From 8493016aadd85f79266233514e2e1588ae62a410 Mon Sep 17 00:00:00 2001 From: Nicolae Vartolomei Date: Fri, 11 Oct 2019 15:44:55 +0100 Subject: [PATCH 0001/1355] Iterate through shards in getStructureOfRemoteTable if skip_unavailable_shards is set --- dbms/src/Common/ErrorCodes.cpp | 1 + .../Storages/getStructureOfRemoteTable.cpp | 38 ++++++++++++++++++- dbms/src/Storages/getStructureOfRemoteTable.h | 10 ++++- .../00183_skip_unavailable_shards.reference | 1 + .../00183_skip_unavailable_shards.sql | 1 + 5 files changed, 47 insertions(+), 4 deletions(-) diff --git a/dbms/src/Common/ErrorCodes.cpp b/dbms/src/Common/ErrorCodes.cpp index cfa89af96d4..c70838749eb 100644 --- a/dbms/src/Common/ErrorCodes.cpp +++ b/dbms/src/Common/ErrorCodes.cpp @@ -459,6 +459,7 @@ namespace ErrorCodes extern const int DICTIONARY_ACCESS_DENIED = 482; extern const int TOO_MANY_REDIRECTS = 483; extern const int INTERNAL_REDIS_ERROR = 484; + extern const int NO_REMOTE_SHARD_AVAILABLE = 485; extern const int KEEPER_EXCEPTION = 999; extern const int POCO_EXCEPTION = 1000; diff --git a/dbms/src/Storages/getStructureOfRemoteTable.cpp b/dbms/src/Storages/getStructureOfRemoteTable.cpp index 2b6924695bf..d6f14a49716 100644 --- a/dbms/src/Storages/getStructureOfRemoteTable.cpp +++ b/dbms/src/Storages/getStructureOfRemoteTable.cpp @@ -21,6 +21,7 @@ namespace DB namespace ErrorCodes { extern const int NO_REMOTE_SHARD_FOUND; + extern const int NO_REMOTE_SHARD_AVAILABLE; } @@ -31,9 +32,42 @@ ColumnsDescription getStructureOfRemoteTable( const Context & context, const ASTPtr & table_func_ptr) { - /// Send to the first any remote shard. - const auto & shard_info = cluster.getAnyShardInfo(); + const auto & shards_info = cluster.getShardsInfo(); + std::string fail_messages; + + for (auto & shard_info : shards_info) + { + try { + const auto & res = getStructureOfRemoteTableInShard(shard_info, database, table, context, table_func_ptr); + return res; + } + catch (const DB::NetException & e) + { + /// THIS DOES NOT WORK AND IS ALWAYS FALSE??? + if (context.getSettingsRef().skip_unavailable_shards) + { + std::string fail_message = getCurrentExceptionMessage(false); + fail_messages += fail_message + '\n'; + + continue; + } + throw; + } + } + + throw DB::NetException( + "All attempts to get table structure failed. Log: \n\n" + fail_messages + "\n", + DB::ErrorCodes::NO_REMOTE_SHARD_AVAILABLE); +} + +ColumnsDescription getStructureOfRemoteTableInShard( + const Cluster::ShardInfo & shard_info, + const std::string & database, + const std::string & table, + const Context & context, + const ASTPtr & table_func_ptr) +{ String query; if (table_func_ptr) diff --git a/dbms/src/Storages/getStructureOfRemoteTable.h b/dbms/src/Storages/getStructureOfRemoteTable.h index 9f1769a7096..a9646271035 100644 --- a/dbms/src/Storages/getStructureOfRemoteTable.h +++ b/dbms/src/Storages/getStructureOfRemoteTable.h @@ -3,12 +3,11 @@ #include #include #include +#include namespace DB { - -class Cluster; class Context; /// Find the names and types of the table columns on any server in the cluster. @@ -20,4 +19,11 @@ ColumnsDescription getStructureOfRemoteTable( const Context & context, const ASTPtr & table_func_ptr = nullptr); +ColumnsDescription getStructureOfRemoteTableInShard( + const Cluster::ShardInfo & shard_info, + const std::string & database, + const std::string & table, + const Context & context, + const ASTPtr & table_func_ptr = nullptr); + } diff --git a/dbms/tests/queries/0_stateless/00183_skip_unavailable_shards.reference b/dbms/tests/queries/0_stateless/00183_skip_unavailable_shards.reference index 0cfbf08886f..51993f072d5 100644 --- a/dbms/tests/queries/0_stateless/00183_skip_unavailable_shards.reference +++ b/dbms/tests/queries/0_stateless/00183_skip_unavailable_shards.reference @@ -1 +1,2 @@ 2 +2 diff --git a/dbms/tests/queries/0_stateless/00183_skip_unavailable_shards.sql b/dbms/tests/queries/0_stateless/00183_skip_unavailable_shards.sql index db7cb24dfd8..2f2fe407400 100644 --- a/dbms/tests/queries/0_stateless/00183_skip_unavailable_shards.sql +++ b/dbms/tests/queries/0_stateless/00183_skip_unavailable_shards.sql @@ -1,2 +1,3 @@ SET send_logs_level = 'none'; 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; From 7b6ede0ee665267e446a06f5546ac4273c13504b Mon Sep 17 00:00:00 2001 From: Nicolae Vartolomei Date: Fri, 11 Oct 2019 18:06:21 +0100 Subject: [PATCH 0002/1355] hack to handle the empty block case returned by Connection when skip_unavailable_shards is set --- dbms/src/Storages/ColumnsDescription.h | 5 +++++ dbms/src/Storages/getStructureOfRemoteTable.cpp | 7 +++++++ 2 files changed, 12 insertions(+) diff --git a/dbms/src/Storages/ColumnsDescription.h b/dbms/src/Storages/ColumnsDescription.h index d0d042498fa..978260af6ab 100644 --- a/dbms/src/Storages/ColumnsDescription.h +++ b/dbms/src/Storages/ColumnsDescription.h @@ -103,6 +103,11 @@ public: String toString() const; static ColumnsDescription parse(const String & str); + size_t size() const + { + return columns.size(); + } + /// Keep the sequence of columns and allow to lookup by name. using Container = boost::multi_index_container< ColumnDescription, diff --git a/dbms/src/Storages/getStructureOfRemoteTable.cpp b/dbms/src/Storages/getStructureOfRemoteTable.cpp index d6f14a49716..5ee6c525955 100644 --- a/dbms/src/Storages/getStructureOfRemoteTable.cpp +++ b/dbms/src/Storages/getStructureOfRemoteTable.cpp @@ -40,6 +40,13 @@ ColumnsDescription getStructureOfRemoteTable( { try { const auto & res = getStructureOfRemoteTableInShard(shard_info, database, table, context, table_func_ptr); + /// Expect at least some columns. + /// This is a hack to handle the empty block case returned by Connection when skip_unavailable_shards is set. + if (res.size() == 0) + { + continue; + } + return res; } catch (const DB::NetException & e) From 18430b7f189eae1bb880322fbad3d6098ac49ac6 Mon Sep 17 00:00:00 2001 From: Nicolae Vartolomei Date: Sat, 12 Oct 2019 12:40:03 +0100 Subject: [PATCH 0003/1355] Update query context with query settings I think the idea of global/session/query context should go away completely as it is quite hard to keep track of them. Just have one type of context and as query is processed make copies. --- dbms/src/Interpreters/InterpreterSelectQuery.cpp | 3 +++ dbms/src/Storages/getStructureOfRemoteTable.cpp | 3 +-- 2 files changed, 4 insertions(+), 2 deletions(-) diff --git a/dbms/src/Interpreters/InterpreterSelectQuery.cpp b/dbms/src/Interpreters/InterpreterSelectQuery.cpp index dbdf3696b9c..52ba2910451 100644 --- a/dbms/src/Interpreters/InterpreterSelectQuery.cpp +++ b/dbms/src/Interpreters/InterpreterSelectQuery.cpp @@ -2694,7 +2694,10 @@ void InterpreterSelectQuery::initSettings() { auto & query = getSelectQuery(); if (query.settings()) + { InterpreterSetQuery(query.settings(), context).executeForCurrentContext(); + InterpreterSetQuery(query.settings(), context.getQueryContext()).executeForCurrentContext(); + } } } diff --git a/dbms/src/Storages/getStructureOfRemoteTable.cpp b/dbms/src/Storages/getStructureOfRemoteTable.cpp index 5ee6c525955..3171e94e884 100644 --- a/dbms/src/Storages/getStructureOfRemoteTable.cpp +++ b/dbms/src/Storages/getStructureOfRemoteTable.cpp @@ -49,9 +49,8 @@ ColumnsDescription getStructureOfRemoteTable( return res; } - catch (const DB::NetException & e) + catch (const DB::NetException &) { - /// THIS DOES NOT WORK AND IS ALWAYS FALSE??? if (context.getSettingsRef().skip_unavailable_shards) { std::string fail_message = getCurrentExceptionMessage(false); From b914c4262df43a9bcc901324f4271f9c756a6135 Mon Sep 17 00:00:00 2001 From: liyang Date: Fri, 17 Jan 2020 21:52:02 +0800 Subject: [PATCH 0004/1355] add array function auc --- dbms/src/Functions/array/arrayScalarProduct.h | 157 ++++++++++++++++++ dbms/src/Functions/array/auc.cpp | 97 +++++++++++ .../array/registerFunctionsArray.cpp | 3 +- .../0_stateless/01064_array_auc.reference | 1 + .../queries/0_stateless/01064_array_auc.sql | 1 + .../functions/array_functions.md | 6 + 6 files changed, 264 insertions(+), 1 deletion(-) create mode 100644 dbms/src/Functions/array/arrayScalarProduct.h create mode 100644 dbms/src/Functions/array/auc.cpp create mode 100644 dbms/tests/queries/0_stateless/01064_array_auc.reference create mode 100644 dbms/tests/queries/0_stateless/01064_array_auc.sql diff --git a/dbms/src/Functions/array/arrayScalarProduct.h b/dbms/src/Functions/array/arrayScalarProduct.h new file mode 100644 index 00000000000..9d25f551483 --- /dev/null +++ b/dbms/src/Functions/array/arrayScalarProduct.h @@ -0,0 +1,157 @@ +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int LOGICAL_ERROR; + extern const int ILLEGAL_COLUMN; + extern const int ILLEGAL_TYPE_OF_ARGUMENT; +} + +template +class FunctionArrayScalarProduct : public IFunction +{ +public: + static constexpr auto name = Name::name; + static FunctionPtr create(const Context & context) { return std::make_shared(context); } + FunctionArrayScalarProduct(const Context & context_): context(context_) {} + +private: + using ResultColumnType = ColumnVector; + + template + bool executeNumber(Block & block, const ColumnNumbers & arguments, size_t result) + { + return executeNumberNumber(block, arguments, result) + || executeNumberNumber(block, arguments, result) + || executeNumberNumber(block, arguments, result) + || executeNumberNumber(block, arguments, result) + || executeNumberNumber(block, arguments, result) + || executeNumberNumber(block, arguments, result) + || executeNumberNumber(block, arguments, result) + || executeNumberNumber(block, arguments, result) + || executeNumberNumber(block, arguments, result) + || executeNumberNumber(block, arguments, result); + } + + + template + bool executeNumberNumber(Block & block, const ColumnNumbers & arguments, size_t result) + { + ColumnPtr col1 = block.getByPosition(arguments[0]).column->convertToFullColumnIfConst(); + ColumnPtr col2 = block.getByPosition(arguments[1]).column->convertToFullColumnIfConst(); + if (! col1 || ! col2) + return false; + + const ColumnArray* col_array1 = checkAndGetColumn(col1.get()); + const ColumnArray* col_array2 = checkAndGetColumn(col2.get()); + if (! col_array1 || ! col_array2) + return false; + + const ColumnVector * col_nested1 = checkAndGetColumn>(col_array1->getData()); + const ColumnVector * col_nested2 = checkAndGetColumn>(col_array2->getData()); + if (! col_nested1 || ! col_nested2) + return false; + + auto col_res = ResultColumnType::create(); + vector(col_nested1->getData(), col_array1->getOffsets(), + col_nested2->getData(), col_array2->getOffsets(), col_res->getData()); + block.getByPosition(result).column = std::move(col_res); + return true; + } + + template + static void vector(const PaddedPODArray & data1, const ColumnArray::Offsets & offsets1, + const PaddedPODArray & data2, const ColumnArray::Offsets & offsets2, + PaddedPODArray & result) + { + size_t size = offsets1.size(); + result.resize(size); + + ColumnArray::Offset current_offset1 = 0; + ColumnArray::Offset current_offset2 = 0; + for (size_t i = 0; i < size; ++i) { + size_t array1_size = offsets1[i] - current_offset1; + size_t array2_size = offsets2[i] - current_offset2; + result[i] = Method::apply(data1, current_offset1, array1_size, data2, current_offset2, array2_size); + + current_offset1 = offsets1[i]; + current_offset2 = offsets2[i]; + } + } + +public: + /// Get function name. + String getName() const override + { + return name; + } + + size_t getNumberOfArguments() const override { return 2; } + + DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override + { + // Basic type check + std::vector nested_types(2, nullptr); + for (size_t i = 0; i < getNumberOfArguments(); ++i) + { + const DataTypeArray * array_type = checkAndGetDataType(arguments[i].get()); + if (!array_type) + throw Exception("All argument for function " + getName() + " must be an array.", + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + + auto & nested_type = array_type->getNestedType(); + WhichDataType which(nested_type); + bool is_number = which.isNativeInt() || which.isNativeUInt() || which.isFloat(); + if (! is_number) + { + throw Exception(getName() + " cannot process values of type " + nested_type->getName(), + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + } + nested_types[i] = nested_type; + } + + // Detail type check in Method, then return ReturnType + return Method::getReturnType(nested_types[0], nested_types[1]); + } + + void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t /* input_rows_count */) override + { + if (!(executeNumber(block, arguments, result) + || executeNumber(block, arguments, result) + || executeNumber(block, arguments, result) + || executeNumber(block, arguments, result) + || executeNumber(block, arguments, result) + || executeNumber(block, arguments, result) + || executeNumber(block, arguments, result) + || executeNumber(block, arguments, result) + || executeNumber(block, arguments, result) + || executeNumber(block, arguments, result))) + throw Exception{"Illegal column " + block.getByPosition(arguments[0]).column->getName() + + " of first argument of function " + getName(), ErrorCodes::ILLEGAL_COLUMN}; + + } + +private: + const Context & context; +}; + +} \ No newline at end of file diff --git a/dbms/src/Functions/array/auc.cpp b/dbms/src/Functions/array/auc.cpp new file mode 100644 index 00000000000..2ac7d85a501 --- /dev/null +++ b/dbms/src/Functions/array/auc.cpp @@ -0,0 +1,97 @@ +#include +#include +#include +#include "arrayScalarProduct.h" + +namespace DB +{ + +struct NameAUC { static constexpr auto name = "auc"; }; + +class AUCImpl +{ +public: + using ResultType = Float64; + + struct ScoreLabel + { + ResultType score; + UInt8 label; + }; + + static DataTypePtr getReturnType(const DataTypePtr & /* nested_type1 */, const DataTypePtr & nested_type2) + { + WhichDataType which2(nested_type2); + if (! which2.isUInt8()) { + throw Exception(std::string(NameAUC::name) + "lable type must be UInt8", + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + } + return std::make_shared>(); + } + + template + static ResultType apply(const PaddedPODArray & scores, ColumnArray::Offset score_offset, size_t score_len, + const PaddedPODArray & labels, ColumnArray::Offset label_offset, size_t label_len) + { + if (score_len != label_len) + throw Exception{"Unmatched length of arrays in " + std::string(NameAUC::name), + ErrorCodes::LOGICAL_ERROR}; + if (score_len == 0) + return {}; + + // Order pairs of score and lable by score ascending + size_t num_pos = 0; + size_t num_neg = 0; + std::vector pairs(score_len); + for (size_t i = 0; i < score_len; ++i) + { + pairs[i].score = scores[i + score_offset]; + pairs[i].label = (labels[i + label_offset] ? 1 : 0); + if (pairs[i].label) + ++num_pos; + else + ++num_neg; + } + std::sort(pairs.begin(), pairs.end(), + [](const auto & lhs, const auto & rhs) {return lhs.score < rhs.score; }); + + // Calculate AUC + size_t curr_cnt = 0; + size_t curr_pos_cnt = 0; + size_t curr_sum = 0; + ResultType last_score = -1; + ResultType rank_sum = 0; + for (size_t i = 0; i < pairs.size(); ++i) + { + if (pairs[i].score == last_score) + { + curr_sum += i + 1; + ++curr_cnt; + if (pairs[i].label) + ++curr_pos_cnt; + } + else + { + if (i > 0) + rank_sum += ResultType(curr_sum * curr_pos_cnt) / curr_cnt; + curr_sum = i + 1; + curr_cnt = 1; + curr_pos_cnt = pairs[i].label ? 1 : 0; + } + last_score = pairs[i].score; + } + rank_sum += ResultType(curr_sum * curr_pos_cnt) / curr_cnt; + return (rank_sum - num_pos*(num_pos+1)/2)/(num_pos * num_neg); + } +}; + +/// auc(array_score, array_label) - Calculate AUC with array of score and label +using FunctionAUC = FunctionArrayScalarProduct; + +void registerFunctionAUC(FunctionFactory & factory) +{ + factory.registerFunction(); +} + + +} \ No newline at end of file diff --git a/dbms/src/Functions/array/registerFunctionsArray.cpp b/dbms/src/Functions/array/registerFunctionsArray.cpp index ababc7603e3..96f19f547c0 100644 --- a/dbms/src/Functions/array/registerFunctionsArray.cpp +++ b/dbms/src/Functions/array/registerFunctionsArray.cpp @@ -33,7 +33,7 @@ void registerFunctionArrayDistinct(FunctionFactory & factory); void registerFunctionArrayFlatten(FunctionFactory & factory); void registerFunctionArrayWithConstant(FunctionFactory & factory); void registerFunctionArrayZip(FunctionFactory & factory); - +void registerFunctionAUC(FunctionFactory &); void registerFunctionsArray(FunctionFactory & factory) { @@ -67,6 +67,7 @@ void registerFunctionsArray(FunctionFactory & factory) registerFunctionArrayFlatten(factory); registerFunctionArrayWithConstant(factory); registerFunctionArrayZip(factory); + registerFunctionAUC(factory); } } diff --git a/dbms/tests/queries/0_stateless/01064_array_auc.reference b/dbms/tests/queries/0_stateless/01064_array_auc.reference new file mode 100644 index 00000000000..d6e1fa0f619 --- /dev/null +++ b/dbms/tests/queries/0_stateless/01064_array_auc.reference @@ -0,0 +1 @@ +0.75 \ No newline at end of file diff --git a/dbms/tests/queries/0_stateless/01064_array_auc.sql b/dbms/tests/queries/0_stateless/01064_array_auc.sql new file mode 100644 index 00000000000..33b54e7db45 --- /dev/null +++ b/dbms/tests/queries/0_stateless/01064_array_auc.sql @@ -0,0 +1 @@ +select select auc([0.1, 0.4, 0.35, 0.8], [0, 0, 1, 1]) \ No newline at end of file diff --git a/docs/en/query_language/functions/array_functions.md b/docs/en/query_language/functions/array_functions.md index 6d5d9ccd6bf..64f631a84e4 100644 --- a/docs/en/query_language/functions/array_functions.md +++ b/docs/en/query_language/functions/array_functions.md @@ -896,5 +896,11 @@ Result: │ [('a','d'),('b','e'),('c','f')] │ └────────────────────────────────────────────┘ ``` +## auc(arr_scores, arr_labels) + +Returns AUC(Area Under the Curve, which is a concept in machine learning, see more details: https://developers.google.com/machine-learning/crash-course/classification/roc-and-auc); + +`arr_scores` represents scores prediction model gives, while `arr_labels` represents labels of samples, usually 1 for positive sample and 0 for negtive sample. + [Original article](https://clickhouse.yandex/docs/en/query_language/functions/array_functions/) From 1dab2b756d08c4a4e0f0d8ba7bc2d0a21dce52f5 Mon Sep 17 00:00:00 2001 From: liyang Date: Sat, 18 Jan 2020 12:16:34 +0800 Subject: [PATCH 0005/1355] fix code style --- dbms/src/Functions/array/arrayScalarProduct.h | 104 ++++++++---------- dbms/src/Functions/array/auc.cpp | 53 +++++---- 2 files changed, 75 insertions(+), 82 deletions(-) diff --git a/dbms/src/Functions/array/arrayScalarProduct.h b/dbms/src/Functions/array/arrayScalarProduct.h index 9d25f551483..82e73b8c17a 100644 --- a/dbms/src/Functions/array/arrayScalarProduct.h +++ b/dbms/src/Functions/array/arrayScalarProduct.h @@ -1,24 +1,23 @@ -#include -#include -#include +#include +#include +#include +#include +#include #include #include #include -#include -#include -#include -#include -#include -#include -#include -#include -#include +#include +#include +#include #include +#include +#include +#include +#include namespace DB { - namespace ErrorCodes { extern const int LOGICAL_ERROR; @@ -32,7 +31,7 @@ class FunctionArrayScalarProduct : public IFunction public: static constexpr auto name = Name::name; static FunctionPtr create(const Context & context) { return std::make_shared(context); } - FunctionArrayScalarProduct(const Context & context_): context(context_) {} + FunctionArrayScalarProduct(const Context & context_) : context(context_) {} private: using ResultColumnType = ColumnVector; @@ -40,16 +39,11 @@ private: template bool executeNumber(Block & block, const ColumnNumbers & arguments, size_t result) { - return executeNumberNumber(block, arguments, result) - || executeNumberNumber(block, arguments, result) - || executeNumberNumber(block, arguments, result) - || executeNumberNumber(block, arguments, result) - || executeNumberNumber(block, arguments, result) - || executeNumberNumber(block, arguments, result) - || executeNumberNumber(block, arguments, result) - || executeNumberNumber(block, arguments, result) - || executeNumberNumber(block, arguments, result) - || executeNumberNumber(block, arguments, result); + return executeNumberNumber(block, arguments, result) || executeNumberNumber(block, arguments, result) + || executeNumberNumber(block, arguments, result) || executeNumberNumber(block, arguments, result) + || executeNumberNumber(block, arguments, result) || executeNumberNumber(block, arguments, result) + || executeNumberNumber(block, arguments, result) || executeNumberNumber(block, arguments, result) + || executeNumberNumber(block, arguments, result) || executeNumberNumber(block, arguments, result); } @@ -58,29 +52,31 @@ private: { ColumnPtr col1 = block.getByPosition(arguments[0]).column->convertToFullColumnIfConst(); ColumnPtr col2 = block.getByPosition(arguments[1]).column->convertToFullColumnIfConst(); - if (! col1 || ! col2) + if (!col1 || !col2) return false; - const ColumnArray* col_array1 = checkAndGetColumn(col1.get()); - const ColumnArray* col_array2 = checkAndGetColumn(col2.get()); - if (! col_array1 || ! col_array2) + const ColumnArray * col_array1 = checkAndGetColumn(col1.get()); + const ColumnArray * col_array2 = checkAndGetColumn(col2.get()); + if (!col_array1 || !col_array2) return false; const ColumnVector * col_nested1 = checkAndGetColumn>(col_array1->getData()); const ColumnVector * col_nested2 = checkAndGetColumn>(col_array2->getData()); - if (! col_nested1 || ! col_nested2) + if (!col_nested1 || !col_nested2) return false; auto col_res = ResultColumnType::create(); - vector(col_nested1->getData(), col_array1->getOffsets(), - col_nested2->getData(), col_array2->getOffsets(), col_res->getData()); + vector(col_nested1->getData(), col_array1->getOffsets(), col_nested2->getData(), col_array2->getOffsets(), col_res->getData()); block.getByPosition(result).column = std::move(col_res); return true; } template - static void vector(const PaddedPODArray & data1, const ColumnArray::Offsets & offsets1, - const PaddedPODArray & data2, const ColumnArray::Offsets & offsets2, + static void vector( + const PaddedPODArray & data1, + const ColumnArray::Offsets & offsets1, + const PaddedPODArray & data2, + const ColumnArray::Offsets & offsets2, PaddedPODArray & result) { size_t size = offsets1.size(); @@ -88,7 +84,8 @@ private: ColumnArray::Offset current_offset1 = 0; ColumnArray::Offset current_offset2 = 0; - for (size_t i = 0; i < size; ++i) { + for (size_t i = 0; i < size; ++i) + { size_t array1_size = offsets1[i] - current_offset1; size_t array2_size = offsets2[i] - current_offset2; result[i] = Method::apply(data1, current_offset1, array1_size, data2, current_offset2, array2_size); @@ -100,10 +97,7 @@ private: public: /// Get function name. - String getName() const override - { - return name; - } + String getName() const override { return name; } size_t getNumberOfArguments() const override { return 2; } @@ -111,20 +105,19 @@ public: { // Basic type check std::vector nested_types(2, nullptr); - for (size_t i = 0; i < getNumberOfArguments(); ++i) + for (size_t i = 0; i < getNumberOfArguments(); ++i) { const DataTypeArray * array_type = checkAndGetDataType(arguments[i].get()); if (!array_type) - throw Exception("All argument for function " + getName() + " must be an array.", - ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); - + throw Exception("All argument for function " + getName() + " must be an array.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + auto & nested_type = array_type->getNestedType(); WhichDataType which(nested_type); bool is_number = which.isNativeInt() || which.isNativeUInt() || which.isFloat(); - if (! is_number) + if (!is_number) { - throw Exception(getName() + " cannot process values of type " + nested_type->getName(), - ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + throw Exception( + getName() + " cannot process values of type " + nested_type->getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); } nested_types[i] = nested_type; } @@ -135,19 +128,14 @@ public: void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t /* input_rows_count */) override { - if (!(executeNumber(block, arguments, result) - || executeNumber(block, arguments, result) - || executeNumber(block, arguments, result) - || executeNumber(block, arguments, result) - || executeNumber(block, arguments, result) - || executeNumber(block, arguments, result) - || executeNumber(block, arguments, result) - || executeNumber(block, arguments, result) - || executeNumber(block, arguments, result) - || executeNumber(block, arguments, result))) - throw Exception{"Illegal column " + block.getByPosition(arguments[0]).column->getName() - + " of first argument of function " + getName(), ErrorCodes::ILLEGAL_COLUMN}; - + if (!(executeNumber(block, arguments, result) || executeNumber(block, arguments, result) + || executeNumber(block, arguments, result) || executeNumber(block, arguments, result) + || executeNumber(block, arguments, result) || executeNumber(block, arguments, result) + || executeNumber(block, arguments, result) || executeNumber(block, arguments, result) + || executeNumber(block, arguments, result) || executeNumber(block, arguments, result))) + throw Exception{"Illegal column " + block.getByPosition(arguments[0]).column->getName() + " of first argument of function " + + getName(), + ErrorCodes::ILLEGAL_COLUMN}; } private: diff --git a/dbms/src/Functions/array/auc.cpp b/dbms/src/Functions/array/auc.cpp index 2ac7d85a501..7091452ea0b 100644 --- a/dbms/src/Functions/array/auc.cpp +++ b/dbms/src/Functions/array/auc.cpp @@ -1,19 +1,21 @@ -#include #include +#include #include #include "arrayScalarProduct.h" namespace DB { +struct NameAUC +{ + static constexpr auto name = "auc"; +}; -struct NameAUC { static constexpr auto name = "auc"; }; - -class AUCImpl +class AUCImpl { public: using ResultType = Float64; - struct ScoreLabel + struct ScoreLabel { ResultType score; UInt8 label; @@ -22,38 +24,41 @@ public: static DataTypePtr getReturnType(const DataTypePtr & /* nested_type1 */, const DataTypePtr & nested_type2) { WhichDataType which2(nested_type2); - if (! which2.isUInt8()) { - throw Exception(std::string(NameAUC::name) + "lable type must be UInt8", - ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + if (!which2.isUInt8()) + { + throw Exception(std::string(NameAUC::name) + "lable type must be UInt8", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); } return std::make_shared>(); } template - static ResultType apply(const PaddedPODArray & scores, ColumnArray::Offset score_offset, size_t score_len, - const PaddedPODArray & labels, ColumnArray::Offset label_offset, size_t label_len) + static ResultType apply( + const PaddedPODArray & scores, + ColumnArray::Offset score_offset, + size_t score_len, + const PaddedPODArray & labels, + ColumnArray::Offset label_offset, + size_t label_len) { - if (score_len != label_len) - throw Exception{"Unmatched length of arrays in " + std::string(NameAUC::name), - ErrorCodes::LOGICAL_ERROR}; - if (score_len == 0) + if (score_len != label_len) + throw Exception{"Unmatched length of arrays in " + std::string(NameAUC::name), ErrorCodes::LOGICAL_ERROR}; + if (score_len == 0) return {}; // Order pairs of score and lable by score ascending size_t num_pos = 0; size_t num_neg = 0; std::vector pairs(score_len); - for (size_t i = 0; i < score_len; ++i) + for (size_t i = 0; i < score_len; ++i) { pairs[i].score = scores[i + score_offset]; pairs[i].label = (labels[i + label_offset] ? 1 : 0); - if (pairs[i].label) + if (pairs[i].label) ++num_pos; - else + else ++num_neg; } - std::sort(pairs.begin(), pairs.end(), - [](const auto & lhs, const auto & rhs) {return lhs.score < rhs.score; }); + std::sort(pairs.begin(), pairs.end(), [](const auto & lhs, const auto & rhs) { return lhs.score < rhs.score; }); // Calculate AUC size_t curr_cnt = 0; @@ -61,18 +66,18 @@ public: size_t curr_sum = 0; ResultType last_score = -1; ResultType rank_sum = 0; - for (size_t i = 0; i < pairs.size(); ++i) + for (size_t i = 0; i < pairs.size(); ++i) { if (pairs[i].score == last_score) { curr_sum += i + 1; ++curr_cnt; - if (pairs[i].label) + if (pairs[i].label) ++curr_pos_cnt; } - else + else { - if (i > 0) + if (i > 0) rank_sum += ResultType(curr_sum * curr_pos_cnt) / curr_cnt; curr_sum = i + 1; curr_cnt = 1; @@ -81,7 +86,7 @@ public: last_score = pairs[i].score; } rank_sum += ResultType(curr_sum * curr_pos_cnt) / curr_cnt; - return (rank_sum - num_pos*(num_pos+1)/2)/(num_pos * num_neg); + return (rank_sum - num_pos * (num_pos + 1) / 2) / (num_pos * num_neg); } }; From 07f2d88ed4230ecb724fc4c3ba57b1910e677fcf Mon Sep 17 00:00:00 2001 From: liyang Date: Sat, 18 Jan 2020 14:46:06 +0800 Subject: [PATCH 0006/1355] fix bug --- dbms/tests/queries/0_stateless/01064_array_auc.sql | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/tests/queries/0_stateless/01064_array_auc.sql b/dbms/tests/queries/0_stateless/01064_array_auc.sql index 33b54e7db45..8886426f098 100644 --- a/dbms/tests/queries/0_stateless/01064_array_auc.sql +++ b/dbms/tests/queries/0_stateless/01064_array_auc.sql @@ -1 +1 @@ -select select auc([0.1, 0.4, 0.35, 0.8], [0, 0, 1, 1]) \ No newline at end of file +select auc([0.1, 0.4, 0.35, 0.8], [0, 0, 1, 1]) From c66fc0c8b58af73237c7fae99b96dc93cd86b32b Mon Sep 17 00:00:00 2001 From: liyang Date: Sat, 18 Jan 2020 18:03:14 +0800 Subject: [PATCH 0007/1355] fix bug --- dbms/src/Functions/array/arrayScalarProduct.h | 2 +- dbms/src/Functions/array/auc.cpp | 2 +- dbms/tests/queries/0_stateless/01064_array_auc.reference | 2 +- 3 files changed, 3 insertions(+), 3 deletions(-) diff --git a/dbms/src/Functions/array/arrayScalarProduct.h b/dbms/src/Functions/array/arrayScalarProduct.h index 82e73b8c17a..c450f41f48a 100644 --- a/dbms/src/Functions/array/arrayScalarProduct.h +++ b/dbms/src/Functions/array/arrayScalarProduct.h @@ -142,4 +142,4 @@ private: const Context & context; }; -} \ No newline at end of file +} diff --git a/dbms/src/Functions/array/auc.cpp b/dbms/src/Functions/array/auc.cpp index 7091452ea0b..7a2daf0fdd0 100644 --- a/dbms/src/Functions/array/auc.cpp +++ b/dbms/src/Functions/array/auc.cpp @@ -99,4 +99,4 @@ void registerFunctionAUC(FunctionFactory & factory) } -} \ No newline at end of file +} diff --git a/dbms/tests/queries/0_stateless/01064_array_auc.reference b/dbms/tests/queries/0_stateless/01064_array_auc.reference index d6e1fa0f619..39c64a9f26e 100644 --- a/dbms/tests/queries/0_stateless/01064_array_auc.reference +++ b/dbms/tests/queries/0_stateless/01064_array_auc.reference @@ -1 +1 @@ -0.75 \ No newline at end of file +0.75 From 2fdd211635d3d6d7800ce0cdd63d6b70d7ba26b8 Mon Sep 17 00:00:00 2001 From: liyang Date: Tue, 21 Jan 2020 16:32:35 +0800 Subject: [PATCH 0008/1355] rename auc to arrayAUC --- .../src/Functions/array/auc.cpp => array_auc.cpp | 16 ++++++++-------- .../queries/0_stateless/01064_array_auc.sql | 2 +- .../query_language/functions/array_functions.md | 2 +- 3 files changed, 10 insertions(+), 10 deletions(-) rename dbms/src/Functions/array/auc.cpp => array_auc.cpp (84%) diff --git a/dbms/src/Functions/array/auc.cpp b/array_auc.cpp similarity index 84% rename from dbms/src/Functions/array/auc.cpp rename to array_auc.cpp index 7a2daf0fdd0..41739446e46 100644 --- a/dbms/src/Functions/array/auc.cpp +++ b/array_auc.cpp @@ -5,12 +5,12 @@ namespace DB { -struct NameAUC +struct NameArrayAUC { - static constexpr auto name = "auc"; + static constexpr auto name = "arrayAUC"; }; -class AUCImpl +class ArrayAUCImpl { public: using ResultType = Float64; @@ -26,7 +26,7 @@ public: WhichDataType which2(nested_type2); if (!which2.isUInt8()) { - throw Exception(std::string(NameAUC::name) + "lable type must be UInt8", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + throw Exception(std::string(NameArrayAUC::name) + "lable type must be UInt8", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); } return std::make_shared>(); } @@ -41,7 +41,7 @@ public: size_t label_len) { if (score_len != label_len) - throw Exception{"Unmatched length of arrays in " + std::string(NameAUC::name), ErrorCodes::LOGICAL_ERROR}; + throw Exception{"Unmatched length of arrays in " + std::string(NameArrayAUC::name), ErrorCodes::LOGICAL_ERROR}; if (score_len == 0) return {}; @@ -91,11 +91,11 @@ public: }; /// auc(array_score, array_label) - Calculate AUC with array of score and label -using FunctionAUC = FunctionArrayScalarProduct; +using FunctionArrayAUC = FunctionArrayScalarProduct; -void registerFunctionAUC(FunctionFactory & factory) +void registerFunctionArrayAUC(FunctionFactory & factory) { - factory.registerFunction(); + factory.registerFunction(); } diff --git a/dbms/tests/queries/0_stateless/01064_array_auc.sql b/dbms/tests/queries/0_stateless/01064_array_auc.sql index 8886426f098..ca270937f63 100644 --- a/dbms/tests/queries/0_stateless/01064_array_auc.sql +++ b/dbms/tests/queries/0_stateless/01064_array_auc.sql @@ -1 +1 @@ -select auc([0.1, 0.4, 0.35, 0.8], [0, 0, 1, 1]) +select arrayAUC([0.1, 0.4, 0.35, 0.8], [0, 0, 1, 1]) diff --git a/docs/en/query_language/functions/array_functions.md b/docs/en/query_language/functions/array_functions.md index 64f631a84e4..8c9b2a7c151 100644 --- a/docs/en/query_language/functions/array_functions.md +++ b/docs/en/query_language/functions/array_functions.md @@ -896,7 +896,7 @@ Result: │ [('a','d'),('b','e'),('c','f')] │ └────────────────────────────────────────────┘ ``` -## auc(arr_scores, arr_labels) +## arrayAUC(arr_scores, arr_labels) Returns AUC(Area Under the Curve, which is a concept in machine learning, see more details: https://developers.google.com/machine-learning/crash-course/classification/roc-and-auc); From b9076ef247b7765e2839c170debe6e10f4f202b4 Mon Sep 17 00:00:00 2001 From: liyang Date: Tue, 21 Jan 2020 16:59:09 +0800 Subject: [PATCH 0009/1355] mv file --- array_auc.cpp => dbms/src/Functions/array/array_auc.cpp | 0 1 file changed, 0 insertions(+), 0 deletions(-) rename array_auc.cpp => dbms/src/Functions/array/array_auc.cpp (100%) diff --git a/array_auc.cpp b/dbms/src/Functions/array/array_auc.cpp similarity index 100% rename from array_auc.cpp rename to dbms/src/Functions/array/array_auc.cpp From 3d9f49c0baff29d6d43d332cfb42e8d6c44d5e01 Mon Sep 17 00:00:00 2001 From: liyang Date: Tue, 21 Jan 2020 17:03:35 +0800 Subject: [PATCH 0010/1355] commitagin --- dbms/src/Functions/array/registerFunctionsArray.cpp | 6 ++---- 1 file changed, 2 insertions(+), 4 deletions(-) diff --git a/dbms/src/Functions/array/registerFunctionsArray.cpp b/dbms/src/Functions/array/registerFunctionsArray.cpp index 96f19f547c0..cc360d80979 100644 --- a/dbms/src/Functions/array/registerFunctionsArray.cpp +++ b/dbms/src/Functions/array/registerFunctionsArray.cpp @@ -1,6 +1,5 @@ namespace DB { - class FunctionFactory; void registerFunctionArray(FunctionFactory & factory); @@ -33,7 +32,7 @@ void registerFunctionArrayDistinct(FunctionFactory & factory); void registerFunctionArrayFlatten(FunctionFactory & factory); void registerFunctionArrayWithConstant(FunctionFactory & factory); void registerFunctionArrayZip(FunctionFactory & factory); -void registerFunctionAUC(FunctionFactory &); +void registerFunctionArrayAUC(FunctionFactory &); void registerFunctionsArray(FunctionFactory & factory) { @@ -67,8 +66,7 @@ void registerFunctionsArray(FunctionFactory & factory) registerFunctionArrayFlatten(factory); registerFunctionArrayWithConstant(factory); registerFunctionArrayZip(factory); - registerFunctionAUC(factory); + registerFunctionArrayAUC(factory); } } - From 1147436d60d19c882109915c417a45e383d52e12 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=D0=90=D1=80=D1=82=D0=B5=D0=BC=20=D0=A1=D1=82=D1=80=D0=B5?= =?UTF-8?q?=D0=BB=D1=8C=D1=86=D0=BE=D0=B2?= Date: Wed, 29 Jan 2020 18:50:18 +0300 Subject: [PATCH 0011/1355] Raw add of direct dictionaries (doesnt work) --- dbms/src/Dictionaries/DirectDictionary.cpp | 582 ++++++++++++++++++ dbms/src/Dictionaries/DirectDictionary.h | 294 +++++++++ dbms/src/Dictionaries/DirectDictionary.inc.h | 406 ++++++++++++ .../src/Dictionaries/registerDictionaries.cpp | 3 + 4 files changed, 1285 insertions(+) create mode 100644 dbms/src/Dictionaries/DirectDictionary.cpp create mode 100644 dbms/src/Dictionaries/DirectDictionary.h create mode 100644 dbms/src/Dictionaries/DirectDictionary.inc.h diff --git a/dbms/src/Dictionaries/DirectDictionary.cpp b/dbms/src/Dictionaries/DirectDictionary.cpp new file mode 100644 index 00000000000..9e0a77ebc91 --- /dev/null +++ b/dbms/src/Dictionaries/DirectDictionary.cpp @@ -0,0 +1,582 @@ +#include "DirectDictionary.h" + +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include "DirectDictionary.inc.h" +#include "DictionaryBlockInputStream.h" +#include "DictionaryFactory.h" + + +/* + * + * TODO: CHANGE EVENTS TO DIRECT DICTIONARY EVENTS (WTF? WHERE R THEY DECLARED????) + * +*/ + +namespace ProfileEvents +{ + extern const Event DictCacheKeysRequested; + extern const Event DictCacheKeysRequestedMiss; + extern const Event DictCacheKeysRequestedFound; + extern const Event DictCacheKeysExpired; + extern const Event DictCacheKeysNotFound; + extern const Event DictCacheKeysHit; + extern const Event DictCacheRequestTimeNs; + extern const Event DictCacheRequests; + extern const Event DictCacheLockWriteNs; + extern const Event DictCacheLockReadNs; +} + +namespace CurrentMetrics +{ + extern const Metric DictCacheRequests; +} + + +namespace DB +{ + namespace ErrorCodes + { + extern const int TYPE_MISMATCH; + extern const int BAD_ARGUMENTS; + extern const int UNSUPPORTED_METHOD; + extern const int LOGICAL_ERROR; + extern const int TOO_SMALL_BUFFER_SIZE; + } + + /* + * deleted inline size_t DirectDictionary::getCellIdx(const Key id) const + * + */ + + + DirectDictionary::DirectDictionary( + const std::string & name_, + const DictionaryStructure & dict_struct_, + DictionarySourcePtr source_ptr_, + const DictionaryLifetime dict_lifetime_ + ) + : name{name_} + , dict_struct(dict_struct_) + , source_ptr{std::move(source_ptr_)} + , dict_lifetime(dict_lifetime_) + , log(&Logger::get("ExternalDictionaries")) + , rnd_engine(randomSeed()) + { + if (!this->source_ptr->supportsSelectiveLoad()) + throw Exception{name + ": source cannot be used with DirectDictionary", ErrorCodes::UNSUPPORTED_METHOD}; + + createAttributes(); + } + + + void DirectDictionary::toParent(const PaddedPODArray & ids, PaddedPODArray & out) const + { + const auto null_value = std::get(hierarchical_attribute->null_values); + + getItemsNumberImpl(*hierarchical_attribute, ids, out, [&](const size_t) { return null_value; }); + } + + +/// Allow to use single value in same way as array. + static inline DirectDictionary::Key getAt(const PaddedPODArray & arr, const size_t idx) + { + return arr[idx]; + } + static inline DirectDictionary::Key getAt(const DirectDictionary::Key & value, const size_t) + { + return value; + } + + + template + void DirectDictionary::isInImpl(const PaddedPODArray & child_ids, const AncestorType & ancestor_ids, PaddedPODArray & out) const + { + /// Transform all children to parents until ancestor id or null_value will be reached. + + size_t out_size = out.size(); + memset(out.data(), 0xFF, out_size); /// 0xFF means "not calculated" + + const auto null_value = std::get(hierarchical_attribute->null_values); + + PaddedPODArray children(out_size, 0); + PaddedPODArray parents(child_ids.begin(), child_ids.end()); + + while (true) + { + size_t out_idx = 0; + size_t parents_idx = 0; + size_t new_children_idx = 0; + + while (out_idx < out_size) + { + /// Already calculated + if (out[out_idx] != 0xFF) + { + ++out_idx; + continue; + } + + /// No parent + if (parents[parents_idx] == null_value) + { + out[out_idx] = 0; + } + /// Found ancestor + else if (parents[parents_idx] == getAt(ancestor_ids, parents_idx)) + { + out[out_idx] = 1; + } + /// Loop detected + else if (children[new_children_idx] == parents[parents_idx]) + { + out[out_idx] = 1; + } + /// Found intermediate parent, add this value to search at next loop iteration + else + { + children[new_children_idx] = parents[parents_idx]; + ++new_children_idx; + } + + ++out_idx; + ++parents_idx; + } + + if (new_children_idx == 0) + break; + + /// Transform all children to its parents. + children.resize(new_children_idx); + parents.resize(new_children_idx); + + toParent(children, parents); + } + } + + void DirectDictionary::isInVectorVector( + const PaddedPODArray & child_ids, const PaddedPODArray & ancestor_ids, PaddedPODArray & out) const + { + isInImpl(child_ids, ancestor_ids, out); + } + + void DirectDictionary::isInVectorConstant(const PaddedPODArray & child_ids, const Key ancestor_id, PaddedPODArray & out) const + { + isInImpl(child_ids, ancestor_id, out); + } + + void DirectDictionary::isInConstantVector(const Key child_id, const PaddedPODArray & ancestor_ids, PaddedPODArray & out) const + { + /// Special case with single child value. + + const auto null_value = std::get(hierarchical_attribute->null_values); + + PaddedPODArray child(1, child_id); + PaddedPODArray parent(1); + std::vector ancestors(1, child_id); + + /// Iteratively find all ancestors for child. + while (true) + { + toParent(child, parent); + + if (parent[0] == null_value) + break; + + child[0] = parent[0]; + ancestors.push_back(parent[0]); + } + + /// Assuming short hierarchy, so linear search is Ok. + for (size_t i = 0, out_size = out.size(); i < out_size; ++i) + out[i] = std::find(ancestors.begin(), ancestors.end(), ancestor_ids[i]) != ancestors.end(); + } + + void DirectDictionary::getString(const std::string & attribute_name, const PaddedPODArray & ids, ColumnString * out) const + { + auto & attribute = getAttribute(attribute_name); + checkAttributeType(name, attribute_name, attribute.type, AttributeUnderlyingType::utString); + + const auto null_value = StringRef{std::get(attribute.null_values)}; + + getItemsString(attribute, ids, out, [&](const size_t) { return null_value; }); + } + + void DirectDictionary::getString( + const std::string & attribute_name, const PaddedPODArray & ids, const ColumnString * const def, ColumnString * const out) const + { + auto & attribute = getAttribute(attribute_name); + checkAttributeType(name, attribute_name, attribute.type, AttributeUnderlyingType::utString); + + getItemsString(attribute, ids, out, [&](const size_t row) { return def->getDataAt(row); }); + } + + void DirectDictionary::getString( + const std::string & attribute_name, const PaddedPODArray & ids, const String & def, ColumnString * const out) const + { + auto & attribute = getAttribute(attribute_name); + checkAttributeType(name, attribute_name, attribute.type, AttributeUnderlyingType::utString); + + getItemsString(attribute, ids, out, [&](const size_t) { return StringRef{def}; }); + } + + +/// returns cell_idx (always valid for replacing), 'cell is valid' flag, 'cell is outdated' flag +/// true false found and valid +/// false true not found (something outdated, maybe our cell) +/// false false not found (other id stored with valid data) +/// true true impossible +/// +/// todo: split this func to two: find_for_get and find_for_set + DirectDictionary::FindResult DirectDictionary::findCellIdx(const Key & id, const CellMetadata::time_point_t now) const + { + auto pos = getCellIdx(id); + auto oldest_id = pos; + auto oldest_time = CellMetadata::time_point_t::max(); + const auto stop = pos + max_collision_length; + for (; pos < stop; ++pos) + { + const auto cell_idx = pos & size_overlap_mask; + const auto & cell = cells[cell_idx]; + + if (cell.id != id) + { + /// maybe we already found nearest expired cell (try minimize collision_length on insert) + if (oldest_time > now && oldest_time > cell.expiresAt()) + { + oldest_time = cell.expiresAt(); + oldest_id = cell_idx; + } + continue; + } + + if (cell.expiresAt() < now) + { + return {cell_idx, false, true}; + } + + return {cell_idx, true, false}; + } + + return {oldest_id, false, false}; + } + + + /* + * deleted most part of has, that stood for + * looking for a key in cache + * + * TODO: check whether we need last two arguments + * in update function (seems like no) + * + */ + + void DirectDictionary::has(const PaddedPODArray & ids, PaddedPODArray & out) const + { + std::vector required_ids(ids.size()); + std::copy(std::begin(ids), std::end(ids), std::begin(required_ids)); + + /// request new values + update( + required_ids, + [&](const auto id, const auto) + { + for (const auto row : outdated_ids[id]) + out[row] = true; + }, + [&](const auto id, const auto) + { + for (const auto row : outdated_ids[id]) + out[row] = false; + }); + } + + + void DirectDictionary::createAttributes() + { + const auto attributes_size = dict_struct.attributes.size(); + attributes.reserve(attributes_size); + + bytes_allocated += size * sizeof(CellMetadata); + bytes_allocated += attributes_size * sizeof(attributes.front()); + + for (const auto & attribute : dict_struct.attributes) + { + attribute_index_by_name.emplace(attribute.name, attributes.size()); + attributes.push_back(createAttributeWithType(attribute.underlying_type, attribute.null_value)); + + if (attribute.hierarchical) + { + hierarchical_attribute = &attributes.back(); + + if (hierarchical_attribute->type != AttributeUnderlyingType::utUInt64) + throw Exception{name + ": hierarchical attribute must be UInt64.", ErrorCodes::TYPE_MISMATCH}; + } + } + } + + DirectDictionary::Attribute DirectDictionary::createAttributeWithType(const AttributeUnderlyingType type, const Field & null_value) + { + Attribute attr{type, {}, {}}; + + switch (type) + { +#define DISPATCH(TYPE) \ + case AttributeUnderlyingType::ut##TYPE: \ + attr.null_values = TYPE(null_value.get>()); \ + attr.arrays = std::make_unique>(size); \ + bytes_allocated += size * sizeof(TYPE); \ + break; + DISPATCH(UInt8) + DISPATCH(UInt16) + DISPATCH(UInt32) + DISPATCH(UInt64) + DISPATCH(UInt128) + DISPATCH(Int8) + DISPATCH(Int16) + DISPATCH(Int32) + DISPATCH(Int64) + DISPATCH(Decimal32) + DISPATCH(Decimal64) + DISPATCH(Decimal128) + DISPATCH(Float32) + DISPATCH(Float64) +#undef DISPATCH + case AttributeUnderlyingType::utString: + attr.null_values = null_value.get(); + attr.arrays = std::make_unique>(size); + bytes_allocated += size * sizeof(StringRef); + if (!string_arena) + string_arena = std::make_unique(); + break; + } + + return attr; + } + + void DirectDictionary::setDefaultAttributeValue(Attribute & attribute, const Key idx) const + { + switch (attribute.type) + { + case AttributeUnderlyingType::utUInt8: + std::get>(attribute.arrays)[idx] = std::get(attribute.null_values); + break; + case AttributeUnderlyingType::utUInt16: + std::get>(attribute.arrays)[idx] = std::get(attribute.null_values); + break; + case AttributeUnderlyingType::utUInt32: + std::get>(attribute.arrays)[idx] = std::get(attribute.null_values); + break; + case AttributeUnderlyingType::utUInt64: + std::get>(attribute.arrays)[idx] = std::get(attribute.null_values); + break; + case AttributeUnderlyingType::utUInt128: + std::get>(attribute.arrays)[idx] = std::get(attribute.null_values); + break; + case AttributeUnderlyingType::utInt8: + std::get>(attribute.arrays)[idx] = std::get(attribute.null_values); + break; + case AttributeUnderlyingType::utInt16: + std::get>(attribute.arrays)[idx] = std::get(attribute.null_values); + break; + case AttributeUnderlyingType::utInt32: + std::get>(attribute.arrays)[idx] = std::get(attribute.null_values); + break; + case AttributeUnderlyingType::utInt64: + std::get>(attribute.arrays)[idx] = std::get(attribute.null_values); + break; + case AttributeUnderlyingType::utFloat32: + std::get>(attribute.arrays)[idx] = std::get(attribute.null_values); + break; + case AttributeUnderlyingType::utFloat64: + std::get>(attribute.arrays)[idx] = std::get(attribute.null_values); + break; + + case AttributeUnderlyingType::utDecimal32: + std::get>(attribute.arrays)[idx] = std::get(attribute.null_values); + break; + case AttributeUnderlyingType::utDecimal64: + std::get>(attribute.arrays)[idx] = std::get(attribute.null_values); + break; + case AttributeUnderlyingType::utDecimal128: + std::get>(attribute.arrays)[idx] = std::get(attribute.null_values); + break; + + case AttributeUnderlyingType::utString: + { + const auto & null_value_ref = std::get(attribute.null_values); + auto & string_ref = std::get>(attribute.arrays)[idx]; + + if (string_ref.data != null_value_ref.data()) + { + if (string_ref.data) + string_arena->free(const_cast(string_ref.data), string_ref.size); + + string_ref = StringRef{null_value_ref}; + } + + break; + } + } + } + + void DirectDictionary::setAttributeValue(Attribute & attribute, const Key idx, const Field & value) const + { + switch (attribute.type) + { + case AttributeUnderlyingType::utUInt8: + std::get>(attribute.arrays)[idx] = value.get(); + break; + case AttributeUnderlyingType::utUInt16: + std::get>(attribute.arrays)[idx] = value.get(); + break; + case AttributeUnderlyingType::utUInt32: + std::get>(attribute.arrays)[idx] = value.get(); + break; + case AttributeUnderlyingType::utUInt64: + std::get>(attribute.arrays)[idx] = value.get(); + break; + case AttributeUnderlyingType::utUInt128: + std::get>(attribute.arrays)[idx] = value.get(); + break; + case AttributeUnderlyingType::utInt8: + std::get>(attribute.arrays)[idx] = value.get(); + break; + case AttributeUnderlyingType::utInt16: + std::get>(attribute.arrays)[idx] = value.get(); + break; + case AttributeUnderlyingType::utInt32: + std::get>(attribute.arrays)[idx] = value.get(); + break; + case AttributeUnderlyingType::utInt64: + std::get>(attribute.arrays)[idx] = value.get(); + break; + case AttributeUnderlyingType::utFloat32: + std::get>(attribute.arrays)[idx] = value.get(); + break; + case AttributeUnderlyingType::utFloat64: + std::get>(attribute.arrays)[idx] = value.get(); + break; + + case AttributeUnderlyingType::utDecimal32: + std::get>(attribute.arrays)[idx] = value.get(); + break; + case AttributeUnderlyingType::utDecimal64: + std::get>(attribute.arrays)[idx] = value.get(); + break; + case AttributeUnderlyingType::utDecimal128: + std::get>(attribute.arrays)[idx] = value.get(); + break; + + case AttributeUnderlyingType::utString: + { + const auto & string = value.get(); + auto & string_ref = std::get>(attribute.arrays)[idx]; + const auto & null_value_ref = std::get(attribute.null_values); + + /// free memory unless it points to a null_value + if (string_ref.data && string_ref.data != null_value_ref.data()) + string_arena->free(const_cast(string_ref.data), string_ref.size); + + const auto str_size = string.size(); + if (str_size != 0) + { + auto string_ptr = string_arena->alloc(str_size + 1); + std::copy(string.data(), string.data() + str_size + 1, string_ptr); + string_ref = StringRef{string_ptr, str_size}; + } + else + string_ref = {}; + + break; + } + } + } + + DirectDictionary::Attribute & DirectDictionary::getAttribute(const std::string & attribute_name) const + { + const auto it = attribute_index_by_name.find(attribute_name); + if (it == std::end(attribute_index_by_name)) + throw Exception{name + ": no such attribute '" + attribute_name + "'", ErrorCodes::BAD_ARGUMENTS}; + + return attributes[it->second]; + } + + /* + * I've deleted: + * bool CacheDictionary::isEmptyCell(const UInt64 idx) const + * and + * PaddedPODArray CacheDictionary::getCachedIds() const + */ + + BlockInputStreamPtr DirectDictionary::getBlockInputStream(const Names & column_names, size_t max_block_size) const + { + using BlockInputStreamType = DictionaryBlockInputStream; + + /* deleted pre-last argument getCachedIds() from this return (will something break then?) */ + return std::make_shared(shared_from_this(), max_block_size, column_names); + } + + std::exception_ptr DirectDictionary::getLastException() const + { + const ProfilingScopedReadRWLock read_lock{rw_lock, ProfileEvents::DictCacheLockReadNs}; + return last_exception; + } + + void registerDictionaryDirect(DictionaryFactory & factory) + { + auto create_layout = [=](const std::string & name, + const DictionaryStructure & dict_struct, + const Poco::Util::AbstractConfiguration & config, + const std::string & config_prefix, + DictionarySourcePtr source_ptr) -> DictionaryPtr + { + if (dict_struct.key) + throw Exception{"'key' is not supported for dictionary of layout 'cache'", ErrorCodes::UNSUPPORTED_METHOD}; + + if (dict_struct.range_min || dict_struct.range_max) + throw Exception{name + + ": elements .structure.range_min and .structure.range_max should be defined only " + "for a dictionary of layout 'range_hashed'", + ErrorCodes::BAD_ARGUMENTS}; + const auto & layout_prefix = config_prefix + ".layout"; + + /* + * + * seems like this stands only for cache dictionaries + * + const auto size = config.getInt(layout_prefix + ".cache.size_in_cells"); + if (size == 0) + throw Exception{name + ": dictionary of layout 'cache' cannot have 0 cells", ErrorCodes::TOO_SMALL_BUFFER_SIZE}; + + */ + + const bool require_nonempty = config.getBool(config_prefix + ".require_nonempty", false); + if (require_nonempty) + throw Exception{name + ": dictionary of layout 'cache' cannot have 'require_nonempty' attribute set", + ErrorCodes::BAD_ARGUMENTS}; + + const DictionaryLifetime dict_lifetime{config, config_prefix + ".lifetime"}; + + /* deleted last argument (size) in this return */ + return std::make_unique(name, dict_struct, std::move(source_ptr), dict_lifetime); + }; + factory.registerLayout("direct", create_layout, false); + } + + +} diff --git a/dbms/src/Dictionaries/DirectDictionary.h b/dbms/src/Dictionaries/DirectDictionary.h new file mode 100644 index 00000000000..1431adccf02 --- /dev/null +++ b/dbms/src/Dictionaries/DirectDictionary.h @@ -0,0 +1,294 @@ +#pragma once + +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include "DictionaryStructure.h" +#include "IDictionary.h" +#include "IDictionarySource.h" + + +namespace DB +{ + class DirectDictionary final : public IDictionary + { + public: + /* Removed last argument (size_) */ + DirectDictionary( + const std::string & name_, + const DictionaryStructure & dict_struct_, + DictionarySourcePtr source_ptr_, + const DictionaryLifetime dict_lifetime_ + ); + + std::string getName() const override { return name; } + + std::string getTypeName() const override { return "Cache"; } + + size_t getBytesAllocated() const override { return bytes_allocated + (string_arena ? string_arena->size() : 0); } + + size_t getQueryCount() const override { return query_count.load(std::memory_order_relaxed); } + + double getHitRate() const override + { + return static_cast(hit_count.load(std::memory_order_acquire)) / query_count.load(std::memory_order_relaxed); + } + + size_t getElementCount() const override { return element_count.load(std::memory_order_relaxed); } + + double getLoadFactor() const override { return static_cast(element_count.load(std::memory_order_relaxed)) / size; } + + bool isCached() const override { return false; } + + std::shared_ptr clone() const override + { + return std::make_shared(name, dict_struct, source_ptr->clone(), dict_lifetime, size); + } + + const IDictionarySource * getSource() const override { return source_ptr.get(); } + + const DictionaryLifetime & getLifetime() const override { return dict_lifetime; } + + const DictionaryStructure & getStructure() const override { return dict_struct; } + + bool isInjective(const std::string & attribute_name) const override + { + return dict_struct.attributes[&getAttribute(attribute_name) - attributes.data()].injective; + } + + bool hasHierarchy() const override { return hierarchical_attribute; } + + void toParent(const PaddedPODArray & ids, PaddedPODArray & out) const override; + + void isInVectorVector( + const PaddedPODArray & child_ids, const PaddedPODArray & ancestor_ids, PaddedPODArray & out) const override; + void isInVectorConstant(const PaddedPODArray & child_ids, const Key ancestor_id, PaddedPODArray & out) const override; + void isInConstantVector(const Key child_id, const PaddedPODArray & ancestor_ids, PaddedPODArray & out) const override; + + std::exception_ptr getLastException() const override; + + template + using ResultArrayType = std::conditional_t, DecimalPaddedPODArray, PaddedPODArray>; + +#define DECLARE(TYPE) \ + void get##TYPE(const std::string & attribute_name, const PaddedPODArray & ids, ResultArrayType & out) const; + DECLARE(UInt8) + DECLARE(UInt16) + DECLARE(UInt32) + DECLARE(UInt64) + DECLARE(UInt128) + DECLARE(Int8) + DECLARE(Int16) + DECLARE(Int32) + DECLARE(Int64) + DECLARE(Float32) + DECLARE(Float64) + DECLARE(Decimal32) + DECLARE(Decimal64) + DECLARE(Decimal128) +#undef DECLARE + + void getString(const std::string & attribute_name, const PaddedPODArray & ids, ColumnString * out) const; + +#define DECLARE(TYPE) \ + void get##TYPE( \ + const std::string & attribute_name, \ + const PaddedPODArray & ids, \ + const PaddedPODArray & def, \ + ResultArrayType & out) const; + DECLARE(UInt8) + DECLARE(UInt16) + DECLARE(UInt32) + DECLARE(UInt64) + DECLARE(UInt128) + DECLARE(Int8) + DECLARE(Int16) + DECLARE(Int32) + DECLARE(Int64) + DECLARE(Float32) + DECLARE(Float64) + DECLARE(Decimal32) + DECLARE(Decimal64) + DECLARE(Decimal128) +#undef DECLARE + + void + getString(const std::string & attribute_name, const PaddedPODArray & ids, const ColumnString * const def, ColumnString * const out) + const; + +#define DECLARE(TYPE) \ + void get##TYPE(const std::string & attribute_name, const PaddedPODArray & ids, const TYPE def, ResultArrayType & out) const; + DECLARE(UInt8) + DECLARE(UInt16) + DECLARE(UInt32) + DECLARE(UInt64) + DECLARE(UInt128) + DECLARE(Int8) + DECLARE(Int16) + DECLARE(Int32) + DECLARE(Int64) + DECLARE(Float32) + DECLARE(Float64) + DECLARE(Decimal32) + DECLARE(Decimal64) + DECLARE(Decimal128) +#undef DECLARE + + void getString(const std::string & attribute_name, const PaddedPODArray & ids, const String & def, ColumnString * const out) const; + + void has(const PaddedPODArray & ids, PaddedPODArray & out) const override; + + BlockInputStreamPtr getBlockInputStream(const Names & column_names, size_t max_block_size) const override; + + private: + template + using ContainerType = Value[]; + template + using ContainerPtrType = std::unique_ptr>; + + struct CellMetadata final + { + using time_point_t = std::chrono::system_clock::time_point; + using time_point_rep_t = time_point_t::rep; + using time_point_urep_t = std::make_unsigned_t; + + static constexpr UInt64 EXPIRES_AT_MASK = std::numeric_limits::max(); + static constexpr UInt64 IS_DEFAULT_MASK = ~EXPIRES_AT_MASK; + + UInt64 id; + /// Stores both expiration time and `is_default` flag in the most significant bit + time_point_urep_t data; + + /// Sets expiration time, resets `is_default` flag to false + time_point_t expiresAt() const { return ext::safe_bit_cast(data & EXPIRES_AT_MASK); } + void setExpiresAt(const time_point_t & t) { data = ext::safe_bit_cast(t); } + + bool isDefault() const { return (data & IS_DEFAULT_MASK) == IS_DEFAULT_MASK; } + void setDefault() { data |= IS_DEFAULT_MASK; } + }; + + struct Attribute final + { + AttributeUnderlyingType type; + std::variant< + UInt8, + UInt16, + UInt32, + UInt64, + UInt128, + Int8, + Int16, + Int32, + Int64, + Decimal32, + Decimal64, + Decimal128, + Float32, + Float64, + String> + null_values; + std::variant< + ContainerPtrType, + ContainerPtrType, + ContainerPtrType, + ContainerPtrType, + ContainerPtrType, + ContainerPtrType, + ContainerPtrType, + ContainerPtrType, + ContainerPtrType, + ContainerPtrType, + ContainerPtrType, + ContainerPtrType, + ContainerPtrType, + ContainerPtrType, + ContainerPtrType> + arrays; + }; + + void createAttributes(); + + Attribute createAttributeWithType(const AttributeUnderlyingType type, const Field & null_value); + + template + void getItemsNumberImpl( + Attribute & attribute, const PaddedPODArray & ids, ResultArrayType & out, DefaultGetter && get_default) const; + + template + void getItemsString(Attribute & attribute, const PaddedPODArray & ids, ColumnString * out, DefaultGetter && get_default) const; + + template + void update(const std::vector & requested_ids, PresentIdHandler && on_cell_updated, AbsentIdHandler && on_id_not_found) const; + + PaddedPODArray getCachedIds() const; + + bool isEmptyCell(const UInt64 idx) const; + + size_t getCellIdx(const Key id) const; + + void setDefaultAttributeValue(Attribute & attribute, const Key idx) const; + + void setAttributeValue(Attribute & attribute, const Key idx, const Field & value) const; + + Attribute & getAttribute(const std::string & attribute_name) const; + + struct FindResult + { + const size_t cell_idx; + const bool valid; + const bool outdated; + }; + + FindResult findCellIdx(const Key & id, const CellMetadata::time_point_t now) const; + + template + void isInImpl(const PaddedPODArray & child_ids, const AncestorType & ancestor_ids, PaddedPODArray & out) const; + + const std::string name; + const DictionaryStructure dict_struct; + mutable DictionarySourcePtr source_ptr; + const DictionaryLifetime dict_lifetime; + Logger * const log; + + mutable std::shared_mutex rw_lock; + + /// Actual size will be increased to match power of 2 + const size_t size; + + /// all bits to 1 mask (size - 1) (0b1000 - 1 = 0b111) + const size_t size_overlap_mask; + + /// Max tries to find cell, overlaped with mask: if size = 16 and start_cell=10: will try cells: 10,11,12,13,14,15,0,1,2,3 + static constexpr size_t max_collision_length = 10; + + const size_t zero_cell_idx{getCellIdx(0)}; + std::map attribute_index_by_name; + mutable std::vector attributes; + mutable std::vector cells; + Attribute * hierarchical_attribute = nullptr; + std::unique_ptr string_arena; + + mutable std::exception_ptr last_exception; + mutable size_t error_count = 0; + mutable std::chrono::system_clock::time_point backoff_end_time; + + mutable pcg64 rnd_engine; + + mutable size_t bytes_allocated = 0; + mutable std::atomic element_count{0}; + mutable std::atomic hit_count{0}; + mutable std::atomic query_count{0}; + }; + +} diff --git a/dbms/src/Dictionaries/DirectDictionary.inc.h b/dbms/src/Dictionaries/DirectDictionary.inc.h new file mode 100644 index 00000000000..68010b0fe19 --- /dev/null +++ b/dbms/src/Dictionaries/DirectDictionary.inc.h @@ -0,0 +1,406 @@ +#include "CacheDictionary.h" + +#include +#include +#include +#include +#include +#include +#include +#include + +namespace ProfileEvents +{ + extern const Event DictCacheKeysRequested; + extern const Event DictCacheKeysRequestedMiss; + extern const Event DictCacheKeysRequestedFound; + extern const Event DictCacheKeysExpired; + extern const Event DictCacheKeysNotFound; + extern const Event DictCacheKeysHit; + extern const Event DictCacheRequestTimeNs; + extern const Event DictCacheRequests; + extern const Event DictCacheLockWriteNs; + extern const Event DictCacheLockReadNs; +} + +namespace CurrentMetrics +{ + extern const Metric DictCacheRequests; +} + +namespace DB +{ + namespace ErrorCodes + { + extern const int TYPE_MISMATCH; + } + + template + void CacheDictionary::getItemsNumberImpl( + Attribute & attribute, const PaddedPODArray & ids, ResultArrayType & out, DefaultGetter && get_default) const + { + /// Mapping: -> { all indices `i` of `ids` such that `ids[i]` = } + std::unordered_map> outdated_ids; + auto & attribute_array = std::get>(attribute.arrays); + const auto rows = ext::size(ids); + + size_t cache_expired = 0, cache_not_found = 0, cache_hit = 0; + + { + const ProfilingScopedReadRWLock read_lock{rw_lock, ProfileEvents::DictCacheLockReadNs}; + + const auto now = std::chrono::system_clock::now(); + /// fetch up-to-date values, decide which ones require update + for (const auto row : ext::range(0, rows)) + { + const auto id = ids[row]; + + /** cell should be updated if either: + * 1. ids do not match, + * 2. cell has expired, + * 3. explicit defaults were specified and cell was set default. */ + + const auto find_result = findCellIdx(id, now); + if (!find_result.valid) + { + outdated_ids[id].push_back(row); + if (find_result.outdated) + ++cache_expired; + else + ++cache_not_found; + } + else + { + ++cache_hit; + const auto & cell_idx = find_result.cell_idx; + const auto & cell = cells[cell_idx]; + out[row] = cell.isDefault() ? get_default(row) : static_cast(attribute_array[cell_idx]); + } + } + } + + ProfileEvents::increment(ProfileEvents::DictCacheKeysExpired, cache_expired); + ProfileEvents::increment(ProfileEvents::DictCacheKeysNotFound, cache_not_found); + ProfileEvents::increment(ProfileEvents::DictCacheKeysHit, cache_hit); + + query_count.fetch_add(rows, std::memory_order_relaxed); + hit_count.fetch_add(rows - outdated_ids.size(), std::memory_order_release); + + if (outdated_ids.empty()) + return; + + std::vector required_ids(outdated_ids.size()); + std::transform(std::begin(outdated_ids), std::end(outdated_ids), std::begin(required_ids), [](auto & pair) { return pair.first; }); + + /// request new values + update( + required_ids, + [&](const auto id, const auto cell_idx) + { + const auto attribute_value = attribute_array[cell_idx]; + + for (const size_t row : outdated_ids[id]) + out[row] = static_cast(attribute_value); + }, + [&](const auto id, const auto) + { + for (const size_t row : outdated_ids[id]) + out[row] = get_default(row); + }); + } + + template + void CacheDictionary::getItemsString( + Attribute & attribute, const PaddedPODArray & ids, ColumnString * out, DefaultGetter && get_default) const + { + const auto rows = ext::size(ids); + + /// save on some allocations + out->getOffsets().reserve(rows); + + auto & attribute_array = std::get>(attribute.arrays); + + auto found_outdated_values = false; + + /// perform optimistic version, fallback to pessimistic if failed + { + const ProfilingScopedReadRWLock read_lock{rw_lock, ProfileEvents::DictCacheLockReadNs}; + + const auto now = std::chrono::system_clock::now(); + /// fetch up-to-date values, discard on fail + for (const auto row : ext::range(0, rows)) + { + const auto id = ids[row]; + + const auto find_result = findCellIdx(id, now); + if (!find_result.valid) + { + found_outdated_values = true; + break; + } + else + { + const auto & cell_idx = find_result.cell_idx; + const auto & cell = cells[cell_idx]; + const auto string_ref = cell.isDefault() ? get_default(row) : attribute_array[cell_idx]; + out->insertData(string_ref.data, string_ref.size); + } + } + } + + /// optimistic code completed successfully + if (!found_outdated_values) + { + query_count.fetch_add(rows, std::memory_order_relaxed); + hit_count.fetch_add(rows, std::memory_order_release); + return; + } + + /// now onto the pessimistic one, discard possible partial results from the optimistic path + out->getChars().resize_assume_reserved(0); + out->getOffsets().resize_assume_reserved(0); + + /// Mapping: -> { all indices `i` of `ids` such that `ids[i]` = } + std::unordered_map> outdated_ids; + /// we are going to store every string separately + std::unordered_map map; + + size_t total_length = 0; + size_t cache_expired = 0, cache_not_found = 0, cache_hit = 0; + { + const ProfilingScopedReadRWLock read_lock{rw_lock, ProfileEvents::DictCacheLockReadNs}; + + const auto now = std::chrono::system_clock::now(); + for (const auto row : ext::range(0, ids.size())) + { + const auto id = ids[row]; + + const auto find_result = findCellIdx(id, now); + if (!find_result.valid) + { + outdated_ids[id].push_back(row); + if (find_result.outdated) + ++cache_expired; + else + ++cache_not_found; + } + else + { + ++cache_hit; + const auto & cell_idx = find_result.cell_idx; + const auto & cell = cells[cell_idx]; + const auto string_ref = cell.isDefault() ? get_default(row) : attribute_array[cell_idx]; + + if (!cell.isDefault()) + map[id] = String{string_ref}; + + total_length += string_ref.size + 1; + } + } + } + + ProfileEvents::increment(ProfileEvents::DictCacheKeysExpired, cache_expired); + ProfileEvents::increment(ProfileEvents::DictCacheKeysNotFound, cache_not_found); + ProfileEvents::increment(ProfileEvents::DictCacheKeysHit, cache_hit); + + query_count.fetch_add(rows, std::memory_order_relaxed); + hit_count.fetch_add(rows - outdated_ids.size(), std::memory_order_release); + + /// request new values + if (!outdated_ids.empty()) + { + std::vector required_ids(outdated_ids.size()); + std::transform(std::begin(outdated_ids), std::end(outdated_ids), std::begin(required_ids), [](auto & pair) { return pair.first; }); + + update( + required_ids, + [&](const auto id, const auto cell_idx) + { + const auto attribute_value = attribute_array[cell_idx]; + + map[id] = String{attribute_value}; + total_length += (attribute_value.size + 1) * outdated_ids[id].size(); + }, + [&](const auto id, const auto) + { + for (const auto row : outdated_ids[id]) + total_length += get_default(row).size + 1; + }); + } + + out->getChars().reserve(total_length); + + for (const auto row : ext::range(0, ext::size(ids))) + { + const auto id = ids[row]; + const auto it = map.find(id); + + const auto string_ref = it != std::end(map) ? StringRef{it->second} : get_default(row); + out->insertData(string_ref.data, string_ref.size); + } + } + + template + void CacheDictionary::update( + const std::vector & requested_ids, PresentIdHandler && on_cell_updated, AbsentIdHandler && on_id_not_found) const + { + CurrentMetrics::Increment metric_increment{CurrentMetrics::DictCacheRequests}; + ProfileEvents::increment(ProfileEvents::DictCacheKeysRequested, requested_ids.size()); + + std::unordered_map remaining_ids{requested_ids.size()}; + for (const auto id : requested_ids) + remaining_ids.insert({id, 0}); + + const auto now = std::chrono::system_clock::now(); + + const ProfilingScopedWriteRWLock write_lock{rw_lock, ProfileEvents::DictCacheLockWriteNs}; + + if (now > backoff_end_time) + { + try + { + if (error_count) + { + /// Recover after error: we have to clone the source here because + /// it could keep connections which should be reset after error. + source_ptr = source_ptr->clone(); + } + + Stopwatch watch; + auto stream = source_ptr->loadIds(requested_ids); + stream->readPrefix(); + + while (const auto block = stream->read()) + { + const auto id_column = typeid_cast(block.safeGetByPosition(0).column.get()); + if (!id_column) + throw Exception{name + ": id column has type different from UInt64.", ErrorCodes::TYPE_MISMATCH}; + + const auto & ids = id_column->getData(); + + /// cache column pointers + const auto column_ptrs = ext::map( + ext::range(0, attributes.size()), [&block](size_t i) { return block.safeGetByPosition(i + 1).column.get(); }); + + for (const auto i : ext::range(0, ids.size())) + { + const auto id = ids[i]; + + const auto find_result = findCellIdx(id, now); + const auto & cell_idx = find_result.cell_idx; + + auto & cell = cells[cell_idx]; + + for (const auto attribute_idx : ext::range(0, attributes.size())) + { + const auto & attribute_column = *column_ptrs[attribute_idx]; + auto & attribute = attributes[attribute_idx]; + + setAttributeValue(attribute, cell_idx, attribute_column[i]); + } + + /// if cell id is zero and zero does not map to this cell, then the cell is unused + if (cell.id == 0 && cell_idx != zero_cell_idx) + element_count.fetch_add(1, std::memory_order_relaxed); + + cell.id = id; + if (dict_lifetime.min_sec != 0 && dict_lifetime.max_sec != 0) + { + std::uniform_int_distribution distribution{dict_lifetime.min_sec, dict_lifetime.max_sec}; + cell.setExpiresAt(now + std::chrono::seconds{distribution(rnd_engine)}); + } + else + cell.setExpiresAt(std::chrono::time_point::max()); + + /// inform caller + on_cell_updated(id, cell_idx); + /// mark corresponding id as found + remaining_ids[id] = 1; + } + } + + stream->readSuffix(); + + error_count = 0; + last_exception = std::exception_ptr{}; + backoff_end_time = std::chrono::system_clock::time_point{}; + + ProfileEvents::increment(ProfileEvents::DictCacheRequestTimeNs, watch.elapsed()); + } + catch (...) + { + ++error_count; + last_exception = std::current_exception(); + backoff_end_time = now + std::chrono::seconds(calculateDurationWithBackoff(rnd_engine, error_count)); + + tryLogException(last_exception, log, "Could not update cache dictionary '" + getName() + + "', next update is scheduled at " + DateLUT::instance().timeToString(std::chrono::system_clock::to_time_t(backoff_end_time))); + } + } + + size_t not_found_num = 0, found_num = 0; + + /// Check which ids have not been found and require setting null_value + for (const auto & id_found_pair : remaining_ids) + { + if (id_found_pair.second) + { + ++found_num; + continue; + } + ++not_found_num; + + const auto id = id_found_pair.first; + + const auto find_result = findCellIdx(id, now); + const auto & cell_idx = find_result.cell_idx; + auto & cell = cells[cell_idx]; + + if (error_count) + { + if (find_result.outdated) + { + /// We have expired data for that `id` so we can continue using it. + bool was_default = cell.isDefault(); + cell.setExpiresAt(backoff_end_time); + if (was_default) + cell.setDefault(); + if (was_default) + on_id_not_found(id, cell_idx); + else + on_cell_updated(id, cell_idx); + continue; + } + /// We don't have expired data for that `id` so all we can do is to rethrow `last_exception`. + std::rethrow_exception(last_exception); + } + + /// Check if cell had not been occupied before and increment element counter if it hadn't + if (cell.id == 0 && cell_idx != zero_cell_idx) + element_count.fetch_add(1, std::memory_order_relaxed); + + cell.id = id; + + if (dict_lifetime.min_sec != 0 && dict_lifetime.max_sec != 0) + { + std::uniform_int_distribution distribution{dict_lifetime.min_sec, dict_lifetime.max_sec}; + cell.setExpiresAt(now + std::chrono::seconds{distribution(rnd_engine)}); + } + else + cell.setExpiresAt(std::chrono::time_point::max()); + + /// Set null_value for each attribute + cell.setDefault(); + for (auto & attribute : attributes) + setDefaultAttributeValue(attribute, cell_idx); + + /// inform caller that the cell has not been found + on_id_not_found(id, cell_idx); + } + + ProfileEvents::increment(ProfileEvents::DictCacheKeysRequestedMiss, not_found_num); + ProfileEvents::increment(ProfileEvents::DictCacheKeysRequestedFound, found_num); + ProfileEvents::increment(ProfileEvents::DictCacheRequests); + } + +} diff --git a/dbms/src/Dictionaries/registerDictionaries.cpp b/dbms/src/Dictionaries/registerDictionaries.cpp index ee320d7177b..44cb1f2858f 100644 --- a/dbms/src/Dictionaries/registerDictionaries.cpp +++ b/dbms/src/Dictionaries/registerDictionaries.cpp @@ -21,6 +21,7 @@ void registerDictionaryTrie(DictionaryFactory & factory); void registerDictionaryFlat(DictionaryFactory & factory); void registerDictionaryHashed(DictionaryFactory & factory); void registerDictionaryCache(DictionaryFactory & factory); +void registerDictionaryDirect(DictionaryFactory & factory); void registerDictionaries() @@ -48,6 +49,8 @@ void registerDictionaries() registerDictionaryFlat(factory); registerDictionaryHashed(factory); registerDictionaryCache(factory); + /// added new dictionary type (i bet i wont forget to remove this comment) + registerDictionaryDirect(factory); } } From b470ff7aaedd10c24752d68fa7899d52f258980a Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Mon, 3 Feb 2020 06:11:12 +0300 Subject: [PATCH 0012/1355] Update array_functions.md --- docs/en/query_language/functions/array_functions.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/query_language/functions/array_functions.md b/docs/en/query_language/functions/array_functions.md index 8c9b2a7c151..35aa9c98dfb 100644 --- a/docs/en/query_language/functions/array_functions.md +++ b/docs/en/query_language/functions/array_functions.md @@ -898,7 +898,7 @@ Result: ``` ## arrayAUC(arr_scores, arr_labels) -Returns AUC(Area Under the Curve, which is a concept in machine learning, see more details: https://developers.google.com/machine-learning/crash-course/classification/roc-and-auc); +Returns AUC (Area Under the Curve, which is a concept in machine learning, see more details: https://developers.google.com/machine-learning/crash-course/classification/roc-and-auc). `arr_scores` represents scores prediction model gives, while `arr_labels` represents labels of samples, usually 1 for positive sample and 0 for negtive sample. From 71d46bb5219e620126347643708990dda8a25f0e Mon Sep 17 00:00:00 2001 From: liyang Date: Sat, 8 Feb 2020 14:31:03 +0800 Subject: [PATCH 0013/1355] modify to suggestions --- dbms/src/Functions/array/arrayScalarProduct.h | 5 +-- dbms/src/Functions/array/array_auc.cpp | 43 ++++++++++++++++--- .../functions/array_functions.md | 28 ++++++++++-- 3 files changed, 63 insertions(+), 13 deletions(-) diff --git a/dbms/src/Functions/array/arrayScalarProduct.h b/dbms/src/Functions/array/arrayScalarProduct.h index c450f41f48a..87f618b5b4f 100644 --- a/dbms/src/Functions/array/arrayScalarProduct.h +++ b/dbms/src/Functions/array/arrayScalarProduct.h @@ -109,11 +109,10 @@ public: { const DataTypeArray * array_type = checkAndGetDataType(arguments[i].get()); if (!array_type) - throw Exception("All argument for function " + getName() + " must be an array.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + throw Exception("All arguments for function " + getName() + " must be an array.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); auto & nested_type = array_type->getNestedType(); - WhichDataType which(nested_type); - bool is_number = which.isNativeInt() || which.isNativeUInt() || which.isFloat(); + bool is_number = isNativeNumber(nested_type) if (!is_number) { throw Exception( diff --git a/dbms/src/Functions/array/array_auc.cpp b/dbms/src/Functions/array/array_auc.cpp index 41739446e46..861b617e3ea 100644 --- a/dbms/src/Functions/array/array_auc.cpp +++ b/dbms/src/Functions/array/array_auc.cpp @@ -18,16 +18,35 @@ public: struct ScoreLabel { ResultType score; - UInt8 label; + bool label; }; - static DataTypePtr getReturnType(const DataTypePtr & /* nested_type1 */, const DataTypePtr & nested_type2) + static DataTypePtr getReturnType(const DataTypePtr & /* score_type */, const DataTypePtr & label_type) { - WhichDataType which2(nested_type2); - if (!which2.isUInt8()) + WhichDataType which(label_type); + // Labels values are either {0, 1} or {-1, 1}, and its type must be one of (Enum8, UInt8, Int8) + if (!which.isUInt8() && !which.isEnum8() && !which.isInt8()) { throw Exception(std::string(NameArrayAUC::name) + "lable type must be UInt8", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); } + + if (which.isEnum8()) + { + auto type8 = checkAndGetDataType(label_type.get()); + if (type8) + throw Exception(std::string(NameArrayAUC::name) + "lable type not valid Enum8", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + + std::set valSet; + const auto & values = type8->getValues(); + for (const auto & value : values) + { + valSet.insert(value.second); + } + + if (valSet != {0, 1} || valSet != {-1, 1}) + throw Exception( + std::string(NameArrayAUC::name) + "lable values must be {0, 1} or {-1, 1}", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + } return std::make_shared>(); } @@ -45,25 +64,35 @@ public: if (score_len == 0) return {}; - // Order pairs of score and lable by score ascending + // Calculate positive and negative label number and restore scores and labels in vector size_t num_pos = 0; size_t num_neg = 0; + std::set labelValSet; std::vector pairs(score_len); for (size_t i = 0; i < score_len; ++i) { pairs[i].score = scores[i + score_offset]; - pairs[i].label = (labels[i + label_offset] ? 1 : 0); + pairs[i].label = (labels[i + label_offset] == 1); if (pairs[i].label) ++num_pos; else ++num_neg; + + labelValSet.insert(labels[i + label_offset]); } + + // Label values must be {0, 1} or {-1, 1} + if (labelValSet != {0, 1} && labelValSet != {-1, 1}) + throw Exception( + std::string(NameArrayAUC::name) + "lable values must be {0, 1} or {-1, 1}", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + + // Order pairs of score and lable by score ascending std::sort(pairs.begin(), pairs.end(), [](const auto & lhs, const auto & rhs) { return lhs.score < rhs.score; }); // Calculate AUC size_t curr_cnt = 0; size_t curr_pos_cnt = 0; - size_t curr_sum = 0; + Int64 curr_sum = 0; ResultType last_score = -1; ResultType rank_sum = 0; for (size_t i = 0; i < pairs.size(); ++i) diff --git a/docs/en/query_language/functions/array_functions.md b/docs/en/query_language/functions/array_functions.md index 35aa9c98dfb..3574f37a509 100644 --- a/docs/en/query_language/functions/array_functions.md +++ b/docs/en/query_language/functions/array_functions.md @@ -896,11 +896,33 @@ Result: │ [('a','d'),('b','e'),('c','f')] │ └────────────────────────────────────────────┘ ``` -## arrayAUC(arr_scores, arr_labels) +## arrayAUC {#arrayauc} +Calculate AUC(Area Under the Curve, which is a concept in machine learning, see more details: https://en.wikipedia.org/wiki/Receiver_operating_characteristic#Area_under_the_curve). -Returns AUC (Area Under the Curve, which is a concept in machine learning, see more details: https://developers.google.com/machine-learning/crash-course/classification/roc-and-auc). +**Syntax** +```sql +arrayAUC(arr_scores, arr_labels) +``` -`arr_scores` represents scores prediction model gives, while `arr_labels` represents labels of samples, usually 1 for positive sample and 0 for negtive sample. +**Parameters** +- `arr_scores` — scores prediction model gives. +- `arr_labels` — labels of samples, usually 1 for positive sample and 0 for negtive sample. +**Returned value** +return AUC value with type Float64. + +**Example** +Query: +```sql +select auc([0.1, 0.4, 0.35, 0.8], [0, 0, 1, 1]) +``` + +Result: + +```text +┌─arrayAUC([0.1, 0.4, 0.35, 0.8], [0, 0, 1, 1])─┐ +│ 0.75 │ +└────────────────────────────────────────-----──┘ +``` [Original article](https://clickhouse.yandex/docs/en/query_language/functions/array_functions/) From f0f0768e81504e9af675337a0d3d39d1b0b7351f Mon Sep 17 00:00:00 2001 From: liyang Date: Sun, 9 Feb 2020 10:45:58 +0800 Subject: [PATCH 0014/1355] fix bugs --- dbms/src/Functions/array/arrayScalarProduct.h | 6 ++--- dbms/src/Functions/array/array_auc.cpp | 26 ++++++++++--------- .../0_stateless/01064_array_auc.reference | 15 +++++++++++ .../queries/0_stateless/01064_array_auc.sql | 17 +++++++++++- 4 files changed, 47 insertions(+), 17 deletions(-) diff --git a/dbms/src/Functions/array/arrayScalarProduct.h b/dbms/src/Functions/array/arrayScalarProduct.h index 87f618b5b4f..1352038195b 100644 --- a/dbms/src/Functions/array/arrayScalarProduct.h +++ b/dbms/src/Functions/array/arrayScalarProduct.h @@ -4,6 +4,7 @@ #include #include #include +#include #include #include #include @@ -112,12 +113,9 @@ public: throw Exception("All arguments for function " + getName() + " must be an array.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); auto & nested_type = array_type->getNestedType(); - bool is_number = isNativeNumber(nested_type) - if (!is_number) - { + if (!isNativeNumber(nested_type) && !isEnum(nested_type)) throw Exception( getName() + " cannot process values of type " + nested_type->getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); - } nested_types[i] = nested_type; } diff --git a/dbms/src/Functions/array/array_auc.cpp b/dbms/src/Functions/array/array_auc.cpp index 861b617e3ea..3ed9cd275d3 100644 --- a/dbms/src/Functions/array/array_auc.cpp +++ b/dbms/src/Functions/array/array_auc.cpp @@ -14,6 +14,10 @@ class ArrayAUCImpl { public: using ResultType = Float64; + using LabelValueSet = std::set; + using LabelValueSets = std::vector; + + inline static const LabelValueSets expect_label_value_sets = {{0, 1}, {-1, 1}}; struct ScoreLabel { @@ -27,23 +31,22 @@ public: // Labels values are either {0, 1} or {-1, 1}, and its type must be one of (Enum8, UInt8, Int8) if (!which.isUInt8() && !which.isEnum8() && !which.isInt8()) { - throw Exception(std::string(NameArrayAUC::name) + "lable type must be UInt8", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + throw Exception( + std::string(NameArrayAUC::name) + "lable type must be UInt8, Enum8 or Int8", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); } if (which.isEnum8()) { auto type8 = checkAndGetDataType(label_type.get()); - if (type8) + if (!type8) throw Exception(std::string(NameArrayAUC::name) + "lable type not valid Enum8", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); - std::set valSet; + LabelValueSet value_set; const auto & values = type8->getValues(); for (const auto & value : values) - { - valSet.insert(value.second); - } + value_set.insert(value.second); - if (valSet != {0, 1} || valSet != {-1, 1}) + if (std::find(expect_label_value_sets.begin(), expect_label_value_sets.end(), value_set) == expect_label_value_sets.end()) throw Exception( std::string(NameArrayAUC::name) + "lable values must be {0, 1} or {-1, 1}", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); } @@ -67,7 +70,7 @@ public: // Calculate positive and negative label number and restore scores and labels in vector size_t num_pos = 0; size_t num_neg = 0; - std::set labelValSet; + LabelValueSet label_value_set; std::vector pairs(score_len); for (size_t i = 0; i < score_len; ++i) { @@ -78,11 +81,11 @@ public: else ++num_neg; - labelValSet.insert(labels[i + label_offset]); + label_value_set.insert(labels[i + label_offset]); } // Label values must be {0, 1} or {-1, 1} - if (labelValSet != {0, 1} && labelValSet != {-1, 1}) + if (std::find(expect_label_value_sets.begin(), expect_label_value_sets.end(), label_value_set) == expect_label_value_sets.end()) throw Exception( std::string(NameArrayAUC::name) + "lable values must be {0, 1} or {-1, 1}", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); @@ -119,6 +122,7 @@ public: } }; + /// auc(array_score, array_label) - Calculate AUC with array of score and label using FunctionArrayAUC = FunctionArrayScalarProduct; @@ -126,6 +130,4 @@ void registerFunctionArrayAUC(FunctionFactory & factory) { factory.registerFunction(); } - - } diff --git a/dbms/tests/queries/0_stateless/01064_array_auc.reference b/dbms/tests/queries/0_stateless/01064_array_auc.reference index 39c64a9f26e..8c17bba359a 100644 --- a/dbms/tests/queries/0_stateless/01064_array_auc.reference +++ b/dbms/tests/queries/0_stateless/01064_array_auc.reference @@ -1 +1,16 @@ 0.75 +0.75 +0.75 +0.75 +0.75 +0.75 +0.75 +0.75 +0.75 +0.25 +0.25 +0.25 +0.25 +0.25 +0.125 +0.25 diff --git a/dbms/tests/queries/0_stateless/01064_array_auc.sql b/dbms/tests/queries/0_stateless/01064_array_auc.sql index ca270937f63..de05c47c51b 100644 --- a/dbms/tests/queries/0_stateless/01064_array_auc.sql +++ b/dbms/tests/queries/0_stateless/01064_array_auc.sql @@ -1 +1,16 @@ -select arrayAUC([0.1, 0.4, 0.35, 0.8], [0, 0, 1, 1]) +select arrayAUC([0.1, 0.4, 0.35, 0.8], [0, 0, 1, 1]); +select arrayAUC([0.1, 0.4, 0.35, 0.8], cast([0, 0, 1, 1] as Array(Int8))); +select arrayAUC([0.1, 0.4, 0.35, 0.8], cast([-1, -1, 1, 1] as Array(Int8))); +select arrayAUC([0.1, 0.4, 0.35, 0.8], cast(['false', 'false', 'true', 'true'] as Array(Enum8('false' = 0, 'true' = 1)))); +select arrayAUC([0.1, 0.4, 0.35, 0.8], cast(['false', 'false', 'true', 'true'] as Array(Enum8('false' = -1, 'true' = 1)))); +select arrayAUC(cast([10, 40, 35, 80] as Array(UInt8)), [0, 0, 1, 1]); +select arrayAUC(cast([10, 40, 35, 80] as Array(UInt16)), [0, 0, 1, 1]); +select arrayAUC(cast([10, 40, 35, 80] as Array(UInt32)), [0, 0, 1, 1]); +select arrayAUC(cast([10, 40, 35, 80] as Array(UInt64)), [0, 0, 1, 1]); +select arrayAUC(cast([-10, -40, -35, -80] as Array(Int8)), [0, 0, 1, 1]); +select arrayAUC(cast([-10, -40, -35, -80] as Array(Int16)), [0, 0, 1, 1]); +select arrayAUC(cast([-10, -40, -35, -80] as Array(Int32)), [0, 0, 1, 1]); +select arrayAUC(cast([-10, -40, -35, -80] as Array(Int64)), [0, 0, 1, 1]); +select arrayAUC(cast([-0.1, -0.4, -0.35, -0.8] as Array(Float32)) , [0, 0, 1, 1]); +select arrayAUC([0, 3, 5, 6, 7.5, 8], [1, 0, 1, 0, 0, 0]); +select arrayAUC([0.1, 0.35, 0.4, 0.8], [1, 0, 1, 0]); \ No newline at end of file From fd87a5bb2d72ce1bf6df1ae7d7f760e936199954 Mon Sep 17 00:00:00 2001 From: liyang Date: Mon, 10 Feb 2020 09:45:30 +0800 Subject: [PATCH 0015/1355] fix error --- docs/en/query_language/functions/array_functions.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/query_language/functions/array_functions.md b/docs/en/query_language/functions/array_functions.md index 3574f37a509..101ba8a8b7f 100644 --- a/docs/en/query_language/functions/array_functions.md +++ b/docs/en/query_language/functions/array_functions.md @@ -914,7 +914,7 @@ return AUC value with type Float64. **Example** Query: ```sql -select auc([0.1, 0.4, 0.35, 0.8], [0, 0, 1, 1]) +select arrayAUC([0.1, 0.4, 0.35, 0.8], [0, 0, 1, 1]) ``` Result: From 1c41d73d9b232d674e1a64524d154a03ccc55099 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Mon, 10 Feb 2020 12:27:57 +0300 Subject: [PATCH 0016/1355] small refactoring + comments adding --- dbms/programs/copier/Internals.h | 8 ++++++++ dbms/src/Parsers/ASTLiteral.h | 4 ++-- 2 files changed, 10 insertions(+), 2 deletions(-) create mode 100644 dbms/programs/copier/Internals.h diff --git a/dbms/programs/copier/Internals.h b/dbms/programs/copier/Internals.h new file mode 100644 index 00000000000..a25ae7b973c --- /dev/null +++ b/dbms/programs/copier/Internals.h @@ -0,0 +1,8 @@ +// +// Created by jakalletti on 2/7/20. +// + +#ifndef CLICKHOUSE_INTERNALS_H +#define CLICKHOUSE_INTERNALS_H + +#endif //CLICKHOUSE_INTERNALS_H diff --git a/dbms/src/Parsers/ASTLiteral.h b/dbms/src/Parsers/ASTLiteral.h index 552f5da04a2..1d307a4101e 100644 --- a/dbms/src/Parsers/ASTLiteral.h +++ b/dbms/src/Parsers/ASTLiteral.h @@ -21,8 +21,8 @@ public: std::optional begin; std::optional end; - ASTLiteral(Field && value_) : value(value_) {} - ASTLiteral(const Field & value_) : value(value_) {} + explicit ASTLiteral(Field && value_) : value(value_) {} + explicit ASTLiteral(const Field & value_) : value(value_) {} /** Get the text that identifies this element. */ String getID(char delim) const override { return "Literal" + (delim + applyVisitor(FieldVisitorDump(), value)); } From b481682a1e067d44219bd47e752f685f1e704513 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Tue, 11 Feb 2020 14:19:19 +0300 Subject: [PATCH 0017/1355] just comments --- dbms/programs/copier/ClusterCopier.cpp | 872 ++++--------------------- dbms/programs/copier/ClusterCopier.h | 4 +- dbms/programs/copier/Internals.h | 750 ++++++++++++++++++++- 3 files changed, 854 insertions(+), 772 deletions(-) diff --git a/dbms/programs/copier/ClusterCopier.cpp b/dbms/programs/copier/ClusterCopier.cpp index 2c6b16a7ae4..e8b3018a861 100644 --- a/dbms/programs/copier/ClusterCopier.cpp +++ b/dbms/programs/copier/ClusterCopier.cpp @@ -1,87 +1,10 @@ #include "ClusterCopier.h" -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include - +#include "Internals.h" namespace DB { -namespace ErrorCodes -{ - extern const int NO_ZOOKEEPER; - extern const int BAD_ARGUMENTS; - extern const int UNKNOWN_TABLE; - extern const int UNFINISHED; - extern const int UNKNOWN_ELEMENT_IN_CONFIG; -} - using ConfigurationPtr = Poco::AutoPtr; @@ -92,661 +15,19 @@ static ConfigurationPtr getConfigurationFromXMLString(const std::string & xml_da return {new Poco::Util::XMLConfiguration{&input_source}}; } -namespace -{ - - -using DatabaseAndTableName = std::pair; - -String getQuotedTable(const String & database, const String & table) -{ - if (database.empty()) - { - return backQuoteIfNeed(table); - } - - return backQuoteIfNeed(database) + "." + backQuoteIfNeed(table); -} - -String getQuotedTable(const DatabaseAndTableName & db_and_table) -{ - return getQuotedTable(db_and_table.first, db_and_table.second); -} - - -enum class TaskState -{ - Started = 0, - Finished, - Unknown -}; - -/// Used to mark status of shard partition tasks -struct TaskStateWithOwner -{ - TaskStateWithOwner() = default; - TaskStateWithOwner(TaskState state_, const String & owner_) : state(state_), owner(owner_) {} - - TaskState state{TaskState::Unknown}; - String owner; - - static String getData(TaskState state, const String & owner) - { - return TaskStateWithOwner(state, owner).toString(); - } - - String toString() - { - WriteBufferFromOwnString wb; - wb << static_cast(state) << "\n" << escape << owner; - return wb.str(); - } - - static TaskStateWithOwner fromString(const String & data) - { - ReadBufferFromString rb(data); - TaskStateWithOwner res; - UInt32 state; - - rb >> state >> "\n" >> escape >> res.owner; - - if (state >= static_cast(TaskState::Unknown)) - throw Exception("Unknown state " + data, ErrorCodes::LOGICAL_ERROR); - - res.state = static_cast(state); - return res; - } -}; - - -/// Hierarchical description of the tasks -struct ShardPartition; -struct TaskShard; -struct TaskTable; -struct TaskCluster; -struct ClusterPartition; - -using TasksPartition = std::map>; -using ShardInfo = Cluster::ShardInfo; -using TaskShardPtr = std::shared_ptr; -using TasksShard = std::vector; -using TasksTable = std::list; -using ClusterPartitions = std::map>; - - -/// Just destination partition of a shard -struct ShardPartition -{ - ShardPartition(TaskShard & parent, const String & name_quoted_) : task_shard(parent), name(name_quoted_) {} - - String getPartitionPath() const; - String getPartitionCleanStartPath() const; - String getCommonPartitionIsDirtyPath() const; - String getCommonPartitionIsCleanedPath() const; - String getPartitionActiveWorkersPath() const; - String getActiveWorkerPath() const; - String getPartitionShardsPath() const; - String getShardStatusPath() const; - - TaskShard & task_shard; - String name; -}; - - -struct ShardPriority -{ - UInt8 is_remote = 1; - size_t hostname_difference = 0; - UInt8 random = 0; - - static bool greaterPriority(const ShardPriority & current, const ShardPriority & other) - { - return std::forward_as_tuple(current.is_remote, current.hostname_difference, current.random) - < std::forward_as_tuple(other.is_remote, other.hostname_difference, other.random); - } -}; - - -struct TaskShard -{ - TaskShard(TaskTable & parent, const ShardInfo & info_) : task_table(parent), info(info_) {} - - TaskTable & task_table; - - ShardInfo info; - UInt32 numberInCluster() const { return info.shard_num; } - UInt32 indexInCluster() const { return info.shard_num - 1; } - - String getDescription() const; - String getHostNameExample() const; - - /// Used to sort clusters by their proximity - ShardPriority priority; - - /// Column with unique destination partitions (computed from engine_push_partition_key expr.) in the shard - ColumnWithTypeAndName partition_key_column; - - /// There is a task for each destination partition - TasksPartition partition_tasks; - - /// Which partitions have been checked for existence - /// If some partition from this lists is exists, it is in partition_tasks - std::set checked_partitions; - - /// Last CREATE TABLE query of the table of the shard - ASTPtr current_pull_table_create_query; - - /// Internal distributed tables - DatabaseAndTableName table_read_shard; - DatabaseAndTableName table_split_shard; -}; - - -/// Contains info about all shards that contain a partition -struct ClusterPartition -{ - double elapsed_time_seconds = 0; - UInt64 bytes_copied = 0; - UInt64 rows_copied = 0; - UInt64 blocks_copied = 0; - - UInt64 total_tries = 0; -}; - - -struct TaskTable -{ - TaskTable(TaskCluster & parent, const Poco::Util::AbstractConfiguration & config, const String & prefix, - const String & table_key); - - TaskCluster & task_cluster; - - String getPartitionPath(const String & partition_name) const; - String getPartitionIsDirtyPath(const String & partition_name) const; - String getPartitionIsCleanedPath(const String & partition_name) const; - String getPartitionTaskStatusPath(const String & partition_name) const; - - String name_in_config; - - /// Used as task ID - String table_id; - - /// Source cluster and table - String cluster_pull_name; - DatabaseAndTableName table_pull; - - /// Destination cluster and table - String cluster_push_name; - DatabaseAndTableName table_push; - - /// Storage of destination table - String engine_push_str; - ASTPtr engine_push_ast; - ASTPtr engine_push_partition_key_ast; - - /// A Distributed table definition used to split data - String sharding_key_str; - ASTPtr sharding_key_ast; - ASTPtr engine_split_ast; - - /// Additional WHERE expression to filter input data - String where_condition_str; - ASTPtr where_condition_ast; - - /// Resolved clusters - ClusterPtr cluster_pull; - ClusterPtr cluster_push; - - /// Filter partitions that should be copied - bool has_enabled_partitions = false; - Strings enabled_partitions; - NameSet enabled_partitions_set; - - /// Prioritized list of shards - TasksShard all_shards; - TasksShard local_shards; - - ClusterPartitions cluster_partitions; - NameSet finished_cluster_partitions; - - /// Parition names to process in user-specified order - Strings ordered_partition_names; - - ClusterPartition & getClusterPartition(const String & partition_name) - { - auto it = cluster_partitions.find(partition_name); - if (it == cluster_partitions.end()) - throw Exception("There are no cluster partition " + partition_name + " in " + table_id, ErrorCodes::LOGICAL_ERROR); - return it->second; - } - - Stopwatch watch; - UInt64 bytes_copied = 0; - UInt64 rows_copied = 0; - - template - void initShards(RandomEngine && random_engine); -}; - - -struct TaskCluster -{ - TaskCluster(const String & task_zookeeper_path_, const String & default_local_database_) - : task_zookeeper_path(task_zookeeper_path_), default_local_database(default_local_database_) {} - - void loadTasks(const Poco::Util::AbstractConfiguration & config, const String & base_key = ""); - - /// Set (or update) settings and max_workers param - void reloadSettings(const Poco::Util::AbstractConfiguration & config, const String & base_key = ""); - - /// Base node for all tasks. Its structure: - /// workers/ - directory with active workers (amount of them is less or equal max_workers) - /// description - node with task configuration - /// table_table1/ - directories with per-partition copying status - String task_zookeeper_path; - - /// Database used to create temporary Distributed tables - String default_local_database; - - /// Limits number of simultaneous workers - UInt64 max_workers = 0; - - /// Base settings for pull and push - Settings settings_common; - /// Settings used to fetch data - Settings settings_pull; - /// Settings used to insert data - Settings settings_push; - - String clusters_prefix; - - /// Subtasks - TasksTable table_tasks; - - std::random_device random_device; - pcg64 random_engine; -}; - - -struct MultiTransactionInfo -{ - int32_t code; - Coordination::Requests requests; - Coordination::Responses responses; -}; - -// Creates AST representing 'ENGINE = Distributed(cluster, db, table, [sharding_key]) -std::shared_ptr createASTStorageDistributed( - const String & cluster_name, const String & database, const String & table, const ASTPtr & sharding_key_ast = nullptr) -{ - auto args = std::make_shared(); - args->children.emplace_back(std::make_shared(cluster_name)); - args->children.emplace_back(std::make_shared(database)); - args->children.emplace_back(std::make_shared(table)); - if (sharding_key_ast) - args->children.emplace_back(sharding_key_ast); - - auto engine = std::make_shared(); - engine->name = "Distributed"; - engine->arguments = args; - - auto storage = std::make_shared(); - storage->set(storage->engine, engine); - - return storage; -} - - -BlockInputStreamPtr squashStreamIntoOneBlock(const BlockInputStreamPtr & stream) -{ - return std::make_shared( - stream, - std::numeric_limits::max(), - std::numeric_limits::max()); -} - -Block getBlockWithAllStreamData(const BlockInputStreamPtr & stream) -{ - return squashStreamIntoOneBlock(stream)->read(); -} - - -/// Path getters - -String TaskTable::getPartitionPath(const String & partition_name) const -{ - return task_cluster.task_zookeeper_path // root - + "/tables/" + table_id // tables/dst_cluster.merge.hits - + "/" + escapeForFileName(partition_name); // 201701 -} - -String ShardPartition::getPartitionCleanStartPath() const -{ - return getPartitionPath() + "/clean_start"; -} - -String ShardPartition::getPartitionPath() const -{ - return task_shard.task_table.getPartitionPath(name); -} - -String ShardPartition::getShardStatusPath() const -{ - // schema: //tables///shards/ - // e.g. /root/table_test.hits/201701/shards/1 - return getPartitionShardsPath() + "/" + toString(task_shard.numberInCluster()); -} - -String ShardPartition::getPartitionShardsPath() const -{ - return getPartitionPath() + "/shards"; -} - -String ShardPartition::getPartitionActiveWorkersPath() const -{ - return getPartitionPath() + "/partition_active_workers"; -} - -String ShardPartition::getActiveWorkerPath() const -{ - return getPartitionActiveWorkersPath() + "/" + toString(task_shard.numberInCluster()); -} - -String ShardPartition::getCommonPartitionIsDirtyPath() const -{ - return getPartitionPath() + "/is_dirty"; -} - -String ShardPartition::getCommonPartitionIsCleanedPath() const -{ - return getCommonPartitionIsDirtyPath() + "/cleaned"; -} - -String TaskTable::getPartitionIsDirtyPath(const String & partition_name) const -{ - return getPartitionPath(partition_name) + "/is_dirty"; -} - -String TaskTable::getPartitionIsCleanedPath(const String & partition_name) const -{ - return getPartitionIsDirtyPath(partition_name) + "/cleaned"; -} - -String TaskTable::getPartitionTaskStatusPath(const String & partition_name) const -{ - return getPartitionPath(partition_name) + "/shards"; -} - -String DB::TaskShard::getDescription() const -{ - std::stringstream ss; - ss << "N" << numberInCluster() - << " (having a replica " << getHostNameExample() - << ", pull table " + getQuotedTable(task_table.table_pull) - << " of cluster " + task_table.cluster_pull_name << ")"; - return ss.str(); -} - -String DB::TaskShard::getHostNameExample() const -{ - auto & replicas = task_table.cluster_pull->getShardsAddresses().at(indexInCluster()); - return replicas.at(0).readableString(); -} - - -static bool isExtendedDefinitionStorage(const ASTPtr & storage_ast) -{ - const auto & storage = storage_ast->as(); - return storage.partition_by || storage.order_by || storage.sample_by; -} - -static ASTPtr extractPartitionKey(const ASTPtr & storage_ast) -{ - String storage_str = queryToString(storage_ast); - - const auto & storage = storage_ast->as(); - const auto & engine = storage.engine->as(); - - if (!endsWith(engine.name, "MergeTree")) - { - throw Exception("Unsupported engine was specified in " + storage_str + ", only *MergeTree engines are supported", - ErrorCodes::BAD_ARGUMENTS); - } - - if (isExtendedDefinitionStorage(storage_ast)) - { - if (storage.partition_by) - return storage.partition_by->clone(); - - static const char * all = "all"; - return std::make_shared(Field(all, strlen(all))); - } - else - { - bool is_replicated = startsWith(engine.name, "Replicated"); - size_t min_args = is_replicated ? 3 : 1; - - if (!engine.arguments) - throw Exception("Expected arguments in " + storage_str, ErrorCodes::BAD_ARGUMENTS); - - ASTPtr arguments_ast = engine.arguments->clone(); - ASTs & arguments = arguments_ast->children; - - if (arguments.size() < min_args) - throw Exception("Expected at least " + toString(min_args) + " arguments in " + storage_str, ErrorCodes::BAD_ARGUMENTS); - - ASTPtr & month_arg = is_replicated ? arguments[2] : arguments[1]; - return makeASTFunction("toYYYYMM", month_arg->clone()); - } -} - - -TaskTable::TaskTable(TaskCluster & parent, const Poco::Util::AbstractConfiguration & config, const String & prefix_, - const String & table_key) -: task_cluster(parent) -{ - String table_prefix = prefix_ + "." + table_key + "."; - - name_in_config = table_key; - - cluster_pull_name = config.getString(table_prefix + "cluster_pull"); - cluster_push_name = config.getString(table_prefix + "cluster_push"); - - table_pull.first = config.getString(table_prefix + "database_pull"); - table_pull.second = config.getString(table_prefix + "table_pull"); - - table_push.first = config.getString(table_prefix + "database_push"); - table_push.second = config.getString(table_prefix + "table_push"); - - /// Used as node name in ZooKeeper - table_id = escapeForFileName(cluster_push_name) - + "." + escapeForFileName(table_push.first) - + "." + escapeForFileName(table_push.second); - - engine_push_str = config.getString(table_prefix + "engine"); - { - ParserStorage parser_storage; - engine_push_ast = parseQuery(parser_storage, engine_push_str, 0); - engine_push_partition_key_ast = extractPartitionKey(engine_push_ast); - } - - sharding_key_str = config.getString(table_prefix + "sharding_key"); - { - ParserExpressionWithOptionalAlias parser_expression(false); - sharding_key_ast = parseQuery(parser_expression, sharding_key_str, 0); - engine_split_ast = createASTStorageDistributed(cluster_push_name, table_push.first, table_push.second, sharding_key_ast); - } - - where_condition_str = config.getString(table_prefix + "where_condition", ""); - if (!where_condition_str.empty()) - { - ParserExpressionWithOptionalAlias parser_expression(false); - where_condition_ast = parseQuery(parser_expression, where_condition_str, 0); - - // Will use canonical expression form - where_condition_str = queryToString(where_condition_ast); - } - - String enabled_partitions_prefix = table_prefix + "enabled_partitions"; - has_enabled_partitions = config.has(enabled_partitions_prefix); - - if (has_enabled_partitions) - { - Strings keys; - config.keys(enabled_partitions_prefix, keys); - - if (keys.empty()) - { - /// Parse list of partition from space-separated string - String partitions_str = config.getString(table_prefix + "enabled_partitions"); - boost::trim_if(partitions_str, isWhitespaceASCII); - boost::split(enabled_partitions, partitions_str, isWhitespaceASCII, boost::token_compress_on); - } - else - { - /// Parse sequence of ... - for (const String & key : keys) - { - if (!startsWith(key, "partition")) - throw Exception("Unknown key " + key + " in " + enabled_partitions_prefix, ErrorCodes::UNKNOWN_ELEMENT_IN_CONFIG); - - enabled_partitions.emplace_back(config.getString(enabled_partitions_prefix + "." + key)); - } - } - - std::copy(enabled_partitions.begin(), enabled_partitions.end(), std::inserter(enabled_partitions_set, enabled_partitions_set.begin())); - } -} - - -static ShardPriority getReplicasPriority(const Cluster::Addresses & replicas, const std::string & local_hostname, UInt8 random) -{ - ShardPriority res; - - if (replicas.empty()) - return res; - - res.is_remote = 1; - for (auto & replica : replicas) - { - if (isLocalAddress(DNSResolver::instance().resolveHost(replica.host_name))) - { - res.is_remote = 0; - break; - } - } - - res.hostname_difference = std::numeric_limits::max(); - for (auto & replica : replicas) - { - size_t difference = getHostNameDifference(local_hostname, replica.host_name); - res.hostname_difference = std::min(difference, res.hostname_difference); - } - - res.random = random; - return res; -} - -template -void TaskTable::initShards(RandomEngine && random_engine) -{ - const String & fqdn_name = getFQDNOrHostName(); - std::uniform_int_distribution get_urand(0, std::numeric_limits::max()); - - // Compute the priority - for (auto & shard_info : cluster_pull->getShardsInfo()) - { - TaskShardPtr task_shard = std::make_shared(*this, shard_info); - const auto & replicas = cluster_pull->getShardsAddresses().at(task_shard->indexInCluster()); - task_shard->priority = getReplicasPriority(replicas, fqdn_name, get_urand(random_engine)); - - all_shards.emplace_back(task_shard); - } - - // Sort by priority - std::sort(all_shards.begin(), all_shards.end(), - [] (const TaskShardPtr & lhs, const TaskShardPtr & rhs) - { - return ShardPriority::greaterPriority(lhs->priority, rhs->priority); - }); - - // Cut local shards - auto it_first_remote = std::lower_bound(all_shards.begin(), all_shards.end(), 1, - [] (const TaskShardPtr & lhs, UInt8 is_remote) - { - return lhs->priority.is_remote < is_remote; - }); - - local_shards.assign(all_shards.begin(), it_first_remote); -} - - -void DB::TaskCluster::loadTasks(const Poco::Util::AbstractConfiguration & config, const String & base_key) -{ - String prefix = base_key.empty() ? "" : base_key + "."; - - clusters_prefix = prefix + "remote_servers"; - if (!config.has(clusters_prefix)) - throw Exception("You should specify list of clusters in " + clusters_prefix, ErrorCodes::BAD_ARGUMENTS); - - Poco::Util::AbstractConfiguration::Keys tables_keys; - config.keys(prefix + "tables", tables_keys); - - for (const auto & table_key : tables_keys) - { - table_tasks.emplace_back(*this, config, prefix + "tables", table_key); - } -} - -void DB::TaskCluster::reloadSettings(const Poco::Util::AbstractConfiguration & config, const String & base_key) -{ - String prefix = base_key.empty() ? "" : base_key + "."; - - max_workers = config.getUInt64(prefix + "max_workers"); - - settings_common = Settings(); - if (config.has(prefix + "settings")) - settings_common.loadSettingsFromConfig(prefix + "settings", config); - - settings_pull = settings_common; - if (config.has(prefix + "settings_pull")) - settings_pull.loadSettingsFromConfig(prefix + "settings_pull", config); - - settings_push = settings_common; - if (config.has(prefix + "settings_push")) - settings_push.loadSettingsFromConfig(prefix + "settings_push", config); - - auto set_default_value = [] (auto && setting, auto && default_value) - { - setting = setting.changed ? setting.value : default_value; - }; - - /// Override important settings - settings_pull.readonly = 1; - settings_push.insert_distributed_sync = 1; - set_default_value(settings_pull.load_balancing, LoadBalancing::NEAREST_HOSTNAME); - set_default_value(settings_pull.max_threads, 1); - set_default_value(settings_pull.max_block_size, 8192UL); - set_default_value(settings_pull.preferred_block_size_bytes, 0); - set_default_value(settings_push.insert_distributed_timeout, 0); -} - - -} // end of an anonymous namespace - class ClusterCopier { public: - ClusterCopier(const String & task_path_, - const String & host_id_, - const String & proxy_database_name_, + ClusterCopier(String task_path_, + String host_id_, + String proxy_database_name_, Context & context_) : - task_zookeeper_path(task_path_), - host_id(host_id_), - working_database_name(proxy_database_name_), + task_zookeeper_path(std::move(task_path_)), + host_id(std::move(host_id_)), + working_database_name(std::move(proxy_database_name_)), context(context_), log(&Poco::Logger::get("ClusterCopier")) { @@ -940,14 +221,14 @@ public: task_description_watch_zookeeper = zookeeper; String task_config_str; - Coordination::Stat stat; + Coordination::Stat stat{}; int code; zookeeper->tryGetWatch(task_description_path, task_config_str, &stat, task_description_watch_callback, &code); if (code) throw Exception("Can't get description node " + task_description_path, ErrorCodes::BAD_ARGUMENTS); - LOG_DEBUG(log, "Loading description, zxid=" << task_descprtion_current_stat.czxid); + LOG_DEBUG(log, "Loading description, zxid=" << task_description_current_stat.czxid); auto config = getConfigurationFromXMLString(task_config_str); /// Setup settings @@ -955,7 +236,7 @@ public: context.getSettingsRef() = task_cluster->settings_common; task_cluster_current_config = config; - task_descprtion_current_stat = stat; + task_description_current_stat = stat; } void updateConfigIfNeeded() @@ -1092,7 +373,7 @@ protected: { updateConfigIfNeeded(); - Coordination::Stat stat; + Coordination::Stat stat{}; zookeeper->get(workers_version_path, &stat); auto version = stat.version; zookeeper->get(workers_path, &stat); @@ -1186,7 +467,7 @@ protected: task_table.getPartitionIsDirtyPath(partition_name), task_table.getPartitionIsCleanedPath(partition_name) ); - Coordination::Stat stat; + Coordination::Stat stat{}; LogicalClock task_start_clock; if (zookeeper->exists(task_table.getPartitionTaskStatusPath(partition_name), &stat)) task_start_clock = LogicalClock(stat.mzxid); @@ -1264,7 +545,7 @@ protected: } /// Replaces ENGINE and table name in a create query - std::shared_ptr rewriteCreateQueryStorage(const ASTPtr & create_query_ast, const DatabaseAndTableName & new_table, const ASTPtr & new_storage_ast) + static std::shared_ptr rewriteCreateQueryStorage(const ASTPtr & create_query_ast, const DatabaseAndTableName & new_table, const ASTPtr & new_storage_ast) { const auto & create = create_query_ast->as(); auto res = std::make_shared(create); @@ -1291,7 +572,7 @@ protected: public: UInt32 value; - WrappingUInt32(UInt32 _value) + explicit WrappingUInt32(UInt32 _value) : value(_value) {} @@ -1315,13 +596,20 @@ protected: /** Conforming Zxid definition. * cf. https://github.com/apache/zookeeper/blob/631d1b284f0edb1c4f6b0fb221bf2428aec71aaa/zookeeper-docs/src/main/resources/markdown/zookeeperInternals.md#guarantees-properties-and-definitions + * + * But it is better to read this: https://zookeeper.apache.org/doc/r3.1.2/zookeeperProgrammers.html + * + * Actually here is the definition of Zxid. + * Every change to the ZooKeeper state receives a stamp in the form of a zxid (ZooKeeper Transaction Id). + * This exposes the total ordering of all changes to ZooKeeper. Each change will have a unique zxid + * and if zxid1 is smaller than zxid2 then zxid1 happened before zxid2. */ class Zxid { public: WrappingUInt32 epoch; WrappingUInt32 counter; - Zxid(UInt64 _zxid) + explicit Zxid(UInt64 _zxid) : epoch(_zxid >> 32) , counter(_zxid) {} @@ -1338,6 +626,51 @@ protected: } }; + /* When multiple ClusterCopiers discover that the target partition is not empty, + * they will attempt to clean up this partition before proceeding to copying. + * + * Instead of purging is_dirty, the history of cleaning work is preserved and partition hygiene is established + * based on a happens-before relation between the events. + * This relation is encoded by LogicalClock based on the mzxid of the is_dirty ZNode and is_dirty/cleaned. + * The fact of the partition hygiene is encoded by CleanStateClock. + * + * For you to know what mzxid means: + * + * ZooKeeper Stat Structure: + * The Stat structure for each znode in ZooKeeper is made up of the following fields: + * + * -- czxid + * The zxid of the change that caused this znode to be created. + * + * -- mzxid + * The zxid of the change that last modified this znode. + * + * -- ctime + * The time in milliseconds from epoch when this znode was created. + * + * -- mtime + * The time in milliseconds from epoch when this znode was last modified. + * + * -- version + * The number of changes to the data of this znode. + * + * -- cversion + * The number of changes to the children of this znode. + * + * -- aversion + * The number of changes to the ACL of this znode. + * + * -- ephemeralOwner + * The session id of the owner of this znode if the znode is an ephemeral node. + * If it is not an ephemeral node, it will be zero. + * + * -- dataLength + * The length of the data field of this znode. + * + * -- numChildren + * The number of children of this znode. + * */ + class LogicalClock { public: @@ -1345,7 +678,7 @@ protected: LogicalClock() = default; - LogicalClock(UInt64 _zxid) + explicit LogicalClock(UInt64 _zxid) : zxid(_zxid) {} @@ -1354,7 +687,7 @@ protected: return bool(zxid); } - // happens-before relation with a reasonable time bound + /// happens-before relation with a reasonable time bound bool happensBefore(const LogicalClock & other) const { return !zxid @@ -1366,13 +699,14 @@ protected: return happensBefore(other); } - // strict equality check + /// strict equality check bool operator==(const LogicalClock & other) const { return zxid == other.zxid; } }; + class CleanStateClock { public: @@ -1404,7 +738,7 @@ protected: const String & clean_state_path) : stale(std::make_shared(false)) { - Coordination::Stat stat; + Coordination::Stat stat{}; String _some_data; auto watch_callback = [stale = stale] (const Coordination::WatchResponse & rsp) @@ -1462,13 +796,13 @@ protected: const String current_shards_path = task_partition.getPartitionShardsPath(); const String current_partition_active_workers_dir = task_partition.getPartitionActiveWorkersPath(); const String is_dirty_flag_path = task_partition.getCommonPartitionIsDirtyPath(); - const String dirt_cleaner_path = is_dirty_flag_path + "/cleaner"; - const String is_dirt_cleaned_path = task_partition.getCommonPartitionIsCleanedPath(); + const String dirty_cleaner_path = is_dirty_flag_path + "/cleaner"; + const String is_dirty_cleaned_path = task_partition.getCommonPartitionIsCleanedPath(); zkutil::EphemeralNodeHolder::Ptr cleaner_holder; try { - cleaner_holder = zkutil::EphemeralNodeHolder::create(dirt_cleaner_path, *zookeeper, host_id); + cleaner_holder = zkutil::EphemeralNodeHolder::create(dirty_cleaner_path, *zookeeper, host_id); } catch (const Coordination::Exception & e) { @@ -1482,7 +816,7 @@ protected: throw; } - Coordination::Stat stat; + Coordination::Stat stat{}; if (zookeeper->exists(current_partition_active_workers_dir, &stat)) { if (stat.numChildren != 0) @@ -1517,7 +851,7 @@ protected: // Lock the dirty flag zookeeper->set(is_dirty_flag_path, host_id, clean_state_clock.discovery_version.value()); zookeeper->tryRemove(task_partition.getPartitionCleanStartPath()); - CleanStateClock my_clock(zookeeper, is_dirty_flag_path, is_dirt_cleaned_path); + CleanStateClock my_clock(zookeeper, is_dirty_flag_path, is_dirty_cleaned_path); /// Remove all status nodes { @@ -1556,9 +890,9 @@ protected: { zookeeper->set(is_dirty_flag_path, host_id, my_clock.discovery_version.value()); if (my_clock.clean_state_version) - zookeeper->set(is_dirt_cleaned_path, host_id, my_clock.clean_state_version.value()); + zookeeper->set(is_dirty_cleaned_path, host_id, my_clock.clean_state_version.value()); else - zookeeper->create(is_dirt_cleaned_path, host_id, zkutil::CreateMode::Persistent); + zookeeper->create(is_dirty_cleaned_path, host_id, zkutil::CreateMode::Persistent); } else { @@ -1582,7 +916,7 @@ protected: bool tryProcessTable(const ConnectionTimeouts & timeouts, TaskTable & task_table) { - /// An heuristic: if previous shard is already done, then check next one without sleeps due to max_workers constraint + /// A heuristic: if previous shard is already done, then check next one without sleeps due to max_workers constraint bool previous_shard_is_instantly_finished = false; /// Process each partition that is present in cluster @@ -1594,6 +928,7 @@ protected: ClusterPartition & cluster_partition = task_table.cluster_partitions[partition_name]; Stopwatch watch; + /// We will check all the shards of the table and check if they contain current partition. TasksShard expected_shards; UInt64 num_failed_shards = 0; @@ -1637,6 +972,8 @@ protected: } auto it_shard_partition = shard->partition_tasks.find(partition_name); + /// Previously when we discovered that shard does not contain current partition, we skipped it. + /// At this moment partition have to be present. if (it_shard_partition == shard->partition_tasks.end()) throw Exception("There are no such partition in a shard. This is a bug.", ErrorCodes::LOGICAL_ERROR); auto & partition = it_shard_partition->second; @@ -1738,6 +1075,7 @@ protected: Error, }; + /// Job for copying partition from particular shard. PartitionTaskStatus tryProcessPartitionTask(const ConnectionTimeouts & timeouts, ShardPartition & task_partition, bool is_unprioritized_task) { PartitionTaskStatus res; @@ -1770,6 +1108,8 @@ protected: TaskShard & task_shard = task_partition.task_shard; TaskTable & task_table = task_shard.task_table; ClusterPartition & cluster_partition = task_table.getClusterPartition(task_partition.name); + const size_t number_of_splits = task_table.number_of_splits; + UNUSED(number_of_splits); /// We need to update table definitions for each partition, it could be changed after ALTER createShardInternalTables(timeouts, task_shard); @@ -1777,7 +1117,7 @@ protected: auto zookeeper = context.getZooKeeper(); const String is_dirty_flag_path = task_partition.getCommonPartitionIsDirtyPath(); - const String is_dirt_cleaned_path = task_partition.getCommonPartitionIsCleanedPath(); + const String is_dirty_cleaned_path = task_partition.getCommonPartitionIsCleanedPath(); const String current_task_is_active_path = task_partition.getActiveWorkerPath(); const String current_task_status_path = task_partition.getShardStatusPath(); @@ -1803,12 +1143,15 @@ protected: }; /// Returns SELECT query filtering current partition and applying user filter - auto get_select_query = [&] (const DatabaseAndTableName & from_table, const String & fields, String limit = "") + auto get_select_query = [&] (const DatabaseAndTableName & from_table, const String & fields, String limit = "", + bool enable_splitting = false, size_t current_piece_number = 0) { String query; query += "SELECT " + fields + " FROM " + getQuotedTable(from_table); /// TODO: Bad, it is better to rewrite with ASTLiteral(partition_key_field) query += " WHERE (" + queryToString(task_table.engine_push_partition_key_ast) + " = (" + task_partition.name + " AS partition_key))"; + if (enable_splitting) + query += " AND ( cityHash64(*) = " + std::to_string(current_piece_number) + " )"; if (!task_table.where_condition_str.empty()) query += " AND (" + task_table.where_condition_str + ")"; if (!limit.empty()) @@ -1823,11 +1166,11 @@ protected: LOG_DEBUG(log, "Processing " << current_task_status_path); - CleanStateClock clean_state_clock (zookeeper, is_dirty_flag_path, is_dirt_cleaned_path); + CleanStateClock clean_state_clock (zookeeper, is_dirty_flag_path, is_dirty_cleaned_path); LogicalClock task_start_clock; { - Coordination::Stat stat; + Coordination::Stat stat{}; if (zookeeper->exists(task_partition.getPartitionShardsPath(), &stat)) task_start_clock = LogicalClock(stat.mzxid); } @@ -1887,7 +1230,8 @@ protected: } // Task is abandoned, initialize DROP PARTITION - LOG_DEBUG(log, "Task " << current_task_status_path << " has not been successfully finished by " << status.owner << ". Partition will be dropped and refilled."); + LOG_DEBUG(log, "Task " << current_task_status_path << " has not been successfully finished by " << + status.owner << ". Partition will be dropped and refilled."); create_is_dirty_node(clean_state_clock); return PartitionTaskStatus::Error; @@ -1900,7 +1244,8 @@ protected: if (!zookeeper->tryGet(task_partition.getPartitionCleanStartPath(), clean_start_status) || clean_start_status != "ok") { zookeeper->createIfNotExists(task_partition.getPartitionCleanStartPath(), ""); - auto checker = zkutil::EphemeralNodeHolder::create(task_partition.getPartitionCleanStartPath() + "/checker", *zookeeper, host_id); + auto checker = zkutil::EphemeralNodeHolder::create(task_partition.getPartitionCleanStartPath() + "/checker", + *zookeeper, host_id); // Maybe we are the first worker ASTPtr query_select_ast = get_select_query(task_shard.table_split_shard, "count()"); UInt64 count; @@ -1916,7 +1261,7 @@ protected: if (count != 0) { - Coordination::Stat stat_shards; + Coordination::Stat stat_shards{}; zookeeper->get(task_partition.getPartitionShardsPath(), &stat_shards); /// NOTE: partition is still fresh if dirt discovery happens before cleaning @@ -1938,7 +1283,7 @@ protected: /// Try start processing, create node about it { String start_state = TaskStateWithOwner::getData(TaskState::Started, host_id); - CleanStateClock new_clean_state_clock (zookeeper, is_dirty_flag_path, is_dirt_cleaned_path); + CleanStateClock new_clean_state_clock (zookeeper, is_dirty_flag_path, is_dirty_cleaned_path); if (clean_state_clock != new_clean_state_clock) { LOG_INFO(log, "Partition " << task_partition.name << " clean state changed, cowardly bailing"); @@ -1955,15 +1300,17 @@ protected: /// Try create table (if not exists) on each shard { - auto create_query_push_ast = rewriteCreateQueryStorage(task_shard.current_pull_table_create_query, task_table.table_push, task_table.engine_push_ast); + auto create_query_push_ast = rewriteCreateQueryStorage(task_shard.current_pull_table_create_query, + task_table.table_push, task_table.engine_push_ast); create_query_push_ast->as().if_not_exists = true; String query = queryToString(create_query_push_ast); LOG_DEBUG(log, "Create destination tables. Query: " << query); - UInt64 shards = executeQueryOnCluster(task_table.cluster_push, query, create_query_push_ast, &task_cluster->settings_push, + UInt64 shards = executeQueryOnCluster(task_table.cluster_push, query, + create_query_push_ast, &task_cluster->settings_push, PoolMode::GET_MANY); - LOG_DEBUG(log, "Destination tables " << getQuotedTable(task_table.table_push) << " have been created on " << shards - << " shards of " << task_table.cluster_push->getShardCount()); + LOG_DEBUG(log, "Destination tables " << getQuotedTable(task_table.table_push) << + " have been created on " << shards << " shards of " << task_table.cluster_push->getShardCount()); } /// Do the copying @@ -2073,7 +1420,7 @@ protected: /// Finalize the processing, change state of current partition task (and also check is_dirty flag) { String state_finished = TaskStateWithOwner::getData(TaskState::Finished, host_id); - CleanStateClock new_clean_state_clock (zookeeper, is_dirty_flag_path, is_dirt_cleaned_path); + CleanStateClock new_clean_state_clock (zookeeper, is_dirty_flag_path, is_dirty_cleaned_path); if (clean_state_clock != new_clean_state_clock) { LOG_INFO(log, "Partition " << task_partition.name << " clean state changed, cowardly bailing"); @@ -2265,7 +1612,7 @@ protected: UInt64 & num_successful_executions = per_shard_num_successful_replicas.at(shard_index); num_successful_executions = 0; - auto increment_and_check_exit = [&] () + auto increment_and_check_exit = [&] () -> bool { ++num_successful_executions; return max_successful_executions_per_shard && num_successful_executions >= max_successful_executions_per_shard; @@ -2348,7 +1695,7 @@ private: ConfigurationPtr task_cluster_initial_config; ConfigurationPtr task_cluster_current_config; - Coordination::Stat task_descprtion_current_stat{}; + Coordination::Stat task_description_current_stat{}; std::unique_ptr task_cluster; @@ -2373,12 +1720,11 @@ void ClusterCopierApp::initialize(Poco::Util::Application & self) config_xml_path = config().getString("config-file"); task_path = config().getString("task-path"); - log_level = config().getString("log-level", "debug"); + log_level = config().getString("log-level", "trace"); is_safe_mode = config().has("safe-mode"); if (config().has("copy-fault-probability")) copy_fault_probability = std::max(std::min(config().getDouble("copy-fault-probability"), 1.0), 0.0); base_dir = (config().has("base-dir")) ? config().getString("base-dir") : Poco::Path::current(); - // process_id is '#_' time_t timestamp = Poco::Timestamp().epochTime(); auto curr_pid = Poco::Process::id(); diff --git a/dbms/programs/copier/ClusterCopier.h b/dbms/programs/copier/ClusterCopier.h index 89f45df8686..fe228fd6194 100644 --- a/dbms/programs/copier/ClusterCopier.h +++ b/dbms/programs/copier/ClusterCopier.h @@ -71,11 +71,9 @@ private: void mainImpl(); - void setupLogging(); - std::string config_xml_path; std::string task_path; - std::string log_level = "debug"; + std::string log_level = "trace"; bool is_safe_mode = false; double copy_fault_probability = 0; bool is_help = false; diff --git a/dbms/programs/copier/Internals.h b/dbms/programs/copier/Internals.h index a25ae7b973c..5f14604fbf9 100644 --- a/dbms/programs/copier/Internals.h +++ b/dbms/programs/copier/Internals.h @@ -1,8 +1,746 @@ -// -// Created by jakalletti on 2/7/20. -// +#pragma once -#ifndef CLICKHOUSE_INTERNALS_H -#define CLICKHOUSE_INTERNALS_H +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int NO_ZOOKEEPER; + extern const int BAD_ARGUMENTS; + extern const int UNKNOWN_TABLE; + extern const int UNFINISHED; + extern const int UNKNOWN_ELEMENT_IN_CONFIG; +} + +namespace +{ + +using DatabaseAndTableName = std::pair; + +String getQuotedTable(const String & database, const String & table) +{ + if (database.empty()) + { + return backQuoteIfNeed(table); + } + + return backQuoteIfNeed(database) + "." + backQuoteIfNeed(table); +} + +String getQuotedTable(const DatabaseAndTableName & db_and_table) +{ + return getQuotedTable(db_and_table.first, db_and_table.second); +} + + +enum class TaskState +{ + Started = 0, + Finished, + Unknown +}; + +/// Used to mark status of shard partition tasks +struct TaskStateWithOwner +{ + TaskStateWithOwner() = default; + TaskStateWithOwner(TaskState state_, const String & owner_) : state(state_), owner(owner_) {} + + TaskState state{TaskState::Unknown}; + String owner; + + static String getData(TaskState state, const String & owner) + { + return TaskStateWithOwner(state, owner).toString(); + } + + String toString() + { + WriteBufferFromOwnString wb; + wb << static_cast(state) << "\n" << escape << owner; + return wb.str(); + } + + static TaskStateWithOwner fromString(const String & data) + { + ReadBufferFromString rb(data); + TaskStateWithOwner res; + UInt32 state; + + rb >> state >> "\n" >> escape >> res.owner; + + if (state >= static_cast(TaskState::Unknown)) + throw Exception("Unknown state " + data, ErrorCodes::LOGICAL_ERROR); + + res.state = static_cast(state); + return res; + } +}; + + +/// Hierarchical description of the tasks +struct ShardPartition; +struct TaskShard; +struct TaskTable; +struct TaskCluster; +struct ClusterPartition; + +using TasksPartition = std::map>; +using ShardInfo = Cluster::ShardInfo; +using TaskShardPtr = std::shared_ptr; +using TasksShard = std::vector; +using TasksTable = std::list; +using ClusterPartitions = std::map>; + + +/// Just destination partition of a shard +/// I don't know what this comment means. +/// In short, when we discovered what shards contain currently processing partition, +/// This class describes a partition (name) that is stored on the shard (parent). +struct ShardPartition +{ + ShardPartition(TaskShard & parent, String name_quoted_) : task_shard(parent), name(std::move(name_quoted_)) {} + + String getPartitionPath() const; + String getPartitionCleanStartPath() const; + String getCommonPartitionIsDirtyPath() const; + String getCommonPartitionIsCleanedPath() const; + String getPartitionActiveWorkersPath() const; + String getActiveWorkerPath() const; + String getPartitionShardsPath() const; + String getShardStatusPath() const; + + TaskShard & task_shard; + String name; +}; + + +struct ShardPriority +{ + UInt8 is_remote = 1; + size_t hostname_difference = 0; + UInt8 random = 0; + + static bool greaterPriority(const ShardPriority & current, const ShardPriority & other) + { + return std::forward_as_tuple(current.is_remote, current.hostname_difference, current.random) + < std::forward_as_tuple(other.is_remote, other.hostname_difference, other.random); + } +}; + +/// Tables has many shards and table's partiton can be stored on different shards. +/// When we copy partition we have to discover it's shards (shards which store this partition) +/// For simplier retrieval of which partitions are stored in particular shard we created TaskShard. +struct TaskShard +{ + TaskShard(TaskTable & parent, const ShardInfo & info_) : task_table(parent), info(info_) {} + + TaskTable & task_table; + + ShardInfo info; + UInt32 numberInCluster() const { return info.shard_num; } + UInt32 indexInCluster() const { return info.shard_num - 1; } + + String getDescription() const; + String getHostNameExample() const; + + /// Used to sort clusters by their proximity + ShardPriority priority; + + /// Column with unique destination partitions (computed from engine_push_partition_key expr.) in the shard + ColumnWithTypeAndName partition_key_column; + + /// There is a task for each destination partition + TasksPartition partition_tasks; + + /// Which partitions have been checked for existence + /// If some partition from this lists is exists, it is in partition_tasks + std::set checked_partitions; + + /// Last CREATE TABLE query of the table of the shard + ASTPtr current_pull_table_create_query; + + /// Internal distributed tables + DatabaseAndTableName table_read_shard; + DatabaseAndTableName table_split_shard; +}; + + +/// Contains info about all shards that contain a partition +struct ClusterPartition +{ + double elapsed_time_seconds = 0; + UInt64 bytes_copied = 0; + UInt64 rows_copied = 0; + UInt64 blocks_copied = 0; + + UInt64 total_tries = 0; +}; + + +struct TaskTable +{ + TaskTable(TaskCluster & parent, const Poco::Util::AbstractConfiguration & config, const String & prefix, + const String & table_key); + + TaskCluster & task_cluster; + + String getPartitionPath(const String & partition_name) const; + [[maybe_unused]] String getPartitionPathWithPieceNumber(const String & partition_name, size_t current_piece_number) const; + String getPartitionIsDirtyPath(const String & partition_name) const; + String getPartitionIsCleanedPath(const String & partition_name) const; + String getPartitionTaskStatusPath(const String & partition_name) const; + + /// Partitions will be splitted into number-of-splits pieces. + /// Each piece will be copied independently. (10 by default) + size_t number_of_splits; + + String name_in_config; + + /// Used as task ID + String table_id; + + /// Source cluster and table + String cluster_pull_name; + DatabaseAndTableName table_pull; + + /// Destination cluster and table + String cluster_push_name; + DatabaseAndTableName table_push; + + /// Storage of destination table + String engine_push_str; + ASTPtr engine_push_ast; + ASTPtr engine_push_partition_key_ast; + + /// A Distributed table definition used to split data + String sharding_key_str; + ASTPtr sharding_key_ast; + ASTPtr engine_split_ast; + + /// Additional WHERE expression to filter input data + String where_condition_str; + ASTPtr where_condition_ast; + + /// Resolved clusters + ClusterPtr cluster_pull; + ClusterPtr cluster_push; + + /// Filter partitions that should be copied + bool has_enabled_partitions = false; + Strings enabled_partitions; + NameSet enabled_partitions_set; + + /// Prioritized list of shards + /// all_shards contains information about all shards in the table. + /// So we have to check whether particular shard have current partiton or not while processing. + TasksShard all_shards; + TasksShard local_shards; + + /// All partitions of the current table. + ClusterPartitions cluster_partitions; + NameSet finished_cluster_partitions; + + /// Parition names to process in user-specified order + Strings ordered_partition_names; + + ClusterPartition & getClusterPartition(const String & partition_name) + { + auto it = cluster_partitions.find(partition_name); + if (it == cluster_partitions.end()) + throw Exception("There are no cluster partition " + partition_name + " in " + table_id, ErrorCodes::LOGICAL_ERROR); + return it->second; + } + + Stopwatch watch; + UInt64 bytes_copied = 0; + UInt64 rows_copied = 0; + + template + void initShards(RandomEngine && random_engine); +}; + + +struct TaskCluster +{ + TaskCluster(const String & task_zookeeper_path_, const String & default_local_database_) + : task_zookeeper_path(task_zookeeper_path_), default_local_database(default_local_database_) {} + + void loadTasks(const Poco::Util::AbstractConfiguration & config, const String & base_key = ""); + + /// Set (or update) settings and max_workers param + void reloadSettings(const Poco::Util::AbstractConfiguration & config, const String & base_key = ""); + + /// Base node for all tasks. Its structure: + /// workers/ - directory with active workers (amount of them is less or equal max_workers) + /// description - node with task configuration + /// table_table1/ - directories with per-partition copying status + String task_zookeeper_path; + + /// Database used to create temporary Distributed tables + String default_local_database; + + /// Limits number of simultaneous workers + UInt64 max_workers = 0; + + /// Base settings for pull and push + Settings settings_common; + /// Settings used to fetch data + Settings settings_pull; + /// Settings used to insert data + Settings settings_push; + + String clusters_prefix; + + /// Subtasks + TasksTable table_tasks; + + std::random_device random_device; + pcg64 random_engine; +}; + + + +struct MultiTransactionInfo +{ + int32_t code; + Coordination::Requests requests; + Coordination::Responses responses; +}; + +// Creates AST representing 'ENGINE = Distributed(cluster, db, table, [sharding_key]) +std::shared_ptr createASTStorageDistributed( + const String & cluster_name, const String & database, const String & table, const ASTPtr & sharding_key_ast = nullptr) +{ + auto args = std::make_shared(); + args->children.emplace_back(std::make_shared(cluster_name)); + args->children.emplace_back(std::make_shared(database)); + args->children.emplace_back(std::make_shared(table)); + if (sharding_key_ast) + args->children.emplace_back(sharding_key_ast); + + auto engine = std::make_shared(); + engine->name = "Distributed"; + engine->arguments = args; + + auto storage = std::make_shared(); + storage->set(storage->engine, engine); + + return storage; +} + + +BlockInputStreamPtr squashStreamIntoOneBlock(const BlockInputStreamPtr & stream) +{ + return std::make_shared( + stream, + std::numeric_limits::max(), + std::numeric_limits::max()); +} + +Block getBlockWithAllStreamData(const BlockInputStreamPtr & stream) +{ + return squashStreamIntoOneBlock(stream)->read(); +} + + +/// Path getters + +String TaskTable::getPartitionPath(const String & partition_name) const +{ + return task_cluster.task_zookeeper_path // root + + "/tables/" + table_id // tables/dst_cluster.merge.hits + + "/" + escapeForFileName(partition_name); // 201701 +} + +String TaskTable::getPartitionPathWithPieceNumber(const String & partition_name, size_t current_piece_number) const +{ + return getPartitionPath(partition_name) + "/" + std::to_string(current_piece_number); // 1...number_of_splits +} + +String ShardPartition::getPartitionCleanStartPath() const +{ + return getPartitionPath() + "/clean_start"; +} + +String ShardPartition::getPartitionPath() const +{ + return task_shard.task_table.getPartitionPath(name); +} + +String ShardPartition::getShardStatusPath() const +{ + // schema: //tables/
//shards/ + // e.g. /root/table_test.hits/201701/shards/1 + return getPartitionShardsPath() + "/" + toString(task_shard.numberInCluster()); +} + +String ShardPartition::getPartitionShardsPath() const +{ + return getPartitionPath() + "/shards"; +} + +String ShardPartition::getPartitionActiveWorkersPath() const +{ + return getPartitionPath() + "/partition_active_workers"; +} + +String ShardPartition::getActiveWorkerPath() const +{ + return getPartitionActiveWorkersPath() + "/" + toString(task_shard.numberInCluster()); +} + +String ShardPartition::getCommonPartitionIsDirtyPath() const +{ + return getPartitionPath() + "/is_dirty"; +} + +String ShardPartition::getCommonPartitionIsCleanedPath() const +{ + return getCommonPartitionIsDirtyPath() + "/cleaned"; +} + +String TaskTable::getPartitionIsDirtyPath(const String & partition_name) const +{ + return getPartitionPath(partition_name) + "/is_dirty"; +} + +String TaskTable::getPartitionIsCleanedPath(const String & partition_name) const +{ + return getPartitionIsDirtyPath(partition_name) + "/cleaned"; +} + +String TaskTable::getPartitionTaskStatusPath(const String & partition_name) const +{ + return getPartitionPath(partition_name) + "/shards"; +} + +String TaskShard::getDescription() const +{ + std::stringstream ss; + ss << "N" << numberInCluster() + << " (having a replica " << getHostNameExample() + << ", pull table " + getQuotedTable(task_table.table_pull) + << " of cluster " + task_table.cluster_pull_name << ")"; + return ss.str(); +} + +String TaskShard::getHostNameExample() const +{ + auto & replicas = task_table.cluster_pull->getShardsAddresses().at(indexInCluster()); + return replicas.at(0).readableString(); +} + + +static bool isExtendedDefinitionStorage(const ASTPtr & storage_ast) +{ + const auto & storage = storage_ast->as(); + return storage.partition_by || storage.order_by || storage.sample_by; +} + +static ASTPtr extractPartitionKey(const ASTPtr & storage_ast) +{ + String storage_str = queryToString(storage_ast); + + const auto & storage = storage_ast->as(); + const auto & engine = storage.engine->as(); + + if (!endsWith(engine.name, "MergeTree")) + { + throw Exception("Unsupported engine was specified in " + storage_str + ", only *MergeTree engines are supported", + ErrorCodes::BAD_ARGUMENTS); + } + + if (isExtendedDefinitionStorage(storage_ast)) + { + if (storage.partition_by) + return storage.partition_by->clone(); + + static const char * all = "all"; + return std::make_shared(Field(all, strlen(all))); + } + else + { + bool is_replicated = startsWith(engine.name, "Replicated"); + size_t min_args = is_replicated ? 3 : 1; + + if (!engine.arguments) + throw Exception("Expected arguments in " + storage_str, ErrorCodes::BAD_ARGUMENTS); + + ASTPtr arguments_ast = engine.arguments->clone(); + ASTs & arguments = arguments_ast->children; + + if (arguments.size() < min_args) + throw Exception("Expected at least " + toString(min_args) + " arguments in " + storage_str, ErrorCodes::BAD_ARGUMENTS); + + ASTPtr & month_arg = is_replicated ? arguments[2] : arguments[1]; + return makeASTFunction("toYYYYMM", month_arg->clone()); + } +} + + +TaskTable::TaskTable(TaskCluster & parent, const Poco::Util::AbstractConfiguration & config, const String & prefix_, + const String & table_key) + : task_cluster(parent) +{ + String table_prefix = prefix_ + "." + table_key + "."; + + name_in_config = table_key; + + number_of_splits = config.getUInt64("number_of_splits", 10); + + cluster_pull_name = config.getString(table_prefix + "cluster_pull"); + cluster_push_name = config.getString(table_prefix + "cluster_push"); + + table_pull.first = config.getString(table_prefix + "database_pull"); + table_pull.second = config.getString(table_prefix + "table_pull"); + + table_push.first = config.getString(table_prefix + "database_push"); + table_push.second = config.getString(table_prefix + "table_push"); + + /// Used as node name in ZooKeeper + table_id = escapeForFileName(cluster_push_name) + + "." + escapeForFileName(table_push.first) + + "." + escapeForFileName(table_push.second); + + engine_push_str = config.getString(table_prefix + "engine"); + { + ParserStorage parser_storage; + engine_push_ast = parseQuery(parser_storage, engine_push_str, 0); + engine_push_partition_key_ast = extractPartitionKey(engine_push_ast); + } + + sharding_key_str = config.getString(table_prefix + "sharding_key"); + { + ParserExpressionWithOptionalAlias parser_expression(false); + sharding_key_ast = parseQuery(parser_expression, sharding_key_str, 0); + engine_split_ast = createASTStorageDistributed(cluster_push_name, table_push.first, table_push.second, sharding_key_ast); + } + + where_condition_str = config.getString(table_prefix + "where_condition", ""); + if (!where_condition_str.empty()) + { + ParserExpressionWithOptionalAlias parser_expression(false); + where_condition_ast = parseQuery(parser_expression, where_condition_str, 0); + + // Will use canonical expression form + where_condition_str = queryToString(where_condition_ast); + } + + String enabled_partitions_prefix = table_prefix + "enabled_partitions"; + has_enabled_partitions = config.has(enabled_partitions_prefix); + + if (has_enabled_partitions) + { + Strings keys; + config.keys(enabled_partitions_prefix, keys); + + if (keys.empty()) + { + /// Parse list of partition from space-separated string + String partitions_str = config.getString(table_prefix + "enabled_partitions"); + boost::trim_if(partitions_str, isWhitespaceASCII); + boost::split(enabled_partitions, partitions_str, isWhitespaceASCII, boost::token_compress_on); + } + else + { + /// Parse sequence of ... + for (const String & key : keys) + { + if (!startsWith(key, "partition")) + throw Exception("Unknown key " + key + " in " + enabled_partitions_prefix, ErrorCodes::UNKNOWN_ELEMENT_IN_CONFIG); + + enabled_partitions.emplace_back(config.getString(enabled_partitions_prefix + "." + key)); + } + } + + std::copy(enabled_partitions.begin(), enabled_partitions.end(), std::inserter(enabled_partitions_set, enabled_partitions_set.begin())); + } +} + + +static ShardPriority getReplicasPriority(const Cluster::Addresses & replicas, const std::string & local_hostname, UInt8 random) +{ + ShardPriority res; + + if (replicas.empty()) + return res; + + res.is_remote = 1; + for (auto & replica : replicas) + { + if (isLocalAddress(DNSResolver::instance().resolveHost(replica.host_name))) + { + res.is_remote = 0; + break; + } + } + + res.hostname_difference = std::numeric_limits::max(); + for (auto & replica : replicas) + { + size_t difference = getHostNameDifference(local_hostname, replica.host_name); + res.hostname_difference = std::min(difference, res.hostname_difference); + } + + res.random = random; + return res; +} + +template +void TaskTable::initShards(RandomEngine && random_engine) +{ + const String & fqdn_name = getFQDNOrHostName(); + std::uniform_int_distribution get_urand(0, std::numeric_limits::max()); + + // Compute the priority + for (auto & shard_info : cluster_pull->getShardsInfo()) + { + TaskShardPtr task_shard = std::make_shared(*this, shard_info); + const auto & replicas = cluster_pull->getShardsAddresses().at(task_shard->indexInCluster()); + task_shard->priority = getReplicasPriority(replicas, fqdn_name, get_urand(random_engine)); + + all_shards.emplace_back(task_shard); + } + + // Sort by priority + std::sort(all_shards.begin(), all_shards.end(), + [] (const TaskShardPtr & lhs, const TaskShardPtr & rhs) + { + return ShardPriority::greaterPriority(lhs->priority, rhs->priority); + }); + + // Cut local shards + auto it_first_remote = std::lower_bound(all_shards.begin(), all_shards.end(), 1, + [] (const TaskShardPtr & lhs, UInt8 is_remote) + { + return lhs->priority.is_remote < is_remote; + }); + + local_shards.assign(all_shards.begin(), it_first_remote); +} + + +void TaskCluster::loadTasks(const Poco::Util::AbstractConfiguration & config, const String & base_key) +{ + String prefix = base_key.empty() ? "" : base_key + "."; + + clusters_prefix = prefix + "remote_servers"; + if (!config.has(clusters_prefix)) + throw Exception("You should specify list of clusters in " + clusters_prefix, ErrorCodes::BAD_ARGUMENTS); + + Poco::Util::AbstractConfiguration::Keys tables_keys; + config.keys(prefix + "tables", tables_keys); + + for (const auto & table_key : tables_keys) + { + table_tasks.emplace_back(*this, config, prefix + "tables", table_key); + } +} + +void TaskCluster::reloadSettings(const Poco::Util::AbstractConfiguration & config, const String & base_key) +{ + String prefix = base_key.empty() ? "" : base_key + "."; + + max_workers = config.getUInt64(prefix + "max_workers"); + + settings_common = Settings(); + if (config.has(prefix + "settings")) + settings_common.loadSettingsFromConfig(prefix + "settings", config); + + settings_pull = settings_common; + if (config.has(prefix + "settings_pull")) + settings_pull.loadSettingsFromConfig(prefix + "settings_pull", config); + + settings_push = settings_common; + if (config.has(prefix + "settings_push")) + settings_push.loadSettingsFromConfig(prefix + "settings_push", config); + + auto set_default_value = [] (auto && setting, auto && default_value) + { + setting = setting.changed ? setting.value : default_value; + }; + + /// Override important settings + settings_pull.readonly = 1; + settings_push.insert_distributed_sync = 1; + set_default_value(settings_pull.load_balancing, LoadBalancing::NEAREST_HOSTNAME); + set_default_value(settings_pull.max_threads, 1); + set_default_value(settings_pull.max_block_size, 8192UL); + set_default_value(settings_pull.preferred_block_size_bytes, 0); + set_default_value(settings_push.insert_distributed_timeout, 0); +} + + +} // end of an anonymous namespace +} -#endif //CLICKHOUSE_INTERNALS_H From 22789a3b55ab392cf3de32db082224dbea0a72b6 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Tue, 11 Feb 2020 21:34:48 +0300 Subject: [PATCH 0018/1355] ShardPartitionPiece task added --- dbms/programs/copier/CMakeLists.txt | 2 +- dbms/programs/copier/ClusterCopier.cpp | 1879 --------------------- dbms/programs/copier/ClusterCopier.h | 1570 ++++++++++++++++- dbms/programs/copier/ClusterCopierApp.cpp | 173 ++ dbms/programs/copier/ClusterCopierApp.h | 87 + dbms/programs/copier/Internals.h | 74 +- dbms/programs/copier/ZookeeperStaff.h | 224 +++ 7 files changed, 2052 insertions(+), 1957 deletions(-) delete mode 100644 dbms/programs/copier/ClusterCopier.cpp create mode 100644 dbms/programs/copier/ClusterCopierApp.cpp create mode 100644 dbms/programs/copier/ClusterCopierApp.h create mode 100644 dbms/programs/copier/ZookeeperStaff.h diff --git a/dbms/programs/copier/CMakeLists.txt b/dbms/programs/copier/CMakeLists.txt index 8e13040b29d..9852dc92242 100644 --- a/dbms/programs/copier/CMakeLists.txt +++ b/dbms/programs/copier/CMakeLists.txt @@ -1,4 +1,4 @@ -set(CLICKHOUSE_COPIER_SOURCES ${CMAKE_CURRENT_SOURCE_DIR}/ClusterCopier.cpp) +set(CLICKHOUSE_COPIER_SOURCES ${CMAKE_CURRENT_SOURCE_DIR}/ClusterCopierApp.cpp) set(CLICKHOUSE_COPIER_LINK PRIVATE clickhouse_common_zookeeper clickhouse_parsers clickhouse_functions clickhouse_table_functions clickhouse_aggregate_functions clickhouse_dictionaries string_utils ${Poco_XML_LIBRARY} PUBLIC daemon) set(CLICKHOUSE_COPIER_INCLUDE SYSTEM PRIVATE ${PCG_RANDOM_INCLUDE_DIR}) diff --git a/dbms/programs/copier/ClusterCopier.cpp b/dbms/programs/copier/ClusterCopier.cpp deleted file mode 100644 index e8b3018a861..00000000000 --- a/dbms/programs/copier/ClusterCopier.cpp +++ /dev/null @@ -1,1879 +0,0 @@ -#include "ClusterCopier.h" - -#include "Internals.h" - -namespace DB -{ - - -using ConfigurationPtr = Poco::AutoPtr; - -static ConfigurationPtr getConfigurationFromXMLString(const std::string & xml_data) -{ - std::stringstream ss(xml_data); - Poco::XML::InputSource input_source{ss}; - return {new Poco::Util::XMLConfiguration{&input_source}}; -} - - -class ClusterCopier -{ -public: - - ClusterCopier(String task_path_, - String host_id_, - String proxy_database_name_, - Context & context_) - : - task_zookeeper_path(std::move(task_path_)), - host_id(std::move(host_id_)), - working_database_name(std::move(proxy_database_name_)), - context(context_), - log(&Poco::Logger::get("ClusterCopier")) - { - } - - void init() - { - auto zookeeper = context.getZooKeeper(); - - task_description_watch_callback = [this] (const Coordination::WatchResponse & response) - { - if (response.error != Coordination::ZOK) - return; - UInt64 version = ++task_descprtion_version; - LOG_DEBUG(log, "Task description should be updated, local version " << version); - }; - - task_description_path = task_zookeeper_path + "/description"; - task_cluster = std::make_unique(task_zookeeper_path, working_database_name); - - reloadTaskDescription(); - task_cluster_initial_config = task_cluster_current_config; - - task_cluster->loadTasks(*task_cluster_initial_config); - context.setClustersConfig(task_cluster_initial_config, task_cluster->clusters_prefix); - - /// Set up shards and their priority - task_cluster->random_engine.seed(task_cluster->random_device()); - for (auto & task_table : task_cluster->table_tasks) - { - task_table.cluster_pull = context.getCluster(task_table.cluster_pull_name); - task_table.cluster_push = context.getCluster(task_table.cluster_push_name); - task_table.initShards(task_cluster->random_engine); - } - - LOG_DEBUG(log, "Will process " << task_cluster->table_tasks.size() << " table tasks"); - - /// Do not initialize tables, will make deferred initialization in process() - - zookeeper->createAncestors(getWorkersPathVersion() + "/"); - zookeeper->createAncestors(getWorkersPath() + "/"); - } - - template - decltype(auto) retry(T && func, UInt64 max_tries = 100) - { - std::exception_ptr exception; - - for (UInt64 try_number = 1; try_number <= max_tries; ++try_number) - { - try - { - return func(); - } - catch (...) - { - exception = std::current_exception(); - if (try_number < max_tries) - { - tryLogCurrentException(log, "Will retry"); - std::this_thread::sleep_for(default_sleep_time); - } - } - } - - std::rethrow_exception(exception); - } - - - void discoverShardPartitions(const ConnectionTimeouts & timeouts, const TaskShardPtr & task_shard) - { - TaskTable & task_table = task_shard->task_table; - - LOG_INFO(log, "Discover partitions of shard " << task_shard->getDescription()); - - auto get_partitions = [&] () { return getShardPartitions(timeouts, *task_shard); }; - auto existing_partitions_names = retry(get_partitions, 60); - Strings filtered_partitions_names; - Strings missing_partitions; - - /// Check that user specified correct partition names - auto check_partition_format = [] (const DataTypePtr & type, const String & partition_text_quoted) - { - MutableColumnPtr column_dummy = type->createColumn(); - ReadBufferFromString rb(partition_text_quoted); - - try - { - type->deserializeAsTextQuoted(*column_dummy, rb, FormatSettings()); - } - catch (Exception & e) - { - throw Exception("Partition " + partition_text_quoted + " has incorrect format. " + e.displayText(), ErrorCodes::BAD_ARGUMENTS); - } - }; - - if (task_table.has_enabled_partitions) - { - /// Process partition in order specified by - for (const String & partition_name : task_table.enabled_partitions) - { - /// Check that user specified correct partition names - check_partition_format(task_shard->partition_key_column.type, partition_name); - - auto it = existing_partitions_names.find(partition_name); - - /// Do not process partition if it is not in enabled_partitions list - if (it == existing_partitions_names.end()) - { - missing_partitions.emplace_back(partition_name); - continue; - } - - filtered_partitions_names.emplace_back(*it); - } - - for (const String & partition_name : existing_partitions_names) - { - if (!task_table.enabled_partitions_set.count(partition_name)) - { - LOG_DEBUG(log, "Partition " << partition_name << " will not be processed, since it is not in " - << "enabled_partitions of " << task_table.table_id); - } - } - } - else - { - for (const String & partition_name : existing_partitions_names) - filtered_partitions_names.emplace_back(partition_name); - } - - for (const String & partition_name : filtered_partitions_names) - { - task_shard->partition_tasks.emplace(partition_name, ShardPartition(*task_shard, partition_name)); - task_shard->checked_partitions.emplace(partition_name, true); - } - - if (!missing_partitions.empty()) - { - std::stringstream ss; - for (const String & missing_partition : missing_partitions) - ss << " " << missing_partition; - - LOG_WARNING(log, "There are no " << missing_partitions.size() << " partitions from enabled_partitions in shard " - << task_shard->getDescription() << " :" << ss.str()); - } - - LOG_DEBUG(log, "Will copy " << task_shard->partition_tasks.size() << " partitions from shard " << task_shard->getDescription()); - } - - /// Compute set of partitions, assume set of partitions aren't changed during the processing - void discoverTablePartitions(const ConnectionTimeouts & timeouts, TaskTable & task_table, UInt64 num_threads = 0) - { - /// Fetch partitions list from a shard - { - ThreadPool thread_pool(num_threads ? num_threads : 2 * getNumberOfPhysicalCPUCores()); - - for (const TaskShardPtr & task_shard : task_table.all_shards) - thread_pool.scheduleOrThrowOnError([this, timeouts, task_shard]() { discoverShardPartitions(timeouts, task_shard); }); - - LOG_DEBUG(log, "Waiting for " << thread_pool.active() << " setup jobs"); - thread_pool.wait(); - } - } - - void uploadTaskDescription(const std::string & task_path, const std::string & task_file, const bool force) - { - auto local_task_description_path = task_path + "/description"; - - String task_config_str; - { - ReadBufferFromFile in(task_file); - readStringUntilEOF(task_config_str, in); - } - if (task_config_str.empty()) - return; - - auto zookeeper = context.getZooKeeper(); - - zookeeper->createAncestors(local_task_description_path); - auto code = zookeeper->tryCreate(local_task_description_path, task_config_str, zkutil::CreateMode::Persistent); - if (code && force) - zookeeper->createOrUpdate(local_task_description_path, task_config_str, zkutil::CreateMode::Persistent); - - LOG_DEBUG(log, "Task description " << ((code && !force) ? "not " : "") << "uploaded to " << local_task_description_path << " with result " << code << " ("<< zookeeper->error2string(code) << ")"); - } - - void reloadTaskDescription() - { - auto zookeeper = context.getZooKeeper(); - task_description_watch_zookeeper = zookeeper; - - String task_config_str; - Coordination::Stat stat{}; - int code; - - zookeeper->tryGetWatch(task_description_path, task_config_str, &stat, task_description_watch_callback, &code); - if (code) - throw Exception("Can't get description node " + task_description_path, ErrorCodes::BAD_ARGUMENTS); - - LOG_DEBUG(log, "Loading description, zxid=" << task_description_current_stat.czxid); - auto config = getConfigurationFromXMLString(task_config_str); - - /// Setup settings - task_cluster->reloadSettings(*config); - context.getSettingsRef() = task_cluster->settings_common; - - task_cluster_current_config = config; - task_description_current_stat = stat; - } - - void updateConfigIfNeeded() - { - UInt64 version_to_update = task_descprtion_version; - bool is_outdated_version = task_descprtion_current_version != version_to_update; - bool is_expired_session = !task_description_watch_zookeeper || task_description_watch_zookeeper->expired(); - - if (!is_outdated_version && !is_expired_session) - return; - - LOG_DEBUG(log, "Updating task description"); - reloadTaskDescription(); - - task_descprtion_current_version = version_to_update; - } - - void process(const ConnectionTimeouts & timeouts) - { - for (TaskTable & task_table : task_cluster->table_tasks) - { - LOG_INFO(log, "Process table task " << task_table.table_id << " with " - << task_table.all_shards.size() << " shards, " << task_table.local_shards.size() << " of them are local ones"); - - if (task_table.all_shards.empty()) - continue; - - /// Discover partitions of each shard and total set of partitions - if (!task_table.has_enabled_partitions) - { - /// If there are no specified enabled_partitions, we must discover them manually - discoverTablePartitions(timeouts, task_table); - - /// After partitions of each shard are initialized, initialize cluster partitions - for (const TaskShardPtr & task_shard : task_table.all_shards) - { - for (const auto & partition_elem : task_shard->partition_tasks) - { - const String & partition_name = partition_elem.first; - task_table.cluster_partitions.emplace(partition_name, ClusterPartition{}); - } - } - - for (auto & partition_elem : task_table.cluster_partitions) - { - const String & partition_name = partition_elem.first; - - for (const TaskShardPtr & task_shard : task_table.all_shards) - task_shard->checked_partitions.emplace(partition_name); - - task_table.ordered_partition_names.emplace_back(partition_name); - } - } - else - { - /// If enabled_partitions are specified, assume that each shard has all partitions - /// We will refine partition set of each shard in future - - for (const String & partition_name : task_table.enabled_partitions) - { - task_table.cluster_partitions.emplace(partition_name, ClusterPartition{}); - task_table.ordered_partition_names.emplace_back(partition_name); - } - } - - task_table.watch.restart(); - - /// Retry table processing - bool table_is_done = false; - for (UInt64 num_table_tries = 0; num_table_tries < max_table_tries; ++num_table_tries) - { - if (tryProcessTable(timeouts, task_table)) - { - table_is_done = true; - break; - } - } - - if (!table_is_done) - { - throw Exception("Too many tries to process table " + task_table.table_id + ". Abort remaining execution", - ErrorCodes::UNFINISHED); - } - } - } - - /// Disables DROP PARTITION commands that used to clear data after errors - void setSafeMode(bool is_safe_mode_ = true) - { - is_safe_mode = is_safe_mode_; - } - - void setCopyFaultProbability(double copy_fault_probability_) - { - copy_fault_probability = copy_fault_probability_; - } - - -protected: - - String getWorkersPath() const - { - return task_cluster->task_zookeeper_path + "/task_active_workers"; - } - - String getWorkersPathVersion() const - { - return getWorkersPath() + "_version"; - } - - String getCurrentWorkerNodePath() const - { - return getWorkersPath() + "/" + host_id; - } - - zkutil::EphemeralNodeHolder::Ptr createTaskWorkerNodeAndWaitIfNeed( - const zkutil::ZooKeeperPtr & zookeeper, - const String & description, - bool unprioritized) - { - std::chrono::milliseconds current_sleep_time = default_sleep_time; - static constexpr std::chrono::milliseconds max_sleep_time(30000); // 30 sec - - if (unprioritized) - std::this_thread::sleep_for(current_sleep_time); - - String workers_version_path = getWorkersPathVersion(); - String workers_path = getWorkersPath(); - String current_worker_path = getCurrentWorkerNodePath(); - - UInt64 num_bad_version_errors = 0; - - while (true) - { - updateConfigIfNeeded(); - - Coordination::Stat stat{}; - zookeeper->get(workers_version_path, &stat); - auto version = stat.version; - zookeeper->get(workers_path, &stat); - - if (static_cast(stat.numChildren) >= task_cluster->max_workers) - { - LOG_DEBUG(log, "Too many workers (" << stat.numChildren << ", maximum " << task_cluster->max_workers << ")" - << ". Postpone processing " << description); - - if (unprioritized) - current_sleep_time = std::min(max_sleep_time, current_sleep_time + default_sleep_time); - - std::this_thread::sleep_for(current_sleep_time); - num_bad_version_errors = 0; - } - else - { - Coordination::Requests ops; - ops.emplace_back(zkutil::makeSetRequest(workers_version_path, description, version)); - ops.emplace_back(zkutil::makeCreateRequest(current_worker_path, description, zkutil::CreateMode::Ephemeral)); - Coordination::Responses responses; - auto code = zookeeper->tryMulti(ops, responses); - - if (code == Coordination::ZOK || code == Coordination::ZNODEEXISTS) - return std::make_shared(current_worker_path, *zookeeper, false, false, description); - - if (code == Coordination::ZBADVERSION) - { - ++num_bad_version_errors; - - /// Try to make fast retries - if (num_bad_version_errors > 3) - { - LOG_DEBUG(log, "A concurrent worker has just been added, will check free worker slots again"); - std::chrono::milliseconds random_sleep_time(std::uniform_int_distribution(1, 1000)(task_cluster->random_engine)); - std::this_thread::sleep_for(random_sleep_time); - num_bad_version_errors = 0; - } - } - else - throw Coordination::Exception(code); - } - } - } - - /** Checks that the whole partition of a table was copied. We should do it carefully due to dirty lock. - * State of some task could change during the processing. - * We have to ensure that all shards have the finished state and there is no dirty flag. - * Moreover, we have to check status twice and check zxid, because state can change during the checking. - */ - bool checkPartitionIsDone(const TaskTable & task_table, const String & partition_name, const TasksShard & shards_with_partition) - { - LOG_DEBUG(log, "Check that all shards processed partition " << partition_name << " successfully"); - - auto zookeeper = context.getZooKeeper(); - - Strings status_paths; - for (auto & shard : shards_with_partition) - { - ShardPartition & task_shard_partition = shard->partition_tasks.find(partition_name)->second; - status_paths.emplace_back(task_shard_partition.getShardStatusPath()); - } - - std::vector zxid1, zxid2; - - try - { - std::vector get_futures; - for (const String & path : status_paths) - get_futures.emplace_back(zookeeper->asyncGet(path)); - - // Check that state is Finished and remember zxid - for (auto & future : get_futures) - { - auto res = future.get(); - - TaskStateWithOwner status = TaskStateWithOwner::fromString(res.data); - if (status.state != TaskState::Finished) - { - LOG_INFO(log, "The task " << res.data << " is being rewritten by " << status.owner << ". Partition will be rechecked"); - return false; - } - - zxid1.push_back(res.stat.pzxid); - } - - // Check that partition is not dirty - { - CleanStateClock clean_state_clock ( - zookeeper, - task_table.getPartitionIsDirtyPath(partition_name), - task_table.getPartitionIsCleanedPath(partition_name) - ); - Coordination::Stat stat{}; - LogicalClock task_start_clock; - if (zookeeper->exists(task_table.getPartitionTaskStatusPath(partition_name), &stat)) - task_start_clock = LogicalClock(stat.mzxid); - zookeeper->get(task_table.getPartitionTaskStatusPath(partition_name), &stat); - if (!clean_state_clock.is_clean() || task_start_clock <= clean_state_clock.discovery_zxid) - { - LOG_INFO(log, "Partition " << partition_name << " become dirty"); - return false; - } - } - - get_futures.clear(); - for (const String & path : status_paths) - get_futures.emplace_back(zookeeper->asyncGet(path)); - - // Remember zxid of states again - for (auto & future : get_futures) - { - auto res = future.get(); - zxid2.push_back(res.stat.pzxid); - } - } - catch (const Coordination::Exception & e) - { - LOG_INFO(log, "A ZooKeeper error occurred while checking partition " << partition_name - << ". Will recheck the partition. Error: " << e.displayText()); - return false; - } - - // If all task is finished and zxid is not changed then partition could not become dirty again - for (UInt64 shard_num = 0; shard_num < status_paths.size(); ++shard_num) - { - if (zxid1[shard_num] != zxid2[shard_num]) - { - LOG_INFO(log, "The task " << status_paths[shard_num] << " is being modified now. Partition will be rechecked"); - return false; - } - } - - LOG_INFO(log, "Partition " << partition_name << " is copied successfully"); - return true; - } - - /// Removes MATERIALIZED and ALIAS columns from create table query - static ASTPtr removeAliasColumnsFromCreateQuery(const ASTPtr & query_ast) - { - const ASTs & column_asts = query_ast->as().columns_list->columns->children; - auto new_columns = std::make_shared(); - - for (const ASTPtr & column_ast : column_asts) - { - const auto & column = column_ast->as(); - - if (!column.default_specifier.empty()) - { - ColumnDefaultKind kind = columnDefaultKindFromString(column.default_specifier); - if (kind == ColumnDefaultKind::Materialized || kind == ColumnDefaultKind::Alias) - continue; - } - - new_columns->children.emplace_back(column_ast->clone()); - } - - ASTPtr new_query_ast = query_ast->clone(); - auto & new_query = new_query_ast->as(); - - auto new_columns_list = std::make_shared(); - new_columns_list->set(new_columns_list->columns, new_columns); - if (auto indices = query_ast->as()->columns_list->indices) - new_columns_list->set(new_columns_list->indices, indices->clone()); - - new_query.replace(new_query.columns_list, new_columns_list); - - return new_query_ast; - } - - /// Replaces ENGINE and table name in a create query - static std::shared_ptr rewriteCreateQueryStorage(const ASTPtr & create_query_ast, const DatabaseAndTableName & new_table, const ASTPtr & new_storage_ast) - { - const auto & create = create_query_ast->as(); - auto res = std::make_shared(create); - - if (create.storage == nullptr || new_storage_ast == nullptr) - throw Exception("Storage is not specified", ErrorCodes::LOGICAL_ERROR); - - res->database = new_table.first; - res->table = new_table.second; - - res->children.clear(); - res->set(res->columns_list, create.columns_list->clone()); - res->set(res->storage, new_storage_ast->clone()); - - return res; - } - - /** Allows to compare two incremental counters of type UInt32 in presence of possible overflow. - * We assume that we compare values that are not too far away. - * For example, when we increment 0xFFFFFFFF, we get 0. So, 0xFFFFFFFF is less than 0. - */ - class WrappingUInt32 - { - public: - UInt32 value; - - explicit WrappingUInt32(UInt32 _value) - : value(_value) - {} - - bool operator<(const WrappingUInt32 & other) const - { - return value != other.value && *this <= other; - } - - bool operator<=(const WrappingUInt32 & other) const - { - const UInt32 HALF = 1 << 31; - return (value <= other.value && other.value - value < HALF) - || (value > other.value && value - other.value > HALF); - } - - bool operator==(const WrappingUInt32 & other) const - { - return value == other.value; - } - }; - - /** Conforming Zxid definition. - * cf. https://github.com/apache/zookeeper/blob/631d1b284f0edb1c4f6b0fb221bf2428aec71aaa/zookeeper-docs/src/main/resources/markdown/zookeeperInternals.md#guarantees-properties-and-definitions - * - * But it is better to read this: https://zookeeper.apache.org/doc/r3.1.2/zookeeperProgrammers.html - * - * Actually here is the definition of Zxid. - * Every change to the ZooKeeper state receives a stamp in the form of a zxid (ZooKeeper Transaction Id). - * This exposes the total ordering of all changes to ZooKeeper. Each change will have a unique zxid - * and if zxid1 is smaller than zxid2 then zxid1 happened before zxid2. - */ - class Zxid - { - public: - WrappingUInt32 epoch; - WrappingUInt32 counter; - explicit Zxid(UInt64 _zxid) - : epoch(_zxid >> 32) - , counter(_zxid) - {} - - bool operator<=(const Zxid & other) const - { - return (epoch < other.epoch) - || (epoch == other.epoch && counter <= other.counter); - } - - bool operator==(const Zxid & other) const - { - return epoch == other.epoch && counter == other.counter; - } - }; - - /* When multiple ClusterCopiers discover that the target partition is not empty, - * they will attempt to clean up this partition before proceeding to copying. - * - * Instead of purging is_dirty, the history of cleaning work is preserved and partition hygiene is established - * based on a happens-before relation between the events. - * This relation is encoded by LogicalClock based on the mzxid of the is_dirty ZNode and is_dirty/cleaned. - * The fact of the partition hygiene is encoded by CleanStateClock. - * - * For you to know what mzxid means: - * - * ZooKeeper Stat Structure: - * The Stat structure for each znode in ZooKeeper is made up of the following fields: - * - * -- czxid - * The zxid of the change that caused this znode to be created. - * - * -- mzxid - * The zxid of the change that last modified this znode. - * - * -- ctime - * The time in milliseconds from epoch when this znode was created. - * - * -- mtime - * The time in milliseconds from epoch when this znode was last modified. - * - * -- version - * The number of changes to the data of this znode. - * - * -- cversion - * The number of changes to the children of this znode. - * - * -- aversion - * The number of changes to the ACL of this znode. - * - * -- ephemeralOwner - * The session id of the owner of this znode if the znode is an ephemeral node. - * If it is not an ephemeral node, it will be zero. - * - * -- dataLength - * The length of the data field of this znode. - * - * -- numChildren - * The number of children of this znode. - * */ - - class LogicalClock - { - public: - std::optional zxid; - - LogicalClock() = default; - - explicit LogicalClock(UInt64 _zxid) - : zxid(_zxid) - {} - - bool hasHappened() const - { - return bool(zxid); - } - - /// happens-before relation with a reasonable time bound - bool happensBefore(const LogicalClock & other) const - { - return !zxid - || (other.zxid && *zxid <= *other.zxid); - } - - bool operator<=(const LogicalClock & other) const - { - return happensBefore(other); - } - - /// strict equality check - bool operator==(const LogicalClock & other) const - { - return zxid == other.zxid; - } - }; - - - class CleanStateClock - { - public: - LogicalClock discovery_zxid; - std::optional discovery_version; - - LogicalClock clean_state_zxid; - std::optional clean_state_version; - - std::shared_ptr stale; - - bool is_clean() const - { - return - !is_stale() - && ( - !discovery_zxid.hasHappened() - || (clean_state_zxid.hasHappened() && discovery_zxid <= clean_state_zxid)); - } - - bool is_stale() const - { - return stale->load(); - } - - CleanStateClock( - const zkutil::ZooKeeperPtr & zookeeper, - const String & discovery_path, - const String & clean_state_path) - : stale(std::make_shared(false)) - { - Coordination::Stat stat{}; - String _some_data; - auto watch_callback = - [stale = stale] (const Coordination::WatchResponse & rsp) - { - auto logger = &Poco::Logger::get("ClusterCopier"); - if (rsp.error == Coordination::ZOK) - { - switch (rsp.type) - { - case Coordination::CREATED: - LOG_DEBUG(logger, "CleanStateClock change: CREATED, at " << rsp.path); - stale->store(true); - break; - case Coordination::CHANGED: - LOG_DEBUG(logger, "CleanStateClock change: CHANGED, at" << rsp.path); - stale->store(true); - } - } - }; - if (zookeeper->tryGetWatch(discovery_path, _some_data, &stat, watch_callback)) - { - discovery_zxid = LogicalClock(stat.mzxid); - discovery_version = stat.version; - } - if (zookeeper->tryGetWatch(clean_state_path, _some_data, &stat, watch_callback)) - { - clean_state_zxid = LogicalClock(stat.mzxid); - clean_state_version = stat.version; - } - } - - bool operator==(const CleanStateClock & other) const - { - return !is_stale() - && !other.is_stale() - && discovery_zxid == other.discovery_zxid - && discovery_version == other.discovery_version - && clean_state_zxid == other.clean_state_zxid - && clean_state_version == other.clean_state_version; - } - - bool operator!=(const CleanStateClock & other) const - { - return !(*this == other); - } - }; - - bool tryDropPartition(ShardPartition & task_partition, const zkutil::ZooKeeperPtr & zookeeper, const CleanStateClock & clean_state_clock) - { - if (is_safe_mode) - throw Exception("DROP PARTITION is prohibited in safe mode", ErrorCodes::NOT_IMPLEMENTED); - - TaskTable & task_table = task_partition.task_shard.task_table; - - const String current_shards_path = task_partition.getPartitionShardsPath(); - const String current_partition_active_workers_dir = task_partition.getPartitionActiveWorkersPath(); - const String is_dirty_flag_path = task_partition.getCommonPartitionIsDirtyPath(); - const String dirty_cleaner_path = is_dirty_flag_path + "/cleaner"; - const String is_dirty_cleaned_path = task_partition.getCommonPartitionIsCleanedPath(); - - zkutil::EphemeralNodeHolder::Ptr cleaner_holder; - try - { - cleaner_holder = zkutil::EphemeralNodeHolder::create(dirty_cleaner_path, *zookeeper, host_id); - } - catch (const Coordination::Exception & e) - { - if (e.code == Coordination::ZNODEEXISTS) - { - LOG_DEBUG(log, "Partition " << task_partition.name << " is cleaning now by somebody, sleep"); - std::this_thread::sleep_for(default_sleep_time); - return false; - } - - throw; - } - - Coordination::Stat stat{}; - if (zookeeper->exists(current_partition_active_workers_dir, &stat)) - { - if (stat.numChildren != 0) - { - LOG_DEBUG(log, "Partition " << task_partition.name << " contains " << stat.numChildren << " active workers while trying to drop it. Going to sleep."); - std::this_thread::sleep_for(default_sleep_time); - return false; - } - else - { - zookeeper->remove(current_partition_active_workers_dir); - } - } - - { - zkutil::EphemeralNodeHolder::Ptr active_workers_lock; - try - { - active_workers_lock = zkutil::EphemeralNodeHolder::create(current_partition_active_workers_dir, *zookeeper, host_id); - } - catch (const Coordination::Exception & e) - { - if (e.code == Coordination::ZNODEEXISTS) - { - LOG_DEBUG(log, "Partition " << task_partition.name << " is being filled now by somebody, sleep"); - return false; - } - - throw; - } - - // Lock the dirty flag - zookeeper->set(is_dirty_flag_path, host_id, clean_state_clock.discovery_version.value()); - zookeeper->tryRemove(task_partition.getPartitionCleanStartPath()); - CleanStateClock my_clock(zookeeper, is_dirty_flag_path, is_dirty_cleaned_path); - - /// Remove all status nodes - { - Strings children; - if (zookeeper->tryGetChildren(current_shards_path, children) == Coordination::ZOK) - for (const auto & child : children) - { - zookeeper->removeRecursive(current_shards_path + "/" + child); - } - } - - String query = "ALTER TABLE " + getQuotedTable(task_table.table_push); - query += " DROP PARTITION " + task_partition.name + ""; - - /// TODO: use this statement after servers will be updated up to 1.1.54310 - // query += " DROP PARTITION ID '" + task_partition.name + "'"; - - ClusterPtr & cluster_push = task_table.cluster_push; - Settings settings_push = task_cluster->settings_push; - - /// It is important, DROP PARTITION must be done synchronously - settings_push.replication_alter_partitions_sync = 2; - - LOG_DEBUG(log, "Execute distributed DROP PARTITION: " << query); - /// Limit number of max executing replicas to 1 - UInt64 num_shards = executeQueryOnCluster(cluster_push, query, nullptr, &settings_push, PoolMode::GET_ONE, 1); - - if (num_shards < cluster_push->getShardCount()) - { - LOG_INFO(log, "DROP PARTITION wasn't successfully executed on " << cluster_push->getShardCount() - num_shards << " shards"); - return false; - } - - /// Update the locking node - if (!my_clock.is_stale()) - { - zookeeper->set(is_dirty_flag_path, host_id, my_clock.discovery_version.value()); - if (my_clock.clean_state_version) - zookeeper->set(is_dirty_cleaned_path, host_id, my_clock.clean_state_version.value()); - else - zookeeper->create(is_dirty_cleaned_path, host_id, zkutil::CreateMode::Persistent); - } - else - { - LOG_DEBUG(log, "Clean state is altered when dropping the partition, cowardly bailing"); - /// clean state is stale - return false; - } - - LOG_INFO(log, "Partition " << task_partition.name << " was dropped on cluster " << task_table.cluster_push_name); - if (zookeeper->tryCreate(current_shards_path, host_id, zkutil::CreateMode::Persistent) == Coordination::ZNODEEXISTS) - zookeeper->set(current_shards_path, host_id); - } - - LOG_INFO(log, "Partition " << task_partition.name << " is safe for work now."); - return true; - } - - - static constexpr UInt64 max_table_tries = 1000; - static constexpr UInt64 max_shard_partition_tries = 600; - - bool tryProcessTable(const ConnectionTimeouts & timeouts, TaskTable & task_table) - { - /// A heuristic: if previous shard is already done, then check next one without sleeps due to max_workers constraint - bool previous_shard_is_instantly_finished = false; - - /// Process each partition that is present in cluster - for (const String & partition_name : task_table.ordered_partition_names) - { - if (!task_table.cluster_partitions.count(partition_name)) - throw Exception("There are no expected partition " + partition_name + ". It is a bug", ErrorCodes::LOGICAL_ERROR); - - ClusterPartition & cluster_partition = task_table.cluster_partitions[partition_name]; - - Stopwatch watch; - /// We will check all the shards of the table and check if they contain current partition. - TasksShard expected_shards; - UInt64 num_failed_shards = 0; - - ++cluster_partition.total_tries; - - LOG_DEBUG(log, "Processing partition " << partition_name << " for the whole cluster"); - - /// Process each source shard having current partition and copy current partition - /// NOTE: shards are sorted by "distance" to current host - bool has_shard_to_process = false; - for (const TaskShardPtr & shard : task_table.all_shards) - { - /// Does shard have a node with current partition? - if (shard->partition_tasks.count(partition_name) == 0) - { - /// If not, did we check existence of that partition previously? - if (shard->checked_partitions.count(partition_name) == 0) - { - auto check_shard_has_partition = [&] () { return checkShardHasPartition(timeouts, *shard, partition_name); }; - bool has_partition = retry(check_shard_has_partition); - - shard->checked_partitions.emplace(partition_name); - - if (has_partition) - { - shard->partition_tasks.emplace(partition_name, ShardPartition(*shard, partition_name)); - LOG_DEBUG(log, "Discovered partition " << partition_name << " in shard " << shard->getDescription()); - } - else - { - LOG_DEBUG(log, "Found that shard " << shard->getDescription() << " does not contain current partition " << partition_name); - continue; - } - } - else - { - /// We have already checked that partition, but did not discover it - previous_shard_is_instantly_finished = true; - continue; - } - } - - auto it_shard_partition = shard->partition_tasks.find(partition_name); - /// Previously when we discovered that shard does not contain current partition, we skipped it. - /// At this moment partition have to be present. - if (it_shard_partition == shard->partition_tasks.end()) - throw Exception("There are no such partition in a shard. This is a bug.", ErrorCodes::LOGICAL_ERROR); - auto & partition = it_shard_partition->second; - - expected_shards.emplace_back(shard); - - /// Do not sleep if there is a sequence of already processed shards to increase startup - bool is_unprioritized_task = !previous_shard_is_instantly_finished && shard->priority.is_remote; - PartitionTaskStatus task_status = PartitionTaskStatus::Error; - bool was_error = false; - has_shard_to_process = true; - for (UInt64 try_num = 0; try_num < max_shard_partition_tries; ++try_num) - { - task_status = tryProcessPartitionTask(timeouts, partition, is_unprioritized_task); - - /// Exit if success - if (task_status == PartitionTaskStatus::Finished) - break; - - was_error = true; - - /// Skip if the task is being processed by someone - if (task_status == PartitionTaskStatus::Active) - break; - - /// Repeat on errors - std::this_thread::sleep_for(default_sleep_time); - } - - if (task_status == PartitionTaskStatus::Error) - ++num_failed_shards; - - previous_shard_is_instantly_finished = !was_error; - } - - cluster_partition.elapsed_time_seconds += watch.elapsedSeconds(); - - /// Check that whole cluster partition is done - /// Firstly check the number of failed partition tasks, then look into ZooKeeper and ensure that each partition is done - bool partition_is_done = num_failed_shards == 0; - try - { - partition_is_done = - !has_shard_to_process - || (partition_is_done && checkPartitionIsDone(task_table, partition_name, expected_shards)); - } - catch (...) - { - tryLogCurrentException(log); - partition_is_done = false; - } - - if (partition_is_done) - { - task_table.finished_cluster_partitions.emplace(partition_name); - - task_table.bytes_copied += cluster_partition.bytes_copied; - task_table.rows_copied += cluster_partition.rows_copied; - double elapsed = cluster_partition.elapsed_time_seconds; - - LOG_INFO(log, "It took " << std::fixed << std::setprecision(2) << elapsed << " seconds to copy partition " << partition_name - << ": " << formatReadableSizeWithDecimalSuffix(cluster_partition.bytes_copied) << " uncompressed bytes" - << ", " << formatReadableQuantity(cluster_partition.rows_copied) << " rows" - << " and " << cluster_partition.blocks_copied << " source blocks are copied"); - - if (cluster_partition.rows_copied) - { - LOG_INFO(log, "Average partition speed: " - << formatReadableSizeWithDecimalSuffix(cluster_partition.bytes_copied / elapsed) << " per second."); - } - - if (task_table.rows_copied) - { - LOG_INFO(log, "Average table " << task_table.table_id << " speed: " - << formatReadableSizeWithDecimalSuffix(task_table.bytes_copied / elapsed) << " per second."); - } - } - } - - UInt64 required_partitions = task_table.cluster_partitions.size(); - UInt64 finished_partitions = task_table.finished_cluster_partitions.size(); - bool table_is_done = finished_partitions >= required_partitions; - - if (!table_is_done) - { - LOG_INFO(log, "Table " + task_table.table_id + " is not processed yet." - << "Copied " << finished_partitions << " of " << required_partitions << ", will retry"); - } - - return table_is_done; - } - - - /// Execution status of a task - enum class PartitionTaskStatus - { - Active, - Finished, - Error, - }; - - /// Job for copying partition from particular shard. - PartitionTaskStatus tryProcessPartitionTask(const ConnectionTimeouts & timeouts, ShardPartition & task_partition, bool is_unprioritized_task) - { - PartitionTaskStatus res; - - try - { - res = processPartitionTaskImpl(timeouts, task_partition, is_unprioritized_task); - } - catch (...) - { - tryLogCurrentException(log, "An error occurred while processing partition " + task_partition.name); - res = PartitionTaskStatus::Error; - } - - /// At the end of each task check if the config is updated - try - { - updateConfigIfNeeded(); - } - catch (...) - { - tryLogCurrentException(log, "An error occurred while updating the config"); - } - - return res; - } - - PartitionTaskStatus processPartitionTaskImpl(const ConnectionTimeouts & timeouts, ShardPartition & task_partition, bool is_unprioritized_task) - { - TaskShard & task_shard = task_partition.task_shard; - TaskTable & task_table = task_shard.task_table; - ClusterPartition & cluster_partition = task_table.getClusterPartition(task_partition.name); - const size_t number_of_splits = task_table.number_of_splits; - UNUSED(number_of_splits); - - /// We need to update table definitions for each partition, it could be changed after ALTER - createShardInternalTables(timeouts, task_shard); - - auto zookeeper = context.getZooKeeper(); - - const String is_dirty_flag_path = task_partition.getCommonPartitionIsDirtyPath(); - const String is_dirty_cleaned_path = task_partition.getCommonPartitionIsCleanedPath(); - const String current_task_is_active_path = task_partition.getActiveWorkerPath(); - const String current_task_status_path = task_partition.getShardStatusPath(); - - /// Auxiliary functions: - - /// Creates is_dirty node to initialize DROP PARTITION - auto create_is_dirty_node = [&, this] (const CleanStateClock & clock) - { - if (clock.is_stale()) - LOG_DEBUG(log, "Clean state clock is stale while setting dirty flag, cowardly bailing"); - else if (!clock.is_clean()) - LOG_DEBUG(log, "Thank you, Captain Obvious"); - else if (clock.discovery_version) - { - LOG_DEBUG(log, "Updating clean state clock"); - zookeeper->set(is_dirty_flag_path, host_id, clock.discovery_version.value()); - } - else - { - LOG_DEBUG(log, "Creating clean state clock"); - zookeeper->create(is_dirty_flag_path, host_id, zkutil::CreateMode::Persistent); - } - }; - - /// Returns SELECT query filtering current partition and applying user filter - auto get_select_query = [&] (const DatabaseAndTableName & from_table, const String & fields, String limit = "", - bool enable_splitting = false, size_t current_piece_number = 0) - { - String query; - query += "SELECT " + fields + " FROM " + getQuotedTable(from_table); - /// TODO: Bad, it is better to rewrite with ASTLiteral(partition_key_field) - query += " WHERE (" + queryToString(task_table.engine_push_partition_key_ast) + " = (" + task_partition.name + " AS partition_key))"; - if (enable_splitting) - query += " AND ( cityHash64(*) = " + std::to_string(current_piece_number) + " )"; - if (!task_table.where_condition_str.empty()) - query += " AND (" + task_table.where_condition_str + ")"; - if (!limit.empty()) - query += " LIMIT " + limit; - - ParserQuery p_query(query.data() + query.size()); - return parseQuery(p_query, query, 0); - }; - - /// Load balancing - auto worker_node_holder = createTaskWorkerNodeAndWaitIfNeed(zookeeper, current_task_status_path, is_unprioritized_task); - - LOG_DEBUG(log, "Processing " << current_task_status_path); - - CleanStateClock clean_state_clock (zookeeper, is_dirty_flag_path, is_dirty_cleaned_path); - - LogicalClock task_start_clock; - { - Coordination::Stat stat{}; - if (zookeeper->exists(task_partition.getPartitionShardsPath(), &stat)) - task_start_clock = LogicalClock(stat.mzxid); - } - - /// Do not start if partition is dirty, try to clean it - if (clean_state_clock.is_clean() - && (!task_start_clock.hasHappened() || clean_state_clock.discovery_zxid <= task_start_clock)) - { - LOG_DEBUG(log, "Partition " << task_partition.name << " appears to be clean"); - zookeeper->createAncestors(current_task_status_path); - } - else - { - LOG_DEBUG(log, "Partition " << task_partition.name << " is dirty, try to drop it"); - - try - { - tryDropPartition(task_partition, zookeeper, clean_state_clock); - } - catch (...) - { - tryLogCurrentException(log, "An error occurred when clean partition"); - } - - return PartitionTaskStatus::Error; - } - - /// Create ephemeral node to mark that we are active and process the partition - zookeeper->createAncestors(current_task_is_active_path); - zkutil::EphemeralNodeHolderPtr partition_task_node_holder; - try - { - partition_task_node_holder = zkutil::EphemeralNodeHolder::create(current_task_is_active_path, *zookeeper, host_id); - } - catch (const Coordination::Exception & e) - { - if (e.code == Coordination::ZNODEEXISTS) - { - LOG_DEBUG(log, "Someone is already processing " << current_task_is_active_path); - return PartitionTaskStatus::Active; - } - - throw; - } - - /// Exit if task has been already processed; - /// create blocking node to signal cleaning up if it is abandoned - { - String status_data; - if (zookeeper->tryGet(current_task_status_path, status_data)) - { - TaskStateWithOwner status = TaskStateWithOwner::fromString(status_data); - if (status.state == TaskState::Finished) - { - LOG_DEBUG(log, "Task " << current_task_status_path << " has been successfully executed by " << status.owner); - return PartitionTaskStatus::Finished; - } - - // Task is abandoned, initialize DROP PARTITION - LOG_DEBUG(log, "Task " << current_task_status_path << " has not been successfully finished by " << - status.owner << ". Partition will be dropped and refilled."); - - create_is_dirty_node(clean_state_clock); - return PartitionTaskStatus::Error; - } - } - - /// Check that destination partition is empty if we are first worker - /// NOTE: this check is incorrect if pull and push tables have different partition key! - String clean_start_status; - if (!zookeeper->tryGet(task_partition.getPartitionCleanStartPath(), clean_start_status) || clean_start_status != "ok") - { - zookeeper->createIfNotExists(task_partition.getPartitionCleanStartPath(), ""); - auto checker = zkutil::EphemeralNodeHolder::create(task_partition.getPartitionCleanStartPath() + "/checker", - *zookeeper, host_id); - // Maybe we are the first worker - ASTPtr query_select_ast = get_select_query(task_shard.table_split_shard, "count()"); - UInt64 count; - { - Context local_context = context; - // Use pull (i.e. readonly) settings, but fetch data from destination servers - local_context.getSettingsRef() = task_cluster->settings_pull; - local_context.getSettingsRef().skip_unavailable_shards = true; - - Block block = getBlockWithAllStreamData(InterpreterFactory::get(query_select_ast, local_context)->execute().in); - count = (block) ? block.safeGetByPosition(0).column->getUInt(0) : 0; - } - - if (count != 0) - { - Coordination::Stat stat_shards{}; - zookeeper->get(task_partition.getPartitionShardsPath(), &stat_shards); - - /// NOTE: partition is still fresh if dirt discovery happens before cleaning - if (stat_shards.numChildren == 0) - { - LOG_WARNING(log, "There are no workers for partition " << task_partition.name - << ", but destination table contains " << count << " rows" - << ". Partition will be dropped and refilled."); - - create_is_dirty_node(clean_state_clock); - return PartitionTaskStatus::Error; - } - } - zookeeper->set(task_partition.getPartitionCleanStartPath(), "ok"); - } - /// At this point, we need to sync that the destination table is clean - /// before any actual work - - /// Try start processing, create node about it - { - String start_state = TaskStateWithOwner::getData(TaskState::Started, host_id); - CleanStateClock new_clean_state_clock (zookeeper, is_dirty_flag_path, is_dirty_cleaned_path); - if (clean_state_clock != new_clean_state_clock) - { - LOG_INFO(log, "Partition " << task_partition.name << " clean state changed, cowardly bailing"); - return PartitionTaskStatus::Error; - } - else if (!new_clean_state_clock.is_clean()) - { - LOG_INFO(log, "Partition " << task_partition.name << " is dirty and will be dropped and refilled"); - create_is_dirty_node(new_clean_state_clock); - return PartitionTaskStatus::Error; - } - zookeeper->create(current_task_status_path, start_state, zkutil::CreateMode::Persistent); - } - - /// Try create table (if not exists) on each shard - { - auto create_query_push_ast = rewriteCreateQueryStorage(task_shard.current_pull_table_create_query, - task_table.table_push, task_table.engine_push_ast); - create_query_push_ast->as().if_not_exists = true; - String query = queryToString(create_query_push_ast); - - LOG_DEBUG(log, "Create destination tables. Query: " << query); - UInt64 shards = executeQueryOnCluster(task_table.cluster_push, query, - create_query_push_ast, &task_cluster->settings_push, - PoolMode::GET_MANY); - LOG_DEBUG(log, "Destination tables " << getQuotedTable(task_table.table_push) << - " have been created on " << shards << " shards of " << task_table.cluster_push->getShardCount()); - } - - /// Do the copying - { - bool inject_fault = false; - if (copy_fault_probability > 0) - { - double value = std::uniform_real_distribution<>(0, 1)(task_table.task_cluster.random_engine); - inject_fault = value < copy_fault_probability; - } - - // Select all fields - ASTPtr query_select_ast = get_select_query(task_shard.table_read_shard, "*", inject_fault ? "1" : ""); - - LOG_DEBUG(log, "Executing SELECT query and pull from " << task_shard.getDescription() - << " : " << queryToString(query_select_ast)); - - ASTPtr query_insert_ast; - { - String query; - query += "INSERT INTO " + getQuotedTable(task_shard.table_split_shard) + " VALUES "; - - ParserQuery p_query(query.data() + query.size()); - query_insert_ast = parseQuery(p_query, query, 0); - - LOG_DEBUG(log, "Executing INSERT query: " << query); - } - - try - { - /// Custom INSERT SELECT implementation - Context context_select = context; - context_select.getSettingsRef() = task_cluster->settings_pull; - - Context context_insert = context; - context_insert.getSettingsRef() = task_cluster->settings_push; - - BlockInputStreamPtr input; - BlockOutputStreamPtr output; - { - BlockIO io_select = InterpreterFactory::get(query_select_ast, context_select)->execute(); - BlockIO io_insert = InterpreterFactory::get(query_insert_ast, context_insert)->execute(); - - input = io_select.in; - output = io_insert.out; - } - - /// Fail-fast optimization to abort copying when the current clean state expires - std::future future_is_dirty_checker; - - Stopwatch watch(CLOCK_MONOTONIC_COARSE); - constexpr UInt64 check_period_milliseconds = 500; - - /// Will asynchronously check that ZooKeeper connection and is_dirty flag appearing while copying data - auto cancel_check = [&] () - { - if (zookeeper->expired()) - throw Exception("ZooKeeper session is expired, cancel INSERT SELECT", ErrorCodes::UNFINISHED); - - if (!future_is_dirty_checker.valid()) - future_is_dirty_checker = zookeeper->asyncExists(is_dirty_flag_path); - - /// check_period_milliseconds should less than average insert time of single block - /// Otherwise, the insertion will slow a little bit - if (watch.elapsedMilliseconds() >= check_period_milliseconds) - { - Coordination::ExistsResponse status = future_is_dirty_checker.get(); - - if (status.error != Coordination::ZNONODE) - { - LogicalClock dirt_discovery_epoch (status.stat.mzxid); - if (dirt_discovery_epoch == clean_state_clock.discovery_zxid) - return false; - throw Exception("Partition is dirty, cancel INSERT SELECT", ErrorCodes::UNFINISHED); - } - } - - return false; - }; - - /// Update statistics - /// It is quite rough: bytes_copied don't take into account DROP PARTITION. - auto update_stats = [&cluster_partition] (const Block & block) - { - cluster_partition.bytes_copied += block.bytes(); - cluster_partition.rows_copied += block.rows(); - cluster_partition.blocks_copied += 1; - }; - - /// Main work is here - copyData(*input, *output, cancel_check, update_stats); - - // Just in case - if (future_is_dirty_checker.valid()) - future_is_dirty_checker.get(); - - if (inject_fault) - throw Exception("Copy fault injection is activated", ErrorCodes::UNFINISHED); - } - catch (...) - { - tryLogCurrentException(log, "An error occurred during copying, partition will be marked as dirty"); - return PartitionTaskStatus::Error; - } - } - - /// Finalize the processing, change state of current partition task (and also check is_dirty flag) - { - String state_finished = TaskStateWithOwner::getData(TaskState::Finished, host_id); - CleanStateClock new_clean_state_clock (zookeeper, is_dirty_flag_path, is_dirty_cleaned_path); - if (clean_state_clock != new_clean_state_clock) - { - LOG_INFO(log, "Partition " << task_partition.name << " clean state changed, cowardly bailing"); - return PartitionTaskStatus::Error; - } - else if (!new_clean_state_clock.is_clean()) - { - LOG_INFO(log, "Partition " << task_partition.name << " became dirty and will be dropped and refilled"); - create_is_dirty_node(new_clean_state_clock); - return PartitionTaskStatus::Error; - } - zookeeper->set(current_task_status_path, state_finished, 0); - } - - LOG_INFO(log, "Partition " << task_partition.name << " copied"); - return PartitionTaskStatus::Finished; - } - - void dropAndCreateLocalTable(const ASTPtr & create_ast) - { - const auto & create = create_ast->as(); - dropLocalTableIfExists({create.database, create.table}); - - InterpreterCreateQuery interpreter(create_ast, context); - interpreter.execute(); - } - - void dropLocalTableIfExists(const DatabaseAndTableName & table_name) const - { - auto drop_ast = std::make_shared(); - drop_ast->if_exists = true; - drop_ast->database = table_name.first; - drop_ast->table = table_name.second; - - InterpreterDropQuery interpreter(drop_ast, context); - interpreter.execute(); - } - - String getRemoteCreateTable(const DatabaseAndTableName & table, Connection & connection, const Settings * settings = nullptr) - { - String query = "SHOW CREATE TABLE " + getQuotedTable(table); - Block block = getBlockWithAllStreamData(std::make_shared( - connection, query, InterpreterShowCreateQuery::getSampleBlock(), context, settings)); - - return typeid_cast(*block.safeGetByPosition(0).column).getDataAt(0).toString(); - } - - ASTPtr getCreateTableForPullShard(const ConnectionTimeouts & timeouts, TaskShard & task_shard) - { - /// Fetch and parse (possibly) new definition - auto connection_entry = task_shard.info.pool->get(timeouts, &task_cluster->settings_pull); - String create_query_pull_str = getRemoteCreateTable( - task_shard.task_table.table_pull, - *connection_entry, - &task_cluster->settings_pull); - - ParserCreateQuery parser_create_query; - return parseQuery(parser_create_query, create_query_pull_str, 0); - } - - void createShardInternalTables(const ConnectionTimeouts & timeouts, TaskShard & task_shard, bool create_split = true) - { - TaskTable & task_table = task_shard.task_table; - - /// We need to update table definitions for each part, it could be changed after ALTER - task_shard.current_pull_table_create_query = getCreateTableForPullShard(timeouts, task_shard); - - /// Create local Distributed tables: - /// a table fetching data from current shard and a table inserting data to the whole destination cluster - String read_shard_prefix = ".read_shard_" + toString(task_shard.indexInCluster()) + "."; - String split_shard_prefix = ".split."; - task_shard.table_read_shard = DatabaseAndTableName(working_database_name, read_shard_prefix + task_table.table_id); - task_shard.table_split_shard = DatabaseAndTableName(working_database_name, split_shard_prefix + task_table.table_id); - - /// Create special cluster with single shard - String shard_read_cluster_name = read_shard_prefix + task_table.cluster_pull_name; - ClusterPtr cluster_pull_current_shard = task_table.cluster_pull->getClusterWithSingleShard(task_shard.indexInCluster()); - context.setCluster(shard_read_cluster_name, cluster_pull_current_shard); - - auto storage_shard_ast = createASTStorageDistributed(shard_read_cluster_name, task_table.table_pull.first, task_table.table_pull.second); - const auto & storage_split_ast = task_table.engine_split_ast; - - auto create_query_ast = removeAliasColumnsFromCreateQuery(task_shard.current_pull_table_create_query); - auto create_table_pull_ast = rewriteCreateQueryStorage(create_query_ast, task_shard.table_read_shard, storage_shard_ast); - auto create_table_split_ast = rewriteCreateQueryStorage(create_query_ast, task_shard.table_split_shard, storage_split_ast); - - dropAndCreateLocalTable(create_table_pull_ast); - - if (create_split) - dropAndCreateLocalTable(create_table_split_ast); - } - - - std::set getShardPartitions(const ConnectionTimeouts & timeouts, TaskShard & task_shard) - { - createShardInternalTables(timeouts, task_shard, false); - - TaskTable & task_table = task_shard.task_table; - - String query; - { - WriteBufferFromOwnString wb; - wb << "SELECT DISTINCT " << queryToString(task_table.engine_push_partition_key_ast) << " AS partition FROM" - << " " << getQuotedTable(task_shard.table_read_shard) << " ORDER BY partition DESC"; - query = wb.str(); - } - - ParserQuery parser_query(query.data() + query.size()); - ASTPtr query_ast = parseQuery(parser_query, query, 0); - - LOG_DEBUG(log, "Computing destination partition set, executing query: " << query); - - Context local_context = context; - local_context.setSettings(task_cluster->settings_pull); - Block block = getBlockWithAllStreamData(InterpreterFactory::get(query_ast, local_context)->execute().in); - - std::set res; - if (block) - { - ColumnWithTypeAndName & column = block.getByPosition(0); - task_shard.partition_key_column = column; - - for (size_t i = 0; i < column.column->size(); ++i) - { - WriteBufferFromOwnString wb; - column.type->serializeAsTextQuoted(*column.column, i, wb, FormatSettings()); - res.emplace(wb.str()); - } - } - - LOG_DEBUG(log, "There are " << res.size() << " destination partitions in shard " << task_shard.getDescription()); - - return res; - } - - bool checkShardHasPartition(const ConnectionTimeouts & timeouts, TaskShard & task_shard, const String & partition_quoted_name) - { - createShardInternalTables(timeouts, task_shard, false); - - TaskTable & task_table = task_shard.task_table; - - std::string query = "SELECT 1 FROM " + getQuotedTable(task_shard.table_read_shard) - + " WHERE (" + queryToString(task_table.engine_push_partition_key_ast) + " = (" + partition_quoted_name + " AS partition_key))"; - - if (!task_table.where_condition_str.empty()) - query += " AND (" + task_table.where_condition_str + ")"; - - query += " LIMIT 1"; - - LOG_DEBUG(log, "Checking shard " << task_shard.getDescription() << " for partition " - << partition_quoted_name << " existence, executing query: " << query); - - ParserQuery parser_query(query.data() + query.size()); - ASTPtr query_ast = parseQuery(parser_query, query, 0); - - Context local_context = context; - local_context.setSettings(task_cluster->settings_pull); - return InterpreterFactory::get(query_ast, local_context)->execute().in->read().rows() != 0; - } - - /** Executes simple query (without output streams, for example DDL queries) on each shard of the cluster - * Returns number of shards for which at least one replica executed query successfully - */ - UInt64 executeQueryOnCluster( - const ClusterPtr & cluster, - const String & query, - const ASTPtr & query_ast_ = nullptr, - const Settings * settings = nullptr, - PoolMode pool_mode = PoolMode::GET_ALL, - UInt64 max_successful_executions_per_shard = 0) const - { - auto num_shards = cluster->getShardsInfo().size(); - std::vector per_shard_num_successful_replicas(num_shards, 0); - - ASTPtr query_ast; - if (query_ast_ == nullptr) - { - ParserQuery p_query(query.data() + query.size()); - query_ast = parseQuery(p_query, query, 0); - } - else - query_ast = query_ast_; - - - /// We need to execute query on one replica at least - auto do_for_shard = [&] (UInt64 shard_index) - { - const Cluster::ShardInfo & shard = cluster->getShardsInfo().at(shard_index); - UInt64 & num_successful_executions = per_shard_num_successful_replicas.at(shard_index); - num_successful_executions = 0; - - auto increment_and_check_exit = [&] () -> bool - { - ++num_successful_executions; - return max_successful_executions_per_shard && num_successful_executions >= max_successful_executions_per_shard; - }; - - UInt64 num_replicas = cluster->getShardsAddresses().at(shard_index).size(); - UInt64 num_local_replicas = shard.getLocalNodeCount(); - UInt64 num_remote_replicas = num_replicas - num_local_replicas; - - /// In that case we don't have local replicas, but do it just in case - for (UInt64 i = 0; i < num_local_replicas; ++i) - { - auto interpreter = InterpreterFactory::get(query_ast, context); - interpreter->execute(); - - if (increment_and_check_exit()) - return; - } - - /// Will try to make as many as possible queries - if (shard.hasRemoteConnections()) - { - Settings current_settings = settings ? *settings : task_cluster->settings_common; - current_settings.max_parallel_replicas = num_remote_replicas ? num_remote_replicas : 1; - - auto timeouts = ConnectionTimeouts::getTCPTimeoutsWithFailover(current_settings).getSaturated(current_settings.max_execution_time); - auto connections = shard.pool->getMany(timeouts, ¤t_settings, pool_mode); - - for (auto & connection : connections) - { - if (connection.isNull()) - continue; - - try - { - /// CREATE TABLE and DROP PARTITION queries return empty block - RemoteBlockInputStream stream{*connection, query, Block{}, context, ¤t_settings}; - NullBlockOutputStream output{Block{}}; - copyData(stream, output); - - if (increment_and_check_exit()) - return; - } - catch (const Exception &) - { - LOG_INFO(log, getCurrentExceptionMessage(false, true)); - } - } - } - }; - - { - ThreadPool thread_pool(std::min(num_shards, getNumberOfPhysicalCPUCores())); - - for (UInt64 shard_index = 0; shard_index < num_shards; ++shard_index) - thread_pool.scheduleOrThrowOnError([=] { do_for_shard(shard_index); }); - - thread_pool.wait(); - } - - UInt64 successful_shards = 0; - for (UInt64 num_replicas : per_shard_num_successful_replicas) - successful_shards += (num_replicas > 0); - - return successful_shards; - } - -private: - String task_zookeeper_path; - String task_description_path; - String host_id; - String working_database_name; - - /// Auto update config stuff - UInt64 task_descprtion_current_version = 1; - std::atomic task_descprtion_version{1}; - Coordination::WatchCallback task_description_watch_callback; - /// ZooKeeper session used to set the callback - zkutil::ZooKeeperPtr task_description_watch_zookeeper; - - ConfigurationPtr task_cluster_initial_config; - ConfigurationPtr task_cluster_current_config; - Coordination::Stat task_description_current_stat{}; - - std::unique_ptr task_cluster; - - bool is_safe_mode = false; - double copy_fault_probability = 0.0; - - Context & context; - Poco::Logger * log; - - std::chrono::milliseconds default_sleep_time{1000}; -}; - - -/// ClusterCopierApp - - -void ClusterCopierApp::initialize(Poco::Util::Application & self) -{ - is_help = config().has("help"); - if (is_help) - return; - - config_xml_path = config().getString("config-file"); - task_path = config().getString("task-path"); - log_level = config().getString("log-level", "trace"); - is_safe_mode = config().has("safe-mode"); - if (config().has("copy-fault-probability")) - copy_fault_probability = std::max(std::min(config().getDouble("copy-fault-probability"), 1.0), 0.0); - base_dir = (config().has("base-dir")) ? config().getString("base-dir") : Poco::Path::current(); - // process_id is '#_' - time_t timestamp = Poco::Timestamp().epochTime(); - auto curr_pid = Poco::Process::id(); - - process_id = std::to_string(DateLUT::instance().toNumYYYYMMDDhhmmss(timestamp)) + "_" + std::to_string(curr_pid); - host_id = escapeForFileName(getFQDNOrHostName()) + '#' + process_id; - process_path = Poco::Path(base_dir + "/clickhouse-copier_" + process_id).absolute().toString(); - Poco::File(process_path).createDirectories(); - - /// Override variables for BaseDaemon - if (config().has("log-level")) - config().setString("logger.level", config().getString("log-level")); - - if (config().has("base-dir") || !config().has("logger.log")) - config().setString("logger.log", process_path + "/log.log"); - - if (config().has("base-dir") || !config().has("logger.errorlog")) - config().setString("logger.errorlog", process_path + "/log.err.log"); - - Base::initialize(self); -} - - -void ClusterCopierApp::handleHelp(const std::string &, const std::string &) -{ - Poco::Util::HelpFormatter helpFormatter(options()); - helpFormatter.setCommand(commandName()); - helpFormatter.setHeader("Copies tables from one cluster to another"); - helpFormatter.setUsage("--config-file --task-path "); - helpFormatter.format(std::cerr); - - stopOptionsProcessing(); -} - - -void ClusterCopierApp::defineOptions(Poco::Util::OptionSet & options) -{ - Base::defineOptions(options); - - options.addOption(Poco::Util::Option("task-path", "", "path to task in ZooKeeper") - .argument("task-path").binding("task-path")); - options.addOption(Poco::Util::Option("task-file", "", "path to task file for uploading in ZooKeeper to task-path") - .argument("task-file").binding("task-file")); - options.addOption(Poco::Util::Option("task-upload-force", "", "Force upload task-file even node already exists") - .argument("task-upload-force").binding("task-upload-force")); - options.addOption(Poco::Util::Option("safe-mode", "", "disables ALTER DROP PARTITION in case of errors") - .binding("safe-mode")); - options.addOption(Poco::Util::Option("copy-fault-probability", "", "the copying fails with specified probability (used to test partition state recovering)") - .argument("copy-fault-probability").binding("copy-fault-probability")); - options.addOption(Poco::Util::Option("log-level", "", "sets log level") - .argument("log-level").binding("log-level")); - options.addOption(Poco::Util::Option("base-dir", "", "base directory for copiers, consecutive copier launches will populate /base-dir/launch_id/* directories") - .argument("base-dir").binding("base-dir")); - - using Me = std::decay_t; - options.addOption(Poco::Util::Option("help", "", "produce this help message").binding("help") - .callback(Poco::Util::OptionCallback(this, &Me::handleHelp))); -} - - -void ClusterCopierApp::mainImpl() -{ - StatusFile status_file(process_path + "/status"); - ThreadStatus thread_status; - - auto log = &logger(); - LOG_INFO(log, "Starting clickhouse-copier (" - << "id " << process_id << ", " - << "host_id " << host_id << ", " - << "path " << process_path << ", " - << "revision " << ClickHouseRevision::get() << ")"); - - auto context = std::make_unique(Context::createGlobal()); - context->makeGlobalContext(); - SCOPE_EXIT(context->shutdown()); - - context->setConfig(loaded_config.configuration); - context->setApplicationType(Context::ApplicationType::LOCAL); - context->setPath(process_path); - - registerFunctions(); - registerAggregateFunctions(); - registerTableFunctions(); - registerStorages(); - registerDictionaries(); - registerDisks(); - - static const std::string default_database = "_local"; - context->addDatabase(default_database, std::make_shared(default_database)); - context->setCurrentDatabase(default_database); - - /// Initialize query scope just in case. - CurrentThread::QueryScope query_scope(*context); - - auto copier = std::make_unique(task_path, host_id, default_database, *context); - copier->setSafeMode(is_safe_mode); - copier->setCopyFaultProbability(copy_fault_probability); - - auto task_file = config().getString("task-file", ""); - if (!task_file.empty()) - copier->uploadTaskDescription(task_path, task_file, config().getBool("task-upload-force", false)); - - copier->init(); - copier->process(ConnectionTimeouts::getTCPTimeoutsWithoutFailover(context->getSettingsRef())); - - /// Reset ZooKeeper before removing ClusterCopier. - /// Otherwise zookeeper watch can call callback which use already removed ClusterCopier object. - context->resetZooKeeper(); -} - - -int ClusterCopierApp::main(const std::vector &) -{ - if (is_help) - return 0; - - try - { - mainImpl(); - } - catch (...) - { - tryLogCurrentException(&Poco::Logger::root(), __PRETTY_FUNCTION__); - auto code = getCurrentExceptionCode(); - - return (code) ? code : -1; - } - - return 0; -} - - -} - -#pragma GCC diagnostic ignored "-Wunused-function" -#pragma GCC diagnostic ignored "-Wmissing-declarations" - -int mainEntryClickHouseClusterCopier(int argc, char ** argv) -{ - try - { - DB::ClusterCopierApp app; - return app.run(argc, argv); - } - catch (...) - { - std::cerr << DB::getCurrentExceptionMessage(true) << "\n"; - auto code = DB::getCurrentExceptionCode(); - - return (code) ? code : -1; - } -} diff --git a/dbms/programs/copier/ClusterCopier.h b/dbms/programs/copier/ClusterCopier.h index fe228fd6194..f4f37b97c26 100644 --- a/dbms/programs/copier/ClusterCopier.h +++ b/dbms/programs/copier/ClusterCopier.h @@ -1,87 +1,1525 @@ #pragma once -#include -#include -/* clickhouse cluster copier util - * Copies tables data from one cluster to new tables of other (possibly the same) cluster in distributed fault-tolerant manner. - * - * See overview in the docs: docs/en/utils/clickhouse-copier.md - * - * Implementation details: - * - * cluster-copier workers pull each partition of each shard of the source cluster and push it to the destination cluster through - * Distributed table (to preform data resharding). So, worker job is a partition of a source shard. - * A job has three states: Active, Finished and Abandoned. Abandoned means that worker died and did not finish the job. - * - * If an error occurred during the copying (a worker failed or a worker did not finish the INSERT), then the whole partition (on - * all destination servers) should be dropped and refilled. So, copying entity is a partition of all destination shards. - * If a failure is detected a special /is_dirty node is created in ZooKeeper signalling that other workers copying the same partition - * should stop, after a refilling procedure should start. - * - * ZooKeeper task node has the following structure: - * /task/path_root - path passed in --task-path parameter - * /description - contains user-defined XML config of the task - * /task_active_workers - contains ephemeral nodes of all currently active workers, used to implement max_workers limitation - * /server_fqdn#PID_timestamp - cluster-copier worker ID - * ... - * /tables - directory with table tasks - * /cluster.db.table1 - directory of table_hits task - * /partition1 - directory for partition1 - * /shards - directory for source cluster shards - * /1 - worker job for the first shard of partition1 of table test.hits - * Contains info about current status (Active or Finished) and worker ID. - * /2 - * ... - * /partition_active_workers - * /1 - for each job in /shards a corresponding ephemeral node created in /partition_active_workers - * It is used to detect Abandoned jobs (if there is Active node in /shards and there is no node in - * /partition_active_workers). - * Also, it is used to track active workers in the partition (when we need to refill the partition we do - * not DROP PARTITION while there are active workers) - * /2 - * ... - * /is_dirty - the node is set if some worker detected that an error occurred (the INSERT is failed or an Abandoned node is - * detected). If the node appeared workers in this partition should stop and start cleaning and refilling - * partition procedure. - * During this procedure a single 'cleaner' worker is selected. The worker waits for stopping all partition - * workers, removes /shards node, executes DROP PARTITION on each destination node and removes /is_dirty node. - * /cleaner- An ephemeral node used to select 'cleaner' worker. Contains ID of the worker. - * /cluster.db.table2 - * ... - */ +#include "Internals.h" +#include "ZookeeperStaff.h" namespace DB { -class ClusterCopierApp : public BaseDaemon +using ConfigurationPtr = Poco::AutoPtr; + +static ConfigurationPtr getConfigurationFromXMLString(const std::string & xml_data) +{ + std::stringstream ss(xml_data); + Poco::XML::InputSource input_source{ss}; + return {new Poco::Util::XMLConfiguration{&input_source}}; +} + +class ClusterCopier { public: - void initialize(Poco::Util::Application & self) override; + ClusterCopier(String task_path_, + String host_id_, + String proxy_database_name_, + Context & context_) + : + task_zookeeper_path(std::move(task_path_)), + host_id(std::move(host_id_)), + working_database_name(std::move(proxy_database_name_)), + context(context_), + log(&Poco::Logger::get("ClusterCopier")) + { + } - void handleHelp(const std::string &, const std::string &); + void init() + { + auto zookeeper = context.getZooKeeper(); - void defineOptions(Poco::Util::OptionSet & options) override; + task_description_watch_callback = [this] (const Coordination::WatchResponse & response) + { + if (response.error != Coordination::ZOK) + return; + UInt64 version = ++task_descprtion_version; + LOG_DEBUG(log, "Task description should be updated, local version " << version); + }; - int main(const std::vector &) override; + task_description_path = task_zookeeper_path + "/description"; + task_cluster = std::make_unique(task_zookeeper_path, working_database_name); + + reloadTaskDescription(); + task_cluster_initial_config = task_cluster_current_config; + + task_cluster->loadTasks(*task_cluster_initial_config); + context.setClustersConfig(task_cluster_initial_config, task_cluster->clusters_prefix); + + /// Set up shards and their priority + task_cluster->random_engine.seed(task_cluster->random_device()); + for (auto & task_table : task_cluster->table_tasks) + { + task_table.cluster_pull = context.getCluster(task_table.cluster_pull_name); + task_table.cluster_push = context.getCluster(task_table.cluster_push_name); + task_table.initShards(task_cluster->random_engine); + } + + LOG_DEBUG(log, "Will process " << task_cluster->table_tasks.size() << " table tasks"); + + /// Do not initialize tables, will make deferred initialization in process() + + zookeeper->createAncestors(getWorkersPathVersion() + "/"); + zookeeper->createAncestors(getWorkersPath() + "/"); + } + + template + decltype(auto) retry(T && func, UInt64 max_tries = 100) + { + std::exception_ptr exception; + + for (UInt64 try_number = 1; try_number <= max_tries; ++try_number) + { + try + { + return func(); + } + catch (...) + { + exception = std::current_exception(); + if (try_number < max_tries) + { + tryLogCurrentException(log, "Will retry"); + std::this_thread::sleep_for(default_sleep_time); + } + } + } + + std::rethrow_exception(exception); + } + + + void discoverShardPartitions(const ConnectionTimeouts & timeouts, const TaskShardPtr & task_shard) + { + TaskTable & task_table = task_shard->task_table; + + LOG_INFO(log, "Discover partitions of shard " << task_shard->getDescription()); + + auto get_partitions = [&] () { return getShardPartitions(timeouts, *task_shard); }; + auto existing_partitions_names = retry(get_partitions, 60); + Strings filtered_partitions_names; + Strings missing_partitions; + + /// Check that user specified correct partition names + auto check_partition_format = [] (const DataTypePtr & type, const String & partition_text_quoted) + { + MutableColumnPtr column_dummy = type->createColumn(); + ReadBufferFromString rb(partition_text_quoted); + + try + { + type->deserializeAsTextQuoted(*column_dummy, rb, FormatSettings()); + } + catch (Exception & e) + { + throw Exception("Partition " + partition_text_quoted + " has incorrect format. " + e.displayText(), ErrorCodes::BAD_ARGUMENTS); + } + }; + + if (task_table.has_enabled_partitions) + { + /// Process partition in order specified by + for (const String & partition_name : task_table.enabled_partitions) + { + /// Check that user specified correct partition names + check_partition_format(task_shard->partition_key_column.type, partition_name); + + auto it = existing_partitions_names.find(partition_name); + + /// Do not process partition if it is not in enabled_partitions list + if (it == existing_partitions_names.end()) + { + missing_partitions.emplace_back(partition_name); + continue; + } + + filtered_partitions_names.emplace_back(*it); + } + + for (const String & partition_name : existing_partitions_names) + { + if (!task_table.enabled_partitions_set.count(partition_name)) + { + LOG_DEBUG(log, "Partition " << partition_name << " will not be processed, since it is not in " + << "enabled_partitions of " << task_table.table_id); + } + } + } + else + { + for (const String & partition_name : existing_partitions_names) + filtered_partitions_names.emplace_back(partition_name); + } + + for (const String & partition_name : filtered_partitions_names) + { + task_shard->partition_tasks.emplace(partition_name, ShardPartition(*task_shard, partition_name, 10)); + task_shard->checked_partitions.emplace(partition_name, true); + } + + if (!missing_partitions.empty()) + { + std::stringstream ss; + for (const String & missing_partition : missing_partitions) + ss << " " << missing_partition; + + LOG_WARNING(log, "There are no " << missing_partitions.size() << " partitions from enabled_partitions in shard " + << task_shard->getDescription() << " :" << ss.str()); + } + + LOG_DEBUG(log, "Will copy " << task_shard->partition_tasks.size() << " partitions from shard " << task_shard->getDescription()); + } + + /// Compute set of partitions, assume set of partitions aren't changed during the processing + void discoverTablePartitions(const ConnectionTimeouts & timeouts, TaskTable & task_table, UInt64 num_threads = 0) + { + /// Fetch partitions list from a shard + { + ThreadPool thread_pool(num_threads ? num_threads : 2 * getNumberOfPhysicalCPUCores()); + + for (const TaskShardPtr & task_shard : task_table.all_shards) + thread_pool.scheduleOrThrowOnError([this, timeouts, task_shard]() { discoverShardPartitions(timeouts, task_shard); }); + + LOG_DEBUG(log, "Waiting for " << thread_pool.active() << " setup jobs"); + thread_pool.wait(); + } + } + + void uploadTaskDescription(const std::string & task_path, const std::string & task_file, const bool force) + { + auto local_task_description_path = task_path + "/description"; + + String task_config_str; + { + ReadBufferFromFile in(task_file); + readStringUntilEOF(task_config_str, in); + } + if (task_config_str.empty()) + return; + + auto zookeeper = context.getZooKeeper(); + + zookeeper->createAncestors(local_task_description_path); + auto code = zookeeper->tryCreate(local_task_description_path, task_config_str, zkutil::CreateMode::Persistent); + if (code && force) + zookeeper->createOrUpdate(local_task_description_path, task_config_str, zkutil::CreateMode::Persistent); + + LOG_DEBUG(log, "Task description " << ((code && !force) ? "not " : "") << "uploaded to " << local_task_description_path << " with result " << code << " ("<< zookeeper->error2string(code) << ")"); + } + + void reloadTaskDescription() + { + auto zookeeper = context.getZooKeeper(); + task_description_watch_zookeeper = zookeeper; + + String task_config_str; + Coordination::Stat stat{}; + int code; + + zookeeper->tryGetWatch(task_description_path, task_config_str, &stat, task_description_watch_callback, &code); + if (code) + throw Exception("Can't get description node " + task_description_path, ErrorCodes::BAD_ARGUMENTS); + + LOG_DEBUG(log, "Loading description, zxid=" << task_description_current_stat.czxid); + auto config = getConfigurationFromXMLString(task_config_str); + + /// Setup settings + task_cluster->reloadSettings(*config); + context.getSettingsRef() = task_cluster->settings_common; + + task_cluster_current_config = config; + task_description_current_stat = stat; + } + + void updateConfigIfNeeded() + { + UInt64 version_to_update = task_descprtion_version; + bool is_outdated_version = task_descprtion_current_version != version_to_update; + bool is_expired_session = !task_description_watch_zookeeper || task_description_watch_zookeeper->expired(); + + if (!is_outdated_version && !is_expired_session) + return; + + LOG_DEBUG(log, "Updating task description"); + reloadTaskDescription(); + + task_descprtion_current_version = version_to_update; + } + + void process(const ConnectionTimeouts & timeouts) + { + for (TaskTable & task_table : task_cluster->table_tasks) + { + LOG_INFO(log, "Process table task " << task_table.table_id << " with " + << task_table.all_shards.size() << " shards, " << task_table.local_shards.size() << " of them are local ones"); + + if (task_table.all_shards.empty()) + continue; + + /// Discover partitions of each shard and total set of partitions + if (!task_table.has_enabled_partitions) + { + /// If there are no specified enabled_partitions, we must discover them manually + discoverTablePartitions(timeouts, task_table); + + /// After partitions of each shard are initialized, initialize cluster partitions + for (const TaskShardPtr & task_shard : task_table.all_shards) + { + for (const auto & partition_elem : task_shard->partition_tasks) + { + const String & partition_name = partition_elem.first; + task_table.cluster_partitions.emplace(partition_name, ClusterPartition{}); + } + } + + for (auto & partition_elem : task_table.cluster_partitions) + { + const String & partition_name = partition_elem.first; + + for (const TaskShardPtr & task_shard : task_table.all_shards) + task_shard->checked_partitions.emplace(partition_name); + + task_table.ordered_partition_names.emplace_back(partition_name); + } + } + else + { + /// If enabled_partitions are specified, assume that each shard has all partitions + /// We will refine partition set of each shard in future + + for (const String & partition_name : task_table.enabled_partitions) + { + task_table.cluster_partitions.emplace(partition_name, ClusterPartition{}); + task_table.ordered_partition_names.emplace_back(partition_name); + } + } + + task_table.watch.restart(); + + /// Retry table processing + bool table_is_done = false; + for (UInt64 num_table_tries = 0; num_table_tries < max_table_tries; ++num_table_tries) + { + if (tryProcessTable(timeouts, task_table)) + { + table_is_done = true; + break; + } + } + + if (!table_is_done) + { + throw Exception("Too many tries to process table " + task_table.table_id + ". Abort remaining execution", + ErrorCodes::UNFINISHED); + } + } + } + + /// Disables DROP PARTITION commands that used to clear data after errors + void setSafeMode(bool is_safe_mode_ = true) + { + is_safe_mode = is_safe_mode_; + } + + void setCopyFaultProbability(double copy_fault_probability_) + { + copy_fault_probability = copy_fault_probability_; + } + + + protected: + + String getWorkersPath() const + { + return task_cluster->task_zookeeper_path + "/task_active_workers"; + } + + String getWorkersPathVersion() const + { + return getWorkersPath() + "_version"; + } + + String getCurrentWorkerNodePath() const + { + return getWorkersPath() + "/" + host_id; + } + + zkutil::EphemeralNodeHolder::Ptr createTaskWorkerNodeAndWaitIfNeed( + const zkutil::ZooKeeperPtr & zookeeper, + const String & description, + bool unprioritized) + { + std::chrono::milliseconds current_sleep_time = default_sleep_time; + static constexpr std::chrono::milliseconds max_sleep_time(30000); // 30 sec + + if (unprioritized) + std::this_thread::sleep_for(current_sleep_time); + + String workers_version_path = getWorkersPathVersion(); + String workers_path = getWorkersPath(); + String current_worker_path = getCurrentWorkerNodePath(); + + UInt64 num_bad_version_errors = 0; + + while (true) + { + updateConfigIfNeeded(); + + Coordination::Stat stat{}; + zookeeper->get(workers_version_path, &stat); + auto version = stat.version; + zookeeper->get(workers_path, &stat); + + if (static_cast(stat.numChildren) >= task_cluster->max_workers) + { + LOG_DEBUG(log, "Too many workers (" << stat.numChildren << ", maximum " << task_cluster->max_workers << ")" + << ". Postpone processing " << description); + + if (unprioritized) + current_sleep_time = std::min(max_sleep_time, current_sleep_time + default_sleep_time); + + std::this_thread::sleep_for(current_sleep_time); + num_bad_version_errors = 0; + } + else + { + Coordination::Requests ops; + ops.emplace_back(zkutil::makeSetRequest(workers_version_path, description, version)); + ops.emplace_back(zkutil::makeCreateRequest(current_worker_path, description, zkutil::CreateMode::Ephemeral)); + Coordination::Responses responses; + auto code = zookeeper->tryMulti(ops, responses); + + if (code == Coordination::ZOK || code == Coordination::ZNODEEXISTS) + return std::make_shared(current_worker_path, *zookeeper, false, false, description); + + if (code == Coordination::ZBADVERSION) + { + ++num_bad_version_errors; + + /// Try to make fast retries + if (num_bad_version_errors > 3) + { + LOG_DEBUG(log, "A concurrent worker has just been added, will check free worker slots again"); + std::chrono::milliseconds random_sleep_time(std::uniform_int_distribution(1, 1000)(task_cluster->random_engine)); + std::this_thread::sleep_for(random_sleep_time); + num_bad_version_errors = 0; + } + } + else + throw Coordination::Exception(code); + } + } + } + + /** Checks that the whole partition of a table was copied. We should do it carefully due to dirty lock. + * State of some task could change during the processing. + * We have to ensure that all shards have the finished state and there is no dirty flag. + * Moreover, we have to check status twice and check zxid, because state can change during the checking. + */ + bool checkPartitionIsDone(const TaskTable & task_table, const String & partition_name, const TasksShard & shards_with_partition) + { + LOG_DEBUG(log, "Check that all shards processed partition " << partition_name << " successfully"); + + auto zookeeper = context.getZooKeeper(); + + Strings status_paths; + for (auto & shard : shards_with_partition) + { + ShardPartition & task_shard_partition = shard->partition_tasks.find(partition_name)->second; + status_paths.emplace_back(task_shard_partition.getShardStatusPath()); + } + + std::vector zxid1, zxid2; + + try + { + std::vector get_futures; + for (const String & path : status_paths) + get_futures.emplace_back(zookeeper->asyncGet(path)); + + // Check that state is Finished and remember zxid + for (auto & future : get_futures) + { + auto res = future.get(); + + TaskStateWithOwner status = TaskStateWithOwner::fromString(res.data); + if (status.state != TaskState::Finished) + { + LOG_INFO(log, "The task " << res.data << " is being rewritten by " << status.owner << ". Partition will be rechecked"); + return false; + } + + zxid1.push_back(res.stat.pzxid); + } + + // Check that partition is not dirty + { + CleanStateClock clean_state_clock ( + zookeeper, + task_table.getPartitionIsDirtyPath(partition_name), + task_table.getPartitionIsCleanedPath(partition_name) + ); + Coordination::Stat stat{}; + LogicalClock task_start_clock; + if (zookeeper->exists(task_table.getPartitionTaskStatusPath(partition_name), &stat)) + task_start_clock = LogicalClock(stat.mzxid); + zookeeper->get(task_table.getPartitionTaskStatusPath(partition_name), &stat); + if (!clean_state_clock.is_clean() || task_start_clock <= clean_state_clock.discovery_zxid) + { + LOG_INFO(log, "Partition " << partition_name << " become dirty"); + return false; + } + } + + get_futures.clear(); + for (const String & path : status_paths) + get_futures.emplace_back(zookeeper->asyncGet(path)); + + // Remember zxid of states again + for (auto & future : get_futures) + { + auto res = future.get(); + zxid2.push_back(res.stat.pzxid); + } + } + catch (const Coordination::Exception & e) + { + LOG_INFO(log, "A ZooKeeper error occurred while checking partition " << partition_name + << ". Will recheck the partition. Error: " << e.displayText()); + return false; + } + + // If all task is finished and zxid is not changed then partition could not become dirty again + for (UInt64 shard_num = 0; shard_num < status_paths.size(); ++shard_num) + { + if (zxid1[shard_num] != zxid2[shard_num]) + { + LOG_INFO(log, "The task " << status_paths[shard_num] << " is being modified now. Partition will be rechecked"); + return false; + } + } + + LOG_INFO(log, "Partition " << partition_name << " is copied successfully"); + return true; + } + + /// Removes MATERIALIZED and ALIAS columns from create table query + static ASTPtr removeAliasColumnsFromCreateQuery(const ASTPtr & query_ast) + { + const ASTs & column_asts = query_ast->as().columns_list->columns->children; + auto new_columns = std::make_shared(); + + for (const ASTPtr & column_ast : column_asts) + { + const auto & column = column_ast->as(); + + if (!column.default_specifier.empty()) + { + ColumnDefaultKind kind = columnDefaultKindFromString(column.default_specifier); + if (kind == ColumnDefaultKind::Materialized || kind == ColumnDefaultKind::Alias) + continue; + } + + new_columns->children.emplace_back(column_ast->clone()); + } + + ASTPtr new_query_ast = query_ast->clone(); + auto & new_query = new_query_ast->as(); + + auto new_columns_list = std::make_shared(); + new_columns_list->set(new_columns_list->columns, new_columns); + if (auto indices = query_ast->as()->columns_list->indices) + new_columns_list->set(new_columns_list->indices, indices->clone()); + + new_query.replace(new_query.columns_list, new_columns_list); + + return new_query_ast; + } + + /// Replaces ENGINE and table name in a create query + static std::shared_ptr rewriteCreateQueryStorage(const ASTPtr & create_query_ast, const DatabaseAndTableName & new_table, const ASTPtr & new_storage_ast) + { + const auto & create = create_query_ast->as(); + auto res = std::make_shared(create); + + if (create.storage == nullptr || new_storage_ast == nullptr) + throw Exception("Storage is not specified", ErrorCodes::LOGICAL_ERROR); + + res->database = new_table.first; + res->table = new_table.second; + + res->children.clear(); + res->set(res->columns_list, create.columns_list->clone()); + res->set(res->storage, new_storage_ast->clone()); + + return res; + } + + bool tryDropPartition(ShardPartition & task_partition, const zkutil::ZooKeeperPtr & zookeeper, const CleanStateClock & clean_state_clock) + { + if (is_safe_mode) + throw Exception("DROP PARTITION is prohibited in safe mode", ErrorCodes::NOT_IMPLEMENTED); + + TaskTable & task_table = task_partition.task_shard.task_table; + + const String current_shards_path = task_partition.getPartitionShardsPath(); + const String current_partition_active_workers_dir = task_partition.getPartitionActiveWorkersPath(); + const String is_dirty_flag_path = task_partition.getCommonPartitionIsDirtyPath(); + const String dirty_cleaner_path = is_dirty_flag_path + "/cleaner"; + const String is_dirty_cleaned_path = task_partition.getCommonPartitionIsCleanedPath(); + + zkutil::EphemeralNodeHolder::Ptr cleaner_holder; + try + { + cleaner_holder = zkutil::EphemeralNodeHolder::create(dirty_cleaner_path, *zookeeper, host_id); + } + catch (const Coordination::Exception & e) + { + if (e.code == Coordination::ZNODEEXISTS) + { + LOG_DEBUG(log, "Partition " << task_partition.name << " is cleaning now by somebody, sleep"); + std::this_thread::sleep_for(default_sleep_time); + return false; + } + + throw; + } + + Coordination::Stat stat{}; + if (zookeeper->exists(current_partition_active_workers_dir, &stat)) + { + if (stat.numChildren != 0) + { + LOG_DEBUG(log, "Partition " << task_partition.name << " contains " << stat.numChildren << " active workers while trying to drop it. Going to sleep."); + std::this_thread::sleep_for(default_sleep_time); + return false; + } + else + { + zookeeper->remove(current_partition_active_workers_dir); + } + } + + { + zkutil::EphemeralNodeHolder::Ptr active_workers_lock; + try + { + active_workers_lock = zkutil::EphemeralNodeHolder::create(current_partition_active_workers_dir, *zookeeper, host_id); + } + catch (const Coordination::Exception & e) + { + if (e.code == Coordination::ZNODEEXISTS) + { + LOG_DEBUG(log, "Partition " << task_partition.name << " is being filled now by somebody, sleep"); + return false; + } + + throw; + } + + // Lock the dirty flag + zookeeper->set(is_dirty_flag_path, host_id, clean_state_clock.discovery_version.value()); + zookeeper->tryRemove(task_partition.getPartitionCleanStartPath()); + CleanStateClock my_clock(zookeeper, is_dirty_flag_path, is_dirty_cleaned_path); + + /// Remove all status nodes + { + Strings children; + if (zookeeper->tryGetChildren(current_shards_path, children) == Coordination::ZOK) + for (const auto & child : children) + { + zookeeper->removeRecursive(current_shards_path + "/" + child); + } + } + + String query = "ALTER TABLE " + getQuotedTable(task_table.table_push); + query += " DROP PARTITION " + task_partition.name + ""; + + /// TODO: use this statement after servers will be updated up to 1.1.54310 + // query += " DROP PARTITION ID '" + task_partition.name + "'"; + + ClusterPtr & cluster_push = task_table.cluster_push; + Settings settings_push = task_cluster->settings_push; + + /// It is important, DROP PARTITION must be done synchronously + settings_push.replication_alter_partitions_sync = 2; + + LOG_DEBUG(log, "Execute distributed DROP PARTITION: " << query); + /// Limit number of max executing replicas to 1 + UInt64 num_shards = executeQueryOnCluster(cluster_push, query, nullptr, &settings_push, PoolMode::GET_ONE, 1); + + if (num_shards < cluster_push->getShardCount()) + { + LOG_INFO(log, "DROP PARTITION wasn't successfully executed on " << cluster_push->getShardCount() - num_shards << " shards"); + return false; + } + + /// Update the locking node + if (!my_clock.is_stale()) + { + zookeeper->set(is_dirty_flag_path, host_id, my_clock.discovery_version.value()); + if (my_clock.clean_state_version) + zookeeper->set(is_dirty_cleaned_path, host_id, my_clock.clean_state_version.value()); + else + zookeeper->create(is_dirty_cleaned_path, host_id, zkutil::CreateMode::Persistent); + } + else + { + LOG_DEBUG(log, "Clean state is altered when dropping the partition, cowardly bailing"); + /// clean state is stale + return false; + } + + LOG_INFO(log, "Partition " << task_partition.name << " was dropped on cluster " << task_table.cluster_push_name); + if (zookeeper->tryCreate(current_shards_path, host_id, zkutil::CreateMode::Persistent) == Coordination::ZNODEEXISTS) + zookeeper->set(current_shards_path, host_id); + } + + LOG_INFO(log, "Partition " << task_partition.name << " is safe for work now."); + return true; + } + + + static constexpr UInt64 max_table_tries = 1000; + static constexpr UInt64 max_shard_partition_tries = 600; + + bool tryProcessTable(const ConnectionTimeouts & timeouts, TaskTable & task_table) + { + /// A heuristic: if previous shard is already done, then check next one without sleeps due to max_workers constraint + bool previous_shard_is_instantly_finished = false; + + /// Process each partition that is present in cluster + for (const String & partition_name : task_table.ordered_partition_names) + { + if (!task_table.cluster_partitions.count(partition_name)) + throw Exception("There are no expected partition " + partition_name + ". It is a bug", ErrorCodes::LOGICAL_ERROR); + + ClusterPartition & cluster_partition = task_table.cluster_partitions[partition_name]; + + Stopwatch watch; + /// We will check all the shards of the table and check if they contain current partition. + TasksShard expected_shards; + UInt64 num_failed_shards = 0; + + ++cluster_partition.total_tries; + + LOG_DEBUG(log, "Processing partition " << partition_name << " for the whole cluster"); + + /// Process each source shard having current partition and copy current partition + /// NOTE: shards are sorted by "distance" to current host + bool has_shard_to_process = false; + for (const TaskShardPtr & shard : task_table.all_shards) + { + /// Does shard have a node with current partition? + if (shard->partition_tasks.count(partition_name) == 0) + { + /// If not, did we check existence of that partition previously? + if (shard->checked_partitions.count(partition_name) == 0) + { + auto check_shard_has_partition = [&] () { return checkShardHasPartition(timeouts, *shard, partition_name); }; + bool has_partition = retry(check_shard_has_partition); + + shard->checked_partitions.emplace(partition_name); + + if (has_partition) + { + shard->partition_tasks.emplace(partition_name, ShardPartition(*shard, partition_name, 10)); + LOG_DEBUG(log, "Discovered partition " << partition_name << " in shard " << shard->getDescription()); + /// To save references in the future. + auto shard_partition_it = shard->partition_tasks.find(partition_name); + PartitionPieces & shard_partition_pieces = shard_partition_it->second.pieces; + ///FIXME: Remove 10 + for (int piece_number = 0; piece_number < 10; ++piece_number) + { + auto res = checkPresentPartitionPiecesOnCurrentShard(timeouts, *shard, partition_name, piece_number); + shard_partition_pieces.emplace_back(shard_partition_it->second, piece_number, res); + } + } + else + { + LOG_DEBUG(log, "Found that shard " << shard->getDescription() << " does not contain current partition " << partition_name); + continue; + } + } + else + { + /// We have already checked that partition, but did not discover it + previous_shard_is_instantly_finished = true; + continue; + } + } + + auto it_shard_partition = shard->partition_tasks.find(partition_name); + /// Previously when we discovered that shard does not contain current partition, we skipped it. + /// At this moment partition have to be present. + if (it_shard_partition == shard->partition_tasks.end()) + throw Exception("There are no such partition in a shard. This is a bug.", ErrorCodes::LOGICAL_ERROR); + auto & partition = it_shard_partition->second; + + expected_shards.emplace_back(shard); + + /// Do not sleep if there is a sequence of already processed shards to increase startup + bool is_unprioritized_task = !previous_shard_is_instantly_finished && shard->priority.is_remote; + PartitionTaskStatus task_status = PartitionTaskStatus::Error; + bool was_error = false; + has_shard_to_process = true; + for (UInt64 try_num = 0; try_num < max_shard_partition_tries; ++try_num) + { + task_status = tryProcessPartitionTask(timeouts, partition, is_unprioritized_task); + + /// Exit if success + if (task_status == PartitionTaskStatus::Finished) + break; + + was_error = true; + + /// Skip if the task is being processed by someone + if (task_status == PartitionTaskStatus::Active) + break; + + /// Repeat on errors + std::this_thread::sleep_for(default_sleep_time); + } + + if (task_status == PartitionTaskStatus::Error) + ++num_failed_shards; + + previous_shard_is_instantly_finished = !was_error; + } + + cluster_partition.elapsed_time_seconds += watch.elapsedSeconds(); + + /// Check that whole cluster partition is done + /// Firstly check the number of failed partition tasks, then look into ZooKeeper and ensure that each partition is done + bool partition_is_done = num_failed_shards == 0; + try + { + partition_is_done = + !has_shard_to_process + || (partition_is_done && checkPartitionIsDone(task_table, partition_name, expected_shards)); + } + catch (...) + { + tryLogCurrentException(log); + partition_is_done = false; + } + + if (partition_is_done) + { + task_table.finished_cluster_partitions.emplace(partition_name); + + task_table.bytes_copied += cluster_partition.bytes_copied; + task_table.rows_copied += cluster_partition.rows_copied; + double elapsed = cluster_partition.elapsed_time_seconds; + + LOG_INFO(log, "It took " << std::fixed << std::setprecision(2) << elapsed << " seconds to copy partition " << partition_name + << ": " << formatReadableSizeWithDecimalSuffix(cluster_partition.bytes_copied) << " uncompressed bytes" + << ", " << formatReadableQuantity(cluster_partition.rows_copied) << " rows" + << " and " << cluster_partition.blocks_copied << " source blocks are copied"); + + if (cluster_partition.rows_copied) + { + LOG_INFO(log, "Average partition speed: " + << formatReadableSizeWithDecimalSuffix(cluster_partition.bytes_copied / elapsed) << " per second."); + } + + if (task_table.rows_copied) + { + LOG_INFO(log, "Average table " << task_table.table_id << " speed: " + << formatReadableSizeWithDecimalSuffix(task_table.bytes_copied / elapsed) << " per second."); + } + } + } + + UInt64 required_partitions = task_table.cluster_partitions.size(); + UInt64 finished_partitions = task_table.finished_cluster_partitions.size(); + bool table_is_done = finished_partitions >= required_partitions; + + if (!table_is_done) + { + LOG_INFO(log, "Table " + task_table.table_id + " is not processed yet." + << "Copied " << finished_partitions << " of " << required_partitions << ", will retry"); + } + + return table_is_done; + } + + + /// Execution status of a task + enum class PartitionTaskStatus + { + Active, + Finished, + Error, + }; + + /// Job for copying partition from particular shard. + PartitionTaskStatus tryProcessPartitionTask(const ConnectionTimeouts & timeouts, ShardPartition & task_partition, bool is_unprioritized_task) + { + PartitionTaskStatus res; + + try + { + res = processPartitionTaskImpl(timeouts, task_partition, is_unprioritized_task); + } + catch (...) + { + tryLogCurrentException(log, "An error occurred while processing partition " + task_partition.name); + res = PartitionTaskStatus::Error; + } + + /// At the end of each task check if the config is updated + try + { + updateConfigIfNeeded(); + } + catch (...) + { + tryLogCurrentException(log, "An error occurred while updating the config"); + } + + return res; + } + + PartitionTaskStatus processPartitionTaskImpl(const ConnectionTimeouts & timeouts, ShardPartition & task_partition, bool is_unprioritized_task) + { + TaskShard & task_shard = task_partition.task_shard; + TaskTable & task_table = task_shard.task_table; + ClusterPartition & cluster_partition = task_table.getClusterPartition(task_partition.name); + const size_t number_of_splits = task_table.number_of_splits; + UNUSED(number_of_splits); + + /// We need to update table definitions for each partition, it could be changed after ALTER + createShardInternalTables(timeouts, task_shard); + + auto zookeeper = context.getZooKeeper(); + + const String is_dirty_flag_path = task_partition.getCommonPartitionIsDirtyPath(); + const String is_dirty_cleaned_path = task_partition.getCommonPartitionIsCleanedPath(); + const String current_task_is_active_path = task_partition.getActiveWorkerPath(); + const String current_task_status_path = task_partition.getShardStatusPath(); + + /// Auxiliary functions: + + /// Creates is_dirty node to initialize DROP PARTITION + auto create_is_dirty_node = [&, this] (const CleanStateClock & clock) + { + if (clock.is_stale()) + LOG_DEBUG(log, "Clean state clock is stale while setting dirty flag, cowardly bailing"); + else if (!clock.is_clean()) + LOG_DEBUG(log, "Thank you, Captain Obvious"); + else if (clock.discovery_version) + { + LOG_DEBUG(log, "Updating clean state clock"); + zookeeper->set(is_dirty_flag_path, host_id, clock.discovery_version.value()); + } + else + { + LOG_DEBUG(log, "Creating clean state clock"); + zookeeper->create(is_dirty_flag_path, host_id, zkutil::CreateMode::Persistent); + } + }; + + /// Returns SELECT query filtering current partition and applying user filter + auto get_select_query = [&] (const DatabaseAndTableName & from_table, const String & fields, String limit = "", + bool enable_splitting = false, size_t current_piece_number = 0) + { + String query; + query += "SELECT " + fields + " FROM " + getQuotedTable(from_table); + /// TODO: Bad, it is better to rewrite with ASTLiteral(partition_key_field) + query += " WHERE (" + queryToString(task_table.engine_push_partition_key_ast) + " = (" + task_partition.name + " AS partition_key))"; + if (enable_splitting) + query += " AND ( cityHash64(*) = " + std::to_string(current_piece_number) + " )"; + if (!task_table.where_condition_str.empty()) + query += " AND (" + task_table.where_condition_str + ")"; + if (!limit.empty()) + query += " LIMIT " + limit; + + ParserQuery p_query(query.data() + query.size()); + return parseQuery(p_query, query, 0); + }; + + /// Load balancing + auto worker_node_holder = createTaskWorkerNodeAndWaitIfNeed(zookeeper, current_task_status_path, is_unprioritized_task); + + LOG_DEBUG(log, "Processing " << current_task_status_path); + + CleanStateClock clean_state_clock (zookeeper, is_dirty_flag_path, is_dirty_cleaned_path); + + LogicalClock task_start_clock; + { + Coordination::Stat stat{}; + if (zookeeper->exists(task_partition.getPartitionShardsPath(), &stat)) + task_start_clock = LogicalClock(stat.mzxid); + } + + /// Do not start if partition is dirty, try to clean it + if (clean_state_clock.is_clean() + && (!task_start_clock.hasHappened() || clean_state_clock.discovery_zxid <= task_start_clock)) + { + LOG_DEBUG(log, "Partition " << task_partition.name << " appears to be clean"); + zookeeper->createAncestors(current_task_status_path); + } + else + { + LOG_DEBUG(log, "Partition " << task_partition.name << " is dirty, try to drop it"); + + try + { + tryDropPartition(task_partition, zookeeper, clean_state_clock); + } + catch (...) + { + tryLogCurrentException(log, "An error occurred when clean partition"); + } + + return PartitionTaskStatus::Error; + } + + /// Create ephemeral node to mark that we are active and process the partition + zookeeper->createAncestors(current_task_is_active_path); + zkutil::EphemeralNodeHolderPtr partition_task_node_holder; + try + { + partition_task_node_holder = zkutil::EphemeralNodeHolder::create(current_task_is_active_path, *zookeeper, host_id); + } + catch (const Coordination::Exception & e) + { + if (e.code == Coordination::ZNODEEXISTS) + { + LOG_DEBUG(log, "Someone is already processing " << current_task_is_active_path); + return PartitionTaskStatus::Active; + } + + throw; + } + + /// Exit if task has been already processed; + /// create blocking node to signal cleaning up if it is abandoned + { + String status_data; + if (zookeeper->tryGet(current_task_status_path, status_data)) + { + TaskStateWithOwner status = TaskStateWithOwner::fromString(status_data); + if (status.state == TaskState::Finished) + { + LOG_DEBUG(log, "Task " << current_task_status_path << " has been successfully executed by " << status.owner); + return PartitionTaskStatus::Finished; + } + + // Task is abandoned, initialize DROP PARTITION + LOG_DEBUG(log, "Task " << current_task_status_path << " has not been successfully finished by " << + status.owner << ". Partition will be dropped and refilled."); + + create_is_dirty_node(clean_state_clock); + return PartitionTaskStatus::Error; + } + } + + /// Check that destination partition is empty if we are first worker + /// NOTE: this check is incorrect if pull and push tables have different partition key! + String clean_start_status; + if (!zookeeper->tryGet(task_partition.getPartitionCleanStartPath(), clean_start_status) || clean_start_status != "ok") + { + zookeeper->createIfNotExists(task_partition.getPartitionCleanStartPath(), ""); + auto checker = zkutil::EphemeralNodeHolder::create(task_partition.getPartitionCleanStartPath() + "/checker", + *zookeeper, host_id); + // Maybe we are the first worker + ASTPtr query_select_ast = get_select_query(task_shard.table_split_shard, "count()"); + UInt64 count; + { + Context local_context = context; + // Use pull (i.e. readonly) settings, but fetch data from destination servers + local_context.getSettingsRef() = task_cluster->settings_pull; + local_context.getSettingsRef().skip_unavailable_shards = true; + + Block block = getBlockWithAllStreamData(InterpreterFactory::get(query_select_ast, local_context)->execute().in); + count = (block) ? block.safeGetByPosition(0).column->getUInt(0) : 0; + } + + if (count != 0) + { + Coordination::Stat stat_shards{}; + zookeeper->get(task_partition.getPartitionShardsPath(), &stat_shards); + + /// NOTE: partition is still fresh if dirt discovery happens before cleaning + if (stat_shards.numChildren == 0) + { + LOG_WARNING(log, "There are no workers for partition " << task_partition.name + << ", but destination table contains " << count << " rows" + << ". Partition will be dropped and refilled."); + + create_is_dirty_node(clean_state_clock); + return PartitionTaskStatus::Error; + } + } + zookeeper->set(task_partition.getPartitionCleanStartPath(), "ok"); + } + /// At this point, we need to sync that the destination table is clean + /// before any actual work + + /// Try start processing, create node about it + { + String start_state = TaskStateWithOwner::getData(TaskState::Started, host_id); + CleanStateClock new_clean_state_clock (zookeeper, is_dirty_flag_path, is_dirty_cleaned_path); + if (clean_state_clock != new_clean_state_clock) + { + LOG_INFO(log, "Partition " << task_partition.name << " clean state changed, cowardly bailing"); + return PartitionTaskStatus::Error; + } + else if (!new_clean_state_clock.is_clean()) + { + LOG_INFO(log, "Partition " << task_partition.name << " is dirty and will be dropped and refilled"); + create_is_dirty_node(new_clean_state_clock); + return PartitionTaskStatus::Error; + } + zookeeper->create(current_task_status_path, start_state, zkutil::CreateMode::Persistent); + } + + /// Try create table (if not exists) on each shard + { + auto create_query_push_ast = rewriteCreateQueryStorage(task_shard.current_pull_table_create_query, + task_table.table_push, task_table.engine_push_ast); + create_query_push_ast->as().if_not_exists = true; + String query = queryToString(create_query_push_ast); + + LOG_DEBUG(log, "Create destination tables. Query: " << query); + UInt64 shards = executeQueryOnCluster(task_table.cluster_push, query, + create_query_push_ast, &task_cluster->settings_push, + PoolMode::GET_MANY); + LOG_DEBUG(log, "Destination tables " << getQuotedTable(task_table.table_push) << + " have been created on " << shards << " shards of " << task_table.cluster_push->getShardCount()); + } + + /// Do the copying + { + bool inject_fault = false; + if (copy_fault_probability > 0) + { + double value = std::uniform_real_distribution<>(0, 1)(task_table.task_cluster.random_engine); + inject_fault = value < copy_fault_probability; + } + + // Select all fields + ASTPtr query_select_ast = get_select_query(task_shard.table_read_shard, "*", inject_fault ? "1" : ""); + + LOG_DEBUG(log, "Executing SELECT query and pull from " << task_shard.getDescription() + << " : " << queryToString(query_select_ast)); + + ASTPtr query_insert_ast; + { + String query; + query += "INSERT INTO " + getQuotedTable(task_shard.table_split_shard) + " VALUES "; + + ParserQuery p_query(query.data() + query.size()); + query_insert_ast = parseQuery(p_query, query, 0); + + LOG_DEBUG(log, "Executing INSERT query: " << query); + } + + try + { + /// Custom INSERT SELECT implementation + Context context_select = context; + context_select.getSettingsRef() = task_cluster->settings_pull; + + Context context_insert = context; + context_insert.getSettingsRef() = task_cluster->settings_push; + + BlockInputStreamPtr input; + BlockOutputStreamPtr output; + { + BlockIO io_select = InterpreterFactory::get(query_select_ast, context_select)->execute(); + BlockIO io_insert = InterpreterFactory::get(query_insert_ast, context_insert)->execute(); + + input = io_select.in; + output = io_insert.out; + } + + /// Fail-fast optimization to abort copying when the current clean state expires + std::future future_is_dirty_checker; + + Stopwatch watch(CLOCK_MONOTONIC_COARSE); + constexpr UInt64 check_period_milliseconds = 500; + + /// Will asynchronously check that ZooKeeper connection and is_dirty flag appearing while copying data + auto cancel_check = [&] () + { + if (zookeeper->expired()) + throw Exception("ZooKeeper session is expired, cancel INSERT SELECT", ErrorCodes::UNFINISHED); + + if (!future_is_dirty_checker.valid()) + future_is_dirty_checker = zookeeper->asyncExists(is_dirty_flag_path); + + /// check_period_milliseconds should less than average insert time of single block + /// Otherwise, the insertion will slow a little bit + if (watch.elapsedMilliseconds() >= check_period_milliseconds) + { + Coordination::ExistsResponse status = future_is_dirty_checker.get(); + + if (status.error != Coordination::ZNONODE) + { + LogicalClock dirt_discovery_epoch (status.stat.mzxid); + if (dirt_discovery_epoch == clean_state_clock.discovery_zxid) + return false; + throw Exception("Partition is dirty, cancel INSERT SELECT", ErrorCodes::UNFINISHED); + } + } + + return false; + }; + + /// Update statistics + /// It is quite rough: bytes_copied don't take into account DROP PARTITION. + auto update_stats = [&cluster_partition] (const Block & block) + { + cluster_partition.bytes_copied += block.bytes(); + cluster_partition.rows_copied += block.rows(); + cluster_partition.blocks_copied += 1; + }; + + /// Main work is here + copyData(*input, *output, cancel_check, update_stats); + + // Just in case + if (future_is_dirty_checker.valid()) + future_is_dirty_checker.get(); + + if (inject_fault) + throw Exception("Copy fault injection is activated", ErrorCodes::UNFINISHED); + } + catch (...) + { + tryLogCurrentException(log, "An error occurred during copying, partition will be marked as dirty"); + return PartitionTaskStatus::Error; + } + } + + /// Finalize the processing, change state of current partition task (and also check is_dirty flag) + { + String state_finished = TaskStateWithOwner::getData(TaskState::Finished, host_id); + CleanStateClock new_clean_state_clock (zookeeper, is_dirty_flag_path, is_dirty_cleaned_path); + if (clean_state_clock != new_clean_state_clock) + { + LOG_INFO(log, "Partition " << task_partition.name << " clean state changed, cowardly bailing"); + return PartitionTaskStatus::Error; + } + else if (!new_clean_state_clock.is_clean()) + { + LOG_INFO(log, "Partition " << task_partition.name << " became dirty and will be dropped and refilled"); + create_is_dirty_node(new_clean_state_clock); + return PartitionTaskStatus::Error; + } + zookeeper->set(current_task_status_path, state_finished, 0); + } + + LOG_INFO(log, "Partition " << task_partition.name << " copied"); + return PartitionTaskStatus::Finished; + } + + void dropAndCreateLocalTable(const ASTPtr & create_ast) + { + const auto & create = create_ast->as(); + dropLocalTableIfExists({create.database, create.table}); + + InterpreterCreateQuery interpreter(create_ast, context); + interpreter.execute(); + } + + void dropLocalTableIfExists(const DatabaseAndTableName & table_name) const + { + auto drop_ast = std::make_shared(); + drop_ast->if_exists = true; + drop_ast->database = table_name.first; + drop_ast->table = table_name.second; + + InterpreterDropQuery interpreter(drop_ast, context); + interpreter.execute(); + } + + String getRemoteCreateTable(const DatabaseAndTableName & table, Connection & connection, const Settings * settings = nullptr) + { + String query = "SHOW CREATE TABLE " + getQuotedTable(table); + Block block = getBlockWithAllStreamData(std::make_shared( + connection, query, InterpreterShowCreateQuery::getSampleBlock(), context, settings)); + + return typeid_cast(*block.safeGetByPosition(0).column).getDataAt(0).toString(); + } + + ASTPtr getCreateTableForPullShard(const ConnectionTimeouts & timeouts, TaskShard & task_shard) + { + /// Fetch and parse (possibly) new definition + auto connection_entry = task_shard.info.pool->get(timeouts, &task_cluster->settings_pull); + String create_query_pull_str = getRemoteCreateTable( + task_shard.task_table.table_pull, + *connection_entry, + &task_cluster->settings_pull); + + ParserCreateQuery parser_create_query; + return parseQuery(parser_create_query, create_query_pull_str, 0); + } + + void createShardInternalTables(const ConnectionTimeouts & timeouts, TaskShard & task_shard, bool create_split = true) + { + TaskTable & task_table = task_shard.task_table; + + /// We need to update table definitions for each part, it could be changed after ALTER + task_shard.current_pull_table_create_query = getCreateTableForPullShard(timeouts, task_shard); + + /// Create local Distributed tables: + /// a table fetching data from current shard and a table inserting data to the whole destination cluster + String read_shard_prefix = ".read_shard_" + toString(task_shard.indexInCluster()) + "."; + String split_shard_prefix = ".split."; + task_shard.table_read_shard = DatabaseAndTableName(working_database_name, read_shard_prefix + task_table.table_id); + task_shard.table_split_shard = DatabaseAndTableName(working_database_name, split_shard_prefix + task_table.table_id); + + /// Create special cluster with single shard + String shard_read_cluster_name = read_shard_prefix + task_table.cluster_pull_name; + ClusterPtr cluster_pull_current_shard = task_table.cluster_pull->getClusterWithSingleShard(task_shard.indexInCluster()); + context.setCluster(shard_read_cluster_name, cluster_pull_current_shard); + + auto storage_shard_ast = createASTStorageDistributed(shard_read_cluster_name, task_table.table_pull.first, task_table.table_pull.second); + const auto & storage_split_ast = task_table.engine_split_ast; + + auto create_query_ast = removeAliasColumnsFromCreateQuery(task_shard.current_pull_table_create_query); + auto create_table_pull_ast = rewriteCreateQueryStorage(create_query_ast, task_shard.table_read_shard, storage_shard_ast); + auto create_table_split_ast = rewriteCreateQueryStorage(create_query_ast, task_shard.table_split_shard, storage_split_ast); + + dropAndCreateLocalTable(create_table_pull_ast); + + if (create_split) + dropAndCreateLocalTable(create_table_split_ast); + } + + + std::set getShardPartitions(const ConnectionTimeouts & timeouts, TaskShard & task_shard) + { + createShardInternalTables(timeouts, task_shard, false); + + TaskTable & task_table = task_shard.task_table; + + String query; + { + WriteBufferFromOwnString wb; + wb << "SELECT DISTINCT " << queryToString(task_table.engine_push_partition_key_ast) << " AS partition FROM" + << " " << getQuotedTable(task_shard.table_read_shard) << " ORDER BY partition DESC"; + query = wb.str(); + } + + ParserQuery parser_query(query.data() + query.size()); + ASTPtr query_ast = parseQuery(parser_query, query, 0); + + LOG_DEBUG(log, "Computing destination partition set, executing query: " << query); + + Context local_context = context; + local_context.setSettings(task_cluster->settings_pull); + Block block = getBlockWithAllStreamData(InterpreterFactory::get(query_ast, local_context)->execute().in); + + std::set res; + if (block) + { + ColumnWithTypeAndName & column = block.getByPosition(0); + task_shard.partition_key_column = column; + + for (size_t i = 0; i < column.column->size(); ++i) + { + WriteBufferFromOwnString wb; + column.type->serializeAsTextQuoted(*column.column, i, wb, FormatSettings()); + res.emplace(wb.str()); + } + } + + LOG_DEBUG(log, "There are " << res.size() << " destination partitions in shard " << task_shard.getDescription()); + + return res; + } + + bool checkShardHasPartition(const ConnectionTimeouts & timeouts, TaskShard & task_shard, const String & partition_quoted_name) + { + createShardInternalTables(timeouts, task_shard, false); + + TaskTable & task_table = task_shard.task_table; + + std::string query = "SELECT 1 FROM " + getQuotedTable(task_shard.table_read_shard) + + " WHERE (" + queryToString(task_table.engine_push_partition_key_ast) + " = (" + partition_quoted_name + " AS partition_key))"; + + if (!task_table.where_condition_str.empty()) + query += " AND (" + task_table.where_condition_str + ")"; + + query += " LIMIT 1"; + + LOG_DEBUG(log, "Checking shard " << task_shard.getDescription() << " for partition " + << partition_quoted_name << " existence, executing query: " << query); + + ParserQuery parser_query(query.data() + query.size()); + ASTPtr query_ast = parseQuery(parser_query, query, 0); + + Context local_context = context; + local_context.setSettings(task_cluster->settings_pull); + return InterpreterFactory::get(query_ast, local_context)->execute().in->read().rows() != 0; + } + + /// TODO: Implement checkPresentPartitionPiecesOnCurrentShard(); + /// Just copypaste the function above + bool checkPresentPartitionPiecesOnCurrentShard(const ConnectionTimeouts & timeouts, + TaskShard & task_shard, const String & partition_quoted_name, size_t current_piece_number) + { + createShardInternalTables(timeouts, task_shard, false); + + TaskTable & task_table = task_shard.task_table; + + std::string query = "SELECT 1 FROM " + getQuotedTable(task_shard.table_read_shard) + + " WHERE (" + queryToString(task_table.engine_push_partition_key_ast) + " = (" + partition_quoted_name + " AS partition_key))"; + + query += " AND (cityHash64(*) = " + std::to_string(current_piece_number) + " )"; + + if (!task_table.where_condition_str.empty()) + query += " AND (" + task_table.where_condition_str + ")"; + + query += " LIMIT 1"; + + LOG_DEBUG(log, "Checking shard " << task_shard.getDescription() << " for partition " + << partition_quoted_name << " piece " << std::to_string(current_piece_number) << "existence, executing query: " << query); + + ParserQuery parser_query(query.data() + query.size()); + ASTPtr query_ast = parseQuery(parser_query, query, 0); + + Context local_context = context; + local_context.setSettings(task_cluster->settings_pull); + return InterpreterFactory::get(query_ast, local_context)->execute().in->read().rows() != 0; + } + + /** Executes simple query (without output streams, for example DDL queries) on each shard of the cluster + * Returns number of shards for which at least one replica executed query successfully + */ + UInt64 executeQueryOnCluster( + const ClusterPtr & cluster, + const String & query, + const ASTPtr & query_ast_ = nullptr, + const Settings * settings = nullptr, + PoolMode pool_mode = PoolMode::GET_ALL, + UInt64 max_successful_executions_per_shard = 0) const + { + auto num_shards = cluster->getShardsInfo().size(); + std::vector per_shard_num_successful_replicas(num_shards, 0); + + ASTPtr query_ast; + if (query_ast_ == nullptr) + { + ParserQuery p_query(query.data() + query.size()); + query_ast = parseQuery(p_query, query, 0); + } + else + query_ast = query_ast_; + + + /// We need to execute query on one replica at least + auto do_for_shard = [&] (UInt64 shard_index) + { + const Cluster::ShardInfo & shard = cluster->getShardsInfo().at(shard_index); + UInt64 & num_successful_executions = per_shard_num_successful_replicas.at(shard_index); + num_successful_executions = 0; + + auto increment_and_check_exit = [&] () -> bool + { + ++num_successful_executions; + return max_successful_executions_per_shard && num_successful_executions >= max_successful_executions_per_shard; + }; + + UInt64 num_replicas = cluster->getShardsAddresses().at(shard_index).size(); + UInt64 num_local_replicas = shard.getLocalNodeCount(); + UInt64 num_remote_replicas = num_replicas - num_local_replicas; + + /// In that case we don't have local replicas, but do it just in case + for (UInt64 i = 0; i < num_local_replicas; ++i) + { + auto interpreter = InterpreterFactory::get(query_ast, context); + interpreter->execute(); + + if (increment_and_check_exit()) + return; + } + + /// Will try to make as many as possible queries + if (shard.hasRemoteConnections()) + { + Settings current_settings = settings ? *settings : task_cluster->settings_common; + current_settings.max_parallel_replicas = num_remote_replicas ? num_remote_replicas : 1; + + auto timeouts = ConnectionTimeouts::getTCPTimeoutsWithFailover(current_settings).getSaturated(current_settings.max_execution_time); + auto connections = shard.pool->getMany(timeouts, ¤t_settings, pool_mode); + + for (auto & connection : connections) + { + if (connection.isNull()) + continue; + + try + { + /// CREATE TABLE and DROP PARTITION queries return empty block + RemoteBlockInputStream stream{*connection, query, Block{}, context, ¤t_settings}; + NullBlockOutputStream output{Block{}}; + copyData(stream, output); + + if (increment_and_check_exit()) + return; + } + catch (const Exception &) + { + LOG_INFO(log, getCurrentExceptionMessage(false, true)); + } + } + } + }; + + { + ThreadPool thread_pool(std::min(num_shards, getNumberOfPhysicalCPUCores())); + + for (UInt64 shard_index = 0; shard_index < num_shards; ++shard_index) + thread_pool.scheduleOrThrowOnError([=] { do_for_shard(shard_index); }); + + thread_pool.wait(); + } + + UInt64 successful_shards = 0; + for (UInt64 num_replicas : per_shard_num_successful_replicas) + successful_shards += (num_replicas > 0); + + return successful_shards; + } private: + String task_zookeeper_path; + String task_description_path; + String host_id; + String working_database_name; - using Base = BaseDaemon; + /// Auto update config stuff + UInt64 task_descprtion_current_version = 1; + std::atomic task_descprtion_version{1}; + Coordination::WatchCallback task_description_watch_callback; + /// ZooKeeper session used to set the callback + zkutil::ZooKeeperPtr task_description_watch_zookeeper; - void mainImpl(); + ConfigurationPtr task_cluster_initial_config; + ConfigurationPtr task_cluster_current_config; + Coordination::Stat task_description_current_stat{}; + + std::unique_ptr task_cluster; - std::string config_xml_path; - std::string task_path; - std::string log_level = "trace"; bool is_safe_mode = false; - double copy_fault_probability = 0; - bool is_help = false; + double copy_fault_probability = 0.0; - std::string base_dir; - std::string process_path; - std::string process_id; - std::string host_id; + Context & context; + Poco::Logger * log; + + std::chrono::milliseconds default_sleep_time{1000}; }; - } diff --git a/dbms/programs/copier/ClusterCopierApp.cpp b/dbms/programs/copier/ClusterCopierApp.cpp new file mode 100644 index 00000000000..37b02296086 --- /dev/null +++ b/dbms/programs/copier/ClusterCopierApp.cpp @@ -0,0 +1,173 @@ +#include "ClusterCopierApp.h" +#include "ClusterCopier.h" + +namespace DB +{ + +/// ClusterCopierApp + +void ClusterCopierApp::initialize(Poco::Util::Application & self) +{ + is_help = config().has("help"); + if (is_help) + return; + + config_xml_path = config().getString("config-file"); + task_path = config().getString("task-path"); + log_level = config().getString("log-level", "trace"); + is_safe_mode = config().has("safe-mode"); + if (config().has("copy-fault-probability")) + copy_fault_probability = std::max(std::min(config().getDouble("copy-fault-probability"), 1.0), 0.0); + base_dir = (config().has("base-dir")) ? config().getString("base-dir") : Poco::Path::current(); + // process_id is '#_' + time_t timestamp = Poco::Timestamp().epochTime(); + auto curr_pid = Poco::Process::id(); + + process_id = std::to_string(DateLUT::instance().toNumYYYYMMDDhhmmss(timestamp)) + "_" + std::to_string(curr_pid); + host_id = escapeForFileName(getFQDNOrHostName()) + '#' + process_id; + process_path = Poco::Path(base_dir + "/clickhouse-copier_" + process_id).absolute().toString(); + Poco::File(process_path).createDirectories(); + + /// Override variables for BaseDaemon + if (config().has("log-level")) + config().setString("logger.level", config().getString("log-level")); + + if (config().has("base-dir") || !config().has("logger.log")) + config().setString("logger.log", process_path + "/log.log"); + + if (config().has("base-dir") || !config().has("logger.errorlog")) + config().setString("logger.errorlog", process_path + "/log.err.log"); + + Base::initialize(self); +} + + +void ClusterCopierApp::handleHelp(const std::string &, const std::string &) +{ + Poco::Util::HelpFormatter helpFormatter(options()); + helpFormatter.setCommand(commandName()); + helpFormatter.setHeader("Copies tables from one cluster to another"); + helpFormatter.setUsage("--config-file --task-path "); + helpFormatter.format(std::cerr); + + stopOptionsProcessing(); +} + + +void ClusterCopierApp::defineOptions(Poco::Util::OptionSet & options) +{ + Base::defineOptions(options); + + options.addOption(Poco::Util::Option("task-path", "", "path to task in ZooKeeper") + .argument("task-path").binding("task-path")); + options.addOption(Poco::Util::Option("task-file", "", "path to task file for uploading in ZooKeeper to task-path") + .argument("task-file").binding("task-file")); + options.addOption(Poco::Util::Option("task-upload-force", "", "Force upload task-file even node already exists") + .argument("task-upload-force").binding("task-upload-force")); + options.addOption(Poco::Util::Option("safe-mode", "", "disables ALTER DROP PARTITION in case of errors") + .binding("safe-mode")); + options.addOption(Poco::Util::Option("copy-fault-probability", "", "the copying fails with specified probability (used to test partition state recovering)") + .argument("copy-fault-probability").binding("copy-fault-probability")); + options.addOption(Poco::Util::Option("log-level", "", "sets log level") + .argument("log-level").binding("log-level")); + options.addOption(Poco::Util::Option("base-dir", "", "base directory for copiers, consecutive copier launches will populate /base-dir/launch_id/* directories") + .argument("base-dir").binding("base-dir")); + + using Me = std::decay_t; + options.addOption(Poco::Util::Option("help", "", "produce this help message").binding("help") + .callback(Poco::Util::OptionCallback(this, &Me::handleHelp))); +} + + +void ClusterCopierApp::mainImpl() +{ + StatusFile status_file(process_path + "/status"); + ThreadStatus thread_status; + + auto log = &logger(); + LOG_INFO(log, "Starting clickhouse-copier (" + << "id " << process_id << ", " + << "host_id " << host_id << ", " + << "path " << process_path << ", " + << "revision " << ClickHouseRevision::get() << ")"); + + auto context = std::make_unique(Context::createGlobal()); + context->makeGlobalContext(); + SCOPE_EXIT(context->shutdown()); + + context->setConfig(loaded_config.configuration); + context->setApplicationType(Context::ApplicationType::LOCAL); + context->setPath(process_path); + + registerFunctions(); + registerAggregateFunctions(); + registerTableFunctions(); + registerStorages(); + registerDictionaries(); + registerDisks(); + + static const std::string default_database = "_local"; + context->addDatabase(default_database, std::make_shared(default_database)); + context->setCurrentDatabase(default_database); + + /// Initialize query scope just in case. + CurrentThread::QueryScope query_scope(*context); + + auto copier = std::make_unique(task_path, host_id, default_database, *context); + copier->setSafeMode(is_safe_mode); + copier->setCopyFaultProbability(copy_fault_probability); + + auto task_file = config().getString("task-file", ""); + if (!task_file.empty()) + copier->uploadTaskDescription(task_path, task_file, config().getBool("task-upload-force", false)); + + copier->init(); + copier->process(ConnectionTimeouts::getTCPTimeoutsWithoutFailover(context->getSettingsRef())); + + /// Reset ZooKeeper before removing ClusterCopier. + /// Otherwise zookeeper watch can call callback which use already removed ClusterCopier object. + context->resetZooKeeper(); +} + + +int ClusterCopierApp::main(const std::vector &) +{ + if (is_help) + return 0; + + try + { + mainImpl(); + } + catch (...) + { + tryLogCurrentException(&Poco::Logger::root(), __PRETTY_FUNCTION__); + auto code = getCurrentExceptionCode(); + + return (code) ? code : -1; + } + + return 0; +} + + +} + +#pragma GCC diagnostic ignored "-Wunused-function" +#pragma GCC diagnostic ignored "-Wmissing-declarations" + +int mainEntryClickHouseClusterCopier(int argc, char ** argv) +{ + try + { + DB::ClusterCopierApp app; + return app.run(argc, argv); + } + catch (...) + { + std::cerr << DB::getCurrentExceptionMessage(true) << "\n"; + auto code = DB::getCurrentExceptionCode(); + + return (code) ? code : -1; + } +} diff --git a/dbms/programs/copier/ClusterCopierApp.h b/dbms/programs/copier/ClusterCopierApp.h new file mode 100644 index 00000000000..fe228fd6194 --- /dev/null +++ b/dbms/programs/copier/ClusterCopierApp.h @@ -0,0 +1,87 @@ +#pragma once +#include +#include + +/* clickhouse cluster copier util + * Copies tables data from one cluster to new tables of other (possibly the same) cluster in distributed fault-tolerant manner. + * + * See overview in the docs: docs/en/utils/clickhouse-copier.md + * + * Implementation details: + * + * cluster-copier workers pull each partition of each shard of the source cluster and push it to the destination cluster through + * Distributed table (to preform data resharding). So, worker job is a partition of a source shard. + * A job has three states: Active, Finished and Abandoned. Abandoned means that worker died and did not finish the job. + * + * If an error occurred during the copying (a worker failed or a worker did not finish the INSERT), then the whole partition (on + * all destination servers) should be dropped and refilled. So, copying entity is a partition of all destination shards. + * If a failure is detected a special /is_dirty node is created in ZooKeeper signalling that other workers copying the same partition + * should stop, after a refilling procedure should start. + * + * ZooKeeper task node has the following structure: + * /task/path_root - path passed in --task-path parameter + * /description - contains user-defined XML config of the task + * /task_active_workers - contains ephemeral nodes of all currently active workers, used to implement max_workers limitation + * /server_fqdn#PID_timestamp - cluster-copier worker ID + * ... + * /tables - directory with table tasks + * /cluster.db.table1 - directory of table_hits task + * /partition1 - directory for partition1 + * /shards - directory for source cluster shards + * /1 - worker job for the first shard of partition1 of table test.hits + * Contains info about current status (Active or Finished) and worker ID. + * /2 + * ... + * /partition_active_workers + * /1 - for each job in /shards a corresponding ephemeral node created in /partition_active_workers + * It is used to detect Abandoned jobs (if there is Active node in /shards and there is no node in + * /partition_active_workers). + * Also, it is used to track active workers in the partition (when we need to refill the partition we do + * not DROP PARTITION while there are active workers) + * /2 + * ... + * /is_dirty - the node is set if some worker detected that an error occurred (the INSERT is failed or an Abandoned node is + * detected). If the node appeared workers in this partition should stop and start cleaning and refilling + * partition procedure. + * During this procedure a single 'cleaner' worker is selected. The worker waits for stopping all partition + * workers, removes /shards node, executes DROP PARTITION on each destination node and removes /is_dirty node. + * /cleaner- An ephemeral node used to select 'cleaner' worker. Contains ID of the worker. + * /cluster.db.table2 + * ... + */ + +namespace DB +{ + +class ClusterCopierApp : public BaseDaemon +{ +public: + + void initialize(Poco::Util::Application & self) override; + + void handleHelp(const std::string &, const std::string &); + + void defineOptions(Poco::Util::OptionSet & options) override; + + int main(const std::vector &) override; + +private: + + using Base = BaseDaemon; + + void mainImpl(); + + std::string config_xml_path; + std::string task_path; + std::string log_level = "trace"; + bool is_safe_mode = false; + double copy_fault_probability = 0; + bool is_help = false; + + std::string base_dir; + std::string process_path; + std::string process_id; + std::string host_id; +}; + +} diff --git a/dbms/programs/copier/Internals.h b/dbms/programs/copier/Internals.h index 5f14604fbf9..6e81f16035c 100644 --- a/dbms/programs/copier/Internals.h +++ b/dbms/programs/copier/Internals.h @@ -148,12 +148,14 @@ struct TaskStateWithOwner /// Hierarchical description of the tasks +struct ShardPartitionPiece; struct ShardPartition; struct TaskShard; struct TaskTable; struct TaskCluster; struct ClusterPartition; +using PartitionPieces = std::vector; using TasksPartition = std::map>; using ShardInfo = Cluster::ShardInfo; using TaskShardPtr = std::shared_ptr; @@ -162,22 +164,59 @@ using TasksTable = std::list; using ClusterPartitions = std::map>; +struct ShardPartitionPiece +{ + ShardPartitionPiece(ShardPartition & parent, size_t current_piece_number_, bool is_absent_piece_) + : is_absent_piece(is_absent_piece_) + , current_piece_number(current_piece_number_) + , shard_partition(parent) {} + + [[maybe_unused]] String getPartitionPiecePath() const {return "Not implemented.";} + [[maybe_unused]] String getPartitionPieceCleanStartPath() const {return "Not implemented.";} + [[maybe_unused]] String getCommonPartitionPieceIsDirtyPath() const {return "Not implemented.";} + [[maybe_unused]] String getCommonPartitionPieceIsCleanedPath() const {return "Not implemented.";} + + [[maybe_unused]] String getPartitionPieceActiveWorkersPath() const {return "Not implemented.";} + [[maybe_unused]] String getActiveWorkerPath() const {return "Not implemented.";} + + /// On what shards do we have current partition. + [[maybe_unused]] String getPartitionPieceShardsPath() const {return "Not implemented.";} + [[maybe_unused]] String getShardStatusPath() const {return "Not implemented.";} + + bool is_absent_piece; + const size_t current_piece_number; + + ShardPartition & shard_partition; +}; + + /// Just destination partition of a shard /// I don't know what this comment means. /// In short, when we discovered what shards contain currently processing partition, /// This class describes a partition (name) that is stored on the shard (parent). struct ShardPartition { - ShardPartition(TaskShard & parent, String name_quoted_) : task_shard(parent), name(std::move(name_quoted_)) {} + ShardPartition(TaskShard & parent, String name_quoted_, size_t number_of_splits = 10) + : task_shard(parent) + , name(std::move(name_quoted_)) + { pieces.reserve(number_of_splits); } - String getPartitionPath() const; - String getPartitionCleanStartPath() const; - String getCommonPartitionIsDirtyPath() const; - String getCommonPartitionIsCleanedPath() const; - String getPartitionActiveWorkersPath() const; - String getActiveWorkerPath() const; - String getPartitionShardsPath() const; - String getShardStatusPath() const; + /*useful*/ String getPartitionPath() const; + [[maybe_unused]] String getPartitionPiecePath(size_t current_piece_number) const; + /*useful*/ String getPartitionCleanStartPath() const; + [[maybe_unused]] String getPartitionPieceCleanStartPath(size_t current_piece_number) const; + /*useful*/ String getCommonPartitionIsDirtyPath() const; + /*useful*/ String getCommonPartitionIsCleanedPath() const; + /*??????*/ String getPartitionActiveWorkersPath() const; + /*??????*/ String getActiveWorkerPath() const; + /*useful*/ String getPartitionShardsPath() const; + /*useful*/ String getShardStatusPath() const; + + /// What partition pieces are present in current shard. + /// FYI: Piece is a part of partition which has modulo equals to concrete constant (less than number_of_splits obliously) + /// For example SELECT ... from ... WHERE partition=current_partition AND cityHash64(*) == const; + /// Absent pieces have field is_absent_piece equals to true. + PartitionPieces pieces; TaskShard & task_shard; String name; @@ -255,7 +294,7 @@ struct TaskTable TaskCluster & task_cluster; String getPartitionPath(const String & partition_name) const; - [[maybe_unused]] String getPartitionPathWithPieceNumber(const String & partition_name, size_t current_piece_number) const; + [[maybe_unused]] String getPartitionPiecePath(const String & partition_name, size_t current_piece_number) const; String getPartitionIsDirtyPath(const String & partition_name) const; String getPartitionIsCleanedPath(const String & partition_name) const; String getPartitionTaskStatusPath(const String & partition_name) const; @@ -422,8 +461,9 @@ String TaskTable::getPartitionPath(const String & partition_name) const + "/" + escapeForFileName(partition_name); // 201701 } -String TaskTable::getPartitionPathWithPieceNumber(const String & partition_name, size_t current_piece_number) const +String TaskTable::getPartitionPiecePath(const String & partition_name, size_t current_piece_number) const { + assert(current_piece_number < number_of_splits); return getPartitionPath(partition_name) + "/" + std::to_string(current_piece_number); // 1...number_of_splits } @@ -432,11 +472,23 @@ String ShardPartition::getPartitionCleanStartPath() const return getPartitionPath() + "/clean_start"; } +String ShardPartition::getPartitionPieceCleanStartPath(size_t current_piece_number) const +{ + assert(current_piece_number < task_shard.task_table.number_of_splits); + return getPartitionPiecePath(current_piece_number) + "/clean_start"; +} + String ShardPartition::getPartitionPath() const { return task_shard.task_table.getPartitionPath(name); } +String ShardPartition::getPartitionPiecePath(size_t current_piece_number) const +{ + assert(current_piece_number < task_shard.task_table.number_of_splits); + return task_shard.task_table.getPartitionPiecePath(name, current_piece_number); +} + String ShardPartition::getShardStatusPath() const { // schema: //tables/
//shards/ diff --git a/dbms/programs/copier/ZookeeperStaff.h b/dbms/programs/copier/ZookeeperStaff.h new file mode 100644 index 00000000000..3133c68933d --- /dev/null +++ b/dbms/programs/copier/ZookeeperStaff.h @@ -0,0 +1,224 @@ +#pragma once + +/** Allows to compare two incremental counters of type UInt32 in presence of possible overflow. + * We assume that we compare values that are not too far away. + * For example, when we increment 0xFFFFFFFF, we get 0. So, 0xFFFFFFFF is less than 0. + */ +class WrappingUInt32 +{ +public: + UInt32 value; + + explicit WrappingUInt32(UInt32 _value) + : value(_value) + {} + + bool operator<(const WrappingUInt32 & other) const + { + return value != other.value && *this <= other; + } + + bool operator<=(const WrappingUInt32 & other) const + { + const UInt32 HALF = 1 << 31; + return (value <= other.value && other.value - value < HALF) + || (value > other.value && value - other.value > HALF); + } + + bool operator==(const WrappingUInt32 & other) const + { + return value == other.value; + } +}; + +/** Conforming Zxid definition. + * cf. https://github.com/apache/zookeeper/blob/631d1b284f0edb1c4f6b0fb221bf2428aec71aaa/zookeeper-docs/src/main/resources/markdown/zookeeperInternals.md#guarantees-properties-and-definitions + * + * But it is better to read this: https://zookeeper.apache.org/doc/r3.1.2/zookeeperProgrammers.html + * + * Actually here is the definition of Zxid. + * Every change to the ZooKeeper state receives a stamp in the form of a zxid (ZooKeeper Transaction Id). + * This exposes the total ordering of all changes to ZooKeeper. Each change will have a unique zxid + * and if zxid1 is smaller than zxid2 then zxid1 happened before zxid2. + */ +class Zxid +{ +public: + WrappingUInt32 epoch; + WrappingUInt32 counter; + explicit Zxid(UInt64 _zxid) + : epoch(_zxid >> 32) + , counter(_zxid) + {} + + bool operator<=(const Zxid & other) const + { + return (epoch < other.epoch) + || (epoch == other.epoch && counter <= other.counter); + } + + bool operator==(const Zxid & other) const + { + return epoch == other.epoch && counter == other.counter; + } +}; + +/* When multiple ClusterCopiers discover that the target partition is not empty, + * they will attempt to clean up this partition before proceeding to copying. + * + * Instead of purging is_dirty, the history of cleaning work is preserved and partition hygiene is established + * based on a happens-before relation between the events. + * This relation is encoded by LogicalClock based on the mzxid of the is_dirty ZNode and is_dirty/cleaned. + * The fact of the partition hygiene is encoded by CleanStateClock. + * + * For you to know what mzxid means: + * + * ZooKeeper Stat Structure: + * The Stat structure for each znode in ZooKeeper is made up of the following fields: + * + * -- czxid + * The zxid of the change that caused this znode to be created. + * + * -- mzxid + * The zxid of the change that last modified this znode. + * + * -- ctime + * The time in milliseconds from epoch when this znode was created. + * + * -- mtime + * The time in milliseconds from epoch when this znode was last modified. + * + * -- version + * The number of changes to the data of this znode. + * + * -- cversion + * The number of changes to the children of this znode. + * + * -- aversion + * The number of changes to the ACL of this znode. + * + * -- ephemeralOwner + * The session id of the owner of this znode if the znode is an ephemeral node. + * If it is not an ephemeral node, it will be zero. + * + * -- dataLength + * The length of the data field of this znode. + * + * -- numChildren + * The number of children of this znode. + * */ + +class LogicalClock +{ +public: + std::optional zxid; + + LogicalClock() = default; + + explicit LogicalClock(UInt64 _zxid) + : zxid(_zxid) + {} + + bool hasHappened() const + { + return bool(zxid); + } + + /// happens-before relation with a reasonable time bound + bool happensBefore(const LogicalClock & other) const + { + return !zxid + || (other.zxid && *zxid <= *other.zxid); + } + + bool operator<=(const LogicalClock & other) const + { + return happensBefore(other); + } + + /// strict equality check + bool operator==(const LogicalClock & other) const + { + return zxid == other.zxid; + } +}; + + +class CleanStateClock +{ +public: + LogicalClock discovery_zxid; + std::optional discovery_version; + + LogicalClock clean_state_zxid; + std::optional clean_state_version; + + std::shared_ptr stale; + + bool is_clean() const + { + return + !is_stale() + && ( + !discovery_zxid.hasHappened() + || (clean_state_zxid.hasHappened() && discovery_zxid <= clean_state_zxid)); + } + + bool is_stale() const + { + return stale->load(); + } + + CleanStateClock( + const zkutil::ZooKeeperPtr & zookeeper, + const String & discovery_path, + const String & clean_state_path) + : stale(std::make_shared(false)) + { + Coordination::Stat stat{}; + String _some_data; + auto watch_callback = + [stale = stale] (const Coordination::WatchResponse & rsp) + { + auto logger = &Poco::Logger::get("ClusterCopier"); + if (rsp.error == Coordination::ZOK) + { + switch (rsp.type) + { + case Coordination::CREATED: + LOG_DEBUG(logger, "CleanStateClock change: CREATED, at " << rsp.path); + stale->store(true); + break; + case Coordination::CHANGED: + LOG_DEBUG(logger, "CleanStateClock change: CHANGED, at" << rsp.path); + stale->store(true); + } + } + }; + if (zookeeper->tryGetWatch(discovery_path, _some_data, &stat, watch_callback)) + { + discovery_zxid = LogicalClock(stat.mzxid); + discovery_version = stat.version; + } + if (zookeeper->tryGetWatch(clean_state_path, _some_data, &stat, watch_callback)) + { + clean_state_zxid = LogicalClock(stat.mzxid); + clean_state_version = stat.version; + } + } + + bool operator==(const CleanStateClock & other) const + { + return !is_stale() + && !other.is_stale() + && discovery_zxid == other.discovery_zxid + && discovery_version == other.discovery_version + && clean_state_zxid == other.clean_state_zxid + && clean_state_version == other.clean_state_version; + } + + bool operator!=(const CleanStateClock & other) const + { + return !(*this == other); + } +}; From b97aa2074deb42a364f64c3abea0ffe334304d82 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Wed, 12 Feb 2020 19:15:34 +0300 Subject: [PATCH 0019/1355] piece observing + config --- dbms/programs/copier/ClusterCopier.h | 22 ++++++++++++++++++++-- dbms/programs/copier/Internals.h | 2 +- 2 files changed, 21 insertions(+), 3 deletions(-) diff --git a/dbms/programs/copier/ClusterCopier.h b/dbms/programs/copier/ClusterCopier.h index f4f37b97c26..42519964199 100644 --- a/dbms/programs/copier/ClusterCopier.h +++ b/dbms/programs/copier/ClusterCopier.h @@ -162,6 +162,15 @@ public: { task_shard->partition_tasks.emplace(partition_name, ShardPartition(*task_shard, partition_name, 10)); task_shard->checked_partitions.emplace(partition_name, true); + + auto shard_partition_it = task_shard->partition_tasks.find(partition_name); + PartitionPieces & shard_partition_pieces = shard_partition_it->second.pieces; + + for (int piece_number = 0; piece_number < 10; ++piece_number) + { + bool res = checkPresentPartitionPiecesOnCurrentShard(timeouts, *task_shard, partition_name, piece_number); + shard_partition_pieces.emplace_back(shard_partition_it->second, piece_number, res); + } } if (!missing_partitions.empty()) @@ -1378,7 +1387,9 @@ public: std::string query = "SELECT 1 FROM " + getQuotedTable(task_shard.table_read_shard) + " WHERE (" + queryToString(task_table.engine_push_partition_key_ast) + " = (" + partition_quoted_name + " AS partition_key))"; - query += " AND (cityHash64(*) = " + std::to_string(current_piece_number) + " )"; + const size_t number_of_splits = task_table.number_of_splits; + + query += " AND (cityHash64(*) % " + std::to_string(number_of_splits) + " = " + std::to_string(current_piece_number) + " )"; if (!task_table.where_condition_str.empty()) query += " AND (" + task_table.where_condition_str + ")"; @@ -1393,7 +1404,14 @@ public: Context local_context = context; local_context.setSettings(task_cluster->settings_pull); - return InterpreterFactory::get(query_ast, local_context)->execute().in->read().rows() != 0; + auto result = InterpreterFactory::get(query_ast, local_context)->execute().in->read().rows(); + if (result != 0) + LOG_DEBUG(log, "Partition " << partition_quoted_name << " piece number " + << std::to_string(current_piece_number) << " is PRESENT on shard " << task_shard.getDescription()); + else + LOG_DEBUG(log, "Partition " << partition_quoted_name << " piece number " + << std::to_string(current_piece_number) << " is ABSENT on shard " << task_shard.getDescription()); + return result != 0; } /** Executes simple query (without output streams, for example DDL queries) on each shard of the cluster diff --git a/dbms/programs/copier/Internals.h b/dbms/programs/copier/Internals.h index 6e81f16035c..c2f9509a866 100644 --- a/dbms/programs/copier/Internals.h +++ b/dbms/programs/copier/Internals.h @@ -608,7 +608,7 @@ TaskTable::TaskTable(TaskCluster & parent, const Poco::Util::AbstractConfigurati name_in_config = table_key; - number_of_splits = config.getUInt64("number_of_splits", 10); + number_of_splits = config.getUInt64(table_prefix + "number_of_splits", 10); cluster_pull_name = config.getString(table_prefix + "cluster_pull"); cluster_push_name = config.getString(table_prefix + "cluster_push"); From 5f1fd17b7c2c62c6d1e9fd9cd82c10c095caff9a Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Thu, 13 Feb 2020 13:52:46 +0300 Subject: [PATCH 0020/1355] ShardPartitionPiece path getters --- dbms/programs/copier/ClusterCopier.h | 22 +- dbms/programs/copier/Internals.h | 447 --------------------- dbms/programs/copier/ShardPartition.h | 95 +++++ dbms/programs/copier/ShardPartitionPiece.h | 83 ++++ dbms/programs/copier/TaskCluster.h | 96 +++++ dbms/programs/copier/TaskTable.h | 243 +++++++++++ 6 files changed, 532 insertions(+), 454 deletions(-) create mode 100644 dbms/programs/copier/ShardPartition.h create mode 100644 dbms/programs/copier/ShardPartitionPiece.h create mode 100644 dbms/programs/copier/TaskCluster.h create mode 100644 dbms/programs/copier/TaskTable.h diff --git a/dbms/programs/copier/ClusterCopier.h b/dbms/programs/copier/ClusterCopier.h index 42519964199..8cc0cf21211 100644 --- a/dbms/programs/copier/ClusterCopier.h +++ b/dbms/programs/copier/ClusterCopier.h @@ -1,6 +1,10 @@ #pragma once #include "Internals.h" +#include "TaskCluster.h" +#include "ShardPartition.h" +#include "ShardPartitionPiece.h" +#include "TaskTable.h" #include "ZookeeperStaff.h" namespace DB @@ -166,7 +170,9 @@ public: auto shard_partition_it = task_shard->partition_tasks.find(partition_name); PartitionPieces & shard_partition_pieces = shard_partition_it->second.pieces; - for (int piece_number = 0; piece_number < 10; ++piece_number) + const size_t number_of_splits = task_table.number_of_splits; + + for (size_t piece_number = 0; piece_number < number_of_splits; ++piece_number) { bool res = checkPresentPartitionPiecesOnCurrentShard(timeouts, *task_shard, partition_name, piece_number); shard_partition_pieces.emplace_back(shard_partition_it->second, piece_number, res); @@ -472,14 +478,14 @@ public: { CleanStateClock clean_state_clock ( zookeeper, - task_table.getPartitionIsDirtyPath(partition_name), - task_table.getPartitionIsCleanedPath(partition_name) + task_table.getCertainPartitionIsDirtyPath(partition_name), + task_table.getCertainPartitionIsCleanedPath(partition_name) ); Coordination::Stat stat{}; LogicalClock task_start_clock; - if (zookeeper->exists(task_table.getPartitionTaskStatusPath(partition_name), &stat)) + if (zookeeper->exists(task_table.getCertainPartitionTaskStatusPath(partition_name), &stat)) task_start_clock = LogicalClock(stat.mzxid); - zookeeper->get(task_table.getPartitionTaskStatusPath(partition_name), &stat); + zookeeper->get(task_table.getCertainPartitionTaskStatusPath(partition_name), &stat); if (!clean_state_clock.is_clean() || task_start_clock <= clean_state_clock.discovery_zxid) { LOG_INFO(log, "Partition " << partition_name << " become dirty"); @@ -744,8 +750,10 @@ public: /// To save references in the future. auto shard_partition_it = shard->partition_tasks.find(partition_name); PartitionPieces & shard_partition_pieces = shard_partition_it->second.pieces; - ///FIXME: Remove 10 - for (int piece_number = 0; piece_number < 10; ++piece_number) + + const size_t number_of_splits = task_table.number_of_splits; + + for (size_t piece_number = 0; piece_number < number_of_splits; ++piece_number) { auto res = checkPresentPartitionPiecesOnCurrentShard(timeouts, *shard, partition_name, piece_number); shard_partition_pieces.emplace_back(shard_partition_it->second, piece_number, res); diff --git a/dbms/programs/copier/Internals.h b/dbms/programs/copier/Internals.h index c2f9509a866..4b5cb4b7a83 100644 --- a/dbms/programs/copier/Internals.h +++ b/dbms/programs/copier/Internals.h @@ -101,7 +101,6 @@ String getQuotedTable(const DatabaseAndTableName & db_and_table) return getQuotedTable(db_and_table.first, db_and_table.second); } - enum class TaskState { Started = 0, @@ -164,64 +163,6 @@ using TasksTable = std::list; using ClusterPartitions = std::map>; -struct ShardPartitionPiece -{ - ShardPartitionPiece(ShardPartition & parent, size_t current_piece_number_, bool is_absent_piece_) - : is_absent_piece(is_absent_piece_) - , current_piece_number(current_piece_number_) - , shard_partition(parent) {} - - [[maybe_unused]] String getPartitionPiecePath() const {return "Not implemented.";} - [[maybe_unused]] String getPartitionPieceCleanStartPath() const {return "Not implemented.";} - [[maybe_unused]] String getCommonPartitionPieceIsDirtyPath() const {return "Not implemented.";} - [[maybe_unused]] String getCommonPartitionPieceIsCleanedPath() const {return "Not implemented.";} - - [[maybe_unused]] String getPartitionPieceActiveWorkersPath() const {return "Not implemented.";} - [[maybe_unused]] String getActiveWorkerPath() const {return "Not implemented.";} - - /// On what shards do we have current partition. - [[maybe_unused]] String getPartitionPieceShardsPath() const {return "Not implemented.";} - [[maybe_unused]] String getShardStatusPath() const {return "Not implemented.";} - - bool is_absent_piece; - const size_t current_piece_number; - - ShardPartition & shard_partition; -}; - - -/// Just destination partition of a shard -/// I don't know what this comment means. -/// In short, when we discovered what shards contain currently processing partition, -/// This class describes a partition (name) that is stored on the shard (parent). -struct ShardPartition -{ - ShardPartition(TaskShard & parent, String name_quoted_, size_t number_of_splits = 10) - : task_shard(parent) - , name(std::move(name_quoted_)) - { pieces.reserve(number_of_splits); } - - /*useful*/ String getPartitionPath() const; - [[maybe_unused]] String getPartitionPiecePath(size_t current_piece_number) const; - /*useful*/ String getPartitionCleanStartPath() const; - [[maybe_unused]] String getPartitionPieceCleanStartPath(size_t current_piece_number) const; - /*useful*/ String getCommonPartitionIsDirtyPath() const; - /*useful*/ String getCommonPartitionIsCleanedPath() const; - /*??????*/ String getPartitionActiveWorkersPath() const; - /*??????*/ String getActiveWorkerPath() const; - /*useful*/ String getPartitionShardsPath() const; - /*useful*/ String getShardStatusPath() const; - - /// What partition pieces are present in current shard. - /// FYI: Piece is a part of partition which has modulo equals to concrete constant (less than number_of_splits obliously) - /// For example SELECT ... from ... WHERE partition=current_partition AND cityHash64(*) == const; - /// Absent pieces have field is_absent_piece equals to true. - PartitionPieces pieces; - - TaskShard & task_shard; - String name; -}; - struct ShardPriority { @@ -286,129 +227,6 @@ struct ClusterPartition }; -struct TaskTable -{ - TaskTable(TaskCluster & parent, const Poco::Util::AbstractConfiguration & config, const String & prefix, - const String & table_key); - - TaskCluster & task_cluster; - - String getPartitionPath(const String & partition_name) const; - [[maybe_unused]] String getPartitionPiecePath(const String & partition_name, size_t current_piece_number) const; - String getPartitionIsDirtyPath(const String & partition_name) const; - String getPartitionIsCleanedPath(const String & partition_name) const; - String getPartitionTaskStatusPath(const String & partition_name) const; - - /// Partitions will be splitted into number-of-splits pieces. - /// Each piece will be copied independently. (10 by default) - size_t number_of_splits; - - String name_in_config; - - /// Used as task ID - String table_id; - - /// Source cluster and table - String cluster_pull_name; - DatabaseAndTableName table_pull; - - /// Destination cluster and table - String cluster_push_name; - DatabaseAndTableName table_push; - - /// Storage of destination table - String engine_push_str; - ASTPtr engine_push_ast; - ASTPtr engine_push_partition_key_ast; - - /// A Distributed table definition used to split data - String sharding_key_str; - ASTPtr sharding_key_ast; - ASTPtr engine_split_ast; - - /// Additional WHERE expression to filter input data - String where_condition_str; - ASTPtr where_condition_ast; - - /// Resolved clusters - ClusterPtr cluster_pull; - ClusterPtr cluster_push; - - /// Filter partitions that should be copied - bool has_enabled_partitions = false; - Strings enabled_partitions; - NameSet enabled_partitions_set; - - /// Prioritized list of shards - /// all_shards contains information about all shards in the table. - /// So we have to check whether particular shard have current partiton or not while processing. - TasksShard all_shards; - TasksShard local_shards; - - /// All partitions of the current table. - ClusterPartitions cluster_partitions; - NameSet finished_cluster_partitions; - - /// Parition names to process in user-specified order - Strings ordered_partition_names; - - ClusterPartition & getClusterPartition(const String & partition_name) - { - auto it = cluster_partitions.find(partition_name); - if (it == cluster_partitions.end()) - throw Exception("There are no cluster partition " + partition_name + " in " + table_id, ErrorCodes::LOGICAL_ERROR); - return it->second; - } - - Stopwatch watch; - UInt64 bytes_copied = 0; - UInt64 rows_copied = 0; - - template - void initShards(RandomEngine && random_engine); -}; - - -struct TaskCluster -{ - TaskCluster(const String & task_zookeeper_path_, const String & default_local_database_) - : task_zookeeper_path(task_zookeeper_path_), default_local_database(default_local_database_) {} - - void loadTasks(const Poco::Util::AbstractConfiguration & config, const String & base_key = ""); - - /// Set (or update) settings and max_workers param - void reloadSettings(const Poco::Util::AbstractConfiguration & config, const String & base_key = ""); - - /// Base node for all tasks. Its structure: - /// workers/ - directory with active workers (amount of them is less or equal max_workers) - /// description - node with task configuration - /// table_table1/ - directories with per-partition copying status - String task_zookeeper_path; - - /// Database used to create temporary Distributed tables - String default_local_database; - - /// Limits number of simultaneous workers - UInt64 max_workers = 0; - - /// Base settings for pull and push - Settings settings_common; - /// Settings used to fetch data - Settings settings_pull; - /// Settings used to insert data - Settings settings_push; - - String clusters_prefix; - - /// Subtasks - TasksTable table_tasks; - - std::random_device random_device; - pcg64 random_engine; -}; - - - struct MultiTransactionInfo { int32_t code; @@ -452,107 +270,6 @@ Block getBlockWithAllStreamData(const BlockInputStreamPtr & stream) } -/// Path getters - -String TaskTable::getPartitionPath(const String & partition_name) const -{ - return task_cluster.task_zookeeper_path // root - + "/tables/" + table_id // tables/dst_cluster.merge.hits - + "/" + escapeForFileName(partition_name); // 201701 -} - -String TaskTable::getPartitionPiecePath(const String & partition_name, size_t current_piece_number) const -{ - assert(current_piece_number < number_of_splits); - return getPartitionPath(partition_name) + "/" + std::to_string(current_piece_number); // 1...number_of_splits -} - -String ShardPartition::getPartitionCleanStartPath() const -{ - return getPartitionPath() + "/clean_start"; -} - -String ShardPartition::getPartitionPieceCleanStartPath(size_t current_piece_number) const -{ - assert(current_piece_number < task_shard.task_table.number_of_splits); - return getPartitionPiecePath(current_piece_number) + "/clean_start"; -} - -String ShardPartition::getPartitionPath() const -{ - return task_shard.task_table.getPartitionPath(name); -} - -String ShardPartition::getPartitionPiecePath(size_t current_piece_number) const -{ - assert(current_piece_number < task_shard.task_table.number_of_splits); - return task_shard.task_table.getPartitionPiecePath(name, current_piece_number); -} - -String ShardPartition::getShardStatusPath() const -{ - // schema: //tables/
//shards/ - // e.g. /root/table_test.hits/201701/shards/1 - return getPartitionShardsPath() + "/" + toString(task_shard.numberInCluster()); -} - -String ShardPartition::getPartitionShardsPath() const -{ - return getPartitionPath() + "/shards"; -} - -String ShardPartition::getPartitionActiveWorkersPath() const -{ - return getPartitionPath() + "/partition_active_workers"; -} - -String ShardPartition::getActiveWorkerPath() const -{ - return getPartitionActiveWorkersPath() + "/" + toString(task_shard.numberInCluster()); -} - -String ShardPartition::getCommonPartitionIsDirtyPath() const -{ - return getPartitionPath() + "/is_dirty"; -} - -String ShardPartition::getCommonPartitionIsCleanedPath() const -{ - return getCommonPartitionIsDirtyPath() + "/cleaned"; -} - -String TaskTable::getPartitionIsDirtyPath(const String & partition_name) const -{ - return getPartitionPath(partition_name) + "/is_dirty"; -} - -String TaskTable::getPartitionIsCleanedPath(const String & partition_name) const -{ - return getPartitionIsDirtyPath(partition_name) + "/cleaned"; -} - -String TaskTable::getPartitionTaskStatusPath(const String & partition_name) const -{ - return getPartitionPath(partition_name) + "/shards"; -} - -String TaskShard::getDescription() const -{ - std::stringstream ss; - ss << "N" << numberInCluster() - << " (having a replica " << getHostNameExample() - << ", pull table " + getQuotedTable(task_table.table_pull) - << " of cluster " + task_table.cluster_pull_name << ")"; - return ss.str(); -} - -String TaskShard::getHostNameExample() const -{ - auto & replicas = task_table.cluster_pull->getShardsAddresses().at(indexInCluster()); - return replicas.at(0).readableString(); -} - - static bool isExtendedDefinitionStorage(const ASTPtr & storage_ast) { const auto & storage = storage_ast->as(); @@ -600,85 +317,6 @@ static ASTPtr extractPartitionKey(const ASTPtr & storage_ast) } -TaskTable::TaskTable(TaskCluster & parent, const Poco::Util::AbstractConfiguration & config, const String & prefix_, - const String & table_key) - : task_cluster(parent) -{ - String table_prefix = prefix_ + "." + table_key + "."; - - name_in_config = table_key; - - number_of_splits = config.getUInt64(table_prefix + "number_of_splits", 10); - - cluster_pull_name = config.getString(table_prefix + "cluster_pull"); - cluster_push_name = config.getString(table_prefix + "cluster_push"); - - table_pull.first = config.getString(table_prefix + "database_pull"); - table_pull.second = config.getString(table_prefix + "table_pull"); - - table_push.first = config.getString(table_prefix + "database_push"); - table_push.second = config.getString(table_prefix + "table_push"); - - /// Used as node name in ZooKeeper - table_id = escapeForFileName(cluster_push_name) - + "." + escapeForFileName(table_push.first) - + "." + escapeForFileName(table_push.second); - - engine_push_str = config.getString(table_prefix + "engine"); - { - ParserStorage parser_storage; - engine_push_ast = parseQuery(parser_storage, engine_push_str, 0); - engine_push_partition_key_ast = extractPartitionKey(engine_push_ast); - } - - sharding_key_str = config.getString(table_prefix + "sharding_key"); - { - ParserExpressionWithOptionalAlias parser_expression(false); - sharding_key_ast = parseQuery(parser_expression, sharding_key_str, 0); - engine_split_ast = createASTStorageDistributed(cluster_push_name, table_push.first, table_push.second, sharding_key_ast); - } - - where_condition_str = config.getString(table_prefix + "where_condition", ""); - if (!where_condition_str.empty()) - { - ParserExpressionWithOptionalAlias parser_expression(false); - where_condition_ast = parseQuery(parser_expression, where_condition_str, 0); - - // Will use canonical expression form - where_condition_str = queryToString(where_condition_ast); - } - - String enabled_partitions_prefix = table_prefix + "enabled_partitions"; - has_enabled_partitions = config.has(enabled_partitions_prefix); - - if (has_enabled_partitions) - { - Strings keys; - config.keys(enabled_partitions_prefix, keys); - - if (keys.empty()) - { - /// Parse list of partition from space-separated string - String partitions_str = config.getString(table_prefix + "enabled_partitions"); - boost::trim_if(partitions_str, isWhitespaceASCII); - boost::split(enabled_partitions, partitions_str, isWhitespaceASCII, boost::token_compress_on); - } - else - { - /// Parse sequence of ... - for (const String & key : keys) - { - if (!startsWith(key, "partition")) - throw Exception("Unknown key " + key + " in " + enabled_partitions_prefix, ErrorCodes::UNKNOWN_ELEMENT_IN_CONFIG); - - enabled_partitions.emplace_back(config.getString(enabled_partitions_prefix + "." + key)); - } - } - - std::copy(enabled_partitions.begin(), enabled_partitions.end(), std::inserter(enabled_partitions_set, enabled_partitions_set.begin())); - } -} - static ShardPriority getReplicasPriority(const Cluster::Addresses & replicas, const std::string & local_hostname, UInt8 random) { @@ -708,91 +346,6 @@ static ShardPriority getReplicasPriority(const Cluster::Addresses & replicas, co return res; } -template -void TaskTable::initShards(RandomEngine && random_engine) -{ - const String & fqdn_name = getFQDNOrHostName(); - std::uniform_int_distribution get_urand(0, std::numeric_limits::max()); - - // Compute the priority - for (auto & shard_info : cluster_pull->getShardsInfo()) - { - TaskShardPtr task_shard = std::make_shared(*this, shard_info); - const auto & replicas = cluster_pull->getShardsAddresses().at(task_shard->indexInCluster()); - task_shard->priority = getReplicasPriority(replicas, fqdn_name, get_urand(random_engine)); - - all_shards.emplace_back(task_shard); - } - - // Sort by priority - std::sort(all_shards.begin(), all_shards.end(), - [] (const TaskShardPtr & lhs, const TaskShardPtr & rhs) - { - return ShardPriority::greaterPriority(lhs->priority, rhs->priority); - }); - - // Cut local shards - auto it_first_remote = std::lower_bound(all_shards.begin(), all_shards.end(), 1, - [] (const TaskShardPtr & lhs, UInt8 is_remote) - { - return lhs->priority.is_remote < is_remote; - }); - - local_shards.assign(all_shards.begin(), it_first_remote); -} - - -void TaskCluster::loadTasks(const Poco::Util::AbstractConfiguration & config, const String & base_key) -{ - String prefix = base_key.empty() ? "" : base_key + "."; - - clusters_prefix = prefix + "remote_servers"; - if (!config.has(clusters_prefix)) - throw Exception("You should specify list of clusters in " + clusters_prefix, ErrorCodes::BAD_ARGUMENTS); - - Poco::Util::AbstractConfiguration::Keys tables_keys; - config.keys(prefix + "tables", tables_keys); - - for (const auto & table_key : tables_keys) - { - table_tasks.emplace_back(*this, config, prefix + "tables", table_key); - } -} - -void TaskCluster::reloadSettings(const Poco::Util::AbstractConfiguration & config, const String & base_key) -{ - String prefix = base_key.empty() ? "" : base_key + "."; - - max_workers = config.getUInt64(prefix + "max_workers"); - - settings_common = Settings(); - if (config.has(prefix + "settings")) - settings_common.loadSettingsFromConfig(prefix + "settings", config); - - settings_pull = settings_common; - if (config.has(prefix + "settings_pull")) - settings_pull.loadSettingsFromConfig(prefix + "settings_pull", config); - - settings_push = settings_common; - if (config.has(prefix + "settings_push")) - settings_push.loadSettingsFromConfig(prefix + "settings_push", config); - - auto set_default_value = [] (auto && setting, auto && default_value) - { - setting = setting.changed ? setting.value : default_value; - }; - - /// Override important settings - settings_pull.readonly = 1; - settings_push.insert_distributed_sync = 1; - set_default_value(settings_pull.load_balancing, LoadBalancing::NEAREST_HOSTNAME); - set_default_value(settings_pull.max_threads, 1); - set_default_value(settings_pull.max_block_size, 8192UL); - set_default_value(settings_pull.preferred_block_size_bytes, 0); - set_default_value(settings_push.insert_distributed_timeout, 0); -} - } // end of an anonymous namespace } - diff --git a/dbms/programs/copier/ShardPartition.h b/dbms/programs/copier/ShardPartition.h new file mode 100644 index 00000000000..2881d3e2f79 --- /dev/null +++ b/dbms/programs/copier/ShardPartition.h @@ -0,0 +1,95 @@ +#pragma once + +#include "Internals.h" +#include "TaskTable.h" + +namespace DB { + +namespace { + + +/// Just destination partition of a shard +/// I don't know what this comment means. +/// In short, when we discovered what shards contain currently processing partition, +/// This class describes a partition (name) that is stored on the shard (parent). +struct ShardPartition { + ShardPartition(TaskShard &parent, String name_quoted_, size_t number_of_splits = 10) + : task_shard(parent), name(std::move(name_quoted_)) { pieces.reserve(number_of_splits); } + + /*useful*/ String getPartitionPath() const; + + [[maybe_unused]] String getPartitionPiecePath(size_t current_piece_number) const; + + /*useful*/ String getPartitionCleanStartPath() const; + + [[maybe_unused]] String getPartitionPieceCleanStartPath(size_t current_piece_number) const; + + /*useful*/ String getCommonPartitionIsDirtyPath() const; + + /*useful*/ String getCommonPartitionIsCleanedPath() const; + + /*??????*/ String getPartitionActiveWorkersPath() const; + + /*??????*/ String getActiveWorkerPath() const; + + /*useful*/ String getPartitionShardsPath() const; + + /*useful*/ String getShardStatusPath() const; + + /// What partition pieces are present in current shard. + /// FYI: Piece is a part of partition which has modulo equals to concrete constant (less than number_of_splits obliously) + /// For example SELECT ... from ... WHERE partition=current_partition AND cityHash64(*) == const; + /// Absent pieces have field is_absent_piece equals to true. + PartitionPieces pieces; + + TaskShard & task_shard; + String name; +}; + +String ShardPartition::getPartitionCleanStartPath() const { + return getPartitionPath() + "/clean_start"; +} + +String ShardPartition::getPartitionPieceCleanStartPath(size_t current_piece_number) const { + assert(current_piece_number < task_shard.task_table.number_of_splits); + return getPartitionPiecePath(current_piece_number) + "/clean_start"; +} + +String ShardPartition::getPartitionPath() const { + return task_shard.task_table.getPartitionPath(name); +} + +String ShardPartition::getPartitionPiecePath(size_t current_piece_number) const { + assert(current_piece_number < task_shard.task_table.number_of_splits); + return task_shard.task_table.getPartitionPiecePath(name, current_piece_number); +} + +String ShardPartition::getShardStatusPath() const { + // schema: //tables/
//shards/ + // e.g. /root/table_test.hits/201701/shards/1 + return getPartitionShardsPath() + "/" + toString(task_shard.numberInCluster()); +} + +String ShardPartition::getPartitionShardsPath() const { + return getPartitionPath() + "/shards"; +} + +String ShardPartition::getPartitionActiveWorkersPath() const { + return getPartitionPath() + "/partition_active_workers"; +} + +String ShardPartition::getActiveWorkerPath() const { + return getPartitionActiveWorkersPath() + "/" + toString(task_shard.numberInCluster()); +} + +String ShardPartition::getCommonPartitionIsDirtyPath() const { + return getPartitionPath() + "/is_dirty"; +} + +String ShardPartition::getCommonPartitionIsCleanedPath() const { + return getCommonPartitionIsDirtyPath() + "/cleaned"; +} + +} //end of anonymous namespace + +} diff --git a/dbms/programs/copier/ShardPartitionPiece.h b/dbms/programs/copier/ShardPartitionPiece.h new file mode 100644 index 00000000000..867a5c4c65a --- /dev/null +++ b/dbms/programs/copier/ShardPartitionPiece.h @@ -0,0 +1,83 @@ +#pragma once + +#include "Internals.h" + +namespace DB { + +namespace { + +struct ShardPartitionPiece { + + ShardPartitionPiece(ShardPartition &parent, size_t current_piece_number_, bool is_absent_piece_) + : is_absent_piece(is_absent_piece_), current_piece_number(current_piece_number_), + shard_partition(parent) {} + + [[maybe_unused]] String getPartitionPiecePath() const; + + [[maybe_unused]] String getPartitionPieceCleanStartPath() const; + + [[maybe_unused]] String getPartitionPieceIsDirtyPath() const; + + [[maybe_unused]] String getPartitionPieceIsCleanedPath() const; + + [[maybe_unused]] String getPartitionPieceActiveWorkersPath() const; + + [[maybe_unused]] String getActiveWorkerPath() const ; + + /// On what shards do we have current partition. + [[maybe_unused]] String getPartitionPieceShardsPath() const; + + [[maybe_unused]] String getShardStatusPath() const; + + bool is_absent_piece; + const size_t current_piece_number; + + ShardPartition & shard_partition; +}; + + +String ShardPartitionPiece::getPartitionPiecePath() const +{ + return shard_partition.getPartitionPath() + "/piece" + std::to_string(current_piece_number); +} + +String ShardPartitionPiece::getPartitionPieceCleanStartPath() const +{ + return getPartitionPiecePath() + "/clean_start"; +} + +String ShardPartitionPiece::getPartitionPieceIsDirtyPath() const +{ + return getPartitionPiecePath() + "/is_dirty"; +} + +String ShardPartitionPiece::getPartitionPieceIsCleanedPath() const +{ + return getPartitionPieceIsDirtyPath() + "/is_cleaned"; +} + +String ShardPartitionPiece::getPartitionPieceActiveWorkersPath() const +{ + return getPartitionPiecePath() + "/partition_active_workers"; +} + +String ShardPartitionPiece::getActiveWorkerPath() const +{ + return getPartitionPieceActiveWorkersPath() + "/" + toString(shard_partition.task_shard.numberInCluster()); +} + +/// On what shards do we have current partition. +String ShardPartitionPiece::getPartitionPieceShardsPath() const +{ + return getPartitionPiecePath() + "/shards"; +} + +String ShardPartitionPiece::getShardStatusPath() const +{ + return getPartitionPieceShardsPath() + "/" + toString(shard_partition.task_shard.numberInCluster()); +} + + +} // end of anonymous namespace + +} diff --git a/dbms/programs/copier/TaskCluster.h b/dbms/programs/copier/TaskCluster.h new file mode 100644 index 00000000000..8cf67ad78fb --- /dev/null +++ b/dbms/programs/copier/TaskCluster.h @@ -0,0 +1,96 @@ +#pragma once + +#include "Internals.h" + +namespace DB { + +namespace { + +struct TaskCluster { + TaskCluster(const String &task_zookeeper_path_, const String &default_local_database_) + : task_zookeeper_path(task_zookeeper_path_), default_local_database(default_local_database_) {} + + void loadTasks(const Poco::Util::AbstractConfiguration &config, const String &base_key = ""); + + /// Set (or update) settings and max_workers param + void reloadSettings(const Poco::Util::AbstractConfiguration &config, const String &base_key = ""); + + /// Base node for all tasks. Its structure: + /// workers/ - directory with active workers (amount of them is less or equal max_workers) + /// description - node with task configuration + /// table_table1/ - directories with per-partition copying status + String task_zookeeper_path; + + /// Database used to create temporary Distributed tables + String default_local_database; + + /// Limits number of simultaneous workers + UInt64 max_workers = 0; + + /// Base settings for pull and push + Settings settings_common; + /// Settings used to fetch data + Settings settings_pull; + /// Settings used to insert data + Settings settings_push; + + String clusters_prefix; + + /// Subtasks + TasksTable table_tasks; + + std::random_device random_device; + pcg64 random_engine; +}; + + +void TaskCluster::loadTasks(const Poco::Util::AbstractConfiguration &config, const String &base_key) { + String prefix = base_key.empty() ? "" : base_key + "."; + + clusters_prefix = prefix + "remote_servers"; + if (!config.has(clusters_prefix)) + throw Exception("You should specify list of clusters in " + clusters_prefix, ErrorCodes::BAD_ARGUMENTS); + + Poco::Util::AbstractConfiguration::Keys tables_keys; + config.keys(prefix + "tables", tables_keys); + + for (const auto &table_key : tables_keys) { + table_tasks.emplace_back(*this, config, prefix + "tables", table_key); + } +} + +void TaskCluster::reloadSettings(const Poco::Util::AbstractConfiguration &config, const String &base_key) { + String prefix = base_key.empty() ? "" : base_key + "."; + + max_workers = config.getUInt64(prefix + "max_workers"); + + settings_common = Settings(); + if (config.has(prefix + "settings")) + settings_common.loadSettingsFromConfig(prefix + "settings", config); + + settings_pull = settings_common; + if (config.has(prefix + "settings_pull")) + settings_pull.loadSettingsFromConfig(prefix + "settings_pull", config); + + settings_push = settings_common; + if (config.has(prefix + "settings_push")) + settings_push.loadSettingsFromConfig(prefix + "settings_push", config); + + auto set_default_value = [](auto &&setting, auto &&default_value) { + setting = setting.changed ? setting.value : default_value; + }; + + /// Override important settings + settings_pull.readonly = 1; + settings_push.insert_distributed_sync = 1; + set_default_value(settings_pull.load_balancing, LoadBalancing::NEAREST_HOSTNAME); + set_default_value(settings_pull.max_threads, 1); + set_default_value(settings_pull.max_block_size, 8192UL); + set_default_value(settings_pull.preferred_block_size_bytes, 0); + set_default_value(settings_push.insert_distributed_timeout, 0); +} + +} // end of anonymous namespace + +} + diff --git a/dbms/programs/copier/TaskTable.h b/dbms/programs/copier/TaskTable.h new file mode 100644 index 00000000000..04c8d747ea4 --- /dev/null +++ b/dbms/programs/copier/TaskTable.h @@ -0,0 +1,243 @@ +#pragma once + +#include "Internals.h" +#include "TaskCluster.h" + +namespace DB { + +namespace { + + +struct TaskTable { + TaskTable(TaskCluster &parent, const Poco::Util::AbstractConfiguration &config, const String &prefix, + const String &table_key); + + TaskCluster &task_cluster; + + String getPartitionPath(const String &partition_name) const; + + [[maybe_unused]] String + getPartitionPiecePath(const String &partition_name, size_t current_piece_number) const; + + String getCertainPartitionIsDirtyPath(const String &partition_name) const; + + String getCertainPartitionIsCleanedPath(const String &partition_name) const; + + String getCertainPartitionTaskStatusPath(const String &partition_name) const; + + /// Partitions will be splitted into number-of-splits pieces. + /// Each piece will be copied independently. (10 by default) + size_t number_of_splits; + + String name_in_config; + + /// Used as task ID + String table_id; + + /// Source cluster and table + String cluster_pull_name; + DatabaseAndTableName table_pull; + + /// Destination cluster and table + String cluster_push_name; + DatabaseAndTableName table_push; + + /// Storage of destination table + String engine_push_str; + ASTPtr engine_push_ast; + ASTPtr engine_push_partition_key_ast; + + /// A Distributed table definition used to split data + String sharding_key_str; + ASTPtr sharding_key_ast; + ASTPtr engine_split_ast; + + /// Additional WHERE expression to filter input data + String where_condition_str; + ASTPtr where_condition_ast; + + /// Resolved clusters + ClusterPtr cluster_pull; + ClusterPtr cluster_push; + + /// Filter partitions that should be copied + bool has_enabled_partitions = false; + Strings enabled_partitions; + NameSet enabled_partitions_set; + + /// Prioritized list of shards + /// all_shards contains information about all shards in the table. + /// So we have to check whether particular shard have current partiton or not while processing. + TasksShard all_shards; + TasksShard local_shards; + + /// All partitions of the current table. + ClusterPartitions cluster_partitions; + NameSet finished_cluster_partitions; + + /// Parition names to process in user-specified order + Strings ordered_partition_names; + + ClusterPartition &getClusterPartition(const String &partition_name) { + auto it = cluster_partitions.find(partition_name); + if (it == cluster_partitions.end()) + throw Exception("There are no cluster partition " + partition_name + " in " + table_id, + ErrorCodes::LOGICAL_ERROR); + return it->second; + } + + Stopwatch watch; + UInt64 bytes_copied = 0; + UInt64 rows_copied = 0; + + template + void initShards(RandomEngine &&random_engine); +}; + + +String TaskTable::getPartitionPath(const String &partition_name) const { + return task_cluster.task_zookeeper_path // root + + "/tables/" + table_id // tables/dst_cluster.merge.hits + + "/" + escapeForFileName(partition_name); // 201701 +} + +String TaskTable::getPartitionPiecePath(const String &partition_name, size_t current_piece_number) const { + assert(current_piece_number < number_of_splits); + return getPartitionPath(partition_name) + "/" + + std::to_string(current_piece_number); // 1...number_of_splits +} + +String TaskTable::getCertainPartitionIsDirtyPath(const String &partition_name) const { + return getPartitionPath(partition_name) + "/is_dirty"; +} + +String TaskTable::getCertainPartitionIsCleanedPath(const String &partition_name) const { + return getCertainPartitionIsDirtyPath(partition_name) + "/cleaned"; +} + +String TaskTable::getCertainPartitionTaskStatusPath(const String &partition_name) const { + return getPartitionPath(partition_name) + "/shards"; +} + +String TaskShard::getDescription() const { + std::stringstream ss; + ss << "N" << numberInCluster() + << " (having a replica " << getHostNameExample() + << ", pull table " + getQuotedTable(task_table.table_pull) + << " of cluster " + task_table.cluster_pull_name << ")"; + return ss.str(); +} + +String TaskShard::getHostNameExample() const { + auto &replicas = task_table.cluster_pull->getShardsAddresses().at(indexInCluster()); + return replicas.at(0).readableString(); +} + +TaskTable::TaskTable(TaskCluster &parent, const Poco::Util::AbstractConfiguration &config, + const String &prefix_, + const String &table_key) + : task_cluster(parent) { + String table_prefix = prefix_ + "." + table_key + "."; + + name_in_config = table_key; + + number_of_splits = config.getUInt64(table_prefix + "number_of_splits", 10); + + cluster_pull_name = config.getString(table_prefix + "cluster_pull"); + cluster_push_name = config.getString(table_prefix + "cluster_push"); + + table_pull.first = config.getString(table_prefix + "database_pull"); + table_pull.second = config.getString(table_prefix + "table_pull"); + + table_push.first = config.getString(table_prefix + "database_push"); + table_push.second = config.getString(table_prefix + "table_push"); + + /// Used as node name in ZooKeeper + table_id = escapeForFileName(cluster_push_name) + + "." + escapeForFileName(table_push.first) + + "." + escapeForFileName(table_push.second); + + engine_push_str = config.getString(table_prefix + "engine"); + { + ParserStorage parser_storage; + engine_push_ast = parseQuery(parser_storage, engine_push_str, 0); + engine_push_partition_key_ast = extractPartitionKey(engine_push_ast); + } + + sharding_key_str = config.getString(table_prefix + "sharding_key"); + { + ParserExpressionWithOptionalAlias parser_expression(false); + sharding_key_ast = parseQuery(parser_expression, sharding_key_str, 0); + engine_split_ast = createASTStorageDistributed(cluster_push_name, table_push.first, table_push.second, + sharding_key_ast); + } + + where_condition_str = config.getString(table_prefix + "where_condition", ""); + if (!where_condition_str.empty()) { + ParserExpressionWithOptionalAlias parser_expression(false); + where_condition_ast = parseQuery(parser_expression, where_condition_str, 0); + + // Will use canonical expression form + where_condition_str = queryToString(where_condition_ast); + } + + String enabled_partitions_prefix = table_prefix + "enabled_partitions"; + has_enabled_partitions = config.has(enabled_partitions_prefix); + + if (has_enabled_partitions) { + Strings keys; + config.keys(enabled_partitions_prefix, keys); + + if (keys.empty()) { + /// Parse list of partition from space-separated string + String partitions_str = config.getString(table_prefix + "enabled_partitions"); + boost::trim_if(partitions_str, isWhitespaceASCII); + boost::split(enabled_partitions, partitions_str, isWhitespaceASCII, boost::token_compress_on); + } else { + /// Parse sequence of ... + for (const String &key : keys) { + if (!startsWith(key, "partition")) + throw Exception("Unknown key " + key + " in " + enabled_partitions_prefix, + ErrorCodes::UNKNOWN_ELEMENT_IN_CONFIG); + + enabled_partitions.emplace_back(config.getString(enabled_partitions_prefix + "." + key)); + } + } + + std::copy(enabled_partitions.begin(), enabled_partitions.end(), + std::inserter(enabled_partitions_set, enabled_partitions_set.begin())); + } +} + +template +void TaskTable::initShards(RandomEngine &&random_engine) { + const String &fqdn_name = getFQDNOrHostName(); + std::uniform_int_distribution get_urand(0, std::numeric_limits::max()); + + // Compute the priority + for (auto &shard_info : cluster_pull->getShardsInfo()) { + TaskShardPtr task_shard = std::make_shared(*this, shard_info); + const auto &replicas = cluster_pull->getShardsAddresses().at(task_shard->indexInCluster()); + task_shard->priority = getReplicasPriority(replicas, fqdn_name, get_urand(random_engine)); + + all_shards.emplace_back(task_shard); + } + + // Sort by priority + std::sort(all_shards.begin(), all_shards.end(), + [](const TaskShardPtr &lhs, const TaskShardPtr &rhs) { + return ShardPriority::greaterPriority(lhs->priority, rhs->priority); + }); + + // Cut local shards + auto it_first_remote = std::lower_bound(all_shards.begin(), all_shards.end(), 1, + [](const TaskShardPtr &lhs, UInt8 is_remote) { + return lhs->priority.is_remote < is_remote; + }); + + local_shards.assign(all_shards.begin(), it_first_remote); +} + +} // end of anonymouse namespace + +} From 5795dfa00c7307a3b328c9cfe8e1e67589a7f802 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Thu, 13 Feb 2020 17:23:41 +0300 Subject: [PATCH 0021/1355] remove anonymous namespaces --- dbms/programs/copier/Internals.h | 5 ----- dbms/programs/copier/ShardPartition.h | 3 --- dbms/programs/copier/ShardPartitionPiece.h | 4 ---- dbms/programs/copier/TaskCluster.h | 2 -- dbms/programs/copier/TaskTable.h | 3 --- 5 files changed, 17 deletions(-) diff --git a/dbms/programs/copier/Internals.h b/dbms/programs/copier/Internals.h index 4b5cb4b7a83..dfb52dcfc34 100644 --- a/dbms/programs/copier/Internals.h +++ b/dbms/programs/copier/Internals.h @@ -81,9 +81,6 @@ namespace ErrorCodes extern const int UNKNOWN_ELEMENT_IN_CONFIG; } -namespace -{ - using DatabaseAndTableName = std::pair; String getQuotedTable(const String & database, const String & table) @@ -346,6 +343,4 @@ static ShardPriority getReplicasPriority(const Cluster::Addresses & replicas, co return res; } - -} // end of an anonymous namespace } diff --git a/dbms/programs/copier/ShardPartition.h b/dbms/programs/copier/ShardPartition.h index 2881d3e2f79..37d027eacef 100644 --- a/dbms/programs/copier/ShardPartition.h +++ b/dbms/programs/copier/ShardPartition.h @@ -5,8 +5,6 @@ namespace DB { -namespace { - /// Just destination partition of a shard /// I don't know what this comment means. @@ -90,6 +88,5 @@ String ShardPartition::getCommonPartitionIsCleanedPath() const { return getCommonPartitionIsDirtyPath() + "/cleaned"; } -} //end of anonymous namespace } diff --git a/dbms/programs/copier/ShardPartitionPiece.h b/dbms/programs/copier/ShardPartitionPiece.h index 867a5c4c65a..28f17a40541 100644 --- a/dbms/programs/copier/ShardPartitionPiece.h +++ b/dbms/programs/copier/ShardPartitionPiece.h @@ -4,8 +4,6 @@ namespace DB { -namespace { - struct ShardPartitionPiece { ShardPartitionPiece(ShardPartition &parent, size_t current_piece_number_, bool is_absent_piece_) @@ -78,6 +76,4 @@ String ShardPartitionPiece::getShardStatusPath() const } -} // end of anonymous namespace - } diff --git a/dbms/programs/copier/TaskCluster.h b/dbms/programs/copier/TaskCluster.h index 8cf67ad78fb..3ed7d610d17 100644 --- a/dbms/programs/copier/TaskCluster.h +++ b/dbms/programs/copier/TaskCluster.h @@ -4,7 +4,6 @@ namespace DB { -namespace { struct TaskCluster { TaskCluster(const String &task_zookeeper_path_, const String &default_local_database_) @@ -90,7 +89,6 @@ void TaskCluster::reloadSettings(const Poco::Util::AbstractConfiguration &config set_default_value(settings_push.insert_distributed_timeout, 0); } -} // end of anonymous namespace } diff --git a/dbms/programs/copier/TaskTable.h b/dbms/programs/copier/TaskTable.h index 04c8d747ea4..c8c292ea3a2 100644 --- a/dbms/programs/copier/TaskTable.h +++ b/dbms/programs/copier/TaskTable.h @@ -5,8 +5,6 @@ namespace DB { -namespace { - struct TaskTable { TaskTable(TaskCluster &parent, const Poco::Util::AbstractConfiguration &config, const String &prefix, @@ -238,6 +236,5 @@ void TaskTable::initShards(RandomEngine &&random_engine) { local_shards.assign(all_shards.begin(), it_first_remote); } -} // end of anonymouse namespace } From 93301b8f9f3b07b2d1630ce063fcf2fcbf5f0095 Mon Sep 17 00:00:00 2001 From: Avogar Date: Fri, 14 Feb 2020 22:48:45 +0300 Subject: [PATCH 0022/1355] Additional of new input format: RegexpRowInputFormat. --- dbms/src/Core/Settings.h | 4 + dbms/src/Formats/FormatFactory.cpp | 5 + dbms/src/Formats/FormatFactory.h | 2 + dbms/src/Formats/FormatSettings.h | 9 + .../Formats/Impl/RegexpRowInputFormat.cpp | 155 ++++++++++++++++++ .../Formats/Impl/RegexpRowInputFormat.h | 43 +++++ .../01079_regexp_input_format.reference | 12 ++ .../0_stateless/01079_regexp_input_format.sh | 27 +++ ...gexp_input_format_skip_unmatched.reference | 3 + ...1080_regexp_input_format_skip_unmatched.sh | 16 ++ 10 files changed, 276 insertions(+) create mode 100644 dbms/src/Processors/Formats/Impl/RegexpRowInputFormat.cpp create mode 100644 dbms/src/Processors/Formats/Impl/RegexpRowInputFormat.h create mode 100644 dbms/tests/queries/0_stateless/01079_regexp_input_format.reference create mode 100755 dbms/tests/queries/0_stateless/01079_regexp_input_format.sh create mode 100644 dbms/tests/queries/0_stateless/01080_regexp_input_format_skip_unmatched.reference create mode 100644 dbms/tests/queries/0_stateless/01080_regexp_input_format_skip_unmatched.sh diff --git a/dbms/src/Core/Settings.h b/dbms/src/Core/Settings.h index 08c555beb03..13e7f2ae70e 100644 --- a/dbms/src/Core/Settings.h +++ b/dbms/src/Core/Settings.h @@ -247,6 +247,10 @@ struct Settings : public SettingsCollection M(SettingString, format_custom_result_before_delimiter, "", "Prefix before result set (for CustomSeparated format)", 0) \ M(SettingString, format_custom_result_after_delimiter, "", "Suffix after result set (for CustomSeparated format)", 0) \ \ + M(SettingString, format_regexp, "", "Regular expression (for Regexp format)", 0) \ + M(SettingString, format_regexp_escaping_rule, "Escaped", "Field escaping rule (for Regexp format)", 0) \ + M(SettingBool, format_regexp_skip_unmatched, false, "Skip lines unmatched by regular expression (for Regexp format", 0) \ + \ M(SettingBool, insert_allow_materialized_columns, 0, "If setting is enabled, Allow materialized columns in INSERT.", 0) \ M(SettingSeconds, http_connection_timeout, DEFAULT_HTTP_READ_BUFFER_CONNECTION_TIMEOUT, "HTTP connection timeout.", 0) \ M(SettingSeconds, http_send_timeout, DEFAULT_HTTP_READ_BUFFER_TIMEOUT, "HTTP send timeout", 0) \ diff --git a/dbms/src/Formats/FormatFactory.cpp b/dbms/src/Formats/FormatFactory.cpp index a8e27054704..c2b890ec631 100644 --- a/dbms/src/Formats/FormatFactory.cpp +++ b/dbms/src/Formats/FormatFactory.cpp @@ -68,6 +68,9 @@ static FormatSettings getInputFormatSetting(const Settings & settings, const Con format_settings.custom.row_before_delimiter = settings.format_custom_row_before_delimiter; format_settings.custom.row_after_delimiter = settings.format_custom_row_after_delimiter; format_settings.custom.row_between_delimiter = settings.format_custom_row_between_delimiter; + format_settings.regexp.regexp = settings.format_regexp; + format_settings.regexp.escaping_rule = settings.format_regexp_escaping_rule; + format_settings.regexp.skip_unmatched = settings.format_regexp_skip_unmatched; /// Validate avro_schema_registry_url with RemoteHostFilter when non-empty and in Server context if (context.hasGlobalContext() && (context.getGlobalContext().getApplicationType() == Context::ApplicationType::SERVER)) @@ -352,10 +355,12 @@ FormatFactory::FormatFactory() registerOutputFormatProcessorAvro(*this); registerInputFormatProcessorTemplate(*this); registerOutputFormatProcessorTemplate(*this); + registerInputFormatProcessorRegexp(*this); registerFileSegmentationEngineTabSeparated(*this); registerFileSegmentationEngineCSV(*this); registerFileSegmentationEngineJSONEachRow(*this); + registerFileSegmentationEngineRegexp(*this); registerOutputFormatNull(*this); diff --git a/dbms/src/Formats/FormatFactory.h b/dbms/src/Formats/FormatFactory.h index 7c18971e0eb..7c515dbce90 100644 --- a/dbms/src/Formats/FormatFactory.h +++ b/dbms/src/Formats/FormatFactory.h @@ -178,6 +178,7 @@ void registerOutputFormatProcessorTemplate(FormatFactory &factory); void registerFileSegmentationEngineTabSeparated(FormatFactory & factory); void registerFileSegmentationEngineCSV(FormatFactory & factory); void registerFileSegmentationEngineJSONEachRow(FormatFactory & factory); +void registerFileSegmentationEngineRegexp(FormatFactory & factory); /// Output only (presentational) formats. @@ -198,5 +199,6 @@ void registerOutputFormatProcessorMySQLWrite(FormatFactory & factory); /// Input only formats. void registerInputFormatProcessorCapnProto(FormatFactory & factory); +void registerInputFormatProcessorRegexp(FormatFactory & factory); } diff --git a/dbms/src/Formats/FormatSettings.h b/dbms/src/Formats/FormatSettings.h index 610768e5d08..1eb95ce2dbf 100644 --- a/dbms/src/Formats/FormatSettings.h +++ b/dbms/src/Formats/FormatSettings.h @@ -122,6 +122,15 @@ struct FormatSettings Avro avro; + struct Regexp + { + std::string regexp; + std::string escaping_rule; + bool skip_unmatched = false; + }; + + Regexp regexp; + }; } diff --git a/dbms/src/Processors/Formats/Impl/RegexpRowInputFormat.cpp b/dbms/src/Processors/Formats/Impl/RegexpRowInputFormat.cpp new file mode 100644 index 00000000000..d53c6819c91 --- /dev/null +++ b/dbms/src/Processors/Formats/Impl/RegexpRowInputFormat.cpp @@ -0,0 +1,155 @@ +#include +#include +#include +#include +#include + +#include + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int INCORRECT_DATA; + extern const int BAD_ARGUMENTS; +} + +RegexpRowInputFormat::RegexpRowInputFormat( + ReadBuffer & in_, const Block & header_, Params params_, const FormatSettings & format_settings_) + : IRowInputFormat(header_, in_, std::move(params_)), format_settings(format_settings_), regexp(format_settings_.regexp.regexp) +{ + field_format = stringToFormat(format_settings_.regexp.escaping_rule); +} + +RegexpRowInputFormat::FieldFormat RegexpRowInputFormat::stringToFormat(const String & format) +{ + if (format == "Escaped") + return FieldFormat::Escaped; + if (format == "Quoted") + return FieldFormat::Quoted; + if (format == "CSV") + return FieldFormat::Csv; + if (format == "JSON") + return FieldFormat::Json; + throw Exception("Unknown field format \"" + format + "\".", ErrorCodes::BAD_ARGUMENTS); +} + +bool RegexpRowInputFormat::readField(size_t index, MutableColumns & columns) +{ + const auto & type = getPort().getHeader().getByPosition(index).type; + bool parse_as_nullable = format_settings.null_as_default && !type->isNullable(); + bool read = true; + ReadBuffer field_buf(matched_fields[index + 1].first, matched_fields[index + 1].length(), 0); + try + { + switch (field_format) + { + case FieldFormat::Escaped: + if (parse_as_nullable) + read = DataTypeNullable::deserializeTextEscaped(*columns[index], field_buf, format_settings, type); + else + type->deserializeAsTextEscaped(*columns[index], field_buf, format_settings); + break; + case FieldFormat::Quoted: + if (parse_as_nullable) + read = DataTypeNullable::deserializeTextQuoted(*columns[index], field_buf, format_settings, type); + else + type->deserializeAsTextQuoted(*columns[index], field_buf, format_settings); + break; + case FieldFormat::Csv: + if (parse_as_nullable) + read = DataTypeNullable::deserializeTextCSV(*columns[index], field_buf, format_settings, type); + else + type->deserializeAsTextCSV(*columns[index], field_buf, format_settings); + break; + case FieldFormat::Json: + if (parse_as_nullable) + read = DataTypeNullable::deserializeTextJSON(*columns[index], field_buf, format_settings, type); + else + type->deserializeAsTextJSON(*columns[index], field_buf, format_settings); + break; + default: + __builtin_unreachable(); + } + } + catch (Exception & e) + { + throw; + } + return read; +} + +void RegexpRowInputFormat::readFieldsFromMatch(MutableColumns & columns, RowReadExtension & ext) +{ + if (matched_fields.size() != columns.size() + 1) + throw Exception("The number of matched fields in line doesn't match the number of columns.", ErrorCodes::INCORRECT_DATA); + + ext.read_columns.assign(columns.size(), false); + for (size_t columns_index = 0; columns_index < columns.size(); ++columns_index) + { + ext.read_columns[columns_index] = readField(columns_index, columns); + } +} + +bool RegexpRowInputFormat::readRow(MutableColumns & columns, RowReadExtension & ext) +{ + if (in.eof()) + return false; + + char * line_end = find_first_symbols<'\n', '\r'>(in.position(), in.buffer().end()); + bool match = std::regex_match(in.position(), line_end, matched_fields, regexp); + + if (!match) + { + if (!format_settings.regexp.skip_unmatched) + throw Exception("Line \"" + std::string(in.position(), line_end) + "\" doesn't match the regexp.", ErrorCodes::INCORRECT_DATA); + in.position() = line_end + 1; + return true; + } + + readFieldsFromMatch(columns, ext); + + in.position() = line_end + 1; + return true; +} + +void registerInputFormatProcessorRegexp(FormatFactory & factory) +{ + factory.registerInputFormatProcessor("Regexp", []( + ReadBuffer & buf, + const Block & sample, + IRowInputFormat::Params params, + const FormatSettings & settings) + { + return std::make_shared(buf, sample, std::move(params), settings); + }); +} + +static bool fileSegmentationEngineRegexpImpl(ReadBuffer & in, DB::Memory<> & memory, size_t min_chunk_size) +{ + char * pos = in.position(); + bool need_more_data = true; + + while (loadAtPosition(in, memory, pos) && need_more_data) + { + pos = find_first_symbols<'\n', '\r'>(pos, in.buffer().end()); + if (pos == in.buffer().end()) + continue; + + if (memory.size() + static_cast(pos - in.position()) >= min_chunk_size) + need_more_data = false; + ++pos; + } + + saveUpToPosition(in, memory, pos); + + return loadAtPosition(in, memory, pos); +} + +void registerFileSegmentationEngineRegexp(FormatFactory & factory) +{ + factory.registerFileSegmentationEngine("Regexp", &fileSegmentationEngineRegexpImpl); +} + +} diff --git a/dbms/src/Processors/Formats/Impl/RegexpRowInputFormat.h b/dbms/src/Processors/Formats/Impl/RegexpRowInputFormat.h new file mode 100644 index 00000000000..fe920f26fed --- /dev/null +++ b/dbms/src/Processors/Formats/Impl/RegexpRowInputFormat.h @@ -0,0 +1,43 @@ +#pragma once + +#include +#include +#include +#include +#include + +namespace DB +{ + +class ReadBuffer; + + +class RegexpRowInputFormat : public IRowInputFormat +{ +public: + RegexpRowInputFormat(ReadBuffer & in_, const Block & header_, Params params_, const FormatSettings & format_settings_); + + String getName() const override { return "RegexpRowInputFormat"; } + + bool readRow(MutableColumns & columns, RowReadExtension & ext) override; + +private: + enum class FieldFormat + { + Escaped, + Quoted, + Csv, + Json, + }; + + bool readField(size_t index, MutableColumns & columns); + void readFieldsFromMatch(MutableColumns & columns, RowReadExtension & ext); + FieldFormat stringToFormat(const String & format); + + const FormatSettings format_settings; + std::regex regexp; + std::match_results matched_fields; + FieldFormat field_format; +}; + +} diff --git a/dbms/tests/queries/0_stateless/01079_regexp_input_format.reference b/dbms/tests/queries/0_stateless/01079_regexp_input_format.reference new file mode 100644 index 00000000000..61435bccefc --- /dev/null +++ b/dbms/tests/queries/0_stateless/01079_regexp_input_format.reference @@ -0,0 +1,12 @@ +1 [1,2,3] str1 2020-01-01 +2 [1,2,3] str2 2020-01-02 +3 [1,2,3] str3 2020-01-03 +4 [1,2,3] str4 2020-01-04 +5 [1,2,3] str5 2020-01-05 +6 [1,2,3] str6 2020-01-06 +7 [1,2,3] str7 2020-01-07 +8 [1,2,3] str8 2020-01-08 +9 [1,2,3] str9 2020-01-09 +10 [1,2,3] str10 2020-01-10 +11 [1,2,3] str11 2020-01-11 +12 [1,2,3] str12 2020-01-12 diff --git a/dbms/tests/queries/0_stateless/01079_regexp_input_format.sh b/dbms/tests/queries/0_stateless/01079_regexp_input_format.sh new file mode 100755 index 00000000000..cbaa1bd2162 --- /dev/null +++ b/dbms/tests/queries/0_stateless/01079_regexp_input_format.sh @@ -0,0 +1,27 @@ +#!/usr/bin/env bash#!/usr/bin/env bash + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +. $CURDIR/../shell_config.sh + +$CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS regexp"; +$CLICKHOUSE_CLIENT --query="CREATE TABLE regexp (id UInt32, array Array(UInt32), string String, date Date) ENGINE = Memory"; + +echo 'id: 1 array: [1,2,3] string: str1 date: 2020-01-01 +id: 2 array: [1,2,3] string: str2 date: 2020-01-02 +id: 3 array: [1,2,3] string: str3 date: 2020-01-03' | $CLICKHOUSE_CLIENT --query="INSERT INTO regexp FORMAT Regexp SETTINGS format_regexp='id: (.+?) array: (.+?) string: (.+?) date: (.+?)', format_regexp_escaping_rule='Escaped'"; + +echo 'id: 4 array: "[1,2,3]" string: "str4" date: "2020-01-04" +id: 5 array: "[1,2,3]" string: "str5" date: "2020-01-05" +id: 6 array: "[1,2,3]" string: "str6" date: "2020-01-06"' | $CLICKHOUSE_CLIENT --query="INSERT INTO regexp FORMAT Regexp SETTINGS format_regexp='id: (.+?) array: (.+?) string: (.+?) date: (.+?)', format_regexp_escaping_rule='CSV'"; + +echo "id: 7 array: [1,2,3] string: 'str7' date: '2020-01-07' +id: 8 array: [1,2,3] string: 'str8' date: '2020-01-08' +id: 9 array: [1,2,3] string: 'str9' date: '2020-01-09'" | $CLICKHOUSE_CLIENT --query="INSERT INTO regexp FORMAT Regexp SETTINGS format_regexp='id: (.+?) array: (.+?) string: (.+?) date: (.+?)', format_regexp_escaping_rule='Quoted'"; + +echo 'id: 10 array: [1,2,3] string: "str10" date: "2020-01-10" +id: 11 array: [1,2,3] string: "str11" date: "2020-01-11" +id: 12 array: [1,2,3] string: "str12" date: "2020-01-12"' | $CLICKHOUSE_CLIENT --query="INSERT INTO regexp FORMAT Regexp SETTINGS format_regexp='id: (.+?) array: (.+?) string: (.+?) date: (.+?)', format_regexp_escaping_rule='JSON'"; + +$CLICKHOUSE_CLIENT --query="SELECT * FROM regexp ORDER BY id"; +$CLICKHOUSE_CLIENT --query="DROP TABLE regexp"; + diff --git a/dbms/tests/queries/0_stateless/01080_regexp_input_format_skip_unmatched.reference b/dbms/tests/queries/0_stateless/01080_regexp_input_format_skip_unmatched.reference new file mode 100644 index 00000000000..ff57df7316e --- /dev/null +++ b/dbms/tests/queries/0_stateless/01080_regexp_input_format_skip_unmatched.reference @@ -0,0 +1,3 @@ +1 str1 +2 str2 +4 str4 diff --git a/dbms/tests/queries/0_stateless/01080_regexp_input_format_skip_unmatched.sh b/dbms/tests/queries/0_stateless/01080_regexp_input_format_skip_unmatched.sh new file mode 100644 index 00000000000..98bcb0a256a --- /dev/null +++ b/dbms/tests/queries/0_stateless/01080_regexp_input_format_skip_unmatched.sh @@ -0,0 +1,16 @@ +#!/usr/bin/env bash + +# CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# . $CURDIR/../shell_config.sh + +$CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS regexp"; +$CLICKHOUSE_CLIENT --query="CREATE TABLE regexp (id UInt32, string String) ENGINE = Memory"; + +echo 'id: 1 string: str1 +id: 2 string: str2 +id=3, string=str3 +id: 4 string: str4' | $CLICKHOUSE_CLIENT --query="INSERT INTO regexp FORMAT Regexp SETTINGS format_regexp='id: (.+?) string: (.+?)', format_regexp_escaping_rule='Escaped', format_regexp_skip_unmatched=1"; + +$CLICKHOUSE_CLIENT --query="SELECT * FROM regexp"; +$CLICKHOUSE_CLIENT --query="DROP TABLE regexp"; + From 038c9a889d5a6ac52582f0461f5f0e50d284295d Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Tue, 18 Feb 2020 16:26:08 +0300 Subject: [PATCH 0023/1355] something works --- dbms/programs/copier/ClusterCopier.h | 219 +++++++++++++++++---- dbms/programs/copier/Internals.h | 73 +++++++ dbms/programs/copier/ShardPartitionPiece.h | 2 +- dbms/programs/copier/TaskCluster.h | 2 +- dbms/programs/copier/TaskTable.h | 109 +++++++--- 5 files changed, 342 insertions(+), 63 deletions(-) diff --git a/dbms/programs/copier/ClusterCopier.h b/dbms/programs/copier/ClusterCopier.h index 8cc0cf21211..cd0481684dc 100644 --- a/dbms/programs/copier/ClusterCopier.h +++ b/dbms/programs/copier/ClusterCopier.h @@ -273,7 +273,8 @@ public: for (TaskTable & task_table : task_cluster->table_tasks) { LOG_INFO(log, "Process table task " << task_table.table_id << " with " - << task_table.all_shards.size() << " shards, " << task_table.local_shards.size() << " of them are local ones"); + << task_table.all_shards.size() << " shards, " + << task_table.local_shards.size() << " of them are local ones"); if (task_table.all_shards.empty()) continue; @@ -507,7 +508,7 @@ public: catch (const Coordination::Exception & e) { LOG_INFO(log, "A ZooKeeper error occurred while checking partition " << partition_name - << ". Will recheck the partition. Error: " << e.displayText()); + << ". Will recheck the partition. Error: " << e.displayText()); return false; } @@ -525,6 +526,112 @@ public: return true; } + bool checkAllPieceInPartitionDone(const TaskTable & task_table, const String & partition_name, const TasksShard & shards_with_partition) + { + bool answer = true; + for (size_t piece_number = 0; piece_number < task_table.number_of_splits; piece_number++) + answer &= checkPartitionPieceIsDone(task_table, partition_name, piece_number, shards_with_partition); + return answer; + } + + + /* The same as function above + * Assume that we don't know on which shards do we have partition certain piece. + * We'll check them all (I mean shards that contain the whole partition) + * And shards that don't have certain piece MUST mark that piece is_done true. + * */ + bool checkPartitionPieceIsDone(const TaskTable & task_table, const String & partition_name, + size_t piece_number, const TasksShard & shards_with_partition) + { + LOG_DEBUG(log, "Check that all shards processed partition " << partition_name + << " piece number" + toString(piece_number) + " successfully"); + + auto zookeeper = context.getZooKeeper(); + + /// Collect all shards that contain partition piece number piece_number. + Strings piece_status_paths; + for (auto & shard : shards_with_partition) + { + ShardPartition & task_shard_partition = shard->partition_tasks.find(partition_name)->second; + ShardPartitionPiece & shard_partition_piece = task_shard_partition.pieces[piece_number]; + piece_status_paths.emplace_back(shard_partition_piece.getShardStatusPath()); + } + + std::vector zxid1, zxid2; + + try + { + std::vector get_futures; + for (const String & path : piece_status_paths) + get_futures.emplace_back(zookeeper->asyncGet(path)); + + // Check that state is Finished and remember zxid + for (auto & future : get_futures) + { + auto res = future.get(); + + TaskStateWithOwner status = TaskStateWithOwner::fromString(res.data); + if (status.state != TaskState::Finished) + { + LOG_INFO(log, "The task " << res.data << " is being rewritten by " << status.owner << ". Partition piece will be rechecked"); + return false; + } + + zxid1.push_back(res.stat.pzxid); + } + + // Check that partition is not dirty + { + CleanStateClock clean_state_clock ( + zookeeper, + task_table.getCertainPartitionIsDirtyPath(partition_name), + task_table.getCertainPartitionIsCleanedPath(partition_name) + ); + Coordination::Stat stat{}; + LogicalClock task_start_clock; + if (zookeeper->exists(task_table.getCertainPartitionTaskStatusPath(partition_name), &stat)) + task_start_clock = LogicalClock(stat.mzxid); + zookeeper->get(task_table.getCertainPartitionTaskStatusPath(partition_name), &stat); + if (!clean_state_clock.is_clean() || task_start_clock <= clean_state_clock.discovery_zxid) + { + LOG_INFO(log, "Partition " << partition_name << " piece number " << toString(piece_number) << " become dirty"); + return false; + } + } + + get_futures.clear(); + for (const String & path : piece_status_paths) + get_futures.emplace_back(zookeeper->asyncGet(path)); + + // Remember zxid of states again + for (auto & future : get_futures) + { + auto res = future.get(); + zxid2.push_back(res.stat.pzxid); + } + } + catch (const Coordination::Exception & e) + { + LOG_INFO(log, "A ZooKeeper error occurred while checking partition " << partition_name << " piece number " + << toString(piece_number) << ". Will recheck the partition. Error: " << e.displayText()); + return false; + } + + // If all task is finished and zxid is not changed then partition could not become dirty again + for (UInt64 shard_num = 0; shard_num < piece_status_paths.size(); ++shard_num) + { + if (zxid1[shard_num] != zxid2[shard_num]) + { + LOG_INFO(log, "The task " << piece_status_paths[shard_num] << " is being modified now. Partition piece will be rechecked"); + return false; + } + } + + LOG_INFO(log, "Partition " << partition_name << " is copied successfully"); + return true; + + } + /// Removes MATERIALIZED and ALIAS columns from create table query static ASTPtr removeAliasColumnsFromCreateQuery(const ASTPtr & query_ast) { @@ -877,6 +984,14 @@ public: Error, }; + + enum class PartititonPieceTaskStatus + { + Active, + Finished, + Error, + }; + /// Job for copying partition from particular shard. PartitionTaskStatus tryProcessPartitionTask(const ConnectionTimeouts & timeouts, ShardPartition & task_partition, bool is_unprioritized_task) { @@ -884,7 +999,7 @@ public: try { - res = processPartitionTaskImpl(timeouts, task_partition, is_unprioritized_task); + res = iterateThroughAllPiecesInPartition(timeouts, task_partition, is_unprioritized_task); } catch (...) { @@ -905,23 +1020,40 @@ public: return res; } - PartitionTaskStatus processPartitionTaskImpl(const ConnectionTimeouts & timeouts, ShardPartition & task_partition, bool is_unprioritized_task) + PartitionTaskStatus iterateThroughAllPiecesInPartition(const ConnectionTimeouts & timeouts, ShardPartition & task_partition, + bool is_unprioritized_task) + { + const size_t total_number_of_pieces = task_partition.task_shard.task_table.number_of_splits; + + /// ThreadPool maybe ?? + for (size_t piece_number = 0; piece_number < total_number_of_pieces; piece_number++) + processPartitionPieceTaskImpl(timeouts, task_partition, piece_number, is_unprioritized_task); + + return PartitionTaskStatus::Finished; + + } + + PartitionTaskStatus processPartitionPieceTaskImpl(const ConnectionTimeouts & timeouts, ShardPartition & task_partition, + const size_t current_piece_number, bool is_unprioritized_task) { TaskShard & task_shard = task_partition.task_shard; TaskTable & task_table = task_shard.task_table; ClusterPartition & cluster_partition = task_table.getClusterPartition(task_partition.name); - const size_t number_of_splits = task_table.number_of_splits; - UNUSED(number_of_splits); + ShardPartitionPiece & partition_piece = task_partition.pieces[current_piece_number]; + const size_t number_of_splits = task_table.number_of_splits; + const String primary_key_comma_separated = task_table.primary_key_comma_separated; + UNUSED(number_of_splits); + UNUSED(partition_piece); /// We need to update table definitions for each partition, it could be changed after ALTER - createShardInternalTables(timeouts, task_shard); + createShardInternalTables(timeouts, task_shard, true, current_piece_number); auto zookeeper = context.getZooKeeper(); - const String is_dirty_flag_path = task_partition.getCommonPartitionIsDirtyPath(); - const String is_dirty_cleaned_path = task_partition.getCommonPartitionIsCleanedPath(); - const String current_task_is_active_path = task_partition.getActiveWorkerPath(); - const String current_task_status_path = task_partition.getShardStatusPath(); + const String piece_is_dirty_flag_path = partition_piece.getPartitionPieceIsDirtyPath(); + const String piece_is_dirty_cleaned_path = partition_piece.getPartitionPieceIsCleanedPath(); + const String current_task_piece_is_active_path = partition_piece.getActiveWorkerPath(); + const String current_task_piece_status_path = partition_piece.getShardStatusPath(); /// Auxiliary functions: @@ -935,25 +1067,27 @@ public: else if (clock.discovery_version) { LOG_DEBUG(log, "Updating clean state clock"); - zookeeper->set(is_dirty_flag_path, host_id, clock.discovery_version.value()); + zookeeper->set(piece_is_dirty_flag_path, host_id, clock.discovery_version.value()); } else { LOG_DEBUG(log, "Creating clean state clock"); - zookeeper->create(is_dirty_flag_path, host_id, zkutil::CreateMode::Persistent); + zookeeper->create(piece_is_dirty_flag_path, host_id, zkutil::CreateMode::Persistent); } }; /// Returns SELECT query filtering current partition and applying user filter auto get_select_query = [&] (const DatabaseAndTableName & from_table, const String & fields, String limit = "", - bool enable_splitting = false, size_t current_piece_number = 0) + bool enable_splitting = false) { String query; query += "SELECT " + fields + " FROM " + getQuotedTable(from_table); /// TODO: Bad, it is better to rewrite with ASTLiteral(partition_key_field) query += " WHERE (" + queryToString(task_table.engine_push_partition_key_ast) + " = (" + task_partition.name + " AS partition_key))"; + if (enable_splitting) - query += " AND ( cityHash64(*) = " + std::to_string(current_piece_number) + " )"; + query += " AND ( cityHash64(" + primary_key_comma_separated + ") = " + toString(current_piece_number) + " )"; + if (!task_table.where_condition_str.empty()) query += " AND (" + task_table.where_condition_str + ")"; if (!limit.empty()) @@ -964,11 +1098,11 @@ public: }; /// Load balancing - auto worker_node_holder = createTaskWorkerNodeAndWaitIfNeed(zookeeper, current_task_status_path, is_unprioritized_task); + auto worker_node_holder = createTaskWorkerNodeAndWaitIfNeed(zookeeper, current_task_piece_status_path, is_unprioritized_task); - LOG_DEBUG(log, "Processing " << current_task_status_path); + LOG_DEBUG(log, "Processing " << current_task_piece_status_path); - CleanStateClock clean_state_clock (zookeeper, is_dirty_flag_path, is_dirty_cleaned_path); + CleanStateClock clean_state_clock (zookeeper, piece_is_dirty_flag_path, piece_is_dirty_cleaned_path); LogicalClock task_start_clock; { @@ -982,7 +1116,7 @@ public: && (!task_start_clock.hasHappened() || clean_state_clock.discovery_zxid <= task_start_clock)) { LOG_DEBUG(log, "Partition " << task_partition.name << " appears to be clean"); - zookeeper->createAncestors(current_task_status_path); + zookeeper->createAncestors(current_task_piece_status_path); } else { @@ -1001,17 +1135,17 @@ public: } /// Create ephemeral node to mark that we are active and process the partition - zookeeper->createAncestors(current_task_is_active_path); + zookeeper->createAncestors(current_task_piece_is_active_path); zkutil::EphemeralNodeHolderPtr partition_task_node_holder; try { - partition_task_node_holder = zkutil::EphemeralNodeHolder::create(current_task_is_active_path, *zookeeper, host_id); + partition_task_node_holder = zkutil::EphemeralNodeHolder::create(current_task_piece_is_active_path, *zookeeper, host_id); } catch (const Coordination::Exception & e) { if (e.code == Coordination::ZNODEEXISTS) { - LOG_DEBUG(log, "Someone is already processing " << current_task_is_active_path); + LOG_DEBUG(log, "Someone is already processing " << current_task_piece_is_active_path); return PartitionTaskStatus::Active; } @@ -1022,17 +1156,17 @@ public: /// create blocking node to signal cleaning up if it is abandoned { String status_data; - if (zookeeper->tryGet(current_task_status_path, status_data)) + if (zookeeper->tryGet(current_task_piece_status_path, status_data)) { TaskStateWithOwner status = TaskStateWithOwner::fromString(status_data); if (status.state == TaskState::Finished) { - LOG_DEBUG(log, "Task " << current_task_status_path << " has been successfully executed by " << status.owner); + LOG_DEBUG(log, "Task " << current_task_piece_status_path << " has been successfully executed by " << status.owner); return PartitionTaskStatus::Finished; } // Task is abandoned, initialize DROP PARTITION - LOG_DEBUG(log, "Task " << current_task_status_path << " has not been successfully finished by " << + LOG_DEBUG(log, "Task " << current_task_piece_status_path << " has not been successfully finished by " << status.owner << ". Partition will be dropped and refilled."); create_is_dirty_node(clean_state_clock); @@ -1064,7 +1198,7 @@ public: if (count != 0) { Coordination::Stat stat_shards{}; - zookeeper->get(task_partition.getPartitionShardsPath(), &stat_shards); + zookeeper->get(task_partition.getPartitionShardsPath(), & stat_shards); /// NOTE: partition is still fresh if dirt discovery happens before cleaning if (stat_shards.numChildren == 0) @@ -1085,7 +1219,7 @@ public: /// Try start processing, create node about it { String start_state = TaskStateWithOwner::getData(TaskState::Started, host_id); - CleanStateClock new_clean_state_clock (zookeeper, is_dirty_flag_path, is_dirty_cleaned_path); + CleanStateClock new_clean_state_clock (zookeeper, piece_is_dirty_flag_path, piece_is_dirty_cleaned_path); if (clean_state_clock != new_clean_state_clock) { LOG_INFO(log, "Partition " << task_partition.name << " clean state changed, cowardly bailing"); @@ -1097,13 +1231,18 @@ public: create_is_dirty_node(new_clean_state_clock); return PartitionTaskStatus::Error; } - zookeeper->create(current_task_status_path, start_state, zkutil::CreateMode::Persistent); + zookeeper->create(current_task_piece_status_path, start_state, zkutil::CreateMode::Persistent); } /// Try create table (if not exists) on each shard { + /// Define push table for current partition piece + auto database_and_table_for_current_piece= std::pair( + task_table.table_push.first, + task_table.table_push.second + ".piece_" + toString(current_piece_number)); + auto create_query_push_ast = rewriteCreateQueryStorage(task_shard.current_pull_table_create_query, - task_table.table_push, task_table.engine_push_ast); + database_and_table_for_current_piece, task_table.engine_push_ast); create_query_push_ast->as().if_not_exists = true; String query = queryToString(create_query_push_ast); @@ -1173,7 +1312,7 @@ public: throw Exception("ZooKeeper session is expired, cancel INSERT SELECT", ErrorCodes::UNFINISHED); if (!future_is_dirty_checker.valid()) - future_is_dirty_checker = zookeeper->asyncExists(is_dirty_flag_path); + future_is_dirty_checker = zookeeper->asyncExists(piece_is_dirty_flag_path); /// check_period_milliseconds should less than average insert time of single block /// Otherwise, the insertion will slow a little bit @@ -1222,7 +1361,7 @@ public: /// Finalize the processing, change state of current partition task (and also check is_dirty flag) { String state_finished = TaskStateWithOwner::getData(TaskState::Finished, host_id); - CleanStateClock new_clean_state_clock (zookeeper, is_dirty_flag_path, is_dirty_cleaned_path); + CleanStateClock new_clean_state_clock (zookeeper, piece_is_dirty_flag_path, piece_is_dirty_cleaned_path); if (clean_state_clock != new_clean_state_clock) { LOG_INFO(log, "Partition " << task_partition.name << " clean state changed, cowardly bailing"); @@ -1234,7 +1373,7 @@ public: create_is_dirty_node(new_clean_state_clock); return PartitionTaskStatus::Error; } - zookeeper->set(current_task_status_path, state_finished, 0); + zookeeper->set(current_task_piece_status_path, state_finished, 0); } LOG_INFO(log, "Partition " << task_partition.name << " copied"); @@ -1283,7 +1422,8 @@ public: return parseQuery(parser_create_query, create_query_pull_str, 0); } - void createShardInternalTables(const ConnectionTimeouts & timeouts, TaskShard & task_shard, bool create_split = true) + /// If it is implicitly asked to create split Distributed table for certain piece on current shard, we will do it. + void createShardInternalTables(const ConnectionTimeouts & timeouts, TaskShard & task_shard, bool create_split = true, const size_t piece_number = 0) { TaskTable & task_table = task_shard.task_table; @@ -1295,7 +1435,8 @@ public: String read_shard_prefix = ".read_shard_" + toString(task_shard.indexInCluster()) + "."; String split_shard_prefix = ".split."; task_shard.table_read_shard = DatabaseAndTableName(working_database_name, read_shard_prefix + task_table.table_id); - task_shard.table_split_shard = DatabaseAndTableName(working_database_name, split_shard_prefix + task_table.table_id); + task_shard.table_split_shard = DatabaseAndTableName( + working_database_name, split_shard_prefix + task_table.table_id + ".piece_" + toString(piece_number)); /// Create special cluster with single shard String shard_read_cluster_name = read_shard_prefix + task_table.cluster_pull_name; @@ -1303,16 +1444,16 @@ public: context.setCluster(shard_read_cluster_name, cluster_pull_current_shard); auto storage_shard_ast = createASTStorageDistributed(shard_read_cluster_name, task_table.table_pull.first, task_table.table_pull.second); - const auto & storage_split_ast = task_table.engine_split_ast; + const auto & storage_piece_split_ast = task_table.auxiliary_engine_split_asts[piece_number]; auto create_query_ast = removeAliasColumnsFromCreateQuery(task_shard.current_pull_table_create_query); auto create_table_pull_ast = rewriteCreateQueryStorage(create_query_ast, task_shard.table_read_shard, storage_shard_ast); - auto create_table_split_ast = rewriteCreateQueryStorage(create_query_ast, task_shard.table_split_shard, storage_split_ast); + auto create_table_split_piece_ast = rewriteCreateQueryStorage(create_query_ast, task_shard.table_split_shard, storage_piece_split_ast); dropAndCreateLocalTable(create_table_pull_ast); if (create_split) - dropAndCreateLocalTable(create_table_split_ast); + dropAndCreateLocalTable(create_table_split_piece_ast); } @@ -1396,8 +1537,10 @@ public: + " WHERE (" + queryToString(task_table.engine_push_partition_key_ast) + " = (" + partition_quoted_name + " AS partition_key))"; const size_t number_of_splits = task_table.number_of_splits; + const String & primary_key_comma_separated = task_table.primary_key_comma_separated; - query += " AND (cityHash64(*) % " + std::to_string(number_of_splits) + " = " + std::to_string(current_piece_number) + " )"; + query += " AND (cityHash64(" + primary_key_comma_separated + ") % " + + std::to_string(number_of_splits) + " = " + std::to_string(current_piece_number) + " )"; if (!task_table.where_condition_str.empty()) query += " AND (" + task_table.where_condition_str + ")"; diff --git a/dbms/programs/copier/Internals.h b/dbms/programs/copier/Internals.h index dfb52dcfc34..d4f565b63df 100644 --- a/dbms/programs/copier/Internals.h +++ b/dbms/programs/copier/Internals.h @@ -313,6 +313,79 @@ static ASTPtr extractPartitionKey(const ASTPtr & storage_ast) } } +/* + * Choosing a Primary Key that Differs from the Sorting Key + * It is possible to specify a primary key (an expression with values that are written in the index file for each mark) + * that is different from the sorting key (an expression for sorting the rows in data parts). + * In this case the primary key expression tuple must be a prefix of the sorting key expression tuple. + * This feature is helpful when using the SummingMergeTree and AggregatingMergeTree table engines. + * In a common case when using these engines, the table has two types of columns: dimensions and measures. + * Typical queries aggregate values of measure columns with arbitrary GROUP BY and filtering by dimensions. + * Because SummingMergeTree and AggregatingMergeTree aggregate rows with the same value of the sorting key, + * it is natural to add all dimensions to it. As a result, the key expression consists of a long list of columns + * and this list must be frequently updated with newly added dimensions. + * In this case it makes sense to leave only a few columns in the primary key that will provide efficient + * range scans and add the remaining dimension columns to the sorting key tuple. + * ALTER of the sorting key is a lightweight operation because when a new column is simultaneously added t + * o the table and to the sorting key, existing data parts don't need to be changed. + * Since the old sorting key is a prefix of the new sorting key and there is no data in the newly added column, + * the data is sorted by both the old and new sorting keys at the moment of table modification. + * + * */ +[[maybe_unused]] static ASTPtr extractPrimaryKeyOrOrderBy(const ASTPtr & storage_ast) +{ + String storage_str = queryToString(storage_ast); + + const auto & storage = storage_ast->as(); + const auto & engine = storage.engine->as(); + + if (!endsWith(engine.name, "MergeTree")) + { + throw Exception("Unsupported engine was specified in " + storage_str + ", only *MergeTree engines are supported", + ErrorCodes::BAD_ARGUMENTS); + } + + /// FIXME + if (!isExtendedDefinitionStorage(storage_ast)) + { + throw Exception("Is not extended deginition storage " + storage_str + " Will be fixed later.", + ErrorCodes::BAD_ARGUMENTS); + } + + if (storage.primary_key) + return storage.primary_key->clone(); + + return storage.order_by->clone(); +} + +[[maybe_unused]] static String createCommaSeparatedStringFrom(const Strings & strings) +{ + String answer; + for (auto & string: strings) + answer += string + ", "; + + /// Remove last comma and space + answer.pop_back(); + answer.pop_back(); + return answer; +} + +[[maybe_unused]] static Strings extractPrimaryKeyString(const ASTPtr & storage_ast) +{ + const auto primary_key_or_order_by = extractPrimaryKeyOrOrderBy(storage_ast)->as(); + + ASTPtr primary_key_or_order_by_arguments_ast = primary_key_or_order_by.arguments->clone(); + ASTs & primary_key_or_order_by_arguments = primary_key_or_order_by_arguments_ast->children; + + Strings answer; + answer.reserve(primary_key_or_order_by_arguments.size()); + + for (auto & column : primary_key_or_order_by_arguments) + answer.push_back(column->getColumnName()); + + return answer; +} + static ShardPriority getReplicasPriority(const Cluster::Addresses & replicas, const std::string & local_hostname, UInt8 random) diff --git a/dbms/programs/copier/ShardPartitionPiece.h b/dbms/programs/copier/ShardPartitionPiece.h index 28f17a40541..303407d1d5b 100644 --- a/dbms/programs/copier/ShardPartitionPiece.h +++ b/dbms/programs/copier/ShardPartitionPiece.h @@ -36,7 +36,7 @@ struct ShardPartitionPiece { String ShardPartitionPiece::getPartitionPiecePath() const { - return shard_partition.getPartitionPath() + "/piece" + std::to_string(current_piece_number); + return shard_partition.getPartitionPath() + "/piece_" + std::to_string(current_piece_number); } String ShardPartitionPiece::getPartitionPieceCleanStartPath() const diff --git a/dbms/programs/copier/TaskCluster.h b/dbms/programs/copier/TaskCluster.h index 3ed7d610d17..0df6f7e6a6b 100644 --- a/dbms/programs/copier/TaskCluster.h +++ b/dbms/programs/copier/TaskCluster.h @@ -53,7 +53,7 @@ void TaskCluster::loadTasks(const Poco::Util::AbstractConfiguration &config, con Poco::Util::AbstractConfiguration::Keys tables_keys; config.keys(prefix + "tables", tables_keys); - for (const auto &table_key : tables_keys) { + for (const auto & table_key : tables_keys) { table_tasks.emplace_back(*this, config, prefix + "tables", table_key); } } diff --git a/dbms/programs/copier/TaskTable.h b/dbms/programs/copier/TaskTable.h index c8c292ea3a2..10aaf9334a6 100644 --- a/dbms/programs/copier/TaskTable.h +++ b/dbms/programs/copier/TaskTable.h @@ -2,26 +2,50 @@ #include "Internals.h" #include "TaskCluster.h" +#include "ext/range.h" namespace DB { struct TaskTable { - TaskTable(TaskCluster &parent, const Poco::Util::AbstractConfiguration &config, const String &prefix, - const String &table_key); + TaskTable(TaskCluster & parent, const Poco::Util::AbstractConfiguration & config, const String & prefix, + const String & table_key); - TaskCluster &task_cluster; + TaskCluster & task_cluster; - String getPartitionPath(const String &partition_name) const; + /// These functions used in checkPartitionIsDone() or checkPartitionPieceIsDone() + /// They are implemented here not to call task_table.tasks_shard[partition_name].second.pieces[current_piece_number] etc. - [[maybe_unused]] String - getPartitionPiecePath(const String &partition_name, size_t current_piece_number) const; + String getPartitionPath(const String & partition_name) const; - String getCertainPartitionIsDirtyPath(const String &partition_name) const; + [[maybe_unused]] String getPartitionPiecePath(const String & partition_name, const size_t piece_number) const; - String getCertainPartitionIsCleanedPath(const String &partition_name) const; + String getCertainPartitionIsDirtyPath(const String & partition_name) const; - String getCertainPartitionTaskStatusPath(const String &partition_name) const; + [[maybe_unused]] String getCertainPartitionPieceIsDirtyPath(const String & partition_name, const size_t piece_number) const + { + UNUSED(partition_name); + UNUSED(piece_number); + return "Not Implemented"; + } + + String getCertainPartitionIsCleanedPath(const String & partition_name) const; + + [[maybe_unused]] String getCertainPartitionPieceIsCleanedPath(const String & partition_name, const size_t piece_number) const + { + UNUSED(partition_name); + UNUSED(piece_number); + return "Not implemented"; + } + + String getCertainPartitionTaskStatusPath(const String & partition_name) const; + + [[maybe_unused]] String getCertainPartitionPieceTaskStatusPath(const String & partition_name, const size_t piece_number) const + { + UNUSED(partition_name); + UNUSED(piece_number); + return "Not implemented"; + } /// Partitions will be splitted into number-of-splits pieces. /// Each piece will be copied independently. (10 by default) @@ -32,6 +56,9 @@ struct TaskTable { /// Used as task ID String table_id; + /// Column names in primary key + String primary_key_comma_separated; + /// Source cluster and table String cluster_pull_name; DatabaseAndTableName table_pull; @@ -41,14 +68,37 @@ struct TaskTable { DatabaseAndTableName table_push; /// Storage of destination table + /// (tables that are stored on each shard of target cluster) String engine_push_str; ASTPtr engine_push_ast; ASTPtr engine_push_partition_key_ast; - /// A Distributed table definition used to split data + /* + * A Distributed table definition used to split data + * Distributed table will be created on each shard of default + * cluster to perform data copying and resharding + * */ String sharding_key_str; ASTPtr sharding_key_ast; - ASTPtr engine_split_ast; + ASTPtr main_engine_split_ast; + + + /* + * Auxuliary table engines used to perform partition piece copying. + * Each AST represent table engine for certatin piece number. + * After copying partition piece is Ok, this piece will be moved to the main + * target table. All this tables are stored on each shard as the main table. + * We have to use separate tables for partition pieces because of the atomicity of copying. + * Also if we want to move some partition to another table, the partition keys have to be the same. + * */ + + + /* + * To copy partiton piece form one cluster to another we have to use Distributed table. + * In case of usage separate table (engine_push) for each partiton piece, + * we have to use many Distributed tables. + * */ + ASTs auxiliary_engine_split_asts; /// Additional WHERE expression to filter input data String where_condition_str; @@ -63,9 +113,11 @@ struct TaskTable { Strings enabled_partitions; NameSet enabled_partitions_set; - /// Prioritized list of shards - /// all_shards contains information about all shards in the table. - /// So we have to check whether particular shard have current partiton or not while processing. + /** + * Prioritized list of shards + * all_shards contains information about all shards in the table. + * So we have to check whether particular shard have current partiton or not while processing. + */ TasksShard all_shards; TasksShard local_shards; @@ -76,7 +128,7 @@ struct TaskTable { /// Parition names to process in user-specified order Strings ordered_partition_names; - ClusterPartition &getClusterPartition(const String &partition_name) { + ClusterPartition & getClusterPartition(const String &partition_name) { auto it = cluster_partitions.find(partition_name); if (it == cluster_partitions.end()) throw Exception("There are no cluster partition " + partition_name + " in " + table_id, @@ -99,10 +151,10 @@ String TaskTable::getPartitionPath(const String &partition_name) const { + "/" + escapeForFileName(partition_name); // 201701 } -String TaskTable::getPartitionPiecePath(const String &partition_name, size_t current_piece_number) const { - assert(current_piece_number < number_of_splits); +String TaskTable::getPartitionPiecePath(const String & partition_name, size_t piece_number) const { + assert(piece_number < number_of_splits); return getPartitionPath(partition_name) + "/" + - std::to_string(current_piece_number); // 1...number_of_splits + std::to_string(piece_number); // 1...number_of_splits } String TaskTable::getCertainPartitionIsDirtyPath(const String &partition_name) const { @@ -131,9 +183,8 @@ String TaskShard::getHostNameExample() const { return replicas.at(0).readableString(); } -TaskTable::TaskTable(TaskCluster &parent, const Poco::Util::AbstractConfiguration &config, - const String &prefix_, - const String &table_key) +TaskTable::TaskTable(TaskCluster & parent, const Poco::Util::AbstractConfiguration & config, + const String & prefix_, const String & table_key) : task_cluster(parent) { String table_prefix = prefix_ + "." + table_key + "."; @@ -160,14 +211,26 @@ TaskTable::TaskTable(TaskCluster &parent, const Poco::Util::AbstractConfiguratio ParserStorage parser_storage; engine_push_ast = parseQuery(parser_storage, engine_push_str, 0); engine_push_partition_key_ast = extractPartitionKey(engine_push_ast); + primary_key_comma_separated = createCommaSeparatedStringFrom(extractPrimaryKeyString(engine_push_ast)); } sharding_key_str = config.getString(table_prefix + "sharding_key"); + + auxiliary_engine_split_asts.reserve(number_of_splits); { ParserExpressionWithOptionalAlias parser_expression(false); sharding_key_ast = parseQuery(parser_expression, sharding_key_str, 0); - engine_split_ast = createASTStorageDistributed(cluster_push_name, table_push.first, table_push.second, + main_engine_split_ast = createASTStorageDistributed(cluster_push_name, table_push.first, table_push.second, sharding_key_ast); + + for (const auto piece_number : ext::range(0, number_of_splits)) + { + auxiliary_engine_split_asts.emplace_back + ( + createASTStorageDistributed(cluster_push_name, table_push.first, + table_push.second + ".piece_" + toString(piece_number), sharding_key_ast) + ); + } } where_condition_str = config.getString(table_prefix + "where_condition", ""); @@ -209,7 +272,7 @@ TaskTable::TaskTable(TaskCluster &parent, const Poco::Util::AbstractConfiguratio template void TaskTable::initShards(RandomEngine &&random_engine) { - const String &fqdn_name = getFQDNOrHostName(); + const String & fqdn_name = getFQDNOrHostName(); std::uniform_int_distribution get_urand(0, std::numeric_limits::max()); // Compute the priority From 6527df83a1997d97e73b477c3320c467e8cc5be6 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Tue, 18 Feb 2020 16:39:22 +0300 Subject: [PATCH 0024/1355] copier split --- dbms/programs/copier/ClusterCopier.cpp | 1615 ++++++++++++++++++++++++ dbms/programs/copier/ClusterCopier.h | 1573 +---------------------- 2 files changed, 1649 insertions(+), 1539 deletions(-) create mode 100644 dbms/programs/copier/ClusterCopier.cpp diff --git a/dbms/programs/copier/ClusterCopier.cpp b/dbms/programs/copier/ClusterCopier.cpp new file mode 100644 index 00000000000..684bfddf7d4 --- /dev/null +++ b/dbms/programs/copier/ClusterCopier.cpp @@ -0,0 +1,1615 @@ +#include "ClusterCopier.h" + +namespace DB +{ + +void ClusterCopier::init() +{ + auto zookeeper = context.getZooKeeper(); + + task_description_watch_callback = [this] (const Coordination::WatchResponse & response) + { + if (response.error != Coordination::ZOK) + return; + UInt64 version = ++task_descprtion_version; + LOG_DEBUG(log, "Task description should be updated, local version " << version); + }; + + task_description_path = task_zookeeper_path + "/description"; + task_cluster = std::make_unique(task_zookeeper_path, working_database_name); + + reloadTaskDescription(); + task_cluster_initial_config = task_cluster_current_config; + + task_cluster->loadTasks(*task_cluster_initial_config); + context.setClustersConfig(task_cluster_initial_config, task_cluster->clusters_prefix); + + /// Set up shards and their priority + task_cluster->random_engine.seed(task_cluster->random_device()); + for (auto & task_table : task_cluster->table_tasks) + { + task_table.cluster_pull = context.getCluster(task_table.cluster_pull_name); + task_table.cluster_push = context.getCluster(task_table.cluster_push_name); + task_table.initShards(task_cluster->random_engine); + } + + LOG_DEBUG(log, "Will process " << task_cluster->table_tasks.size() << " table tasks"); + + /// Do not initialize tables, will make deferred initialization in process() + + zookeeper->createAncestors(getWorkersPathVersion() + "/"); + zookeeper->createAncestors(getWorkersPath() + "/"); +} + +template +decltype(auto) ClusterCopier::retry(T && func, UInt64 max_tries = 100) +{ + std::exception_ptr exception; + + for (UInt64 try_number = 1; try_number <= max_tries; ++try_number) + { + try + { + return func(); + } + catch (...) + { + exception = std::current_exception(); + if (try_number < max_tries) + { + tryLogCurrentException(log, "Will retry"); + std::this_thread::sleep_for(default_sleep_time); + } + } + } + + std::rethrow_exception(exception); +} + + +void ClusterCopier::discoverShardPartitions(const ConnectionTimeouts & timeouts, const TaskShardPtr & task_shard) +{ + TaskTable & task_table = task_shard->task_table; + + LOG_INFO(log, "Discover partitions of shard " << task_shard->getDescription()); + + auto get_partitions = [&] () { return getShardPartitions(timeouts, *task_shard); }; + auto existing_partitions_names = retry(get_partitions, 60); + Strings filtered_partitions_names; + Strings missing_partitions; + + /// Check that user specified correct partition names + auto check_partition_format = [] (const DataTypePtr & type, const String & partition_text_quoted) + { + MutableColumnPtr column_dummy = type->createColumn(); + ReadBufferFromString rb(partition_text_quoted); + + try + { + type->deserializeAsTextQuoted(*column_dummy, rb, FormatSettings()); + } + catch (Exception & e) + { + throw Exception("Partition " + partition_text_quoted + " has incorrect format. " + e.displayText(), ErrorCodes::BAD_ARGUMENTS); + } + }; + + if (task_table.has_enabled_partitions) + { + /// Process partition in order specified by + for (const String & partition_name : task_table.enabled_partitions) + { + /// Check that user specified correct partition names + check_partition_format(task_shard->partition_key_column.type, partition_name); + + auto it = existing_partitions_names.find(partition_name); + + /// Do not process partition if it is not in enabled_partitions list + if (it == existing_partitions_names.end()) + { + missing_partitions.emplace_back(partition_name); + continue; + } + + filtered_partitions_names.emplace_back(*it); + } + + for (const String & partition_name : existing_partitions_names) + { + if (!task_table.enabled_partitions_set.count(partition_name)) + { + LOG_DEBUG(log, "Partition " << partition_name << " will not be processed, since it is not in " + << "enabled_partitions of " << task_table.table_id); + } + } + } + else + { + for (const String & partition_name : existing_partitions_names) + filtered_partitions_names.emplace_back(partition_name); + } + + for (const String & partition_name : filtered_partitions_names) + { + task_shard->partition_tasks.emplace(partition_name, ShardPartition(*task_shard, partition_name, 10)); + task_shard->checked_partitions.emplace(partition_name, true); + + auto shard_partition_it = task_shard->partition_tasks.find(partition_name); + PartitionPieces & shard_partition_pieces = shard_partition_it->second.pieces; + + const size_t number_of_splits = task_table.number_of_splits; + + for (size_t piece_number = 0; piece_number < number_of_splits; ++piece_number) + { + bool res = checkPresentPartitionPiecesOnCurrentShard(timeouts, *task_shard, partition_name, piece_number); + shard_partition_pieces.emplace_back(shard_partition_it->second, piece_number, res); + } + } + + if (!missing_partitions.empty()) + { + std::stringstream ss; + for (const String & missing_partition : missing_partitions) + ss << " " << missing_partition; + + LOG_WARNING(log, "There are no " << missing_partitions.size() << " partitions from enabled_partitions in shard " + << task_shard->getDescription() << " :" << ss.str()); + } + + LOG_DEBUG(log, "Will copy " << task_shard->partition_tasks.size() << " partitions from shard " << task_shard->getDescription()); +} + +/// Compute set of partitions, assume set of partitions aren't changed during the processing +void ClusterCopier::discoverTablePartitions(const ConnectionTimeouts & timeouts, TaskTable & task_table, UInt64 num_threads = 0) +{ + /// Fetch partitions list from a shard + { + ThreadPool thread_pool(num_threads ? num_threads : 2 * getNumberOfPhysicalCPUCores()); + + for (const TaskShardPtr & task_shard : task_table.all_shards) + thread_pool.scheduleOrThrowOnError([this, timeouts, task_shard]() { discoverShardPartitions(timeouts, task_shard); }); + + LOG_DEBUG(log, "Waiting for " << thread_pool.active() << " setup jobs"); + thread_pool.wait(); + } +} + +void ClusterCopier::uploadTaskDescription(const std::string & task_path, const std::string & task_file, const bool force) +{ + auto local_task_description_path = task_path + "/description"; + + String task_config_str; + { + ReadBufferFromFile in(task_file); + readStringUntilEOF(task_config_str, in); + } + if (task_config_str.empty()) + return; + + auto zookeeper = context.getZooKeeper(); + + zookeeper->createAncestors(local_task_description_path); + auto code = zookeeper->tryCreate(local_task_description_path, task_config_str, zkutil::CreateMode::Persistent); + if (code && force) + zookeeper->createOrUpdate(local_task_description_path, task_config_str, zkutil::CreateMode::Persistent); + + LOG_DEBUG(log, "Task description " << ((code && !force) ? "not " : "") << "uploaded to " << local_task_description_path << " with result " << code << " ("<< zookeeper->error2string(code) << ")"); +} + +void ClusterCopier::reloadTaskDescription() +{ + auto zookeeper = context.getZooKeeper(); + task_description_watch_zookeeper = zookeeper; + + String task_config_str; + Coordination::Stat stat{}; + int code; + + zookeeper->tryGetWatch(task_description_path, task_config_str, &stat, task_description_watch_callback, &code); + if (code) + throw Exception("Can't get description node " + task_description_path, ErrorCodes::BAD_ARGUMENTS); + + LOG_DEBUG(log, "Loading description, zxid=" << task_description_current_stat.czxid); + auto config = getConfigurationFromXMLString(task_config_str); + + /// Setup settings + task_cluster->reloadSettings(*config); + context.getSettingsRef() = task_cluster->settings_common; + + task_cluster_current_config = config; + task_description_current_stat = stat; +} + +void ClusterCopier::updateConfigIfNeeded() +{ + UInt64 version_to_update = task_descprtion_version; + bool is_outdated_version = task_descprtion_current_version != version_to_update; + bool is_expired_session = !task_description_watch_zookeeper || task_description_watch_zookeeper->expired(); + + if (!is_outdated_version && !is_expired_session) + return; + + LOG_DEBUG(log, "Updating task description"); + reloadTaskDescription(); + + task_descprtion_current_version = version_to_update; +} + +void ClusterCopier::process(const ConnectionTimeouts & timeouts) +{ + for (TaskTable & task_table : task_cluster->table_tasks) + { + LOG_INFO(log, "Process table task " << task_table.table_id << " with " + << task_table.all_shards.size() << " shards, " + << task_table.local_shards.size() << " of them are local ones"); + + if (task_table.all_shards.empty()) + continue; + + /// Discover partitions of each shard and total set of partitions + if (!task_table.has_enabled_partitions) + { + /// If there are no specified enabled_partitions, we must discover them manually + discoverTablePartitions(timeouts, task_table); + + /// After partitions of each shard are initialized, initialize cluster partitions + for (const TaskShardPtr & task_shard : task_table.all_shards) + { + for (const auto & partition_elem : task_shard->partition_tasks) + { + const String & partition_name = partition_elem.first; + task_table.cluster_partitions.emplace(partition_name, ClusterPartition{}); + } + } + + for (auto & partition_elem : task_table.cluster_partitions) + { + const String & partition_name = partition_elem.first; + + for (const TaskShardPtr & task_shard : task_table.all_shards) + task_shard->checked_partitions.emplace(partition_name); + + task_table.ordered_partition_names.emplace_back(partition_name); + } + } + else + { + /// If enabled_partitions are specified, assume that each shard has all partitions + /// We will refine partition set of each shard in future + + for (const String & partition_name : task_table.enabled_partitions) + { + task_table.cluster_partitions.emplace(partition_name, ClusterPartition{}); + task_table.ordered_partition_names.emplace_back(partition_name); + } + } + + task_table.watch.restart(); + + /// Retry table processing + bool table_is_done = false; + for (UInt64 num_table_tries = 0; num_table_tries < max_table_tries; ++num_table_tries) + { + if (tryProcessTable(timeouts, task_table)) + { + table_is_done = true; + break; + } + } + + if (!table_is_done) + { + throw Exception("Too many tries to process table " + task_table.table_id + ". Abort remaining execution", + ErrorCodes::UNFINISHED); + } + } +} + +/// Disables DROP PARTITION commands that used to clear data after errors +void ClusterCopier::setSafeMode(bool is_safe_mode_ = true) +{ + is_safe_mode = is_safe_mode_; +} + +void ClusterCopier::setCopyFaultProbability(double copy_fault_probability_) +{ + copy_fault_probability = copy_fault_probability_; +} + +/// Protected section + +String ClusterCopier::getWorkersPath() const +{ + return task_cluster->task_zookeeper_path + "/task_active_workers"; +} + +String ClusterCopier::getWorkersPathVersion() const +{ + return getWorkersPath() + "_version"; +} + +String ClusterCopier::getCurrentWorkerNodePath() const +{ + return getWorkersPath() + "/" + host_id; +} + +zkutil::EphemeralNodeHolder::Ptr ClusterCopier::createTaskWorkerNodeAndWaitIfNeed( + const zkutil::ZooKeeperPtr & zookeeper, + const String & description, + bool unprioritized) +{ + std::chrono::milliseconds current_sleep_time = default_sleep_time; + static constexpr std::chrono::milliseconds max_sleep_time(30000); // 30 sec + + if (unprioritized) + std::this_thread::sleep_for(current_sleep_time); + + String workers_version_path = getWorkersPathVersion(); + String workers_path = getWorkersPath(); + String current_worker_path = getCurrentWorkerNodePath(); + + UInt64 num_bad_version_errors = 0; + + while (true) + { + updateConfigIfNeeded(); + + Coordination::Stat stat{}; + zookeeper->get(workers_version_path, &stat); + auto version = stat.version; + zookeeper->get(workers_path, &stat); + + if (static_cast(stat.numChildren) >= task_cluster->max_workers) + { + LOG_DEBUG(log, "Too many workers (" << stat.numChildren << ", maximum " << task_cluster->max_workers << ")" + << ". Postpone processing " << description); + + if (unprioritized) + current_sleep_time = std::min(max_sleep_time, current_sleep_time + default_sleep_time); + + std::this_thread::sleep_for(current_sleep_time); + num_bad_version_errors = 0; + } + else + { + Coordination::Requests ops; + ops.emplace_back(zkutil::makeSetRequest(workers_version_path, description, version)); + ops.emplace_back(zkutil::makeCreateRequest(current_worker_path, description, zkutil::CreateMode::Ephemeral)); + Coordination::Responses responses; + auto code = zookeeper->tryMulti(ops, responses); + + if (code == Coordination::ZOK || code == Coordination::ZNODEEXISTS) + return std::make_shared(current_worker_path, *zookeeper, false, false, description); + + if (code == Coordination::ZBADVERSION) + { + ++num_bad_version_errors; + + /// Try to make fast retries + if (num_bad_version_errors > 3) + { + LOG_DEBUG(log, "A concurrent worker has just been added, will check free worker slots again"); + std::chrono::milliseconds random_sleep_time(std::uniform_int_distribution(1, 1000)(task_cluster->random_engine)); + std::this_thread::sleep_for(random_sleep_time); + num_bad_version_errors = 0; + } + } + else + throw Coordination::Exception(code); + } + } +} + +/** Checks that the whole partition of a table was copied. We should do it carefully due to dirty lock. + * State of some task could change during the processing. + * We have to ensure that all shards have the finished state and there is no dirty flag. + * Moreover, we have to check status twice and check zxid, because state can change during the checking. + */ +bool ClusterCopier::checkPartitionIsDone(const TaskTable & task_table, const String & partition_name, const TasksShard & shards_with_partition) +{ + LOG_DEBUG(log, "Check that all shards processed partition " << partition_name << " successfully"); + + auto zookeeper = context.getZooKeeper(); + + Strings status_paths; + for (auto & shard : shards_with_partition) + { + ShardPartition & task_shard_partition = shard->partition_tasks.find(partition_name)->second; + status_paths.emplace_back(task_shard_partition.getShardStatusPath()); + } + + std::vector zxid1, zxid2; + + try + { + std::vector get_futures; + for (const String & path : status_paths) + get_futures.emplace_back(zookeeper->asyncGet(path)); + + // Check that state is Finished and remember zxid + for (auto & future : get_futures) + { + auto res = future.get(); + + TaskStateWithOwner status = TaskStateWithOwner::fromString(res.data); + if (status.state != TaskState::Finished) + { + LOG_INFO(log, "The task " << res.data << " is being rewritten by " << status.owner << ". Partition will be rechecked"); + return false; + } + + zxid1.push_back(res.stat.pzxid); + } + + // Check that partition is not dirty + { + CleanStateClock clean_state_clock ( + zookeeper, + task_table.getCertainPartitionIsDirtyPath(partition_name), + task_table.getCertainPartitionIsCleanedPath(partition_name) + ); + Coordination::Stat stat{}; + LogicalClock task_start_clock; + if (zookeeper->exists(task_table.getCertainPartitionTaskStatusPath(partition_name), &stat)) + task_start_clock = LogicalClock(stat.mzxid); + zookeeper->get(task_table.getCertainPartitionTaskStatusPath(partition_name), &stat); + if (!clean_state_clock.is_clean() || task_start_clock <= clean_state_clock.discovery_zxid) + { + LOG_INFO(log, "Partition " << partition_name << " become dirty"); + return false; + } + } + + get_futures.clear(); + for (const String & path : status_paths) + get_futures.emplace_back(zookeeper->asyncGet(path)); + + // Remember zxid of states again + for (auto & future : get_futures) + { + auto res = future.get(); + zxid2.push_back(res.stat.pzxid); + } + } + catch (const Coordination::Exception & e) + { + LOG_INFO(log, "A ZooKeeper error occurred while checking partition " << partition_name + << ". Will recheck the partition. Error: " << e.displayText()); + return false; + } + + // If all task is finished and zxid is not changed then partition could not become dirty again + for (UInt64 shard_num = 0; shard_num < status_paths.size(); ++shard_num) + { + if (zxid1[shard_num] != zxid2[shard_num]) + { + LOG_INFO(log, "The task " << status_paths[shard_num] << " is being modified now. Partition will be rechecked"); + return false; + } + } + + LOG_INFO(log, "Partition " << partition_name << " is copied successfully"); + return true; +} + +bool ClusterCopier::checkAllPieceInPartitionDone(const TaskTable & task_table, const String & partition_name, const TasksShard & shards_with_partition) +{ + bool answer = true; + for (size_t piece_number = 0; piece_number < task_table.number_of_splits; piece_number++) + answer &= checkPartitionPieceIsDone(task_table, partition_name, piece_number, shards_with_partition); + return answer; +} + + +/* The same as function above + * Assume that we don't know on which shards do we have partition certain piece. + * We'll check them all (I mean shards that contain the whole partition) + * And shards that don't have certain piece MUST mark that piece is_done true. + * */ +bool ClusterCopier::checkPartitionPieceIsDone(const TaskTable & task_table, const String & partition_name, + size_t piece_number, const TasksShard & shards_with_partition) +{ + LOG_DEBUG(log, "Check that all shards processed partition " << partition_name + << " piece number" + toString(piece_number) + " successfully"); + + auto zookeeper = context.getZooKeeper(); + + /// Collect all shards that contain partition piece number piece_number. + Strings piece_status_paths; + for (auto & shard : shards_with_partition) + { + ShardPartition & task_shard_partition = shard->partition_tasks.find(partition_name)->second; + ShardPartitionPiece & shard_partition_piece = task_shard_partition.pieces[piece_number]; + piece_status_paths.emplace_back(shard_partition_piece.getShardStatusPath()); + } + + std::vector zxid1, zxid2; + + try + { + std::vector get_futures; + for (const String & path : piece_status_paths) + get_futures.emplace_back(zookeeper->asyncGet(path)); + + // Check that state is Finished and remember zxid + for (auto & future : get_futures) + { + auto res = future.get(); + + TaskStateWithOwner status = TaskStateWithOwner::fromString(res.data); + if (status.state != TaskState::Finished) + { + LOG_INFO(log, "The task " << res.data << " is being rewritten by " << status.owner << ". Partition piece will be rechecked"); + return false; + } + + zxid1.push_back(res.stat.pzxid); + } + + // Check that partition is not dirty + { + CleanStateClock clean_state_clock ( + zookeeper, + task_table.getCertainPartitionIsDirtyPath(partition_name), + task_table.getCertainPartitionIsCleanedPath(partition_name) + ); + Coordination::Stat stat{}; + LogicalClock task_start_clock; + if (zookeeper->exists(task_table.getCertainPartitionTaskStatusPath(partition_name), &stat)) + task_start_clock = LogicalClock(stat.mzxid); + zookeeper->get(task_table.getCertainPartitionTaskStatusPath(partition_name), &stat); + if (!clean_state_clock.is_clean() || task_start_clock <= clean_state_clock.discovery_zxid) + { + LOG_INFO(log, "Partition " << partition_name << " piece number " << toString(piece_number) << " become dirty"); + return false; + } + } + + get_futures.clear(); + for (const String & path : piece_status_paths) + get_futures.emplace_back(zookeeper->asyncGet(path)); + + // Remember zxid of states again + for (auto & future : get_futures) + { + auto res = future.get(); + zxid2.push_back(res.stat.pzxid); + } + } + catch (const Coordination::Exception & e) + { + LOG_INFO(log, "A ZooKeeper error occurred while checking partition " << partition_name << " piece number " + << toString(piece_number) << ". Will recheck the partition. Error: " << e.displayText()); + return false; + } + + // If all task is finished and zxid is not changed then partition could not become dirty again + for (UInt64 shard_num = 0; shard_num < piece_status_paths.size(); ++shard_num) + { + if (zxid1[shard_num] != zxid2[shard_num]) + { + LOG_INFO(log, "The task " << piece_status_paths[shard_num] << " is being modified now. Partition piece will be rechecked"); + return false; + } + } + + LOG_INFO(log, "Partition " << partition_name << " is copied successfully"); + return true; + +} + +/// Removes MATERIALIZED and ALIAS columns from create table query +static ASTPtr ClusterCopier::removeAliasColumnsFromCreateQuery(const ASTPtr & query_ast) +{ + const ASTs & column_asts = query_ast->as().columns_list->columns->children; + auto new_columns = std::make_shared(); + + for (const ASTPtr & column_ast : column_asts) + { + const auto & column = column_ast->as(); + + if (!column.default_specifier.empty()) + { + ColumnDefaultKind kind = columnDefaultKindFromString(column.default_specifier); + if (kind == ColumnDefaultKind::Materialized || kind == ColumnDefaultKind::Alias) + continue; + } + + new_columns->children.emplace_back(column_ast->clone()); + } + + ASTPtr new_query_ast = query_ast->clone(); + auto & new_query = new_query_ast->as(); + + auto new_columns_list = std::make_shared(); + new_columns_list->set(new_columns_list->columns, new_columns); + if (auto indices = query_ast->as()->columns_list->indices) + new_columns_list->set(new_columns_list->indices, indices->clone()); + + new_query.replace(new_query.columns_list, new_columns_list); + + return new_query_ast; +} + +/// Replaces ENGINE and table name in a create query +static std::shared_ptr ClusterCopier::rewriteCreateQueryStorage(const ASTPtr & create_query_ast, const DatabaseAndTableName & new_table, const ASTPtr & new_storage_ast) +{ + const auto & create = create_query_ast->as(); + auto res = std::make_shared(create); + + if (create.storage == nullptr || new_storage_ast == nullptr) + throw Exception("Storage is not specified", ErrorCodes::LOGICAL_ERROR); + + res->database = new_table.first; + res->table = new_table.second; + + res->children.clear(); + res->set(res->columns_list, create.columns_list->clone()); + res->set(res->storage, new_storage_ast->clone()); + + return res; +} + +bool ClusterCopier::tryDropPartition(ShardPartition & task_partition, const zkutil::ZooKeeperPtr & zookeeper, const CleanStateClock & clean_state_clock) +{ + if (is_safe_mode) + throw Exception("DROP PARTITION is prohibited in safe mode", ErrorCodes::NOT_IMPLEMENTED); + + TaskTable & task_table = task_partition.task_shard.task_table; + + const String current_shards_path = task_partition.getPartitionShardsPath(); + const String current_partition_active_workers_dir = task_partition.getPartitionActiveWorkersPath(); + const String is_dirty_flag_path = task_partition.getCommonPartitionIsDirtyPath(); + const String dirty_cleaner_path = is_dirty_flag_path + "/cleaner"; + const String is_dirty_cleaned_path = task_partition.getCommonPartitionIsCleanedPath(); + + zkutil::EphemeralNodeHolder::Ptr cleaner_holder; + try + { + cleaner_holder = zkutil::EphemeralNodeHolder::create(dirty_cleaner_path, *zookeeper, host_id); + } + catch (const Coordination::Exception & e) + { + if (e.code == Coordination::ZNODEEXISTS) + { + LOG_DEBUG(log, "Partition " << task_partition.name << " is cleaning now by somebody, sleep"); + std::this_thread::sleep_for(default_sleep_time); + return false; + } + + throw; + } + + Coordination::Stat stat{}; + if (zookeeper->exists(current_partition_active_workers_dir, &stat)) + { + if (stat.numChildren != 0) + { + LOG_DEBUG(log, "Partition " << task_partition.name << " contains " << stat.numChildren << " active workers while trying to drop it. Going to sleep."); + std::this_thread::sleep_for(default_sleep_time); + return false; + } + else + { + zookeeper->remove(current_partition_active_workers_dir); + } + } + + { + zkutil::EphemeralNodeHolder::Ptr active_workers_lock; + try + { + active_workers_lock = zkutil::EphemeralNodeHolder::create(current_partition_active_workers_dir, *zookeeper, host_id); + } + catch (const Coordination::Exception & e) + { + if (e.code == Coordination::ZNODEEXISTS) + { + LOG_DEBUG(log, "Partition " << task_partition.name << " is being filled now by somebody, sleep"); + return false; + } + + throw; + } + + // Lock the dirty flag + zookeeper->set(is_dirty_flag_path, host_id, clean_state_clock.discovery_version.value()); + zookeeper->tryRemove(task_partition.getPartitionCleanStartPath()); + CleanStateClock my_clock(zookeeper, is_dirty_flag_path, is_dirty_cleaned_path); + + /// Remove all status nodes + { + Strings children; + if (zookeeper->tryGetChildren(current_shards_path, children) == Coordination::ZOK) + for (const auto & child : children) + { + zookeeper->removeRecursive(current_shards_path + "/" + child); + } + } + + String query = "ALTER TABLE " + getQuotedTable(task_table.table_push); + query += " DROP PARTITION " + task_partition.name + ""; + + /// TODO: use this statement after servers will be updated up to 1.1.54310 + // query += " DROP PARTITION ID '" + task_partition.name + "'"; + + ClusterPtr & cluster_push = task_table.cluster_push; + Settings settings_push = task_cluster->settings_push; + + /// It is important, DROP PARTITION must be done synchronously + settings_push.replication_alter_partitions_sync = 2; + + LOG_DEBUG(log, "Execute distributed DROP PARTITION: " << query); + /// Limit number of max executing replicas to 1 + UInt64 num_shards = executeQueryOnCluster(cluster_push, query, nullptr, &settings_push, PoolMode::GET_ONE, 1); + + if (num_shards < cluster_push->getShardCount()) + { + LOG_INFO(log, "DROP PARTITION wasn't successfully executed on " << cluster_push->getShardCount() - num_shards << " shards"); + return false; + } + + /// Update the locking node + if (!my_clock.is_stale()) + { + zookeeper->set(is_dirty_flag_path, host_id, my_clock.discovery_version.value()); + if (my_clock.clean_state_version) + zookeeper->set(is_dirty_cleaned_path, host_id, my_clock.clean_state_version.value()); + else + zookeeper->create(is_dirty_cleaned_path, host_id, zkutil::CreateMode::Persistent); + } + else + { + LOG_DEBUG(log, "Clean state is altered when dropping the partition, cowardly bailing"); + /// clean state is stale + return false; + } + + LOG_INFO(log, "Partition " << task_partition.name << " was dropped on cluster " << task_table.cluster_push_name); + if (zookeeper->tryCreate(current_shards_path, host_id, zkutil::CreateMode::Persistent) == Coordination::ZNODEEXISTS) + zookeeper->set(current_shards_path, host_id); + } + + LOG_INFO(log, "Partition " << task_partition.name << " is safe for work now."); + return true; +} + + +static constexpr UInt64 max_table_tries = 1000; +static constexpr UInt64 max_shard_partition_tries = 600; + +bool ClusterCopier::tryProcessTable(const ConnectionTimeouts & timeouts, TaskTable & task_table) +{ + /// A heuristic: if previous shard is already done, then check next one without sleeps due to max_workers constraint + bool previous_shard_is_instantly_finished = false; + + /// Process each partition that is present in cluster + for (const String & partition_name : task_table.ordered_partition_names) + { + if (!task_table.cluster_partitions.count(partition_name)) + throw Exception("There are no expected partition " + partition_name + ". It is a bug", ErrorCodes::LOGICAL_ERROR); + + ClusterPartition & cluster_partition = task_table.cluster_partitions[partition_name]; + + Stopwatch watch; + /// We will check all the shards of the table and check if they contain current partition. + TasksShard expected_shards; + UInt64 num_failed_shards = 0; + + ++cluster_partition.total_tries; + + LOG_DEBUG(log, "Processing partition " << partition_name << " for the whole cluster"); + + /// Process each source shard having current partition and copy current partition + /// NOTE: shards are sorted by "distance" to current host + bool has_shard_to_process = false; + for (const TaskShardPtr & shard : task_table.all_shards) + { + /// Does shard have a node with current partition? + if (shard->partition_tasks.count(partition_name) == 0) + { + /// If not, did we check existence of that partition previously? + if (shard->checked_partitions.count(partition_name) == 0) + { + auto check_shard_has_partition = [&] () { return checkShardHasPartition(timeouts, *shard, partition_name); }; + bool has_partition = retry(check_shard_has_partition); + + shard->checked_partitions.emplace(partition_name); + + if (has_partition) + { + shard->partition_tasks.emplace(partition_name, ShardPartition(*shard, partition_name, 10)); + LOG_DEBUG(log, "Discovered partition " << partition_name << " in shard " << shard->getDescription()); + /// To save references in the future. + auto shard_partition_it = shard->partition_tasks.find(partition_name); + PartitionPieces & shard_partition_pieces = shard_partition_it->second.pieces; + + const size_t number_of_splits = task_table.number_of_splits; + + for (size_t piece_number = 0; piece_number < number_of_splits; ++piece_number) + { + auto res = checkPresentPartitionPiecesOnCurrentShard(timeouts, *shard, partition_name, piece_number); + shard_partition_pieces.emplace_back(shard_partition_it->second, piece_number, res); + } + } + else + { + LOG_DEBUG(log, "Found that shard " << shard->getDescription() << " does not contain current partition " << partition_name); + continue; + } + } + else + { + /// We have already checked that partition, but did not discover it + previous_shard_is_instantly_finished = true; + continue; + } + } + + auto it_shard_partition = shard->partition_tasks.find(partition_name); + /// Previously when we discovered that shard does not contain current partition, we skipped it. + /// At this moment partition have to be present. + if (it_shard_partition == shard->partition_tasks.end()) + throw Exception("There are no such partition in a shard. This is a bug.", ErrorCodes::LOGICAL_ERROR); + auto & partition = it_shard_partition->second; + + expected_shards.emplace_back(shard); + + /// Do not sleep if there is a sequence of already processed shards to increase startup + bool is_unprioritized_task = !previous_shard_is_instantly_finished && shard->priority.is_remote; + PartitionTaskStatus task_status = PartitionTaskStatus::Error; + bool was_error = false; + has_shard_to_process = true; + for (UInt64 try_num = 0; try_num < max_shard_partition_tries; ++try_num) + { + task_status = tryProcessPartitionTask(timeouts, partition, is_unprioritized_task); + + /// Exit if success + if (task_status == PartitionTaskStatus::Finished) + break; + + was_error = true; + + /// Skip if the task is being processed by someone + if (task_status == PartitionTaskStatus::Active) + break; + + /// Repeat on errors + std::this_thread::sleep_for(default_sleep_time); + } + + if (task_status == PartitionTaskStatus::Error) + ++num_failed_shards; + + previous_shard_is_instantly_finished = !was_error; + } + + cluster_partition.elapsed_time_seconds += watch.elapsedSeconds(); + + /// Check that whole cluster partition is done + /// Firstly check the number of failed partition tasks, then look into ZooKeeper and ensure that each partition is done + bool partition_is_done = num_failed_shards == 0; + try + { + partition_is_done = + !has_shard_to_process + || (partition_is_done && checkPartitionIsDone(task_table, partition_name, expected_shards)); + } + catch (...) + { + tryLogCurrentException(log); + partition_is_done = false; + } + + if (partition_is_done) + { + task_table.finished_cluster_partitions.emplace(partition_name); + + task_table.bytes_copied += cluster_partition.bytes_copied; + task_table.rows_copied += cluster_partition.rows_copied; + double elapsed = cluster_partition.elapsed_time_seconds; + + LOG_INFO(log, "It took " << std::fixed << std::setprecision(2) << elapsed << " seconds to copy partition " << partition_name + << ": " << formatReadableSizeWithDecimalSuffix(cluster_partition.bytes_copied) << " uncompressed bytes" + << ", " << formatReadableQuantity(cluster_partition.rows_copied) << " rows" + << " and " << cluster_partition.blocks_copied << " source blocks are copied"); + + if (cluster_partition.rows_copied) + { + LOG_INFO(log, "Average partition speed: " + << formatReadableSizeWithDecimalSuffix(cluster_partition.bytes_copied / elapsed) << " per second."); + } + + if (task_table.rows_copied) + { + LOG_INFO(log, "Average table " << task_table.table_id << " speed: " + << formatReadableSizeWithDecimalSuffix(task_table.bytes_copied / elapsed) << " per second."); + } + } + } + + UInt64 required_partitions = task_table.cluster_partitions.size(); + UInt64 finished_partitions = task_table.finished_cluster_partitions.size(); + bool table_is_done = finished_partitions >= required_partitions; + + if (!table_is_done) + { + LOG_INFO(log, "Table " + task_table.table_id + " is not processed yet." + << "Copied " << finished_partitions << " of " << required_partitions << ", will retry"); + } + + return table_is_done; +} + +/// Job for copying partition from particular shard. +PartitionTaskStatus ClusterCopier::tryProcessPartitionTask(const ConnectionTimeouts & timeouts, ShardPartition & task_partition, bool is_unprioritized_task) +{ + PartitionTaskStatus res; + + try + { + res = iterateThroughAllPiecesInPartition(timeouts, task_partition, is_unprioritized_task); + } + catch (...) + { + tryLogCurrentException(log, "An error occurred while processing partition " + task_partition.name); + res = PartitionTaskStatus::Error; + } + + /// At the end of each task check if the config is updated + try + { + updateConfigIfNeeded(); + } + catch (...) + { + tryLogCurrentException(log, "An error occurred while updating the config"); + } + + return res; +} + +PartitionTaskStatus ClusterCopier::iterateThroughAllPiecesInPartition(const ConnectionTimeouts & timeouts, ShardPartition & task_partition, + bool is_unprioritized_task) +{ + const size_t total_number_of_pieces = task_partition.task_shard.task_table.number_of_splits; + + /// ThreadPool maybe ?? + for (size_t piece_number = 0; piece_number < total_number_of_pieces; piece_number++) + processPartitionPieceTaskImpl(timeouts, task_partition, piece_number, is_unprioritized_task); + + return PartitionTaskStatus::Finished; + +} + +PartitionTaskStatus ClusterCopier::processPartitionPieceTaskImpl(const ConnectionTimeouts & timeouts, ShardPartition & task_partition, + const size_t current_piece_number, bool is_unprioritized_task) +{ + TaskShard & task_shard = task_partition.task_shard; + TaskTable & task_table = task_shard.task_table; + ClusterPartition & cluster_partition = task_table.getClusterPartition(task_partition.name); + ShardPartitionPiece & partition_piece = task_partition.pieces[current_piece_number]; + + const size_t number_of_splits = task_table.number_of_splits; + const String primary_key_comma_separated = task_table.primary_key_comma_separated; + UNUSED(number_of_splits); + UNUSED(partition_piece); + /// We need to update table definitions for each partition, it could be changed after ALTER + createShardInternalTables(timeouts, task_shard, true, current_piece_number); + + auto zookeeper = context.getZooKeeper(); + + const String piece_is_dirty_flag_path = partition_piece.getPartitionPieceIsDirtyPath(); + const String piece_is_dirty_cleaned_path = partition_piece.getPartitionPieceIsCleanedPath(); + const String current_task_piece_is_active_path = partition_piece.getActiveWorkerPath(); + const String current_task_piece_status_path = partition_piece.getShardStatusPath(); + + /// Auxiliary functions: + + /// Creates is_dirty node to initialize DROP PARTITION + auto create_is_dirty_node = [&, this] (const CleanStateClock & clock) + { + if (clock.is_stale()) + LOG_DEBUG(log, "Clean state clock is stale while setting dirty flag, cowardly bailing"); + else if (!clock.is_clean()) + LOG_DEBUG(log, "Thank you, Captain Obvious"); + else if (clock.discovery_version) + { + LOG_DEBUG(log, "Updating clean state clock"); + zookeeper->set(piece_is_dirty_flag_path, host_id, clock.discovery_version.value()); + } + else + { + LOG_DEBUG(log, "Creating clean state clock"); + zookeeper->create(piece_is_dirty_flag_path, host_id, zkutil::CreateMode::Persistent); + } + }; + + /// Returns SELECT query filtering current partition and applying user filter + auto get_select_query = [&] (const DatabaseAndTableName & from_table, const String & fields, String limit = "", + bool enable_splitting = false) + { + String query; + query += "SELECT " + fields + " FROM " + getQuotedTable(from_table); + /// TODO: Bad, it is better to rewrite with ASTLiteral(partition_key_field) + query += " WHERE (" + queryToString(task_table.engine_push_partition_key_ast) + " = (" + task_partition.name + " AS partition_key))"; + + if (enable_splitting) + query += " AND ( cityHash64(" + primary_key_comma_separated + ") = " + toString(current_piece_number) + " )"; + + if (!task_table.where_condition_str.empty()) + query += " AND (" + task_table.where_condition_str + ")"; + if (!limit.empty()) + query += " LIMIT " + limit; + + ParserQuery p_query(query.data() + query.size()); + return parseQuery(p_query, query, 0); + }; + + /// Load balancing + auto worker_node_holder = createTaskWorkerNodeAndWaitIfNeed(zookeeper, current_task_piece_status_path, is_unprioritized_task); + + LOG_DEBUG(log, "Processing " << current_task_piece_status_path); + + CleanStateClock clean_state_clock (zookeeper, piece_is_dirty_flag_path, piece_is_dirty_cleaned_path); + + LogicalClock task_start_clock; + { + Coordination::Stat stat{}; + if (zookeeper->exists(task_partition.getPartitionShardsPath(), &stat)) + task_start_clock = LogicalClock(stat.mzxid); + } + + /// Do not start if partition is dirty, try to clean it + if (clean_state_clock.is_clean() + && (!task_start_clock.hasHappened() || clean_state_clock.discovery_zxid <= task_start_clock)) + { + LOG_DEBUG(log, "Partition " << task_partition.name << " appears to be clean"); + zookeeper->createAncestors(current_task_piece_status_path); + } + else + { + LOG_DEBUG(log, "Partition " << task_partition.name << " is dirty, try to drop it"); + + try + { + tryDropPartition(task_partition, zookeeper, clean_state_clock); + } + catch (...) + { + tryLogCurrentException(log, "An error occurred when clean partition"); + } + + return PartitionTaskStatus::Error; + } + + /// Create ephemeral node to mark that we are active and process the partition + zookeeper->createAncestors(current_task_piece_is_active_path); + zkutil::EphemeralNodeHolderPtr partition_task_node_holder; + try + { + partition_task_node_holder = zkutil::EphemeralNodeHolder::create(current_task_piece_is_active_path, *zookeeper, host_id); + } + catch (const Coordination::Exception & e) + { + if (e.code == Coordination::ZNODEEXISTS) + { + LOG_DEBUG(log, "Someone is already processing " << current_task_piece_is_active_path); + return PartitionTaskStatus::Active; + } + + throw; + } + + /// Exit if task has been already processed; + /// create blocking node to signal cleaning up if it is abandoned + { + String status_data; + if (zookeeper->tryGet(current_task_piece_status_path, status_data)) + { + TaskStateWithOwner status = TaskStateWithOwner::fromString(status_data); + if (status.state == TaskState::Finished) + { + LOG_DEBUG(log, "Task " << current_task_piece_status_path << " has been successfully executed by " << status.owner); + return PartitionTaskStatus::Finished; + } + + // Task is abandoned, initialize DROP PARTITION + LOG_DEBUG(log, "Task " << current_task_piece_status_path << " has not been successfully finished by " << + status.owner << ". Partition will be dropped and refilled."); + + create_is_dirty_node(clean_state_clock); + return PartitionTaskStatus::Error; + } + } + + /// Check that destination partition is empty if we are first worker + /// NOTE: this check is incorrect if pull and push tables have different partition key! + String clean_start_status; + if (!zookeeper->tryGet(task_partition.getPartitionCleanStartPath(), clean_start_status) || clean_start_status != "ok") + { + zookeeper->createIfNotExists(task_partition.getPartitionCleanStartPath(), ""); + auto checker = zkutil::EphemeralNodeHolder::create(task_partition.getPartitionCleanStartPath() + "/checker", + *zookeeper, host_id); + // Maybe we are the first worker + ASTPtr query_select_ast = get_select_query(task_shard.table_split_shard, "count()"); + UInt64 count; + { + Context local_context = context; + // Use pull (i.e. readonly) settings, but fetch data from destination servers + local_context.getSettingsRef() = task_cluster->settings_pull; + local_context.getSettingsRef().skip_unavailable_shards = true; + + Block block = getBlockWithAllStreamData(InterpreterFactory::get(query_select_ast, local_context)->execute().in); + count = (block) ? block.safeGetByPosition(0).column->getUInt(0) : 0; + } + + if (count != 0) + { + Coordination::Stat stat_shards{}; + zookeeper->get(task_partition.getPartitionShardsPath(), & stat_shards); + + /// NOTE: partition is still fresh if dirt discovery happens before cleaning + if (stat_shards.numChildren == 0) + { + LOG_WARNING(log, "There are no workers for partition " << task_partition.name + << ", but destination table contains " << count << " rows" + << ". Partition will be dropped and refilled."); + + create_is_dirty_node(clean_state_clock); + return PartitionTaskStatus::Error; + } + } + zookeeper->set(task_partition.getPartitionCleanStartPath(), "ok"); + } + /// At this point, we need to sync that the destination table is clean + /// before any actual work + + /// Try start processing, create node about it + { + String start_state = TaskStateWithOwner::getData(TaskState::Started, host_id); + CleanStateClock new_clean_state_clock (zookeeper, piece_is_dirty_flag_path, piece_is_dirty_cleaned_path); + if (clean_state_clock != new_clean_state_clock) + { + LOG_INFO(log, "Partition " << task_partition.name << " clean state changed, cowardly bailing"); + return PartitionTaskStatus::Error; + } + else if (!new_clean_state_clock.is_clean()) + { + LOG_INFO(log, "Partition " << task_partition.name << " is dirty and will be dropped and refilled"); + create_is_dirty_node(new_clean_state_clock); + return PartitionTaskStatus::Error; + } + zookeeper->create(current_task_piece_status_path, start_state, zkutil::CreateMode::Persistent); + } + + /// Try create table (if not exists) on each shard + { + /// Define push table for current partition piece + auto database_and_table_for_current_piece= std::pair( + task_table.table_push.first, + task_table.table_push.second + ".piece_" + toString(current_piece_number)); + + auto create_query_push_ast = rewriteCreateQueryStorage(task_shard.current_pull_table_create_query, + database_and_table_for_current_piece, task_table.engine_push_ast); + create_query_push_ast->as().if_not_exists = true; + String query = queryToString(create_query_push_ast); + + LOG_DEBUG(log, "Create destination tables. Query: " << query); + UInt64 shards = executeQueryOnCluster(task_table.cluster_push, query, + create_query_push_ast, &task_cluster->settings_push, + PoolMode::GET_MANY); + LOG_DEBUG(log, "Destination tables " << getQuotedTable(task_table.table_push) << + " have been created on " << shards << " shards of " << task_table.cluster_push->getShardCount()); + } + + /// Do the copying + { + bool inject_fault = false; + if (copy_fault_probability > 0) + { + double value = std::uniform_real_distribution<>(0, 1)(task_table.task_cluster.random_engine); + inject_fault = value < copy_fault_probability; + } + + // Select all fields + ASTPtr query_select_ast = get_select_query(task_shard.table_read_shard, "*", inject_fault ? "1" : ""); + + LOG_DEBUG(log, "Executing SELECT query and pull from " << task_shard.getDescription() + << " : " << queryToString(query_select_ast)); + + ASTPtr query_insert_ast; + { + String query; + query += "INSERT INTO " + getQuotedTable(task_shard.table_split_shard) + " VALUES "; + + ParserQuery p_query(query.data() + query.size()); + query_insert_ast = parseQuery(p_query, query, 0); + + LOG_DEBUG(log, "Executing INSERT query: " << query); + } + + try + { + /// Custom INSERT SELECT implementation + Context context_select = context; + context_select.getSettingsRef() = task_cluster->settings_pull; + + Context context_insert = context; + context_insert.getSettingsRef() = task_cluster->settings_push; + + BlockInputStreamPtr input; + BlockOutputStreamPtr output; + { + BlockIO io_select = InterpreterFactory::get(query_select_ast, context_select)->execute(); + BlockIO io_insert = InterpreterFactory::get(query_insert_ast, context_insert)->execute(); + + input = io_select.in; + output = io_insert.out; + } + + /// Fail-fast optimization to abort copying when the current clean state expires + std::future future_is_dirty_checker; + + Stopwatch watch(CLOCK_MONOTONIC_COARSE); + constexpr UInt64 check_period_milliseconds = 500; + + /// Will asynchronously check that ZooKeeper connection and is_dirty flag appearing while copying data + auto cancel_check = [&] () + { + if (zookeeper->expired()) + throw Exception("ZooKeeper session is expired, cancel INSERT SELECT", ErrorCodes::UNFINISHED); + + if (!future_is_dirty_checker.valid()) + future_is_dirty_checker = zookeeper->asyncExists(piece_is_dirty_flag_path); + + /// check_period_milliseconds should less than average insert time of single block + /// Otherwise, the insertion will slow a little bit + if (watch.elapsedMilliseconds() >= check_period_milliseconds) + { + Coordination::ExistsResponse status = future_is_dirty_checker.get(); + + if (status.error != Coordination::ZNONODE) + { + LogicalClock dirt_discovery_epoch (status.stat.mzxid); + if (dirt_discovery_epoch == clean_state_clock.discovery_zxid) + return false; + throw Exception("Partition is dirty, cancel INSERT SELECT", ErrorCodes::UNFINISHED); + } + } + + return false; + }; + + /// Update statistics + /// It is quite rough: bytes_copied don't take into account DROP PARTITION. + auto update_stats = [&cluster_partition] (const Block & block) + { + cluster_partition.bytes_copied += block.bytes(); + cluster_partition.rows_copied += block.rows(); + cluster_partition.blocks_copied += 1; + }; + + /// Main work is here + copyData(*input, *output, cancel_check, update_stats); + + // Just in case + if (future_is_dirty_checker.valid()) + future_is_dirty_checker.get(); + + if (inject_fault) + throw Exception("Copy fault injection is activated", ErrorCodes::UNFINISHED); + } + catch (...) + { + tryLogCurrentException(log, "An error occurred during copying, partition will be marked as dirty"); + return PartitionTaskStatus::Error; + } + } + + /// Finalize the processing, change state of current partition task (and also check is_dirty flag) + { + String state_finished = TaskStateWithOwner::getData(TaskState::Finished, host_id); + CleanStateClock new_clean_state_clock (zookeeper, piece_is_dirty_flag_path, piece_is_dirty_cleaned_path); + if (clean_state_clock != new_clean_state_clock) + { + LOG_INFO(log, "Partition " << task_partition.name << " clean state changed, cowardly bailing"); + return PartitionTaskStatus::Error; + } + else if (!new_clean_state_clock.is_clean()) + { + LOG_INFO(log, "Partition " << task_partition.name << " became dirty and will be dropped and refilled"); + create_is_dirty_node(new_clean_state_clock); + return PartitionTaskStatus::Error; + } + zookeeper->set(current_task_piece_status_path, state_finished, 0); + } + + LOG_INFO(log, "Partition " << task_partition.name << " copied"); + return PartitionTaskStatus::Finished; +} + +void ClusterCopier::dropAndCreateLocalTable(const ASTPtr & create_ast) +{ + const auto & create = create_ast->as(); + dropLocalTableIfExists({create.database, create.table}); + + InterpreterCreateQuery interpreter(create_ast, context); + interpreter.execute(); +} + +void ClusterCopier::dropLocalTableIfExists(const DatabaseAndTableName & table_name) const +{ + auto drop_ast = std::make_shared(); + drop_ast->if_exists = true; + drop_ast->database = table_name.first; + drop_ast->table = table_name.second; + + InterpreterDropQuery interpreter(drop_ast, context); + interpreter.execute(); +} + +String ClusterCopier::getRemoteCreateTable(const DatabaseAndTableName & table, Connection & connection, const Settings * settings = nullptr) +{ + String query = "SHOW CREATE TABLE " + getQuotedTable(table); + Block block = getBlockWithAllStreamData(std::make_shared( + connection, query, InterpreterShowCreateQuery::getSampleBlock(), context, settings)); + + return typeid_cast(*block.safeGetByPosition(0).column).getDataAt(0).toString(); +} + +ASTPtr ClusterCopier::getCreateTableForPullShard(const ConnectionTimeouts & timeouts, TaskShard & task_shard) +{ + /// Fetch and parse (possibly) new definition + auto connection_entry = task_shard.info.pool->get(timeouts, &task_cluster->settings_pull); + String create_query_pull_str = getRemoteCreateTable( + task_shard.task_table.table_pull, + *connection_entry, + &task_cluster->settings_pull); + + ParserCreateQuery parser_create_query; + return parseQuery(parser_create_query, create_query_pull_str, 0); +} + +/// If it is implicitly asked to create split Distributed table for certain piece on current shard, we will do it. +void ClusterCopier::createShardInternalTables(const ConnectionTimeouts & timeouts, TaskShard & task_shard, bool create_split = true, const size_t piece_number = 0) +{ + TaskTable & task_table = task_shard.task_table; + + /// We need to update table definitions for each part, it could be changed after ALTER + task_shard.current_pull_table_create_query = getCreateTableForPullShard(timeouts, task_shard); + + /// Create local Distributed tables: + /// a table fetching data from current shard and a table inserting data to the whole destination cluster + String read_shard_prefix = ".read_shard_" + toString(task_shard.indexInCluster()) + "."; + String split_shard_prefix = ".split."; + task_shard.table_read_shard = DatabaseAndTableName(working_database_name, read_shard_prefix + task_table.table_id); + task_shard.table_split_shard = DatabaseAndTableName( + working_database_name, split_shard_prefix + task_table.table_id + ".piece_" + toString(piece_number)); + + /// Create special cluster with single shard + String shard_read_cluster_name = read_shard_prefix + task_table.cluster_pull_name; + ClusterPtr cluster_pull_current_shard = task_table.cluster_pull->getClusterWithSingleShard(task_shard.indexInCluster()); + context.setCluster(shard_read_cluster_name, cluster_pull_current_shard); + + auto storage_shard_ast = createASTStorageDistributed(shard_read_cluster_name, task_table.table_pull.first, task_table.table_pull.second); + const auto & storage_piece_split_ast = task_table.auxiliary_engine_split_asts[piece_number]; + + auto create_query_ast = removeAliasColumnsFromCreateQuery(task_shard.current_pull_table_create_query); + auto create_table_pull_ast = rewriteCreateQueryStorage(create_query_ast, task_shard.table_read_shard, storage_shard_ast); + auto create_table_split_piece_ast = rewriteCreateQueryStorage(create_query_ast, task_shard.table_split_shard, storage_piece_split_ast); + + dropAndCreateLocalTable(create_table_pull_ast); + + if (create_split) + dropAndCreateLocalTable(create_table_split_piece_ast); +} + + +std::set ClusterCopier::getShardPartitions(const ConnectionTimeouts & timeouts, TaskShard & task_shard) +{ + createShardInternalTables(timeouts, task_shard, false); + + TaskTable & task_table = task_shard.task_table; + + String query; + { + WriteBufferFromOwnString wb; + wb << "SELECT DISTINCT " << queryToString(task_table.engine_push_partition_key_ast) << " AS partition FROM" + << " " << getQuotedTable(task_shard.table_read_shard) << " ORDER BY partition DESC"; + query = wb.str(); + } + + ParserQuery parser_query(query.data() + query.size()); + ASTPtr query_ast = parseQuery(parser_query, query, 0); + + LOG_DEBUG(log, "Computing destination partition set, executing query: " << query); + + Context local_context = context; + local_context.setSettings(task_cluster->settings_pull); + Block block = getBlockWithAllStreamData(InterpreterFactory::get(query_ast, local_context)->execute().in); + + std::set res; + if (block) + { + ColumnWithTypeAndName & column = block.getByPosition(0); + task_shard.partition_key_column = column; + + for (size_t i = 0; i < column.column->size(); ++i) + { + WriteBufferFromOwnString wb; + column.type->serializeAsTextQuoted(*column.column, i, wb, FormatSettings()); + res.emplace(wb.str()); + } + } + + LOG_DEBUG(log, "There are " << res.size() << " destination partitions in shard " << task_shard.getDescription()); + + return res; +} + +bool ClusterCopier::checkShardHasPartition(const ConnectionTimeouts & timeouts, TaskShard & task_shard, const String & partition_quoted_name) +{ + createShardInternalTables(timeouts, task_shard, false); + + TaskTable & task_table = task_shard.task_table; + + std::string query = "SELECT 1 FROM " + getQuotedTable(task_shard.table_read_shard) + + " WHERE (" + queryToString(task_table.engine_push_partition_key_ast) + " = (" + partition_quoted_name + " AS partition_key))"; + + if (!task_table.where_condition_str.empty()) + query += " AND (" + task_table.where_condition_str + ")"; + + query += " LIMIT 1"; + + LOG_DEBUG(log, "Checking shard " << task_shard.getDescription() << " for partition " + << partition_quoted_name << " existence, executing query: " << query); + + ParserQuery parser_query(query.data() + query.size()); + ASTPtr query_ast = parseQuery(parser_query, query, 0); + + Context local_context = context; + local_context.setSettings(task_cluster->settings_pull); + return InterpreterFactory::get(query_ast, local_context)->execute().in->read().rows() != 0; +} + +/// TODO: Implement checkPresentPartitionPiecesOnCurrentShard(); +/// Just copypaste the function above +bool ClusterCopier::checkPresentPartitionPiecesOnCurrentShard(const ConnectionTimeouts & timeouts, + TaskShard & task_shard, const String & partition_quoted_name, size_t current_piece_number) +{ + createShardInternalTables(timeouts, task_shard, false); + + TaskTable & task_table = task_shard.task_table; + + std::string query = "SELECT 1 FROM " + getQuotedTable(task_shard.table_read_shard) + + " WHERE (" + queryToString(task_table.engine_push_partition_key_ast) + " = (" + partition_quoted_name + " AS partition_key))"; + + const size_t number_of_splits = task_table.number_of_splits; + const String & primary_key_comma_separated = task_table.primary_key_comma_separated; + + query += " AND (cityHash64(" + primary_key_comma_separated + ") % " + + std::to_string(number_of_splits) + " = " + std::to_string(current_piece_number) + " )"; + + if (!task_table.where_condition_str.empty()) + query += " AND (" + task_table.where_condition_str + ")"; + + query += " LIMIT 1"; + + LOG_DEBUG(log, "Checking shard " << task_shard.getDescription() << " for partition " + << partition_quoted_name << " piece " << std::to_string(current_piece_number) << "existence, executing query: " << query); + + ParserQuery parser_query(query.data() + query.size()); + ASTPtr query_ast = parseQuery(parser_query, query, 0); + + Context local_context = context; + local_context.setSettings(task_cluster->settings_pull); + auto result = InterpreterFactory::get(query_ast, local_context)->execute().in->read().rows(); + if (result != 0) + LOG_DEBUG(log, "Partition " << partition_quoted_name << " piece number " + << std::to_string(current_piece_number) << " is PRESENT on shard " << task_shard.getDescription()); + else + LOG_DEBUG(log, "Partition " << partition_quoted_name << " piece number " + << std::to_string(current_piece_number) << " is ABSENT on shard " << task_shard.getDescription()); + return result != 0; +} + +/** Executes simple query (without output streams, for example DDL queries) on each shard of the cluster + * Returns number of shards for which at least one replica executed query successfully + */ +UInt64 ClusterCopier::executeQueryOnCluster( + const ClusterPtr & cluster, + const String & query, + const ASTPtr & query_ast_ = nullptr, + const Settings * settings = nullptr, + PoolMode pool_mode = PoolMode::GET_ALL, + UInt64 max_successful_executions_per_shard = 0) const +{ + auto num_shards = cluster->getShardsInfo().size(); + std::vector per_shard_num_successful_replicas(num_shards, 0); + + ASTPtr query_ast; + if (query_ast_ == nullptr) + { + ParserQuery p_query(query.data() + query.size()); + query_ast = parseQuery(p_query, query, 0); + } + else + query_ast = query_ast_; + + + /// We need to execute query on one replica at least + auto do_for_shard = [&] (UInt64 shard_index) + { + const Cluster::ShardInfo & shard = cluster->getShardsInfo().at(shard_index); + UInt64 & num_successful_executions = per_shard_num_successful_replicas.at(shard_index); + num_successful_executions = 0; + + auto increment_and_check_exit = [&] () -> bool + { + ++num_successful_executions; + return max_successful_executions_per_shard && num_successful_executions >= max_successful_executions_per_shard; + }; + + UInt64 num_replicas = cluster->getShardsAddresses().at(shard_index).size(); + UInt64 num_local_replicas = shard.getLocalNodeCount(); + UInt64 num_remote_replicas = num_replicas - num_local_replicas; + + /// In that case we don't have local replicas, but do it just in case + for (UInt64 i = 0; i < num_local_replicas; ++i) + { + auto interpreter = InterpreterFactory::get(query_ast, context); + interpreter->execute(); + + if (increment_and_check_exit()) + return; + } + + /// Will try to make as many as possible queries + if (shard.hasRemoteConnections()) + { + Settings current_settings = settings ? *settings : task_cluster->settings_common; + current_settings.max_parallel_replicas = num_remote_replicas ? num_remote_replicas : 1; + + auto timeouts = ConnectionTimeouts::getTCPTimeoutsWithFailover(current_settings).getSaturated(current_settings.max_execution_time); + auto connections = shard.pool->getMany(timeouts, ¤t_settings, pool_mode); + + for (auto & connection : connections) + { + if (connection.isNull()) + continue; + + try + { + /// CREATE TABLE and DROP PARTITION queries return empty block + RemoteBlockInputStream stream{*connection, query, Block{}, context, ¤t_settings}; + NullBlockOutputStream output{Block{}}; + copyData(stream, output); + + if (increment_and_check_exit()) + return; + } + catch (const Exception &) + { + LOG_INFO(log, getCurrentExceptionMessage(false, true)); + } + } + } + }; + + { + ThreadPool thread_pool(std::min(num_shards, getNumberOfPhysicalCPUCores())); + + for (UInt64 shard_index = 0; shard_index < num_shards; ++shard_index) + thread_pool.scheduleOrThrowOnError([=] { do_for_shard(shard_index); }); + + thread_pool.wait(); + } + + UInt64 successful_shards = 0; + for (UInt64 num_replicas : per_shard_num_successful_replicas) + successful_shards += (num_replicas > 0); + + return successful_shards; +} +} diff --git a/dbms/programs/copier/ClusterCopier.h b/dbms/programs/copier/ClusterCopier.h index cd0481684dc..5c438427ee7 100644 --- a/dbms/programs/copier/ClusterCopier.h +++ b/dbms/programs/copier/ClusterCopier.h @@ -36,504 +36,50 @@ public: { } - void init() - { - auto zookeeper = context.getZooKeeper(); - - task_description_watch_callback = [this] (const Coordination::WatchResponse & response) - { - if (response.error != Coordination::ZOK) - return; - UInt64 version = ++task_descprtion_version; - LOG_DEBUG(log, "Task description should be updated, local version " << version); - }; - - task_description_path = task_zookeeper_path + "/description"; - task_cluster = std::make_unique(task_zookeeper_path, working_database_name); - - reloadTaskDescription(); - task_cluster_initial_config = task_cluster_current_config; - - task_cluster->loadTasks(*task_cluster_initial_config); - context.setClustersConfig(task_cluster_initial_config, task_cluster->clusters_prefix); - - /// Set up shards and their priority - task_cluster->random_engine.seed(task_cluster->random_device()); - for (auto & task_table : task_cluster->table_tasks) - { - task_table.cluster_pull = context.getCluster(task_table.cluster_pull_name); - task_table.cluster_push = context.getCluster(task_table.cluster_push_name); - task_table.initShards(task_cluster->random_engine); - } - - LOG_DEBUG(log, "Will process " << task_cluster->table_tasks.size() << " table tasks"); - - /// Do not initialize tables, will make deferred initialization in process() - - zookeeper->createAncestors(getWorkersPathVersion() + "/"); - zookeeper->createAncestors(getWorkersPath() + "/"); - } + void init(); template - decltype(auto) retry(T && func, UInt64 max_tries = 100) - { - std::exception_ptr exception; + decltype(auto) retry(T && func, UInt64 max_tries = 100); - for (UInt64 try_number = 1; try_number <= max_tries; ++try_number) - { - try - { - return func(); - } - catch (...) - { - exception = std::current_exception(); - if (try_number < max_tries) - { - tryLogCurrentException(log, "Will retry"); - std::this_thread::sleep_for(default_sleep_time); - } - } - } - - std::rethrow_exception(exception); - } - - - void discoverShardPartitions(const ConnectionTimeouts & timeouts, const TaskShardPtr & task_shard) - { - TaskTable & task_table = task_shard->task_table; - - LOG_INFO(log, "Discover partitions of shard " << task_shard->getDescription()); - - auto get_partitions = [&] () { return getShardPartitions(timeouts, *task_shard); }; - auto existing_partitions_names = retry(get_partitions, 60); - Strings filtered_partitions_names; - Strings missing_partitions; - - /// Check that user specified correct partition names - auto check_partition_format = [] (const DataTypePtr & type, const String & partition_text_quoted) - { - MutableColumnPtr column_dummy = type->createColumn(); - ReadBufferFromString rb(partition_text_quoted); - - try - { - type->deserializeAsTextQuoted(*column_dummy, rb, FormatSettings()); - } - catch (Exception & e) - { - throw Exception("Partition " + partition_text_quoted + " has incorrect format. " + e.displayText(), ErrorCodes::BAD_ARGUMENTS); - } - }; - - if (task_table.has_enabled_partitions) - { - /// Process partition in order specified by - for (const String & partition_name : task_table.enabled_partitions) - { - /// Check that user specified correct partition names - check_partition_format(task_shard->partition_key_column.type, partition_name); - - auto it = existing_partitions_names.find(partition_name); - - /// Do not process partition if it is not in enabled_partitions list - if (it == existing_partitions_names.end()) - { - missing_partitions.emplace_back(partition_name); - continue; - } - - filtered_partitions_names.emplace_back(*it); - } - - for (const String & partition_name : existing_partitions_names) - { - if (!task_table.enabled_partitions_set.count(partition_name)) - { - LOG_DEBUG(log, "Partition " << partition_name << " will not be processed, since it is not in " - << "enabled_partitions of " << task_table.table_id); - } - } - } - else - { - for (const String & partition_name : existing_partitions_names) - filtered_partitions_names.emplace_back(partition_name); - } - - for (const String & partition_name : filtered_partitions_names) - { - task_shard->partition_tasks.emplace(partition_name, ShardPartition(*task_shard, partition_name, 10)); - task_shard->checked_partitions.emplace(partition_name, true); - - auto shard_partition_it = task_shard->partition_tasks.find(partition_name); - PartitionPieces & shard_partition_pieces = shard_partition_it->second.pieces; - - const size_t number_of_splits = task_table.number_of_splits; - - for (size_t piece_number = 0; piece_number < number_of_splits; ++piece_number) - { - bool res = checkPresentPartitionPiecesOnCurrentShard(timeouts, *task_shard, partition_name, piece_number); - shard_partition_pieces.emplace_back(shard_partition_it->second, piece_number, res); - } - } - - if (!missing_partitions.empty()) - { - std::stringstream ss; - for (const String & missing_partition : missing_partitions) - ss << " " << missing_partition; - - LOG_WARNING(log, "There are no " << missing_partitions.size() << " partitions from enabled_partitions in shard " - << task_shard->getDescription() << " :" << ss.str()); - } - - LOG_DEBUG(log, "Will copy " << task_shard->partition_tasks.size() << " partitions from shard " << task_shard->getDescription()); - } + void discoverShardPartitions(const ConnectionTimeouts & timeouts, const TaskShardPtr & task_shard); /// Compute set of partitions, assume set of partitions aren't changed during the processing - void discoverTablePartitions(const ConnectionTimeouts & timeouts, TaskTable & task_table, UInt64 num_threads = 0) - { - /// Fetch partitions list from a shard - { - ThreadPool thread_pool(num_threads ? num_threads : 2 * getNumberOfPhysicalCPUCores()); + void discoverTablePartitions(const ConnectionTimeouts & timeouts, TaskTable & task_table, UInt64 num_threads = 0); - for (const TaskShardPtr & task_shard : task_table.all_shards) - thread_pool.scheduleOrThrowOnError([this, timeouts, task_shard]() { discoverShardPartitions(timeouts, task_shard); }); + void uploadTaskDescription(const std::string & task_path, const std::string & task_file, const bool force); - LOG_DEBUG(log, "Waiting for " << thread_pool.active() << " setup jobs"); - thread_pool.wait(); - } - } + void reloadTaskDescription(); - void uploadTaskDescription(const std::string & task_path, const std::string & task_file, const bool force) - { - auto local_task_description_path = task_path + "/description"; + void updateConfigIfNeeded(); - String task_config_str; - { - ReadBufferFromFile in(task_file); - readStringUntilEOF(task_config_str, in); - } - if (task_config_str.empty()) - return; - - auto zookeeper = context.getZooKeeper(); - - zookeeper->createAncestors(local_task_description_path); - auto code = zookeeper->tryCreate(local_task_description_path, task_config_str, zkutil::CreateMode::Persistent); - if (code && force) - zookeeper->createOrUpdate(local_task_description_path, task_config_str, zkutil::CreateMode::Persistent); - - LOG_DEBUG(log, "Task description " << ((code && !force) ? "not " : "") << "uploaded to " << local_task_description_path << " with result " << code << " ("<< zookeeper->error2string(code) << ")"); - } - - void reloadTaskDescription() - { - auto zookeeper = context.getZooKeeper(); - task_description_watch_zookeeper = zookeeper; - - String task_config_str; - Coordination::Stat stat{}; - int code; - - zookeeper->tryGetWatch(task_description_path, task_config_str, &stat, task_description_watch_callback, &code); - if (code) - throw Exception("Can't get description node " + task_description_path, ErrorCodes::BAD_ARGUMENTS); - - LOG_DEBUG(log, "Loading description, zxid=" << task_description_current_stat.czxid); - auto config = getConfigurationFromXMLString(task_config_str); - - /// Setup settings - task_cluster->reloadSettings(*config); - context.getSettingsRef() = task_cluster->settings_common; - - task_cluster_current_config = config; - task_description_current_stat = stat; - } - - void updateConfigIfNeeded() - { - UInt64 version_to_update = task_descprtion_version; - bool is_outdated_version = task_descprtion_current_version != version_to_update; - bool is_expired_session = !task_description_watch_zookeeper || task_description_watch_zookeeper->expired(); - - if (!is_outdated_version && !is_expired_session) - return; - - LOG_DEBUG(log, "Updating task description"); - reloadTaskDescription(); - - task_descprtion_current_version = version_to_update; - } - - void process(const ConnectionTimeouts & timeouts) - { - for (TaskTable & task_table : task_cluster->table_tasks) - { - LOG_INFO(log, "Process table task " << task_table.table_id << " with " - << task_table.all_shards.size() << " shards, " - << task_table.local_shards.size() << " of them are local ones"); - - if (task_table.all_shards.empty()) - continue; - - /// Discover partitions of each shard and total set of partitions - if (!task_table.has_enabled_partitions) - { - /// If there are no specified enabled_partitions, we must discover them manually - discoverTablePartitions(timeouts, task_table); - - /// After partitions of each shard are initialized, initialize cluster partitions - for (const TaskShardPtr & task_shard : task_table.all_shards) - { - for (const auto & partition_elem : task_shard->partition_tasks) - { - const String & partition_name = partition_elem.first; - task_table.cluster_partitions.emplace(partition_name, ClusterPartition{}); - } - } - - for (auto & partition_elem : task_table.cluster_partitions) - { - const String & partition_name = partition_elem.first; - - for (const TaskShardPtr & task_shard : task_table.all_shards) - task_shard->checked_partitions.emplace(partition_name); - - task_table.ordered_partition_names.emplace_back(partition_name); - } - } - else - { - /// If enabled_partitions are specified, assume that each shard has all partitions - /// We will refine partition set of each shard in future - - for (const String & partition_name : task_table.enabled_partitions) - { - task_table.cluster_partitions.emplace(partition_name, ClusterPartition{}); - task_table.ordered_partition_names.emplace_back(partition_name); - } - } - - task_table.watch.restart(); - - /// Retry table processing - bool table_is_done = false; - for (UInt64 num_table_tries = 0; num_table_tries < max_table_tries; ++num_table_tries) - { - if (tryProcessTable(timeouts, task_table)) - { - table_is_done = true; - break; - } - } - - if (!table_is_done) - { - throw Exception("Too many tries to process table " + task_table.table_id + ". Abort remaining execution", - ErrorCodes::UNFINISHED); - } - } - } + void process(const ConnectionTimeouts & timeouts); /// Disables DROP PARTITION commands that used to clear data after errors - void setSafeMode(bool is_safe_mode_ = true) - { - is_safe_mode = is_safe_mode_; - } - - void setCopyFaultProbability(double copy_fault_probability_) - { - copy_fault_probability = copy_fault_probability_; - } + void setSafeMode(bool is_safe_mode_ = true); + void setCopyFaultProbability(double copy_fault_probability_); protected: - String getWorkersPath() const - { - return task_cluster->task_zookeeper_path + "/task_active_workers"; - } + String getWorkersPath() const; - String getWorkersPathVersion() const - { - return getWorkersPath() + "_version"; - } + String getWorkersPathVersion() const; - String getCurrentWorkerNodePath() const - { - return getWorkersPath() + "/" + host_id; - } + String getCurrentWorkerNodePath() const; zkutil::EphemeralNodeHolder::Ptr createTaskWorkerNodeAndWaitIfNeed( const zkutil::ZooKeeperPtr & zookeeper, const String & description, - bool unprioritized) - { - std::chrono::milliseconds current_sleep_time = default_sleep_time; - static constexpr std::chrono::milliseconds max_sleep_time(30000); // 30 sec - - if (unprioritized) - std::this_thread::sleep_for(current_sleep_time); - - String workers_version_path = getWorkersPathVersion(); - String workers_path = getWorkersPath(); - String current_worker_path = getCurrentWorkerNodePath(); - - UInt64 num_bad_version_errors = 0; - - while (true) - { - updateConfigIfNeeded(); - - Coordination::Stat stat{}; - zookeeper->get(workers_version_path, &stat); - auto version = stat.version; - zookeeper->get(workers_path, &stat); - - if (static_cast(stat.numChildren) >= task_cluster->max_workers) - { - LOG_DEBUG(log, "Too many workers (" << stat.numChildren << ", maximum " << task_cluster->max_workers << ")" - << ". Postpone processing " << description); - - if (unprioritized) - current_sleep_time = std::min(max_sleep_time, current_sleep_time + default_sleep_time); - - std::this_thread::sleep_for(current_sleep_time); - num_bad_version_errors = 0; - } - else - { - Coordination::Requests ops; - ops.emplace_back(zkutil::makeSetRequest(workers_version_path, description, version)); - ops.emplace_back(zkutil::makeCreateRequest(current_worker_path, description, zkutil::CreateMode::Ephemeral)); - Coordination::Responses responses; - auto code = zookeeper->tryMulti(ops, responses); - - if (code == Coordination::ZOK || code == Coordination::ZNODEEXISTS) - return std::make_shared(current_worker_path, *zookeeper, false, false, description); - - if (code == Coordination::ZBADVERSION) - { - ++num_bad_version_errors; - - /// Try to make fast retries - if (num_bad_version_errors > 3) - { - LOG_DEBUG(log, "A concurrent worker has just been added, will check free worker slots again"); - std::chrono::milliseconds random_sleep_time(std::uniform_int_distribution(1, 1000)(task_cluster->random_engine)); - std::this_thread::sleep_for(random_sleep_time); - num_bad_version_errors = 0; - } - } - else - throw Coordination::Exception(code); - } - } - } + bool unprioritized); /** Checks that the whole partition of a table was copied. We should do it carefully due to dirty lock. * State of some task could change during the processing. * We have to ensure that all shards have the finished state and there is no dirty flag. * Moreover, we have to check status twice and check zxid, because state can change during the checking. */ - bool checkPartitionIsDone(const TaskTable & task_table, const String & partition_name, const TasksShard & shards_with_partition) - { - LOG_DEBUG(log, "Check that all shards processed partition " << partition_name << " successfully"); - - auto zookeeper = context.getZooKeeper(); - - Strings status_paths; - for (auto & shard : shards_with_partition) - { - ShardPartition & task_shard_partition = shard->partition_tasks.find(partition_name)->second; - status_paths.emplace_back(task_shard_partition.getShardStatusPath()); - } - - std::vector zxid1, zxid2; - - try - { - std::vector get_futures; - for (const String & path : status_paths) - get_futures.emplace_back(zookeeper->asyncGet(path)); - - // Check that state is Finished and remember zxid - for (auto & future : get_futures) - { - auto res = future.get(); - - TaskStateWithOwner status = TaskStateWithOwner::fromString(res.data); - if (status.state != TaskState::Finished) - { - LOG_INFO(log, "The task " << res.data << " is being rewritten by " << status.owner << ". Partition will be rechecked"); - return false; - } - - zxid1.push_back(res.stat.pzxid); - } - - // Check that partition is not dirty - { - CleanStateClock clean_state_clock ( - zookeeper, - task_table.getCertainPartitionIsDirtyPath(partition_name), - task_table.getCertainPartitionIsCleanedPath(partition_name) - ); - Coordination::Stat stat{}; - LogicalClock task_start_clock; - if (zookeeper->exists(task_table.getCertainPartitionTaskStatusPath(partition_name), &stat)) - task_start_clock = LogicalClock(stat.mzxid); - zookeeper->get(task_table.getCertainPartitionTaskStatusPath(partition_name), &stat); - if (!clean_state_clock.is_clean() || task_start_clock <= clean_state_clock.discovery_zxid) - { - LOG_INFO(log, "Partition " << partition_name << " become dirty"); - return false; - } - } - - get_futures.clear(); - for (const String & path : status_paths) - get_futures.emplace_back(zookeeper->asyncGet(path)); - - // Remember zxid of states again - for (auto & future : get_futures) - { - auto res = future.get(); - zxid2.push_back(res.stat.pzxid); - } - } - catch (const Coordination::Exception & e) - { - LOG_INFO(log, "A ZooKeeper error occurred while checking partition " << partition_name - << ". Will recheck the partition. Error: " << e.displayText()); - return false; - } - - // If all task is finished and zxid is not changed then partition could not become dirty again - for (UInt64 shard_num = 0; shard_num < status_paths.size(); ++shard_num) - { - if (zxid1[shard_num] != zxid2[shard_num]) - { - LOG_INFO(log, "The task " << status_paths[shard_num] << " is being modified now. Partition will be rechecked"); - return false; - } - } - - LOG_INFO(log, "Partition " << partition_name << " is copied successfully"); - return true; - } - - bool checkAllPieceInPartitionDone(const TaskTable & task_table, const String & partition_name, const TasksShard & shards_with_partition) - { - bool answer = true; - for (size_t piece_number = 0; piece_number < task_table.number_of_splits; piece_number++) - answer &= checkPartitionPieceIsDone(task_table, partition_name, piece_number, shards_with_partition); - return answer; - } + bool checkPartitionIsDone(const TaskTable & task_table, const String & partition_name, const TasksShard & shards_with_partition); + bool checkAllPieceInPartitionDone(const TaskTable & task_table, const String & partition_name, const TasksShard & shards_with_partition); /* The same as function above * Assume that we don't know on which shards do we have partition certain piece. @@ -541,440 +87,21 @@ public: * And shards that don't have certain piece MUST mark that piece is_done true. * */ bool checkPartitionPieceIsDone(const TaskTable & task_table, const String & partition_name, - size_t piece_number, const TasksShard & shards_with_partition) - { - LOG_DEBUG(log, "Check that all shards processed partition " << partition_name - << " piece number" + toString(piece_number) + " successfully"); - - auto zookeeper = context.getZooKeeper(); - - /// Collect all shards that contain partition piece number piece_number. - Strings piece_status_paths; - for (auto & shard : shards_with_partition) - { - ShardPartition & task_shard_partition = shard->partition_tasks.find(partition_name)->second; - ShardPartitionPiece & shard_partition_piece = task_shard_partition.pieces[piece_number]; - piece_status_paths.emplace_back(shard_partition_piece.getShardStatusPath()); - } - - std::vector zxid1, zxid2; - - try - { - std::vector get_futures; - for (const String & path : piece_status_paths) - get_futures.emplace_back(zookeeper->asyncGet(path)); - - // Check that state is Finished and remember zxid - for (auto & future : get_futures) - { - auto res = future.get(); - - TaskStateWithOwner status = TaskStateWithOwner::fromString(res.data); - if (status.state != TaskState::Finished) - { - LOG_INFO(log, "The task " << res.data << " is being rewritten by " << status.owner << ". Partition piece will be rechecked"); - return false; - } - - zxid1.push_back(res.stat.pzxid); - } - - // Check that partition is not dirty - { - CleanStateClock clean_state_clock ( - zookeeper, - task_table.getCertainPartitionIsDirtyPath(partition_name), - task_table.getCertainPartitionIsCleanedPath(partition_name) - ); - Coordination::Stat stat{}; - LogicalClock task_start_clock; - if (zookeeper->exists(task_table.getCertainPartitionTaskStatusPath(partition_name), &stat)) - task_start_clock = LogicalClock(stat.mzxid); - zookeeper->get(task_table.getCertainPartitionTaskStatusPath(partition_name), &stat); - if (!clean_state_clock.is_clean() || task_start_clock <= clean_state_clock.discovery_zxid) - { - LOG_INFO(log, "Partition " << partition_name << " piece number " << toString(piece_number) << " become dirty"); - return false; - } - } - - get_futures.clear(); - for (const String & path : piece_status_paths) - get_futures.emplace_back(zookeeper->asyncGet(path)); - - // Remember zxid of states again - for (auto & future : get_futures) - { - auto res = future.get(); - zxid2.push_back(res.stat.pzxid); - } - } - catch (const Coordination::Exception & e) - { - LOG_INFO(log, "A ZooKeeper error occurred while checking partition " << partition_name << " piece number " - << toString(piece_number) << ". Will recheck the partition. Error: " << e.displayText()); - return false; - } - - // If all task is finished and zxid is not changed then partition could not become dirty again - for (UInt64 shard_num = 0; shard_num < piece_status_paths.size(); ++shard_num) - { - if (zxid1[shard_num] != zxid2[shard_num]) - { - LOG_INFO(log, "The task " << piece_status_paths[shard_num] << " is being modified now. Partition piece will be rechecked"); - return false; - } - } - - LOG_INFO(log, "Partition " << partition_name << " is copied successfully"); - return true; - - } + size_t piece_number, const TasksShard & shards_with_partition); /// Removes MATERIALIZED and ALIAS columns from create table query - static ASTPtr removeAliasColumnsFromCreateQuery(const ASTPtr & query_ast) - { - const ASTs & column_asts = query_ast->as().columns_list->columns->children; - auto new_columns = std::make_shared(); - - for (const ASTPtr & column_ast : column_asts) - { - const auto & column = column_ast->as(); - - if (!column.default_specifier.empty()) - { - ColumnDefaultKind kind = columnDefaultKindFromString(column.default_specifier); - if (kind == ColumnDefaultKind::Materialized || kind == ColumnDefaultKind::Alias) - continue; - } - - new_columns->children.emplace_back(column_ast->clone()); - } - - ASTPtr new_query_ast = query_ast->clone(); - auto & new_query = new_query_ast->as(); - - auto new_columns_list = std::make_shared(); - new_columns_list->set(new_columns_list->columns, new_columns); - if (auto indices = query_ast->as()->columns_list->indices) - new_columns_list->set(new_columns_list->indices, indices->clone()); - - new_query.replace(new_query.columns_list, new_columns_list); - - return new_query_ast; - } + static ASTPtr removeAliasColumnsFromCreateQuery(const ASTPtr & query_ast); /// Replaces ENGINE and table name in a create query - static std::shared_ptr rewriteCreateQueryStorage(const ASTPtr & create_query_ast, const DatabaseAndTableName & new_table, const ASTPtr & new_storage_ast) - { - const auto & create = create_query_ast->as(); - auto res = std::make_shared(create); - - if (create.storage == nullptr || new_storage_ast == nullptr) - throw Exception("Storage is not specified", ErrorCodes::LOGICAL_ERROR); - - res->database = new_table.first; - res->table = new_table.second; - - res->children.clear(); - res->set(res->columns_list, create.columns_list->clone()); - res->set(res->storage, new_storage_ast->clone()); - - return res; - } - - bool tryDropPartition(ShardPartition & task_partition, const zkutil::ZooKeeperPtr & zookeeper, const CleanStateClock & clean_state_clock) - { - if (is_safe_mode) - throw Exception("DROP PARTITION is prohibited in safe mode", ErrorCodes::NOT_IMPLEMENTED); - - TaskTable & task_table = task_partition.task_shard.task_table; - - const String current_shards_path = task_partition.getPartitionShardsPath(); - const String current_partition_active_workers_dir = task_partition.getPartitionActiveWorkersPath(); - const String is_dirty_flag_path = task_partition.getCommonPartitionIsDirtyPath(); - const String dirty_cleaner_path = is_dirty_flag_path + "/cleaner"; - const String is_dirty_cleaned_path = task_partition.getCommonPartitionIsCleanedPath(); - - zkutil::EphemeralNodeHolder::Ptr cleaner_holder; - try - { - cleaner_holder = zkutil::EphemeralNodeHolder::create(dirty_cleaner_path, *zookeeper, host_id); - } - catch (const Coordination::Exception & e) - { - if (e.code == Coordination::ZNODEEXISTS) - { - LOG_DEBUG(log, "Partition " << task_partition.name << " is cleaning now by somebody, sleep"); - std::this_thread::sleep_for(default_sleep_time); - return false; - } - - throw; - } - - Coordination::Stat stat{}; - if (zookeeper->exists(current_partition_active_workers_dir, &stat)) - { - if (stat.numChildren != 0) - { - LOG_DEBUG(log, "Partition " << task_partition.name << " contains " << stat.numChildren << " active workers while trying to drop it. Going to sleep."); - std::this_thread::sleep_for(default_sleep_time); - return false; - } - else - { - zookeeper->remove(current_partition_active_workers_dir); - } - } - - { - zkutil::EphemeralNodeHolder::Ptr active_workers_lock; - try - { - active_workers_lock = zkutil::EphemeralNodeHolder::create(current_partition_active_workers_dir, *zookeeper, host_id); - } - catch (const Coordination::Exception & e) - { - if (e.code == Coordination::ZNODEEXISTS) - { - LOG_DEBUG(log, "Partition " << task_partition.name << " is being filled now by somebody, sleep"); - return false; - } - - throw; - } - - // Lock the dirty flag - zookeeper->set(is_dirty_flag_path, host_id, clean_state_clock.discovery_version.value()); - zookeeper->tryRemove(task_partition.getPartitionCleanStartPath()); - CleanStateClock my_clock(zookeeper, is_dirty_flag_path, is_dirty_cleaned_path); - - /// Remove all status nodes - { - Strings children; - if (zookeeper->tryGetChildren(current_shards_path, children) == Coordination::ZOK) - for (const auto & child : children) - { - zookeeper->removeRecursive(current_shards_path + "/" + child); - } - } - - String query = "ALTER TABLE " + getQuotedTable(task_table.table_push); - query += " DROP PARTITION " + task_partition.name + ""; - - /// TODO: use this statement after servers will be updated up to 1.1.54310 - // query += " DROP PARTITION ID '" + task_partition.name + "'"; - - ClusterPtr & cluster_push = task_table.cluster_push; - Settings settings_push = task_cluster->settings_push; - - /// It is important, DROP PARTITION must be done synchronously - settings_push.replication_alter_partitions_sync = 2; - - LOG_DEBUG(log, "Execute distributed DROP PARTITION: " << query); - /// Limit number of max executing replicas to 1 - UInt64 num_shards = executeQueryOnCluster(cluster_push, query, nullptr, &settings_push, PoolMode::GET_ONE, 1); - - if (num_shards < cluster_push->getShardCount()) - { - LOG_INFO(log, "DROP PARTITION wasn't successfully executed on " << cluster_push->getShardCount() - num_shards << " shards"); - return false; - } - - /// Update the locking node - if (!my_clock.is_stale()) - { - zookeeper->set(is_dirty_flag_path, host_id, my_clock.discovery_version.value()); - if (my_clock.clean_state_version) - zookeeper->set(is_dirty_cleaned_path, host_id, my_clock.clean_state_version.value()); - else - zookeeper->create(is_dirty_cleaned_path, host_id, zkutil::CreateMode::Persistent); - } - else - { - LOG_DEBUG(log, "Clean state is altered when dropping the partition, cowardly bailing"); - /// clean state is stale - return false; - } - - LOG_INFO(log, "Partition " << task_partition.name << " was dropped on cluster " << task_table.cluster_push_name); - if (zookeeper->tryCreate(current_shards_path, host_id, zkutil::CreateMode::Persistent) == Coordination::ZNODEEXISTS) - zookeeper->set(current_shards_path, host_id); - } - - LOG_INFO(log, "Partition " << task_partition.name << " is safe for work now."); - return true; - } + static std::shared_ptr rewriteCreateQueryStorage(const ASTPtr & create_query_ast, + const DatabaseAndTableName & new_table, const ASTPtr & new_storage_ast); + bool tryDropPartition(ShardPartition & task_partition, const zkutil::ZooKeeperPtr & zookeeper, const CleanStateClock & clean_state_clock); static constexpr UInt64 max_table_tries = 1000; static constexpr UInt64 max_shard_partition_tries = 600; - bool tryProcessTable(const ConnectionTimeouts & timeouts, TaskTable & task_table) - { - /// A heuristic: if previous shard is already done, then check next one without sleeps due to max_workers constraint - bool previous_shard_is_instantly_finished = false; - - /// Process each partition that is present in cluster - for (const String & partition_name : task_table.ordered_partition_names) - { - if (!task_table.cluster_partitions.count(partition_name)) - throw Exception("There are no expected partition " + partition_name + ". It is a bug", ErrorCodes::LOGICAL_ERROR); - - ClusterPartition & cluster_partition = task_table.cluster_partitions[partition_name]; - - Stopwatch watch; - /// We will check all the shards of the table and check if they contain current partition. - TasksShard expected_shards; - UInt64 num_failed_shards = 0; - - ++cluster_partition.total_tries; - - LOG_DEBUG(log, "Processing partition " << partition_name << " for the whole cluster"); - - /// Process each source shard having current partition and copy current partition - /// NOTE: shards are sorted by "distance" to current host - bool has_shard_to_process = false; - for (const TaskShardPtr & shard : task_table.all_shards) - { - /// Does shard have a node with current partition? - if (shard->partition_tasks.count(partition_name) == 0) - { - /// If not, did we check existence of that partition previously? - if (shard->checked_partitions.count(partition_name) == 0) - { - auto check_shard_has_partition = [&] () { return checkShardHasPartition(timeouts, *shard, partition_name); }; - bool has_partition = retry(check_shard_has_partition); - - shard->checked_partitions.emplace(partition_name); - - if (has_partition) - { - shard->partition_tasks.emplace(partition_name, ShardPartition(*shard, partition_name, 10)); - LOG_DEBUG(log, "Discovered partition " << partition_name << " in shard " << shard->getDescription()); - /// To save references in the future. - auto shard_partition_it = shard->partition_tasks.find(partition_name); - PartitionPieces & shard_partition_pieces = shard_partition_it->second.pieces; - - const size_t number_of_splits = task_table.number_of_splits; - - for (size_t piece_number = 0; piece_number < number_of_splits; ++piece_number) - { - auto res = checkPresentPartitionPiecesOnCurrentShard(timeouts, *shard, partition_name, piece_number); - shard_partition_pieces.emplace_back(shard_partition_it->second, piece_number, res); - } - } - else - { - LOG_DEBUG(log, "Found that shard " << shard->getDescription() << " does not contain current partition " << partition_name); - continue; - } - } - else - { - /// We have already checked that partition, but did not discover it - previous_shard_is_instantly_finished = true; - continue; - } - } - - auto it_shard_partition = shard->partition_tasks.find(partition_name); - /// Previously when we discovered that shard does not contain current partition, we skipped it. - /// At this moment partition have to be present. - if (it_shard_partition == shard->partition_tasks.end()) - throw Exception("There are no such partition in a shard. This is a bug.", ErrorCodes::LOGICAL_ERROR); - auto & partition = it_shard_partition->second; - - expected_shards.emplace_back(shard); - - /// Do not sleep if there is a sequence of already processed shards to increase startup - bool is_unprioritized_task = !previous_shard_is_instantly_finished && shard->priority.is_remote; - PartitionTaskStatus task_status = PartitionTaskStatus::Error; - bool was_error = false; - has_shard_to_process = true; - for (UInt64 try_num = 0; try_num < max_shard_partition_tries; ++try_num) - { - task_status = tryProcessPartitionTask(timeouts, partition, is_unprioritized_task); - - /// Exit if success - if (task_status == PartitionTaskStatus::Finished) - break; - - was_error = true; - - /// Skip if the task is being processed by someone - if (task_status == PartitionTaskStatus::Active) - break; - - /// Repeat on errors - std::this_thread::sleep_for(default_sleep_time); - } - - if (task_status == PartitionTaskStatus::Error) - ++num_failed_shards; - - previous_shard_is_instantly_finished = !was_error; - } - - cluster_partition.elapsed_time_seconds += watch.elapsedSeconds(); - - /// Check that whole cluster partition is done - /// Firstly check the number of failed partition tasks, then look into ZooKeeper and ensure that each partition is done - bool partition_is_done = num_failed_shards == 0; - try - { - partition_is_done = - !has_shard_to_process - || (partition_is_done && checkPartitionIsDone(task_table, partition_name, expected_shards)); - } - catch (...) - { - tryLogCurrentException(log); - partition_is_done = false; - } - - if (partition_is_done) - { - task_table.finished_cluster_partitions.emplace(partition_name); - - task_table.bytes_copied += cluster_partition.bytes_copied; - task_table.rows_copied += cluster_partition.rows_copied; - double elapsed = cluster_partition.elapsed_time_seconds; - - LOG_INFO(log, "It took " << std::fixed << std::setprecision(2) << elapsed << " seconds to copy partition " << partition_name - << ": " << formatReadableSizeWithDecimalSuffix(cluster_partition.bytes_copied) << " uncompressed bytes" - << ", " << formatReadableQuantity(cluster_partition.rows_copied) << " rows" - << " and " << cluster_partition.blocks_copied << " source blocks are copied"); - - if (cluster_partition.rows_copied) - { - LOG_INFO(log, "Average partition speed: " - << formatReadableSizeWithDecimalSuffix(cluster_partition.bytes_copied / elapsed) << " per second."); - } - - if (task_table.rows_copied) - { - LOG_INFO(log, "Average table " << task_table.table_id << " speed: " - << formatReadableSizeWithDecimalSuffix(task_table.bytes_copied / elapsed) << " per second."); - } - } - } - - UInt64 required_partitions = task_table.cluster_partitions.size(); - UInt64 finished_partitions = task_table.finished_cluster_partitions.size(); - bool table_is_done = finished_partitions >= required_partitions; - - if (!table_is_done) - { - LOG_INFO(log, "Table " + task_table.table_id + " is not processed yet." - << "Copied " << finished_partitions << " of " << required_partitions << ", will retry"); - } - - return table_is_done; - } - + bool tryProcessTable(const ConnectionTimeouts & timeouts, TaskTable & task_table); /// Execution status of a task enum class PartitionTaskStatus @@ -993,577 +120,33 @@ public: }; /// Job for copying partition from particular shard. - PartitionTaskStatus tryProcessPartitionTask(const ConnectionTimeouts & timeouts, ShardPartition & task_partition, bool is_unprioritized_task) - { - PartitionTaskStatus res; - - try - { - res = iterateThroughAllPiecesInPartition(timeouts, task_partition, is_unprioritized_task); - } - catch (...) - { - tryLogCurrentException(log, "An error occurred while processing partition " + task_partition.name); - res = PartitionTaskStatus::Error; - } - - /// At the end of each task check if the config is updated - try - { - updateConfigIfNeeded(); - } - catch (...) - { - tryLogCurrentException(log, "An error occurred while updating the config"); - } - - return res; - } + PartitionTaskStatus tryProcessPartitionTask(const ConnectionTimeouts & timeouts, ShardPartition & task_partition, bool is_unprioritized_task); PartitionTaskStatus iterateThroughAllPiecesInPartition(const ConnectionTimeouts & timeouts, ShardPartition & task_partition, - bool is_unprioritized_task) - { - const size_t total_number_of_pieces = task_partition.task_shard.task_table.number_of_splits; - - /// ThreadPool maybe ?? - for (size_t piece_number = 0; piece_number < total_number_of_pieces; piece_number++) - processPartitionPieceTaskImpl(timeouts, task_partition, piece_number, is_unprioritized_task); - - return PartitionTaskStatus::Finished; - - } + bool is_unprioritized_task); PartitionTaskStatus processPartitionPieceTaskImpl(const ConnectionTimeouts & timeouts, ShardPartition & task_partition, - const size_t current_piece_number, bool is_unprioritized_task) - { - TaskShard & task_shard = task_partition.task_shard; - TaskTable & task_table = task_shard.task_table; - ClusterPartition & cluster_partition = task_table.getClusterPartition(task_partition.name); - ShardPartitionPiece & partition_piece = task_partition.pieces[current_piece_number]; + const size_t current_piece_number, bool is_unprioritized_task); - const size_t number_of_splits = task_table.number_of_splits; - const String primary_key_comma_separated = task_table.primary_key_comma_separated; - UNUSED(number_of_splits); - UNUSED(partition_piece); - /// We need to update table definitions for each partition, it could be changed after ALTER - createShardInternalTables(timeouts, task_shard, true, current_piece_number); + void dropAndCreateLocalTable(const ASTPtr & create_ast); - auto zookeeper = context.getZooKeeper(); + void dropLocalTableIfExists(const DatabaseAndTableName & table_name) const; - const String piece_is_dirty_flag_path = partition_piece.getPartitionPieceIsDirtyPath(); - const String piece_is_dirty_cleaned_path = partition_piece.getPartitionPieceIsCleanedPath(); - const String current_task_piece_is_active_path = partition_piece.getActiveWorkerPath(); - const String current_task_piece_status_path = partition_piece.getShardStatusPath(); + String getRemoteCreateTable(const DatabaseAndTableName & table, Connection & connection, const Settings * settings = nullptr); - /// Auxiliary functions: - - /// Creates is_dirty node to initialize DROP PARTITION - auto create_is_dirty_node = [&, this] (const CleanStateClock & clock) - { - if (clock.is_stale()) - LOG_DEBUG(log, "Clean state clock is stale while setting dirty flag, cowardly bailing"); - else if (!clock.is_clean()) - LOG_DEBUG(log, "Thank you, Captain Obvious"); - else if (clock.discovery_version) - { - LOG_DEBUG(log, "Updating clean state clock"); - zookeeper->set(piece_is_dirty_flag_path, host_id, clock.discovery_version.value()); - } - else - { - LOG_DEBUG(log, "Creating clean state clock"); - zookeeper->create(piece_is_dirty_flag_path, host_id, zkutil::CreateMode::Persistent); - } - }; - - /// Returns SELECT query filtering current partition and applying user filter - auto get_select_query = [&] (const DatabaseAndTableName & from_table, const String & fields, String limit = "", - bool enable_splitting = false) - { - String query; - query += "SELECT " + fields + " FROM " + getQuotedTable(from_table); - /// TODO: Bad, it is better to rewrite with ASTLiteral(partition_key_field) - query += " WHERE (" + queryToString(task_table.engine_push_partition_key_ast) + " = (" + task_partition.name + " AS partition_key))"; - - if (enable_splitting) - query += " AND ( cityHash64(" + primary_key_comma_separated + ") = " + toString(current_piece_number) + " )"; - - if (!task_table.where_condition_str.empty()) - query += " AND (" + task_table.where_condition_str + ")"; - if (!limit.empty()) - query += " LIMIT " + limit; - - ParserQuery p_query(query.data() + query.size()); - return parseQuery(p_query, query, 0); - }; - - /// Load balancing - auto worker_node_holder = createTaskWorkerNodeAndWaitIfNeed(zookeeper, current_task_piece_status_path, is_unprioritized_task); - - LOG_DEBUG(log, "Processing " << current_task_piece_status_path); - - CleanStateClock clean_state_clock (zookeeper, piece_is_dirty_flag_path, piece_is_dirty_cleaned_path); - - LogicalClock task_start_clock; - { - Coordination::Stat stat{}; - if (zookeeper->exists(task_partition.getPartitionShardsPath(), &stat)) - task_start_clock = LogicalClock(stat.mzxid); - } - - /// Do not start if partition is dirty, try to clean it - if (clean_state_clock.is_clean() - && (!task_start_clock.hasHappened() || clean_state_clock.discovery_zxid <= task_start_clock)) - { - LOG_DEBUG(log, "Partition " << task_partition.name << " appears to be clean"); - zookeeper->createAncestors(current_task_piece_status_path); - } - else - { - LOG_DEBUG(log, "Partition " << task_partition.name << " is dirty, try to drop it"); - - try - { - tryDropPartition(task_partition, zookeeper, clean_state_clock); - } - catch (...) - { - tryLogCurrentException(log, "An error occurred when clean partition"); - } - - return PartitionTaskStatus::Error; - } - - /// Create ephemeral node to mark that we are active and process the partition - zookeeper->createAncestors(current_task_piece_is_active_path); - zkutil::EphemeralNodeHolderPtr partition_task_node_holder; - try - { - partition_task_node_holder = zkutil::EphemeralNodeHolder::create(current_task_piece_is_active_path, *zookeeper, host_id); - } - catch (const Coordination::Exception & e) - { - if (e.code == Coordination::ZNODEEXISTS) - { - LOG_DEBUG(log, "Someone is already processing " << current_task_piece_is_active_path); - return PartitionTaskStatus::Active; - } - - throw; - } - - /// Exit if task has been already processed; - /// create blocking node to signal cleaning up if it is abandoned - { - String status_data; - if (zookeeper->tryGet(current_task_piece_status_path, status_data)) - { - TaskStateWithOwner status = TaskStateWithOwner::fromString(status_data); - if (status.state == TaskState::Finished) - { - LOG_DEBUG(log, "Task " << current_task_piece_status_path << " has been successfully executed by " << status.owner); - return PartitionTaskStatus::Finished; - } - - // Task is abandoned, initialize DROP PARTITION - LOG_DEBUG(log, "Task " << current_task_piece_status_path << " has not been successfully finished by " << - status.owner << ". Partition will be dropped and refilled."); - - create_is_dirty_node(clean_state_clock); - return PartitionTaskStatus::Error; - } - } - - /// Check that destination partition is empty if we are first worker - /// NOTE: this check is incorrect if pull and push tables have different partition key! - String clean_start_status; - if (!zookeeper->tryGet(task_partition.getPartitionCleanStartPath(), clean_start_status) || clean_start_status != "ok") - { - zookeeper->createIfNotExists(task_partition.getPartitionCleanStartPath(), ""); - auto checker = zkutil::EphemeralNodeHolder::create(task_partition.getPartitionCleanStartPath() + "/checker", - *zookeeper, host_id); - // Maybe we are the first worker - ASTPtr query_select_ast = get_select_query(task_shard.table_split_shard, "count()"); - UInt64 count; - { - Context local_context = context; - // Use pull (i.e. readonly) settings, but fetch data from destination servers - local_context.getSettingsRef() = task_cluster->settings_pull; - local_context.getSettingsRef().skip_unavailable_shards = true; - - Block block = getBlockWithAllStreamData(InterpreterFactory::get(query_select_ast, local_context)->execute().in); - count = (block) ? block.safeGetByPosition(0).column->getUInt(0) : 0; - } - - if (count != 0) - { - Coordination::Stat stat_shards{}; - zookeeper->get(task_partition.getPartitionShardsPath(), & stat_shards); - - /// NOTE: partition is still fresh if dirt discovery happens before cleaning - if (stat_shards.numChildren == 0) - { - LOG_WARNING(log, "There are no workers for partition " << task_partition.name - << ", but destination table contains " << count << " rows" - << ". Partition will be dropped and refilled."); - - create_is_dirty_node(clean_state_clock); - return PartitionTaskStatus::Error; - } - } - zookeeper->set(task_partition.getPartitionCleanStartPath(), "ok"); - } - /// At this point, we need to sync that the destination table is clean - /// before any actual work - - /// Try start processing, create node about it - { - String start_state = TaskStateWithOwner::getData(TaskState::Started, host_id); - CleanStateClock new_clean_state_clock (zookeeper, piece_is_dirty_flag_path, piece_is_dirty_cleaned_path); - if (clean_state_clock != new_clean_state_clock) - { - LOG_INFO(log, "Partition " << task_partition.name << " clean state changed, cowardly bailing"); - return PartitionTaskStatus::Error; - } - else if (!new_clean_state_clock.is_clean()) - { - LOG_INFO(log, "Partition " << task_partition.name << " is dirty and will be dropped and refilled"); - create_is_dirty_node(new_clean_state_clock); - return PartitionTaskStatus::Error; - } - zookeeper->create(current_task_piece_status_path, start_state, zkutil::CreateMode::Persistent); - } - - /// Try create table (if not exists) on each shard - { - /// Define push table for current partition piece - auto database_and_table_for_current_piece= std::pair( - task_table.table_push.first, - task_table.table_push.second + ".piece_" + toString(current_piece_number)); - - auto create_query_push_ast = rewriteCreateQueryStorage(task_shard.current_pull_table_create_query, - database_and_table_for_current_piece, task_table.engine_push_ast); - create_query_push_ast->as().if_not_exists = true; - String query = queryToString(create_query_push_ast); - - LOG_DEBUG(log, "Create destination tables. Query: " << query); - UInt64 shards = executeQueryOnCluster(task_table.cluster_push, query, - create_query_push_ast, &task_cluster->settings_push, - PoolMode::GET_MANY); - LOG_DEBUG(log, "Destination tables " << getQuotedTable(task_table.table_push) << - " have been created on " << shards << " shards of " << task_table.cluster_push->getShardCount()); - } - - /// Do the copying - { - bool inject_fault = false; - if (copy_fault_probability > 0) - { - double value = std::uniform_real_distribution<>(0, 1)(task_table.task_cluster.random_engine); - inject_fault = value < copy_fault_probability; - } - - // Select all fields - ASTPtr query_select_ast = get_select_query(task_shard.table_read_shard, "*", inject_fault ? "1" : ""); - - LOG_DEBUG(log, "Executing SELECT query and pull from " << task_shard.getDescription() - << " : " << queryToString(query_select_ast)); - - ASTPtr query_insert_ast; - { - String query; - query += "INSERT INTO " + getQuotedTable(task_shard.table_split_shard) + " VALUES "; - - ParserQuery p_query(query.data() + query.size()); - query_insert_ast = parseQuery(p_query, query, 0); - - LOG_DEBUG(log, "Executing INSERT query: " << query); - } - - try - { - /// Custom INSERT SELECT implementation - Context context_select = context; - context_select.getSettingsRef() = task_cluster->settings_pull; - - Context context_insert = context; - context_insert.getSettingsRef() = task_cluster->settings_push; - - BlockInputStreamPtr input; - BlockOutputStreamPtr output; - { - BlockIO io_select = InterpreterFactory::get(query_select_ast, context_select)->execute(); - BlockIO io_insert = InterpreterFactory::get(query_insert_ast, context_insert)->execute(); - - input = io_select.in; - output = io_insert.out; - } - - /// Fail-fast optimization to abort copying when the current clean state expires - std::future future_is_dirty_checker; - - Stopwatch watch(CLOCK_MONOTONIC_COARSE); - constexpr UInt64 check_period_milliseconds = 500; - - /// Will asynchronously check that ZooKeeper connection and is_dirty flag appearing while copying data - auto cancel_check = [&] () - { - if (zookeeper->expired()) - throw Exception("ZooKeeper session is expired, cancel INSERT SELECT", ErrorCodes::UNFINISHED); - - if (!future_is_dirty_checker.valid()) - future_is_dirty_checker = zookeeper->asyncExists(piece_is_dirty_flag_path); - - /// check_period_milliseconds should less than average insert time of single block - /// Otherwise, the insertion will slow a little bit - if (watch.elapsedMilliseconds() >= check_period_milliseconds) - { - Coordination::ExistsResponse status = future_is_dirty_checker.get(); - - if (status.error != Coordination::ZNONODE) - { - LogicalClock dirt_discovery_epoch (status.stat.mzxid); - if (dirt_discovery_epoch == clean_state_clock.discovery_zxid) - return false; - throw Exception("Partition is dirty, cancel INSERT SELECT", ErrorCodes::UNFINISHED); - } - } - - return false; - }; - - /// Update statistics - /// It is quite rough: bytes_copied don't take into account DROP PARTITION. - auto update_stats = [&cluster_partition] (const Block & block) - { - cluster_partition.bytes_copied += block.bytes(); - cluster_partition.rows_copied += block.rows(); - cluster_partition.blocks_copied += 1; - }; - - /// Main work is here - copyData(*input, *output, cancel_check, update_stats); - - // Just in case - if (future_is_dirty_checker.valid()) - future_is_dirty_checker.get(); - - if (inject_fault) - throw Exception("Copy fault injection is activated", ErrorCodes::UNFINISHED); - } - catch (...) - { - tryLogCurrentException(log, "An error occurred during copying, partition will be marked as dirty"); - return PartitionTaskStatus::Error; - } - } - - /// Finalize the processing, change state of current partition task (and also check is_dirty flag) - { - String state_finished = TaskStateWithOwner::getData(TaskState::Finished, host_id); - CleanStateClock new_clean_state_clock (zookeeper, piece_is_dirty_flag_path, piece_is_dirty_cleaned_path); - if (clean_state_clock != new_clean_state_clock) - { - LOG_INFO(log, "Partition " << task_partition.name << " clean state changed, cowardly bailing"); - return PartitionTaskStatus::Error; - } - else if (!new_clean_state_clock.is_clean()) - { - LOG_INFO(log, "Partition " << task_partition.name << " became dirty and will be dropped and refilled"); - create_is_dirty_node(new_clean_state_clock); - return PartitionTaskStatus::Error; - } - zookeeper->set(current_task_piece_status_path, state_finished, 0); - } - - LOG_INFO(log, "Partition " << task_partition.name << " copied"); - return PartitionTaskStatus::Finished; - } - - void dropAndCreateLocalTable(const ASTPtr & create_ast) - { - const auto & create = create_ast->as(); - dropLocalTableIfExists({create.database, create.table}); - - InterpreterCreateQuery interpreter(create_ast, context); - interpreter.execute(); - } - - void dropLocalTableIfExists(const DatabaseAndTableName & table_name) const - { - auto drop_ast = std::make_shared(); - drop_ast->if_exists = true; - drop_ast->database = table_name.first; - drop_ast->table = table_name.second; - - InterpreterDropQuery interpreter(drop_ast, context); - interpreter.execute(); - } - - String getRemoteCreateTable(const DatabaseAndTableName & table, Connection & connection, const Settings * settings = nullptr) - { - String query = "SHOW CREATE TABLE " + getQuotedTable(table); - Block block = getBlockWithAllStreamData(std::make_shared( - connection, query, InterpreterShowCreateQuery::getSampleBlock(), context, settings)); - - return typeid_cast(*block.safeGetByPosition(0).column).getDataAt(0).toString(); - } - - ASTPtr getCreateTableForPullShard(const ConnectionTimeouts & timeouts, TaskShard & task_shard) - { - /// Fetch and parse (possibly) new definition - auto connection_entry = task_shard.info.pool->get(timeouts, &task_cluster->settings_pull); - String create_query_pull_str = getRemoteCreateTable( - task_shard.task_table.table_pull, - *connection_entry, - &task_cluster->settings_pull); - - ParserCreateQuery parser_create_query; - return parseQuery(parser_create_query, create_query_pull_str, 0); - } + ASTPtr getCreateTableForPullShard(const ConnectionTimeouts & timeouts, TaskShard & task_shard); /// If it is implicitly asked to create split Distributed table for certain piece on current shard, we will do it. - void createShardInternalTables(const ConnectionTimeouts & timeouts, TaskShard & task_shard, bool create_split = true, const size_t piece_number = 0) - { - TaskTable & task_table = task_shard.task_table; + void createShardInternalTables(const ConnectionTimeouts & timeouts, TaskShard & task_shard, bool create_split = true, const size_t piece_number = 0); - /// We need to update table definitions for each part, it could be changed after ALTER - task_shard.current_pull_table_create_query = getCreateTableForPullShard(timeouts, task_shard); + std::set getShardPartitions(const ConnectionTimeouts & timeouts, TaskShard & task_shard); - /// Create local Distributed tables: - /// a table fetching data from current shard and a table inserting data to the whole destination cluster - String read_shard_prefix = ".read_shard_" + toString(task_shard.indexInCluster()) + "."; - String split_shard_prefix = ".split."; - task_shard.table_read_shard = DatabaseAndTableName(working_database_name, read_shard_prefix + task_table.table_id); - task_shard.table_split_shard = DatabaseAndTableName( - working_database_name, split_shard_prefix + task_table.table_id + ".piece_" + toString(piece_number)); - - /// Create special cluster with single shard - String shard_read_cluster_name = read_shard_prefix + task_table.cluster_pull_name; - ClusterPtr cluster_pull_current_shard = task_table.cluster_pull->getClusterWithSingleShard(task_shard.indexInCluster()); - context.setCluster(shard_read_cluster_name, cluster_pull_current_shard); - - auto storage_shard_ast = createASTStorageDistributed(shard_read_cluster_name, task_table.table_pull.first, task_table.table_pull.second); - const auto & storage_piece_split_ast = task_table.auxiliary_engine_split_asts[piece_number]; - - auto create_query_ast = removeAliasColumnsFromCreateQuery(task_shard.current_pull_table_create_query); - auto create_table_pull_ast = rewriteCreateQueryStorage(create_query_ast, task_shard.table_read_shard, storage_shard_ast); - auto create_table_split_piece_ast = rewriteCreateQueryStorage(create_query_ast, task_shard.table_split_shard, storage_piece_split_ast); - - dropAndCreateLocalTable(create_table_pull_ast); - - if (create_split) - dropAndCreateLocalTable(create_table_split_piece_ast); - } - - - std::set getShardPartitions(const ConnectionTimeouts & timeouts, TaskShard & task_shard) - { - createShardInternalTables(timeouts, task_shard, false); - - TaskTable & task_table = task_shard.task_table; - - String query; - { - WriteBufferFromOwnString wb; - wb << "SELECT DISTINCT " << queryToString(task_table.engine_push_partition_key_ast) << " AS partition FROM" - << " " << getQuotedTable(task_shard.table_read_shard) << " ORDER BY partition DESC"; - query = wb.str(); - } - - ParserQuery parser_query(query.data() + query.size()); - ASTPtr query_ast = parseQuery(parser_query, query, 0); - - LOG_DEBUG(log, "Computing destination partition set, executing query: " << query); - - Context local_context = context; - local_context.setSettings(task_cluster->settings_pull); - Block block = getBlockWithAllStreamData(InterpreterFactory::get(query_ast, local_context)->execute().in); - - std::set res; - if (block) - { - ColumnWithTypeAndName & column = block.getByPosition(0); - task_shard.partition_key_column = column; - - for (size_t i = 0; i < column.column->size(); ++i) - { - WriteBufferFromOwnString wb; - column.type->serializeAsTextQuoted(*column.column, i, wb, FormatSettings()); - res.emplace(wb.str()); - } - } - - LOG_DEBUG(log, "There are " << res.size() << " destination partitions in shard " << task_shard.getDescription()); - - return res; - } - - bool checkShardHasPartition(const ConnectionTimeouts & timeouts, TaskShard & task_shard, const String & partition_quoted_name) - { - createShardInternalTables(timeouts, task_shard, false); - - TaskTable & task_table = task_shard.task_table; - - std::string query = "SELECT 1 FROM " + getQuotedTable(task_shard.table_read_shard) - + " WHERE (" + queryToString(task_table.engine_push_partition_key_ast) + " = (" + partition_quoted_name + " AS partition_key))"; - - if (!task_table.where_condition_str.empty()) - query += " AND (" + task_table.where_condition_str + ")"; - - query += " LIMIT 1"; - - LOG_DEBUG(log, "Checking shard " << task_shard.getDescription() << " for partition " - << partition_quoted_name << " existence, executing query: " << query); - - ParserQuery parser_query(query.data() + query.size()); - ASTPtr query_ast = parseQuery(parser_query, query, 0); - - Context local_context = context; - local_context.setSettings(task_cluster->settings_pull); - return InterpreterFactory::get(query_ast, local_context)->execute().in->read().rows() != 0; - } + bool checkShardHasPartition(const ConnectionTimeouts & timeouts, TaskShard & task_shard, const String & partition_quoted_name); /// TODO: Implement checkPresentPartitionPiecesOnCurrentShard(); /// Just copypaste the function above bool checkPresentPartitionPiecesOnCurrentShard(const ConnectionTimeouts & timeouts, - TaskShard & task_shard, const String & partition_quoted_name, size_t current_piece_number) - { - createShardInternalTables(timeouts, task_shard, false); - - TaskTable & task_table = task_shard.task_table; - - std::string query = "SELECT 1 FROM " + getQuotedTable(task_shard.table_read_shard) - + " WHERE (" + queryToString(task_table.engine_push_partition_key_ast) + " = (" + partition_quoted_name + " AS partition_key))"; - - const size_t number_of_splits = task_table.number_of_splits; - const String & primary_key_comma_separated = task_table.primary_key_comma_separated; - - query += " AND (cityHash64(" + primary_key_comma_separated + ") % " - + std::to_string(number_of_splits) + " = " + std::to_string(current_piece_number) + " )"; - - if (!task_table.where_condition_str.empty()) - query += " AND (" + task_table.where_condition_str + ")"; - - query += " LIMIT 1"; - - LOG_DEBUG(log, "Checking shard " << task_shard.getDescription() << " for partition " - << partition_quoted_name << " piece " << std::to_string(current_piece_number) << "existence, executing query: " << query); - - ParserQuery parser_query(query.data() + query.size()); - ASTPtr query_ast = parseQuery(parser_query, query, 0); - - Context local_context = context; - local_context.setSettings(task_cluster->settings_pull); - auto result = InterpreterFactory::get(query_ast, local_context)->execute().in->read().rows(); - if (result != 0) - LOG_DEBUG(log, "Partition " << partition_quoted_name << " piece number " - << std::to_string(current_piece_number) << " is PRESENT on shard " << task_shard.getDescription()); - else - LOG_DEBUG(log, "Partition " << partition_quoted_name << " piece number " - << std::to_string(current_piece_number) << " is ABSENT on shard " << task_shard.getDescription()); - return result != 0; - } + TaskShard & task_shard, const String & partition_quoted_name, size_t current_piece_number); /** Executes simple query (without output streams, for example DDL queries) on each shard of the cluster * Returns number of shards for which at least one replica executed query successfully @@ -1574,95 +157,7 @@ public: const ASTPtr & query_ast_ = nullptr, const Settings * settings = nullptr, PoolMode pool_mode = PoolMode::GET_ALL, - UInt64 max_successful_executions_per_shard = 0) const - { - auto num_shards = cluster->getShardsInfo().size(); - std::vector per_shard_num_successful_replicas(num_shards, 0); - - ASTPtr query_ast; - if (query_ast_ == nullptr) - { - ParserQuery p_query(query.data() + query.size()); - query_ast = parseQuery(p_query, query, 0); - } - else - query_ast = query_ast_; - - - /// We need to execute query on one replica at least - auto do_for_shard = [&] (UInt64 shard_index) - { - const Cluster::ShardInfo & shard = cluster->getShardsInfo().at(shard_index); - UInt64 & num_successful_executions = per_shard_num_successful_replicas.at(shard_index); - num_successful_executions = 0; - - auto increment_and_check_exit = [&] () -> bool - { - ++num_successful_executions; - return max_successful_executions_per_shard && num_successful_executions >= max_successful_executions_per_shard; - }; - - UInt64 num_replicas = cluster->getShardsAddresses().at(shard_index).size(); - UInt64 num_local_replicas = shard.getLocalNodeCount(); - UInt64 num_remote_replicas = num_replicas - num_local_replicas; - - /// In that case we don't have local replicas, but do it just in case - for (UInt64 i = 0; i < num_local_replicas; ++i) - { - auto interpreter = InterpreterFactory::get(query_ast, context); - interpreter->execute(); - - if (increment_and_check_exit()) - return; - } - - /// Will try to make as many as possible queries - if (shard.hasRemoteConnections()) - { - Settings current_settings = settings ? *settings : task_cluster->settings_common; - current_settings.max_parallel_replicas = num_remote_replicas ? num_remote_replicas : 1; - - auto timeouts = ConnectionTimeouts::getTCPTimeoutsWithFailover(current_settings).getSaturated(current_settings.max_execution_time); - auto connections = shard.pool->getMany(timeouts, ¤t_settings, pool_mode); - - for (auto & connection : connections) - { - if (connection.isNull()) - continue; - - try - { - /// CREATE TABLE and DROP PARTITION queries return empty block - RemoteBlockInputStream stream{*connection, query, Block{}, context, ¤t_settings}; - NullBlockOutputStream output{Block{}}; - copyData(stream, output); - - if (increment_and_check_exit()) - return; - } - catch (const Exception &) - { - LOG_INFO(log, getCurrentExceptionMessage(false, true)); - } - } - } - }; - - { - ThreadPool thread_pool(std::min(num_shards, getNumberOfPhysicalCPUCores())); - - for (UInt64 shard_index = 0; shard_index < num_shards; ++shard_index) - thread_pool.scheduleOrThrowOnError([=] { do_for_shard(shard_index); }); - - thread_pool.wait(); - } - - UInt64 successful_shards = 0; - for (UInt64 num_replicas : per_shard_num_successful_replicas) - successful_shards += (num_replicas > 0); - - return successful_shards; - } + UInt64 max_successful_executions_per_shard = 0) const; private: String task_zookeeper_path; From 70e8e1db26b13800324ce001f2798e4e7afbf47d Mon Sep 17 00:00:00 2001 From: Avogar Date: Tue, 18 Feb 2020 16:53:12 +0300 Subject: [PATCH 0025/1355] Fix style errors and tests --- dbms/src/Processors/Formats/Impl/RegexpRowInputFormat.cpp | 2 +- dbms/src/Processors/Formats/Impl/RegexpRowInputFormat.h | 2 +- dbms/tests/queries/0_stateless/01079_regexp_input_format.sh | 2 +- .../0_stateless/01080_regexp_input_format_skip_unmatched.sh | 4 ++-- 4 files changed, 5 insertions(+), 5 deletions(-) mode change 100644 => 100755 dbms/tests/queries/0_stateless/01080_regexp_input_format_skip_unmatched.sh diff --git a/dbms/src/Processors/Formats/Impl/RegexpRowInputFormat.cpp b/dbms/src/Processors/Formats/Impl/RegexpRowInputFormat.cpp index d53c6819c91..f0f3d10755b 100644 --- a/dbms/src/Processors/Formats/Impl/RegexpRowInputFormat.cpp +++ b/dbms/src/Processors/Formats/Impl/RegexpRowInputFormat.cpp @@ -16,7 +16,7 @@ namespace ErrorCodes } RegexpRowInputFormat::RegexpRowInputFormat( - ReadBuffer & in_, const Block & header_, Params params_, const FormatSettings & format_settings_) + ReadBuffer & in_, const Block & header_, Params params_, const FormatSettings & format_settings_) : IRowInputFormat(header_, in_, std::move(params_)), format_settings(format_settings_), regexp(format_settings_.regexp.regexp) { field_format = stringToFormat(format_settings_.regexp.escaping_rule); diff --git a/dbms/src/Processors/Formats/Impl/RegexpRowInputFormat.h b/dbms/src/Processors/Formats/Impl/RegexpRowInputFormat.h index fe920f26fed..48a711d0cde 100644 --- a/dbms/src/Processors/Formats/Impl/RegexpRowInputFormat.h +++ b/dbms/src/Processors/Formats/Impl/RegexpRowInputFormat.h @@ -15,7 +15,7 @@ class ReadBuffer; class RegexpRowInputFormat : public IRowInputFormat { public: - RegexpRowInputFormat(ReadBuffer & in_, const Block & header_, Params params_, const FormatSettings & format_settings_); + RegexpRowInputFormat(ReadBuffer & in_, const Block & header_, Params params_, const FormatSettings & format_settings_); String getName() const override { return "RegexpRowInputFormat"; } diff --git a/dbms/tests/queries/0_stateless/01079_regexp_input_format.sh b/dbms/tests/queries/0_stateless/01079_regexp_input_format.sh index cbaa1bd2162..81ccf35e678 100755 --- a/dbms/tests/queries/0_stateless/01079_regexp_input_format.sh +++ b/dbms/tests/queries/0_stateless/01079_regexp_input_format.sh @@ -1,4 +1,4 @@ -#!/usr/bin/env bash#!/usr/bin/env bash +#!/usr/bin/env bash CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) . $CURDIR/../shell_config.sh diff --git a/dbms/tests/queries/0_stateless/01080_regexp_input_format_skip_unmatched.sh b/dbms/tests/queries/0_stateless/01080_regexp_input_format_skip_unmatched.sh old mode 100644 new mode 100755 index 98bcb0a256a..f46a6239795 --- a/dbms/tests/queries/0_stateless/01080_regexp_input_format_skip_unmatched.sh +++ b/dbms/tests/queries/0_stateless/01080_regexp_input_format_skip_unmatched.sh @@ -1,7 +1,7 @@ #!/usr/bin/env bash -# CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) -# . $CURDIR/../shell_config.sh +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +. $CURDIR/../shell_config.sh $CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS regexp"; $CLICKHOUSE_CLIENT --query="CREATE TABLE regexp (id UInt32, string String) ENGINE = Memory"; From b08db134c1b3bec91b869ced70daeea4f371d6f0 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Thu, 20 Feb 2020 12:00:41 +0300 Subject: [PATCH 0026/1355] merging --- dbms/programs/copier/TaskTable.h | 303 -------------------------- dbms/programs/copier/ZookeeperStaff.h | 224 ------------------- 2 files changed, 527 deletions(-) delete mode 100644 dbms/programs/copier/TaskTable.h delete mode 100644 dbms/programs/copier/ZookeeperStaff.h diff --git a/dbms/programs/copier/TaskTable.h b/dbms/programs/copier/TaskTable.h deleted file mode 100644 index 10aaf9334a6..00000000000 --- a/dbms/programs/copier/TaskTable.h +++ /dev/null @@ -1,303 +0,0 @@ -#pragma once - -#include "Internals.h" -#include "TaskCluster.h" -#include "ext/range.h" - -namespace DB { - - -struct TaskTable { - TaskTable(TaskCluster & parent, const Poco::Util::AbstractConfiguration & config, const String & prefix, - const String & table_key); - - TaskCluster & task_cluster; - - /// These functions used in checkPartitionIsDone() or checkPartitionPieceIsDone() - /// They are implemented here not to call task_table.tasks_shard[partition_name].second.pieces[current_piece_number] etc. - - String getPartitionPath(const String & partition_name) const; - - [[maybe_unused]] String getPartitionPiecePath(const String & partition_name, const size_t piece_number) const; - - String getCertainPartitionIsDirtyPath(const String & partition_name) const; - - [[maybe_unused]] String getCertainPartitionPieceIsDirtyPath(const String & partition_name, const size_t piece_number) const - { - UNUSED(partition_name); - UNUSED(piece_number); - return "Not Implemented"; - } - - String getCertainPartitionIsCleanedPath(const String & partition_name) const; - - [[maybe_unused]] String getCertainPartitionPieceIsCleanedPath(const String & partition_name, const size_t piece_number) const - { - UNUSED(partition_name); - UNUSED(piece_number); - return "Not implemented"; - } - - String getCertainPartitionTaskStatusPath(const String & partition_name) const; - - [[maybe_unused]] String getCertainPartitionPieceTaskStatusPath(const String & partition_name, const size_t piece_number) const - { - UNUSED(partition_name); - UNUSED(piece_number); - return "Not implemented"; - } - - /// Partitions will be splitted into number-of-splits pieces. - /// Each piece will be copied independently. (10 by default) - size_t number_of_splits; - - String name_in_config; - - /// Used as task ID - String table_id; - - /// Column names in primary key - String primary_key_comma_separated; - - /// Source cluster and table - String cluster_pull_name; - DatabaseAndTableName table_pull; - - /// Destination cluster and table - String cluster_push_name; - DatabaseAndTableName table_push; - - /// Storage of destination table - /// (tables that are stored on each shard of target cluster) - String engine_push_str; - ASTPtr engine_push_ast; - ASTPtr engine_push_partition_key_ast; - - /* - * A Distributed table definition used to split data - * Distributed table will be created on each shard of default - * cluster to perform data copying and resharding - * */ - String sharding_key_str; - ASTPtr sharding_key_ast; - ASTPtr main_engine_split_ast; - - - /* - * Auxuliary table engines used to perform partition piece copying. - * Each AST represent table engine for certatin piece number. - * After copying partition piece is Ok, this piece will be moved to the main - * target table. All this tables are stored on each shard as the main table. - * We have to use separate tables for partition pieces because of the atomicity of copying. - * Also if we want to move some partition to another table, the partition keys have to be the same. - * */ - - - /* - * To copy partiton piece form one cluster to another we have to use Distributed table. - * In case of usage separate table (engine_push) for each partiton piece, - * we have to use many Distributed tables. - * */ - ASTs auxiliary_engine_split_asts; - - /// Additional WHERE expression to filter input data - String where_condition_str; - ASTPtr where_condition_ast; - - /// Resolved clusters - ClusterPtr cluster_pull; - ClusterPtr cluster_push; - - /// Filter partitions that should be copied - bool has_enabled_partitions = false; - Strings enabled_partitions; - NameSet enabled_partitions_set; - - /** - * Prioritized list of shards - * all_shards contains information about all shards in the table. - * So we have to check whether particular shard have current partiton or not while processing. - */ - TasksShard all_shards; - TasksShard local_shards; - - /// All partitions of the current table. - ClusterPartitions cluster_partitions; - NameSet finished_cluster_partitions; - - /// Parition names to process in user-specified order - Strings ordered_partition_names; - - ClusterPartition & getClusterPartition(const String &partition_name) { - auto it = cluster_partitions.find(partition_name); - if (it == cluster_partitions.end()) - throw Exception("There are no cluster partition " + partition_name + " in " + table_id, - ErrorCodes::LOGICAL_ERROR); - return it->second; - } - - Stopwatch watch; - UInt64 bytes_copied = 0; - UInt64 rows_copied = 0; - - template - void initShards(RandomEngine &&random_engine); -}; - - -String TaskTable::getPartitionPath(const String &partition_name) const { - return task_cluster.task_zookeeper_path // root - + "/tables/" + table_id // tables/dst_cluster.merge.hits - + "/" + escapeForFileName(partition_name); // 201701 -} - -String TaskTable::getPartitionPiecePath(const String & partition_name, size_t piece_number) const { - assert(piece_number < number_of_splits); - return getPartitionPath(partition_name) + "/" + - std::to_string(piece_number); // 1...number_of_splits -} - -String TaskTable::getCertainPartitionIsDirtyPath(const String &partition_name) const { - return getPartitionPath(partition_name) + "/is_dirty"; -} - -String TaskTable::getCertainPartitionIsCleanedPath(const String &partition_name) const { - return getCertainPartitionIsDirtyPath(partition_name) + "/cleaned"; -} - -String TaskTable::getCertainPartitionTaskStatusPath(const String &partition_name) const { - return getPartitionPath(partition_name) + "/shards"; -} - -String TaskShard::getDescription() const { - std::stringstream ss; - ss << "N" << numberInCluster() - << " (having a replica " << getHostNameExample() - << ", pull table " + getQuotedTable(task_table.table_pull) - << " of cluster " + task_table.cluster_pull_name << ")"; - return ss.str(); -} - -String TaskShard::getHostNameExample() const { - auto &replicas = task_table.cluster_pull->getShardsAddresses().at(indexInCluster()); - return replicas.at(0).readableString(); -} - -TaskTable::TaskTable(TaskCluster & parent, const Poco::Util::AbstractConfiguration & config, - const String & prefix_, const String & table_key) - : task_cluster(parent) { - String table_prefix = prefix_ + "." + table_key + "."; - - name_in_config = table_key; - - number_of_splits = config.getUInt64(table_prefix + "number_of_splits", 10); - - cluster_pull_name = config.getString(table_prefix + "cluster_pull"); - cluster_push_name = config.getString(table_prefix + "cluster_push"); - - table_pull.first = config.getString(table_prefix + "database_pull"); - table_pull.second = config.getString(table_prefix + "table_pull"); - - table_push.first = config.getString(table_prefix + "database_push"); - table_push.second = config.getString(table_prefix + "table_push"); - - /// Used as node name in ZooKeeper - table_id = escapeForFileName(cluster_push_name) - + "." + escapeForFileName(table_push.first) - + "." + escapeForFileName(table_push.second); - - engine_push_str = config.getString(table_prefix + "engine"); - { - ParserStorage parser_storage; - engine_push_ast = parseQuery(parser_storage, engine_push_str, 0); - engine_push_partition_key_ast = extractPartitionKey(engine_push_ast); - primary_key_comma_separated = createCommaSeparatedStringFrom(extractPrimaryKeyString(engine_push_ast)); - } - - sharding_key_str = config.getString(table_prefix + "sharding_key"); - - auxiliary_engine_split_asts.reserve(number_of_splits); - { - ParserExpressionWithOptionalAlias parser_expression(false); - sharding_key_ast = parseQuery(parser_expression, sharding_key_str, 0); - main_engine_split_ast = createASTStorageDistributed(cluster_push_name, table_push.first, table_push.second, - sharding_key_ast); - - for (const auto piece_number : ext::range(0, number_of_splits)) - { - auxiliary_engine_split_asts.emplace_back - ( - createASTStorageDistributed(cluster_push_name, table_push.first, - table_push.second + ".piece_" + toString(piece_number), sharding_key_ast) - ); - } - } - - where_condition_str = config.getString(table_prefix + "where_condition", ""); - if (!where_condition_str.empty()) { - ParserExpressionWithOptionalAlias parser_expression(false); - where_condition_ast = parseQuery(parser_expression, where_condition_str, 0); - - // Will use canonical expression form - where_condition_str = queryToString(where_condition_ast); - } - - String enabled_partitions_prefix = table_prefix + "enabled_partitions"; - has_enabled_partitions = config.has(enabled_partitions_prefix); - - if (has_enabled_partitions) { - Strings keys; - config.keys(enabled_partitions_prefix, keys); - - if (keys.empty()) { - /// Parse list of partition from space-separated string - String partitions_str = config.getString(table_prefix + "enabled_partitions"); - boost::trim_if(partitions_str, isWhitespaceASCII); - boost::split(enabled_partitions, partitions_str, isWhitespaceASCII, boost::token_compress_on); - } else { - /// Parse sequence of ... - for (const String &key : keys) { - if (!startsWith(key, "partition")) - throw Exception("Unknown key " + key + " in " + enabled_partitions_prefix, - ErrorCodes::UNKNOWN_ELEMENT_IN_CONFIG); - - enabled_partitions.emplace_back(config.getString(enabled_partitions_prefix + "." + key)); - } - } - - std::copy(enabled_partitions.begin(), enabled_partitions.end(), - std::inserter(enabled_partitions_set, enabled_partitions_set.begin())); - } -} - -template -void TaskTable::initShards(RandomEngine &&random_engine) { - const String & fqdn_name = getFQDNOrHostName(); - std::uniform_int_distribution get_urand(0, std::numeric_limits::max()); - - // Compute the priority - for (auto &shard_info : cluster_pull->getShardsInfo()) { - TaskShardPtr task_shard = std::make_shared(*this, shard_info); - const auto &replicas = cluster_pull->getShardsAddresses().at(task_shard->indexInCluster()); - task_shard->priority = getReplicasPriority(replicas, fqdn_name, get_urand(random_engine)); - - all_shards.emplace_back(task_shard); - } - - // Sort by priority - std::sort(all_shards.begin(), all_shards.end(), - [](const TaskShardPtr &lhs, const TaskShardPtr &rhs) { - return ShardPriority::greaterPriority(lhs->priority, rhs->priority); - }); - - // Cut local shards - auto it_first_remote = std::lower_bound(all_shards.begin(), all_shards.end(), 1, - [](const TaskShardPtr &lhs, UInt8 is_remote) { - return lhs->priority.is_remote < is_remote; - }); - - local_shards.assign(all_shards.begin(), it_first_remote); -} - - -} diff --git a/dbms/programs/copier/ZookeeperStaff.h b/dbms/programs/copier/ZookeeperStaff.h deleted file mode 100644 index 3133c68933d..00000000000 --- a/dbms/programs/copier/ZookeeperStaff.h +++ /dev/null @@ -1,224 +0,0 @@ -#pragma once - -/** Allows to compare two incremental counters of type UInt32 in presence of possible overflow. - * We assume that we compare values that are not too far away. - * For example, when we increment 0xFFFFFFFF, we get 0. So, 0xFFFFFFFF is less than 0. - */ -class WrappingUInt32 -{ -public: - UInt32 value; - - explicit WrappingUInt32(UInt32 _value) - : value(_value) - {} - - bool operator<(const WrappingUInt32 & other) const - { - return value != other.value && *this <= other; - } - - bool operator<=(const WrappingUInt32 & other) const - { - const UInt32 HALF = 1 << 31; - return (value <= other.value && other.value - value < HALF) - || (value > other.value && value - other.value > HALF); - } - - bool operator==(const WrappingUInt32 & other) const - { - return value == other.value; - } -}; - -/** Conforming Zxid definition. - * cf. https://github.com/apache/zookeeper/blob/631d1b284f0edb1c4f6b0fb221bf2428aec71aaa/zookeeper-docs/src/main/resources/markdown/zookeeperInternals.md#guarantees-properties-and-definitions - * - * But it is better to read this: https://zookeeper.apache.org/doc/r3.1.2/zookeeperProgrammers.html - * - * Actually here is the definition of Zxid. - * Every change to the ZooKeeper state receives a stamp in the form of a zxid (ZooKeeper Transaction Id). - * This exposes the total ordering of all changes to ZooKeeper. Each change will have a unique zxid - * and if zxid1 is smaller than zxid2 then zxid1 happened before zxid2. - */ -class Zxid -{ -public: - WrappingUInt32 epoch; - WrappingUInt32 counter; - explicit Zxid(UInt64 _zxid) - : epoch(_zxid >> 32) - , counter(_zxid) - {} - - bool operator<=(const Zxid & other) const - { - return (epoch < other.epoch) - || (epoch == other.epoch && counter <= other.counter); - } - - bool operator==(const Zxid & other) const - { - return epoch == other.epoch && counter == other.counter; - } -}; - -/* When multiple ClusterCopiers discover that the target partition is not empty, - * they will attempt to clean up this partition before proceeding to copying. - * - * Instead of purging is_dirty, the history of cleaning work is preserved and partition hygiene is established - * based on a happens-before relation between the events. - * This relation is encoded by LogicalClock based on the mzxid of the is_dirty ZNode and is_dirty/cleaned. - * The fact of the partition hygiene is encoded by CleanStateClock. - * - * For you to know what mzxid means: - * - * ZooKeeper Stat Structure: - * The Stat structure for each znode in ZooKeeper is made up of the following fields: - * - * -- czxid - * The zxid of the change that caused this znode to be created. - * - * -- mzxid - * The zxid of the change that last modified this znode. - * - * -- ctime - * The time in milliseconds from epoch when this znode was created. - * - * -- mtime - * The time in milliseconds from epoch when this znode was last modified. - * - * -- version - * The number of changes to the data of this znode. - * - * -- cversion - * The number of changes to the children of this znode. - * - * -- aversion - * The number of changes to the ACL of this znode. - * - * -- ephemeralOwner - * The session id of the owner of this znode if the znode is an ephemeral node. - * If it is not an ephemeral node, it will be zero. - * - * -- dataLength - * The length of the data field of this znode. - * - * -- numChildren - * The number of children of this znode. - * */ - -class LogicalClock -{ -public: - std::optional zxid; - - LogicalClock() = default; - - explicit LogicalClock(UInt64 _zxid) - : zxid(_zxid) - {} - - bool hasHappened() const - { - return bool(zxid); - } - - /// happens-before relation with a reasonable time bound - bool happensBefore(const LogicalClock & other) const - { - return !zxid - || (other.zxid && *zxid <= *other.zxid); - } - - bool operator<=(const LogicalClock & other) const - { - return happensBefore(other); - } - - /// strict equality check - bool operator==(const LogicalClock & other) const - { - return zxid == other.zxid; - } -}; - - -class CleanStateClock -{ -public: - LogicalClock discovery_zxid; - std::optional discovery_version; - - LogicalClock clean_state_zxid; - std::optional clean_state_version; - - std::shared_ptr stale; - - bool is_clean() const - { - return - !is_stale() - && ( - !discovery_zxid.hasHappened() - || (clean_state_zxid.hasHappened() && discovery_zxid <= clean_state_zxid)); - } - - bool is_stale() const - { - return stale->load(); - } - - CleanStateClock( - const zkutil::ZooKeeperPtr & zookeeper, - const String & discovery_path, - const String & clean_state_path) - : stale(std::make_shared(false)) - { - Coordination::Stat stat{}; - String _some_data; - auto watch_callback = - [stale = stale] (const Coordination::WatchResponse & rsp) - { - auto logger = &Poco::Logger::get("ClusterCopier"); - if (rsp.error == Coordination::ZOK) - { - switch (rsp.type) - { - case Coordination::CREATED: - LOG_DEBUG(logger, "CleanStateClock change: CREATED, at " << rsp.path); - stale->store(true); - break; - case Coordination::CHANGED: - LOG_DEBUG(logger, "CleanStateClock change: CHANGED, at" << rsp.path); - stale->store(true); - } - } - }; - if (zookeeper->tryGetWatch(discovery_path, _some_data, &stat, watch_callback)) - { - discovery_zxid = LogicalClock(stat.mzxid); - discovery_version = stat.version; - } - if (zookeeper->tryGetWatch(clean_state_path, _some_data, &stat, watch_callback)) - { - clean_state_zxid = LogicalClock(stat.mzxid); - clean_state_version = stat.version; - } - } - - bool operator==(const CleanStateClock & other) const - { - return !is_stale() - && !other.is_stale() - && discovery_zxid == other.discovery_zxid - && discovery_version == other.discovery_version - && clean_state_zxid == other.clean_state_zxid - && clean_state_version == other.clean_state_version; - } - - bool operator!=(const CleanStateClock & other) const - { - return !(*this == other); - } -}; From 6bc23f5eae2e8b2e12811802e91237333cd1dc66 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Thu, 20 Feb 2020 12:01:06 +0300 Subject: [PATCH 0027/1355] merging --- dbms/programs/copier/Aliases.h | 3 + dbms/programs/copier/ClusterCopier.cpp | 8 +- dbms/programs/copier/ClusterCopier.h | 14 +-- dbms/programs/copier/Internals.cpp | 54 +++++++++ dbms/programs/copier/Internals.h | 25 ++++ dbms/programs/copier/ShardPartition.h | 4 +- dbms/programs/copier/TaskTableAndShard.h | 148 +++++++++++++++++------ 7 files changed, 206 insertions(+), 50 deletions(-) diff --git a/dbms/programs/copier/Aliases.h b/dbms/programs/copier/Aliases.h index 4beff891bf4..d088d33f259 100644 --- a/dbms/programs/copier/Aliases.h +++ b/dbms/programs/copier/Aliases.h @@ -2,6 +2,7 @@ #include + namespace DB { @@ -10,12 +11,14 @@ namespace DB using DatabaseAndTableName = std::pair; /// Hierarchical description of the tasks + struct ShardPartitionPiece; struct ShardPartition; struct TaskShard; struct TaskTable; struct TaskCluster; struct ClusterPartition; + using PartitionPieces = std::vector; using TasksPartition = std::map>; using ShardInfo = Cluster::ShardInfo; using TaskShardPtr = std::shared_ptr; diff --git a/dbms/programs/copier/ClusterCopier.cpp b/dbms/programs/copier/ClusterCopier.cpp index 24ac6fdde0e..f133aadf6f2 100644 --- a/dbms/programs/copier/ClusterCopier.cpp +++ b/dbms/programs/copier/ClusterCopier.cpp @@ -578,7 +578,7 @@ bool ClusterCopier::checkPartitionPieceIsDone(const TaskTable & task_table, cons } /// Removes MATERIALIZED and ALIAS columns from create table query -static ASTPtr ClusterCopier::removeAliasColumnsFromCreateQuery(const ASTPtr & query_ast) +ASTPtr ClusterCopier::removeAliasColumnsFromCreateQuery(const ASTPtr & query_ast) { const ASTs & column_asts = query_ast->as().columns_list->columns->children; auto new_columns = std::make_shared(); @@ -611,7 +611,7 @@ static ASTPtr ClusterCopier::removeAliasColumnsFromCreateQuery(const ASTPtr & qu } /// Replaces ENGINE and table name in a create query -static std::shared_ptr ClusterCopier::rewriteCreateQueryStorage(const ASTPtr & create_query_ast, const DatabaseAndTableName & new_table, const ASTPtr & new_storage_ast) +std::shared_ptr ClusterCopier::rewriteCreateQueryStorage(const ASTPtr & create_query_ast, const DatabaseAndTableName & new_table, const ASTPtr & new_storage_ast) { const auto & create = create_query_ast->as(); auto res = std::make_shared(create); @@ -919,7 +919,7 @@ bool ClusterCopier::tryProcessTable(const ConnectionTimeouts & timeouts, TaskTab /// Job for copying partition from particular shard. PartitionTaskStatus ClusterCopier::tryProcessPartitionTask(const ConnectionTimeouts & timeouts, ShardPartition & task_partition, bool is_unprioritized_task) { - PartitionTaskStatus res; + PartitionTaskStatus res{Active}; try { @@ -941,7 +941,7 @@ PartitionTaskStatus ClusterCopier::tryProcessPartitionTask(const ConnectionTimeo tryLogCurrentException(log, "An error occurred while updating the config"); } - return res; + return PartitionTaskStatus::Finished; } PartitionTaskStatus ClusterCopier::iterateThroughAllPiecesInPartition(const ConnectionTimeouts & timeouts, ShardPartition & task_partition, diff --git a/dbms/programs/copier/ClusterCopier.h b/dbms/programs/copier/ClusterCopier.h index e8ca88cb2a2..30a18b8b1a2 100644 --- a/dbms/programs/copier/ClusterCopier.h +++ b/dbms/programs/copier/ClusterCopier.h @@ -1,20 +1,20 @@ #pragma once -#include -#include +#include "Aliases.h" #include "Internals.h" #include "TaskCluster.h" +#include "TaskTableAndShard.h" #include "ShardPartition.h" #include "ShardPartitionPiece.h" -#include "TaskTable.h" -#include "ZookeeperStaff.h" +#include "ZooKeeperStaff.h" + namespace DB { using ConfigurationPtr = Poco::AutoPtr; -static ConfigurationPtr getConfigurationFromXMLString(const std::string & xml_data) +ConfigurationPtr getConfigurationFromXMLString(const std::string & xml_data) { std::stringstream ss(xml_data); Poco::XML::InputSource input_source{ss}; @@ -92,10 +92,10 @@ public: size_t piece_number, const TasksShard & shards_with_partition); /// Removes MATERIALIZED and ALIAS columns from create table query - static ASTPtr removeAliasColumnsFromCreateQuery(const ASTPtr & query_ast); + ASTPtr removeAliasColumnsFromCreateQuery(const ASTPtr & query_ast); /// Replaces ENGINE and table name in a create query - static std::shared_ptr rewriteCreateQueryStorage(const ASTPtr & create_query_ast, + std::shared_ptr rewriteCreateQueryStorage(const ASTPtr & create_query_ast, const DatabaseAndTableName & new_table, const ASTPtr & new_storage_ast); bool tryDropPartition(ShardPartition & task_partition, const zkutil::ZooKeeperPtr & zookeeper, const CleanStateClock & clean_state_clock); diff --git a/dbms/programs/copier/Internals.cpp b/dbms/programs/copier/Internals.cpp index 05286515970..c7163d5eef0 100644 --- a/dbms/programs/copier/Internals.cpp +++ b/dbms/programs/copier/Internals.cpp @@ -109,6 +109,60 @@ ASTPtr extractPartitionKey(const ASTPtr & storage_ast) } } +ASTPtr extractPrimaryKeyOrOrderBy(const ASTPtr & storage_ast) +{ + String storage_str = queryToString(storage_ast); + + const auto & storage = storage_ast->as(); + const auto & engine = storage.engine->as(); + + if (!endsWith(engine.name, "MergeTree")) + { + throw Exception("Unsupported engine was specified in " + storage_str + ", only *MergeTree engines are supported", + ErrorCodes::BAD_ARGUMENTS); + } + + /// FIXME + if (!isExtendedDefinitionStorage(storage_ast)) + { + throw Exception("Is not extended deginition storage " + storage_str + " Will be fixed later.", + ErrorCodes::BAD_ARGUMENTS); + } + + if (storage.primary_key) + return storage.primary_key->clone(); + + return storage.order_by->clone(); +} + +String createCommaSeparatedStringFrom(const Strings & strings) +{ + String answer; + for (auto & string: strings) + answer += string + ", "; + + /// Remove last comma and space + answer.pop_back(); + answer.pop_back(); + return answer; +} + +Strings extractPrimaryKeyString(const ASTPtr & storage_ast) +{ + const auto primary_key_or_order_by = extractPrimaryKeyOrOrderBy(storage_ast)->as(); + + ASTPtr primary_key_or_order_by_arguments_ast = primary_key_or_order_by.arguments->clone(); + ASTs & primary_key_or_order_by_arguments = primary_key_or_order_by_arguments_ast->children; + + Strings answer; + answer.reserve(primary_key_or_order_by_arguments.size()); + + for (auto & column : primary_key_or_order_by_arguments) + answer.push_back(column->getColumnName()); + + return answer; +} + ShardPriority getReplicasPriority(const Cluster::Addresses & replicas, const std::string & local_hostname, UInt8 random) { ShardPriority res; diff --git a/dbms/programs/copier/Internals.h b/dbms/programs/copier/Internals.h index 7177f6900de..5deca5d8d34 100644 --- a/dbms/programs/copier/Internals.h +++ b/dbms/programs/copier/Internals.h @@ -177,6 +177,31 @@ bool isExtendedDefinitionStorage(const ASTPtr & storage_ast); ASTPtr extractPartitionKey(const ASTPtr & storage_ast); +/* +* Choosing a Primary Key that Differs from the Sorting Key +* It is possible to specify a primary key (an expression with values that are written in the index file for each mark) +* that is different from the sorting key (an expression for sorting the rows in data parts). +* In this case the primary key expression tuple must be a prefix of the sorting key expression tuple. +* This feature is helpful when using the SummingMergeTree and AggregatingMergeTree table engines. +* In a common case when using these engines, the table has two types of columns: dimensions and measures. +* Typical queries aggregate values of measure columns with arbitrary GROUP BY and filtering by dimensions. +* Because SummingMergeTree and AggregatingMergeTree aggregate rows with the same value of the sorting key, +* it is natural to add all dimensions to it. As a result, the key expression consists of a long list of columns +* and this list must be frequently updated with newly added dimensions. +* In this case it makes sense to leave only a few columns in the primary key that will provide efficient +* range scans and add the remaining dimension columns to the sorting key tuple. +* ALTER of the sorting key is a lightweight operation because when a new column is simultaneously added t +* o the table and to the sorting key, existing data parts don't need to be changed. +* Since the old sorting key is a prefix of the new sorting key and there is no data in the newly added column, +* the data is sorted by both the old and new sorting keys at the moment of table modification. +* +* */ +ASTPtr extractPrimaryKeyOrOrderBy(const ASTPtr & storage_ast); + +String createCommaSeparatedStringFrom(const Strings & strings); + +Strings extractPrimaryKeyString(const ASTPtr & storage_ast); + ShardPriority getReplicasPriority(const Cluster::Addresses & replicas, const std::string & local_hostname, UInt8 random); } diff --git a/dbms/programs/copier/ShardPartition.h b/dbms/programs/copier/ShardPartition.h index 03b3502efba..f7384b977e3 100644 --- a/dbms/programs/copier/ShardPartition.h +++ b/dbms/programs/copier/ShardPartition.h @@ -1,7 +1,7 @@ #pragma once -#include "Internals.h" -#include "TaskTable.h" +#include "Aliases.h" +#include "TaskTableAndShard.h" namespace DB { diff --git a/dbms/programs/copier/TaskTableAndShard.h b/dbms/programs/copier/TaskTableAndShard.h index 05b1c3f543a..a73b7bfafa5 100644 --- a/dbms/programs/copier/TaskTableAndShard.h +++ b/dbms/programs/copier/TaskTableAndShard.h @@ -9,23 +9,58 @@ namespace DB struct TaskShard; -struct TaskTable -{ +struct TaskTable { TaskTable(TaskCluster & parent, const Poco::Util::AbstractConfiguration & config, const String & prefix, const String & table_key); TaskCluster & task_cluster; + /// These functions used in checkPartitionIsDone() or checkPartitionPieceIsDone() + /// They are implemented here not to call task_table.tasks_shard[partition_name].second.pieces[current_piece_number] etc. + String getPartitionPath(const String & partition_name) const; - String getPartitionIsDirtyPath(const String & partition_name) const; - String getPartitionIsCleanedPath(const String & partition_name) const; - String getPartitionTaskStatusPath(const String & partition_name) const; + + [[maybe_unused]] String getPartitionPiecePath(const String & partition_name, const size_t piece_number) const; + + String getCertainPartitionIsDirtyPath(const String & partition_name) const; + + [[maybe_unused]] String getCertainPartitionPieceIsDirtyPath(const String & partition_name, const size_t piece_number) const + { + UNUSED(partition_name); + UNUSED(piece_number); + return "Not Implemented"; + } + + String getCertainPartitionIsCleanedPath(const String & partition_name) const; + + [[maybe_unused]] String getCertainPartitionPieceIsCleanedPath(const String & partition_name, const size_t piece_number) const + { + UNUSED(partition_name); + UNUSED(piece_number); + return "Not implemented"; + } + + String getCertainPartitionTaskStatusPath(const String & partition_name) const; + + [[maybe_unused]] String getCertainPartitionPieceTaskStatusPath(const String & partition_name, const size_t piece_number) const + { + UNUSED(partition_name); + UNUSED(piece_number); + return "Not implemented"; + } + + /// Partitions will be splitted into number-of-splits pieces. + /// Each piece will be copied independently. (10 by default) + size_t number_of_splits; String name_in_config; /// Used as task ID String table_id; + /// Column names in primary key + String primary_key_comma_separated; + /// Source cluster and table String cluster_pull_name; DatabaseAndTableName table_pull; @@ -35,14 +70,37 @@ struct TaskTable DatabaseAndTableName table_push; /// Storage of destination table + /// (tables that are stored on each shard of target cluster) String engine_push_str; ASTPtr engine_push_ast; ASTPtr engine_push_partition_key_ast; - /// A Distributed table definition used to split data + /* + * A Distributed table definition used to split data + * Distributed table will be created on each shard of default + * cluster to perform data copying and resharding + * */ String sharding_key_str; ASTPtr sharding_key_ast; - ASTPtr engine_split_ast; + ASTPtr main_engine_split_ast; + + + /* + * Auxuliary table engines used to perform partition piece copying. + * Each AST represent table engine for certatin piece number. + * After copying partition piece is Ok, this piece will be moved to the main + * target table. All this tables are stored on each shard as the main table. + * We have to use separate tables for partition pieces because of the atomicity of copying. + * Also if we want to move some partition to another table, the partition keys have to be the same. + * */ + + + /* + * To copy partiton piece form one cluster to another we have to use Distributed table. + * In case of usage separate table (engine_push) for each partiton piece, + * we have to use many Distributed tables. + * */ + ASTs auxiliary_engine_split_asts; /// Additional WHERE expression to filter input data String where_condition_str; @@ -57,21 +115,26 @@ struct TaskTable Strings enabled_partitions; NameSet enabled_partitions_set; - /// Prioritized list of shards + /** + * Prioritized list of shards + * all_shards contains information about all shards in the table. + * So we have to check whether particular shard have current partiton or not while processing. + */ TasksShard all_shards; TasksShard local_shards; + /// All partitions of the current table. ClusterPartitions cluster_partitions; NameSet finished_cluster_partitions; /// Parition names to process in user-specified order Strings ordered_partition_names; - ClusterPartition & getClusterPartition(const String & partition_name) - { + ClusterPartition & getClusterPartition(const String &partition_name) { auto it = cluster_partitions.find(partition_name); if (it == cluster_partitions.end()) - throw Exception("There are no cluster partition " + partition_name + " in " + table_id, ErrorCodes::LOGICAL_ERROR); + throw Exception("There are no cluster partition " + partition_name + " in " + table_id, + ErrorCodes::LOGICAL_ERROR); return it->second; } @@ -79,8 +142,8 @@ struct TaskTable UInt64 bytes_copied = 0; UInt64 rows_copied = 0; - template - void initShards(RandomEngine && random_engine); + template + void initShards(RandomEngine &&random_engine); }; @@ -121,36 +184,38 @@ struct TaskTable }; -inline String TaskTable::getPartitionPath(const String & partition_name) const +inline String TaskTable::getPartitionPiecePath(const String & partition_name, size_t piece_number) const { - return task_cluster.task_zookeeper_path // root - + "/tables/" + table_id // tables/dst_cluster.merge.hits - + "/" + escapeForFileName(partition_name); // 201701 + assert(piece_number < number_of_splits); + return getPartitionPath(partition_name) + "/" + + std::to_string(piece_number); // 1...number_of_splits } -inline String TaskTable::getPartitionIsDirtyPath(const String & partition_name) const +inline String TaskTable::getCertainPartitionIsDirtyPath(const String &partition_name) const { return getPartitionPath(partition_name) + "/is_dirty"; } -inline String TaskTable::getPartitionIsCleanedPath(const String & partition_name) const +inline String TaskTable::getCertainPartitionIsCleanedPath(const String &partition_name) const { - return getPartitionIsDirtyPath(partition_name) + "/cleaned"; + return getCertainPartitionIsDirtyPath(partition_name) + "/cleaned"; } -inline String TaskTable::getPartitionTaskStatusPath(const String & partition_name) const +inline String TaskTable::getCertainPartitionTaskStatusPath(const String &partition_name) const { return getPartitionPath(partition_name) + "/shards"; } -inline TaskTable::TaskTable(TaskCluster & parent, const Poco::Util::AbstractConfiguration & config, const String & prefix_, - const String & table_key) +inline TaskTable::TaskTable(TaskCluster & parent, const Poco::Util::AbstractConfiguration & config, + const String & prefix_, const String & table_key) : task_cluster(parent) { String table_prefix = prefix_ + "." + table_key + "."; name_in_config = table_key; + number_of_splits = config.getUInt64(table_prefix + "number_of_splits", 10); + cluster_pull_name = config.getString(table_prefix + "cluster_pull"); cluster_push_name = config.getString(table_prefix + "cluster_push"); @@ -170,18 +235,30 @@ inline TaskTable::TaskTable(TaskCluster & parent, const Poco::Util::AbstractConf ParserStorage parser_storage; engine_push_ast = parseQuery(parser_storage, engine_push_str, 0); engine_push_partition_key_ast = extractPartitionKey(engine_push_ast); + primary_key_comma_separated = createCommaSeparatedStringFrom(extractPrimaryKeyString(engine_push_ast)); } sharding_key_str = config.getString(table_prefix + "sharding_key"); + + auxiliary_engine_split_asts.reserve(number_of_splits); { ParserExpressionWithOptionalAlias parser_expression(false); sharding_key_ast = parseQuery(parser_expression, sharding_key_str, 0); - engine_split_ast = createASTStorageDistributed(cluster_push_name, table_push.first, table_push.second, sharding_key_ast); + main_engine_split_ast = createASTStorageDistributed(cluster_push_name, table_push.first, table_push.second, + sharding_key_ast); + + for (const auto piece_number : ext::range(0, number_of_splits)) + { + auxiliary_engine_split_asts.emplace_back + ( + createASTStorageDistributed(cluster_push_name, table_push.first, + table_push.second + ".piece_" + toString(piece_number), sharding_key_ast) + ); + } } where_condition_str = config.getString(table_prefix + "where_condition", ""); - if (!where_condition_str.empty()) - { + if (!where_condition_str.empty()) { ParserExpressionWithOptionalAlias parser_expression(false); where_condition_ast = parseQuery(parser_expression, where_condition_str, 0); @@ -192,31 +269,28 @@ inline TaskTable::TaskTable(TaskCluster & parent, const Poco::Util::AbstractConf String enabled_partitions_prefix = table_prefix + "enabled_partitions"; has_enabled_partitions = config.has(enabled_partitions_prefix); - if (has_enabled_partitions) - { + if (has_enabled_partitions) { Strings keys; config.keys(enabled_partitions_prefix, keys); - if (keys.empty()) - { + if (keys.empty()) { /// Parse list of partition from space-separated string String partitions_str = config.getString(table_prefix + "enabled_partitions"); boost::trim_if(partitions_str, isWhitespaceASCII); boost::split(enabled_partitions, partitions_str, isWhitespaceASCII, boost::token_compress_on); - } - else - { + } else { /// Parse sequence of ... - for (const String & key : keys) - { + for (const String &key : keys) { if (!startsWith(key, "partition")) - throw Exception("Unknown key " + key + " in " + enabled_partitions_prefix, ErrorCodes::UNKNOWN_ELEMENT_IN_CONFIG); + throw Exception("Unknown key " + key + " in " + enabled_partitions_prefix, + ErrorCodes::UNKNOWN_ELEMENT_IN_CONFIG); enabled_partitions.emplace_back(config.getString(enabled_partitions_prefix + "." + key)); } } - std::copy(enabled_partitions.begin(), enabled_partitions.end(), std::inserter(enabled_partitions_set, enabled_partitions_set.begin())); + std::copy(enabled_partitions.begin(), enabled_partitions.end(), + std::inserter(enabled_partitions_set, enabled_partitions_set.begin())); } } From b035dd815b24a6e27d05c63cfa5c0b959bde284f Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Thu, 20 Feb 2020 13:01:02 +0300 Subject: [PATCH 0028/1355] merge finished --- dbms/programs/copier/CMakeLists.txt | 2 +- dbms/programs/copier/ClusterCopier.cpp | 26 +++++----- dbms/programs/copier/ClusterCopier.h | 56 +++++++++------------- dbms/programs/copier/ClusterCopierApp.cpp | 1 - dbms/programs/copier/ClusterCopierApp.h | 3 ++ dbms/programs/copier/Internals.cpp | 3 +- dbms/programs/copier/Internals.h | 6 +++ dbms/programs/copier/ShardPartitionPiece.h | 16 +++---- dbms/programs/copier/TaskTableAndShard.h | 18 +++---- 9 files changed, 67 insertions(+), 64 deletions(-) diff --git a/dbms/programs/copier/CMakeLists.txt b/dbms/programs/copier/CMakeLists.txt index ff9ba2f250f..5573fbc5e5d 100644 --- a/dbms/programs/copier/CMakeLists.txt +++ b/dbms/programs/copier/CMakeLists.txt @@ -12,6 +12,6 @@ set(CLICKHOUSE_COPIER_LINK PRIVATE clickhouse_dictionaries string_utils ${Poco_XML_LIBRARY} PUBLIC daemon) -set(CLICKHOUSE_COPIER_INCLUDE SYSTEM PRIVATE ${PCG_RANDOM_INCLUDE_DIR}) +set(CLICKHOUSE_COPIER_INCLUDE SYSTEM PRIVATE ${PCG_RANDOM_INCLUDE_DIR} ${CMAKE_CURRENT_SOURCE_DIR}) clickhouse_program_add(copier) diff --git a/dbms/programs/copier/ClusterCopier.cpp b/dbms/programs/copier/ClusterCopier.cpp index f133aadf6f2..65a82983d09 100644 --- a/dbms/programs/copier/ClusterCopier.cpp +++ b/dbms/programs/copier/ClusterCopier.cpp @@ -919,7 +919,7 @@ bool ClusterCopier::tryProcessTable(const ConnectionTimeouts & timeouts, TaskTab /// Job for copying partition from particular shard. PartitionTaskStatus ClusterCopier::tryProcessPartitionTask(const ConnectionTimeouts & timeouts, ShardPartition & task_partition, bool is_unprioritized_task) { - PartitionTaskStatus res{Active}; + PartitionTaskStatus res; try { @@ -1324,7 +1324,7 @@ void ClusterCopier::dropLocalTableIfExists(const DatabaseAndTableName & table_na interpreter.execute(); } -String ClusterCopier::getRemoteCreateTable(const DatabaseAndTableName & table, Connection & connection, const Settings * settings = nullptr) +String ClusterCopier::getRemoteCreateTable(const DatabaseAndTableName & table, Connection & connection, const Settings * settings) { String query = "SHOW CREATE TABLE " + getQuotedTable(table); Block block = getBlockWithAllStreamData(std::make_shared( @@ -1347,7 +1347,8 @@ ASTPtr ClusterCopier::getCreateTableForPullShard(const ConnectionTimeouts & time } /// If it is implicitly asked to create split Distributed table for certain piece on current shard, we will do it. -void ClusterCopier::createShardInternalTables(const ConnectionTimeouts & timeouts, TaskShard & task_shard, bool create_split = true, const size_t piece_number = 0) +void ClusterCopier::createShardInternalTables(const ConnectionTimeouts & timeouts, + TaskShard & task_shard, bool create_split, const size_t piece_number) { TaskTable & task_table = task_shard.task_table; @@ -1423,14 +1424,16 @@ std::set ClusterCopier::getShardPartitions(const ConnectionTimeouts & ti return res; } -bool ClusterCopier::checkShardHasPartition(const ConnectionTimeouts & timeouts, TaskShard & task_shard, const String & partition_quoted_name) +bool ClusterCopier::checkShardHasPartition(const ConnectionTimeouts & timeouts, + TaskShard & task_shard, const String & partition_quoted_name) { createShardInternalTables(timeouts, task_shard, false); TaskTable & task_table = task_shard.task_table; std::string query = "SELECT 1 FROM " + getQuotedTable(task_shard.table_read_shard) - + " WHERE (" + queryToString(task_table.engine_push_partition_key_ast) + " = (" + partition_quoted_name + " AS partition_key))"; + + " WHERE (" + queryToString(task_table.engine_push_partition_key_ast) + + " = (" + partition_quoted_name + " AS partition_key))"; if (!task_table.where_condition_str.empty()) query += " AND (" + task_table.where_condition_str + ")"; @@ -1451,14 +1454,15 @@ bool ClusterCopier::checkShardHasPartition(const ConnectionTimeouts & timeouts, /// TODO: Implement checkPresentPartitionPiecesOnCurrentShard(); /// Just copypaste the function above bool ClusterCopier::checkPresentPartitionPiecesOnCurrentShard(const ConnectionTimeouts & timeouts, - TaskShard & task_shard, const String & partition_quoted_name, size_t current_piece_number) + TaskShard & task_shard, const String & partition_quoted_name, size_t current_piece_number) { createShardInternalTables(timeouts, task_shard, false); TaskTable & task_table = task_shard.task_table; std::string query = "SELECT 1 FROM " + getQuotedTable(task_shard.table_read_shard) - + " WHERE (" + queryToString(task_table.engine_push_partition_key_ast) + " = (" + partition_quoted_name + " AS partition_key))"; + + " WHERE (" + queryToString(task_table.engine_push_partition_key_ast) + + " = (" + partition_quoted_name + " AS partition_key))"; const size_t number_of_splits = task_table.number_of_splits; const String & primary_key_comma_separated = task_table.primary_key_comma_separated; @@ -1495,10 +1499,10 @@ bool ClusterCopier::checkPresentPartitionPiecesOnCurrentShard(const ConnectionTi UInt64 ClusterCopier::executeQueryOnCluster( const ClusterPtr & cluster, const String & query, - const ASTPtr & query_ast_ = nullptr, - const Settings * settings = nullptr, - PoolMode pool_mode = PoolMode::GET_ALL, - UInt64 max_successful_executions_per_shard = 0) const + const ASTPtr & query_ast_, + const Settings * settings, + PoolMode pool_mode, + UInt64 max_successful_executions_per_shard) const { auto num_shards = cluster->getShardsInfo().size(); std::vector per_shard_num_successful_replicas(num_shards, 0); diff --git a/dbms/programs/copier/ClusterCopier.h b/dbms/programs/copier/ClusterCopier.h index 30a18b8b1a2..0555b2d5c04 100644 --- a/dbms/programs/copier/ClusterCopier.h +++ b/dbms/programs/copier/ClusterCopier.h @@ -12,15 +12,6 @@ namespace DB { -using ConfigurationPtr = Poco::AutoPtr; - -ConfigurationPtr getConfigurationFromXMLString(const std::string & xml_data) -{ - std::stringstream ss(xml_data); - Poco::XML::InputSource input_source{ss}; - return {new Poco::Util::XMLConfiguration{&input_source}}; -} - class ClusterCopier { public: @@ -34,9 +25,7 @@ public: host_id(std::move(host_id_)), working_database_name(std::move(proxy_database_name_)), context(context_), - log(&Poco::Logger::get("ClusterCopier")) - { - } + log(&Poco::Logger::get("ClusterCopier")) {} void init(); @@ -57,17 +46,32 @@ public: void process(const ConnectionTimeouts & timeouts); /// Disables DROP PARTITION commands that used to clear data after errors - void setSafeMode(bool is_safe_mode_ = true); + void setSafeMode(bool is_safe_mode_ = true) + { + is_safe_mode = is_safe_mode_; + } - void setCopyFaultProbability(double copy_fault_probability_); + void setCopyFaultProbability(double copy_fault_probability_) + { + copy_fault_probability = copy_fault_probability_; + } - protected: +protected: - String getWorkersPath() const; + String getWorkersPath() const + { + return task_cluster->task_zookeeper_path + "/task_active_workers"; + } - String getWorkersPathVersion() const; + String getWorkersPathVersion() const + { + return getWorkersPath() + "_version"; + } - String getCurrentWorkerNodePath() const; + String getCurrentWorkerNodePath() const + { + return getWorkersPath() + "/" + host_id; + } zkutil::EphemeralNodeHolder::Ptr createTaskWorkerNodeAndWaitIfNeed( const zkutil::ZooKeeperPtr & zookeeper, @@ -105,22 +109,6 @@ public: bool tryProcessTable(const ConnectionTimeouts & timeouts, TaskTable & task_table); - /// Execution status of a task - enum class PartitionTaskStatus - { - Active, - Finished, - Error, - }; - - - enum class PartititonPieceTaskStatus - { - Active, - Finished, - Error, - }; - /// Job for copying partition from particular shard. PartitionTaskStatus tryProcessPartitionTask(const ConnectionTimeouts & timeouts, ShardPartition & task_partition, bool is_unprioritized_task); diff --git a/dbms/programs/copier/ClusterCopierApp.cpp b/dbms/programs/copier/ClusterCopierApp.cpp index 37b02296086..e6ff4c521ef 100644 --- a/dbms/programs/copier/ClusterCopierApp.cpp +++ b/dbms/programs/copier/ClusterCopierApp.cpp @@ -1,5 +1,4 @@ #include "ClusterCopierApp.h" -#include "ClusterCopier.h" namespace DB { diff --git a/dbms/programs/copier/ClusterCopierApp.h b/dbms/programs/copier/ClusterCopierApp.h index fe228fd6194..25350aefd3a 100644 --- a/dbms/programs/copier/ClusterCopierApp.h +++ b/dbms/programs/copier/ClusterCopierApp.h @@ -1,7 +1,10 @@ #pragma once + #include #include +#include "ClusterCopier.h" + /* clickhouse cluster copier util * Copies tables data from one cluster to new tables of other (possibly the same) cluster in distributed fault-tolerant manner. * diff --git a/dbms/programs/copier/Internals.cpp b/dbms/programs/copier/Internals.cpp index c7163d5eef0..173a5dc3c59 100644 --- a/dbms/programs/copier/Internals.cpp +++ b/dbms/programs/copier/Internals.cpp @@ -2,6 +2,8 @@ namespace DB { +using ConfigurationPtr = Poco::AutoPtr; + ConfigurationPtr getConfigurationFromXMLString(const std::string & xml_data) { std::stringstream ss(xml_data); @@ -9,7 +11,6 @@ ConfigurationPtr getConfigurationFromXMLString(const std::string & xml_data) return {new Poco::Util::XMLConfiguration{&input_source}}; } - String getQuotedTable(const String & database, const String & table) { if (database.empty()) diff --git a/dbms/programs/copier/Internals.h b/dbms/programs/copier/Internals.h index 5deca5d8d34..2937171903c 100644 --- a/dbms/programs/copier/Internals.h +++ b/dbms/programs/copier/Internals.h @@ -155,6 +155,12 @@ enum class PartitionTaskStatus Error, }; +enum class PartititonPieceTaskStatus +{ + Active, + Finished, + Error, +}; struct MultiTransactionInfo { diff --git a/dbms/programs/copier/ShardPartitionPiece.h b/dbms/programs/copier/ShardPartitionPiece.h index 303407d1d5b..f7ae8013b47 100644 --- a/dbms/programs/copier/ShardPartitionPiece.h +++ b/dbms/programs/copier/ShardPartitionPiece.h @@ -34,43 +34,43 @@ struct ShardPartitionPiece { }; -String ShardPartitionPiece::getPartitionPiecePath() const +inline String ShardPartitionPiece::getPartitionPiecePath() const { return shard_partition.getPartitionPath() + "/piece_" + std::to_string(current_piece_number); } -String ShardPartitionPiece::getPartitionPieceCleanStartPath() const +inline String ShardPartitionPiece::getPartitionPieceCleanStartPath() const { return getPartitionPiecePath() + "/clean_start"; } -String ShardPartitionPiece::getPartitionPieceIsDirtyPath() const +inline String ShardPartitionPiece::getPartitionPieceIsDirtyPath() const { return getPartitionPiecePath() + "/is_dirty"; } -String ShardPartitionPiece::getPartitionPieceIsCleanedPath() const +inline String ShardPartitionPiece::getPartitionPieceIsCleanedPath() const { return getPartitionPieceIsDirtyPath() + "/is_cleaned"; } -String ShardPartitionPiece::getPartitionPieceActiveWorkersPath() const +inline String ShardPartitionPiece::getPartitionPieceActiveWorkersPath() const { return getPartitionPiecePath() + "/partition_active_workers"; } -String ShardPartitionPiece::getActiveWorkerPath() const +inline String ShardPartitionPiece::getActiveWorkerPath() const { return getPartitionPieceActiveWorkersPath() + "/" + toString(shard_partition.task_shard.numberInCluster()); } /// On what shards do we have current partition. -String ShardPartitionPiece::getPartitionPieceShardsPath() const +inline String ShardPartitionPiece::getPartitionPieceShardsPath() const { return getPartitionPiecePath() + "/shards"; } -String ShardPartitionPiece::getShardStatusPath() const +inline String ShardPartitionPiece::getShardStatusPath() const { return getPartitionPieceShardsPath() + "/" + toString(shard_partition.task_shard.numberInCluster()); } diff --git a/dbms/programs/copier/TaskTableAndShard.h b/dbms/programs/copier/TaskTableAndShard.h index a73b7bfafa5..806b6b66cd4 100644 --- a/dbms/programs/copier/TaskTableAndShard.h +++ b/dbms/programs/copier/TaskTableAndShard.h @@ -184,25 +184,27 @@ struct TaskTable { }; -inline String TaskTable::getPartitionPiecePath(const String & partition_name, size_t piece_number) const -{ +inline String TaskTable::getPartitionPath(const String &partition_name) const { + return task_cluster.task_zookeeper_path // root + + "/tables/" + table_id // tables/dst_cluster.merge.hits + + "/" + escapeForFileName(partition_name); // 201701 +} + +inline String TaskTable::getPartitionPiecePath(const String & partition_name, size_t piece_number) const { assert(piece_number < number_of_splits); return getPartitionPath(partition_name) + "/" + std::to_string(piece_number); // 1...number_of_splits } -inline String TaskTable::getCertainPartitionIsDirtyPath(const String &partition_name) const -{ +inline String TaskTable::getCertainPartitionIsDirtyPath(const String &partition_name) const { return getPartitionPath(partition_name) + "/is_dirty"; } -inline String TaskTable::getCertainPartitionIsCleanedPath(const String &partition_name) const -{ +inline String TaskTable::getCertainPartitionIsCleanedPath(const String &partition_name) const { return getCertainPartitionIsDirtyPath(partition_name) + "/cleaned"; } -inline String TaskTable::getCertainPartitionTaskStatusPath(const String &partition_name) const -{ +inline String TaskTable::getCertainPartitionTaskStatusPath(const String &partition_name) const { return getPartitionPath(partition_name) + "/shards"; } From 1be4a35f15c74ca327668185fd3f24bd9865c2b7 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Thu, 20 Feb 2020 20:26:20 +0300 Subject: [PATCH 0029/1355] successful copying --- dbms/programs/copier/ClusterCopier.cpp | 112 +++++++++++++-------- dbms/programs/copier/ClusterCopier.h | 4 +- dbms/programs/copier/ShardPartitionPiece.h | 2 +- dbms/programs/copier/TaskTableAndShard.h | 13 ++- dbms/programs/copier/ZooKeeperStaff.h | 7 +- 5 files changed, 81 insertions(+), 57 deletions(-) diff --git a/dbms/programs/copier/ClusterCopier.cpp b/dbms/programs/copier/ClusterCopier.cpp index 65a82983d09..d6859b54cbf 100644 --- a/dbms/programs/copier/ClusterCopier.cpp +++ b/dbms/programs/copier/ClusterCopier.cpp @@ -17,7 +17,7 @@ void ClusterCopier::init() { if (response.error != Coordination::ZOK) return; - UInt64 version = ++task_descprtion_version; + UInt64 version = ++task_description_version; LOG_DEBUG(log, "Task description should be updated, local version " << version); }; @@ -227,9 +227,9 @@ void ClusterCopier::reloadTaskDescription() void ClusterCopier::updateConfigIfNeeded() { - UInt64 version_to_update = task_descprtion_version; - bool is_outdated_version = task_descprtion_current_version != version_to_update; - bool is_expired_session = !task_description_watch_zookeeper || task_description_watch_zookeeper->expired(); + UInt64 version_to_update = task_description_version; + bool is_outdated_version = task_description_current_version != version_to_update; + bool is_expired_session = !task_description_watch_zookeeper || task_description_watch_zookeeper->expired(); if (!is_outdated_version && !is_expired_session) return; @@ -237,7 +237,7 @@ void ClusterCopier::updateConfigIfNeeded() LOG_DEBUG(log, "Updating task description"); reloadTaskDescription(); - task_descprtion_current_version = version_to_update; + task_description_current_version = version_to_update; } void ClusterCopier::process(const ConnectionTimeouts & timeouts) @@ -312,6 +312,13 @@ void ClusterCopier::process(const ConnectionTimeouts & timeouts) /// Protected section + +/* + * Creates task worker node and checks maximum number of workers not to exceed the limit. + * To achive this we have to check version of workers_version_path node and create current_worker_path + * node atomically. + * */ + zkutil::EphemeralNodeHolder::Ptr ClusterCopier::createTaskWorkerNodeAndWaitIfNeed( const zkutil::ZooKeeperPtr & zookeeper, const String & description, @@ -324,8 +331,8 @@ zkutil::EphemeralNodeHolder::Ptr ClusterCopier::createTaskWorkerNodeAndWaitIfNee std::this_thread::sleep_for(current_sleep_time); String workers_version_path = getWorkersPathVersion(); - String workers_path = getWorkersPath(); - String current_worker_path = getCurrentWorkerNodePath(); + String workers_path = getWorkersPath(); + String current_worker_path = getCurrentWorkerNodePath(); UInt64 num_bad_version_errors = 0; @@ -629,6 +636,9 @@ std::shared_ptr ClusterCopier::rewriteCreateQueryStorage(const A return res; } + +/// TODO: implement tryDropPartitionPiece which is simply tryDropPartition, but on different table. + bool ClusterCopier::tryDropPartition(ShardPartition & task_partition, const zkutil::ZooKeeperPtr & zookeeper, const CleanStateClock & clean_state_clock) { if (is_safe_mode) @@ -636,11 +646,11 @@ bool ClusterCopier::tryDropPartition(ShardPartition & task_partition, const zkut TaskTable & task_table = task_partition.task_shard.task_table; - const String current_shards_path = task_partition.getPartitionShardsPath(); + const String current_shards_path = task_partition.getPartitionShardsPath(); const String current_partition_active_workers_dir = task_partition.getPartitionActiveWorkersPath(); - const String is_dirty_flag_path = task_partition.getCommonPartitionIsDirtyPath(); - const String dirty_cleaner_path = is_dirty_flag_path + "/cleaner"; - const String is_dirty_cleaned_path = task_partition.getCommonPartitionIsCleanedPath(); + const String is_dirty_flag_path = task_partition.getCommonPartitionIsDirtyPath(); + const String dirty_cleaner_path = is_dirty_flag_path + "/cleaner"; + const String is_dirty_cleaned_path = task_partition.getCommonPartitionIsCleanedPath(); zkutil::EphemeralNodeHolder::Ptr cleaner_holder; try @@ -957,12 +967,15 @@ PartitionTaskStatus ClusterCopier::iterateThroughAllPiecesInPartition(const Conn } -PartitionTaskStatus ClusterCopier::processPartitionPieceTaskImpl(const ConnectionTimeouts & timeouts, ShardPartition & task_partition, - const size_t current_piece_number, bool is_unprioritized_task) + +/*...*/ +PartitionTaskStatus ClusterCopier::processPartitionPieceTaskImpl( + const ConnectionTimeouts & timeouts, ShardPartition & task_partition, + const size_t current_piece_number, bool is_unprioritized_task) { TaskShard & task_shard = task_partition.task_shard; TaskTable & task_table = task_shard.task_table; - ClusterPartition & cluster_partition = task_table.getClusterPartition(task_partition.name); + ClusterPartition & cluster_partition = task_table.getClusterPartition(task_partition.name); ShardPartitionPiece & partition_piece = task_partition.pieces[current_piece_number]; const size_t number_of_splits = task_table.number_of_splits; @@ -974,15 +987,15 @@ PartitionTaskStatus ClusterCopier::processPartitionPieceTaskImpl(const Connectio auto zookeeper = context.getZooKeeper(); - const String piece_is_dirty_flag_path = partition_piece.getPartitionPieceIsDirtyPath(); - const String piece_is_dirty_cleaned_path = partition_piece.getPartitionPieceIsCleanedPath(); + const String piece_is_dirty_flag_path = partition_piece.getPartitionPieceIsDirtyPath(); + const String piece_is_dirty_cleaned_path = partition_piece.getPartitionPieceIsCleanedPath(); const String current_task_piece_is_active_path = partition_piece.getActiveWorkerPath(); - const String current_task_piece_status_path = partition_piece.getShardStatusPath(); + const String current_task_piece_status_path = partition_piece.getShardStatusPath(); /// Auxiliary functions: /// Creates is_dirty node to initialize DROP PARTITION - auto create_is_dirty_node = [&, this] (const CleanStateClock & clock) + auto create_is_dirty_node = [&] (const CleanStateClock & clock) { if (clock.is_stale()) LOG_DEBUG(log, "Clean state clock is stale while setting dirty flag, cowardly bailing"); @@ -1001,16 +1014,17 @@ PartitionTaskStatus ClusterCopier::processPartitionPieceTaskImpl(const Connectio }; /// Returns SELECT query filtering current partition and applying user filter - auto get_select_query = [&] (const DatabaseAndTableName & from_table, const String & fields, String limit = "", - bool enable_splitting = false) + auto get_select_query = [&] (const DatabaseAndTableName & from_table, const String & fields, bool enable_splitting, String limit = "") { String query; query += "SELECT " + fields + " FROM " + getQuotedTable(from_table); /// TODO: Bad, it is better to rewrite with ASTLiteral(partition_key_field) query += " WHERE (" + queryToString(task_table.engine_push_partition_key_ast) + " = (" + task_partition.name + " AS partition_key))"; + if (enable_splitting) - query += " AND ( cityHash64(" + primary_key_comma_separated + ") = " + toString(current_piece_number) + " )"; + query += " AND ( cityHash64(" + primary_key_comma_separated + ") %" + toString(number_of_splits) + + " = " + toString(current_piece_number) + " )"; if (!task_table.where_condition_str.empty()) query += " AND (" + task_table.where_condition_str + ")"; @@ -1031,23 +1045,26 @@ PartitionTaskStatus ClusterCopier::processPartitionPieceTaskImpl(const Connectio LogicalClock task_start_clock; { Coordination::Stat stat{}; - if (zookeeper->exists(task_partition.getPartitionShardsPath(), &stat)) + if (zookeeper->exists(partition_piece.getPartitionPieceShardsPath(), &stat)) task_start_clock = LogicalClock(stat.mzxid); } - /// Do not start if partition is dirty, try to clean it + /// Do not start if partition piece is dirty, try to clean it if (clean_state_clock.is_clean() && (!task_start_clock.hasHappened() || clean_state_clock.discovery_zxid <= task_start_clock)) { - LOG_DEBUG(log, "Partition " << task_partition.name << " appears to be clean"); + LOG_DEBUG(log, "Partition " << task_partition.name + << " piece " + toString(current_piece_number) + " appears to be clean"); zookeeper->createAncestors(current_task_piece_status_path); } else { - LOG_DEBUG(log, "Partition " << task_partition.name << " is dirty, try to drop it"); + LOG_DEBUG(log, "Partition " << task_partition.name + << " piece " + toString(current_piece_number) + " is dirty, try to drop it"); try { + /// TODO: tryDropPartitionPiece. tryDropPartition(task_partition, zookeeper, clean_state_clock); } catch (...) @@ -1085,13 +1102,16 @@ PartitionTaskStatus ClusterCopier::processPartitionPieceTaskImpl(const Connectio TaskStateWithOwner status = TaskStateWithOwner::fromString(status_data); if (status.state == TaskState::Finished) { - LOG_DEBUG(log, "Task " << current_task_piece_status_path << " has been successfully executed by " << status.owner); + LOG_DEBUG(log, "Task " << current_task_piece_status_path + << " has been successfully executed by " << status.owner); return PartitionTaskStatus::Finished; } - // Task is abandoned, initialize DROP PARTITION - LOG_DEBUG(log, "Task " << current_task_piece_status_path << " has not been successfully finished by " << - status.owner << ". Partition will be dropped and refilled."); + /// Task is abandoned, because previously we created ephemeral node, possibly in other copier's process. + /// Initialize DROP PARTITION + LOG_DEBUG(log, "Task " << current_task_piece_status_path + << " has not been successfully finished by " << status.owner + << ". Partition will be dropped and refilled."); create_is_dirty_node(clean_state_clock); return PartitionTaskStatus::Error; @@ -1101,13 +1121,14 @@ PartitionTaskStatus ClusterCopier::processPartitionPieceTaskImpl(const Connectio /// Check that destination partition is empty if we are first worker /// NOTE: this check is incorrect if pull and push tables have different partition key! String clean_start_status; - if (!zookeeper->tryGet(task_partition.getPartitionCleanStartPath(), clean_start_status) || clean_start_status != "ok") + if (!zookeeper->tryGet(partition_piece.getPartitionPieceCleanStartPath(), clean_start_status) || clean_start_status != "ok") { - zookeeper->createIfNotExists(task_partition.getPartitionCleanStartPath(), ""); - auto checker = zkutil::EphemeralNodeHolder::create(task_partition.getPartitionCleanStartPath() + "/checker", + zookeeper->createIfNotExists(partition_piece.getPartitionPieceCleanStartPath(), ""); + auto checker = zkutil::EphemeralNodeHolder::create(partition_piece.getPartitionPieceCleanStartPath() + "/checker", *zookeeper, host_id); // Maybe we are the first worker - ASTPtr query_select_ast = get_select_query(task_shard.table_split_shard, "count()"); + /// TODO: Why table_split_shard??? + ASTPtr query_select_ast = get_select_query(task_shard.table_split_shard, "count()", /*enable_splitting*/ true); UInt64 count; { Context local_context = context; @@ -1122,20 +1143,21 @@ PartitionTaskStatus ClusterCopier::processPartitionPieceTaskImpl(const Connectio if (count != 0) { Coordination::Stat stat_shards{}; - zookeeper->get(task_partition.getPartitionShardsPath(), & stat_shards); + zookeeper->get(partition_piece.getPartitionPieceShardsPath(), &stat_shards); /// NOTE: partition is still fresh if dirt discovery happens before cleaning if (stat_shards.numChildren == 0) { LOG_WARNING(log, "There are no workers for partition " << task_partition.name - << ", but destination table contains " << count << " rows" - << ". Partition will be dropped and refilled."); + << " piece " << toString(current_piece_number) + << ", but destination table contains " << count << " rows" + << ". Partition will be dropped and refilled."); create_is_dirty_node(clean_state_clock); return PartitionTaskStatus::Error; } } - zookeeper->set(task_partition.getPartitionCleanStartPath(), "ok"); + zookeeper->set(partition_piece.getPartitionPieceCleanStartPath(), "ok"); } /// At this point, we need to sync that the destination table is clean /// before any actual work @@ -1146,12 +1168,14 @@ PartitionTaskStatus ClusterCopier::processPartitionPieceTaskImpl(const Connectio CleanStateClock new_clean_state_clock (zookeeper, piece_is_dirty_flag_path, piece_is_dirty_cleaned_path); if (clean_state_clock != new_clean_state_clock) { - LOG_INFO(log, "Partition " << task_partition.name << " clean state changed, cowardly bailing"); + LOG_INFO(log, "Partition " << task_partition.name << " piece " + << toString(current_piece_number) << " clean state changed, cowardly bailing"); return PartitionTaskStatus::Error; } else if (!new_clean_state_clock.is_clean()) { - LOG_INFO(log, "Partition " << task_partition.name << " is dirty and will be dropped and refilled"); + LOG_INFO(log, "Partition " << task_partition.name << " piece " + << toString(current_piece_number) << " is dirty and will be dropped and refilled"); create_is_dirty_node(new_clean_state_clock); return PartitionTaskStatus::Error; } @@ -1163,7 +1187,7 @@ PartitionTaskStatus ClusterCopier::processPartitionPieceTaskImpl(const Connectio /// Define push table for current partition piece auto database_and_table_for_current_piece= std::pair( task_table.table_push.first, - task_table.table_push.second + ".piece_" + toString(current_piece_number)); + task_table.table_push.second + "_piece_" + toString(current_piece_number)); auto create_query_push_ast = rewriteCreateQueryStorage(task_shard.current_pull_table_create_query, database_and_table_for_current_piece, task_table.engine_push_ast); @@ -1174,8 +1198,8 @@ PartitionTaskStatus ClusterCopier::processPartitionPieceTaskImpl(const Connectio UInt64 shards = executeQueryOnCluster(task_table.cluster_push, query, create_query_push_ast, &task_cluster->settings_push, PoolMode::GET_MANY); - LOG_DEBUG(log, "Destination tables " << getQuotedTable(task_table.table_push) << - " have been created on " << shards << " shards of " << task_table.cluster_push->getShardCount()); + LOG_DEBUG(log, "Destination tables " << getQuotedTable(task_table.table_push) + << " have been created on " << shards << " shards of " << task_table.cluster_push->getShardCount()); } /// Do the copying @@ -1188,7 +1212,7 @@ PartitionTaskStatus ClusterCopier::processPartitionPieceTaskImpl(const Connectio } // Select all fields - ASTPtr query_select_ast = get_select_query(task_shard.table_read_shard, "*", inject_fault ? "1" : ""); + ASTPtr query_select_ast = get_select_query(task_shard.table_read_shard, "*", /*enable_splitting*/ inject_fault ? "1" : ""); LOG_DEBUG(log, "Executing SELECT query and pull from " << task_shard.getDescription() << " : " << queryToString(query_select_ast)); @@ -1361,7 +1385,7 @@ void ClusterCopier::createShardInternalTables(const ConnectionTimeouts & timeout String split_shard_prefix = ".split."; task_shard.table_read_shard = DatabaseAndTableName(working_database_name, read_shard_prefix + task_table.table_id); task_shard.table_split_shard = DatabaseAndTableName( - working_database_name, split_shard_prefix + task_table.table_id + ".piece_" + toString(piece_number)); + working_database_name, split_shard_prefix + task_table.table_id + "_piece_" + toString(piece_number)); /// Create special cluster with single shard String shard_read_cluster_name = read_shard_prefix + task_table.cluster_pull_name; diff --git a/dbms/programs/copier/ClusterCopier.h b/dbms/programs/copier/ClusterCopier.h index 0555b2d5c04..41fdc326ed2 100644 --- a/dbms/programs/copier/ClusterCopier.h +++ b/dbms/programs/copier/ClusterCopier.h @@ -156,8 +156,8 @@ private: String working_database_name; /// Auto update config stuff - UInt64 task_descprtion_current_version = 1; - std::atomic task_descprtion_version{1}; + UInt64 task_description_current_version = 1; + std::atomic task_description_version{1}; Coordination::WatchCallback task_description_watch_callback; /// ZooKeeper session used to set the callback zkutil::ZooKeeperPtr task_description_watch_zookeeper; diff --git a/dbms/programs/copier/ShardPartitionPiece.h b/dbms/programs/copier/ShardPartitionPiece.h index f7ae8013b47..a99221b8a97 100644 --- a/dbms/programs/copier/ShardPartitionPiece.h +++ b/dbms/programs/copier/ShardPartitionPiece.h @@ -56,7 +56,7 @@ inline String ShardPartitionPiece::getPartitionPieceIsCleanedPath() const inline String ShardPartitionPiece::getPartitionPieceActiveWorkersPath() const { - return getPartitionPiecePath() + "/partition_active_workers"; + return getPartitionPiecePath() + "/partition_piece_active_workers"; } inline String ShardPartitionPiece::getActiveWorkerPath() const diff --git a/dbms/programs/copier/TaskTableAndShard.h b/dbms/programs/copier/TaskTableAndShard.h index d585fb184ed..c0795340e47 100644 --- a/dbms/programs/copier/TaskTableAndShard.h +++ b/dbms/programs/copier/TaskTableAndShard.h @@ -20,11 +20,11 @@ struct TaskTable { String getPartitionPath(const String & partition_name) const; - [[maybe_unused]] String getPartitionPiecePath(const String & partition_name, const size_t piece_number) const; + String getPartitionPiecePath(const String & partition_name, const size_t piece_number) const; String getCertainPartitionIsDirtyPath(const String & partition_name) const; - [[maybe_unused]] String getCertainPartitionPieceIsDirtyPath(const String & partition_name, const size_t piece_number) const + String getCertainPartitionPieceIsDirtyPath(const String & partition_name, const size_t piece_number) const { UNUSED(partition_name); UNUSED(piece_number); @@ -33,7 +33,7 @@ struct TaskTable { String getCertainPartitionIsCleanedPath(const String & partition_name) const; - [[maybe_unused]] String getCertainPartitionPieceIsCleanedPath(const String & partition_name, const size_t piece_number) const + String getCertainPartitionPieceIsCleanedPath(const String & partition_name, const size_t piece_number) const { UNUSED(partition_name); UNUSED(piece_number); @@ -42,7 +42,7 @@ struct TaskTable { String getCertainPartitionTaskStatusPath(const String & partition_name) const; - [[maybe_unused]] String getCertainPartitionPieceTaskStatusPath(const String & partition_name, const size_t piece_number) const + String getCertainPartitionPieceTaskStatusPath(const String & partition_name, const size_t piece_number) const { UNUSED(partition_name); UNUSED(piece_number); @@ -181,7 +181,10 @@ struct TaskShard /// Internal distributed tables DatabaseAndTableName table_read_shard; + DatabaseAndTableName table_split_shard; + + std::vector list_of_split_tables_on_shard; }; @@ -255,7 +258,7 @@ inline TaskTable::TaskTable(TaskCluster & parent, const Poco::Util::AbstractConf auxiliary_engine_split_asts.emplace_back ( createASTStorageDistributed(cluster_push_name, table_push.first, - table_push.second + ".piece_" + toString(piece_number), sharding_key_ast) + table_push.second + "_piece_" + toString(piece_number), sharding_key_ast) ); } } diff --git a/dbms/programs/copier/ZooKeeperStaff.h b/dbms/programs/copier/ZooKeeperStaff.h index 3133c68933d..2fc4d35400d 100644 --- a/dbms/programs/copier/ZooKeeperStaff.h +++ b/dbms/programs/copier/ZooKeeperStaff.h @@ -157,11 +157,8 @@ public: bool is_clean() const { - return - !is_stale() - && ( - !discovery_zxid.hasHappened() - || (clean_state_zxid.hasHappened() && discovery_zxid <= clean_state_zxid)); + return !is_stale() + && (!discovery_zxid.hasHappened() || (clean_state_zxid.hasHappened() && discovery_zxid <= clean_state_zxid)); } bool is_stale() const From 2869c015712f444dcce8a2c7f9c321da8ae112d8 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Thu, 20 Feb 2020 21:58:00 +0300 Subject: [PATCH 0030/1355] drop partition piece --- dbms/programs/copier/ClusterCopier.cpp | 32 ++++++++++++++-------- dbms/programs/copier/ClusterCopier.h | 3 +- dbms/programs/copier/ShardPartitionPiece.h | 7 +++++ 3 files changed, 29 insertions(+), 13 deletions(-) diff --git a/dbms/programs/copier/ClusterCopier.cpp b/dbms/programs/copier/ClusterCopier.cpp index d6859b54cbf..cde272074da 100644 --- a/dbms/programs/copier/ClusterCopier.cpp +++ b/dbms/programs/copier/ClusterCopier.cpp @@ -639,18 +639,23 @@ std::shared_ptr ClusterCopier::rewriteCreateQueryStorage(const A /// TODO: implement tryDropPartitionPiece which is simply tryDropPartition, but on different table. -bool ClusterCopier::tryDropPartition(ShardPartition & task_partition, const zkutil::ZooKeeperPtr & zookeeper, const CleanStateClock & clean_state_clock) +bool ClusterCopier::tryDropPartitionPiece( + ShardPartition & task_partition, + const size_t current_piece_number, + const zkutil::ZooKeeperPtr & zookeeper, + const CleanStateClock & clean_state_clock) { if (is_safe_mode) throw Exception("DROP PARTITION is prohibited in safe mode", ErrorCodes::NOT_IMPLEMENTED); TaskTable & task_table = task_partition.task_shard.task_table; + ShardPartitionPiece & partition_piece = task_partition.pieces[current_piece_number]; - const String current_shards_path = task_partition.getPartitionShardsPath(); - const String current_partition_active_workers_dir = task_partition.getPartitionActiveWorkersPath(); - const String is_dirty_flag_path = task_partition.getCommonPartitionIsDirtyPath(); - const String dirty_cleaner_path = is_dirty_flag_path + "/cleaner"; - const String is_dirty_cleaned_path = task_partition.getCommonPartitionIsCleanedPath(); + const String current_shards_path = partition_piece.getPartitionPieceShardsPath(); + const String current_partition_active_workers_dir = partition_piece.getPartitionPieceActiveWorkersPath(); + const String is_dirty_flag_path = partition_piece.getPartitionPieceIsDirtyPath(); + const String dirty_cleaner_path = partition_piece.getPartitionPieceCleanerPath(); + const String is_dirty_cleaned_path = partition_piece.getPartitionPieceIsCleanedPath(); zkutil::EphemeralNodeHolder::Ptr cleaner_holder; try @@ -661,7 +666,8 @@ bool ClusterCopier::tryDropPartition(ShardPartition & task_partition, const zkut { if (e.code == Coordination::ZNODEEXISTS) { - LOG_DEBUG(log, "Partition " << task_partition.name << " is cleaning now by somebody, sleep"); + LOG_DEBUG(log, "Partition " << task_partition.name << " piece " + << toString(current_piece_number) << " is cleaning now by somebody, sleep"); std::this_thread::sleep_for(default_sleep_time); return false; } @@ -674,7 +680,8 @@ bool ClusterCopier::tryDropPartition(ShardPartition & task_partition, const zkut { if (stat.numChildren != 0) { - LOG_DEBUG(log, "Partition " << task_partition.name << " contains " << stat.numChildren << " active workers while trying to drop it. Going to sleep."); + LOG_DEBUG(log, "Partition " << task_partition.name << " contains " << stat.numChildren + << " active workers while trying to drop it. Going to sleep."); std::this_thread::sleep_for(default_sleep_time); return false; } @@ -703,7 +710,7 @@ bool ClusterCopier::tryDropPartition(ShardPartition & task_partition, const zkut // Lock the dirty flag zookeeper->set(is_dirty_flag_path, host_id, clean_state_clock.discovery_version.value()); - zookeeper->tryRemove(task_partition.getPartitionCleanStartPath()); + zookeeper->tryRemove(partition_piece.getPartitionPieceCleanStartPath()); CleanStateClock my_clock(zookeeper, is_dirty_flag_path, is_dirty_cleaned_path); /// Remove all status nodes @@ -716,7 +723,7 @@ bool ClusterCopier::tryDropPartition(ShardPartition & task_partition, const zkut } } - String query = "ALTER TABLE " + getQuotedTable(task_table.table_push); + String query = "ALTER TABLE " + getQuotedTable(task_table.table_push) + "_piece_" + toString(current_piece_number); query += " DROP PARTITION " + task_partition.name + ""; /// TODO: use this statement after servers will be updated up to 1.1.54310 @@ -754,7 +761,8 @@ bool ClusterCopier::tryDropPartition(ShardPartition & task_partition, const zkut return false; } - LOG_INFO(log, "Partition " << task_partition.name << " was dropped on cluster " << task_table.cluster_push_name); + LOG_INFO(log, "Partition " << task_partition.name << " piece " << toString(current_piece_number) + << " was dropped on cluster " << task_table.cluster_push_name); if (zookeeper->tryCreate(current_shards_path, host_id, zkutil::CreateMode::Persistent) == Coordination::ZNODEEXISTS) zookeeper->set(current_shards_path, host_id); } @@ -1065,7 +1073,7 @@ PartitionTaskStatus ClusterCopier::processPartitionPieceTaskImpl( try { /// TODO: tryDropPartitionPiece. - tryDropPartition(task_partition, zookeeper, clean_state_clock); + tryDropPartitionPiece(task_partition, current_piece_number, zookeeper, clean_state_clock); } catch (...) { diff --git a/dbms/programs/copier/ClusterCopier.h b/dbms/programs/copier/ClusterCopier.h index 41fdc326ed2..2729b295bce 100644 --- a/dbms/programs/copier/ClusterCopier.h +++ b/dbms/programs/copier/ClusterCopier.h @@ -102,7 +102,8 @@ protected: std::shared_ptr rewriteCreateQueryStorage(const ASTPtr & create_query_ast, const DatabaseAndTableName & new_table, const ASTPtr & new_storage_ast); - bool tryDropPartition(ShardPartition & task_partition, const zkutil::ZooKeeperPtr & zookeeper, const CleanStateClock & clean_state_clock); + bool tryDropPartitionPiece(ShardPartition & task_partition, const size_t current_piece_number, + const zkutil::ZooKeeperPtr & zookeeper, const CleanStateClock & clean_state_clock); static constexpr UInt64 max_table_tries = 1000; static constexpr UInt64 max_shard_partition_tries = 600; diff --git a/dbms/programs/copier/ShardPartitionPiece.h b/dbms/programs/copier/ShardPartitionPiece.h index a99221b8a97..aeaa24fef13 100644 --- a/dbms/programs/copier/ShardPartitionPiece.h +++ b/dbms/programs/copier/ShardPartitionPiece.h @@ -27,6 +27,8 @@ struct ShardPartitionPiece { [[maybe_unused]] String getShardStatusPath() const; + String getPartitionPieceCleanerPath() const; + bool is_absent_piece; const size_t current_piece_number; @@ -75,5 +77,10 @@ inline String ShardPartitionPiece::getShardStatusPath() const return getPartitionPieceShardsPath() + "/" + toString(shard_partition.task_shard.numberInCluster()); } +inline String ShardPartitionPiece::getPartitionPieceCleanerPath() const +{ + return getPartitionPieceIsDirtyPath() + "/cleaner"; +} + } From de49d00079e31f6250adad9effa3e648137bbb52 Mon Sep 17 00:00:00 2001 From: Avogar Date: Fri, 21 Feb 2020 18:21:31 +0300 Subject: [PATCH 0031/1355] Use re2 instead of regexp. Use PeekableReadBuffer insted of common ReadBuffer. --- .../Formats/Impl/RegexpRowInputFormat.cpp | 59 ++++++++++++++----- .../Formats/Impl/RegexpRowInputFormat.h | 15 ++++- 2 files changed, 57 insertions(+), 17 deletions(-) diff --git a/dbms/src/Processors/Formats/Impl/RegexpRowInputFormat.cpp b/dbms/src/Processors/Formats/Impl/RegexpRowInputFormat.cpp index f0f3d10755b..387f81f1155 100644 --- a/dbms/src/Processors/Formats/Impl/RegexpRowInputFormat.cpp +++ b/dbms/src/Processors/Formats/Impl/RegexpRowInputFormat.cpp @@ -1,10 +1,8 @@ -#include #include #include #include #include - -#include +#include namespace DB { @@ -17,8 +15,20 @@ namespace ErrorCodes RegexpRowInputFormat::RegexpRowInputFormat( ReadBuffer & in_, const Block & header_, Params params_, const FormatSettings & format_settings_) - : IRowInputFormat(header_, in_, std::move(params_)), format_settings(format_settings_), regexp(format_settings_.regexp.regexp) + : IRowInputFormat(header_, in_, std::move(params_)), buf(in_), format_settings(format_settings_), regexp(format_settings_.regexp.regexp) { + size_t fields_count = regexp.NumberOfCapturingGroups(); + matched_fields.resize(fields_count); + re2_arguments.resize(fields_count); + re2_arguments_ptrs.resize(fields_count); + for (size_t i = 0; i != fields_count; ++i) + { + // Bind an argument to a matched field. + re2_arguments[i] = &matched_fields[i]; + // Save pointer to argument. + re2_arguments_ptrs[i] = &re2_arguments[i]; + } + field_format = stringToFormat(format_settings_.regexp.escaping_rule); } @@ -40,7 +50,7 @@ bool RegexpRowInputFormat::readField(size_t index, MutableColumns & columns) const auto & type = getPort().getHeader().getByPosition(index).type; bool parse_as_nullable = format_settings.null_as_default && !type->isNullable(); bool read = true; - ReadBuffer field_buf(matched_fields[index + 1].first, matched_fields[index + 1].length(), 0); + ReadBuffer field_buf(matched_fields[index].data(), matched_fields[index].size(), 0); try { switch (field_format) @@ -82,7 +92,7 @@ bool RegexpRowInputFormat::readField(size_t index, MutableColumns & columns) void RegexpRowInputFormat::readFieldsFromMatch(MutableColumns & columns, RowReadExtension & ext) { - if (matched_fields.size() != columns.size() + 1) + if (matched_fields.size() != columns.size()) throw Exception("The number of matched fields in line doesn't match the number of columns.", ErrorCodes::INCORRECT_DATA); ext.read_columns.assign(columns.size(), false); @@ -94,23 +104,44 @@ void RegexpRowInputFormat::readFieldsFromMatch(MutableColumns & columns, RowRead bool RegexpRowInputFormat::readRow(MutableColumns & columns, RowReadExtension & ext) { - if (in.eof()) + if (buf.eof()) return false; - char * line_end = find_first_symbols<'\n', '\r'>(in.position(), in.buffer().end()); - bool match = std::regex_match(in.position(), line_end, matched_fields, regexp); + PeekableReadBufferCheckpoint checkpoint{buf}; + + size_t line_size = 0; + + while (!buf.eof() && *buf.position() != '\n' && *buf.position() != '\r') + { + ++buf.position(); + ++line_size; + } + + buf.makeContinuousMemoryFromCheckpointToPos(); + buf.rollbackToCheckpoint(); + + bool match = RE2::FullMatchN(re2::StringPiece(buf.position(), line_size), regexp, re2_arguments_ptrs.data(), re2_arguments_ptrs.size()); + bool read_line = true; if (!match) { if (!format_settings.regexp.skip_unmatched) - throw Exception("Line \"" + std::string(in.position(), line_end) + "\" doesn't match the regexp.", ErrorCodes::INCORRECT_DATA); - in.position() = line_end + 1; - return true; + throw Exception("Line \"" + std::string(buf.position(), line_size) + "\" doesn't match the regexp.", ErrorCodes::INCORRECT_DATA); + read_line = false; } - readFieldsFromMatch(columns, ext); + if (read_line) + readFieldsFromMatch(columns, ext); + + buf.position() += line_size; + + // Two sequential increments are needed to support DOS-style newline ("\r\n"). + if (!buf.eof() && *buf.position() == '\r') + ++buf.position(); + + if (!buf.eof() && *buf.position() == '\n') + ++buf.position(); - in.position() = line_end + 1; return true; } diff --git a/dbms/src/Processors/Formats/Impl/RegexpRowInputFormat.h b/dbms/src/Processors/Formats/Impl/RegexpRowInputFormat.h index 48a711d0cde..218bb71055f 100644 --- a/dbms/src/Processors/Formats/Impl/RegexpRowInputFormat.h +++ b/dbms/src/Processors/Formats/Impl/RegexpRowInputFormat.h @@ -1,10 +1,13 @@ #pragma once -#include +#include +#include +#include #include #include #include #include +#include namespace DB { @@ -34,10 +37,16 @@ private: void readFieldsFromMatch(MutableColumns & columns, RowReadExtension & ext); FieldFormat stringToFormat(const String & format); + PeekableReadBuffer buf; const FormatSettings format_settings; - std::regex regexp; - std::match_results matched_fields; FieldFormat field_format; + + RE2 regexp; + // The vector of fields extracted from line using regexp. + std::vector matched_fields; + // These two vectors are needed to use RE2::FullMatchN (function for extracting fields). + std::vector re2_arguments; + std::vector re2_arguments_ptrs; }; } From 0b4d8bdb08bb7de45f893dc5a6d8212b51afc67b Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Fri, 21 Feb 2020 19:00:50 +0300 Subject: [PATCH 0032/1355] move partition added --- dbms/programs/copier/Aliases.h | 1 + dbms/programs/copier/ClusterCopier.cpp | 287 ++++++++++++--------- dbms/programs/copier/ClusterCopier.h | 18 +- dbms/programs/copier/ShardPartitionPiece.h | 4 +- dbms/programs/copier/TaskTableAndShard.h | 60 +++-- 5 files changed, 212 insertions(+), 158 deletions(-) diff --git a/dbms/programs/copier/Aliases.h b/dbms/programs/copier/Aliases.h index d088d33f259..fff57dd603f 100644 --- a/dbms/programs/copier/Aliases.h +++ b/dbms/programs/copier/Aliases.h @@ -9,6 +9,7 @@ namespace DB using ConfigurationPtr = Poco::AutoPtr; using DatabaseAndTableName = std::pair; + using ListOfDatabasesAndTableNames = std::vector; /// Hierarchical description of the tasks struct ShardPartitionPiece; diff --git a/dbms/programs/copier/ClusterCopier.cpp b/dbms/programs/copier/ClusterCopier.cpp index cde272074da..e9dc613f85e 100644 --- a/dbms/programs/copier/ClusterCopier.cpp +++ b/dbms/programs/copier/ClusterCopier.cpp @@ -386,99 +386,32 @@ zkutil::EphemeralNodeHolder::Ptr ClusterCopier::createTaskWorkerNodeAndWaitIfNee } } -/** Checks that the whole partition of a table was copied. We should do it carefully due to dirty lock. - * State of some task could change during the processing. - * We have to ensure that all shards have the finished state and there is no dirty flag. - * Moreover, we have to check status twice and check zxid, because state can change during the checking. - */ -bool ClusterCopier::checkPartitionIsDone(const TaskTable & task_table, const String & partition_name, const TasksShard & shards_with_partition) + +bool ClusterCopier::checkPartitionPieceIsClean( + const zkutil::ZooKeeperPtr & zookeeper, + const CleanStateClock & clean_state_clock, + const String & task_status_path) const { - LOG_DEBUG(log, "Check that all shards processed partition " << partition_name << " successfully"); + LogicalClock task_start_clock; - auto zookeeper = context.getZooKeeper(); + Coordination::Stat stat{}; + if (zookeeper->exists(task_status_path, &stat)) + task_start_clock = LogicalClock(stat.mzxid); - Strings status_paths; - for (auto & shard : shards_with_partition) + /// If statement for readability. + if (clean_state_clock.is_clean() && (!task_start_clock.hasHappened() || clean_state_clock.discovery_zxid <= task_start_clock)) { - ShardPartition & task_shard_partition = shard->partition_tasks.find(partition_name)->second; - status_paths.emplace_back(task_shard_partition.getShardStatusPath()); + return true; } - - std::vector zxid1, zxid2; - - try + else { - std::vector get_futures; - for (const String & path : status_paths) - get_futures.emplace_back(zookeeper->asyncGet(path)); - - // Check that state is Finished and remember zxid - for (auto & future : get_futures) - { - auto res = future.get(); - - TaskStateWithOwner status = TaskStateWithOwner::fromString(res.data); - if (status.state != TaskState::Finished) - { - LOG_INFO(log, "The task " << res.data << " is being rewritten by " << status.owner << ". Partition will be rechecked"); - return false; - } - - zxid1.push_back(res.stat.pzxid); - } - - // Check that partition is not dirty - { - CleanStateClock clean_state_clock ( - zookeeper, - task_table.getCertainPartitionIsDirtyPath(partition_name), - task_table.getCertainPartitionIsCleanedPath(partition_name) - ); - Coordination::Stat stat{}; - LogicalClock task_start_clock; - if (zookeeper->exists(task_table.getCertainPartitionTaskStatusPath(partition_name), &stat)) - task_start_clock = LogicalClock(stat.mzxid); - zookeeper->get(task_table.getCertainPartitionTaskStatusPath(partition_name), &stat); - if (!clean_state_clock.is_clean() || task_start_clock <= clean_state_clock.discovery_zxid) - { - LOG_INFO(log, "Partition " << partition_name << " become dirty"); - return false; - } - } - - get_futures.clear(); - for (const String & path : status_paths) - get_futures.emplace_back(zookeeper->asyncGet(path)); - - // Remember zxid of states again - for (auto & future : get_futures) - { - auto res = future.get(); - zxid2.push_back(res.stat.pzxid); - } - } - catch (const Coordination::Exception & e) - { - LOG_INFO(log, "A ZooKeeper error occurred while checking partition " << partition_name - << ". Will recheck the partition. Error: " << e.displayText()); return false; } - // If all task is finished and zxid is not changed then partition could not become dirty again - for (UInt64 shard_num = 0; shard_num < status_paths.size(); ++shard_num) - { - if (zxid1[shard_num] != zxid2[shard_num]) - { - LOG_INFO(log, "The task " << status_paths[shard_num] << " is being modified now. Partition will be rechecked"); - return false; - } - } - - LOG_INFO(log, "Partition " << partition_name << " is copied successfully"); - return true; } -bool ClusterCopier::checkAllPieceInPartitionDone(const TaskTable & task_table, const String & partition_name, const TasksShard & shards_with_partition) + +bool ClusterCopier::checkAllPiecesInPartitionAreDone(const TaskTable & task_table, const String & partition_name, const TasksShard & shards_with_partition) { bool answer = true; for (size_t piece_number = 0; piece_number < task_table.number_of_splits; piece_number++) @@ -496,7 +429,7 @@ bool ClusterCopier::checkPartitionPieceIsDone(const TaskTable & task_table, cons size_t piece_number, const TasksShard & shards_with_partition) { LOG_DEBUG(log, "Check that all shards processed partition " << partition_name - << " piece number" + toString(piece_number) + " successfully"); + << " piece " + toString(piece_number) + " successfully"); auto zookeeper = context.getZooKeeper(); @@ -525,30 +458,42 @@ bool ClusterCopier::checkPartitionPieceIsDone(const TaskTable & task_table, cons TaskStateWithOwner status = TaskStateWithOwner::fromString(res.data); if (status.state != TaskState::Finished) { - LOG_INFO(log, "The task " << res.data << " is being rewritten by " << status.owner << ". Partition piece will be rechecked"); + LOG_INFO(log, "The task " << res.data << " is being rewritten by " + << status.owner << ". Partition piece will be rechecked"); return false; } zxid1.push_back(res.stat.pzxid); } - // Check that partition is not dirty + const String piece_is_dirty_flag_path = task_table.getCertainPartitionPieceIsDirtyPath(partition_name, piece_number); + const String piece_is_dirty_cleaned_path = task_table.getCertainPartitionPieceIsCleanedPath(partition_name, piece_number); + const String piece_task_status_path = task_table.getCertainPartitionPieceTaskStatusPath(partition_name, piece_number); + + CleanStateClock clean_state_clock (zookeeper, piece_is_dirty_flag_path, piece_is_dirty_cleaned_path); + + std::cout << piece_is_dirty_flag_path << std::endl; + std::cout << piece_is_dirty_cleaned_path << std::endl; + std::cout << piece_task_status_path << std::endl; + + const bool is_clean = checkPartitionPieceIsClean(zookeeper, clean_state_clock, piece_task_status_path); + + for (size_t i = 0; i < 10; ++i) { - CleanStateClock clean_state_clock ( - zookeeper, - task_table.getCertainPartitionIsDirtyPath(partition_name), - task_table.getCertainPartitionIsCleanedPath(partition_name) - ); - Coordination::Stat stat{}; - LogicalClock task_start_clock; - if (zookeeper->exists(task_table.getCertainPartitionTaskStatusPath(partition_name), &stat)) - task_start_clock = LogicalClock(stat.mzxid); - zookeeper->get(task_table.getCertainPartitionTaskStatusPath(partition_name), &stat); - if (!clean_state_clock.is_clean() || task_start_clock <= clean_state_clock.discovery_zxid) + if (checkPartitionPieceIsClean(zookeeper, clean_state_clock, piece_task_status_path)) { - LOG_INFO(log, "Partition " << partition_name << " piece number " << toString(piece_number) << " become dirty"); - return false; + std::cout << "clean" << std::endl; } + else + { + std::cout << "dirty" << std::endl; + } + } + + if (!is_clean) + { + LOG_INFO(log, "Partition " << partition_name << " become dirty"); + return false; } get_futures.clear(); @@ -565,7 +510,7 @@ bool ClusterCopier::checkPartitionPieceIsDone(const TaskTable & task_table, cons catch (const Coordination::Exception & e) { LOG_INFO(log, "A ZooKeeper error occurred while checking partition " << partition_name << " piece number " - << toString(piece_number) << ". Will recheck the partition. Error: " << e.displayText()); + << toString(piece_number) << ". Will recheck the partition. Error: " << e.displayText()); return false; } @@ -581,7 +526,6 @@ bool ClusterCopier::checkPartitionPieceIsDone(const TaskTable & task_table, cons LOG_INFO(log, "Partition " << partition_name << " is copied successfully"); return true; - } /// Removes MATERIALIZED and ALIAS columns from create table query @@ -637,8 +581,6 @@ std::shared_ptr ClusterCopier::rewriteCreateQueryStorage(const A } -/// TODO: implement tryDropPartitionPiece which is simply tryDropPartition, but on different table. - bool ClusterCopier::tryDropPartitionPiece( ShardPartition & task_partition, const size_t current_piece_number, @@ -886,7 +828,7 @@ bool ClusterCopier::tryProcessTable(const ConnectionTimeouts & timeouts, TaskTab { partition_is_done = !has_shard_to_process - || (partition_is_done && checkPartitionIsDone(task_table, partition_name, expected_shards)); + || (partition_is_done && checkAllPiecesInPartitionAreDone(task_table, partition_name, expected_shards)); } catch (...) { @@ -988,10 +930,11 @@ PartitionTaskStatus ClusterCopier::processPartitionPieceTaskImpl( const size_t number_of_splits = task_table.number_of_splits; const String primary_key_comma_separated = task_table.primary_key_comma_separated; - UNUSED(number_of_splits); - UNUSED(partition_piece); + /// We need to update table definitions for each partition, it could be changed after ALTER - createShardInternalTables(timeouts, task_shard, true, current_piece_number); + createShardInternalTables(timeouts, task_shard, true); + + auto split_table_for_current_piece = task_shard.list_of_split_tables_on_shard[current_piece_number]; auto zookeeper = context.getZooKeeper(); @@ -1029,7 +972,6 @@ PartitionTaskStatus ClusterCopier::processPartitionPieceTaskImpl( /// TODO: Bad, it is better to rewrite with ASTLiteral(partition_key_field) query += " WHERE (" + queryToString(task_table.engine_push_partition_key_ast) + " = (" + task_partition.name + " AS partition_key))"; - if (enable_splitting) query += " AND ( cityHash64(" + primary_key_comma_separated + ") %" + toString(number_of_splits) + " = " + toString(current_piece_number) + " )"; @@ -1048,18 +990,30 @@ PartitionTaskStatus ClusterCopier::processPartitionPieceTaskImpl( LOG_DEBUG(log, "Processing " << current_task_piece_status_path); - CleanStateClock clean_state_clock (zookeeper, piece_is_dirty_flag_path, piece_is_dirty_cleaned_path); + const String piece_status_path = partition_piece.getPartitionPieceShardsPath(); - LogicalClock task_start_clock; + CleanStateClock clean_state_clock(zookeeper, piece_is_dirty_flag_path, piece_is_dirty_cleaned_path); + + std::cout << piece_is_dirty_flag_path << std::endl; + std::cout << piece_is_dirty_cleaned_path << std::endl; + std::cout << piece_status_path << std::endl; + + const bool is_clean = checkPartitionPieceIsClean(zookeeper, clean_state_clock, piece_status_path); + + for (size_t i = 0; i < 10; ++i) { - Coordination::Stat stat{}; - if (zookeeper->exists(partition_piece.getPartitionPieceShardsPath(), &stat)) - task_start_clock = LogicalClock(stat.mzxid); + if (checkPartitionPieceIsClean(zookeeper, clean_state_clock, piece_status_path)) + { + std::cout << "clean" << std::endl; + } + else + { + std::cout << "dirty" << std::endl; + } } /// Do not start if partition piece is dirty, try to clean it - if (clean_state_clock.is_clean() - && (!task_start_clock.hasHappened() || clean_state_clock.discovery_zxid <= task_start_clock)) + if (is_clean) { LOG_DEBUG(log, "Partition " << task_partition.name << " piece " + toString(current_piece_number) + " appears to be clean"); @@ -1136,7 +1090,8 @@ PartitionTaskStatus ClusterCopier::processPartitionPieceTaskImpl( *zookeeper, host_id); // Maybe we are the first worker /// TODO: Why table_split_shard??? - ASTPtr query_select_ast = get_select_query(task_shard.table_split_shard, "count()", /*enable_splitting*/ true); + + ASTPtr query_select_ast = get_select_query(split_table_for_current_piece, "count()", /*enable_splitting*/ true); UInt64 count; { Context local_context = context; @@ -1228,7 +1183,7 @@ PartitionTaskStatus ClusterCopier::processPartitionPieceTaskImpl( ASTPtr query_insert_ast; { String query; - query += "INSERT INTO " + getQuotedTable(task_shard.table_split_shard) + " VALUES "; + query += "INSERT INTO " + getQuotedTable(split_table_for_current_piece) + " VALUES "; ParserQuery p_query(query.data() + query.size()); query_insert_ast = parseQuery(p_query, query, 0); @@ -1314,25 +1269,83 @@ PartitionTaskStatus ClusterCopier::processPartitionPieceTaskImpl( } } + LOG_INFO(log, "Partition " << task_partition.name << " piece " + << toString(current_piece_number) << " copied. But not moved to original destination table."); + + + /// Try create original table (if not exists) on each shard + { + auto create_query_push_ast = rewriteCreateQueryStorage(task_shard.current_pull_table_create_query, + task_table.table_push, task_table.engine_push_ast); + create_query_push_ast->as().if_not_exists = true; + String query = queryToString(create_query_push_ast); + + LOG_DEBUG(log, "Create destination tables. Query: " << query); + UInt64 shards = executeQueryOnCluster(task_table.cluster_push, query, + create_query_push_ast, &task_cluster->settings_push, + PoolMode::GET_MANY); + LOG_DEBUG(log, "Destination tables " << getQuotedTable(task_table.table_push) + << " have been created on " << shards << " shards of " << task_table.cluster_push->getShardCount()); + } + + /// Move partition to original destination table. + { + /// TODO: Execute alter table move partition. + + LOG_DEBUG(log, "Trying to move partition " << task_partition.name + << " piece " << toString(current_piece_number) << " to original table"); + + ASTPtr query_alter_ast; + String query_alter_ast_string; + + DatabaseAndTableName original_table = task_table.table_push; + DatabaseAndTableName helping_table = DatabaseAndTableName(original_table.first, original_table.second + "_piece_" + toString(current_piece_number)); + + query_alter_ast_string += "ALTER TABLE " + getQuotedTable(helping_table) + + " MOVE PARTITION " + task_partition.name + + " TO TABLE " + getQuotedTable(original_table); + + LOG_DEBUG(log, "Executing ALTER query: " << query_alter_ast_string); + + try + { + UInt64 num_shards = executeQueryOnCluster(task_table.cluster_push, query_alter_ast_string, nullptr, &task_cluster->settings_push, PoolMode::GET_ONE, 1); + + LOG_INFO(log, "Number of shard that executed ALTER query successfully : " << toString(num_shards)); + } + catch (...) + { + LOG_DEBUG(log, "Error while moving partition " << task_partition.name + << " piece " << toString(current_piece_number) << "to original table"); + throw; + } + + } + /// Finalize the processing, change state of current partition task (and also check is_dirty flag) { String state_finished = TaskStateWithOwner::getData(TaskState::Finished, host_id); CleanStateClock new_clean_state_clock (zookeeper, piece_is_dirty_flag_path, piece_is_dirty_cleaned_path); if (clean_state_clock != new_clean_state_clock) { - LOG_INFO(log, "Partition " << task_partition.name << " clean state changed, cowardly bailing"); + LOG_INFO(log, "Partition " << task_partition.name << " piece " + << toString(current_piece_number) << " clean state changed, cowardly bailing"); return PartitionTaskStatus::Error; } else if (!new_clean_state_clock.is_clean()) { - LOG_INFO(log, "Partition " << task_partition.name << " became dirty and will be dropped and refilled"); + LOG_INFO(log, "Partition " << task_partition.name << " piece " + << toString(current_piece_number) << " became dirty and will be dropped and refilled"); create_is_dirty_node(new_clean_state_clock); return PartitionTaskStatus::Error; } zookeeper->set(current_task_piece_status_path, state_finished, 0); } - LOG_INFO(log, "Partition " << task_partition.name << " copied"); + /// TODO: LOG_INFO (Piece copied and moved to destination table) + + + return PartitionTaskStatus::Finished; } @@ -1380,7 +1393,7 @@ ASTPtr ClusterCopier::getCreateTableForPullShard(const ConnectionTimeouts & time /// If it is implicitly asked to create split Distributed table for certain piece on current shard, we will do it. void ClusterCopier::createShardInternalTables(const ConnectionTimeouts & timeouts, - TaskShard & task_shard, bool create_split, const size_t piece_number) + TaskShard & task_shard, bool create_split) { TaskTable & task_table = task_shard.task_table; @@ -1392,8 +1405,13 @@ void ClusterCopier::createShardInternalTables(const ConnectionTimeouts & timeout String read_shard_prefix = ".read_shard_" + toString(task_shard.indexInCluster()) + "."; String split_shard_prefix = ".split."; task_shard.table_read_shard = DatabaseAndTableName(working_database_name, read_shard_prefix + task_table.table_id); - task_shard.table_split_shard = DatabaseAndTableName( - working_database_name, split_shard_prefix + task_table.table_id + "_piece_" + toString(piece_number)); + task_shard.main_table_split_shard = DatabaseAndTableName(working_database_name, split_shard_prefix + task_table.table_id); + + for (const auto & piece_number : ext::range(0, task_table.number_of_splits)) + { + task_shard.list_of_split_tables_on_shard[piece_number] = + DatabaseAndTableName(working_database_name, split_shard_prefix + task_table.table_id + "_piece_" + toString(piece_number)); + } /// Create special cluster with single shard String shard_read_cluster_name = read_shard_prefix + task_table.cluster_pull_name; @@ -1401,16 +1419,35 @@ void ClusterCopier::createShardInternalTables(const ConnectionTimeouts & timeout context.setCluster(shard_read_cluster_name, cluster_pull_current_shard); auto storage_shard_ast = createASTStorageDistributed(shard_read_cluster_name, task_table.table_pull.first, task_table.table_pull.second); - const auto & storage_piece_split_ast = task_table.auxiliary_engine_split_asts[piece_number]; auto create_query_ast = removeAliasColumnsFromCreateQuery(task_shard.current_pull_table_create_query); - auto create_table_pull_ast = rewriteCreateQueryStorage(create_query_ast, task_shard.table_read_shard, storage_shard_ast); - auto create_table_split_piece_ast = rewriteCreateQueryStorage(create_query_ast, task_shard.table_split_shard, storage_piece_split_ast); + auto create_table_pull_ast = rewriteCreateQueryStorage(create_query_ast, task_shard.table_read_shard, storage_shard_ast); dropAndCreateLocalTable(create_table_pull_ast); if (create_split) + { + auto create_table_split_piece_ast = rewriteCreateQueryStorage( + create_query_ast, + task_shard.main_table_split_shard, + task_table.main_engine_split_ast); + dropAndCreateLocalTable(create_table_split_piece_ast); + + /// Create auxilary split tables for each piece + for (const auto & piece_number : ext::range(0, task_table.number_of_splits)) + { + const auto & storage_piece_split_ast = task_table.auxiliary_engine_split_asts[piece_number]; + + create_table_split_piece_ast = rewriteCreateQueryStorage( + create_query_ast, + task_shard.list_of_split_tables_on_shard[piece_number], + storage_piece_split_ast); + + dropAndCreateLocalTable(create_table_split_piece_ast); + } + } + } diff --git a/dbms/programs/copier/ClusterCopier.h b/dbms/programs/copier/ClusterCopier.h index 2729b295bce..a5587045299 100644 --- a/dbms/programs/copier/ClusterCopier.h +++ b/dbms/programs/copier/ClusterCopier.h @@ -78,14 +78,23 @@ protected: const String & description, bool unprioritized); + /* + * Checks that partition piece or some other entity is clean. + * The only requirement is that you have to pass is_dirty_flag_path and is_dirty_cleaned_path to the function. + * And is_dirty_flag_path is a parent of is_dirty_cleaned_path. + * */ + bool checkPartitionPieceIsClean( + const zkutil::ZooKeeperPtr & zookeeper, + const CleanStateClock & clean_state_clock, + const String & task_status_path) const; + + bool checkAllPiecesInPartitionAreDone(const TaskTable & task_table, const String & partition_name, const TasksShard & shards_with_partition); + /** Checks that the whole partition of a table was copied. We should do it carefully due to dirty lock. * State of some task could change during the processing. * We have to ensure that all shards have the finished state and there is no dirty flag. * Moreover, we have to check status twice and check zxid, because state can change during the checking. */ - bool checkPartitionIsDone(const TaskTable & task_table, const String & partition_name, const TasksShard & shards_with_partition); - - bool checkAllPieceInPartitionDone(const TaskTable & task_table, const String & partition_name, const TasksShard & shards_with_partition); /* The same as function above * Assume that we don't know on which shards do we have partition certain piece. @@ -128,7 +137,8 @@ protected: ASTPtr getCreateTableForPullShard(const ConnectionTimeouts & timeouts, TaskShard & task_shard); /// If it is implicitly asked to create split Distributed table for certain piece on current shard, we will do it. - void createShardInternalTables(const ConnectionTimeouts & timeouts, TaskShard & task_shard, bool create_split = true, const size_t piece_number = 0); + /// TODO: rewrite comment + void createShardInternalTables(const ConnectionTimeouts & timeouts, TaskShard & task_shard, bool create_split = true); std::set getShardPartitions(const ConnectionTimeouts & timeouts, TaskShard & task_shard); diff --git a/dbms/programs/copier/ShardPartitionPiece.h b/dbms/programs/copier/ShardPartitionPiece.h index aeaa24fef13..04f7f458a00 100644 --- a/dbms/programs/copier/ShardPartitionPiece.h +++ b/dbms/programs/copier/ShardPartitionPiece.h @@ -38,7 +38,7 @@ struct ShardPartitionPiece { inline String ShardPartitionPiece::getPartitionPiecePath() const { - return shard_partition.getPartitionPath() + "/piece_" + std::to_string(current_piece_number); + return shard_partition.getPartitionPath() + "/piece_" + toString(current_piece_number); } inline String ShardPartitionPiece::getPartitionPieceCleanStartPath() const @@ -53,7 +53,7 @@ inline String ShardPartitionPiece::getPartitionPieceIsDirtyPath() const inline String ShardPartitionPiece::getPartitionPieceIsCleanedPath() const { - return getPartitionPieceIsDirtyPath() + "/is_cleaned"; + return getPartitionPieceIsDirtyPath() + "/cleaned"; } inline String ShardPartitionPiece::getPartitionPieceActiveWorkersPath() const diff --git a/dbms/programs/copier/TaskTableAndShard.h b/dbms/programs/copier/TaskTableAndShard.h index c0795340e47..a1b7620bbcd 100644 --- a/dbms/programs/copier/TaskTableAndShard.h +++ b/dbms/programs/copier/TaskTableAndShard.h @@ -24,30 +24,15 @@ struct TaskTable { String getCertainPartitionIsDirtyPath(const String & partition_name) const; - String getCertainPartitionPieceIsDirtyPath(const String & partition_name, const size_t piece_number) const - { - UNUSED(partition_name); - UNUSED(piece_number); - return "Not Implemented"; - } + String getCertainPartitionPieceIsDirtyPath(const String & partition_name, const size_t piece_number) const; String getCertainPartitionIsCleanedPath(const String & partition_name) const; - String getCertainPartitionPieceIsCleanedPath(const String & partition_name, const size_t piece_number) const - { - UNUSED(partition_name); - UNUSED(piece_number); - return "Not implemented"; - } + String getCertainPartitionPieceIsCleanedPath(const String & partition_name, const size_t piece_number) const; String getCertainPartitionTaskStatusPath(const String & partition_name) const; - String getCertainPartitionPieceTaskStatusPath(const String & partition_name, const size_t piece_number) const - { - UNUSED(partition_name); - UNUSED(piece_number); - return "Not implemented"; - } + String getCertainPartitionPieceTaskStatusPath(const String & partition_name, const size_t piece_number) const; /// Partitions will be splitted into number-of-splits pieces. /// Each piece will be copied independently. (10 by default) @@ -149,7 +134,10 @@ struct TaskTable { struct TaskShard { - TaskShard(TaskTable &parent, const ShardInfo &info_) : task_table(parent), info(info_) {} + TaskShard(TaskTable & parent, const ShardInfo & info_) : task_table(parent), info(info_) + { + list_of_split_tables_on_shard.assign(task_table.number_of_splits, DatabaseAndTableName()); + } TaskTable & task_table; @@ -182,9 +170,9 @@ struct TaskShard /// Internal distributed tables DatabaseAndTableName table_read_shard; - DatabaseAndTableName table_split_shard; + DatabaseAndTableName main_table_split_shard; - std::vector list_of_split_tables_on_shard; + ListOfDatabasesAndTableNames list_of_split_tables_on_shard; }; @@ -194,24 +182,42 @@ inline String TaskTable::getPartitionPath(const String &partition_name) const { + "/" + escapeForFileName(partition_name); // 201701 } -inline String TaskTable::getPartitionPiecePath(const String & partition_name, size_t piece_number) const { +inline String TaskTable::getPartitionPiecePath(const String & partition_name, size_t piece_number) const +{ assert(piece_number < number_of_splits); - return getPartitionPath(partition_name) + "/" + - std::to_string(piece_number); // 1...number_of_splits + return getPartitionPath(partition_name) + "/piece_" + toString(piece_number); // 1...number_of_splits } -inline String TaskTable::getCertainPartitionIsDirtyPath(const String &partition_name) const { +inline String TaskTable::getCertainPartitionIsDirtyPath(const String &partition_name) const +{ return getPartitionPath(partition_name) + "/is_dirty"; } -inline String TaskTable::getCertainPartitionIsCleanedPath(const String &partition_name) const { +inline String TaskTable::getCertainPartitionPieceIsDirtyPath(const String & partition_name, const size_t piece_number) const +{ + return getPartitionPiecePath(partition_name, piece_number) + "/is_dirty"; +} + +inline String TaskTable::getCertainPartitionIsCleanedPath(const String &partition_name) const +{ return getCertainPartitionIsDirtyPath(partition_name) + "/cleaned"; } -inline String TaskTable::getCertainPartitionTaskStatusPath(const String &partition_name) const { +inline String TaskTable::getCertainPartitionPieceIsCleanedPath(const String & partition_name, const size_t piece_number) const +{ + return getCertainPartitionPieceIsDirtyPath(partition_name, piece_number) + "/cleaned"; +} + +inline String TaskTable::getCertainPartitionTaskStatusPath(const String & partition_name) const +{ return getPartitionPath(partition_name) + "/shards"; } +inline String TaskTable::getCertainPartitionPieceTaskStatusPath(const String & partition_name, const size_t piece_number) const +{ + return getPartitionPiecePath(partition_name, piece_number) + "/shards"; +} + inline TaskTable::TaskTable(TaskCluster & parent, const Poco::Util::AbstractConfiguration & config, const String & prefix_, const String & table_key) : task_cluster(parent) From 16322b13528eb4ff52412784c453fa8527068513 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Fri, 21 Feb 2020 19:18:16 +0300 Subject: [PATCH 0033/1355] replace partition --- dbms/programs/copier/ClusterCopier.cpp | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/dbms/programs/copier/ClusterCopier.cpp b/dbms/programs/copier/ClusterCopier.cpp index e9dc613f85e..bd4c15b6fdb 100644 --- a/dbms/programs/copier/ClusterCopier.cpp +++ b/dbms/programs/copier/ClusterCopier.cpp @@ -1301,9 +1301,9 @@ PartitionTaskStatus ClusterCopier::processPartitionPieceTaskImpl( DatabaseAndTableName original_table = task_table.table_push; DatabaseAndTableName helping_table = DatabaseAndTableName(original_table.first, original_table.second + "_piece_" + toString(current_piece_number)); - query_alter_ast_string += "ALTER TABLE " + getQuotedTable(helping_table) + - " MOVE PARTITION " + task_partition.name + - " TO TABLE " + getQuotedTable(original_table); + query_alter_ast_string += " ALTER TABLE " + getQuotedTable(original_table) + + " REPLACE PARTITION " + task_partition.name + + " FROM " + getQuotedTable(helping_table); LOG_DEBUG(log, "Executing ALTER query: " << query_alter_ast_string); From 5e01f348c57adc2ddebb860b5a126019e550519b Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Fri, 21 Feb 2020 19:47:43 +0300 Subject: [PATCH 0034/1355] return back alter move --- dbms/programs/copier/ClusterCopier.cpp | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/dbms/programs/copier/ClusterCopier.cpp b/dbms/programs/copier/ClusterCopier.cpp index bd4c15b6fdb..1ab137ca141 100644 --- a/dbms/programs/copier/ClusterCopier.cpp +++ b/dbms/programs/copier/ClusterCopier.cpp @@ -1301,9 +1301,9 @@ PartitionTaskStatus ClusterCopier::processPartitionPieceTaskImpl( DatabaseAndTableName original_table = task_table.table_push; DatabaseAndTableName helping_table = DatabaseAndTableName(original_table.first, original_table.second + "_piece_" + toString(current_piece_number)); - query_alter_ast_string += " ALTER TABLE " + getQuotedTable(original_table) + - " REPLACE PARTITION " + task_partition.name + - " FROM " + getQuotedTable(helping_table); + query_alter_ast_string += " ALTER TABLE " + getQuotedTable(helping_table) + + " MOVE PARTITION " + task_partition.name + + " TO TABLE " + getQuotedTable(original_table); LOG_DEBUG(log, "Executing ALTER query: " << query_alter_ast_string); From 187872517f2f924b4ebd4bd3dc264459b9200e57 Mon Sep 17 00:00:00 2001 From: Avogar Date: Fri, 21 Feb 2020 21:49:18 +0300 Subject: [PATCH 0035/1355] Add DOS-style new line in FileSegmentationEngine. --- .../Formats/Impl/RegexpRowInputFormat.cpp | 16 +++++++++++----- 1 file changed, 11 insertions(+), 5 deletions(-) diff --git a/dbms/src/Processors/Formats/Impl/RegexpRowInputFormat.cpp b/dbms/src/Processors/Formats/Impl/RegexpRowInputFormat.cpp index 387f81f1155..90d1cf99b2e 100644 --- a/dbms/src/Processors/Formats/Impl/RegexpRowInputFormat.cpp +++ b/dbms/src/Processors/Formats/Impl/RegexpRowInputFormat.cpp @@ -3,6 +3,7 @@ #include #include #include +#include namespace DB { @@ -85,6 +86,7 @@ bool RegexpRowInputFormat::readField(size_t index, MutableColumns & columns) } catch (Exception & e) { + e.addMessage("(while read the value of key " + getPort().getHeader().getByPosition(index).name + ")"); throw; } return read; @@ -160,17 +162,21 @@ void registerInputFormatProcessorRegexp(FormatFactory & factory) static bool fileSegmentationEngineRegexpImpl(ReadBuffer & in, DB::Memory<> & memory, size_t min_chunk_size) { char * pos = in.position(); - bool need_more_data = true; - while (loadAtPosition(in, memory, pos) && need_more_data) + while (loadAtPosition(in, memory, pos) && (memory.size() + static_cast(pos - in.position()) >= min_chunk_size)) { pos = find_first_symbols<'\n', '\r'>(pos, in.buffer().end()); if (pos == in.buffer().end()) continue; - if (memory.size() + static_cast(pos - in.position()) >= min_chunk_size) - need_more_data = false; - ++pos; + // Support DOS-style newline ("\r\n") + if (*pos++ == '\r') + { + if (pos == in.buffer().end()) + loadAtPosition(in, memory, pos); + if (*pos == '\n') + ++pos; + } } saveUpToPosition(in, memory, pos); From 1b8174b9d068317cab8b1b1db88cd652c90d6922 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Sun, 23 Feb 2020 17:25:55 +0300 Subject: [PATCH 0036/1355] trigger ci --- dbms/programs/copier/Aliases.h | 1 - 1 file changed, 1 deletion(-) diff --git a/dbms/programs/copier/Aliases.h b/dbms/programs/copier/Aliases.h index fff57dd603f..d91685445f8 100644 --- a/dbms/programs/copier/Aliases.h +++ b/dbms/programs/copier/Aliases.h @@ -4,7 +4,6 @@ namespace DB - { using ConfigurationPtr = Poco::AutoPtr; From 476d25b6b27002678becf3e989e7cfa5c8237988 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Tue, 25 Feb 2020 15:38:11 +0300 Subject: [PATCH 0037/1355] fix build --- dbms/programs/copier/ClusterCopier.cpp | 15 ++++++++++----- 1 file changed, 10 insertions(+), 5 deletions(-) diff --git a/dbms/programs/copier/ClusterCopier.cpp b/dbms/programs/copier/ClusterCopier.cpp index 1ab137ca141..c6409552136 100644 --- a/dbms/programs/copier/ClusterCopier.cpp +++ b/dbms/programs/copier/ClusterCopier.cpp @@ -901,7 +901,7 @@ PartitionTaskStatus ClusterCopier::tryProcessPartitionTask(const ConnectionTimeo tryLogCurrentException(log, "An error occurred while updating the config"); } - return PartitionTaskStatus::Finished; + return res; } PartitionTaskStatus ClusterCopier::iterateThroughAllPiecesInPartition(const ConnectionTimeouts & timeouts, ShardPartition & task_partition, @@ -909,12 +909,17 @@ PartitionTaskStatus ClusterCopier::iterateThroughAllPiecesInPartition(const Conn { const size_t total_number_of_pieces = task_partition.task_shard.task_table.number_of_splits; - /// ThreadPool maybe ?? + PartitionTaskStatus res; + PartitionTaskStatus answer = PartitionTaskStatus::Finished; + for (size_t piece_number = 0; piece_number < total_number_of_pieces; piece_number++) - processPartitionPieceTaskImpl(timeouts, task_partition, piece_number, is_unprioritized_task); - - return PartitionTaskStatus::Finished; + { + res = processPartitionPieceTaskImpl(timeouts, task_partition, piece_number, is_unprioritized_task); + if (res == PartitionTaskStatus::Error) + answer = res; + } + return answer; } From 9591678f84172bcb9246a876c6ac1031c5375f0f Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Wed, 26 Feb 2020 14:44:21 +0300 Subject: [PATCH 0038/1355] useledd --- dbms/programs/copier/Internals.cpp | 10 ++++++++-- dbms/programs/copier/TaskTableAndShard.h | 2 ++ 2 files changed, 10 insertions(+), 2 deletions(-) diff --git a/dbms/programs/copier/Internals.cpp b/dbms/programs/copier/Internals.cpp index 93be1ea00e8..b0d54a65fee 100644 --- a/dbms/programs/copier/Internals.cpp +++ b/dbms/programs/copier/Internals.cpp @@ -73,6 +73,8 @@ ASTPtr extractPartitionKey(const ASTPtr & storage_ast) { String storage_str = queryToString(storage_ast); + std::cout << "inside extractPartitionKey " << storage_str << std::endl; + const auto & storage = storage_ast->as(); const auto & engine = storage.engine->as(); @@ -115,7 +117,12 @@ ASTPtr extractPrimaryKeyOrOrderBy(const ASTPtr & storage_ast) { String storage_str = queryToString(storage_ast); - const auto & storage = storage_ast->as(); + ParserStorage parser_storage; + auto new_storage_ast = parseQuery(parser_storage, storage_str, 0); + + std::cout << "inside extractPrimaryKeyOrOrderBy" << storage_str << std::endl; + + const auto & storage = new_storage_ast->as(); const auto & engine = storage.engine->as(); if (!endsWith(engine.name, "MergeTree")) @@ -124,7 +131,6 @@ ASTPtr extractPrimaryKeyOrOrderBy(const ASTPtr & storage_ast) ErrorCodes::BAD_ARGUMENTS); } - /// FIXME if (!isExtendedDefinitionStorage(storage_ast)) { throw Exception("Is not extended deginition storage " + storage_str + " Will be fixed later.", diff --git a/dbms/programs/copier/TaskTableAndShard.h b/dbms/programs/copier/TaskTableAndShard.h index a1b7620bbcd..a8590d54b9b 100644 --- a/dbms/programs/copier/TaskTableAndShard.h +++ b/dbms/programs/copier/TaskTableAndShard.h @@ -246,7 +246,9 @@ inline TaskTable::TaskTable(TaskCluster & parent, const Poco::Util::AbstractConf { ParserStorage parser_storage; engine_push_ast = parseQuery(parser_storage, engine_push_str, 0); + std::cout << engine_push_str << std::endl; engine_push_partition_key_ast = extractPartitionKey(engine_push_ast); + std::cout << engine_push_str << std::endl; primary_key_comma_separated = createCommaSeparatedStringFrom(extractPrimaryKeyString(engine_push_ast)); } From fec76334cc07c19acd427021edcefd2ed27008bf Mon Sep 17 00:00:00 2001 From: Avogar Date: Thu, 27 Feb 2020 15:33:56 +0300 Subject: [PATCH 0039/1355] Rename tests. --- ...input_format.reference => 01085_regexp_input_format.reference} | 0 ...{01079_regexp_input_format.sh => 01085_regexp_input_format.sh} | 0 ...ference => 01086_regexp_input_format_skip_unmatched.reference} | 0 ...p_unmatched.sh => 01086_regexp_input_format_skip_unmatched.sh} | 0 4 files changed, 0 insertions(+), 0 deletions(-) rename dbms/tests/queries/0_stateless/{01079_regexp_input_format.reference => 01085_regexp_input_format.reference} (100%) rename dbms/tests/queries/0_stateless/{01079_regexp_input_format.sh => 01085_regexp_input_format.sh} (100%) rename dbms/tests/queries/0_stateless/{01080_regexp_input_format_skip_unmatched.reference => 01086_regexp_input_format_skip_unmatched.reference} (100%) rename dbms/tests/queries/0_stateless/{01080_regexp_input_format_skip_unmatched.sh => 01086_regexp_input_format_skip_unmatched.sh} (100%) diff --git a/dbms/tests/queries/0_stateless/01079_regexp_input_format.reference b/dbms/tests/queries/0_stateless/01085_regexp_input_format.reference similarity index 100% rename from dbms/tests/queries/0_stateless/01079_regexp_input_format.reference rename to dbms/tests/queries/0_stateless/01085_regexp_input_format.reference diff --git a/dbms/tests/queries/0_stateless/01079_regexp_input_format.sh b/dbms/tests/queries/0_stateless/01085_regexp_input_format.sh similarity index 100% rename from dbms/tests/queries/0_stateless/01079_regexp_input_format.sh rename to dbms/tests/queries/0_stateless/01085_regexp_input_format.sh diff --git a/dbms/tests/queries/0_stateless/01080_regexp_input_format_skip_unmatched.reference b/dbms/tests/queries/0_stateless/01086_regexp_input_format_skip_unmatched.reference similarity index 100% rename from dbms/tests/queries/0_stateless/01080_regexp_input_format_skip_unmatched.reference rename to dbms/tests/queries/0_stateless/01086_regexp_input_format_skip_unmatched.reference diff --git a/dbms/tests/queries/0_stateless/01080_regexp_input_format_skip_unmatched.sh b/dbms/tests/queries/0_stateless/01086_regexp_input_format_skip_unmatched.sh similarity index 100% rename from dbms/tests/queries/0_stateless/01080_regexp_input_format_skip_unmatched.sh rename to dbms/tests/queries/0_stateless/01086_regexp_input_format_skip_unmatched.sh From aa0d44131d958c6545cee7ae3cbef6d680ef7969 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Tue, 3 Mar 2020 12:49:06 +0300 Subject: [PATCH 0040/1355] normal extracting --- dbms/programs/copier/ClusterCopier.cpp | 23 ------ dbms/programs/copier/Internals.cpp | 89 +++++++++++++++++++----- dbms/programs/copier/Internals.h | 8 ++- dbms/programs/copier/TaskTableAndShard.h | 2 +- 4 files changed, 77 insertions(+), 45 deletions(-) diff --git a/dbms/programs/copier/ClusterCopier.cpp b/dbms/programs/copier/ClusterCopier.cpp index 90ea75708ce..e3013c5bebe 100644 --- a/dbms/programs/copier/ClusterCopier.cpp +++ b/dbms/programs/copier/ClusterCopier.cpp @@ -486,17 +486,6 @@ bool ClusterCopier::checkPartitionPieceIsDone(const TaskTable & task_table, cons const bool is_clean = checkPartitionPieceIsClean(zookeeper, clean_state_clock, piece_task_status_path); - for (size_t i = 0; i < 10; ++i) - { - if (checkPartitionPieceIsClean(zookeeper, clean_state_clock, piece_task_status_path)) - { - std::cout << "clean" << std::endl; - } - else - { - std::cout << "dirty" << std::endl; - } - } if (!is_clean) { @@ -1013,18 +1002,6 @@ PartitionTaskStatus ClusterCopier::processPartitionPieceTaskImpl( const bool is_clean = checkPartitionPieceIsClean(zookeeper, clean_state_clock, piece_status_path); - for (size_t i = 0; i < 10; ++i) - { - if (checkPartitionPieceIsClean(zookeeper, clean_state_clock, piece_status_path)) - { - std::cout << "clean" << std::endl; - } - else - { - std::cout << "dirty" << std::endl; - } - } - /// Do not start if partition piece is dirty, try to clean it if (is_clean) { diff --git a/dbms/programs/copier/Internals.cpp b/dbms/programs/copier/Internals.cpp index e16b55bebb0..a0beccbde2c 100644 --- a/dbms/programs/copier/Internals.cpp +++ b/dbms/programs/copier/Internals.cpp @@ -1,4 +1,5 @@ #include "Internals.h" +#include namespace DB { @@ -117,16 +118,11 @@ ASTPtr extractPartitionKey(const ASTPtr & storage_ast) } } -ASTPtr extractPrimaryKeyOrOrderBy(const ASTPtr & storage_ast) +ASTPtr extractPrimaryKey(const ASTPtr & storage_ast) { String storage_str = queryToString(storage_ast); - ParserStorage parser_storage; - auto new_storage_ast = parseQuery(parser_storage, storage_str, 0); - - std::cout << "inside extractPrimaryKeyOrOrderBy" << storage_str << std::endl; - - const auto & storage = new_storage_ast->as(); + const auto & storage = storage_ast->as(); const auto & engine = storage.engine->as(); if (!endsWith(engine.name, "MergeTree")) @@ -144,10 +140,37 @@ ASTPtr extractPrimaryKeyOrOrderBy(const ASTPtr & storage_ast) if (storage.primary_key) return storage.primary_key->clone(); - return storage.order_by->clone(); + return nullptr; } -String createCommaSeparatedStringFrom(const Strings & strings) + +ASTPtr extractOrderBy(const ASTPtr & storage_ast) +{ + String storage_str = queryToString(storage_ast); + + const auto & storage = storage_ast->as(); + const auto & engine = storage.engine->as(); + + if (!endsWith(engine.name, "MergeTree")) + { + throw Exception("Unsupported engine was specified in " + storage_str + ", only *MergeTree engines are supported", + ErrorCodes::BAD_ARGUMENTS); + } + + if (!isExtendedDefinitionStorage(storage_ast)) + { + throw Exception("Is not extended deginition storage " + storage_str + " Will be fixed later.", + ErrorCodes::BAD_ARGUMENTS); + } + + if (storage.order_by) + return storage.order_by->clone(); + + throw Exception("ORDER BY cannot be empty", ErrorCodes::BAD_ARGUMENTS); +} + + +String createCommaSeparatedStringFrom(const Names & strings) { String answer; for (auto & string: strings) @@ -159,20 +182,50 @@ String createCommaSeparatedStringFrom(const Strings & strings) return answer; } -Strings extractPrimaryKeyString(const ASTPtr & storage_ast) +Names extractPrimaryKeyColumnNames(const ASTPtr & storage_ast) { - const auto primary_key_or_order_by = extractPrimaryKeyOrOrderBy(storage_ast)->as(); + const auto sorting_key_ast = extractOrderBy(storage_ast); + const auto primary_key_ast = extractPrimaryKey(storage_ast); - ASTPtr primary_key_or_order_by_arguments_ast = primary_key_or_order_by.arguments->clone(); - ASTs & primary_key_or_order_by_arguments = primary_key_or_order_by_arguments_ast->children; + const auto sorting_key_expr_list = MergeTreeData::extractKeyExpressionList(sorting_key_ast); + const auto primary_key_expr_list = primary_key_ast + ? MergeTreeData::extractKeyExpressionList(primary_key_ast) : sorting_key_expr_list->clone(); - Strings answer; - answer.reserve(primary_key_or_order_by_arguments.size()); + /// VersionedCollapsing ??? - for (auto & column : primary_key_or_order_by_arguments) - answer.push_back(column->getColumnName()); + size_t primary_key_size = primary_key_expr_list->children.size(); + size_t sorting_key_size = sorting_key_expr_list->children.size(); - return answer; + if (primary_key_size > sorting_key_size) + throw Exception("Primary key must be a prefix of the sorting key, but its length: " + + toString(primary_key_size) + " is greater than the sorting key length: " + toString(sorting_key_size), + ErrorCodes::BAD_ARGUMENTS); + + Names primary_key_columns; + Names sorting_key_columns; + NameSet primary_key_columns_set; + + for (size_t i = 0; i < sorting_key_size; ++i) + { + String sorting_key_column = sorting_key_expr_list->children[i]->getColumnName(); + sorting_key_columns.push_back(sorting_key_column); + + if (i < primary_key_size) + { + String pk_column = primary_key_expr_list->children[i]->getColumnName(); + if (pk_column != sorting_key_column) + throw Exception("Primary key must be a prefix of the sorting key, but in position " + + toString(i) + " its column is " + pk_column + ", not " + sorting_key_column, + ErrorCodes::BAD_ARGUMENTS); + + if (!primary_key_columns_set.emplace(pk_column).second) + throw Exception("Primary key contains duplicate columns", ErrorCodes::BAD_ARGUMENTS); + + primary_key_columns.push_back(pk_column); + } + } + + return primary_key_columns; } ShardPriority getReplicasPriority(const Cluster::Addresses & replicas, const std::string & local_hostname, UInt8 random) diff --git a/dbms/programs/copier/Internals.h b/dbms/programs/copier/Internals.h index 4f0db4a1cc3..ca20f88d870 100644 --- a/dbms/programs/copier/Internals.h +++ b/dbms/programs/copier/Internals.h @@ -202,11 +202,13 @@ ASTPtr extractPartitionKey(const ASTPtr & storage_ast); * the data is sorted by both the old and new sorting keys at the moment of table modification. * * */ -ASTPtr extractPrimaryKeyOrOrderBy(const ASTPtr & storage_ast); +ASTPtr extractPrimaryKey(const ASTPtr & storage_ast); -String createCommaSeparatedStringFrom(const Strings & strings); +ASTPtr extractOrderBy(const ASTPtr & storage_ast); -Strings extractPrimaryKeyString(const ASTPtr & storage_ast); +String createCommaSeparatedStringFrom(const Names & strings); + +Names extractPrimaryKeyColumnNames(const ASTPtr & storage_ast); ShardPriority getReplicasPriority(const Cluster::Addresses & replicas, const std::string & local_hostname, UInt8 random); diff --git a/dbms/programs/copier/TaskTableAndShard.h b/dbms/programs/copier/TaskTableAndShard.h index 47c6fbed948..e3a6a7b1bb1 100644 --- a/dbms/programs/copier/TaskTableAndShard.h +++ b/dbms/programs/copier/TaskTableAndShard.h @@ -254,7 +254,7 @@ inline TaskTable::TaskTable(TaskCluster & parent, const Poco::Util::AbstractConf std::cout << engine_push_str << std::endl; engine_push_partition_key_ast = extractPartitionKey(engine_push_ast); std::cout << engine_push_str << std::endl; - primary_key_comma_separated = createCommaSeparatedStringFrom(extractPrimaryKeyString(engine_push_ast)); + primary_key_comma_separated = createCommaSeparatedStringFrom(extractPrimaryKeyColumnNames(engine_push_ast)); } sharding_key_str = config.getString(table_prefix + "sharding_key"); From d9765991a7c0c94f8485e3040c17e17d11b71a59 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Tue, 3 Mar 2020 16:15:23 +0300 Subject: [PATCH 0041/1355] replicated engine support --- dbms/programs/copier/ClusterCopier.cpp | 24 ++++++++-------- dbms/programs/copier/Internals.cpp | 29 ++++++++++++++++++-- dbms/programs/copier/Internals.h | 2 ++ dbms/programs/copier/TaskTableAndShard.h | 35 ++++++++++++++++++++++-- 4 files changed, 74 insertions(+), 16 deletions(-) diff --git a/dbms/programs/copier/ClusterCopier.cpp b/dbms/programs/copier/ClusterCopier.cpp index e3013c5bebe..1a7ddce7e5e 100644 --- a/dbms/programs/copier/ClusterCopier.cpp +++ b/dbms/programs/copier/ClusterCopier.cpp @@ -480,10 +480,6 @@ bool ClusterCopier::checkPartitionPieceIsDone(const TaskTable & task_table, cons CleanStateClock clean_state_clock (zookeeper, piece_is_dirty_flag_path, piece_is_dirty_cleaned_path); - std::cout << piece_is_dirty_flag_path << std::endl; - std::cout << piece_is_dirty_cleaned_path << std::endl; - std::cout << piece_task_status_path << std::endl; - const bool is_clean = checkPartitionPieceIsClean(zookeeper, clean_state_clock, piece_task_status_path); @@ -559,7 +555,8 @@ ASTPtr ClusterCopier::removeAliasColumnsFromCreateQuery(const ASTPtr & query_ast } /// Replaces ENGINE and table name in a create query -std::shared_ptr ClusterCopier::rewriteCreateQueryStorage(const ASTPtr & create_query_ast, const DatabaseAndTableName & new_table, const ASTPtr & new_storage_ast) +std::shared_ptr ClusterCopier::rewriteCreateQueryStorage( + const ASTPtr & create_query_ast, const DatabaseAndTableName & new_table, const ASTPtr & new_storage_ast) { const auto & create = create_query_ast->as(); auto res = std::make_shared(create); @@ -574,6 +571,7 @@ std::shared_ptr ClusterCopier::rewriteCreateQueryStorage(const A res->set(res->columns_list, create.columns_list->clone()); res->set(res->storage, new_storage_ast->clone()); + return res; } @@ -996,10 +994,6 @@ PartitionTaskStatus ClusterCopier::processPartitionPieceTaskImpl( CleanStateClock clean_state_clock(zookeeper, piece_is_dirty_flag_path, piece_is_dirty_cleaned_path); - std::cout << piece_is_dirty_flag_path << std::endl; - std::cout << piece_is_dirty_cleaned_path << std::endl; - std::cout << piece_status_path << std::endl; - const bool is_clean = checkPartitionPieceIsClean(zookeeper, clean_state_clock, piece_status_path); /// Do not start if partition piece is dirty, try to clean it @@ -1142,8 +1136,16 @@ PartitionTaskStatus ClusterCopier::processPartitionPieceTaskImpl( task_table.table_push.first, task_table.table_push.second + "_piece_" + toString(current_piece_number)); - auto create_query_push_ast = rewriteCreateQueryStorage(task_shard.current_pull_table_create_query, - database_and_table_for_current_piece, task_table.engine_push_ast); + auto new_engine_push_ast = task_table.engine_push_ast; + if (task_table.isReplicatedTable()) + { + new_engine_push_ast = task_table.rewriteParamsForReplicatedTableFor(current_piece_number); + } + + auto create_query_push_ast = rewriteCreateQueryStorage( + task_shard.current_pull_table_create_query, + database_and_table_for_current_piece, new_engine_push_ast); + create_query_push_ast->as().if_not_exists = true; String query = queryToString(create_query_push_ast); diff --git a/dbms/programs/copier/Internals.cpp b/dbms/programs/copier/Internals.cpp index a0beccbde2c..263bf5f8590 100644 --- a/dbms/programs/copier/Internals.cpp +++ b/dbms/programs/copier/Internals.cpp @@ -78,8 +78,6 @@ ASTPtr extractPartitionKey(const ASTPtr & storage_ast) { String storage_str = queryToString(storage_ast); - std::cout << "inside extractPartitionKey " << storage_str << std::endl; - const auto & storage = storage_ast->as(); const auto & engine = storage.engine->as(); @@ -228,6 +226,33 @@ Names extractPrimaryKeyColumnNames(const ASTPtr & storage_ast) return primary_key_columns; } +String extractReplicatedTableZookeeperPath(const ASTPtr & storage_ast) +{ + String storage_str = queryToString(storage_ast); + + const auto & storage = storage_ast->as(); + const auto & engine = storage.engine->as(); + + if (!endsWith(engine.name, "MergeTree")) + { + throw Exception( + "Unsupported engine was specified in " + storage_str + ", only *MergeTree engines are supported", + ErrorCodes::BAD_ARGUMENTS); + } + + if (!startsWith(engine.name, "Replicated")) + { + return ""; + } + + auto replicated_table_arguments = engine.arguments->children; + + auto zk_table_path_ast = replicated_table_arguments[0]->as(); + auto zk_table_path_string = zk_table_path_ast.value.safeGet(); + + return zk_table_path_string; +} + ShardPriority getReplicasPriority(const Cluster::Addresses & replicas, const std::string & local_hostname, UInt8 random) { ShardPriority res; diff --git a/dbms/programs/copier/Internals.h b/dbms/programs/copier/Internals.h index ca20f88d870..14dc888a0b4 100644 --- a/dbms/programs/copier/Internals.h +++ b/dbms/programs/copier/Internals.h @@ -210,6 +210,8 @@ String createCommaSeparatedStringFrom(const Names & strings); Names extractPrimaryKeyColumnNames(const ASTPtr & storage_ast); +String extractReplicatedTableZookeeperPath(const ASTPtr & storage_ast); + ShardPriority getReplicasPriority(const Cluster::Addresses & replicas, const std::string & local_hostname, UInt8 random); } diff --git a/dbms/programs/copier/TaskTableAndShard.h b/dbms/programs/copier/TaskTableAndShard.h index e3a6a7b1bb1..b9f90dc1a77 100644 --- a/dbms/programs/copier/TaskTableAndShard.h +++ b/dbms/programs/copier/TaskTableAndShard.h @@ -39,6 +39,11 @@ struct TaskTable { String getCertainPartitionPieceTaskStatusPath(const String & partition_name, const size_t piece_number) const; + String getReplicatedEngineFirstArgumentForCurrentPiece(const size_t piece_number) const; + + + bool isReplicatedTable() { return engine_push_zk_path != ""; } + /// Partitions will be splitted into number-of-splits pieces. /// Each piece will be copied independently. (10 by default) size_t number_of_splits; @@ -65,6 +70,11 @@ struct TaskTable { ASTPtr engine_push_ast; ASTPtr engine_push_partition_key_ast; + /// First argument of Replicated...MergeTree() + String engine_push_zk_path; + + ASTPtr rewriteParamsForReplicatedTableFor(const size_t current_piece_number) const; + /* * A Distributed table definition used to split data * Distributed table will be created on each shard of default @@ -251,10 +261,9 @@ inline TaskTable::TaskTable(TaskCluster & parent, const Poco::Util::AbstractConf { ParserStorage parser_storage; engine_push_ast = parseQuery(parser_storage, engine_push_str, 0); - std::cout << engine_push_str << std::endl; engine_push_partition_key_ast = extractPartitionKey(engine_push_ast); - std::cout << engine_push_str << std::endl; primary_key_comma_separated = createCommaSeparatedStringFrom(extractPrimaryKeyColumnNames(engine_push_ast)); + engine_push_zk_path = extractReplicatedTableZookeeperPath(engine_push_ast); } sharding_key_str = config.getString(table_prefix + "sharding_key"); @@ -346,6 +355,24 @@ inline void TaskTable::initShards(RandomEngine && random_engine) local_shards.assign(all_shards.begin(), it_first_remote); } +inline String TaskTable::getReplicatedEngineFirstArgumentForCurrentPiece(const size_t piece_number) const +{ + assert (engine_push_zk_path != ""); + return engine_push_zk_path + "/" + toString(piece_number); +} + +inline ASTPtr TaskTable::rewriteParamsForReplicatedTableFor(const size_t current_piece_number) const +{ + const auto & new_engine_ast = engine_push_ast->clone()->as(); + + auto & replicated_table_arguments = new_engine_ast.arguments->children; + + auto & zk_table_path_ast = replicated_table_arguments[0]->as(); + zk_table_path_ast.value = getReplicatedEngineFirstArgumentForCurrentPiece(current_piece_number); + + return new_engine_ast.clone(); +} + inline String DB::TaskShard::getDescription() const { @@ -359,8 +386,10 @@ inline String DB::TaskShard::getDescription() const inline String DB::TaskShard::getHostNameExample() const { - auto &replicas = task_table.cluster_pull->getShardsAddresses().at(indexInCluster()); + auto & replicas = task_table.cluster_pull->getShardsAddresses().at(indexInCluster()); return replicas.at(0).readableString(); } + + } From 649df01627c2e9e388b0d21a5119072080ed53da Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Tue, 3 Mar 2020 19:36:47 +0300 Subject: [PATCH 0042/1355] rewrite replicated tables to plain merge tree --- dbms/programs/copier/ClusterCopier.cpp | 2 +- dbms/programs/copier/TaskTableAndShard.h | 23 +++++++++++++++-------- 2 files changed, 16 insertions(+), 9 deletions(-) diff --git a/dbms/programs/copier/ClusterCopier.cpp b/dbms/programs/copier/ClusterCopier.cpp index 1a7ddce7e5e..cdf530e8437 100644 --- a/dbms/programs/copier/ClusterCopier.cpp +++ b/dbms/programs/copier/ClusterCopier.cpp @@ -1139,7 +1139,7 @@ PartitionTaskStatus ClusterCopier::processPartitionPieceTaskImpl( auto new_engine_push_ast = task_table.engine_push_ast; if (task_table.isReplicatedTable()) { - new_engine_push_ast = task_table.rewriteParamsForReplicatedTableFor(current_piece_number); + new_engine_push_ast = task_table.rewriteReplicatedCreateQueryToPlain(); } auto create_query_push_ast = rewriteCreateQueryStorage( diff --git a/dbms/programs/copier/TaskTableAndShard.h b/dbms/programs/copier/TaskTableAndShard.h index b9f90dc1a77..c879152d6c9 100644 --- a/dbms/programs/copier/TaskTableAndShard.h +++ b/dbms/programs/copier/TaskTableAndShard.h @@ -39,7 +39,7 @@ struct TaskTable { String getCertainPartitionPieceTaskStatusPath(const String & partition_name, const size_t piece_number) const; - String getReplicatedEngineFirstArgumentForCurrentPiece(const size_t piece_number) const; + [[maybe_unused]] String getReplicatedEngineFirstArgumentForCurrentPiece(const size_t piece_number) const; bool isReplicatedTable() { return engine_push_zk_path != ""; } @@ -73,7 +73,7 @@ struct TaskTable { /// First argument of Replicated...MergeTree() String engine_push_zk_path; - ASTPtr rewriteParamsForReplicatedTableFor(const size_t current_piece_number) const; + ASTPtr rewriteReplicatedCreateQueryToPlain(); /* * A Distributed table definition used to split data @@ -358,19 +358,26 @@ inline void TaskTable::initShards(RandomEngine && random_engine) inline String TaskTable::getReplicatedEngineFirstArgumentForCurrentPiece(const size_t piece_number) const { assert (engine_push_zk_path != ""); - return engine_push_zk_path + "/" + toString(piece_number); + return engine_push_zk_path + "/piece_" + toString(piece_number); } -inline ASTPtr TaskTable::rewriteParamsForReplicatedTableFor(const size_t current_piece_number) const +inline ASTPtr TaskTable::rewriteReplicatedCreateQueryToPlain() { - const auto & new_engine_ast = engine_push_ast->clone()->as(); + ASTPtr prev_engine_push_ast = engine_push_ast->clone(); + + auto & new_storage_ast = prev_engine_push_ast->as(); + auto & new_engine_ast = new_storage_ast.engine->as(); auto & replicated_table_arguments = new_engine_ast.arguments->children; - auto & zk_table_path_ast = replicated_table_arguments[0]->as(); - zk_table_path_ast.value = getReplicatedEngineFirstArgumentForCurrentPiece(current_piece_number); + /// Delete first two arguments of Replicated...MergeTree() table. + replicated_table_arguments.erase(replicated_table_arguments.begin()); + replicated_table_arguments.erase(replicated_table_arguments.begin()); - return new_engine_ast.clone(); + /// Remove replicated from name + new_engine_ast.name = new_engine_ast.name.substr(10); + + return new_storage_ast.clone(); } From 12d5900d71d29a26189c9780f457a570ea927a0d Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Sat, 7 Mar 2020 03:05:49 +0300 Subject: [PATCH 0043/1355] most tests ok --- dbms/programs/copier/ClusterCopier.cpp | 62 ++++++++++++++++--- .../Storages/StorageReplicatedMergeTree.cpp | 3 + .../integration/test_cluster_copier/test.py | 10 +++ 3 files changed, 65 insertions(+), 10 deletions(-) diff --git a/dbms/programs/copier/ClusterCopier.cpp b/dbms/programs/copier/ClusterCopier.cpp index cdf530e8437..6eb0ffcfac9 100644 --- a/dbms/programs/copier/ClusterCopier.cpp +++ b/dbms/programs/copier/ClusterCopier.cpp @@ -904,17 +904,43 @@ PartitionTaskStatus ClusterCopier::iterateThroughAllPiecesInPartition(const Conn { const size_t total_number_of_pieces = task_partition.task_shard.task_table.number_of_splits; - PartitionTaskStatus res; - PartitionTaskStatus answer = PartitionTaskStatus::Finished; + PartitionTaskStatus res{PartitionTaskStatus::Finished}; + + bool was_failed_pieces = false; + bool was_active_pieces = false; + bool was_error = false; for (size_t piece_number = 0; piece_number < total_number_of_pieces; piece_number++) { - res = processPartitionPieceTaskImpl(timeouts, task_partition, piece_number, is_unprioritized_task); - if (res == PartitionTaskStatus::Error) - answer = res; + for (UInt64 try_num = 0; try_num < max_shard_partition_tries; ++try_num) + { + res = processPartitionPieceTaskImpl(timeouts, task_partition, piece_number, is_unprioritized_task); + + /// Exit if success + if (res == PartitionTaskStatus::Finished) + break; + + was_error = true; + + /// Skip if the task is being processed by someone + if (res == PartitionTaskStatus::Active) + break; + + /// Repeat on errors + std::this_thread::sleep_for(default_sleep_time); + } + + was_active_pieces = (res == PartitionTaskStatus::Active); + was_failed_pieces = (res == PartitionTaskStatus::Error); } - return answer; + if (was_failed_pieces) + return PartitionTaskStatus::Error; + + if (was_active_pieces) + return PartitionTaskStatus::Active; + + return PartitionTaskStatus::Finished; } @@ -1266,6 +1292,7 @@ PartitionTaskStatus ClusterCopier::processPartitionPieceTaskImpl( /// Try create original table (if not exists) on each shard + try { auto create_query_push_ast = rewriteCreateQueryStorage(task_shard.current_pull_table_create_query, task_table.table_push, task_table.engine_push_ast); @@ -1279,6 +1306,10 @@ PartitionTaskStatus ClusterCopier::processPartitionPieceTaskImpl( LOG_DEBUG(log, "Destination tables " << getQuotedTable(task_table.table_push) << " have been created on " << shards << " shards of " << task_table.cluster_push->getShardCount()); } + catch (...) + { + tryLogCurrentException(log, "Error while creating original table. Maybe we are not first."); + } /// Move partition to original destination table. { @@ -1293,16 +1324,25 @@ PartitionTaskStatus ClusterCopier::processPartitionPieceTaskImpl( DatabaseAndTableName original_table = task_table.table_push; DatabaseAndTableName helping_table = DatabaseAndTableName(original_table.first, original_table.second + "_piece_" + toString(current_piece_number)); - query_alter_ast_string += " ALTER TABLE " + getQuotedTable(helping_table) + - " MOVE PARTITION " + task_partition.name + - " TO TABLE " + getQuotedTable(original_table); + query_alter_ast_string += " ALTER TABLE " + getQuotedTable(original_table) + + " ATTACH PARTITION " + task_partition.name + + " FROM " + getQuotedTable(helping_table); LOG_DEBUG(log, "Executing ALTER query: " << query_alter_ast_string); +// query_alter_ast_string += " INSERT INTO " + getQuotedTable(original_table) + +// " SELECT * FROM " + getQuotedTable(helping_table); +// +// query_alter_ast_string += " WHERE (" + queryToString(task_table.engine_push_partition_key_ast) + " = (" + task_partition.name + " AS partition_key))"; +// +// LOG_DEBUG(log, "Executing ALTER query: " << query_alter_ast_string); + try { - UInt64 num_shards = executeQueryOnCluster(task_table.cluster_push, query_alter_ast_string, nullptr, &task_cluster->settings_push, PoolMode::GET_ONE, 1); + ///FIXME: We have to be sure that every node in cluster executed this query + UInt64 num_shards = executeQueryOnCluster(task_table.cluster_push, query_alter_ast_string, nullptr, &task_cluster->settings_push, PoolMode::GET_MANY); + assert(num_shards > 0); LOG_INFO(log, "Number of shard that executed ALTER query successfully : " << toString(num_shards)); } catch (...) @@ -1436,6 +1476,8 @@ void ClusterCopier::createShardInternalTables(const ConnectionTimeouts & timeout task_shard.list_of_split_tables_on_shard[piece_number], storage_piece_split_ast); + std::cout << "anime" << queryToString(create_table_split_piece_ast) << std::endl; + dropAndCreateLocalTable(create_table_split_piece_ast); } } diff --git a/dbms/src/Storages/StorageReplicatedMergeTree.cpp b/dbms/src/Storages/StorageReplicatedMergeTree.cpp index 0ed2527a981..7ea195d496d 100644 --- a/dbms/src/Storages/StorageReplicatedMergeTree.cpp +++ b/dbms/src/Storages/StorageReplicatedMergeTree.cpp @@ -4877,6 +4877,9 @@ void StorageReplicatedMergeTree::replacePartitionFrom(const StoragePtr & source_ ErrorCodes::LOGICAL_ERROR); String hash_hex = src_part->checksums.getTotalChecksumHex(); + + LOG_INFO(log, "Trying to attach " << src_part->name << "with hash_hex " << hash_hex); + String block_id_path = replace ? "" : (zookeeper_path + "/blocks/" + partition_id + "_replace_from_" + hash_hex); auto lock = allocateBlockNumber(partition_id, zookeeper, block_id_path); diff --git a/dbms/tests/integration/test_cluster_copier/test.py b/dbms/tests/integration/test_cluster_copier/test.py index c223a73f59e..4677c49eb43 100644 --- a/dbms/tests/integration/test_cluster_copier/test.py +++ b/dbms/tests/integration/test_cluster_copier/test.py @@ -92,6 +92,16 @@ class Task1: def check(self): assert TSV(self.cluster.instances['s0_0_0'].query("SELECT count() FROM hits_all")) == TSV("1002\n") + for anime in ['s1_0_0', 's1_0_1', 's1_1_0']: + a = self.cluster.instances[anime].query("SELECT count() FROM hits_piece_0") + b = self.cluster.instances[anime].query("SELECT count() FROM hits_piece_1") + c = self.cluster.instances[anime].query("SELECT count() FROM hits") + print(anime, a, b, int(a) + int(b), c) + print(self.cluster.instances[anime].query("select partition, name, database, table, hash_of_all_files, hash_of_uncompressed_files, uncompressed_hash_of_compressed_files from system.parts where table like '%hits%' format TSV")) + + assert TSV(self.cluster.instances['s1_0_0'].query("SELECT DISTINCT d % 2 FROM hits")) == TSV("1\n") + assert TSV(self.cluster.instances['s1_1_0'].query("SELECT DISTINCT d % 2 FROM hits")) == TSV("0\n") + assert TSV(self.cluster.instances['s1_0_0'].query("SELECT count() FROM hits_all")) == TSV("1002\n") assert TSV(self.cluster.instances['s1_0_0'].query("SELECT DISTINCT d % 2 FROM hits")) == TSV("1\n") From 1acffc674d8c3875c539579c56f29a825f509bd6 Mon Sep 17 00:00:00 2001 From: Vasily Nemkov Date: Tue, 10 Mar 2020 14:14:34 +0800 Subject: [PATCH 0044/1355] Fixed DataTypeDateTime::getName() with constructed from TimezoneMixin --- dbms/src/DataTypes/DataTypeDateTime.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/src/DataTypes/DataTypeDateTime.cpp b/dbms/src/DataTypes/DataTypeDateTime.cpp index 5589a372732..8903d3b03b3 100644 --- a/dbms/src/DataTypes/DataTypeDateTime.cpp +++ b/dbms/src/DataTypes/DataTypeDateTime.cpp @@ -49,7 +49,7 @@ DataTypeDateTime::DataTypeDateTime(const String & time_zone_name, const String & } DataTypeDateTime::DataTypeDateTime(const TimezoneMixin & time_zone_) - : TimezoneMixin(time_zone_) + : TimezoneMixin(time_zone_), type_name(family_name) {} String DataTypeDateTime::doGetName() const From 308b7bfe54f67451e5ce0fe2581d41dfc409bdce Mon Sep 17 00:00:00 2001 From: Vasily Nemkov Date: Tue, 10 Mar 2020 14:15:14 +0800 Subject: [PATCH 0045/1355] Fixed binary operators with DateTime, now result is of same TimeZone as the DateTime operand --- dbms/src/Functions/FunctionBinaryArithmetic.h | 12 ++++++++++++ 1 file changed, 12 insertions(+) diff --git a/dbms/src/Functions/FunctionBinaryArithmetic.h b/dbms/src/Functions/FunctionBinaryArithmetic.h index 233598370ad..63961cc0bb7 100644 --- a/dbms/src/Functions/FunctionBinaryArithmetic.h +++ b/dbms/src/Functions/FunctionBinaryArithmetic.h @@ -823,6 +823,18 @@ public: type_res = std::make_shared(left.getPrecision(), left.getScale()); else if constexpr (IsDataTypeDecimal) type_res = std::make_shared(right.getPrecision(), right.getScale()); + // Special case for DateTime: binary OPS should not looze the timezone of the result type, + // but reuse timezone of DateTime argument. + // NOTE: binary plus/minus are not allowed on DateTime64, and we are not handling it here. + else if constexpr (std::is_same_v) + { + const TimezoneMixin * tz = nullptr; + if constexpr (std::is_same_v) + tz = &right; + if constexpr (std::is_same_v) + tz = &left; + type_res = std::make_shared(*tz); + } else type_res = std::make_shared(); return true; From 99bf8950afb2d0d70b28c4482671cace96585611 Mon Sep 17 00:00:00 2001 From: Vasily Nemkov Date: Tue, 10 Mar 2020 14:15:54 +0800 Subject: [PATCH 0046/1355] Fixed invalid test reference value of toStartOfTheDay --- dbms/tests/queries/0_stateless/00921_datetime64_basic.reference | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/tests/queries/0_stateless/00921_datetime64_basic.reference b/dbms/tests/queries/0_stateless/00921_datetime64_basic.reference index 6352bd34f98..ab2b602412f 100644 --- a/dbms/tests/queries/0_stateless/00921_datetime64_basic.reference +++ b/dbms/tests/queries/0_stateless/00921_datetime64_basic.reference @@ -1,3 +1,3 @@ 2019-09-16 19:20:11.000 -2019-05-03 11:25:25.123 2019-05-03 2019-05-02 21:00:00 2019-04-01 1970-01-02 11:25:25 2019-05-03 11:25:00 +2019-05-03 11:25:25.123 2019-05-03 2019-05-03 00:00:00 2019-04-01 1970-01-02 11:25:25 2019-05-03 11:25:00 2019-09-16 19:20:11.234 From 82a7939a013ba27993385ff8a926d9ac70c766ad Mon Sep 17 00:00:00 2001 From: Vasily Nemkov Date: Tue, 10 Mar 2020 15:06:51 +0800 Subject: [PATCH 0047/1355] Fixed getting Timezone from DateTime64 --- .../Functions/extractTimeZoneFromFunctionArguments.cpp | 8 +++++--- 1 file changed, 5 insertions(+), 3 deletions(-) diff --git a/dbms/src/Functions/extractTimeZoneFromFunctionArguments.cpp b/dbms/src/Functions/extractTimeZoneFromFunctionArguments.cpp index b49ceeedab3..9402359dfd3 100644 --- a/dbms/src/Functions/extractTimeZoneFromFunctionArguments.cpp +++ b/dbms/src/Functions/extractTimeZoneFromFunctionArguments.cpp @@ -42,9 +42,9 @@ std::string extractTimeZoneNameFromFunctionArguments(const ColumnsWithTypeAndNam return {}; /// If time zone is attached to an argument of type DateTime. - if (const DataTypeDateTime * type = checkAndGetDataType(arguments[datetime_arg_num].type.get())) + if (const auto * type = checkAndGetDataType(arguments[datetime_arg_num].type.get())) return type->getTimeZone().getTimeZone(); - if (const DataTypeDateTime64 * type = checkAndGetDataType(arguments[datetime_arg_num].type.get())) + if (const auto * type = checkAndGetDataType(arguments[datetime_arg_num].type.get())) return type->getTimeZone().getTimeZone(); return {}; @@ -61,7 +61,9 @@ const DateLUTImpl & extractTimeZoneFromFunctionArguments(Block & block, const Co return DateLUT::instance(); /// If time zone is attached to an argument of type DateTime. - if (const DataTypeDateTime * type = checkAndGetDataType(block.getByPosition(arguments[datetime_arg_num]).type.get())) + if (const auto * type = checkAndGetDataType(block.getByPosition(arguments[datetime_arg_num]).type.get())) + return type->getTimeZone(); + if (const auto * type = checkAndGetDataType(block.getByPosition(arguments[datetime_arg_num]).type.get())) return type->getTimeZone(); return DateLUT::instance(); From c3651c3a0aa0e034a61695f6d8505dcb4fc2a74b Mon Sep 17 00:00:00 2001 From: Vasily Nemkov Date: Tue, 10 Mar 2020 15:07:13 +0800 Subject: [PATCH 0048/1355] Fixed formatDateTime to utilize timezone of DateTime/DateTime64 argument --- dbms/src/Functions/formatDateTime.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/dbms/src/Functions/formatDateTime.cpp b/dbms/src/Functions/formatDateTime.cpp index 5a3ee7b1c48..ce6a30aac89 100644 --- a/dbms/src/Functions/formatDateTime.cpp +++ b/dbms/src/Functions/formatDateTime.cpp @@ -314,8 +314,8 @@ public: size_t result_size = pattern_to_fill.size(); const DateLUTImpl * time_zone_tmp = nullptr; - if (arguments.size() == 3) - time_zone_tmp = &extractTimeZoneFromFunctionArguments(block, arguments, 2, 0); + if (std::is_same_v || std::is_same_v) + time_zone_tmp = &extractTimeZoneFromFunctionArguments(block, arguments, 3, 0); else time_zone_tmp = &DateLUT::instance(); From b55cca62864624b3736428642d73380fc95d0f04 Mon Sep 17 00:00:00 2001 From: Vasily Nemkov Date: Tue, 10 Mar 2020 15:09:30 +0800 Subject: [PATCH 0049/1355] Updated test reference values to include timezone of DateTime/DateTime64 result type. --- .../00921_datetime64_compatibility.reference | 48 +++++++++---------- 1 file changed, 24 insertions(+), 24 deletions(-) diff --git a/dbms/tests/queries/0_stateless/00921_datetime64_compatibility.reference b/dbms/tests/queries/0_stateless/00921_datetime64_compatibility.reference index 120dbee6434..5ca31f0bfc6 100644 --- a/dbms/tests/queries/0_stateless/00921_datetime64_compatibility.reference +++ b/dbms/tests/queries/0_stateless/00921_datetime64_compatibility.reference @@ -620,19 +620,19 @@ SELECT DT64 >= N ------------------------------------------ SELECT N + toUInt8(1) "Date","2019-09-17" -"DateTime","2019-09-16 19:20:12" +"DateTime('Europe/Minsk')","2019-09-16 19:20:12" Code: 43: Illegal types DateTime64(3, 'Europe/Minsk') and UInt8 of arguments of function plus. ------------------------------------------ SELECT toUInt8(1) + N "Date","2019-09-17" -"DateTime","2019-09-16 19:20:12" +"DateTime('Europe/Minsk')","2019-09-16 19:20:12" Code: 43: Illegal types UInt8 and DateTime64(3, 'Europe/Minsk') of arguments of function plus. ------------------------------------------ SELECT N - toUInt8(1) "Date","2019-09-15" -"DateTime","2019-09-16 19:20:10" +"DateTime('Europe/Minsk')","2019-09-16 19:20:10" Code: 43: Illegal types DateTime64(3, 'Europe/Minsk') and UInt8 of arguments of function minus. ------------------------------------------ @@ -718,19 +718,19 @@ Code: 43: Illegal types of arguments (UInt8, Date) of function greaterOrEquals. ------------------------------------------ SELECT N + toInt8(-1) "Date","2019-09-15" -"DateTime","2019-09-16 19:20:10" +"DateTime('Europe/Minsk')","2019-09-16 19:20:10" Code: 43: Illegal types DateTime64(3, 'Europe/Minsk') and Int8 of arguments of function plus. ------------------------------------------ SELECT toInt8(-1) + N "Date","2019-09-15" -"DateTime","2019-09-16 19:20:10" +"DateTime('Europe/Minsk')","2019-09-16 19:20:10" Code: 43: Illegal types Int8 and DateTime64(3, 'Europe/Minsk') of arguments of function plus. ------------------------------------------ SELECT N - toInt8(-1) "Date","2019-09-17" -"DateTime","2019-09-16 19:20:12" +"DateTime('Europe/Minsk')","2019-09-16 19:20:12" Code: 43: Illegal types DateTime64(3, 'Europe/Minsk') and Int8 of arguments of function minus. ------------------------------------------ @@ -816,19 +816,19 @@ Code: 43: Illegal types of arguments (Int8, Date) of function greaterOrEquals. ------------------------------------------ SELECT N + toUInt16(1) "Date","2019-09-17" -"DateTime","2019-09-16 19:20:12" +"DateTime('Europe/Minsk')","2019-09-16 19:20:12" Code: 43: Illegal types DateTime64(3, 'Europe/Minsk') and UInt16 of arguments of function plus. ------------------------------------------ SELECT toUInt16(1) + N "Date","2019-09-17" -"DateTime","2019-09-16 19:20:12" +"DateTime('Europe/Minsk')","2019-09-16 19:20:12" Code: 43: Illegal types UInt16 and DateTime64(3, 'Europe/Minsk') of arguments of function plus. ------------------------------------------ SELECT N - toUInt16(1) "Date","2019-09-15" -"DateTime","2019-09-16 19:20:10" +"DateTime('Europe/Minsk')","2019-09-16 19:20:10" Code: 43: Illegal types DateTime64(3, 'Europe/Minsk') and UInt16 of arguments of function minus. ------------------------------------------ @@ -914,19 +914,19 @@ Code: 43: Illegal types of arguments (UInt16, Date) of function greaterOrEquals. ------------------------------------------ SELECT N + toInt16(-1) "Date","2019-09-15" -"DateTime","2019-09-16 19:20:10" +"DateTime('Europe/Minsk')","2019-09-16 19:20:10" Code: 43: Illegal types DateTime64(3, 'Europe/Minsk') and Int16 of arguments of function plus. ------------------------------------------ SELECT toInt16(-1) + N "Date","2019-09-15" -"DateTime","2019-09-16 19:20:10" +"DateTime('Europe/Minsk')","2019-09-16 19:20:10" Code: 43: Illegal types Int16 and DateTime64(3, 'Europe/Minsk') of arguments of function plus. ------------------------------------------ SELECT N - toInt16(-1) "Date","2019-09-17" -"DateTime","2019-09-16 19:20:12" +"DateTime('Europe/Minsk')","2019-09-16 19:20:12" Code: 43: Illegal types DateTime64(3, 'Europe/Minsk') and Int16 of arguments of function minus. ------------------------------------------ @@ -1012,19 +1012,19 @@ Code: 43: Illegal types of arguments (Int16, Date) of function greaterOrEquals. ------------------------------------------ SELECT N + toUInt32(1) "Date","2019-09-17" -"DateTime","2019-09-16 19:20:12" +"DateTime('Europe/Minsk')","2019-09-16 19:20:12" Code: 43: Illegal types DateTime64(3, 'Europe/Minsk') and UInt32 of arguments of function plus. ------------------------------------------ SELECT toUInt32(1) + N "Date","2019-09-17" -"DateTime","2019-09-16 19:20:12" +"DateTime('Europe/Minsk')","2019-09-16 19:20:12" Code: 43: Illegal types UInt32 and DateTime64(3, 'Europe/Minsk') of arguments of function plus. ------------------------------------------ SELECT N - toUInt32(1) "Date","2019-09-15" -"DateTime","2019-09-16 19:20:10" +"DateTime('Europe/Minsk')","2019-09-16 19:20:10" Code: 43: Illegal types DateTime64(3, 'Europe/Minsk') and UInt32 of arguments of function minus. ------------------------------------------ @@ -1110,19 +1110,19 @@ Code: 43: Illegal types of arguments (UInt32, Date) of function greaterOrEquals. ------------------------------------------ SELECT N + toInt32(-1) "Date","2019-09-15" -"DateTime","2019-09-16 19:20:10" +"DateTime('Europe/Minsk')","2019-09-16 19:20:10" Code: 43: Illegal types DateTime64(3, 'Europe/Minsk') and Int32 of arguments of function plus. ------------------------------------------ SELECT toInt32(-1) + N "Date","2019-09-15" -"DateTime","2019-09-16 19:20:10" +"DateTime('Europe/Minsk')","2019-09-16 19:20:10" Code: 43: Illegal types Int32 and DateTime64(3, 'Europe/Minsk') of arguments of function plus. ------------------------------------------ SELECT N - toInt32(-1) "Date","2019-09-17" -"DateTime","2019-09-16 19:20:12" +"DateTime('Europe/Minsk')","2019-09-16 19:20:12" Code: 43: Illegal types DateTime64(3, 'Europe/Minsk') and Int32 of arguments of function minus. ------------------------------------------ @@ -1208,19 +1208,19 @@ Code: 43: Illegal types of arguments (Int32, Date) of function greaterOrEquals. ------------------------------------------ SELECT N + toUInt64(1) "Date","2019-09-17" -"DateTime","2019-09-16 19:20:12" +"DateTime('Europe/Minsk')","2019-09-16 19:20:12" Code: 43: Illegal types DateTime64(3, 'Europe/Minsk') and UInt64 of arguments of function plus. ------------------------------------------ SELECT toUInt64(1) + N "Date","2019-09-17" -"DateTime","2019-09-16 19:20:12" +"DateTime('Europe/Minsk')","2019-09-16 19:20:12" Code: 43: Illegal types UInt64 and DateTime64(3, 'Europe/Minsk') of arguments of function plus. ------------------------------------------ SELECT N - toUInt64(1) "Date","2019-09-15" -"DateTime","2019-09-16 19:20:10" +"DateTime('Europe/Minsk')","2019-09-16 19:20:10" Code: 43: Illegal types DateTime64(3, 'Europe/Minsk') and UInt64 of arguments of function minus. ------------------------------------------ @@ -1306,19 +1306,19 @@ Code: 43: Illegal types of arguments (UInt64, Date) of function greaterOrEquals. ------------------------------------------ SELECT N + toInt64(-1) "Date","2019-09-15" -"DateTime","2019-09-16 19:20:10" +"DateTime('Europe/Minsk')","2019-09-16 19:20:10" Code: 43: Illegal types DateTime64(3, 'Europe/Minsk') and Int64 of arguments of function plus. ------------------------------------------ SELECT toInt64(-1) + N "Date","2019-09-15" -"DateTime","2019-09-16 19:20:10" +"DateTime('Europe/Minsk')","2019-09-16 19:20:10" Code: 43: Illegal types Int64 and DateTime64(3, 'Europe/Minsk') of arguments of function plus. ------------------------------------------ SELECT N - toInt64(-1) "Date","2019-09-17" -"DateTime","2019-09-16 19:20:12" +"DateTime('Europe/Minsk')","2019-09-16 19:20:12" Code: 43: Illegal types DateTime64(3, 'Europe/Minsk') and Int64 of arguments of function minus. ------------------------------------------ From 5fc89bce7cb4feedf2c6771bb40b7a96f4c3b635 Mon Sep 17 00:00:00 2001 From: Vasily Nemkov Date: Tue, 10 Mar 2020 16:10:07 +0800 Subject: [PATCH 0050/1355] Explicitly specifying timezone when casting to avoid different results on different machines due to default timezone being used in result datatype. --- .../00921_datetime64_compatibility.python | 10 ++--- .../00921_datetime64_compatibility.reference | 40 +++++++++---------- 2 files changed, 25 insertions(+), 25 deletions(-) diff --git a/dbms/tests/queries/0_stateless/00921_datetime64_compatibility.python b/dbms/tests/queries/0_stateless/00921_datetime64_compatibility.python index 27b9f8dc820..b67bec36947 100755 --- a/dbms/tests/queries/0_stateless/00921_datetime64_compatibility.python +++ b/dbms/tests/queries/0_stateless/00921_datetime64_compatibility.python @@ -68,13 +68,13 @@ subtractHours(N, 1) subtractMinutes(N, 1) subtractSeconds(N, 1) subtractQuarters(N, 1) -CAST(N as DateTime) +CAST(N as DateTime('Europe/Minsk')) CAST(N as Date) CAST(N as UInt64) -CAST(N as DateTime64(0)) -CAST(N as DateTime64(3)) -CAST(N as DateTime64(6)) -CAST(N as DateTime64(9)) +CAST(N as DateTime64(0, 'Europe/Minsk')) +CAST(N as DateTime64(3, 'Europe/Minsk')) +CAST(N as DateTime64(6, 'Europe/Minsk')) +CAST(N as DateTime64(9, 'Europe/Minsk')) # Casting our test values to DateTime(12) will cause an overflow and hence will fail the test under UB sanitizer. # CAST(N as DateTime64(12)) # DateTime64(18) will always fail due to zero precision, but it is Ok to test here: diff --git a/dbms/tests/queries/0_stateless/00921_datetime64_compatibility.reference b/dbms/tests/queries/0_stateless/00921_datetime64_compatibility.reference index 5ca31f0bfc6..5dd0450e400 100644 --- a/dbms/tests/queries/0_stateless/00921_datetime64_compatibility.reference +++ b/dbms/tests/queries/0_stateless/00921_datetime64_compatibility.reference @@ -310,10 +310,10 @@ SELECT subtractQuarters(N, 1) "DateTime('Europe/Minsk')","2019-06-16 19:20:11" "DateTime64(3, 'Europe/Minsk')","2019-06-16 19:20:11.234" ------------------------------------------ -SELECT CAST(N as DateTime) -"DateTime","2019-09-16 00:00:00" -"DateTime","2019-09-16 19:20:11" -"DateTime","2019-09-16 19:20:11" +SELECT CAST(N as DateTime(\'Europe/Minsk\')) +"DateTime('Europe/Minsk')","2019-09-15 19:00:00" +"DateTime('Europe/Minsk')","2019-09-16 19:20:11" +"DateTime('Europe/Minsk')","2019-09-16 19:20:11" ------------------------------------------ SELECT CAST(N as Date) "Date","2019-09-16" @@ -325,25 +325,25 @@ SELECT CAST(N as UInt64) "UInt64",1568650811 "UInt64",1568650811 ------------------------------------------ -SELECT CAST(N as DateTime64(0)) -"DateTime64(0)","2019-09-16 00:00:00" -"DateTime64(0)","2019-09-16 19:20:11" -"DateTime64(0)","2019-09-16 19:20:11" +SELECT CAST(N as DateTime64(0, \'Europe/Minsk\')) +"DateTime64(0, 'Europe/Minsk')","2019-09-15 19:00:00" +"DateTime64(0, 'Europe/Minsk')","2019-09-16 19:20:11" +"DateTime64(0, 'Europe/Minsk')","2019-09-16 19:20:11" ------------------------------------------ -SELECT CAST(N as DateTime64(3)) -"DateTime64(3)","2019-09-16 00:00:00.000" -"DateTime64(3)","2019-09-16 19:20:11.000" -"DateTime64(3)","2019-09-16 19:20:11.234" +SELECT CAST(N as DateTime64(3, \'Europe/Minsk\')) +"DateTime64(3, 'Europe/Minsk')","2019-09-15 19:00:00.000" +"DateTime64(3, 'Europe/Minsk')","2019-09-16 19:20:11.000" +"DateTime64(3, 'Europe/Minsk')","2019-09-16 19:20:11.234" ------------------------------------------ -SELECT CAST(N as DateTime64(6)) -"DateTime64(6)","2019-09-16 00:00:00.000000" -"DateTime64(6)","2019-09-16 19:20:11.000000" -"DateTime64(6)","2019-09-16 19:20:11.234000" +SELECT CAST(N as DateTime64(6, \'Europe/Minsk\')) +"DateTime64(6, 'Europe/Minsk')","2019-09-15 19:00:00.000000" +"DateTime64(6, 'Europe/Minsk')","2019-09-16 19:20:11.000000" +"DateTime64(6, 'Europe/Minsk')","2019-09-16 19:20:11.234000" ------------------------------------------ -SELECT CAST(N as DateTime64(9)) -"DateTime64(9)","2019-09-16 00:00:00.000000000" -"DateTime64(9)","2019-09-16 19:20:11.000000000" -"DateTime64(9)","2019-09-16 19:20:11.234000000" +SELECT CAST(N as DateTime64(9, \'Europe/Minsk\')) +"DateTime64(9, 'Europe/Minsk')","2019-09-15 19:00:00.000000000" +"DateTime64(9, 'Europe/Minsk')","2019-09-16 19:20:11.000000000" +"DateTime64(9, 'Europe/Minsk')","2019-09-16 19:20:11.234000000" ------------------------------------------ SELECT formatDateTime(N, \'%C %d %D %e %F %H %I %j %m %M %p %R %S %T %u %V %w %y %Y %%\') "String","20 16 09/16/19 16 2019-09-16 00 12 259 09 00 AM 00:00 00 00:00:00 1 38 1 19 2019 %" From 2cb3d379131789688ec381ebaea1e96e428a7a14 Mon Sep 17 00:00:00 2001 From: Vasily Nemkov Date: Tue, 10 Mar 2020 16:17:34 +0800 Subject: [PATCH 0051/1355] Minor: Comment reworded and style fixes --- dbms/src/Functions/FunctionBinaryArithmetic.h | 7 ++++--- dbms/src/Functions/formatDateTime.cpp | 2 +- 2 files changed, 5 insertions(+), 4 deletions(-) diff --git a/dbms/src/Functions/FunctionBinaryArithmetic.h b/dbms/src/Functions/FunctionBinaryArithmetic.h index 63961cc0bb7..8314e6fca8b 100644 --- a/dbms/src/Functions/FunctionBinaryArithmetic.h +++ b/dbms/src/Functions/FunctionBinaryArithmetic.h @@ -823,11 +823,12 @@ public: type_res = std::make_shared(left.getPrecision(), left.getScale()); else if constexpr (IsDataTypeDecimal) type_res = std::make_shared(right.getPrecision(), right.getScale()); - // Special case for DateTime: binary OPS should not looze the timezone of the result type, - // but reuse timezone of DateTime argument. - // NOTE: binary plus/minus are not allowed on DateTime64, and we are not handling it here. else if constexpr (std::is_same_v) { + // Special case for DateTime: binary OPS should reuse timezone + // of DateTime argument as timezeone of result type. + // NOTE: binary plus/minus are not allowed on DateTime64, and we are not handling it here. + const TimezoneMixin * tz = nullptr; if constexpr (std::is_same_v) tz = &right; diff --git a/dbms/src/Functions/formatDateTime.cpp b/dbms/src/Functions/formatDateTime.cpp index ce6a30aac89..c7f3f62a4e3 100644 --- a/dbms/src/Functions/formatDateTime.cpp +++ b/dbms/src/Functions/formatDateTime.cpp @@ -315,7 +315,7 @@ public: const DateLUTImpl * time_zone_tmp = nullptr; if (std::is_same_v || std::is_same_v) - time_zone_tmp = &extractTimeZoneFromFunctionArguments(block, arguments, 3, 0); + time_zone_tmp = &extractTimeZoneFromFunctionArguments(block, arguments, 3, 0); else time_zone_tmp = &DateLUT::instance(); From 0df165578bf0f5b9b08040b21356c8c88a089692 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Tue, 10 Mar 2020 17:42:06 +0300 Subject: [PATCH 0052/1355] mark piece as done if it is not present --- dbms/programs/copier/ClusterCopier.cpp | 22 ++++++++++++++++++++-- 1 file changed, 20 insertions(+), 2 deletions(-) diff --git a/dbms/programs/copier/ClusterCopier.cpp b/dbms/programs/copier/ClusterCopier.cpp index 6eb0ffcfac9..45b386a7503 100644 --- a/dbms/programs/copier/ClusterCopier.cpp +++ b/dbms/programs/copier/ClusterCopier.cpp @@ -1065,6 +1065,26 @@ PartitionTaskStatus ClusterCopier::processPartitionPieceTaskImpl( throw; } + + /// Exit if current piece is absent on this shard. Also mark it as finished, because we will check + /// whether each shard have processed each partitition (and its pieces). + if (partition_piece.is_absent_piece) + { + std::cout << "current partition piece is clean?? " << is_clean << std::endl; + std::cout << "######" << "Partition " << task_partition.name + << " piece " + toString(current_piece_number) + " IS ABSENT ON CURRENT SHARD" << std::endl; + std::cout << "current_task_piece_status_path " << current_task_piece_status_path << std::endl; + String state_finished = TaskStateWithOwner::getData(TaskState::Finished, host_id); + auto res = zookeeper->tryCreate(current_task_piece_status_path, state_finished, zkutil::CreateMode::Persistent); + if (res == Coordination::ZNODEEXISTS) + LOG_DEBUG(log, "Partition " << task_partition.name << " piece " + + toString(current_piece_number) + " is absent on current replica of a shard. But other replica has already marked it as done."); + if (res == Coordination::ZOK) + LOG_DEBUG(log, "Partition " << task_partition.name << " piece " + + toString(current_piece_number) + " is absent on current replica of a shard. Will mark it as done. Other replicas will do the same."); + return PartitionTaskStatus::Finished; + } + /// Exit if task has been already processed; /// create blocking node to signal cleaning up if it is abandoned { @@ -1376,8 +1396,6 @@ PartitionTaskStatus ClusterCopier::processPartitionPieceTaskImpl( /// TODO: LOG_INFO (Piece copied and moved to destination table) - - return PartitionTaskStatus::Finished; } From f93aaf061fbb56e3e03615e6ecb9c9507fd6ad18 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Tue, 10 Mar 2020 23:04:08 +0300 Subject: [PATCH 0053/1355] probably all tests are ok --- dbms/programs/copier/ClusterCopier.cpp | 101 ++++++++++++------ dbms/programs/copier/ClusterCopier.h | 13 +++ dbms/programs/copier/ShardPartitionPiece.h | 4 +- dbms/programs/copier/TaskTableAndShard.h | 2 +- dbms/src/Interpreters/Cluster.h | 12 +++ .../Storages/StorageReplicatedMergeTree.cpp | 6 +- .../integration/test_cluster_copier/test.py | 19 ++-- 7 files changed, 112 insertions(+), 45 deletions(-) diff --git a/dbms/programs/copier/ClusterCopier.cpp b/dbms/programs/copier/ClusterCopier.cpp index 45b386a7503..f6f3d2bed55 100644 --- a/dbms/programs/copier/ClusterCopier.cpp +++ b/dbms/programs/copier/ClusterCopier.cpp @@ -674,7 +674,13 @@ bool ClusterCopier::tryDropPartitionPiece( LOG_DEBUG(log, "Execute distributed DROP PARTITION: " << query); /// Limit number of max executing replicas to 1 - UInt64 num_shards = executeQueryOnCluster(cluster_push, query, nullptr, &settings_push, PoolMode::GET_ONE, 1); + UInt64 num_shards = executeQueryOnCluster( + cluster_push, query, + nullptr, + &settings_push, + PoolMode::GET_ONE, + ClusterExecutionMode::ON_EACH_SHARD, + 1); if (num_shards < cluster_push->getShardCount()) { @@ -1065,26 +1071,6 @@ PartitionTaskStatus ClusterCopier::processPartitionPieceTaskImpl( throw; } - - /// Exit if current piece is absent on this shard. Also mark it as finished, because we will check - /// whether each shard have processed each partitition (and its pieces). - if (partition_piece.is_absent_piece) - { - std::cout << "current partition piece is clean?? " << is_clean << std::endl; - std::cout << "######" << "Partition " << task_partition.name - << " piece " + toString(current_piece_number) + " IS ABSENT ON CURRENT SHARD" << std::endl; - std::cout << "current_task_piece_status_path " << current_task_piece_status_path << std::endl; - String state_finished = TaskStateWithOwner::getData(TaskState::Finished, host_id); - auto res = zookeeper->tryCreate(current_task_piece_status_path, state_finished, zkutil::CreateMode::Persistent); - if (res == Coordination::ZNODEEXISTS) - LOG_DEBUG(log, "Partition " << task_partition.name << " piece " - + toString(current_piece_number) + " is absent on current replica of a shard. But other replica has already marked it as done."); - if (res == Coordination::ZOK) - LOG_DEBUG(log, "Partition " << task_partition.name << " piece " - + toString(current_piece_number) + " is absent on current replica of a shard. Will mark it as done. Other replicas will do the same."); - return PartitionTaskStatus::Finished; - } - /// Exit if task has been already processed; /// create blocking node to signal cleaning up if it is abandoned { @@ -1110,6 +1096,22 @@ PartitionTaskStatus ClusterCopier::processPartitionPieceTaskImpl( } } + + /// Exit if current piece is absent on this shard. Also mark it as finished, because we will check + /// whether each shard have processed each partitition (and its pieces). + if (partition_piece.is_absent_piece) + { + String state_finished = TaskStateWithOwner::getData(TaskState::Finished, host_id); + auto res = zookeeper->tryCreate(current_task_piece_status_path, state_finished, zkutil::CreateMode::Persistent); + if (res == Coordination::ZNODEEXISTS) + LOG_DEBUG(log, "Partition " << task_partition.name << " piece " + + toString(current_piece_number) + " is absent on current replica of a shard. But other replicas have already marked it as done."); + if (res == Coordination::ZOK) + LOG_DEBUG(log, "Partition " << task_partition.name << " piece " + + toString(current_piece_number) + " is absent on current replica of a shard. Will mark it as done. Other replicas will do the same."); + return PartitionTaskStatus::Finished; + } + /// Check that destination partition is empty if we are first worker /// NOTE: this check is incorrect if pull and push tables have different partition key! String clean_start_status; @@ -1119,7 +1121,7 @@ PartitionTaskStatus ClusterCopier::processPartitionPieceTaskImpl( auto checker = zkutil::EphemeralNodeHolder::create(partition_piece.getPartitionPieceCleanStartPath() + "/checker", *zookeeper, host_id); // Maybe we are the first worker - /// TODO: Why table_split_shard??? + ///TODO: Why table_split_shard??? ASTPtr query_select_ast = get_select_query(split_table_for_current_piece, "count()", /*enable_splitting*/ true); UInt64 count; @@ -1350,20 +1352,20 @@ PartitionTaskStatus ClusterCopier::processPartitionPieceTaskImpl( LOG_DEBUG(log, "Executing ALTER query: " << query_alter_ast_string); -// query_alter_ast_string += " INSERT INTO " + getQuotedTable(original_table) + -// " SELECT * FROM " + getQuotedTable(helping_table); -// -// query_alter_ast_string += " WHERE (" + queryToString(task_table.engine_push_partition_key_ast) + " = (" + task_partition.name + " AS partition_key))"; -// -// LOG_DEBUG(log, "Executing ALTER query: " << query_alter_ast_string); - try { ///FIXME: We have to be sure that every node in cluster executed this query - UInt64 num_shards = executeQueryOnCluster(task_table.cluster_push, query_alter_ast_string, nullptr, &task_cluster->settings_push, PoolMode::GET_MANY); + UInt64 num_nodes = executeQueryOnCluster( + task_table.cluster_push, + query_alter_ast_string, + nullptr, + &task_cluster->settings_push, + PoolMode::GET_MANY, + ClusterExecutionMode::ON_EACH_NODE); - assert(num_shards > 0); - LOG_INFO(log, "Number of shard that executed ALTER query successfully : " << toString(num_shards)); + // TODO: Throw an exception if num nodes is not equal to original number of nodes + + LOG_INFO(log, "Number of shard that executed ALTER query successfully : " << toString(num_nodes)); } catch (...) { @@ -1623,6 +1625,7 @@ UInt64 ClusterCopier::executeQueryOnCluster( const ASTPtr & query_ast_, const Settings * settings, PoolMode pool_mode, + ClusterExecutionMode execution_mode, UInt64 max_successful_executions_per_shard) const { auto num_shards = cluster->getShardsInfo().size(); @@ -1637,6 +1640,11 @@ UInt64 ClusterCopier::executeQueryOnCluster( else query_ast = query_ast_; + /// We will have to execute query on each replica of a shard. + if (execution_mode == ClusterExecutionMode::ON_EACH_NODE) + max_successful_executions_per_shard = 0; + + std::atomic origin_replicas_number; /// We need to execute query on one replica at least auto do_for_shard = [&] (UInt64 shard_index) @@ -1652,6 +1660,13 @@ UInt64 ClusterCopier::executeQueryOnCluster( }; UInt64 num_replicas = cluster->getShardsAddresses().at(shard_index).size(); + + for (size_t i = 0; i < num_replicas; ++i) + { + std::cout << "host_name " << cluster->getShardsAddresses().at(shard_index)[i].host_name + << " port " << cluster->getShardsAddresses().at(shard_index)[i].port << std::endl; + } + origin_replicas_number += num_replicas; UInt64 num_local_replicas = shard.getLocalNodeCount(); UInt64 num_remote_replicas = num_replicas - num_local_replicas; @@ -1706,10 +1721,26 @@ UInt64 ClusterCopier::executeQueryOnCluster( thread_pool.wait(); } - UInt64 successful_shards = 0; + UInt64 successful_nodes = 0; for (UInt64 num_replicas : per_shard_num_successful_replicas) - successful_shards += (num_replicas > 0); + { + if (execution_mode == ClusterExecutionMode::ON_EACH_NODE) + successful_nodes += num_replicas; + else + /// Count only successful shards + successful_nodes += (num_replicas > 0); + } - return successful_shards; + if (execution_mode == ClusterExecutionMode::ON_EACH_NODE && successful_nodes != origin_replicas_number - 1) + { + LOG_INFO(log, "There was an error while executing ALTER on each node. Query was executed on " + << toString(successful_nodes) << " nodes. But had to be executed on " << toString(origin_replicas_number.load())); + + std::cout << "successful_nodes " << successful_nodes << " origin_replicas_number " << origin_replicas_number << std::endl; + throw Exception("There was an error while executing ALTER on each node.", ErrorCodes::LOGICAL_ERROR); + } + + + return successful_nodes; } } diff --git a/dbms/programs/copier/ClusterCopier.h b/dbms/programs/copier/ClusterCopier.h index a5587045299..4bbf9e58286 100644 --- a/dbms/programs/copier/ClusterCopier.h +++ b/dbms/programs/copier/ClusterCopier.h @@ -149,6 +149,18 @@ protected: bool checkPresentPartitionPiecesOnCurrentShard(const ConnectionTimeouts & timeouts, TaskShard & task_shard, const String & partition_quoted_name, size_t current_piece_number); + /* + * This class is used in executeQueryOnCluster function + * You can execute query on each shard (no sense it is executed on each replica of a shard or not) + * or you can execute query on each replica on each shard. + * First mode is useful for INSERTS queries. + * */ + enum ClusterExecutionMode + { + ON_EACH_SHARD, + ON_EACH_NODE + }; + /** Executes simple query (without output streams, for example DDL queries) on each shard of the cluster * Returns number of shards for which at least one replica executed query successfully */ @@ -158,6 +170,7 @@ protected: const ASTPtr & query_ast_ = nullptr, const Settings * settings = nullptr, PoolMode pool_mode = PoolMode::GET_ALL, + ClusterExecutionMode execution_mode = ClusterExecutionMode::ON_EACH_SHARD, UInt64 max_successful_executions_per_shard = 0) const; private: diff --git a/dbms/programs/copier/ShardPartitionPiece.h b/dbms/programs/copier/ShardPartitionPiece.h index 04f7f458a00..88f1ceddb0d 100644 --- a/dbms/programs/copier/ShardPartitionPiece.h +++ b/dbms/programs/copier/ShardPartitionPiece.h @@ -6,8 +6,8 @@ namespace DB { struct ShardPartitionPiece { - ShardPartitionPiece(ShardPartition &parent, size_t current_piece_number_, bool is_absent_piece_) - : is_absent_piece(is_absent_piece_), current_piece_number(current_piece_number_), + ShardPartitionPiece(ShardPartition &parent, size_t current_piece_number_, bool is_present_piece_) + : is_absent_piece(!is_present_piece_), current_piece_number(current_piece_number_), shard_partition(parent) {} [[maybe_unused]] String getPartitionPiecePath() const; diff --git a/dbms/programs/copier/TaskTableAndShard.h b/dbms/programs/copier/TaskTableAndShard.h index c879152d6c9..91e14598e81 100644 --- a/dbms/programs/copier/TaskTableAndShard.h +++ b/dbms/programs/copier/TaskTableAndShard.h @@ -241,7 +241,7 @@ inline TaskTable::TaskTable(TaskCluster & parent, const Poco::Util::AbstractConf name_in_config = table_key; - number_of_splits = config.getUInt64(table_prefix + "number_of_splits", 10); + number_of_splits = config.getUInt64(table_prefix + "number_of_splits", 2); cluster_pull_name = config.getString(table_prefix + "cluster_pull"); cluster_push_name = config.getString(table_prefix + "cluster_push"); diff --git a/dbms/src/Interpreters/Cluster.h b/dbms/src/Interpreters/Cluster.h index eba046994eb..b6c8b054495 100644 --- a/dbms/src/Interpreters/Cluster.h +++ b/dbms/src/Interpreters/Cluster.h @@ -151,6 +151,18 @@ public: /// The number of all shards. size_t getShardCount() const { return shards_info.size(); } +// /// The number of all nodes (all replicas of each shard). +// size_t getNodesCount() const +// { +// size_t nodes_count = 0; +// std::cout << "addresses_with_failover.size() " << addresses_with_failover.size() << std::endl; +// std::for_each(addresses_with_failover.begin(), addresses_with_failover.end(), [&] (const Addresses & addresses) +// { +// nodes_count += addresses.size(); +// }); +// return nodes_count; +// } + /// Get a subcluster consisting of one shard - index by count (from 0) of the shard of this cluster. std::unique_ptr getClusterWithSingleShard(size_t index) const; diff --git a/dbms/src/Storages/StorageReplicatedMergeTree.cpp b/dbms/src/Storages/StorageReplicatedMergeTree.cpp index 9880df6f116..99cc73b56a6 100644 --- a/dbms/src/Storages/StorageReplicatedMergeTree.cpp +++ b/dbms/src/Storages/StorageReplicatedMergeTree.cpp @@ -4888,7 +4888,7 @@ void StorageReplicatedMergeTree::replacePartitionFrom(const StoragePtr & source_ String hash_hex = src_part->checksums.getTotalChecksumHex(); - LOG_INFO(log, "Trying to attach " << src_part->name << "with hash_hex " << hash_hex); + LOG_INFO(log, "Trying to attach " << src_part->name << " with hash_hex " << hash_hex); String block_id_path = replace ? "" : (zookeeper_path + "/blocks/" + partition_id + "_replace_from_" + hash_hex); @@ -4910,6 +4910,10 @@ void StorageReplicatedMergeTree::replacePartitionFrom(const StoragePtr & source_ part_checksums.emplace_back(hash_hex); } + /// We have nothing to do - return + if (src_parts.empty()) + return; + ReplicatedMergeTreeLogEntryData entry; { auto src_table_id = src_data.getStorageID(); diff --git a/dbms/tests/integration/test_cluster_copier/test.py b/dbms/tests/integration/test_cluster_copier/test.py index 4677c49eb43..b363c02d71d 100644 --- a/dbms/tests/integration/test_cluster_copier/test.py +++ b/dbms/tests/integration/test_cluster_copier/test.py @@ -68,6 +68,13 @@ def started_cluster(): pass cluster.shutdown() +def print_destination_cluster(task): + for anime in ['s1_0_0', 's1_0_1', 's1_1_0']: + a = task.cluster.instances[anime].query("SELECT count() FROM hits_piece_0") + b = task.cluster.instances[anime].query("SELECT count() FROM hits_piece_1") + c = task.cluster.instances[anime].query("SELECT count() FROM hits") + print(anime, a, b, int(a) + int(b), c) + print(task.cluster.instances[anime].query("select partition, name, database, table, hash_of_all_files, hash_of_uncompressed_files, uncompressed_hash_of_compressed_files from system.parts where table like '%hits%' format TSV")) class Task1: @@ -91,13 +98,8 @@ class Task1: def check(self): + print_destination_cluster(self) assert TSV(self.cluster.instances['s0_0_0'].query("SELECT count() FROM hits_all")) == TSV("1002\n") - for anime in ['s1_0_0', 's1_0_1', 's1_1_0']: - a = self.cluster.instances[anime].query("SELECT count() FROM hits_piece_0") - b = self.cluster.instances[anime].query("SELECT count() FROM hits_piece_1") - c = self.cluster.instances[anime].query("SELECT count() FROM hits") - print(anime, a, b, int(a) + int(b), c) - print(self.cluster.instances[anime].query("select partition, name, database, table, hash_of_all_files, hash_of_uncompressed_files, uncompressed_hash_of_compressed_files from system.parts where table like '%hits%' format TSV")) assert TSV(self.cluster.instances['s1_0_0'].query("SELECT DISTINCT d % 2 FROM hits")) == TSV("1\n") assert TSV(self.cluster.instances['s1_1_0'].query("SELECT DISTINCT d % 2 FROM hits")) == TSV("0\n") @@ -136,6 +138,7 @@ class Task2: def check(self): + #print_destination_cluster(self) assert TSV(self.cluster.instances['s0_0_0'].query("SELECT count() FROM cluster(cluster0, default, a)")) == TSV("85\n") assert TSV(self.cluster.instances['s1_0_0'].query("SELECT count(), uniqExact(date) FROM cluster(cluster1, default, b)")) == TSV("85\t85\n") @@ -171,6 +174,8 @@ class Task_test_block_size: def check(self): + #print_destination_cluster(self) + assert TSV(self.cluster.instances['s1_0_0'].query("SELECT count() FROM cluster(cluster1, default, test_block_size)")) == TSV("{}\n".format(self.rows)) instance = cluster.instances['s0_0_0'] @@ -194,6 +199,7 @@ class Task_no_index: def check(self): + #print_destination_cluster(self) assert TSV(self.cluster.instances['s1_1_0'].query("SELECT Year FROM ontime22")) == TSV("2017\n") instance = cluster.instances['s0_0_0'] instance.query("DROP TABLE ontime") @@ -217,6 +223,7 @@ class Task_no_arg: def check(self): + #print_destination_cluster(self) assert TSV(self.cluster.instances['s1_1_0'].query("SELECT date FROM copier_test1_1")) == TSV("2016-01-01\n") instance = cluster.instances['s0_0_0'] instance.query("DROP TABLE copier_test1") From acf09b6630489cf34f9a92330735fafa942a2846 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Wed, 11 Mar 2020 16:01:17 +0300 Subject: [PATCH 0054/1355] fix build --- dbms/programs/copier/ClusterCopier.cpp | 3 --- 1 file changed, 3 deletions(-) diff --git a/dbms/programs/copier/ClusterCopier.cpp b/dbms/programs/copier/ClusterCopier.cpp index f6f3d2bed55..27f2bb53e09 100644 --- a/dbms/programs/copier/ClusterCopier.cpp +++ b/dbms/programs/copier/ClusterCopier.cpp @@ -914,7 +914,6 @@ PartitionTaskStatus ClusterCopier::iterateThroughAllPiecesInPartition(const Conn bool was_failed_pieces = false; bool was_active_pieces = false; - bool was_error = false; for (size_t piece_number = 0; piece_number < total_number_of_pieces; piece_number++) { @@ -926,8 +925,6 @@ PartitionTaskStatus ClusterCopier::iterateThroughAllPiecesInPartition(const Conn if (res == PartitionTaskStatus::Finished) break; - was_error = true; - /// Skip if the task is being processed by someone if (res == PartitionTaskStatus::Active) break; From 36d4f40d586cba9ecf9026043f2c6ac22229c4b8 Mon Sep 17 00:00:00 2001 From: Guillaume Tassery Date: Wed, 11 Mar 2020 16:30:02 +0100 Subject: [PATCH 0055/1355] add dictionary status on prometheus server --- .../server/PrometheusMetricsWriter.cpp | 36 +++++++++++- .../programs/server/PrometheusMetricsWriter.h | 2 + dbms/src/Common/CurrentStatusInfo.cpp | 58 +++++++++++++++++++ dbms/src/Common/CurrentStatusInfo.h | 42 ++++++++++++++ dbms/src/Interpreters/ExternalLoader.cpp | 8 +++ 5 files changed, 145 insertions(+), 1 deletion(-) create mode 100644 dbms/src/Common/CurrentStatusInfo.cpp create mode 100644 dbms/src/Common/CurrentStatusInfo.h diff --git a/dbms/programs/server/PrometheusMetricsWriter.cpp b/dbms/programs/server/PrometheusMetricsWriter.cpp index 11782710104..17c05d14936 100644 --- a/dbms/programs/server/PrometheusMetricsWriter.cpp +++ b/dbms/programs/server/PrometheusMetricsWriter.cpp @@ -3,6 +3,7 @@ #include #include +#include namespace { @@ -40,6 +41,7 @@ PrometheusMetricsWriter::PrometheusMetricsWriter( , send_events(config.getBool(config_name + ".events", true)) , send_metrics(config.getBool(config_name + ".metrics", true)) , send_asynchronous_metrics(config.getBool(config_name + ".asynchronous_metrics", true)) + , send_status_info(config.getBool(config_name + ".status_info", true)) { } @@ -86,7 +88,7 @@ void PrometheusMetricsWriter::write(WriteBuffer & wb) const auto async_metrics_values = async_metrics.getValues(); for (const auto & name_value : async_metrics_values) { - std::string key{asynchronous_metrics_prefix + name_value.first}; + std::string key{current_status_prefix + name_value.first}; replaceInvalidChars(key); auto value = name_value.second; @@ -96,6 +98,38 @@ void PrometheusMetricsWriter::write(WriteBuffer & wb) const writeOutLine(wb, key, value); } } + + if (send_status_info) + { + for (size_t i = 0, end = CurrentStatusInfo::end(); i < end; ++i) + { + std::string metric_name{CurrentStatusInfo::getName(static_cast(i))}; + std::string metric_doc{CurrentStatusInfo::getDocumentation(static_cast(i))}; + + replaceInvalidChars(metric_name); + std::string key{current_status_prefix + metric_name}; + + writeOutLine(wb, "# HELP", key, metric_doc); + writeOutLine(wb, "# TYPE", key, "gauge"); + + for (const auto & value: CurrentStatusInfo::values[i]) + { + for (const auto & enum_value: CurrentStatusInfo::getAllPossibleValues(static_cast(i))) + { + DB::writeText(key, wb); + DB::writeChar('{', wb); + DB::writeText(key, wb); + DB::writeText("=\"", wb); + DB::writeText(enum_value.first, wb); + DB::writeText("\",name=\"", wb); + DB::writeText(value.first, wb); + DB::writeText("\"} ", wb); + DB::writeText(value.second == enum_value.first, wb); + DB::writeChar('\n', wb); + } + } + } + } } } diff --git a/dbms/programs/server/PrometheusMetricsWriter.h b/dbms/programs/server/PrometheusMetricsWriter.h index ba1f0cde61b..4422ced625e 100644 --- a/dbms/programs/server/PrometheusMetricsWriter.h +++ b/dbms/programs/server/PrometheusMetricsWriter.h @@ -27,10 +27,12 @@ private: const bool send_events; const bool send_metrics; const bool send_asynchronous_metrics; + const bool send_status_info; static inline constexpr auto profile_events_prefix = "ClickHouseProfileEvents_"; static inline constexpr auto current_metrics_prefix = "ClickHouseMetrics_"; static inline constexpr auto asynchronous_metrics_prefix = "ClickHouseAsyncMetrics_"; + static inline constexpr auto current_status_prefix = "ClickHouseStatusInfo_"; }; } diff --git a/dbms/src/Common/CurrentStatusInfo.cpp b/dbms/src/Common/CurrentStatusInfo.cpp new file mode 100644 index 00000000000..693b21080fd --- /dev/null +++ b/dbms/src/Common/CurrentStatusInfo.cpp @@ -0,0 +1,58 @@ +#include +#include + +/// Available status. Add something here as you wish. +#define APPLY_FOR_STATUS(M) \ + M(DictionaryStatus, "Dictionary Status.", DB::ExternalLoader::getStatusEnumAllPossibleValues()) \ + + +namespace CurrentStatusInfo +{ + #define M(NAME, DOCUMENTATION, ENUM) extern const Metric NAME = __COUNTER__; + APPLY_FOR_STATUS(M) + #undef M + constexpr Metric END = __COUNTER__; + + std::mutex locks[END] {}; + std::unordered_map values[END] {}; + + + const char * getName(Metric event) + { + static const char * strings[] = + { + #define M(NAME, DOCUMENTATION, ENUM) #NAME, + APPLY_FOR_STATUS(M) + #undef M + }; + + return strings[event]; + } + + const char * getDocumentation(Metric event) + { + static const char * strings[] = + { + #define M(NAME, DOCUMENTATION, ENUM) DOCUMENTATION, + APPLY_FOR_STATUS(M) + #undef M + }; + + return strings[event]; + } + + const std::vector> & getAllPossibleValues(Metric event) + { + static const std::vector> enum_values [] = + { + #define M(NAME, DOCUMENTATION, ENUM) ENUM, + APPLY_FOR_STATUS(M) + #undef M + }; + return enum_values[event]; + } + + Metric end() { return END; } +} + +#undef APPLY_FOR_STATUS diff --git a/dbms/src/Common/CurrentStatusInfo.h b/dbms/src/Common/CurrentStatusInfo.h new file mode 100644 index 00000000000..646d8fce842 --- /dev/null +++ b/dbms/src/Common/CurrentStatusInfo.h @@ -0,0 +1,42 @@ +#pragma once + +#include +#include +#include +#include +#include +#include +#include + + +namespace CurrentStatusInfo +{ + /// Metric identifier (index in array). + using Metric = size_t; + using Key = std::string; + + /// Get name of metric by identifier. Returns statically allocated string. + const char * getName(Metric event); + /// Get text description of metric by identifier. Returns statically allocated string. + const char * getDocumentation(Metric event); + const std::vector> & getAllPossibleValues(Metric event); + + extern std::unordered_map values[]; + extern std::mutex locks[]; + + /// Get index just after last metric identifier. + Metric end(); + + /// Set status of specified. + inline void set(Metric metric, Key key, String value) + { + std::lock_guard lock(locks[metric]); + values[metric][key] = value; + } + + inline void unset(Metric metric, Key key) + { + std::lock_guard lock(locks[metric]); + values[metric].erase(key); + } +} diff --git a/dbms/src/Interpreters/ExternalLoader.cpp b/dbms/src/Interpreters/ExternalLoader.cpp index 5a012bfb5a2..6488bf8f1c0 100644 --- a/dbms/src/Interpreters/ExternalLoader.cpp +++ b/dbms/src/Interpreters/ExternalLoader.cpp @@ -8,6 +8,7 @@ #include #include #include +#include #include #include #include @@ -15,6 +16,12 @@ #include +namespace CurrentStatusInfo +{ + extern const Metric DictionaryStatus; +} + + namespace DB { namespace ErrorCodes @@ -1035,6 +1042,7 @@ private: it->second.detach(); loading_threads.erase(it); } + CurrentStatusInfo::set(CurrentStatusInfo::DictionaryStatus, name, toString(info->status())); } /// Calculate next update time for loaded_object. Can be called without mutex locking, From 52251dceb5b2eeaef7bd3aff896c77d6aa5bfdc4 Mon Sep 17 00:00:00 2001 From: Guillaume Tassery Date: Wed, 11 Mar 2020 16:34:12 +0100 Subject: [PATCH 0056/1355] revert change --- dbms/programs/server/PrometheusMetricsWriter.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/programs/server/PrometheusMetricsWriter.cpp b/dbms/programs/server/PrometheusMetricsWriter.cpp index 17c05d14936..473655a2bc9 100644 --- a/dbms/programs/server/PrometheusMetricsWriter.cpp +++ b/dbms/programs/server/PrometheusMetricsWriter.cpp @@ -88,7 +88,7 @@ void PrometheusMetricsWriter::write(WriteBuffer & wb) const auto async_metrics_values = async_metrics.getValues(); for (const auto & name_value : async_metrics_values) { - std::string key{current_status_prefix + name_value.first}; + std::string key{asynchronous_metrics_prefix + name_value.first}; replaceInvalidChars(key); auto value = name_value.second; From 6baf671017208e1adbf6132fa401d3afdfb5b3ce Mon Sep 17 00:00:00 2001 From: Guillaume Tassery Date: Wed, 11 Mar 2020 17:20:30 +0100 Subject: [PATCH 0057/1355] optimization and factoring --- dbms/programs/server/PrometheusMetricsWriter.cpp | 2 +- dbms/src/Interpreters/ExternalLoader.cpp | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/dbms/programs/server/PrometheusMetricsWriter.cpp b/dbms/programs/server/PrometheusMetricsWriter.cpp index 473655a2bc9..6f9252e5b6c 100644 --- a/dbms/programs/server/PrometheusMetricsWriter.cpp +++ b/dbms/programs/server/PrometheusMetricsWriter.cpp @@ -124,7 +124,7 @@ void PrometheusMetricsWriter::write(WriteBuffer & wb) const DB::writeText("\",name=\"", wb); DB::writeText(value.first, wb); DB::writeText("\"} ", wb); - DB::writeText(value.second == enum_value.first, wb); + DB::writeText(value.second == enum_value.second, wb); DB::writeChar('\n', wb); } } diff --git a/dbms/src/Interpreters/ExternalLoader.cpp b/dbms/src/Interpreters/ExternalLoader.cpp index 6488bf8f1c0..a07db4ef08d 100644 --- a/dbms/src/Interpreters/ExternalLoader.cpp +++ b/dbms/src/Interpreters/ExternalLoader.cpp @@ -1042,7 +1042,7 @@ private: it->second.detach(); loading_threads.erase(it); } - CurrentStatusInfo::set(CurrentStatusInfo::DictionaryStatus, name, toString(info->status())); + CurrentStatusInfo::set(CurrentStatusInfo::DictionaryStatus, name, static_cast(info->status())); } /// Calculate next update time for loaded_object. Can be called without mutex locking, From a8611da113107409c13a3def9a3c882678b05b9b Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Wed, 11 Mar 2020 22:55:27 +0300 Subject: [PATCH 0058/1355] maybe better --- dbms/programs/copier/ClusterCopier.cpp | 40 +++++++++++-------- dbms/programs/copier/ClusterCopier.h | 1 + dbms/src/Interpreters/Cluster.cpp | 4 ++ .../Storages/StorageReplicatedMergeTree.cpp | 2 +- .../integration/test_cluster_copier/test.py | 35 ++++++++++++---- 5 files changed, 58 insertions(+), 24 deletions(-) diff --git a/dbms/programs/copier/ClusterCopier.cpp b/dbms/programs/copier/ClusterCopier.cpp index 27f2bb53e09..d7a9a7922dc 100644 --- a/dbms/programs/copier/ClusterCopier.cpp +++ b/dbms/programs/copier/ClusterCopier.cpp @@ -43,6 +43,13 @@ void ClusterCopier::init() for (auto & task_table : task_cluster->table_tasks) { task_table.cluster_pull = context.getCluster(task_table.cluster_pull_name); + auto pull_shards_info = task_table.cluster_pull->getShardsInfo(); + for (auto & shard_info : pull_shards_info) + { + std::cout << "current_shard " << toString(shard_info.shard_num) << "has remote connections " + << toString(shard_info.hasRemoteConnections()) << std::endl; + } + std::cout << "CLUSTER PULL " << std::endl; task_table.cluster_push = context.getCluster(task_table.cluster_push_name); task_table.initShards(task_cluster->random_engine); } @@ -1212,7 +1219,7 @@ PartitionTaskStatus ClusterCopier::processPartitionPieceTaskImpl( } // Select all fields - ASTPtr query_select_ast = get_select_query(task_shard.table_read_shard, "*", /*enable_splitting*/ inject_fault ? "1" : ""); + ASTPtr query_select_ast = get_select_query(task_shard.table_read_shard, "*", /*enable_splitting*/ true, inject_fault ? "1" : ""); LOG_DEBUG(log, "Executing SELECT query and pull from " << task_shard.getDescription() << " : " << queryToString(query_select_ast)); @@ -1345,23 +1352,27 @@ PartitionTaskStatus ClusterCopier::processPartitionPieceTaskImpl( query_alter_ast_string += " ALTER TABLE " + getQuotedTable(original_table) + " ATTACH PARTITION " + task_partition.name + - " FROM " + getQuotedTable(helping_table); + " FROM " + getQuotedTable(helping_table) + + " SETTINGS replication_alter_partitions_sync=2;"; LOG_DEBUG(log, "Executing ALTER query: " << query_alter_ast_string); try { - ///FIXME: We have to be sure that every node in cluster executed this query - UInt64 num_nodes = executeQueryOnCluster( - task_table.cluster_push, - query_alter_ast_string, - nullptr, - &task_cluster->settings_push, - PoolMode::GET_MANY, - ClusterExecutionMode::ON_EACH_NODE); - - // TODO: Throw an exception if num nodes is not equal to original number of nodes + size_t num_nodes = 0; + for (UInt64 try_num = 0; try_num < max_shard_partition_piece_tries_for_alter; ++try_num) + { + ///FIXME: We have to be sure that every node in cluster executed this query + UInt64 current_num_nodes = executeQueryOnCluster( + task_table.cluster_push, + query_alter_ast_string, + nullptr, + &task_cluster->settings_push, + PoolMode::GET_MANY, + ClusterExecutionMode::ON_EACH_NODE); + num_nodes = std::max(current_num_nodes, num_nodes); + } LOG_INFO(log, "Number of shard that executed ALTER query successfully : " << toString(num_nodes)); } catch (...) @@ -1728,13 +1739,10 @@ UInt64 ClusterCopier::executeQueryOnCluster( successful_nodes += (num_replicas > 0); } - if (execution_mode == ClusterExecutionMode::ON_EACH_NODE && successful_nodes != origin_replicas_number - 1) + if (execution_mode == ClusterExecutionMode::ON_EACH_NODE && successful_nodes != origin_replicas_number) { LOG_INFO(log, "There was an error while executing ALTER on each node. Query was executed on " << toString(successful_nodes) << " nodes. But had to be executed on " << toString(origin_replicas_number.load())); - - std::cout << "successful_nodes " << successful_nodes << " origin_replicas_number " << origin_replicas_number << std::endl; - throw Exception("There was an error while executing ALTER on each node.", ErrorCodes::LOGICAL_ERROR); } diff --git a/dbms/programs/copier/ClusterCopier.h b/dbms/programs/copier/ClusterCopier.h index 4bbf9e58286..bed6352e129 100644 --- a/dbms/programs/copier/ClusterCopier.h +++ b/dbms/programs/copier/ClusterCopier.h @@ -116,6 +116,7 @@ protected: static constexpr UInt64 max_table_tries = 1000; static constexpr UInt64 max_shard_partition_tries = 600; + static constexpr UInt64 max_shard_partition_piece_tries_for_alter = 5; bool tryProcessTable(const ConnectionTimeouts & timeouts, TaskTable & task_table); diff --git a/dbms/src/Interpreters/Cluster.cpp b/dbms/src/Interpreters/Cluster.cpp index 3261c0caf59..446b258d30f 100644 --- a/dbms/src/Interpreters/Cluster.cpp +++ b/dbms/src/Interpreters/Cluster.cpp @@ -75,6 +75,8 @@ bool Cluster::Address::isLocal(UInt16 clickhouse_port) const Cluster::Address::Address(const Poco::Util::AbstractConfiguration & config, const String & config_prefix, UInt32 shard_index_, UInt32 replica_index_) : shard_index(shard_index_), replica_index(replica_index_) { + + std::cout << "FROM Address constructor " << " shard index " << shard_index_ << " replica index " << replica_index_ << std::endl; host_name = config.getString(config_prefix + ".host"); port = static_cast(config.getInt(config_prefix + ".port")); if (config.has(config_prefix + ".user")) @@ -327,6 +329,8 @@ Cluster::Cluster(const Poco::Util::AbstractConfiguration & config, const Setting if (startsWith(replica_key, "replica")) { replica_addresses.emplace_back(config, partial_prefix + replica_key, current_shard_num, current_replica_num); + + std::cout << "replica num " << current_replica_num << " is a replica of shard number " << current_shard_num << std::endl; ++current_replica_num; if (!replica_addresses.back().is_local) diff --git a/dbms/src/Storages/StorageReplicatedMergeTree.cpp b/dbms/src/Storages/StorageReplicatedMergeTree.cpp index 99cc73b56a6..244dca2a044 100644 --- a/dbms/src/Storages/StorageReplicatedMergeTree.cpp +++ b/dbms/src/Storages/StorageReplicatedMergeTree.cpp @@ -4828,7 +4828,7 @@ void StorageReplicatedMergeTree::clearBlocksInPartition( void StorageReplicatedMergeTree::replacePartitionFrom(const StoragePtr & source_table, const ASTPtr & partition, bool replace, const Context & context) { - auto lock1 = lockStructureForShare(false, context.getCurrentQueryId()); + auto lock1 = lockStructureForShare(true, context.getCurrentQueryId()); auto lock2 = source_table->lockStructureForShare(false, context.getCurrentQueryId()); Stopwatch watch; diff --git a/dbms/tests/integration/test_cluster_copier/test.py b/dbms/tests/integration/test_cluster_copier/test.py index b363c02d71d..7af58c66907 100644 --- a/dbms/tests/integration/test_cluster_copier/test.py +++ b/dbms/tests/integration/test_cluster_copier/test.py @@ -68,13 +68,32 @@ def started_cluster(): pass cluster.shutdown() -def print_destination_cluster(task): +def print_destination_cluster(task, pattern): for anime in ['s1_0_0', 's1_0_1', 's1_1_0']: a = task.cluster.instances[anime].query("SELECT count() FROM hits_piece_0") b = task.cluster.instances[anime].query("SELECT count() FROM hits_piece_1") c = task.cluster.instances[anime].query("SELECT count() FROM hits") print(anime, a, b, int(a) + int(b), c) - print(task.cluster.instances[anime].query("select partition, name, database, table, hash_of_all_files, hash_of_uncompressed_files, uncompressed_hash_of_compressed_files from system.parts where table like '%hits%' format TSV")) + print(task.cluster.instances[anime].query("select partition, " + "name, database, table, " + "rows, hash_of_all_files, " + "hash_of_uncompressed_files, " + "uncompressed_hash_of_compressed_files " + "from system.parts " + "where database='default'" + "format PrettyCompact".format(pattern))) + + +def print_source_cluster(task, pattern): + for anime in ['s0_0_0', 's0_0_1', 's0_1_0']: + print(task.cluster.instances[anime].query("select partition, " + "name, database, table, " + "rows, min_date, min_time, hash_of_all_files, " + "hash_of_uncompressed_files, " + "uncompressed_hash_of_compressed_files " + "from system.parts " + "where table like '%{}%' " + "format PrettyCompact".format(pattern))) class Task1: @@ -98,7 +117,7 @@ class Task1: def check(self): - print_destination_cluster(self) + print_destination_cluster(self, "hits") assert TSV(self.cluster.instances['s0_0_0'].query("SELECT count() FROM hits_all")) == TSV("1002\n") assert TSV(self.cluster.instances['s1_0_0'].query("SELECT DISTINCT d % 2 FROM hits")) == TSV("1\n") @@ -136,9 +155,11 @@ class Task2: instance.query("INSERT INTO a_all SELECT toDate(17581 + number) AS date, number AS d FROM system.numbers LIMIT 85", settings={"insert_distributed_sync": 1}) + print_source_cluster(self, "a") + def check(self): - #print_destination_cluster(self) + print_destination_cluster(self, "a") assert TSV(self.cluster.instances['s0_0_0'].query("SELECT count() FROM cluster(cluster0, default, a)")) == TSV("85\n") assert TSV(self.cluster.instances['s1_0_0'].query("SELECT count(), uniqExact(date) FROM cluster(cluster1, default, b)")) == TSV("85\t85\n") @@ -174,7 +195,7 @@ class Task_test_block_size: def check(self): - #print_destination_cluster(self) + print_destination_cluster(self, "test_block_size") assert TSV(self.cluster.instances['s1_0_0'].query("SELECT count() FROM cluster(cluster1, default, test_block_size)")) == TSV("{}\n".format(self.rows)) @@ -199,7 +220,7 @@ class Task_no_index: def check(self): - #print_destination_cluster(self) + print_destination_cluster(self, "ontime") assert TSV(self.cluster.instances['s1_1_0'].query("SELECT Year FROM ontime22")) == TSV("2017\n") instance = cluster.instances['s0_0_0'] instance.query("DROP TABLE ontime") @@ -223,7 +244,7 @@ class Task_no_arg: def check(self): - #print_destination_cluster(self) + print_destination_cluster(self, "copier_test1") assert TSV(self.cluster.instances['s1_1_0'].query("SELECT date FROM copier_test1_1")) == TSV("2016-01-01\n") instance = cluster.instances['s0_0_0'] instance.query("DROP TABLE copier_test1") From c2033d2f0183fedc55e647dcd05eb728e16d7b0e Mon Sep 17 00:00:00 2001 From: Avogar Date: Thu, 12 Mar 2020 14:51:59 +0300 Subject: [PATCH 0059/1355] Fix build errors. --- dbms/src/Processors/Formats/Impl/RegexpRowInputFormat.cpp | 2 -- 1 file changed, 2 deletions(-) diff --git a/dbms/src/Processors/Formats/Impl/RegexpRowInputFormat.cpp b/dbms/src/Processors/Formats/Impl/RegexpRowInputFormat.cpp index 90d1cf99b2e..f20ae2b6cfa 100644 --- a/dbms/src/Processors/Formats/Impl/RegexpRowInputFormat.cpp +++ b/dbms/src/Processors/Formats/Impl/RegexpRowInputFormat.cpp @@ -80,8 +80,6 @@ bool RegexpRowInputFormat::readField(size_t index, MutableColumns & columns) else type->deserializeAsTextJSON(*columns[index], field_buf, format_settings); break; - default: - __builtin_unreachable(); } } catch (Exception & e) From 88a5dd3049cf6999f193205690b7d0c4e845ec1d Mon Sep 17 00:00:00 2001 From: Guillaume Tassery Date: Thu, 12 Mar 2020 13:09:15 +0100 Subject: [PATCH 0060/1355] cosmetic --- dbms/src/Common/CurrentStatusInfo.cpp | 2 +- dbms/src/Common/CurrentStatusInfo.h | 9 ++------- dbms/src/Databases/DatabaseWithDictionaries.cpp | 9 +++++++++ 3 files changed, 12 insertions(+), 8 deletions(-) diff --git a/dbms/src/Common/CurrentStatusInfo.cpp b/dbms/src/Common/CurrentStatusInfo.cpp index 693b21080fd..430c34cc98e 100644 --- a/dbms/src/Common/CurrentStatusInfo.cpp +++ b/dbms/src/Common/CurrentStatusInfo.cpp @@ -14,7 +14,7 @@ namespace CurrentStatusInfo constexpr Metric END = __COUNTER__; std::mutex locks[END] {}; - std::unordered_map values[END] {}; + std::unordered_map values[END] {}; const char * getName(Metric event) diff --git a/dbms/src/Common/CurrentStatusInfo.h b/dbms/src/Common/CurrentStatusInfo.h index 646d8fce842..b2c8827c2c9 100644 --- a/dbms/src/Common/CurrentStatusInfo.h +++ b/dbms/src/Common/CurrentStatusInfo.h @@ -11,24 +11,19 @@ namespace CurrentStatusInfo { - /// Metric identifier (index in array). using Metric = size_t; using Key = std::string; - /// Get name of metric by identifier. Returns statically allocated string. const char * getName(Metric event); - /// Get text description of metric by identifier. Returns statically allocated string. const char * getDocumentation(Metric event); const std::vector> & getAllPossibleValues(Metric event); - extern std::unordered_map values[]; + extern std::unordered_map values[]; extern std::mutex locks[]; - /// Get index just after last metric identifier. Metric end(); - /// Set status of specified. - inline void set(Metric metric, Key key, String value) + inline void set(Metric metric, Key key, Int8 value) { std::lock_guard lock(locks[metric]); values[metric][key] = value; diff --git a/dbms/src/Databases/DatabaseWithDictionaries.cpp b/dbms/src/Databases/DatabaseWithDictionaries.cpp index cd72b703ecc..444c200b4ab 100644 --- a/dbms/src/Databases/DatabaseWithDictionaries.cpp +++ b/dbms/src/Databases/DatabaseWithDictionaries.cpp @@ -1,4 +1,5 @@ #include +#include #include #include #include @@ -10,6 +11,11 @@ #include +namespace CurrentStatusInfo +{ + extern const Metric DictionaryStatus; +} + namespace DB { @@ -31,6 +37,7 @@ void DatabaseWithDictionaries::attachDictionary(const String & dictionary_name, throw Exception("Dictionary " + full_name + " already exists.", ErrorCodes::DICTIONARY_ALREADY_EXISTS); } + CurrentStatusInfo::set(CurrentStatusInfo::DictionaryStatus, full_name, static_cast(ExternalLoader::Status::NOT_LOADED)); /// ExternalLoader::reloadConfig() will find out that the dictionary's config has been added /// and in case `dictionaries_lazy_load == false` it will load the dictionary. const auto & external_loader = context.getExternalDictionariesLoader(); @@ -48,6 +55,7 @@ void DatabaseWithDictionaries::detachDictionary(const String & dictionary_name, dictionaries.erase(it); } + CurrentStatusInfo::unset(CurrentStatusInfo::DictionaryStatus, full_name); /// ExternalLoader::reloadConfig() will find out that the dictionary's config has been removed /// and therefore it will unload the dictionary. const auto & external_loader = context.getExternalDictionariesLoader(); @@ -146,6 +154,7 @@ void DatabaseWithDictionaries::removeDictionary(const Context & context, const S try { Poco::File(dictionary_metadata_path).remove(); + CurrentStatusInfo::unset(CurrentStatusInfo::DictionaryStatus, getDatabaseName() + "." + dictionary_name); } catch (...) { From ea535e254d8d980559ee2a008c37501d2dde5f8a Mon Sep 17 00:00:00 2001 From: Guillaume Tassery Date: Thu, 12 Mar 2020 13:29:28 +0100 Subject: [PATCH 0061/1355] cosmetic --- .../server/PrometheusMetricsWriter.cpp | 7 +++--- dbms/src/Common/CurrentStatusInfo.cpp | 12 +++++----- dbms/src/Common/CurrentStatusInfo.h | 22 +++++++++---------- .../Databases/DatabaseWithDictionaries.cpp | 2 +- dbms/src/Interpreters/ExternalLoader.cpp | 2 +- 5 files changed, 23 insertions(+), 22 deletions(-) diff --git a/dbms/programs/server/PrometheusMetricsWriter.cpp b/dbms/programs/server/PrometheusMetricsWriter.cpp index 6f9252e5b6c..952141e9437 100644 --- a/dbms/programs/server/PrometheusMetricsWriter.cpp +++ b/dbms/programs/server/PrometheusMetricsWriter.cpp @@ -103,8 +103,9 @@ void PrometheusMetricsWriter::write(WriteBuffer & wb) const { for (size_t i = 0, end = CurrentStatusInfo::end(); i < end; ++i) { - std::string metric_name{CurrentStatusInfo::getName(static_cast(i))}; - std::string metric_doc{CurrentStatusInfo::getDocumentation(static_cast(i))}; + std::lock_guard lock(CurrentStatusInfo::locks[static_cast(i)]); + std::string metric_name{CurrentStatusInfo::getName(static_cast(i))}; + std::string metric_doc{CurrentStatusInfo::getDocumentation(static_cast(i))}; replaceInvalidChars(metric_name); std::string key{current_status_prefix + metric_name}; @@ -114,7 +115,7 @@ void PrometheusMetricsWriter::write(WriteBuffer & wb) const for (const auto & value: CurrentStatusInfo::values[i]) { - for (const auto & enum_value: CurrentStatusInfo::getAllPossibleValues(static_cast(i))) + for (const auto & enum_value: CurrentStatusInfo::getAllPossibleValues(static_cast(i))) { DB::writeText(key, wb); DB::writeChar('{', wb); diff --git a/dbms/src/Common/CurrentStatusInfo.cpp b/dbms/src/Common/CurrentStatusInfo.cpp index 430c34cc98e..6019c69939f 100644 --- a/dbms/src/Common/CurrentStatusInfo.cpp +++ b/dbms/src/Common/CurrentStatusInfo.cpp @@ -8,16 +8,16 @@ namespace CurrentStatusInfo { - #define M(NAME, DOCUMENTATION, ENUM) extern const Metric NAME = __COUNTER__; + #define M(NAME, DOCUMENTATION, ENUM) extern const Status NAME = __COUNTER__; APPLY_FOR_STATUS(M) #undef M - constexpr Metric END = __COUNTER__; + constexpr Status END = __COUNTER__; std::mutex locks[END] {}; std::unordered_map values[END] {}; - const char * getName(Metric event) + const char * getName(Status event) { static const char * strings[] = { @@ -29,7 +29,7 @@ namespace CurrentStatusInfo return strings[event]; } - const char * getDocumentation(Metric event) + const char * getDocumentation(Status event) { static const char * strings[] = { @@ -41,7 +41,7 @@ namespace CurrentStatusInfo return strings[event]; } - const std::vector> & getAllPossibleValues(Metric event) + const std::vector> & getAllPossibleValues(Status event) { static const std::vector> enum_values [] = { @@ -52,7 +52,7 @@ namespace CurrentStatusInfo return enum_values[event]; } - Metric end() { return END; } + Status end() { return END; } } #undef APPLY_FOR_STATUS diff --git a/dbms/src/Common/CurrentStatusInfo.h b/dbms/src/Common/CurrentStatusInfo.h index b2c8827c2c9..89365f0634f 100644 --- a/dbms/src/Common/CurrentStatusInfo.h +++ b/dbms/src/Common/CurrentStatusInfo.h @@ -11,27 +11,27 @@ namespace CurrentStatusInfo { - using Metric = size_t; + using Status = size_t; using Key = std::string; - const char * getName(Metric event); - const char * getDocumentation(Metric event); - const std::vector> & getAllPossibleValues(Metric event); + const char * getName(Status event); + const char * getDocumentation(Status event); + const std::vector> & getAllPossibleValues(Status event); extern std::unordered_map values[]; extern std::mutex locks[]; - Metric end(); + Status end(); - inline void set(Metric metric, Key key, Int8 value) + inline void set(Status status, Key key, Int8 value) { - std::lock_guard lock(locks[metric]); - values[metric][key] = value; + std::lock_guard lock(locks[status]); + values[status][key] = value; } - inline void unset(Metric metric, Key key) + inline void unset(Status status, Key key) { - std::lock_guard lock(locks[metric]); - values[metric].erase(key); + std::lock_guard lock(locks[status]); + values[status].erase(key); } } diff --git a/dbms/src/Databases/DatabaseWithDictionaries.cpp b/dbms/src/Databases/DatabaseWithDictionaries.cpp index 444c200b4ab..d75a2e9d830 100644 --- a/dbms/src/Databases/DatabaseWithDictionaries.cpp +++ b/dbms/src/Databases/DatabaseWithDictionaries.cpp @@ -13,7 +13,7 @@ namespace CurrentStatusInfo { - extern const Metric DictionaryStatus; + extern const Status DictionaryStatus; } namespace DB diff --git a/dbms/src/Interpreters/ExternalLoader.cpp b/dbms/src/Interpreters/ExternalLoader.cpp index a07db4ef08d..4e18dc1bceb 100644 --- a/dbms/src/Interpreters/ExternalLoader.cpp +++ b/dbms/src/Interpreters/ExternalLoader.cpp @@ -18,7 +18,7 @@ namespace CurrentStatusInfo { - extern const Metric DictionaryStatus; + extern const Status DictionaryStatus; } From feea427267b5bd041ebe339fe299116f8fe14b76 Mon Sep 17 00:00:00 2001 From: Avogar Date: Thu, 12 Mar 2020 17:18:39 +0300 Subject: [PATCH 0062/1355] Fix build errors 2 --- dbms/src/Processors/Formats/Impl/RegexpRowInputFormat.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/src/Processors/Formats/Impl/RegexpRowInputFormat.cpp b/dbms/src/Processors/Formats/Impl/RegexpRowInputFormat.cpp index f20ae2b6cfa..e26255deae4 100644 --- a/dbms/src/Processors/Formats/Impl/RegexpRowInputFormat.cpp +++ b/dbms/src/Processors/Formats/Impl/RegexpRowInputFormat.cpp @@ -1,9 +1,9 @@ +#include #include #include #include #include #include -#include namespace DB { From 12dd0c92c09b2ad022ada4fe62a55e30e1806ad3 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Thu, 12 Mar 2020 19:48:28 +0300 Subject: [PATCH 0063/1355] better --- dbms/programs/copier/ClusterCopier.cpp | 56 +++++++++++++++---- dbms/programs/copier/ZooKeeperStaff.h | 3 + .../integration/test_cluster_copier/test.py | 10 ++-- 3 files changed, 55 insertions(+), 14 deletions(-) diff --git a/dbms/programs/copier/ClusterCopier.cpp b/dbms/programs/copier/ClusterCopier.cpp index d7a9a7922dc..8359db65e28 100644 --- a/dbms/programs/copier/ClusterCopier.cpp +++ b/dbms/programs/copier/ClusterCopier.cpp @@ -414,6 +414,9 @@ bool ClusterCopier::checkPartitionPieceIsClean( task_start_clock = LogicalClock(stat.mzxid); /// If statement for readability. + + LOG_INFO(log, "clean_state_clock.is_stale() " << clean_state_clock.is_stale()); + LOG_INFO(log, "clean_state_clock.is_clean() " << clean_state_clock.is_clean()); if (clean_state_clock.is_clean() && (!task_start_clock.hasHappened() || clean_state_clock.discovery_zxid <= task_start_clock)) { return true; @@ -680,20 +683,20 @@ bool ClusterCopier::tryDropPartitionPiece( settings_push.replication_alter_partitions_sync = 2; LOG_DEBUG(log, "Execute distributed DROP PARTITION: " << query); - /// Limit number of max executing replicas to 1 + /// We have to drop partition_piece on each replica UInt64 num_shards = executeQueryOnCluster( cluster_push, query, nullptr, &settings_push, - PoolMode::GET_ONE, - ClusterExecutionMode::ON_EACH_SHARD, - 1); + PoolMode::GET_MANY, + ClusterExecutionMode::ON_EACH_NODE); - if (num_shards < cluster_push->getShardCount()) - { - LOG_INFO(log, "DROP PARTITION wasn't successfully executed on " << cluster_push->getShardCount() - num_shards << " shards"); - return false; - } + UNUSED(num_shards); +// if (num_shards < cluster_push->getShardCount()) +// { +// LOG_INFO(log, "DROP PARTITION wasn't successfully executed on " << cluster_push->getShardCount() - num_shards << " shards"); +// return false; +// } /// Update the locking node if (!my_clock.is_stale()) @@ -717,7 +720,7 @@ bool ClusterCopier::tryDropPartitionPiece( zookeeper->set(current_shards_path, host_id); } - LOG_INFO(log, "Partition " << task_partition.name << " is safe for work now."); + LOG_INFO(log, "Partition " << task_partition.name << " piece " << toString(current_piece_number) << " is safe for work now."); return true; } @@ -926,6 +929,9 @@ PartitionTaskStatus ClusterCopier::iterateThroughAllPiecesInPartition(const Conn { for (UInt64 try_num = 0; try_num < max_shard_partition_tries; ++try_num) { + LOG_INFO(log, "Attempt number " << try_num << " to process partition " << task_partition.name + << " piece number " << piece_number << " on shard number " << task_partition.task_shard.numberInCluster() + << " with index " << task_partition.task_shard.indexInCluster()); res = processPartitionPieceTaskImpl(timeouts, task_partition, piece_number, is_unprioritized_task); /// Exit if success @@ -1141,6 +1147,8 @@ PartitionTaskStatus ClusterCopier::processPartitionPieceTaskImpl( if (count != 0) { + LOG_INFO(log, "Partition " << task_partition.name << " piece " + << current_piece_number << "is not empty. In contains " << count << " rows."); Coordination::Stat stat_shards{}; zookeeper->get(partition_piece.getPartitionPieceShardsPath(), &stat_shards); @@ -1309,6 +1317,7 @@ PartitionTaskStatus ClusterCopier::processPartitionPieceTaskImpl( catch (...) { tryLogCurrentException(log, "An error occurred during copying, partition will be marked as dirty"); + create_is_dirty_node(clean_state_clock); return PartitionTaskStatus::Error; } } @@ -1382,6 +1391,33 @@ PartitionTaskStatus ClusterCopier::processPartitionPieceTaskImpl( throw; } + + try + { + String query_deduplicate_ast_string; + if (!task_table.isReplicatedTable()) + { + query_deduplicate_ast_string += " OPTIMIZE TABLE " + getQuotedTable(original_table) + + " PARTITION " + task_partition.name + " DEDUPLICATE;"; + + LOG_DEBUG(log, "Executing OPTIMIZE DEDUPLICATE query: " << query_alter_ast_string); + + UInt64 num_nodes = executeQueryOnCluster( + task_table.cluster_push, + query_deduplicate_ast_string, + nullptr, + &task_cluster->settings_push, + PoolMode::GET_MANY); + + LOG_INFO(log, "Number of shard that executed OPTIMIZE DEDUPLICATE query successfully : " << toString(num_nodes)); + } + } + catch(...) + { + LOG_DEBUG(log, "Error while executing OPTIMIZE DEDUPLICATE partition " << task_partition.name << "in the original table"); + throw; + } + } /// Finalize the processing, change state of current partition task (and also check is_dirty flag) diff --git a/dbms/programs/copier/ZooKeeperStaff.h b/dbms/programs/copier/ZooKeeperStaff.h index 2fc4d35400d..e22df6bf199 100644 --- a/dbms/programs/copier/ZooKeeperStaff.h +++ b/dbms/programs/copier/ZooKeeperStaff.h @@ -157,6 +157,9 @@ public: bool is_clean() const { + LOG_INFO(&Poco::Logger::get("ClusterCopier"), "!discovery_zxid.hasHappened() " << !discovery_zxid.hasHappened()); + LOG_INFO(&Poco::Logger::get("ClusterCopier"), "clean_state_zxid.hasHappened() " << clean_state_zxid.hasHappened()); + LOG_INFO(&Poco::Logger::get("ClusterCopier"), "discovery_zxid <= clean_state_zxid " << (discovery_zxid <= clean_state_zxid)); return !is_stale() && (!discovery_zxid.hasHappened() || (clean_state_zxid.hasHappened() && discovery_zxid <= clean_state_zxid)); } diff --git a/dbms/tests/integration/test_cluster_copier/test.py b/dbms/tests/integration/test_cluster_copier/test.py index 7af58c66907..9ea26013125 100644 --- a/dbms/tests/integration/test_cluster_copier/test.py +++ b/dbms/tests/integration/test_cluster_copier/test.py @@ -70,10 +70,6 @@ def started_cluster(): def print_destination_cluster(task, pattern): for anime in ['s1_0_0', 's1_0_1', 's1_1_0']: - a = task.cluster.instances[anime].query("SELECT count() FROM hits_piece_0") - b = task.cluster.instances[anime].query("SELECT count() FROM hits_piece_1") - c = task.cluster.instances[anime].query("SELECT count() FROM hits") - print(anime, a, b, int(a) + int(b), c) print(task.cluster.instances[anime].query("select partition, " "name, database, table, " "rows, hash_of_all_files, " @@ -117,6 +113,12 @@ class Task1: def check(self): + for anime in ['s1_0_0', 's1_0_1', 's1_1_0']: + a = self.cluster.instances[anime].query("SELECT count() FROM hits_piece_0") + b = self.cluster.instances[anime].query("SELECT count() FROM hits_piece_1") + c = self.cluster.instances[anime].query("SELECT count() FROM hits") + print(anime, a, b, int(a) + int(b), c) + print_destination_cluster(self, "hits") assert TSV(self.cluster.instances['s0_0_0'].query("SELECT count() FROM hits_all")) == TSV("1002\n") From fc599780b653c6a57e0c4bc88fc49ed8ec6c6851 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Thu, 12 Mar 2020 22:46:48 +0300 Subject: [PATCH 0064/1355] slightly better code --- dbms/programs/copier/ClusterCopier.cpp | 16 +++++++++------- 1 file changed, 9 insertions(+), 7 deletions(-) diff --git a/dbms/programs/copier/ClusterCopier.cpp b/dbms/programs/copier/ClusterCopier.cpp index 8359db65e28..62c51952e95 100644 --- a/dbms/programs/copier/ClusterCopier.cpp +++ b/dbms/programs/copier/ClusterCopier.cpp @@ -152,14 +152,13 @@ void ClusterCopier::discoverShardPartitions(const ConnectionTimeouts & timeouts, for (const String & partition_name : filtered_partitions_names) { - task_shard->partition_tasks.emplace(partition_name, ShardPartition(*task_shard, partition_name, 10)); + const size_t number_of_splits = task_table.number_of_splits; + task_shard->partition_tasks.emplace(partition_name, ShardPartition(*task_shard, partition_name, number_of_splits)); task_shard->checked_partitions.emplace(partition_name, true); auto shard_partition_it = task_shard->partition_tasks.find(partition_name); PartitionPieces & shard_partition_pieces = shard_partition_it->second.pieces; - const size_t number_of_splits = task_table.number_of_splits; - for (size_t piece_number = 0; piece_number < number_of_splits; ++piece_number) { bool res = checkPresentPartitionPiecesOnCurrentShard(timeouts, *task_shard, partition_name, piece_number); @@ -670,7 +669,11 @@ bool ClusterCopier::tryDropPartitionPiece( } } - String query = "ALTER TABLE " + getQuotedTable(task_table.table_push) + "_piece_" + toString(current_piece_number); + + DatabaseAndTableName original_table = task_table.table_push; + DatabaseAndTableName helping_table = DatabaseAndTableName(original_table.first, original_table.second + "_piece_" + toString(current_piece_number)); + + String query = "ALTER TABLE " + getQuotedTable(helping_table); query += " DROP PARTITION " + task_partition.name + ""; /// TODO: use this statement after servers will be updated up to 1.1.54310 @@ -764,14 +767,13 @@ bool ClusterCopier::tryProcessTable(const ConnectionTimeouts & timeouts, TaskTab if (has_partition) { - shard->partition_tasks.emplace(partition_name, ShardPartition(*shard, partition_name, 10)); + const size_t number_of_splits = task_table.number_of_splits; + shard->partition_tasks.emplace(partition_name, ShardPartition(*shard, partition_name, number_of_splits)); LOG_DEBUG(log, "Discovered partition " << partition_name << " in shard " << shard->getDescription()); /// To save references in the future. auto shard_partition_it = shard->partition_tasks.find(partition_name); PartitionPieces & shard_partition_pieces = shard_partition_it->second.pieces; - const size_t number_of_splits = task_table.number_of_splits; - for (size_t piece_number = 0; piece_number < number_of_splits; ++piece_number) { auto res = checkPresentPartitionPiecesOnCurrentShard(timeouts, *shard, partition_name, piece_number); From 65d6cc0aa98f1f513c411be6a1f5fe10712e0c52 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Thu, 12 Mar 2020 22:52:36 +0300 Subject: [PATCH 0065/1355] codestyle --- dbms/programs/copier/Internals.h | 1 - dbms/programs/copier/TaskTableAndShard.h | 22 +++++++++++++--------- 2 files changed, 13 insertions(+), 10 deletions(-) diff --git a/dbms/programs/copier/Internals.h b/dbms/programs/copier/Internals.h index 4e4142a035b..fc4c2f09da7 100644 --- a/dbms/programs/copier/Internals.h +++ b/dbms/programs/copier/Internals.h @@ -14,7 +14,6 @@ #include #include #include -#include #include #include #include diff --git a/dbms/programs/copier/TaskTableAndShard.h b/dbms/programs/copier/TaskTableAndShard.h index 91e14598e81..b33876c679e 100644 --- a/dbms/programs/copier/TaskTableAndShard.h +++ b/dbms/programs/copier/TaskTableAndShard.h @@ -286,7 +286,8 @@ inline TaskTable::TaskTable(TaskCluster & parent, const Poco::Util::AbstractConf } where_condition_str = config.getString(table_prefix + "where_condition", ""); - if (!where_condition_str.empty()) { + if (!where_condition_str.empty()) + { ParserExpressionWithOptionalAlias parser_expression(false); where_condition_ast = parseQuery(parser_expression, where_condition_str, 0); @@ -297,28 +298,31 @@ inline TaskTable::TaskTable(TaskCluster & parent, const Poco::Util::AbstractConf String enabled_partitions_prefix = table_prefix + "enabled_partitions"; has_enabled_partitions = config.has(enabled_partitions_prefix); - if (has_enabled_partitions) { + if (has_enabled_partitions) + { Strings keys; config.keys(enabled_partitions_prefix, keys); - if (keys.empty()) { + if (keys.empty()) + { /// Parse list of partition from space-separated string String partitions_str = config.getString(table_prefix + "enabled_partitions"); boost::trim_if(partitions_str, isWhitespaceASCII); boost::split(enabled_partitions, partitions_str, isWhitespaceASCII, boost::token_compress_on); - } else { + } + else + { /// Parse sequence of ... - for (const String &key : keys) { + for (const String &key : keys) + { if (!startsWith(key, "partition")) - throw Exception("Unknown key " + key + " in " + enabled_partitions_prefix, - ErrorCodes::UNKNOWN_ELEMENT_IN_CONFIG); + throw Exception("Unknown key " + key + " in " + enabled_partitions_prefix, ErrorCodes::UNKNOWN_ELEMENT_IN_CONFIG); enabled_partitions.emplace_back(config.getString(enabled_partitions_prefix + "." + key)); } } - std::copy(enabled_partitions.begin(), enabled_partitions.end(), - std::inserter(enabled_partitions_set, enabled_partitions_set.begin())); + std::copy(enabled_partitions.begin(), enabled_partitions.end(), std::inserter(enabled_partitions_set, enabled_partitions_set.begin())); } } From 88fc6f2e91d3dd65590b34d5427d7c965bc0799e Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Thu, 12 Mar 2020 22:54:37 +0300 Subject: [PATCH 0066/1355] remove unused code --- dbms/src/Parsers/ASTLiteral.h | 4 ++-- dbms/src/Storages/StorageReplicatedMergeTree.cpp | 4 ---- 2 files changed, 2 insertions(+), 6 deletions(-) diff --git a/dbms/src/Parsers/ASTLiteral.h b/dbms/src/Parsers/ASTLiteral.h index 1d307a4101e..552f5da04a2 100644 --- a/dbms/src/Parsers/ASTLiteral.h +++ b/dbms/src/Parsers/ASTLiteral.h @@ -21,8 +21,8 @@ public: std::optional begin; std::optional end; - explicit ASTLiteral(Field && value_) : value(value_) {} - explicit ASTLiteral(const Field & value_) : value(value_) {} + ASTLiteral(Field && value_) : value(value_) {} + ASTLiteral(const Field & value_) : value(value_) {} /** Get the text that identifies this element. */ String getID(char delim) const override { return "Literal" + (delim + applyVisitor(FieldVisitorDump(), value)); } diff --git a/dbms/src/Storages/StorageReplicatedMergeTree.cpp b/dbms/src/Storages/StorageReplicatedMergeTree.cpp index 244dca2a044..891e3beb0c1 100644 --- a/dbms/src/Storages/StorageReplicatedMergeTree.cpp +++ b/dbms/src/Storages/StorageReplicatedMergeTree.cpp @@ -4910,10 +4910,6 @@ void StorageReplicatedMergeTree::replacePartitionFrom(const StoragePtr & source_ part_checksums.emplace_back(hash_hex); } - /// We have nothing to do - return - if (src_parts.empty()) - return; - ReplicatedMergeTreeLogEntryData entry; { auto src_table_id = src_data.getStorageID(); From ad7a9314903b428c2ca6c778f37085200113288d Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Thu, 12 Mar 2020 23:13:31 +0300 Subject: [PATCH 0067/1355] return back fixes in StorageReplicatedMergeTree --- dbms/src/Storages/StorageReplicatedMergeTree.cpp | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/dbms/src/Storages/StorageReplicatedMergeTree.cpp b/dbms/src/Storages/StorageReplicatedMergeTree.cpp index 891e3beb0c1..244dca2a044 100644 --- a/dbms/src/Storages/StorageReplicatedMergeTree.cpp +++ b/dbms/src/Storages/StorageReplicatedMergeTree.cpp @@ -4910,6 +4910,10 @@ void StorageReplicatedMergeTree::replacePartitionFrom(const StoragePtr & source_ part_checksums.emplace_back(hash_hex); } + /// We have nothing to do - return + if (src_parts.empty()) + return; + ReplicatedMergeTreeLogEntryData entry; { auto src_table_id = src_data.getStorageID(); From cce69b0744bd337bbacf8abbc32d1f2ae604ebfb Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Fri, 13 Mar 2020 13:14:48 +0300 Subject: [PATCH 0068/1355] fix mac build --- dbms/programs/copier/ClusterCopier.cpp | 2 ++ 1 file changed, 2 insertions(+) diff --git a/dbms/programs/copier/ClusterCopier.cpp b/dbms/programs/copier/ClusterCopier.cpp index 62c51952e95..f308a9a55f5 100644 --- a/dbms/programs/copier/ClusterCopier.cpp +++ b/dbms/programs/copier/ClusterCopier.cpp @@ -2,6 +2,8 @@ #include "Internals.h" +#include + #include #include From 42fae556c11de775760ec946832bd5a9adfa911c Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Fri, 13 Mar 2020 17:19:20 +0300 Subject: [PATCH 0069/1355] split up copying and moving --- dbms/programs/copier/ClusterCopier.cpp | 259 ++++++++++++++++------- dbms/programs/copier/ClusterCopier.h | 4 + dbms/programs/copier/TaskTableAndShard.h | 21 +- 3 files changed, 199 insertions(+), 85 deletions(-) diff --git a/dbms/programs/copier/ClusterCopier.cpp b/dbms/programs/copier/ClusterCopier.cpp index f308a9a55f5..d7be72022f1 100644 --- a/dbms/programs/copier/ClusterCopier.cpp +++ b/dbms/programs/copier/ClusterCopier.cpp @@ -433,8 +433,14 @@ bool ClusterCopier::checkPartitionPieceIsClean( bool ClusterCopier::checkAllPiecesInPartitionAreDone(const TaskTable & task_table, const String & partition_name, const TasksShard & shards_with_partition) { bool answer = true; - for (size_t piece_number = 0; piece_number < task_table.number_of_splits; piece_number++) - answer &= checkPartitionPieceIsDone(task_table, partition_name, piece_number, shards_with_partition); + for (size_t piece_number = 0; piece_number < task_table.number_of_splits; ++piece_number) + { + bool piece_is_done = checkPartitionPieceIsDone(task_table, partition_name, piece_number, shards_with_partition); + if (!piece_is_done) + LOG_DEBUG(log, "Partition " << partition_name << " piece " + toString(piece_number) + " is not already done."); + answer &= piece_is_done; + } + return answer; } @@ -528,10 +534,156 @@ bool ClusterCopier::checkPartitionPieceIsDone(const TaskTable & task_table, cons } } - LOG_INFO(log, "Partition " << partition_name << " is copied successfully"); + LOG_INFO(log, "Partition " << partition_name << " piece number " << toString(piece_number) << " is copied successfully"); return true; } + +PartitionTaskStatus ClusterCopier::tryMoveAllPiecesToDestinationTable(const TaskTable & task_table, const String & partition_name) +{ + LOG_DEBUG(log, "Try to move " << partition_name << " to destionation table"); + + auto zookeeper = context.getZooKeeper(); + + const auto current_partition_attach_is_active = task_table.getPartitionAttachIsActivePath(partition_name); + const auto current_partition_attach_is_done = task_table.getPartitionAttachIsDonePath(partition_name); + + /// Create ephemeral node to mark that we are active and process the partition + zookeeper->createAncestors(current_partition_attach_is_active); + zkutil::EphemeralNodeHolderPtr partition_attach_node_holder; + try + { + partition_attach_node_holder = zkutil::EphemeralNodeHolder::create(current_partition_attach_is_active, *zookeeper, host_id); + } + catch (const Coordination::Exception & e) + { + if (e.code == Coordination::ZNODEEXISTS) + { + LOG_DEBUG(log, "Someone is already moving pieces " << current_partition_attach_is_active); + return PartitionTaskStatus::Active; + } + + throw; + } + + + /// Exit if task has been already processed; + /// create blocking node to signal cleaning up if it is abandoned + { + String status_data; + if (zookeeper->tryGet(current_partition_attach_is_done, status_data)) + { + TaskStateWithOwner status = TaskStateWithOwner::fromString(status_data); + if (status.state == TaskState::Finished) + { + LOG_DEBUG(log, "All pieces for partition from this task " << current_partition_attach_is_active + << " has been successfully moved to destination table by " << status.owner); + return PartitionTaskStatus::Finished; + } + + /// Task is abandoned, because previously we created ephemeral node, possibly in other copier's process. + /// Initialize DROP PARTITION + LOG_DEBUG(log, "Moving piece for partition " << current_partition_attach_is_active + << " has not been successfully finished by " << status.owner + << ". Will try to move by myself."); + } + } + + + /// Try start processing, create node about it + { + String start_state = TaskStateWithOwner::getData(TaskState::Started, host_id); + zookeeper->create(current_partition_attach_is_done, start_state, zkutil::CreateMode::Persistent); + } + + for (size_t current_piece_number = 0; current_piece_number < task_table.number_of_splits; ++current_piece_number) + { + /// Move partition to original destination table. + { + /// TODO: Execute alter table move partition. + + LOG_DEBUG(log, "Trying to move partition " << partition_name + << " piece " << toString(current_piece_number) << " to original table"); + + ASTPtr query_alter_ast; + String query_alter_ast_string; + + DatabaseAndTableName original_table = task_table.table_push; + DatabaseAndTableName helping_table = DatabaseAndTableName(original_table.first, original_table.second + "_piece_" + toString(current_piece_number)); + + query_alter_ast_string += " ALTER TABLE " + getQuotedTable(original_table) + + " ATTACH PARTITION " + partition_name + + " FROM " + getQuotedTable(helping_table) + + " SETTINGS replication_alter_partitions_sync=2;"; + + LOG_DEBUG(log, "Executing ALTER query: " << query_alter_ast_string); + + try + { + size_t num_nodes = 0; + + for (UInt64 try_num = 0; try_num < max_shard_partition_piece_tries_for_alter; ++try_num) + { + ///FIXME: We have to be sure that every node in cluster executed this query + UInt64 current_num_nodes = executeQueryOnCluster( + task_table.cluster_push, + query_alter_ast_string, + nullptr, + &task_cluster->settings_push, + PoolMode::GET_MANY, + ClusterExecutionMode::ON_EACH_NODE); + + num_nodes = std::max(current_num_nodes, num_nodes); + } + + LOG_INFO(log, "Number of nodes that executed ALTER query successfully : " << toString(num_nodes)); + } + catch (...) + { + LOG_DEBUG(log, "Error while moving partition " << partition_name + << " piece " << toString(current_piece_number) << "to original table"); + throw; + } + + + try + { + String query_deduplicate_ast_string; + if (!task_table.isReplicatedTable()) + { + query_deduplicate_ast_string += " OPTIMIZE TABLE " + getQuotedTable(original_table) + + " PARTITION " + partition_name + " DEDUPLICATE;"; + + LOG_DEBUG(log, "Executing OPTIMIZE DEDUPLICATE query: " << query_alter_ast_string); + + UInt64 num_nodes = executeQueryOnCluster( + task_table.cluster_push, + query_deduplicate_ast_string, + nullptr, + &task_cluster->settings_push, + PoolMode::GET_MANY); + + LOG_INFO(log, "Number of shard that executed OPTIMIZE DEDUPLICATE query successfully : " << toString(num_nodes)); + } + } + catch(...) + { + LOG_DEBUG(log, "Error while executing OPTIMIZE DEDUPLICATE partition " << partition_name << "in the original table"); + throw; + } + } + } + + + /// Create node to signal that we finished moving + { + String state_finished = TaskStateWithOwner::getData(TaskState::Finished, host_id); + zookeeper->set(current_partition_attach_is_done, state_finished, 0); + } + + return PartitionTaskStatus::Finished; +} + /// Removes MATERIALIZED and ALIAS columns from create table query ASTPtr ClusterCopier::removeAliasColumnsFromCreateQuery(const ASTPtr & query_ast) { @@ -838,20 +990,37 @@ bool ClusterCopier::tryProcessTable(const ConnectionTimeouts & timeouts, TaskTab /// Check that whole cluster partition is done /// Firstly check the number of failed partition tasks, then look into ZooKeeper and ensure that each partition is done - bool partition_is_done = num_failed_shards == 0; + bool partition_copying_is_done = num_failed_shards == 0; try { - partition_is_done = + partition_copying_is_done = !has_shard_to_process - || (partition_is_done && checkAllPiecesInPartitionAreDone(task_table, partition_name, expected_shards)); + || (partition_copying_is_done && checkAllPiecesInPartitionAreDone(task_table, partition_name, expected_shards)); } catch (...) { tryLogCurrentException(log); - partition_is_done = false; + partition_copying_is_done = false; } - if (partition_is_done) + + bool partition_moving_is_done = false; + /// Try to move only if all pieces were copied. + if (partition_copying_is_done) + { + try + { + auto res = tryMoveAllPiecesToDestinationTable(task_table, partition_name); + if (res == PartitionTaskStatus::Finished) + partition_moving_is_done = true; + } + catch (...) + { + tryLogCurrentException(log, "Some error occured while moving pieces to destination table for partition " + partition_name); + } + } + + if (partition_copying_is_done && partition_moving_is_done) { task_table.finished_cluster_partitions.emplace(partition_name); @@ -1350,80 +1519,6 @@ PartitionTaskStatus ClusterCopier::processPartitionPieceTaskImpl( tryLogCurrentException(log, "Error while creating original table. Maybe we are not first."); } - /// Move partition to original destination table. - { - /// TODO: Execute alter table move partition. - - LOG_DEBUG(log, "Trying to move partition " << task_partition.name - << " piece " << toString(current_piece_number) << " to original table"); - - ASTPtr query_alter_ast; - String query_alter_ast_string; - - DatabaseAndTableName original_table = task_table.table_push; - DatabaseAndTableName helping_table = DatabaseAndTableName(original_table.first, original_table.second + "_piece_" + toString(current_piece_number)); - - query_alter_ast_string += " ALTER TABLE " + getQuotedTable(original_table) + - " ATTACH PARTITION " + task_partition.name + - " FROM " + getQuotedTable(helping_table) + - " SETTINGS replication_alter_partitions_sync=2;"; - - LOG_DEBUG(log, "Executing ALTER query: " << query_alter_ast_string); - - try - { - size_t num_nodes = 0; - for (UInt64 try_num = 0; try_num < max_shard_partition_piece_tries_for_alter; ++try_num) - { - ///FIXME: We have to be sure that every node in cluster executed this query - UInt64 current_num_nodes = executeQueryOnCluster( - task_table.cluster_push, - query_alter_ast_string, - nullptr, - &task_cluster->settings_push, - PoolMode::GET_MANY, - ClusterExecutionMode::ON_EACH_NODE); - - num_nodes = std::max(current_num_nodes, num_nodes); - } - LOG_INFO(log, "Number of shard that executed ALTER query successfully : " << toString(num_nodes)); - } - catch (...) - { - LOG_DEBUG(log, "Error while moving partition " << task_partition.name - << " piece " << toString(current_piece_number) << "to original table"); - throw; - } - - - try - { - String query_deduplicate_ast_string; - if (!task_table.isReplicatedTable()) - { - query_deduplicate_ast_string += " OPTIMIZE TABLE " + getQuotedTable(original_table) + - " PARTITION " + task_partition.name + " DEDUPLICATE;"; - - LOG_DEBUG(log, "Executing OPTIMIZE DEDUPLICATE query: " << query_alter_ast_string); - - UInt64 num_nodes = executeQueryOnCluster( - task_table.cluster_push, - query_deduplicate_ast_string, - nullptr, - &task_cluster->settings_push, - PoolMode::GET_MANY); - - LOG_INFO(log, "Number of shard that executed OPTIMIZE DEDUPLICATE query successfully : " << toString(num_nodes)); - } - } - catch(...) - { - LOG_DEBUG(log, "Error while executing OPTIMIZE DEDUPLICATE partition " << task_partition.name << "in the original table"); - throw; - } - - } - /// Finalize the processing, change state of current partition task (and also check is_dirty flag) { String state_finished = TaskStateWithOwner::getData(TaskState::Finished, host_id); diff --git a/dbms/programs/copier/ClusterCopier.h b/dbms/programs/copier/ClusterCopier.h index bed6352e129..a39653c6c87 100644 --- a/dbms/programs/copier/ClusterCopier.h +++ b/dbms/programs/copier/ClusterCopier.h @@ -104,6 +104,10 @@ protected: bool checkPartitionPieceIsDone(const TaskTable & task_table, const String & partition_name, size_t piece_number, const TasksShard & shards_with_partition); + + /*Alter successful insertion to helping tables it will move all pieces to destination table*/ + PartitionTaskStatus tryMoveAllPiecesToDestinationTable(const TaskTable & task_table, const String & partition_name); + /// Removes MATERIALIZED and ALIAS columns from create table query ASTPtr removeAliasColumnsFromCreateQuery(const ASTPtr & query_ast); diff --git a/dbms/programs/copier/TaskTableAndShard.h b/dbms/programs/copier/TaskTableAndShard.h index b33876c679e..9a09deb911f 100644 --- a/dbms/programs/copier/TaskTableAndShard.h +++ b/dbms/programs/copier/TaskTableAndShard.h @@ -25,6 +25,10 @@ struct TaskTable { String getPartitionPath(const String & partition_name) const; + String getPartitionAttachIsActivePath(const String & partition_name) const; + + String getPartitionAttachIsDonePath(const String & partition_name) const; + String getPartitionPiecePath(const String & partition_name, const size_t piece_number) const; String getCertainPartitionIsDirtyPath(const String & partition_name) const; @@ -42,7 +46,7 @@ struct TaskTable { [[maybe_unused]] String getReplicatedEngineFirstArgumentForCurrentPiece(const size_t piece_number) const; - bool isReplicatedTable() { return engine_push_zk_path != ""; } + bool isReplicatedTable() const { return engine_push_zk_path != ""; } /// Partitions will be splitted into number-of-splits pieces. /// Each piece will be copied independently. (10 by default) @@ -191,12 +195,23 @@ struct TaskShard }; -inline String TaskTable::getPartitionPath(const String &partition_name) const { +inline String TaskTable::getPartitionPath(const String & partition_name) const +{ return task_cluster.task_zookeeper_path // root + "/tables/" + table_id // tables/dst_cluster.merge.hits + "/" + escapeForFileName(partition_name); // 201701 } +inline String TaskTable::getPartitionAttachIsActivePath(const String & partition_name) const +{ + return getPartitionPath(partition_name) + "/attach_active"; +} + +inline String TaskTable::getPartitionAttachIsDonePath(const String & partition_name) const +{ + return getPartitionPath(partition_name) + "/attach_is_done"; +} + inline String TaskTable::getPartitionPiecePath(const String & partition_name, size_t piece_number) const { assert(piece_number < number_of_splits); @@ -241,7 +256,7 @@ inline TaskTable::TaskTable(TaskCluster & parent, const Poco::Util::AbstractConf name_in_config = table_key; - number_of_splits = config.getUInt64(table_prefix + "number_of_splits", 2); + number_of_splits = config.getUInt64(table_prefix + "number_of_splits", 10); cluster_pull_name = config.getString(table_prefix + "cluster_pull"); cluster_push_name = config.getString(table_prefix + "cluster_push"); From b26a949c372455e11b58e767645cf445a14a0b70 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Fri, 13 Mar 2020 19:25:07 +0300 Subject: [PATCH 0070/1355] drop helping tables --- dbms/programs/copier/ClusterCopier.cpp | 30 ++++++++++++++++++++++++++ dbms/programs/copier/ClusterCopier.h | 2 ++ 2 files changed, 32 insertions(+) diff --git a/dbms/programs/copier/ClusterCopier.cpp b/dbms/programs/copier/ClusterCopier.cpp index d7be72022f1..471db73e715 100644 --- a/dbms/programs/copier/ClusterCopier.cpp +++ b/dbms/programs/copier/ClusterCopier.cpp @@ -318,6 +318,9 @@ void ClusterCopier::process(const ConnectionTimeouts & timeouts) } } + /// Delete helping tables in both cases (whole table is done or not) + dropHelpingTables(task_table); + if (!table_is_done) { throw Exception("Too many tries to process table " + task_table.table_id + ". Abort remaining execution", @@ -1564,6 +1567,33 @@ void ClusterCopier::dropLocalTableIfExists(const DatabaseAndTableName & table_na interpreter.execute(); } + +void ClusterCopier::dropHelpingTables(const TaskTable & task_table) +{ + LOG_DEBUG(log, "Removing helping tables"); + for (size_t current_piece_number = 0; current_piece_number < task_table.number_of_splits; ++current_piece_number) + { + DatabaseAndTableName original_table = task_table.table_push; + DatabaseAndTableName helping_table = DatabaseAndTableName(original_table.first, original_table.second + "_piece_" + toString(current_piece_number)); + + String query = "DROP TABLE IF EXISTS " + getQuotedTable(helping_table); + + const ClusterPtr & cluster_push = task_table.cluster_push; + Settings settings_push = task_cluster->settings_push; + + LOG_DEBUG(log, "Execute distributed DROP TABLE: " << query); + /// We have to drop partition_piece on each replica + UInt64 num_nodes = executeQueryOnCluster( + cluster_push, query, + nullptr, + &settings_push, + PoolMode::GET_MANY, + ClusterExecutionMode::ON_EACH_NODE); + + LOG_DEBUG(log, "DROP TABLE query was successfully executed on " << toString(num_nodes) << " nodes."); + } +} + String ClusterCopier::getRemoteCreateTable(const DatabaseAndTableName & table, Connection & connection, const Settings * settings) { String query = "SHOW CREATE TABLE " + getQuotedTable(table); diff --git a/dbms/programs/copier/ClusterCopier.h b/dbms/programs/copier/ClusterCopier.h index a39653c6c87..ce9ea7cc7ef 100644 --- a/dbms/programs/copier/ClusterCopier.h +++ b/dbms/programs/copier/ClusterCopier.h @@ -137,6 +137,8 @@ protected: void dropLocalTableIfExists(const DatabaseAndTableName & table_name) const; + void dropHelpingTables(const TaskTable & task_table); + String getRemoteCreateTable(const DatabaseAndTableName & table, Connection & connection, const Settings * settings = nullptr); ASTPtr getCreateTableForPullShard(const ConnectionTimeouts & timeouts, TaskShard & task_shard); From 3e394bee2468e9cf2d77e97023f938c168c41eec Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Fri, 13 Mar 2020 19:39:16 +0300 Subject: [PATCH 0071/1355] remove useless prints from test --- .../integration/test_cluster_copier/test.py | 41 ------------------- 1 file changed, 41 deletions(-) diff --git a/dbms/tests/integration/test_cluster_copier/test.py b/dbms/tests/integration/test_cluster_copier/test.py index 9ea26013125..47df976bbcf 100644 --- a/dbms/tests/integration/test_cluster_copier/test.py +++ b/dbms/tests/integration/test_cluster_copier/test.py @@ -68,28 +68,6 @@ def started_cluster(): pass cluster.shutdown() -def print_destination_cluster(task, pattern): - for anime in ['s1_0_0', 's1_0_1', 's1_1_0']: - print(task.cluster.instances[anime].query("select partition, " - "name, database, table, " - "rows, hash_of_all_files, " - "hash_of_uncompressed_files, " - "uncompressed_hash_of_compressed_files " - "from system.parts " - "where database='default'" - "format PrettyCompact".format(pattern))) - - -def print_source_cluster(task, pattern): - for anime in ['s0_0_0', 's0_0_1', 's0_1_0']: - print(task.cluster.instances[anime].query("select partition, " - "name, database, table, " - "rows, min_date, min_time, hash_of_all_files, " - "hash_of_uncompressed_files, " - "uncompressed_hash_of_compressed_files " - "from system.parts " - "where table like '%{}%' " - "format PrettyCompact".format(pattern))) class Task1: @@ -113,18 +91,7 @@ class Task1: def check(self): - for anime in ['s1_0_0', 's1_0_1', 's1_1_0']: - a = self.cluster.instances[anime].query("SELECT count() FROM hits_piece_0") - b = self.cluster.instances[anime].query("SELECT count() FROM hits_piece_1") - c = self.cluster.instances[anime].query("SELECT count() FROM hits") - print(anime, a, b, int(a) + int(b), c) - - print_destination_cluster(self, "hits") assert TSV(self.cluster.instances['s0_0_0'].query("SELECT count() FROM hits_all")) == TSV("1002\n") - - assert TSV(self.cluster.instances['s1_0_0'].query("SELECT DISTINCT d % 2 FROM hits")) == TSV("1\n") - assert TSV(self.cluster.instances['s1_1_0'].query("SELECT DISTINCT d % 2 FROM hits")) == TSV("0\n") - assert TSV(self.cluster.instances['s1_0_0'].query("SELECT count() FROM hits_all")) == TSV("1002\n") assert TSV(self.cluster.instances['s1_0_0'].query("SELECT DISTINCT d % 2 FROM hits")) == TSV("1\n") @@ -157,11 +124,7 @@ class Task2: instance.query("INSERT INTO a_all SELECT toDate(17581 + number) AS date, number AS d FROM system.numbers LIMIT 85", settings={"insert_distributed_sync": 1}) - print_source_cluster(self, "a") - - def check(self): - print_destination_cluster(self, "a") assert TSV(self.cluster.instances['s0_0_0'].query("SELECT count() FROM cluster(cluster0, default, a)")) == TSV("85\n") assert TSV(self.cluster.instances['s1_0_0'].query("SELECT count(), uniqExact(date) FROM cluster(cluster1, default, b)")) == TSV("85\t85\n") @@ -197,8 +160,6 @@ class Task_test_block_size: def check(self): - print_destination_cluster(self, "test_block_size") - assert TSV(self.cluster.instances['s1_0_0'].query("SELECT count() FROM cluster(cluster1, default, test_block_size)")) == TSV("{}\n".format(self.rows)) instance = cluster.instances['s0_0_0'] @@ -222,7 +183,6 @@ class Task_no_index: def check(self): - print_destination_cluster(self, "ontime") assert TSV(self.cluster.instances['s1_1_0'].query("SELECT Year FROM ontime22")) == TSV("2017\n") instance = cluster.instances['s0_0_0'] instance.query("DROP TABLE ontime") @@ -246,7 +206,6 @@ class Task_no_arg: def check(self): - print_destination_cluster(self, "copier_test1") assert TSV(self.cluster.instances['s1_1_0'].query("SELECT date FROM copier_test1_1")) == TSV("2016-01-01\n") instance = cluster.instances['s0_0_0'] instance.query("DROP TABLE copier_test1") From b6d633a09ef7ba812dd95c3e09b273b00d84273b Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Fri, 13 Mar 2020 19:53:32 +0300 Subject: [PATCH 0072/1355] more comments --- dbms/src/Storages/StorageReplicatedMergeTree.cpp | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/dbms/src/Storages/StorageReplicatedMergeTree.cpp b/dbms/src/Storages/StorageReplicatedMergeTree.cpp index 244dca2a044..c37623986c3 100644 --- a/dbms/src/Storages/StorageReplicatedMergeTree.cpp +++ b/dbms/src/Storages/StorageReplicatedMergeTree.cpp @@ -4888,7 +4888,7 @@ void StorageReplicatedMergeTree::replacePartitionFrom(const StoragePtr & source_ String hash_hex = src_part->checksums.getTotalChecksumHex(); - LOG_INFO(log, "Trying to attach " << src_part->name << " with hash_hex " << hash_hex); + LOG_INFO(log, "Trying to " << (replace ? "replace " : "attach ") << src_part->name << " with hash_hex " << hash_hex); String block_id_path = replace ? "" : (zookeeper_path + "/blocks/" + partition_id + "_replace_from_" + hash_hex); @@ -4910,8 +4910,9 @@ void StorageReplicatedMergeTree::replacePartitionFrom(const StoragePtr & source_ part_checksums.emplace_back(hash_hex); } - /// We have nothing to do - return - if (src_parts.empty()) + /// In case of ATTACH PARTITION FROM we have nothing to do - return. Because no data will be added or removed. + /// In case of REPLACE PARTITION we can replace existing partition with empty. + if (!replace && src_parts.empty()) return; ReplicatedMergeTreeLogEntryData entry; From 91f57e46e9e93497b3249b33a6aa8a5c6ad9def9 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Fri, 13 Mar 2020 20:05:05 +0300 Subject: [PATCH 0073/1355] fix mac build --- dbms/programs/copier/ClusterCopier.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/programs/copier/ClusterCopier.cpp b/dbms/programs/copier/ClusterCopier.cpp index 471db73e715..88590205850 100644 --- a/dbms/programs/copier/ClusterCopier.cpp +++ b/dbms/programs/copier/ClusterCopier.cpp @@ -628,7 +628,7 @@ PartitionTaskStatus ClusterCopier::tryMoveAllPiecesToDestinationTable(const Task for (UInt64 try_num = 0; try_num < max_shard_partition_piece_tries_for_alter; ++try_num) { ///FIXME: We have to be sure that every node in cluster executed this query - UInt64 current_num_nodes = executeQueryOnCluster( + size_t current_num_nodes = executeQueryOnCluster( task_table.cluster_push, query_alter_ast_string, nullptr, From 9e0c257162dc62141efbab9527c1e3e2a9d8d37d Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 13 Mar 2020 20:31:50 +0300 Subject: [PATCH 0074/1355] WeakHash32 [part 1]. --- dbms/src/Columns/ColumnString.cpp | 27 +++++++++++++++++++++++++++ dbms/src/Columns/ColumnString.h | 2 ++ dbms/src/Columns/ColumnVector.cpp | 31 +++++++++++++++++++++++++++++++ dbms/src/Columns/ColumnVector.h | 2 ++ dbms/src/Columns/IColumn.h | 6 ++++++ dbms/src/Common/WeakHash.cpp | 2 ++ dbms/src/Common/WeakHash.h | 25 +++++++++++++++++++++++++ 7 files changed, 95 insertions(+) create mode 100644 dbms/src/Common/WeakHash.cpp create mode 100644 dbms/src/Common/WeakHash.h diff --git a/dbms/src/Columns/ColumnString.cpp b/dbms/src/Columns/ColumnString.cpp index ec4d445079f..e1014008000 100644 --- a/dbms/src/Columns/ColumnString.cpp +++ b/dbms/src/Columns/ColumnString.cpp @@ -2,6 +2,8 @@ #include #include #include +#include +#include #include #include #include @@ -63,6 +65,31 @@ MutableColumnPtr ColumnString::cloneResized(size_t to_size) const return res; } +void ColumnString::updateWeakHash32(WeakHash32 & hash) const +{ + auto s = offsets.size(); + + if (hash.getData().size() != s) + throw Exception("Size of WeakHash32 does not match size of column: column size is " + std::to_string(s) + + ", hash size is " + std::to_string(hash.getData().size()), ErrorCodes::LOGICAL_ERROR); + + const UInt8 * begin = &chars[0]; + const UInt8 * str_begin = begin; + const Offset * offset_begin = &offsets[0]; + const Offset * offset_end = offset_begin + s; + UInt32 * hash_data = &hash.getData()[0]; + + while (offset_begin < offset_end) + { + const UInt8 * str_end = begin + *offset_begin; + + auto str_size = str_end - str_begin; + + ++offset_begin; + str_begin = str_end; + } +} + void ColumnString::insertRangeFrom(const IColumn & src, size_t start, size_t length) { diff --git a/dbms/src/Columns/ColumnString.h b/dbms/src/Columns/ColumnString.h index 8f1eced92f9..32116880014 100644 --- a/dbms/src/Columns/ColumnString.h +++ b/dbms/src/Columns/ColumnString.h @@ -188,6 +188,8 @@ public: hash.update(reinterpret_cast(&chars[offset]), string_size); } + void updateWeakHash32(WeakHash32 & hash) const override; + void insertRangeFrom(const IColumn & src, size_t start, size_t length) override; ColumnPtr filter(const Filter & filt, ssize_t result_size_hint) const override; diff --git a/dbms/src/Columns/ColumnVector.cpp b/dbms/src/Columns/ColumnVector.cpp index a1c86953dc9..697da7e7002 100644 --- a/dbms/src/Columns/ColumnVector.cpp +++ b/dbms/src/Columns/ColumnVector.cpp @@ -9,6 +9,8 @@ #include #include #include +#include +#include #include #include #include @@ -51,6 +53,35 @@ void ColumnVector::updateHashWithValue(size_t n, SipHash & hash) const hash.update(data[n]); } +template +void ColumnVector::updateWeakHash32(WeakHash32 & hash) const +{ + auto s = data.size(); + + if (hash.getData().size() != s) + throw Exception("Size of WeakHash32 does not match size of column: column size is " + std::to_string(s) + + ", hash size is " + std::to_string(hash.getData().size()), ErrorCodes::LOGICAL_ERROR); + + const T * begin = &data[0]; + const T * end = begin + s; + UInt32 * hash_data = &hash.getData()[0]; + + while (begin < end) + { + /// Note: it's copy-paste form intHashCRC32 adapted for updating previous hash result. +#ifdef __SSE4_2__ + *hash_data = _mm_crc32_u64(*hash_data, *begin); +#elif defined(__aarch64__) && defined(__ARM_FEATURE_CRC32) + *hash_data = __crc32cd(*hash_data, *begin); +#else + /// On other platforms we do not have CRC32. NOTE This can be confusing. + *hash_data = intHash64(*begin) ^ *hash_data; +#endif + ++begin; + ++hash_data; + } +} + template struct ColumnVector::less { diff --git a/dbms/src/Columns/ColumnVector.h b/dbms/src/Columns/ColumnVector.h index a157a184974..258e7a87252 100644 --- a/dbms/src/Columns/ColumnVector.h +++ b/dbms/src/Columns/ColumnVector.h @@ -160,6 +160,8 @@ public: void updateHashWithValue(size_t n, SipHash & hash) const override; + void updateWeakHash32(WeakHash32 & hash) const override; + size_t byteSize() const override { return data.size() * sizeof(data[0]); diff --git a/dbms/src/Columns/IColumn.h b/dbms/src/Columns/IColumn.h index 208fca7f6fc..090537d6770 100644 --- a/dbms/src/Columns/IColumn.h +++ b/dbms/src/Columns/IColumn.h @@ -23,6 +23,7 @@ namespace ErrorCodes class Arena; class ColumnGathererStream; class Field; +class WeakHash32; /// Declares interface to store columns in memory. class IColumn : public COW @@ -200,6 +201,11 @@ public: /// passed bytes to hash must identify sequence of values unambiguously. virtual void updateHashWithValue(size_t n, SipHash & hash) const = 0; + /// Update hash function value. Hash is calculated for each element. + /// It's a fast weak hash function. Mainly need to scatter data between threads. + /// WeakHash32 must have the same size as column. + virtual void updateWeakHash32(WeakHash32 & hash) const = 0; + /** Removes elements that don't match the filter. * Is used in WHERE and HAVING operations. * If result_size_hint > 0, then makes advance reserve(result_size_hint) for the result column; diff --git a/dbms/src/Common/WeakHash.cpp b/dbms/src/Common/WeakHash.cpp new file mode 100644 index 00000000000..54d973b6296 --- /dev/null +++ b/dbms/src/Common/WeakHash.cpp @@ -0,0 +1,2 @@ +#include + diff --git a/dbms/src/Common/WeakHash.h b/dbms/src/Common/WeakHash.h new file mode 100644 index 00000000000..22cb3ae515a --- /dev/null +++ b/dbms/src/Common/WeakHash.h @@ -0,0 +1,25 @@ +#pragma once +#include + +namespace DB +{ + +/// It's a class which represents the result of weak and fast hash function per row in column. It's usually crc32. +/// Has function result may be combined to calculate hash for tuples. +/// +/// The main purpose why this class needed is to support data initialization. Initially, every bit is 1. +class WeakHash32 +{ +public: + using Container = PaddedPODArray; + + explicit WeakHash32(size_t size) : data(size, ~UInt32(0)) {} + + const Container & getData() const { return data; } + Container & getData() { return data; } + +private: + PaddedPODArray data; +}; + +} From 21d9752277aae6573e94dbe96a7205d10fbafaae Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 13 Mar 2020 21:34:59 +0300 Subject: [PATCH 0075/1355] WeakHash for ColumnVector and ColumnString. --- dbms/src/Columns/ColumnString.cpp | 22 ++++++++++++++++++++++ dbms/src/Columns/ColumnVector.cpp | 10 +--------- dbms/src/Common/HashTable/Hash.h | 12 ++++++++++++ 3 files changed, 35 insertions(+), 9 deletions(-) diff --git a/dbms/src/Columns/ColumnString.cpp b/dbms/src/Columns/ColumnString.cpp index e1014008000..ce8326a1fb9 100644 --- a/dbms/src/Columns/ColumnString.cpp +++ b/dbms/src/Columns/ColumnString.cpp @@ -85,7 +85,29 @@ void ColumnString::updateWeakHash32(WeakHash32 & hash) const auto str_size = str_end - str_begin; + if (str_size < 8) + { + auto value = unalignedLoad(str_begin); + value &= (1ull << UInt64((8 * str_size))) - 1ull; + *hash_data = intHashCRC32(value, *hash_data); + } + else + { + /// Copy from StringRef.h + while (str_begin + 8 < str_end) + { + auto word = unalignedLoad(str_begin); + *hash_data = intHashCRC32(word, *hash_data); + + str_begin += 8; + } + + auto word = unalignedLoad(str_end - 8); + *hash_data = intHashCRC32(word, *hash_data); + } + ++offset_begin; + ++hash_data; str_begin = str_end; } } diff --git a/dbms/src/Columns/ColumnVector.cpp b/dbms/src/Columns/ColumnVector.cpp index 697da7e7002..a96a4605cc6 100644 --- a/dbms/src/Columns/ColumnVector.cpp +++ b/dbms/src/Columns/ColumnVector.cpp @@ -68,15 +68,7 @@ void ColumnVector::updateWeakHash32(WeakHash32 & hash) const while (begin < end) { - /// Note: it's copy-paste form intHashCRC32 adapted for updating previous hash result. -#ifdef __SSE4_2__ - *hash_data = _mm_crc32_u64(*hash_data, *begin); -#elif defined(__aarch64__) && defined(__ARM_FEATURE_CRC32) - *hash_data = __crc32cd(*hash_data, *begin); -#else - /// On other platforms we do not have CRC32. NOTE This can be confusing. - *hash_data = intHash64(*begin) ^ *hash_data; -#endif + *hash_data = intHashCRC32(*begin, *hash_data); ++begin; ++hash_data; } diff --git a/dbms/src/Common/HashTable/Hash.h b/dbms/src/Common/HashTable/Hash.h index befb660a968..7b4a8e0fd7a 100644 --- a/dbms/src/Common/HashTable/Hash.h +++ b/dbms/src/Common/HashTable/Hash.h @@ -58,6 +58,18 @@ inline DB::UInt64 intHashCRC32(DB::UInt64 x) #endif } +inline DB::UInt64 intHashCRC32(DB::UInt64 x, DB::UInt64 updated_value) +{ +#ifdef __SSE4_2__ + return _mm_crc32_u64(updated_value, x); +#elif defined(__aarch64__) && defined(__ARM_FEATURE_CRC32) + return __crc32cd(updated_value, x); +#else + /// On other platforms we do not have CRC32. NOTE This can be confusing. + return intHash64(x) ^ updated_value; +#endif +} + template inline size_t DefaultHash64(T key) From 25ba98d0aec23e2041d102333084d2ac3cecc7c2 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Mon, 16 Mar 2020 11:25:59 +0300 Subject: [PATCH 0076/1355] get rid of logs --- dbms/programs/copier/ZooKeeperStaff.h | 3 --- 1 file changed, 3 deletions(-) diff --git a/dbms/programs/copier/ZooKeeperStaff.h b/dbms/programs/copier/ZooKeeperStaff.h index e22df6bf199..2fc4d35400d 100644 --- a/dbms/programs/copier/ZooKeeperStaff.h +++ b/dbms/programs/copier/ZooKeeperStaff.h @@ -157,9 +157,6 @@ public: bool is_clean() const { - LOG_INFO(&Poco::Logger::get("ClusterCopier"), "!discovery_zxid.hasHappened() " << !discovery_zxid.hasHappened()); - LOG_INFO(&Poco::Logger::get("ClusterCopier"), "clean_state_zxid.hasHappened() " << clean_state_zxid.hasHappened()); - LOG_INFO(&Poco::Logger::get("ClusterCopier"), "discovery_zxid <= clean_state_zxid " << (discovery_zxid <= clean_state_zxid)); return !is_stale() && (!discovery_zxid.hasHappened() || (clean_state_zxid.hasHappened() && discovery_zxid <= clean_state_zxid)); } From d506f977869ac5c8d49bc60f22070c7f6a3f33b4 Mon Sep 17 00:00:00 2001 From: Guillaume Tassery Date: Mon, 16 Mar 2020 10:21:38 +0100 Subject: [PATCH 0077/1355] fix segfault --- dbms/src/Interpreters/ExternalLoader.cpp | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/dbms/src/Interpreters/ExternalLoader.cpp b/dbms/src/Interpreters/ExternalLoader.cpp index 4e18dc1bceb..3adbe6265ca 100644 --- a/dbms/src/Interpreters/ExternalLoader.cpp +++ b/dbms/src/Interpreters/ExternalLoader.cpp @@ -1032,8 +1032,10 @@ private: { Info * info = getInfo(name); if (info && (info->loading_id == loading_id)) + { info->loading_id = info->state_id; - + CurrentStatusInfo::set(CurrentStatusInfo::DictionaryStatus, name, static_cast(info->status())); + } min_id_to_finish_loading_dependencies.erase(std::this_thread::get_id()); auto it = loading_threads.find(loading_id); @@ -1042,7 +1044,6 @@ private: it->second.detach(); loading_threads.erase(it); } - CurrentStatusInfo::set(CurrentStatusInfo::DictionaryStatus, name, static_cast(info->status())); } /// Calculate next update time for loaded_object. Can be called without mutex locking, From 1f92b0a7a3fd5202ebc686d1210e253c2f6bd3e4 Mon Sep 17 00:00:00 2001 From: Guillaume Tassery Date: Mon, 16 Mar 2020 11:00:51 +0100 Subject: [PATCH 0078/1355] double quote string --- dbms/programs/server/PrometheusMetricsWriter.cpp | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/dbms/programs/server/PrometheusMetricsWriter.cpp b/dbms/programs/server/PrometheusMetricsWriter.cpp index 952141e9437..db57221af3d 100644 --- a/dbms/programs/server/PrometheusMetricsWriter.cpp +++ b/dbms/programs/server/PrometheusMetricsWriter.cpp @@ -4,6 +4,7 @@ #include #include +#include namespace { @@ -121,9 +122,9 @@ void PrometheusMetricsWriter::write(WriteBuffer & wb) const DB::writeChar('{', wb); DB::writeText(key, wb); DB::writeText("=\"", wb); - DB::writeText(enum_value.first, wb); + writeDoubleQuotedString(enum_value.first, wb); DB::writeText("\",name=\"", wb); - DB::writeText(value.first, wb); + writeDoubleQuotedString(value.first, wb); DB::writeText("\"} ", wb); DB::writeText(value.second == enum_value.second, wb); DB::writeChar('\n', wb); From 6c3200c75c50b7030bbd5ce7e9cb0498bdcf61cc Mon Sep 17 00:00:00 2001 From: Guillaume Tassery Date: Mon, 16 Mar 2020 11:32:23 +0100 Subject: [PATCH 0079/1355] test --- dbms/programs/server/PrometheusMetricsWriter.cpp | 2 +- dbms/src/Common/{CurrentStatusInfo.cpp => StatusInfo.cpp} | 3 +-- dbms/src/Common/{CurrentStatusInfo.h => StatusInfo.h} | 0 dbms/src/Databases/DatabaseWithDictionaries.cpp | 2 +- dbms/src/Interpreters/ExternalLoader.cpp | 2 +- 5 files changed, 4 insertions(+), 5 deletions(-) rename dbms/src/Common/{CurrentStatusInfo.cpp => StatusInfo.cpp} (97%) rename dbms/src/Common/{CurrentStatusInfo.h => StatusInfo.h} (100%) diff --git a/dbms/programs/server/PrometheusMetricsWriter.cpp b/dbms/programs/server/PrometheusMetricsWriter.cpp index db57221af3d..2bce7ac5abd 100644 --- a/dbms/programs/server/PrometheusMetricsWriter.cpp +++ b/dbms/programs/server/PrometheusMetricsWriter.cpp @@ -3,7 +3,7 @@ #include #include -#include +#include #include namespace diff --git a/dbms/src/Common/CurrentStatusInfo.cpp b/dbms/src/Common/StatusInfo.cpp similarity index 97% rename from dbms/src/Common/CurrentStatusInfo.cpp rename to dbms/src/Common/StatusInfo.cpp index 6019c69939f..bae35d7a41a 100644 --- a/dbms/src/Common/CurrentStatusInfo.cpp +++ b/dbms/src/Common/StatusInfo.cpp @@ -1,4 +1,4 @@ -#include +#include #include /// Available status. Add something here as you wish. @@ -16,7 +16,6 @@ namespace CurrentStatusInfo std::mutex locks[END] {}; std::unordered_map values[END] {}; - const char * getName(Status event) { static const char * strings[] = diff --git a/dbms/src/Common/CurrentStatusInfo.h b/dbms/src/Common/StatusInfo.h similarity index 100% rename from dbms/src/Common/CurrentStatusInfo.h rename to dbms/src/Common/StatusInfo.h diff --git a/dbms/src/Databases/DatabaseWithDictionaries.cpp b/dbms/src/Databases/DatabaseWithDictionaries.cpp index 4d5e4be2f89..7fbbf027333 100644 --- a/dbms/src/Databases/DatabaseWithDictionaries.cpp +++ b/dbms/src/Databases/DatabaseWithDictionaries.cpp @@ -1,5 +1,5 @@ #include -#include +#include #include #include #include diff --git a/dbms/src/Interpreters/ExternalLoader.cpp b/dbms/src/Interpreters/ExternalLoader.cpp index 3adbe6265ca..a6f078dcc0b 100644 --- a/dbms/src/Interpreters/ExternalLoader.cpp +++ b/dbms/src/Interpreters/ExternalLoader.cpp @@ -8,7 +8,7 @@ #include #include #include -#include +#include #include #include #include From b3c1115bc2790a43af6a755cc657e79d5eeee225 Mon Sep 17 00:00:00 2001 From: Guillaume Tassery Date: Mon, 16 Mar 2020 13:40:34 +0100 Subject: [PATCH 0080/1355] an another test --- dbms/src/Common/StatusInfo.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/dbms/src/Common/StatusInfo.cpp b/dbms/src/Common/StatusInfo.cpp index bae35d7a41a..351c9eb22b9 100644 --- a/dbms/src/Common/StatusInfo.cpp +++ b/dbms/src/Common/StatusInfo.cpp @@ -32,7 +32,7 @@ namespace CurrentStatusInfo { static const char * strings[] = { - #define M(NAME, DOCUMENTATION, ENUM) DOCUMENTATION, + #define M(NAME, DOCUMENTATION, ENUM) #DOCUMENTATION, APPLY_FOR_STATUS(M) #undef M }; @@ -44,7 +44,7 @@ namespace CurrentStatusInfo { static const std::vector> enum_values [] = { - #define M(NAME, DOCUMENTATION, ENUM) ENUM, + #define M(NAME, DOCUMENTATION, ENUM) #ENUM, APPLY_FOR_STATUS(M) #undef M }; From bbd6ff8466e1bc4dfbcf2f02338d9e86ab4e97ae Mon Sep 17 00:00:00 2001 From: Guillaume Tassery Date: Mon, 16 Mar 2020 14:35:22 +0100 Subject: [PATCH 0081/1355] test --- dbms/src/Common/StatusInfo.cpp | 4 ++-- dbms/src/Interpreters/ExternalLoader.cpp | 14 -------------- dbms/src/Interpreters/ExternalLoader.h | 15 +++++++++++++-- .../Storages/System/StorageSystemDictionaries.cpp | 2 +- dbms/src/Storages/System/StorageSystemModels.cpp | 2 +- 5 files changed, 17 insertions(+), 20 deletions(-) diff --git a/dbms/src/Common/StatusInfo.cpp b/dbms/src/Common/StatusInfo.cpp index 351c9eb22b9..be343acfe4a 100644 --- a/dbms/src/Common/StatusInfo.cpp +++ b/dbms/src/Common/StatusInfo.cpp @@ -3,7 +3,7 @@ /// Available status. Add something here as you wish. #define APPLY_FOR_STATUS(M) \ - M(DictionaryStatus, "Dictionary Status.", DB::ExternalLoader::getStatusEnumAllPossibleValues()) \ + M(DictionaryStatus, "Dictionary Status.", DB::getStatusEnumAllPossibleValues()) \ namespace CurrentStatusInfo @@ -44,7 +44,7 @@ namespace CurrentStatusInfo { static const std::vector> enum_values [] = { - #define M(NAME, DOCUMENTATION, ENUM) #ENUM, + #define M(NAME, DOCUMENTATION, ENUM) ENUM, APPLY_FOR_STATUS(M) #undef M }; diff --git a/dbms/src/Interpreters/ExternalLoader.cpp b/dbms/src/Interpreters/ExternalLoader.cpp index a6f078dcc0b..90817194b45 100644 --- a/dbms/src/Interpreters/ExternalLoader.cpp +++ b/dbms/src/Interpreters/ExternalLoader.cpp @@ -1362,20 +1362,6 @@ ExternalLoader::LoadablePtr ExternalLoader::createObject( return create(name, *config.config, config.key_in_config, config.repository_name); } -std::vector> ExternalLoader::getStatusEnumAllPossibleValues() -{ - return std::vector>{ - {toString(Status::NOT_LOADED), static_cast(Status::NOT_LOADED)}, - {toString(Status::LOADED), static_cast(Status::LOADED)}, - {toString(Status::FAILED), static_cast(Status::FAILED)}, - {toString(Status::LOADING), static_cast(Status::LOADING)}, - {toString(Status::LOADED_AND_RELOADING), static_cast(Status::LOADED_AND_RELOADING)}, - {toString(Status::FAILED_AND_RELOADING), static_cast(Status::FAILED_AND_RELOADING)}, - {toString(Status::NOT_EXIST), static_cast(Status::NOT_EXIST)}, - }; -} - - String toString(ExternalLoader::Status status) { using Status = ExternalLoader::Status; diff --git a/dbms/src/Interpreters/ExternalLoader.h b/dbms/src/Interpreters/ExternalLoader.h index ff6b06a2029..0d80a5ee137 100644 --- a/dbms/src/Interpreters/ExternalLoader.h +++ b/dbms/src/Interpreters/ExternalLoader.h @@ -59,8 +59,6 @@ public: NOT_EXIST, /// Object with this name wasn't found in the configuration. }; - static std::vector> getStatusEnumAllPossibleValues(); - using Duration = std::chrono::milliseconds; using TimePoint = std::chrono::system_clock::time_point; @@ -230,4 +228,17 @@ private: String toString(ExternalLoader::Status status); std::ostream & operator<<(std::ostream & out, ExternalLoader::Status status); +std::vector> getStatusEnumAllPossibleValues() +{ + return std::vector>{ + {toString(ExternalLoader::Status::NOT_LOADED), static_cast(ExternalLoader::Status::NOT_LOADED)}, + {toString(ExternalLoader::Status::LOADED), static_cast(ExternalLoader::Status::LOADED)}, + {toString(ExternalLoader::Status::FAILED), static_cast(ExternalLoader::Status::FAILED)}, + {toString(ExternalLoader::Status::LOADING), static_cast(ExternalLoader::Status::LOADING)}, + {toString(ExternalLoader::Status::LOADED_AND_RELOADING), static_cast(ExternalLoader::Status::LOADED_AND_RELOADING)}, + {toString(ExternalLoader::Status::FAILED_AND_RELOADING), static_cast(ExternalLoader::Status::FAILED_AND_RELOADING)}, + {toString(ExternalLoader::Status::NOT_EXIST), static_cast(ExternalLoader::Status::NOT_EXIST)}, + }; +}; + } diff --git a/dbms/src/Storages/System/StorageSystemDictionaries.cpp b/dbms/src/Storages/System/StorageSystemDictionaries.cpp index 87a11387e4d..625585aa5e0 100644 --- a/dbms/src/Storages/System/StorageSystemDictionaries.cpp +++ b/dbms/src/Storages/System/StorageSystemDictionaries.cpp @@ -25,7 +25,7 @@ NamesAndTypesList StorageSystemDictionaries::getNamesAndTypes() return { {"database", std::make_shared()}, {"name", std::make_shared()}, - {"status", std::make_shared(ExternalLoader::getStatusEnumAllPossibleValues())}, + {"status", std::make_shared(getStatusEnumAllPossibleValues())}, {"origin", std::make_shared()}, {"type", std::make_shared()}, {"key", std::make_shared()}, diff --git a/dbms/src/Storages/System/StorageSystemModels.cpp b/dbms/src/Storages/System/StorageSystemModels.cpp index 67594b8692d..306829cf6de 100644 --- a/dbms/src/Storages/System/StorageSystemModels.cpp +++ b/dbms/src/Storages/System/StorageSystemModels.cpp @@ -15,7 +15,7 @@ NamesAndTypesList StorageSystemModels::getNamesAndTypes() { return { { "name", std::make_shared() }, - { "status", std::make_shared(ExternalLoader::getStatusEnumAllPossibleValues()) }, + { "status", std::make_shared(getStatusEnumAllPossibleValues()) }, { "origin", std::make_shared() }, { "type", std::make_shared() }, { "loading_start_time", std::make_shared() }, From e350baf58f00cb1f181dd070d97c87dd2bc557dc Mon Sep 17 00:00:00 2001 From: Guillaume Tassery Date: Mon, 16 Mar 2020 16:26:23 +0100 Subject: [PATCH 0082/1355] test --- dbms/src/Interpreters/ExternalLoader.cpp | 12 ++++++++++++ dbms/src/Interpreters/ExternalLoader.h | 14 +------------- 2 files changed, 13 insertions(+), 13 deletions(-) diff --git a/dbms/src/Interpreters/ExternalLoader.cpp b/dbms/src/Interpreters/ExternalLoader.cpp index 90817194b45..21b32996af2 100644 --- a/dbms/src/Interpreters/ExternalLoader.cpp +++ b/dbms/src/Interpreters/ExternalLoader.cpp @@ -1378,6 +1378,18 @@ String toString(ExternalLoader::Status status) __builtin_unreachable(); } +std::vector> getStatusEnumAllPossibleValues() +{ + return std::vector>{ + {toString(ExternalLoader::Status::NOT_LOADED), static_cast(ExternalLoader::Status::NOT_LOADED)}, + {toString(ExternalLoader::Status::LOADED), static_cast(ExternalLoader::Status::LOADED)}, + {toString(ExternalLoader::Status::FAILED), static_cast(ExternalLoader::Status::FAILED)}, + {toString(ExternalLoader::Status::LOADING), static_cast(ExternalLoader::Status::LOADING)}, + {toString(ExternalLoader::Status::LOADED_AND_RELOADING), static_cast(ExternalLoader::Status::LOADED_AND_RELOADING)}, + {toString(ExternalLoader::Status::FAILED_AND_RELOADING), static_cast(ExternalLoader::Status::FAILED_AND_RELOADING)}, + {toString(ExternalLoader::Status::NOT_EXIST), static_cast(ExternalLoader::Status::NOT_EXIST)}, + }; +}; std::ostream & operator<<(std::ostream & out, ExternalLoader::Status status) { diff --git a/dbms/src/Interpreters/ExternalLoader.h b/dbms/src/Interpreters/ExternalLoader.h index 0d80a5ee137..64660c26940 100644 --- a/dbms/src/Interpreters/ExternalLoader.h +++ b/dbms/src/Interpreters/ExternalLoader.h @@ -227,18 +227,6 @@ private: String toString(ExternalLoader::Status status); std::ostream & operator<<(std::ostream & out, ExternalLoader::Status status); - -std::vector> getStatusEnumAllPossibleValues() -{ - return std::vector>{ - {toString(ExternalLoader::Status::NOT_LOADED), static_cast(ExternalLoader::Status::NOT_LOADED)}, - {toString(ExternalLoader::Status::LOADED), static_cast(ExternalLoader::Status::LOADED)}, - {toString(ExternalLoader::Status::FAILED), static_cast(ExternalLoader::Status::FAILED)}, - {toString(ExternalLoader::Status::LOADING), static_cast(ExternalLoader::Status::LOADING)}, - {toString(ExternalLoader::Status::LOADED_AND_RELOADING), static_cast(ExternalLoader::Status::LOADED_AND_RELOADING)}, - {toString(ExternalLoader::Status::FAILED_AND_RELOADING), static_cast(ExternalLoader::Status::FAILED_AND_RELOADING)}, - {toString(ExternalLoader::Status::NOT_EXIST), static_cast(ExternalLoader::Status::NOT_EXIST)}, - }; -}; +std::vector> getStatusEnumAllPossibleValues(); } From d2179339488f682dae6200ff5403a6ee2bf7c9fb Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Tue, 17 Mar 2020 00:05:38 +0300 Subject: [PATCH 0083/1355] add more tests --- dbms/programs/copier/ClusterCopier.cpp | 169 ++++++++++-------- dbms/programs/copier/ClusterCopier.h | 8 +- dbms/programs/copier/ClusterCopierApp.cpp | 5 + dbms/programs/copier/ClusterCopierApp.h | 3 +- .../configs/conf.d/clusters.xml | 19 ++ .../test_cluster_copier/task_trivial.xml | 64 +++++++ .../integration/test_cluster_copier/test.py | 13 +- .../test_cluster_copier/trivial_test.py | 144 +++++++++++++++ 8 files changed, 348 insertions(+), 77 deletions(-) create mode 100644 dbms/tests/integration/test_cluster_copier/task_trivial.xml create mode 100644 dbms/tests/integration/test_cluster_copier/trivial_test.py diff --git a/dbms/programs/copier/ClusterCopier.cpp b/dbms/programs/copier/ClusterCopier.cpp index 88590205850..b1c284122a4 100644 --- a/dbms/programs/copier/ClusterCopier.cpp +++ b/dbms/programs/copier/ClusterCopier.cpp @@ -544,6 +544,13 @@ bool ClusterCopier::checkPartitionPieceIsDone(const TaskTable & task_table, cons PartitionTaskStatus ClusterCopier::tryMoveAllPiecesToDestinationTable(const TaskTable & task_table, const String & partition_name) { + bool inject_fault = false; + if (move_fault_probability > 0) + { + double value = std::uniform_real_distribution<>(0, 1)(task_table.task_cluster.random_engine); + inject_fault = value < move_fault_probability; + } + LOG_DEBUG(log, "Try to move " << partition_name << " to destionation table"); auto zookeeper = context.getZooKeeper(); @@ -599,85 +606,90 @@ PartitionTaskStatus ClusterCopier::tryMoveAllPiecesToDestinationTable(const Task zookeeper->create(current_partition_attach_is_done, start_state, zkutil::CreateMode::Persistent); } + /// Move partition to original destination table. for (size_t current_piece_number = 0; current_piece_number < task_table.number_of_splits; ++current_piece_number) { - /// Move partition to original destination table. + /// TODO: Execute alter table move partition. + + LOG_DEBUG(log, "Trying to move partition " << partition_name + << " piece " << toString(current_piece_number) + << " to original table"); + + ASTPtr query_alter_ast; + String query_alter_ast_string; + + DatabaseAndTableName original_table = task_table.table_push; + DatabaseAndTableName helping_table = DatabaseAndTableName(original_table.first, + original_table.second + "_piece_" + + toString(current_piece_number)); + + query_alter_ast_string += " ALTER TABLE " + getQuotedTable(original_table) + + " ATTACH PARTITION " + partition_name + + " FROM " + getQuotedTable(helping_table) + + " SETTINGS replication_alter_partitions_sync=2;"; + + LOG_DEBUG(log, "Executing ALTER query: " << query_alter_ast_string); + + try { - /// TODO: Execute alter table move partition. + size_t num_nodes = 0; - LOG_DEBUG(log, "Trying to move partition " << partition_name - << " piece " << toString(current_piece_number) << " to original table"); + ///FIXME: We have to be sure that every node in cluster executed this query + size_t current_num_nodes = executeQueryOnCluster( + task_table.cluster_push, + query_alter_ast_string, + nullptr, + &task_cluster->settings_push, + PoolMode::GET_MANY, + ClusterExecutionMode::ON_EACH_NODE); - ASTPtr query_alter_ast; - String query_alter_ast_string; - - DatabaseAndTableName original_table = task_table.table_push; - DatabaseAndTableName helping_table = DatabaseAndTableName(original_table.first, original_table.second + "_piece_" + toString(current_piece_number)); - - query_alter_ast_string += " ALTER TABLE " + getQuotedTable(original_table) + - " ATTACH PARTITION " + partition_name + - " FROM " + getQuotedTable(helping_table) + - " SETTINGS replication_alter_partitions_sync=2;"; - - LOG_DEBUG(log, "Executing ALTER query: " << query_alter_ast_string); - - try - { - size_t num_nodes = 0; - - for (UInt64 try_num = 0; try_num < max_shard_partition_piece_tries_for_alter; ++try_num) - { - ///FIXME: We have to be sure that every node in cluster executed this query - size_t current_num_nodes = executeQueryOnCluster( - task_table.cluster_push, - query_alter_ast_string, - nullptr, - &task_cluster->settings_push, - PoolMode::GET_MANY, - ClusterExecutionMode::ON_EACH_NODE); - - num_nodes = std::max(current_num_nodes, num_nodes); - } - - LOG_INFO(log, "Number of nodes that executed ALTER query successfully : " << toString(num_nodes)); - } - catch (...) - { - LOG_DEBUG(log, "Error while moving partition " << partition_name - << " piece " << toString(current_piece_number) << "to original table"); - throw; - } + num_nodes = std::max(current_num_nodes, num_nodes); - try - { - String query_deduplicate_ast_string; - if (!task_table.isReplicatedTable()) - { - query_deduplicate_ast_string += " OPTIMIZE TABLE " + getQuotedTable(original_table) + - " PARTITION " + partition_name + " DEDUPLICATE;"; + LOG_INFO(log, "Number of nodes that executed ALTER query successfully : " << toString(num_nodes)); + } + catch (...) + { + LOG_DEBUG(log, "Error while moving partition " << partition_name + << " piece " << toString(current_piece_number) + << "to original table"); + throw; + } - LOG_DEBUG(log, "Executing OPTIMIZE DEDUPLICATE query: " << query_alter_ast_string); + if (inject_fault) + throw Exception("Copy fault injection is activated", ErrorCodes::UNFINISHED); - UInt64 num_nodes = executeQueryOnCluster( - task_table.cluster_push, - query_deduplicate_ast_string, - nullptr, - &task_cluster->settings_push, - PoolMode::GET_MANY); + try + { + String query_deduplicate_ast_string; + if (!task_table.isReplicatedTable()) { + query_deduplicate_ast_string += " OPTIMIZE TABLE " + getQuotedTable(original_table) + + " PARTITION " + partition_name + " DEDUPLICATE;"; - LOG_INFO(log, "Number of shard that executed OPTIMIZE DEDUPLICATE query successfully : " << toString(num_nodes)); - } - } - catch(...) - { - LOG_DEBUG(log, "Error while executing OPTIMIZE DEDUPLICATE partition " << partition_name << "in the original table"); - throw; + LOG_DEBUG(log, "Executing OPTIMIZE DEDUPLICATE query: " << query_alter_ast_string); + + UInt64 num_nodes = executeQueryOnCluster( + task_table.cluster_push, + query_deduplicate_ast_string, + nullptr, + &task_cluster->settings_push, + PoolMode::GET_MANY); + + LOG_INFO(log, "Number of shard that executed OPTIMIZE DEDUPLICATE query successfully : " + << toString(num_nodes)); } } + catch (...) + { + LOG_DEBUG(log, "Error while executing OPTIMIZE DEDUPLICATE partition " << partition_name + << "in the original table"); + throw; + } } + + /// Create node to signal that we finished moving { String state_finished = TaskStateWithOwner::getData(TaskState::Finished, host_id); @@ -1011,15 +1023,28 @@ bool ClusterCopier::tryProcessTable(const ConnectionTimeouts & timeouts, TaskTab /// Try to move only if all pieces were copied. if (partition_copying_is_done) { - try + for (UInt64 try_num = 0; try_num < max_shard_partition_piece_tries_for_alter; ++try_num) { - auto res = tryMoveAllPiecesToDestinationTable(task_table, partition_name); - if (res == PartitionTaskStatus::Finished) - partition_moving_is_done = true; - } - catch (...) - { - tryLogCurrentException(log, "Some error occured while moving pieces to destination table for partition " + partition_name); + try + { + auto res = tryMoveAllPiecesToDestinationTable(task_table, partition_name); + if (res == PartitionTaskStatus::Finished) + { + partition_moving_is_done = true; + break; + } + + /// Sleep if this task is active + if (res == PartitionTaskStatus::Active) + std::this_thread::sleep_for(default_sleep_time); + + /// Repeat on errors + } + catch (...) { + tryLogCurrentException(log, + "Some error occured while moving pieces to destination table for partition " + + partition_name); + } } } diff --git a/dbms/programs/copier/ClusterCopier.h b/dbms/programs/copier/ClusterCopier.h index ce9ea7cc7ef..8558a2859e3 100644 --- a/dbms/programs/copier/ClusterCopier.h +++ b/dbms/programs/copier/ClusterCopier.h @@ -56,6 +56,11 @@ public: copy_fault_probability = copy_fault_probability_; } + void setMoveFaultProbability(double move_fault_probability_) + { + move_fault_probability = move_fault_probability_; + } + protected: String getWorkersPath() const @@ -120,7 +125,7 @@ protected: static constexpr UInt64 max_table_tries = 1000; static constexpr UInt64 max_shard_partition_tries = 600; - static constexpr UInt64 max_shard_partition_piece_tries_for_alter = 5; + static constexpr UInt64 max_shard_partition_piece_tries_for_alter = 10; bool tryProcessTable(const ConnectionTimeouts & timeouts, TaskTable & task_table); @@ -201,6 +206,7 @@ private: bool is_safe_mode = false; double copy_fault_probability = 0.0; + double move_fault_probability = 0.0; Context & context; Poco::Logger * log; diff --git a/dbms/programs/copier/ClusterCopierApp.cpp b/dbms/programs/copier/ClusterCopierApp.cpp index fe5b7f72588..37b66b0fb73 100644 --- a/dbms/programs/copier/ClusterCopierApp.cpp +++ b/dbms/programs/copier/ClusterCopierApp.cpp @@ -17,6 +17,8 @@ void ClusterCopierApp::initialize(Poco::Util::Application & self) is_safe_mode = config().has("safe-mode"); if (config().has("copy-fault-probability")) copy_fault_probability = std::max(std::min(config().getDouble("copy-fault-probability"), 1.0), 0.0); + if (config().has("move-fault-probability")) + move_fault_probability = std::max(std::min(config().getDouble("move-fault-probability"), 1.0), 0.0); base_dir = (config().has("base-dir")) ? config().getString("base-dir") : Poco::Path::current(); // process_id is '#_' time_t timestamp = Poco::Timestamp().epochTime(); @@ -67,6 +69,8 @@ void ClusterCopierApp::defineOptions(Poco::Util::OptionSet & options) .binding("safe-mode")); options.addOption(Poco::Util::Option("copy-fault-probability", "", "the copying fails with specified probability (used to test partition state recovering)") .argument("copy-fault-probability").binding("copy-fault-probability")); + options.addOption(Poco::Util::Option("move-fault-probability", "", "the moving fails with specified probability (used to test partition state recovering)") + .argument("move-fault-probability").binding("move-fault-probability")); options.addOption(Poco::Util::Option("log-level", "", "sets log level") .argument("log-level").binding("log-level")); options.addOption(Poco::Util::Option("base-dir", "", "base directory for copiers, consecutive copier launches will populate /base-dir/launch_id/* directories") @@ -115,6 +119,7 @@ void ClusterCopierApp::mainImpl() auto copier = std::make_unique(task_path, host_id, default_database, *context); copier->setSafeMode(is_safe_mode); copier->setCopyFaultProbability(copy_fault_probability); + copier->setMoveFaultProbability(move_fault_probability); auto task_file = config().getString("task-file", ""); if (!task_file.empty()) diff --git a/dbms/programs/copier/ClusterCopierApp.h b/dbms/programs/copier/ClusterCopierApp.h index 25350aefd3a..8cadd9d5dff 100644 --- a/dbms/programs/copier/ClusterCopierApp.h +++ b/dbms/programs/copier/ClusterCopierApp.h @@ -78,7 +78,8 @@ private: std::string task_path; std::string log_level = "trace"; bool is_safe_mode = false; - double copy_fault_probability = 0; + double copy_fault_probability = 0.0; + double move_fault_probability = 0.0; bool is_help = false; std::string base_dir; diff --git a/dbms/tests/integration/test_cluster_copier/configs/conf.d/clusters.xml b/dbms/tests/integration/test_cluster_copier/configs/conf.d/clusters.xml index f00cf1cf351..54a8822fa98 100644 --- a/dbms/tests/integration/test_cluster_copier/configs/conf.d/clusters.xml +++ b/dbms/tests/integration/test_cluster_copier/configs/conf.d/clusters.xml @@ -57,5 +57,24 @@ + + + + s0_0_0 + 9000 + + + + + + + + + s1_0_0 + 9000 + + + + diff --git a/dbms/tests/integration/test_cluster_copier/task_trivial.xml b/dbms/tests/integration/test_cluster_copier/task_trivial.xml new file mode 100644 index 00000000000..c23b9322470 --- /dev/null +++ b/dbms/tests/integration/test_cluster_copier/task_trivial.xml @@ -0,0 +1,64 @@ + + + + 3 + + + + 1 + + + + + 0 + + + + + + + + + + source_trivial_cluster + default + trivial + + destination_trivial_cluster + default + trivial + + + ENGINE=ReplicatedMergeTree('/clickhouse/tables/cluster{cluster}/{shard}/hits', '{replica}') PARTITION BY d % 5 ORDER BY d SETTINGS index_granularity = 16 + + + d + 1 + + + d - d = 0 + + + + + + + + + s0_0_0 + 9000 + + + + + + + + + s1_0_0 + 9000 + + + + + + \ No newline at end of file diff --git a/dbms/tests/integration/test_cluster_copier/test.py b/dbms/tests/integration/test_cluster_copier/test.py index 47df976bbcf..d411eba3974 100644 --- a/dbms/tests/integration/test_cluster_copier/test.py +++ b/dbms/tests/integration/test_cluster_copier/test.py @@ -15,6 +15,7 @@ from helpers.cluster import ClickHouseCluster from helpers.test_tools import TSV COPYING_FAIL_PROBABILITY = 0.33 +MOVING_FAIL_PROBABILITY = 0.33 cluster = None @@ -259,17 +260,23 @@ def execute_task(task, cmd_options): # Tests -def test_copy1_simple(started_cluster): +def test_copy_simple(started_cluster): execute_task(Task1(started_cluster), []) -def test_copy1_with_recovering(started_cluster): +def test_copy_with_recovering(started_cluster): execute_task(Task1(started_cluster), ['--copy-fault-probability', str(COPYING_FAIL_PROBABILITY)]) +def test_copy_with_recovering_after_move_faults(started_cluster): + execute_task(Task1(started_cluster), ['--move-fault-probability', str(MOVING_FAIL_PROBABILITY)]) + def test_copy_month_to_week_partition(started_cluster): execute_task(Task2(started_cluster), []) def test_copy_month_to_week_partition_with_recovering(started_cluster): - execute_task(Task2(started_cluster), ['--copy-fault-probability', str(0.3)]) + execute_task(Task2(started_cluster), ['--copy-fault-probability', str(COPYING_FAIL_PROBABILITY)]) + +def test_copy_month_to_week_partition_with_recovering_after_move_faults(started_cluster): + execute_task(Task2(started_cluster), ['--move-fault-probability', str(MOVING_FAIL_PROBABILITY)]) def test_block_size(started_cluster): execute_task(Task_test_block_size(started_cluster), []) diff --git a/dbms/tests/integration/test_cluster_copier/trivial_test.py b/dbms/tests/integration/test_cluster_copier/trivial_test.py new file mode 100644 index 00000000000..aa204825462 --- /dev/null +++ b/dbms/tests/integration/test_cluster_copier/trivial_test.py @@ -0,0 +1,144 @@ +import os +import os.path as p +import sys +import time +import datetime +import pytest +from contextlib import contextmanager +import docker +from kazoo.client import KazooClient + + +CURRENT_TEST_DIR = os.path.dirname(os.path.abspath(__file__)) +sys.path.insert(0, os.path.dirname(CURRENT_TEST_DIR)) +from helpers.cluster import ClickHouseCluster +from helpers.test_tools import TSV + +COPYING_FAIL_PROBABILITY = 0.33 +MOVING_FAIL_PROBABILITY = 0.1 +cluster = None + +@pytest.fixture(scope="module") +def started_cluster(): + global cluster + try: + clusters_schema = { + "0" : {"0" : ["0"]}, + "1" : {"0" : ["0"]} + } + + cluster = ClickHouseCluster(__file__) + + for cluster_name, shards in clusters_schema.iteritems(): + for shard_name, replicas in shards.iteritems(): + for replica_name in replicas: + name = "s{}_{}_{}".format(cluster_name, shard_name, replica_name) + cluster.add_instance(name, + config_dir="configs", + macros={"cluster": cluster_name, "shard": shard_name, "replica": replica_name}, + with_zookeeper=True) + + cluster.start() + yield cluster + + finally: + pass + cluster.shutdown() + + +class TaskTrivial: + def __init__(self, cluster): + self.cluster = cluster + self.zk_task_path="/clickhouse-copier/task_trivial" + self.copier_task_config = open(os.path.join(CURRENT_TEST_DIR, 'task_trivial.xml'), 'r').read() + + + def start(self): + source = cluster.instances['s0_0_0'] + destination = cluster.instances['s1_0_0'] + + for node in [source, destination]: + node.query("DROP DATABASE IF EXISTS default") + node.query("CREATE DATABASE IF NOT EXISTS default") + + source.query("CREATE TABLE trivial (d UInt64, d1 UInt64 MATERIALIZED d+1) " + "ENGINE=ReplicatedMergeTree('/clickhouse/tables/source_trivial_cluster/1/trivial', '1') " + "PARTITION BY d % 5 ORDER BY d SETTINGS index_granularity = 16") + + source.query("INSERT INTO trivial SELECT * FROM system.numbers LIMIT 1002", settings={"insert_distributed_sync": 1}) + + + def check(self): + source = cluster.instances['s0_0_0'] + destination = cluster.instances['s1_0_0'] + + assert TSV(source.query("SELECT count() FROM trivial")) == TSV("1002\n") + assert TSV(destination.query("SELECT count() FROM trivial")) == TSV("1002\n") + + for node in [source, destination]: + node.query("DROP TABLE trivial") + + +def execute_task(task, cmd_options): + task.start() + + zk = cluster.get_kazoo_client('zoo1') + print "Use ZooKeeper server: {}:{}".format(zk.hosts[0][0], zk.hosts[0][1]) + + zk_task_path = task.zk_task_path + zk.ensure_path(zk_task_path) + zk.create(zk_task_path + "/description", task.copier_task_config) + + # Run cluster-copier processes on each node + docker_api = docker.from_env().api + copiers_exec_ids = [] + + cmd = ['/usr/bin/clickhouse', 'copier', + '--config', '/etc/clickhouse-server/config-copier.xml', + '--task-path', zk_task_path, + '--base-dir', '/var/log/clickhouse-server/copier'] + cmd += cmd_options + + print(cmd) + + for instance_name, instance in cluster.instances.iteritems(): + container = instance.get_docker_handle() + exec_id = docker_api.exec_create(container.id, cmd, stderr=True) + docker_api.exec_start(exec_id, detach=True) + + copiers_exec_ids.append(exec_id) + print "Copier for {} ({}) has started".format(instance.name, instance.ip_address) + + # Wait for copiers stopping and check their return codes + for exec_id, instance in zip(copiers_exec_ids, cluster.instances.itervalues()): + while True: + res = docker_api.exec_inspect(exec_id) + if not res['Running']: + break + time.sleep(1) + + assert res['ExitCode'] == 0, "Instance: {} ({}). Info: {}".format(instance.name, instance.ip_address, repr(res)) + + try: + task.check() + finally: + zk.delete(zk_task_path, recursive=True) + + +# Tests + +def test_trivial_copy(started_cluster): + execute_task(TaskTrivial(started_cluster), []) + +def test_trivial_copy_with_copy_fault(started_cluster): + execute_task(TaskTrivial(started_cluster), ['--copy-fault-probability', str(0.1)]) + +def test_trivial_copy_with_move_fault(started_cluster): + execute_task(TaskTrivial(started_cluster), ['--move-fault-probability', str(0.1)]) + + +if __name__ == '__main__': + with contextmanager(started_cluster)() as cluster: + for name, instance in cluster.instances.items(): + print name, instance.ip_address + raw_input("Cluster created, press any key to destroy...") \ No newline at end of file From 57fc1b1a9b9894c5b902fbd1836c226246657dfc Mon Sep 17 00:00:00 2001 From: Guillaume Tassery Date: Tue, 17 Mar 2020 10:24:24 +0100 Subject: [PATCH 0084/1355] test --- dbms/src/Common/ExternalLoaderStatus.h | 50 +++++++++++++++++++ .../Databases/DatabaseWithDictionaries.cpp | 3 +- dbms/src/Interpreters/ExternalLoader.cpp | 35 ------------- dbms/src/Interpreters/ExternalLoader.h | 17 +------ 4 files changed, 54 insertions(+), 51 deletions(-) create mode 100644 dbms/src/Common/ExternalLoaderStatus.h diff --git a/dbms/src/Common/ExternalLoaderStatus.h b/dbms/src/Common/ExternalLoaderStatus.h new file mode 100644 index 00000000000..2478215d016 --- /dev/null +++ b/dbms/src/Common/ExternalLoaderStatus.h @@ -0,0 +1,50 @@ +#pragma once + +namespace DB +{ + enum class ExternalLoaderStatus + { + NOT_LOADED, /// Object hasn't been tried to load. This is an initial state. + LOADED, /// Object has been loaded successfully. + FAILED, /// Object has been failed to load. + LOADING, /// Object is being loaded right now for the first time. + FAILED_AND_RELOADING, /// Object was failed to load before and it's being reloaded right now. + LOADED_AND_RELOADING, /// Object was loaded successfully before and it's being reloaded right now. + NOT_EXIST, /// Object with this name wasn't found in the configuration. + }; + + String toString(ExternalLoaderStatus status) + { + using Status = ExternalLoaderStatus; + switch (status) + { + case Status::NOT_LOADED: return "NOT_LOADED"; + case Status::LOADED: return "LOADED"; + case Status::FAILED: return "FAILED"; + case Status::LOADING: return "LOADING"; + case Status::FAILED_AND_RELOADING: return "FAILED_AND_RELOADING"; + case Status::LOADED_AND_RELOADING: return "LOADED_AND_RELOADING"; + case Status::NOT_EXIST: return "NOT_EXIST"; + } + __builtin_unreachable(); + } + + std::vector> getStatusEnumAllPossibleValues() + { + using Status = ExternalLoaderStatus; + return std::vector>{ + {toString(Status::NOT_LOADED), static_cast(Status::NOT_LOADED)}, + {toString(Status::LOADED), static_cast(Status::LOADED)}, + {toString(Status::FAILED), static_cast(Status::FAILED)}, + {toString(Status::LOADING), static_cast(Status::LOADING)}, + {toString(Status::LOADED_AND_RELOADING), static_cast(Status::LOADED_AND_RELOADING)}, + {toString(Status::FAILED_AND_RELOADING), static_cast(Status::FAILED_AND_RELOADING)}, + {toString(Status::NOT_EXIST), static_cast(Status::NOT_EXIST)}, + }; + } + + std::ostream & operator<<(std::ostream & out, ExternalLoaderStatus status) + { + return out << toString(status); + } +} diff --git a/dbms/src/Databases/DatabaseWithDictionaries.cpp b/dbms/src/Databases/DatabaseWithDictionaries.cpp index 7fbbf027333..e849962aae3 100644 --- a/dbms/src/Databases/DatabaseWithDictionaries.cpp +++ b/dbms/src/Databases/DatabaseWithDictionaries.cpp @@ -1,5 +1,6 @@ #include #include +#include #include #include #include @@ -37,7 +38,7 @@ void DatabaseWithDictionaries::attachDictionary(const String & dictionary_name, throw Exception("Dictionary " + full_name + " already exists.", ErrorCodes::DICTIONARY_ALREADY_EXISTS); } - CurrentStatusInfo::set(CurrentStatusInfo::DictionaryStatus, full_name, static_cast(ExternalLoader::Status::NOT_LOADED)); + CurrentStatusInfo::set(CurrentStatusInfo::DictionaryStatus, full_name, static_cast(ExternalLoaderStatus::NOT_LOADED)); /// ExternalLoader::reloadConfig() will find out that the dictionary's config has been added /// and in case `dictionaries_lazy_load == false` it will load the dictionary. const auto & external_loader = context.getExternalDictionariesLoader(); diff --git a/dbms/src/Interpreters/ExternalLoader.cpp b/dbms/src/Interpreters/ExternalLoader.cpp index 21b32996af2..07e871099a1 100644 --- a/dbms/src/Interpreters/ExternalLoader.cpp +++ b/dbms/src/Interpreters/ExternalLoader.cpp @@ -1362,41 +1362,6 @@ ExternalLoader::LoadablePtr ExternalLoader::createObject( return create(name, *config.config, config.key_in_config, config.repository_name); } -String toString(ExternalLoader::Status status) -{ - using Status = ExternalLoader::Status; - switch (status) - { - case Status::NOT_LOADED: return "NOT_LOADED"; - case Status::LOADED: return "LOADED"; - case Status::FAILED: return "FAILED"; - case Status::LOADING: return "LOADING"; - case Status::FAILED_AND_RELOADING: return "FAILED_AND_RELOADING"; - case Status::LOADED_AND_RELOADING: return "LOADED_AND_RELOADING"; - case Status::NOT_EXIST: return "NOT_EXIST"; - } - __builtin_unreachable(); -} - -std::vector> getStatusEnumAllPossibleValues() -{ - return std::vector>{ - {toString(ExternalLoader::Status::NOT_LOADED), static_cast(ExternalLoader::Status::NOT_LOADED)}, - {toString(ExternalLoader::Status::LOADED), static_cast(ExternalLoader::Status::LOADED)}, - {toString(ExternalLoader::Status::FAILED), static_cast(ExternalLoader::Status::FAILED)}, - {toString(ExternalLoader::Status::LOADING), static_cast(ExternalLoader::Status::LOADING)}, - {toString(ExternalLoader::Status::LOADED_AND_RELOADING), static_cast(ExternalLoader::Status::LOADED_AND_RELOADING)}, - {toString(ExternalLoader::Status::FAILED_AND_RELOADING), static_cast(ExternalLoader::Status::FAILED_AND_RELOADING)}, - {toString(ExternalLoader::Status::NOT_EXIST), static_cast(ExternalLoader::Status::NOT_EXIST)}, - }; -}; - -std::ostream & operator<<(std::ostream & out, ExternalLoader::Status status) -{ - return out << toString(status); -} - - template ExternalLoader::LoadablePtr ExternalLoader::getCurrentLoadResult(const String &) const; template ExternalLoader::LoadResult ExternalLoader::getCurrentLoadResult(const String &) const; template ExternalLoader::Loadables ExternalLoader::getCurrentLoadResults(const FilterByNameFunction &) const; diff --git a/dbms/src/Interpreters/ExternalLoader.h b/dbms/src/Interpreters/ExternalLoader.h index 64660c26940..a9a94ca615e 100644 --- a/dbms/src/Interpreters/ExternalLoader.h +++ b/dbms/src/Interpreters/ExternalLoader.h @@ -8,6 +8,7 @@ #include #include #include +#include namespace DB @@ -47,17 +48,7 @@ class ExternalLoader public: using LoadablePtr = std::shared_ptr; using Loadables = std::vector; - - enum class Status - { - NOT_LOADED, /// Object hasn't been tried to load. This is an initial state. - LOADED, /// Object has been loaded successfully. - FAILED, /// Object has been failed to load. - LOADING, /// Object is being loaded right now for the first time. - FAILED_AND_RELOADING, /// Object was failed to load before and it's being reloaded right now. - LOADED_AND_RELOADING, /// Object was loaded successfully before and it's being reloaded right now. - NOT_EXIST, /// Object with this name wasn't found in the configuration. - }; + using Status = ExternalLoaderStatus; using Duration = std::chrono::milliseconds; using TimePoint = std::chrono::system_clock::time_point; @@ -225,8 +216,4 @@ private: Poco::Logger * log; }; -String toString(ExternalLoader::Status status); -std::ostream & operator<<(std::ostream & out, ExternalLoader::Status status); -std::vector> getStatusEnumAllPossibleValues(); - } From df3976c9b2a428340ed4e774c89564a113666a93 Mon Sep 17 00:00:00 2001 From: Guillaume Tassery Date: Tue, 17 Mar 2020 11:43:57 +0100 Subject: [PATCH 0085/1355] set code on cpp file --- dbms/src/Common/ExternalLoaderStatus.cpp | 39 ++++++++++++++++++++++++ dbms/src/Common/ExternalLoaderStatus.h | 37 ++-------------------- 2 files changed, 42 insertions(+), 34 deletions(-) create mode 100644 dbms/src/Common/ExternalLoaderStatus.cpp diff --git a/dbms/src/Common/ExternalLoaderStatus.cpp b/dbms/src/Common/ExternalLoaderStatus.cpp new file mode 100644 index 00000000000..a31b13eb574 --- /dev/null +++ b/dbms/src/Common/ExternalLoaderStatus.cpp @@ -0,0 +1,39 @@ +#include + +namespace DB +{ + String toString(ExternalLoaderStatus status) + { + using Status = ExternalLoaderStatus; + switch (status) + { + case Status::NOT_LOADED: return "NOT_LOADED"; + case Status::LOADED: return "LOADED"; + case Status::FAILED: return "FAILED"; + case Status::LOADING: return "LOADING"; + case Status::FAILED_AND_RELOADING: return "FAILED_AND_RELOADING"; + case Status::LOADED_AND_RELOADING: return "LOADED_AND_RELOADING"; + case Status::NOT_EXIST: return "NOT_EXIST"; + } + __builtin_unreachable(); + } + + std::vector> getStatusEnumAllPossibleValues() + { + using Status = ExternalLoaderStatus; + return std::vector>{ + {toString(Status::NOT_LOADED), static_cast(Status::NOT_LOADED)}, + {toString(Status::LOADED), static_cast(Status::LOADED)}, + {toString(Status::FAILED), static_cast(Status::FAILED)}, + {toString(Status::LOADING), static_cast(Status::LOADING)}, + {toString(Status::LOADED_AND_RELOADING), static_cast(Status::LOADED_AND_RELOADING)}, + {toString(Status::FAILED_AND_RELOADING), static_cast(Status::FAILED_AND_RELOADING)}, + {toString(Status::NOT_EXIST), static_cast(Status::NOT_EXIST)}, + }; + } + + std::ostream & operator<<(std::ostream & out, ExternalLoaderStatus status) + { + return out << toString(status); + } +} diff --git a/dbms/src/Common/ExternalLoaderStatus.h b/dbms/src/Common/ExternalLoaderStatus.h index 2478215d016..7a412319952 100644 --- a/dbms/src/Common/ExternalLoaderStatus.h +++ b/dbms/src/Common/ExternalLoaderStatus.h @@ -13,38 +13,7 @@ namespace DB NOT_EXIST, /// Object with this name wasn't found in the configuration. }; - String toString(ExternalLoaderStatus status) - { - using Status = ExternalLoaderStatus; - switch (status) - { - case Status::NOT_LOADED: return "NOT_LOADED"; - case Status::LOADED: return "LOADED"; - case Status::FAILED: return "FAILED"; - case Status::LOADING: return "LOADING"; - case Status::FAILED_AND_RELOADING: return "FAILED_AND_RELOADING"; - case Status::LOADED_AND_RELOADING: return "LOADED_AND_RELOADING"; - case Status::NOT_EXIST: return "NOT_EXIST"; - } - __builtin_unreachable(); - } - - std::vector> getStatusEnumAllPossibleValues() - { - using Status = ExternalLoaderStatus; - return std::vector>{ - {toString(Status::NOT_LOADED), static_cast(Status::NOT_LOADED)}, - {toString(Status::LOADED), static_cast(Status::LOADED)}, - {toString(Status::FAILED), static_cast(Status::FAILED)}, - {toString(Status::LOADING), static_cast(Status::LOADING)}, - {toString(Status::LOADED_AND_RELOADING), static_cast(Status::LOADED_AND_RELOADING)}, - {toString(Status::FAILED_AND_RELOADING), static_cast(Status::FAILED_AND_RELOADING)}, - {toString(Status::NOT_EXIST), static_cast(Status::NOT_EXIST)}, - }; - } - - std::ostream & operator<<(std::ostream & out, ExternalLoaderStatus status) - { - return out << toString(status); - } + String toString(ExternalLoaderStatus status); + std::vector> getStatusEnumAllPossibleValues(); + std::ostream & operator<<(std::ostream & out, ExternalLoaderStatus status); } From cb7bbed8495d04f9adabf3adace1cb260335916d Mon Sep 17 00:00:00 2001 From: Guillaume Tassery Date: Tue, 17 Mar 2020 12:36:41 +0100 Subject: [PATCH 0086/1355] run build --- dbms/src/Common/ExternalLoaderStatus.h | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/dbms/src/Common/ExternalLoaderStatus.h b/dbms/src/Common/ExternalLoaderStatus.h index 7a412319952..44536198b82 100644 --- a/dbms/src/Common/ExternalLoaderStatus.h +++ b/dbms/src/Common/ExternalLoaderStatus.h @@ -1,5 +1,10 @@ #pragma once +#include +#include +#include +#include + namespace DB { enum class ExternalLoaderStatus From 751a574274271ed9825f7a395d85669b305c5615 Mon Sep 17 00:00:00 2001 From: Guillaume Tassery Date: Tue, 17 Mar 2020 14:27:05 +0100 Subject: [PATCH 0087/1355] remove double quoted string --- dbms/programs/server/PrometheusMetricsWriter.cpp | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/dbms/programs/server/PrometheusMetricsWriter.cpp b/dbms/programs/server/PrometheusMetricsWriter.cpp index 2bce7ac5abd..787f0fcd95e 100644 --- a/dbms/programs/server/PrometheusMetricsWriter.cpp +++ b/dbms/programs/server/PrometheusMetricsWriter.cpp @@ -121,11 +121,11 @@ void PrometheusMetricsWriter::write(WriteBuffer & wb) const DB::writeText(key, wb); DB::writeChar('{', wb); DB::writeText(key, wb); - DB::writeText("=\"", wb); + DB::writeChar('=', wb); writeDoubleQuotedString(enum_value.first, wb); - DB::writeText("\",name=\"", wb); + DB::writeText(",name=", wb); writeDoubleQuotedString(value.first, wb); - DB::writeText("\"} ", wb); + DB::writeText("} ", wb); DB::writeText(value.second == enum_value.second, wb); DB::writeChar('\n', wb); } From 784ac0059f97c5851b09aebe91380d2cdd89e09b Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Tue, 17 Mar 2020 19:23:47 +0300 Subject: [PATCH 0088/1355] better --- dbms/programs/copier/ClusterCopier.cpp | 6 +++++- dbms/programs/copier/ClusterCopier.h | 2 +- dbms/src/Storages/StorageReplicatedMergeTree.cpp | 8 ++++---- 3 files changed, 10 insertions(+), 6 deletions(-) diff --git a/dbms/programs/copier/ClusterCopier.cpp b/dbms/programs/copier/ClusterCopier.cpp index b1c284122a4..7edd8953252 100644 --- a/dbms/programs/copier/ClusterCopier.cpp +++ b/dbms/programs/copier/ClusterCopier.cpp @@ -596,6 +596,9 @@ PartitionTaskStatus ClusterCopier::tryMoveAllPiecesToDestinationTable(const Task LOG_DEBUG(log, "Moving piece for partition " << current_partition_attach_is_active << " has not been successfully finished by " << status.owner << ". Will try to move by myself."); + + /// Remove is_done marker. + zookeeper->remove(current_partition_attach_is_done); } } @@ -1036,9 +1039,10 @@ bool ClusterCopier::tryProcessTable(const ConnectionTimeouts & timeouts, TaskTab /// Sleep if this task is active if (res == PartitionTaskStatus::Active) - std::this_thread::sleep_for(default_sleep_time); + break; /// Repeat on errors + std::this_thread::sleep_for(default_sleep_time); } catch (...) { tryLogCurrentException(log, diff --git a/dbms/programs/copier/ClusterCopier.h b/dbms/programs/copier/ClusterCopier.h index 8558a2859e3..0f994e520d2 100644 --- a/dbms/programs/copier/ClusterCopier.h +++ b/dbms/programs/copier/ClusterCopier.h @@ -125,7 +125,7 @@ protected: static constexpr UInt64 max_table_tries = 1000; static constexpr UInt64 max_shard_partition_tries = 600; - static constexpr UInt64 max_shard_partition_piece_tries_for_alter = 10; + static constexpr UInt64 max_shard_partition_piece_tries_for_alter = 100; bool tryProcessTable(const ConnectionTimeouts & timeouts, TaskTable & task_table); diff --git a/dbms/src/Storages/StorageReplicatedMergeTree.cpp b/dbms/src/Storages/StorageReplicatedMergeTree.cpp index 7ea1243ff49..fb2ccfcd323 100644 --- a/dbms/src/Storages/StorageReplicatedMergeTree.cpp +++ b/dbms/src/Storages/StorageReplicatedMergeTree.cpp @@ -4941,10 +4941,10 @@ void StorageReplicatedMergeTree::replacePartitionFrom(const StoragePtr & source_ part_checksums.emplace_back(hash_hex); } - /// In case of ATTACH PARTITION FROM we have nothing to do - return. Because no data will be added or removed. - /// In case of REPLACE PARTITION we can replace existing partition with empty. - if (!replace && src_parts.empty()) - return; +// /// In case of ATTACH PARTITION FROM we have nothing to do - return. Because no data will be added or removed. +// /// In case of REPLACE PARTITION we can replace existing partition with empty. +// if (!replace && src_parts.empty()) +// return; ReplicatedMergeTreeLogEntryData entry; { From 6a991783f47dd3716a8fc0716d84aaee472dd960 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Tue, 17 Mar 2020 19:50:22 +0300 Subject: [PATCH 0089/1355] simplify code --- dbms/programs/copier/Aliases.h | 1 - dbms/programs/copier/ClusterCopier.cpp | 21 +-------------------- 2 files changed, 1 insertion(+), 21 deletions(-) diff --git a/dbms/programs/copier/Aliases.h b/dbms/programs/copier/Aliases.h index d91685445f8..c4d9c40d9f1 100644 --- a/dbms/programs/copier/Aliases.h +++ b/dbms/programs/copier/Aliases.h @@ -2,7 +2,6 @@ #include - namespace DB { using ConfigurationPtr = Poco::AutoPtr; diff --git a/dbms/programs/copier/ClusterCopier.cpp b/dbms/programs/copier/ClusterCopier.cpp index 7edd8953252..a25e3b56348 100644 --- a/dbms/programs/copier/ClusterCopier.cpp +++ b/dbms/programs/copier/ClusterCopier.cpp @@ -45,13 +45,6 @@ void ClusterCopier::init() for (auto & task_table : task_cluster->table_tasks) { task_table.cluster_pull = context.getCluster(task_table.cluster_pull_name); - auto pull_shards_info = task_table.cluster_pull->getShardsInfo(); - for (auto & shard_info : pull_shards_info) - { - std::cout << "current_shard " << toString(shard_info.shard_num) << "has remote connections " - << toString(shard_info.hasRemoteConnections()) << std::endl; - } - std::cout << "CLUSTER PULL " << std::endl; task_table.cluster_push = context.getCluster(task_table.cluster_push_name); task_table.initShards(task_cluster->random_engine); } @@ -417,19 +410,7 @@ bool ClusterCopier::checkPartitionPieceIsClean( if (zookeeper->exists(task_status_path, &stat)) task_start_clock = LogicalClock(stat.mzxid); - /// If statement for readability. - - LOG_INFO(log, "clean_state_clock.is_stale() " << clean_state_clock.is_stale()); - LOG_INFO(log, "clean_state_clock.is_clean() " << clean_state_clock.is_clean()); - if (clean_state_clock.is_clean() && (!task_start_clock.hasHappened() || clean_state_clock.discovery_zxid <= task_start_clock)) - { - return true; - } - else - { - return false; - } - + return clean_state_clock.is_clean() && (!task_start_clock.hasHappened() || clean_state_clock.discovery_zxid <= task_start_clock); } From 1798bbedec264549789b5c1adf227f7f9535b346 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Tue, 17 Mar 2020 21:07:54 +0300 Subject: [PATCH 0090/1355] style issues --- dbms/programs/copier/ClusterCopier.cpp | 40 ++++--------- dbms/programs/copier/ClusterCopier.h | 12 ++-- dbms/programs/copier/ShardPartition.h | 58 +++++++++++-------- dbms/programs/copier/ShardPartitionPiece.h | 22 +++---- dbms/programs/copier/TaskTableAndShard.h | 22 ++----- dbms/src/Interpreters/Cluster.cpp | 4 -- .../Storages/StorageReplicatedMergeTree.cpp | 6 +- 7 files changed, 71 insertions(+), 93 deletions(-) diff --git a/dbms/programs/copier/ClusterCopier.cpp b/dbms/programs/copier/ClusterCopier.cpp index a25e3b56348..b36246e95db 100644 --- a/dbms/programs/copier/ClusterCopier.cpp +++ b/dbms/programs/copier/ClusterCopier.cpp @@ -593,8 +593,6 @@ PartitionTaskStatus ClusterCopier::tryMoveAllPiecesToDestinationTable(const Task /// Move partition to original destination table. for (size_t current_piece_number = 0; current_piece_number < task_table.number_of_splits; ++current_piece_number) { - /// TODO: Execute alter table move partition. - LOG_DEBUG(log, "Trying to move partition " << partition_name << " piece " << toString(current_piece_number) << " to original table"); @@ -646,7 +644,8 @@ PartitionTaskStatus ClusterCopier::tryMoveAllPiecesToDestinationTable(const Task try { String query_deduplicate_ast_string; - if (!task_table.isReplicatedTable()) { + if (!task_table.isReplicatedTable()) + { query_deduplicate_ast_string += " OPTIMIZE TABLE " + getQuotedTable(original_table) + " PARTITION " + partition_name + " DEDUPLICATE;"; @@ -671,9 +670,6 @@ PartitionTaskStatus ClusterCopier::tryMoveAllPiecesToDestinationTable(const Task } } - - - /// Create node to signal that we finished moving { String state_finished = TaskStateWithOwner::getData(TaskState::Finished, host_id); @@ -840,19 +836,14 @@ bool ClusterCopier::tryDropPartitionPiece( LOG_DEBUG(log, "Execute distributed DROP PARTITION: " << query); /// We have to drop partition_piece on each replica - UInt64 num_shards = executeQueryOnCluster( + size_t num_shards = executeQueryOnCluster( cluster_push, query, nullptr, &settings_push, PoolMode::GET_MANY, ClusterExecutionMode::ON_EACH_NODE); - UNUSED(num_shards); -// if (num_shards < cluster_push->getShardCount()) -// { -// LOG_INFO(log, "DROP PARTITION wasn't successfully executed on " << cluster_push->getShardCount() - num_shards << " shards"); -// return false; -// } + LOG_INFO(log, "DROP PARTITION was successfully executed on " << num_shards << " nodes of a cluster."); /// Update the locking node if (!my_clock.is_stale()) @@ -1025,10 +1016,9 @@ bool ClusterCopier::tryProcessTable(const ConnectionTimeouts & timeouts, TaskTab /// Repeat on errors std::this_thread::sleep_for(default_sleep_time); } - catch (...) { - tryLogCurrentException(log, - "Some error occured while moving pieces to destination table for partition " + - partition_name); + catch (...) + { + tryLogCurrentException(log, "Some error occured while moving pieces to destination table for partition " + partition_name); } } } @@ -1201,8 +1191,7 @@ PartitionTaskStatus ClusterCopier::processPartitionPieceTaskImpl( query += " WHERE (" + queryToString(task_table.engine_push_partition_key_ast) + " = (" + task_partition.name + " AS partition_key))"; if (enable_splitting) - query += " AND ( cityHash64(" + primary_key_comma_separated + ") %" + toString(number_of_splits) + - " = " + toString(current_piece_number) + " )"; + query += " AND ( cityHash64(" + primary_key_comma_separated + ") %" + toString(number_of_splits) + " = " + toString(current_piece_number) + " )"; if (!task_table.where_condition_str.empty()) query += " AND (" + task_table.where_condition_str + ")"; @@ -1238,7 +1227,6 @@ PartitionTaskStatus ClusterCopier::processPartitionPieceTaskImpl( try { - /// TODO: tryDropPartitionPiece. tryDropPartitionPiece(task_partition, current_piece_number, zookeeper, clean_state_clock); } catch (...) @@ -1317,7 +1305,6 @@ PartitionTaskStatus ClusterCopier::processPartitionPieceTaskImpl( auto checker = zkutil::EphemeralNodeHolder::create(partition_piece.getPartitionPieceCleanStartPath() + "/checker", *zookeeper, host_id); // Maybe we are the first worker - ///TODO: Why table_split_shard??? ASTPtr query_select_ast = get_select_query(split_table_for_current_piece, "count()", /*enable_splitting*/ true); UInt64 count; @@ -1552,8 +1539,6 @@ PartitionTaskStatus ClusterCopier::processPartitionPieceTaskImpl( zookeeper->set(current_task_piece_status_path, state_finished, 0); } - /// TODO: LOG_INFO (Piece copied and moved to destination table) - return PartitionTaskStatus::Finished; } @@ -1757,8 +1742,6 @@ bool ClusterCopier::checkShardHasPartition(const ConnectionTimeouts & timeouts, return InterpreterFactory::get(query_ast, local_context)->execute().in->read().rows() != 0; } -/// TODO: Implement checkPresentPartitionPiecesOnCurrentShard(); -/// Just copypaste the function above bool ClusterCopier::checkPresentPartitionPiecesOnCurrentShard(const ConnectionTimeouts & timeouts, TaskShard & task_shard, const String & partition_quoted_name, size_t current_piece_number) { @@ -1782,7 +1765,8 @@ bool ClusterCopier::checkPresentPartitionPiecesOnCurrentShard(const ConnectionTi query += " LIMIT 1"; LOG_DEBUG(log, "Checking shard " << task_shard.getDescription() << " for partition " - << partition_quoted_name << " piece " << std::to_string(current_piece_number) << "existence, executing query: " << query); + << partition_quoted_name << " piece " << std::to_string(current_piece_number) + << "existence, executing query: " << query); ParserQuery parser_query(query.data() + query.size()); ASTPtr query_ast = parseQuery(parser_query, query, 0); @@ -1792,10 +1776,10 @@ bool ClusterCopier::checkPresentPartitionPiecesOnCurrentShard(const ConnectionTi auto result = InterpreterFactory::get(query_ast, local_context)->execute().in->read().rows(); if (result != 0) LOG_DEBUG(log, "Partition " << partition_quoted_name << " piece number " - << std::to_string(current_piece_number) << " is PRESENT on shard " << task_shard.getDescription()); + << std::to_string(current_piece_number) << " is PRESENT on shard " << task_shard.getDescription()); else LOG_DEBUG(log, "Partition " << partition_quoted_name << " piece number " - << std::to_string(current_piece_number) << " is ABSENT on shard " << task_shard.getDescription()); + << std::to_string(current_piece_number) << " is ABSENT on shard " << task_shard.getDescription()); return result != 0; } diff --git a/dbms/programs/copier/ClusterCopier.h b/dbms/programs/copier/ClusterCopier.h index 0f994e520d2..84b920992a5 100644 --- a/dbms/programs/copier/ClusterCopier.h +++ b/dbms/programs/copier/ClusterCopier.h @@ -16,14 +16,14 @@ class ClusterCopier { public: - ClusterCopier(String task_path_, - String host_id_, - String proxy_database_name_, + ClusterCopier(const String & task_path_, + const String & host_id_, + const String & proxy_database_name_, Context & context_) : - task_zookeeper_path(std::move(task_path_)), - host_id(std::move(host_id_)), - working_database_name(std::move(proxy_database_name_)), + task_zookeeper_path(task_path_), + host_id(host_id_), + working_database_name(proxy_database_name_), context(context_), log(&Poco::Logger::get("ClusterCopier")) {} diff --git a/dbms/programs/copier/ShardPartition.h b/dbms/programs/copier/ShardPartition.h index f7384b977e3..7de381977f9 100644 --- a/dbms/programs/copier/ShardPartition.h +++ b/dbms/programs/copier/ShardPartition.h @@ -3,36 +3,37 @@ #include "Aliases.h" #include "TaskTableAndShard.h" -namespace DB { - +namespace DB +{ /// Just destination partition of a shard /// I don't know what this comment means. /// In short, when we discovered what shards contain currently processing partition, /// This class describes a partition (name) that is stored on the shard (parent). -struct ShardPartition { +struct ShardPartition +{ ShardPartition(TaskShard &parent, String name_quoted_, size_t number_of_splits = 10) : task_shard(parent), name(std::move(name_quoted_)) { pieces.reserve(number_of_splits); } - /*useful*/ String getPartitionPath() const; + String getPartitionPath() const; - [[maybe_unused]] String getPartitionPiecePath(size_t current_piece_number) const; + String getPartitionPiecePath(size_t current_piece_number) const; - /*useful*/ String getPartitionCleanStartPath() const; + String getPartitionCleanStartPath() const; - [[maybe_unused]] String getPartitionPieceCleanStartPath(size_t current_piece_number) const; + String getPartitionPieceCleanStartPath(size_t current_piece_number) const; - /*useful*/ String getCommonPartitionIsDirtyPath() const; + String getCommonPartitionIsDirtyPath() const; - /*useful*/ String getCommonPartitionIsCleanedPath() const; + String getCommonPartitionIsCleanedPath() const; - /*??????*/ String getPartitionActiveWorkersPath() const; + String getPartitionActiveWorkersPath() const; - /*??????*/ String getActiveWorkerPath() const; + String getActiveWorkerPath() const; - /*useful*/ String getPartitionShardsPath() const; + String getPartitionShardsPath() const; - /*useful*/ String getShardStatusPath() const; + String getShardStatusPath() const; /// What partition pieces are present in current shard. /// FYI: Piece is a part of partition which has modulo equals to concrete constant (less than number_of_splits obliously) @@ -44,49 +45,58 @@ struct ShardPartition { String name; }; -inline String ShardPartition::getPartitionCleanStartPath() const { +inline String ShardPartition::getPartitionCleanStartPath() const +{ return getPartitionPath() + "/clean_start"; } -inline String ShardPartition::getPartitionPieceCleanStartPath(size_t current_piece_number) const { +inline String ShardPartition::getPartitionPieceCleanStartPath(size_t current_piece_number) const +{ assert(current_piece_number < task_shard.task_table.number_of_splits); return getPartitionPiecePath(current_piece_number) + "/clean_start"; } -inline String ShardPartition::getPartitionPath() const { +inline String ShardPartition::getPartitionPath() const +{ return task_shard.task_table.getPartitionPath(name); } -inline String ShardPartition::getPartitionPiecePath(size_t current_piece_number) const { +inline String ShardPartition::getPartitionPiecePath(size_t current_piece_number) const +{ assert(current_piece_number < task_shard.task_table.number_of_splits); return task_shard.task_table.getPartitionPiecePath(name, current_piece_number); } -inline String ShardPartition::getShardStatusPath() const { +inline String ShardPartition::getShardStatusPath() const +{ // schema: //tables/
//shards/ // e.g. /root/table_test.hits/201701/shards/1 return getPartitionShardsPath() + "/" + toString(task_shard.numberInCluster()); } -inline String ShardPartition::getPartitionShardsPath() const { +inline String ShardPartition::getPartitionShardsPath() const +{ return getPartitionPath() + "/shards"; } -inline String ShardPartition::getPartitionActiveWorkersPath() const { +inline String ShardPartition::getPartitionActiveWorkersPath() const +{ return getPartitionPath() + "/partition_active_workers"; } -inline String ShardPartition::getActiveWorkerPath() const { +inline String ShardPartition::getActiveWorkerPath() const +{ return getPartitionActiveWorkersPath() + "/" + toString(task_shard.numberInCluster()); } -inline String ShardPartition::getCommonPartitionIsDirtyPath() const { +inline String ShardPartition::getCommonPartitionIsDirtyPath() const +{ return getPartitionPath() + "/is_dirty"; } -inline String ShardPartition::getCommonPartitionIsCleanedPath() const { +inline String ShardPartition::getCommonPartitionIsCleanedPath() const +{ return getCommonPartitionIsDirtyPath() + "/cleaned"; } - } diff --git a/dbms/programs/copier/ShardPartitionPiece.h b/dbms/programs/copier/ShardPartitionPiece.h index 88f1ceddb0d..a21fd531da4 100644 --- a/dbms/programs/copier/ShardPartitionPiece.h +++ b/dbms/programs/copier/ShardPartitionPiece.h @@ -2,30 +2,32 @@ #include "Internals.h" -namespace DB { +namespace DB +{ -struct ShardPartitionPiece { +struct ShardPartitionPiece +{ ShardPartitionPiece(ShardPartition &parent, size_t current_piece_number_, bool is_present_piece_) : is_absent_piece(!is_present_piece_), current_piece_number(current_piece_number_), shard_partition(parent) {} - [[maybe_unused]] String getPartitionPiecePath() const; + String getPartitionPiecePath() const; - [[maybe_unused]] String getPartitionPieceCleanStartPath() const; + String getPartitionPieceCleanStartPath() const; - [[maybe_unused]] String getPartitionPieceIsDirtyPath() const; + String getPartitionPieceIsDirtyPath() const; - [[maybe_unused]] String getPartitionPieceIsCleanedPath() const; + String getPartitionPieceIsCleanedPath() const; - [[maybe_unused]] String getPartitionPieceActiveWorkersPath() const; + String getPartitionPieceActiveWorkersPath() const; - [[maybe_unused]] String getActiveWorkerPath() const ; + String getActiveWorkerPath() const ; /// On what shards do we have current partition. - [[maybe_unused]] String getPartitionPieceShardsPath() const; + String getPartitionPieceShardsPath() const; - [[maybe_unused]] String getShardStatusPath() const; + String getShardStatusPath() const; String getPartitionPieceCleanerPath() const; diff --git a/dbms/programs/copier/TaskTableAndShard.h b/dbms/programs/copier/TaskTableAndShard.h index 9a09deb911f..8e67cc5053b 100644 --- a/dbms/programs/copier/TaskTableAndShard.h +++ b/dbms/programs/copier/TaskTableAndShard.h @@ -14,7 +14,8 @@ namespace ErrorCodes struct TaskShard; -struct TaskTable { +struct TaskTable + { TaskTable(TaskCluster & parent, const Poco::Util::AbstractConfiguration & config, const String & prefix, const String & table_key); @@ -43,8 +44,6 @@ struct TaskTable { String getCertainPartitionPieceTaskStatusPath(const String & partition_name, const size_t piece_number) const; - [[maybe_unused]] String getReplicatedEngineFirstArgumentForCurrentPiece(const size_t piece_number) const; - bool isReplicatedTable() const { return engine_push_zk_path != ""; } @@ -134,7 +133,8 @@ struct TaskTable { /// Parition names to process in user-specified order Strings ordered_partition_names; - ClusterPartition & getClusterPartition(const String &partition_name) { + ClusterPartition & getClusterPartition(const String &partition_name) + { auto it = cluster_partitions.find(partition_name); if (it == cluster_partitions.end()) throw Exception("There are no cluster partition " + partition_name + " in " + table_id, @@ -146,7 +146,7 @@ struct TaskTable { UInt64 bytes_copied = 0; UInt64 rows_copied = 0; - template + template void initShards(RandomEngine &&random_engine); }; @@ -188,9 +188,7 @@ struct TaskShard /// Internal distributed tables DatabaseAndTableName table_read_shard; - DatabaseAndTableName main_table_split_shard; - ListOfDatabasesAndTableNames list_of_split_tables_on_shard; }; @@ -228,7 +226,7 @@ inline String TaskTable::getCertainPartitionPieceIsDirtyPath(const String & part return getPartitionPiecePath(partition_name, piece_number) + "/is_dirty"; } -inline String TaskTable::getCertainPartitionIsCleanedPath(const String &partition_name) const +inline String TaskTable::getCertainPartitionIsCleanedPath(const String & partition_name) const { return getCertainPartitionIsDirtyPath(partition_name) + "/cleaned"; } @@ -374,12 +372,6 @@ inline void TaskTable::initShards(RandomEngine && random_engine) local_shards.assign(all_shards.begin(), it_first_remote); } -inline String TaskTable::getReplicatedEngineFirstArgumentForCurrentPiece(const size_t piece_number) const -{ - assert (engine_push_zk_path != ""); - return engine_push_zk_path + "/piece_" + toString(piece_number); -} - inline ASTPtr TaskTable::rewriteReplicatedCreateQueryToPlain() { ASTPtr prev_engine_push_ast = engine_push_ast->clone(); @@ -416,6 +408,4 @@ inline String DB::TaskShard::getHostNameExample() const return replicas.at(0).readableString(); } - - } diff --git a/dbms/src/Interpreters/Cluster.cpp b/dbms/src/Interpreters/Cluster.cpp index dc811ffa327..f1790249cc0 100644 --- a/dbms/src/Interpreters/Cluster.cpp +++ b/dbms/src/Interpreters/Cluster.cpp @@ -75,8 +75,6 @@ bool Cluster::Address::isLocal(UInt16 clickhouse_port) const Cluster::Address::Address(const Poco::Util::AbstractConfiguration & config, const String & config_prefix, UInt32 shard_index_, UInt32 replica_index_) : shard_index(shard_index_), replica_index(replica_index_) { - - std::cout << "FROM Address constructor " << " shard index " << shard_index_ << " replica index " << replica_index_ << std::endl; host_name = config.getString(config_prefix + ".host"); port = static_cast(config.getInt(config_prefix + ".port")); if (config.has(config_prefix + ".user")) @@ -340,8 +338,6 @@ Cluster::Cluster(const Poco::Util::AbstractConfiguration & config, const Setting if (startsWith(replica_key, "replica")) { replica_addresses.emplace_back(config, partial_prefix + replica_key, current_shard_num, current_replica_num); - - std::cout << "replica num " << current_replica_num << " is a replica of shard number " << current_shard_num << std::endl; ++current_replica_num; if (!replica_addresses.back().is_local) diff --git a/dbms/src/Storages/StorageReplicatedMergeTree.cpp b/dbms/src/Storages/StorageReplicatedMergeTree.cpp index fb2ccfcd323..77b306d9d97 100644 --- a/dbms/src/Storages/StorageReplicatedMergeTree.cpp +++ b/dbms/src/Storages/StorageReplicatedMergeTree.cpp @@ -4861,6 +4861,7 @@ void StorageReplicatedMergeTree::clearBlocksInPartition( void StorageReplicatedMergeTree::replacePartitionFrom(const StoragePtr & source_table, const ASTPtr & partition, bool replace, const Context & context) { + /// First argument is true, because we possibly will add new data to current table. auto lock1 = lockStructureForShare(true, context.getCurrentQueryId()); auto lock2 = source_table->lockStructureForShare(false, context.getCurrentQueryId()); @@ -4941,11 +4942,6 @@ void StorageReplicatedMergeTree::replacePartitionFrom(const StoragePtr & source_ part_checksums.emplace_back(hash_hex); } -// /// In case of ATTACH PARTITION FROM we have nothing to do - return. Because no data will be added or removed. -// /// In case of REPLACE PARTITION we can replace existing partition with empty. -// if (!replace && src_parts.empty()) -// return; - ReplicatedMergeTreeLogEntryData entry; { auto src_table_id = src_data.getStorageID(); From b3e37a0890a2b181d594735baf0680571dae6c22 Mon Sep 17 00:00:00 2001 From: Guillaume Tassery Date: Wed, 18 Mar 2020 04:09:19 +0100 Subject: [PATCH 0091/1355] add example --- dbms/programs/server/config.xml | 2 ++ 1 file changed, 2 insertions(+) diff --git a/dbms/programs/server/config.xml b/dbms/programs/server/config.xml index 4239cb02523..e0d527f9538 100644 --- a/dbms/programs/server/config.xml +++ b/dbms/programs/server/config.xml @@ -325,6 +325,7 @@ metrics - send data from table system.metrics events - send data from table system.events asynchronous_metrics - send data from table system.asynchronous_metrics + status_info - send data from different component from CH, ex: Dictionaries status --> From 134ac8c9b4ff79d2a299ab3a6b4f3f28fec1d84b Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Wed, 18 Mar 2020 11:49:05 +0300 Subject: [PATCH 0092/1355] tune tests --- .../integration/test_cluster_copier/task0_description.xml | 2 ++ .../test_cluster_copier/task_month_to_week_description.xml | 2 ++ dbms/tests/integration/test_cluster_copier/trivial_test.py | 4 ++-- 3 files changed, 6 insertions(+), 2 deletions(-) diff --git a/dbms/tests/integration/test_cluster_copier/task0_description.xml b/dbms/tests/integration/test_cluster_copier/task0_description.xml index e8e4df99254..6fa490cd02f 100644 --- a/dbms/tests/integration/test_cluster_copier/task0_description.xml +++ b/dbms/tests/integration/test_cluster_copier/task0_description.xml @@ -28,6 +28,8 @@ default hits + 2 + 3 4 5 6 1 2 0 diff --git a/dbms/tests/integration/test_cluster_copier/task_month_to_week_description.xml b/dbms/tests/integration/test_cluster_copier/task_month_to_week_description.xml index e212d1a3d04..ee134603310 100644 --- a/dbms/tests/integration/test_cluster_copier/task_month_to_week_description.xml +++ b/dbms/tests/integration/test_cluster_copier/task_month_to_week_description.xml @@ -30,6 +30,8 @@ --> + 2 + ENGINE= ReplicatedMergeTree('/clickhouse/tables/cluster{cluster}/{shard}/b', '{replica}') diff --git a/dbms/tests/integration/test_cluster_copier/trivial_test.py b/dbms/tests/integration/test_cluster_copier/trivial_test.py index aa204825462..9b71b08674c 100644 --- a/dbms/tests/integration/test_cluster_copier/trivial_test.py +++ b/dbms/tests/integration/test_cluster_copier/trivial_test.py @@ -131,10 +131,10 @@ def test_trivial_copy(started_cluster): execute_task(TaskTrivial(started_cluster), []) def test_trivial_copy_with_copy_fault(started_cluster): - execute_task(TaskTrivial(started_cluster), ['--copy-fault-probability', str(0.1)]) + execute_task(TaskTrivial(started_cluster), ['--copy-fault-probability', str(COPYING_FAIL_PROBABILITY)]) def test_trivial_copy_with_move_fault(started_cluster): - execute_task(TaskTrivial(started_cluster), ['--move-fault-probability', str(0.1)]) + execute_task(TaskTrivial(started_cluster), ['--move-fault-probability', str(MOVING_FAIL_PROBABILITY)]) if __name__ == '__main__': From 528e42bc4cd2ce7735040a45ea878de8a540c361 Mon Sep 17 00:00:00 2001 From: Ivan Lezhankin Date: Mon, 16 Mar 2020 16:50:30 +0300 Subject: [PATCH 0093/1355] Improve jemalloc contrib --- CMakeLists.txt | 8 +- base/common/argsToConfig.cpp | 2 +- base/common/argsToConfig.h | 1 + base/common/config_common.h.in | 1 - base/common/coverage.cpp | 19 ++- cmake/Modules/FindJeMalloc.cmake | 46 ------ cmake/find/jemalloc.cmake | 42 ----- contrib/CMakeLists.txt | 4 +- contrib/jemalloc-cmake/CMakeLists.txt | 156 +++++++++++------- dbms/CMakeLists.txt | 16 +- ...StorageSystemBuildOptions.generated.cpp.in | 1 - 11 files changed, 114 insertions(+), 182 deletions(-) delete mode 100644 cmake/Modules/FindJeMalloc.cmake delete mode 100644 cmake/find/jemalloc.cmake diff --git a/CMakeLists.txt b/CMakeLists.txt index 08b3ed9fdf4..29f97c7c0ae 100644 --- a/CMakeLists.txt +++ b/CMakeLists.txt @@ -351,18 +351,12 @@ if (ENABLE_TESTS) endif () # Need to process before "contrib" dir: -include (cmake/find/jemalloc.cmake) include (cmake/find/cctz.cmake) include (cmake/find/mysqlclient.cmake) # When testing for memory leaks with Valgrind, don't link tcmalloc or jemalloc. -if (USE_JEMALLOC) - message (STATUS "Link jemalloc: ${JEMALLOC_LIBRARIES}") - set (MALLOC_LIBRARIES ${JEMALLOC_LIBRARIES}) -elseif (SANITIZE) - message (STATUS "Will use ${SANITIZE} sanitizer.") -elseif (OS_LINUX) +if (OS_LINUX AND NOT ENABLE_JEMALLOC) message (WARNING "Non default allocator is disabled. This is not recommended for production Linux builds.") endif () diff --git a/base/common/argsToConfig.cpp b/base/common/argsToConfig.cpp index b0ec2900268..e6b65c7bb01 100644 --- a/base/common/argsToConfig.cpp +++ b/base/common/argsToConfig.cpp @@ -1,4 +1,4 @@ -#include +#include "argsToConfig.h" #include #include diff --git a/base/common/argsToConfig.h b/base/common/argsToConfig.h index 1c1607bc4c5..134eed64fd2 100644 --- a/base/common/argsToConfig.h +++ b/base/common/argsToConfig.h @@ -1,4 +1,5 @@ #pragma once + #include namespace Poco::Util diff --git a/base/common/config_common.h.in b/base/common/config_common.h.in index 41999bb5cde..2c9a2b82374 100644 --- a/base/common/config_common.h.in +++ b/base/common/config_common.h.in @@ -2,6 +2,5 @@ // .h autogenerated by cmake ! -#cmakedefine01 USE_JEMALLOC #cmakedefine01 UNBUNDLED #cmakedefine01 WITH_COVERAGE diff --git a/base/common/coverage.cpp b/base/common/coverage.cpp index d8d3b71edd1..0bb36504db6 100644 --- a/base/common/coverage.cpp +++ b/base/common/coverage.cpp @@ -1,16 +1,17 @@ -#include +#include "coverage.h" + #include #if WITH_COVERAGE -#include -#include +# include +# include -#if defined(__clang__) +# if defined(__clang__) extern "C" void __llvm_profile_dump(); -#elif defined(__GNUC__) || defined(__GNUG__) +# elif defined(__GNUC__) || defined(__GNUG__) extern "C" void __gcov_exit(); -#endif +# endif #endif @@ -21,11 +22,11 @@ void dumpCoverageReportIfPossible() static std::mutex mutex; std::lock_guard lock(mutex); -#if defined(__clang__) +# if defined(__clang__) __llvm_profile_dump(); -#elif defined(__GNUC__) || defined(__GNUG__) +# elif defined(__GNUC__) || defined(__GNUG__) __gcov_exit(); -#endif +# endif #endif } diff --git a/cmake/Modules/FindJeMalloc.cmake b/cmake/Modules/FindJeMalloc.cmake deleted file mode 100644 index 264415dc9b2..00000000000 --- a/cmake/Modules/FindJeMalloc.cmake +++ /dev/null @@ -1,46 +0,0 @@ -# https://github.com/bro/cmake/blob/master/FindJeMalloc.cmake -# -# - Try to find jemalloc headers and libraries. -# -# Usage of this module as follows: -# -# find_package(JeMalloc) -# -# Variables used by this module, they can change the default behaviour and need -# to be set before calling find_package: -# -# JEMALLOC_ROOT_DIR Set this variable to the root installation of -# jemalloc if the module has problems finding -# the proper installation path. -# -# Variables defined by this module: -# -# JEMALLOC_FOUND System has jemalloc libs/headers -# JEMALLOC_LIBRARIES The jemalloc library/libraries -# JEMALLOC_INCLUDE_DIR The location of jemalloc headers - -find_path(JEMALLOC_ROOT_DIR - NAMES include/jemalloc/jemalloc.h -) - -find_library(JEMALLOC_LIBRARIES - NAMES jemalloc - HINTS ${JEMALLOC_ROOT_DIR}/lib -) - -find_path(JEMALLOC_INCLUDE_DIR - NAMES jemalloc/jemalloc.h - HINTS ${JEMALLOC_ROOT_DIR}/include -) - -include(FindPackageHandleStandardArgs) -find_package_handle_standard_args(JeMalloc DEFAULT_MSG - JEMALLOC_LIBRARIES - JEMALLOC_INCLUDE_DIR -) - -mark_as_advanced( - JEMALLOC_ROOT_DIR - JEMALLOC_LIBRARIES - JEMALLOC_INCLUDE_DIR -) diff --git a/cmake/find/jemalloc.cmake b/cmake/find/jemalloc.cmake deleted file mode 100644 index 6508f1b675f..00000000000 --- a/cmake/find/jemalloc.cmake +++ /dev/null @@ -1,42 +0,0 @@ -if (OS_LINUX AND NOT SANITIZE AND NOT ARCH_32 AND NOT ARCH_PPC64LE) - set(ENABLE_JEMALLOC_DEFAULT ${ENABLE_LIBRARIES}) -else () - set(ENABLE_JEMALLOC_DEFAULT 0) -endif () - -option (ENABLE_JEMALLOC "Set to TRUE to use jemalloc" ${ENABLE_JEMALLOC_DEFAULT}) -if (OS_LINUX) - option (USE_INTERNAL_JEMALLOC_LIBRARY "Set to FALSE to use system jemalloc library instead of bundled" ${NOT_UNBUNDLED}) -else() - option (USE_INTERNAL_JEMALLOC_LIBRARY "Set to FALSE to use system jemalloc library instead of bundled" OFF) -endif() - -if (ENABLE_JEMALLOC) - if (USE_INTERNAL_JEMALLOC_LIBRARY AND NOT EXISTS "${ClickHouse_SOURCE_DIR}/contrib/jemalloc/src/jemalloc.c") - message (WARNING "submodule contrib/jemalloc is missing. to fix try run: \n git submodule update --init --recursive") - set (USE_INTERNAL_JEMALLOC_LIBRARY 0) - set (MISSING_INTERNAL_JEMALLOC_LIBRARY 1) - endif () - - if (NOT USE_INTERNAL_JEMALLOC_LIBRARY) - find_package (JeMalloc) - endif () - - if ((NOT JEMALLOC_LIBRARIES OR NOT JEMALLOC_INCLUDE_DIR) AND NOT MISSING_INTERNAL_JEMALLOC_LIBRARY ) - set (JEMALLOC_LIBRARIES "jemalloc") - set (JEMALLOC_INCLUDE_DIR "${ClickHouse_SOURCE_DIR}/contrib/jemalloc-cmake/include" "${ClickHouse_SOURCE_DIR}/contrib/jemalloc-cmake/include_linux_x86_64") - set (USE_INTERNAL_JEMALLOC_LIBRARY 1) - endif () - - if (JEMALLOC_LIBRARIES) - set (USE_JEMALLOC 1) - elseif (NOT MISSING_INTERNAL_JEMALLOC_LIBRARY) - message (FATAL_ERROR "ENABLE_JEMALLOC is set to true, but library was not found") - endif () - - if (SANITIZE) - message (FATAL_ERROR "ENABLE_JEMALLOC is set to true, but it cannot be used with sanitizers") - endif () - - message (STATUS "Using jemalloc=${USE_JEMALLOC}: ${JEMALLOC_INCLUDE_DIR} : ${JEMALLOC_LIBRARIES}") -endif () diff --git a/contrib/CMakeLists.txt b/contrib/CMakeLists.txt index bc9c2528fb0..46a9105cefe 100644 --- a/contrib/CMakeLists.txt +++ b/contrib/CMakeLists.txt @@ -79,9 +79,7 @@ if (USE_INTERNAL_CCTZ_LIBRARY) add_subdirectory (cctz-cmake) endif () -if (ENABLE_JEMALLOC AND USE_INTERNAL_JEMALLOC_LIBRARY) - add_subdirectory (jemalloc-cmake) -endif () +add_subdirectory (jemalloc-cmake) if (USE_INTERNAL_CPUID_LIBRARY) add_subdirectory (libcpuid) diff --git a/contrib/jemalloc-cmake/CMakeLists.txt b/contrib/jemalloc-cmake/CMakeLists.txt index 5b420246168..65b5d5fdcfa 100644 --- a/contrib/jemalloc-cmake/CMakeLists.txt +++ b/contrib/jemalloc-cmake/CMakeLists.txt @@ -1,70 +1,108 @@ -set(JEMALLOC_SOURCE_DIR ${ClickHouse_SOURCE_DIR}/contrib/jemalloc) +option (ENABLE_JEMALLOC "Enable jemalloc allocator" ${ENABLE_LIBRARIES}) -set(SRCS -${JEMALLOC_SOURCE_DIR}/src/arena.c -${JEMALLOC_SOURCE_DIR}/src/background_thread.c -${JEMALLOC_SOURCE_DIR}/src/base.c -${JEMALLOC_SOURCE_DIR}/src/bin.c -${JEMALLOC_SOURCE_DIR}/src/bitmap.c -${JEMALLOC_SOURCE_DIR}/src/ckh.c -${JEMALLOC_SOURCE_DIR}/src/ctl.c -${JEMALLOC_SOURCE_DIR}/src/div.c -${JEMALLOC_SOURCE_DIR}/src/extent.c -${JEMALLOC_SOURCE_DIR}/src/extent_dss.c -${JEMALLOC_SOURCE_DIR}/src/extent_mmap.c -${JEMALLOC_SOURCE_DIR}/src/hash.c -${JEMALLOC_SOURCE_DIR}/src/hook.c -${JEMALLOC_SOURCE_DIR}/src/jemalloc.c -${JEMALLOC_SOURCE_DIR}/src/large.c -${JEMALLOC_SOURCE_DIR}/src/log.c -${JEMALLOC_SOURCE_DIR}/src/malloc_io.c -${JEMALLOC_SOURCE_DIR}/src/mutex.c -${JEMALLOC_SOURCE_DIR}/src/mutex_pool.c -${JEMALLOC_SOURCE_DIR}/src/nstime.c -${JEMALLOC_SOURCE_DIR}/src/pages.c -${JEMALLOC_SOURCE_DIR}/src/prng.c -${JEMALLOC_SOURCE_DIR}/src/prof.c -${JEMALLOC_SOURCE_DIR}/src/rtree.c -${JEMALLOC_SOURCE_DIR}/src/sc.c -${JEMALLOC_SOURCE_DIR}/src/stats.c -${JEMALLOC_SOURCE_DIR}/src/sz.c -${JEMALLOC_SOURCE_DIR}/src/tcache.c -${JEMALLOC_SOURCE_DIR}/src/test_hooks.c -${JEMALLOC_SOURCE_DIR}/src/ticker.c -${JEMALLOC_SOURCE_DIR}/src/tsd.c -${JEMALLOC_SOURCE_DIR}/src/witness.c -) - -if(OS_DARWIN) - list(APPEND SRCS ${JEMALLOC_SOURCE_DIR}/src/zone.c) -endif() - -if (CMAKE_CXX_COMPILER_ID STREQUAL "GNU") - set (CMAKE_C_FLAGS "${CMAKE_C_FLAGS} -w") +if (SANITIZE) + set (ENABLE_JEMALLOC FALSE) + message (STATUS "jemalloc is disabled because of sanitizers") endif () -add_library(jemalloc STATIC ${SRCS}) +if (ENABLE_JEMALLOC) + option (USE_INTERNAL_JEMALLOC "Use internal jemalloc library" ${NOT_UNBUNDLED}) -target_include_directories(jemalloc PUBLIC ${CMAKE_CURRENT_SOURCE_DIR}/include) + if (USE_INTERNAL_JEMALLOC) + if (NOT OS_LINUX OR NOT (ARCH_AMD64 OR ARCH_ARM)) + message (FATAL_ERROR "Internal jemalloc can only be used on Linux with x86_64 or aarch64.") + endif () -if (ARCH_AMD64) - target_include_directories(jemalloc PUBLIC ${CMAKE_CURRENT_SOURCE_DIR}/include_linux_x86_64) -elseif (ARCH_ARM) - target_include_directories(jemalloc PUBLIC ${CMAKE_CURRENT_SOURCE_DIR}/include_linux_aarch64) -else () - message (FATAL_ERROR "jemalloc can only be used on x86_64 or aarch64.") -endif () + set (LIBRARY_DIR "${ClickHouse_SOURCE_DIR}/contrib/jemalloc") -target_include_directories(jemalloc PRIVATE - ${JEMALLOC_SOURCE_DIR}/include) + set (SRCS + ${LIBRARY_DIR}/src/arena.c + ${LIBRARY_DIR}/src/background_thread.c + ${LIBRARY_DIR}/src/base.c + ${LIBRARY_DIR}/src/bin.c + ${LIBRARY_DIR}/src/bitmap.c + ${LIBRARY_DIR}/src/ckh.c + ${LIBRARY_DIR}/src/ctl.c + ${LIBRARY_DIR}/src/div.c + ${LIBRARY_DIR}/src/extent.c + ${LIBRARY_DIR}/src/extent_dss.c + ${LIBRARY_DIR}/src/extent_mmap.c + ${LIBRARY_DIR}/src/hash.c + ${LIBRARY_DIR}/src/hook.c + ${LIBRARY_DIR}/src/jemalloc.c + ${LIBRARY_DIR}/src/large.c + ${LIBRARY_DIR}/src/log.c + ${LIBRARY_DIR}/src/malloc_io.c + ${LIBRARY_DIR}/src/mutex.c + ${LIBRARY_DIR}/src/mutex_pool.c + ${LIBRARY_DIR}/src/nstime.c + ${LIBRARY_DIR}/src/pages.c + ${LIBRARY_DIR}/src/prng.c + ${LIBRARY_DIR}/src/prof.c + ${LIBRARY_DIR}/src/rtree.c + ${LIBRARY_DIR}/src/sc.c + ${LIBRARY_DIR}/src/stats.c + ${LIBRARY_DIR}/src/sz.c + ${LIBRARY_DIR}/src/tcache.c + ${LIBRARY_DIR}/src/test_hooks.c + ${LIBRARY_DIR}/src/ticker.c + ${LIBRARY_DIR}/src/tsd.c + ${LIBRARY_DIR}/src/witness.c + ) + if (OS_DARWIN) + list(APPEND SRCS ${LIBRARY_DIR}/src/zone.c) + endif () -target_compile_definitions(jemalloc PRIVATE -DJEMALLOC_NO_PRIVATE_NAMESPACE) + add_library(jemalloc STATIC ${SRCS}) + target_include_directories(jemalloc PRIVATE ${LIBRARY_DIR}/include) + target_include_directories(jemalloc PUBLIC include) + if (ARCH_AMD64) + target_include_directories(jemalloc PUBLIC include_linux_x86_64) + elseif (ARCH_ARM) + target_include_directories(jemalloc PUBLIC include_linux_aarch64) + endif () -if (CMAKE_BUILD_TYPE_UC STREQUAL "DEBUG") - target_compile_definitions(jemalloc PRIVATE -DJEMALLOC_DEBUG=1 -DJEMALLOC_PROF=1) + target_compile_definitions(jemalloc PRIVATE -DJEMALLOC_NO_PRIVATE_NAMESPACE) - if (USE_UNWIND) - target_compile_definitions (jemalloc PRIVATE -DJEMALLOC_PROF_LIBUNWIND=1) - target_link_libraries (jemalloc PRIVATE ${UNWIND_LIBRARIES}) + if (CMAKE_BUILD_TYPE_UC STREQUAL "DEBUG") + target_compile_definitions(jemalloc PRIVATE -DJEMALLOC_DEBUG=1 -DJEMALLOC_PROF=1) + + if (USE_UNWIND) + target_compile_definitions (jemalloc PRIVATE -DJEMALLOC_PROF_LIBUNWIND=1) + target_link_libraries (jemalloc PRIVATE unwind) + endif () + endif () + else () + find_library(LIBRARY_JEMALLOC jemalloc) + find_path(INCLUDE_JEMALLOC jemalloc/jemalloc.h) + + add_library (jemalloc STATIC IMPORTED) + set_property (TARGET jemalloc PROPERTY IMPORTED_LOCATION ${LIBRARY_JEMALLOC}) + target_include_directories (jemalloc PUBLIC ${INCLUDE_JEMALLOC}) + + set (CMAKE_REQUIRED_LIBRARIES jemalloc) + check_cxx_source_compiles ( + " + #include + + int main() { + free(mallocx(1, 0)); + } + " + EXTERNAL_JEMALLOC_WORKS + ) + + if (NOT EXTERNAL_JEMALLOC_WORKS) + message (FATAL_ERROR "jemalloc is unusable: ${LIBRARY_JEMALLOC} ${INCLUDE_JEMALLOC}") + endif () endif () + + target_compile_definitions(jemalloc PUBLIC USE_JEMALLOC=1) + + message (STATUS "Using jemalloc") +else () + add_library(jemalloc INTERFACE) + target_compile_definitions(jemalloc INTERFACE USE_JEMALLOC=0) + + message (STATUS "Not using jemalloc") endif () diff --git a/dbms/CMakeLists.txt b/dbms/CMakeLists.txt index b254c986511..81cb5afbc43 100644 --- a/dbms/CMakeLists.txt +++ b/dbms/CMakeLists.txt @@ -256,7 +256,7 @@ add_library (clickhouse_malloc OBJECT src/Common/malloc.cpp) set_source_files_properties(src/Common/malloc.cpp PROPERTIES COMPILE_FLAGS "-fno-builtin") add_library (clickhouse_new_delete STATIC src/Common/new_delete.cpp) -target_link_libraries (clickhouse_new_delete PRIVATE clickhouse_common_io) +target_link_libraries (clickhouse_new_delete PRIVATE clickhouse_common_io jemalloc) if (OS_FREEBSD) target_compile_definitions (clickhouse_common_io PUBLIC CLOCK_MONOTONIC_COARSE=CLOCK_MONOTONIC_FAST) @@ -303,10 +303,12 @@ add_object_library(clickhouse_processors_sources src/Processors/Sources) if (MAKE_STATIC_LIBRARIES OR NOT SPLIT_SHARED_LIBRARIES) add_library (dbms STATIC ${dbms_headers} ${dbms_sources}) + target_link_libraries (dbms PRIVATE jemalloc) set (all_modules dbms) else() add_library (dbms SHARED ${dbms_headers} ${dbms_sources}) target_link_libraries (dbms PUBLIC ${all_modules}) + target_link_libraries (clickhouse_interpreters PRIVATE jemalloc) list (APPEND all_modules dbms) # force all split libs to be linked set (CMAKE_SHARED_LINKER_FLAGS "${CMAKE_SHARED_LINKER_FLAGS} -Wl,--no-as-needed") @@ -555,18 +557,6 @@ if (USE_BROTLI) target_include_directories (clickhouse_common_io SYSTEM BEFORE PRIVATE ${BROTLI_INCLUDE_DIR}) endif() -if (USE_JEMALLOC) - dbms_target_include_directories (SYSTEM BEFORE PRIVATE ${JEMALLOC_INCLUDE_DIR}) # used in Interpreters/AsynchronousMetrics.cpp - target_include_directories (clickhouse_new_delete SYSTEM BEFORE PRIVATE ${JEMALLOC_INCLUDE_DIR}) - - if(NOT MAKE_STATIC_LIBRARIES AND ${JEMALLOC_LIBRARIES} MATCHES "${CMAKE_STATIC_LIBRARY_SUFFIX}$") - # mallctl in dbms/src/Interpreters/AsynchronousMetrics.cpp - # Actually we link JEMALLOC to almost all libraries. - # This is just hotfix for some uninvestigated problem. - target_link_libraries(clickhouse_interpreters PRIVATE ${JEMALLOC_LIBRARIES}) - endif() -endif () - dbms_target_include_directories (PUBLIC ${DBMS_INCLUDE_DIR}) target_include_directories (clickhouse_common_io PUBLIC ${DBMS_INCLUDE_DIR}) diff --git a/dbms/src/Storages/System/StorageSystemBuildOptions.generated.cpp.in b/dbms/src/Storages/System/StorageSystemBuildOptions.generated.cpp.in index ec51100e8cb..6ae1e2cbfbd 100644 --- a/dbms/src/Storages/System/StorageSystemBuildOptions.generated.cpp.in +++ b/dbms/src/Storages/System/StorageSystemBuildOptions.generated.cpp.in @@ -34,7 +34,6 @@ const char * auto_config_build[] "SPLIT_BINARY", "@CLICKHOUSE_SPLIT_BINARY@", "USE_EMBEDDED_COMPILER", "@USE_EMBEDDED_COMPILER@", "USE_GLIBC_COMPATIBILITY", "@GLIBC_COMPATIBILITY@", - "USE_JEMALLOC", "@USE_JEMALLOC@", "USE_MIMALLOC", "@USE_MIMALLOC@", "USE_UNWIND", "@USE_UNWIND@", "USE_ICU", "@USE_ICU@", From 5a2e82948968022c02d8494160680e0f344d346a Mon Sep 17 00:00:00 2001 From: Ivan Lezhankin Date: Tue, 17 Mar 2020 14:22:15 +0300 Subject: [PATCH 0094/1355] Some fixes --- contrib/jemalloc-cmake/CMakeLists.txt | 12 +++++------- 1 file changed, 5 insertions(+), 7 deletions(-) diff --git a/contrib/jemalloc-cmake/CMakeLists.txt b/contrib/jemalloc-cmake/CMakeLists.txt index 65b5d5fdcfa..3f685bfefd4 100644 --- a/contrib/jemalloc-cmake/CMakeLists.txt +++ b/contrib/jemalloc-cmake/CMakeLists.txt @@ -1,18 +1,14 @@ option (ENABLE_JEMALLOC "Enable jemalloc allocator" ${ENABLE_LIBRARIES}) -if (SANITIZE) - set (ENABLE_JEMALLOC FALSE) - message (STATUS "jemalloc is disabled because of sanitizers") +if (SANITIZE OR NOT OS_LINUX OR NOT (ARCH_AMD64 OR ARCH_ARM)) + set (ENABLE_JEMALLOC OFF) + message (STATUS "jemalloc is disabled implicitly: it doesn't work with sanitizers and can only be used on Linux with x86_64 or aarch64.") endif () if (ENABLE_JEMALLOC) option (USE_INTERNAL_JEMALLOC "Use internal jemalloc library" ${NOT_UNBUNDLED}) if (USE_INTERNAL_JEMALLOC) - if (NOT OS_LINUX OR NOT (ARCH_AMD64 OR ARCH_ARM)) - message (FATAL_ERROR "Internal jemalloc can only be used on Linux with x86_64 or aarch64.") - endif () - set (LIBRARY_DIR "${ClickHouse_SOURCE_DIR}/contrib/jemalloc") set (SRCS @@ -72,6 +68,8 @@ if (ENABLE_JEMALLOC) target_link_libraries (jemalloc PRIVATE unwind) endif () endif () + + target_compile_options(jemalloc PUBLIC -Wno-redundant-decls) else () find_library(LIBRARY_JEMALLOC jemalloc) find_path(INCLUDE_JEMALLOC jemalloc/jemalloc.h) From 40095f4ba514fc7025e351ab267cd6fe3d03d776 Mon Sep 17 00:00:00 2001 From: Ivan Lezhankin Date: Tue, 17 Mar 2020 14:28:41 +0300 Subject: [PATCH 0095/1355] Return jemalloc to system.build_options --- .../Storages/System/StorageSystemBuildOptions.generated.cpp.in | 1 + 1 file changed, 1 insertion(+) diff --git a/dbms/src/Storages/System/StorageSystemBuildOptions.generated.cpp.in b/dbms/src/Storages/System/StorageSystemBuildOptions.generated.cpp.in index 6ae1e2cbfbd..842ad1b1f5d 100644 --- a/dbms/src/Storages/System/StorageSystemBuildOptions.generated.cpp.in +++ b/dbms/src/Storages/System/StorageSystemBuildOptions.generated.cpp.in @@ -34,6 +34,7 @@ const char * auto_config_build[] "SPLIT_BINARY", "@CLICKHOUSE_SPLIT_BINARY@", "USE_EMBEDDED_COMPILER", "@USE_EMBEDDED_COMPILER@", "USE_GLIBC_COMPATIBILITY", "@GLIBC_COMPATIBILITY@", + "USE_JEMALLOC", "@ENABLE_JEMALLOC@", "USE_MIMALLOC", "@USE_MIMALLOC@", "USE_UNWIND", "@USE_UNWIND@", "USE_ICU", "@USE_ICU@", From 6f526db293558f49fab920e5bba7ee5dddba93e1 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Wed, 18 Mar 2020 16:25:49 +0300 Subject: [PATCH 0096/1355] also better code --- dbms/programs/copier/ClusterCopier.cpp | 89 +++++++++++------------- dbms/programs/copier/ClusterCopier.h | 17 +++-- dbms/programs/copier/Internals.cpp | 19 +++-- dbms/programs/copier/Internals.h | 12 +--- dbms/programs/copier/TaskTableAndShard.h | 15 +--- dbms/src/Interpreters/Cluster.h | 12 ---- 6 files changed, 65 insertions(+), 99 deletions(-) diff --git a/dbms/programs/copier/ClusterCopier.cpp b/dbms/programs/copier/ClusterCopier.cpp index b36246e95db..df524b6d314 100644 --- a/dbms/programs/copier/ClusterCopier.cpp +++ b/dbms/programs/copier/ClusterCopier.cpp @@ -1,8 +1,6 @@ #include "ClusterCopier.h" -#include "Internals.h" - -#include +ls#include "Internals.h" #include #include @@ -523,7 +521,7 @@ bool ClusterCopier::checkPartitionPieceIsDone(const TaskTable & task_table, cons } -PartitionTaskStatus ClusterCopier::tryMoveAllPiecesToDestinationTable(const TaskTable & task_table, const String & partition_name) +TaskStatus ClusterCopier::tryMoveAllPiecesToDestinationTable(const TaskTable & task_table, const String & partition_name) { bool inject_fault = false; if (move_fault_probability > 0) @@ -551,7 +549,7 @@ PartitionTaskStatus ClusterCopier::tryMoveAllPiecesToDestinationTable(const Task if (e.code == Coordination::ZNODEEXISTS) { LOG_DEBUG(log, "Someone is already moving pieces " << current_partition_attach_is_active); - return PartitionTaskStatus::Active; + return TaskStatus::Active; } throw; @@ -569,7 +567,7 @@ PartitionTaskStatus ClusterCopier::tryMoveAllPiecesToDestinationTable(const Task { LOG_DEBUG(log, "All pieces for partition from this task " << current_partition_attach_is_active << " has been successfully moved to destination table by " << status.owner); - return PartitionTaskStatus::Finished; + return TaskStatus::Finished; } /// Task is abandoned, because previously we created ephemeral node, possibly in other copier's process. @@ -614,10 +612,7 @@ PartitionTaskStatus ClusterCopier::tryMoveAllPiecesToDestinationTable(const Task try { - size_t num_nodes = 0; - - ///FIXME: We have to be sure that every node in cluster executed this query - size_t current_num_nodes = executeQueryOnCluster( + size_t num_nodes = executeQueryOnCluster( task_table.cluster_push, query_alter_ast_string, nullptr, @@ -625,9 +620,6 @@ PartitionTaskStatus ClusterCopier::tryMoveAllPiecesToDestinationTable(const Task PoolMode::GET_MANY, ClusterExecutionMode::ON_EACH_NODE); - num_nodes = std::max(current_num_nodes, num_nodes); - - LOG_INFO(log, "Number of nodes that executed ALTER query successfully : " << toString(num_nodes)); } catch (...) @@ -676,7 +668,7 @@ PartitionTaskStatus ClusterCopier::tryMoveAllPiecesToDestinationTable(const Task zookeeper->set(current_partition_attach_is_done, state_finished, 0); } - return PartitionTaskStatus::Finished; + return TaskStatus::Finished; } /// Removes MATERIALIZED and ALIAS columns from create table query @@ -729,7 +721,6 @@ std::shared_ptr ClusterCopier::rewriteCreateQueryStorage( res->set(res->columns_list, create.columns_list->clone()); res->set(res->storage, new_storage_ast->clone()); - return res; } @@ -949,7 +940,7 @@ bool ClusterCopier::tryProcessTable(const ConnectionTimeouts & timeouts, TaskTab /// Do not sleep if there is a sequence of already processed shards to increase startup bool is_unprioritized_task = !previous_shard_is_instantly_finished && shard->priority.is_remote; - PartitionTaskStatus task_status = PartitionTaskStatus::Error; + TaskStatus task_status = TaskStatus::Error; bool was_error = false; has_shard_to_process = true; for (UInt64 try_num = 0; try_num < max_shard_partition_tries; ++try_num) @@ -957,20 +948,20 @@ bool ClusterCopier::tryProcessTable(const ConnectionTimeouts & timeouts, TaskTab task_status = tryProcessPartitionTask(timeouts, partition, is_unprioritized_task); /// Exit if success - if (task_status == PartitionTaskStatus::Finished) + if (task_status == TaskStatus::Finished) break; was_error = true; /// Skip if the task is being processed by someone - if (task_status == PartitionTaskStatus::Active) + if (task_status == TaskStatus::Active) break; /// Repeat on errors std::this_thread::sleep_for(default_sleep_time); } - if (task_status == PartitionTaskStatus::Error) + if (task_status == TaskStatus::Error) ++num_failed_shards; previous_shard_is_instantly_finished = !was_error; @@ -1003,17 +994,18 @@ bool ClusterCopier::tryProcessTable(const ConnectionTimeouts & timeouts, TaskTab try { auto res = tryMoveAllPiecesToDestinationTable(task_table, partition_name); - if (res == PartitionTaskStatus::Finished) + /// Exit and mark current task is done. + if (res == TaskStatus::Finished) { partition_moving_is_done = true; break; } - /// Sleep if this task is active - if (res == PartitionTaskStatus::Active) + /// Exit if this task is active. + if (res == TaskStatus::Active) break; - /// Repeat on errors + /// Repeat on errors. std::this_thread::sleep_for(default_sleep_time); } catch (...) @@ -1064,9 +1056,9 @@ bool ClusterCopier::tryProcessTable(const ConnectionTimeouts & timeouts, TaskTab } /// Job for copying partition from particular shard. -PartitionTaskStatus ClusterCopier::tryProcessPartitionTask(const ConnectionTimeouts & timeouts, ShardPartition & task_partition, bool is_unprioritized_task) +TaskStatus ClusterCopier::tryProcessPartitionTask(const ConnectionTimeouts & timeouts, ShardPartition & task_partition, bool is_unprioritized_task) { - PartitionTaskStatus res; + TaskStatus res; try { @@ -1075,7 +1067,7 @@ PartitionTaskStatus ClusterCopier::tryProcessPartitionTask(const ConnectionTimeo catch (...) { tryLogCurrentException(log, "An error occurred while processing partition " + task_partition.name); - res = PartitionTaskStatus::Error; + res = TaskStatus::Error; } /// At the end of each task check if the config is updated @@ -1091,12 +1083,12 @@ PartitionTaskStatus ClusterCopier::tryProcessPartitionTask(const ConnectionTimeo return res; } -PartitionTaskStatus ClusterCopier::iterateThroughAllPiecesInPartition(const ConnectionTimeouts & timeouts, ShardPartition & task_partition, +TaskStatus ClusterCopier::iterateThroughAllPiecesInPartition(const ConnectionTimeouts & timeouts, ShardPartition & task_partition, bool is_unprioritized_task) { const size_t total_number_of_pieces = task_partition.task_shard.task_table.number_of_splits; - PartitionTaskStatus res{PartitionTaskStatus::Finished}; + TaskStatus res{TaskStatus::Finished}; bool was_failed_pieces = false; bool was_active_pieces = false; @@ -1111,33 +1103,32 @@ PartitionTaskStatus ClusterCopier::iterateThroughAllPiecesInPartition(const Conn res = processPartitionPieceTaskImpl(timeouts, task_partition, piece_number, is_unprioritized_task); /// Exit if success - if (res == PartitionTaskStatus::Finished) + if (res == TaskStatus::Finished) break; /// Skip if the task is being processed by someone - if (res == PartitionTaskStatus::Active) + if (res == TaskStatus::Active) break; /// Repeat on errors std::this_thread::sleep_for(default_sleep_time); } - was_active_pieces = (res == PartitionTaskStatus::Active); - was_failed_pieces = (res == PartitionTaskStatus::Error); + was_active_pieces = (res == TaskStatus::Active); + was_failed_pieces = (res == TaskStatus::Error); } if (was_failed_pieces) - return PartitionTaskStatus::Error; + return TaskStatus::Error; if (was_active_pieces) - return PartitionTaskStatus::Active; + return TaskStatus::Active; - return PartitionTaskStatus::Finished; + return TaskStatus::Finished; } -/*...*/ -PartitionTaskStatus ClusterCopier::processPartitionPieceTaskImpl( +TaskStatus ClusterCopier::processPartitionPieceTaskImpl( const ConnectionTimeouts & timeouts, ShardPartition & task_partition, const size_t current_piece_number, bool is_unprioritized_task) { @@ -1234,7 +1225,7 @@ PartitionTaskStatus ClusterCopier::processPartitionPieceTaskImpl( tryLogCurrentException(log, "An error occurred when clean partition"); } - return PartitionTaskStatus::Error; + return TaskStatus::Error; } /// Create ephemeral node to mark that we are active and process the partition @@ -1249,7 +1240,7 @@ PartitionTaskStatus ClusterCopier::processPartitionPieceTaskImpl( if (e.code == Coordination::ZNODEEXISTS) { LOG_DEBUG(log, "Someone is already processing " << current_task_piece_is_active_path); - return PartitionTaskStatus::Active; + return TaskStatus::Active; } throw; @@ -1266,7 +1257,7 @@ PartitionTaskStatus ClusterCopier::processPartitionPieceTaskImpl( { LOG_DEBUG(log, "Task " << current_task_piece_status_path << " has been successfully executed by " << status.owner); - return PartitionTaskStatus::Finished; + return TaskStatus::Finished; } /// Task is abandoned, because previously we created ephemeral node, possibly in other copier's process. @@ -1276,7 +1267,7 @@ PartitionTaskStatus ClusterCopier::processPartitionPieceTaskImpl( << ". Partition will be dropped and refilled."); create_is_dirty_node(clean_state_clock); - return PartitionTaskStatus::Error; + return TaskStatus::Error; } } @@ -1293,7 +1284,7 @@ PartitionTaskStatus ClusterCopier::processPartitionPieceTaskImpl( if (res == Coordination::ZOK) LOG_DEBUG(log, "Partition " << task_partition.name << " piece " + toString(current_piece_number) + " is absent on current replica of a shard. Will mark it as done. Other replicas will do the same."); - return PartitionTaskStatus::Finished; + return TaskStatus::Finished; } /// Check that destination partition is empty if we are first worker @@ -1334,7 +1325,7 @@ PartitionTaskStatus ClusterCopier::processPartitionPieceTaskImpl( << ". Partition will be dropped and refilled."); create_is_dirty_node(clean_state_clock); - return PartitionTaskStatus::Error; + return TaskStatus::Error; } } zookeeper->set(partition_piece.getPartitionPieceCleanStartPath(), "ok"); @@ -1350,14 +1341,14 @@ PartitionTaskStatus ClusterCopier::processPartitionPieceTaskImpl( { LOG_INFO(log, "Partition " << task_partition.name << " piece " << toString(current_piece_number) << " clean state changed, cowardly bailing"); - return PartitionTaskStatus::Error; + return TaskStatus::Error; } else if (!new_clean_state_clock.is_clean()) { LOG_INFO(log, "Partition " << task_partition.name << " piece " << toString(current_piece_number) << " is dirty and will be dropped and refilled"); create_is_dirty_node(new_clean_state_clock); - return PartitionTaskStatus::Error; + return TaskStatus::Error; } zookeeper->create(current_task_piece_status_path, start_state, zkutil::CreateMode::Persistent); } @@ -1491,7 +1482,7 @@ PartitionTaskStatus ClusterCopier::processPartitionPieceTaskImpl( { tryLogCurrentException(log, "An error occurred during copying, partition will be marked as dirty"); create_is_dirty_node(clean_state_clock); - return PartitionTaskStatus::Error; + return TaskStatus::Error; } } @@ -1527,19 +1518,19 @@ PartitionTaskStatus ClusterCopier::processPartitionPieceTaskImpl( { LOG_INFO(log, "Partition " << task_partition.name << " piece " << toString(current_piece_number) << " clean state changed, cowardly bailing"); - return PartitionTaskStatus::Error; + return TaskStatus::Error; } else if (!new_clean_state_clock.is_clean()) { LOG_INFO(log, "Partition " << task_partition.name << " piece " << toString(current_piece_number) << " became dirty and will be dropped and refilled"); create_is_dirty_node(new_clean_state_clock); - return PartitionTaskStatus::Error; + return TaskStatus::Error; } zookeeper->set(current_task_piece_status_path, state_finished, 0); } - return PartitionTaskStatus::Finished; + return TaskStatus::Finished; } void ClusterCopier::dropAndCreateLocalTable(const ASTPtr & create_ast) diff --git a/dbms/programs/copier/ClusterCopier.h b/dbms/programs/copier/ClusterCopier.h index 84b920992a5..60d63e37cf6 100644 --- a/dbms/programs/copier/ClusterCopier.h +++ b/dbms/programs/copier/ClusterCopier.h @@ -111,7 +111,7 @@ protected: /*Alter successful insertion to helping tables it will move all pieces to destination table*/ - PartitionTaskStatus tryMoveAllPiecesToDestinationTable(const TaskTable & task_table, const String & partition_name); + TaskStatus tryMoveAllPiecesToDestinationTable(const TaskTable & task_table, const String & partition_name); /// Removes MATERIALIZED and ALIAS columns from create table query ASTPtr removeAliasColumnsFromCreateQuery(const ASTPtr & query_ast); @@ -130,13 +130,18 @@ protected: bool tryProcessTable(const ConnectionTimeouts & timeouts, TaskTable & task_table); /// Job for copying partition from particular shard. - PartitionTaskStatus tryProcessPartitionTask(const ConnectionTimeouts & timeouts, ShardPartition & task_partition, bool is_unprioritized_task); + TaskStatus tryProcessPartitionTask(const ConnectionTimeouts & timeouts, + ShardPartition & task_partition, + bool is_unprioritized_task); - PartitionTaskStatus iterateThroughAllPiecesInPartition(const ConnectionTimeouts & timeouts, ShardPartition & task_partition, - bool is_unprioritized_task); + TaskStatus iterateThroughAllPiecesInPartition(const ConnectionTimeouts & timeouts, + ShardPartition & task_partition, + bool is_unprioritized_task); - PartitionTaskStatus processPartitionPieceTaskImpl(const ConnectionTimeouts & timeouts, ShardPartition & task_partition, - const size_t current_piece_number, bool is_unprioritized_task); + TaskStatus processPartitionPieceTaskImpl(const ConnectionTimeouts & timeouts, + ShardPartition & task_partition, + const size_t current_piece_number, + bool is_unprioritized_task); void dropAndCreateLocalTable(const ASTPtr & create_ast); diff --git a/dbms/programs/copier/Internals.cpp b/dbms/programs/copier/Internals.cpp index 263bf5f8590..af41ce3ebbe 100644 --- a/dbms/programs/copier/Internals.cpp +++ b/dbms/programs/copier/Internals.cpp @@ -168,16 +168,15 @@ ASTPtr extractOrderBy(const ASTPtr & storage_ast) } -String createCommaSeparatedStringFrom(const Names & strings) +String createCommaSeparatedStringFrom(const Names & names) { - String answer; - for (auto & string: strings) - answer += string + ", "; - - /// Remove last comma and space - answer.pop_back(); - answer.pop_back(); - return answer; + std::ostringstream ss; + if(!names.empty()) + { + std::copy(names.begin(), std::prev(names.end()), std::ostream_iterator(ss, ", ")); + ss << names.back(); + } + return ss.str(); } Names extractPrimaryKeyColumnNames(const ASTPtr & storage_ast) @@ -189,7 +188,7 @@ Names extractPrimaryKeyColumnNames(const ASTPtr & storage_ast) const auto primary_key_expr_list = primary_key_ast ? MergeTreeData::extractKeyExpressionList(primary_key_ast) : sorting_key_expr_list->clone(); - /// VersionedCollapsing ??? + /// Maybe we have to handle VersionedCollapsing engine separately. But in our case in looks pointless. size_t primary_key_size = primary_key_expr_list->children.size(); size_t sorting_key_size = sorting_key_expr_list->children.size(); diff --git a/dbms/programs/copier/Internals.h b/dbms/programs/copier/Internals.h index fc4c2f09da7..63ef9483228 100644 --- a/dbms/programs/copier/Internals.h +++ b/dbms/programs/copier/Internals.h @@ -145,15 +145,9 @@ struct ShardPriority } }; -/// Execution status of a task -enum class PartitionTaskStatus -{ - Active, - Finished, - Error, -}; - -enum class PartititonPieceTaskStatus +/// Execution status of a task. +/// Is used for: partition copying task status, partition piece copying task status, partition moving task status. +enum class TaskStatus { Active, Finished, diff --git a/dbms/programs/copier/TaskTableAndShard.h b/dbms/programs/copier/TaskTableAndShard.h index 8e67cc5053b..615ad297b79 100644 --- a/dbms/programs/copier/TaskTableAndShard.h +++ b/dbms/programs/copier/TaskTableAndShard.h @@ -15,7 +15,7 @@ namespace ErrorCodes struct TaskShard; struct TaskTable - { +{ TaskTable(TaskCluster & parent, const Poco::Util::AbstractConfiguration & config, const String & prefix, const String & table_key); @@ -87,17 +87,6 @@ struct TaskTable ASTPtr sharding_key_ast; ASTPtr main_engine_split_ast; - - /* - * Auxuliary table engines used to perform partition piece copying. - * Each AST represent table engine for certatin piece number. - * After copying partition piece is Ok, this piece will be moved to the main - * target table. All this tables are stored on each shard as the main table. - * We have to use separate tables for partition pieces because of the atomicity of copying. - * Also if we want to move some partition to another table, the partition keys have to be the same. - * */ - - /* * To copy partiton piece form one cluster to another we have to use Distributed table. * In case of usage separate table (engine_push) for each partiton piece, @@ -133,7 +122,7 @@ struct TaskTable /// Parition names to process in user-specified order Strings ordered_partition_names; - ClusterPartition & getClusterPartition(const String &partition_name) + ClusterPartition & getClusterPartition(const String & partition_name) { auto it = cluster_partitions.find(partition_name); if (it == cluster_partitions.end()) diff --git a/dbms/src/Interpreters/Cluster.h b/dbms/src/Interpreters/Cluster.h index 35fe088a216..b029580969f 100644 --- a/dbms/src/Interpreters/Cluster.h +++ b/dbms/src/Interpreters/Cluster.h @@ -162,18 +162,6 @@ public: /// The number of all shards. size_t getShardCount() const { return shards_info.size(); } -// /// The number of all nodes (all replicas of each shard). -// size_t getNodesCount() const -// { -// size_t nodes_count = 0; -// std::cout << "addresses_with_failover.size() " << addresses_with_failover.size() << std::endl; -// std::for_each(addresses_with_failover.begin(), addresses_with_failover.end(), [&] (const Addresses & addresses) -// { -// nodes_count += addresses.size(); -// }); -// return nodes_count; -// } - /// Get a subcluster consisting of one shard - index by count (from 0) of the shard of this cluster. std::unique_ptr getClusterWithSingleShard(size_t index) const; From a19937a0a4d5c539995749afcd3bb05fef7b54b3 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Wed, 18 Mar 2020 16:44:46 +0300 Subject: [PATCH 0097/1355] Refactor weakHash for ColumnString. --- base/common/StringRef.h | 30 ++++++++++++++++++++++++ dbms/src/Columns/ColumnString.cpp | 38 ++++++------------------------- 2 files changed, 37 insertions(+), 31 deletions(-) diff --git a/base/common/StringRef.h b/base/common/StringRef.h index 54010f15085..a00ded07b1c 100644 --- a/base/common/StringRef.h +++ b/base/common/StringRef.h @@ -7,6 +7,7 @@ #include #include +#include #include @@ -264,6 +265,35 @@ struct CRC32Hash return res; } + + static UInt32 updateWeakHash(StringRef x, UInt32 updated_value) + { + const char * pos = x.data; + size_t size = x.size; + + if (size < 8) + { + auto value = unalignedLoad(pos); + /// 8 bytes were loaded to UInt64 value, but string size is less then 8 bytes. + /// We need to zero excessive bytes to remove the garbage. + /// But instead we move bits to the right, so that we had zeros at left. + /// It helps to have different hash for strings like 'a' and 'a\0' + value >>= UInt8(8 * (8 - size)); + return intHashCRC32(value, updated_value); + } + + const char * end = pos + size; + while (pos + 8 < end) + { + auto word = unalignedLoad(pos); + updated_value = intHashCRC32(word, updated_value); + + pos += 8; + } + + auto word = unalignedLoad(pos - 8); + return intHashCRC32(word, updated_value); + } }; struct StringRefHash : CRC32Hash {}; diff --git a/dbms/src/Columns/ColumnString.cpp b/dbms/src/Columns/ColumnString.cpp index ce8326a1fb9..956d900af41 100644 --- a/dbms/src/Columns/ColumnString.cpp +++ b/dbms/src/Columns/ColumnString.cpp @@ -73,42 +73,18 @@ void ColumnString::updateWeakHash32(WeakHash32 & hash) const throw Exception("Size of WeakHash32 does not match size of column: column size is " + std::to_string(s) + ", hash size is " + std::to_string(hash.getData().size()), ErrorCodes::LOGICAL_ERROR); - const UInt8 * begin = &chars[0]; - const UInt8 * str_begin = begin; - const Offset * offset_begin = &offsets[0]; - const Offset * offset_end = offset_begin + s; + const UInt8 * pos = &chars[0]; UInt32 * hash_data = &hash.getData()[0]; + Offset prev_offset = 0; - while (offset_begin < offset_end) + for (auto & offset : offsets) { - const UInt8 * str_end = begin + *offset_begin; + auto str_size = offset - prev_offset; + *hash_data = StringRefHash::updateWeakHash(StringRef(pos, str_size), *hash_data); - auto str_size = str_end - str_begin; - - if (str_size < 8) - { - auto value = unalignedLoad(str_begin); - value &= (1ull << UInt64((8 * str_size))) - 1ull; - *hash_data = intHashCRC32(value, *hash_data); - } - else - { - /// Copy from StringRef.h - while (str_begin + 8 < str_end) - { - auto word = unalignedLoad(str_begin); - *hash_data = intHashCRC32(word, *hash_data); - - str_begin += 8; - } - - auto word = unalignedLoad(str_end - 8); - *hash_data = intHashCRC32(word, *hash_data); - } - - ++offset_begin; + pos += str_size; + prev_offset = offset; ++hash_data; - str_begin = str_end; } } From 23f027e6dadbf60bd4d1d8d6dc2142ddfc64fbb0 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Wed, 18 Mar 2020 17:40:24 +0300 Subject: [PATCH 0098/1355] typo --- dbms/programs/copier/ClusterCopier.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/programs/copier/ClusterCopier.cpp b/dbms/programs/copier/ClusterCopier.cpp index df524b6d314..d3b9d0c3512 100644 --- a/dbms/programs/copier/ClusterCopier.cpp +++ b/dbms/programs/copier/ClusterCopier.cpp @@ -1,6 +1,6 @@ #include "ClusterCopier.h" -ls#include "Internals.h" +#include "Internals.h" #include #include From e26eac46495630e0f89005063db8f6600a32ff94 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Wed, 18 Mar 2020 19:03:55 +0300 Subject: [PATCH 0099/1355] Implement updateWeakHash32 for other columns. --- dbms/src/Columns/ColumnAggregateFunction.cpp | 19 +++++++++++ dbms/src/Columns/ColumnAggregateFunction.h | 2 ++ dbms/src/Columns/ColumnArray.cpp | 25 ++++++++++++++ dbms/src/Columns/ColumnArray.h | 1 + dbms/src/Columns/ColumnConst.cpp | 15 +++++++++ dbms/src/Columns/ColumnConst.h | 2 ++ dbms/src/Columns/ColumnDecimal.cpp | 35 ++++++++++++++++++++ dbms/src/Columns/ColumnDecimal.h | 1 + dbms/src/Columns/ColumnFixedString.cpp | 22 ++++++++++++ dbms/src/Columns/ColumnFixedString.h | 2 ++ dbms/src/Columns/ColumnFunction.h | 5 +++ dbms/src/Columns/ColumnLowCardinality.cpp | 34 +++++++++++++++++++ dbms/src/Columns/ColumnLowCardinality.h | 4 +++ dbms/src/Columns/ColumnNullable.cpp | 21 ++++++++++++ dbms/src/Columns/ColumnNullable.h | 1 + dbms/src/Columns/ColumnTuple.cpp | 13 ++++++++ dbms/src/Columns/ColumnTuple.h | 1 + dbms/src/Columns/IColumnDummy.h | 4 +++ dbms/src/Common/WeakHash.h | 1 + 19 files changed, 208 insertions(+) diff --git a/dbms/src/Columns/ColumnAggregateFunction.cpp b/dbms/src/Columns/ColumnAggregateFunction.cpp index 507ff6c2db9..8b8f0a32d84 100644 --- a/dbms/src/Columns/ColumnAggregateFunction.cpp +++ b/dbms/src/Columns/ColumnAggregateFunction.cpp @@ -10,6 +10,8 @@ #include #include #include +#include +#include #include @@ -282,6 +284,23 @@ void ColumnAggregateFunction::updateHashWithValue(size_t n, SipHash & hash) cons hash.update(wbuf.str().c_str(), wbuf.str().size()); } +void ColumnAggregateFunction::updateWeakHash32(WeakHash32 & hash) const +{ + auto s = data.size(); + if (hash.getData().size() != data.size()) + throw Exception("Size of WeakHash32 does not match size of column: column size is " + std::to_string(s) + + ", hash size is " + std::to_string(hash.getData().size()), ErrorCodes::LOGICAL_ERROR); + + auto & hash_data = hash.getData(); + + for (size_t i = 0; i < s; ++i) + { + WriteBufferFromOwnString wbuf; + func->serialize(data[i], wbuf); + hash_data[i] = StringRefHash::updateWeakHash(StringRef(wbuf.str().c_str(), wbuf.str().size()), hash_data[i]); + } +} + /// The returned size is less than real size. The reason is that some parts of /// aggregate function data may be allocated on shared arenas. These arenas are /// used for several blocks, and also may be updated concurrently from other diff --git a/dbms/src/Columns/ColumnAggregateFunction.h b/dbms/src/Columns/ColumnAggregateFunction.h index 9a744b141f5..4fd75674fe7 100644 --- a/dbms/src/Columns/ColumnAggregateFunction.h +++ b/dbms/src/Columns/ColumnAggregateFunction.h @@ -160,6 +160,8 @@ public: void updateHashWithValue(size_t n, SipHash & hash) const override; + void updateWeakHash32(WeakHash32 & hash) const override; + size_t byteSize() const override; size_t allocatedBytes() const override; diff --git a/dbms/src/Columns/ColumnArray.cpp b/dbms/src/Columns/ColumnArray.cpp index 2c98e873cb6..28f33b9de47 100644 --- a/dbms/src/Columns/ColumnArray.cpp +++ b/dbms/src/Columns/ColumnArray.cpp @@ -17,6 +17,7 @@ #include #include #include +#include namespace DB @@ -214,6 +215,30 @@ void ColumnArray::updateHashWithValue(size_t n, SipHash & hash) const getData().updateHashWithValue(offset + i, hash); } +void ColumnArray::updateWeakHash32(WeakHash32 & hash) const +{ + auto s = offsets->size(); + if (hash.getData().size() != s) + throw Exception("Size of WeakHash32 does not match size of column: column size is " + std::to_string(s) + + ", hash size is " + std::to_string(hash.getData().size()), ErrorCodes::LOGICAL_ERROR); + + WeakHash32 internal_hash(data->size()); + data->updateWeakHash32(internal_hash); + + Offset prev_offset = 0; + auto & offsets_data = getOffsets(); + auto & hash_data = hash.getData(); + auto & internal_hash_data = internal_hash.getData(); + + for (size_t i = 0; i < s; ++i) + { + for (size_t row = prev_offset; row < offsets_data[i]; ++row) + /// It is probably not the best way to combine hashes. + /// But much better then xor which lead to similar hash for arrays like [1], [1, 1, 1], [1, 1, 1, 1, 1], ... + /// Much better implementation - to add offsets as an optional argument to updateWeakHash32. + hash_data[i] = intHashCRC32(internal_hash_data[row], hash_data[i]); + } +} void ColumnArray::insert(const Field & x) { diff --git a/dbms/src/Columns/ColumnArray.h b/dbms/src/Columns/ColumnArray.h index ab414f4917c..15a1d1bd91a 100644 --- a/dbms/src/Columns/ColumnArray.h +++ b/dbms/src/Columns/ColumnArray.h @@ -61,6 +61,7 @@ public: StringRef serializeValueIntoArena(size_t n, Arena & arena, char const *& begin) const override; const char * deserializeAndInsertFromArena(const char * pos) override; void updateHashWithValue(size_t n, SipHash & hash) const override; + void updateWeakHash32(WeakHash32 & hash) const override; void insertRangeFrom(const IColumn & src, size_t start, size_t length) override; void insert(const Field & x) override; void insertFrom(const IColumn & src_, size_t n) override; diff --git a/dbms/src/Columns/ColumnConst.cpp b/dbms/src/Columns/ColumnConst.cpp index 4324d439fb0..bfe94ff0bae 100644 --- a/dbms/src/Columns/ColumnConst.cpp +++ b/dbms/src/Columns/ColumnConst.cpp @@ -4,6 +4,7 @@ #include #include #include +#include namespace DB @@ -103,4 +104,18 @@ void ColumnConst::getPermutation(bool /*reverse*/, size_t /*limit*/, int /*nan_d res[i] = i; } +void ColumnConst::updateWeakHash32(WeakHash32 & hash) const +{ + if (hash.getData().size() != s) + throw Exception("Size of WeakHash32 does not match size of column: column size is " + std::to_string(s) + + ", hash size is " + std::to_string(hash.getData().size()), ErrorCodes::LOGICAL_ERROR); + + WeakHash32 element_hash(1); + data->updateWeakHash32(element_hash); + size_t data_hash = element_hash.getData()[0]; + + for (auto & value : hash.getData()) + value = intHashCRC32(data_hash, value); +} + } diff --git a/dbms/src/Columns/ColumnConst.h b/dbms/src/Columns/ColumnConst.h index 0b8ca38e823..560d4d63a10 100644 --- a/dbms/src/Columns/ColumnConst.h +++ b/dbms/src/Columns/ColumnConst.h @@ -163,6 +163,8 @@ public: data->updateHashWithValue(0, hash); } + void updateWeakHash32(WeakHash32 & hash) const override; + ColumnPtr filter(const Filter & filt, ssize_t result_size_hint) const override; ColumnPtr replicate(const Offsets & offsets) const override; ColumnPtr permute(const Permutation & perm, size_t limit) const override; diff --git a/dbms/src/Columns/ColumnDecimal.cpp b/dbms/src/Columns/ColumnDecimal.cpp index 38c17ccacb4..373255963d0 100644 --- a/dbms/src/Columns/ColumnDecimal.cpp +++ b/dbms/src/Columns/ColumnDecimal.cpp @@ -2,6 +2,7 @@ #include #include #include +#include #include @@ -65,6 +66,40 @@ void ColumnDecimal::updateHashWithValue(size_t n, SipHash & hash) const hash.update(data[n]); } +template +void ColumnDecimal::updateWeakHash32(WeakHash32 & hash) const +{ + auto s = data.size(); + + if (hash.getData().size() != s) + throw Exception("Size of WeakHash32 does not match size of column: column size is " + std::to_string(s) + + ", hash size is " + std::to_string(hash.getData().size()), ErrorCodes::LOGICAL_ERROR); + + const T * begin = &data[0]; + const T * end = begin + s; + UInt32 * hash_data = &hash.getData()[0]; + + while (begin < end) + { + if constexpr (sizeof(T) <= sizeof(UInt64)) + { + *hash_data = intHashCRC32(*begin, *hash_data); + } + else + { + auto * begin64 = reinterpret_cast(begin); + for (size_t i = 0; i < sizeof(T); i += sizeof(UInt64)) + { + *hash_data = intHashCRC32(*begin64, *hash_data); + ++begin64; + } + } + + ++begin; + ++hash_data; + } +} + template void ColumnDecimal::getPermutation(bool reverse, size_t limit, int , IColumn::Permutation & res) const { diff --git a/dbms/src/Columns/ColumnDecimal.h b/dbms/src/Columns/ColumnDecimal.h index 038e96016f1..62e414a676b 100644 --- a/dbms/src/Columns/ColumnDecimal.h +++ b/dbms/src/Columns/ColumnDecimal.h @@ -105,6 +105,7 @@ public: StringRef serializeValueIntoArena(size_t n, Arena & arena, char const *& begin) const override; const char * deserializeAndInsertFromArena(const char * pos) override; void updateHashWithValue(size_t n, SipHash & hash) const override; + void updateWeakHash32(WeakHash32 & hash) const override; int compareAt(size_t n, size_t m, const IColumn & rhs_, int nan_direction_hint) const override; void getPermutation(bool reverse, size_t limit, int nan_direction_hint, IColumn::Permutation & res) const override; diff --git a/dbms/src/Columns/ColumnFixedString.cpp b/dbms/src/Columns/ColumnFixedString.cpp index 02df8d010ea..5c7e1da98dc 100644 --- a/dbms/src/Columns/ColumnFixedString.cpp +++ b/dbms/src/Columns/ColumnFixedString.cpp @@ -6,6 +6,8 @@ #include #include #include +#include +#include #include @@ -101,6 +103,26 @@ void ColumnFixedString::updateHashWithValue(size_t index, SipHash & hash) const hash.update(reinterpret_cast(&chars[n * index]), n); } +void ColumnFixedString::updateWeakHash32(WeakHash32 & hash) const +{ + auto s = size(); + + if (hash.getData().size() != s) + throw Exception("Size of WeakHash32 does not match size of column: column size is " + std::to_string(s) + + ", hash size is " + std::to_string(hash.getData().size()), ErrorCodes::LOGICAL_ERROR); + + const UInt8 * pos = &chars[0]; + UInt32 * hash_data = &hash.getData()[0]; + + for (size_t row = 0; row < s; ++row) + { + *hash_data = StringRefHash::updateWeakHash(StringRef(pos, n), *hash_data); + + pos += n; + ++hash_data; + } +} + template struct ColumnFixedString::less { diff --git a/dbms/src/Columns/ColumnFixedString.h b/dbms/src/Columns/ColumnFixedString.h index a91a82d8524..74c4f3c74f2 100644 --- a/dbms/src/Columns/ColumnFixedString.h +++ b/dbms/src/Columns/ColumnFixedString.h @@ -108,6 +108,8 @@ public: void updateHashWithValue(size_t index, SipHash & hash) const override; + void updateWeakHash32(WeakHash32 & hash) const override; + int compareAt(size_t p1, size_t p2, const IColumn & rhs_, int /*nan_direction_hint*/) const override { const ColumnFixedString & rhs = assert_cast(rhs_); diff --git a/dbms/src/Columns/ColumnFunction.h b/dbms/src/Columns/ColumnFunction.h index 9cd496f72a0..1bde48559fe 100644 --- a/dbms/src/Columns/ColumnFunction.h +++ b/dbms/src/Columns/ColumnFunction.h @@ -101,6 +101,11 @@ public: throw Exception("updateHashWithValue is not implemented for " + getName(), ErrorCodes::NOT_IMPLEMENTED); } + void updateWeakHash32(WeakHash32 &) const override + { + throw Exception("updateWeakHash32 is not implemented for " + getName(), ErrorCodes::NOT_IMPLEMENTED); + } + void popBack(size_t) override { throw Exception("popBack is not implemented for " + getName(), ErrorCodes::NOT_IMPLEMENTED); diff --git a/dbms/src/Columns/ColumnLowCardinality.cpp b/dbms/src/Columns/ColumnLowCardinality.cpp index 23d43579bbd..b209f5938ec 100644 --- a/dbms/src/Columns/ColumnLowCardinality.cpp +++ b/dbms/src/Columns/ColumnLowCardinality.cpp @@ -4,6 +4,7 @@ #include #include #include +#include namespace DB @@ -235,6 +236,21 @@ const char * ColumnLowCardinality::deserializeAndInsertFromArena(const char * po return new_pos; } +void ColumnLowCardinality::updateWeakHash32(WeakHash32 & hash) const +{ + auto s = size(); + + if (hash.getData().size() != s) + throw Exception("Size of WeakHash32 does not match size of column: column size is " + std::to_string(s) + + ", hash size is " + std::to_string(hash.getData().size()), ErrorCodes::LOGICAL_ERROR); + + auto & dict = getDictionary().getNestedColumn(); + WeakHash32 dict_hash(dict->size()); + dict->updateWeakHash32(dict_hash); + + idx.updateWeakHash(hash, dict_hash); +} + void ColumnLowCardinality::gather(ColumnGathererStream & gatherer) { gatherer.gather(*this); @@ -639,6 +655,24 @@ bool ColumnLowCardinality::Index::containsDefault() const return contains; } +void ColumnLowCardinality::Index::updateWeakHash(WeakHash32 & hash, WeakHash32 & dict_hash) const +{ + auto & hash_data = hash.getData(); + auto & dict_hash_data = dict_hash.getData(); + + auto update_weak_hash = [&](auto x) + { + using CurIndexType = decltype(x); + auto & data = getPositionsData(); + auto size = data.size(); + + for (size_t i = 0; i < size; ++i) + hash_data[i] = intHashCRC32(dict_hash_data[data[i]], hash_data[i]); + }; + + callForType(std::move(update_weak_hash), size_of_type); +} + ColumnLowCardinality::Dictionary::Dictionary(MutableColumnPtr && column_unique_, bool is_shared) : column_unique(std::move(column_unique_)), shared(is_shared) diff --git a/dbms/src/Columns/ColumnLowCardinality.h b/dbms/src/Columns/ColumnLowCardinality.h index 621fffb4a19..7b4c75a2eb1 100644 --- a/dbms/src/Columns/ColumnLowCardinality.h +++ b/dbms/src/Columns/ColumnLowCardinality.h @@ -90,6 +90,8 @@ public: return getDictionary().updateHashWithValue(getIndexes().getUInt(n), hash); } + void updateWeakHash32(WeakHash32 & hash) const override; + ColumnPtr filter(const Filter & filt, ssize_t result_size_hint) const override { return ColumnLowCardinality::create(dictionary.getColumnUniquePtr(), getIndexes().filter(filt, result_size_hint)); @@ -230,6 +232,8 @@ public: bool containsDefault() const; + void updateWeakHash(WeakHash32 & hash, WeakHash32 & dict_hash) const; + private: WrappedPtr positions; size_t size_of_type = 0; diff --git a/dbms/src/Columns/ColumnNullable.cpp b/dbms/src/Columns/ColumnNullable.cpp index 575a9f4ac83..79b3bff1ccc 100644 --- a/dbms/src/Columns/ColumnNullable.cpp +++ b/dbms/src/Columns/ColumnNullable.cpp @@ -3,6 +3,7 @@ #include #include #include +#include #include #include #include @@ -42,6 +43,26 @@ void ColumnNullable::updateHashWithValue(size_t n, SipHash & hash) const getNestedColumn().updateHashWithValue(n, hash); } +void ColumnNullable::updateWeakHash32(WeakHash32 & hash) const +{ + auto s = size(); + + if (hash.getData().size() != s) + throw Exception("Size of WeakHash32 does not match size of column: column size is " + std::to_string(s) + + ", hash size is " + std::to_string(hash.getData().size()), ErrorCodes::LOGICAL_ERROR); + + WeakHash32 old_hash = hash; + nested_column->updateWeakHash32(hash); + + auto & null_map_data = getNullMapData(); + auto & hash_data = hash.getData(); + auto & old_hash_data = old_hash.getData(); + + /// Use old data for nulls. + for (size_t row = 0; row < s; ++row) + if (null_map_data[row]) + hash_data[row] = old_hash_data[row]; +} MutableColumnPtr ColumnNullable::cloneResized(size_t new_size) const { diff --git a/dbms/src/Columns/ColumnNullable.h b/dbms/src/Columns/ColumnNullable.h index b1fa32fd6db..5443d8b0187 100644 --- a/dbms/src/Columns/ColumnNullable.h +++ b/dbms/src/Columns/ColumnNullable.h @@ -84,6 +84,7 @@ public: void protect() override; ColumnPtr replicate(const Offsets & replicate_offsets) const override; void updateHashWithValue(size_t n, SipHash & hash) const override; + void updateWeakHash32(WeakHash32 & hash) const override; void getExtremes(Field & min, Field & max) const override; MutableColumns scatter(ColumnIndex num_columns, const Selector & selector) const override diff --git a/dbms/src/Columns/ColumnTuple.cpp b/dbms/src/Columns/ColumnTuple.cpp index d7aacec8d09..4a68f6a7ed6 100644 --- a/dbms/src/Columns/ColumnTuple.cpp +++ b/dbms/src/Columns/ColumnTuple.cpp @@ -6,6 +6,7 @@ #include #include #include +#include #include @@ -180,6 +181,18 @@ void ColumnTuple::updateHashWithValue(size_t n, SipHash & hash) const column->updateHashWithValue(n, hash); } +void ColumnTuple::updateWeakHash32(WeakHash32 & hash) const +{ + auto s = size(); + + if (hash.getData().size() != s) + throw Exception("Size of WeakHash32 does not match size of column: column size is " + std::to_string(s) + + ", hash size is " + std::to_string(hash.getData().size()), ErrorCodes::LOGICAL_ERROR); + + for (auto & column : columns) + column->updateWeakHash32(hash); +} + void ColumnTuple::insertRangeFrom(const IColumn & src, size_t start, size_t length) { const size_t tuple_size = columns.size(); diff --git a/dbms/src/Columns/ColumnTuple.h b/dbms/src/Columns/ColumnTuple.h index e5e47ac74db..3533b602a1b 100644 --- a/dbms/src/Columns/ColumnTuple.h +++ b/dbms/src/Columns/ColumnTuple.h @@ -61,6 +61,7 @@ public: StringRef serializeValueIntoArena(size_t n, Arena & arena, char const *& begin) const override; const char * deserializeAndInsertFromArena(const char * pos) override; void updateHashWithValue(size_t n, SipHash & hash) const override; + void updateWeakHash32(WeakHash32 & hash) const override; void insertRangeFrom(const IColumn & src, size_t start, size_t length) override; ColumnPtr filter(const Filter & filt, ssize_t result_size_hint) const override; ColumnPtr permute(const Permutation & perm, size_t limit) const override; diff --git a/dbms/src/Columns/IColumnDummy.h b/dbms/src/Columns/IColumnDummy.h index 0b56647d5a2..00604fb87d0 100644 --- a/dbms/src/Columns/IColumnDummy.h +++ b/dbms/src/Columns/IColumnDummy.h @@ -65,6 +65,10 @@ public: { } + void updateWeakHash32(WeakHash32 & /*hash*/) const override + { + } + void insertFrom(const IColumn &, size_t) override { ++s; diff --git a/dbms/src/Common/WeakHash.h b/dbms/src/Common/WeakHash.h index 22cb3ae515a..4c42bb930d3 100644 --- a/dbms/src/Common/WeakHash.h +++ b/dbms/src/Common/WeakHash.h @@ -14,6 +14,7 @@ public: using Container = PaddedPODArray; explicit WeakHash32(size_t size) : data(size, ~UInt32(0)) {} + WeakHash32(const WeakHash32 & other) { data.assign(other.data); } const Container & getData() const { return data; } Container & getData() { return data; } From e76877c5ed5c2c025388409d6c97bac4b355d04b Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Wed, 18 Mar 2020 19:18:37 +0300 Subject: [PATCH 0100/1355] Fix build. --- base/common/StringRef.h | 29 -------------------------- dbms/src/Columns/ColumnFixedString.cpp | 4 ++-- dbms/src/Columns/ColumnString.cpp | 2 +- dbms/src/Common/HashTable/Hash.h | 25 ++++++++++++++++++++++ 4 files changed, 28 insertions(+), 32 deletions(-) diff --git a/base/common/StringRef.h b/base/common/StringRef.h index a00ded07b1c..3f5d7beb932 100644 --- a/base/common/StringRef.h +++ b/base/common/StringRef.h @@ -265,35 +265,6 @@ struct CRC32Hash return res; } - - static UInt32 updateWeakHash(StringRef x, UInt32 updated_value) - { - const char * pos = x.data; - size_t size = x.size; - - if (size < 8) - { - auto value = unalignedLoad(pos); - /// 8 bytes were loaded to UInt64 value, but string size is less then 8 bytes. - /// We need to zero excessive bytes to remove the garbage. - /// But instead we move bits to the right, so that we had zeros at left. - /// It helps to have different hash for strings like 'a' and 'a\0' - value >>= UInt8(8 * (8 - size)); - return intHashCRC32(value, updated_value); - } - - const char * end = pos + size; - while (pos + 8 < end) - { - auto word = unalignedLoad(pos); - updated_value = intHashCRC32(word, updated_value); - - pos += 8; - } - - auto word = unalignedLoad(pos - 8); - return intHashCRC32(word, updated_value); - } }; struct StringRefHash : CRC32Hash {}; diff --git a/dbms/src/Columns/ColumnFixedString.cpp b/dbms/src/Columns/ColumnFixedString.cpp index 5c7e1da98dc..9056e0675f7 100644 --- a/dbms/src/Columns/ColumnFixedString.cpp +++ b/dbms/src/Columns/ColumnFixedString.cpp @@ -7,7 +7,7 @@ #include #include #include -#include +#include #include @@ -116,7 +116,7 @@ void ColumnFixedString::updateWeakHash32(WeakHash32 & hash) const for (size_t row = 0; row < s; ++row) { - *hash_data = StringRefHash::updateWeakHash(StringRef(pos, n), *hash_data); + *hash_data = ::updateWeakHash32(pos, n, *hash_data); pos += n; ++hash_data; diff --git a/dbms/src/Columns/ColumnString.cpp b/dbms/src/Columns/ColumnString.cpp index 956d900af41..ec868d6ec11 100644 --- a/dbms/src/Columns/ColumnString.cpp +++ b/dbms/src/Columns/ColumnString.cpp @@ -80,7 +80,7 @@ void ColumnString::updateWeakHash32(WeakHash32 & hash) const for (auto & offset : offsets) { auto str_size = offset - prev_offset; - *hash_data = StringRefHash::updateWeakHash(StringRef(pos, str_size), *hash_data); + *hash_data = ::updateWeakHash32(pos, str_size, *hash_data); pos += str_size; prev_offset = offset; diff --git a/dbms/src/Common/HashTable/Hash.h b/dbms/src/Common/HashTable/Hash.h index 7b4a8e0fd7a..cdcad677088 100644 --- a/dbms/src/Common/HashTable/Hash.h +++ b/dbms/src/Common/HashTable/Hash.h @@ -70,6 +70,31 @@ inline DB::UInt64 intHashCRC32(DB::UInt64 x, DB::UInt64 updated_value) #endif } +UInt32 updateWeakHash32(const DB::UInt8 * pos, size_t size, DB::UInt32 updated_value) +{ + if (size < 8) + { + auto value = unalignedLoad(pos); + /// 8 bytes were loaded to UInt64 value, but string size is less then 8 bytes. + /// We need to zero excessive bytes to remove the garbage. + /// But instead we move bits to the right, so that we had zeros at left. + /// It helps to have different hash for strings like 'a' and 'a\0' + value >>= UInt8(8 * (8 - size)); + return intHashCRC32(value, updated_value); + } + + const auto * end = pos + size; + while (pos + 8 < end) + { + auto word = unalignedLoad(pos); + updated_value = intHashCRC32(word, updated_value); + + pos += 8; + } + + auto word = unalignedLoad(pos - 8); + return intHashCRC32(word, updated_value); +} template inline size_t DefaultHash64(T key) From 0a61f325fe80d6bf68f8990333c0a6955988874d Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Wed, 18 Mar 2020 19:21:28 +0300 Subject: [PATCH 0101/1355] Fix build. --- base/common/StringRef.h | 1 - 1 file changed, 1 deletion(-) diff --git a/base/common/StringRef.h b/base/common/StringRef.h index 3f5d7beb932..54010f15085 100644 --- a/base/common/StringRef.h +++ b/base/common/StringRef.h @@ -7,7 +7,6 @@ #include #include -#include #include From e94f572c9574b9a496eceb23e0ea66f3defe3d57 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Wed, 18 Mar 2020 19:28:50 +0300 Subject: [PATCH 0102/1355] Fix build. --- dbms/src/Columns/IColumnUnique.h | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/dbms/src/Columns/IColumnUnique.h b/dbms/src/Columns/IColumnUnique.h index 9cccb6edbb4..af5d9878a3b 100644 --- a/dbms/src/Columns/IColumnUnique.h +++ b/dbms/src/Columns/IColumnUnique.h @@ -136,6 +136,11 @@ public: { throw Exception("Method scatter is not supported for ColumnUnique.", ErrorCodes::NOT_IMPLEMENTED); } + + void updateWeakHash32(WeakHash32 &) const override + { + throw Exception("Method updateWeakHash32 is not supported for ColumnUnique.", ErrorCodes::NOT_IMPLEMENTED); + } }; using ColumnUniquePtr = IColumnUnique::ColumnUniquePtr; From 9f82f43fddf27f94cc4f615042b8f8d1d40a5c26 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Wed, 18 Mar 2020 19:36:21 +0300 Subject: [PATCH 0103/1355] Fix build. --- dbms/src/Columns/ColumnConst.cpp | 1 + 1 file changed, 1 insertion(+) diff --git a/dbms/src/Columns/ColumnConst.cpp b/dbms/src/Columns/ColumnConst.cpp index bfe94ff0bae..85387b6b84a 100644 --- a/dbms/src/Columns/ColumnConst.cpp +++ b/dbms/src/Columns/ColumnConst.cpp @@ -5,6 +5,7 @@ #include #include #include +#include namespace DB From 6d1640643adfe5fddb9ee3a56f851bf76480237c Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Wed, 18 Mar 2020 19:40:23 +0300 Subject: [PATCH 0104/1355] Fix build. --- dbms/src/Columns/ColumnDecimal.cpp | 1 + dbms/src/Columns/ColumnVector.cpp | 15 ++++++++++++++- 2 files changed, 15 insertions(+), 1 deletion(-) diff --git a/dbms/src/Columns/ColumnDecimal.cpp b/dbms/src/Columns/ColumnDecimal.cpp index 373255963d0..2a0e9c8291e 100644 --- a/dbms/src/Columns/ColumnDecimal.cpp +++ b/dbms/src/Columns/ColumnDecimal.cpp @@ -3,6 +3,7 @@ #include #include #include +#include #include diff --git a/dbms/src/Columns/ColumnVector.cpp b/dbms/src/Columns/ColumnVector.cpp index a96a4605cc6..fa60d562814 100644 --- a/dbms/src/Columns/ColumnVector.cpp +++ b/dbms/src/Columns/ColumnVector.cpp @@ -68,7 +68,20 @@ void ColumnVector::updateWeakHash32(WeakHash32 & hash) const while (begin < end) { - *hash_data = intHashCRC32(*begin, *hash_data); + if constexpr (sizeof(T) <= sizeof(UInt64)) + { + *hash_data = intHashCRC32(*begin, *hash_data); + } + else + { + auto * begin64 = reinterpret_cast(begin); + for (size_t i = 0; i < sizeof(T); i += sizeof(UInt64)) + { + *hash_data = intHashCRC32(*begin64, *hash_data); + ++begin64; + } + } + ++begin; ++hash_data; } From e6f518154b8ab2040ce0c3e692417611b6a1339a Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Wed, 18 Mar 2020 19:46:07 +0300 Subject: [PATCH 0105/1355] Fix build. --- dbms/src/Columns/ColumnAggregateFunction.cpp | 8 +++++--- dbms/src/Columns/ColumnArray.cpp | 1 + dbms/src/Common/HashTable/Hash.h | 1 + 3 files changed, 7 insertions(+), 3 deletions(-) diff --git a/dbms/src/Columns/ColumnAggregateFunction.cpp b/dbms/src/Columns/ColumnAggregateFunction.cpp index 8b8f0a32d84..adb0f55e291 100644 --- a/dbms/src/Columns/ColumnAggregateFunction.cpp +++ b/dbms/src/Columns/ColumnAggregateFunction.cpp @@ -10,8 +10,8 @@ #include #include #include -#include #include +#include #include @@ -295,9 +295,11 @@ void ColumnAggregateFunction::updateWeakHash32(WeakHash32 & hash) const for (size_t i = 0; i < s; ++i) { - WriteBufferFromOwnString wbuf; + std::vector v; + WriteBufferFromVector> wbuf(v); func->serialize(data[i], wbuf); - hash_data[i] = StringRefHash::updateWeakHash(StringRef(wbuf.str().c_str(), wbuf.str().size()), hash_data[i]); + wbuf.finalize(); + hash_data[i] = ::updateWeakHash32(v.data(), v.size(), hash_data[i]); } } diff --git a/dbms/src/Columns/ColumnArray.cpp b/dbms/src/Columns/ColumnArray.cpp index 28f33b9de47..42ec73327a8 100644 --- a/dbms/src/Columns/ColumnArray.cpp +++ b/dbms/src/Columns/ColumnArray.cpp @@ -18,6 +18,7 @@ #include #include #include +#include namespace DB diff --git a/dbms/src/Common/HashTable/Hash.h b/dbms/src/Common/HashTable/Hash.h index cdcad677088..1d249c8e48a 100644 --- a/dbms/src/Common/HashTable/Hash.h +++ b/dbms/src/Common/HashTable/Hash.h @@ -2,6 +2,7 @@ #include #include +#include #include From 7246f819cde006890136fd2225547cf336fbf9d3 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Wed, 18 Mar 2020 20:08:27 +0300 Subject: [PATCH 0106/1355] Fix build. --- dbms/src/Common/HashTable/Hash.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/src/Common/HashTable/Hash.h b/dbms/src/Common/HashTable/Hash.h index 1d249c8e48a..97df2915417 100644 --- a/dbms/src/Common/HashTable/Hash.h +++ b/dbms/src/Common/HashTable/Hash.h @@ -71,7 +71,7 @@ inline DB::UInt64 intHashCRC32(DB::UInt64 x, DB::UInt64 updated_value) #endif } -UInt32 updateWeakHash32(const DB::UInt8 * pos, size_t size, DB::UInt32 updated_value) +inline UInt32 updateWeakHash32(const DB::UInt8 * pos, size_t size, DB::UInt32 updated_value) { if (size < 8) { From c33771105fefc17a1da7116cca03abfe690f826a Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Wed, 18 Mar 2020 21:35:58 +0300 Subject: [PATCH 0107/1355] partition deletion added --- dbms/programs/copier/ClusterCopier.cpp | 35 ++++++++++++++++++++------ dbms/programs/copier/ClusterCopier.h | 5 ++++ 2 files changed, 33 insertions(+), 7 deletions(-) diff --git a/dbms/programs/copier/ClusterCopier.cpp b/dbms/programs/copier/ClusterCopier.cpp index d3b9d0c3512..915cf36fa7c 100644 --- a/dbms/programs/copier/ClusterCopier.cpp +++ b/dbms/programs/copier/ClusterCopier.cpp @@ -1580,6 +1580,34 @@ void ClusterCopier::dropHelpingTables(const TaskTable & task_table) } } + +void ClusterCopier::dropParticularPartitionPieceFromAllHelpingTables(const TaskTable & task_table, const String & partition_name) +{ + LOG_DEBUG(log, "Try drop partition partition from all helping tables."); + for (size_t current_piece_number = 0; current_piece_number < task_table.number_of_splits; ++current_piece_number) + { + DatabaseAndTableName original_table = task_table.table_push; + DatabaseAndTableName helping_table = DatabaseAndTableName(original_table.first, original_table.second + "_piece_" + toString(current_piece_number)); + + String query = "ALTER TABLE " + getQuotedTable(helping_table) + " DROP PARTITION " + partition_name; + + const ClusterPtr & cluster_push = task_table.cluster_push; + Settings settings_push = task_cluster->settings_push; + + LOG_DEBUG(log, "Execute distributed DROP PARTITION: " << query); + /// We have to drop partition_piece on each replica + UInt64 num_nodes = executeQueryOnCluster( + cluster_push, query, + nullptr, + &settings_push, + PoolMode::GET_MANY, + ClusterExecutionMode::ON_EACH_NODE); + + LOG_DEBUG(log, "DROP PARTITION query was successfully executed on " << toString(num_nodes) << " nodes."); + } + LOG_DEBUG(log, "All helping tables dropped partition " << partition_name); +} + String ClusterCopier::getRemoteCreateTable(const DatabaseAndTableName & table, Connection & connection, const Settings * settings) { String query = "SHOW CREATE TABLE " + getQuotedTable(table); @@ -1655,8 +1683,6 @@ void ClusterCopier::createShardInternalTables(const ConnectionTimeouts & timeout task_shard.list_of_split_tables_on_shard[piece_number], storage_piece_split_ast); - std::cout << "anime" << queryToString(create_table_split_piece_ast) << std::endl; - dropAndCreateLocalTable(create_table_split_piece_ast); } } @@ -1819,11 +1845,6 @@ UInt64 ClusterCopier::executeQueryOnCluster( UInt64 num_replicas = cluster->getShardsAddresses().at(shard_index).size(); - for (size_t i = 0; i < num_replicas; ++i) - { - std::cout << "host_name " << cluster->getShardsAddresses().at(shard_index)[i].host_name - << " port " << cluster->getShardsAddresses().at(shard_index)[i].port << std::endl; - } origin_replicas_number += num_replicas; UInt64 num_local_replicas = shard.getLocalNodeCount(); UInt64 num_remote_replicas = num_replicas - num_local_replicas; diff --git a/dbms/programs/copier/ClusterCopier.h b/dbms/programs/copier/ClusterCopier.h index 60d63e37cf6..53415df515c 100644 --- a/dbms/programs/copier/ClusterCopier.h +++ b/dbms/programs/copier/ClusterCopier.h @@ -149,6 +149,11 @@ protected: void dropHelpingTables(const TaskTable & task_table); + /// Is used for usage less disk space. + /// After all pieces were successfully moved to original destination + /// table we can get rid of partition pieces (partitions in helping tables). + void dropParticularPartitionPieceFromAllHelpingTables(const TaskTable & task_table, const String & partition_name); + String getRemoteCreateTable(const DatabaseAndTableName & table, Connection & connection, const Settings * settings = nullptr); ASTPtr getCreateTableForPullShard(const ConnectionTimeouts & timeouts, TaskShard & task_shard); From fdd12bb8d7a9d9882ff0924a460750dd58d41642 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Wed, 18 Mar 2020 22:19:37 +0300 Subject: [PATCH 0108/1355] remote TODO --- dbms/programs/copier/ClusterCopier.h | 3 --- 1 file changed, 3 deletions(-) diff --git a/dbms/programs/copier/ClusterCopier.h b/dbms/programs/copier/ClusterCopier.h index 228e33b7b62..12ac5bdf47e 100644 --- a/dbms/programs/copier/ClusterCopier.h +++ b/dbms/programs/copier/ClusterCopier.h @@ -155,15 +155,12 @@ protected: ASTPtr getCreateTableForPullShard(const ConnectionTimeouts & timeouts, TaskShard & task_shard); /// If it is implicitly asked to create split Distributed table for certain piece on current shard, we will do it. - /// TODO: rewrite comment void createShardInternalTables(const ConnectionTimeouts & timeouts, TaskShard & task_shard, bool create_split = true); std::set getShardPartitions(const ConnectionTimeouts & timeouts, TaskShard & task_shard); bool checkShardHasPartition(const ConnectionTimeouts & timeouts, TaskShard & task_shard, const String & partition_quoted_name); - /// TODO: Implement checkPresentPartitionPiecesOnCurrentShard(); - /// Just copypaste the function above bool checkPresentPartitionPiecesOnCurrentShard(const ConnectionTimeouts & timeouts, TaskShard & task_shard, const String & partition_quoted_name, size_t current_piece_number); From 0a7f4e08cf803577cc2e42fde5f03d825dabd301 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Wed, 18 Mar 2020 23:18:01 +0300 Subject: [PATCH 0109/1355] style check fix --- dbms/programs/copier/Internals.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/programs/copier/Internals.cpp b/dbms/programs/copier/Internals.cpp index af41ce3ebbe..545df2e779c 100644 --- a/dbms/programs/copier/Internals.cpp +++ b/dbms/programs/copier/Internals.cpp @@ -171,7 +171,7 @@ ASTPtr extractOrderBy(const ASTPtr & storage_ast) String createCommaSeparatedStringFrom(const Names & names) { std::ostringstream ss; - if(!names.empty()) + if (!names.empty()) { std::copy(names.begin(), std::prev(names.end()), std::ostream_iterator(ss, ", ")); ss << names.back(); From 88782b4dced1b39028a75b2c64614ffbdd2fecb9 Mon Sep 17 00:00:00 2001 From: Artem Zuikov Date: Thu, 19 Mar 2020 00:38:27 +0300 Subject: [PATCH 0110/1355] Rewrite multiple JOINs v2 (set multiple_joins_rewriter_version = 2) --- dbms/src/Common/ErrorCodes.cpp | 2 +- dbms/src/Core/Settings.h | 1 + .../Interpreters/CrossToInnerJoinVisitor.cpp | 13 +- .../Interpreters/CrossToInnerJoinVisitor.h | 2 + .../Interpreters/InterpreterSelectQuery.cpp | 19 +- .../JoinToSubqueryTransformVisitor.cpp | 276 ++++++++++++++++-- .../JoinToSubqueryTransformVisitor.h | 12 +- dbms/src/Interpreters/QueryAliasesVisitor.cpp | 33 ++- dbms/src/Interpreters/QueryAliasesVisitor.h | 21 +- dbms/src/Interpreters/SyntaxAnalyzer.cpp | 3 +- 10 files changed, 329 insertions(+), 53 deletions(-) diff --git a/dbms/src/Common/ErrorCodes.cpp b/dbms/src/Common/ErrorCodes.cpp index dfb39f1624e..72e25db1b80 100644 --- a/dbms/src/Common/ErrorCodes.cpp +++ b/dbms/src/Common/ErrorCodes.cpp @@ -443,7 +443,7 @@ namespace ErrorCodes extern const int CANNOT_PTHREAD_ATTR = 468; extern const int VIOLATED_CONSTRAINT = 469; extern const int QUERY_IS_NOT_SUPPORTED_IN_LIVE_VIEW = 470; - extern const int SETTINGS_ARE_NOT_SUPPORTED = 471; + extern const int INVALID_SETTING_VALUE = 471; extern const int READONLY_SETTING = 472; extern const int DEADLOCK_AVOIDED = 473; extern const int INVALID_TEMPLATE_FORMAT = 474; diff --git a/dbms/src/Core/Settings.h b/dbms/src/Core/Settings.h index a3b94ad035b..445641b0e29 100644 --- a/dbms/src/Core/Settings.h +++ b/dbms/src/Core/Settings.h @@ -398,6 +398,7 @@ struct Settings : public SettingsCollection M(SettingBool, partial_revokes, false, "Makes it possible to revoke privileges partially.", 0) \ M(SettingBool, deduplicate_blocks_in_dependent_materialized_views, false, "Should deduplicate blocks for materialized views if the block is not a duplicate for the table. Use true to always deduplicate in dependent tables.", 0) \ M(SettingBool, use_compact_format_in_distributed_parts_names, false, "Changes format of directories names for distributed table insert parts.", 0) \ + M(SettingUInt64, multiple_joins_rewriter_version, 1, "1 or 2. Second rewriter version knows about table columns and keep not clashed names as is.", 0) \ \ /** Obsolete settings that do nothing but left for compatibility reasons. Remove each one after half a year of obsolescence. */ \ \ diff --git a/dbms/src/Interpreters/CrossToInnerJoinVisitor.cpp b/dbms/src/Interpreters/CrossToInnerJoinVisitor.cpp index c05e617f84d..0d07ad24efa 100644 --- a/dbms/src/Interpreters/CrossToInnerJoinVisitor.cpp +++ b/dbms/src/Interpreters/CrossToInnerJoinVisitor.cpp @@ -4,7 +4,6 @@ #include #include #include -#include #include #include #include @@ -95,7 +94,7 @@ public: CheckExpressionVisitorData(const std::vector & tables_, const std::vector & tables_with_columns, - Aliases && aliases_) + const Aliases & aliases_) : joined_tables(tables_) , tables(tables_with_columns) , aliases(aliases_) @@ -168,7 +167,7 @@ private: const std::vector & joined_tables; const std::vector & tables; std::map> asts_to_join_on; - Aliases aliases; + const Aliases & aliases; bool ands_only; size_t canMoveEqualsToJoinOn(const ASTFunction & node) @@ -323,13 +322,7 @@ void CrossToInnerJoinMatcher::visit(ASTSelectQuery & select, ASTPtr &, Data & da if (!select.where()) return; - Aliases aliases; - QueryAliasesVisitor::Data query_aliases_data{aliases}; - if (ASTPtr with = select.with()) - QueryAliasesVisitor(query_aliases_data).visit(with); - QueryAliasesVisitor(query_aliases_data).visit(select.select()); - - CheckExpressionVisitor::Data visitor_data{joined_tables, data.tables_with_columns, std::move(aliases)}; + CheckExpressionVisitor::Data visitor_data{joined_tables, data.tables_with_columns, data.aliases}; CheckExpressionVisitor(visitor_data).visit(select.where()); if (visitor_data.complex()) diff --git a/dbms/src/Interpreters/CrossToInnerJoinVisitor.h b/dbms/src/Interpreters/CrossToInnerJoinVisitor.h index 4c5ae97bc34..7cd5c93b1e3 100644 --- a/dbms/src/Interpreters/CrossToInnerJoinVisitor.h +++ b/dbms/src/Interpreters/CrossToInnerJoinVisitor.h @@ -1,6 +1,7 @@ #pragma once #include +#include namespace DB { @@ -15,6 +16,7 @@ public: struct Data { const std::vector & tables_with_columns; + const Aliases & aliases; const String current_database; bool done = false; }; diff --git a/dbms/src/Interpreters/InterpreterSelectQuery.cpp b/dbms/src/Interpreters/InterpreterSelectQuery.cpp index 3adc2eadbfc..93210b2aae2 100644 --- a/dbms/src/Interpreters/InterpreterSelectQuery.cpp +++ b/dbms/src/Interpreters/InterpreterSelectQuery.cpp @@ -52,6 +52,7 @@ #include #include #include +#include #include #include @@ -115,6 +116,7 @@ namespace ErrorCodes extern const int PARAMETER_OUT_OF_BOUND; extern const int INVALID_LIMIT_EXPRESSION; extern const int INVALID_WITH_FILL_EXPRESSION; + extern const int INVALID_SETTING_VALUE; } /// Assumes `storage` is set and the table filter (row-level security) is not empty. @@ -264,13 +266,24 @@ InterpreterSelectQuery::InterpreterSelectQuery( /// Rewrite JOINs if (!has_input && joined_tables.tablesCount() > 1) { - CrossToInnerJoinVisitor::Data cross_to_inner{joined_tables.tablesWithColumns(), context->getCurrentDatabase()}; + ASTSelectQuery & select = getSelectQuery(); + + Aliases aliases; + if (ASTPtr with = select.with()) + QueryAliasesNoSubqueriesVisitor(aliases).visit(with); + QueryAliasesNoSubqueriesVisitor(aliases).visit(select.select()); + + CrossToInnerJoinVisitor::Data cross_to_inner{joined_tables.tablesWithColumns(), aliases, context->getCurrentDatabase()}; CrossToInnerJoinVisitor(cross_to_inner).visit(query_ptr); - JoinToSubqueryTransformVisitor::Data join_to_subs_data{*context}; + size_t rewriter_version = settings.multiple_joins_rewriter_version; + if (!rewriter_version || rewriter_version > 2) + throw Exception("Bad multiple_joins_rewriter_version setting value: " + settings.multiple_joins_rewriter_version.toString(), + ErrorCodes::INVALID_SETTING_VALUE); + JoinToSubqueryTransformVisitor::Data join_to_subs_data{*context, joined_tables.tablesWithColumns(), aliases, rewriter_version}; JoinToSubqueryTransformVisitor(join_to_subs_data).visit(query_ptr); - joined_tables.reset(getSelectQuery()); + joined_tables.reset(select); joined_tables.resolveTables(); if (storage && joined_tables.isLeftTableSubquery()) diff --git a/dbms/src/Interpreters/JoinToSubqueryTransformVisitor.cpp b/dbms/src/Interpreters/JoinToSubqueryTransformVisitor.cpp index 39037613dda..67d6a4786d5 100644 --- a/dbms/src/Interpreters/JoinToSubqueryTransformVisitor.cpp +++ b/dbms/src/Interpreters/JoinToSubqueryTransformVisitor.cpp @@ -31,6 +31,15 @@ namespace ErrorCodes namespace { +ASTPtr makeSubqueryTemplate() +{ + ParserTablesInSelectQueryElement parser(true); + ASTPtr subquery_template = parseQuery(parser, "(select * from _t)", 0); + if (!subquery_template) + throw Exception("Cannot parse subquery template", ErrorCodes::LOGICAL_ERROR); + return subquery_template; +} + /// Replace asterisks in select_expression_list with column identifiers class ExtractAsterisksMatcher { @@ -296,6 +305,7 @@ struct SetSubqueryAliasVisitorData } }; +template bool needRewrite(ASTSelectQuery & select, std::vector & table_expressions) { if (!select.tables()) @@ -346,7 +356,7 @@ bool needRewrite(ASTSelectQuery & select, std::vector using AppendSemanticMatcher = OneTypeMatcher; using AppendSemanticVisitor = InDepthNodeVisitor; +/// V2 specific visitors + +struct CollectColumnIdentifiersMatcher +{ + using Data = std::vector; + + static bool needChildVisit(const ASTPtr & node, const ASTPtr &) + { + /// Do not go into subqueries. Do not collect table identifiers. + return !node->as() && + !node->as(); + } + + static void visit(const ASTPtr & ast, Data & data) + { + if (auto * t = ast->as()) + visit(*t, ast, data); + } + + static void visit(const ASTIdentifier & ident, const ASTPtr &, Data & data) + { + data.push_back(const_cast(&ident)); + } +}; + +struct TableNeededColumns +{ + const DatabaseAndTableWithAlias & table; + NameSet no_clashes = {}; + NameSet column_clashes = {}; /// It's column for sure + NameSet alias_clashes = {}; /// It's column or alias + + void fillExpressionList(ASTExpressionList & expression_list) const + { + size_t columns_count = no_clashes.size() + column_clashes.size() + alias_clashes.size(); + expression_list.children.reserve(expression_list.children.size() + columns_count); + + String table_name = table.getQualifiedNamePrefix(false); + + for (auto & column : no_clashes) + addShortName(column, expression_list); + + for (auto & column : column_clashes) + addAliasedName(table_name, column, expression_list); + + for (auto & column : alias_clashes) + addShortName(column, expression_list); + } + + static void addShortName(const String & column, ASTExpressionList & expression_list) + { + auto ident = std::make_shared(column); + expression_list.children.emplace_back(std::move(ident)); + } + + /// t.x as `t.x` + static void addAliasedName(const String & table, const String & column, ASTExpressionList & expression_list) + { + auto ident = std::make_shared(std::vector{table, column}); + ident->setAlias(table + '.' + column); + expression_list.children.emplace_back(std::move(ident)); + } +}; + +class SubqueryExpressionsRewriteMatcher +{ +public: + struct Data + { + ASTPtr expression_list; + const String & alias; + bool rewritten = false; + bool aliased = false; + }; + + static bool needChildVisit(ASTPtr & node, ASTPtr &) + { + return !node->as(); + } + + static void visit(ASTPtr & ast, Data & data) + { + if (auto * t = ast->as()) + visit(*t, ast, data); + if (auto * t = ast->as()) + visit(*t, ast, data); + } + +private: + static void visit(ASTSelectQuery & select, ASTPtr &, Data & data) + { + if (!data.rewritten) + select.setExpression(ASTSelectQuery::Expression::SELECT, std::move(data.expression_list)); + data.rewritten = true; + } + + static void visit(ASTSubquery &, ASTPtr & ast, Data & data) + { + if (!data.aliased) + ast->setAlias(data.alias); + data.aliased = true; + } +}; + +using CollectColumnIdentifiersVisitor = ConstInDepthNodeVisitor; +using SubqueryExpressionsRewriteVisitor = InDepthNodeVisitor; + } /// namelesspace void JoinToSubqueryTransformMatcher::visit(ASTPtr & ast, Data & data) { if (auto * t = ast->as()) - visit(*t, ast, data); + { + if (data.version == 1) + visitV1(*t, ast, data); + else + visitV2(*t, ast, data); + } } -void JoinToSubqueryTransformMatcher::visit(ASTSelectQuery & select, ASTPtr &, Data & data) +/// The reason for V2: not to alias columns without clashes, use better `t.x` style aliases for others. +void JoinToSubqueryTransformMatcher::visitV2(ASTSelectQuery & select, ASTPtr & ast, Data & data) +{ + std::vector table_expressions; + if (!needRewrite<2>(select, table_expressions)) + return; + + /// TODO: check table_expressions vs data.tables consistency + + /// Collect column identifiers + + std::vector identifiers; + CollectColumnIdentifiersVisitor(identifiers).visit(ast); + + /// JOIN sections + for (auto & child : select.tables()->children) + { + auto * table = child->as(); + if (table->table_join) + { + auto & join = table->table_join->as(); + if (join.on_expression) + CollectColumnIdentifiersVisitor(identifiers).visit(join.on_expression); + /// Nothing special for join.using_expression_list cause it contains short names + } + } + + /// Find clashes and normalize names: + /// 1. If column name has no clashes make all its occurrences short: 'table.column' -> 'column', 'table_alias.column' -> 'column'. + /// 2. If column name can't be short cause of same alias we keep it long converting 'table.column' -> 'table_alias.column' if any. + /// 3. If column clashes with another column keep their names long but convert 'table.column' -> 'table_alias.column' if any. + /// 4. If column clashes with another column and it's short - it's 'ambiguous column' error. + /// 5. If column clashes with alias add short column name to select list. It would be removed later if not needed. + /// @note Source query aliases should not clash with qualified names. + + std::vector needed_columns; + needed_columns.reserve(data.tables.size()); + for (auto & table : data.tables) + needed_columns.push_back(TableNeededColumns{table.table}); + NameSet alias_uses; + + for (ASTIdentifier * ident : identifiers) + { + bool got_alias = data.aliases.count(ident->name); + + if (auto table_pos = IdentifierSemantic::chooseTable(*ident, data.tables)) + { + const String & short_name = ident->shortName(); + if (!ident->isShort()) + { + if (got_alias) + throw Exception("Alias clashes with qualified column '" + ident->name + "'", ErrorCodes::AMBIGUOUS_COLUMN_NAME); + + size_t count = 0; + for (auto & table : data.tables) + if (table.hasColumn(short_name)) + ++count; + + if (count > 1 || data.aliases.count(short_name)) + { + auto & table = data.tables[*table_pos]; + IdentifierSemantic::setColumnLongName(*ident, table.table); /// table.column -> table_alias.column + needed_columns[*table_pos].column_clashes.emplace(short_name); + } + else + { + ident->setShortName(short_name); /// table.column -> column + needed_columns[*table_pos].no_clashes.emplace(short_name); + } + } + else if (got_alias) + needed_columns[*table_pos].alias_clashes.emplace(short_name); + else + needed_columns[*table_pos].no_clashes.emplace(short_name); + } + else if (got_alias) + alias_uses.insert(ident->name); + else + throw Exception("Unknown column name '" + ident->name + "'", ErrorCodes::UNKNOWN_IDENTIFIER); + } + + /// Rewrite tables + + auto & src_tables = select.tables()->children; + ASTPtr left_table = src_tables[0]; + + static ASTPtr subquery_template = makeSubqueryTemplate(); + static constexpr const char * join_subquery_alias = "--join"; + + for (size_t i = 1; i < src_tables.size() - 1; ++i) + { + String prev_join_alias = String(join_subquery_alias) + std::to_string(i-1); + String current_join_alias = String(join_subquery_alias) + std::to_string(i); + + auto expression_list = std::make_shared(); + { + if (i == 1) + { + /// First time extract needed left table columns manually + needed_columns[0].fillExpressionList(*expression_list); + } + else + { + /// Next times extract left tables via QualifiedAsterisk + auto asterisk = std::make_shared(); + asterisk->children.emplace_back(std::make_shared(prev_join_alias)); + expression_list->children.emplace_back(std::move(asterisk)); + } + + /// Add needed right table columns + needed_columns[i].fillExpressionList(*expression_list); + } + + ASTPtr subquery = subquery_template->clone(); + SubqueryExpressionsRewriteVisitor::Data expr_rewrite_data{std::move(expression_list), current_join_alias}; + SubqueryExpressionsRewriteVisitor(expr_rewrite_data).visit(subquery); + + left_table = replaceJoin(left_table, src_tables[i], subquery); + } + + RewriteVisitor::Data visitor_data{left_table, src_tables.back()}; + RewriteVisitor(visitor_data).visit(select.refTables()); + + data.done = true; +} + +void JoinToSubqueryTransformMatcher::visitV1(ASTSelectQuery & select, ASTPtr &, Data & data) { using RevertedAliases = AsteriskSemantic::RevertedAliases; @@ -425,9 +673,11 @@ void JoinToSubqueryTransformMatcher::visit(ASTSelectQuery & select, ASTPtr &, Da auto & src_tables = select.tables()->children; ASTPtr left_table = src_tables[0]; + static ASTPtr subquery_template = makeSubqueryTemplate(); + for (size_t i = 1; i < src_tables.size() - 1; ++i) { - left_table = replaceJoin(left_table, src_tables[i]); + left_table = replaceJoin(left_table, src_tables[i], subquery_template->clone()); if (!left_table) throw Exception("Cannot replace tables with subselect", ErrorCodes::LOGICAL_ERROR); @@ -451,16 +701,7 @@ void JoinToSubqueryTransformMatcher::visit(ASTSelectQuery & select, ASTPtr &, Da data.done = true; } -static ASTPtr makeSubqueryTemplate() -{ - ParserTablesInSelectQueryElement parser(true); - ASTPtr subquery_template = parseQuery(parser, "(select * from _t)", 0); - if (!subquery_template) - throw Exception("Cannot parse subquery template", ErrorCodes::LOGICAL_ERROR); - return subquery_template; -} - -ASTPtr JoinToSubqueryTransformMatcher::replaceJoin(ASTPtr ast_left, ASTPtr ast_right) +ASTPtr JoinToSubqueryTransformMatcher::replaceJoin(ASTPtr ast_left, ASTPtr ast_right, ASTPtr subquery_template) { const auto * left = ast_left->as(); const auto * right = ast_right->as(); @@ -470,13 +711,10 @@ ASTPtr JoinToSubqueryTransformMatcher::replaceJoin(ASTPtr ast_left, ASTPtr ast_r if (!right->table_join) throw Exception("Table join expected", ErrorCodes::LOGICAL_ERROR); - static ASTPtr subquery_template = makeSubqueryTemplate(); - /// replace '_t' with pair of joined tables - ASTPtr res = subquery_template->clone(); RewriteVisitor::Data visitor_data{ast_left, ast_right}; - RewriteVisitor(visitor_data).visit(res); - return res; + RewriteVisitor(visitor_data).visit(subquery_template); + return subquery_template; } } diff --git a/dbms/src/Interpreters/JoinToSubqueryTransformVisitor.h b/dbms/src/Interpreters/JoinToSubqueryTransformVisitor.h index 7b325a7cff2..e8d5e59802e 100644 --- a/dbms/src/Interpreters/JoinToSubqueryTransformVisitor.h +++ b/dbms/src/Interpreters/JoinToSubqueryTransformVisitor.h @@ -1,6 +1,8 @@ #pragma once #include +#include +#include namespace DB { @@ -17,6 +19,9 @@ public: struct Data { const Context & context; + const std::vector & tables; + const Aliases & aliases; + size_t version = 1; bool done = false; }; @@ -39,10 +44,13 @@ private: /// TablesInSelectQueryElement [source1] /// TablesInSelectQueryElement [source2] /// - static void visit(ASTSelectQuery & select, ASTPtr & ast, Data & data); + static void visitV1(ASTSelectQuery & select, ASTPtr & ast, Data & data); + + /// V2 uses information about tables' columns to rewrite queries. + static void visitV2(ASTSelectQuery & select, ASTPtr & ast, Data & data); /// @return combined TablesInSelectQueryElement or nullptr if cannot rewrite - static ASTPtr replaceJoin(ASTPtr left, ASTPtr right); + static ASTPtr replaceJoin(ASTPtr left, ASTPtr right, ASTPtr subquery_template); }; using JoinToSubqueryTransformVisitor = InDepthNodeVisitor; diff --git a/dbms/src/Interpreters/QueryAliasesVisitor.cpp b/dbms/src/Interpreters/QueryAliasesVisitor.cpp index 9caed01ca6d..f61eb5cddff 100644 --- a/dbms/src/Interpreters/QueryAliasesVisitor.cpp +++ b/dbms/src/Interpreters/QueryAliasesVisitor.cpp @@ -30,13 +30,21 @@ static String wrongAliasMessage(const ASTPtr & ast, const ASTPtr & prev_ast, con } -bool QueryAliasesMatcher::needChildVisit(const ASTPtr & node, const ASTPtr &) +bool QueryAliasesWithSubqueries::needChildVisit(const ASTPtr & node, const ASTPtr &) { /// Don't descent into table functions and subqueries and special case for ArrayJoin. return !(node->as() || node->as() || node->as()); } -void QueryAliasesMatcher::visit(const ASTPtr & ast, Data & data) +bool QueryAliasesNoSubqueries::needChildVisit(const ASTPtr & node, const ASTPtr & child) +{ + if (node->as()) + return false; + return QueryAliasesWithSubqueries::needChildVisit(node, child); +} + +template +void QueryAliasesMatcher::visit(const ASTPtr & ast, Data & data) { if (auto * s = ast->as()) visit(*s, ast, data); @@ -48,7 +56,8 @@ void QueryAliasesMatcher::visit(const ASTPtr & ast, Data & data) visitOther(ast, data); } -void QueryAliasesMatcher::visit(const ASTSelectQuery & select, const ASTPtr &, Data &) +template +void QueryAliasesMatcher::visit(const ASTSelectQuery & select, const ASTPtr &, Data &) { ASTPtr with = select.with(); if (!with) @@ -61,7 +70,8 @@ void QueryAliasesMatcher::visit(const ASTSelectQuery & select, const ASTPtr &, D /// The top-level aliases in the ARRAY JOIN section have a special meaning, we will not add them /// (skip the expression list itself and its children). -void QueryAliasesMatcher::visit(const ASTArrayJoin &, const ASTPtr & ast, Data & data) +template +void QueryAliasesMatcher::visit(const ASTArrayJoin &, const ASTPtr & ast, Data & data) { visitOther(ast, data); @@ -79,10 +89,10 @@ void QueryAliasesMatcher::visit(const ASTArrayJoin &, const ASTPtr & ast, Data & /// set unique aliases for all subqueries. this is needed, because: /// 1) content of subqueries could change after recursive analysis, and auto-generated column names could become incorrect /// 2) result of different scalar subqueries can be cached inside expressions compilation cache and must have different names -void QueryAliasesMatcher::visit(const ASTSubquery & const_subquery, const ASTPtr & ast, Data & data) +template +void QueryAliasesMatcher::visit(const ASTSubquery & const_subquery, const ASTPtr & ast, Data & aliases) { ASTSubquery & subquery = const_cast(const_subquery); - Aliases & aliases = data.aliases; static std::atomic_uint64_t subquery_index = 0; @@ -99,15 +109,14 @@ void QueryAliasesMatcher::visit(const ASTSubquery & const_subquery, const ASTPtr aliases[alias] = ast; } else - visitOther(ast, data); + visitOther(ast, aliases); subquery.prefer_alias_to_column_name = true; } -void QueryAliasesMatcher::visitOther(const ASTPtr & ast, Data & data) +template +void QueryAliasesMatcher::visitOther(const ASTPtr & ast, Data & aliases) { - Aliases & aliases = data.aliases; - String alias = ast->tryGetAlias(); if (!alias.empty()) { @@ -118,4 +127,8 @@ void QueryAliasesMatcher::visitOther(const ASTPtr & ast, Data & data) } } +/// Explicit template instantiations +template class QueryAliasesMatcher; +template class QueryAliasesMatcher; + } diff --git a/dbms/src/Interpreters/QueryAliasesVisitor.h b/dbms/src/Interpreters/QueryAliasesVisitor.h index 62242b500f7..6e79cfc77be 100644 --- a/dbms/src/Interpreters/QueryAliasesVisitor.h +++ b/dbms/src/Interpreters/QueryAliasesVisitor.h @@ -11,19 +11,27 @@ class ASTSubquery; struct ASTTableExpression; struct ASTArrayJoin; +struct QueryAliasesWithSubqueries +{ + static bool needChildVisit(const ASTPtr & node, const ASTPtr & child); +}; + +struct QueryAliasesNoSubqueries +{ + static bool needChildVisit(const ASTPtr & node, const ASTPtr & child); +}; + /// Visits AST node to collect aliases. +template class QueryAliasesMatcher { public: using Visitor = ConstInDepthNodeVisitor; - struct Data - { - Aliases & aliases; - }; + using Data = Aliases; static void visit(const ASTPtr & ast, Data & data); - static bool needChildVisit(const ASTPtr & node, const ASTPtr & child); + static bool needChildVisit(const ASTPtr & node, const ASTPtr & child) { return Helper::needChildVisit(node, child); } private: static void visit(const ASTSelectQuery & select, const ASTPtr & ast, Data & data); @@ -33,6 +41,7 @@ private: }; /// Visits AST nodes and collect their aliases in one map (with links to source nodes). -using QueryAliasesVisitor = QueryAliasesMatcher::Visitor; +using QueryAliasesVisitor = QueryAliasesMatcher::Visitor; +using QueryAliasesNoSubqueriesVisitor = QueryAliasesMatcher::Visitor; } diff --git a/dbms/src/Interpreters/SyntaxAnalyzer.cpp b/dbms/src/Interpreters/SyntaxAnalyzer.cpp index ce6f163ab5b..a4e08dd9ccb 100644 --- a/dbms/src/Interpreters/SyntaxAnalyzer.cpp +++ b/dbms/src/Interpreters/SyntaxAnalyzer.cpp @@ -893,8 +893,7 @@ void SyntaxAnalyzer::normalize(ASTPtr & query, Aliases & aliases, const Settings CustomizeFunctionsVisitor(data).visit(query); /// Creates a dictionary `aliases`: alias -> ASTPtr - QueryAliasesVisitor::Data query_aliases_data{aliases}; - QueryAliasesVisitor(query_aliases_data).visit(query); + QueryAliasesVisitor(aliases).visit(query); /// Mark table ASTIdentifiers with not a column marker MarkTableIdentifiersVisitor::Data identifiers_data{aliases}; From b623e3425b3e02453dd1f6669553adc8b8a08c9d Mon Sep 17 00:00:00 2001 From: nikitamikhaylov Date: Thu, 19 Mar 2020 02:01:43 +0400 Subject: [PATCH 0111/1355] fix build --- dbms/programs/copier/Internals.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/programs/copier/Internals.h b/dbms/programs/copier/Internals.h index 64a4fafd490..8e4f7afeb6e 100644 --- a/dbms/programs/copier/Internals.h +++ b/dbms/programs/copier/Internals.h @@ -198,7 +198,7 @@ ASTPtr extractPrimaryKey(const ASTPtr & storage_ast); ASTPtr extractOrderBy(const ASTPtr & storage_ast); -String createCommaSeparatedStringFrom(const Names & strings); +String createCommaSeparatedStringFrom(const Names & names); Names extractPrimaryKeyColumnNames(const ASTPtr & storage_ast); From f08771e563e4d59f9f43ad418c3cc67460060964 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Thu, 19 Mar 2020 04:15:01 +0300 Subject: [PATCH 0112/1355] parse tuple as literal if possible --- dbms/src/Parsers/ASTLiteral.cpp | 12 +++-- dbms/src/Parsers/ExpressionElementParsers.cpp | 26 +++++++---- dbms/src/Parsers/ExpressionElementParsers.h | 45 ++++++++++++++++--- dbms/src/Parsers/ParserPartition.cpp | 27 ++++++++--- dbms/tests/performance/set_index.xml | 13 +++++- 5 files changed, 97 insertions(+), 26 deletions(-) diff --git a/dbms/src/Parsers/ASTLiteral.cpp b/dbms/src/Parsers/ASTLiteral.cpp index 04504fb3594..1a75ad019e5 100644 --- a/dbms/src/Parsers/ASTLiteral.cpp +++ b/dbms/src/Parsers/ASTLiteral.cpp @@ -16,17 +16,21 @@ void ASTLiteral::updateTreeHashImpl(SipHash & hash_state) const void ASTLiteral::appendColumnNameImpl(WriteBuffer & ostr) const { - /// Special case for very large arrays. Instead of listing all elements, will use hash of them. + /// 100 - just arbitrary value. + constexpr auto MIN_ELEMENTS_FOR_HASHING = 100; + + /// Special case for very large arrays and tuples. Instead of listing all elements, will use hash of them. /// (Otherwise column name will be too long, that will lead to significant slowdown of expression analysis.) - if (value.getType() == Field::Types::Array - && value.get().size() > 100) /// 100 - just arbitrary value. + auto type = value.getType(); + if ((type == Field::Types::Array && value.get().size() > MIN_ELEMENTS_FOR_HASHING) + || (type == Field::Types::Tuple && value.get().size() > MIN_ELEMENTS_FOR_HASHING)) { SipHash hash; applyVisitor(FieldVisitorHash(hash), value); UInt64 low, high; hash.get128(low, high); - writeCString("__array_", ostr); + writeCString(type == Field::Types::Array ? "__array_" : "__tuple_", ostr); writeText(low, ostr); ostr.write('_'); writeText(high, ostr); diff --git a/dbms/src/Parsers/ExpressionElementParsers.cpp b/dbms/src/Parsers/ExpressionElementParsers.cpp index eb77d77a5c8..af676d94f77 100644 --- a/dbms/src/Parsers/ExpressionElementParsers.cpp +++ b/dbms/src/Parsers/ExpressionElementParsers.cpp @@ -990,15 +990,15 @@ bool ParserStringLiteral::parseImpl(Pos & pos, ASTPtr & node, Expected & expecte return true; } - -bool ParserArrayOfLiterals::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) +template +bool ParserCollectionOfLiterals::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) { - if (pos->type != TokenType::OpeningSquareBracket) + if (pos->type != opening_bracket) return false; Pos literal_begin = pos; - Array arr; + Collection arr; ParserLiteral literal_p; @@ -1008,9 +1008,16 @@ bool ParserArrayOfLiterals::parseImpl(Pos & pos, ASTPtr & node, Expected & expec { if (!arr.empty()) { - if (pos->type == TokenType::ClosingSquareBracket) + if (pos->type == closing_bracket) { - auto literal = std::make_shared(arr); + std::shared_ptr literal; + + /// Parse one-element tuples (e.g. (1)) as single values for backward compatibility. + if (std::is_same_v && arr.size() == 1) + literal = std::make_shared(arr[0]); + else + literal = std::make_shared(arr); + literal->begin = literal_begin; literal->end = ++pos; node = literal; @@ -1022,7 +1029,9 @@ bool ParserArrayOfLiterals::parseImpl(Pos & pos, ASTPtr & node, Expected & expec } else { - expected.add(pos, "comma or closing square bracket"); + std::stringstream msg; + msg << "comma or " << getTokenName(closing_bracket); + expected.add(pos, msg.str().c_str()); return false; } } @@ -1034,7 +1043,7 @@ bool ParserArrayOfLiterals::parseImpl(Pos & pos, ASTPtr & node, Expected & expec arr.push_back(literal_node->as().value); } - expected.add(pos, "closing square bracket"); + expected.add(pos, getTokenName(closing_bracket)); return false; } @@ -1235,6 +1244,7 @@ bool ParserSubstitution::parseImpl(Pos & pos, ASTPtr & node, Expected & expected bool ParserExpressionElement::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) { return ParserSubquery().parse(pos, node, expected) + || ParserTupleOfLiterals().parse(pos, node, expected) || ParserParenthesisExpression().parse(pos, node, expected) || ParserArrayOfLiterals().parse(pos, node, expected) || ParserArray().parse(pos, node, expected) diff --git a/dbms/src/Parsers/ExpressionElementParsers.h b/dbms/src/Parsers/ExpressionElementParsers.h index c82dfd7a62f..b2923118225 100644 --- a/dbms/src/Parsers/ExpressionElementParsers.h +++ b/dbms/src/Parsers/ExpressionElementParsers.h @@ -1,5 +1,6 @@ #pragma once +#include #include @@ -217,17 +218,49 @@ protected: }; -/** An array of literals. - * Arrays can also be parsed as an application of [] operator. - * But parsing the whole array as a whole constant seriously speeds up the analysis of expressions in the case of very large arrays. - * We try to parse the array as an array of literals first (fast path), - * and if it did not work out (when the array consists of complex expressions) - parse as an application of [] operator (slow path). +/** An array or tuple of literals. + * Arrays can also be parsed as an application of [] operator and tuples as an application of 'tuple' function. + * But parsing the whole array/tuple as a whole constant seriously speeds up the analysis of expressions in the case of very large collection. + * We try to parse the array or tuple as a collection of literals first (fast path), + * and if it did not work out (when the collection consists of complex expressions) - + * parse as an application of [] operator or 'tuple' function (slow path). */ +template +class ParserCollectionOfLiterals : public IParserBase +{ +public: + ParserCollectionOfLiterals(TokenType opening_bracket_, TokenType closing_bracket_) + : opening_bracket(opening_bracket_), closing_bracket(closing_bracket_) {} +protected: + const char * getName() const override { return "collection of literals"; } + bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override; +private: + TokenType opening_bracket; + TokenType closing_bracket; +}; + +class ParserTupleOfLiterals : public IParserBase +{ +public: + ParserCollectionOfLiterals tuple_parser{TokenType::OpeningRoundBracket, TokenType::ClosingRoundBracket}; +protected: + const char * getName() const override { return "tuple"; } + bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override + { + return tuple_parser.parse(pos, node, expected); + } +}; + class ParserArrayOfLiterals : public IParserBase { +public: + ParserCollectionOfLiterals array_parser{TokenType::OpeningSquareBracket, TokenType::ClosingSquareBracket}; protected: const char * getName() const override { return "array"; } - bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override; + bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override + { + return array_parser.parse(pos, node, expected); + } }; diff --git a/dbms/src/Parsers/ParserPartition.cpp b/dbms/src/Parsers/ParserPartition.cpp index 511db9a53b3..d5ba5739ab1 100644 --- a/dbms/src/Parsers/ParserPartition.cpp +++ b/dbms/src/Parsers/ParserPartition.cpp @@ -38,14 +38,34 @@ bool ParserPartition::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) String fields_str; const auto * tuple_ast = value->as(); + bool surrounded_by_parens = false; if (tuple_ast && tuple_ast->name == "tuple") { + surrounded_by_parens = true; const auto * arguments_ast = tuple_ast->arguments->as(); if (arguments_ast) fields_count = arguments_ast->children.size(); else fields_count = 0; + } + else if (auto literal = value->as()) + { + if (literal->value.getType() == Field::Types::Tuple) + { + surrounded_by_parens = true; + fields_count = literal->value.get().size(); + } + else + { + fields_count = 1; + fields_str = String(begin->begin, pos->begin - begin->begin); + } + } + else + return false; + if (surrounded_by_parens) + { Pos left_paren = begin; Pos right_paren = pos; @@ -61,13 +81,6 @@ bool ParserPartition::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) fields_str = String(left_paren->end, right_paren->begin - left_paren->end); } - else if (value->as()) - { - fields_count = 1; - fields_str = String(begin->begin, pos->begin - begin->begin); - } - else - return false; partition->value = value; partition->children.push_back(value); diff --git a/dbms/tests/performance/set_index.xml b/dbms/tests/performance/set_index.xml index e5848b08124..090d8ac8c08 100644 --- a/dbms/tests/performance/set_index.xml +++ b/dbms/tests/performance/set_index.xml @@ -13,8 +13,19 @@ - + SELECT count() FROM test_in WHERE a IN (SELECT rand(1) FROM zeros(100000)) SETTINGS max_rows_to_read = 1, read_overflow_mode = 'break' + + SELECT count() FROM numbers(1000) WHERE toString(number) IN ('41577', '83972', '51697', '50014', '37553', '93459', '87438', '95971', '83186', '74326', '67871', '50406', '83678', '29655', '18580', '83905', '61518', '29059', '56700', '82787', '98672', '30884', '81822', '39850', '80852', '57627', '91346', '64522', '17781', '49467', '41099', '41929', '85618', '91389', '68564', '91769', '81219', '52218', '37220', '97097', '2129', '9886', '52049', '34847', '25364', '36429', '76897', '71868', '58121', '71199', '84819', '69991', '34046', '64507', '34892', '24228', '36986', '28588', '51159', '53444', '80531', '9941', '20256', '48103', '32565', '62890', '5379', '60302', '46434', '3205', '18821', '31030', '19794', '71557', '71703', '15024', '14004', '82164', '95659', '40227', '83358', '24395', '9610', '19814', '48491', '66412', '16012', '71586', '42143', '51103', '24463', '89949', '35694', '39193', '63904', '40489', '77144', '94014', '84836', '9980', '46554', '43905', '25588', '25205', '72624', '10249', '35888', '98478', '99030', '26834', '31', '81499', '14847', '82997', '92357', '92893', '17426', '56630', '22252', '68119', '62710', '8740', '82144', '79916', '23391', '30192', '99271', '96435', '44237', '98327', '69481', '16691', '13643', '84554', '38571', '70926', '99283', '79000', '20926', '86495', '4834', '1222', '39486', '57697', '58002', '40790', '15623', '3999', '31515', '12694', '26143', '35951', '54085', '97534', '35329', '73535', '88715', '29572', '75799', '45166', '32066', '48023', '69523', '93150', '8740', '96790', '15534', '63252', '5142', '67045', '93992', '16663', '292', '63924', '6588', '12190', '31506', '69590', '35394', '55168', '65223', '79183', '32600', '69676', '28316', '72111', '53531', '15073', '41127', '73451', '24725', '61647', '65315', '41143', '26493', '95608', '34407', '76098', '53105', '83691', '48755', '35696', '62587', '81826', '3963', '45766', '82751', '12430', '97685', '29919', '78155', '71636', '50215', '89734', '9892', '47151', '54855', '3428', '9712', '52592', '2403', '79602', '81243', '79859', '57361', '82000', '42107', '28860', '99591', '28296', '57337', '64969', '32332', '25535', '30924', '21313', '32554', '17342', '87311', '19825', '24898', '61323', '83209', '79322', '79009', '50746', '33396', '62033', '16548', '17427', '24073', '34640', '52368', '4724', '80408', '40', '33787', '16666', '19665', '86751', '27264', '2241', '88134', '53566', '10589', '79711', '92823', '58972', '91767', '60885', '51659', '7867', '96849', '30360', '20914', '9584', '1250', '22871', '23282', '99312', '4683', '33429', '68361', '82614', '81440', '47863', '69790', '11968', '75210', '66854', '37002', '61142', '71514', '1588', '42336', '11069', '26291', '2261', '71056', '13492', '9133', '91216', '72207', '71586', '86535', '83898', '24392', '45384', '48545', '61972', '503', '80180', '35834', '97025', '70411', '55039', '35430', '27631', '82533', '96831', '74077', '42533', '14451', '26943', '53783', '69489', '71969', '8432', '37230', '61348', '19472', '59115', '9886', '50951', '57109', '7141', '1902', '84130', '4323', '55889', '47784', '2220', '75988', '66988', '63721', '8131', '95601', '95207', '2311', '26541', '50991', '6717', '2969', '71857', '51034', '65958', '94716', '90275', '21012', '46859', '7984', '31131', '46457', '69578', '44540', '7294', '80117', '9925', '60155', '90608', '82684', '32193', '87071', '28006', '87604', '24501', '79087', '2848', '29237', '11221', '81319', '40966', '87641', '35325', '78705', '88636', '78717', '62831', '56390', '99271', '43821', '14453', '17923', '62695', '77322', '21038', '67677', '41271', '4376', '65426', '46091', '19887', '97251', '55583', '58763', '3826', '35037', '73533', '64267', '82319', '9836', '42622', '96829', '16363', '10455', '49290', '99992', '98229', '66356', '59087', '73998', '25986', '4279', '56790', '69540', '588', '36620', '60358', '45056', '89297', '42740', '8323', '19245', '82417', '41431', '699', '11554', '73910', '44491', '56019', '68901', '45816', '68126', '89379', '23885', '13263', '56395', '73130', '19089', '23771', '10335', '48547', '16903', '6453', '33560', '89668', '38159', '43177', '90655', '49712', '62', '66920', '34180', '12150', '48564', '39538', '85026', '87195', '14928', '8956', '71157', '53287', '39161', '67583', '83309', '92054', '86977', '56188', '15229', '88170', '60894', '58497', '89254', '40082', '86890', '60161', '97291', '45878', '23368', '14577', '92870', '37017', '97356', '99426', '76061', '89186', '99751', '85153', '61580', '39360', '90107', '25603', '26798', '76224', '6469', '7912', '69838', '16404', '67497', '28965', '80836', '80365', '91249', '48713', '17113', '33090', '40793', '70450', '66689', '83698', '17802', '43869', '13355', '18959', '79411', '87930', '9265', '37504', '44876', '97234', '94149', '35040', '22049', '49248', '6535', '36080', '28346', '94437', '78319', '17961', '89056', '56161', '35810', '41632', '45494', '53351', '89729', '99510', '51584', '59688', '6193', '70809', '51093', '92589', '90247', '34910', '78235', '17362', '49423', '63324', '525', '37638', '72325', '89356', '15298', '59116', '17848', '65429', '27029', '84781', '70247', '8825', '35082', '70451', '22522', '58125', '91879', '90531', '2478', '463', '37902', '54405', '87267', '72688', '22803', '33134', '35177', '84551', '44974', '88375', '76407', '27774', '33849', '19915', '82014', '80434', '26380', '48777', '53811', '14838', '26829', '56441', '99869', '49574', '85476', '19723', '16907', '4018', '37338', '78510', '47912', '13030', '65277', '95716', '67363', '21393', '89887', '78842', '81650', '903', '17436', '30704', '49223', '27198', '25500', '52214', '54258', '70082', '53950', '49312', '43615', '99473', '94348', '53661', '96213', '96346', '62010', '38268', '32861', '75660', '10392', '89491', '68335', '29817', '88706', '24184', '36298', '43440', '21626', '26535', '44560', '46363', '12534', '99070', '95606', '33714', '73070', '8303', '29853', '23014', '99982', '4530', '14955', '45803', '50', '90750', '30394', '81276', '95563', '47314', '58520', '91299', '88944', '54402', '67405', '29253', '47079', '71734', '99728', '17652', '13307', '35556', '18962', '26780', '17771', '53712', '60055', '37628', '35830', '90739', '61151', '41309', '27652', '3051', '53167', '98417', '19382', '36833', '75085', '65374', '87732', '30352', '31776', '32765', '97565', '92199', '49050', '29503', '51024', '18834', '8515', '24069', '96216', '10777', '90680', '18974', '68884', '85305', '36007', '56707', '4212', '47352', '34426', '13185', '92939', '95782', '70577', '58080', '98279', '3906', '5065', '56896', '16382', '31273', '17117', '98602', '12786', '24086', '63970', '72756', '35798', '82367', '7356', '53398', '68503', '2962', '16425', '67334', '68461', '65439', '15620', '70906', '29649', '46461', '74602', '38012', '71714', '16825', '89480', '53386', '88532', '35104', '28556', '82120', '23155', '23347', '24797', '60061', '54962', '99427', '82248', '82447', '39968', '63727', '27431', '81511', '91168', '71425', '80740', '84127', '40717', '15503', '15419', '46594', '61263', '19212', '53175', '70724', '74445', '23034', '71818', '40246', '18886', '53066', '4880', '83701', '86107', '87862', '44751', '392', '73440', '90291', '93395', '20894', '38463', '32664', '55158', '20090', '50004', '79070', '98471', '85478', '96615', '68149', '78334', '97752', '73207', '71678', '91238', '96757', '82598', '194', '35797', '45120', '60782', '28721', '17676', '78066', '60957', '11826', '51563', '50516', '16485', '47053', '31738', '48923', '23554', '96850', '42033', '73701', '78607', '45979', '54571', '12415', '31693', '15356', '36902', '9126', '3767', '3295', '90402', '24005', '95350', '67033', '49137', '72606', '51899', '17522', '31957', '44641', '53982', '23767', '68257', '15766', '19995', '2107', '48788', '11765', '91055', '46576', '54651', '50381', '62827', '73636', '46606', '98753', '37631', '70441', '87916', '66983', '33870', '31125', '12904', '57040', '4874', '58632', '42037', '18782', '5998', '18974', '57949', '81010', '90407', '99874', '20462', '89949', '10952', '71454', '95130', '46115', '3518', '13384', '69039', '79482', '22076', '59782', '32042', '40930', '60243', '29298', '6790', '46985', '44398', '85631', '14380', '66179', '2629', '32126', '49833', '14118', '58492', '31493', '81172', '96638', '8745', '89663', '76842', '78633', '41373', '83721', '42886', '11123', '32739', '11051', '1303', '92314', '83324', '85600', '44276', '69064', '56125', '84650', '31028', '12628', '14502', '64764', '39405', '44855', '79046', '51716', '46824', '83389', '1941', '1257', '9280', '73176', '84729', '2579', '63366', '22606', '35541', '51096', '13447', '18355', '68037', '28436', '94116', '81070', '78355', '67897', '5296', '32742', '77645', '91853', '18767', '67949', '40963', '5792', '17278', '25597', '41884', '80829', '7099', '18645', '60295', '12082', '81800', '78415', '18082', '38789', '16295', '72377', '74949', '55583', '66853', '15402', '72977', '15123', '99434', '34999', '21687', '76049', '42987', '83748', '88256', '66688', '21766', '20304', '29271', '10069', '19822', '11792', '42526', '74143', '17289', '30253', '6367', '20888', '12975', '94073', '98639', '30134', '26320', '65507', '69002', '53120', '4550', '38893', '18954', '38283', '54863', '17698', '99670', '10521', '92467', '60994', '18052', '48673', '35811', '87282', '62706', '16061', '53112', '22652', '37780', '55662', '26331', '49410', '79074', '10623', '69577', '79613', '9491', '31229', '43922', '84231', '58409', '36386', '46875', '74431', '76735', '38776', '23350', '7314', '9079', '51519', '98544', '70216', '63380', '90381', '1295', '46901', '58225', '55339', '89918', '75522', '35431', '89460', '49552', '89302', '23068', '28493', '3042', '25194', '59520', '9810', '95706', '81297', '89638', '54794', '94527', '45262', '97932', '78685', '6947', '22818', '48700', '9153', '12289', '22011', '58825', '93854', '65438', '4509', '33741', '28208', '69061', '48578', '40247', '77725', '31837', '39003', '69363', '78113', '76398', '97262', '67795', + '68446', '58896', '60969', '19849', '6722', '91854', '49519', '13949', '67109', '48824', '31723', '75554', '69575', '94986', '75350', '18628', '15284', '41943', '15433', '52607', '41', '22340', '29528', '24059', '34145', '72517', '46316', '10667', '54510', '19882', '47764', '69124', '41963', '84350', '48420', '4646', '24958', '69020', '97121', '26178', '62664', '50622', '32554', '49655', '45398', '11267', '72222', '73363', '89554', '89046', '57384', '29259', '37888', '24850', '74353', '57343', '34762', '2900', '11393', '42154', '94306', '70552', '75265', '921', '26003', '64352', '89857', '83171', '58249', '48940', '53512', '66335', '44865', '68729', '19299', '58003', '39854', '99122', '3860', '80173', '52242', '90966', '53183', '71982', '82325', '87842', '15000', '55627', '71132', '6354', '42402', '91719', '91644', '94533', '74925', '66278', '66911', '85576', '40495', '70919', '71797', '87835', '29845', '71832', '3390', '7994', '33499', '70715', '54897', '82710', '63077', '78105', '24758', '89585', '84607', '46477', '78618', '10989', '39222', '98749', '51685', '94664', '31008', '32823', '89521', '72160', '26952', '4001', '21421', '5737', '74027', '88179', '45553', '83743', '19913', '49435', '65616', '82641', '5149', '76959', '40681', '73023', '2670', '30845', '18863', '35094', '88400', '80963', '9154', '16571', '64192', '59694', '41317', '59942', '58856', '99281', '67260', '66971', '22716', '76089', '58047', '67071', '53707', '462', '52518', '72277', '10681', '69', '98855', '12593', '88842', '67242', '73543', '37439', '18413', '67211', '93495', '45576', '70614', '27988', '53210', '18618', '21318', '68059', '25518', '55917', '56522', '16548', '2404', '93538', '61452', '66358', '3709', '23914', '92426', '81439', '38070', '28988', '29939', '2948', '85720', '45628', '51101', '89431', '86365', '17571', '50987', '83849', '11015', '83812', '66187', '26362', '66786', '22024', '93866', '36161', '90080', '64874', '37294', '83860', '73821', '80279', '36766', '73117', '44620', '84556', '42070', '90383', '27862', '20665', '67576', '34997', '57958', '80638', '84351', '63961', '1362', '14338', '80377', '24192', '41294', '57368', '51189', '27287', '45764', '86289', '65600', '708', '84090', '96005', '55676', '84855', '72385', '70018', '9336', '82701', '3710', '52083', '74045', '96454', '30956', '67369', '78941', '81810', '71906', '23194', '33042', '50794', '61256', '24449', '48639', '22916', '78303', '13666', '40762', '43942', '51075', '89783', '95786', '90462', '6181', '36482', '40675', '4970', '6388', '91849', '72579', '94983', '86084', '20140', '68427', '48123', '43122', '98066', '37560', '6927', '72803', '5546', '62259', '98439', '6457', '98568', '70499', '33022', '28226', '29675', '20917', '75365', '20900', '8190', '56736', '99153', '77779', '49333', '50293', '97650', '4067', '47278', '42761', '71875', '13966', '11223', '46783', '18059', '61355', '29638', '75681', '24466', '89634', '20759', '83252', '37780', '15931', '74893', '6703', '64524', '80656', '85990', '78427', '18411', '20696', '86432', '93176', '69889', '15072', '15180', '9935', '10467', '60248', '42430', '62590', '89596', '27743', '26398', '79912', '60048', '50943', '38870', '69383', '72261', '98059', '55242', '74905', '5667', '54321', '70415', '39903', '49711', '85318', '79979', '59262', '82321', '15263', '17416', '74554', '94733', '72112', '49872', '54849', '73883', '78250', '74935', '68559', '57564', '50541', '45730', '41595', '5588', '83723', '42891', '11898', '14348', '99732', '14481', '85233', '21277', '94508', '52551', '74187', '7634', '42912', '25100', '43536', '35798', '48190', '86477', '22680', '48148', '59501', '56563', '16802', '81496', '97568', '68657', '51462', '67953', '99660', '39002', '54170', '57190', '68086', '52700', '6487', '55709', '70418', '62629', '70420', '35695', '36152', '45360', '53503', '46623', '76000', '50648', '97876', '44815', '29163', '1356', '64123', '71388', '17658', '99084', '58727', '59437', '38773', '71254', '81286', '97545', '18786', '56834', '20346', '36401', '62316', '58082', '67959', '99876', '69895', '80099', '62747', '20517', '99777', '6472', '49189', '31321', '39992', '68073', '13378', '51806', '21776', '52060', '96983', '25754', '93709', '96627', '8644', '93726', '14002', '37716', '87620', '34507', '76339', '24491', '5849', '44110', '522', '66521', '12776', '44887', '80535', '14548', '75248', '671', '73071', '35715', '59474', '7061', '82243', '56170', '20179', '59717', '1725', '24634', '11270', '77023', '63840', '46608', '44667', '22422', '59771', '94768', '73033', '82905', '16463', '40971', '22204', '58366', '28721', '14907', '76468', '81872', '38418', '36989', '61439', '10610', '131', '44296', '35453', '10117', '75856', '94603', '99602', '68075', '35949', '13599', '50030', '69633', '55956', '85465', '16429', '86081', '11145', '6195', '82207', '90598', '92814', '23725', '83204', '80346', '71542', '46634', '15820', '54123', '45397', '15322', '61743', '9273', '71347', '6835', '64006', '91718', '43677', '32923', '21486', '17098', '61694', '43347', '40019', '4071', '52443', '42386', '56839', '83514', '27633', '40780', '51749', '92101', '62384', '92206', '56044', '66174', '11137', '73966', '78471', '30468', '31643', '33197', '6888', '8066', '86603', '74383', '6098', '54411', '98819', '89862', '88639', '94422', '89371', '80526', '91747', '91220', '64944', '76658', '42046', '58518', '27249', '6646', '3028', '1346', '33763', '9734', '31737', '65527', '5892', '60813', '3410', '35464', '43009', '98382', '70580', '93898', '56404', '32995', '62771', '71556', '40538', '55612', '45656', '10758', '20268', '33603', '38310', '14242', '74397', '10722', '71575', '22590', '49043', '91439', '9055', '23668', '9101', '5268', '64133', '77501', '64684', '11337', '47575', '50732', '88680', '93730', '46785', '17589', '3520', '57595', '71241', '34994', '8753', '36147', '88844', '41914', '11250', '94632', '71927', '4624', '86279', '7664', '2659', '94853', '65386', '30438', '86005', '92883', '84629', '59910', '44484', '1306', '8404', '56962', '29990', '38445', '96191', '73013', '66590', '40951', '24712', '18825', '37268', '87843', '18972', '12154', '7779', '52149', '76152', '65799', '86011', '35475', '78083', '88232', '91551', '65532', '93516', '73827', '24227', '44687', '55759', '83819', '45088', '10856', '60488', '39051', '14103', '76650', '81181', '46731', '737', '58788', '78945', '42096', '66731', '66740', '72273', '88969', '5655', '86590', '41096', '80038', '32430', '51877', '23970', '91900', '13082', '45880', '94367', '19739', '61998', '71665', '16083', '57035', '26916', '10166', '18834', '46798', '66881', '28444', '68840', '10459', '81087', '4728', '76224', '39257', '23470', '93524', '37345', '30074', '49856', '22022', '55279', '5159', '5193', '58030', '57539', '12514', '49759', '96222', '52597', '67192', '88187', '53614', '16084', '79915', '28212', '79334', '85283', '32306', '31058', '43113', '74707', '74869', '2213', '32134', '6379', '85426', '87098', '35984', '51105', '69287', '16803', '83337', '14913', '62531', '58098', '7914', '20105', '28850', '1384', '43173', '62983', '87113', '76066', '86320', '77684', '45191', '95225', '41503', '36713', '48404', '91228', '53865', '98981', '59161', '61237', '84561', '17455', '14379', '57789', '80895', '99260', '84595', '72942', '53220', '84448', '81332', '49437', '83086', '93414', '54519', '52288', '74772', '22460', '49324', '11168', '96071', '61985', '38284', '6405', '54698', '71727', '60093', '37340', '87884', '83403', '4542', '94949', '19636', '15855', '39105', '10424', '67418', '91022', '69254', '8481', '38411', '3832', '44354', '93548', '57172', '28481', '372', '81497', '52179', '41060', '72141', '41396', '65590', '70432', '82819', '93814', '26118', '84780', '88485', '70821', '8222', '83000', '47067', '38516', '33347', '47681', '48202', '60749', '52112', '7937', '28105', '11394', '45746', '43252', '34494', '2979', '69715', '42486', '82315', '71760', '97413', '66137', '94487', '7429', '74434', '22964', '55251', '3448', '53534', '2574', '9693', '96157', '2955', '4348', '19566', '56930', '83319', '31310', '53905', '1148', '41726', '22233', '76045', '37351', '10545', '17581', '28047', '30199', '4741', '58111', '33497', '67796', '67730', '31247', '43772', '29461', '45970', '73353', '22534', '53962', '32147', '71392', '62579', '66345', '58246', '33442', '9581', '29705', '14058', '86471', '76125', '59363', '94982', '74810', '89149', '20066', '3366', '3568', '25752', '80036', '64119', '27270', '40061', '91052', '69022', '9852', '77112', '83075', '43924', '61661', '56133', '96652', '57944', '72576', '82170', '79236', '55745', '15309', '88878', '72761', '37647', '67465', '12777', '97309', '93202', '41470', '8787', '64920', '48514', '18917', '35157', '59151', '4640', '5317', '38134', '76548', '82788', '9214', '58418', '73185', '90554', '10543', '47182', '62936', '91765', '89751', '68931', '48865', '64607', '7150', '77862', '14297', '14828', '33013', '91698', '67593', '98096', '16595', '51639', '86531', '24719', '1703', '78788', '43810', '38918', '95491', '99903', '82671', '8291', '68288', '31224', '39863', '4265', '77798', '7698', '33804', '92286', '4744', '37038', '44203', '98212', '17369', '77442', '62879', '4145', '96881', '15646', '36824', '19959', '45451', '76049', '54272', '97577', '95298', '81115', '30204', '82041', '8037', '10052', '8756', '76833', '82851', '24276', '75574', '36037', '78079', '92807', '29064', '90000', '84150', '17102', '75092', '49424', '35597', '4693', '82853', '42511', '16119', '23478', '65240', '55585', '91762', '71671', '46682', '72479', '97696', '24615', '12579', '30274', '48255', '2336', '90202', '5808', '45426', '76308', '74639', '31245', '99894', '89638', '6233', '33893', '71899', '85273', '89429', '29761', '50231', '57249', '99347', '22642', '66972', '86221', '47514', '88274', '10819', '73150', '53754', '13304', '20478', '38099', '619', '14669', '8011', '97657', '26569', '65430', '13467', '38180', '23675', '72350', '42257', '39875', '23529', '53407', '11833', + '29599', '95621', '7727', '59527', '86846', '22860', '5358', '3730', '87555', '362', '95755', '54565', '29935', '68950', '52349', '98344', '86576', '7420', '12236', '15844', '48099', '97535', '97081', '50261', '31187', '60496', '24123', '24042', '6376', '6679', '99806', '20306', '60676', '36881', '77309', '5247', '96569', '53417', '73252', '64179', '35318', '75732', '65119', '32621', '40464', '22887', '96152', '65161', '83381', '8915', '68142', '7328', '85031', '15688', '72519', '93992', '86927', '75538', '38205', '50877', '70039', '97538', '94822', '52131', '49643', '85206', '1347', '14574', '88736', '53442', '49991', '64925', '72283', '82213', '60905', '36118', '62963', '16983', '79185', '15111', '26059', '17792', '98218', '33214', '1094', '41754', '77275', '65173', '13190', '91004', '90422', '44387', '92672', '98641', '54609', '83295', '37395', '70104', '32986', '72524', '82478', '5837', '83916', '52736', '57112', '55985', '42642', '42136', '89642', '35712', '49489', '19726', '65824', '24384', '48112', '15366', '99206', '68384', '51389', '529', '21475', '75749', '95182', '60110', '70571', '74174', '38105', '78107', '4101', '8982', '11215', '23987', '3303', '28706', '54629', '98000', '67510', '30036', '99140', '48896', '40971', '7735', '79984', '50134', '94928', '57023', '52880', '83067', '41940', '62994', '89213', '38593', '19283', '68206', '22234', '19245', '26266', '32403', '65889', '17022', '64280', '42797', '27161', '57675', '42313', '93606', '93082', '20659', '90824', '1226', '66266', '12503', '57104', '15247', '51160', '92398', '71967', '59476', '44465', '35765', '10787', '47737', '45792', '2292', '47599', '89612', '8162', '87622', '69410', '45727', '31158', '99791', '89544', '27214', '99588', '40516', '75616', '36505', '46079', '95448', '97999', '47462', '47799', '82729', '34038', '60789', '96938', '22682', '79062', '93307', '36038', '49016', '90983', '48219', '50889', '32517', '72219', '71229', '82643', '1195', '70543', '17', '22178', '23544', '72371', '1163', '28527', '7336', '39846', '31956', '80963', '41804', '59791', '41831', '1940', '52377', '79494', '12531', '81112', '44320', '18746', '5774', '63869', '4085', '59922', '12751', '99443', '13530', '23872', '36026', '83360', '32711', '92980', '11140', '99323', '57263', '98149', '29265', '25548', '65995', '4818', '15593', '8535', '37863', '12217', '14474', '66584', '89272', '86690', '58777', '39666', '44756', '18442', '52586', '98030', '40850', '38708', '49304', '68923', '65008', '84388', '83639', '29866', '63675', '26793', '49227', '82099', '24090', '57535', '24201', '65776', '74054', '89833', '62979', '26613', '5851', '99766', '63484', '66605', '37179', '90760', '59336', '58390', '93239', '84578', '11396', '93994', '73818', '23972', '37720', '72369', '25063', '32952', '71036', '76612', '31285', '34090', '19136', '53783', '66436', '61478', '96749', '43658', '7399', '31574', '67073', '40480', '20727', '70993', '65549', '30800', '21507', '53785', '89574', '86381', '56492', '62603', '44856', '68687', '63794', '70996', '7475', '84238', '71939', '86886', '94792', '15036', '36936', '95722', '17771', '67850', '33371', '49314', '40744', '5432', '81057', '41201', '75986', '22961', '15323', '1570', '18657', '95219', '19130', '53127', '15867', '81135', '73206', '76668', '36386', '48828', '31417', '56916', '70891', '60534', '95777', '10022', '94053', '2928', '56326', '16559', '79656', '6414', '81247', '78270', '55687', '19151', '61597', '99857', '81142', '27725', '53493', '12185', '1455', '48501', '59425', '20591', '24900', '66079', '84889', '32024', '18919', '2043', '7076', '71201', '88258', '86521', '93348', '26395', '39646', '44145', '33911', '46231', '67054', '39979', '11630', '23020', '76278', '88056', '11480', '4723', '78612', '70211', '60622', '84687', '59092', '65675', '38479', '64399', '64699', '95964', '42764', '69060', '28189', '4193', '95805', '75462', '17245', '59640', '94773', '84292', '53092', '98507', '61353', '32483', '53027', '48912', '87221', '47788', '59263', '65196', '35567', '17494', '64253', '50223', '7057', '87467', '62414', '2523', '50910', '72353', '78986', '78104', '47719', '29108', '12957', '5114', '64435', '66707', '37449', '70399', '45334', '71606', '55338', '55072', '58765', '12151', '22012', '16954', '87366', '14240', '98041', '72296', '47408', '56879', '99584', '63172', '92316', '28071', '29880', '19608', '13839', '87484', '56541', '88662', '87098', '72124', '78282', '27653', '38993', '31870', '67239', '99445', '7376', '78487', '98880', '12180', '86773', '67773', '15416', '58172', '13075', '67559', '97510', '29705', '86985', '57024', '11827', '31236', '91920', '26116', '94614', '14486', '46252', '78847', '43786', '70048', '96739', '35240', '39933', '58209', '27852', '65669', '47323', '58150', '84444', '44344', '95882', '41258', '31314', '69060', '19916', '6979', '19436', '45572', '16259', '74566', '6306', '24705', '53422', '593', '97031', '22308', '26875', '23042', '78035', '34229', '61976', '23175', '50072', '90896', '50810', '71730', '86468', '94807', '8218', '36032', '58628', '60560', '51206', '37943', '27987', '15014', '49905', '70018', '66799', '80851', '23594', '29982', '6438', '97381', '47715', '96294', '17985', '48545', '12672', '5250', '9988', '24601', '3736', '97815', '54363', '64703', '44167', '68376', '16595', '38073', '29630', '59630', '1858', '71823', '75580', '70083', '14493', '93821', '93394', '85369', '3818', '8435', '59988', '43966', '13961', '15855', '83332', '80312', '27299', '88840', '76964', '56173', '62794', '79389', '82642', '85843', '47116', '43064', '16061', '28905', '54415', '72832', '91252', '93488', '79457', '99336', '70744', '80432', '6487', '880', '87701', '154', '86574', '86677', '17892', '81488', '95260', '12515', '43189', '9211', '55403', '41417', '60046', '54785', '83655', '28274', '65745', '63062', '44549', '36391', '48051', '7328', '3572', '33226', '49177', '25123', '59065', '19691', '15109', '10172', '95578', '29497', '48152', '20276', '36270', '78866', '48309', '53209', '55475', '30073', '19717', '16004', '45692', '83430', '9291', '45935', '57030', '92613', '91656', '67697', '34915', '28156', '56594', '3273', '11194', '98270', '34370', '2621', '66679', '97451', '97717', '87923', '48310', '37725', '69743', '75103', '84956', '75163', '16069', '65304', '19397', '18071', '27273', '49823', '57595', '98324', '82174', '10293', '80943', '64184', '19472', '4198', '9410', '25927', '65961', '33155', '95168', '33692', '61712', '69877', '13308', '17415', '10022', '2491', '67310', '96140', '68050', '76272', '17143', '76805', '57176', '7539', '22690', '95483', '87592', '27221', '90821', '51154', '99828', '68998', '54581', '74222', '10269', '65057', '45467', '96089', '55058', '89779', '60837', '74122', '52886', '58055', '14880', '93208', '66652', '68830', '24121', '62407', '87257', '18802', '14925', '45423', '98624', '55195', '59072', '41414', '77840', '66075', '62705', '26549', '19063', '57552', '2507', '52069', '57620', '66688', '14833', '33700', '90666', '98052', '5367', '2268', '43093', '69063', '22030', '85564', '92258', '1847', '24446', '65835', '38660', '91899', '87732', '52396', '31952', '36000', '86944', '16109', '80729', '53757', '60226', '59103', '84187', '36674', '72823', '29884', '4654', '69139', '20440', '57413', '3651', '39639', '44564', '57492', '84159', '751', '99748', '9659', '72661', '39220', '99742', '74734', '75729', '38071', '69934', '73640', '65294', '54524', '64372', '37927', '17187', '7863', '12732', '40296', '36197', '15821', '76831', '4400', '71933', '4040', '22072', '33064', '25702', '13324', '91275', '27388', '97729', '14620', '45989', '80737', '17934', '4219', '3032', '43457', '31051', '24469', '67041', '29328', '75499', '80951', '88212', '92595', '49969', '24612', '58732', '2718', '3805', '50918', '99426', '8614', '35580', '93273', '989', '24385', '41185', '25687', '47146', '25227', '95839', '56355', '98536', '79824', '31725', '46447', '26690', '68418', '47783', '33725', '21729', '70797', '59038', '60376', '25087', '68332', '67950', '12411', '95918', '64736', '65336', '74947', '64605', '4106', '42712', '96640', '28492', '28648', '42429', '821', '24333', '69677', '38959', '23484', '92005', '29352', '29159', '52873', '99947', '21834', '85347', '93479', '28298', '55608', '3226', '69714', '80283', '6577', '18849', '44605', '75286', '28139', '26541', '12867', '57500', '86617', '33005', '57498', '60223', '74954', '51401', '55246', '5648', '16513', '40930', '43821', '32090', '66002', '65530', '76083', '6047', '6879', '94987', '80787', '11688', '77161', '92670', '6696', '400', '28572', '47234', '51375', '88518', '762', '92617', '54260', '7560', '60180', '43331', '64059', '27616', '75839', '21392', '47756', '46254', '19486', '88533', '30130', '93694', '8557', '66534', '94447', '16910', '6480', '77440', '24366', '6195', '48946', '28597', '44429', '50300', '73556', '40638', '98709', '94413', '15987', '43860', '64871', '93953', '34506', '7296', '31753', '30626', '77510', '39829', '25696', '39776', '69185', '36540', '65413', '31528', '43446', '73532', '49776', '30282', '30004', '26725', '15200', '33958', '90320', '71836', '48051', '31970', '5326', '96194', '69695', '60898', '60945', '18271', '50868', '61468', '23593', '68985', '20628', '58044', '8942', '34849', '7384', '50500', '62895', '78780', '48946', '65278', '4067', '973', '34761', '15512', '73739', '23138', '47322', '55568', '32259', '71816', '49277', '75218', '76104', '19579', '68312', '67904', '33886', '53888', '26421', '43859', '40291', '39068', '31711', '36542', '10195', '39781', '72352', '13188', '34113', '9428', '60443', '4987', '13783', '80744', '63483', '18266', '11961', '87167', '46987', '28480', '74214', '39191', '8146', '38090', '75727', '79245', '47720', '52547', '45321', '4972', '49701', '74354', '69672', '63455', '41902', '5667', '54166', '4962', '25873', '44509', '73332', '73383', '29438', '21455', '12320', '11997', '16921', '49379', '63027', '86175', '8110', '76149', '2520', '11256', '25863', '50518', '69001', + '79113', '9447', '91840', '5242', '10998', '46496', '2448', '56058', '20970', '10517', '17783', '25723', '97137', '62840', '1264', '78691', '81020', '55335', '48524', '2088', '90413', '76651', '26855', '16177', '14954', '62914', '21344', '5708', '75560', '39311', '95865', '28783', '64902', '95657', '46276', '33426', '4799', '11588', '57513', '73689', '77677', '63011', '97795', '34954', '76866', '32043', '32697', '26643', '36890', '53476', '3011', '13963', '49551', '87671', '67761', '17488', '94770', '50599', '33272', '23091', '38079', '41177', '22395', '91656', '79679', '38687', '57384', '80118', '42507', '4098', '78949', '45669', '48802', '83915', '78292', '4369', '57657', '49146', '45192', '98491', '72457', '46331', '207', '81601', '7409', '70856', '91605', '70295', '9171', '72293', '32997', '78025', '16795', '73534', '68780', '21284', '31767', '94381', '86439', '12420', '53285', '99563', '60502', '67954', '55012', '99809', '5431', '69978', '99712', '14401', '79498', '4495', '3045', '528', '72542', '91604', '72725', '39378', '80378', '41996', '20138', '54545', '59730', '36951', '45157', '37964', '97690', '12184', '4944', '53803', '93605', '60851', '68938', '46285', '89663', '90309', '6907', '87239', '81791', '83292', '90013', '68927', '14725', '81840', '63836', '52068', '43830', '4794', '931', '59255', '8263', '99057', '94401', '69033', '7437', '20364', '92884', '28193', '43932', '37629', '59426', '18891', '8583', '79551', '87242', '1483', '6725', '65786', '16844', '12650', '99305', '42841', '9811', '18800', '39313', '51373', '31874', '84558', '27831', '48614', '48975', '55509', '83363', '31854', '64001', '94028', '76125', '79314', '24893', '81132', '9441', '86015', '28356', '40358', '10160', '23328', '7330', '76538', '37611', '89351', '84132', '97047', '26109', '95222', '35130', '75600', '88602', '15073', '87835', '71649', '28948', '81615', '37498', '28674', '59776', '44095', '65924', '64368', '94536', '12518', '61711', '55619', '82949', '4114', '21540', '70544', '28022', '79983', '28781', '7749', '97873', '4951', '50076', '47611', '99522', '56820', '38653', '49047', '36283', '83908', '72452', '85625', '10811', '36998', '44083', '34864', '44975', '39057', '4551', '68450', '24781', '1503', '9871', '46885', '11424', '21259', '54900', '97669', '85669', '6015', '2521', '37661', '14915', '57423', '91903', '94789', '32059', '64972', '4600', '61465', '27118', '79785', '13547', '49766', '38410', '68860', '63756', '23621', '64387', '46255', '63408', '11297', '41081', '56326', '58349', '98703', '72268', '73574', '32098', '42534', '91502', '38083', '11241', '56828', '12098', '25377', '37054', '56328', '30034', '26922', '68401', '93478', '63275', '62650', '81407', '773', '79499', '14970', '47217', '1187', '57428', '69980', '77764', '74791', '22107', '54363', '39247', '56028', '56982', '84244', '21464', '18716', '25533', '94589', '94768', '21537', '18436', '81135', '27654', '79713', '56630', '61571', '58453', '26758', '68450', '68449', '2994', '15347', '83954', '71823', '6428', '44210', '79597', '95144', '32871', '1991', '320', '77157', '63607', '31154', '48846', '71125', '61750', '59608', '33038', '35733', '68915', '94127', '50383', '64242', '49708', '57270', '65019', '8581', '12111', '18487', '50013', '58664', '22214', '19033', '33681', '44754', '28830', '10381', '52318', '34959', '20682', '55453', '53800', '65774', '99164', '72102', '36986', '44157', '56716', '7974', '81475', '25926', '39402', '33688', '99671', '95312', '42268', '26536', '14482', '67377', '57993', '89147', '15834', '64995', '4700', '18714', '30221', '39095', '32749', '69257', '55204', '30497', '31839', '63045', '30009', '62683', '31232', '77680', '93551', '63589', '6989', '77246', '42169', '46117', '73226', '37427', '1858', '83649', '37410', '86369', '4641', '74481', '66168', '48041', '22597', '14670', '27464', '57165', '20939', '36282', '76940', '73358', '50521', '69603', '8895', '81793', '57743', '81903', '64025', '91641', '25276', '34040', '62642', '64015', '57657', '84890', '73832', '782', '60160', '16998', '40023', '24590', '88613', '76640', '53091', '67600', '80183', '45674', '64464', '25163', '42384', '66972', '13953', '41966', '66048', '15135', '73745', '19466', '53657', '34619', '13462', '15905', '48257', '73297', '238', '93525', '80556', '5942', '5411', '66169', '9090', '95130', '74316', '57321', '48083', '62355', '68113', '15239', '36644', '80326', '65817', '54428', '61955', '58849', '77206', '16073', '98261', '92091', '39178', '35464', '85109', '85452', '21128', '25665', '81860', '44664', '24024', '56960', '95124', '39786', '18836', '11121', '44163', '81074', '79064', '46219', '94694', '44233', '81469', '24642', '15030', '21995', '13587', '40755', '6669', '81093', '74305', '1881', '55649', '37273', '80827', '98643', '46694', '59281', '79231', '42813', '84984', '7052', '98113', '17296', '84434', '31205', '46894', '71219', '74530', '44686', '70744', '91388', '20692', '96853', '73803', '15836', '18126', '49686', '4179', '47588', '87892', '65425', '68012', '97468', '92510', '99271', '58694', '11918', '37051', '18644', '57228', '14265', '57572', '57022', '52186', '30193', '93570', '87872', '5257', '26784', '6476', '61746', '68559', '1720', '26202', '16519', '27688', '10645', '87174', '60845', '73385', '82075', '6933', '98828', '56895', '17344', '84253', '36561', '51648', '24939', '63470', '31034', '95052', '51090', '51465', '87979', '68650', '30181', '29598', '19137', '43221', '81353', '90170', '96985', '61115', '17385', '92314', '80650', '55821', '17874', '84333', '93272', '48260', '87272', '22764', '59957', '51870', '85988', '39222', '77241', '62535', '28344', '6011', '80831', '64551', '46299', '75195', '71177', '8660', '58943', '57003', '3306', '74413', '74068', '15073', '89016', '93140', '13911', '57170', '19880', '41870', '9131', '57495', '73032', '86979', '60094', '87026', '30880', '4736', '86301', '92707', '21689', '83565', '71275', '47665', '65687', '71184', '89897', '32490', '97577', '38723', '79113', '37531', '97500', '94450', '15699', '58019', '84423', '27057', '56017', '97148', '47365', '30669', '33818', '80406', '99690', '33012', '95178', '46809', '48448', '79350', '9146', '99701', '98976', '71197', '44161', '75069', '36602', '79650', '97301', '12020', '56658', '25701', '46392', '78609', '63073', '69419', '57736', '20102', '42415', '79044', '20277', '56280', '47903', '94311', '25558', '40336', '91305', '90505', '66769', '64562', '83737', '62892', '10375', '71024', '19988', '56946', '76110', '21847', '43162', '50578', '46086', '54167', '61722', '53463', '63134', '69288', '12838', '14116', '71687', '50846', '59810', '24826', '84138', '82885', '91496', '98600', '82769', '40049', '4125', '50694', '1294', '2805', '29691', '82321', '76462', '85945', '115', '29188', '66918', '71340', '31585', '61638', '95472', '52978', '50622', '81990', '60955', '70519', '22270', '35610', '95871', '89222', '41038', '52546', '1163', '67943', '1793', '92010', '35755', '74509', '66665', '95759', '8568', '44299', '67822', '5806', '85839', '13895', '87675', '31357', '88014', '40026', '53050', '28951', '31992', '42495', '82892', '51567', '2869', '45808', '20238', '20781', '56098', '66307', '95701', '614', '60833', '3091', '81339', '24195', '65639', '85976', '28116', '66224', '51502', '73637', '13207', '88302', '36488', '65518', '98187', '26', '74367', '64706', '53943', '86760', '25783', '82112', '34958', '86621', '20848', '63459', '14049', '84943', '91873', '50238', '77773', '64109', '8602', '87934', '47583', '66053', '30287', '5507', '80312', '37464', '57457', '86200', '17806', '16522', '38843', '94334', '59958', '63864', '53427', '74506', '33980', '90449', '30842', '53616', '36738', '52', '13595', '53051', '13174', '60163', '71420', '73835', '67119', '79018', '42782', '45059', '952', '46360', '85879', '71552', '84741', '29746', '32577', '10041', '7208', '97528', '51256', '916', '55973', '17684', '99046', '38782', '58660', '97798', '66032', '48339', '51329', '12532', '97904', '95454', '42737', '62541', '96702', '82953', '94610', '26645', '86813', '25480', '99713', '26078', '23028', '93056', '21445', '73209', '89318', '69987', '34705', '30064', '17094', '51135', '54141', '26625', '1086', '13082', '30843', '98672', '56864', '42605', '5833', '60850', '69366', '27351', '16456', '92609', '48030', '54322', '69891', '46502', '34578', '77918', '63276', '75958', '42519', '60266', '85576', '4855', '14258', '67017', '10545', '35078', '53012', '71922', '85784', '73402', '74363', '58457', '94102', '23510', '51559', '39482', '87057', '9377', '10106', '82985', '33931', '16523', '6484', '97749', '83172', '53753', '27466', '23073', '96083', '67302', '57465', '21877', '18013', '99804', '32873', '43123', '72365', '53197', '80578', '69770', '97471', '86954', '67183', '98497', '78474', '28450', '63183', '98699', '42738', '61433', '3491', '27304', '49311', '94980', '92740', '43272', '86549', '11406', '79636', '85582', '38086', '657', '2354', '26567', '77450', '42086', '21600', '49011', '44059', '47872', '75761', '96577', '11642', '83471', '79616', '23749', '77082', '96876', '65302', '84027', '48955', '59887', '20657', '75090', '9058', '50347', '66088', '70745', '76342', '58026', '95568', '61504', '93473', '84590', '47089', '74717', '93090', '46334', '68273', '59500', '54345', '72608', '54048', '86156', '40296', '74046', '6813', '36369', '74543', '18305', '85236', '31316', '37061', '96893', '23112', '5529', '10166', '19037', '1467', '70810', '30932', '18410', '92837', '81324', '12268', '54705', '25207', '90366', '56528', '3392', '88747', '39951', '97957', '99404', '23685', '13533', '15640', '11434', '66516', '71025', '65770', '88000', '52232', '32360', '10787', '37438', '2264', '94460', '80214', '42288', '59062', '29010', '64093', '21225', '22297', '36935', '19202', '5925', '85373', '27414', '28991', '9191', '42273', '56587', '89719', '77191', '64334', '61542', '28763', '28978', '79184', '59815', '95200', '30246', '54022', '287', '91808', '66347', '50833', '15356', '78614', + '65087', '9664', '67555', '58222', '20787', '72382', '83712', '15858', '72040', '59451', '7428', '47586', '83509', '45969', '28759', '94033', '88618', '46', '1753', '28290', '71934', '99070', '52517', '65865', '15904', '68134', '87595', '39779', '99174', '13366', '50847', '69037', '77911', '19709', '84404', '24370', '18920', '1746', '28059', '4653', '6892', '25496', '48810', '82943', '7069', '44018', '1617', '56497', '27582', '29576', '47883', '8385', '54975', '96245', '77257', '31194', '34512', '19198', '50926', '50886', '91577', '49260', '49604', '58464', '18198', '90887', '15379', '74883', '36644', '63967', '34545', '20539', '12884', '6010', '98636', '94832', '47702', '98318', '6078', '42517', '97030', '93707', '39286', '10245', '66230', '99574', '74563', '81937', '22153', '10659', '67579', '85147', '69686', '32017', '8199', '20027', '64266', '60707', '55426', '22027', '13823', '81404', '86757', '3754', '197', '45469', '68736', '17656', '92453', '6784', '35531', '33538', '27558', '35825', '13471', '16377', '66978', '44601', '69943', '84635', '7335', '67725', '88064', '78201', '64380', '66771', '62221', '60427', '32028', '37857', '5491', '2757', '10537', '2199', '60244', '14726', '29734', '34433', '81241', '39778', '65831', '56153', '81696', '25113', '17371', '79022', '3587', '60439', '96582', '37801', '82496', '68242', '19561', '53140', '17171', '13165', '58750', '80612', '19818', '16753', '70009', '75290', '48846', '89851', '25190', '35525', '84348', '65298', '23161', '60583', '89611', '18798', '85422', '60713', '17660', '41915', '65757', '4781', '48374', '10712', '40330', '2409', '41020', '28042', '51857', '81224', '93917', '90366', '58848', '8012', '92070', '75349', '80066', '83228', '55170', '78376', '94127', '29418', '58660', '74006', '63909', '59502', '63255', '32318', '28904', '26117', '88103', '50656', '58517', '16372', '43392', '77876', '65997', '63501', '45927', '17444', '84021', '54886', '57082', '47456', '73816', '66900', '82958', '49722', '95843', '23458', '4560', '67276', '59080', '81486', '2967', '26133', '87694', '74183', '22100', '45306', '33895', '26512', '71426', '54994', '47367', '10171', '45681', '36268', '3166', '2179', '64197', '90037', '4641', '96073', '72989', '55123', '48825', '2862', '66974', '30115', '41554', '26727', '54025', '69504', '38284', '7816', '36370', '75764', '69774', '43208', '43598', '39461', '21655', '12169', '77790', '44578', '70959', '13500', '40823', '52952', '88095', '82193', '66424', '79814', '25651', '58125', '42724', '11752', '64533', '34827', '72755', '96727', '82066', '60268', '55535', '4230', '73018', '69602', '16912', '66088', '21859', '38654', '38638', '51444', '37999', '18595', '15209', '21801', '21130', '53741', '54954', '91590', '83660', '74071', '82627', '4311', '45268', '94735', '65747', '72428', '22621', '70828', '64394', '647', '23570', '17677', '84377', '63993', '10268', '47426', '72750', '83772', '29090', '73284', '36683', '91143', '64546', '99836', '21999', '61793', '41557', '16714', '20586', '14251', '56633', '46146', '84730', '5445', '8367', '52621', '79252', '89737', '18431', '32134', '86975', '58939', '63385', '8504', '25998', '79549', '70558', '4870', '57604', '48374', '32388', '97871', '75590', '10919', '43825', '82858', '25807', '50366', '2841', '2716', '51315', '79719', '33159', '63382', '11327', '65184', '87243', '17850', '40149', '12171', '48486', '26380', '85432', '35427', '59256', '57601', '29529', '20553', '3006', '69997', '87919', '47626', '96026', '37445', '51735', '89296', '26702', '22837', '61279', '85193', '30624', '10676', '83075', '21477', '54708', '47485', '80297', '86204', '25192', '55197', '81177', '70551', '11003', '7001', '55017', '487', '40926', '99463', '79514', '69697', '52854', '88640', '44464', '66314', '48240', '49787', '81333', '14914', '12919', '51037', '88609', '25572', '61019', '85184', '12051', '61246', '52108', '98895', '24549', '86944', '73303', '9012', '25151', '97424', '43531', '44790', '63034', '86713', '76483', '17499', '86891', '3267', '45091', '39126', '94419', '35750', '35239', '49413', '62381', '92010', '7246', '26997', '47446', '26930', '22668', '43961', '51741', '63058', '41977', '62933', '9197', '94303', '29044', '53755', '22391', '85373', '66572', '74650', '7824', '49712', '40021', '99958', '60404', '37370', '89264', '52343', '90901', '44730', '4252', '5811', '22283', '25102', '86636', '75509', '47088', '62971', '85519', '7495', '16798', '25708', '59258', '89361', '43455', '86768', '29961', '4362', '60956', '81059', '87604', '46241', '98362', '21030', '76182', '83545', '59150', '99333', '68271', '32629', '3364', '96467', '40427', '307', '84267', '239', '11711', '40933', '24518', '33854', '60291', '94179', '5564', '33196', '73818', '22201', '97078', '69360', '94513', '42623', '6510', '67378', '79468', '10394', '96430', '87701', '2667', '64162', '93266', '48941', '55345', '56127', '22704', '8793', '57008', '36920', '80845', '81203', '98715', '84158', '49194', '72718', '52919', '4951', '8715', '82384', '8570', '26733', '62399', '27297', '79038', '57916', '46155', '82381', '1027', '20541', '40580', '44836', '17336', '55548', '87454', '63549', '61401', '88819', '23326', '37546', '49089', '50707', '23099', '20724', '67184', '3394', '95941', '76045', '20523', '30393', '14157', '69381', '94104', '15406', '46707', '36234', '9136', '96805', '4904', '20310', '77164', '15400', '87198', '48310', '59729', '35230', '61503', '42015', '86910', '23264', '23542', '49105', '58334', '52855', '78865', '73516', '43820', '96593', '33168', '26170', '88241', '4016', '79862', '39617', '95329', '57881', '71750', '89690', '22519', '59367', '19825', '12638', '36100', '14355', '79540', '8411', '94571', '80462', '11953', '55629', '14598', '81066', '48968', '74149', '36938', '4564', '59157', '75517', '14309', '64905', '59893', '62803', '6968', '33003', '40894', '48816', '77710', '16692', '19135', '98381', '49397', '24048', '17804', '26829', '24046', '44546', '28174', '2639', '27957', '38625', '85902', '9908', '77666', '14096', '89851', '23600', '20717', '32860', '44070', '67853', '85907', '23761', '789', '73370', '15649', '26472', '9174', '20081', '49415', '97463', '14565', '19834', '94738', '68045', '67542', '3552', '77093', '99071', '16832', '95177', '49571', '85821', '9368', '85170', '58449', '89633', '99302', '94158', '50274', '50223', '51128', '52847', '53598', '47826', '2883', '28192', '84324', '9530', '14385', '37292', '74795', '16821', '5010', '77101', '96596', '72384', '41399', '83046', '37927', '48213', '84574', '55438', '79315', '71005', '45190', '56133', '14016', '92180', '28365', '34307', '90665', '74758', '45080', '38017', '80599', '91174', '47907', '19299', '9707', '92260', '36784', '55855', '25939', '74472', '47402', '12352', '75591', '31459', '76711', '57940', '95535', '55231', '80645', '84613', '90280', '3500', '83695', '42992', '77226', '25326', '8093', '49838', '6457', '44814', '82819', '94986', '81818', '65186', '21378', '49711', '66248', '72391', '65826', '21284', '41399', '72858', '32223', '35915', '45221', '36871', '16193', '2522', '52963', '24506', '98239', '56752', '15192', '8451', '62802', '54777', '67983', '82177', '7303', '71445', '40559', '58655', '73468', '40491', '52864', '81558', '36789', '53134', '38801', '90600', '36385', '90996', '20526', '13627', '62253', '96989', '88864', '11928', '14681', '77082', '95659', '47182', '16085', '26894', '39758', '32803', '26458', '65779', '66110', '38888', '15490', '48120', '90238', '8736', '233', '76851', '86938', '8873', '95387', '33400', '50788', '13568', '43139', '29821', '70427', '21022', '93285', '57243', '51675', '77459', '33170', '81485', '85303', '44133', '203', '44175', '89151', '26705', '32990', '10885', '10208', '58477', '97729', '699', '40581', '8065', '95864', '89797', '47506', '19160', '85728', '58599', '96271', '75849', '30736', '18808', '25828', '52011', '66179', '70244', '81312', '19570', '14287', '61078', '81922', '63361', '80858', '306', '22156', '82923', '53176', '24864', '49193', '2961', '96536', '62449', '13213', '85347', '11442', '76615', '43037', '86122', '99676', '74309', '46017', '16175', '12685', '53550', '19080', '37283', '44921', '34991', '4433', '99561', '38215', '35662', '13892', '7814', '41718', '41231', '85589', '74048', '72169', '26946', '61732', '9135', '39009', '63534', '4114', '40417', '83412', '22562', '65546', '720', '90064', '73027', '50736', '71086', '12445', '3681', '65140', '56291', '47452', '48916', '65136', '53941', '15484', '21038', '47251', '78461', '31031', '4838', '87483', '732', '16111', '68263', '4811', '65979', '44283', '51659', '29997', '11280', '3880', '12399', '63450', '93862', '50981', '62790', '76784', '76696', '40906', '49261', '35863', '43747', '65954', '36125', '73713', '98678', '13409', '38773', '59417', '9763', '429', '66203', '7350', '32754', '36079', '72447', '63925', '12958', '65292', '80270', '78432', '50925', '74946', '7649', '13689', '68001', '52318', '5804', '51584', '63637', '85697', '50929', '25279', '20963', '83038', '31199', '64189', '17152', '20459', '66108', '66005', '53159', '53971', '32528', '77145', '52981', '88941', '68593', '86343', '97770', '25625', '17598', '55249', '89640', '94162', '65874', '36511', '21906', '70401', '30914', '24085', '42945', '55597', '78393', '98120', '38283', '44967', '76077', '45739', '92432', '11846', '59131', '26322', '82651', '94792', '37726', '23301', '11275', '98813', '34229', '65440', '51369', '55580', '35207', '84314', '35696', '50231', '9399', '18679', '27219', '22519', '26725', '63451', '79357', '55915', '77027', '30354', '45890', '10745', '60035', '7914', '43562', '12219', '30220', '29382', '72520', '87475', '83634', '15163', '32468', '72904', '18200', '84666', '43560', '58483', '64613', '52170', '55490', '45954', '99323', '26401', '61210', '10450', '70360', '42906', '32851', '70022', '49966', '62331', '84249', '98093', '62146', '88171', '73246', '48314', '45730', '56348', '59081', '16642', '57872', '81564', '32305', '4158', '47263', + '97234', '74415', '36159', '30014', '22241', '77574', '27990', '67395', '81471', '18512', '47546', '23386', '6156', '72893', '15332', '57667', '92657', '35148', '45316', '90482', '93819', '38065', '74899', '10850', '63285', '50968', '88081', '42486', '56089', '43288', '11239', '55127', '26637', '85974', '93879', '78540', '42451', '35082', '93809', '63793', '51039', '13518', '20239', '26933', '19322', '26648', '83711', '93163', '95951', '77389', '43771', '36873', '66976', '38082', '59476', '53984', '6951', '42615', '80733', '1120', '80595', '10500', '67420', '38885', '8892', '36205', '17170', '46564', '66415', '36372', '68305', '41470', '38699', '60900', '23468', '6927', '498', '55896', '4005', '50692', '67984', '91012', '97314', '94177', '57676', '36734', '32671', '44733', '34981', '52186', '15599', '75642', '82652', '22773', '97106', '20691', '38085', '50368', '74916', '76281', '54002', '85721', '99186', '78438', '27865', '34827', '61315', '8817', '93105', '71539', '35484', '58033', '17540', '32806', '87253', '15780', '80123', '34241', '70383', '95515', '27998', '47682', '32682', '93658', '15116', '93030', '71866', '35516', '70161', '7556', '64122', '54622', '14312', '64375', '28863', '27886', '34040', '10010', '1460', '58765', '71496', '37785', '5616', '84850', '2194', '55514', '33636', '7138', '36937', '44763', '61308', '63574', '8069', '24109', '17821', '82144', '53309', '26183', '84238', '36684', '45285', '41353', '29380', '93726', '59560', '96422', '34218', '66851', '49001', '4855', '46849', '19877', '76540', '61415', '32053', '38801', '74623', '92808', '6660', '85031', '97706', '34417', '51085', '72292', '30903', '45126', '69978', '11572', '68622', '64123', '59344', '67118', '46014', '59248', '69321', '31897', '72232', '48016', '36455', '87125', '85356', '96888', '28316', '23149', '34823', '29944', '18522', '14045', '973', '49472', '89165', '84112', '94087', '47881', '87600', '86330', '13384', '4528', '94572', '30634', '86728', '48525', '53243', '35911', '96382', '22525', '89914', '52213', '23007', '54827', '35721', '49121', '40203', '99040', '55039', '94543', '85440', '75098', '12566', '15579', '40603', '51321', '28805', '46550', '42499', '69794', '51322', '37286', '72035', '15494', '39787', '97639', '24343', '36299', '72759', '96672', '58801', '57467', '97185', '19493', '4431', '33940', '950', '46713', '12503', '35185', '31106', '93821', '1086', '31767', '17484', '52751', '70411', '12196', '502', '45391', '70720', '27507', '1282', '18633', '45031', '10475', '83570', '29539', '16753', '63494', '17010', '76069', '38913', '53948', '34175', '36761', '37575', '73655', '76272', '1069', '74456', '10440', '65502', '95286', '39384', '83461', '76477', '4167', '46503', '22373', '24523', '46024', '28959', '83219', '27807', '80905', '88626', '70234', '86888', '60954', '95397', '47973', '47944', '30154', '42698', '15144', '53223', '21123', '19281', '62933', '24952', '84744', '58456', '38446', '26768', '40797', '107', '52101', '8348', '8531', '17772', '97826', '34009', '15825', '48977', '54939', '78886', '31784', '42842', '22773', '9095', '25113', '65362', '83953', '55921', '92797', '26776', '59312', '56428', '52218', '50141', '52535', '22564', '50708', '50384', '16261', '529', '31406', '46452', '30954', '5385', '60190', '38790', '65440', '11891', '2196', '83560', '68884', '72400', '47795', '7501', '8916', '63267', '35567', '39889', '61818', '94343', '46612', '75029', '46988', '12411', '6123', '60904', '43704', '31143', '72544', '45919', '67953', '97776', '49517', '60476', '54437', '46261', '99778', '18541', '77626', '12537', '68459', '12688', '36284', '54901', '89375', '10538', '38415', '33160', '78777', '33532', '82538', '21329', '34359', '29544', '57035', '61945', '49098', '32484', '68236', '11088', '57533', '11044', '96786', '58861', '51390', '9582', '24229', '32604', '66195', '7311', '46441', '73383', '58033', '41866', '19293', '73085', '50627', '19818', '40498', '56436', '99865', '55516', '38917', '89889', '49571', '99408', '20144', '76122', '49593', '47906', '94763', '26488', '86343', '98334', '70112', '85244', '45622', '47828', '35306', '86838', '57186', '69907', '90594', '68206', '25118', '75445', '71715', '15800', '67558', '17783', '56525', '22970', '17354', '17139', '76827', '50325', '95314', '65432', '44045', '57928', '66247', '10673', '69744', '99485', '57467', '66889', '55246', '15095', '23540', '32890', '22959', '59472', '62237', '6956', '34669', '16547', '54868', '22598', '65120', '48525', '72750', '70559', '75706', '61831', '14032', '36146', '90211', '36082', '79712', '43018', '53903', '82490', '45698', '54202', '83079', '4867', '18044', '55144', '49189', '33278', '48387', '68822', '73057', '58118', '89796', '47318', '98383', '58759', '172', '56627', '79861', '86734', '25758', '80779', '55741', '40924', '24568', '23090', '62934', '94264', '58331', '18747', '15802', '81585', '92542', '97426', '18205', '11820', '62702', '16439', '18613', '45918', '47584', '98743', '39808', '33357', '16033', '87378', '87036', '88920', '99300', '89343', '7384', '36867', '36712', '96947', '92141', '85779', '81735', '86121', '19511', '70519', '62537', '10398', '99308', '63922', '45419', '17708', '2093', '85127', '55753', '97149', '82877', '64642', '30417', '151', '75464', '75992', '99440', '58040', '32083', '90604', '47830', '40087', '78899', '43638', '59066', '46579', '44828', '35130', '10670', '62051', '94127', '32024', '85455', '80408', '18295', '8106', '89380', '90499', '3196', '3826', '63462', '64317', '55570', '77162', '48564', '59842', '80241', '76221', '57327', '48049', '74336', '43168', '19559', '32059', '99226', '46384', '48111', '56013', '58953', '68426', '22', '72641', '16207', '595', '71706', '61213', '65921', '67517', '92618', '57660', '76414', '60998', '1829', '90176', '62932', '15052', '94997', '41', '58848', '56411', '18129', '86542', '16330', '2803', '5651', '47351', '57866', '3560', '97530', '8338', '59434', '50183', '39076', '2516', '16293', '6559', '31189', '83713', '58214', '32016', '56581', '46226', '74599', '33513', '51530', '20163', '56512', '1694', '8165', '17370', '2510', '46015', '80417', '77195', '69062', '717', '33699', '37638', '82238', '5436', '67991', '98554', '26668', '19565', '26500', '30133', '54349', '36668', '56773', '55187', '25458', '54126', '11833', '32007', '38881', '20038', '42186', '67850', '12001', '96089', '63397', '75035', '16744', '45208', '84186', '82044', '34818', '55858', '1623', '41600', '34753', '92242', '16416', '49800', '45309', '56520', '38893', '42865', '37476', '78860', '32503', '18086', '55004', '65795', '82926', '75725', '52917', '75463', '82202', '35158', '67065', '38454', '90106', '79681', '57317', '68357', '66487', '99640', '13182', '68684', '96739', '89887', '54307', '98357', '4597', '47645', '38560', '17334', '46291', '91096', '22675', '12995', '35767', '76173', '54754', '75206', '13345', '78860', '19929', '12740', '13007', '69737', '52454', '6784', '77198', '59565', '44548', '22924', '38773', '9522', '59392', '45244', '64754', '39640', '75795', '22725', '78314', '24738', '83167', '31049', '92122', '62117', '47314', '34032', '76951', '63164', '22515', '46057', '81625', '72164', '60013', '26171', '55608', '50486', '51422', '18277', '13682', '70130', '49689', '32278', '25430', '84394', '57371', '93398', '14139', '461', '37302', '25705', '25642', '70311', '81778', '66948', '51187', '99649', '44509', '12753', '39827', '57393', '40288', '26496', '1500', '32979', '62062', '5170', '25273', '26343', '21058', '66749', '52257', '26335', '4347', '82427', '51381', '66857', '58330', '15097', '23881', '93865', '97929', '11233', '67910', '22731', '70911', '93941', '46169', '31920', '16027', '1635', '16338', '49712', '29345', '26316', '15221', '54771', '34715', '35399', '80287', '90881', '33758', '46569', '52450', '40564', '34885', '5679', '53232', '85632', '87510', '4008', '87014', '49611', '94569', '96432', '57394', '66064', '34144', '67269', '83546', '5901', '25386', '57588', '55055', '6940', '8414', '31432', '75407', '86922', '33925', '24600', '48261', '94439', '34234', '69028', '37145', '32386', '68275', '13237', '3521', '57395', '55029', '16426', '87797', '18559', '72744', '1155', '30384', '19799', '50001', '85831', '19583', '83898', '13465', '58160', '51160', '89216', '90725', '37151', '32241', '6319', '95146', '10376', '55137', '64094', '14990', '19747', '85852', '50692', '63214', '53123', '97058', '60203', '29272', '14439', '69712', '13023', '75808', '67150', '76276', '54521', '46621', '78807', '89302', '30273', '85103', '63180', '75468', '98474', '37749', '66275', '61712', '39381', '2740', '83025', '18083', '42030', '3391', '56204', '3960', '34780', '22476', '50074', '79854', '89447', '3695', '2986', '85090', '71529', '77995', '51572', '60278', '69314', '70777', '46754', '22085', '94517', '98052', '5700', '43157', '12345', '34663', '40350', '5739', '84384', '8354', '81566', '75500', '88934', '67062', '52877', '30428', '95593', '13288', '51836', '25264', '19278', '81290', '47555', '7606', '46694', '29735', '25723', '81597', '26221', '24545', '98576', '63395', '36061', '58752', '8486', '97817', '27235', '18579', '2387', '88017', '42855', '5111', '49094', '28448', '82227', '63436', '89135', '86591', '2922', '72163', '83000', '14378', '81829', '5610', '93513', '78135', '55292', '80776', '21212', '40604', '44442', '50505', '22750', '47985', '19367', '63494', '4363', '64500', '79444', '36504', '43418', '72523', '85981', '13740', '302', '76267', '14962', '79643', '26144', '27125', '13364', '82935', '8936', '8340', '38025', '80742', '42829', '95357', '11131', '6806', '10497', '70527', '3004', '54217', '51051', '34722', '57348', '50489', '76514', '76546', '29498', '40587', '52697', '37808', '32509', '68927', '68051', '53328', '41926', '9344', '67795', '98626', '44058', '46817', '95773', '57539', '13352', '96292', '72647', '47111', '48595', '82684', '43566', '93381', '3081', '27653', '25045', '34753', '80870', '92013', '21935', '89454', '14127', '73278', '87015', + '65232', '97216', '26840', '21272', '8226', '42347', '44358', '26715', '7860', '31989', '14091', '15648', '85801', '20463', '85221', '77969', '87196', '28563', '69663', '55169', '11372', '38915', '74678', '1183', '14845', '85696', '54995', '62342', '83227', '73536', '64251', '46556', '78421', '89058', '41036', '79746', '42148', '77720', '33923', '24979', '98428', '80202', '36845', '20584', '13506', '97375', '48434', '14863', '29237', '99014', '20650', '48496', '61825', '89100', '51501', '22632', '89030', '22901', '54072', '68093', '98140', '1631', '43266', '34532', '35780', '17763', '41731', '19580', '72266', '66530', '55650', '51506', '92256', '5332', '85143', '65002', '30530', '21358', '88420', '81349', '94086', '68143', '95343', '30490', '61605', '40296', '61665', '21032', '498', '67165', '3151', '94788', '36502', '77434', '88776', '67565', '4404', '51177', '49030', '88502', '92552', '31170', '92051', '79960', '43784', '17864', '54179', '91590', '39153', '75017', '11161', '31100', '27964', '30263', '54908', '57991', '89272', '14642', '40576', '20224', '48887', '93479', '87862', '19858', '78238', '4113', '32369', '70072', '82752', '85122', '72450', '53113', '53806', '45347', '55032', '77242', '70401', '83685', '41229', '47489', '36964', '40556', '2619', '64080', '7354', '10159', '74827', '11846', '64587', '88667', '20940', '93594', '43669', '38592', '20211', '96055', '65995', '89957', '18941', '82750', '90753', '49989', '63331', '52911', '1675', '98447', '35516', '28539', '91592', '80615', '68066', '63922', '52509', '26025', '14634', '69439', '19820', '89983', '38306', '84048', '66264', '26855', '3634', '92094', '61088', '28293', '11428', '68748', '10665', '27114', '20151', '65883', '86423', '77069', '74997', '147', '41427', '26124', '22163', '82452', '61186', '89370', '58583', '2692', '54969', '84679', '26237', '46229', '16041', '28473', '38684', '43587', '37886', '68790', '92030', '85066', '13673', '30280', '91844', '43115', '10005', '1481', '84393', '10008', '60291', '90687', '99338', '95570', '39916', '68643', '41309', '24724', '57188', '75473', '21647', '50206', '6820', '73756', '51102', '76169', '99840', '32863', '17306', '43063', '90483', '15894', '59042', '53797', '93961', '12248', '12125', '39393', '17056', '11576', '6145', '43960', '74245', '86805', '95665', '58855', '50039', '14867', '85076', '3939', '55501', '18184', '62019', '16312', '37518', '18544', '67431', '68250', '70766', '74635', '41835', '74538', '77822', '47918', '90863', '17526', '78930', '47473', '67945', '70385', '19404', '45030', '13476', '46186', '28191', '22019', '18700', '90915', '41408', '63333', '52902', '87336', '72833', '47707', '46692', '47873', '82012', '22597', '46379', '69109', '72108', '47439', '51130', '41894', '29827', '88403', '75040', '37504', '69925', '19034', '57998', '46633', '1021', '48358', '55923', '3536', '26350', '36696', '29658', '25247', '65614', '19042', '34463', '85938', '26155', '49801', '74604', '17581', '91632', '12935', '97966', '14947', '71196', '32270', '51915', '41692', '83274', '6608', '37190', '35363', '85313', '97545', '5696', '23700', '39334', '38434', '20192', '27349', '71340', '19732', '96142', '17152', '32995', '34829', '71157', '36376', '11522', '26646', '19239', '34033', '4200', '39023', '64291', '7931', '42807', '2800', '75490', '4515', '84571', '59557', '13421', '69465', '69121', '82358', '18134', '18536', '74294', '24013', '43777', '94192', '77031', '33341', '98070', '27534', '34419', '40107', '17826', '48196', '82843', '37592', '40484', '49019', '59822', '26118', '89788', '5990', '99595', '35178', '49473', '54699', '81618', '23497', '87025', '29421', '1685', '31406', '54115', '25193', '44525', '44886', '4885', '74482', '63280', '57532', '29773', '83104', '5354', '35313', '12558', '16165', '52712', '82282', '73541', '97334', '7283', '29405', '87694', '64717', '98840', '2868', '17727', '61847', '32417', '91009', '12117', '35077', '36279', '87552', '12441', '70576', '505', '57332', '56191', '33846', '78645', '58506', '48692', '27712', '86722', '46677', '81512', '14807', '30337', '7863', '457', '71893', '32256', '21897', '95811', '73552', '24929', '4893', '49448', '33836', '24781', '43236', '30853', '81985', '60625', '20087', '30674', '39280', '10779', '97785', '11298', '65057', '17589', '44085', '10126', '72351', '4165', '30539', '97124', '20545', '50087', '13490', '4055', '62365', '24046', '63110', '99045', '47122', '74389', '29843', '89082', '71686', '40321', '15789', '63062', '23403', '27718', '72550', '36098', '47309', '60568', '22266', '20542', '59018', '76386', '89381', '9164', '42093', '19931', '90426', '73695', '26948', '17363', '15727', '75439', '45425', '42518', '32966', '80703', '4659', '74566', '22464', '49383', '53167', '80720', '45464', '179', '29789', '19444', '97663', '6466', '51290', '56374', '17869', '97520', '45148', '92149', '54757', '94455', '80125', '98584', '70174', '77708', '59260', '50275', '49211', '60730', '29875', '90935', '29676', '95390', '18263', '73189', '41364', '39707', '75110', '59156', '11770', '38655', '55173', '62547', '16328', '70585', '81958', '61191', '90906', '45978', '59134', '24609', '32956', '30518', '27341', '84512', '24987', '65295', '75210', '97931', '56717', '6751', '83338', '82808', '96570', '27038', '93274', '399', '9330', '54319', '24848', '39269', '92692', '50625', '52306', '93262', '48756', '66694', '50517', '41763', '42231', '8589', '41923', '87839', '9579', '5104', '41634', '63030', '15100', '59593', '38288', '88852', '81957', '84267', '56380', '89329', '74372', '16026', '15795', '29456', '93173', '76367', '37598', '68153', '72405', '55181', '77543', '30730', '27564', '98764', '48711', '22180', '51352', '80150', '50091', '19481', '97962', '74879', '75369', '84410', '54712', '68399', '60910', '57922', '8896', '94118', '5276', '54049', '11614', '98854', '65979', '75156', '39456', '30226', '27357', '71087', '30814', '6416', '56480', '70259', '56525', '69945', '63874', '87262', '97398', '77972', '76745', '97191', '26739', '10957', '48743', '18175', '74688', '49853', '83675', '79232', '72211', '77255', '44445', '81053', '5506', '21366', '99015', '57598', '9467', '65131', '73794', '25679', '3609', '70041', '14653', '93792', '7076', '87326', '12978', '20886', '24089', '82187', '55203', '96649', '51205', '26250', '98219', '64275', '18634', '94004', '50999', '99611', '72380', '28685', '33246', '49116', '44972', '44273', '26558', '84799', '51310', '39480', '16713', '34371', '94308', '38743', '14592', '47430', '74505', '38866', '69596', '84705', '20236', '49353', '28259', '16255', '27665', '94271', '98652', '18279', '4805', '57471', '7944', '72567', '55603', '47327', '32268', '1256', '47608', '80225', '15744', '75492', '74652', '96812', '77976', '72595', '86791', '97736', '83149', '57030', '58553', '7121', '53544', '94894', '57236', '55850', '52762', '36303', '97740', '25314', '43196', '74248', '52320', '60448', '10606', '7010', '21765', '20357', '61825', '34331', '15898', '67485', '70353', '41657', '49212', '21628', '11783', '90514', '33883', '85668', '73432', '55374', '53880', '864', '66975', '51190', '18268', '45344', '99680', '91098', '3893', '90773', '55611', '5914', '14977', '29463', '38339', '94390', '15379', '74717', '78721', '10689', '90966', '33191', '32384', '4363', '61992', '37456', '28402', '51633', '14358', '99226', '23024', '22114', '86438', '70682', '27242', '46237', '92467', '83997', '71994', '59256', '44728', '51372', '50328', '66925', '16487', '7348', '69501', '23328', '55945', '98064', '92561', '5275', '67097', '37616', '8561', '14378', '83858', '44306', '60514', '89238', '28781', '11864', '31540', '3809', '16629', '4336', '70961', '10787', '58719', '39587', '46840', '48993', '18578', '30890', '71438', '29672', '54785', '77570', '25012', '70696', '25935', '279', '74327', '33944', '55060', '72194', '28269', '47600', '2132', '56618', '77029', '26410', '25682', '13744', '31968', '27115', '57065', '49313', '31237', '70475', '38549', '33723', '66622', '19646', '93518', '4052', '29390', '10163', '19879', '10765', '71024', '71339', '69114', '78385', '64440', '79577', '48109', '81114', '4708', '44863', '72410', '71161', '55491', '3433', '88675', '48496', '97875', '17184', '840', '16960', '32880', '48533', '82430', '44005', '36654', '96011', '16158', '21530', '88485', '91231', '77831', '39266', '9833', '96154', '89200', '2021', '57813', '897', '27643', '65750', '27447', '29043', '50691', '74683', '78027', '99590', '6052', '22867', '4567', '3918', '27143', '77750', '32003', '14011', '59006', '33140', '12165', '72413', '89931', '59389', '45281', '52596', '31993', '88613', '85332', '8016', '34419', '80901', '76317', '34932', '71099', '23354', '19296', '42671', '12646', '47189', '67982', '79803', '28449', '6229', '44873', '48970', '10012', '6871', '88240', '1277', '71039', '47959', '45051', '74358', '68394', '49093', '78063', '26981', '8704', '18470', '14458', '38195', '75691', '64683', '44910', '93423', '67591', '36833', '63204', '29507', '38894', '59524', '12141', '36543', '86914', '91250', '45459', '30156', '119', '95390', '75872', '28827', '67609', '99869', '91237', '70358', '60292', '3499', '3133', '45478', '5800', '47576', '50558', '71500', '65579', '84252', '68999', '59495', '79395', '28634', '44121', '26503', '11922', '22058', '34164', '32668', '10422', '62793', '41303', '2030', '76875', '17151', '42101', '22133', '82998', '19499', '14247', '95259', '746', '38449', '72450', '99138', '61259', '65623', '13439', '45597', '55234', '19347', '70266', '4828', '79316', '16983', '49723', '95468', '9747', '81126', '31999', '33796', '15270', '60644', '61048', '98848', '26685', '40980', '23733', '13676', '98363', '17970', '76451', '42657', '13165', '27870', '49193', '56279', '34647', '1652', '47674', '95555', '80556', '52419', '61110', '26124', '49097', '90600', '53605', '97544', '8694', '94739', '59652', '41807', '87155', '83539', '61468', '20614', '12652', '86562', '82894', '94360', '56182', '30513', '65379', '6522', '86486', + '37674', '44977', '54370', '97381', '60218', '2423', '99591', '69913', '26507', '19708', '6279', '58955', '20126', '1495', '57894', '7638', '38700', '77148', '36844', '7539', '91452', '6914', '74349', '66850', '49104', '6516', '58535', '20851', '27859', '32881', '72919', '28203', '32882', '2419', '77583', '63822', '37703', '66793', '65784', '62281', '55867', '70703', '89344', '1498', '33770', '87176', '95636', '64891', '90736', '95521', '10989', '5237', '99010', '21106', '11422', '1831', '67239', '52557', '36468', '71713', '39637', '49574', '50455', '14953', '96900', '70852', '96982', '4341', '44585', '95651', '79669', '29652', '87294', '74692', '16221', '768', '35380', '21352', '50907', '27259', '11718', '5017', '55964', '94137', '52347', '10595', '12968', '85602', '97965', '18836', '90511', '70960', '97336', '44575', '23791', '42195', '64776', '29363', '42379', '1805', '28919', '6772', '78143', '54797', '27362', '56149', '59048', '38567', '6339', '27787', '42167', '45990', '95532', '54839', '26572', '38496', '89797', '6634', '16468', '24898', '66814', '98126', '31762', '36133', '64539', '43167', '87022', '61295', '30364', '89249', '25756', '63570', '91484', '10564', '79648', '5756', '41376', '61897', '40388', '88927', '62891', '79708', '25495', '22204', '33892', '36871', '19879', '58646', '57061', '73100', '75831', '20029', '67462', '54675', '7766', '2409', '24506', '7877', '11720', '86252', '9897', '8080', '70684', '74497', '2242', '24604', '31969', '83999', '56635', '5283', '64971', '79152', '27470', '89042', '22835', '21476', '50292', '56081', '96342', '32763', '84487', '64856', '79152', '64656', '72169', '69971', '93094', '52804', '80917', '53152', '56016', '28496', '79110', '17133', '12581', '91742', '78929', '2676', '46700', '59528', '93808', '4535', '54035', '40161', '62796', '3598', '97088', '13599', '36337', '73395', '17494', '86275', '62058', '61937', '87747', '94883', '90677', '88544', '72553', '50210', '75481', '64378', '74464', '21659', '30970', '71989', '84846', '72289', '88716', '39143', '8487', '4912', '91013', '18623', '19122', '36507', '76438', '7516', '67970', '72350', '69873', '33635', '55983', '69008', '49545', '3134', '60056', '52509', '63304', '15560', '23651', '81090', '7027', '8317', '33060', '37295', '51961', '53037', '97431', '40512', '23536', '25168', '78455', '85613', '12304', '40733', '99890', '51238', '55439', '96201', '73559', '92533', '90173', '16721', '6078', '29854', '38894', '31117', '63040', '86795', '81786', '21149', '38998', '61811', '48622', '73019', '59296', '13576', '92559', '36300', '77294', '26794', '50912', '98380', '13176', '57746', '75286', '15330', '40921', '7337', '4664', '20384', '4674', '44516', '27633', '31950', '88210', '54536', '9839', '80137', '77491', '18434', '45152', '96942', '41005', '76103', '34825', '86869', '14772', '13384', '21051', '37348', '34434', '97210', '54960', '26598', '60981', '41889', '6446', '64492', '95310', '86236', '81885', '35684', '16539', '98476', '32028', '96470', '6318', '99576', '93935', '48609', '86090', '2476', '65576', '80636', '44817', '99646', '98963', '20486', '26261', '27334', '72946', '82023', '33506', '80193', '13762', '98133', '21134', '33268', '63477', '74609', '30454', '51477', '93391', '96805', '68653', '2714', '63642', '51520', '22972', '13305', '96058', '42336', '74461', '31597', '12050', '81712', '37977', '25718', '4834', '56608', '75731', '406', '28585', '63924', '23702', '29849', '16941', '91921', '65842', '76525', '68534', '50902', '17609', '23852', '53703', '31286', '58526', '9633', '87596', '10654', '2085', '52766', '22135', '76524', '32295', '90072', '70078', '77786', '93741', '87320', '70309', '44024', '95286', '12361', '29682', '59766', '26685', '90686', '81691', '49704', '23431', '53955', '39023', '47261', '1530', '58265', '80065', '95620', '90621', '63760', '90676', '81653', '36397', '20252', '81754', '20256', '67098', '7838', '49408', '88400', '87941', '84533', '6570', '22567', '18850', '55472', '40129', '48425', '23497', '39308', '34698', '53092', '89480', '47785', '57282', '25508', '19006', '50604', '86917', '9436', '88921', '3168', '70537', '3185', '34988', '5462', '69482', '45768', '91955', '56898', '15307', '99731', '89292', '19356', '20646', '66712', '7281', '12856', '31174', '19577', '8726', '62971', '33008', '37118', '59055', '84101', '68445', '91957', '47526', '15627', '79914', '20013', '26147', '80821', '56372', '74205', '28531', '25352', '51775', '93948', '55212', '17863', '91521', '74911', '88160', '2360', '98260', '18294', '62402', '84268', '9580', '42668', '1467', '40059', '5221', '4216', '9917', '35420', '16496', '34369', '50253', '95234', '95114', '84193', '28322', '37031', '81284', '88628', '36782', '42572', '73347', '66188', '43342', '77285', '16513', '89064', '63066', '72645', '67075', '48208', '18181', '77898', '65795', '53707', '39856', '92883', '92567', '49733', '30236', '10273', '53029', '69773', '78379', '72108', '47696', '97557', '95184', '14688', '29853', '62694', '70431', '88435', '58799', '21883', '99866', '69178', '55870', '14414', '85274', '27321', '55555', '613', '15067', '88217', '73655', '99548', '13631', '78789', '36690', '7952', '60830', '77438', '40059', '95602', '43097', '3429', '93731', '90537', '2932', '35702', '16125', '6652', '39632', '39349', '9910', '38103', '78608', '73565', '48556', '28978', '7128', '82326', '53980', '28059', '28212', '87101', '77752', '99170', '56753', '30484', '71470', '32607', '24674', '32687', '25098', '94712', '64024', '48239', '90408', '17316', '99243', '3656', '67402', '48009', '98427', '52800', '56024', '4417', '89747', '93338', '18758', '56411', '44810', '82456', '30808', '75470', '67115', '66876', '53906', '78403', '56059', '34383', '60056', '89136', '7237', '11129', '21351', '78662', '43606', '37454', '45465', '9292', '38099', '81699', '50195', '49368', '47503', '44605', '6523', '81478', '37910', '397', '20256', '6835', '2787', '80383', '4241', '65986', '83870', '21205', '10879', '26593', '44357', '72604', '56131', '43423', '80206', '26240', '87198', '99445', '53504', '10632', '2465', '31793', '89575', '64184', '39988', '60049', '87100', '37151', '61585', '82180', '52065', '72519', '72935', '3201', '5862', '20560', '95339', '21661', '17533', '17182', '71189', '91564', '57999', '35490', '94773', '95056', '51583', '59394', '10727', '8655', '48123', '10701', '25314', '20100', '6533', '46435', '43188', '23001', '23018', '76637', '32018', '36603', '18701', '9550', '61550', '47541', '36500', '67507', '81574', '95490', '69169', '32584', '30045', '64699', '83539', '89396', '42517', '61979', '41528', '8271', '88377', '61423', '1158', '89724', '70789', '14886', '64823', '56675', '97747', '23990', '58495', '82064', '17062', '90258', '86854', '93304', '12925', '49975', '45074', '87155', '72223', '67344', '42733', '42516', '40110', '15444', '88285', '39371', '23198', '61544', '90205', '6192', '15718', '19803', '92712', '20081', '31397', '5555', '70463', '19521', '80401', '74097', '32060', '26495', '20507', '40473', '1449', '57215', '46142', '39303', '50359', '35898', '46908', '90752', '7823', '27416', '73770', '98790', '17907', '29999', '76417', '49926', '76752', '21608', '26524', '88209', '6000', '88897', '19541', '41451', '59538', '56560', '1456', '67828', '82407', '45722', '93344', '54279', '78594', '38354', '93807', '10929', '91560', '60681', '70615', '32527', '10108', '48303', '63134', '28500', '18257', '57081', '24801', '99077', '52197', '15390', '52300', '57116', '417', '7503', '20054', '75315', '81359', '69091', '18853', '2465', '25600', '13522', '74575', '12661', '83071', '15191', '27543', '21730', '60853', '18961', '14773', '89185', '33694', '51143', '1449', '68831', '78062', '65173', '32697', '41674', '9429', '22156', '96022', '46305', '97534', '5685', '48870', '89988', '20686', '66705', '6865', '94250', '16872', '13178', '7420', '73531', '92723', '60620', '48843', '74207', '60016', '50943', '62699', '63507', '76537', '87066', '76922', '24711', '34809', '5021', '31293', '53854', '77607', '52322', '10934', '50284', '87804', '36730', '86946', '80749', '43325', '97958', '7362', '39582', '10042', '42053', '66236', '69931', '23463', '87996', '33563', '4468', '32905', '50815', '79478', '28658', '46018', '23186', '26080', '13494', '6237', '42762', '86440', '77407', '10426', '62902', '73251', '36861', '92357', '98754', '1839', '46391', '11420', '27132', '93028', '39609', '42015', '68218', '54228', '5456', '38705', '64307', '49483', '878', '54360', '54480', '66684', '55089', '4537', '82073', '72602', '96238', '56708', '58625', '32991', '74205', '72868', '79086', '64250', '56376', '10621', '76607', '47706', '72760', '70303', '60715', '14644', '44186', '36264', '29489', '14184', '62699', '30567', '16700', '31222', '15650', '1500', '22950', '54628', '41004', '96094', '70028', '74178', '65328', '26605', '63076', '75271', '79285', '8151', '42101', '56362', '25961', '87864', '972', '29510', '2747', '8877', '9780', '61052', '84105', '15573', '27475', '44570', '25334', '18517', '44237', '84094', '67524', '76761', '65678', '79284', '2462', '42631', '22696', '19223', '29728', '67742', '11883', '59027', '12377', '80538', '2165', '17377', '15030', '49838', '23920', '26025', '68179', '75894', '43783', '97106', '75558', '35528', '52081', '16951', '68855', '402', '21459', '97550', '16948', '5369', '4641', '2663', '15233', '79974', '71093', '15234', '42690', '22322', '54282', '95845', '90010', '40530', '88298', '41885', '7079', '6098', '72786', '36603', '77378', '48393', '45723', '41996', '96025', '89297', '75586', '8422', '24360', '170', '46036', '46725', '67944', '74029', '73069', '45371', '99916', '71085', '42608', '89904', '6393', '51274', '42729', '58924', '82497', '64143', '88622', '18818', '89041', '56090', '21369', '78224', '90450', '45488', '58830', '4133', '98062', '81113', '11285', '51457', '3183', '38800', '65278', '42169', '28602', '52648', '44683', '75647', '11778', '32151', '33528', '23773', '68268', '23367', '70964', '23548', + '35575', '67570', '77681', '74158', '25374', '62714', '43100', '4977', '51678', '83460', '29755', '15890', '64626', '54044', '14793', '64339', '94008', '97126', '49202', '33889', '12601', '12275', '56123', '94557', '68226', '67200', '9374', '70687', '29211', '8039', '14598', '74548', '37433', '98991', '29933', '37203', '23973', '96482', '64774', '58350', '61781', '31824', '57193', '26476', '21814', '32297', '32627', '44277', '33876', '55468', '81715', '82505', '61462', '20324', '84293', '40116', '51087', '43594', '6854', '59077', '39841', '26023', '22777', '66859', '82460', '89515', '41712', '33711', '71875', '10685', '12655', '50138', '31063', '37040', '95819', '38919', '27391', '29833', '34350', '65646', '7697', '2688', '41146', '13241', '50305', '86568', '24487', '78741', '96370', '21015', '31719', '39750', '25014', '72415', '8486', '90668', '51143', '49488', '21057', '92803', '53528', '39550', '76039', '44185', '32404', '30217', '19796', '38084', '49161', '80140', '20241', '39357', '68908', '93083', '77231', '6952', '36322', '50790', '623', '29730', '13616', '57546', '17434', '93811', '35148', '81419', '40250', '40329', '89126', '72402', '16053', '27107', '28919', '16829', '96582', '65057', '28416', '30801', '77742', '27420', '73118', '89352', '54706', '23035', '88413', '64608', '61930', '15037', '47327', '59596', '18700', '57576', '63628', '56823', '60091', '68209', '21001', '14962', '72257', '83802', '33721', '86343', '11133', '65737', '68477', '90725', '86869', '98403', '47393', '25356', '61372', '8873', '19888', '48836', '66005', '23531', '72520', '26461', '78508', '28213', '96394', '22983', '37856', '71814', '27425', '72753', '27511', '65471', '38592', '3683', '24652', '64505', '92543', '53201', '40639', '99542', '53425', '35321', '47669', '14134', '47727', '48202', '71931', '32119', '50086', '50266', '67159', '89317', '81905', '30315', '49154', '8690', '69365', '56881', '46473', '64100', '38365', '59377', '65630', '54871', '52745', '91536', '16106', '70066', '62063', '84530', '88103', '33599', '51063', '87299', '41880', '25335', '51252', '42788', '13568', '1721', '62424', '83308', '36787', '91536', '92555', '27600', '24030', '12267', '66336', '30242', '7183', '67624', '28471', '48593', '79766', '31178', '47818', '94522', '88855', '45262', '43670', '18065', '25062', '44558', '37189', '69225', '35216', '42683', '26289', '72816', '31947', '65871', '45715', '59452', '22014', '56669', '60331', '33450', '60601', '95047', '30789', '90107', '81565', '32266', '3252', '5446', '58756', '55370', '34034', '81071', '2560', '39054', '39564', '15010', '5389', '60002', '53320', '49545', '48444', '31415', '39278', '79879', '30148', '10186', '60358', '29011', '14419', '95159', '94815', '55251', '90910', '80582', '92304', '11697', '60061', '38577', '84439', '76196', '34542', '50963', '36294', '11123', '59763', '29873', '47383', '12979', '22119', '21723', '64725', '48377', '77132', '9817', '79920', '47653', '60069', '12924', '53808', '55962', '66969', '13757', '60615', '10994', '9138', '34119', '58436', '64407', '75170', '73524', '51864', '94183', '86847', '15585', '57616', '96267', '5340', '52929', '49096', '50291', '5559', '32382', '84077', '6598', '87921', '59719', '31726', '44772', '63373', '75420', '66829', '47275', '98264', '61387', '94945', '44540', '50098', '13078', '44729', '95332', '63555', '30782', '63203', '15071', '60996', '72812', '17418', '80215', '37610', '30670', '44674', '74822', '15471', '25236', '16266', '76213', '35820', '19567', '8715', '72003', '90606', '1434', '53545', '88170', '75014', '62287', '35436', '38669', '12927', '83877', '38622', '28313', '82884', '73969', '38671', '10450', '24158', '22941', '73162', '86548', '42482', '95315', '92016', '96156', '44012', '35962', '6366', '3881', '74300', '26248', '30182', '19164', '67105', '66771', '52587', '69894', '61820', '16551', '50743', '10096', '69030', '24451', '89165', '23929', '96291', '30685', '64413', '19913', '9049', '71383', '61684', '45384', '45927', '81840', '49521', '89594', '30055', '83430', '14930', '60316', '86585', '99375', '80170', '14207', '19584', '20067', '82874', '30159', '46647', '6942', '66777', '32638', '55662', '75470', '77622', '26893', '96149', '14373', '33252', '50574', '7945', '20696', '56662', '94348', '3384', '20956', '89668', '99052', '65131', '56847', '17589', '16419', '2670', '10705', '59587', '92902', '92424', '48570', '11034', '69149', '35733', '17315', '84966', '69353', '69590', '52834', '32561', '6049', '50156', '71676', '76423', '32361', '61509', '8845', '75709', '35956', '21912', '31188', '59083', '43459', '38614', '92206', '55645', '38737', '34193', '6451', '94163', '24326', '49976', '71600', '58024', '67160', '4365', '38270', '59558', '80834', '60739', '54318', '19738', '42196', '43191', '13463', '88914', '99239', '66869', '75691', '33085', '4323', '7170', '46184', '41423', '89835', '46877', '20349', '14365', '32727', '35322', '841', '23597', '43370', '57527', '73250', '32553', '71489', '44617', '98323', '37672', '59549', '96023', '63176', '13524', '15621', '30448', '28136', '45549', '3513', '64153', '19839', '24219', '41987', '51083', '90268', '52052', '31430', '4727', '99409', '43595', '82374', '61251', '51470', '66562', '98724', '23529', '53895', '67562', '87573', '89964', '30821', '15733', '33062', '86963', '33450', '75338', '32570', '14453', '38080', '36335', '84226', '52790', '42883', '61156', '42789', '57846', '60096', '29946', '80178', '15882', '1971', '60722', '62458', '8754', '59991', '89321', '584', '70565', '36458', '21226', '23561', '9837', '39364', '23065', '30675', '9306', '40085', '52082', '89976', '73283', '77851', '36174', '54470', '63250', '72111', '70853', '26723', '42590', '91230', '47512', '13983', '70898', '70927', '40721', '30642', '41628', '90010', '27306', '1933', '43304', '44499', '87890', '22201', '89249', '63935', '48438', '58588', '1061', '70061', '63075', '9676', '65820', '82156', '82668', '111', '54350', '10328', '23466', '98936', '18285', '53919', '32422', '84859', '58387', '24022', '32423', '6010', '56417', '49452', '69999', '14885', '47102', '59577', '24999', '75984', '96464', '59088', '85987', '71442', '88789', '4753', '8229', '76883', '15284', '90610', '40507', '78882', '55575', '25315', '7214', '70602', '4796', '35767', '54657', '42153', '16050', '93607', '99249', '77236', '59949', '52871', '47837', '33534', '30023', '89137', '99938', '35824', '50775', '30282', '82798', '53312', '65277', '68375', '91445', '58166', '43344', '6589', '82515', '34632', '78588', '152', '67554', '15877', '74334', '32783', '45147', '39483', '92067', '59029', '38298', '55229', '28268', '85140', '33451', '15424', '46695', '23201', '83329', '28372', '19518', '89198', '33305', '43892', '470', '37662', '9407', '14376', '80310', '21459', '72381', '80414', '88305', '69073', '63101', '91054', '47190', '48595', '24696', '41426', '35133', '94399', '21790', '55040', '73279', '20809', '67805', '94115', '58633', '78053', '89444', '4112', '8', '34517', '22106', '85934', '86814', '53333', '93437', '85062', '32791', '72744', '99843', '51161', '22730', '34908', '82918', '92566', '22467', '41226', '98518', '29235', '94042', '84371', '79100', '25214', '7764', '59427', '47891', '61092', '23775', '13641', '30837', '77377', '43032', '38441', '29462', '20300', '19070', '20982', '73987', '87836', '68062', '6419', '51563', '40084', '85694', '86677', '47142', '27222', '17844', '19158', '45120', '88524', '74724', '73229', '42470', '38751', '1132', '28603', '61188', '55021', '88825', '58005', '62411', '8843', '94852', '93664', '39253', '27473', '247', '43824', '1804', '8905', '11509', '95659', '7811', '80691', '15779', '49794', '8991', '76099', '29223', '36060', '85399', '41369', '22885', '38473', '22376', '50446', '89578', '25818', '61333', '78787', '47605', '83654', '99068', '52120', '48367', '86381', '19803', '72600', '31998', '37755', '88031', '83969', '42319', '27974', '35780', '93662', '46808', '60529', '15491', '10447', '48829', '33886', '68333', '44855', '86554', '64794', '66376', '58222', '14021', '52043', '56375', '1300', '38105', '89159', '97456', '26800', '93124', '3673', '32279', '30658', '84475', '3708', '93952', '39245', '91980', '55333', '79440', '64407', '46559', '60759', '10688', '49872', '45810', '87405', '66932', '56530', '57751', '9619', '27361', '6356', '65848', '7524', '20273', '22362', '20504', '28042', '39475', '51677', '85733', '32426', '54558', '17222', '56485', '34928', '90917', '70528', '51732', '61014', '98420', '67265', '41383', '3883', '47642', '53324', '93679', '93088', '57534', '44449', '46779', '81482', '54279', '80135', '11216', '92545', '18426', '96005', '57801', '21898', '5104', '83467', '72015', '43783', '89674', '57468', '96686', '95167', '38507', '95187', '64923', '71214', '42834', '93219', '47342', '24476', '84834', '29080', '86533', '30687', '68400', '26933', '37396', '65169', '89767', '20642', '53843', '85167', '77306', '46723', '68501', '4243', '35044', '15950', '40388', '53630', '76125', '10816', '83285', '4120', '11402', '91344', '95169') + DROP TABLE IF EXISTS test_in From a9b94b11d6513641ee8a04ca65b4298bdcb711d4 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Thu, 19 Mar 2020 13:29:49 +0300 Subject: [PATCH 0113/1355] empty commit From 783a898b9dac0f09e609ffbdd47b50109a7171e8 Mon Sep 17 00:00:00 2001 From: Avogar Date: Thu, 12 Mar 2020 20:19:40 +0300 Subject: [PATCH 0114/1355] Add MsgPackRowOutputFormat. --- dbms/programs/server/metadata/test_n2zcw0.sql | 2 + .../server/metadata/test_n2zcw0/t.sql | 8 + dbms/src/Formats/FormatFactory.cpp | 1 + dbms/src/Formats/FormatFactory.h | 4 +- .../Formats/Impl/MsgPackRowOutputFormat.cpp | 164 ++++++++++++++++++ .../Formats/Impl/MsgPackRowOutputFormat.h | 29 ++++ 6 files changed, 207 insertions(+), 1 deletion(-) create mode 100644 dbms/programs/server/metadata/test_n2zcw0.sql create mode 100644 dbms/programs/server/metadata/test_n2zcw0/t.sql create mode 100644 dbms/src/Processors/Formats/Impl/MsgPackRowOutputFormat.cpp create mode 100644 dbms/src/Processors/Formats/Impl/MsgPackRowOutputFormat.h diff --git a/dbms/programs/server/metadata/test_n2zcw0.sql b/dbms/programs/server/metadata/test_n2zcw0.sql new file mode 100644 index 00000000000..80046cd585c --- /dev/null +++ b/dbms/programs/server/metadata/test_n2zcw0.sql @@ -0,0 +1,2 @@ +ATTACH DATABASE test_n2zcw0 +ENGINE = Ordinary diff --git a/dbms/programs/server/metadata/test_n2zcw0/t.sql b/dbms/programs/server/metadata/test_n2zcw0/t.sql new file mode 100644 index 00000000000..64e3abcfe34 --- /dev/null +++ b/dbms/programs/server/metadata/test_n2zcw0/t.sql @@ -0,0 +1,8 @@ +ATTACH TABLE t +( + `a` Int, + `b` Int +) +ENGINE = MergeTree +ORDER BY (a, b) +SETTINGS index_granularity = 400 diff --git a/dbms/src/Formats/FormatFactory.cpp b/dbms/src/Formats/FormatFactory.cpp index a8e27054704..8b6034dad9d 100644 --- a/dbms/src/Formats/FormatFactory.cpp +++ b/dbms/src/Formats/FormatFactory.cpp @@ -352,6 +352,7 @@ FormatFactory::FormatFactory() registerOutputFormatProcessorAvro(*this); registerInputFormatProcessorTemplate(*this); registerOutputFormatProcessorTemplate(*this); + registerOutputFormatProcessorMsgPack(*this); registerFileSegmentationEngineTabSeparated(*this); registerFileSegmentationEngineCSV(*this); diff --git a/dbms/src/Formats/FormatFactory.h b/dbms/src/Formats/FormatFactory.h index 7c18971e0eb..68ba2155642 100644 --- a/dbms/src/Formats/FormatFactory.h +++ b/dbms/src/Formats/FormatFactory.h @@ -171,7 +171,9 @@ void registerOutputFormatProcessorProtobuf(FormatFactory & factory); void registerInputFormatProcessorAvro(FormatFactory & factory); void registerOutputFormatProcessorAvro(FormatFactory & factory); void registerInputFormatProcessorTemplate(FormatFactory & factory); -void registerOutputFormatProcessorTemplate(FormatFactory &factory); +void registerOutputFormatProcessorTemplate(FormatFactory & factory); +void registerOutputFormatProcessorMsgPack(FormatFactory &factory); + /// File Segmentation Engines for parallel reading diff --git a/dbms/src/Processors/Formats/Impl/MsgPackRowOutputFormat.cpp b/dbms/src/Processors/Formats/Impl/MsgPackRowOutputFormat.cpp new file mode 100644 index 00000000000..061f4228158 --- /dev/null +++ b/dbms/src/Processors/Formats/Impl/MsgPackRowOutputFormat.cpp @@ -0,0 +1,164 @@ +#include +#include +#include + +#include +#include +#include + +#include +#include +#include +#include +#include + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int ILLEGAL_COLUMN; +} + +MsgPackRowOutputFormat::MsgPackRowOutputFormat(WriteBuffer & out_, const Block & header_, FormatFactory::WriteCallback callback, const FormatSettings & settings_) + : IRowOutputFormat(header_, out_, callback), settings(settings_), packer(out_) {} + +void MsgPackRowOutputFormat::serializeField(const IColumn & column, DataTypePtr data_type, size_t row_num) +{ + switch (data_type->getTypeId()) + { + case TypeIndex::UInt8: + { + packer.pack_uint8(assert_cast(column).getElement(row_num)); + return; + } + case TypeIndex::UInt16: + { + packer.pack_uint16(assert_cast(column).getElement(row_num)); + return; + } + case TypeIndex::UInt32: + { + packer.pack_uint32(assert_cast(column).getElement(row_num)); + return; + } + case TypeIndex::UInt64: + { + packer.pack_uint64(assert_cast(column).getElement(row_num)); + return; + } + case TypeIndex::Int8: + { + packer.pack_int8(assert_cast(column).getElement(row_num)); + return; + } + case TypeIndex::Int16: + { + packer.pack_int16(assert_cast(column).getElement(row_num)); + return; + } + case TypeIndex::Int32: + { + packer.pack_int32(assert_cast(column).getElement(row_num)); + return; + } + case TypeIndex::Int64: + { + packer.pack_int64(assert_cast(column).getElement(row_num)); + return; + } + case TypeIndex::Float32: + { + packer.pack_float(assert_cast(column).getElement(row_num)); + return; + } + case TypeIndex::Float64: + { + packer.pack_double(assert_cast(column).getElement(row_num)); + return; + } + case TypeIndex::Date: + { + packer.pack_uint16(assert_cast(column).getElement(row_num)); + return; + } + case TypeIndex::DateTime: + { + UInt32 datetime = assert_cast(column).getElement(row_num); + // Timestamp extension type in MsgPack is -1. + packer.pack_ext(sizeof(datetime), -1); + packer.pack_ext_body(reinterpret_cast(&datetime), sizeof(datetime)); + return; + } + case TypeIndex::String: + { + const StringRef & string = assert_cast(column).getDataAt(row_num); + packer.pack_str(string.size); + packer.pack_str_body(string.data, string.size); + return; + } + case TypeIndex::FixedString: + { + const StringRef & string = assert_cast(column).getDataAt(row_num); + packer.pack_str(string.size); + packer.pack_str_body(string.data, string.size); + return; + } + case TypeIndex::Array: + { + auto nested_type = assert_cast(*data_type).getNestedType(); + const ColumnArray & column_array = assert_cast(column); + const IColumn & nested_column = column_array.getData(); + const ColumnArray::Offsets & offsets = column_array.getOffsets(); + size_t offset = offsets[row_num - 1]; + size_t size = offsets[row_num] - offset; + packer.pack_array(size); + for (size_t i = 0; i < size; ++i) + { + serializeField(nested_column, nested_type, offset + i); + } + return; + } + case TypeIndex::Nullable: + { + auto nested_type = removeNullable(data_type); + const ColumnNullable & column_nullable = assert_cast(column); + if (!column_nullable.isNullAt(row_num)) + serializeField(column_nullable.getNestedColumn(), nested_type, row_num); + else + packer.pack_nil(); + return; + } + case TypeIndex::Nothing: + { + packer.pack_nil(); + return; + } + default: + break; + } + throw Exception("Type " + data_type->getName() + " is not supported for MsgPack output format", ErrorCodes::ILLEGAL_COLUMN); +} + +void MsgPackRowOutputFormat::write(const Columns & columns, size_t row_num) +{ + size_t num_columns = columns.size(); + for (size_t i = 0; i < num_columns; ++i) + { + serializeField(*columns[i], types[i], row_num); + } +} + +void registerOutputFormatProcessorMsgPack(FormatFactory & factory) +{ + factory.registerOutputFormatProcessor("MsgPack", []( + WriteBuffer & buf, + const Block & sample, + FormatFactory::WriteCallback callback, + const FormatSettings & settings) + { + return std::make_shared(buf, sample, callback, settings); + }); +} + +} \ No newline at end of file diff --git a/dbms/src/Processors/Formats/Impl/MsgPackRowOutputFormat.h b/dbms/src/Processors/Formats/Impl/MsgPackRowOutputFormat.h new file mode 100644 index 00000000000..20df018b60c --- /dev/null +++ b/dbms/src/Processors/Formats/Impl/MsgPackRowOutputFormat.h @@ -0,0 +1,29 @@ +#pragma once + +#include +#include +#include +#include +#include + + +namespace DB +{ + +class MsgPackRowOutputFormat : public IRowOutputFormat +{ +public: + MsgPackRowOutputFormat(WriteBuffer & out_, const Block & header_, FormatFactory::WriteCallback callback, const FormatSettings & settings_); + + String getName() const override { return "MsgPackRowOutputFormat"; } + + void write(const Columns & columns, size_t row_num) override; + void writeField(const IColumn &, const IDataType &, size_t) override {} + void serializeField(const IColumn & column, DataTypePtr data_type, size_t row_num); + +private: + FormatSettings settings; + msgpack::packer packer; +}; + +} From 28dd5439731b3086b6eee0b57c564f52de6ece24 Mon Sep 17 00:00:00 2001 From: Vxider Date: Fri, 20 Mar 2020 00:51:09 +0800 Subject: [PATCH 0115/1355] Enable parallel distributed INSERT SELECT --- dbms/src/Core/Settings.h | 1 + .../Interpreters/InterpreterInsertQuery.cpp | 164 +++++++++++++----- .../01099_distributed_insert_select.reference | 12 ++ .../01099_distributed_insert_select.sql | 70 ++++++++ 4 files changed, 204 insertions(+), 43 deletions(-) create mode 100644 dbms/tests/queries/0_stateless/01099_distributed_insert_select.reference create mode 100644 dbms/tests/queries/0_stateless/01099_distributed_insert_select.sql diff --git a/dbms/src/Core/Settings.h b/dbms/src/Core/Settings.h index a3b94ad035b..a61dce49a75 100644 --- a/dbms/src/Core/Settings.h +++ b/dbms/src/Core/Settings.h @@ -112,6 +112,7 @@ struct Settings : public SettingsCollection M(SettingBool, skip_unavailable_shards, false, "If 1, ClickHouse silently skips unavailable shards and nodes unresolvable through DNS. Shard is marked as unavailable when none of the replicas can be reached.", 0) \ \ M(SettingBool, distributed_group_by_no_merge, false, "Do not merge aggregation states from different servers for distributed query processing - in case it is for certain that there are different keys on different shards.", 0) \ + M(SettingBool, distributed_insert_select, false, "If true, distributed insert select query in the same cluster will be processed on local tables on every shard", 0) \ M(SettingBool, optimize_skip_unused_shards, false, "Assumes that data is distributed by sharding_key. Optimization to skip unused shards if SELECT query filters by sharding_key.", 0) \ M(SettingUInt64, force_optimize_skip_unused_shards, 0, "Throw an exception if unused shards cannot be skipped (1 - throw only if the table has the sharding key, 2 - always throw.", 0) \ \ diff --git a/dbms/src/Interpreters/InterpreterInsertQuery.cpp b/dbms/src/Interpreters/InterpreterInsertQuery.cpp index 0a49a291c75..6b51360b714 100644 --- a/dbms/src/Interpreters/InterpreterInsertQuery.cpp +++ b/dbms/src/Interpreters/InterpreterInsertQuery.cpp @@ -1,5 +1,6 @@ #include +#include #include #include #include @@ -7,18 +8,23 @@ #include #include #include +#include #include #include +#include #include #include #include #include #include -#include +#include #include #include +#include #include +#include #include +#include #include #include @@ -109,61 +115,133 @@ BlockIO InterpreterInsertQuery::execute() context.checkAccess(AccessType::INSERT, query.table_id, query_sample_block.getNames()); BlockInputStreams in_streams; - size_t out_streams_size = 1; - if (query.select) - { - /// Passing 1 as subquery_depth will disable limiting size of intermediate result. - InterpreterSelectWithUnionQuery interpreter_select{query.select, context, SelectQueryOptions(QueryProcessingStage::Complete, 1)}; + BlockOutputStreams out_streams; + bool is_distributed_insert_select = false; - if (table->supportsParallelInsert() && settings.max_insert_threads > 1) + if (query.select && table->isRemote() && settings.distributed_insert_select) + { + // Distributed INSERT SELECT + std::shared_ptr storage_src; + auto & select_ = query.select->as(); + auto new_query = dynamic_pointer_cast(query.clone()); + if (select_.list_of_selects->children.size() == 1) { - in_streams = interpreter_select.executeWithMultipleStreams(res.pipeline); - out_streams_size = std::min(size_t(settings.max_insert_threads), in_streams.size()); + auto & select_query = select_.list_of_selects->children.at(0)->as(); + JoinedTables joined_tables(Context(context), select_query); + + if (joined_tables.tablesCount() == 1) + { + storage_src = dynamic_pointer_cast(joined_tables.getLeftTableStorage()); + if (storage_src) + { + const auto select_with_union_query = std::make_shared(); + select_with_union_query->list_of_selects = std::make_shared(); + + auto new_select_query = dynamic_pointer_cast(select_query.clone()); + select_with_union_query->list_of_selects->children.push_back(new_select_query); + + new_select_query->replaceDatabaseAndTable(storage_src->getRemoteDatabaseName(), storage_src->getRemoteTableName()); + + new_query->select = select_with_union_query; + } + } } - else + + auto storage_dst = dynamic_pointer_cast(table); + + if (storage_src && storage_dst && storage_src->cluster_name == storage_dst->cluster_name) { - res = interpreter_select.execute(); - in_streams.emplace_back(res.in); - res.in = nullptr; - res.out = nullptr; + is_distributed_insert_select = true; + + const auto & cluster = storage_src->getCluster(); + const auto & shards_info = cluster->getShardsInfo(); + + String new_query_str = queryToString(new_query); + for (size_t shard_index : ext::range(0, shards_info.size())) + { + const auto & shard_info = shards_info[shard_index]; + if (shard_info.isLocal()) + { + InterpreterInsertQuery interpreter(new_query, context); + auto block_io = interpreter.execute(); + in_streams.push_back(block_io.in); + } + else + { + auto timeouts = ConnectionTimeouts::getTCPTimeoutsWithFailover(settings); + auto connections = shard_info.pool->getMany(timeouts, &settings, PoolMode::GET_ONE); + if (connections.empty() || connections.front().isNull()) + throw Exception( + "Expected exactly one connection for shard " + toString(shard_info.shard_num), ErrorCodes::LOGICAL_ERROR); + + /// INSERT SELECT querie return empty block + auto in_stream = std::make_shared(*connections.front(), new_query_str, Block{}, context); + in_streams.push_back(in_stream); + } + out_streams.push_back(std::make_shared(Block())); + } } } - BlockOutputStreams out_streams; - - for (size_t i = 0; i < out_streams_size; i++) + if (!is_distributed_insert_select) { - /// We create a pipeline of several streams, into which we will write data. - BlockOutputStreamPtr out; - - /// 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); - else - out = std::make_shared(table, context, query_ptr, no_destination); - - /// Do not squash blocks if it is a sync INSERT into Distributed, since it lead to double bufferization on client and server side. - /// Client-side bufferization might cause excessive timeouts (especially in case of big blocks). - if (!(context.getSettingsRef().insert_distributed_sync && table->isRemote()) && !no_squash) + size_t out_streams_size = 1; + if (query.select) { - out = std::make_shared( - out, out->getHeader(), context.getSettingsRef().min_insert_block_size_rows, context.getSettingsRef().min_insert_block_size_bytes); + /// Passing 1 as subquery_depth will disable limiting size of intermediate result. + InterpreterSelectWithUnionQuery interpreter_select{ query.select, context, SelectQueryOptions(QueryProcessingStage::Complete, 1)}; + + if (table->supportsParallelInsert() && settings.max_insert_threads > 1) + { + in_streams = interpreter_select.executeWithMultipleStreams(res.pipeline); + out_streams_size = std::min(size_t(settings.max_insert_threads), in_streams.size()); + } + else + { + res = interpreter_select.execute(); + in_streams.emplace_back(res.in); + res.in = nullptr; + res.out = nullptr; + } } - /// 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); + for (size_t i = 0; i < out_streams_size; i++) + { + /// We create a pipeline of several streams, into which we will write data. + BlockOutputStreamPtr out; - if (const auto & constraints = table->getConstraints(); !constraints.empty()) - out = std::make_shared(query.table_id, - out, query_sample_block, table->getConstraints(), context); + /// 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); + else + out = std::make_shared(table, context, query_ptr, no_destination); - auto out_wrapper = std::make_shared(out); - out_wrapper->setProcessListElement(context.getProcessListElement()); - out = std::move(out_wrapper); - out_streams.emplace_back(std::move(out)); + /// Do not squash blocks if it is a sync INSERT into Distributed, since it lead to double bufferization on client and server side. + /// Client-side bufferization might cause excessive timeouts (especially in case of big blocks). + if (!(context.getSettingsRef().insert_distributed_sync && table->isRemote()) && !no_squash) + { + out = std::make_shared( + out, + out->getHeader(), + context.getSettingsRef().min_insert_block_size_rows, + context.getSettingsRef().min_insert_block_size_bytes); + } + + /// 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); + + if (const auto & constraints = table->getConstraints(); !constraints.empty()) + out = std::make_shared( + query.table_id, out, query_sample_block, table->getConstraints(), context); + + auto out_wrapper = std::make_shared(out); + out_wrapper->setProcessListElement(context.getProcessListElement()); + out = std::move(out_wrapper); + out_streams.emplace_back(std::move(out)); + } } /// What type of query: INSERT or INSERT SELECT? diff --git a/dbms/tests/queries/0_stateless/01099_distributed_insert_select.reference b/dbms/tests/queries/0_stateless/01099_distributed_insert_select.reference new file mode 100644 index 00000000000..dc58f3ae4a8 --- /dev/null +++ b/dbms/tests/queries/0_stateless/01099_distributed_insert_select.reference @@ -0,0 +1,12 @@ +test_shard_localhost +0 +1 +2 +test_cluster_two_shards_localhost +0 2 +1 2 +2 2 +test_cluster_two_shards +0 2 +1 2 +2 2 diff --git a/dbms/tests/queries/0_stateless/01099_distributed_insert_select.sql b/dbms/tests/queries/0_stateless/01099_distributed_insert_select.sql new file mode 100644 index 00000000000..66a46d0ec2c --- /dev/null +++ b/dbms/tests/queries/0_stateless/01099_distributed_insert_select.sql @@ -0,0 +1,70 @@ +DROP TABLE IF EXISTS local_01099_a; +DROP TABLE IF EXISTS local_01099_b; +DROP TABLE IF EXISTS distributed_01099_a; +DROP TABLE IF EXISTS distributed_01099_b; + +SET distributed_insert_select=1; + +-- +-- test_shard_localhost +-- + +SELECT 'test_shard_localhost'; + +CREATE TABLE local_01099_a (number UInt64) ENGINE = Log; +CREATE TABLE local_01099_b (number UInt64) ENGINE = Log; +CREATE TABLE distributed_01099_a AS local_01099_a ENGINE = Distributed('test_shard_localhost', currentDatabase(), local_01099_a, rand()); +CREATE TABLE distributed_01099_b AS local_01099_b ENGINE = Distributed('test_shard_localhost', currentDatabase(), local_01099_b, rand()); + +INSERT INTO local_01099_a SELECT number from system.numbers limit 3; +INSERT INTO distributed_01099_b SELECT * from distributed_01099_a; + +SELECT * FROM distributed_01099_b; + +DROP TABLE local_01099_a; +DROP TABLE local_01099_b; +DROP TABLE distributed_01099_a; +DROP TABLE distributed_01099_b; + +-- +-- test_cluster_two_shards_localhost +-- + +SELECT 'test_cluster_two_shards_localhost'; + +CREATE TABLE local_01099_a (number UInt64) ENGINE = Log; +CREATE TABLE local_01099_b (number UInt64) ENGINE = Log; +CREATE TABLE distributed_01099_a AS local_01099_a ENGINE = Distributed('test_cluster_two_shards_localhost', currentDatabase(), local_01099_a, rand()); +CREATE TABLE distributed_01099_b AS local_01099_b ENGINE = Distributed('test_cluster_two_shards_localhost', currentDatabase(), local_01099_b, rand()); + +INSERT INTO local_01099_a SELECT number from system.numbers limit 3; +INSERT INTO distributed_01099_b SELECT * from distributed_01099_a; + +SELECT number, count(number) FROM local_01099_b group by number order by number; + +DROP TABLE local_01099_a; +DROP TABLE local_01099_b; +DROP TABLE distributed_01099_a; +DROP TABLE distributed_01099_b; + +-- +-- test_cluster_two_shards +-- + +SELECT 'test_cluster_two_shards'; + +CREATE TABLE local_01099_a (number UInt64) ENGINE = Log; +CREATE TABLE local_01099_b (number UInt64) ENGINE = Log; +CREATE TABLE distributed_01099_a AS local_01099_a ENGINE = Distributed('test_cluster_two_shards', currentDatabase(), local_01099_a, rand()); +CREATE TABLE distributed_01099_b AS local_01099_b ENGINE = Distributed('test_cluster_two_shards', currentDatabase(), local_01099_b, rand()); + +INSERT INTO local_01099_a SELECT number from system.numbers limit 3; +INSERT INTO distributed_01099_b SELECT * from distributed_01099_a; +SYSTEM FLUSH DISTRIBUTED distributed_01099_b; + +SELECT number, count(number) FROM local_01099_b group by number order by number; + +DROP TABLE local_01099_a; +DROP TABLE local_01099_b; +DROP TABLE distributed_01099_a; +DROP TABLE distributed_01099_b; From 706eeb67a8b4dad06a721e385bb28789fe841e4b Mon Sep 17 00:00:00 2001 From: Vxider Date: Fri, 20 Mar 2020 01:04:59 +0800 Subject: [PATCH 0116/1355] update setting name --- dbms/src/Core/Settings.h | 2 +- dbms/src/Interpreters/InterpreterInsertQuery.cpp | 2 +- ...rence => 01099_parallel_distributed_insert_select.reference} | 0 ..._select.sql => 01099_parallel_distributed_insert_select.sql} | 2 +- 4 files changed, 3 insertions(+), 3 deletions(-) rename dbms/tests/queries/0_stateless/{01099_distributed_insert_select.reference => 01099_parallel_distributed_insert_select.reference} (100%) rename dbms/tests/queries/0_stateless/{01099_distributed_insert_select.sql => 01099_parallel_distributed_insert_select.sql} (98%) diff --git a/dbms/src/Core/Settings.h b/dbms/src/Core/Settings.h index a61dce49a75..7dab0924771 100644 --- a/dbms/src/Core/Settings.h +++ b/dbms/src/Core/Settings.h @@ -112,7 +112,7 @@ struct Settings : public SettingsCollection M(SettingBool, skip_unavailable_shards, false, "If 1, ClickHouse silently skips unavailable shards and nodes unresolvable through DNS. Shard is marked as unavailable when none of the replicas can be reached.", 0) \ \ M(SettingBool, distributed_group_by_no_merge, false, "Do not merge aggregation states from different servers for distributed query processing - in case it is for certain that there are different keys on different shards.", 0) \ - M(SettingBool, distributed_insert_select, false, "If true, distributed insert select query in the same cluster will be processed on local tables on every shard", 0) \ + M(SettingBool, parallel_distributed_insert_select, false, "If true, distributed insert select query in the same cluster will be processed on local tables on every shard", 0) \ M(SettingBool, optimize_skip_unused_shards, false, "Assumes that data is distributed by sharding_key. Optimization to skip unused shards if SELECT query filters by sharding_key.", 0) \ M(SettingUInt64, force_optimize_skip_unused_shards, 0, "Throw an exception if unused shards cannot be skipped (1 - throw only if the table has the sharding key, 2 - always throw.", 0) \ \ diff --git a/dbms/src/Interpreters/InterpreterInsertQuery.cpp b/dbms/src/Interpreters/InterpreterInsertQuery.cpp index 6b51360b714..2e6a39b9bd1 100644 --- a/dbms/src/Interpreters/InterpreterInsertQuery.cpp +++ b/dbms/src/Interpreters/InterpreterInsertQuery.cpp @@ -118,7 +118,7 @@ BlockIO InterpreterInsertQuery::execute() BlockOutputStreams out_streams; bool is_distributed_insert_select = false; - if (query.select && table->isRemote() && settings.distributed_insert_select) + if (query.select && table->isRemote() && settings.parallel_distributed_insert_select) { // Distributed INSERT SELECT std::shared_ptr storage_src; diff --git a/dbms/tests/queries/0_stateless/01099_distributed_insert_select.reference b/dbms/tests/queries/0_stateless/01099_parallel_distributed_insert_select.reference similarity index 100% rename from dbms/tests/queries/0_stateless/01099_distributed_insert_select.reference rename to dbms/tests/queries/0_stateless/01099_parallel_distributed_insert_select.reference diff --git a/dbms/tests/queries/0_stateless/01099_distributed_insert_select.sql b/dbms/tests/queries/0_stateless/01099_parallel_distributed_insert_select.sql similarity index 98% rename from dbms/tests/queries/0_stateless/01099_distributed_insert_select.sql rename to dbms/tests/queries/0_stateless/01099_parallel_distributed_insert_select.sql index 66a46d0ec2c..6e2146ad68c 100644 --- a/dbms/tests/queries/0_stateless/01099_distributed_insert_select.sql +++ b/dbms/tests/queries/0_stateless/01099_parallel_distributed_insert_select.sql @@ -3,7 +3,7 @@ DROP TABLE IF EXISTS local_01099_b; DROP TABLE IF EXISTS distributed_01099_a; DROP TABLE IF EXISTS distributed_01099_b; -SET distributed_insert_select=1; +SET parallel_distributed_insert_select=1; -- -- test_shard_localhost From 404fbc275e6285aa15df5a1976c6b38dd8597b88 Mon Sep 17 00:00:00 2001 From: Vxider Date: Fri, 20 Mar 2020 01:07:58 +0800 Subject: [PATCH 0117/1355] undo code format --- dbms/src/Interpreters/InterpreterInsertQuery.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/src/Interpreters/InterpreterInsertQuery.cpp b/dbms/src/Interpreters/InterpreterInsertQuery.cpp index 2e6a39b9bd1..8f2cbb4b65d 100644 --- a/dbms/src/Interpreters/InterpreterInsertQuery.cpp +++ b/dbms/src/Interpreters/InterpreterInsertQuery.cpp @@ -1,6 +1,5 @@ #include -#include #include #include #include @@ -17,6 +16,7 @@ #include #include #include +#include #include #include #include From fe36c2715238e644c60b70586584178d3731a686 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 19 Mar 2020 20:35:08 +0300 Subject: [PATCH 0118/1355] Fix style. --- dbms/src/Columns/ColumnConst.cpp | 1 + dbms/src/Columns/ColumnDecimal.cpp | 1 + dbms/src/Columns/ColumnFixedString.cpp | 1 + dbms/src/Columns/ColumnString.cpp | 1 + dbms/src/Columns/ColumnTuple.cpp | 1 + dbms/src/Columns/ColumnVector.cpp | 1 + 6 files changed, 6 insertions(+) diff --git a/dbms/src/Columns/ColumnConst.cpp b/dbms/src/Columns/ColumnConst.cpp index 85387b6b84a..29aa09cb554 100644 --- a/dbms/src/Columns/ColumnConst.cpp +++ b/dbms/src/Columns/ColumnConst.cpp @@ -14,6 +14,7 @@ namespace DB namespace ErrorCodes { extern const int SIZES_OF_COLUMNS_DOESNT_MATCH; + extern const int LOGICAL_ERROR; } ColumnConst::ColumnConst(const ColumnPtr & data_, size_t s_) diff --git a/dbms/src/Columns/ColumnDecimal.cpp b/dbms/src/Columns/ColumnDecimal.cpp index 2a0e9c8291e..aa87473c948 100644 --- a/dbms/src/Columns/ColumnDecimal.cpp +++ b/dbms/src/Columns/ColumnDecimal.cpp @@ -24,6 +24,7 @@ namespace ErrorCodes extern const int PARAMETER_OUT_OF_BOUND; extern const int SIZES_OF_COLUMNS_DOESNT_MATCH; extern const int NOT_IMPLEMENTED; + extern const int LOGICAL_ERROR; } template diff --git a/dbms/src/Columns/ColumnFixedString.cpp b/dbms/src/Columns/ColumnFixedString.cpp index 9056e0675f7..d60c487eca9 100644 --- a/dbms/src/Columns/ColumnFixedString.cpp +++ b/dbms/src/Columns/ColumnFixedString.cpp @@ -27,6 +27,7 @@ namespace ErrorCodes extern const int SIZE_OF_FIXED_STRING_DOESNT_MATCH; extern const int SIZES_OF_COLUMNS_DOESNT_MATCH; extern const int PARAMETER_OUT_OF_BOUND; + extern const int LOGICAL_ERROR; } diff --git a/dbms/src/Columns/ColumnString.cpp b/dbms/src/Columns/ColumnString.cpp index ec868d6ec11..dbf90b02f57 100644 --- a/dbms/src/Columns/ColumnString.cpp +++ b/dbms/src/Columns/ColumnString.cpp @@ -19,6 +19,7 @@ namespace ErrorCodes { extern const int PARAMETER_OUT_OF_BOUND; extern const int SIZES_OF_COLUMNS_DOESNT_MATCH; + extern const int LOGICAL_ERROR; } diff --git a/dbms/src/Columns/ColumnTuple.cpp b/dbms/src/Columns/ColumnTuple.cpp index 4a68f6a7ed6..1d331377e2b 100644 --- a/dbms/src/Columns/ColumnTuple.cpp +++ b/dbms/src/Columns/ColumnTuple.cpp @@ -18,6 +18,7 @@ namespace ErrorCodes extern const int ILLEGAL_COLUMN; extern const int NOT_IMPLEMENTED; extern const int CANNOT_INSERT_VALUE_OF_DIFFERENT_SIZE_INTO_TUPLE; + extern const int LOGICAL_ERROR; } diff --git a/dbms/src/Columns/ColumnVector.cpp b/dbms/src/Columns/ColumnVector.cpp index fa60d562814..df995dbfb9e 100644 --- a/dbms/src/Columns/ColumnVector.cpp +++ b/dbms/src/Columns/ColumnVector.cpp @@ -29,6 +29,7 @@ namespace ErrorCodes { extern const int PARAMETER_OUT_OF_BOUND; extern const int SIZES_OF_COLUMNS_DOESNT_MATCH; + extern const int LOGICAL_ERROR; } From dcd8321c7dd7136a8b0578e2297e66410bafbbbb Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 19 Mar 2020 22:02:54 +0300 Subject: [PATCH 0119/1355] Add some tests. --- dbms/src/Columns/tests/gtest_weak_hash_32.cpp | 416 ++++++++++++++++++ 1 file changed, 416 insertions(+) create mode 100644 dbms/src/Columns/tests/gtest_weak_hash_32.cpp diff --git a/dbms/src/Columns/tests/gtest_weak_hash_32.cpp b/dbms/src/Columns/tests/gtest_weak_hash_32.cpp new file mode 100644 index 00000000000..c607d5b0eff --- /dev/null +++ b/dbms/src/Columns/tests/gtest_weak_hash_32.cpp @@ -0,0 +1,416 @@ +#include + +#include +#include +#include +#include + +#include + +#include + +using namespace DB; + +template +void checkColumn(const WeakHash32::Container & hash, const PaddedPODArray & eq_class, size_t allowed_collisions = 0) +{ + /// Check equal rows has equal hash. + { + std::unordered_map map; + + for (size_t i = 0; i < eq_class.size(); ++i) + { + auto & val = eq_class[i]; + auto it = map.find(val); + + if (it == map.end()) + map[val] = hash[i]; + else + ASSERT_EQ(it->second, hash[i]); + } + } + + /// Check have not many collisions. + { + std::unordered_map map; + size_t num_collisions = 0; + + for (size_t i = 0; i < eq_class.size(); ++i) + { + auto & val = eq_class[i]; + auto it = map.find(hash[i]); + + if (it == map.end()) + map[hash[i]] = val; + else if (it->second != val) + ++num_collisions; + } + + ASSERT_LE(num_collisions, allowed_collisions); + } +} + +TEST(WeakHash32, ColumnVectorU8) +{ + auto col = ColumnUInt8::create(); + auto & data = col->getData(); + + for (int _i [[maybe_unused]] : {1, 2}) + { + for (size_t i = 0; i < 265; ++i) + data.push_back(i); + } + + WeakHash32 hash(col->size()); + col->updateWeakHash32(hash); + + checkColumn(hash.getData(), col->getData()); +} + +TEST(WeakHash32, ColumnVectorI8) +{ + auto col = ColumnInt8::create(); + auto & data = col->getData(); + + for (int _i [[maybe_unused]] : {1, 2}) + { + for (int i = -128; i < 128; ++i) + data.push_back(i); + } + + WeakHash32 hash(col->size()); + col->updateWeakHash32(hash); + + checkColumn(hash.getData(), col->getData()); +} + +TEST(WeakHash32, ColumnVectorU16) +{ + auto col = ColumnUInt16::create(); + auto & data = col->getData(); + + for (int _i [[maybe_unused]] : {1, 2}) + { + for (size_t i = 0; i < 65536; ++i) + data.push_back(i); + } + + WeakHash32 hash(col->size()); + col->updateWeakHash32(hash); + + checkColumn(hash.getData(), col->getData()); +} + +TEST(WeakHash32, ColumnVectorI16) +{ + auto col = ColumnInt16::create(); + auto & data = col->getData(); + + for (int _i [[maybe_unused]] : {1, 2}) + { + for (int i = 32768; i < 32768; ++i) + data.push_back(i); + } + + WeakHash32 hash(col->size()); + col->updateWeakHash32(hash); + + checkColumn(hash.getData(), col->getData()); +} + +TEST(WeakHash32, ColumnVectorU32) +{ + auto col = ColumnUInt32::create(); + auto & data = col->getData(); + + for (int _i [[maybe_unused]] : {1, 2}) + { + for (uint64_t i = 0; i < 65536; ++i) + data.push_back(i << 16u); + } + + WeakHash32 hash(col->size()); + col->updateWeakHash32(hash); + + checkColumn(hash.getData(), col->getData()); +} + +TEST(WeakHash32, ColumnVectorI32) +{ + auto col = ColumnInt32::create(); + auto & data = col->getData(); + + for (int _i [[maybe_unused]] : {1, 2}) + { + for (int64_t i = -32768; i < 32768; ++i) + data.push_back(i << 16); + } + + WeakHash32 hash(col->size()); + col->updateWeakHash32(hash); + + checkColumn(hash.getData(), col->getData()); +} + +TEST(WeakHash32, ColumnVectorU64) +{ + auto col = ColumnUInt64::create(); + auto & data = col->getData(); + + for (int _i [[maybe_unused]] : {1, 2}) + { + for (uint64_t i = 0; i < 65536; ++i) + data.push_back(i << 32u); + } + + WeakHash32 hash(col->size()); + col->updateWeakHash32(hash); + + checkColumn(hash.getData(), col->getData()); +} + +TEST(WeakHash32, ColumnVectorI64) +{ + auto col = ColumnInt64::create(); + auto & data = col->getData(); + + for (int _i [[maybe_unused]] : {1, 2}) + { + for (int64_t i = -32768; i < 32768; ++i) + data.push_back(i << 32); + } + + WeakHash32 hash(col->size()); + col->updateWeakHash32(hash); + + checkColumn(hash.getData(), col->getData()); +} + +TEST(WeakHash32, ColumnString_1) +{ + auto col = ColumnString::create(); + auto eq = ColumnUInt32::create(); + auto & data = eq->getData(); + + for (int _i [[maybe_unused]] : {1, 2}) + { + for (int64_t i = 0; i < 65536; ++i) + { + data.push_back(i); + auto str = std::to_string(i); + col->insertData(str.data(), str.size()); + } + } + + WeakHash32 hash(col->size()); + col->updateWeakHash32(hash); + + checkColumn(hash.getData(), data); +} + +TEST(WeakHash32, ColumnString_2) +{ + auto col = ColumnString::create(); + auto eq = ColumnUInt32::create(); + auto & data = eq->getData(); + + /* + * a + * aaa + * aaaa + * ... + * b + * bb + * bbb + */ + for (int _i [[maybe_unused]] : {1, 2}) + { + size_t max_size = 3000; + char letter = 'a'; + for (int64_t i = 0; i < 65536; ++i) + { + data.push_back(i); + size_t s = (i % max_size) + 1; + std::string str(letter, s); + col->insertData(str.data(), str.size()); + + if (s == max_size) + ++letter; + } + } + + WeakHash32 hash(col->size()); + col->updateWeakHash32(hash); + + checkColumn(hash.getData(), data); +} + +TEST(WeakHash32, ColumnString_3) +{ + auto col = ColumnString::create(); + auto eq = ColumnUInt32::create(); + auto & data = eq->getData(); + + /* + * a + * a\0 + * a\0\0 + * ... + * b + * b\0 + * b\0\0 + */ + for (int _i [[maybe_unused]] : {1, 2}) + { + size_t max_size = 3000; + char letter = 'a'; + for (int64_t i = 0; i < 65536; ++i) + { + data.push_back(i); + size_t s = (i % max_size) + 1; + std::string str(s,'\0'); + str[0] = letter; + col->insertData(str.data(), str.size()); + + if (s == max_size) + ++letter; + } + } + + WeakHash32 hash(col->size()); + col->updateWeakHash32(hash); + + checkColumn(hash.getData(), data); +} + +TEST(WeakHash32, ColumnFixedString) +{ + size_t max_size = 3000; + auto col = ColumnFixedString::create(max_size); + auto eq = ColumnUInt32::create(); + auto & data = eq->getData(); + + for (int _i [[maybe_unused]] : {1, 2}) + { + char letter = 'a'; + for (int64_t i = 0; i < 65536; ++i) + { + data.push_back(i); + size_t s = (i % max_size) + 1; + std::string str(letter, s); + col->insertData(str.data(), str.size()); + + if (s == max_size) + ++letter; + } + } + + WeakHash32 hash(col->size()); + col->updateWeakHash32(hash); + + checkColumn(hash.getData(), data); +} + +TEST(WeakHash32, ColumnArray) +{ + size_t max_size = 3000; + auto val = ColumnUInt32::create(); + auto off = ColumnUInt64::create(); + auto eq = ColumnUInt32::create(); + auto & eq_data = eq->getData(); + auto & val_data = val->getData(); + auto & off_data = off->getData(); + + /* [1] + * [1, 1] + * [1, 1, 1] + * ... + * [2] + * [2, 2] + * [2, 2, 2] + * ... + */ + for (int _i [[maybe_unused]] : {1, 2}) + { + UInt32 cur = 0; + UInt64 cur_off = 0; + for (int64_t i = 0; i < 65536; ++i) + { + eq_data.push_back(i); + size_t s = (i % max_size) + 1; + + cur_off += s; + off_data.push_back(cur_off); + + for (size_t j = 0; j < s; ++j) + val_data.push_back(cur); + + if (s == max_size) + ++cur; + } + } + + auto col_arr = ColumnArray::create(std::move(val), std::move(off)); + + WeakHash32 hash(col_arr->size()); + col_arr->updateWeakHash32(hash); + + checkColumn(hash.getData(), eq_data); +} + +TEST(WeakHash32, ColumnArrayArray) +{ + size_t max_size = 3000; + auto val = ColumnUInt32::create(); + auto off = ColumnUInt64::create(); + auto off2 = ColumnUInt64::create(); + auto eq = ColumnUInt32::create(); + auto & eq_data = eq->getData(); + auto & val_data = val->getData(); + auto & off_data = off->getData(); + auto & off2_data = off->getData(); + + /* [[0]] + * [[0], [0]] + * [[0], [0], [0]] + * ... + * [[0, 0]] + * [[0, 0], [0, 0]] + * [[0, 0], [0, 0], [0, 0]] + * ... + */ + for (int _i [[maybe_unused]] : {1, 2}) + { + UInt32 cur = 1; + UInt64 cur_off = 0; + UInt64 cur_off2 = 0; + for (int64_t i = 0; i < 65536; ++i) + { + eq_data.push_back(i); + size_t s = (i % max_size) + 1; + + cur_off2 += s; + off2_data.push_back(cur_off2); + + for (size_t j = 0; j < s; ++j) + { + for (size_t k = 0; k < cur; ++k) + val_data.push_back(0); + + cur_off += cur; + off_data.push_back(cur_off); + } + + if (s == max_size) + ++cur; + } + } + + auto col_arr = ColumnArray::create(std::move(val), std::move(off)); + auto col_arr_arr = ColumnArray::create(std::move(col_arr), std::move(off2)); + + WeakHash32 hash(col_arr_arr->size()); + col_arr->updateWeakHash32(hash); + + checkColumn(hash.getData(), eq_data); +} From 8f71c743457b5f86e1fd40bf871ca988e96ef6f7 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 19 Mar 2020 22:54:36 +0300 Subject: [PATCH 0120/1355] Added a test that checks that read from MergeTree with single thread is performed in order --- .../01201_read_single_thread_in_order.reference | 1 + .../01201_read_single_thread_in_order.sql | 16 ++++++++++++++++ 2 files changed, 17 insertions(+) create mode 100644 dbms/tests/queries/0_stateless/01201_read_single_thread_in_order.reference create mode 100644 dbms/tests/queries/0_stateless/01201_read_single_thread_in_order.sql diff --git a/dbms/tests/queries/0_stateless/01201_read_single_thread_in_order.reference b/dbms/tests/queries/0_stateless/01201_read_single_thread_in_order.reference new file mode 100644 index 00000000000..7660873d103 --- /dev/null +++ b/dbms/tests/queries/0_stateless/01201_read_single_thread_in_order.reference @@ -0,0 +1 @@ +[1] diff --git a/dbms/tests/queries/0_stateless/01201_read_single_thread_in_order.sql b/dbms/tests/queries/0_stateless/01201_read_single_thread_in_order.sql new file mode 100644 index 00000000000..7110131905f --- /dev/null +++ b/dbms/tests/queries/0_stateless/01201_read_single_thread_in_order.sql @@ -0,0 +1,16 @@ +DROP TABLE IF EXISTS t; + +CREATE TABLE t +( + number UInt64 +) +ENGINE = MergeTree +ORDER BY number +SETTINGS index_granularity = 128; + +INSERT INTO t SELECT number FROM numbers(10000000); + +SET max_threads = 1, max_block_size = 12345; +SELECT arrayDistinct(arrayPopFront(arrayDifference(groupArray(number)))) FROM t; + +DROP TABLE t; From 93daa16271220b4385522a6a46b06abe262dc584 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 19 Mar 2020 22:55:16 +0300 Subject: [PATCH 0121/1355] Better test --- .../queries/0_stateless/01201_read_single_thread_in_order.sql | 1 + 1 file changed, 1 insertion(+) diff --git a/dbms/tests/queries/0_stateless/01201_read_single_thread_in_order.sql b/dbms/tests/queries/0_stateless/01201_read_single_thread_in_order.sql index 7110131905f..bfe03192891 100644 --- a/dbms/tests/queries/0_stateless/01201_read_single_thread_in_order.sql +++ b/dbms/tests/queries/0_stateless/01201_read_single_thread_in_order.sql @@ -8,6 +8,7 @@ ENGINE = MergeTree ORDER BY number SETTINGS index_granularity = 128; +SET min_insert_block_size_rows = 0, min_insert_block_size_bytes = 0; INSERT INTO t SELECT number FROM numbers(10000000); SET max_threads = 1, max_block_size = 12345; From c45002a2eebff403632ead4ed5a34957dd443fdc Mon Sep 17 00:00:00 2001 From: Artem Zuikov Date: Fri, 20 Mar 2020 03:58:20 +0300 Subject: [PATCH 0122/1355] multiple join rewriter v2 (in progress) --- .../JoinToSubqueryTransformVisitor.cpp | 422 ++++++++++++------ 1 file changed, 292 insertions(+), 130 deletions(-) diff --git a/dbms/src/Interpreters/JoinToSubqueryTransformVisitor.cpp b/dbms/src/Interpreters/JoinToSubqueryTransformVisitor.cpp index 00cb6059a6a..6cf280f4fa7 100644 --- a/dbms/src/Interpreters/JoinToSubqueryTransformVisitor.cpp +++ b/dbms/src/Interpreters/JoinToSubqueryTransformVisitor.cpp @@ -34,12 +34,19 @@ namespace ASTPtr makeSubqueryTemplate() { ParserTablesInSelectQueryElement parser(true); - ASTPtr subquery_template = parseQuery(parser, "(select * from _t)", 0); + ASTPtr subquery_template = parseQuery(parser, "(select * from _t) as `--s`", 0); if (!subquery_template) throw Exception("Cannot parse subquery template", ErrorCodes::LOGICAL_ERROR); return subquery_template; } +ASTPtr makeSubqueryQualifiedAsterisk() +{ + auto asterisk = std::make_shared(); + asterisk->children.emplace_back(std::make_shared("--s")); + return asterisk; +} + /// Replace asterisks in select_expression_list with column identifiers class ExtractAsterisksMatcher { @@ -50,6 +57,7 @@ public: std::vector tables_order; std::shared_ptr new_select_expression_list; + /// V1 Data(const Context & context, const std::vector & table_expressions) { tables_order.reserve(table_expressions.size()); @@ -69,6 +77,19 @@ public: } } + /// V2 + Data(const std::vector & tables) + { + tables_order.reserve(tables.size()); + for (const auto & table : tables) + { + String table_name = table.table.getQualifiedNamePrefix(false); + NamesAndTypesList columns = table.columns; + tables_order.push_back(table_name); + table_columns.emplace(std::move(table_name), std::move(columns)); + } + } + void addTableColumns(const String & table_name) { auto it = table_columns.find(table_name); @@ -380,9 +401,10 @@ struct CollectColumnIdentifiersMatcher static bool needChildVisit(const ASTPtr & node, const ASTPtr &) { - /// Do not go into subqueries. Do not collect table identifiers. + /// Do not go into subqueries. Do not collect table identifiers. Do not get identifier from 't.*'. return !node->as() && - !node->as(); + !node->as() && + !node->as(); } static void visit(const ASTPtr & ast, Data & data) @@ -396,45 +418,45 @@ struct CollectColumnIdentifiersMatcher data.push_back(const_cast(&ident)); } }; +using CollectColumnIdentifiersVisitor = ConstInDepthNodeVisitor; -struct TableNeededColumns +struct CheckAliasDependencyVisitorData { - const DatabaseAndTableWithAlias & table; - NameSet no_clashes = {}; - NameSet column_clashes = {}; /// It's column for sure - NameSet alias_clashes = {}; /// It's column or alias + using TypeToVisit = ASTIdentifier; - void fillExpressionList(ASTExpressionList & expression_list) const + const Aliases & aliases; + const ASTIdentifier * dependency = nullptr; + + void visit(ASTIdentifier & ident, ASTPtr &) { - size_t columns_count = no_clashes.size() + column_clashes.size() + alias_clashes.size(); - expression_list.children.reserve(expression_list.children.size() + columns_count); - - String table_name = table.getQualifiedNamePrefix(false); - - for (auto & column : no_clashes) - addShortName(column, expression_list); - - for (auto & column : column_clashes) - addAliasedName(table_name, column, expression_list); - - for (auto & column : alias_clashes) - addShortName(column, expression_list); - } - - static void addShortName(const String & column, ASTExpressionList & expression_list) - { - auto ident = std::make_shared(column); - expression_list.children.emplace_back(std::move(ident)); - } - - /// t.x as `t.x` - static void addAliasedName(const String & table, const String & column, ASTExpressionList & expression_list) - { - auto ident = std::make_shared(std::vector{table, column}); - ident->setAlias(table + '.' + column); - expression_list.children.emplace_back(std::move(ident)); + if (!dependency && aliases.count(ident.name)) + dependency = &ident; } }; +using CheckAliasDependencyMatcher = OneTypeMatcher; +using CheckAliasDependencyVisitor = InDepthNodeVisitor; + +struct RewriteWithAliasMatcher +{ + using Data = std::unordered_map; + + static bool needChildVisit(const ASTPtr & node, const ASTPtr &) + { + return !node->as(); + } + + static void visit(ASTPtr & ast, Data & data) + { + String alias = ast->tryGetAlias(); + if (!alias.empty()) + { + auto it = data.find(alias); + if (it != data.end() && it->second.get() == ast.get()) + ast = std::make_shared(alias); + } + } +}; +using RewriteWithAliasVisitor = InDepthNodeVisitor; class SubqueryExpressionsRewriteMatcher { @@ -442,9 +464,7 @@ public: struct Data { ASTPtr expression_list; - const String & alias; - bool rewritten = false; - bool aliased = false; + bool done = false; }; static bool needChildVisit(ASTPtr & node, ASTPtr &) @@ -456,28 +476,179 @@ public: { if (auto * t = ast->as()) visit(*t, ast, data); - if (auto * t = ast->as()) - visit(*t, ast, data); } private: static void visit(ASTSelectQuery & select, ASTPtr &, Data & data) { - if (!data.rewritten) + if (!data.done) select.setExpression(ASTSelectQuery::Expression::SELECT, std::move(data.expression_list)); - data.rewritten = true; + data.done = true; + } +}; +using SubqueryExpressionsRewriteVisitor = InDepthNodeVisitor; + +struct TableNeededColumns +{ + const DatabaseAndTableWithAlias & table; + NameSet no_clashes = {}; + NameSet alias_clashes = {}; + std::unordered_map column_clashes = {}; + + void fillExpressionList(ASTExpressionList & expression_list) const + { + size_t columns_count = no_clashes.size() + column_clashes.size() + alias_clashes.size(); + expression_list.children.reserve(expression_list.children.size() + columns_count); + + String table_name = table.getQualifiedNamePrefix(false); + + for (auto & column : no_clashes) + addShortName(column, expression_list); + + for (auto & column : alias_clashes) + addShortName(column, expression_list); + + for (auto & [column, alias] : column_clashes) + addAliasedName(table_name, column, alias, expression_list); } - static void visit(ASTSubquery &, ASTPtr & ast, Data & data) + static void addShortName(const String & column, ASTExpressionList & expression_list) { - if (!data.aliased) - ast->setAlias(data.alias); - data.aliased = true; + auto ident = std::make_shared(column); + expression_list.children.emplace_back(std::move(ident)); + } + + /// t.x as `some` + static void addAliasedName(const String & table, const String & column, const String & alias, ASTExpressionList & expression_list) + { + auto ident = std::make_shared(std::vector{table, column}); + ident->setAlias(alias); + expression_list.children.emplace_back(std::move(ident)); } }; -using CollectColumnIdentifiersVisitor = ConstInDepthNodeVisitor; -using SubqueryExpressionsRewriteVisitor = InDepthNodeVisitor; +class UniqueShortNames +{ +public: + String get(const String & long_name) + { + auto it = names.find(long_name); + if (it != names.end()) + return it->second; + + String unique_name = generateUniqueName(); + names.emplace(long_name, unique_name); + return unique_name; + } + +private: + std::unordered_map names; + size_t counter = 0; + + String generateUniqueName() + { + static constexpr const char * pattern = "--x"; + return String(pattern) + std::to_string(counter++); + } +}; + +size_t countSuchColumns(const std::vector & tables, const String & short_name) +{ + size_t count = 0; + for (auto & table : tables) + if (table.hasColumn(short_name)) + ++count; + return count; +} + +/// Find clashes and normalize names +/// 1. If column name has no clashes make all its occurrences short: 'table.column' -> 'column', 'table_alias.column' -> 'column'. +/// 2. If column name can't be short cause of alias with same name generate and use unique name for it. +/// 3. If column clashes with another column generate and use unique names for them. +/// 4. If column clashes with another column and it's short - it's 'ambiguous column' error. +/// 5. If column clashes with alias add short column name to select list. It would be removed later if not needed. +std::vector normalizeColumnNamesExtractNeeded( + const std::vector & tables, + const Aliases & aliases, + std::vector & identifiers) +{ + UniqueShortNames unique_names; + size_t last_table_pos = tables.size() - 1; + + std::vector needed_columns; + needed_columns.reserve(tables.size()); + for (auto & table : tables) + needed_columns.push_back(TableNeededColumns{table.table}); + + for (ASTIdentifier * ident : identifiers) + { + bool got_alias = aliases.count(ident->name); + + if (auto table_pos = IdentifierSemantic::chooseTable(*ident, tables)) + { + if (!ident->isShort()) + { + if (got_alias) + throw Exception("Alias clashes with qualified column '" + ident->name + "'", ErrorCodes::AMBIGUOUS_COLUMN_NAME); + + String short_name = ident->shortName(); + size_t count = countSuchColumns(tables, short_name); + + if (count > 1 || aliases.count(short_name)) + { + auto & table = tables[*table_pos]; + IdentifierSemantic::setColumnLongName(*ident, table.table); /// table.column -> table_alias.column + + /// For tables moved into subselects we need unique short names for clashed names + if (*table_pos != last_table_pos) + { + auto & unique_long_name = ident->name; + String unique_short_name = unique_names.get(unique_long_name); + ident->setShortName(unique_short_name); + needed_columns[*table_pos].column_clashes.emplace(short_name, unique_short_name); + } + } + else + { + ident->setShortName(short_name); /// table.column -> column + needed_columns[*table_pos].no_clashes.emplace(std::move(short_name)); + } + } + else if (got_alias) + needed_columns[*table_pos].alias_clashes.emplace(ident->shortName()); + else + needed_columns[*table_pos].no_clashes.emplace(ident->shortName()); + } + else if (!got_alias) + throw Exception("Unknown column name '" + ident->name + "'", ErrorCodes::UNKNOWN_IDENTIFIER); + } + + return needed_columns; +} + +/// Make expression list for current subselect +std::shared_ptr subqueryExpressionList( + size_t table_pos, + const std::vector & needed_columns, + const std::vector> & alias_pushdown) +{ + auto expression_list = std::make_shared(); + + /// First time extract needed left table columns manually. + /// Next times extract left table columns via QualifiedAsterisk: `--s`.* + if (table_pos == 1) + needed_columns[0].fillExpressionList(*expression_list); + else + expression_list->children.emplace_back(makeSubqueryQualifiedAsterisk()); + + /// Add needed right table columns + needed_columns[table_pos].fillExpressionList(*expression_list); + + for (auto & expr : alias_pushdown[table_pos]) + expression_list->children.emplace_back(std::move(expr)); + + return expression_list; +} } /// namelesspace @@ -493,121 +664,112 @@ void JoinToSubqueryTransformMatcher::visit(ASTPtr & ast, Data & data) } } -/// The reason for V2: not to alias columns without clashes, use better `t.x` style aliases for others. +/// The reason for V2: not to alias columns without clashes. +/// It allows USING and 'select *' for queries with subselects. It doesn't need AsterisksSemantic and related stuff. +/// 1. Expand asterisks in select expression list. +/// 2. Normalize column names and find name clashes +/// 3. Rewrite multiple JOINs with subqueries: +/// SELECT ... FROM (SELECT `--join`.*, ... FROM (...) AS `--join` JOIN tableY ON ...) AS `--join` JOIN tableZ ON ...' +/// 4. Push down expressions of aliases used in ON section into expression list of first reletad subquery void JoinToSubqueryTransformMatcher::visitV2(ASTSelectQuery & select, ASTPtr & ast, Data & data) { std::vector table_expressions; if (!needRewrite<2>(select, table_expressions)) return; - /// TODO: check table_expressions vs data.tables consistency + auto & src_tables = select.tables()->children; + size_t tables_count = src_tables.size(); + + if (table_expressions.size() != data.tables.size() || + tables_count != data.tables.size()) + throw Exception("Inconsistent tables count in JOIN rewriter", ErrorCodes::LOGICAL_ERROR); + + /// Replace * and t.* with columns in select expression list. + { + ExtractAsterisksVisitor::Data asterisks_data(data.tables); + ExtractAsterisksVisitor(asterisks_data).visit(select.select()); + if (asterisks_data.new_select_expression_list) + select.setExpression(ASTSelectQuery::Expression::SELECT, std::move(asterisks_data.new_select_expression_list)); + } /// Collect column identifiers std::vector identifiers; CollectColumnIdentifiersVisitor(identifiers).visit(ast); - /// JOIN sections - for (auto & child : select.tables()->children) + std::vector using_identifiers; + std::vector> alias_pushdown(tables_count); + std::unordered_map on_aliases; + + /// Collect columns from JOIN sections. Detect if we have aliases there (they need pushdown). + for (size_t table_pos = 0; table_pos < tables_count; ++table_pos) { - auto * table = child->as(); + auto * table = src_tables[table_pos]->as(); if (table->table_join) { auto & join = table->table_join->as(); if (join.on_expression) - CollectColumnIdentifiersVisitor(identifiers).visit(join.on_expression); - /// Nothing special for join.using_expression_list cause it contains short names - } - } - - /// Find clashes and normalize names: - /// 1. If column name has no clashes make all its occurrences short: 'table.column' -> 'column', 'table_alias.column' -> 'column'. - /// 2. If column name can't be short cause of same alias we keep it long converting 'table.column' -> 'table_alias.column' if any. - /// 3. If column clashes with another column keep their names long but convert 'table.column' -> 'table_alias.column' if any. - /// 4. If column clashes with another column and it's short - it's 'ambiguous column' error. - /// 5. If column clashes with alias add short column name to select list. It would be removed later if not needed. - /// @note Source query aliases should not clash with qualified names. - - std::vector needed_columns; - needed_columns.reserve(data.tables.size()); - for (auto & table : data.tables) - needed_columns.push_back(TableNeededColumns{table.table}); - NameSet alias_uses; - - for (ASTIdentifier * ident : identifiers) - { - bool got_alias = data.aliases.count(ident->name); - - if (auto table_pos = IdentifierSemantic::chooseTable(*ident, data.tables)) - { - const String & short_name = ident->shortName(); - if (!ident->isShort()) { - if (got_alias) - throw Exception("Alias clashes with qualified column '" + ident->name + "'", ErrorCodes::AMBIGUOUS_COLUMN_NAME); + std::vector on_identifiers; + CollectColumnIdentifiersVisitor(on_identifiers).visit(join.on_expression); + identifiers.insert(identifiers.end(), on_identifiers.begin(), on_identifiers.end()); - size_t count = 0; - for (auto & table : data.tables) - if (table.hasColumn(short_name)) - ++count; - - if (count > 1 || data.aliases.count(short_name)) + /// Extract aliases used in ON section for pushdown. Exclude the last table. + if (table_pos < tables_count - 1) { - auto & table = data.tables[*table_pos]; - IdentifierSemantic::setColumnLongName(*ident, table.table); /// table.column -> table_alias.column - needed_columns[*table_pos].column_clashes.emplace(short_name); - } - else - { - ident->setShortName(short_name); /// table.column -> column - needed_columns[*table_pos].no_clashes.emplace(short_name); + for (auto * ident : on_identifiers) + { + auto it = data.aliases.find(ident->name); + if (!on_aliases.count(ident->name) && it != data.aliases.end()) + { + auto alias_expression = it->second; + alias_pushdown[table_pos].push_back(alias_expression); + on_aliases[ident->name] = alias_expression; + } + } } } - else if (got_alias) - needed_columns[*table_pos].alias_clashes.emplace(short_name); - else - needed_columns[*table_pos].no_clashes.emplace(short_name); + else if (join.using_expression_list) + CollectColumnIdentifiersVisitor(using_identifiers).visit(join.on_expression); } - else if (got_alias) - alias_uses.insert(ident->name); - else - throw Exception("Unknown column name '" + ident->name + "'", ErrorCodes::UNKNOWN_IDENTIFIER); } - /// Rewrite tables + /// Check if alias expression is too complex to push it down. + for (auto & expr : on_aliases) + { + CheckAliasDependencyVisitor::Data check{data.aliases}; + CheckAliasDependencyVisitor(check).visit(expr.second); + if (check.dependency) + throw Exception("Cannot rewrite JOINs. Alias '" + expr.first + + "' used in ON section depends on another alias '" + check.dependency->name + "'", + ErrorCodes::NOT_IMPLEMENTED); + } + + /// Check same name in aliases, USING and ON sections. Cannot push down alias to ON through USING cause of name masquerading. + for (auto * ident : using_identifiers) + if (on_aliases.count(ident->name)) + throw Exception("Cannot rewrite JOINs. Alias '" + ident->name + "' appears both in ON and USING", ErrorCodes::NOT_IMPLEMENTED); + using_identifiers.clear(); + + /// Replace pushdowned expressions with aliases names in original expression lists. + RewriteWithAliasVisitor(on_aliases).visit(ast); + on_aliases.clear(); + + std::vector needed_columns = + normalizeColumnNamesExtractNeeded(data.tables, data.aliases, identifiers); + + /// Rewrite JOINs with subselects - auto & src_tables = select.tables()->children; ASTPtr left_table = src_tables[0]; static ASTPtr subquery_template = makeSubqueryTemplate(); - static constexpr const char * join_subquery_alias = "--join"; for (size_t i = 1; i < src_tables.size() - 1; ++i) { - String prev_join_alias = String(join_subquery_alias) + std::to_string(i-1); - String current_join_alias = String(join_subquery_alias) + std::to_string(i); - - auto expression_list = std::make_shared(); - { - if (i == 1) - { - /// First time extract needed left table columns manually - needed_columns[0].fillExpressionList(*expression_list); - } - else - { - /// Next times extract left tables via QualifiedAsterisk - auto asterisk = std::make_shared(); - asterisk->children.emplace_back(std::make_shared(prev_join_alias)); - expression_list->children.emplace_back(std::move(asterisk)); - } - - /// Add needed right table columns - needed_columns[i].fillExpressionList(*expression_list); - } + auto expression_list = subqueryExpressionList(i, needed_columns, alias_pushdown); ASTPtr subquery = subquery_template->clone(); - SubqueryExpressionsRewriteVisitor::Data expr_rewrite_data{std::move(expression_list), current_join_alias}; + SubqueryExpressionsRewriteVisitor::Data expr_rewrite_data{std::move(expression_list)}; SubqueryExpressionsRewriteVisitor(expr_rewrite_data).visit(subquery); left_table = replaceJoin(left_table, src_tables[i], subquery); From a8c8466ffb76be8669cdbcb0f25a09e8647cfa70 Mon Sep 17 00:00:00 2001 From: Vxider Date: Fri, 20 Mar 2020 12:48:20 +0800 Subject: [PATCH 0123/1355] fix clang build --- dbms/src/Interpreters/InterpreterInsertQuery.cpp | 9 +++++---- 1 file changed, 5 insertions(+), 4 deletions(-) diff --git a/dbms/src/Interpreters/InterpreterInsertQuery.cpp b/dbms/src/Interpreters/InterpreterInsertQuery.cpp index 8f2cbb4b65d..ab3b64840a0 100644 --- a/dbms/src/Interpreters/InterpreterInsertQuery.cpp +++ b/dbms/src/Interpreters/InterpreterInsertQuery.cpp @@ -37,6 +37,7 @@ namespace ErrorCodes extern const int NO_SUCH_COLUMN_IN_TABLE; extern const int ILLEGAL_COLUMN; extern const int DUPLICATE_COLUMN; + extern const int LOGICAL_ERROR; } @@ -123,7 +124,7 @@ BlockIO InterpreterInsertQuery::execute() // Distributed INSERT SELECT std::shared_ptr storage_src; auto & select_ = query.select->as(); - auto new_query = dynamic_pointer_cast(query.clone()); + auto new_query = std::dynamic_pointer_cast(query.clone()); if (select_.list_of_selects->children.size() == 1) { auto & select_query = select_.list_of_selects->children.at(0)->as(); @@ -131,13 +132,13 @@ BlockIO InterpreterInsertQuery::execute() if (joined_tables.tablesCount() == 1) { - storage_src = dynamic_pointer_cast(joined_tables.getLeftTableStorage()); + storage_src = std::dynamic_pointer_cast(joined_tables.getLeftTableStorage()); if (storage_src) { const auto select_with_union_query = std::make_shared(); select_with_union_query->list_of_selects = std::make_shared(); - auto new_select_query = dynamic_pointer_cast(select_query.clone()); + auto new_select_query = std::dynamic_pointer_cast(select_query.clone()); select_with_union_query->list_of_selects->children.push_back(new_select_query); new_select_query->replaceDatabaseAndTable(storage_src->getRemoteDatabaseName(), storage_src->getRemoteTableName()); @@ -147,7 +148,7 @@ BlockIO InterpreterInsertQuery::execute() } } - auto storage_dst = dynamic_pointer_cast(table); + auto storage_dst = std::dynamic_pointer_cast(table); if (storage_src && storage_dst && storage_src->cluster_name == storage_dst->cluster_name) { From b3c7d2f1848a0269163c08fedbd7d118fd6d1f2c Mon Sep 17 00:00:00 2001 From: Artem Zuikov Date: Fri, 20 Mar 2020 14:31:16 +0300 Subject: [PATCH 0124/1355] fix pubclshed columns --- .../Interpreters/InterpreterSelectQuery.cpp | 2 +- .../JoinToSubqueryTransformVisitor.cpp | 137 +++++++++--------- .../JoinToSubqueryTransformVisitor.h | 1 - 3 files changed, 67 insertions(+), 73 deletions(-) diff --git a/dbms/src/Interpreters/InterpreterSelectQuery.cpp b/dbms/src/Interpreters/InterpreterSelectQuery.cpp index 93210b2aae2..fd25f03717a 100644 --- a/dbms/src/Interpreters/InterpreterSelectQuery.cpp +++ b/dbms/src/Interpreters/InterpreterSelectQuery.cpp @@ -280,7 +280,7 @@ InterpreterSelectQuery::InterpreterSelectQuery( if (!rewriter_version || rewriter_version > 2) throw Exception("Bad multiple_joins_rewriter_version setting value: " + settings.multiple_joins_rewriter_version.toString(), ErrorCodes::INVALID_SETTING_VALUE); - JoinToSubqueryTransformVisitor::Data join_to_subs_data{*context, joined_tables.tablesWithColumns(), aliases, rewriter_version}; + JoinToSubqueryTransformVisitor::Data join_to_subs_data{joined_tables.tablesWithColumns(), aliases, rewriter_version}; JoinToSubqueryTransformVisitor(join_to_subs_data).visit(query_ptr); joined_tables.reset(select); diff --git a/dbms/src/Interpreters/JoinToSubqueryTransformVisitor.cpp b/dbms/src/Interpreters/JoinToSubqueryTransformVisitor.cpp index 6cf280f4fa7..390b67ac794 100644 --- a/dbms/src/Interpreters/JoinToSubqueryTransformVisitor.cpp +++ b/dbms/src/Interpreters/JoinToSubqueryTransformVisitor.cpp @@ -4,8 +4,6 @@ #include #include #include -#include -#include #include #include #include @@ -31,10 +29,12 @@ namespace ErrorCodes namespace { +/// @note we use `--` prefix for unique short names and `--.` for subqueries. +/// It expects that user do not use names starting with `--` and column names starting with dot. ASTPtr makeSubqueryTemplate() { ParserTablesInSelectQueryElement parser(true); - ASTPtr subquery_template = parseQuery(parser, "(select * from _t) as `--s`", 0); + ASTPtr subquery_template = parseQuery(parser, "(select * from _t) as `--.s`", 0); if (!subquery_template) throw Exception("Cannot parse subquery template", ErrorCodes::LOGICAL_ERROR); return subquery_template; @@ -43,7 +43,7 @@ ASTPtr makeSubqueryTemplate() ASTPtr makeSubqueryQualifiedAsterisk() { auto asterisk = std::make_shared(); - asterisk->children.emplace_back(std::make_shared("--s")); + asterisk->children.emplace_back(std::make_shared("--.s")); return asterisk; } @@ -57,27 +57,6 @@ public: std::vector tables_order; std::shared_ptr new_select_expression_list; - /// V1 - Data(const Context & context, const std::vector & table_expressions) - { - tables_order.reserve(table_expressions.size()); - for (const auto & expr : table_expressions) - { - if (expr->subquery) - { - table_columns.clear(); - tables_order.clear(); - break; - } - - String table_name = DatabaseAndTableWithAlias(*expr, context.getCurrentDatabase()).getQualifiedNamePrefix(false); - NamesAndTypesList columns = getColumnsFromTableExpression(*expr, context); - tables_order.push_back(table_name); - table_columns.emplace(std::move(table_name), std::move(columns)); - } - } - - /// V2 Data(const std::vector & tables) { tables_order.reserve(tables.size()); @@ -309,23 +288,6 @@ struct RewriteTablesVisitorData } }; -/// Attach alias to the first visited subquery -struct SetSubqueryAliasVisitorData -{ - using TypeToVisit = ASTSubquery; - - const String & alias; - bool done = false; - - void visit(ASTSubquery &, ASTPtr & ast) - { - if (done) - return; - ast->setAlias(alias); - done = true; - } -}; - template bool needRewrite(ASTSelectQuery & select, std::vector & table_expressions) { @@ -386,8 +348,6 @@ bool needRewrite(ASTSelectQuery & select, std::vector; using RewriteVisitor = InDepthNodeVisitor; -using SetSubqueryAliasMatcher = OneTypeMatcher; -using SetSubqueryAliasVisitor = InDepthNodeVisitor; using ExtractAsterisksVisitor = ConstInDepthNodeVisitor; using ColumnAliasesVisitor = ConstInDepthNodeVisitor; using AppendSemanticMatcher = OneTypeMatcher; @@ -530,25 +490,27 @@ struct TableNeededColumns class UniqueShortNames { public: - String get(const String & long_name) + /// We know that long names are unique (do not clashes with others). + /// So we could make unique names base on this knolage by adding some unused prefix. + static constexpr const char * pattern = "--"; + + String longToShort(const String & long_name) { - auto it = names.find(long_name); - if (it != names.end()) + auto it = long_to_short.find(long_name); + if (it != long_to_short.end()) return it->second; - String unique_name = generateUniqueName(); - names.emplace(long_name, unique_name); - return unique_name; + String short_name = generateUniqueName(long_name); + long_to_short.emplace(long_name, short_name); + return short_name; } private: - std::unordered_map names; - size_t counter = 0; + std::unordered_map long_to_short; - String generateUniqueName() + String generateUniqueName(const String & long_name) { - static constexpr const char * pattern = "--x"; - return String(pattern) + std::to_string(counter++); + return String(pattern) + long_name; } }; @@ -561,6 +523,23 @@ size_t countSuchColumns(const std::vector & tables return count; } +/// 'select `--t.x`, `--t.x`, ...' -> 'select `--t.x` as `t.x`, `t.x`, ...' +void restoreName(ASTIdentifier & ident, const String & original_name, NameSet & restored_names) +{ + if (!ident.tryGetAlias().empty()) + return; + if (original_name.empty()) + return; + + if (!restored_names.count(original_name)) + { + ident.setAlias(original_name); + restored_names.emplace(original_name); + } + else + ident.setShortName(original_name); +} + /// Find clashes and normalize names /// 1. If column name has no clashes make all its occurrences short: 'table.column' -> 'column', 'table_alias.column' -> 'column'. /// 2. If column name can't be short cause of alias with same name generate and use unique name for it. @@ -570,11 +549,13 @@ size_t countSuchColumns(const std::vector & tables std::vector normalizeColumnNamesExtractNeeded( const std::vector & tables, const Aliases & aliases, - std::vector & identifiers) + const std::vector & identifiers, + const std::unordered_set & public_identifiers, + UniqueShortNames & unique_names) { - UniqueShortNames unique_names; size_t last_table_pos = tables.size() - 1; + NameSet restored_names; std::vector needed_columns; needed_columns.reserve(tables.size()); for (auto & table : tables) @@ -592,18 +573,22 @@ std::vector normalizeColumnNamesExtractNeeded( throw Exception("Alias clashes with qualified column '" + ident->name + "'", ErrorCodes::AMBIGUOUS_COLUMN_NAME); String short_name = ident->shortName(); + String original_long_name; + if (public_identifiers.count(ident)) + original_long_name = ident->name; + size_t count = countSuchColumns(tables, short_name); if (count > 1 || aliases.count(short_name)) { auto & table = tables[*table_pos]; IdentifierSemantic::setColumnLongName(*ident, table.table); /// table.column -> table_alias.column + auto & unique_long_name = ident->name; /// For tables moved into subselects we need unique short names for clashed names if (*table_pos != last_table_pos) { - auto & unique_long_name = ident->name; - String unique_short_name = unique_names.get(unique_long_name); + String unique_short_name = unique_names.longToShort(unique_long_name); ident->setShortName(unique_short_name); needed_columns[*table_pos].column_clashes.emplace(short_name, unique_short_name); } @@ -611,8 +596,10 @@ std::vector normalizeColumnNamesExtractNeeded( else { ident->setShortName(short_name); /// table.column -> column - needed_columns[*table_pos].no_clashes.emplace(std::move(short_name)); + needed_columns[*table_pos].no_clashes.emplace(short_name); } + + restoreName(*ident, original_long_name, restored_names); } else if (got_alias) needed_columns[*table_pos].alias_clashes.emplace(ident->shortName()); @@ -755,8 +742,15 @@ void JoinToSubqueryTransformMatcher::visitV2(ASTSelectQuery & select, ASTPtr & a RewriteWithAliasVisitor(on_aliases).visit(ast); on_aliases.clear(); + /// We need to know if identifier is public. If so we have too keep its output name. + std::unordered_set public_identifiers; + for (auto & top_level_child : select.select()->children) + if (auto * ident = top_level_child->as()) + public_identifiers.insert(ident); + + UniqueShortNames unique_names; std::vector needed_columns = - normalizeColumnNamesExtractNeeded(data.tables, data.aliases, identifiers); + normalizeColumnNamesExtractNeeded(data.tables, data.aliases, identifiers, public_identifiers, unique_names); /// Rewrite JOINs with subselects @@ -789,9 +783,17 @@ void JoinToSubqueryTransformMatcher::visitV1(ASTSelectQuery & select, ASTPtr &, if (!needRewrite(select, table_expressions)) return; - ExtractAsterisksVisitor::Data asterisks_data(data.context, table_expressions); - if (!asterisks_data.table_columns.empty()) + if (table_expressions.size() != data.tables.size()) + throw Exception("Inconsistent tables count in JOIN rewriter", ErrorCodes::LOGICAL_ERROR); + + bool has_subquery = false; + for (const auto & expr : table_expressions) + if (expr->subquery) + has_subquery = true; + + if (!has_subquery) { + ExtractAsterisksVisitor::Data asterisks_data(data.tables); ExtractAsterisksVisitor(asterisks_data).visit(select.select()); if (asterisks_data.new_select_expression_list) select.setExpression(ASTSelectQuery::Expression::SELECT, std::move(asterisks_data.new_select_expression_list)); @@ -800,6 +802,7 @@ void JoinToSubqueryTransformMatcher::visitV1(ASTSelectQuery & select, ASTPtr &, ColumnAliasesVisitor::Data aliases_data(getDatabaseAndTables(select, "")); if (select.select()) { + /// TODO: there's a bug here. We need to publish only top-level ASTIdentifiers but visitor extracts all. aliases_data.public_names = true; ColumnAliasesVisitor(aliases_data).visit(select.select()); aliases_data.public_names = false; @@ -843,14 +846,6 @@ void JoinToSubqueryTransformMatcher::visitV1(ASTSelectQuery & select, ASTPtr &, if (!left_table) throw Exception("Cannot replace tables with subselect", ErrorCodes::LOGICAL_ERROR); - /// attach an alias to subquery. - /// TODO: remove setting check after testing period - if (data.context.getSettingsRef().joined_subquery_requires_alias) - { - SetSubqueryAliasVisitor::Data alias_data{String("--.join") + std::to_string(i)}; - SetSubqueryAliasVisitor(alias_data).visit(left_table); - } - /// attach data to generated asterisk AppendSemanticVisitor::Data semantic_data{rev_aliases, false}; AppendSemanticVisitor(semantic_data).visit(left_table); diff --git a/dbms/src/Interpreters/JoinToSubqueryTransformVisitor.h b/dbms/src/Interpreters/JoinToSubqueryTransformVisitor.h index e8d5e59802e..a4f841d9923 100644 --- a/dbms/src/Interpreters/JoinToSubqueryTransformVisitor.h +++ b/dbms/src/Interpreters/JoinToSubqueryTransformVisitor.h @@ -18,7 +18,6 @@ class JoinToSubqueryTransformMatcher public: struct Data { - const Context & context; const std::vector & tables; const Aliases & aliases; size_t version = 1; From ae69a53f2be93bbb5e6dded555696f4a5322941b Mon Sep 17 00:00:00 2001 From: Artem Zuikov Date: Fri, 20 Mar 2020 15:09:53 +0300 Subject: [PATCH 0125/1355] update tests --- .../JoinToSubqueryTransformVisitor.cpp | 5 ++--- .../00847_multiple_join_same_column.reference | 6 ++--- .../00849_multiple_comma_join.reference | 22 +++++++++---------- .../0_stateless/00849_multiple_comma_join.sql | 2 +- .../00854_multiple_join_asterisks.reference | 1 + .../00854_multiple_join_asterisks.sql | 8 +++++++ 6 files changed, 26 insertions(+), 18 deletions(-) diff --git a/dbms/src/Interpreters/JoinToSubqueryTransformVisitor.cpp b/dbms/src/Interpreters/JoinToSubqueryTransformVisitor.cpp index 390b67ac794..11237afebb2 100644 --- a/dbms/src/Interpreters/JoinToSubqueryTransformVisitor.cpp +++ b/dbms/src/Interpreters/JoinToSubqueryTransformVisitor.cpp @@ -267,7 +267,6 @@ struct AppendSemanticVisitorData } }; - /// Replaces table elements with pair. struct RewriteTablesVisitorData { @@ -514,7 +513,7 @@ private: } }; -size_t countSuchColumns(const std::vector & tables, const String & short_name) +size_t countTablesWithColumn(const std::vector & tables, const String & short_name) { size_t count = 0; for (auto & table : tables) @@ -577,7 +576,7 @@ std::vector normalizeColumnNamesExtractNeeded( if (public_identifiers.count(ident)) original_long_name = ident->name; - size_t count = countSuchColumns(tables, short_name); + size_t count = countTablesWithColumn(tables, short_name); if (count > 1 || aliases.count(short_name)) { diff --git a/dbms/tests/queries/0_stateless/00847_multiple_join_same_column.reference b/dbms/tests/queries/0_stateless/00847_multiple_join_same_column.reference index 1685a298042..91bd62ca5a3 100644 --- a/dbms/tests/queries/0_stateless/00847_multiple_join_same_column.reference +++ b/dbms/tests/queries/0_stateless/00847_multiple_join_same_column.reference @@ -31,15 +31,15 @@ y.b: 0 │ 1 │ 1 │ 1 │ │ 2 │ 2 │ 2 │ └─────┴─────┴─────┘ -┌─s.a─┬─s.a─┬─s_b─┬─s_b─┐ +┌─s.a─┬─s.a─┬─s_b─┬─s.b─┐ │ 1 │ 1 │ 1 │ 1 │ │ 0 │ 0 │ 0 │ 0 │ └─────┴─────┴─────┴─────┘ -┌─y.a─┬─y.a─┬─y_b─┬─y_b─┐ +┌─y.a─┬─y.a─┬─y_b─┬─y.b─┐ │ 1 │ 1 │ 1 │ 1 │ │ 0 │ 0 │ 0 │ 0 │ └─────┴─────┴─────┴─────┘ -┌─t_a─┬─t_a─┬─s_a─┬─s_a─┬─y_a─┬─y_a─┐ +┌─t.a─┬─t_a─┬─s.a─┬─s_a─┬─y.a─┬─y_a─┐ │ 1 │ 1 │ 1 │ 1 │ 1 │ 1 │ │ 2 │ 2 │ 0 │ 0 │ 0 │ 0 │ └─────┴─────┴─────┴─────┴─────┴─────┘ diff --git a/dbms/tests/queries/0_stateless/00849_multiple_comma_join.reference b/dbms/tests/queries/0_stateless/00849_multiple_comma_join.reference index 453458a6ecf..a0664f1a53c 100644 --- a/dbms/tests/queries/0_stateless/00849_multiple_comma_join.reference +++ b/dbms/tests/queries/0_stateless/00849_multiple_comma_join.reference @@ -1,17 +1,17 @@ SELECT a\nFROM t1_00849\nCROSS JOIN t2_00849 SELECT a\nFROM t1_00849\nALL INNER JOIN t2_00849 ON a = t2_00849.a\nWHERE a = t2_00849.a SELECT a\nFROM t1_00849\nALL INNER JOIN t2_00849 ON b = t2_00849.b\nWHERE b = t2_00849.b -SELECT `--t1_00849.a` AS `t1_00849.a`\nFROM \n(\n SELECT \n a AS `--t1_00849.a`, \n b, \n t2_00849.a AS `--t2_00849.a`, \n t2_00849.b\n FROM t1_00849\n ALL INNER JOIN t2_00849 ON `--t1_00849.a` = `--t2_00849.a`\n)\nALL INNER JOIN t3_00849 ON `--t1_00849.a` = a\nWHERE (`--t1_00849.a` = `--t2_00849.a`) AND (`--t1_00849.a` = a) -SELECT `--t1_00849.a` AS `t1_00849.a`\nFROM \n(\n SELECT \n a AS `--t1_00849.a`, \n b AS `--t1_00849.b`, \n t2_00849.a, \n t2_00849.b AS `--t2_00849.b`\n FROM t1_00849\n ALL INNER JOIN t2_00849 ON `--t1_00849.b` = `--t2_00849.b`\n)\nALL INNER JOIN t3_00849 ON `--t1_00849.b` = b\nWHERE (`--t1_00849.b` = `--t2_00849.b`) AND (`--t1_00849.b` = b) -SELECT `--t1_00849.a` AS `t1_00849.a`\nFROM \n(\n SELECT \n `--t1_00849.a`, \n b, \n `--t2_00849.a`, \n `t2_00849.b`, \n a AS `--t3_00849.a`, \n t3_00849.b\n FROM \n (\n SELECT \n a AS `--t1_00849.a`, \n b, \n t2_00849.a AS `--t2_00849.a`, \n t2_00849.b\n FROM t1_00849\n ALL INNER JOIN t2_00849 ON `--t1_00849.a` = `--t2_00849.a`\n )\n ALL INNER JOIN t3_00849 ON `--t1_00849.a` = `--t3_00849.a`\n)\nALL INNER JOIN t4_00849 ON `--t1_00849.a` = a\nWHERE (`--t1_00849.a` = `--t2_00849.a`) AND (`--t1_00849.a` = `--t3_00849.a`) AND (`--t1_00849.a` = a) -SELECT `--t1_00849.a` AS `t1_00849.a`\nFROM \n(\n SELECT \n `--t1_00849.a`, \n `--t1_00849.b`, \n `t2_00849.a`, \n `--t2_00849.b`, \n a, \n b AS `--t3_00849.b`\n FROM \n (\n SELECT \n a AS `--t1_00849.a`, \n b AS `--t1_00849.b`, \n t2_00849.a, \n t2_00849.b AS `--t2_00849.b`\n FROM t1_00849\n ALL INNER JOIN t2_00849 ON `--t1_00849.b` = `--t2_00849.b`\n )\n ALL INNER JOIN t3_00849 ON `--t1_00849.b` = `--t3_00849.b`\n)\nALL INNER JOIN t4_00849 ON `--t1_00849.b` = b\nWHERE (`--t1_00849.b` = `--t2_00849.b`) AND (`--t1_00849.b` = `--t3_00849.b`) AND (`--t1_00849.b` = b) -SELECT `--t1_00849.a` AS `t1_00849.a`\nFROM \n(\n SELECT \n `--t1_00849.a`, \n b, \n `--t2_00849.a`, \n `t2_00849.b`, \n a AS `--t3_00849.a`, \n t3_00849.b\n FROM \n (\n SELECT \n a AS `--t1_00849.a`, \n b, \n t2_00849.a AS `--t2_00849.a`, \n t2_00849.b\n FROM t1_00849\n ALL INNER JOIN t2_00849 ON `--t2_00849.a` = `--t1_00849.a`\n )\n ALL INNER JOIN t3_00849 ON `--t2_00849.a` = `--t3_00849.a`\n)\nALL INNER JOIN t4_00849 ON `--t2_00849.a` = a\nWHERE (`--t2_00849.a` = `--t1_00849.a`) AND (`--t2_00849.a` = `--t3_00849.a`) AND (`--t2_00849.a` = a) -SELECT `--t1_00849.a` AS `t1_00849.a`\nFROM \n(\n SELECT \n `--t1_00849.a`, \n b, \n `--t2_00849.a`, \n `t2_00849.b`, \n a AS `--t3_00849.a`, \n t3_00849.b\n FROM \n (\n SELECT \n a AS `--t1_00849.a`, \n b, \n t2_00849.a AS `--t2_00849.a`, \n t2_00849.b\n FROM t1_00849\n CROSS JOIN t2_00849\n )\n ALL INNER JOIN t3_00849 ON (`--t3_00849.a` = `--t1_00849.a`) AND (`--t3_00849.a` = `--t2_00849.a`)\n)\nALL INNER JOIN t4_00849 ON `--t3_00849.a` = a\nWHERE (`--t3_00849.a` = `--t1_00849.a`) AND (`--t3_00849.a` = `--t2_00849.a`) AND (`--t3_00849.a` = a) -SELECT `--t1_00849.a` AS `t1_00849.a`\nFROM \n(\n SELECT \n `--t1_00849.a`, \n b, \n `--t2_00849.a`, \n `t2_00849.b`, \n a AS `--t3_00849.a`, \n t3_00849.b\n FROM \n (\n SELECT \n a AS `--t1_00849.a`, \n b, \n t2_00849.a AS `--t2_00849.a`, \n t2_00849.b\n FROM t1_00849\n CROSS JOIN t2_00849\n )\n CROSS JOIN t3_00849\n)\nALL INNER JOIN t4_00849 ON (a = `--t1_00849.a`) AND (a = `--t2_00849.a`) AND (a = `--t3_00849.a`)\nWHERE (a = `--t1_00849.a`) AND (a = `--t2_00849.a`) AND (a = `--t3_00849.a`) -SELECT `--t1_00849.a` AS `t1_00849.a`\nFROM \n(\n SELECT \n `--t1_00849.a`, \n b, \n `--t2_00849.a`, \n `t2_00849.b`, \n a AS `--t3_00849.a`, \n t3_00849.b\n FROM \n (\n SELECT \n a AS `--t1_00849.a`, \n b, \n t2_00849.a AS `--t2_00849.a`, \n t2_00849.b\n FROM t1_00849\n ALL INNER JOIN t2_00849 ON `--t1_00849.a` = `--t2_00849.a`\n )\n ALL INNER JOIN t3_00849 ON `--t2_00849.a` = `--t3_00849.a`\n)\nALL INNER JOIN t4_00849 ON `--t3_00849.a` = a\nWHERE (`--t1_00849.a` = `--t2_00849.a`) AND (`--t2_00849.a` = `--t3_00849.a`) AND (`--t3_00849.a` = a) -SELECT `--t1_00849.a` AS `t1_00849.a`\nFROM \n(\n SELECT \n `--t1_00849.a`, \n b, \n `t2_00849.a`, \n `t2_00849.b`, \n a, \n t3_00849.b\n FROM \n (\n SELECT \n a AS `--t1_00849.a`, \n b, \n t2_00849.a, \n t2_00849.b\n FROM t1_00849\n CROSS JOIN t2_00849\n )\n CROSS JOIN t3_00849\n)\nCROSS JOIN t4_00849 -SELECT `--t1_00849.a` AS `t1_00849.a`\nFROM \n(\n SELECT \n `--t1_00849.a`, \n b, \n `t2_00849.a`, \n `t2_00849.b`, \n a, \n t3_00849.b\n FROM \n (\n SELECT \n a AS `--t1_00849.a`, \n b, \n t2_00849.a, \n t2_00849.b\n FROM t1_00849\n CROSS JOIN t2_00849\n )\n CROSS JOIN t3_00849\n)\nCROSS JOIN t4_00849 -SELECT `--t1_00849.a` AS `t1_00849.a`\nFROM \n(\n SELECT \n a AS `--t1_00849.a`, \n b, \n t2_00849.a AS `--t2_00849.a`, \n t2_00849.b\n FROM t1_00849\n ALL INNER JOIN t2_00849 ON `--t1_00849.a` = `--t2_00849.a`\n)\nCROSS JOIN t3_00849 +SELECT `--t1_00849.a` AS `t1_00849.a`\nFROM \n(\n SELECT \n a AS `--t1_00849.a`, \n b, \n t2_00849.a AS `--t2_00849.a`, \n t2_00849.b\n FROM t1_00849\n ALL INNER JOIN t2_00849 ON `--t1_00849.a` = `--t2_00849.a`\n) AS `--.s`\nALL INNER JOIN t3_00849 ON `--t1_00849.a` = a\nWHERE (`--t1_00849.a` = `--t2_00849.a`) AND (`--t1_00849.a` = a) +SELECT `--t1_00849.a` AS `t1_00849.a`\nFROM \n(\n SELECT \n a AS `--t1_00849.a`, \n b AS `--t1_00849.b`, \n t2_00849.a, \n t2_00849.b AS `--t2_00849.b`\n FROM t1_00849\n ALL INNER JOIN t2_00849 ON `--t1_00849.b` = `--t2_00849.b`\n) AS `--.s`\nALL INNER JOIN t3_00849 ON `--t1_00849.b` = b\nWHERE (`--t1_00849.b` = `--t2_00849.b`) AND (`--t1_00849.b` = b) +SELECT `--t1_00849.a` AS `t1_00849.a`\nFROM \n(\n SELECT \n `--t1_00849.a`, \n b, \n `--t2_00849.a`, \n `t2_00849.b`, \n a AS `--t3_00849.a`, \n t3_00849.b\n FROM \n (\n SELECT \n a AS `--t1_00849.a`, \n b, \n t2_00849.a AS `--t2_00849.a`, \n t2_00849.b\n FROM t1_00849\n ALL INNER JOIN t2_00849 ON `--t1_00849.a` = `--t2_00849.a`\n ) AS `--.s`\n ALL INNER JOIN t3_00849 ON `--t1_00849.a` = `--t3_00849.a`\n) AS `--.s`\nALL INNER JOIN t4_00849 ON `--t1_00849.a` = a\nWHERE (`--t1_00849.a` = `--t2_00849.a`) AND (`--t1_00849.a` = `--t3_00849.a`) AND (`--t1_00849.a` = a) +SELECT `--t1_00849.a` AS `t1_00849.a`\nFROM \n(\n SELECT \n `--t1_00849.a`, \n `--t1_00849.b`, \n `t2_00849.a`, \n `--t2_00849.b`, \n a, \n b AS `--t3_00849.b`\n FROM \n (\n SELECT \n a AS `--t1_00849.a`, \n b AS `--t1_00849.b`, \n t2_00849.a, \n t2_00849.b AS `--t2_00849.b`\n FROM t1_00849\n ALL INNER JOIN t2_00849 ON `--t1_00849.b` = `--t2_00849.b`\n ) AS `--.s`\n ALL INNER JOIN t3_00849 ON `--t1_00849.b` = `--t3_00849.b`\n) AS `--.s`\nALL INNER JOIN t4_00849 ON `--t1_00849.b` = b\nWHERE (`--t1_00849.b` = `--t2_00849.b`) AND (`--t1_00849.b` = `--t3_00849.b`) AND (`--t1_00849.b` = b) +SELECT `--t1_00849.a` AS `t1_00849.a`\nFROM \n(\n SELECT \n `--t1_00849.a`, \n b, \n `--t2_00849.a`, \n `t2_00849.b`, \n a AS `--t3_00849.a`, \n t3_00849.b\n FROM \n (\n SELECT \n a AS `--t1_00849.a`, \n b, \n t2_00849.a AS `--t2_00849.a`, \n t2_00849.b\n FROM t1_00849\n ALL INNER JOIN t2_00849 ON `--t2_00849.a` = `--t1_00849.a`\n ) AS `--.s`\n ALL INNER JOIN t3_00849 ON `--t2_00849.a` = `--t3_00849.a`\n) AS `--.s`\nALL INNER JOIN t4_00849 ON `--t2_00849.a` = a\nWHERE (`--t2_00849.a` = `--t1_00849.a`) AND (`--t2_00849.a` = `--t3_00849.a`) AND (`--t2_00849.a` = a) +SELECT `--t1_00849.a` AS `t1_00849.a`\nFROM \n(\n SELECT \n `--t1_00849.a`, \n b, \n `--t2_00849.a`, \n `t2_00849.b`, \n a AS `--t3_00849.a`, \n t3_00849.b\n FROM \n (\n SELECT \n a AS `--t1_00849.a`, \n b, \n t2_00849.a AS `--t2_00849.a`, \n t2_00849.b\n FROM t1_00849\n CROSS JOIN t2_00849\n ) AS `--.s`\n ALL INNER JOIN t3_00849 ON (`--t3_00849.a` = `--t1_00849.a`) AND (`--t3_00849.a` = `--t2_00849.a`)\n) AS `--.s`\nALL INNER JOIN t4_00849 ON `--t3_00849.a` = a\nWHERE (`--t3_00849.a` = `--t1_00849.a`) AND (`--t3_00849.a` = `--t2_00849.a`) AND (`--t3_00849.a` = a) +SELECT `--t1_00849.a` AS `t1_00849.a`\nFROM \n(\n SELECT \n `--t1_00849.a`, \n b, \n `--t2_00849.a`, \n `t2_00849.b`, \n a AS `--t3_00849.a`, \n t3_00849.b\n FROM \n (\n SELECT \n a AS `--t1_00849.a`, \n b, \n t2_00849.a AS `--t2_00849.a`, \n t2_00849.b\n FROM t1_00849\n CROSS JOIN t2_00849\n ) AS `--.s`\n CROSS JOIN t3_00849\n) AS `--.s`\nALL INNER JOIN t4_00849 ON (a = `--t1_00849.a`) AND (a = `--t2_00849.a`) AND (a = `--t3_00849.a`)\nWHERE (a = `--t1_00849.a`) AND (a = `--t2_00849.a`) AND (a = `--t3_00849.a`) +SELECT `--t1_00849.a` AS `t1_00849.a`\nFROM \n(\n SELECT \n `--t1_00849.a`, \n b, \n `--t2_00849.a`, \n `t2_00849.b`, \n a AS `--t3_00849.a`, \n t3_00849.b\n FROM \n (\n SELECT \n a AS `--t1_00849.a`, \n b, \n t2_00849.a AS `--t2_00849.a`, \n t2_00849.b\n FROM t1_00849\n ALL INNER JOIN t2_00849 ON `--t1_00849.a` = `--t2_00849.a`\n ) AS `--.s`\n ALL INNER JOIN t3_00849 ON `--t2_00849.a` = `--t3_00849.a`\n) AS `--.s`\nALL INNER JOIN t4_00849 ON `--t3_00849.a` = a\nWHERE (`--t1_00849.a` = `--t2_00849.a`) AND (`--t2_00849.a` = `--t3_00849.a`) AND (`--t3_00849.a` = a) +SELECT `--t1_00849.a` AS `t1_00849.a`\nFROM \n(\n SELECT \n `--t1_00849.a`, \n b, \n `t2_00849.a`, \n `t2_00849.b`, \n a, \n t3_00849.b\n FROM \n (\n SELECT \n a AS `--t1_00849.a`, \n b, \n t2_00849.a, \n t2_00849.b\n FROM t1_00849\n CROSS JOIN t2_00849\n ) AS `--.s`\n CROSS JOIN t3_00849\n) AS `--.s`\nCROSS JOIN t4_00849 +SELECT `--t1_00849.a` AS `t1_00849.a`\nFROM \n(\n SELECT \n `--t1_00849.a`, \n b, \n `t2_00849.a`, \n `t2_00849.b`, \n a, \n t3_00849.b\n FROM \n (\n SELECT \n a AS `--t1_00849.a`, \n b, \n t2_00849.a, \n t2_00849.b\n FROM t1_00849\n CROSS JOIN t2_00849\n ) AS `--.s`\n CROSS JOIN t3_00849\n) AS `--.s`\nCROSS JOIN t4_00849 +SELECT `--t1_00849.a` AS `t1_00849.a`\nFROM \n(\n SELECT \n a AS `--t1_00849.a`, \n b, \n t2_00849.a AS `--t2_00849.a`, \n t2_00849.b\n FROM t1_00849\n ALL INNER JOIN t2_00849 ON `--t1_00849.a` = `--t2_00849.a`\n) AS `--.s`\nCROSS JOIN t3_00849 SELECT * FROM t1, t2 1 1 1 1 1 1 1 \N diff --git a/dbms/tests/queries/0_stateless/00849_multiple_comma_join.sql b/dbms/tests/queries/0_stateless/00849_multiple_comma_join.sql index f80daecbe87..332491683b9 100644 --- a/dbms/tests/queries/0_stateless/00849_multiple_comma_join.sql +++ b/dbms/tests/queries/0_stateless/00849_multiple_comma_join.sql @@ -1,6 +1,6 @@ SET enable_debug_queries = 1; SET enable_optimize_predicate_expression = 0; -SET joined_subquery_requires_alias = 0; +SET multiple_joins_rewriter_version = 1; DROP TABLE IF EXISTS t1_00849; DROP TABLE IF EXISTS t2_00849; diff --git a/dbms/tests/queries/0_stateless/00854_multiple_join_asterisks.reference b/dbms/tests/queries/0_stateless/00854_multiple_join_asterisks.reference index 9a2733a6d15..4500bdba755 100644 --- a/dbms/tests/queries/0_stateless/00854_multiple_join_asterisks.reference +++ b/dbms/tests/queries/0_stateless/00854_multiple_join_asterisks.reference @@ -3,3 +3,4 @@ 0 0 0 0 0 0 +0 0 0 diff --git a/dbms/tests/queries/0_stateless/00854_multiple_join_asterisks.sql b/dbms/tests/queries/0_stateless/00854_multiple_join_asterisks.sql index 3697a957c8a..43aef45440c 100644 --- a/dbms/tests/queries/0_stateless/00854_multiple_join_asterisks.sql +++ b/dbms/tests/queries/0_stateless/00854_multiple_join_asterisks.sql @@ -4,6 +4,14 @@ select t1.* from system.one t1 join system.one t2 on t1.dummy = t2.dummy join sy select t2.*, t3.* from system.one t1 join system.one t2 on t1.dummy = t2.dummy join system.one t3 ON t1.dummy = t3.dummy; select t1.dummy, t2.*, t3.dummy from system.one t1 join system.one t2 on t1.dummy = t2.dummy join system.one t3 ON t1.dummy = t3.dummy; +set multiple_joins_rewriter_version = 1; + select t1.dummy, t2.*, t3.dummy from (select * from system.one) t1 join system.one t2 on t1.dummy = t2.dummy join system.one t3 ON t1.dummy = t3.dummy; -- { serverError 48 } + +set multiple_joins_rewriter_version = 2; + +select t1.dummy, t2.*, t3.dummy from (select * from system.one) t1 +join system.one t2 on t1.dummy = t2.dummy +join system.one t3 ON t1.dummy = t3.dummy; From b6d2895511263e0c825e2a5293de36b4d92edd32 Mon Sep 17 00:00:00 2001 From: Artem Zuikov Date: Fri, 20 Mar 2020 15:16:00 +0300 Subject: [PATCH 0126/1355] add test --- .../00849_multiple_comma_join_2.reference | 55 +++++++++++++++ .../00849_multiple_comma_join_2.sql | 69 +++++++++++++++++++ 2 files changed, 124 insertions(+) create mode 100644 dbms/tests/queries/0_stateless/00849_multiple_comma_join_2.reference create mode 100644 dbms/tests/queries/0_stateless/00849_multiple_comma_join_2.sql diff --git a/dbms/tests/queries/0_stateless/00849_multiple_comma_join_2.reference b/dbms/tests/queries/0_stateless/00849_multiple_comma_join_2.reference new file mode 100644 index 00000000000..581e4efca6a --- /dev/null +++ b/dbms/tests/queries/0_stateless/00849_multiple_comma_join_2.reference @@ -0,0 +1,55 @@ +SELECT a\nFROM t1\nCROSS JOIN t2 +SELECT a\nFROM t1\nALL INNER JOIN t2 ON a = t2.a\nWHERE a = t2.a +SELECT a\nFROM t1\nALL INNER JOIN t2 ON b = t2.b\nWHERE b = t2.b +SELECT `--t1.a` AS `t1.a`\nFROM \n(\n SELECT \n a AS `--t1.a`, \n t2.a AS `--t2.a`\n FROM t1\n ALL INNER JOIN t2 ON `--t1.a` = `--t2.a`\n) AS `--.s`\nALL INNER JOIN t3 ON `--t1.a` = a\nWHERE (`--t1.a` = `--t2.a`) AND (`--t1.a` = a) +SELECT `--t1.a` AS `t1.a`\nFROM \n(\n SELECT \n b AS `--t1.b`, \n a AS `--t1.a`, \n t2.b AS `--t2.b`\n FROM t1\n ALL INNER JOIN t2 ON `--t1.b` = `--t2.b`\n) AS `--.s`\nALL INNER JOIN t3 ON `--t1.b` = b\nWHERE (`--t1.b` = `--t2.b`) AND (`--t1.b` = b) +SELECT `--t1.a` AS `t1.a`\nFROM \n(\n SELECT \n `--t1.a`, \n `--t2.a`, \n a AS `--t3.a`\n FROM \n (\n SELECT \n a AS `--t1.a`, \n t2.a AS `--t2.a`\n FROM t1\n ALL INNER JOIN t2 ON `--t1.a` = `--t2.a`\n ) AS `--.s`\n ALL INNER JOIN t3 ON `--t1.a` = `--t3.a`\n) AS `--.s`\nALL INNER JOIN t4 ON `--t1.a` = a\nWHERE (`--t1.a` = `--t2.a`) AND (`--t1.a` = `--t3.a`) AND (`--t1.a` = a) +SELECT `--t1.a` AS `t1.a`\nFROM \n(\n SELECT \n `--t1.b`, \n `--t1.a`, \n `--t2.b`, \n b AS `--t3.b`\n FROM \n (\n SELECT \n b AS `--t1.b`, \n a AS `--t1.a`, \n t2.b AS `--t2.b`\n FROM t1\n ALL INNER JOIN t2 ON `--t1.b` = `--t2.b`\n ) AS `--.s`\n ALL INNER JOIN t3 ON `--t1.b` = `--t3.b`\n) AS `--.s`\nALL INNER JOIN t4 ON `--t1.b` = b\nWHERE (`--t1.b` = `--t2.b`) AND (`--t1.b` = `--t3.b`) AND (`--t1.b` = b) +SELECT `--t1.a` AS `t1.a`\nFROM \n(\n SELECT \n `--t1.a`, \n `--t2.a`, \n a AS `--t3.a`\n FROM \n (\n SELECT \n a AS `--t1.a`, \n t2.a AS `--t2.a`\n FROM t1\n ALL INNER JOIN t2 ON `--t2.a` = `--t1.a`\n ) AS `--.s`\n ALL INNER JOIN t3 ON `--t2.a` = `--t3.a`\n) AS `--.s`\nALL INNER JOIN t4 ON `--t2.a` = a\nWHERE (`--t2.a` = `--t1.a`) AND (`--t2.a` = `--t3.a`) AND (`--t2.a` = a) +SELECT `--t1.a` AS `t1.a`\nFROM \n(\n SELECT \n `--t1.a`, \n `--t2.a`, \n a AS `--t3.a`\n FROM \n (\n SELECT \n a AS `--t1.a`, \n t2.a AS `--t2.a`\n FROM t1\n CROSS JOIN t2\n ) AS `--.s`\n ALL INNER JOIN t3 ON (`--t3.a` = `--t1.a`) AND (`--t3.a` = `--t2.a`)\n) AS `--.s`\nALL INNER JOIN t4 ON `--t3.a` = a\nWHERE (`--t3.a` = `--t1.a`) AND (`--t3.a` = `--t2.a`) AND (`--t3.a` = a) +SELECT `--t1.a` AS `t1.a`\nFROM \n(\n SELECT \n `--t1.a`, \n `--t2.a`, \n a AS `--t3.a`\n FROM \n (\n SELECT \n a AS `--t1.a`, \n t2.a AS `--t2.a`\n FROM t1\n CROSS JOIN t2\n ) AS `--.s`\n CROSS JOIN t3\n) AS `--.s`\nALL INNER JOIN t4 ON (a = `--t1.a`) AND (a = `--t2.a`) AND (a = `--t3.a`)\nWHERE (a = `--t1.a`) AND (a = `--t2.a`) AND (a = `--t3.a`) +SELECT `--t1.a` AS `t1.a`\nFROM \n(\n SELECT \n `--t1.a`, \n `--t2.a`, \n a AS `--t3.a`\n FROM \n (\n SELECT \n a AS `--t1.a`, \n t2.a AS `--t2.a`\n FROM t1\n ALL INNER JOIN t2 ON `--t1.a` = `--t2.a`\n ) AS `--.s`\n ALL INNER JOIN t3 ON `--t2.a` = `--t3.a`\n) AS `--.s`\nALL INNER JOIN t4 ON `--t3.a` = a\nWHERE (`--t1.a` = `--t2.a`) AND (`--t2.a` = `--t3.a`) AND (`--t3.a` = a) +SELECT `--t1.a` AS `t1.a`\nFROM \n(\n SELECT `--t1.a`\n FROM \n (\n SELECT a AS `--t1.a`\n FROM t1\n CROSS JOIN t2\n ) AS `--.s`\n CROSS JOIN t3\n) AS `--.s`\nCROSS JOIN t4 +SELECT `--t1.a` AS `t1.a`\nFROM \n(\n SELECT `--t1.a`\n FROM \n (\n SELECT a AS `--t1.a`\n FROM t1\n CROSS JOIN t2\n ) AS `--.s`\n CROSS JOIN t3\n) AS `--.s`\nCROSS JOIN t4 +SELECT `--t1.a` AS `t1.a`\nFROM \n(\n SELECT \n a AS `--t1.a`, \n t2.a AS `--t2.a`\n FROM t1\n ALL INNER JOIN t2 ON `--t1.a` = `--t2.a`\n) AS `--.s`\nCROSS JOIN t3 +SELECT * FROM t1, t2 +1 1 1 1 +1 1 1 \N +2 2 1 1 +2 2 1 \N +3 3 1 1 +3 3 1 \N +4 4 1 1 +4 4 1 \N +SELECT * FROM t1, t2 WHERE t1.a = t2.a +1 1 1 1 +1 1 1 \N +SELECT t1.a, t2.a FROM t1, t2 WHERE t1.b = t2.b +1 1 +SELECT t1.a, t2.b, t3.b FROM t1, t2, t3 WHERE t1.a = t2.a AND t1.a = t3.a +1 1 1 +1 1 \N +1 \N 1 +1 \N \N +SELECT t1.a, t2.b, t3.b FROM t1, t2, t3 WHERE t1.b = t2.b AND t1.b = t3.b +1 1 1 +SELECT t1.a, t2.b, t3.b, t4.b FROM t1, t2, t3, t4 WHERE t1.a = t2.a AND t1.a = t3.a AND t1.a = t4.a +1 1 1 1 +1 1 1 \N +1 1 \N 1 +1 1 \N \N +1 \N 1 1 +1 \N 1 \N +1 \N \N 1 +1 \N \N \N +SELECT t1.a, t2.b, t3.b, t4.b FROM t1, t2, t3, t4 WHERE t1.b = t2.b AND t1.b = t3.b AND t1.b = t4.b +1 1 1 1 +SELECT t1.a, t2.b, t3.b, t4.b FROM t1, t2, t3, t4 WHERE t1.a = t2.a AND t2.a = t3.a AND t3.a = t4.a +1 1 1 1 +1 1 1 \N +1 1 \N 1 +1 1 \N \N +1 \N 1 1 +1 \N 1 \N +1 \N \N 1 +1 \N \N \N diff --git a/dbms/tests/queries/0_stateless/00849_multiple_comma_join_2.sql b/dbms/tests/queries/0_stateless/00849_multiple_comma_join_2.sql new file mode 100644 index 00000000000..8c4202ac090 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00849_multiple_comma_join_2.sql @@ -0,0 +1,69 @@ +SET enable_debug_queries = 1; +SET enable_optimize_predicate_expression = 0; +SET multiple_joins_rewriter_version = 2; + +DROP TABLE IF EXISTS t1; +DROP TABLE IF EXISTS t2; +DROP TABLE IF EXISTS t3; +DROP TABLE IF EXISTS t4; + +CREATE TABLE t1 (a UInt32, b Nullable(Int32)) ENGINE = Memory; +CREATE TABLE t2 (a UInt32, b Nullable(Int32)) ENGINE = Memory; +CREATE TABLE t3 (a UInt32, b Nullable(Int32)) ENGINE = Memory; +CREATE TABLE t4 (a UInt32, b Nullable(Int32)) ENGINE = Memory; + +ANALYZE SELECT t1.a FROM t1, t2; +ANALYZE SELECT t1.a FROM t1, t2 WHERE t1.a = t2.a; +ANALYZE SELECT t1.a FROM t1, t2 WHERE t1.b = t2.b; +ANALYZE SELECT t1.a FROM t1, t2, t3 WHERE t1.a = t2.a AND t1.a = t3.a; +ANALYZE SELECT t1.a FROM t1, t2, t3 WHERE t1.b = t2.b AND t1.b = t3.b; +ANALYZE SELECT t1.a FROM t1, t2, t3, t4 WHERE t1.a = t2.a AND t1.a = t3.a AND t1.a = t4.a; +ANALYZE SELECT t1.a FROM t1, t2, t3, t4 WHERE t1.b = t2.b AND t1.b = t3.b AND t1.b = t4.b; + +ANALYZE SELECT t1.a FROM t1, t2, t3, t4 WHERE t2.a = t1.a AND t2.a = t3.a AND t2.a = t4.a; +ANALYZE SELECT t1.a FROM t1, t2, t3, t4 WHERE t3.a = t1.a AND t3.a = t2.a AND t3.a = t4.a; +ANALYZE SELECT t1.a FROM t1, t2, t3, t4 WHERE t4.a = t1.a AND t4.a = t2.a AND t4.a = t3.a; +ANALYZE SELECT t1.a FROM t1, t2, t3, t4 WHERE t1.a = t2.a AND t2.a = t3.a AND t3.a = t4.a; + +ANALYZE SELECT t1.a FROM t1, t2, t3, t4; +ANALYZE SELECT t1.a FROM t1 CROSS JOIN t2 CROSS JOIN t3 CROSS JOIN t4; + +ANALYZE SELECT t1.a FROM t1, t2 CROSS JOIN t3; -- { serverError 48 } +ANALYZE SELECT t1.a FROM t1 JOIN t2 USING a CROSS JOIN t3; -- { serverError 48 } +ANALYZE SELECT t1.a FROM t1 JOIN t2 ON t1.a = t2.a CROSS JOIN t3; + +INSERT INTO t1 values (1,1), (2,2), (3,3), (4,4); +INSERT INTO t2 values (1,1), (1, Null); +INSERT INTO t3 values (1,1), (1, Null); +INSERT INTO t4 values (1,1), (1, Null); + +SELECT 'SELECT * FROM t1, t2'; +SELECT * FROM t1, t2 +ORDER BY t1.a, t2.b; +SELECT 'SELECT * FROM t1, t2 WHERE t1.a = t2.a'; +SELECT * FROM t1, t2 WHERE t1.a = t2.a +ORDER BY t1.a, t2.b; +SELECT 'SELECT t1.a, t2.a FROM t1, t2 WHERE t1.b = t2.b'; +SELECT t1.a, t2.b FROM t1, t2 WHERE t1.b = t2.b; +SELECT 'SELECT t1.a, t2.b, t3.b FROM t1, t2, t3 WHERE t1.a = t2.a AND t1.a = t3.a'; +SELECT t1.a, t2.b, t3.b FROM t1, t2, t3 +WHERE t1.a = t2.a AND t1.a = t3.a +ORDER BY t2.b, t3.b; +SELECT 'SELECT t1.a, t2.b, t3.b FROM t1, t2, t3 WHERE t1.b = t2.b AND t1.b = t3.b'; +SELECT t1.a, t2.b, t3.b FROM t1, t2, t3 WHERE t1.b = t2.b AND t1.b = t3.b; +SELECT 'SELECT t1.a, t2.b, t3.b, t4.b FROM t1, t2, t3, t4 WHERE t1.a = t2.a AND t1.a = t3.a AND t1.a = t4.a'; +SELECT t1.a, t2.b, t3.b, t4.b FROM t1, t2, t3, t4 +WHERE t1.a = t2.a AND t1.a = t3.a AND t1.a = t4.a +ORDER BY t2.b, t3.b, t4.b; +SELECT 'SELECT t1.a, t2.b, t3.b, t4.b FROM t1, t2, t3, t4 WHERE t1.b = t2.b AND t1.b = t3.b AND t1.b = t4.b'; +SELECT t1.a, t2.b, t3.b, t4.b FROM t1, t2, t3, t4 +WHERE t1.b = t2.b AND t1.b = t3.b AND t1.b = t4.b; +SELECT 'SELECT t1.a, t2.b, t3.b, t4.b FROM t1, t2, t3, t4 WHERE t1.a = t2.a AND t2.a = t3.a AND t3.a = t4.a'; +SELECT t1.a, t2.b, t3.b, t4.b FROM t1, t2, t3, t4 +WHERE t1.a = t2.a AND t2.a = t3.a AND t3.a = t4.a +ORDER BY t2.b, t3.b, t4.b; + +DROP TABLE t1; +DROP TABLE t2; +DROP TABLE t3; +DROP TABLE t4; From 74327e7f4776d38f3d3498ec4471bac4569d49c5 Mon Sep 17 00:00:00 2001 From: nikitamikhaylov Date: Fri, 20 Mar 2020 16:18:26 +0400 Subject: [PATCH 0127/1355] remove settings from query --- dbms/programs/copier/ClusterCopier.cpp | 12 +++++++++--- 1 file changed, 9 insertions(+), 3 deletions(-) diff --git a/dbms/programs/copier/ClusterCopier.cpp b/dbms/programs/copier/ClusterCopier.cpp index c5266b3e43a..5f4480e6d5c 100644 --- a/dbms/programs/copier/ClusterCopier.cpp +++ b/dbms/programs/copier/ClusterCopier.cpp @@ -603,10 +603,16 @@ TaskStatus ClusterCopier::tryMoveAllPiecesToDestinationTable(const TaskTable & t original_table.second + "_piece_" + toString(current_piece_number)); + Settings settings_push = task_cluster->settings_push; + + /// It is important, ALTER ATTACH PARTITION must be done synchronously + /// And we will execute this ALTER query on each replica of a shard. + /// It is correct, because this query is idempotent. + settings_push.replication_alter_partitions_sync = 2; + query_alter_ast_string += " ALTER TABLE " + getQuotedTable(original_table) + " ATTACH PARTITION " + partition_name + - " FROM " + getQuotedTable(helping_table) + - " SETTINGS replication_alter_partitions_sync=2;"; + " FROM " + getQuotedTable(helping_table); LOG_DEBUG(log, "Executing ALTER query: " << query_alter_ast_string); @@ -616,7 +622,7 @@ TaskStatus ClusterCopier::tryMoveAllPiecesToDestinationTable(const TaskTable & t task_table.cluster_push, query_alter_ast_string, nullptr, - &task_cluster->settings_push, + &settings_push, PoolMode::GET_MANY, ClusterExecutionMode::ON_EACH_NODE); From b0d54e474b90e1542861d508f35b21fbe6357528 Mon Sep 17 00:00:00 2001 From: Artem Zuikov Date: Fri, 20 Mar 2020 15:21:02 +0300 Subject: [PATCH 0128/1355] update comment --- dbms/src/Interpreters/JoinToSubqueryTransformVisitor.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/src/Interpreters/JoinToSubqueryTransformVisitor.cpp b/dbms/src/Interpreters/JoinToSubqueryTransformVisitor.cpp index 11237afebb2..ea508b27dd0 100644 --- a/dbms/src/Interpreters/JoinToSubqueryTransformVisitor.cpp +++ b/dbms/src/Interpreters/JoinToSubqueryTransformVisitor.cpp @@ -655,7 +655,7 @@ void JoinToSubqueryTransformMatcher::visit(ASTPtr & ast, Data & data) /// 1. Expand asterisks in select expression list. /// 2. Normalize column names and find name clashes /// 3. Rewrite multiple JOINs with subqueries: -/// SELECT ... FROM (SELECT `--join`.*, ... FROM (...) AS `--join` JOIN tableY ON ...) AS `--join` JOIN tableZ ON ...' +/// SELECT ... FROM (SELECT `--.s`.*, ... FROM (...) AS `--.s` JOIN tableY ON ...) AS `--.s` JOIN tableZ ON ...' /// 4. Push down expressions of aliases used in ON section into expression list of first reletad subquery void JoinToSubqueryTransformMatcher::visitV2(ASTSelectQuery & select, ASTPtr & ast, Data & data) { From 61dee9ffaa6bdbb94d32970a8eb3e8fb2f716994 Mon Sep 17 00:00:00 2001 From: Artem Zuikov Date: Fri, 20 Mar 2020 15:36:35 +0300 Subject: [PATCH 0129/1355] fix clang-8 warning --- dbms/src/Interpreters/QueryAliasesVisitor.cpp | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/dbms/src/Interpreters/QueryAliasesVisitor.cpp b/dbms/src/Interpreters/QueryAliasesVisitor.cpp index f61eb5cddff..6eae5594810 100644 --- a/dbms/src/Interpreters/QueryAliasesVisitor.cpp +++ b/dbms/src/Interpreters/QueryAliasesVisitor.cpp @@ -90,8 +90,9 @@ void QueryAliasesMatcher::visit(const ASTArrayJoin &, const ASTPtr & ast, Dat /// 1) content of subqueries could change after recursive analysis, and auto-generated column names could become incorrect /// 2) result of different scalar subqueries can be cached inside expressions compilation cache and must have different names template -void QueryAliasesMatcher::visit(const ASTSubquery & const_subquery, const ASTPtr & ast, Data & aliases) +void QueryAliasesMatcher::visit(const ASTSubquery & const_subquery, const ASTPtr & ast, Data & data) { + auto & aliases = data; ASTSubquery & subquery = const_cast(const_subquery); static std::atomic_uint64_t subquery_index = 0; @@ -115,8 +116,9 @@ void QueryAliasesMatcher::visit(const ASTSubquery & const_subquery, const AST } template -void QueryAliasesMatcher::visitOther(const ASTPtr & ast, Data & aliases) +void QueryAliasesMatcher::visitOther(const ASTPtr & ast, Data & data) { + auto & aliases = data; String alias = ast->tryGetAlias(); if (!alias.empty()) { From c9d808e509ad3dd179d485b5d713321a7ae0d277 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 20 Mar 2020 16:03:51 +0300 Subject: [PATCH 0130/1355] Fix weakHash --- dbms/src/Common/HashTable/Hash.h | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/dbms/src/Common/HashTable/Hash.h b/dbms/src/Common/HashTable/Hash.h index 97df2915417..6fb5c137813 100644 --- a/dbms/src/Common/HashTable/Hash.h +++ b/dbms/src/Common/HashTable/Hash.h @@ -78,9 +78,9 @@ inline UInt32 updateWeakHash32(const DB::UInt8 * pos, size_t size, DB::UInt32 u auto value = unalignedLoad(pos); /// 8 bytes were loaded to UInt64 value, but string size is less then 8 bytes. /// We need to zero excessive bytes to remove the garbage. - /// But instead we move bits to the right, so that we had zeros at left. + /// But instead we move bits, so that we had zeros at left. /// It helps to have different hash for strings like 'a' and 'a\0' - value >>= UInt8(8 * (8 - size)); + value <<= UInt8(8 * (8 - size)); return intHashCRC32(value, updated_value); } From c4e73c10c93fd285b0b6942564f9bea098779047 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 20 Mar 2020 16:54:43 +0300 Subject: [PATCH 0131/1355] Fix weakHash --- dbms/src/Common/HashTable/Hash.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/src/Common/HashTable/Hash.h b/dbms/src/Common/HashTable/Hash.h index 6fb5c137813..e6c817c889e 100644 --- a/dbms/src/Common/HashTable/Hash.h +++ b/dbms/src/Common/HashTable/Hash.h @@ -93,7 +93,7 @@ inline UInt32 updateWeakHash32(const DB::UInt8 * pos, size_t size, DB::UInt32 u pos += 8; } - auto word = unalignedLoad(pos - 8); + auto word = unalignedLoad(end - 8); return intHashCRC32(word, updated_value); } From 33db7f3988b8b0e55f3a91895ca1eefab050831f Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 20 Mar 2020 18:48:09 +0300 Subject: [PATCH 0132/1355] Fix weakHash --- dbms/src/Common/HashTable/Hash.h | 1 + 1 file changed, 1 insertion(+) diff --git a/dbms/src/Common/HashTable/Hash.h b/dbms/src/Common/HashTable/Hash.h index e6c817c889e..3ba5a18a5e7 100644 --- a/dbms/src/Common/HashTable/Hash.h +++ b/dbms/src/Common/HashTable/Hash.h @@ -94,6 +94,7 @@ inline UInt32 updateWeakHash32(const DB::UInt8 * pos, size_t size, DB::UInt32 u } auto word = unalignedLoad(end - 8); + word &= (~UInt64(0)) << DB::UInt8(8 * (8 - (end - pos))); return intHashCRC32(word, updated_value); } From ebba6a6100b688ce85f569c8fd0b2719208ed716 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 20 Mar 2020 18:57:05 +0300 Subject: [PATCH 0133/1355] Fix weakHash --- dbms/src/Columns/tests/gtest_weak_hash_32.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/dbms/src/Columns/tests/gtest_weak_hash_32.cpp b/dbms/src/Columns/tests/gtest_weak_hash_32.cpp index c607d5b0eff..0d8fca14243 100644 --- a/dbms/src/Columns/tests/gtest_weak_hash_32.cpp +++ b/dbms/src/Columns/tests/gtest_weak_hash_32.cpp @@ -216,8 +216,8 @@ TEST(WeakHash32, ColumnString_2) /* * a + * aa * aaa - * aaaa * ... * b * bb @@ -231,7 +231,7 @@ TEST(WeakHash32, ColumnString_2) { data.push_back(i); size_t s = (i % max_size) + 1; - std::string str(letter, s); + std::string str(s, letter); col->insertData(str.data(), str.size()); if (s == max_size) From 75192d44aa1269b814abaa612f7e8e22f1da70b5 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 20 Mar 2020 20:05:04 +0300 Subject: [PATCH 0134/1355] Fix weakHash --- dbms/src/Common/HashTable/Hash.h | 15 +++++++++++---- 1 file changed, 11 insertions(+), 4 deletions(-) diff --git a/dbms/src/Common/HashTable/Hash.h b/dbms/src/Common/HashTable/Hash.h index 3ba5a18a5e7..11d84408b0a 100644 --- a/dbms/src/Common/HashTable/Hash.h +++ b/dbms/src/Common/HashTable/Hash.h @@ -85,7 +85,7 @@ inline UInt32 updateWeakHash32(const DB::UInt8 * pos, size_t size, DB::UInt32 u } const auto * end = pos + size; - while (pos + 8 < end) + while (pos + 8 <= end) { auto word = unalignedLoad(pos); updated_value = intHashCRC32(word, updated_value); @@ -93,9 +93,16 @@ inline UInt32 updateWeakHash32(const DB::UInt8 * pos, size_t size, DB::UInt32 u pos += 8; } - auto word = unalignedLoad(end - 8); - word &= (~UInt64(0)) << DB::UInt8(8 * (8 - (end - pos))); - return intHashCRC32(word, updated_value); + if (pos < end) + { + DB::UInt8 tail_size = end - pos; + auto word = unalignedLoad(end - 8); + word &= (~UInt64(0)) << DB::UInt8(8 * (8 - tail_size)); + word |= tail_size; + updated_value = intHashCRC32(word, updated_value); + } + + return updated_value; } template From ad8b579f3a6a44cd426d2c879a3cb53eae2fed64 Mon Sep 17 00:00:00 2001 From: Artem Zuikov Date: Fri, 20 Mar 2020 20:16:36 +0300 Subject: [PATCH 0135/1355] fix clang-8 warning --- dbms/src/Interpreters/JoinToSubqueryTransformVisitor.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/src/Interpreters/JoinToSubqueryTransformVisitor.cpp b/dbms/src/Interpreters/JoinToSubqueryTransformVisitor.cpp index ea508b27dd0..51e5ab44fc2 100644 --- a/dbms/src/Interpreters/JoinToSubqueryTransformVisitor.cpp +++ b/dbms/src/Interpreters/JoinToSubqueryTransformVisitor.cpp @@ -57,7 +57,7 @@ public: std::vector tables_order; std::shared_ptr new_select_expression_list; - Data(const std::vector & tables) + explicit Data(const std::vector & tables) { tables_order.reserve(tables.size()); for (const auto & table : tables) From 0344978634e79cf10408ce8a90ceaddb992563d1 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 20 Mar 2020 20:28:45 +0300 Subject: [PATCH 0136/1355] Fix weakHash --- dbms/src/Columns/tests/gtest_weak_hash_32.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/src/Columns/tests/gtest_weak_hash_32.cpp b/dbms/src/Columns/tests/gtest_weak_hash_32.cpp index 0d8fca14243..394e207f0b1 100644 --- a/dbms/src/Columns/tests/gtest_weak_hash_32.cpp +++ b/dbms/src/Columns/tests/gtest_weak_hash_32.cpp @@ -297,7 +297,7 @@ TEST(WeakHash32, ColumnFixedString) { data.push_back(i); size_t s = (i % max_size) + 1; - std::string str(letter, s); + std::string str(s, letter); col->insertData(str.data(), str.size()); if (s == max_size) From 46f63c1dd8b99787a6897233ba16be85364cfa99 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 20 Mar 2020 20:31:05 +0300 Subject: [PATCH 0137/1355] Fix weakHash --- dbms/src/Columns/ColumnArray.cpp | 2 ++ 1 file changed, 2 insertions(+) diff --git a/dbms/src/Columns/ColumnArray.cpp b/dbms/src/Columns/ColumnArray.cpp index 42ec73327a8..ae4a911c9e4 100644 --- a/dbms/src/Columns/ColumnArray.cpp +++ b/dbms/src/Columns/ColumnArray.cpp @@ -238,6 +238,8 @@ void ColumnArray::updateWeakHash32(WeakHash32 & hash) const /// But much better then xor which lead to similar hash for arrays like [1], [1, 1, 1], [1, 1, 1, 1, 1], ... /// Much better implementation - to add offsets as an optional argument to updateWeakHash32. hash_data[i] = intHashCRC32(internal_hash_data[row], hash_data[i]); + + prev_offset = offsets_data[i]; } } From 62b60a09a4b07445dc2fc77db1e7d65ec88105d4 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Fri, 28 Feb 2020 22:53:05 +0300 Subject: [PATCH 0138/1355] Add missing space in StorageDistributed logging (sharding key) --- dbms/src/Storages/StorageDistributed.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/src/Storages/StorageDistributed.cpp b/dbms/src/Storages/StorageDistributed.cpp index 4ef8f39d1f7..c925e4088c4 100644 --- a/dbms/src/Storages/StorageDistributed.cpp +++ b/dbms/src/Storages/StorageDistributed.cpp @@ -392,7 +392,7 @@ Pipes StorageDistributed::read( if (!smaller_cluster) { LOG_DEBUG(log, "Reading from " << table_id.getNameForLogs() << - (has_sharding_key ? "" : "(no sharding key)") << ": " + (has_sharding_key ? "" : " (no sharding key)") << ": " "Unable to figure out irrelevant shards from WHERE/PREWHERE clauses - " "the query will be sent to all shards of the cluster"); From 391054039ec8e9830d0718080ca5248473a7b295 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Wed, 4 Mar 2020 23:31:12 +0300 Subject: [PATCH 0139/1355] Test for max_distributed_connections (via TCP) --- .../01085_max_distributed_connections.reference | 2 ++ .../01085_max_distributed_connections.sh | 13 +++++++++++++ 2 files changed, 15 insertions(+) create mode 100644 dbms/tests/queries/0_stateless/01085_max_distributed_connections.reference create mode 100755 dbms/tests/queries/0_stateless/01085_max_distributed_connections.sh diff --git a/dbms/tests/queries/0_stateless/01085_max_distributed_connections.reference b/dbms/tests/queries/0_stateless/01085_max_distributed_connections.reference new file mode 100644 index 00000000000..aa47d0d46d4 --- /dev/null +++ b/dbms/tests/queries/0_stateless/01085_max_distributed_connections.reference @@ -0,0 +1,2 @@ +0 +0 diff --git a/dbms/tests/queries/0_stateless/01085_max_distributed_connections.sh b/dbms/tests/queries/0_stateless/01085_max_distributed_connections.sh new file mode 100755 index 00000000000..44d20d048b8 --- /dev/null +++ b/dbms/tests/queries/0_stateless/01085_max_distributed_connections.sh @@ -0,0 +1,13 @@ +#!/usr/bin/env bash + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +. $CURDIR/../shell_config.sh + +opts=( + --max_distributed_connections 2 + --max_threads 1 + --query "SELECT sleepEachRow(1) FROM remote('127.{2,3}', system.one)" +) +# 1.8 less then 2 seconds, but long enough to cover possible load peaks +# "$@" left to pass manual options (like --experimental_use_processors 0) during manual testing +timeout 1.8s $CLICKHOUSE_CLIENT "${opts[@]}" "$@" From 977849df5a53518480878d4649e3b2e84c15e531 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Tue, 17 Mar 2020 10:34:20 +0300 Subject: [PATCH 0140/1355] Test for max_distributed_connections via HTTP --- .../01085_max_distributed_connections_http.reference | 2 ++ .../0_stateless/01085_max_distributed_connections_http.sh | 8 ++++++++ 2 files changed, 10 insertions(+) create mode 100644 dbms/tests/queries/0_stateless/01085_max_distributed_connections_http.reference create mode 100755 dbms/tests/queries/0_stateless/01085_max_distributed_connections_http.sh diff --git a/dbms/tests/queries/0_stateless/01085_max_distributed_connections_http.reference b/dbms/tests/queries/0_stateless/01085_max_distributed_connections_http.reference new file mode 100644 index 00000000000..aa47d0d46d4 --- /dev/null +++ b/dbms/tests/queries/0_stateless/01085_max_distributed_connections_http.reference @@ -0,0 +1,2 @@ +0 +0 diff --git a/dbms/tests/queries/0_stateless/01085_max_distributed_connections_http.sh b/dbms/tests/queries/0_stateless/01085_max_distributed_connections_http.sh new file mode 100755 index 00000000000..7c91fb67de0 --- /dev/null +++ b/dbms/tests/queries/0_stateless/01085_max_distributed_connections_http.sh @@ -0,0 +1,8 @@ +#!/usr/bin/env bash + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +. $CURDIR/../shell_config.sh + +query="SELECT sleepEachRow(1) FROM remote('127.{2,3}', system.one)" +# 1.8 less then 2 seconds, but long enough to cover possible load peaks +timeout 1.8s ${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&max_distributed_connections=2&max_threads=1" -d "$query" From b40eac426f92adac3b086af24c7ee7911ae09f1b Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Mon, 16 Mar 2020 00:22:55 +0300 Subject: [PATCH 0141/1355] Fix max_distributed_connections w/ Processors With Processors max_distributed_connections does not used anymore, since it always uses not more then max_threads (see TCPHandler::processOrdinaryQueryWithProcessors()/ QueryPipeline::execute()/PipelineExecutor::execute()) So just use QueryPipeline::getNumThreads(), that takes max_distributed_connections into account. --- dbms/programs/server/TCPHandler.cpp | 9 +++------ dbms/programs/server/TCPHandler.h | 2 +- dbms/src/Interpreters/InterpreterSelectQuery.cpp | 3 ++- dbms/src/Interpreters/InterpreterSelectQuery.h | 7 +++++++ dbms/src/Interpreters/executeQuery.cpp | 2 +- 5 files changed, 14 insertions(+), 9 deletions(-) diff --git a/dbms/programs/server/TCPHandler.cpp b/dbms/programs/server/TCPHandler.cpp index 40ed50e45a2..d301a6794da 100644 --- a/dbms/programs/server/TCPHandler.cpp +++ b/dbms/programs/server/TCPHandler.cpp @@ -265,7 +265,7 @@ void TCPHandler::runImpl() state.io.onFinish(); } else if (state.io.pipeline.initialized()) - processOrdinaryQueryWithProcessors(query_context->getSettingsRef().max_threads); + processOrdinaryQueryWithProcessors(); else processOrdinaryQuery(); @@ -544,13 +544,10 @@ void TCPHandler::processOrdinaryQuery() } -void TCPHandler::processOrdinaryQueryWithProcessors(size_t num_threads) +void TCPHandler::processOrdinaryQueryWithProcessors() { auto & pipeline = state.io.pipeline; - /// Reduce the number of threads to recommended value. - num_threads = std::min(num_threads, pipeline.getNumThreads()); - /// Send header-block, to allow client to prepare output format for data to send. { auto & header = pipeline.getHeader(); @@ -585,7 +582,7 @@ void TCPHandler::processOrdinaryQueryWithProcessors(size_t num_threads) try { - executor->execute(num_threads); + executor->execute(pipeline.getNumThreads()); } catch (...) { diff --git a/dbms/programs/server/TCPHandler.h b/dbms/programs/server/TCPHandler.h index cac9c8dd2fe..3fec89264be 100644 --- a/dbms/programs/server/TCPHandler.h +++ b/dbms/programs/server/TCPHandler.h @@ -171,7 +171,7 @@ private: /// Process a request that does not require the receiving of data blocks from the client void processOrdinaryQuery(); - void processOrdinaryQueryWithProcessors(size_t num_threads); + void processOrdinaryQueryWithProcessors(); void processTablesStatusRequest(); diff --git a/dbms/src/Interpreters/InterpreterSelectQuery.cpp b/dbms/src/Interpreters/InterpreterSelectQuery.cpp index 085ebe52557..7fd987a300f 100644 --- a/dbms/src/Interpreters/InterpreterSelectQuery.cpp +++ b/dbms/src/Interpreters/InterpreterSelectQuery.cpp @@ -456,7 +456,6 @@ QueryPipeline InterpreterSelectQuery::executeWithProcessors() { QueryPipeline query_pipeline; executeImpl(query_pipeline, input, std::move(input_pipe), query_pipeline); - query_pipeline.setMaxThreads(max_streams); query_pipeline.addInterpreterContext(context); query_pipeline.addStorageHolder(storage); return query_pipeline; @@ -1291,6 +1290,7 @@ void InterpreterSelectQuery::executeFetchColumns( { is_remote = true; max_streams = settings.max_distributed_connections; + pipeline.setMaxThreads(max_streams); } UInt64 max_block_size = settings.max_block_size; @@ -1315,6 +1315,7 @@ void InterpreterSelectQuery::executeFetchColumns( { max_block_size = std::max(UInt64(1), limit_length + limit_offset); max_streams = 1; + pipeline.setMaxThreads(max_streams); } if (!max_block_size) diff --git a/dbms/src/Interpreters/InterpreterSelectQuery.h b/dbms/src/Interpreters/InterpreterSelectQuery.h index d3007813218..2a8662e3d26 100644 --- a/dbms/src/Interpreters/InterpreterSelectQuery.h +++ b/dbms/src/Interpreters/InterpreterSelectQuery.h @@ -122,6 +122,9 @@ private: BlockInputStreamPtr stream_with_non_joined_data; bool union_stream = false; + /// Cache value of InterpreterSelectQuery::max_streams + size_t max_threads = 1; + BlockInputStreamPtr & firstStream() { return streams.at(0); } template @@ -147,6 +150,10 @@ private: bool hasDelayedStream() const { return stream_with_non_joined_data != nullptr; } bool initialized() const { return !streams.empty(); } + + /// Compatibility with QueryPipeline (Processors) + void setMaxThreads(size_t max_threads_) { max_threads = max_threads_; } + size_t getNumThreads() const { return max_threads; } }; template diff --git a/dbms/src/Interpreters/executeQuery.cpp b/dbms/src/Interpreters/executeQuery.cpp index fefca6b580f..3ec43711d9d 100644 --- a/dbms/src/Interpreters/executeQuery.cpp +++ b/dbms/src/Interpreters/executeQuery.cpp @@ -746,7 +746,7 @@ void executeQuery( { auto executor = pipeline.execute(); - executor->execute(context.getSettingsRef().max_threads); + executor->execute(pipeline.getNumThreads()); } } } From 550cfef2ba35fecc6c4cc7c576a2c6c125ef5694 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Mon, 16 Mar 2020 21:45:39 +0300 Subject: [PATCH 0142/1355] Fix max_distributed_connections w/o Processors --- dbms/src/Interpreters/InterpreterSelectQuery.h | 2 ++ .../InterpreterSelectWithUnionQuery.cpp | 18 ++++++++++++++---- .../InterpreterSelectWithUnionQuery.h | 2 ++ 3 files changed, 18 insertions(+), 4 deletions(-) diff --git a/dbms/src/Interpreters/InterpreterSelectQuery.h b/dbms/src/Interpreters/InterpreterSelectQuery.h index 2a8662e3d26..0208af2431f 100644 --- a/dbms/src/Interpreters/InterpreterSelectQuery.h +++ b/dbms/src/Interpreters/InterpreterSelectQuery.h @@ -92,6 +92,8 @@ public: ASTPtr getQuery() const { return query_ptr; } + size_t getMaxStreams() const { return max_streams; } + private: InterpreterSelectQuery( const ASTPtr & query_ptr_, diff --git a/dbms/src/Interpreters/InterpreterSelectWithUnionQuery.cpp b/dbms/src/Interpreters/InterpreterSelectWithUnionQuery.cpp index e76f2668d3d..feec18a1af3 100644 --- a/dbms/src/Interpreters/InterpreterSelectWithUnionQuery.cpp +++ b/dbms/src/Interpreters/InterpreterSelectWithUnionQuery.cpp @@ -34,7 +34,8 @@ InterpreterSelectWithUnionQuery::InterpreterSelectWithUnionQuery( const Names & required_result_column_names) : options(options_), query_ptr(query_ptr_), - context(std::make_shared(context_)) + context(std::make_shared(context_)), + max_streams(context->getSettingsRef().max_threads) { const auto & ast = query_ptr->as(); @@ -196,14 +197,23 @@ BlockInputStreams InterpreterSelectWithUnionQuery::executeWithMultipleStreams(Qu parent_pipeline.addInterpreterContext(context); } + /// Update max_streams due to: + /// - max_distributed_connections for Distributed() engine + /// - max_streams_to_max_threads_ratio + /// + /// XXX: res.pipeline.getMaxThreads() cannot be used since it is capped to + /// number of streams, which is empty for non-Processors case. + max_streams = (*std::min_element(nested_interpreters.begin(), nested_interpreters.end(), [](const auto &a, const auto &b) + { + return a->getMaxStreams() < b->getMaxStreams(); + }))->getMaxStreams(); + return nested_streams; } BlockIO InterpreterSelectWithUnionQuery::execute() { - const Settings & settings = context->getSettingsRef(); - BlockIO res; BlockInputStreams nested_streams = executeWithMultipleStreams(res.pipeline); BlockInputStreamPtr result_stream; @@ -219,7 +229,7 @@ BlockIO InterpreterSelectWithUnionQuery::execute() } else { - result_stream = std::make_shared(nested_streams, nullptr, settings.max_threads); + result_stream = std::make_shared(nested_streams, nullptr, max_streams); nested_streams.clear(); } diff --git a/dbms/src/Interpreters/InterpreterSelectWithUnionQuery.h b/dbms/src/Interpreters/InterpreterSelectWithUnionQuery.h index e18627fec2a..ad78572ab77 100644 --- a/dbms/src/Interpreters/InterpreterSelectWithUnionQuery.h +++ b/dbms/src/Interpreters/InterpreterSelectWithUnionQuery.h @@ -56,6 +56,8 @@ private: Block result_header; + size_t max_streams = 1; + static Block getCommonHeaderForUnion(const Blocks & headers); }; From 27f59c73a2c99863ecccd9cef03b864fef92a999 Mon Sep 17 00:00:00 2001 From: Artem Zuikov Date: Fri, 20 Mar 2020 22:18:42 +0300 Subject: [PATCH 0143/1355] enable new setting for CI and improve test --- dbms/src/Core/Settings.h | 2 +- .../00847_multiple_join_same_column.reference | 12 ++++++++++ .../00847_multiple_join_same_column.sql | 22 +++++++++++++++++++ 3 files changed, 35 insertions(+), 1 deletion(-) diff --git a/dbms/src/Core/Settings.h b/dbms/src/Core/Settings.h index 445641b0e29..a567fbaf627 100644 --- a/dbms/src/Core/Settings.h +++ b/dbms/src/Core/Settings.h @@ -398,7 +398,7 @@ struct Settings : public SettingsCollection M(SettingBool, partial_revokes, false, "Makes it possible to revoke privileges partially.", 0) \ M(SettingBool, deduplicate_blocks_in_dependent_materialized_views, false, "Should deduplicate blocks for materialized views if the block is not a duplicate for the table. Use true to always deduplicate in dependent tables.", 0) \ M(SettingBool, use_compact_format_in_distributed_parts_names, false, "Changes format of directories names for distributed table insert parts.", 0) \ - M(SettingUInt64, multiple_joins_rewriter_version, 1, "1 or 2. Second rewriter version knows about table columns and keep not clashed names as is.", 0) \ + M(SettingUInt64, multiple_joins_rewriter_version, 2, "1 or 2. Second rewriter version knows about table columns and keep not clashed names as is.", 0) \ \ /** Obsolete settings that do nothing but left for compatibility reasons. Remove each one after half a year of obsolescence. */ \ \ diff --git a/dbms/tests/queries/0_stateless/00847_multiple_join_same_column.reference b/dbms/tests/queries/0_stateless/00847_multiple_join_same_column.reference index 91bd62ca5a3..4e813d5a677 100644 --- a/dbms/tests/queries/0_stateless/00847_multiple_join_same_column.reference +++ b/dbms/tests/queries/0_stateless/00847_multiple_join_same_column.reference @@ -31,6 +31,18 @@ y.b: 0 │ 1 │ 1 │ 1 │ │ 2 │ 2 │ 2 │ └─────┴─────┴─────┘ +┌─s.a─┬─s.a─┬─s_b─┬─s_b─┐ +│ 1 │ 1 │ 1 │ 1 │ +│ 0 │ 0 │ 0 │ 0 │ +└─────┴─────┴─────┴─────┘ +┌─y.a─┬─y.a─┬─y_b─┬─y_b─┐ +│ 1 │ 1 │ 1 │ 1 │ +│ 0 │ 0 │ 0 │ 0 │ +└─────┴─────┴─────┴─────┘ +┌─t_a─┬─t_a─┬─s_a─┬─s_a─┬─y_a─┬─y_a─┐ +│ 1 │ 1 │ 1 │ 1 │ 1 │ 1 │ +│ 2 │ 2 │ 0 │ 0 │ 0 │ 0 │ +└─────┴─────┴─────┴─────┴─────┴─────┘ ┌─s.a─┬─s.a─┬─s_b─┬─s.b─┐ │ 1 │ 1 │ 1 │ 1 │ │ 0 │ 0 │ 0 │ 0 │ diff --git a/dbms/tests/queries/0_stateless/00847_multiple_join_same_column.sql b/dbms/tests/queries/0_stateless/00847_multiple_join_same_column.sql index c7f0c6383c2..eae18dba7f3 100644 --- a/dbms/tests/queries/0_stateless/00847_multiple_join_same_column.sql +++ b/dbms/tests/queries/0_stateless/00847_multiple_join_same_column.sql @@ -39,6 +39,28 @@ left join y on y.b = s.b order by t.a format PrettyCompactNoEscapes; +set multiple_joins_rewriter_version = 1; + +select s.a, s.a, s.b as s_b, s.b from t +left join s on s.a = t.a +left join y on s.b = y.b +order by t.a +format PrettyCompactNoEscapes; + +select y.a, y.a, y.b as y_b, y.b from t +left join s on s.a = t.a +left join y on y.b = s.b +order by t.a +format PrettyCompactNoEscapes; + +select t.a, t.a as t_a, s.a, s.a as s_a, y.a, y.a as y_a from t +left join s on t.a = s.a +left join y on y.b = s.b +order by t.a +format PrettyCompactNoEscapes; + +set multiple_joins_rewriter_version = 2; + select s.a, s.a, s.b as s_b, s.b from t left join s on s.a = t.a left join y on s.b = y.b From d3b5620ad6cb5b53d9153586901c53a702781f4c Mon Sep 17 00:00:00 2001 From: Artem Zuikov Date: Sat, 21 Mar 2020 19:13:06 +0300 Subject: [PATCH 0144/1355] multiple_joins_rewriter_version = 1 --- dbms/src/Core/Settings.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/src/Core/Settings.h b/dbms/src/Core/Settings.h index a567fbaf627..445641b0e29 100644 --- a/dbms/src/Core/Settings.h +++ b/dbms/src/Core/Settings.h @@ -398,7 +398,7 @@ struct Settings : public SettingsCollection M(SettingBool, partial_revokes, false, "Makes it possible to revoke privileges partially.", 0) \ M(SettingBool, deduplicate_blocks_in_dependent_materialized_views, false, "Should deduplicate blocks for materialized views if the block is not a duplicate for the table. Use true to always deduplicate in dependent tables.", 0) \ M(SettingBool, use_compact_format_in_distributed_parts_names, false, "Changes format of directories names for distributed table insert parts.", 0) \ - M(SettingUInt64, multiple_joins_rewriter_version, 2, "1 or 2. Second rewriter version knows about table columns and keep not clashed names as is.", 0) \ + M(SettingUInt64, multiple_joins_rewriter_version, 1, "1 or 2. Second rewriter version knows about table columns and keep not clashed names as is.", 0) \ \ /** Obsolete settings that do nothing but left for compatibility reasons. Remove each one after half a year of obsolescence. */ \ \ From 86cb4811b5da8cbab5923906aa87606993a4908a Mon Sep 17 00:00:00 2001 From: Ivan Lezhankin Date: Sat, 21 Mar 2020 22:49:26 +0300 Subject: [PATCH 0145/1355] Build shared jemalloc in splitted mode --- base/common/coverage.cpp | 4 ++-- contrib/jemalloc-cmake/CMakeLists.txt | 2 +- dbms/programs/client/Client.cpp | 4 ++-- dbms/src/Interpreters/evaluateConstantExpression.h | 2 +- 4 files changed, 6 insertions(+), 6 deletions(-) diff --git a/base/common/coverage.cpp b/base/common/coverage.cpp index 0bb36504db6..9f3c5ca653a 100644 --- a/base/common/coverage.cpp +++ b/base/common/coverage.cpp @@ -1,12 +1,12 @@ #include "coverage.h" -#include - #if WITH_COVERAGE # include + # include + # if defined(__clang__) extern "C" void __llvm_profile_dump(); # elif defined(__GNUC__) || defined(__GNUG__) diff --git a/contrib/jemalloc-cmake/CMakeLists.txt b/contrib/jemalloc-cmake/CMakeLists.txt index 3f685bfefd4..4e3a817b481 100644 --- a/contrib/jemalloc-cmake/CMakeLists.txt +++ b/contrib/jemalloc-cmake/CMakeLists.txt @@ -49,7 +49,7 @@ if (ENABLE_JEMALLOC) list(APPEND SRCS ${LIBRARY_DIR}/src/zone.c) endif () - add_library(jemalloc STATIC ${SRCS}) + add_library(jemalloc ${SRCS}) target_include_directories(jemalloc PRIVATE ${LIBRARY_DIR}/include) target_include_directories(jemalloc PUBLIC include) if (ARCH_AMD64) diff --git a/dbms/programs/client/Client.cpp b/dbms/programs/client/Client.cpp index 4885b08ad91..24afea67090 100644 --- a/dbms/programs/client/Client.cpp +++ b/dbms/programs/client/Client.cpp @@ -4,7 +4,7 @@ #if USE_REPLXX # include -#elif USE_READLINE +#elif defined(USE_READLINE) && USE_READLINE # include #else # include @@ -504,7 +504,7 @@ private: #if USE_REPLXX ReplxxLineReader lr(Suggest::instance(), history_file, '\\', config().has("multiline") ? ';' : 0); -#elif USE_READLINE +#elif defined(USE_READLINE) && USE_READLINE ReadlineLineReader lr(Suggest::instance(), history_file, '\\', config().has("multiline") ? ';' : 0); #else LineReader lr(history_file, '\\', config().has("multiline") ? ';' : 0); diff --git a/dbms/src/Interpreters/evaluateConstantExpression.h b/dbms/src/Interpreters/evaluateConstantExpression.h index f3ed1b730dc..8e3fa08a626 100644 --- a/dbms/src/Interpreters/evaluateConstantExpression.h +++ b/dbms/src/Interpreters/evaluateConstantExpression.h @@ -44,7 +44,7 @@ ASTPtr evaluateConstantExpressionOrIdentifierAsLiteral(const ASTPtr & node, cons ASTPtr evaluateConstantExpressionForDatabaseName(const ASTPtr & node, const Context & context); /** Try to fold condition to countable set of constant values. - * @param condition a condition that we try to fold. + * @param node a condition that we try to fold. * @param target_expr expression evaluated over a set of constants. * @return optional blocks each with a single row and a single column for target expression, * or empty blocks if condition is always false, From 598b373b351e50ff9c576141cf00e174f0a7a6c6 Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Thu, 5 Mar 2020 17:42:02 +0300 Subject: [PATCH 0146/1355] Improve ext::scope_guard. --- base/ext/scope_guard.h | 30 +++++++++++++++++++++--------- 1 file changed, 21 insertions(+), 9 deletions(-) diff --git a/base/ext/scope_guard.h b/base/ext/scope_guard.h index f5b986e7ab6..79bad56f360 100644 --- a/base/ext/scope_guard.h +++ b/base/ext/scope_guard.h @@ -12,20 +12,20 @@ class [[nodiscard]] basic_scope_guard { public: constexpr basic_scope_guard() = default; - constexpr basic_scope_guard(basic_scope_guard && src) : function{std::exchange(src.function, {})} {} + constexpr basic_scope_guard(basic_scope_guard && src) : function{src.release()} {} constexpr basic_scope_guard & operator=(basic_scope_guard && src) { if (this != &src) { invoke(); - function = std::exchange(src.function, {}); + function = src.release(); } return *this; } template , void>> - constexpr basic_scope_guard(basic_scope_guard && src) : function{std::exchange(src.function, {})} {} + constexpr basic_scope_guard(basic_scope_guard && src) : function{src.release()} {} template , void>> constexpr basic_scope_guard & operator=(basic_scope_guard && src) @@ -33,7 +33,7 @@ public: if (this != &src) { invoke(); - function = std::exchange(src.function, {}); + function = src.release(); } return *this; } @@ -46,14 +46,26 @@ public: ~basic_scope_guard() { invoke(); } + static constexpr bool is_nullable = std::is_constructible_v; + explicit operator bool() const { - if constexpr (std::is_constructible_v) + if constexpr (is_nullable) return static_cast(function); return true; } - void reset() { function = {}; } + void reset() + { + invoke(); + release(); + } + + F release() + { + static_assert(is_nullable); + return std::exchange(function, {}); + } template , void>> basic_scope_guard & join(basic_scope_guard && other) @@ -62,14 +74,14 @@ public: { if (function) { - function = [x = std::make_shared>(std::move(function), std::exchange(other.function, {}))]() + function = [x = std::make_shared>(std::move(function), other.release())]() { std::move(x->first)(); std::move(x->second)(); }; } else - function = std::exchange(other.function, {}); + function = other.release(); } return *this; } @@ -77,7 +89,7 @@ public: private: void invoke() { - if constexpr (std::is_constructible_v) + if constexpr (is_nullable) { if (!function) return; From 5eb79c2303d162a44c64b5b35148d19a63959418 Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Sun, 15 Mar 2020 16:52:46 +0300 Subject: [PATCH 0147/1355] Cast "const char *" to StringRef is now implicit and constexpr. --- base/common/StringRef.h | 10 +++++----- dbms/src/IO/WriteHelpers.h | 5 ----- dbms/src/Interpreters/Context.cpp | 4 ++-- dbms/src/Interpreters/Context.h | 4 ++-- 4 files changed, 9 insertions(+), 14 deletions(-) diff --git a/base/common/StringRef.h b/base/common/StringRef.h index 961aab58980..076b8982b1d 100644 --- a/base/common/StringRef.h +++ b/base/common/StringRef.h @@ -27,17 +27,17 @@ struct StringRef size_t size = 0; template > - StringRef(const CharT * data_, size_t size_) : data(reinterpret_cast(data_)), size(size_) {} + constexpr StringRef(const CharT * data_, size_t size_) : data(reinterpret_cast(data_)), size(size_) {} StringRef(const std::string & s) : data(s.data()), size(s.size()) {} - StringRef(const std::string_view & s) : data(s.data()), size(s.size()) {} - explicit StringRef(const char * data_) : data(data_), size(strlen(data_)) {} - StringRef() = default; + constexpr StringRef(const std::string_view & s) : data(s.data()), size(s.size()) {} + constexpr StringRef(const char * data_) : StringRef(std::string_view{data_}) {} + constexpr StringRef() = default; std::string toString() const { return std::string(data, size); } explicit operator std::string() const { return toString(); } - explicit operator std::string_view() const { return {data, size}; } + constexpr explicit operator std::string_view() const { return {data, size}; } }; using StringRefs = std::vector; diff --git a/dbms/src/IO/WriteHelpers.h b/dbms/src/IO/WriteHelpers.h index 44b0322ee83..10918fb7b61 100644 --- a/dbms/src/IO/WriteHelpers.h +++ b/dbms/src/IO/WriteHelpers.h @@ -239,11 +239,6 @@ inline void writeFloatText(T x, WriteBuffer & buf) } -inline void writeString(const String & s, WriteBuffer & buf) -{ - buf.write(s.data(), s.size()); -} - inline void writeString(const char * data, size_t size, WriteBuffer & buf) { buf.write(data, size); diff --git a/dbms/src/Interpreters/Context.cpp b/dbms/src/Interpreters/Context.cpp index ab9b4a2c31b..ef45fd60e81 100644 --- a/dbms/src/Interpreters/Context.cpp +++ b/dbms/src/Interpreters/Context.cpp @@ -947,7 +947,7 @@ void Context::setSettings(const Settings & settings_) } -void Context::setSetting(const String & name, const String & value) +void Context::setSetting(const StringRef & name, const String & value) { auto lock = getLock(); if (name == "profile") @@ -962,7 +962,7 @@ void Context::setSetting(const String & name, const String & value) } -void Context::setSetting(const String & name, const Field & value) +void Context::setSetting(const StringRef & name, const Field & value) { auto lock = getLock(); if (name == "profile") diff --git a/dbms/src/Interpreters/Context.h b/dbms/src/Interpreters/Context.h index 5d8351ed598..2f049d14cd3 100644 --- a/dbms/src/Interpreters/Context.h +++ b/dbms/src/Interpreters/Context.h @@ -344,8 +344,8 @@ public: void setSettings(const Settings & settings_); /// Set settings by name. - void setSetting(const String & name, const String & value); - void setSetting(const String & name, const Field & value); + void setSetting(const StringRef & name, const String & value); + void setSetting(const StringRef & name, const Field & value); void applySettingChange(const SettingChange & change); void applySettingsChanges(const SettingsChanges & changes); From 4af36ee5a473245581e2319699f4d7cb5927d416 Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Fri, 13 Mar 2020 17:50:26 +0300 Subject: [PATCH 0148/1355] Fix updating of the access rights after changing of the `readonly` setting. Remove non-const function Context::getSettingsRef(). --- dbms/programs/client/Client.cpp | 11 +++++++---- dbms/programs/copier/ClusterCopier.cpp | 10 +++++----- dbms/programs/local/LocalServer.cpp | 3 ++- dbms/programs/server/HTTPHandler.cpp | 4 ++-- dbms/programs/server/Server.cpp | 2 +- dbms/programs/server/TCPHandler.cpp | 4 ++-- dbms/src/Core/SettingsCollection.cpp | 4 ++-- .../DataStreams/PushingToViewsBlockOutputStream.cpp | 2 +- dbms/src/Dictionaries/ClickHouseDictionarySource.cpp | 2 +- dbms/src/Interpreters/Context.cpp | 6 +++--- dbms/src/Interpreters/Context.h | 1 - dbms/src/Interpreters/MutationsInterpreter.cpp | 4 ++-- .../tests/in_join_subqueries_preprocessor.cpp | 4 +--- .../Storages/MergeTree/MergeTreeDataMergerMutator.cpp | 4 ++-- dbms/src/Storages/StorageMerge.cpp | 6 +++--- .../integration/test_format_avro_confluent/test.py | 10 ++++++---- 16 files changed, 40 insertions(+), 37 deletions(-) diff --git a/dbms/programs/client/Client.cpp b/dbms/programs/client/Client.cpp index 142f5edc4da..21768911beb 100644 --- a/dbms/programs/client/Client.cpp +++ b/dbms/programs/client/Client.cpp @@ -225,11 +225,11 @@ private: context.setQueryParameters(query_parameters); /// settings and limits could be specified in config file, but passed settings has higher priority - for (auto && setting : context.getSettingsRef()) + for (const auto & setting : context.getSettingsRef()) { const String & name = setting.getName().toString(); if (config().has(name) && !setting.isChanged()) - setting.setValue(config().getString(name)); + context.setSetting(name, config().getString(name)); } /// Set path for format schema files @@ -1736,8 +1736,8 @@ public: ("server_logs_file", po::value(), "put server logs into specified file") ; - context.makeGlobalContext(); - context.getSettingsRef().addProgramOptions(main_description); + Settings cmd_settings; + cmd_settings.addProgramOptions(main_description); /// Commandline options related to external tables. po::options_description external_description = createOptionsDescription("External tables options", terminal_width); @@ -1805,6 +1805,9 @@ public: } } + context.makeGlobalContext(); + context.setSettings(cmd_settings); + /// Copy settings-related program options to config. /// TODO: Is this code necessary? for (const auto & setting : context.getSettingsRef()) diff --git a/dbms/programs/copier/ClusterCopier.cpp b/dbms/programs/copier/ClusterCopier.cpp index 4431362913d..fcb2a69d2a4 100644 --- a/dbms/programs/copier/ClusterCopier.cpp +++ b/dbms/programs/copier/ClusterCopier.cpp @@ -216,7 +216,7 @@ void ClusterCopier::reloadTaskDescription() /// Setup settings task_cluster->reloadSettings(*config); - context.getSettingsRef() = task_cluster->settings_common; + context.setSettings(task_cluster->settings_common); task_cluster_current_config = config; task_descprtion_current_stat = stat; @@ -964,8 +964,8 @@ PartitionTaskStatus ClusterCopier::processPartitionTaskImpl(const ConnectionTime { Context local_context = context; // Use pull (i.e. readonly) settings, but fetch data from destination servers - local_context.getSettingsRef() = task_cluster->settings_pull; - local_context.getSettingsRef().skip_unavailable_shards = true; + local_context.setSettings(task_cluster->settings_pull); + local_context.setSetting("skip_unavailable_shards", true); Block block = getBlockWithAllStreamData(InterpreterFactory::get(query_select_ast, local_context)->execute().in); count = (block) ? block.safeGetByPosition(0).column->getUInt(0) : 0; @@ -1053,10 +1053,10 @@ PartitionTaskStatus ClusterCopier::processPartitionTaskImpl(const ConnectionTime { /// Custom INSERT SELECT implementation Context context_select = context; - context_select.getSettingsRef() = task_cluster->settings_pull; + context_select.setSettings(task_cluster->settings_pull); Context context_insert = context; - context_insert.getSettingsRef() = task_cluster->settings_push; + context_insert.setSettings(task_cluster->settings_push); BlockInputStreamPtr input; BlockOutputStreamPtr output; diff --git a/dbms/programs/local/LocalServer.cpp b/dbms/programs/local/LocalServer.cpp index 2d93c792350..ec2c01924f6 100644 --- a/dbms/programs/local/LocalServer.cpp +++ b/dbms/programs/local/LocalServer.cpp @@ -21,6 +21,7 @@ #include #include #include +#include #include #include #include @@ -92,7 +93,7 @@ void LocalServer::initialize(Poco::Util::Application & self) void LocalServer::applyCmdSettings() { - context->getSettingsRef().copyChangesFrom(cmd_settings); + context->applySettingsChanges(cmd_settings.changes()); } /// If path is specified and not empty, will try to setup server environment and load existing metadata diff --git a/dbms/programs/server/HTTPHandler.cpp b/dbms/programs/server/HTTPHandler.cpp index 0d447a56740..65b605c993d 100644 --- a/dbms/programs/server/HTTPHandler.cpp +++ b/dbms/programs/server/HTTPHandler.cpp @@ -439,13 +439,13 @@ void HTTPHandler::processQuery( /// In theory if initially readonly = 0, the client can change any setting and then set readonly /// to some other value. - auto & settings = context.getSettingsRef(); + const auto & settings = context.getSettingsRef(); /// Only readonly queries are allowed for HTTP GET requests. if (request.getMethod() == Poco::Net::HTTPServerRequest::HTTP_GET) { if (settings.readonly == 0) - settings.readonly = 2; + context.setSetting("readonly", 2); } bool has_external_data = startsWith(request.getContentType(), "multipart/form-data"); diff --git a/dbms/programs/server/Server.cpp b/dbms/programs/server/Server.cpp index 86f65fb09f1..96ba2883480 100644 --- a/dbms/programs/server/Server.cpp +++ b/dbms/programs/server/Server.cpp @@ -527,7 +527,7 @@ int Server::main(const std::vector & /*args*/) /// Load global settings from default_profile and system_profile. global_context->setDefaultProfiles(config()); - Settings & settings = global_context->getSettingsRef(); + const Settings & settings = global_context->getSettingsRef(); /// Size of cache for marks (index of MergeTree family of tables). It is mandatory. size_t mark_cache_size = config().getUInt64("mark_cache_size"); diff --git a/dbms/programs/server/TCPHandler.cpp b/dbms/programs/server/TCPHandler.cpp index a5ecf2963ea..f9df1e4cf9a 100644 --- a/dbms/programs/server/TCPHandler.cpp +++ b/dbms/programs/server/TCPHandler.cpp @@ -950,11 +950,11 @@ void TCPHandler::receiveUnexpectedQuery() readStringBinary(skip_string, *in); - ClientInfo & skip_client_info = query_context->getClientInfo(); + ClientInfo skip_client_info; if (client_revision >= DBMS_MIN_REVISION_WITH_CLIENT_INFO) skip_client_info.read(*in, client_revision); - Settings & skip_settings = query_context->getSettingsRef(); + Settings skip_settings; auto settings_format = (client_revision >= DBMS_MIN_REVISION_WITH_SETTINGS_SERIALIZED_AS_STRINGS) ? SettingsBinaryFormat::STRINGS : SettingsBinaryFormat::OLD; skip_settings.deserialize(*in, settings_format); diff --git a/dbms/src/Core/SettingsCollection.cpp b/dbms/src/Core/SettingsCollection.cpp index d45c082eb0b..6d879b27181 100644 --- a/dbms/src/Core/SettingsCollection.cpp +++ b/dbms/src/Core/SettingsCollection.cpp @@ -165,7 +165,7 @@ void SettingMaxThreads::set(const Field & x) if (x.getType() == Field::Types::String) set(get(x)); else - set(safeGet(x)); + set(applyVisitor(FieldVisitorConvertToNumber(), x)); } void SettingMaxThreads::set(const String & x) @@ -246,7 +246,7 @@ void SettingTimespan::set(const Field & x) if (x.getType() == Field::Types::String) set(get(x)); else - set(safeGet(x)); + set(applyVisitor(FieldVisitorConvertToNumber(), x)); } template diff --git a/dbms/src/DataStreams/PushingToViewsBlockOutputStream.cpp b/dbms/src/DataStreams/PushingToViewsBlockOutputStream.cpp index f6dbf0b6c0b..991d206777a 100644 --- a/dbms/src/DataStreams/PushingToViewsBlockOutputStream.cpp +++ b/dbms/src/DataStreams/PushingToViewsBlockOutputStream.cpp @@ -42,7 +42,7 @@ PushingToViewsBlockOutputStream::PushingToViewsBlockOutputStream( views_context = std::make_unique(context); // Do not deduplicate insertions into MV if the main insertion is Ok if (disable_deduplication_for_children) - views_context->getSettingsRef().insert_deduplicate = false; + views_context->setSetting("insert_deduplicate", false); } for (const auto & database_table : dependencies) diff --git a/dbms/src/Dictionaries/ClickHouseDictionarySource.cpp b/dbms/src/Dictionaries/ClickHouseDictionarySource.cpp index 97ae125abbb..e7f38173d8a 100644 --- a/dbms/src/Dictionaries/ClickHouseDictionarySource.cpp +++ b/dbms/src/Dictionaries/ClickHouseDictionarySource.cpp @@ -74,7 +74,7 @@ ClickHouseDictionarySource::ClickHouseDictionarySource( /// We should set user info even for the case when the dictionary is loaded in-process (without TCP communication). context.setUser(user, password, Poco::Net::SocketAddress("127.0.0.1", 0), {}); /// Processors are not supported here yet. - context.getSettingsRef().experimental_use_processors = false; + context.setSetting("experimental_use_processors", false); /// Query context is needed because some code in executeQuery function may assume it exists. /// Current example is Context::getSampleBlockCache from InterpreterSelectWithUnionQuery::getSampleBlock. context.makeQueryContext(); diff --git a/dbms/src/Interpreters/Context.cpp b/dbms/src/Interpreters/Context.cpp index ef45fd60e81..81fdcae69c8 100644 --- a/dbms/src/Interpreters/Context.cpp +++ b/dbms/src/Interpreters/Context.cpp @@ -936,9 +936,9 @@ Settings Context::getSettings() const void Context::setSettings(const Settings & settings_) { auto lock = getLock(); - bool old_readonly = settings.readonly; - bool old_allow_ddl = settings.allow_ddl; - bool old_allow_introspection_functions = settings.allow_introspection_functions; + auto old_readonly = settings.readonly; + auto old_allow_ddl = settings.allow_ddl; + auto old_allow_introspection_functions = settings.allow_introspection_functions; settings = settings_; diff --git a/dbms/src/Interpreters/Context.h b/dbms/src/Interpreters/Context.h index 2f049d14cd3..f21348a061c 100644 --- a/dbms/src/Interpreters/Context.h +++ b/dbms/src/Interpreters/Context.h @@ -427,7 +427,6 @@ public: } const Settings & getSettingsRef() const { return settings; } - Settings & getSettingsRef() { return settings; } void setProgressCallback(ProgressCallback callback); /// Used in InterpreterSelectQuery to pass it to the IBlockInputStream. diff --git a/dbms/src/Interpreters/MutationsInterpreter.cpp b/dbms/src/Interpreters/MutationsInterpreter.cpp index 3acd04d99f6..056fd5b597a 100644 --- a/dbms/src/Interpreters/MutationsInterpreter.cpp +++ b/dbms/src/Interpreters/MutationsInterpreter.cpp @@ -164,8 +164,8 @@ bool isStorageTouchedByMutations( return true; } - context_copy.getSettingsRef().max_streams_to_max_threads_ratio = 1; - context_copy.getSettingsRef().max_threads = 1; + context_copy.setSetting("max_streams_to_max_threads_ratio", 1); + context_copy.setSetting("max_threads", 1); ASTPtr select_query = prepareQueryAffectedAST(commands); diff --git a/dbms/src/Interpreters/tests/in_join_subqueries_preprocessor.cpp b/dbms/src/Interpreters/tests/in_join_subqueries_preprocessor.cpp index 47dcb406114..9a6d7ca4162 100644 --- a/dbms/src/Interpreters/tests/in_join_subqueries_preprocessor.cpp +++ b/dbms/src/Interpreters/tests/in_join_subqueries_preprocessor.cpp @@ -1170,9 +1170,7 @@ TestResult check(const TestEntry & entry) database->attachTable("visits_all", storage_distributed_visits); database->attachTable("hits_all", storage_distributed_hits); context.setCurrentDatabase("test"); - - auto & settings = context.getSettingsRef(); - settings.distributed_product_mode = entry.mode; + context.setSetting("distributed_product_mode", entry.mode); /// Parse and process the incoming query. DB::ASTPtr ast_input; diff --git a/dbms/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp b/dbms/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp index ac7d5e4b541..7857720f862 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp @@ -958,8 +958,8 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mutatePartToTempor auto storage_from_source_part = StorageFromMergeTreeDataPart::create(source_part); auto context_for_reading = context; - context_for_reading.getSettingsRef().max_streams_to_max_threads_ratio = 1; - context_for_reading.getSettingsRef().max_threads = 1; + context_for_reading.setSetting("max_streams_to_max_threads_ratio", 1); + context_for_reading.setSetting("max_threads", 1); MutationCommands commands_for_part; for (const auto & command : commands) diff --git a/dbms/src/Storages/StorageMerge.cpp b/dbms/src/Storages/StorageMerge.cpp index 46e76a3fcde..e23aa608ec7 100644 --- a/dbms/src/Storages/StorageMerge.cpp +++ b/dbms/src/Storages/StorageMerge.cpp @@ -186,7 +186,7 @@ Pipes StorageMerge::read( * since there is no certainty that it works when one of table is MergeTree and other is not. */ auto modified_context = std::make_shared(context); - modified_context->getSettingsRef().optimize_move_to_prewhere = false; + 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); @@ -300,8 +300,8 @@ Pipes StorageMerge::createSources(const SelectQueryInfo & query_info, const Quer modified_query_info.query->as()->replaceDatabaseAndTable(source_database, table_name); /// Maximum permissible parallelism is streams_num - modified_context->getSettingsRef().max_threads = UInt64(streams_num); - modified_context->getSettingsRef().max_streams_to_max_threads_ratio = 1; + modified_context->setSetting("max_threads", streams_num); + modified_context->setSetting("max_streams_to_max_threads_ratio", 1); InterpreterSelectQuery interpreter{modified_query_info.query, *modified_context, SelectQueryOptions(processed_stage)}; diff --git a/dbms/tests/integration/test_format_avro_confluent/test.py b/dbms/tests/integration/test_format_avro_confluent/test.py index 42ebf05d161..a93b5585f8d 100644 --- a/dbms/tests/integration/test_format_avro_confluent/test.py +++ b/dbms/tests/integration/test_format_avro_confluent/test.py @@ -29,12 +29,14 @@ def cluster(): cluster.shutdown() -def run_query(instance, query, stdin=None, settings=None): +def run_query(instance, query, data=None, settings=None): # type: (ClickHouseInstance, str, object, dict) -> str logging.info("Running query '{}'...".format(query)) # use http to force parsing on server - result = instance.http_query(query, data=stdin, params=settings) + if not data: + data = " " # make POST request + result = instance.http_query(query, data=data, params=settings) logging.info("Query finished") return result @@ -64,7 +66,7 @@ def test_select(cluster): 'test_subject', schema, {'value': x} ) buf.write(message) - stdin = buf.getvalue() + data = buf.getvalue() instance = cluster.instances["dummy"] # type: ClickHouseInstance schema_registry_url = "http://{}:{}".format( @@ -74,7 +76,7 @@ def test_select(cluster): run_query(instance, "create table avro_data(value Int64) engine = Memory()") settings = {'format_avro_schema_registry_url': schema_registry_url} - run_query(instance, "insert into avro_data format AvroConfluent", stdin, settings) + run_query(instance, "insert into avro_data format AvroConfluent", data, settings) stdout = run_query(instance, "select * from avro_data") assert list(map(str.split, stdout.splitlines())) == [ ["0"], From c7a10665d7de21b1f5236993c11f0092e22d3b1e Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Thu, 5 Mar 2020 20:02:11 +0300 Subject: [PATCH 0149/1355] Remove non-partial revoke. --- dbms/src/Access/AccessRights.cpp | 490 ++++++++---------- dbms/src/Access/AccessRights.h | 66 +-- dbms/src/Access/AccessRightsContext.cpp | 10 +- dbms/src/Access/UsersConfigAccessStorage.cpp | 4 +- dbms/src/Core/Settings.h | 1 - .../Interpreters/InterpreterGrantQuery.cpp | 17 +- .../01074_partial_revokes.reference | 3 - .../0_stateless/01074_partial_revokes.sql | 7 - 8 files changed, 231 insertions(+), 367 deletions(-) diff --git a/dbms/src/Access/AccessRights.cpp b/dbms/src/Access/AccessRights.cpp index 80de185ed8f..8b1ef225cec 100644 --- a/dbms/src/Access/AccessRights.cpp +++ b/dbms/src/Access/AccessRights.cpp @@ -23,13 +23,6 @@ namespace COLUMN_LEVEL, }; - enum RevokeMode - { - NORMAL_REVOKE_MODE, /// for AccessRights::revoke() - PARTIAL_REVOKE_MODE, /// for AccessRights::partialRevoke() - FULL_REVOKE_MODE, /// for AccessRights::fullRevoke() - }; - struct Helper { static const Helper & instance() @@ -41,6 +34,7 @@ namespace const AccessFlags database_level_flags = AccessFlags::databaseLevel(); const AccessFlags table_level_flags = AccessFlags::tableLevel(); const AccessFlags column_level_flags = AccessFlags::columnLevel(); + const AccessFlags show_flag = AccessType::SHOW; const AccessFlags exists_flag = AccessType::EXISTS; const AccessFlags create_table_flag = AccessType::CREATE_TABLE; @@ -61,13 +55,10 @@ struct AccessRights::Node public: std::shared_ptr node_name; Level level = GLOBAL_LEVEL; - AccessFlags explicit_grants; - AccessFlags partial_revokes; - AccessFlags inherited_access; /// the access inherited from the parent node - AccessFlags raw_access; /// raw_access = (inherited_access - partial_revokes) | explicit_grants - AccessFlags access; /// access = raw_access | implicit_access - AccessFlags min_access; /// min_access = access & child[0].access & ... | child[N-1].access - AccessFlags max_access; /// max_access = access | child[0].access | ... | child[N-1].access + AccessFlags access; /// access = (inherited_access - partial_revokes) | explicit_grants + AccessFlags final_access; /// final_access = access | implicit_access + AccessFlags min_access; /// min_access = final_access & child[0].final_access & ... & child[N-1].final_access + AccessFlags max_access; /// max_access = final_access | child[0].final_access | ... | child[N-1].final_access std::unique_ptr> children; Node() = default; @@ -80,11 +71,8 @@ public: node_name = src.node_name; level = src.level; - inherited_access = src.inherited_access; - explicit_grants = src.explicit_grants; - partial_revokes = src.partial_revokes; - raw_access = src.raw_access; access = src.access; + final_access = src.final_access; min_access = src.min_access; max_access = src.max_access; if (src.children) @@ -94,9 +82,9 @@ public: return *this; } - void grant(AccessFlags access_to_grant, const Helper & helper) + void grant(AccessFlags flags, const Helper & helper) { - if (!access_to_grant) + if (!flags) return; if (level == GLOBAL_LEVEL) @@ -105,126 +93,77 @@ public: } else if (level == DATABASE_LEVEL) { - AccessFlags grantable = access_to_grant & helper.database_level_flags; + AccessFlags grantable = flags & helper.database_level_flags; if (!grantable) - throw Exception(access_to_grant.toString() + " cannot be granted on the database level", ErrorCodes::INVALID_GRANT); - access_to_grant = grantable; + throw Exception(flags.toString() + " cannot be granted on the database level", ErrorCodes::INVALID_GRANT); + flags = grantable; } else if (level == TABLE_LEVEL) { - AccessFlags grantable = access_to_grant & helper.table_level_flags; + AccessFlags grantable = flags & helper.table_level_flags; if (!grantable) - throw Exception(access_to_grant.toString() + " cannot be granted on the table level", ErrorCodes::INVALID_GRANT); - access_to_grant = grantable; + throw Exception(flags.toString() + " cannot be granted on the table level", ErrorCodes::INVALID_GRANT); + flags = grantable; } else if (level == COLUMN_LEVEL) { - AccessFlags grantable = access_to_grant & helper.column_level_flags; + AccessFlags grantable = flags & helper.column_level_flags; if (!grantable) - throw Exception(access_to_grant.toString() + " cannot be granted on the column level", ErrorCodes::INVALID_GRANT); - access_to_grant = grantable; + throw Exception(flags.toString() + " cannot be granted on the column level", ErrorCodes::INVALID_GRANT); + flags = grantable; } - AccessFlags new_explicit_grants = access_to_grant - partial_revokes; - if (level == TABLE_LEVEL) - removeExplicitGrantsRec(new_explicit_grants); - removePartialRevokesRec(access_to_grant); - explicit_grants |= new_explicit_grants; - - calculateAllAccessRec(helper); + addGrantsRec(flags); + calculateFinalAccessRec(helper); } template - void grant(const AccessFlags & access_to_grant, const Helper & helper, const std::string_view & name, const Args &... subnames) + void grant(const AccessFlags & flags, const Helper & helper, const std::string_view & name, const Args &... subnames) { auto & child = getChild(name); - child.grant(access_to_grant, helper, subnames...); - eraseChildIfEmpty(child); - calculateImplicitAccess(helper); - calculateMinAndMaxAccess(); + child.grant(flags, helper, subnames...); + eraseChildIfPossible(child); + calculateFinalAccess(helper); } template - void grant(const AccessFlags & access_to_grant, const Helper & helper, const std::vector & names) + void grant(const AccessFlags & flags, const Helper & helper, const std::vector & names) { for (const auto & name : names) { auto & child = getChild(name); - child.grant(access_to_grant, helper); - eraseChildIfEmpty(child); + child.grant(flags, helper); + eraseChildIfPossible(child); } - calculateImplicitAccess(helper); - calculateMinAndMaxAccess(); + calculateFinalAccess(helper); } - template - void revoke(const AccessFlags & access_to_revoke, const Helper & helper) + void revoke(const AccessFlags & flags, const Helper & helper) { - if constexpr (mode == NORMAL_REVOKE_MODE) - { // NOLINT - if (level == TABLE_LEVEL) - removeExplicitGrantsRec(access_to_revoke); - else - removeExplicitGrants(access_to_revoke); - } - else if constexpr (mode == PARTIAL_REVOKE_MODE) - { - if (level == TABLE_LEVEL) - removeExplicitGrantsRec(access_to_revoke); - else - removeExplicitGrants(access_to_revoke); - - AccessFlags new_partial_revokes = access_to_revoke - explicit_grants; - removePartialRevokesRec(new_partial_revokes); - partial_revokes |= new_partial_revokes; - } - else /// mode == FULL_REVOKE_MODE - { - AccessFlags new_partial_revokes = access_to_revoke - explicit_grants; - removeExplicitGrantsRec(access_to_revoke); - removePartialRevokesRec(new_partial_revokes); - partial_revokes |= new_partial_revokes; - } - calculateAllAccessRec(helper); + removeGrantsRec(flags); + calculateFinalAccessRec(helper); } - template - void revoke(const AccessFlags & access_to_revoke, const Helper & helper, const std::string_view & name, const Args &... subnames) + template + void revoke(const AccessFlags & flags, const Helper & helper, const std::string_view & name, const Args &... subnames) { - Node * child; - if (mode == NORMAL_REVOKE_MODE) - { - if (!(child = tryGetChild(name))) - return; - } - else - child = &getChild(name); + auto & child = getChild(name); - child->revoke(access_to_revoke, helper, subnames...); - eraseChildIfEmpty(*child); - calculateImplicitAccess(helper); - calculateMinAndMaxAccess(); + child.revoke(flags, helper, subnames...); + eraseChildIfPossible(child); + calculateFinalAccess(helper); } - template - void revoke(const AccessFlags & access_to_revoke, const Helper & helper, const std::vector & names) + template + void revoke(const AccessFlags & flags, const Helper & helper, const std::vector & names) { - Node * child; for (const auto & name : names) { - if (mode == NORMAL_REVOKE_MODE) - { - if (!(child = tryGetChild(name))) - continue; - } - else - child = &getChild(name); - - child->revoke(access_to_revoke, helper); - eraseChildIfEmpty(*child); + auto & child = getChild(name); + child.revoke(flags, helper); + eraseChildIfPossible(child); } - calculateImplicitAccess(helper); - calculateMinAndMaxAccess(); + calculateFinalAccess(helper); } bool isGranted(const AccessFlags & flags) const @@ -244,7 +183,7 @@ public: if (child) return child->isGranted(flags, subnames...); else - return access.contains(flags); + return final_access.contains(flags); } template @@ -265,7 +204,7 @@ public: } else { - if (!access.contains(flags)) + if (!final_access.contains(flags)) return false; } } @@ -274,7 +213,7 @@ public: friend bool operator ==(const Node & left, const Node & right) { - if ((left.explicit_grants != right.explicit_grants) || (left.partial_revokes != right.partial_revokes)) + if (left.access != right.access) return false; if (!left.children) @@ -287,33 +226,24 @@ public: friend bool operator!=(const Node & left, const Node & right) { return !(left == right); } - bool isEmpty() const - { - return !explicit_grants && !partial_revokes && !children; - } - void merge(const Node & other, const Helper & helper) { - mergeRawAccessRec(other); - calculateGrantsAndPartialRevokesRec(); - calculateAllAccessRec(helper); + mergeAccessRec(other); + calculateFinalAccessRec(helper); } - void traceTree(Poco::Logger * log) const + void logTree(Poco::Logger * log) const { LOG_TRACE(log, "Tree(" << level << "): name=" << (node_name ? *node_name : "NULL") - << ", explicit_grants=" << explicit_grants.toString() - << ", partial_revokes=" << partial_revokes.toString() - << ", inherited_access=" << inherited_access.toString() - << ", raw_access=" << raw_access.toString() << ", access=" << access.toString() + << ", final_access=" << final_access.toString() << ", min_access=" << min_access.toString() << ", max_access=" << max_access.toString() << ", num_children=" << (children ? children->size() : 0)); if (children) { for (auto & child : *children | boost::adaptors::map_values) - child.traceTree(log); + child.logTree(log); } } @@ -349,14 +279,13 @@ private: Node & new_child = (*children)[*new_child_name]; new_child.node_name = std::move(new_child_name); new_child.level = static_cast(level + 1); - new_child.inherited_access = raw_access; - new_child.raw_access = raw_access; + new_child.access = access; return new_child; } - void eraseChildIfEmpty(Node & child) + void eraseChildIfPossible(Node & child) { - if (!child.isEmpty()) + if (!canEraseChild(child)) return; auto it = children->find(*child.node_name); children->erase(it); @@ -364,46 +293,59 @@ private: children = nullptr; } - void calculateImplicitAccess(const Helper & helper) + bool canEraseChild(const Node & child) const { - access = raw_access; - if (access & helper.database_level_flags) - access |= helper.show_flag | helper.exists_flag; - else if ((level >= DATABASE_LEVEL) && children) - access |= helper.exists_flag; - - if ((level == GLOBAL_LEVEL) && (access & helper.create_table_flag)) - access |= helper.create_temporary_table_flag; + return (access == child.access) && !child.children; } - void calculateMinAndMaxAccess() + void addGrantsRec(const AccessFlags & flags) { - min_access = access; - max_access = access; + access |= flags; if (children) { - for (const auto & child : *children | boost::adaptors::map_values) + for (auto it = children->begin(); it != children->end();) { - min_access &= child.min_access; - max_access |= child.max_access; + auto & child = it->second; + child.addGrantsRec(flags); + if (canEraseChild(child)) + it = children->erase(it); + else + ++it; } + if (children->empty()) + children = nullptr; } } - void calculateAllAccessRec(const Helper & helper) + void removeGrantsRec(const AccessFlags & flags) { - partial_revokes &= inherited_access; - raw_access = (inherited_access - partial_revokes) | explicit_grants; + access &= ~flags; + if (children) + { + for (auto it = children->begin(); it != children->end();) + { + auto & child = it->second; + child.removeGrantsRec(flags); + if (canEraseChild(child)) + it = children->erase(it); + else + ++it; + } + if (children->empty()) + children = nullptr; + } + } + void calculateFinalAccessRec(const Helper & helper) + { /// Traverse tree. if (children) { for (auto it = children->begin(); it != children->end();) { auto & child = it->second; - child.inherited_access = raw_access; - child.calculateAllAccessRec(helper); - if (child.isEmpty()) + child.calculateFinalAccessRec(helper); + if (canEraseChild(child)) it = children->erase(it); else ++it; @@ -412,64 +354,59 @@ private: children = nullptr; } - calculateImplicitAccess(helper); - calculateMinAndMaxAccess(); + calculateFinalAccess(helper); } - void removeExplicitGrants(const AccessFlags & change) + void calculateFinalAccess(const Helper & helper) { - explicit_grants -= change; - } - - void removeExplicitGrantsRec(const AccessFlags & change) - { - removeExplicitGrants(change); + /// Calculate min and max access among children. + AccessFlags min_access_among_children = AccessType::ALL; + AccessFlags max_access_among_children; if (children) { - for (auto & child : *children | boost::adaptors::map_values) - child.removeExplicitGrantsRec(change); + for (const auto & child : *children | boost::adaptors::map_values) + { + min_access &= child.min_access; + max_access |= child.max_access; + } } + + /// Calculate implicit access: + AccessFlags implicit_access; + if (access & helper.database_level_flags) + implicit_access |= helper.show_flag | helper.exists_flag; + else if ((level >= DATABASE_LEVEL) && children) + implicit_access |= helper.exists_flag; + + if ((level == GLOBAL_LEVEL) && (final_access & helper.create_table_flag)) + implicit_access |= helper.create_temporary_table_flag; + + final_access = access | implicit_access; + + /// Calculate min and max access: + /// min_access = final_access & child[0].final_access & ... & child[N-1].final_access + /// max_access = final_access | child[0].final_access | ... | child[N-1].final_access + min_access = final_access & min_access_among_children; + max_access = final_access | max_access_among_children; } - void removePartialRevokesRec(const AccessFlags & change) - { - partial_revokes -= change; - if (children) - { - for (auto & child : *children | boost::adaptors::map_values) - child.removePartialRevokesRec(change); - } - } - - void mergeRawAccessRec(const Node & rhs) + void mergeAccessRec(const Node & rhs) { if (rhs.children) { for (const auto & [rhs_childname, rhs_child] : *rhs.children) - getChild(rhs_childname).mergeRawAccessRec(rhs_child); + getChild(rhs_childname).mergeAccessRec(rhs_child); } - raw_access |= rhs.raw_access; + access |= rhs.access; if (children) { for (auto & [lhs_childname, lhs_child] : *children) { - lhs_child.inherited_access = raw_access; if (!rhs.tryGetChild(lhs_childname)) - lhs_child.raw_access |= rhs.raw_access; + lhs_child.access |= rhs.access; } } } - - void calculateGrantsAndPartialRevokesRec() - { - explicit_grants = raw_access - inherited_access; - partial_revokes = inherited_access - raw_access; - if (children) - { - for (auto & child : *children | boost::adaptors::map_values) - child.calculateGrantsAndPartialRevokesRec(); - } - } }; @@ -514,165 +451,150 @@ void AccessRights::clear() template -void AccessRights::grantImpl(const AccessFlags & access, const Args &... args) +void AccessRights::grantImpl(const AccessFlags & flags, const Args &... args) { if (!root) root = std::make_unique(); - root->grant(access, Helper::instance(), args...); - if (root->isEmpty()) + root->grant(flags, Helper::instance(), args...); + if (!root->access && !root->children) root = nullptr; } -void AccessRights::grantImpl(const AccessRightsElement & element, std::string_view current_database) +void AccessRights::grant(const AccessFlags & flags) { grantImpl(flags); } +void AccessRights::grant(const AccessFlags & flags, const std::string_view & database) { grantImpl(flags, database); } +void AccessRights::grant(const AccessFlags & flags, const std::string_view & database, const std::string_view & table) { grantImpl(flags, database, table); } +void AccessRights::grant(const AccessFlags & flags, const std::string_view & database, const std::string_view & table, const std::string_view & column) { grantImpl(flags, database, table, column); } +void AccessRights::grant(const AccessFlags & flags, const std::string_view & database, const std::string_view & table, const std::vector & columns) { grantImpl(flags, database, table, columns); } +void AccessRights::grant(const AccessFlags & flags, const std::string_view & database, const std::string_view & table, const Strings & columns) { grantImpl(flags, database, table, columns); } + +void AccessRights::grant(const AccessRightsElement & element, std::string_view current_database) { if (element.any_database) { - grantImpl(element.access_flags); + grant(element.access_flags); } else if (element.any_table) { if (element.database.empty()) - grantImpl(element.access_flags, checkCurrentDatabase(current_database)); + grant(element.access_flags, checkCurrentDatabase(current_database)); else - grantImpl(element.access_flags, element.database); + grant(element.access_flags, element.database); } else if (element.any_column) { if (element.database.empty()) - grantImpl(element.access_flags, checkCurrentDatabase(current_database), element.table); + grant(element.access_flags, checkCurrentDatabase(current_database), element.table); else - grantImpl(element.access_flags, element.database, element.table); + grant(element.access_flags, element.database, element.table); } else { if (element.database.empty()) - grantImpl(element.access_flags, checkCurrentDatabase(current_database), element.table, element.columns); + grant(element.access_flags, checkCurrentDatabase(current_database), element.table, element.columns); else - grantImpl(element.access_flags, element.database, element.table, element.columns); + grant(element.access_flags, element.database, element.table, element.columns); } } -void AccessRights::grantImpl(const AccessRightsElements & elements, std::string_view current_database) +void AccessRights::grant(const AccessRightsElements & elements, std::string_view current_database) { for (const auto & element : elements) - grantImpl(element, current_database); + grant(element, current_database); } -void AccessRights::grant(const AccessFlags & access) { grantImpl(access); } -void AccessRights::grant(const AccessFlags & access, const std::string_view & database) { grantImpl(access, database); } -void AccessRights::grant(const AccessFlags & access, const std::string_view & database, const std::string_view & table) { grantImpl(access, database, table); } -void AccessRights::grant(const AccessFlags & access, const std::string_view & database, const std::string_view & table, const std::string_view & column) { grantImpl(access, database, table, column); } -void AccessRights::grant(const AccessFlags & access, const std::string_view & database, const std::string_view & table, const std::vector & columns) { grantImpl(access, database, table, columns); } -void AccessRights::grant(const AccessFlags & access, const std::string_view & database, const std::string_view & table, const Strings & columns) { grantImpl(access, database, table, columns); } -void AccessRights::grant(const AccessRightsElement & element, std::string_view current_database) { grantImpl(element, current_database); } -void AccessRights::grant(const AccessRightsElements & elements, std::string_view current_database) { grantImpl(elements, current_database); } -template -void AccessRights::revokeImpl(const AccessFlags & access, const Args &... args) +template +void AccessRights::revokeImpl(const AccessFlags & flags, const Args &... args) { if (!root) return; - root->revoke(access, Helper::instance(), args...); - if (root->isEmpty()) + root->revoke(flags, Helper::instance(), args...); + if (!root->access && !root->children) root = nullptr; } -template -void AccessRights::revokeImpl(const AccessRightsElement & element, std::string_view current_database) +void AccessRights::revoke(const AccessFlags & flags) { revokeImpl(flags); } +void AccessRights::revoke(const AccessFlags & flags, const std::string_view & database) { revokeImpl(flags, database); } +void AccessRights::revoke(const AccessFlags & flags, const std::string_view & database, const std::string_view & table) { revokeImpl(flags, database, table); } +void AccessRights::revoke(const AccessFlags & flags, const std::string_view & database, const std::string_view & table, const std::string_view & column) { revokeImpl(flags, database, table, column); } +void AccessRights::revoke(const AccessFlags & flags, const std::string_view & database, const std::string_view & table, const std::vector & columns) { revokeImpl(flags, database, table, columns); } +void AccessRights::revoke(const AccessFlags & flags, const std::string_view & database, const std::string_view & table, const Strings & columns) { revokeImpl(flags, database, table, columns); } + + +void AccessRights::revoke(const AccessRightsElement & element, std::string_view current_database) { if (element.any_database) { - revokeImpl(element.access_flags); + revoke(element.access_flags); } else if (element.any_table) { if (element.database.empty()) - revokeImpl(element.access_flags, checkCurrentDatabase(current_database)); + revoke(element.access_flags, checkCurrentDatabase(current_database)); else - revokeImpl(element.access_flags, element.database); + revoke(element.access_flags, element.database); } else if (element.any_column) { if (element.database.empty()) - revokeImpl(element.access_flags, checkCurrentDatabase(current_database), element.table); + revoke(element.access_flags, checkCurrentDatabase(current_database), element.table); else - revokeImpl(element.access_flags, element.database, element.table); + revoke(element.access_flags, element.database, element.table); } else { if (element.database.empty()) - revokeImpl(element.access_flags, checkCurrentDatabase(current_database), element.table, element.columns); + revoke(element.access_flags, checkCurrentDatabase(current_database), element.table, element.columns); else - revokeImpl(element.access_flags, element.database, element.table, element.columns); + revoke(element.access_flags, element.database, element.table, element.columns); } } -template -void AccessRights::revokeImpl(const AccessRightsElements & elements, std::string_view current_database) +void AccessRights::revoke(const AccessRightsElements & elements, std::string_view current_database) { for (const auto & element : elements) - revokeImpl(element, current_database); + revoke(element, current_database); } -void AccessRights::revoke(const AccessFlags & access) { revokeImpl(access); } -void AccessRights::revoke(const AccessFlags & access, const std::string_view & database) { revokeImpl(access, database); } -void AccessRights::revoke(const AccessFlags & access, const std::string_view & database, const std::string_view & table) { revokeImpl(access, database, table); } -void AccessRights::revoke(const AccessFlags & access, const std::string_view & database, const std::string_view & table, const std::string_view & column) { revokeImpl(access, database, table, column); } -void AccessRights::revoke(const AccessFlags & access, const std::string_view & database, const std::string_view & table, const std::vector & columns) { revokeImpl(access, database, table, columns); } -void AccessRights::revoke(const AccessFlags & access, const std::string_view & database, const std::string_view & table, const Strings & columns) { revokeImpl(access, database, table, columns); } -void AccessRights::revoke(const AccessRightsElement & element, std::string_view current_database) { revokeImpl(element, current_database); } -void AccessRights::revoke(const AccessRightsElements & elements, std::string_view current_database) { revokeImpl(elements, current_database); } - -void AccessRights::partialRevoke(const AccessFlags & access) { revokeImpl(access); } -void AccessRights::partialRevoke(const AccessFlags & access, const std::string_view & database) { revokeImpl(access, database); } -void AccessRights::partialRevoke(const AccessFlags & access, const std::string_view & database, const std::string_view & table) { revokeImpl(access, database, table); } -void AccessRights::partialRevoke(const AccessFlags & access, const std::string_view & database, const std::string_view & table, const std::string_view & column) { revokeImpl(access, database, table, column); } -void AccessRights::partialRevoke(const AccessFlags & access, const std::string_view & database, const std::string_view & table, const std::vector & columns) { revokeImpl(access, database, table, columns); } -void AccessRights::partialRevoke(const AccessFlags & access, const std::string_view & database, const std::string_view & table, const Strings & columns) { revokeImpl(access, database, table, columns); } -void AccessRights::partialRevoke(const AccessRightsElement & element, std::string_view current_database) { revokeImpl(element, current_database); } -void AccessRights::partialRevoke(const AccessRightsElements & elements, std::string_view current_database) { revokeImpl(elements, current_database); } - -void AccessRights::fullRevoke(const AccessFlags & access) { revokeImpl(access); } -void AccessRights::fullRevoke(const AccessFlags & access, const std::string_view & database) { revokeImpl(access, database); } -void AccessRights::fullRevoke(const AccessFlags & access, const std::string_view & database, const std::string_view & table) { revokeImpl(access, database, table); } -void AccessRights::fullRevoke(const AccessFlags & access, const std::string_view & database, const std::string_view & table, const std::string_view & column) { revokeImpl(access, database, table, column); } -void AccessRights::fullRevoke(const AccessFlags & access, const std::string_view & database, const std::string_view & table, const std::vector & columns) { revokeImpl(access, database, table, columns); } -void AccessRights::fullRevoke(const AccessFlags & access, const std::string_view & database, const std::string_view & table, const Strings & columns) { revokeImpl(access, database, table, columns); } -void AccessRights::fullRevoke(const AccessRightsElement & element, std::string_view current_database) { revokeImpl(element, current_database); } -void AccessRights::fullRevoke(const AccessRightsElements & elements, std::string_view current_database) { revokeImpl(elements, current_database); } - AccessRights::Elements AccessRights::getElements() const { if (!root) return {}; Elements res; - if (root->explicit_grants) - res.grants.push_back({root->explicit_grants}); + auto global_access = root->access; + if (global_access) + res.grants.push_back({global_access}); if (root->children) { for (const auto & [db_name, db_node] : *root->children) { - if (db_node.partial_revokes) - res.partial_revokes.push_back({db_node.partial_revokes, db_name}); - if (db_node.explicit_grants) - res.grants.push_back({db_node.explicit_grants, db_name}); + auto db_grants = db_node.access - global_access; + auto db_partial_revokes = global_access - db_node.access; + if (db_partial_revokes) + res.partial_revokes.push_back({db_partial_revokes, db_name}); + if (db_grants) + res.grants.push_back({db_grants, db_name}); if (db_node.children) { for (const auto & [table_name, table_node] : *db_node.children) { - if (table_node.partial_revokes) - res.partial_revokes.push_back({table_node.partial_revokes, db_name, table_name}); - if (table_node.explicit_grants) - res.grants.push_back({table_node.explicit_grants, db_name, table_name}); + auto table_grants = table_node.access - db_node.access; + auto table_partial_revokes = db_node.access - table_node.access; + if (table_partial_revokes) + res.partial_revokes.push_back({table_partial_revokes, db_name, table_name}); + if (table_grants) + res.grants.push_back({table_grants, db_name, table_name}); if (table_node.children) { for (const auto & [column_name, column_node] : *table_node.children) { - if (column_node.partial_revokes) - res.partial_revokes.push_back({column_node.partial_revokes, db_name, table_name, column_name}); - if (column_node.explicit_grants) - res.grants.push_back({column_node.explicit_grants, db_name, table_name, column_name}); + auto column_grants = column_node.access - table_node.access; + auto column_partial_revokes = table_node.access - column_node.access; + if (column_partial_revokes) + res.partial_revokes.push_back({column_partial_revokes, db_name, table_name, column_name}); + if (column_grants) + res.grants.push_back({column_grants, db_name, table_name, column_name}); } } } @@ -706,59 +628,57 @@ String AccessRights::toString() const template -bool AccessRights::isGrantedImpl(const AccessFlags & access, const Args &... args) const +bool AccessRights::isGrantedImpl(const AccessFlags & flags, const Args &... args) const { if (!root) - return access.isEmpty(); - return root->isGranted(access, args...); + return flags.isEmpty(); + return root->isGranted(flags, args...); } -bool AccessRights::isGrantedImpl(const AccessRightsElement & element, std::string_view current_database) const +bool AccessRights::isGranted(const AccessFlags & flags) const { return isGrantedImpl(flags); } +bool AccessRights::isGranted(const AccessFlags & flags, const std::string_view & database) const { return isGrantedImpl(flags, database); } +bool AccessRights::isGranted(const AccessFlags & flags, const std::string_view & database, const std::string_view & table) const { return isGrantedImpl(flags, database, table); } +bool AccessRights::isGranted(const AccessFlags & flags, const std::string_view & database, const std::string_view & table, const std::string_view & column) const { return isGrantedImpl(flags, database, table, column); } +bool AccessRights::isGranted(const AccessFlags & flags, const std::string_view & database, const std::string_view & table, const std::vector & columns) const { return isGrantedImpl(flags, database, table, columns); } +bool AccessRights::isGranted(const AccessFlags & flags, const std::string_view & database, const std::string_view & table, const Strings & columns) const { return isGrantedImpl(flags, database, table, columns); } + +bool AccessRights::isGranted(const AccessRightsElement & element, std::string_view current_database) const { if (element.any_database) { - return isGrantedImpl(element.access_flags); + return isGranted(element.access_flags); } else if (element.any_table) { if (element.database.empty()) - return isGrantedImpl(element.access_flags, checkCurrentDatabase(current_database)); + return isGranted(element.access_flags, checkCurrentDatabase(current_database)); else - return isGrantedImpl(element.access_flags, element.database); + return isGranted(element.access_flags, element.database); } else if (element.any_column) { if (element.database.empty()) - return isGrantedImpl(element.access_flags, checkCurrentDatabase(current_database), element.table); + return isGranted(element.access_flags, checkCurrentDatabase(current_database), element.table); else - return isGrantedImpl(element.access_flags, element.database, element.table); + return isGranted(element.access_flags, element.database, element.table); } else { if (element.database.empty()) - return isGrantedImpl(element.access_flags, checkCurrentDatabase(current_database), element.table, element.columns); + return isGranted(element.access_flags, checkCurrentDatabase(current_database), element.table, element.columns); else - return isGrantedImpl(element.access_flags, element.database, element.table, element.columns); + return isGranted(element.access_flags, element.database, element.table, element.columns); } } -bool AccessRights::isGrantedImpl(const AccessRightsElements & elements, std::string_view current_database) const +bool AccessRights::isGranted(const AccessRightsElements & elements, std::string_view current_database) const { for (const auto & element : elements) - if (!isGrantedImpl(element, current_database)) + if (!isGranted(element, current_database)) return false; return true; } -bool AccessRights::isGranted(const AccessFlags & access) const { return isGrantedImpl(access); } -bool AccessRights::isGranted(const AccessFlags & access, const std::string_view & database) const { return isGrantedImpl(access, database); } -bool AccessRights::isGranted(const AccessFlags & access, const std::string_view & database, const std::string_view & table) const { return isGrantedImpl(access, database, table); } -bool AccessRights::isGranted(const AccessFlags & access, const std::string_view & database, const std::string_view & table, const std::string_view & column) const { return isGrantedImpl(access, database, table, column); } -bool AccessRights::isGranted(const AccessFlags & access, const std::string_view & database, const std::string_view & table, const std::vector & columns) const { return isGrantedImpl(access, database, table, columns); } -bool AccessRights::isGranted(const AccessFlags & access, const std::string_view & database, const std::string_view & table, const Strings & columns) const { return isGrantedImpl(access, database, table, columns); } -bool AccessRights::isGranted(const AccessRightsElement & element, std::string_view current_database) const { return isGrantedImpl(element, current_database); } -bool AccessRights::isGranted(const AccessRightsElements & elements, std::string_view current_database) const { return isGrantedImpl(elements, current_database); } - bool operator ==(const AccessRights & left, const AccessRights & right) { @@ -780,17 +700,17 @@ void AccessRights::merge(const AccessRights & other) if (other.root) { root->merge(*other.root, Helper::instance()); - if (root->isEmpty()) + if (!root->access && !root->children) root = nullptr; } } -void AccessRights::traceTree() const +void AccessRights::logTree() const { auto * log = &Poco::Logger::get("AccessRights"); if (root) - root->traceTree(log); + root->logTree(log); else LOG_TRACE(log, "Tree: NULL"); } diff --git a/dbms/src/Access/AccessRights.h b/dbms/src/Access/AccessRights.h index 67d205ec6dc..48ffa0bb616 100644 --- a/dbms/src/Access/AccessRights.h +++ b/dbms/src/Access/AccessRights.h @@ -23,7 +23,7 @@ public: bool isEmpty() const; - /// Revokes everything. It's the same as fullRevoke(AccessType::ALL). + /// Revokes everything. It's the same as revoke(AccessType::ALL). void clear(); /// Grants access on a specified database/table/column. @@ -38,10 +38,7 @@ public: void grant(const AccessRightsElements & elements, std::string_view current_database = {}); /// Revokes a specified access granted earlier on a specified database/table/column. - /// Does nothing if the specified access is not granted. - /// If the specified access is granted but on upper level (e.g. database for table, table for columns) - /// or lower level, the function also does nothing. - /// This function implements the standard SQL REVOKE behaviour. + /// For example, revoke(AccessType::ALL) revokes all grants at all, just like clear(); void revoke(const AccessFlags & access); void revoke(const AccessFlags & access, const std::string_view & database); void revoke(const AccessFlags & access, const std::string_view & database, const std::string_view & table); @@ -51,32 +48,6 @@ public: void revoke(const AccessRightsElement & element, std::string_view current_database = {}); void revoke(const AccessRightsElements & elements, std::string_view current_database = {}); - /// Revokes a specified access granted earlier on a specified database/table/column or on lower levels. - /// The function also restricts access if it's granted on upper level. - /// For example, an access could be granted on a database and then revoked on a table in this database. - /// This function implements the MySQL REVOKE behaviour with partial_revokes is ON. - void partialRevoke(const AccessFlags & access); - void partialRevoke(const AccessFlags & access, const std::string_view & database); - void partialRevoke(const AccessFlags & access, const std::string_view & database, const std::string_view & table); - void partialRevoke(const AccessFlags & access, const std::string_view & database, const std::string_view & table, const std::string_view & column); - void partialRevoke(const AccessFlags & access, const std::string_view & database, const std::string_view & table, const std::vector & columns); - void partialRevoke(const AccessFlags & access, const std::string_view & database, const std::string_view & table, const Strings & columns); - void partialRevoke(const AccessRightsElement & element, std::string_view current_database = {}); - void partialRevoke(const AccessRightsElements & elements, std::string_view current_database = {}); - - /// Revokes a specified access granted earlier on a specified database/table/column or on lower levels. - /// The function also restricts access if it's granted on upper level. - /// For example, fullRevoke(AccessType::ALL) revokes all grants at all, just like clear(); - /// fullRevoke(AccessType::SELECT, db) means it's not allowed to execute SELECT in that database anymore (from any table). - void fullRevoke(const AccessFlags & access); - void fullRevoke(const AccessFlags & access, const std::string_view & database); - void fullRevoke(const AccessFlags & access, const std::string_view & database, const std::string_view & table); - void fullRevoke(const AccessFlags & access, const std::string_view & database, const std::string_view & table, const std::string_view & column); - void fullRevoke(const AccessFlags & access, const std::string_view & database, const std::string_view & table, const std::vector & columns); - void fullRevoke(const AccessFlags & access, const std::string_view & database, const std::string_view & table, const Strings & columns); - void fullRevoke(const AccessRightsElement & element, std::string_view current_database = {}); - void fullRevoke(const AccessRightsElements & elements, std::string_view current_database = {}); - /// Returns the information about all the access granted. struct Elements { @@ -89,12 +60,12 @@ public: String toString() const; /// Whether a specified access granted. - bool isGranted(const AccessFlags & access) const; - bool isGranted(const AccessFlags & access, const std::string_view & database) const; - bool isGranted(const AccessFlags & access, const std::string_view & database, const std::string_view & table) const; - bool isGranted(const AccessFlags & access, const std::string_view & database, const std::string_view & table, const std::string_view & column) const; - bool isGranted(const AccessFlags & access, const std::string_view & database, const std::string_view & table, const std::vector & columns) const; - bool isGranted(const AccessFlags & access, const std::string_view & database, const std::string_view & table, const Strings & columns) const; + bool isGranted(const AccessFlags & flags) const; + bool isGranted(const AccessFlags & flags, const std::string_view & database) const; + bool isGranted(const AccessFlags & flags, const std::string_view & database, const std::string_view & table) const; + bool isGranted(const AccessFlags & flags, const std::string_view & database, const std::string_view & table, const std::string_view & column) const; + bool isGranted(const AccessFlags & flags, const std::string_view & database, const std::string_view & table, const std::vector & columns) const; + bool isGranted(const AccessFlags & flags, const std::string_view & database, const std::string_view & table, const Strings & columns) const; bool isGranted(const AccessRightsElement & element, std::string_view current_database = {}) const; bool isGranted(const AccessRightsElements & elements, std::string_view current_database = {}) const; @@ -107,22 +78,13 @@ public: private: template - void grantImpl(const AccessFlags & access, const Args &... args); - - void grantImpl(const AccessRightsElement & element, std::string_view current_database); - void grantImpl(const AccessRightsElements & elements, std::string_view current_database); - - template - void revokeImpl(const AccessFlags & access, const Args &... args); - - template - void revokeImpl(const AccessRightsElement & element, std::string_view current_database); - - template - void revokeImpl(const AccessRightsElements & elements, std::string_view current_database); + void grantImpl(const AccessFlags & flags, const Args &... args); template - bool isGrantedImpl(const AccessFlags & access, const Args &... args) const; + void revokeImpl(const AccessFlags & flags, const Args &... args); + + template + bool isGrantedImpl(const AccessFlags & flags, const Args &... args) const; bool isGrantedImpl(const AccessRightsElement & element, std::string_view current_database) const; bool isGrantedImpl(const AccessRightsElements & elements, std::string_view current_database) const; @@ -130,7 +92,7 @@ private: template AccessFlags getAccessImpl(const Args &... args) const; - void traceTree() const; + void logTree() const; struct Node; std::unique_ptr root; diff --git a/dbms/src/Access/AccessRightsContext.cpp b/dbms/src/Access/AccessRightsContext.cpp index 9e781cbe280..bfc9c386ef4 100644 --- a/dbms/src/Access/AccessRightsContext.cpp +++ b/dbms/src/Access/AccessRightsContext.cpp @@ -444,19 +444,19 @@ boost::shared_ptr AccessRightsContext::calculateResultAccess result.grant(AccessType::SELECT, DatabaseCatalog::TEMPORARY_DATABASE); if (readonly_) - result.fullRevoke(write_table_access | all_dcl | AccessType::SYSTEM | AccessType::KILL); + result.revoke(write_table_access | all_dcl | AccessType::SYSTEM | AccessType::KILL); if (readonly_ || !allow_ddl_) - result.fullRevoke(table_and_dictionary_ddl); + result.revoke(table_and_dictionary_ddl); if (readonly_ && grant_option) - result.fullRevoke(AccessType::ALL); + result.revoke(AccessType::ALL); if (readonly_ == 1) { /// Table functions are forbidden in readonly mode. /// For example, for readonly = 2 - allowed. - result.fullRevoke(AccessType::CREATE_TEMPORARY_TABLE | AccessType::TABLE_FUNCTIONS); + result.revoke(AccessType::CREATE_TEMPORARY_TABLE | AccessType::TABLE_FUNCTIONS); } else if (readonly_ == 2) { @@ -465,7 +465,7 @@ boost::shared_ptr AccessRightsContext::calculateResultAccess } if (!allow_introspection_) - result.fullRevoke(AccessType::INTROSPECTION); + result.revoke(AccessType::INTROSPECTION); result_access_cache[cache_index].store(result_ptr); diff --git a/dbms/src/Access/UsersConfigAccessStorage.cpp b/dbms/src/Access/UsersConfigAccessStorage.cpp index 20ee2a628a6..ecb681ad7e0 100644 --- a/dbms/src/Access/UsersConfigAccessStorage.cpp +++ b/dbms/src/Access/UsersConfigAccessStorage.cpp @@ -141,14 +141,14 @@ namespace if (databases) { - user->access.fullRevoke(AccessFlags::databaseLevel()); + user->access.revoke(AccessFlags::databaseLevel()); for (const String & database : *databases) user->access.grant(AccessFlags::databaseLevel(), database); } if (dictionaries) { - user->access.fullRevoke(AccessType::dictGet, IDictionary::NO_DATABASE_TAG); + user->access.revoke(AccessType::dictGet, IDictionary::NO_DATABASE_TAG); for (const String & dictionary : *dictionaries) user->access.grant(AccessType::dictGet, IDictionary::NO_DATABASE_TAG, dictionary); } diff --git a/dbms/src/Core/Settings.h b/dbms/src/Core/Settings.h index a3b94ad035b..29f13aa968e 100644 --- a/dbms/src/Core/Settings.h +++ b/dbms/src/Core/Settings.h @@ -395,7 +395,6 @@ struct Settings : public SettingsCollection M(SettingBool, allow_experimental_alter_materialized_view_structure, false, "Allow atomic alter on Materialized views. Work in progress.", 0) \ M(SettingBool, enable_early_constant_folding, true, "Enable query optimization where we analyze function and subqueries results and rewrite query if there're constants there", 0) \ \ - M(SettingBool, partial_revokes, false, "Makes it possible to revoke privileges partially.", 0) \ M(SettingBool, deduplicate_blocks_in_dependent_materialized_views, false, "Should deduplicate blocks for materialized views if the block is not a duplicate for the table. Use true to always deduplicate in dependent tables.", 0) \ M(SettingBool, use_compact_format_in_distributed_parts_names, false, "Changes format of directories names for distributed table insert parts.", 0) \ \ diff --git a/dbms/src/Interpreters/InterpreterGrantQuery.cpp b/dbms/src/Interpreters/InterpreterGrantQuery.cpp index 6d1b2262637..fd38b1a8788 100644 --- a/dbms/src/Interpreters/InterpreterGrantQuery.cpp +++ b/dbms/src/Interpreters/InterpreterGrantQuery.cpp @@ -14,7 +14,7 @@ namespace DB namespace { template - void updateFromQueryImpl(T & grantee, const ASTGrantQuery & query, const std::vector & roles_from_query, const String & current_database, bool partial_revokes) + void updateFromQueryImpl(T & grantee, const ASTGrantQuery & query, const std::vector & roles_from_query, const String & current_database) { using Kind = ASTGrantQuery::Kind; if (!query.access_rights_elements.empty()) @@ -25,12 +25,6 @@ namespace if (query.grant_option) grantee.access_with_grant_option.grant(query.access_rights_elements, current_database); } - else if (partial_revokes) - { - grantee.access_with_grant_option.partialRevoke(query.access_rights_elements, current_database); - if (!query.grant_option) - grantee.access.partialRevoke(query.access_rights_elements, current_database); - } else { grantee.access_with_grant_option.revoke(query.access_rights_elements, current_database); @@ -79,19 +73,18 @@ BlockIO InterpreterGrantQuery::execute() std::vector to_roles = GenericRoleSet{*query.to_roles, access_control, context.getUserID()}.getMatchingUsersAndRoles(access_control); String current_database = context.getCurrentDatabase(); - bool partial_revokes = context.getSettingsRef().partial_revokes; auto update_func = [&](const AccessEntityPtr & entity) -> AccessEntityPtr { auto clone = entity->clone(); if (auto user = typeid_cast>(clone)) { - updateFromQueryImpl(*user, query, roles_from_query, current_database, partial_revokes); + updateFromQueryImpl(*user, query, roles_from_query, current_database); return user; } else if (auto role = typeid_cast>(clone)) { - updateFromQueryImpl(*role, query, roles_from_query, current_database, partial_revokes); + updateFromQueryImpl(*role, query, roles_from_query, current_database); return role; } else @@ -109,7 +102,7 @@ void InterpreterGrantQuery::updateUserFromQuery(User & user, const ASTGrantQuery std::vector roles_from_query; if (query.roles) roles_from_query = GenericRoleSet{*query.roles}.getMatchingIDs(); - updateFromQueryImpl(user, query, roles_from_query, {}, true); + updateFromQueryImpl(user, query, roles_from_query, {}); } @@ -118,7 +111,7 @@ void InterpreterGrantQuery::updateRoleFromQuery(Role & role, const ASTGrantQuery std::vector roles_from_query; if (query.roles) roles_from_query = GenericRoleSet{*query.roles}.getMatchingIDs(); - updateFromQueryImpl(role, query, roles_from_query, {}, true); + updateFromQueryImpl(role, query, roles_from_query, {}); } } diff --git a/dbms/tests/queries/0_stateless/01074_partial_revokes.reference b/dbms/tests/queries/0_stateless/01074_partial_revokes.reference index e64d439b5b2..19a70679143 100644 --- a/dbms/tests/queries/0_stateless/01074_partial_revokes.reference +++ b/dbms/tests/queries/0_stateless/01074_partial_revokes.reference @@ -1,5 +1,2 @@ -A -GRANT SELECT ON *.* TO test_user_01074 -B GRANT SELECT ON *.* TO test_user_01074 REVOKE SELECT ON db.* FROM test_user_01074 diff --git a/dbms/tests/queries/0_stateless/01074_partial_revokes.sql b/dbms/tests/queries/0_stateless/01074_partial_revokes.sql index af7048a0815..4406341cc4f 100644 --- a/dbms/tests/queries/0_stateless/01074_partial_revokes.sql +++ b/dbms/tests/queries/0_stateless/01074_partial_revokes.sql @@ -1,15 +1,8 @@ DROP USER IF EXISTS test_user_01074; CREATE USER test_user_01074; -SELECT 'A'; -SET partial_revokes=0; GRANT SELECT ON *.* TO test_user_01074; REVOKE SELECT ON db.* FROM test_user_01074; SHOW GRANTS FOR test_user_01074; -SELECT 'B'; -SET partial_revokes=1; -REVOKE SELECT ON db.* FROM test_user_01074; -SHOW GRANTS FOR test_user_01074; - DROP USER test_user_01074; From 66172cc2eb7562128f993131b2af0ada71387d5e Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Thu, 5 Mar 2020 20:32:17 +0300 Subject: [PATCH 0150/1355] Improve access rights: CREATE TABLE now implicitly enables CREATE VIEW, the same thing for DROP TABLE, ALTER TABLE, TRUNCATE_TABLE. --- dbms/src/Access/AccessRights.cpp | 22 ++++++++++++++++++++++ 1 file changed, 22 insertions(+) diff --git a/dbms/src/Access/AccessRights.cpp b/dbms/src/Access/AccessRights.cpp index 8b1ef225cec..ae5c043a75f 100644 --- a/dbms/src/Access/AccessRights.cpp +++ b/dbms/src/Access/AccessRights.cpp @@ -38,7 +38,14 @@ namespace const AccessFlags show_flag = AccessType::SHOW; const AccessFlags exists_flag = AccessType::EXISTS; const AccessFlags create_table_flag = AccessType::CREATE_TABLE; + const AccessFlags create_view_flag = AccessType::CREATE_VIEW; const AccessFlags create_temporary_table_flag = AccessType::CREATE_TEMPORARY_TABLE; + const AccessFlags alter_table_flag = AccessType::ALTER_TABLE; + const AccessFlags alter_view_flag = AccessType::ALTER_VIEW; + const AccessFlags truncate_table_flag = AccessType::TRUNCATE_TABLE; + const AccessFlags truncate_view_flag = AccessType::TRUNCATE_VIEW; + const AccessFlags drop_table_flag = AccessType::DROP_TABLE; + const AccessFlags drop_view_flag = AccessType::DROP_VIEW; }; std::string_view checkCurrentDatabase(const std::string_view & current_database) @@ -381,6 +388,21 @@ private: if ((level == GLOBAL_LEVEL) && (final_access & helper.create_table_flag)) implicit_access |= helper.create_temporary_table_flag; + if (level <= TABLE_LEVEL) + { + if (access & helper.create_table_flag) + implicit_access |= helper.create_view_flag; + + if (access & helper.drop_table_flag) + implicit_access |= helper.drop_view_flag; + + if (access & helper.alter_table_flag) + implicit_access |= helper.alter_view_flag; + + if (access & helper.truncate_table_flag) + implicit_access |= helper.truncate_view_flag; + } + final_access = access | implicit_access; /// Calculate min and max access: From c1f5f8bc89f606e4c65f1e5ae7dbb8a014dd4513 Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Fri, 6 Mar 2020 17:36:01 +0300 Subject: [PATCH 0151/1355] Improve access rights: SHOW splitted into SHOW_DATABASES, SHOW_TABLES, SHOW_COLUMNS, SHOW_DICTIONARIES, EXISTS deleted. --- dbms/src/Access/AccessFlags.h | 284 ++++++++++-------- dbms/src/Access/AccessRights.cpp | 54 +++- dbms/src/Access/AccessRightsContext.cpp | 31 +- dbms/src/Access/AccessType.h | 17 +- dbms/src/Access/UsersConfigAccessStorage.cpp | 11 +- .../Interpreters/InterpreterCheckQuery.cpp | 2 +- .../Interpreters/InterpreterDescribeQuery.cpp | 3 +- .../Interpreters/InterpreterExistsQuery.cpp | 5 +- .../InterpreterShowCreateQuery.cpp | 6 +- dbms/src/Interpreters/InterpreterUseQuery.cpp | 2 +- .../Storages/System/StorageSystemColumns.cpp | 6 +- .../System/StorageSystemDatabases.cpp | 4 +- .../System/StorageSystemDictionaries.cpp | 4 +- .../Storages/System/StorageSystemMerges.cpp | 4 +- .../System/StorageSystemMutations.cpp | 6 +- .../System/StorageSystemPartsBase.cpp | 4 +- .../Storages/System/StorageSystemReplicas.cpp | 6 +- .../System/StorageSystemReplicationQueue.cpp | 6 +- .../Storages/System/StorageSystemTables.cpp | 6 +- 19 files changed, 255 insertions(+), 206 deletions(-) diff --git a/dbms/src/Access/AccessFlags.h b/dbms/src/Access/AccessFlags.h index 5af804ddc48..2cf59b4886d 100644 --- a/dbms/src/Access/AccessFlags.h +++ b/dbms/src/Access/AccessFlags.h @@ -63,15 +63,24 @@ public: /// Returns a list of keywords. std::vector toKeywords() const; - /// Returns the access types which could be granted on the database level. - /// For example, SELECT can be granted on the database level, but CREATE_USER cannot. - static AccessFlags databaseLevel(); + /// Returns all the flags. + /// These are the same as (allGlobalFlags() | allDatabaseFlags() | allTableFlags() | allColumnsFlags() | allDictionaryFlags()). + static AccessFlags allFlags(); - /// Returns the access types which could be granted on the table/dictionary level. - static AccessFlags tableLevel(); + /// Returns all the global flags. + static AccessFlags allGlobalFlags(); - /// Returns the access types which could be granted on the column/attribute level. - static AccessFlags columnLevel(); + /// Returns all the flags related to a database. + static AccessFlags allDatabaseFlags(); + + /// Returns all the flags related to a table. + static AccessFlags allTableFlags(); + + /// Returns all the flags related to a column. + static AccessFlags allColumnFlags(); + + /// Returns all the flags related to a dictionary. + static AccessFlags allDictionaryFlags(); private: static constexpr size_t NUM_FLAGS = 128; @@ -158,22 +167,27 @@ public: return str; } - const Flags & getDatabaseLevelFlags() const { return all_grantable_on_level[DATABASE_LEVEL]; } - const Flags & getTableLevelFlags() const { return all_grantable_on_level[TABLE_LEVEL]; } - const Flags & getColumnLevelFlags() const { return all_grantable_on_level[COLUMN_LEVEL]; } + const Flags & getAllFlags() const { return all_flags; } + const Flags & getGlobalFlags() const { return all_flags_for_target[GLOBAL]; } + const Flags & getDatabaseFlags() const { return all_flags_for_target[DATABASE]; } + const Flags & getTableFlags() const { return all_flags_for_target[TABLE]; } + const Flags & getColumnFlags() const { return all_flags_for_target[COLUMN]; } + const Flags & getDictionaryFlags() const { return all_flags_for_target[DICTIONARY]; } private: - enum Level + enum Target { - UNKNOWN_LEVEL = -1, - GLOBAL_LEVEL = 0, - DATABASE_LEVEL = 1, - TABLE_LEVEL = 2, - VIEW_LEVEL = 2, - DICTIONARY_LEVEL = 2, - COLUMN_LEVEL = 3, + UNKNOWN_TARGET, + GLOBAL, + DATABASE, + TABLE, + VIEW = TABLE, + COLUMN, + DICTIONARY, }; + static constexpr size_t NUM_TARGETS = static_cast(DICTIONARY) + 1; + struct Node; using NodePtr = std::unique_ptr; using Nodes = std::vector; @@ -191,11 +205,11 @@ private: std::string_view keyword; std::vector aliases; Flags flags; - Level level = UNKNOWN_LEVEL; + Target target = UNKNOWN_TARGET; Nodes children; - Node(std::string_view keyword_, size_t flag_, Level level_) - : keyword(keyword_), level(level_) + Node(std::string_view keyword_, size_t flag_, Target target_) + : keyword(keyword_), target(target_) { flags.set(flag_); } @@ -229,216 +243,220 @@ private: } } - static void makeFlagsToKeywordTree(NodePtr & flags_to_keyword_tree_) + static NodePtr makeFlagsToKeywordTree() { size_t next_flag = 0; Nodes all; - auto show = std::make_unique("SHOW", next_flag++, COLUMN_LEVEL); - auto exists = std::make_unique("EXISTS", next_flag++, COLUMN_LEVEL); - ext::push_back(all, std::move(show), std::move(exists)); + auto show_databases = std::make_unique("SHOW DATABASES", next_flag++, DATABASE); + auto show_tables = std::make_unique("SHOW TABLES", next_flag++, TABLE); + auto show_columns = std::make_unique("SHOW COLUMNS", next_flag++, COLUMN); + auto show_dictionaries = std::make_unique("SHOW DICTIONARIES", next_flag++, DICTIONARY); + auto show = std::make_unique("SHOW", std::move(show_databases), std::move(show_tables), std::move(show_columns), std::move(show_dictionaries)); + ext::push_back(all, std::move(show)); - auto select = std::make_unique("SELECT", next_flag++, COLUMN_LEVEL); - auto insert = std::make_unique("INSERT", next_flag++, COLUMN_LEVEL); + auto select = std::make_unique("SELECT", next_flag++, COLUMN); + auto insert = std::make_unique("INSERT", next_flag++, COLUMN); ext::push_back(all, std::move(select), std::move(insert)); - auto update = std::make_unique("UPDATE", next_flag++, COLUMN_LEVEL); + auto update = std::make_unique("UPDATE", next_flag++, COLUMN); ext::push_back(update->aliases, "ALTER UPDATE"); - auto delet = std::make_unique("DELETE", next_flag++, TABLE_LEVEL); + auto delet = std::make_unique("DELETE", next_flag++, TABLE); ext::push_back(delet->aliases, "ALTER DELETE"); - auto add_column = std::make_unique("ADD COLUMN", next_flag++, COLUMN_LEVEL); + auto add_column = std::make_unique("ADD COLUMN", next_flag++, COLUMN); add_column->aliases.push_back("ALTER ADD COLUMN"); - auto modify_column = std::make_unique("MODIFY COLUMN", next_flag++, COLUMN_LEVEL); + auto modify_column = std::make_unique("MODIFY COLUMN", next_flag++, COLUMN); modify_column->aliases.push_back("ALTER MODIFY COLUMN"); - auto drop_column = std::make_unique("DROP COLUMN", next_flag++, COLUMN_LEVEL); + auto drop_column = std::make_unique("DROP COLUMN", next_flag++, COLUMN); drop_column->aliases.push_back("ALTER DROP COLUMN"); - auto comment_column = std::make_unique("COMMENT COLUMN", next_flag++, COLUMN_LEVEL); + auto comment_column = std::make_unique("COMMENT COLUMN", next_flag++, COLUMN); comment_column->aliases.push_back("ALTER COMMENT COLUMN"); - auto clear_column = std::make_unique("CLEAR COLUMN", next_flag++, COLUMN_LEVEL); + auto clear_column = std::make_unique("CLEAR COLUMN", next_flag++, COLUMN); clear_column->aliases.push_back("ALTER CLEAR COLUMN"); auto alter_column = std::make_unique("ALTER COLUMN", std::move(add_column), std::move(modify_column), std::move(drop_column), std::move(comment_column), std::move(clear_column)); - auto alter_order_by = std::make_unique("ALTER ORDER BY", next_flag++, TABLE_LEVEL); + auto alter_order_by = std::make_unique("ALTER ORDER BY", next_flag++, TABLE); alter_order_by->aliases.push_back("MODIFY ORDER BY"); alter_order_by->aliases.push_back("ALTER MODIFY ORDER BY"); - auto add_index = std::make_unique("ADD INDEX", next_flag++, TABLE_LEVEL); + auto add_index = std::make_unique("ADD INDEX", next_flag++, TABLE); add_index->aliases.push_back("ALTER ADD INDEX"); - auto drop_index = std::make_unique("DROP INDEX", next_flag++, TABLE_LEVEL); + auto drop_index = std::make_unique("DROP INDEX", next_flag++, TABLE); drop_index->aliases.push_back("ALTER DROP INDEX"); - auto materialize_index = std::make_unique("MATERIALIZE INDEX", next_flag++, TABLE_LEVEL); + auto materialize_index = std::make_unique("MATERIALIZE INDEX", next_flag++, TABLE); materialize_index->aliases.push_back("ALTER MATERIALIZE INDEX"); - auto clear_index = std::make_unique("CLEAR INDEX", next_flag++, TABLE_LEVEL); + auto clear_index = std::make_unique("CLEAR INDEX", next_flag++, TABLE); clear_index->aliases.push_back("ALTER CLEAR INDEX"); auto index = std::make_unique("INDEX", std::move(alter_order_by), std::move(add_index), std::move(drop_index), std::move(materialize_index), std::move(clear_index)); index->aliases.push_back("ALTER INDEX"); - auto add_constraint = std::make_unique("ADD CONSTRAINT", next_flag++, TABLE_LEVEL); + auto add_constraint = std::make_unique("ADD CONSTRAINT", next_flag++, TABLE); add_constraint->aliases.push_back("ALTER ADD CONSTRAINT"); - auto drop_constraint = std::make_unique("DROP CONSTRAINT", next_flag++, TABLE_LEVEL); + auto drop_constraint = std::make_unique("DROP CONSTRAINT", next_flag++, TABLE); drop_constraint->aliases.push_back("ALTER DROP CONSTRAINT"); auto alter_constraint = std::make_unique("CONSTRAINT", std::move(add_constraint), std::move(drop_constraint)); alter_constraint->aliases.push_back("ALTER CONSTRAINT"); - auto modify_ttl = std::make_unique("MODIFY TTL", next_flag++, TABLE_LEVEL); + auto modify_ttl = std::make_unique("MODIFY TTL", next_flag++, TABLE); modify_ttl->aliases.push_back("ALTER MODIFY TTL"); - auto materialize_ttl = std::make_unique("MATERIALIZE TTL", next_flag++, TABLE_LEVEL); + auto materialize_ttl = std::make_unique("MATERIALIZE TTL", next_flag++, TABLE); materialize_ttl->aliases.push_back("ALTER MATERIALIZE TTL"); - auto modify_setting = std::make_unique("MODIFY SETTING", next_flag++, TABLE_LEVEL); + auto modify_setting = std::make_unique("MODIFY SETTING", next_flag++, TABLE); modify_setting->aliases.push_back("ALTER MODIFY SETTING"); - auto move_partition = std::make_unique("MOVE PARTITION", next_flag++, TABLE_LEVEL); + auto move_partition = std::make_unique("MOVE PARTITION", next_flag++, TABLE); ext::push_back(move_partition->aliases, "ALTER MOVE PARTITION", "MOVE PART", "ALTER MOVE PART"); - auto fetch_partition = std::make_unique("FETCH PARTITION", next_flag++, TABLE_LEVEL); + auto fetch_partition = std::make_unique("FETCH PARTITION", next_flag++, TABLE); ext::push_back(fetch_partition->aliases, "ALTER FETCH PARTITION"); - auto freeze_partition = std::make_unique("FREEZE PARTITION", next_flag++, TABLE_LEVEL); + auto freeze_partition = std::make_unique("FREEZE PARTITION", next_flag++, TABLE); ext::push_back(freeze_partition->aliases, "ALTER FREEZE PARTITION"); auto alter_table = std::make_unique("ALTER TABLE", std::move(update), std::move(delet), std::move(alter_column), std::move(index), std::move(alter_constraint), std::move(modify_ttl), std::move(materialize_ttl), std::move(modify_setting), std::move(move_partition), std::move(fetch_partition), std::move(freeze_partition)); - auto refresh_view = std::make_unique("REFRESH VIEW", next_flag++, VIEW_LEVEL); + auto refresh_view = std::make_unique("REFRESH VIEW", next_flag++, VIEW); ext::push_back(refresh_view->aliases, "ALTER LIVE VIEW REFRESH"); - auto modify_view_query = std::make_unique("MODIFY VIEW QUERY", next_flag++, VIEW_LEVEL); + auto modify_view_query = std::make_unique("MODIFY VIEW QUERY", next_flag++, VIEW); auto alter_view = std::make_unique("ALTER VIEW", std::move(refresh_view), std::move(modify_view_query)); auto alter = std::make_unique("ALTER", std::move(alter_table), std::move(alter_view)); ext::push_back(all, std::move(alter)); - auto create_database = std::make_unique("CREATE DATABASE", next_flag++, DATABASE_LEVEL); - auto create_table = std::make_unique("CREATE TABLE", next_flag++, TABLE_LEVEL); - auto create_view = std::make_unique("CREATE VIEW", next_flag++, VIEW_LEVEL); - auto create_dictionary = std::make_unique("CREATE DICTIONARY", next_flag++, DICTIONARY_LEVEL); + auto create_database = std::make_unique("CREATE DATABASE", next_flag++, DATABASE); + auto create_table = std::make_unique("CREATE TABLE", next_flag++, TABLE); + auto create_view = std::make_unique("CREATE VIEW", next_flag++, VIEW); + auto create_dictionary = std::make_unique("CREATE DICTIONARY", next_flag++, DICTIONARY); auto create = std::make_unique("CREATE", std::move(create_database), std::move(create_table), std::move(create_view), std::move(create_dictionary)); ext::push_back(all, std::move(create)); - auto create_temporary_table = std::make_unique("CREATE TEMPORARY TABLE", next_flag++, GLOBAL_LEVEL); + auto create_temporary_table = std::make_unique("CREATE TEMPORARY TABLE", next_flag++, GLOBAL); ext::push_back(all, std::move(create_temporary_table)); - auto drop_database = std::make_unique("DROP DATABASE", next_flag++, DATABASE_LEVEL); - auto drop_table = std::make_unique("DROP TABLE", next_flag++, TABLE_LEVEL); - auto drop_view = std::make_unique("DROP VIEW", next_flag++, VIEW_LEVEL); - auto drop_dictionary = std::make_unique("DROP DICTIONARY", next_flag++, DICTIONARY_LEVEL); + auto drop_database = std::make_unique("DROP DATABASE", next_flag++, DATABASE); + auto drop_table = std::make_unique("DROP TABLE", next_flag++, TABLE); + auto drop_view = std::make_unique("DROP VIEW", next_flag++, VIEW); + auto drop_dictionary = std::make_unique("DROP DICTIONARY", next_flag++, DICTIONARY); auto drop = std::make_unique("DROP", std::move(drop_database), std::move(drop_table), std::move(drop_view), std::move(drop_dictionary)); ext::push_back(all, std::move(drop)); - auto truncate_table = std::make_unique("TRUNCATE TABLE", next_flag++, TABLE_LEVEL); - auto truncate_view = std::make_unique("TRUNCATE VIEW", next_flag++, VIEW_LEVEL); + auto truncate_table = std::make_unique("TRUNCATE TABLE", next_flag++, TABLE); + auto truncate_view = std::make_unique("TRUNCATE VIEW", next_flag++, VIEW); auto truncate = std::make_unique("TRUNCATE", std::move(truncate_table), std::move(truncate_view)); ext::push_back(all, std::move(truncate)); - auto optimize = std::make_unique("OPTIMIZE", next_flag++, TABLE_LEVEL); + auto optimize = std::make_unique("OPTIMIZE", next_flag++, TABLE); optimize->aliases.push_back("OPTIMIZE TABLE"); ext::push_back(all, std::move(optimize)); - auto kill_query = std::make_unique("KILL QUERY", next_flag++, GLOBAL_LEVEL); - auto kill_mutation = std::make_unique("KILL MUTATION", next_flag++, TABLE_LEVEL); + auto kill_query = std::make_unique("KILL QUERY", next_flag++, GLOBAL); + auto kill_mutation = std::make_unique("KILL MUTATION", next_flag++, TABLE); auto kill = std::make_unique("KILL", std::move(kill_query), std::move(kill_mutation)); ext::push_back(all, std::move(kill)); - auto create_user = std::make_unique("CREATE USER", next_flag++, GLOBAL_LEVEL); - auto alter_user = std::make_unique("ALTER USER", next_flag++, GLOBAL_LEVEL); - auto drop_user = std::make_unique("DROP USER", next_flag++, GLOBAL_LEVEL); - auto create_role = std::make_unique("CREATE ROLE", next_flag++, GLOBAL_LEVEL); - auto drop_role = std::make_unique("DROP ROLE", next_flag++, GLOBAL_LEVEL); - auto create_policy = std::make_unique("CREATE POLICY", next_flag++, GLOBAL_LEVEL); - auto alter_policy = std::make_unique("ALTER POLICY", next_flag++, GLOBAL_LEVEL); - auto drop_policy = std::make_unique("DROP POLICY", next_flag++, GLOBAL_LEVEL); - auto create_quota = std::make_unique("CREATE QUOTA", next_flag++, GLOBAL_LEVEL); - auto alter_quota = std::make_unique("ALTER QUOTA", next_flag++, GLOBAL_LEVEL); - auto drop_quota = std::make_unique("DROP QUOTA", next_flag++, GLOBAL_LEVEL); - auto role_admin = std::make_unique("ROLE ADMIN", next_flag++, GLOBAL_LEVEL); + auto create_user = std::make_unique("CREATE USER", next_flag++, GLOBAL); + auto alter_user = std::make_unique("ALTER USER", next_flag++, GLOBAL); + auto drop_user = std::make_unique("DROP USER", next_flag++, GLOBAL); + auto create_role = std::make_unique("CREATE ROLE", next_flag++, GLOBAL); + auto drop_role = std::make_unique("DROP ROLE", next_flag++, GLOBAL); + auto create_policy = std::make_unique("CREATE POLICY", next_flag++, GLOBAL); + auto alter_policy = std::make_unique("ALTER POLICY", next_flag++, GLOBAL); + auto drop_policy = std::make_unique("DROP POLICY", next_flag++, GLOBAL); + auto create_quota = std::make_unique("CREATE QUOTA", next_flag++, GLOBAL); + auto alter_quota = std::make_unique("ALTER QUOTA", next_flag++, GLOBAL); + auto drop_quota = std::make_unique("DROP QUOTA", next_flag++, GLOBAL); + auto role_admin = std::make_unique("ROLE ADMIN", next_flag++, GLOBAL); ext::push_back(all, std::move(create_user), std::move(alter_user), std::move(drop_user), std::move(create_role), std::move(drop_role), std::move(create_policy), std::move(alter_policy), std::move(drop_policy), std::move(create_quota), std::move(alter_quota), std::move(drop_quota), std::move(role_admin)); - auto shutdown = std::make_unique("SHUTDOWN", next_flag++, GLOBAL_LEVEL); + auto shutdown = std::make_unique("SHUTDOWN", next_flag++, GLOBAL); ext::push_back(shutdown->aliases, "SYSTEM SHUTDOWN", "SYSTEM KILL"); - auto drop_cache = std::make_unique("DROP CACHE", next_flag++, GLOBAL_LEVEL); + auto drop_cache = std::make_unique("DROP CACHE", next_flag++, GLOBAL); ext::push_back(drop_cache->aliases, "SYSTEM DROP CACHE", "DROP DNS CACHE", "SYSTEM DROP DNS CACHE", "DROP MARK CACHE", "SYSTEM DROP MARK CACHE", "DROP UNCOMPRESSED CACHE", "SYSTEM DROP UNCOMPRESSED CACHE", "DROP COMPILED EXPRESSION CACHE", "SYSTEM DROP COMPILED EXPRESSION CACHE"); - auto reload_config = std::make_unique("RELOAD CONFIG", next_flag++, GLOBAL_LEVEL); + auto reload_config = std::make_unique("RELOAD CONFIG", next_flag++, GLOBAL); ext::push_back(reload_config->aliases, "SYSTEM RELOAD CONFIG"); - auto reload_dictionary = std::make_unique("RELOAD DICTIONARY", next_flag++, GLOBAL_LEVEL); + auto reload_dictionary = std::make_unique("RELOAD DICTIONARY", next_flag++, GLOBAL); ext::push_back(reload_dictionary->aliases, "SYSTEM RELOAD DICTIONARY", "RELOAD DICTIONARIES", "SYSTEM RELOAD DICTIONARIES", "RELOAD EMBEDDED DICTIONARIES", "SYSTEM RELOAD EMBEDDED DICTIONARIES"); - auto stop_merges = std::make_unique("STOP MERGES", next_flag++, TABLE_LEVEL); + auto stop_merges = std::make_unique("STOP MERGES", next_flag++, TABLE); ext::push_back(stop_merges->aliases, "SYSTEM STOP MERGES", "START MERGES", "SYSTEM START MERGES"); - auto stop_ttl_merges = std::make_unique("STOP TTL MERGES", next_flag++, TABLE_LEVEL); + auto stop_ttl_merges = std::make_unique("STOP TTL MERGES", next_flag++, TABLE); ext::push_back(stop_ttl_merges->aliases, "SYSTEM STOP TTL MERGES", "START TTL MERGES", "SYSTEM START TTL MERGES"); - auto stop_fetches = std::make_unique("STOP FETCHES", next_flag++, TABLE_LEVEL); + auto stop_fetches = std::make_unique("STOP FETCHES", next_flag++, TABLE); ext::push_back(stop_fetches->aliases, "SYSTEM STOP FETCHES", "START FETCHES", "SYSTEM START FETCHES"); - auto stop_moves = std::make_unique("STOP MOVES", next_flag++, TABLE_LEVEL); + auto stop_moves = std::make_unique("STOP MOVES", next_flag++, TABLE); ext::push_back(stop_moves->aliases, "SYSTEM STOP MOVES", "START MOVES", "SYSTEM START MOVES"); - auto stop_distributed_sends = std::make_unique("STOP DISTRIBUTED SENDS", next_flag++, TABLE_LEVEL); + auto stop_distributed_sends = std::make_unique("STOP DISTRIBUTED SENDS", next_flag++, TABLE); ext::push_back(stop_distributed_sends->aliases, "SYSTEM STOP DISTRIBUTED SENDS", "START DISTRIBUTED SENDS", "SYSTEM START DISTRIBUTED SENDS"); - auto stop_replicated_sends = std::make_unique("STOP REPLICATED SENDS", next_flag++, TABLE_LEVEL); + auto stop_replicated_sends = std::make_unique("STOP REPLICATED SENDS", next_flag++, TABLE); ext::push_back(stop_replicated_sends->aliases, "SYSTEM STOP REPLICATED SENDS", "START REPLICATED SENDS", "SYSTEM START REPLICATED SENDS"); - auto stop_replication_queues = std::make_unique("STOP REPLICATION QUEUES", next_flag++, TABLE_LEVEL); + auto stop_replication_queues = std::make_unique("STOP REPLICATION QUEUES", next_flag++, TABLE); ext::push_back(stop_replication_queues->aliases, "SYSTEM STOP REPLICATION QUEUES", "START REPLICATION QUEUES", "SYSTEM START REPLICATION QUEUES"); - auto sync_replica = std::make_unique("SYNC REPLICA", next_flag++, TABLE_LEVEL); + auto sync_replica = std::make_unique("SYNC REPLICA", next_flag++, TABLE); ext::push_back(sync_replica->aliases, "SYSTEM SYNC REPLICA"); - auto restart_replica = std::make_unique("RESTART REPLICA", next_flag++, TABLE_LEVEL); + auto restart_replica = std::make_unique("RESTART REPLICA", next_flag++, TABLE); ext::push_back(restart_replica->aliases, "SYSTEM RESTART REPLICA"); - auto flush_distributed = std::make_unique("FLUSH DISTRIBUTED", next_flag++, TABLE_LEVEL); + auto flush_distributed = std::make_unique("FLUSH DISTRIBUTED", next_flag++, TABLE); ext::push_back(flush_distributed->aliases, "SYSTEM FLUSH DISTRIBUTED"); - auto flush_logs = std::make_unique("FLUSH LOGS", next_flag++, GLOBAL_LEVEL); + auto flush_logs = std::make_unique("FLUSH LOGS", next_flag++, GLOBAL); ext::push_back(flush_logs->aliases, "SYSTEM FLUSH LOGS"); auto system = std::make_unique("SYSTEM", std::move(shutdown), std::move(drop_cache), std::move(reload_config), std::move(reload_dictionary), std::move(stop_merges), std::move(stop_ttl_merges), std::move(stop_fetches), std::move(stop_moves), std::move(stop_distributed_sends), std::move(stop_replicated_sends), std::move(stop_replication_queues), std::move(sync_replica), std::move(restart_replica), std::move(flush_distributed), std::move(flush_logs)); ext::push_back(all, std::move(system)); - auto dict_get = std::make_unique("dictGet()", next_flag++, DICTIONARY_LEVEL); + auto dict_get = std::make_unique("dictGet()", next_flag++, DICTIONARY); dict_get->aliases.push_back("dictHas()"); dict_get->aliases.push_back("dictGetHierarchy()"); dict_get->aliases.push_back("dictIsIn()"); ext::push_back(all, std::move(dict_get)); - auto address_to_line = std::make_unique("addressToLine()", next_flag++, GLOBAL_LEVEL); - auto address_to_symbol = std::make_unique("addressToSymbol()", next_flag++, GLOBAL_LEVEL); - auto demangle = std::make_unique("demangle()", next_flag++, GLOBAL_LEVEL); + auto address_to_line = std::make_unique("addressToLine()", next_flag++, GLOBAL); + auto address_to_symbol = std::make_unique("addressToSymbol()", next_flag++, GLOBAL); + auto demangle = std::make_unique("demangle()", next_flag++, GLOBAL); auto introspection = std::make_unique("INTROSPECTION", std::move(address_to_line), std::move(address_to_symbol), std::move(demangle)); ext::push_back(introspection->aliases, "INTROSPECTION FUNCTIONS"); ext::push_back(all, std::move(introspection)); - auto file = std::make_unique("file()", next_flag++, GLOBAL_LEVEL); - auto url = std::make_unique("url()", next_flag++, GLOBAL_LEVEL); - auto input = std::make_unique("input()", next_flag++, GLOBAL_LEVEL); - auto values = std::make_unique("values()", next_flag++, GLOBAL_LEVEL); - auto numbers = std::make_unique("numbers()", next_flag++, GLOBAL_LEVEL); - auto zeros = std::make_unique("zeros()", next_flag++, GLOBAL_LEVEL); - auto merge = std::make_unique("merge()", next_flag++, DATABASE_LEVEL); - auto remote = std::make_unique("remote()", next_flag++, GLOBAL_LEVEL); + auto file = std::make_unique("file()", next_flag++, GLOBAL); + auto url = std::make_unique("url()", next_flag++, GLOBAL); + auto input = std::make_unique("input()", next_flag++, GLOBAL); + auto values = std::make_unique("values()", next_flag++, GLOBAL); + auto numbers = std::make_unique("numbers()", next_flag++, GLOBAL); + auto zeros = std::make_unique("zeros()", next_flag++, GLOBAL); + auto merge = std::make_unique("merge()", next_flag++, DATABASE); + auto remote = std::make_unique("remote()", next_flag++, GLOBAL); ext::push_back(remote->aliases, "remoteSecure()", "cluster()"); - auto mysql = std::make_unique("mysql()", next_flag++, GLOBAL_LEVEL); - auto odbc = std::make_unique("odbc()", next_flag++, GLOBAL_LEVEL); - auto jdbc = std::make_unique("jdbc()", next_flag++, GLOBAL_LEVEL); - auto hdfs = std::make_unique("hdfs()", next_flag++, GLOBAL_LEVEL); - auto s3 = std::make_unique("s3()", next_flag++, GLOBAL_LEVEL); + auto mysql = std::make_unique("mysql()", next_flag++, GLOBAL); + auto odbc = std::make_unique("odbc()", next_flag++, GLOBAL); + auto jdbc = std::make_unique("jdbc()", next_flag++, GLOBAL); + auto hdfs = std::make_unique("hdfs()", next_flag++, GLOBAL); + auto s3 = std::make_unique("s3()", next_flag++, GLOBAL); auto table_functions = std::make_unique("TABLE FUNCTIONS", std::move(file), std::move(url), std::move(input), std::move(values), std::move(numbers), std::move(zeros), std::move(merge), std::move(remote), std::move(mysql), std::move(odbc), std::move(jdbc), std::move(hdfs), std::move(s3)); ext::push_back(all, std::move(table_functions)); - flags_to_keyword_tree_ = std::make_unique("ALL", std::move(all)); - flags_to_keyword_tree_->aliases.push_back("ALL PRIVILEGES"); + auto node_all = std::make_unique("ALL", std::move(all)); + node_all->aliases.push_back("ALL PRIVILEGES"); + return node_all; } - void makeKeywordToFlagsMap(std::unordered_map & keyword_to_flags_map_, Node * start_node = nullptr) + void makeKeywordToFlagsMap(Node * start_node = nullptr) { if (!start_node) { start_node = flags_to_keyword_tree.get(); - keyword_to_flags_map_["USAGE"] = {}; - keyword_to_flags_map_["NONE"] = {}; - keyword_to_flags_map_["NO PRIVILEGES"] = {}; + keyword_to_flags_map["USAGE"] = {}; + keyword_to_flags_map["NONE"] = {}; + keyword_to_flags_map["NO PRIVILEGES"] = {}; } start_node->aliases.emplace_back(start_node->keyword); for (auto & alias : start_node->aliases) { boost::to_upper(alias); - keyword_to_flags_map_[alias] = start_node->flags; + keyword_to_flags_map[alias] = start_node->flags; } for (auto & child : start_node->children) - makeKeywordToFlagsMap(keyword_to_flags_map_, child.get()); + makeKeywordToFlagsMap(child.get()); } - void makeAccessTypeToFlagsMapping(std::vector & access_type_to_flags_mapping_) + void makeAccessTypeToFlagsMapping() { - access_type_to_flags_mapping_.resize(MAX_ACCESS_TYPE); + access_type_to_flags_mapping.resize(MAX_ACCESS_TYPE); for (auto access_type : ext::range_with_static_cast(0, MAX_ACCESS_TYPE)) { auto str = toKeyword(access_type); @@ -449,35 +467,36 @@ private: boost::to_upper(uppercased); it = keyword_to_flags_map.find(uppercased); } - access_type_to_flags_mapping_[static_cast(access_type)] = it->second; + access_type_to_flags_mapping[static_cast(access_type)] = it->second; } } - void collectAllGrantableOnLevel(std::vector & all_grantable_on_level_, const Node * start_node = nullptr) + void collectAllFlags(const Node * start_node = nullptr) { if (!start_node) { start_node = flags_to_keyword_tree.get(); - all_grantable_on_level.resize(COLUMN_LEVEL + 1); + all_flags = start_node->flags; } - for (int i = 0; i <= start_node->level; ++i) - all_grantable_on_level_[i] |= start_node->flags; + if (start_node->target != UNKNOWN_TARGET) + all_flags_for_target[start_node->target] |= start_node->flags; for (const auto & child : start_node->children) - collectAllGrantableOnLevel(all_grantable_on_level_, child.get()); + collectAllFlags(child.get()); } Impl() { - makeFlagsToKeywordTree(flags_to_keyword_tree); - makeKeywordToFlagsMap(keyword_to_flags_map); - makeAccessTypeToFlagsMapping(access_type_to_flags_mapping); - collectAllGrantableOnLevel(all_grantable_on_level); + flags_to_keyword_tree = makeFlagsToKeywordTree(); + makeKeywordToFlagsMap(); + makeAccessTypeToFlagsMapping(); + collectAllFlags(); } std::unique_ptr flags_to_keyword_tree; std::unordered_map keyword_to_flags_map; std::vector access_type_to_flags_mapping; - std::vector all_grantable_on_level; + Flags all_flags; + Flags all_flags_for_target[NUM_TARGETS]; }; @@ -487,9 +506,12 @@ inline AccessFlags::AccessFlags(const std::vector & keywords) inline AccessFlags::AccessFlags(const Strings & keywords) : flags(Impl<>::instance().keywordsToFlags(keywords)) {} inline String AccessFlags::toString() const { return Impl<>::instance().flagsToString(flags); } inline std::vector AccessFlags::toKeywords() const { return Impl<>::instance().flagsToKeywords(flags); } -inline AccessFlags AccessFlags::databaseLevel() { return Impl<>::instance().getDatabaseLevelFlags(); } -inline AccessFlags AccessFlags::tableLevel() { return Impl<>::instance().getTableLevelFlags(); } -inline AccessFlags AccessFlags::columnLevel() { return Impl<>::instance().getColumnLevelFlags(); } +inline AccessFlags AccessFlags::allFlags() { return Impl<>::instance().getAllFlags(); } +inline AccessFlags AccessFlags::allGlobalFlags() { return Impl<>::instance().getGlobalFlags(); } +inline AccessFlags AccessFlags::allDatabaseFlags() { return Impl<>::instance().getDatabaseFlags(); } +inline AccessFlags AccessFlags::allTableFlags() { return Impl<>::instance().getTableFlags(); } +inline AccessFlags AccessFlags::allColumnFlags() { return Impl<>::instance().getColumnFlags(); } +inline AccessFlags AccessFlags::allDictionaryFlags() { return Impl<>::instance().getDictionaryFlags(); } inline AccessFlags operator |(AccessType left, AccessType right) { return AccessFlags(left) | right; } inline AccessFlags operator &(AccessType left, AccessType right) { return AccessFlags(left) & right; } diff --git a/dbms/src/Access/AccessRights.cpp b/dbms/src/Access/AccessRights.cpp index ae5c043a75f..6f94cfac286 100644 --- a/dbms/src/Access/AccessRights.cpp +++ b/dbms/src/Access/AccessRights.cpp @@ -31,12 +31,19 @@ namespace return res; } - const AccessFlags database_level_flags = AccessFlags::databaseLevel(); - const AccessFlags table_level_flags = AccessFlags::tableLevel(); - const AccessFlags column_level_flags = AccessFlags::columnLevel(); + const AccessFlags all_flags = AccessFlags::allFlags(); + const AccessFlags database_flags = AccessFlags::allDatabaseFlags(); + const AccessFlags table_flags = AccessFlags::allTableFlags(); + const AccessFlags column_flags = AccessFlags::allColumnFlags(); + const AccessFlags dictionary_flags = AccessFlags::allDictionaryFlags(); + const AccessFlags column_level_flags = column_flags; + const AccessFlags table_level_flags = table_flags | dictionary_flags | column_level_flags; + const AccessFlags database_level_flags = database_flags | table_level_flags; - const AccessFlags show_flag = AccessType::SHOW; - const AccessFlags exists_flag = AccessType::EXISTS; + const AccessFlags show_databases_flag = AccessType::SHOW_DATABASES; + const AccessFlags show_tables_flag = AccessType::SHOW_TABLES; + const AccessFlags show_columns_flag = AccessType::SHOW_COLUMNS; + const AccessFlags show_dictionaries_flag = AccessType::SHOW_DICTIONARIES; const AccessFlags create_table_flag = AccessType::CREATE_TABLE; const AccessFlags create_view_flag = AccessType::CREATE_VIEW; const AccessFlags create_temporary_table_flag = AccessType::CREATE_TEMPORARY_TABLE; @@ -367,25 +374,46 @@ private: void calculateFinalAccess(const Helper & helper) { /// Calculate min and max access among children. - AccessFlags min_access_among_children = AccessType::ALL; + AccessFlags min_access_among_children = helper.all_flags; AccessFlags max_access_among_children; if (children) { for (const auto & child : *children | boost::adaptors::map_values) { - min_access &= child.min_access; - max_access |= child.max_access; + min_access_among_children &= child.min_access; + max_access_among_children |= child.max_access; } } /// Calculate implicit access: AccessFlags implicit_access; - if (access & helper.database_level_flags) - implicit_access |= helper.show_flag | helper.exists_flag; - else if ((level >= DATABASE_LEVEL) && children) - implicit_access |= helper.exists_flag; - if ((level == GLOBAL_LEVEL) && (final_access & helper.create_table_flag)) + if (level <= DATABASE_LEVEL) + { + if (access & helper.database_flags) + implicit_access |= helper.show_databases_flag; + } + if (level <= TABLE_LEVEL) + { + if (access & helper.table_flags) + implicit_access |= helper.show_tables_flag; + if (access & helper.dictionary_flags) + implicit_access |= helper.show_dictionaries_flag; + } + if (level <= COLUMN_LEVEL) + { + if (access & helper.column_flags) + implicit_access |= helper.show_columns_flag; + } + if (children && max_access_among_children) + { + if (level == DATABASE_LEVEL) + implicit_access |= helper.show_databases_flag; + else if (level == TABLE_LEVEL) + implicit_access |= helper.show_tables_flag; + } + + if ((level == GLOBAL_LEVEL) && ((access | max_access_among_children) & helper.create_table_flag)) implicit_access |= helper.create_temporary_table_flag; if (level <= TABLE_LEVEL) diff --git a/dbms/src/Access/AccessRightsContext.cpp b/dbms/src/Access/AccessRightsContext.cpp index bfc9c386ef4..f847e353a41 100644 --- a/dbms/src/Access/AccessRightsContext.cpp +++ b/dbms/src/Access/AccessRightsContext.cpp @@ -435,38 +435,37 @@ boost::shared_ptr AccessRightsContext::calculateResultAccess | AccessType::CREATE_QUOTA | AccessType::ALTER_USER | AccessType::ALTER_POLICY | AccessType::ALTER_QUOTA | AccessType::DROP_USER | AccessType::DROP_ROLE | AccessType::DROP_POLICY | AccessType::DROP_QUOTA | AccessType::ROLE_ADMIN; - /// Anyone has access to the "system" database. - if (!result.isGranted(AccessType::SELECT, DatabaseCatalog::SYSTEM_DATABASE)) - result.grant(AccessType::SELECT, DatabaseCatalog::SYSTEM_DATABASE); - - /// User has access to temporary or external table if such table was resolved in session or query context - if (!result.isGranted(AccessType::SELECT, DatabaseCatalog::TEMPORARY_DATABASE)) - result.grant(AccessType::SELECT, DatabaseCatalog::TEMPORARY_DATABASE); - if (readonly_) result.revoke(write_table_access | all_dcl | AccessType::SYSTEM | AccessType::KILL); if (readonly_ || !allow_ddl_) result.revoke(table_and_dictionary_ddl); - if (readonly_ && grant_option) - result.revoke(AccessType::ALL); - if (readonly_ == 1) { /// Table functions are forbidden in readonly mode. /// For example, for readonly = 2 - allowed. result.revoke(AccessType::CREATE_TEMPORARY_TABLE | AccessType::TABLE_FUNCTIONS); } - else if (readonly_ == 2) - { - /// Allow INSERT into temporary tables - result.grant(AccessType::INSERT, DatabaseCatalog::TEMPORARY_DATABASE); - } if (!allow_introspection_) result.revoke(AccessType::INTROSPECTION); + /// Anyone has access to the "system" database. + result.grant(AccessType::SELECT, DatabaseCatalog::SYSTEM_DATABASE); + + if (readonly_ != 1) + { + /// User has access to temporary or external table if such table was resolved in session or query context + result.grant(AccessFlags::allTableFlags() | AccessFlags::allColumnFlags(), DatabaseCatalog::TEMPORARY_DATABASE); + } + + if (readonly_ && grant_option) + { + /// No grant option in readonly mode. + result.revoke(AccessType::ALL); + } + result_access_cache[cache_index].store(result_ptr); if (trace_log && (params.readonly == readonly_) && (params.allow_ddl == allow_ddl_) && (params.allow_introspection == allow_introspection_)) diff --git a/dbms/src/Access/AccessType.h b/dbms/src/Access/AccessType.h index 205840eecdf..ea51d66451a 100644 --- a/dbms/src/Access/AccessType.h +++ b/dbms/src/Access/AccessType.h @@ -14,12 +14,11 @@ enum class AccessType NONE, /// no access ALL, /// full access - SHOW, /// allows to execute SHOW TABLES, SHOW CREATE TABLE, SHOW DATABASES and so on - /// (granted implicitly with any other grant) - - EXISTS, /// allows to execute EXISTS, USE, i.e. to check existence - /// (granted implicitly on the database level with any other grant on the database and lower levels, - /// e.g. "GRANT SELECT(x) ON db.table" also grants EXISTS on db.*) + SHOW_DATABASES, /// allows to execute SHOW DATABASES, SHOW CREATE DATABASE, USE + SHOW_TABLES, /// allows to execute SHOW TABLES, EXISTS
, CHECK
+ SHOW_COLUMNS, /// allows to execute SHOW CREATE TABLE, DESCRIBE + SHOW_DICTIONARIES, /// allows to execute SHOW DICTIONARIES, SHOW CREATE DICTIONARY, EXISTS + SHOW, /// allows to execute SHOW, USE, EXISTS, CHECK, DESCRIBE SELECT, INSERT, @@ -179,8 +178,12 @@ namespace impl ACCESS_TYPE_TO_KEYWORD_CASE(NONE); ACCESS_TYPE_TO_KEYWORD_CASE(ALL); + + ACCESS_TYPE_TO_KEYWORD_CASE(SHOW_DATABASES); + ACCESS_TYPE_TO_KEYWORD_CASE(SHOW_TABLES); + ACCESS_TYPE_TO_KEYWORD_CASE(SHOW_COLUMNS); + ACCESS_TYPE_TO_KEYWORD_CASE(SHOW_DICTIONARIES); ACCESS_TYPE_TO_KEYWORD_CASE(SHOW); - ACCESS_TYPE_TO_KEYWORD_CASE(EXISTS); ACCESS_TYPE_TO_KEYWORD_CASE(SELECT); ACCESS_TYPE_TO_KEYWORD_CASE(INSERT); diff --git a/dbms/src/Access/UsersConfigAccessStorage.cpp b/dbms/src/Access/UsersConfigAccessStorage.cpp index ecb681ad7e0..fb3c3d5b80c 100644 --- a/dbms/src/Access/UsersConfigAccessStorage.cpp +++ b/dbms/src/Access/UsersConfigAccessStorage.cpp @@ -141,19 +141,18 @@ namespace if (databases) { - user->access.revoke(AccessFlags::databaseLevel()); + user->access.revoke(AccessFlags::allFlags() - AccessFlags::allGlobalFlags()); + user->access.grant(AccessFlags::allDictionaryFlags(), IDictionary::NO_DATABASE_TAG); for (const String & database : *databases) - user->access.grant(AccessFlags::databaseLevel(), database); + user->access.grant(AccessFlags::allFlags(), database); } if (dictionaries) { - user->access.revoke(AccessType::dictGet, IDictionary::NO_DATABASE_TAG); + user->access.revoke(AccessFlags::allDictionaryFlags(), IDictionary::NO_DATABASE_TAG); for (const String & dictionary : *dictionaries) - user->access.grant(AccessType::dictGet, IDictionary::NO_DATABASE_TAG, dictionary); + user->access.grant(AccessFlags::allDictionaryFlags(), IDictionary::NO_DATABASE_TAG, dictionary); } - else if (databases) - user->access.grant(AccessType::dictGet, IDictionary::NO_DATABASE_TAG); user->access_with_grant_option = user->access; diff --git a/dbms/src/Interpreters/InterpreterCheckQuery.cpp b/dbms/src/Interpreters/InterpreterCheckQuery.cpp index 25fac1d4982..b8f7203e607 100644 --- a/dbms/src/Interpreters/InterpreterCheckQuery.cpp +++ b/dbms/src/Interpreters/InterpreterCheckQuery.cpp @@ -40,7 +40,7 @@ BlockIO InterpreterCheckQuery::execute() const auto & check = query_ptr->as(); auto table_id = context.resolveStorageID(check, Context::ResolveOrdinary); - context.checkAccess(AccessType::SHOW, table_id); + context.checkAccess(AccessType::SHOW_TABLES, table_id); StoragePtr table = DatabaseCatalog::instance().getTable(table_id); auto check_results = table->checkData(query_ptr, context); diff --git a/dbms/src/Interpreters/InterpreterDescribeQuery.cpp b/dbms/src/Interpreters/InterpreterDescribeQuery.cpp index c2660f63169..1353c01ebf6 100644 --- a/dbms/src/Interpreters/InterpreterDescribeQuery.cpp +++ b/dbms/src/Interpreters/InterpreterDescribeQuery.cpp @@ -85,8 +85,7 @@ BlockInputStreamPtr InterpreterDescribeQuery::executeImpl() else { auto table_id = context.resolveStorageID(table_expression.database_and_table_name); - context.checkAccess(AccessType::SHOW, table_id); - + context.checkAccess(AccessType::SHOW_COLUMNS, table_id); table = DatabaseCatalog::instance().getTable(table_id); } diff --git a/dbms/src/Interpreters/InterpreterExistsQuery.cpp b/dbms/src/Interpreters/InterpreterExistsQuery.cpp index 7cd864fddb7..993b3631e06 100644 --- a/dbms/src/Interpreters/InterpreterExistsQuery.cpp +++ b/dbms/src/Interpreters/InterpreterExistsQuery.cpp @@ -44,13 +44,12 @@ BlockInputStreamPtr InterpreterExistsQuery::executeImpl() { if (exists_query->temporary) { - context.checkAccess(AccessType::EXISTS, "", exists_query->table); result = context.tryResolveStorageID({"", exists_query->table}, Context::ResolveExternal); } else { String database = context.resolveDatabase(exists_query->database); - context.checkAccess(AccessType::EXISTS, database, exists_query->table); + context.checkAccess(AccessType::SHOW_TABLES, database, exists_query->table); result = DatabaseCatalog::instance().isTableExist({database, exists_query->table}); } } @@ -59,7 +58,7 @@ BlockInputStreamPtr InterpreterExistsQuery::executeImpl() if (exists_query->temporary) throw Exception("Temporary dictionaries are not possible.", ErrorCodes::SYNTAX_ERROR); String database = context.resolveDatabase(exists_query->database); - context.checkAccess(AccessType::EXISTS, database, exists_query->table); + context.checkAccess(AccessType::SHOW_DICTIONARIES, database, exists_query->table); result = DatabaseCatalog::instance().isDictionaryExist({database, exists_query->table}); } diff --git a/dbms/src/Interpreters/InterpreterShowCreateQuery.cpp b/dbms/src/Interpreters/InterpreterShowCreateQuery.cpp index 71f08f12f5d..8bee0b88fe8 100644 --- a/dbms/src/Interpreters/InterpreterShowCreateQuery.cpp +++ b/dbms/src/Interpreters/InterpreterShowCreateQuery.cpp @@ -49,7 +49,7 @@ BlockInputStreamPtr InterpreterShowCreateQuery::executeImpl() { auto resolve_table_type = show_query->temporary ? Context::ResolveExternal : Context::ResolveOrdinary; auto table_id = context.resolveStorageID(*show_query, resolve_table_type); - context.checkAccess(AccessType::SHOW, table_id); + context.checkAccess(AccessType::SHOW_COLUMNS, table_id); create_query = DatabaseCatalog::instance().getDatabase(table_id.database_name)->getCreateTableQuery(context, table_id.table_name); } else if ((show_query = query_ptr->as())) @@ -57,7 +57,7 @@ BlockInputStreamPtr InterpreterShowCreateQuery::executeImpl() if (show_query->temporary) throw Exception("Temporary databases are not possible.", ErrorCodes::SYNTAX_ERROR); show_query->database = context.resolveDatabase(show_query->database); - context.checkAccess(AccessType::SHOW, show_query->database); + context.checkAccess(AccessType::SHOW_DATABASES, show_query->database); create_query = DatabaseCatalog::instance().getDatabase(show_query->database)->getCreateDatabaseQuery(context); } else if ((show_query = query_ptr->as())) @@ -65,7 +65,7 @@ BlockInputStreamPtr InterpreterShowCreateQuery::executeImpl() if (show_query->temporary) throw Exception("Temporary dictionaries are not possible.", ErrorCodes::SYNTAX_ERROR); show_query->database = context.resolveDatabase(show_query->database); - context.checkAccess(AccessType::SHOW, show_query->database, show_query->table); + context.checkAccess(AccessType::SHOW_DICTIONARIES, show_query->database, show_query->table); create_query = DatabaseCatalog::instance().getDatabase(show_query->database)->getCreateDictionaryQuery(context, show_query->table); } diff --git a/dbms/src/Interpreters/InterpreterUseQuery.cpp b/dbms/src/Interpreters/InterpreterUseQuery.cpp index 0cddaf26c11..58f5b6c9a32 100644 --- a/dbms/src/Interpreters/InterpreterUseQuery.cpp +++ b/dbms/src/Interpreters/InterpreterUseQuery.cpp @@ -11,7 +11,7 @@ namespace DB BlockIO InterpreterUseQuery::execute() { const String & new_database = query_ptr->as().database; - context.checkAccess(AccessType::EXISTS, new_database); + context.checkAccess(AccessType::SHOW_DATABASES, new_database); context.getSessionContext().setCurrentDatabase(new_database); return {}; } diff --git a/dbms/src/Storages/System/StorageSystemColumns.cpp b/dbms/src/Storages/System/StorageSystemColumns.cpp index 9bf96f95264..4e210f7935c 100644 --- a/dbms/src/Storages/System/StorageSystemColumns.cpp +++ b/dbms/src/Storages/System/StorageSystemColumns.cpp @@ -82,7 +82,7 @@ protected: MutableColumns res_columns = getPort().getHeader().cloneEmptyColumns(); size_t rows_count = 0; - const bool check_access_for_tables = !access_rights->isGranted(AccessType::SHOW); + const bool check_access_for_tables = !access_rights->isGranted(AccessType::SHOW_COLUMNS); while (rows_count < max_block_size && db_table_num < total_tables) { @@ -128,14 +128,14 @@ protected: column_sizes = storage->getColumnSizes(); } - bool check_access_for_columns = check_access_for_tables && !access_rights->isGranted(AccessType::SHOW, database_name, table_name); + bool check_access_for_columns = check_access_for_tables && !access_rights->isGranted(AccessType::SHOW_COLUMNS, database_name, table_name); for (const auto & column : columns) { if (column.is_virtual) continue; - if (check_access_for_columns && !access_rights->isGranted(AccessType::SHOW, database_name, table_name, column.name)) + if (check_access_for_columns && !access_rights->isGranted(AccessType::SHOW_COLUMNS, database_name, table_name, column.name)) continue; size_t src_index = 0; diff --git a/dbms/src/Storages/System/StorageSystemDatabases.cpp b/dbms/src/Storages/System/StorageSystemDatabases.cpp index 4588fd28482..835ed074a55 100644 --- a/dbms/src/Storages/System/StorageSystemDatabases.cpp +++ b/dbms/src/Storages/System/StorageSystemDatabases.cpp @@ -21,12 +21,12 @@ NamesAndTypesList StorageSystemDatabases::getNamesAndTypes() void StorageSystemDatabases::fillData(MutableColumns & res_columns, const Context & context, const SelectQueryInfo &) const { const auto access_rights = context.getAccessRights(); - const bool check_access_for_databases = !access_rights->isGranted(AccessType::SHOW); + const bool check_access_for_databases = !access_rights->isGranted(AccessType::SHOW_DATABASES); auto databases = DatabaseCatalog::instance().getDatabases(); for (const auto & database : databases) { - if (check_access_for_databases && !access_rights->isGranted(AccessType::SHOW, database.first)) + if (check_access_for_databases && !access_rights->isGranted(AccessType::SHOW_DATABASES, database.first)) continue; res_columns[0]->insert(database.first); diff --git a/dbms/src/Storages/System/StorageSystemDictionaries.cpp b/dbms/src/Storages/System/StorageSystemDictionaries.cpp index 87a11387e4d..e3d4f7c773c 100644 --- a/dbms/src/Storages/System/StorageSystemDictionaries.cpp +++ b/dbms/src/Storages/System/StorageSystemDictionaries.cpp @@ -50,7 +50,7 @@ NamesAndTypesList StorageSystemDictionaries::getNamesAndTypes() void StorageSystemDictionaries::fillData(MutableColumns & res_columns, const Context & context, const SelectQueryInfo & /*query_info*/) const { const auto access_rights = context.getAccessRights(); - const bool check_access_for_dictionaries = !access_rights->isGranted(AccessType::SHOW); + const bool check_access_for_dictionaries = !access_rights->isGranted(AccessType::SHOW_DICTIONARIES); const auto & external_dictionaries = context.getExternalDictionariesLoader(); for (const auto & load_result : external_dictionaries.getCurrentLoadResults()) @@ -74,7 +74,7 @@ void StorageSystemDictionaries::fillData(MutableColumns & res_columns, const Con } if (check_access_for_dictionaries - && !access_rights->isGranted(AccessType::SHOW, database.empty() ? IDictionary::NO_DATABASE_TAG : database, short_name)) + && !access_rights->isGranted(AccessType::SHOW_DICTIONARIES, database.empty() ? IDictionary::NO_DATABASE_TAG : database, short_name)) continue; size_t i = 0; diff --git a/dbms/src/Storages/System/StorageSystemMerges.cpp b/dbms/src/Storages/System/StorageSystemMerges.cpp index 7e71d90120a..04e05a31534 100644 --- a/dbms/src/Storages/System/StorageSystemMerges.cpp +++ b/dbms/src/Storages/System/StorageSystemMerges.cpp @@ -37,11 +37,11 @@ NamesAndTypesList StorageSystemMerges::getNamesAndTypes() void StorageSystemMerges::fillData(MutableColumns & res_columns, const Context & context, const SelectQueryInfo &) const { const auto access_rights = context.getAccessRights(); - const bool check_access_for_tables = !access_rights->isGranted(AccessType::SHOW); + const bool check_access_for_tables = !access_rights->isGranted(AccessType::SHOW_TABLES); for (const auto & merge : context.getMergeList().get()) { - if (check_access_for_tables && !access_rights->isGranted(AccessType::SHOW, merge.database, merge.table)) + if (check_access_for_tables && !access_rights->isGranted(AccessType::SHOW_TABLES, merge.database, merge.table)) continue; size_t i = 0; diff --git a/dbms/src/Storages/System/StorageSystemMutations.cpp b/dbms/src/Storages/System/StorageSystemMutations.cpp index 51c5bd47c6d..4e324ae6bd2 100644 --- a/dbms/src/Storages/System/StorageSystemMutations.cpp +++ b/dbms/src/Storages/System/StorageSystemMutations.cpp @@ -38,7 +38,7 @@ NamesAndTypesList StorageSystemMutations::getNamesAndTypes() void StorageSystemMutations::fillData(MutableColumns & res_columns, const Context & context, const SelectQueryInfo & query_info) const { const auto access_rights = context.getAccessRights(); - const bool check_access_for_databases = !access_rights->isGranted(AccessType::SHOW); + const bool check_access_for_databases = !access_rights->isGranted(AccessType::SHOW_TABLES); /// Collect a set of *MergeTree tables. std::map> merge_tree_tables; @@ -48,14 +48,14 @@ void StorageSystemMutations::fillData(MutableColumns & res_columns, const Contex if (db.second->getEngineName() == "Lazy") continue; - const bool check_access_for_tables = check_access_for_databases && !access_rights->isGranted(AccessType::SHOW, db.first); + const bool check_access_for_tables = check_access_for_databases && !access_rights->isGranted(AccessType::SHOW_TABLES, db.first); for (auto iterator = db.second->getTablesIterator(context); iterator->isValid(); iterator->next()) { if (!dynamic_cast(iterator->table().get())) continue; - if (check_access_for_tables && !access_rights->isGranted(AccessType::SHOW, db.first, iterator->name())) + if (check_access_for_tables && !access_rights->isGranted(AccessType::SHOW_TABLES, db.first, iterator->name())) continue; merge_tree_tables[db.first][iterator->name()] = iterator->table(); diff --git a/dbms/src/Storages/System/StorageSystemPartsBase.cpp b/dbms/src/Storages/System/StorageSystemPartsBase.cpp index 4bf3737dcd4..235c17611fc 100644 --- a/dbms/src/Storages/System/StorageSystemPartsBase.cpp +++ b/dbms/src/Storages/System/StorageSystemPartsBase.cpp @@ -74,7 +74,7 @@ StoragesInfoStream::StoragesInfoStream(const SelectQueryInfo & query_info, const MutableColumnPtr active_column_mut = ColumnUInt8::create(); const auto access_rights = context.getAccessRights(); - const bool check_access_for_tables = !access_rights->isGranted(AccessType::SHOW); + const bool check_access_for_tables = !access_rights->isGranted(AccessType::SHOW_TABLES); { Databases databases = DatabaseCatalog::instance().getDatabases(); @@ -119,7 +119,7 @@ StoragesInfoStream::StoragesInfoStream(const SelectQueryInfo & query_info, const if (!dynamic_cast(storage.get())) continue; - if (check_access_for_tables && !access_rights->isGranted(AccessType::SHOW, database_name, table_name)) + if (check_access_for_tables && !access_rights->isGranted(AccessType::SHOW_TABLES, database_name, table_name)) continue; storages[std::make_pair(database_name, iterator->name())] = storage; diff --git a/dbms/src/Storages/System/StorageSystemReplicas.cpp b/dbms/src/Storages/System/StorageSystemReplicas.cpp index 16a2a8d07de..02ae587d1b5 100644 --- a/dbms/src/Storages/System/StorageSystemReplicas.cpp +++ b/dbms/src/Storages/System/StorageSystemReplicas.cpp @@ -66,7 +66,7 @@ Pipes StorageSystemReplicas::read( check(column_names); const auto access_rights = context.getAccessRights(); - const bool check_access_for_databases = !access_rights->isGranted(AccessType::SHOW); + const bool check_access_for_databases = !access_rights->isGranted(AccessType::SHOW_TABLES); /// We collect a set of replicated tables. std::map> replicated_tables; @@ -75,12 +75,12 @@ Pipes StorageSystemReplicas::read( /// Lazy database can not contain replicated tables if (db.second->getEngineName() == "Lazy") continue; - const bool check_access_for_tables = check_access_for_databases && !access_rights->isGranted(AccessType::SHOW, db.first); + const bool check_access_for_tables = check_access_for_databases && !access_rights->isGranted(AccessType::SHOW_TABLES, db.first); for (auto iterator = db.second->getTablesIterator(context); iterator->isValid(); iterator->next()) { if (!dynamic_cast(iterator->table().get())) continue; - if (check_access_for_tables && !access_rights->isGranted(AccessType::SHOW, db.first, iterator->name())) + if (check_access_for_tables && !access_rights->isGranted(AccessType::SHOW_TABLES, db.first, iterator->name())) continue; replicated_tables[db.first][iterator->name()] = iterator->table(); } diff --git a/dbms/src/Storages/System/StorageSystemReplicationQueue.cpp b/dbms/src/Storages/System/StorageSystemReplicationQueue.cpp index 5148e0a9ec8..4519d735161 100644 --- a/dbms/src/Storages/System/StorageSystemReplicationQueue.cpp +++ b/dbms/src/Storages/System/StorageSystemReplicationQueue.cpp @@ -49,7 +49,7 @@ NamesAndTypesList StorageSystemReplicationQueue::getNamesAndTypes() void StorageSystemReplicationQueue::fillData(MutableColumns & res_columns, const Context & context, const SelectQueryInfo & query_info) const { const auto access_rights = context.getAccessRights(); - const bool check_access_for_databases = !access_rights->isGranted(AccessType::SHOW); + const bool check_access_for_databases = !access_rights->isGranted(AccessType::SHOW_TABLES); std::map> replicated_tables; for (const auto & db : DatabaseCatalog::instance().getDatabases()) @@ -58,13 +58,13 @@ void StorageSystemReplicationQueue::fillData(MutableColumns & res_columns, const if (db.second->getEngineName() == "Lazy") continue; - const bool check_access_for_tables = check_access_for_databases && !access_rights->isGranted(AccessType::SHOW, db.first); + const bool check_access_for_tables = check_access_for_databases && !access_rights->isGranted(AccessType::SHOW_TABLES, db.first); for (auto iterator = db.second->getTablesIterator(context); iterator->isValid(); iterator->next()) { if (!dynamic_cast(iterator->table().get())) continue; - if (check_access_for_tables && !access_rights->isGranted(AccessType::SHOW, db.first, iterator->name())) + if (check_access_for_tables && !access_rights->isGranted(AccessType::SHOW_TABLES, db.first, iterator->name())) continue; replicated_tables[db.first][iterator->name()] = iterator->table(); } diff --git a/dbms/src/Storages/System/StorageSystemTables.cpp b/dbms/src/Storages/System/StorageSystemTables.cpp index 7bc0799e795..2ada4a64b45 100644 --- a/dbms/src/Storages/System/StorageSystemTables.cpp +++ b/dbms/src/Storages/System/StorageSystemTables.cpp @@ -106,7 +106,7 @@ protected: MutableColumns res_columns = getPort().getHeader().cloneEmptyColumns(); const auto access_rights = context.getAccessRights(); - const bool check_access_for_databases = !access_rights->isGranted(AccessType::SHOW); + const bool check_access_for_databases = !access_rights->isGranted(AccessType::SHOW_TABLES); size_t rows_count = 0; while (rows_count < max_block_size) @@ -196,7 +196,7 @@ protected: return Chunk(std::move(res_columns), num_rows); } - const bool check_access_for_tables = check_access_for_databases && !access_rights->isGranted(AccessType::SHOW, database_name); + const bool check_access_for_tables = check_access_for_databases && !access_rights->isGranted(AccessType::SHOW_TABLES, database_name); if (!tables_it || !tables_it->isValid()) tables_it = database->getTablesWithDictionaryTablesIterator(context); @@ -206,7 +206,7 @@ protected: for (; rows_count < max_block_size && tables_it->isValid(); tables_it->next()) { auto table_name = tables_it->name(); - if (check_access_for_tables && !access_rights->isGranted(AccessType::SHOW, database_name, table_name)) + if (check_access_for_tables && !access_rights->isGranted(AccessType::SHOW_TABLES, database_name, table_name)) continue; StoragePtr table = nullptr; From f1e9e3dec0d154b41fab9699369720377317a281 Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Fri, 6 Mar 2020 20:13:28 +0300 Subject: [PATCH 0152/1355] Improve access rights: KILL_MUTATION deleted, rights for corresponding ALTER commands are checked instead. --- dbms/src/Access/AccessFlags.h | 4 +- dbms/src/Access/AccessRightsContext.cpp | 2 +- dbms/src/Access/AccessType.h | 4 - .../Interpreters/InterpreterAlterQuery.cpp | 301 +++++++++--------- dbms/src/Interpreters/InterpreterAlterQuery.h | 3 + .../InterpreterKillQueryQuery.cpp | 35 +- .../0_stateless/00834_kill_mutation.reference | 4 +- ...ll_mutation_replicated_zookeeper.reference | 4 +- 8 files changed, 189 insertions(+), 168 deletions(-) diff --git a/dbms/src/Access/AccessFlags.h b/dbms/src/Access/AccessFlags.h index 2cf59b4886d..61a160fc69a 100644 --- a/dbms/src/Access/AccessFlags.h +++ b/dbms/src/Access/AccessFlags.h @@ -349,9 +349,7 @@ private: ext::push_back(all, std::move(optimize)); auto kill_query = std::make_unique("KILL QUERY", next_flag++, GLOBAL); - auto kill_mutation = std::make_unique("KILL MUTATION", next_flag++, TABLE); - auto kill = std::make_unique("KILL", std::move(kill_query), std::move(kill_mutation)); - ext::push_back(all, std::move(kill)); + ext::push_back(all, std::move(kill_query)); auto create_user = std::make_unique("CREATE USER", next_flag++, GLOBAL); auto alter_user = std::make_unique("ALTER USER", next_flag++, GLOBAL); diff --git a/dbms/src/Access/AccessRightsContext.cpp b/dbms/src/Access/AccessRightsContext.cpp index f847e353a41..6eff1f050b3 100644 --- a/dbms/src/Access/AccessRightsContext.cpp +++ b/dbms/src/Access/AccessRightsContext.cpp @@ -436,7 +436,7 @@ boost::shared_ptr AccessRightsContext::calculateResultAccess | AccessType::DROP_ROLE | AccessType::DROP_POLICY | AccessType::DROP_QUOTA | AccessType::ROLE_ADMIN; if (readonly_) - result.revoke(write_table_access | all_dcl | AccessType::SYSTEM | AccessType::KILL); + result.revoke(write_table_access | all_dcl | AccessType::SYSTEM | AccessType::KILL_QUERY); if (readonly_ || !allow_ddl_) result.revoke(table_and_dictionary_ddl); diff --git a/dbms/src/Access/AccessType.h b/dbms/src/Access/AccessType.h index ea51d66451a..4084d180013 100644 --- a/dbms/src/Access/AccessType.h +++ b/dbms/src/Access/AccessType.h @@ -79,8 +79,6 @@ enum class AccessType OPTIMIZE, /// allows to execute OPTIMIZE TABLE KILL_QUERY, /// allows to kill a query started by another user (anyone can kill his own queries) - KILL_MUTATION, /// allows to kill a mutation - KILL, /// allows to execute KILL {MUTATION|QUERY} CREATE_USER, ALTER_USER, @@ -244,8 +242,6 @@ namespace impl ACCESS_TYPE_TO_KEYWORD_CASE(OPTIMIZE); ACCESS_TYPE_TO_KEYWORD_CASE(KILL_QUERY); - ACCESS_TYPE_TO_KEYWORD_CASE(KILL_MUTATION); - ACCESS_TYPE_TO_KEYWORD_CASE(KILL); ACCESS_TYPE_TO_KEYWORD_CASE(CREATE_USER); ACCESS_TYPE_TO_KEYWORD_CASE(ALTER_USER); diff --git a/dbms/src/Interpreters/InterpreterAlterQuery.cpp b/dbms/src/Interpreters/InterpreterAlterQuery.cpp index 5462fc16a81..315527765ef 100644 --- a/dbms/src/Interpreters/InterpreterAlterQuery.cpp +++ b/dbms/src/Interpreters/InterpreterAlterQuery.cpp @@ -13,7 +13,7 @@ #include #include #include - +#include #include @@ -125,155 +125,162 @@ AccessRightsElements InterpreterAlterQuery::getRequiredAccess() const { AccessRightsElements required_access; const auto & alter = query_ptr->as(); - for (ASTAlterCommand * command_ast : alter.command_list->commands) - { - auto column_name = [&]() -> String { return getIdentifierName(command_ast->column); }; - auto column_name_from_col_decl = [&]() -> std::string_view { return command_ast->col_decl->as().name; }; - auto column_names_from_update_assignments = [&]() -> std::vector - { - std::vector column_names; - for (const ASTPtr & assignment_ast : command_ast->update_assignments->children) - column_names.emplace_back(assignment_ast->as().column_name); - return column_names; - }; + for (ASTAlterCommand * command : alter.command_list->commands) + boost::range::push_back(required_access, getRequiredAccessForCommand(*command, alter.database, alter.table)); + return required_access; +} - switch (command_ast->type) + +AccessRightsElements InterpreterAlterQuery::getRequiredAccessForCommand(const ASTAlterCommand & command, const String & database, const String & table) +{ + AccessRightsElements required_access; + + auto column_name = [&]() -> String { return getIdentifierName(command.column); }; + auto column_name_from_col_decl = [&]() -> std::string_view { return command.col_decl->as().name; }; + auto column_names_from_update_assignments = [&]() -> std::vector + { + std::vector column_names; + for (const ASTPtr & assignment_ast : command.update_assignments->children) + column_names.emplace_back(assignment_ast->as().column_name); + return column_names; + }; + + switch (command.type) + { + case ASTAlterCommand::UPDATE: { - case ASTAlterCommand::UPDATE: - { - required_access.emplace_back(AccessType::UPDATE, alter.database, alter.table, column_names_from_update_assignments()); - break; - } - case ASTAlterCommand::DELETE: - { - required_access.emplace_back(AccessType::DELETE, alter.database, alter.table); - break; - } - case ASTAlterCommand::ADD_COLUMN: - { - required_access.emplace_back(AccessType::ADD_COLUMN, alter.database, alter.table, column_name_from_col_decl()); - break; - } - case ASTAlterCommand::DROP_COLUMN: - { - if (command_ast->clear_column) - required_access.emplace_back(AccessType::CLEAR_COLUMN, alter.database, alter.table, column_name()); - else - required_access.emplace_back(AccessType::DROP_COLUMN, alter.database, alter.table, column_name()); - break; - } - case ASTAlterCommand::MODIFY_COLUMN: - { - required_access.emplace_back(AccessType::MODIFY_COLUMN, alter.database, alter.table, column_name_from_col_decl()); - break; - } - case ASTAlterCommand::COMMENT_COLUMN: - { - required_access.emplace_back(AccessType::COMMENT_COLUMN, alter.database, alter.table, column_name()); - break; - } - case ASTAlterCommand::MODIFY_ORDER_BY: - { - required_access.emplace_back(AccessType::ALTER_ORDER_BY, alter.database, alter.table); - break; - } - case ASTAlterCommand::ADD_INDEX: - { - required_access.emplace_back(AccessType::ADD_INDEX, alter.database, alter.table); - break; - } - case ASTAlterCommand::DROP_INDEX: - { - if (command_ast->clear_index) - required_access.emplace_back(AccessType::CLEAR_INDEX, alter.database, alter.table); - else - required_access.emplace_back(AccessType::DROP_INDEX, alter.database, alter.table); - break; - } - case ASTAlterCommand::MATERIALIZE_INDEX: - { - required_access.emplace_back(AccessType::MATERIALIZE_INDEX, alter.database, alter.table); - break; - } - case ASTAlterCommand::ADD_CONSTRAINT: - { - required_access.emplace_back(AccessType::ADD_CONSTRAINT, alter.database, alter.table); - break; - } - case ASTAlterCommand::DROP_CONSTRAINT: - { - required_access.emplace_back(AccessType::DROP_CONSTRAINT, alter.database, alter.table); - break; - } - case ASTAlterCommand::MODIFY_TTL: - { - required_access.emplace_back(AccessType::MODIFY_TTL, alter.database, alter.table); - break; - } - case ASTAlterCommand::MATERIALIZE_TTL: - { - required_access.emplace_back(AccessType::MATERIALIZE_TTL, alter.database, alter.table); - break; - } - case ASTAlterCommand::MODIFY_SETTING: - { - required_access.emplace_back(AccessType::MODIFY_SETTING, alter.database, alter.table); - break; - } - case ASTAlterCommand::ATTACH_PARTITION: - { - required_access.emplace_back(AccessType::INSERT, alter.database, alter.table); - break; - } - case ASTAlterCommand::DROP_PARTITION: [[fallthrough]]; - case ASTAlterCommand::DROP_DETACHED_PARTITION: - { - required_access.emplace_back(AccessType::DELETE, alter.database, alter.table); - break; - } - case ASTAlterCommand::MOVE_PARTITION: - { - if ((command_ast->move_destination_type == PartDestinationType::DISK) - || (command_ast->move_destination_type == PartDestinationType::VOLUME)) - { - required_access.emplace_back(AccessType::MOVE_PARTITION, alter.database, alter.table); - } - else if (command_ast->move_destination_type == PartDestinationType::TABLE) - { - required_access.emplace_back(AccessType::SELECT | AccessType::DELETE, alter.database, alter.table); - required_access.emplace_back(AccessType::INSERT, command_ast->to_database, command_ast->to_table); - } - break; - } - case ASTAlterCommand::REPLACE_PARTITION: - { - required_access.emplace_back(AccessType::SELECT, command_ast->from_database, command_ast->from_table); - required_access.emplace_back(AccessType::DELETE | AccessType::INSERT, alter.database, alter.table); - break; - } - case ASTAlterCommand::FETCH_PARTITION: - { - required_access.emplace_back(AccessType::FETCH_PARTITION, alter.database, alter.table); - break; - } - case ASTAlterCommand::FREEZE_PARTITION: [[fallthrough]]; - case ASTAlterCommand::FREEZE_ALL: - { - required_access.emplace_back(AccessType::FREEZE_PARTITION, alter.database, alter.table); - break; - } - case ASTAlterCommand::MODIFY_QUERY: - { - required_access.emplace_back(AccessType::MODIFY_VIEW_QUERY, alter.database, alter.table); - break; - } - case ASTAlterCommand::LIVE_VIEW_REFRESH: - { - required_access.emplace_back(AccessType::REFRESH_VIEW, alter.database, alter.table); - break; - } - case ASTAlterCommand::NO_TYPE: break; + required_access.emplace_back(AccessType::UPDATE, database, table, column_names_from_update_assignments()); + break; } + case ASTAlterCommand::DELETE: + { + required_access.emplace_back(AccessType::DELETE, database, table); + break; + } + case ASTAlterCommand::ADD_COLUMN: + { + required_access.emplace_back(AccessType::ADD_COLUMN, database, table, column_name_from_col_decl()); + break; + } + case ASTAlterCommand::DROP_COLUMN: + { + if (command.clear_column) + required_access.emplace_back(AccessType::CLEAR_COLUMN, database, table, column_name()); + else + required_access.emplace_back(AccessType::DROP_COLUMN, database, table, column_name()); + break; + } + case ASTAlterCommand::MODIFY_COLUMN: + { + required_access.emplace_back(AccessType::MODIFY_COLUMN, database, table, column_name_from_col_decl()); + break; + } + case ASTAlterCommand::COMMENT_COLUMN: + { + required_access.emplace_back(AccessType::COMMENT_COLUMN, database, table, column_name()); + break; + } + case ASTAlterCommand::MODIFY_ORDER_BY: + { + required_access.emplace_back(AccessType::ALTER_ORDER_BY, database, table); + break; + } + case ASTAlterCommand::ADD_INDEX: + { + required_access.emplace_back(AccessType::ADD_INDEX, database, table); + break; + } + case ASTAlterCommand::DROP_INDEX: + { + if (command.clear_index) + required_access.emplace_back(AccessType::CLEAR_INDEX, database, table); + else + required_access.emplace_back(AccessType::DROP_INDEX, database, table); + break; + } + case ASTAlterCommand::MATERIALIZE_INDEX: + { + required_access.emplace_back(AccessType::MATERIALIZE_INDEX, database, table); + break; + } + case ASTAlterCommand::ADD_CONSTRAINT: + { + required_access.emplace_back(AccessType::ADD_CONSTRAINT, database, table); + break; + } + case ASTAlterCommand::DROP_CONSTRAINT: + { + required_access.emplace_back(AccessType::DROP_CONSTRAINT, database, table); + break; + } + case ASTAlterCommand::MODIFY_TTL: + { + required_access.emplace_back(AccessType::MODIFY_TTL, database, table); + break; + } + case ASTAlterCommand::MATERIALIZE_TTL: + { + required_access.emplace_back(AccessType::MATERIALIZE_TTL, database, table); + break; + } + case ASTAlterCommand::MODIFY_SETTING: + { + required_access.emplace_back(AccessType::MODIFY_SETTING, database, table); + break; + } + case ASTAlterCommand::ATTACH_PARTITION: + { + required_access.emplace_back(AccessType::INSERT, database, table); + break; + } + case ASTAlterCommand::DROP_PARTITION: [[fallthrough]]; + case ASTAlterCommand::DROP_DETACHED_PARTITION: + { + required_access.emplace_back(AccessType::DELETE, database, table); + break; + } + case ASTAlterCommand::MOVE_PARTITION: + { + if ((command.move_destination_type == PartDestinationType::DISK) + || (command.move_destination_type == PartDestinationType::VOLUME)) + { + required_access.emplace_back(AccessType::MOVE_PARTITION, database, table); + } + else if (command.move_destination_type == PartDestinationType::TABLE) + { + required_access.emplace_back(AccessType::SELECT | AccessType::DELETE, database, table); + required_access.emplace_back(AccessType::INSERT, command.to_database, command.to_table); + } + break; + } + case ASTAlterCommand::REPLACE_PARTITION: + { + required_access.emplace_back(AccessType::SELECT, command.from_database, command.from_table); + required_access.emplace_back(AccessType::DELETE | AccessType::INSERT, database, table); + break; + } + case ASTAlterCommand::FETCH_PARTITION: + { + required_access.emplace_back(AccessType::FETCH_PARTITION, database, table); + break; + } + case ASTAlterCommand::FREEZE_PARTITION: [[fallthrough]]; + case ASTAlterCommand::FREEZE_ALL: + { + required_access.emplace_back(AccessType::FREEZE_PARTITION, database, table); + break; + } + case ASTAlterCommand::MODIFY_QUERY: + { + required_access.emplace_back(AccessType::MODIFY_VIEW_QUERY, database, table); + break; + } + case ASTAlterCommand::LIVE_VIEW_REFRESH: + { + required_access.emplace_back(AccessType::REFRESH_VIEW, database, table); + break; + } + case ASTAlterCommand::NO_TYPE: break; } return required_access; diff --git a/dbms/src/Interpreters/InterpreterAlterQuery.h b/dbms/src/Interpreters/InterpreterAlterQuery.h index fd395a0de52..a7609eb81f1 100644 --- a/dbms/src/Interpreters/InterpreterAlterQuery.h +++ b/dbms/src/Interpreters/InterpreterAlterQuery.h @@ -8,6 +8,7 @@ namespace DB { class Context; class AccessRightsElements; +class ASTAlterCommand; /** Allows you add or remove a column in the table. @@ -20,6 +21,8 @@ public: BlockIO execute() override; + static AccessRightsElements getRequiredAccessForCommand(const ASTAlterCommand & command, const String & database, const String & table); + private: AccessRightsElements getRequiredAccess() const; diff --git a/dbms/src/Interpreters/InterpreterKillQueryQuery.cpp b/dbms/src/Interpreters/InterpreterKillQueryQuery.cpp index 81a093f4eae..dc365990794 100644 --- a/dbms/src/Interpreters/InterpreterKillQueryQuery.cpp +++ b/dbms/src/Interpreters/InterpreterKillQueryQuery.cpp @@ -6,6 +6,10 @@ #include #include #include +#include +#include +#include +#include #include #include #include @@ -55,7 +59,7 @@ struct QueryDescriptor size_t source_num; bool processed = false; - QueryDescriptor(String && query_id_, String && user_, size_t source_num_, bool processed_ = false) + QueryDescriptor(String query_id_, String user_, size_t source_num_, bool processed_ = false) : query_id(std::move(query_id_)), user(std::move(user_)), source_num(source_num_), processed(processed_) {} }; @@ -81,6 +85,7 @@ static QueryDescriptors extractQueriesExceptMeAndCheckAccess(const Block & proce const ClientInfo & my_client = context.getProcessListElement()->getClientInfo(); std::optional can_kill_query_started_by_another_user; String query_user; + bool access_denied = false; for (size_t i = 0; i < num_processes; ++i) { @@ -95,14 +100,17 @@ static QueryDescriptors extractQueriesExceptMeAndCheckAccess(const Block & proce { if (!can_kill_query_started_by_another_user) can_kill_query_started_by_another_user = context.getAccessRights()->isGranted(&Poco::Logger::get("InterpreterKillQueryQuery"), AccessType::KILL_QUERY); - if (!can_kill_query_started_by_another_user.value()) + if (!*can_kill_query_started_by_another_user) + { + access_denied = true; continue; + } } - res.emplace_back(std::move(query_id), std::move(query_user), i, false); + res.emplace_back(std::move(query_id), query_user, i, false); } - if (res.empty() && !query_user.empty()) // NOLINT + if (res.empty() && access_denied) throw Exception("User " + my_client.current_user + " attempts to kill query created by " + query_user, ErrorCodes::ACCESS_DENIED); return res; @@ -221,19 +229,22 @@ BlockIO InterpreterKillQueryQuery::execute() } case ASTKillQueryQuery::Type::Mutation: { - Block mutations_block = getSelectResult("database, table, mutation_id", "system.mutations"); + Block mutations_block = getSelectResult("database, table, mutation_id, command", "system.mutations"); if (!mutations_block) return res_io; const ColumnString & database_col = typeid_cast(*mutations_block.getByName("database").column); const ColumnString & table_col = typeid_cast(*mutations_block.getByName("table").column); const ColumnString & mutation_id_col = typeid_cast(*mutations_block.getByName("mutation_id").column); + const ColumnString & command_col = typeid_cast(*mutations_block.getByName("command").column); auto header = mutations_block.cloneEmpty(); header.insert(0, {ColumnString::create(), std::make_shared(), "kill_status"}); MutableColumns res_columns = header.cloneEmptyColumns(); auto table_id = StorageID::createEmpty(); + AccessRightsElements required_access_rights; + bool access_denied = false; for (size_t i = 0; i < mutations_block.rows(); ++i) { @@ -248,8 +259,14 @@ BlockIO InterpreterKillQueryQuery::execute() code = CancellationCode::NotFound; else { - if (!context.getAccessRights()->isGranted(&Poco::Logger::get("InterpreterKillQueryQuery"), AccessType::KILL_MUTATION, table_id.database_name, table_id.table_name)) + ParserAlterCommand parser; + auto command_ast = parseQuery(parser, command_col.getDataAt(i).toString(), 0); + required_access_rights = InterpreterAlterQuery::getRequiredAccessForCommand(command_ast->as(), table_id.database_name, table_id.table_name); + if (!context.getAccessRights()->isGranted(&Poco::Logger::get("InterpreterKillQueryQuery"), required_access_rights)) + { + access_denied = true; continue; + } code = storage->killMutation(mutation_id); } } @@ -257,9 +274,9 @@ BlockIO InterpreterKillQueryQuery::execute() insertResultRow(i, code, mutations_block, header, res_columns); } - if (res_columns[0]->empty() && table_id) + if (res_columns[0]->empty() && access_denied) throw Exception( - "Not allowed to kill mutation on " + table_id.getNameForLogs(), + "Not allowed to kill mutation. To execute this query it's necessary to have the grant " + required_access_rights.toString(), ErrorCodes::ACCESS_DENIED); res_io.in = std::make_shared(header.cloneWithColumns(std::move(res_columns))); @@ -295,7 +312,7 @@ AccessRightsElements InterpreterKillQueryQuery::getRequiredAccessForDDLOnCluster if (query.type == ASTKillQueryQuery::Type::Query) required_access.emplace_back(AccessType::KILL_QUERY); else if (query.type == ASTKillQueryQuery::Type::Mutation) - required_access.emplace_back(AccessType::KILL_MUTATION); + required_access.emplace_back(AccessType::UPDATE | AccessType::DELETE | AccessType::MATERIALIZE_INDEX | AccessType::MATERIALIZE_TTL); return required_access; } diff --git a/dbms/tests/queries/0_stateless/00834_kill_mutation.reference b/dbms/tests/queries/0_stateless/00834_kill_mutation.reference index cbee44069d8..1e4a67b66ea 100644 --- a/dbms/tests/queries/0_stateless/00834_kill_mutation.reference +++ b/dbms/tests/queries/0_stateless/00834_kill_mutation.reference @@ -1,7 +1,7 @@ *** Create and kill a single invalid mutation *** 1 -waiting test kill_mutation mutation_3.txt +waiting test kill_mutation mutation_3.txt DELETE WHERE toUInt32(s) = 1 *** Create and kill invalid mutation that blocks another mutation *** 1 -waiting test kill_mutation mutation_4.txt +waiting test kill_mutation mutation_4.txt DELETE WHERE toUInt32(s) = 1 2001-01-01 2 b diff --git a/dbms/tests/queries/0_stateless/00834_kill_mutation_replicated_zookeeper.reference b/dbms/tests/queries/0_stateless/00834_kill_mutation_replicated_zookeeper.reference index a997ebe1dc9..d6a82e48836 100644 --- a/dbms/tests/queries/0_stateless/00834_kill_mutation_replicated_zookeeper.reference +++ b/dbms/tests/queries/0_stateless/00834_kill_mutation_replicated_zookeeper.reference @@ -1,9 +1,9 @@ *** Create and kill a single invalid mutation *** 1 Mutation 0000000000 was killed -waiting test kill_mutation_r1 0000000000 +waiting test kill_mutation_r1 0000000000 DELETE WHERE toUInt32(s) = 1 0 *** Create and kill invalid mutation that blocks another mutation *** 1 -waiting test kill_mutation_r1 0000000001 +waiting test kill_mutation_r1 0000000001 DELETE WHERE toUInt32(s) = 1 2001-01-01 2 b From 18d5f63b3114180c57839e27ad6c4c56c510d0a8 Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Sat, 7 Mar 2020 20:37:38 +0300 Subject: [PATCH 0153/1355] Mass rename: AccessRightsContext -> ContextAccess, QuotaContext -> EnabledQuota, RoleContext -> EnabledRoles, and so on. --- dbms/src/Access/AccessControlManager.cpp | 76 ++- dbms/src/Access/AccessControlManager.h | 38 +- dbms/src/Access/AccessRights.h | 24 +- dbms/src/Access/AccessRightsContext.cpp | 585 ------------------ dbms/src/Access/AccessRightsContext.h | 157 ----- .../src/Access/AccessRightsContextFactory.cpp | 48 -- dbms/src/Access/AccessRightsContextFactory.h | 29 - dbms/src/Access/ContextAccess.cpp | 535 ++++++++++++++++ dbms/src/Access/ContextAccess.h | 156 +++++ .../{QuotaContext.cpp => EnabledQuota.cpp} | 75 +-- .../Access/{QuotaContext.h => EnabledQuota.h} | 74 +-- dbms/src/Access/EnabledRoles.cpp | 51 ++ dbms/src/Access/EnabledRoles.h | 53 ++ ...rentRolesInfo.cpp => EnabledRolesInfo.cpp} | 8 +- ...{CurrentRolesInfo.h => EnabledRolesInfo.h} | 8 +- ...licyContext.cpp => EnabledRowPolicies.cpp} | 40 +- ...owPolicyContext.h => EnabledRowPolicies.h} | 41 +- ...GenericRoleSet.cpp => ExtendedRoleSet.cpp} | 96 ++- dbms/src/Access/ExtendedRoleSet.h | 75 +++ dbms/src/Access/GenericRoleSet.h | 77 --- dbms/src/Access/Quota.cpp | 2 +- dbms/src/Access/Quota.h | 6 +- ...QuotaContextFactory.cpp => QuotaCache.cpp} | 116 ++-- .../{QuotaContextFactory.h => QuotaCache.h} | 27 +- dbms/src/Access/QuotaUsageInfo.cpp | 17 + dbms/src/Access/QuotaUsageInfo.h | 32 + dbms/src/Access/RoleCache.cpp | 186 ++++++ dbms/src/Access/RoleCache.h | 36 ++ dbms/src/Access/RoleContext.cpp | 200 ------ dbms/src/Access/RoleContext.h | 64 -- dbms/src/Access/RoleContextFactory.cpp | 52 -- dbms/src/Access/RoleContextFactory.h | 29 - dbms/src/Access/RowPolicy.cpp | 6 +- dbms/src/Access/RowPolicy.h | 14 +- ...yContextFactory.cpp => RowPolicyCache.cpp} | 113 ++-- ...olicyContextFactory.h => RowPolicyCache.h} | 22 +- dbms/src/Access/User.h | 4 +- dbms/src/Access/UsersConfigAccessStorage.cpp | 4 +- dbms/src/DataStreams/IBlockInputStream.cpp | 2 +- dbms/src/DataStreams/IBlockInputStream.h | 7 +- dbms/src/Functions/currentQuota.cpp | 3 +- dbms/src/Functions/currentRowPolicies.cpp | 50 +- dbms/src/Interpreters/Context.cpp | 86 +-- dbms/src/Interpreters/Context.h | 47 +- .../InterpreterCreateQuotaQuery.cpp | 14 +- .../InterpreterCreateRowPolicyQuery.cpp | 14 +- .../InterpreterCreateUserQuery.cpp | 19 +- .../Interpreters/InterpreterGrantQuery.cpp | 17 +- .../InterpreterKillQueryQuery.cpp | 29 +- .../Interpreters/InterpreterSelectQuery.cpp | 9 +- .../Interpreters/InterpreterSetRoleQuery.cpp | 12 +- .../Interpreters/InterpreterSetRoleQuery.h | 4 +- ...InterpreterShowCreateAccessEntityQuery.cpp | 25 +- .../InterpreterShowGrantsQuery.cpp | 8 +- .../Interpreters/InterpreterSystemQuery.cpp | 16 +- dbms/src/Interpreters/executeQuery.cpp | 19 +- dbms/src/Parsers/ASTCreateQuotaQuery.cpp | 4 +- dbms/src/Parsers/ASTCreateQuotaQuery.h | 4 +- dbms/src/Parsers/ASTCreateRowPolicyQuery.cpp | 18 +- dbms/src/Parsers/ASTCreateRowPolicyQuery.h | 8 +- dbms/src/Parsers/ASTCreateUserQuery.cpp | 4 +- dbms/src/Parsers/ASTCreateUserQuery.h | 4 +- ...ericRoleSet.cpp => ASTExtendedRoleSet.cpp} | 4 +- ...TGenericRoleSet.h => ASTExtendedRoleSet.h} | 6 +- dbms/src/Parsers/ASTGrantQuery.cpp | 4 +- dbms/src/Parsers/ASTGrantQuery.h | 6 +- dbms/src/Parsers/ASTSetRoleQuery.cpp | 2 +- dbms/src/Parsers/ASTSetRoleQuery.h | 6 +- dbms/src/Parsers/ParserCreateQuotaQuery.cpp | 12 +- .../Parsers/ParserCreateRowPolicyQuery.cpp | 28 +- dbms/src/Parsers/ParserCreateUserQuery.cpp | 12 +- ...cRoleSet.cpp => ParserExtendedRoleSet.cpp} | 8 +- dbms/src/Parsers/ParserExtendedRoleSet.h | 28 + dbms/src/Parsers/ParserGenericRoleSet.h | 28 - dbms/src/Parsers/ParserGrantQuery.cpp | 20 +- dbms/src/Parsers/ParserSetRoleQuery.cpp | 20 +- .../TreeExecutorBlockInputStream.cpp | 2 +- .../Executors/TreeExecutorBlockInputStream.h | 2 +- dbms/src/Processors/Pipe.cpp | 2 +- dbms/src/Processors/Pipe.h | 2 +- .../Sources/SourceFromInputStream.h | 2 +- .../Processors/Sources/SourceWithProgress.cpp | 2 +- .../Processors/Sources/SourceWithProgress.h | 6 +- .../Transforms/LimitsCheckingTransform.cpp | 2 +- .../Transforms/LimitsCheckingTransform.h | 4 +- .../Storages/System/StorageSystemColumns.cpp | 16 +- .../System/StorageSystemDatabases.cpp | 8 +- .../System/StorageSystemDictionaries.cpp | 8 +- .../Storages/System/StorageSystemMerges.cpp | 8 +- .../System/StorageSystemMutations.cpp | 10 +- .../System/StorageSystemPartsBase.cpp | 8 +- .../System/StorageSystemQuotaUsage.cpp | 3 +- .../Storages/System/StorageSystemQuotas.cpp | 2 +- .../Storages/System/StorageSystemReplicas.cpp | 10 +- .../System/StorageSystemReplicationQueue.cpp | 10 +- .../System/StorageSystemRowPolicies.cpp | 6 +- .../Storages/System/StorageSystemTables.cpp | 10 +- 97 files changed, 1918 insertions(+), 1988 deletions(-) delete mode 100644 dbms/src/Access/AccessRightsContext.cpp delete mode 100644 dbms/src/Access/AccessRightsContext.h delete mode 100644 dbms/src/Access/AccessRightsContextFactory.cpp delete mode 100644 dbms/src/Access/AccessRightsContextFactory.h create mode 100644 dbms/src/Access/ContextAccess.cpp create mode 100644 dbms/src/Access/ContextAccess.h rename dbms/src/Access/{QuotaContext.cpp => EnabledQuota.cpp} (77%) rename dbms/src/Access/{QuotaContext.h => EnabledQuota.h} (60%) create mode 100644 dbms/src/Access/EnabledRoles.cpp create mode 100644 dbms/src/Access/EnabledRoles.h rename dbms/src/Access/{CurrentRolesInfo.cpp => EnabledRolesInfo.cpp} (77%) rename dbms/src/Access/{CurrentRolesInfo.h => EnabledRolesInfo.h} (62%) rename dbms/src/Access/{RowPolicyContext.cpp => EnabledRowPolicies.cpp} (55%) rename dbms/src/Access/{RowPolicyContext.h => EnabledRowPolicies.h} (55%) rename dbms/src/Access/{GenericRoleSet.cpp => ExtendedRoleSet.cpp} (65%) create mode 100644 dbms/src/Access/ExtendedRoleSet.h delete mode 100644 dbms/src/Access/GenericRoleSet.h rename dbms/src/Access/{QuotaContextFactory.cpp => QuotaCache.cpp} (67%) rename dbms/src/Access/{QuotaContextFactory.h => QuotaCache.h} (60%) create mode 100644 dbms/src/Access/QuotaUsageInfo.cpp create mode 100644 dbms/src/Access/QuotaUsageInfo.h create mode 100644 dbms/src/Access/RoleCache.cpp create mode 100644 dbms/src/Access/RoleCache.h delete mode 100644 dbms/src/Access/RoleContext.cpp delete mode 100644 dbms/src/Access/RoleContext.h delete mode 100644 dbms/src/Access/RoleContextFactory.cpp delete mode 100644 dbms/src/Access/RoleContextFactory.h rename dbms/src/Access/{RowPolicyContextFactory.cpp => RowPolicyCache.cpp} (68%) rename dbms/src/Access/{RowPolicyContextFactory.h => RowPolicyCache.h} (56%) rename dbms/src/Parsers/{ASTGenericRoleSet.cpp => ASTExtendedRoleSet.cpp} (93%) rename dbms/src/Parsers/{ASTGenericRoleSet.h => ASTExtendedRoleSet.h} (77%) rename dbms/src/Parsers/{ParserGenericRoleSet.cpp => ParserExtendedRoleSet.cpp} (94%) create mode 100644 dbms/src/Parsers/ParserExtendedRoleSet.h delete mode 100644 dbms/src/Parsers/ParserGenericRoleSet.h diff --git a/dbms/src/Access/AccessControlManager.cpp b/dbms/src/Access/AccessControlManager.cpp index b3854e69eec..541400fe7a5 100644 --- a/dbms/src/Access/AccessControlManager.cpp +++ b/dbms/src/Access/AccessControlManager.cpp @@ -3,10 +3,14 @@ #include #include #include -#include -#include -#include -#include +#include +#include +#include +#include +#include +#include +#include +#include namespace DB @@ -27,12 +31,48 @@ namespace } +class AccessControlManager::ContextAccessCache +{ +public: + explicit ContextAccessCache(const AccessControlManager & manager_) : manager(manager_) {} + + std::shared_ptr getContextAccess(const UUID & user_id, const std::vector & current_roles, bool use_default_roles, const Settings & settings, const String & current_database, const ClientInfo & client_info) + { + ContextAccess::Params params; + params.user_id = user_id; + params.current_roles = current_roles; + params.use_default_roles = use_default_roles; + params.current_database = current_database; + params.readonly = settings.readonly; + params.allow_ddl = settings.allow_ddl; + params.allow_introspection = settings.allow_introspection_functions; + params.interface = client_info.interface; + params.http_method = client_info.http_method; + params.address = client_info.current_address.host(); + params.quota_key = client_info.quota_key; + + std::lock_guard lock{mutex}; + auto x = cache.get(params); + if (x) + return *x; + auto res = std::shared_ptr(new ContextAccess(manager, params)); + cache.add(params, res); + return res; + } + +private: + const AccessControlManager & manager; + Poco::ExpireCache> cache; + std::mutex mutex; +}; + + AccessControlManager::AccessControlManager() : MultipleAccessStorage(createStorages()), - access_rights_context_factory(std::make_unique(*this)), - role_context_factory(std::make_unique(*this)), - row_policy_context_factory(std::make_unique(*this)), - quota_context_factory(std::make_unique(*this)) + context_access_cache(std::make_unique(*this)), + role_cache(std::make_unique(*this)), + row_policy_cache(std::make_unique(*this)), + quota_cache(std::make_unique(*this)) { } @@ -54,7 +94,7 @@ void AccessControlManager::setUsersConfig(const Poco::Util::AbstractConfiguratio } -AccessRightsContextPtr AccessControlManager::getAccessRightsContext( +std::shared_ptr AccessControlManager::getContextAccess( const UUID & user_id, const std::vector & current_roles, bool use_default_roles, @@ -62,34 +102,34 @@ AccessRightsContextPtr AccessControlManager::getAccessRightsContext( const String & current_database, const ClientInfo & client_info) const { - return access_rights_context_factory->createContext(user_id, current_roles, use_default_roles, settings, current_database, client_info); + return context_access_cache->getContextAccess(user_id, current_roles, use_default_roles, settings, current_database, client_info); } -RoleContextPtr AccessControlManager::getRoleContext( +std::shared_ptr AccessControlManager::getEnabledRoles( const std::vector & current_roles, const std::vector & current_roles_with_admin_option) const { - return role_context_factory->createContext(current_roles, current_roles_with_admin_option); + return role_cache->getEnabledRoles(current_roles, current_roles_with_admin_option); } -RowPolicyContextPtr AccessControlManager::getRowPolicyContext(const UUID & user_id, const std::vector & enabled_roles) const +std::shared_ptr AccessControlManager::getEnabledRowPolicies(const UUID & user_id, const std::vector & enabled_roles) const { - return row_policy_context_factory->createContext(user_id, enabled_roles); + return row_policy_cache->getEnabledRowPolicies(user_id, enabled_roles); } -QuotaContextPtr AccessControlManager::getQuotaContext( - const String & user_name, const UUID & user_id, const std::vector & enabled_roles, const Poco::Net::IPAddress & address, const String & custom_quota_key) const +std::shared_ptr AccessControlManager::getEnabledQuota( + const UUID & user_id, const String & user_name, const std::vector & enabled_roles, const Poco::Net::IPAddress & address, const String & custom_quota_key) const { - return quota_context_factory->createContext(user_name, user_id, enabled_roles, address, custom_quota_key); + return quota_cache->getEnabledQuota(user_id, user_name, enabled_roles, address, custom_quota_key); } std::vector AccessControlManager::getQuotaUsageInfo() const { - return quota_context_factory->getUsageInfo(); + return quota_cache->getUsageInfo(); } } diff --git a/dbms/src/Access/AccessControlManager.h b/dbms/src/Access/AccessControlManager.h index bd5720bb0f4..1e7c1e6df1d 100644 --- a/dbms/src/Access/AccessControlManager.h +++ b/dbms/src/Access/AccessControlManager.h @@ -19,20 +19,15 @@ namespace Poco namespace DB { -class AccessRightsContext; -using AccessRightsContextPtr = std::shared_ptr; -class AccessRightsContextFactory; +class ContextAccess; struct User; using UserPtr = std::shared_ptr; -class RoleContext; -using RoleContextPtr = std::shared_ptr; -class RoleContextFactory; -class RowPolicyContext; -using RowPolicyContextPtr = std::shared_ptr; -class RowPolicyContextFactory; -class QuotaContext; -using QuotaContextPtr = std::shared_ptr; -class QuotaContextFactory; +class EnabledRoles; +class RoleCache; +class EnabledRowPolicies; +class RowPolicyCache; +class EnabledQuota; +class QuotaCache; struct QuotaUsageInfo; class ClientInfo; struct Settings; @@ -48,7 +43,7 @@ public: void setLocalDirectory(const String & directory); void setUsersConfig(const Poco::Util::AbstractConfiguration & users_config); - AccessRightsContextPtr getAccessRightsContext( + std::shared_ptr getContextAccess( const UUID & user_id, const std::vector & current_roles, bool use_default_roles, @@ -56,17 +51,17 @@ public: const String & current_database, const ClientInfo & client_info) const; - RoleContextPtr getRoleContext( + std::shared_ptr getEnabledRoles( const std::vector & current_roles, const std::vector & current_roles_with_admin_option) const; - RowPolicyContextPtr getRowPolicyContext( + std::shared_ptr getEnabledRowPolicies( const UUID & user_id, const std::vector & enabled_roles) const; - QuotaContextPtr getQuotaContext( - const String & user_name, + std::shared_ptr getEnabledQuota( const UUID & user_id, + const String & user_name, const std::vector & enabled_roles, const Poco::Net::IPAddress & address, const String & custom_quota_key) const; @@ -74,10 +69,11 @@ public: std::vector getQuotaUsageInfo() const; private: - std::unique_ptr access_rights_context_factory; - std::unique_ptr role_context_factory; - std::unique_ptr row_policy_context_factory; - std::unique_ptr quota_context_factory; + class ContextAccessCache; + std::unique_ptr context_access_cache; + std::unique_ptr role_cache; + std::unique_ptr row_policy_cache; + std::unique_ptr quota_cache; }; } diff --git a/dbms/src/Access/AccessRights.h b/dbms/src/Access/AccessRights.h index 48ffa0bb616..133038f2d44 100644 --- a/dbms/src/Access/AccessRights.h +++ b/dbms/src/Access/AccessRights.h @@ -28,23 +28,23 @@ public: /// Grants access on a specified database/table/column. /// Does nothing if the specified access has been already granted. - void grant(const AccessFlags & access); - void grant(const AccessFlags & access, const std::string_view & database); - void grant(const AccessFlags & access, const std::string_view & database, const std::string_view & table); - void grant(const AccessFlags & access, const std::string_view & database, const std::string_view & table, const std::string_view & column); - void grant(const AccessFlags & access, const std::string_view & database, const std::string_view & table, const std::vector & columns); - void grant(const AccessFlags & access, const std::string_view & database, const std::string_view & table, const Strings & columns); + void grant(const AccessFlags & flags); + void grant(const AccessFlags & flags, const std::string_view & database); + void grant(const AccessFlags & flags, const std::string_view & database, const std::string_view & table); + void grant(const AccessFlags & flags, const std::string_view & database, const std::string_view & table, const std::string_view & column); + void grant(const AccessFlags & flags, const std::string_view & database, const std::string_view & table, const std::vector & columns); + void grant(const AccessFlags & flags, const std::string_view & database, const std::string_view & table, const Strings & columns); void grant(const AccessRightsElement & element, std::string_view current_database = {}); void grant(const AccessRightsElements & elements, std::string_view current_database = {}); /// Revokes a specified access granted earlier on a specified database/table/column. /// For example, revoke(AccessType::ALL) revokes all grants at all, just like clear(); - void revoke(const AccessFlags & access); - void revoke(const AccessFlags & access, const std::string_view & database); - void revoke(const AccessFlags & access, const std::string_view & database, const std::string_view & table); - void revoke(const AccessFlags & access, const std::string_view & database, const std::string_view & table, const std::string_view & column); - void revoke(const AccessFlags & access, const std::string_view & database, const std::string_view & table, const std::vector & columns); - void revoke(const AccessFlags & access, const std::string_view & database, const std::string_view & table, const Strings & columns); + void revoke(const AccessFlags & flags); + void revoke(const AccessFlags & flags, const std::string_view & database); + void revoke(const AccessFlags & flags, const std::string_view & database, const std::string_view & table); + void revoke(const AccessFlags & flags, const std::string_view & database, const std::string_view & table, const std::string_view & column); + void revoke(const AccessFlags & flags, const std::string_view & database, const std::string_view & table, const std::vector & columns); + void revoke(const AccessFlags & flags, const std::string_view & database, const std::string_view & table, const Strings & columns); void revoke(const AccessRightsElement & element, std::string_view current_database = {}); void revoke(const AccessRightsElements & elements, std::string_view current_database = {}); diff --git a/dbms/src/Access/AccessRightsContext.cpp b/dbms/src/Access/AccessRightsContext.cpp deleted file mode 100644 index 6eff1f050b3..00000000000 --- a/dbms/src/Access/AccessRightsContext.cpp +++ /dev/null @@ -1,585 +0,0 @@ -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include - - -namespace DB -{ -namespace ErrorCodes -{ - extern const int ACCESS_DENIED; - extern const int READONLY; - extern const int QUERY_IS_PROHIBITED; - extern const int FUNCTION_NOT_ALLOWED; - extern const int UNKNOWN_USER; -} - - -namespace -{ - enum CheckAccessRightsMode - { - RETURN_FALSE_IF_ACCESS_DENIED, - LOG_WARNING_IF_ACCESS_DENIED, - THROW_IF_ACCESS_DENIED, - }; - - - String formatSkippedMessage() - { - return ""; - } - - String formatSkippedMessage(const std::string_view & database) - { - return ". Skipped database " + backQuoteIfNeed(database); - } - - String formatSkippedMessage(const std::string_view & database, const std::string_view & table) - { - String str = ". Skipped table "; - if (!database.empty()) - str += backQuoteIfNeed(database) + "."; - str += backQuoteIfNeed(table); - return str; - } - - String formatSkippedMessage(const std::string_view & database, const std::string_view & table, const std::string_view & column) - { - String str = ". Skipped column " + backQuoteIfNeed(column) + " ON "; - if (!database.empty()) - str += backQuoteIfNeed(database) + "."; - str += backQuoteIfNeed(table); - return str; - } - - template - String formatSkippedMessage(const std::string_view & database, const std::string_view & table, const std::vector & columns) - { - if (columns.size() == 1) - return formatSkippedMessage(database, table, columns[0]); - - String str = ". Skipped columns "; - bool need_comma = false; - for (const auto & column : columns) - { - if (std::exchange(need_comma, true)) - str += ", "; - str += backQuoteIfNeed(column); - } - str += " ON "; - if (!database.empty()) - str += backQuoteIfNeed(database) + "."; - str += backQuoteIfNeed(table); - return str; - } -} - - -AccessRightsContext::AccessRightsContext() -{ - auto everything_granted = boost::make_shared(); - everything_granted->grant(AccessType::ALL); - boost::range::fill(result_access_cache, everything_granted); - - enabled_roles_with_admin_option = boost::make_shared>(); - - row_policy_context = std::make_shared(); - quota_context = std::make_shared(); -} - - -AccessRightsContext::AccessRightsContext(const AccessControlManager & manager_, const Params & params_) - : manager(&manager_) - , params(params_) -{ - subscription_for_user_change = manager->subscribeForChanges( - *params.user_id, [this](const UUID &, const AccessEntityPtr & entity) - { - UserPtr changed_user = entity ? typeid_cast(entity) : nullptr; - std::lock_guard lock{mutex}; - setUser(changed_user); - }); - - setUser(manager->read(*params.user_id)); -} - - -void AccessRightsContext::setUser(const UserPtr & user_) const -{ - user = user_; - if (!user) - { - /// User has been dropped. - auto nothing_granted = boost::make_shared(); - boost::range::fill(result_access_cache, nothing_granted); - subscription_for_user_change = {}; - subscription_for_roles_info_change = {}; - role_context = nullptr; - enabled_roles_with_admin_option = boost::make_shared>(); - row_policy_context = std::make_shared(); - quota_context = std::make_shared(); - return; - } - - user_name = user->getName(); - trace_log = &Poco::Logger::get("AccessRightsContext (" + user_name + ")"); - - std::vector current_roles, current_roles_with_admin_option; - if (params.use_default_roles) - { - for (const UUID & id : user->granted_roles) - { - if (user->default_roles.match(id)) - current_roles.push_back(id); - } - boost::range::set_intersection(current_roles, user->granted_roles_with_admin_option, - std::back_inserter(current_roles_with_admin_option)); - } - else - { - current_roles.reserve(params.current_roles.size()); - for (const auto & id : params.current_roles) - { - if (user->granted_roles.contains(id)) - current_roles.push_back(id); - if (user->granted_roles_with_admin_option.contains(id)) - current_roles_with_admin_option.push_back(id); - } - } - - subscription_for_roles_info_change = {}; - role_context = manager->getRoleContext(current_roles, current_roles_with_admin_option); - subscription_for_roles_info_change = role_context->subscribeForChanges([this](const CurrentRolesInfoPtr & roles_info_) - { - std::lock_guard lock{mutex}; - setRolesInfo(roles_info_); - }); - - setRolesInfo(role_context->getInfo()); -} - - -void AccessRightsContext::setRolesInfo(const CurrentRolesInfoPtr & roles_info_) const -{ - assert(roles_info_); - roles_info = roles_info_; - enabled_roles_with_admin_option.store(nullptr /* need to recalculate */); - boost::range::fill(result_access_cache, nullptr /* need recalculate */); - row_policy_context = manager->getRowPolicyContext(*params.user_id, roles_info->enabled_roles); - quota_context = manager->getQuotaContext(user_name, *params.user_id, roles_info->enabled_roles, params.address, params.quota_key); -} - - -bool AccessRightsContext::isCorrectPassword(const String & password) const -{ - std::lock_guard lock{mutex}; - if (!user) - return false; - return user->authentication.isCorrectPassword(password); -} - -bool AccessRightsContext::isClientHostAllowed() const -{ - std::lock_guard lock{mutex}; - if (!user) - return false; - return user->allowed_client_hosts.contains(params.address); -} - - -template -bool AccessRightsContext::checkAccessImpl(Poco::Logger * log_, const AccessFlags & access, const Args &... args) const -{ - auto result_access = calculateResultAccess(grant_option); - bool is_granted = result_access->isGranted(access, args...); - - if (trace_log) - LOG_TRACE(trace_log, "Access " << (is_granted ? "granted" : "denied") << ": " << (AccessRightsElement{access, args...}.toString())); - - if (is_granted) - return true; - - if constexpr (mode == RETURN_FALSE_IF_ACCESS_DENIED) - return false; - - if constexpr (mode == LOG_WARNING_IF_ACCESS_DENIED) - { - if (!log_) - return false; - } - - auto show_error = [&](const String & msg, [[maybe_unused]] int error_code) - { - if constexpr (mode == THROW_IF_ACCESS_DENIED) - throw Exception(user_name + ": " + msg, error_code); - else if constexpr (mode == LOG_WARNING_IF_ACCESS_DENIED) - LOG_WARNING(log_, user_name + ": " + msg + formatSkippedMessage(args...)); - }; - - if (!user) - { - show_error("User has been dropped", ErrorCodes::UNKNOWN_USER); - } - else if (grant_option && calculateResultAccess(false, params.readonly, params.allow_ddl, params.allow_introspection)->isGranted(access, args...)) - { - show_error( - "Not enough privileges. " - "The required privileges have been granted, but without grant option. " - "To execute this query it's necessary to have the grant " - + AccessRightsElement{access, args...}.toString() + " WITH GRANT OPTION", - ErrorCodes::ACCESS_DENIED); - } - else if (params.readonly && calculateResultAccess(false, false, params.allow_ddl, params.allow_introspection)->isGranted(access, args...)) - { - if (params.interface == ClientInfo::Interface::HTTP && params.http_method == ClientInfo::HTTPMethod::GET) - show_error( - "Cannot execute query in readonly mode. " - "For queries over HTTP, method GET implies readonly. You should use method POST for modifying queries", - ErrorCodes::READONLY); - else - show_error("Cannot execute query in readonly mode", ErrorCodes::READONLY); - } - else if (!params.allow_ddl && calculateResultAccess(false, params.readonly, true, params.allow_introspection)->isGranted(access, args...)) - { - show_error("Cannot execute query. DDL queries are prohibited for the user", ErrorCodes::QUERY_IS_PROHIBITED); - } - else if (!params.allow_introspection && calculateResultAccess(false, params.readonly, params.allow_ddl, true)->isGranted(access, args...)) - { - show_error("Introspection functions are disabled, because setting 'allow_introspection_functions' is set to 0", ErrorCodes::FUNCTION_NOT_ALLOWED); - } - else - { - show_error( - "Not enough privileges. To execute this query it's necessary to have the grant " - + AccessRightsElement{access, args...}.toString() + (grant_option ? " WITH GRANT OPTION" : ""), - ErrorCodes::ACCESS_DENIED); - } - - return false; -} - - -template -bool AccessRightsContext::checkAccessImpl(Poco::Logger * log_, const AccessRightsElement & element) const -{ - if (element.any_database) - { - return checkAccessImpl(log_, element.access_flags); - } - else if (element.any_table) - { - if (element.database.empty()) - return checkAccessImpl(log_, element.access_flags, params.current_database); - else - return checkAccessImpl(log_, element.access_flags, element.database); - } - else if (element.any_column) - { - if (element.database.empty()) - return checkAccessImpl(log_, element.access_flags, params.current_database, element.table); - else - return checkAccessImpl(log_, element.access_flags, element.database, element.table); - } - else - { - if (element.database.empty()) - return checkAccessImpl(log_, element.access_flags, params.current_database, element.table, element.columns); - else - return checkAccessImpl(log_, element.access_flags, element.database, element.table, element.columns); - } -} - - -template -bool AccessRightsContext::checkAccessImpl(Poco::Logger * log_, const AccessRightsElements & elements) const -{ - for (const auto & element : elements) - if (!checkAccessImpl(log_, element)) - return false; - return true; -} - - -void AccessRightsContext::checkAccess(const AccessFlags & access) const { checkAccessImpl(nullptr, access); } -void AccessRightsContext::checkAccess(const AccessFlags & access, const std::string_view & database) const { checkAccessImpl(nullptr, access, database); } -void AccessRightsContext::checkAccess(const AccessFlags & access, const std::string_view & database, const std::string_view & table) const { checkAccessImpl(nullptr, access, database, table); } -void AccessRightsContext::checkAccess(const AccessFlags & access, const std::string_view & database, const std::string_view & table, const std::string_view & column) const { checkAccessImpl(nullptr, access, database, table, column); } -void AccessRightsContext::checkAccess(const AccessFlags & access, const std::string_view & database, const std::string_view & table, const std::vector & columns) const { checkAccessImpl(nullptr, access, database, table, columns); } -void AccessRightsContext::checkAccess(const AccessFlags & access, const std::string_view & database, const std::string_view & table, const Strings & columns) const { checkAccessImpl(nullptr, access, database, table, columns); } -void AccessRightsContext::checkAccess(const AccessRightsElement & access) const { checkAccessImpl(nullptr, access); } -void AccessRightsContext::checkAccess(const AccessRightsElements & access) const { checkAccessImpl(nullptr, access); } - -bool AccessRightsContext::isGranted(const AccessFlags & access) const { return checkAccessImpl(nullptr, access); } -bool AccessRightsContext::isGranted(const AccessFlags & access, const std::string_view & database) const { return checkAccessImpl(nullptr, access, database); } -bool AccessRightsContext::isGranted(const AccessFlags & access, const std::string_view & database, const std::string_view & table) const { return checkAccessImpl(nullptr, access, database, table); } -bool AccessRightsContext::isGranted(const AccessFlags & access, const std::string_view & database, const std::string_view & table, const std::string_view & column) const { return checkAccessImpl(nullptr, access, database, table, column); } -bool AccessRightsContext::isGranted(const AccessFlags & access, const std::string_view & database, const std::string_view & table, const std::vector & columns) const { return checkAccessImpl(nullptr, access, database, table, columns); } -bool AccessRightsContext::isGranted(const AccessFlags & access, const std::string_view & database, const std::string_view & table, const Strings & columns) const { return checkAccessImpl(nullptr, access, database, table, columns); } -bool AccessRightsContext::isGranted(const AccessRightsElement & access) const { return checkAccessImpl(nullptr, access); } -bool AccessRightsContext::isGranted(const AccessRightsElements & access) const { return checkAccessImpl(nullptr, access); } - -bool AccessRightsContext::isGranted(Poco::Logger * log_, const AccessFlags & access) const { return checkAccessImpl(log_, access); } -bool AccessRightsContext::isGranted(Poco::Logger * log_, const AccessFlags & access, const std::string_view & database) const { return checkAccessImpl(log_, access, database); } -bool AccessRightsContext::isGranted(Poco::Logger * log_, const AccessFlags & access, const std::string_view & database, const std::string_view & table) const { return checkAccessImpl(log_, access, database, table); } -bool AccessRightsContext::isGranted(Poco::Logger * log_, const AccessFlags & access, const std::string_view & database, const std::string_view & table, const std::string_view & column) const { return checkAccessImpl(log_, access, database, table, column); } -bool AccessRightsContext::isGranted(Poco::Logger * log_, const AccessFlags & access, const std::string_view & database, const std::string_view & table, const std::vector & columns) const { return checkAccessImpl(log_, access, database, table, columns); } -bool AccessRightsContext::isGranted(Poco::Logger * log_, const AccessFlags & access, const std::string_view & database, const std::string_view & table, const Strings & columns) const { return checkAccessImpl(log_, access, database, table, columns); } -bool AccessRightsContext::isGranted(Poco::Logger * log_, const AccessRightsElement & access) const { return checkAccessImpl(log_, access); } -bool AccessRightsContext::isGranted(Poco::Logger * log_, const AccessRightsElements & access) const { return checkAccessImpl(log_, access); } - -void AccessRightsContext::checkGrantOption(const AccessFlags & access) const { checkAccessImpl(nullptr, access); } -void AccessRightsContext::checkGrantOption(const AccessFlags & access, const std::string_view & database) const { checkAccessImpl(nullptr, access, database); } -void AccessRightsContext::checkGrantOption(const AccessFlags & access, const std::string_view & database, const std::string_view & table) const { checkAccessImpl(nullptr, access, database, table); } -void AccessRightsContext::checkGrantOption(const AccessFlags & access, const std::string_view & database, const std::string_view & table, const std::string_view & column) const { checkAccessImpl(nullptr, access, database, table, column); } -void AccessRightsContext::checkGrantOption(const AccessFlags & access, const std::string_view & database, const std::string_view & table, const std::vector & columns) const { checkAccessImpl(nullptr, access, database, table, columns); } -void AccessRightsContext::checkGrantOption(const AccessFlags & access, const std::string_view & database, const std::string_view & table, const Strings & columns) const { checkAccessImpl(nullptr, access, database, table, columns); } -void AccessRightsContext::checkGrantOption(const AccessRightsElement & access) const { checkAccessImpl(nullptr, access); } -void AccessRightsContext::checkGrantOption(const AccessRightsElements & access) const { checkAccessImpl(nullptr, access); } - - -void AccessRightsContext::checkAdminOption(const UUID & role_id) const -{ - if (isGranted(AccessType::ROLE_ADMIN)) - return; - - boost::shared_ptr> enabled_roles = enabled_roles_with_admin_option.load(); - if (!enabled_roles) - { - std::lock_guard lock{mutex}; - enabled_roles = enabled_roles_with_admin_option.load(); - if (!enabled_roles) - { - if (roles_info) - enabled_roles = boost::make_shared>(roles_info->enabled_roles_with_admin_option.begin(), roles_info->enabled_roles_with_admin_option.end()); - else - enabled_roles = boost::make_shared>(); - enabled_roles_with_admin_option.store(enabled_roles); - } - } - - if (enabled_roles->contains(role_id)) - return; - - std::optional role_name = manager->readName(role_id); - if (!role_name) - role_name = "ID {" + toString(role_id) + "}"; - throw Exception( - getUserName() + ": Not enough privileges. To execute this query it's necessary to have the grant " + backQuoteIfNeed(*role_name) - + " WITH ADMIN OPTION ", - ErrorCodes::ACCESS_DENIED); -} - - -boost::shared_ptr AccessRightsContext::calculateResultAccess(bool grant_option) const -{ - return calculateResultAccess(grant_option, params.readonly, params.allow_ddl, params.allow_introspection); -} - - -boost::shared_ptr AccessRightsContext::calculateResultAccess(bool grant_option, UInt64 readonly_, bool allow_ddl_, bool allow_introspection_) const -{ - size_t cache_index = static_cast(readonly_ != params.readonly) - + static_cast(allow_ddl_ != params.allow_ddl) * 2 + - + static_cast(allow_introspection_ != params.allow_introspection) * 3 - + static_cast(grant_option) * 4; - assert(cache_index < std::size(result_access_cache)); - auto cached = result_access_cache[cache_index].load(); - if (cached) - return cached; - - std::lock_guard lock{mutex}; - cached = result_access_cache[cache_index].load(); - if (cached) - return cached; - - auto result_ptr = boost::make_shared(); - auto & result = *result_ptr; - - if (grant_option) - { - result = user->access_with_grant_option; - if (roles_info) - result.merge(roles_info->access_with_grant_option); - } - else - { - result = user->access; - if (roles_info) - result.merge(roles_info->access); - } - - static const AccessFlags table_ddl = AccessType::CREATE_DATABASE | AccessType::CREATE_TABLE | AccessType::CREATE_VIEW - | AccessType::ALTER_TABLE | AccessType::ALTER_VIEW | AccessType::DROP_DATABASE | AccessType::DROP_TABLE | AccessType::DROP_VIEW - | AccessType::TRUNCATE; - static const AccessFlags dictionary_ddl = AccessType::CREATE_DICTIONARY | AccessType::DROP_DICTIONARY; - static const AccessFlags table_and_dictionary_ddl = table_ddl | dictionary_ddl; - static const AccessFlags write_table_access = AccessType::INSERT | AccessType::OPTIMIZE; - static const AccessFlags all_dcl = AccessType::CREATE_USER | AccessType::CREATE_ROLE | AccessType::CREATE_POLICY - | AccessType::CREATE_QUOTA | AccessType::ALTER_USER | AccessType::ALTER_POLICY | AccessType::ALTER_QUOTA | AccessType::DROP_USER - | AccessType::DROP_ROLE | AccessType::DROP_POLICY | AccessType::DROP_QUOTA | AccessType::ROLE_ADMIN; - - if (readonly_) - result.revoke(write_table_access | all_dcl | AccessType::SYSTEM | AccessType::KILL_QUERY); - - if (readonly_ || !allow_ddl_) - result.revoke(table_and_dictionary_ddl); - - if (readonly_ == 1) - { - /// Table functions are forbidden in readonly mode. - /// For example, for readonly = 2 - allowed. - result.revoke(AccessType::CREATE_TEMPORARY_TABLE | AccessType::TABLE_FUNCTIONS); - } - - if (!allow_introspection_) - result.revoke(AccessType::INTROSPECTION); - - /// Anyone has access to the "system" database. - result.grant(AccessType::SELECT, DatabaseCatalog::SYSTEM_DATABASE); - - if (readonly_ != 1) - { - /// User has access to temporary or external table if such table was resolved in session or query context - result.grant(AccessFlags::allTableFlags() | AccessFlags::allColumnFlags(), DatabaseCatalog::TEMPORARY_DATABASE); - } - - if (readonly_ && grant_option) - { - /// No grant option in readonly mode. - result.revoke(AccessType::ALL); - } - - result_access_cache[cache_index].store(result_ptr); - - if (trace_log && (params.readonly == readonly_) && (params.allow_ddl == allow_ddl_) && (params.allow_introspection == allow_introspection_)) - { - LOG_TRACE(trace_log, "List of all grants: " << result_ptr->toString() << (grant_option ? " WITH GRANT OPTION" : "")); - if (roles_info && !roles_info->getCurrentRolesNames().empty()) - { - LOG_TRACE( - trace_log, - "Current_roles: " << boost::algorithm::join(roles_info->getCurrentRolesNames(), ", ") - << ", enabled_roles: " << boost::algorithm::join(roles_info->getEnabledRolesNames(), ", ")); - } - } - - return result_ptr; -} - - -UserPtr AccessRightsContext::getUser() const -{ - std::lock_guard lock{mutex}; - return user; -} - -String AccessRightsContext::getUserName() const -{ - std::lock_guard lock{mutex}; - return user_name; -} - -CurrentRolesInfoPtr AccessRightsContext::getRolesInfo() const -{ - std::lock_guard lock{mutex}; - return roles_info; -} - -std::vector AccessRightsContext::getCurrentRoles() const -{ - std::lock_guard lock{mutex}; - return roles_info ? roles_info->current_roles : std::vector{}; -} - -Strings AccessRightsContext::getCurrentRolesNames() const -{ - std::lock_guard lock{mutex}; - return roles_info ? roles_info->getCurrentRolesNames() : Strings{}; -} - -std::vector AccessRightsContext::getEnabledRoles() const -{ - std::lock_guard lock{mutex}; - return roles_info ? roles_info->enabled_roles : std::vector{}; -} - -Strings AccessRightsContext::getEnabledRolesNames() const -{ - std::lock_guard lock{mutex}; - return roles_info ? roles_info->getEnabledRolesNames() : Strings{}; -} - -RowPolicyContextPtr AccessRightsContext::getRowPolicy() const -{ - std::lock_guard lock{mutex}; - return row_policy_context; -} - -QuotaContextPtr AccessRightsContext::getQuota() const -{ - std::lock_guard lock{mutex}; - return quota_context; -} - - -bool operator <(const AccessRightsContext::Params & lhs, const AccessRightsContext::Params & rhs) -{ -#define ACCESS_RIGHTS_CONTEXT_PARAMS_COMPARE_HELPER(field) \ - if (lhs.field < rhs.field) \ - return true; \ - if (lhs.field > rhs.field) \ - return false - ACCESS_RIGHTS_CONTEXT_PARAMS_COMPARE_HELPER(user_id); - ACCESS_RIGHTS_CONTEXT_PARAMS_COMPARE_HELPER(current_roles); - ACCESS_RIGHTS_CONTEXT_PARAMS_COMPARE_HELPER(use_default_roles); - ACCESS_RIGHTS_CONTEXT_PARAMS_COMPARE_HELPER(address); - ACCESS_RIGHTS_CONTEXT_PARAMS_COMPARE_HELPER(quota_key); - ACCESS_RIGHTS_CONTEXT_PARAMS_COMPARE_HELPER(current_database); - ACCESS_RIGHTS_CONTEXT_PARAMS_COMPARE_HELPER(readonly); - ACCESS_RIGHTS_CONTEXT_PARAMS_COMPARE_HELPER(allow_ddl); - ACCESS_RIGHTS_CONTEXT_PARAMS_COMPARE_HELPER(allow_introspection); - ACCESS_RIGHTS_CONTEXT_PARAMS_COMPARE_HELPER(interface); - ACCESS_RIGHTS_CONTEXT_PARAMS_COMPARE_HELPER(http_method); - return false; -#undef ACCESS_RIGHTS_CONTEXT_PARAMS_COMPARE_HELPER -} - - -bool operator ==(const AccessRightsContext::Params & lhs, const AccessRightsContext::Params & rhs) -{ -#define ACCESS_RIGHTS_CONTEXT_PARAMS_COMPARE_HELPER(field) \ - if (lhs.field != rhs.field) \ - return false - ACCESS_RIGHTS_CONTEXT_PARAMS_COMPARE_HELPER(user_id); - ACCESS_RIGHTS_CONTEXT_PARAMS_COMPARE_HELPER(current_roles); - ACCESS_RIGHTS_CONTEXT_PARAMS_COMPARE_HELPER(use_default_roles); - ACCESS_RIGHTS_CONTEXT_PARAMS_COMPARE_HELPER(address); - ACCESS_RIGHTS_CONTEXT_PARAMS_COMPARE_HELPER(quota_key); - ACCESS_RIGHTS_CONTEXT_PARAMS_COMPARE_HELPER(current_database); - ACCESS_RIGHTS_CONTEXT_PARAMS_COMPARE_HELPER(readonly); - ACCESS_RIGHTS_CONTEXT_PARAMS_COMPARE_HELPER(allow_ddl); - ACCESS_RIGHTS_CONTEXT_PARAMS_COMPARE_HELPER(allow_introspection); - ACCESS_RIGHTS_CONTEXT_PARAMS_COMPARE_HELPER(interface); - ACCESS_RIGHTS_CONTEXT_PARAMS_COMPARE_HELPER(http_method); - return true; -#undef ACCESS_RIGHTS_CONTEXT_PARAMS_COMPARE_HELPER -} - -} diff --git a/dbms/src/Access/AccessRightsContext.h b/dbms/src/Access/AccessRightsContext.h deleted file mode 100644 index 8fc5066cfe4..00000000000 --- a/dbms/src/Access/AccessRightsContext.h +++ /dev/null @@ -1,157 +0,0 @@ -#pragma once - -#include -#include -#include -#include -#include -#include -#include -#include - - -namespace Poco { class Logger; } - -namespace DB -{ -struct User; -using UserPtr = std::shared_ptr; -struct CurrentRolesInfo; -using CurrentRolesInfoPtr = std::shared_ptr; -class RoleContext; -using RoleContextPtr = std::shared_ptr; -class RowPolicyContext; -using RowPolicyContextPtr = std::shared_ptr; -class QuotaContext; -using QuotaContextPtr = std::shared_ptr; -struct Settings; -class AccessControlManager; - - -class AccessRightsContext -{ -public: - struct Params - { - std::optional user_id; - std::vector current_roles; - bool use_default_roles = false; - UInt64 readonly = 0; - bool allow_ddl = false; - bool allow_introspection = false; - String current_database; - ClientInfo::Interface interface = ClientInfo::Interface::TCP; - ClientInfo::HTTPMethod http_method = ClientInfo::HTTPMethod::UNKNOWN; - Poco::Net::IPAddress address; - String quota_key; - - friend bool operator ==(const Params & lhs, const Params & rhs); - friend bool operator !=(const Params & lhs, const Params & rhs) { return !(lhs == rhs); } - friend bool operator <(const Params & lhs, const Params & rhs); - friend bool operator >(const Params & lhs, const Params & rhs) { return rhs < lhs; } - friend bool operator <=(const Params & lhs, const Params & rhs) { return !(rhs < lhs); } - friend bool operator >=(const Params & lhs, const Params & rhs) { return !(lhs < rhs); } - }; - - /// Default constructor creates access rights' context which allows everything. - AccessRightsContext(); - - const Params & getParams() const { return params; } - UserPtr getUser() const; - String getUserName() const; - - bool isCorrectPassword(const String & password) const; - bool isClientHostAllowed() const; - - CurrentRolesInfoPtr getRolesInfo() const; - std::vector getCurrentRoles() const; - Strings getCurrentRolesNames() const; - std::vector getEnabledRoles() const; - Strings getEnabledRolesNames() const; - - RowPolicyContextPtr getRowPolicy() const; - QuotaContextPtr getQuota() const; - - /// Checks if a specified access is granted, and throws an exception if not. - /// Empty database means the current database. - void checkAccess(const AccessFlags & access) const; - void checkAccess(const AccessFlags & access, const std::string_view & database) const; - void checkAccess(const AccessFlags & access, const std::string_view & database, const std::string_view & table) const; - void checkAccess(const AccessFlags & access, const std::string_view & database, const std::string_view & table, const std::string_view & column) const; - void checkAccess(const AccessFlags & access, const std::string_view & database, const std::string_view & table, const std::vector & columns) const; - void checkAccess(const AccessFlags & access, const std::string_view & database, const std::string_view & table, const Strings & columns) const; - void checkAccess(const AccessRightsElement & access) const; - void checkAccess(const AccessRightsElements & access) const; - - /// Checks if a specified access is granted. - bool isGranted(const AccessFlags & access) const; - bool isGranted(const AccessFlags & access, const std::string_view & database) const; - bool isGranted(const AccessFlags & access, const std::string_view & database, const std::string_view & table) const; - bool isGranted(const AccessFlags & access, const std::string_view & database, const std::string_view & table, const std::string_view & column) const; - bool isGranted(const AccessFlags & access, const std::string_view & database, const std::string_view & table, const std::vector & columns) const; - bool isGranted(const AccessFlags & access, const std::string_view & database, const std::string_view & table, const Strings & columns) const; - bool isGranted(const AccessRightsElement & access) const; - bool isGranted(const AccessRightsElements & access) const; - - /// Checks if a specified access is granted, and logs a warning if not. - bool isGranted(Poco::Logger * log_, const AccessFlags & access) const; - bool isGranted(Poco::Logger * log_, const AccessFlags & access, const std::string_view & database) const; - bool isGranted(Poco::Logger * log_, const AccessFlags & access, const std::string_view & database, const std::string_view & table) const; - bool isGranted(Poco::Logger * log_, const AccessFlags & access, const std::string_view & database, const std::string_view & table, const std::string_view & column) const; - bool isGranted(Poco::Logger * log_, const AccessFlags & access, const std::string_view & database, const std::string_view & table, const std::vector & columns) const; - bool isGranted(Poco::Logger * log_, const AccessFlags & access, const std::string_view & database, const std::string_view & table, const Strings & columns) const; - bool isGranted(Poco::Logger * log_, const AccessRightsElement & access) const; - bool isGranted(Poco::Logger * log_, const AccessRightsElements & access) const; - - /// Checks if a specified access is granted with grant option, and throws an exception if not. - void checkGrantOption(const AccessFlags & access) const; - void checkGrantOption(const AccessFlags & access, const std::string_view & database) const; - void checkGrantOption(const AccessFlags & access, const std::string_view & database, const std::string_view & table) const; - void checkGrantOption(const AccessFlags & access, const std::string_view & database, const std::string_view & table, const std::string_view & column) const; - void checkGrantOption(const AccessFlags & access, const std::string_view & database, const std::string_view & table, const std::vector & columns) const; - void checkGrantOption(const AccessFlags & access, const std::string_view & database, const std::string_view & table, const Strings & columns) const; - void checkGrantOption(const AccessRightsElement & access) const; - void checkGrantOption(const AccessRightsElements & access) const; - - /// Checks if a specified role is granted with admin option, and throws an exception if not. - void checkAdminOption(const UUID & role_id) const; - -private: - friend class AccessRightsContextFactory; - friend struct ext::shared_ptr_helper; - AccessRightsContext(const AccessControlManager & manager_, const Params & params_); /// AccessRightsContext should be created by AccessRightsContextFactory. - - void setUser(const UserPtr & user_) const; - void setRolesInfo(const CurrentRolesInfoPtr & roles_info_) const; - - template - bool checkAccessImpl(Poco::Logger * log_, const AccessFlags & access, const Args &... args) const; - - template - bool checkAccessImpl(Poco::Logger * log_, const AccessRightsElement & element) const; - - template - bool checkAccessImpl(Poco::Logger * log_, const AccessRightsElements & elements) const; - - boost::shared_ptr calculateResultAccess(bool grant_option) const; - boost::shared_ptr calculateResultAccess(bool grant_option, UInt64 readonly_, bool allow_ddl_, bool allow_introspection_) const; - - const AccessControlManager * manager = nullptr; - const Params params; - mutable Poco::Logger * trace_log = nullptr; - mutable UserPtr user; - mutable String user_name; - mutable ext::scope_guard subscription_for_user_change; - mutable RoleContextPtr role_context; - mutable ext::scope_guard subscription_for_roles_info_change; - mutable CurrentRolesInfoPtr roles_info; - mutable boost::atomic_shared_ptr> enabled_roles_with_admin_option; - mutable boost::atomic_shared_ptr result_access_cache[7]; - mutable RowPolicyContextPtr row_policy_context; - mutable QuotaContextPtr quota_context; - mutable std::mutex mutex; -}; - -using AccessRightsContextPtr = std::shared_ptr; - -} diff --git a/dbms/src/Access/AccessRightsContextFactory.cpp b/dbms/src/Access/AccessRightsContextFactory.cpp deleted file mode 100644 index 8d542a5f439..00000000000 --- a/dbms/src/Access/AccessRightsContextFactory.cpp +++ /dev/null @@ -1,48 +0,0 @@ -#include -#include -#include - - -namespace DB -{ -AccessRightsContextFactory::AccessRightsContextFactory(const AccessControlManager & manager_) - : manager(manager_), cache(600000 /* 10 minutes */) {} - -AccessRightsContextFactory::~AccessRightsContextFactory() = default; - - -AccessRightsContextPtr AccessRightsContextFactory::createContext(const Params & params) -{ - std::lock_guard lock{mutex}; - auto x = cache.get(params); - if (x) - return *x; - auto res = ext::shared_ptr_helper::create(manager, params); - cache.add(params, res); - return res; -} - -AccessRightsContextPtr AccessRightsContextFactory::createContext( - const UUID & user_id, - const std::vector & current_roles, - bool use_default_roles, - const Settings & settings, - const String & current_database, - const ClientInfo & client_info) -{ - Params params; - params.user_id = user_id; - params.current_roles = current_roles; - params.use_default_roles = use_default_roles; - params.current_database = current_database; - params.readonly = settings.readonly; - params.allow_ddl = settings.allow_ddl; - params.allow_introspection = settings.allow_introspection_functions; - params.interface = client_info.interface; - params.http_method = client_info.http_method; - params.address = client_info.current_address.host(); - params.quota_key = client_info.quota_key; - return createContext(params); -} - -} diff --git a/dbms/src/Access/AccessRightsContextFactory.h b/dbms/src/Access/AccessRightsContextFactory.h deleted file mode 100644 index c480307757a..00000000000 --- a/dbms/src/Access/AccessRightsContextFactory.h +++ /dev/null @@ -1,29 +0,0 @@ -#pragma once - -#include -#include -#include - - -namespace DB -{ -class AccessControlManager; - - -class AccessRightsContextFactory -{ -public: - AccessRightsContextFactory(const AccessControlManager & manager_); - ~AccessRightsContextFactory(); - - using Params = AccessRightsContext::Params; - AccessRightsContextPtr createContext(const Params & params); - AccessRightsContextPtr createContext(const UUID & user_id, const std::vector & current_roles, bool use_default_roles, const Settings & settings, const String & current_database, const ClientInfo & client_info); - -private: - const AccessControlManager & manager; - Poco::ExpireCache cache; - std::mutex mutex; -}; - -} diff --git a/dbms/src/Access/ContextAccess.cpp b/dbms/src/Access/ContextAccess.cpp new file mode 100644 index 00000000000..4867694396c --- /dev/null +++ b/dbms/src/Access/ContextAccess.cpp @@ -0,0 +1,535 @@ +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + + +namespace DB +{ +namespace ErrorCodes +{ + extern const int ACCESS_DENIED; + extern const int READONLY; + extern const int QUERY_IS_PROHIBITED; + extern const int FUNCTION_NOT_ALLOWED; + extern const int UNKNOWN_USER; +} + + +namespace +{ + enum CheckAccessRightsMode + { + RETURN_FALSE_IF_ACCESS_DENIED, + LOG_WARNING_IF_ACCESS_DENIED, + THROW_IF_ACCESS_DENIED, + }; + + + String formatSkippedMessage() + { + return ""; + } + + String formatSkippedMessage(const std::string_view & database) + { + return ". Skipped database " + backQuoteIfNeed(database); + } + + String formatSkippedMessage(const std::string_view & database, const std::string_view & table) + { + String str = ". Skipped table "; + if (!database.empty()) + str += backQuoteIfNeed(database) + "."; + str += backQuoteIfNeed(table); + return str; + } + + String formatSkippedMessage(const std::string_view & database, const std::string_view & table, const std::string_view & column) + { + String str = ". Skipped column " + backQuoteIfNeed(column) + " ON "; + if (!database.empty()) + str += backQuoteIfNeed(database) + "."; + str += backQuoteIfNeed(table); + return str; + } + + template + String formatSkippedMessage(const std::string_view & database, const std::string_view & table, const std::vector & columns) + { + if (columns.size() == 1) + return formatSkippedMessage(database, table, columns[0]); + + String str = ". Skipped columns "; + bool need_comma = false; + for (const auto & column : columns) + { + if (std::exchange(need_comma, true)) + str += ", "; + str += backQuoteIfNeed(column); + } + str += " ON "; + if (!database.empty()) + str += backQuoteIfNeed(database) + "."; + str += backQuoteIfNeed(table); + return str; + } +} + + +ContextAccess::ContextAccess(const AccessControlManager & manager_, const Params & params_) + : manager(&manager_) + , params(params_) +{ + subscription_for_user_change = manager->subscribeForChanges( + *params.user_id, [this](const UUID &, const AccessEntityPtr & entity) + { + UserPtr changed_user = entity ? typeid_cast(entity) : nullptr; + std::lock_guard lock{mutex}; + setUser(changed_user); + }); + + setUser(manager->read(*params.user_id)); +} + + +void ContextAccess::setUser(const UserPtr & user_) const +{ + user = user_; + if (!user) + { + /// User has been dropped. + auto nothing_granted = boost::make_shared(); + boost::range::fill(result_access, nothing_granted); + subscription_for_user_change = {}; + subscription_for_roles_changes = {}; + enabled_roles = nullptr; + roles_info = nullptr; + roles_with_admin_option = nullptr; + enabled_row_policies = nullptr; + enabled_quota = nullptr; + return; + } + + user_name = user->getName(); + trace_log = &Poco::Logger::get("ContextAccess (" + user_name + ")"); + + std::vector current_roles, current_roles_with_admin_option; + if (params.use_default_roles) + { + for (const UUID & id : user->granted_roles) + { + if (user->default_roles.match(id)) + current_roles.push_back(id); + } + boost::range::set_intersection(current_roles, user->granted_roles_with_admin_option, + std::back_inserter(current_roles_with_admin_option)); + } + else + { + current_roles.reserve(params.current_roles.size()); + for (const auto & id : params.current_roles) + { + if (user->granted_roles.contains(id)) + current_roles.push_back(id); + if (user->granted_roles_with_admin_option.contains(id)) + current_roles_with_admin_option.push_back(id); + } + } + + subscription_for_roles_changes = {}; + enabled_roles = manager->getEnabledRoles(current_roles, current_roles_with_admin_option); + subscription_for_roles_changes = enabled_roles->subscribeForChanges([this](const std::shared_ptr & roles_info_) + { + std::lock_guard lock{mutex}; + setRolesInfo(roles_info_); + }); + + setRolesInfo(enabled_roles->getRolesInfo()); +} + + +void ContextAccess::setRolesInfo(const std::shared_ptr & roles_info_) const +{ + assert(roles_info_); + roles_info = roles_info_; + roles_with_admin_option.store(boost::make_shared>(roles_info->enabled_roles_with_admin_option.begin(), roles_info->enabled_roles_with_admin_option.end())); + boost::range::fill(result_access, nullptr /* need recalculate */); + enabled_row_policies = manager->getEnabledRowPolicies(*params.user_id, roles_info->enabled_roles); + enabled_quota = manager->getEnabledQuota(*params.user_id, user_name, roles_info->enabled_roles, params.address, params.quota_key); +} + + +bool ContextAccess::isCorrectPassword(const String & password) const +{ + std::lock_guard lock{mutex}; + if (!user) + return false; + return user->authentication.isCorrectPassword(password); +} + +bool ContextAccess::isClientHostAllowed() const +{ + std::lock_guard lock{mutex}; + if (!user) + return false; + return user->allowed_client_hosts.contains(params.address); +} + + +template +bool ContextAccess::checkAccessImpl(Poco::Logger * log_, const AccessFlags & flags, const Args &... args) const +{ + auto access = calculateResultAccess(grant_option); + bool is_granted = access->isGranted(flags, args...); + + if (trace_log) + LOG_TRACE(trace_log, "Access " << (is_granted ? "granted" : "denied") << ": " << (AccessRightsElement{flags, args...}.toString())); + + if (is_granted) + return true; + + if constexpr (mode == RETURN_FALSE_IF_ACCESS_DENIED) + return false; + + if constexpr (mode == LOG_WARNING_IF_ACCESS_DENIED) + { + if (!log_) + return false; + } + + auto show_error = [&](const String & msg, [[maybe_unused]] int error_code) + { + if constexpr (mode == THROW_IF_ACCESS_DENIED) + throw Exception(user_name + ": " + msg, error_code); + else if constexpr (mode == LOG_WARNING_IF_ACCESS_DENIED) + LOG_WARNING(log_, user_name + ": " + msg + formatSkippedMessage(args...)); + }; + + if (!user) + { + show_error("User has been dropped", ErrorCodes::UNKNOWN_USER); + } + else if (grant_option && calculateResultAccess(false, params.readonly, params.allow_ddl, params.allow_introspection)->isGranted(flags, args...)) + { + show_error( + "Not enough privileges. " + "The required privileges have been granted, but without grant option. " + "To execute this query it's necessary to have the grant " + + AccessRightsElement{flags, args...}.toString() + " WITH GRANT OPTION", + ErrorCodes::ACCESS_DENIED); + } + else if (params.readonly && calculateResultAccess(false, false, params.allow_ddl, params.allow_introspection)->isGranted(flags, args...)) + { + if (params.interface == ClientInfo::Interface::HTTP && params.http_method == ClientInfo::HTTPMethod::GET) + show_error( + "Cannot execute query in readonly mode. " + "For queries over HTTP, method GET implies readonly. You should use method POST for modifying queries", + ErrorCodes::READONLY); + else + show_error("Cannot execute query in readonly mode", ErrorCodes::READONLY); + } + else if (!params.allow_ddl && calculateResultAccess(false, params.readonly, true, params.allow_introspection)->isGranted(flags, args...)) + { + show_error("Cannot execute query. DDL queries are prohibited for the user", ErrorCodes::QUERY_IS_PROHIBITED); + } + else if (!params.allow_introspection && calculateResultAccess(false, params.readonly, params.allow_ddl, true)->isGranted(flags, args...)) + { + show_error("Introspection functions are disabled, because setting 'allow_introspection_functions' is set to 0", ErrorCodes::FUNCTION_NOT_ALLOWED); + } + else + { + show_error( + "Not enough privileges. To execute this query it's necessary to have the grant " + + AccessRightsElement{flags, args...}.toString() + (grant_option ? " WITH GRANT OPTION" : ""), + ErrorCodes::ACCESS_DENIED); + } + + return false; +} + + +template +bool ContextAccess::checkAccessImpl(Poco::Logger * log_, const AccessRightsElement & element) const +{ + if (element.any_database) + { + return checkAccessImpl(log_, element.access_flags); + } + else if (element.any_table) + { + if (element.database.empty()) + return checkAccessImpl(log_, element.access_flags, params.current_database); + else + return checkAccessImpl(log_, element.access_flags, element.database); + } + else if (element.any_column) + { + if (element.database.empty()) + return checkAccessImpl(log_, element.access_flags, params.current_database, element.table); + else + return checkAccessImpl(log_, element.access_flags, element.database, element.table); + } + else + { + if (element.database.empty()) + return checkAccessImpl(log_, element.access_flags, params.current_database, element.table, element.columns); + else + return checkAccessImpl(log_, element.access_flags, element.database, element.table, element.columns); + } +} + + +template +bool ContextAccess::checkAccessImpl(Poco::Logger * log_, const AccessRightsElements & elements) const +{ + for (const auto & element : elements) + if (!checkAccessImpl(log_, element)) + return false; + return true; +} + + +void ContextAccess::checkAccess(const AccessFlags & flags) const { checkAccessImpl(nullptr, flags); } +void ContextAccess::checkAccess(const AccessFlags & flags, const std::string_view & database) const { checkAccessImpl(nullptr, flags, database); } +void ContextAccess::checkAccess(const AccessFlags & flags, const std::string_view & database, const std::string_view & table) const { checkAccessImpl(nullptr, flags, database, table); } +void ContextAccess::checkAccess(const AccessFlags & flags, const std::string_view & database, const std::string_view & table, const std::string_view & column) const { checkAccessImpl(nullptr, flags, database, table, column); } +void ContextAccess::checkAccess(const AccessFlags & flags, const std::string_view & database, const std::string_view & table, const std::vector & columns) const { checkAccessImpl(nullptr, flags, database, table, columns); } +void ContextAccess::checkAccess(const AccessFlags & flags, const std::string_view & database, const std::string_view & table, const Strings & columns) const { checkAccessImpl(nullptr, flags, database, table, columns); } +void ContextAccess::checkAccess(const AccessRightsElement & element) const { checkAccessImpl(nullptr, element); } +void ContextAccess::checkAccess(const AccessRightsElements & elements) const { checkAccessImpl(nullptr, elements); } + +bool ContextAccess::isGranted(const AccessFlags & flags) const { return checkAccessImpl(nullptr, flags); } +bool ContextAccess::isGranted(const AccessFlags & flags, const std::string_view & database) const { return checkAccessImpl(nullptr, flags, database); } +bool ContextAccess::isGranted(const AccessFlags & flags, const std::string_view & database, const std::string_view & table) const { return checkAccessImpl(nullptr, flags, database, table); } +bool ContextAccess::isGranted(const AccessFlags & flags, const std::string_view & database, const std::string_view & table, const std::string_view & column) const { return checkAccessImpl(nullptr, flags, database, table, column); } +bool ContextAccess::isGranted(const AccessFlags & flags, const std::string_view & database, const std::string_view & table, const std::vector & columns) const { return checkAccessImpl(nullptr, flags, database, table, columns); } +bool ContextAccess::isGranted(const AccessFlags & flags, const std::string_view & database, const std::string_view & table, const Strings & columns) const { return checkAccessImpl(nullptr, flags, database, table, columns); } +bool ContextAccess::isGranted(const AccessRightsElement & element) const { return checkAccessImpl(nullptr, element); } +bool ContextAccess::isGranted(const AccessRightsElements & elements) const { return checkAccessImpl(nullptr, elements); } + +bool ContextAccess::isGranted(Poco::Logger * log_, const AccessFlags & flags) const { return checkAccessImpl(log_, flags); } +bool ContextAccess::isGranted(Poco::Logger * log_, const AccessFlags & flags, const std::string_view & database) const { return checkAccessImpl(log_, flags, database); } +bool ContextAccess::isGranted(Poco::Logger * log_, const AccessFlags & flags, const std::string_view & database, const std::string_view & table) const { return checkAccessImpl(log_, flags, database, table); } +bool ContextAccess::isGranted(Poco::Logger * log_, const AccessFlags & flags, const std::string_view & database, const std::string_view & table, const std::string_view & column) const { return checkAccessImpl(log_, flags, database, table, column); } +bool ContextAccess::isGranted(Poco::Logger * log_, const AccessFlags & flags, const std::string_view & database, const std::string_view & table, const std::vector & columns) const { return checkAccessImpl(log_, flags, database, table, columns); } +bool ContextAccess::isGranted(Poco::Logger * log_, const AccessFlags & flags, const std::string_view & database, const std::string_view & table, const Strings & columns) const { return checkAccessImpl(log_, flags, database, table, columns); } +bool ContextAccess::isGranted(Poco::Logger * log_, const AccessRightsElement & element) const { return checkAccessImpl(log_, element); } +bool ContextAccess::isGranted(Poco::Logger * log_, const AccessRightsElements & elements) const { return checkAccessImpl(log_, elements); } + +void ContextAccess::checkGrantOption(const AccessFlags & flags) const { checkAccessImpl(nullptr, flags); } +void ContextAccess::checkGrantOption(const AccessFlags & flags, const std::string_view & database) const { checkAccessImpl(nullptr, flags, database); } +void ContextAccess::checkGrantOption(const AccessFlags & flags, const std::string_view & database, const std::string_view & table) const { checkAccessImpl(nullptr, flags, database, table); } +void ContextAccess::checkGrantOption(const AccessFlags & flags, const std::string_view & database, const std::string_view & table, const std::string_view & column) const { checkAccessImpl(nullptr, flags, database, table, column); } +void ContextAccess::checkGrantOption(const AccessFlags & flags, const std::string_view & database, const std::string_view & table, const std::vector & columns) const { checkAccessImpl(nullptr, flags, database, table, columns); } +void ContextAccess::checkGrantOption(const AccessFlags & flags, const std::string_view & database, const std::string_view & table, const Strings & columns) const { checkAccessImpl(nullptr, flags, database, table, columns); } +void ContextAccess::checkGrantOption(const AccessRightsElement & element) const { checkAccessImpl(nullptr, element); } +void ContextAccess::checkGrantOption(const AccessRightsElements & elements) const { checkAccessImpl(nullptr, elements); } + + +void ContextAccess::checkAdminOption(const UUID & role_id) const +{ + if (isGranted(AccessType::ROLE_ADMIN)) + return; + + auto roles_with_admin_option_loaded = roles_with_admin_option.load(); + if (roles_with_admin_option_loaded && roles_with_admin_option_loaded->contains(role_id)) + return; + + std::optional role_name = manager->readName(role_id); + if (!role_name) + role_name = "ID {" + toString(role_id) + "}"; + throw Exception( + getUserName() + ": Not enough privileges. To execute this query it's necessary to have the grant " + backQuoteIfNeed(*role_name) + + " WITH ADMIN OPTION ", + ErrorCodes::ACCESS_DENIED); +} + + +boost::shared_ptr ContextAccess::calculateResultAccess(bool grant_option) const +{ + return calculateResultAccess(grant_option, params.readonly, params.allow_ddl, params.allow_introspection); +} + + +boost::shared_ptr ContextAccess::calculateResultAccess(bool grant_option, UInt64 readonly_, bool allow_ddl_, bool allow_introspection_) const +{ + size_t cache_index = static_cast(readonly_ != params.readonly) + + static_cast(allow_ddl_ != params.allow_ddl) * 2 + + + static_cast(allow_introspection_ != params.allow_introspection) * 3 + + static_cast(grant_option) * 4; + assert(cache_index < std::size(result_access)); + auto res = result_access[cache_index].load(); + if (res) + return res; + + std::lock_guard lock{mutex}; + res = result_access[cache_index].load(); + if (res) + return res; + + auto merged_access = boost::make_shared(); + + if (grant_option) + { + *merged_access = user->access_with_grant_option; + if (roles_info) + merged_access->merge(roles_info->access_with_grant_option); + } + else + { + *merged_access = user->access; + if (roles_info) + merged_access->merge(roles_info->access); + } + + static const AccessFlags table_ddl = AccessType::CREATE_DATABASE | AccessType::CREATE_TABLE | AccessType::CREATE_VIEW + | AccessType::ALTER_TABLE | AccessType::ALTER_VIEW | AccessType::DROP_DATABASE | AccessType::DROP_TABLE | AccessType::DROP_VIEW + | AccessType::TRUNCATE; + static const AccessFlags dictionary_ddl = AccessType::CREATE_DICTIONARY | AccessType::DROP_DICTIONARY; + static const AccessFlags table_and_dictionary_ddl = table_ddl | dictionary_ddl; + static const AccessFlags write_table_access = AccessType::INSERT | AccessType::OPTIMIZE; + static const AccessFlags all_dcl = AccessType::CREATE_USER | AccessType::CREATE_ROLE | AccessType::CREATE_POLICY + | AccessType::CREATE_QUOTA | AccessType::ALTER_USER | AccessType::ALTER_POLICY | AccessType::ALTER_QUOTA | AccessType::DROP_USER + | AccessType::DROP_ROLE | AccessType::DROP_POLICY | AccessType::DROP_QUOTA | AccessType::ROLE_ADMIN; + + if (readonly_) + merged_access->revoke(write_table_access | all_dcl | AccessType::SYSTEM | AccessType::KILL_QUERY); + + if (readonly_ || !allow_ddl_) + merged_access->revoke(table_and_dictionary_ddl); + + if (readonly_ == 1) + { + /// Table functions are forbidden in readonly mode. + /// For example, for readonly = 2 - allowed. + merged_access->revoke(AccessType::CREATE_TEMPORARY_TABLE | AccessType::TABLE_FUNCTIONS); + } + + if (!allow_introspection_) + merged_access->revoke(AccessType::INTROSPECTION); + + /// Anyone has access to the "system" database. + merged_access->grant(AccessType::SELECT, DatabaseCatalog::SYSTEM_DATABASE); + + if (readonly_ != 1) + { + /// User has access to temporary or external table if such table was resolved in session or query context + merged_access->grant(AccessFlags::allTableFlags() | AccessFlags::allColumnFlags(), DatabaseCatalog::TEMPORARY_DATABASE); + } + + if (readonly_ && grant_option) + { + /// No grant option in readonly mode. + merged_access->revoke(AccessType::ALL); + } + + if (trace_log && (params.readonly == readonly_) && (params.allow_ddl == allow_ddl_) && (params.allow_introspection == allow_introspection_)) + { + LOG_TRACE(trace_log, "List of all grants: " << merged_access->toString() << (grant_option ? " WITH GRANT OPTION" : "")); + if (roles_info && !roles_info->getCurrentRolesNames().empty()) + { + LOG_TRACE( + trace_log, + "Current_roles: " << boost::algorithm::join(roles_info->getCurrentRolesNames(), ", ") + << ", enabled_roles: " << boost::algorithm::join(roles_info->getEnabledRolesNames(), ", ")); + } + } + + res = std::move(merged_access); + result_access[cache_index].store(res); + return res; +} + + +UserPtr ContextAccess::getUser() const +{ + std::lock_guard lock{mutex}; + return user; +} + +String ContextAccess::getUserName() const +{ + std::lock_guard lock{mutex}; + return user_name; +} + +std::shared_ptr ContextAccess::getRolesInfo() const +{ + std::lock_guard lock{mutex}; + return roles_info; +} + +std::vector ContextAccess::getCurrentRoles() const +{ + std::lock_guard lock{mutex}; + return roles_info ? roles_info->current_roles : std::vector{}; +} + +Strings ContextAccess::getCurrentRolesNames() const +{ + std::lock_guard lock{mutex}; + return roles_info ? roles_info->getCurrentRolesNames() : Strings{}; +} + +std::vector ContextAccess::getEnabledRoles() const +{ + std::lock_guard lock{mutex}; + return roles_info ? roles_info->enabled_roles : std::vector{}; +} + +Strings ContextAccess::getEnabledRolesNames() const +{ + std::lock_guard lock{mutex}; + return roles_info ? roles_info->getEnabledRolesNames() : Strings{}; +} + +std::shared_ptr ContextAccess::getRowPolicies() const +{ + std::lock_guard lock{mutex}; + return enabled_row_policies; +} + +ASTPtr ContextAccess::getRowPolicyCondition(const String & database, const String & table_name, RowPolicy::ConditionType index, const ASTPtr & extra_condition) const +{ + std::lock_guard lock{mutex}; + return enabled_row_policies ? enabled_row_policies->getCondition(database, table_name, index, extra_condition) : nullptr; +} + +std::shared_ptr ContextAccess::getQuota() const +{ + std::lock_guard lock{mutex}; + return enabled_quota; +} + + +std::shared_ptr ContextAccess::getFullAccess() +{ + static const std::shared_ptr res = [] + { + auto full_access = std::shared_ptr(new ContextAccess); + auto everything_granted = boost::make_shared(); + everything_granted->grant(AccessType::ALL); + boost::range::fill(full_access->result_access, everything_granted); + full_access->enabled_quota = EnabledQuota::getUnlimitedQuota(); + return full_access; + }(); + return res; +} + +} diff --git a/dbms/src/Access/ContextAccess.h b/dbms/src/Access/ContextAccess.h new file mode 100644 index 00000000000..dc84e51f9e6 --- /dev/null +++ b/dbms/src/Access/ContextAccess.h @@ -0,0 +1,156 @@ +#pragma once + +#include +#include +#include +#include +#include +#include +#include +#include +#include + + +namespace Poco { class Logger; } + +namespace DB +{ +struct User; +using UserPtr = std::shared_ptr; +struct EnabledRolesInfo; +class EnabledRoles; +class EnabledRowPolicies; +class EnabledQuota; +struct Settings; +class AccessControlManager; +class IAST; +using ASTPtr = std::shared_ptr; + + +class ContextAccess +{ +public: + struct Params + { + std::optional user_id; + std::vector current_roles; + bool use_default_roles = false; + UInt64 readonly = 0; + bool allow_ddl = false; + bool allow_introspection = false; + String current_database; + ClientInfo::Interface interface = ClientInfo::Interface::TCP; + ClientInfo::HTTPMethod http_method = ClientInfo::HTTPMethod::UNKNOWN; + Poco::Net::IPAddress address; + String quota_key; + + auto toTuple() const { return std::tie(user_id, current_roles, use_default_roles, readonly, allow_ddl, allow_introspection, current_database, interface, http_method, address, quota_key); } + friend bool operator ==(const Params & lhs, const Params & rhs) { return lhs.toTuple() == rhs.toTuple(); } + friend bool operator !=(const Params & lhs, const Params & rhs) { return !(lhs == rhs); } + friend bool operator <(const Params & lhs, const Params & rhs) { return lhs.toTuple() < rhs.toTuple(); } + friend bool operator >(const Params & lhs, const Params & rhs) { return rhs < lhs; } + friend bool operator <=(const Params & lhs, const Params & rhs) { return !(rhs < lhs); } + friend bool operator >=(const Params & lhs, const Params & rhs) { return !(lhs < rhs); } + }; + + const Params & getParams() const { return params; } + UserPtr getUser() const; + String getUserName() const; + + bool isCorrectPassword(const String & password) const; + bool isClientHostAllowed() const; + + std::shared_ptr getRolesInfo() const; + std::vector getCurrentRoles() const; + Strings getCurrentRolesNames() const; + std::vector getEnabledRoles() const; + Strings getEnabledRolesNames() const; + + std::shared_ptr getRowPolicies() const; + ASTPtr getRowPolicyCondition(const String & database, const String & table_name, RowPolicy::ConditionType index, const ASTPtr & extra_condition = nullptr) const; + std::shared_ptr getQuota() const; + + /// Checks if a specified access is granted, and throws an exception if not. + /// Empty database means the current database. + void checkAccess(const AccessFlags & flags) const; + void checkAccess(const AccessFlags & flags, const std::string_view & database) const; + void checkAccess(const AccessFlags & flags, const std::string_view & database, const std::string_view & table) const; + void checkAccess(const AccessFlags & flags, const std::string_view & database, const std::string_view & table, const std::string_view & column) const; + void checkAccess(const AccessFlags & flags, const std::string_view & database, const std::string_view & table, const std::vector & columns) const; + void checkAccess(const AccessFlags & flags, const std::string_view & database, const std::string_view & table, const Strings & columns) const; + void checkAccess(const AccessRightsElement & element) const; + void checkAccess(const AccessRightsElements & elements) const; + + /// Checks if a specified access is granted. + bool isGranted(const AccessFlags & flags) const; + bool isGranted(const AccessFlags & flags, const std::string_view & database) const; + bool isGranted(const AccessFlags & flags, const std::string_view & database, const std::string_view & table) const; + bool isGranted(const AccessFlags & flags, const std::string_view & database, const std::string_view & table, const std::string_view & column) const; + bool isGranted(const AccessFlags & flags, const std::string_view & database, const std::string_view & table, const std::vector & columns) const; + bool isGranted(const AccessFlags & flags, const std::string_view & database, const std::string_view & table, const Strings & columns) const; + bool isGranted(const AccessRightsElement & element) const; + bool isGranted(const AccessRightsElements & elements) const; + + /// Checks if a specified access is granted, and logs a warning if not. + bool isGranted(Poco::Logger * log_, const AccessFlags & flags) const; + bool isGranted(Poco::Logger * log_, const AccessFlags & flags, const std::string_view & database) const; + bool isGranted(Poco::Logger * log_, const AccessFlags & flags, const std::string_view & database, const std::string_view & table) const; + bool isGranted(Poco::Logger * log_, const AccessFlags & flags, const std::string_view & database, const std::string_view & table, const std::string_view & column) const; + bool isGranted(Poco::Logger * log_, const AccessFlags & flags, const std::string_view & database, const std::string_view & table, const std::vector & columns) const; + bool isGranted(Poco::Logger * log_, const AccessFlags & flags, const std::string_view & database, const std::string_view & table, const Strings & columns) const; + bool isGranted(Poco::Logger * log_, const AccessRightsElement & element) const; + bool isGranted(Poco::Logger * log_, const AccessRightsElements & elements) const; + + /// Checks if a specified access is granted with grant option, and throws an exception if not. + void checkGrantOption(const AccessFlags & flags) const; + void checkGrantOption(const AccessFlags & flags, const std::string_view & database) const; + void checkGrantOption(const AccessFlags & flags, const std::string_view & database, const std::string_view & table) const; + void checkGrantOption(const AccessFlags & flags, const std::string_view & database, const std::string_view & table, const std::string_view & column) const; + void checkGrantOption(const AccessFlags & flags, const std::string_view & database, const std::string_view & table, const std::vector & columns) const; + void checkGrantOption(const AccessFlags & flags, const std::string_view & database, const std::string_view & table, const Strings & columns) const; + void checkGrantOption(const AccessRightsElement & element) const; + void checkGrantOption(const AccessRightsElements & elements) const; + + /// Checks if a specified role is granted with admin option, and throws an exception if not. + void checkAdminOption(const UUID & role_id) const; + + /// Returns an instance of ContextAccess which has full access to everything. + static std::shared_ptr getFullAccess(); + +private: + friend class AccessControlManager; + ContextAccess() {} + ContextAccess(const AccessControlManager & manager_, const Params & params_); + + void setUser(const UserPtr & user_) const; + void setRolesInfo(const std::shared_ptr & roles_info_) const; + + template + bool checkAccessImpl(Poco::Logger * log_, const AccessFlags & flags, const Args &... args) const; + + template + bool checkAccessImpl(Poco::Logger * log_, const AccessRightsElement & element) const; + + template + bool checkAccessImpl(Poco::Logger * log_, const AccessRightsElements & elements) const; + + boost::shared_ptr calculateResultAccess(bool grant_option) const; + boost::shared_ptr calculateResultAccess(bool grant_option, UInt64 readonly_, bool allow_ddl_, bool allow_introspection_) const; + + const AccessControlManager * manager = nullptr; + const Params params; + mutable Poco::Logger * trace_log = nullptr; + mutable UserPtr user; + mutable String user_name; + mutable ext::scope_guard subscription_for_user_change; + mutable std::shared_ptr enabled_roles; + mutable ext::scope_guard subscription_for_roles_changes; + mutable std::shared_ptr roles_info; + mutable boost::atomic_shared_ptr> roles_with_admin_option; + mutable boost::atomic_shared_ptr result_access[7]; + mutable std::shared_ptr enabled_row_policies; + mutable std::shared_ptr enabled_quota; + mutable std::mutex mutex; +}; + +} diff --git a/dbms/src/Access/QuotaContext.cpp b/dbms/src/Access/EnabledQuota.cpp similarity index 77% rename from dbms/src/Access/QuotaContext.cpp rename to dbms/src/Access/EnabledQuota.cpp index a48c41dc419..92257ce0002 100644 --- a/dbms/src/Access/QuotaContext.cpp +++ b/dbms/src/Access/EnabledQuota.cpp @@ -1,4 +1,5 @@ -#include +#include +#include #include #include #include @@ -14,7 +15,7 @@ namespace ErrorCodes extern const int QUOTA_EXPIRED; } -struct QuotaContext::Impl +struct EnabledQuota::Impl { [[noreturn]] static void throwQuotaExceed( const String & user_name, @@ -133,7 +134,7 @@ struct QuotaContext::Impl }; -QuotaContext::Interval & QuotaContext::Interval::operator =(const Interval & src) +EnabledQuota::Interval & EnabledQuota::Interval::operator =(const Interval & src) { if (this == &src) return *this; @@ -150,7 +151,7 @@ QuotaContext::Interval & QuotaContext::Interval::operator =(const Interval & src } -QuotaUsageInfo QuotaContext::Intervals::getUsageInfo(std::chrono::system_clock::time_point current_time) const +QuotaUsageInfo EnabledQuota::Intervals::getUsageInfo(std::chrono::system_clock::time_point current_time) const { QuotaUsageInfo info; info.quota_id = quota_id; @@ -174,97 +175,85 @@ QuotaUsageInfo QuotaContext::Intervals::getUsageInfo(std::chrono::system_clock:: } -QuotaContext::QuotaContext() - : intervals(boost::make_shared()) /// Unlimited quota. +EnabledQuota::EnabledQuota(const Params & params_) : params(params_) { } - -QuotaContext::QuotaContext( - const String & user_name_, - const UUID & user_id_, - const std::vector & enabled_roles_, - const Poco::Net::IPAddress & address_, - const String & client_key_) - : user_name(user_name_), user_id(user_id_), enabled_roles(enabled_roles_), address(address_), client_key(client_key_) -{ -} +EnabledQuota::~EnabledQuota() = default; -QuotaContext::~QuotaContext() = default; - - -void QuotaContext::used(ResourceType resource_type, ResourceAmount amount, bool check_exceeded) const +void EnabledQuota::used(ResourceType resource_type, ResourceAmount amount, bool check_exceeded) const { used({resource_type, amount}, check_exceeded); } -void QuotaContext::used(const std::pair & resource, bool check_exceeded) const +void EnabledQuota::used(const std::pair & resource, bool check_exceeded) const { auto loaded = intervals.load(); auto current_time = std::chrono::system_clock::now(); - Impl::used(user_name, *loaded, resource.first, resource.second, current_time, check_exceeded); + Impl::used(getUserName(), *loaded, resource.first, resource.second, current_time, check_exceeded); } -void QuotaContext::used(const std::pair & resource1, const std::pair & resource2, bool check_exceeded) const +void EnabledQuota::used(const std::pair & resource1, const std::pair & resource2, bool check_exceeded) const { auto loaded = intervals.load(); auto current_time = std::chrono::system_clock::now(); - Impl::used(user_name, *loaded, resource1.first, resource1.second, current_time, check_exceeded); - Impl::used(user_name, *loaded, resource2.first, resource2.second, current_time, check_exceeded); + Impl::used(getUserName(), *loaded, resource1.first, resource1.second, current_time, check_exceeded); + Impl::used(getUserName(), *loaded, resource2.first, resource2.second, current_time, check_exceeded); } -void QuotaContext::used(const std::pair & resource1, const std::pair & resource2, const std::pair & resource3, bool check_exceeded) const +void EnabledQuota::used(const std::pair & resource1, const std::pair & resource2, const std::pair & resource3, bool check_exceeded) const { auto loaded = intervals.load(); auto current_time = std::chrono::system_clock::now(); - Impl::used(user_name, *loaded, resource1.first, resource1.second, current_time, check_exceeded); - Impl::used(user_name, *loaded, resource2.first, resource2.second, current_time, check_exceeded); - Impl::used(user_name, *loaded, resource3.first, resource3.second, current_time, check_exceeded); + Impl::used(getUserName(), *loaded, resource1.first, resource1.second, current_time, check_exceeded); + Impl::used(getUserName(), *loaded, resource2.first, resource2.second, current_time, check_exceeded); + Impl::used(getUserName(), *loaded, resource3.first, resource3.second, current_time, check_exceeded); } -void QuotaContext::used(const std::vector> & resources, bool check_exceeded) const +void EnabledQuota::used(const std::vector> & resources, bool check_exceeded) const { auto loaded = intervals.load(); auto current_time = std::chrono::system_clock::now(); for (const auto & resource : resources) - Impl::used(user_name, *loaded, resource.first, resource.second, current_time, check_exceeded); + Impl::used(getUserName(), *loaded, resource.first, resource.second, current_time, check_exceeded); } -void QuotaContext::checkExceeded() const +void EnabledQuota::checkExceeded() const { auto loaded = intervals.load(); - Impl::checkExceeded(user_name, *loaded, std::chrono::system_clock::now()); + Impl::checkExceeded(getUserName(), *loaded, std::chrono::system_clock::now()); } -void QuotaContext::checkExceeded(ResourceType resource_type) const +void EnabledQuota::checkExceeded(ResourceType resource_type) const { auto loaded = intervals.load(); - Impl::checkExceeded(user_name, *loaded, resource_type, std::chrono::system_clock::now()); + Impl::checkExceeded(getUserName(), *loaded, resource_type, std::chrono::system_clock::now()); } -QuotaUsageInfo QuotaContext::getUsageInfo() const +QuotaUsageInfo EnabledQuota::getUsageInfo() const { auto loaded = intervals.load(); return loaded->getUsageInfo(std::chrono::system_clock::now()); } -QuotaUsageInfo::QuotaUsageInfo() : quota_id(UUID(UInt128(0))) +std::shared_ptr EnabledQuota::getUnlimitedQuota() { + static const std::shared_ptr res = [] + { + auto unlimited_quota = std::shared_ptr(new EnabledQuota); + unlimited_quota->intervals = boost::make_shared(); + return unlimited_quota; + }(); + return res; } - -QuotaUsageInfo::Interval::Interval() -{ - boost::range::fill(used, 0); - boost::range::fill(max, 0); -} } diff --git a/dbms/src/Access/QuotaContext.h b/dbms/src/Access/EnabledQuota.h similarity index 60% rename from dbms/src/Access/QuotaContext.h rename to dbms/src/Access/EnabledQuota.h index d788a08ea17..5a624c651af 100644 --- a/dbms/src/Access/QuotaContext.h +++ b/dbms/src/Access/EnabledQuota.h @@ -3,7 +3,6 @@ #include #include #include -#include #include #include #include @@ -16,17 +15,31 @@ namespace DB struct QuotaUsageInfo; -/// Instances of `QuotaContext` are used to track resource consumption. -class QuotaContext : public boost::noncopyable +/// Instances of `EnabledQuota` are used to track resource consumption. +class EnabledQuota : public boost::noncopyable { public: + struct Params + { + UUID user_id; + String user_name; + std::vector enabled_roles; + Poco::Net::IPAddress client_address; + String client_key; + + auto toTuple() const { return std::tie(user_id, enabled_roles, user_name, client_address, client_key); } + friend bool operator ==(const Params & lhs, const Params & rhs) { return lhs.toTuple() == rhs.toTuple(); } + friend bool operator !=(const Params & lhs, const Params & rhs) { return !(lhs == rhs); } + friend bool operator <(const Params & lhs, const Params & rhs) { return lhs.toTuple() < rhs.toTuple(); } + friend bool operator >(const Params & lhs, const Params & rhs) { return rhs < lhs; } + friend bool operator <=(const Params & lhs, const Params & rhs) { return !(rhs < lhs); } + friend bool operator >=(const Params & lhs, const Params & rhs) { return !(lhs < rhs); } + }; + using ResourceType = Quota::ResourceType; using ResourceAmount = Quota::ResourceAmount; - /// Default constructors makes an unlimited quota. - QuotaContext(); - - ~QuotaContext(); + ~EnabledQuota(); /// Tracks resource consumption. If the quota exceeded and `check_exceeded == true`, throws an exception. void used(ResourceType resource_type, ResourceAmount amount, bool check_exceeded = true) const; @@ -39,15 +52,18 @@ public: void checkExceeded() const; void checkExceeded(ResourceType resource_type) const; - /// Returns the information about this quota context. + /// Returns the information about quota consumption. QuotaUsageInfo getUsageInfo() const; -private: - friend class QuotaContextFactory; - friend struct ext::shared_ptr_helper; + /// Returns an instance of EnabledQuota which is never exceeded. + static std::shared_ptr getUnlimitedQuota(); - /// Instances of this class are created by QuotaContextFactory. - QuotaContext(const String & user_name_, const UUID & user_id_, const std::vector & enabled_roles_, const Poco::Net::IPAddress & address_, const String & client_key_); +private: + friend class QuotaCache; + EnabledQuota(const Params & params_); + EnabledQuota() {} + + const String & getUserName() const { return params.user_name; } static constexpr size_t MAX_RESOURCE_TYPE = Quota::MAX_RESOURCE_TYPE; @@ -76,38 +92,8 @@ private: struct Impl; - const String user_name; - const UUID user_id; - const std::vector enabled_roles; - const Poco::Net::IPAddress address; - const String client_key; + const Params params; boost::atomic_shared_ptr intervals; /// atomically changed by QuotaUsageManager }; -using QuotaContextPtr = std::shared_ptr; - - -/// The information about a quota context. -struct QuotaUsageInfo -{ - using ResourceType = Quota::ResourceType; - using ResourceAmount = Quota::ResourceAmount; - static constexpr size_t MAX_RESOURCE_TYPE = Quota::MAX_RESOURCE_TYPE; - - struct Interval - { - ResourceAmount used[MAX_RESOURCE_TYPE]; - ResourceAmount max[MAX_RESOURCE_TYPE]; - std::chrono::seconds duration = std::chrono::seconds::zero(); - bool randomize_interval = false; - std::chrono::system_clock::time_point end_of_interval; - Interval(); - }; - - std::vector intervals; - UUID quota_id; - String quota_name; - String quota_key; - QuotaUsageInfo(); -}; } diff --git a/dbms/src/Access/EnabledRoles.cpp b/dbms/src/Access/EnabledRoles.cpp new file mode 100644 index 00000000000..fd48eb6830a --- /dev/null +++ b/dbms/src/Access/EnabledRoles.cpp @@ -0,0 +1,51 @@ +#include +#include +#include +#include + + +namespace DB +{ +EnabledRoles::EnabledRoles(const Params & params_) : params(params_) +{ +} + +EnabledRoles::~EnabledRoles() = default; + + +std::shared_ptr EnabledRoles::getRolesInfo() const +{ + std::lock_guard lock{mutex}; + return info; +} + + +ext::scope_guard EnabledRoles::subscribeForChanges(const OnChangeHandler & handler) const +{ + std::lock_guard lock{mutex}; + handlers.push_back(handler); + auto it = std::prev(handlers.end()); + + return [this, it] + { + std::lock_guard lock2{mutex}; + handlers.erase(it); + }; +} + + +void EnabledRoles::setRolesInfo(const std::shared_ptr & info_) +{ + std::vector handlers_to_notify; + SCOPE_EXIT({ for (const auto & handler : handlers_to_notify) handler(info_); }); + + std::lock_guard lock{mutex}; + + if (info && info_ && *info == *info_) + return; + + info = info_; + boost::range::copy(handlers, std::back_inserter(handlers_to_notify)); +} + +} diff --git a/dbms/src/Access/EnabledRoles.h b/dbms/src/Access/EnabledRoles.h new file mode 100644 index 00000000000..122b1a16fe3 --- /dev/null +++ b/dbms/src/Access/EnabledRoles.h @@ -0,0 +1,53 @@ +#pragma once + +#include +#include +#include +#include +#include + + +namespace DB +{ +struct EnabledRolesInfo; + +class EnabledRoles +{ +public: + struct Params + { + std::vector current_roles; + std::vector current_roles_with_admin_option; + + auto toTuple() const { return std::tie(current_roles, current_roles_with_admin_option); } + friend bool operator ==(const Params & lhs, const Params & rhs) { return lhs.toTuple() == rhs.toTuple(); } + friend bool operator !=(const Params & lhs, const Params & rhs) { return !(lhs == rhs); } + friend bool operator <(const Params & lhs, const Params & rhs) { return lhs.toTuple() < rhs.toTuple(); } + friend bool operator >(const Params & lhs, const Params & rhs) { return rhs < lhs; } + friend bool operator <=(const Params & lhs, const Params & rhs) { return !(rhs < lhs); } + friend bool operator >=(const Params & lhs, const Params & rhs) { return !(lhs < rhs); } + }; + + ~EnabledRoles(); + + /// Returns all the roles specified in the constructor. + std::shared_ptr getRolesInfo() const; + + using OnChangeHandler = std::function & info)>; + + /// Called when either the specified roles or the roles granted to the specified roles are changed. + ext::scope_guard subscribeForChanges(const OnChangeHandler & handler) const; + +private: + friend class RoleCache; + EnabledRoles(const Params & params_); + + void setRolesInfo(const std::shared_ptr & info_); + + const Params params; + mutable std::shared_ptr info; + mutable std::list handlers; + mutable std::mutex mutex; +}; + +} diff --git a/dbms/src/Access/CurrentRolesInfo.cpp b/dbms/src/Access/EnabledRolesInfo.cpp similarity index 77% rename from dbms/src/Access/CurrentRolesInfo.cpp rename to dbms/src/Access/EnabledRolesInfo.cpp index f4cbd739021..7481e707033 100644 --- a/dbms/src/Access/CurrentRolesInfo.cpp +++ b/dbms/src/Access/EnabledRolesInfo.cpp @@ -1,10 +1,10 @@ -#include +#include namespace DB { -Strings CurrentRolesInfo::getCurrentRolesNames() const +Strings EnabledRolesInfo::getCurrentRolesNames() const { Strings result; result.reserve(current_roles.size()); @@ -14,7 +14,7 @@ Strings CurrentRolesInfo::getCurrentRolesNames() const } -Strings CurrentRolesInfo::getEnabledRolesNames() const +Strings EnabledRolesInfo::getEnabledRolesNames() const { Strings result; result.reserve(enabled_roles.size()); @@ -24,7 +24,7 @@ Strings CurrentRolesInfo::getEnabledRolesNames() const } -bool operator==(const CurrentRolesInfo & lhs, const CurrentRolesInfo & rhs) +bool operator==(const EnabledRolesInfo & lhs, const EnabledRolesInfo & rhs) { return (lhs.current_roles == rhs.current_roles) && (lhs.enabled_roles == rhs.enabled_roles) && (lhs.enabled_roles_with_admin_option == rhs.enabled_roles_with_admin_option) && (lhs.names_of_roles == rhs.names_of_roles) diff --git a/dbms/src/Access/CurrentRolesInfo.h b/dbms/src/Access/EnabledRolesInfo.h similarity index 62% rename from dbms/src/Access/CurrentRolesInfo.h rename to dbms/src/Access/EnabledRolesInfo.h index a4dd26be0f7..1fb69e6e871 100644 --- a/dbms/src/Access/CurrentRolesInfo.h +++ b/dbms/src/Access/EnabledRolesInfo.h @@ -10,7 +10,7 @@ namespace DB { /// Information about a role. -struct CurrentRolesInfo +struct EnabledRolesInfo { std::vector current_roles; std::vector enabled_roles; @@ -22,10 +22,8 @@ struct CurrentRolesInfo Strings getCurrentRolesNames() const; Strings getEnabledRolesNames() const; - friend bool operator ==(const CurrentRolesInfo & lhs, const CurrentRolesInfo & rhs); - friend bool operator !=(const CurrentRolesInfo & lhs, const CurrentRolesInfo & rhs) { return !(lhs == rhs); } + friend bool operator ==(const EnabledRolesInfo & lhs, const EnabledRolesInfo & rhs); + friend bool operator !=(const EnabledRolesInfo & lhs, const EnabledRolesInfo & rhs) { return !(lhs == rhs); } }; -using CurrentRolesInfoPtr = std::shared_ptr; - } diff --git a/dbms/src/Access/RowPolicyContext.cpp b/dbms/src/Access/EnabledRowPolicies.cpp similarity index 55% rename from dbms/src/Access/RowPolicyContext.cpp rename to dbms/src/Access/EnabledRowPolicies.cpp index 661a6cb4b5f..a525fb65606 100644 --- a/dbms/src/Access/RowPolicyContext.cpp +++ b/dbms/src/Access/EnabledRowPolicies.cpp @@ -1,4 +1,4 @@ -#include +#include #include #include #include @@ -8,55 +8,50 @@ namespace DB { -size_t RowPolicyContext::Hash::operator()(const DatabaseAndTableNameRef & database_and_table_name) const +size_t EnabledRowPolicies::Hash::operator()(const DatabaseAndTableNameRef & database_and_table_name) const { return std::hash{}(database_and_table_name.first) - std::hash{}(database_and_table_name.second); } -RowPolicyContext::RowPolicyContext() - : map_of_mixed_conditions(boost::make_shared()) +EnabledRowPolicies::EnabledRowPolicies(const Params & params_) + : params(params_) { } - -RowPolicyContext::~RowPolicyContext() = default; +EnabledRowPolicies::~EnabledRowPolicies() = default; -RowPolicyContext::RowPolicyContext(const UUID & user_id_, const std::vector & enabled_roles_) - : user_id(user_id_), enabled_roles(enabled_roles_) -{} - - -ASTPtr RowPolicyContext::getCondition(const String & database, const String & table_name, ConditionIndex index) const +ASTPtr EnabledRowPolicies::getCondition(const String & database, const String & table_name, ConditionType type) const { /// We don't lock `mutex` here. auto loaded = map_of_mixed_conditions.load(); auto it = loaded->find({database, table_name}); if (it == loaded->end()) return {}; - return it->second.mixed_conditions[index]; + return it->second.mixed_conditions[type]; } -ASTPtr RowPolicyContext::combineConditionsUsingAnd(const ASTPtr & lhs, const ASTPtr & rhs) +ASTPtr EnabledRowPolicies::getCondition(const String & database, const String & table_name, ConditionType type, const ASTPtr & extra_condition) const { - if (!lhs) - return rhs; - if (!rhs) - return lhs; + ASTPtr main_condition = getCondition(database, table_name, type); + if (!main_condition) + return extra_condition; + if (!extra_condition) + return main_condition; auto function = std::make_shared(); auto exp_list = std::make_shared(); function->name = "and"; function->arguments = exp_list; function->children.push_back(exp_list); - exp_list->children.push_back(lhs); - exp_list->children.push_back(rhs); + exp_list->children.push_back(main_condition); + exp_list->children.push_back(extra_condition); return function; } -std::vector RowPolicyContext::getCurrentPolicyIDs() const +std::vector EnabledRowPolicies::getCurrentPolicyIDs() const { /// We don't lock `mutex` here. auto loaded = map_of_mixed_conditions.load(); @@ -67,7 +62,7 @@ std::vector RowPolicyContext::getCurrentPolicyIDs() const } -std::vector RowPolicyContext::getCurrentPolicyIDs(const String & database, const String & table_name) const +std::vector EnabledRowPolicies::getCurrentPolicyIDs(const String & database, const String & table_name) const { /// We don't lock `mutex` here. auto loaded = map_of_mixed_conditions.load(); @@ -76,4 +71,5 @@ std::vector RowPolicyContext::getCurrentPolicyIDs(const String & database, return {}; return it->second.policy_ids; } + } diff --git a/dbms/src/Access/RowPolicyContext.h b/dbms/src/Access/EnabledRowPolicies.h similarity index 55% rename from dbms/src/Access/RowPolicyContext.h rename to dbms/src/Access/EnabledRowPolicies.h index 2042b85bf7a..9befb65ff0b 100644 --- a/dbms/src/Access/RowPolicyContext.h +++ b/dbms/src/Access/EnabledRowPolicies.h @@ -15,23 +15,32 @@ using ASTPtr = std::shared_ptr; /// Provides fast access to row policies' conditions for a specific user and tables. -class RowPolicyContext +class EnabledRowPolicies { public: - /// Default constructor makes a row policy usage context which restricts nothing. - RowPolicyContext(); + struct Params + { + UUID user_id; + std::vector enabled_roles; - ~RowPolicyContext(); + auto toTuple() const { return std::tie(user_id, enabled_roles); } + friend bool operator ==(const Params & lhs, const Params & rhs) { return lhs.toTuple() == rhs.toTuple(); } + friend bool operator !=(const Params & lhs, const Params & rhs) { return !(lhs == rhs); } + friend bool operator <(const Params & lhs, const Params & rhs) { return lhs.toTuple() < rhs.toTuple(); } + friend bool operator >(const Params & lhs, const Params & rhs) { return rhs < lhs; } + friend bool operator <=(const Params & lhs, const Params & rhs) { return !(rhs < lhs); } + friend bool operator >=(const Params & lhs, const Params & rhs) { return !(lhs < rhs); } + }; - using ConditionIndex = RowPolicy::ConditionIndex; + ~EnabledRowPolicies(); + + using ConditionType = RowPolicy::ConditionType; /// Returns prepared filter for a specific table and operations. /// The function can return nullptr, that means there is no filters applied. /// The returned filter can be a combination of the filters defined by multiple row policies. - ASTPtr getCondition(const String & database, const String & table_name, ConditionIndex index) const; - - /// Combines two conditions into one by using the logical AND operator. - static ASTPtr combineConditionsUsingAnd(const ASTPtr & lhs, const ASTPtr & rhs); + ASTPtr getCondition(const String & database, const String & table_name, ConditionType type) const; + ASTPtr getCondition(const String & database, const String & table_name, ConditionType type, const ASTPtr & extra_condition) const; /// Returns IDs of all the policies used by the current user. std::vector getCurrentPolicyIDs() const; @@ -40,9 +49,8 @@ public: std::vector getCurrentPolicyIDs(const String & database, const String & table_name) const; private: - friend class RowPolicyContextFactory; - friend struct ext::shared_ptr_helper; - RowPolicyContext(const UUID & user_id_, const std::vector & enabled_roles_); /// RowPolicyContext should be created by RowPolicyContextFactory. + friend class RowPolicyCache; + EnabledRowPolicies(const Params & params_); using DatabaseAndTableName = std::pair; using DatabaseAndTableNameRef = std::pair; @@ -50,8 +58,8 @@ private: { size_t operator()(const DatabaseAndTableNameRef & database_and_table_name) const; }; - static constexpr size_t MAX_CONDITION_INDEX = RowPolicy::MAX_CONDITION_INDEX; - using ParsedConditions = std::array; + static constexpr size_t MAX_CONDITION_TYPE = RowPolicy::MAX_CONDITION_TYPE; + using ParsedConditions = std::array; struct MixedConditions { std::unique_ptr database_and_table_name_keeper; @@ -60,11 +68,8 @@ private: }; using MapOfMixedConditions = std::unordered_map; - const UUID user_id; - const std::vector enabled_roles; + const Params params; mutable boost::atomic_shared_ptr map_of_mixed_conditions; }; - -using RowPolicyContextPtr = std::shared_ptr; } diff --git a/dbms/src/Access/GenericRoleSet.cpp b/dbms/src/Access/ExtendedRoleSet.cpp similarity index 65% rename from dbms/src/Access/GenericRoleSet.cpp rename to dbms/src/Access/ExtendedRoleSet.cpp index 1e751f995c1..b59dc7ac232 100644 --- a/dbms/src/Access/GenericRoleSet.cpp +++ b/dbms/src/Access/ExtendedRoleSet.cpp @@ -1,8 +1,8 @@ -#include +#include #include #include #include -#include +#include #include #include #include @@ -17,57 +17,59 @@ namespace ErrorCodes { extern const int LOGICAL_ERROR; } -GenericRoleSet::GenericRoleSet() = default; -GenericRoleSet::GenericRoleSet(const GenericRoleSet & src) = default; -GenericRoleSet & GenericRoleSet::operator =(const GenericRoleSet & src) = default; -GenericRoleSet::GenericRoleSet(GenericRoleSet && src) = default; -GenericRoleSet & GenericRoleSet::operator =(GenericRoleSet && src) = default; -GenericRoleSet::GenericRoleSet(AllTag) +ExtendedRoleSet::ExtendedRoleSet() = default; +ExtendedRoleSet::ExtendedRoleSet(const ExtendedRoleSet & src) = default; +ExtendedRoleSet & ExtendedRoleSet::operator =(const ExtendedRoleSet & src) = default; +ExtendedRoleSet::ExtendedRoleSet(ExtendedRoleSet && src) = default; +ExtendedRoleSet & ExtendedRoleSet::operator =(ExtendedRoleSet && src) = default; + + +ExtendedRoleSet::ExtendedRoleSet(AllTag) { all = true; } -GenericRoleSet::GenericRoleSet(const UUID & id) +ExtendedRoleSet::ExtendedRoleSet(const UUID & id) { add(id); } -GenericRoleSet::GenericRoleSet(const std::vector & ids_) +ExtendedRoleSet::ExtendedRoleSet(const std::vector & ids_) { add(ids_); } -GenericRoleSet::GenericRoleSet(const boost::container::flat_set & ids_) +ExtendedRoleSet::ExtendedRoleSet(const boost::container::flat_set & ids_) { add(ids_); } -GenericRoleSet::GenericRoleSet(const ASTGenericRoleSet & ast) +ExtendedRoleSet::ExtendedRoleSet(const ASTExtendedRoleSet & ast) { init(ast, nullptr, nullptr); } -GenericRoleSet::GenericRoleSet(const ASTGenericRoleSet & ast, const UUID & current_user_id) +ExtendedRoleSet::ExtendedRoleSet(const ASTExtendedRoleSet & ast, const UUID & current_user_id) { init(ast, nullptr, ¤t_user_id); } -GenericRoleSet::GenericRoleSet(const ASTGenericRoleSet & ast, const AccessControlManager & manager) +ExtendedRoleSet::ExtendedRoleSet(const ASTExtendedRoleSet & ast, const AccessControlManager & manager) { init(ast, &manager, nullptr); } -GenericRoleSet::GenericRoleSet(const ASTGenericRoleSet & ast, const AccessControlManager & manager, const UUID & current_user_id) +ExtendedRoleSet::ExtendedRoleSet(const ASTExtendedRoleSet & ast, const AccessControlManager & manager, const UUID & current_user_id) { init(ast, &manager, ¤t_user_id); } -void GenericRoleSet::init(const ASTGenericRoleSet & ast, const AccessControlManager * manager, const UUID * current_user_id) +void ExtendedRoleSet::init(const ASTExtendedRoleSet & ast, const AccessControlManager * manager, const UUID * current_user_id) { all = ast.all; @@ -113,9 +115,9 @@ void GenericRoleSet::init(const ASTGenericRoleSet & ast, const AccessControlMana } -std::shared_ptr GenericRoleSet::toAST() const +std::shared_ptr ExtendedRoleSet::toAST() const { - auto ast = std::make_shared(); + auto ast = std::make_shared(); ast->id_mode = true; ast->all = all; @@ -137,14 +139,14 @@ std::shared_ptr GenericRoleSet::toAST() const } -String GenericRoleSet::toString() const +String ExtendedRoleSet::toString() const { auto ast = toAST(); return serializeAST(*ast); } -Strings GenericRoleSet::toStrings() const +Strings ExtendedRoleSet::toStrings() const { if (all || !except_ids.empty()) return {toString()}; @@ -157,9 +159,9 @@ Strings GenericRoleSet::toStrings() const } -std::shared_ptr GenericRoleSet::toASTWithNames(const AccessControlManager & manager) const +std::shared_ptr ExtendedRoleSet::toASTWithNames(const AccessControlManager & manager) const { - auto ast = std::make_shared(); + auto ast = std::make_shared(); ast->all = all; if (!ids.empty()) @@ -190,14 +192,14 @@ std::shared_ptr GenericRoleSet::toASTWithNames(const AccessCo } -String GenericRoleSet::toStringWithNames(const AccessControlManager & manager) const +String ExtendedRoleSet::toStringWithNames(const AccessControlManager & manager) const { auto ast = toASTWithNames(manager); return serializeAST(*ast); } -Strings GenericRoleSet::toStringsWithNames(const AccessControlManager & manager) const +Strings ExtendedRoleSet::toStringsWithNames(const AccessControlManager & manager) const { if (all || !except_ids.empty()) return {toStringWithNames(manager)}; @@ -215,13 +217,13 @@ Strings GenericRoleSet::toStringsWithNames(const AccessControlManager & manager) } -bool GenericRoleSet::empty() const +bool ExtendedRoleSet::empty() const { return ids.empty() && !all; } -void GenericRoleSet::clear() +void ExtendedRoleSet::clear() { ids.clear(); all = false; @@ -229,33 +231,33 @@ void GenericRoleSet::clear() } -void GenericRoleSet::add(const UUID & id) +void ExtendedRoleSet::add(const UUID & id) { ids.insert(id); } -void GenericRoleSet::add(const std::vector & ids_) +void ExtendedRoleSet::add(const std::vector & ids_) { for (const auto & id : ids_) add(id); } -void GenericRoleSet::add(const boost::container::flat_set & ids_) +void ExtendedRoleSet::add(const boost::container::flat_set & ids_) { for (const auto & id : ids_) add(id); } -bool GenericRoleSet::match(const UUID & id) const +bool ExtendedRoleSet::match(const UUID & id) const { return (all || ids.contains(id)) && !except_ids.contains(id); } -bool GenericRoleSet::match(const UUID & user_id, const std::vector & enabled_roles) const +bool ExtendedRoleSet::match(const UUID & user_id, const std::vector & enabled_roles) const { if (!all && !ids.contains(user_id)) { @@ -274,7 +276,7 @@ bool GenericRoleSet::match(const UUID & user_id, const std::vector & enabl } -bool GenericRoleSet::match(const UUID & user_id, const boost::container::flat_set & enabled_roles) const +bool ExtendedRoleSet::match(const UUID & user_id, const boost::container::flat_set & enabled_roles) const { if (!all && !ids.contains(user_id)) { @@ -293,17 +295,17 @@ bool GenericRoleSet::match(const UUID & user_id, const boost::container::flat_se } -std::vector GenericRoleSet::getMatchingIDs() const +std::vector ExtendedRoleSet::getMatchingIDs() const { if (all) - throw Exception("getAllMatchingIDs() can't get ALL ids", ErrorCodes::LOGICAL_ERROR); + throw Exception("getAllMatchingIDs() can't get ALL ids without manager", ErrorCodes::LOGICAL_ERROR); std::vector res; boost::range::set_difference(ids, except_ids, std::back_inserter(res)); return res; } -std::vector GenericRoleSet::getMatchingUsers(const AccessControlManager & manager) const +std::vector ExtendedRoleSet::getMatchingIDs(const AccessControlManager & manager) const { if (!all) return getMatchingIDs(); @@ -314,37 +316,17 @@ std::vector GenericRoleSet::getMatchingUsers(const AccessControlManager & if (match(id)) res.push_back(id); } - return res; -} - - -std::vector GenericRoleSet::getMatchingRoles(const AccessControlManager & manager) const -{ - if (!all) - return getMatchingIDs(); - - std::vector res; for (const UUID & id : manager.findAll()) { if (match(id)) res.push_back(id); } + return res; } -std::vector GenericRoleSet::getMatchingUsersAndRoles(const AccessControlManager & manager) const -{ - if (!all) - return getMatchingIDs(); - - std::vector vec = getMatchingUsers(manager); - boost::range::push_back(vec, getMatchingRoles(manager)); - return vec; -} - - -bool operator ==(const GenericRoleSet & lhs, const GenericRoleSet & rhs) +bool operator ==(const ExtendedRoleSet & lhs, const ExtendedRoleSet & rhs) { return (lhs.all == rhs.all) && (lhs.ids == rhs.ids) && (lhs.except_ids == rhs.except_ids); } diff --git a/dbms/src/Access/ExtendedRoleSet.h b/dbms/src/Access/ExtendedRoleSet.h new file mode 100644 index 00000000000..61a4db6e0ae --- /dev/null +++ b/dbms/src/Access/ExtendedRoleSet.h @@ -0,0 +1,75 @@ +#pragma once + +#include +#include +#include +#include + + +namespace DB +{ +class ASTExtendedRoleSet; +class AccessControlManager; + + +/// Represents a set of users/roles like +/// {user_name | role_name | CURRENT_USER} [,...] | NONE | ALL | ALL EXCEPT {user_name | role_name | CURRENT_USER} [,...] +/// Similar to ASTExtendedRoleSet, but with IDs instead of names. +struct ExtendedRoleSet +{ + ExtendedRoleSet(); + ExtendedRoleSet(const ExtendedRoleSet & src); + ExtendedRoleSet & operator =(const ExtendedRoleSet & src); + ExtendedRoleSet(ExtendedRoleSet && src); + ExtendedRoleSet & operator =(ExtendedRoleSet && src); + + struct AllTag {}; + ExtendedRoleSet(AllTag); + + ExtendedRoleSet(const UUID & id); + ExtendedRoleSet(const std::vector & ids_); + ExtendedRoleSet(const boost::container::flat_set & ids_); + + /// The constructor from AST requires the AccessControlManager if `ast.id_mode == false`. + ExtendedRoleSet(const ASTExtendedRoleSet & ast); + ExtendedRoleSet(const ASTExtendedRoleSet & ast, const UUID & current_user_id); + ExtendedRoleSet(const ASTExtendedRoleSet & ast, const AccessControlManager & manager); + ExtendedRoleSet(const ASTExtendedRoleSet & ast, const AccessControlManager & manager, const UUID & current_user_id); + + std::shared_ptr toAST() const; + String toString() const; + Strings toStrings() const; + + std::shared_ptr toASTWithNames(const AccessControlManager & manager) const; + String toStringWithNames(const AccessControlManager & manager) const; + Strings toStringsWithNames(const AccessControlManager & manager) const; + + bool empty() const; + void clear(); + void add(const UUID & id); + void add(const std::vector & ids_); + void add(const boost::container::flat_set & ids_); + + /// Checks if a specified ID matches this ExtendedRoleSet. + bool match(const UUID & id) const; + bool match(const UUID & user_id, const std::vector & enabled_roles) const; + bool match(const UUID & user_id, const boost::container::flat_set & enabled_roles) const; + + /// Returns a list of matching IDs. The function must not be called if `all` == `true`. + std::vector getMatchingIDs() const; + + /// Returns a list of matching users and roles. + std::vector getMatchingIDs(const AccessControlManager & manager) const; + + friend bool operator ==(const ExtendedRoleSet & lhs, const ExtendedRoleSet & rhs); + friend bool operator !=(const ExtendedRoleSet & lhs, const ExtendedRoleSet & rhs) { return !(lhs == rhs); } + + boost::container::flat_set ids; + bool all = false; + boost::container::flat_set except_ids; + +private: + void init(const ASTExtendedRoleSet & ast, const AccessControlManager * manager = nullptr, const UUID * current_user_id = nullptr); +}; + +} diff --git a/dbms/src/Access/GenericRoleSet.h b/dbms/src/Access/GenericRoleSet.h deleted file mode 100644 index e276eb4066a..00000000000 --- a/dbms/src/Access/GenericRoleSet.h +++ /dev/null @@ -1,77 +0,0 @@ -#pragma once - -#include -#include -#include -#include - - -namespace DB -{ -class ASTGenericRoleSet; -class AccessControlManager; - - -/// Represents a set of users/roles like -/// {user_name | role_name | CURRENT_USER} [,...] | NONE | ALL | ALL EXCEPT {user_name | role_name | CURRENT_USER} [,...] -/// Similar to ASTGenericRoleSet, but with IDs instead of names. -struct GenericRoleSet -{ - GenericRoleSet(); - GenericRoleSet(const GenericRoleSet & src); - GenericRoleSet & operator =(const GenericRoleSet & src); - GenericRoleSet(GenericRoleSet && src); - GenericRoleSet & operator =(GenericRoleSet && src); - - struct AllTag {}; - GenericRoleSet(AllTag); - - GenericRoleSet(const UUID & id); - GenericRoleSet(const std::vector & ids_); - GenericRoleSet(const boost::container::flat_set & ids_); - - /// The constructor from AST requires the AccessControlManager if `ast.id_mode == false`. - GenericRoleSet(const ASTGenericRoleSet & ast); - GenericRoleSet(const ASTGenericRoleSet & ast, const UUID & current_user_id); - GenericRoleSet(const ASTGenericRoleSet & ast, const AccessControlManager & manager); - GenericRoleSet(const ASTGenericRoleSet & ast, const AccessControlManager & manager, const UUID & current_user_id); - - std::shared_ptr toAST() const; - String toString() const; - Strings toStrings() const; - - std::shared_ptr toASTWithNames(const AccessControlManager & manager) const; - String toStringWithNames(const AccessControlManager & manager) const; - Strings toStringsWithNames(const AccessControlManager & manager) const; - - bool empty() const; - void clear(); - void add(const UUID & id); - void add(const std::vector & ids_); - void add(const boost::container::flat_set & ids_); - - /// Checks if a specified ID matches this GenericRoleSet. - bool match(const UUID & id) const; - bool match(const UUID & user_id, const std::vector & enabled_roles) const; - bool match(const UUID & user_id, const boost::container::flat_set & enabled_roles) const; - - /// Returns a list of matching IDs. The function must not be called if `all` == `true`. - std::vector getMatchingIDs() const; - - /// Returns a list of matching users. - std::vector getMatchingUsers(const AccessControlManager & manager) const; - std::vector getMatchingRoles(const AccessControlManager & manager) const; - std::vector getMatchingUsersAndRoles(const AccessControlManager & manager) const; - - friend bool operator ==(const GenericRoleSet & lhs, const GenericRoleSet & rhs); - friend bool operator !=(const GenericRoleSet & lhs, const GenericRoleSet & rhs) { return !(lhs == rhs); } - - boost::container::flat_set ids; - bool all = false; - boost::container::flat_set except_ids; - -private: - void init(const ASTGenericRoleSet & ast, const AccessControlManager * manager = nullptr, const UUID * current_user_id = nullptr); -}; - -} diff --git a/dbms/src/Access/Quota.cpp b/dbms/src/Access/Quota.cpp index d9e9e0b35fc..e3a9e11eb10 100644 --- a/dbms/src/Access/Quota.cpp +++ b/dbms/src/Access/Quota.cpp @@ -23,7 +23,7 @@ bool Quota::equal(const IAccessEntity & other) const if (!IAccessEntity::equal(other)) return false; const auto & other_quota = typeid_cast(other); - return (all_limits == other_quota.all_limits) && (key_type == other_quota.key_type) && (roles == other_quota.roles); + return (all_limits == other_quota.all_limits) && (key_type == other_quota.key_type) && (to_roles == other_quota.to_roles); } diff --git a/dbms/src/Access/Quota.h b/dbms/src/Access/Quota.h index 4968e5d92c9..714d582e98f 100644 --- a/dbms/src/Access/Quota.h +++ b/dbms/src/Access/Quota.h @@ -1,7 +1,7 @@ -#pragma once +#pragma once #include -#include +#include #include @@ -63,7 +63,7 @@ struct Quota : public IAccessEntity KeyType key_type = KeyType::NONE; /// Which roles or users should use this quota. - GenericRoleSet roles; + ExtendedRoleSet to_roles; bool equal(const IAccessEntity & other) const override; std::shared_ptr clone() const override { return cloneImpl(); } diff --git a/dbms/src/Access/QuotaContextFactory.cpp b/dbms/src/Access/QuotaCache.cpp similarity index 67% rename from dbms/src/Access/QuotaContextFactory.cpp rename to dbms/src/Access/QuotaCache.cpp index 7c585bdddee..cdf298d0e57 100644 --- a/dbms/src/Access/QuotaContextFactory.cpp +++ b/dbms/src/Access/QuotaCache.cpp @@ -1,5 +1,6 @@ -#include -#include +#include +#include +#include #include #include #include @@ -8,7 +9,6 @@ #include #include #include -#include #include @@ -31,58 +31,53 @@ namespace } -void QuotaContextFactory::QuotaInfo::setQuota(const QuotaPtr & quota_, const UUID & quota_id_) +void QuotaCache::QuotaInfo::setQuota(const QuotaPtr & quota_, const UUID & quota_id_) { quota = quota_; quota_id = quota_id_; - roles = "a->roles; + roles = "a->to_roles; rebuildAllIntervals(); } -bool QuotaContextFactory::QuotaInfo::canUseWithContext(const QuotaContext & context) const -{ - return roles->match(context.user_id, context.enabled_roles); -} - - -String QuotaContextFactory::QuotaInfo::calculateKey(const QuotaContext & context) const +String QuotaCache::QuotaInfo::calculateKey(const EnabledQuota & enabled) const { + const auto & params = enabled.params; using KeyType = Quota::KeyType; switch (quota->key_type) { case KeyType::NONE: return ""; case KeyType::USER_NAME: - return context.user_name; + return params.user_name; case KeyType::IP_ADDRESS: - return context.address.toString(); + return params.client_address.toString(); case KeyType::CLIENT_KEY: { - if (!context.client_key.empty()) - return context.client_key; + if (!params.client_key.empty()) + return params.client_key; throw Exception( - "Quota " + quota->getName() + " (for user " + context.user_name + ") requires a client supplied key.", + "Quota " + quota->getName() + " (for user " + params.user_name + ") requires a client supplied key.", ErrorCodes::QUOTA_REQUIRES_CLIENT_KEY); } case KeyType::CLIENT_KEY_OR_USER_NAME: { - if (!context.client_key.empty()) - return context.client_key; - return context.user_name; + if (!params.client_key.empty()) + return params.client_key; + return params.user_name; } case KeyType::CLIENT_KEY_OR_IP_ADDRESS: { - if (!context.client_key.empty()) - return context.client_key; - return context.address.toString(); + if (!params.client_key.empty()) + return params.client_key; + return params.client_address.toString(); } } __builtin_unreachable(); } -boost::shared_ptr QuotaContextFactory::QuotaInfo::getOrBuildIntervals(const String & key) +boost::shared_ptr QuotaCache::QuotaInfo::getOrBuildIntervals(const String & key) { auto it = key_to_intervals.find(key); if (it != key_to_intervals.end()) @@ -91,14 +86,14 @@ boost::shared_ptr QuotaContextFactory::QuotaInfo: } -void QuotaContextFactory::QuotaInfo::rebuildAllIntervals() +void QuotaCache::QuotaInfo::rebuildAllIntervals() { for (const String & key : key_to_intervals | boost::adaptors::map_keys) rebuildIntervals(key); } -boost::shared_ptr QuotaContextFactory::QuotaInfo::rebuildIntervals(const String & key) +boost::shared_ptr QuotaCache::QuotaInfo::rebuildIntervals(const String & key) { auto new_intervals = boost::make_shared(); new_intervals->quota_name = quota->getName(); @@ -164,27 +159,42 @@ boost::shared_ptr QuotaContextFactory::QuotaInfo: } -QuotaContextFactory::QuotaContextFactory(const AccessControlManager & access_control_manager_) +QuotaCache::QuotaCache(const AccessControlManager & access_control_manager_) : access_control_manager(access_control_manager_) { } - -QuotaContextFactory::~QuotaContextFactory() = default; +QuotaCache::~QuotaCache() = default; -QuotaContextPtr QuotaContextFactory::createContext(const String & user_name, const UUID & user_id, const std::vector & enabled_roles, const Poco::Net::IPAddress & address, const String & client_key) +std::shared_ptr QuotaCache::getEnabledQuota(const UUID & user_id, const String & user_name, const std::vector & enabled_roles, const Poco::Net::IPAddress & client_address, const String & client_key) { std::lock_guard lock{mutex}; ensureAllQuotasRead(); - auto context = ext::shared_ptr_helper::create(user_name, user_id, enabled_roles, address, client_key); - contexts.push_back(context); - chooseQuotaForContext(context); - return context; + + EnabledQuota::Params params; + params.user_id = user_id; + params.user_name = user_name; + params.enabled_roles = enabled_roles; + params.client_address = client_address; + params.client_key = client_key; + auto it = enabled_quotas.find(params); + if (it != enabled_quotas.end()) + { + auto from_cache = it->second.lock(); + if (from_cache) + return from_cache; + enabled_quotas.erase(it); + } + + auto res = std::shared_ptr(new EnabledQuota(params)); + enabled_quotas.emplace(std::move(params), res); + chooseQuotaToConsumeFor(*res); + return res; } -void QuotaContextFactory::ensureAllQuotasRead() +void QuotaCache::ensureAllQuotasRead() { /// `mutex` is already locked. if (all_quotas_read) @@ -209,7 +219,7 @@ void QuotaContextFactory::ensureAllQuotasRead() } -void QuotaContextFactory::quotaAddedOrChanged(const UUID & quota_id, const std::shared_ptr & new_quota) +void QuotaCache::quotaAddedOrChanged(const UUID & quota_id, const std::shared_ptr & new_quota) { std::lock_guard lock{mutex}; auto it = all_quotas.find(quota_id); @@ -225,42 +235,42 @@ void QuotaContextFactory::quotaAddedOrChanged(const UUID & quota_id, const std:: auto & info = it->second; info.setQuota(new_quota, quota_id); - chooseQuotaForAllContexts(); + chooseQuotaToConsume(); } -void QuotaContextFactory::quotaRemoved(const UUID & quota_id) +void QuotaCache::quotaRemoved(const UUID & quota_id) { std::lock_guard lock{mutex}; all_quotas.erase(quota_id); - chooseQuotaForAllContexts(); + chooseQuotaToConsume(); } -void QuotaContextFactory::chooseQuotaForAllContexts() +void QuotaCache::chooseQuotaToConsume() { /// `mutex` is already locked. - boost::range::remove_erase_if( - contexts, - [&](const std::weak_ptr & weak) + std::erase_if( + enabled_quotas, + [&](const std::pair> & pr) { - auto context = weak.lock(); - if (!context) - return true; // remove from the `contexts` list. - chooseQuotaForContext(context); - return false; // keep in the `contexts` list. + auto elem = pr.second.lock(); + if (!elem) + return true; // remove from the `enabled_quotas` list. + chooseQuotaToConsumeFor(*elem); + return false; // keep in the `enabled_quotas` list. }); } -void QuotaContextFactory::chooseQuotaForContext(const std::shared_ptr & context) +void QuotaCache::chooseQuotaToConsumeFor(EnabledQuota & enabled) { /// `mutex` is already locked. boost::shared_ptr intervals; for (auto & info : all_quotas | boost::adaptors::map_values) { - if (info.canUseWithContext(*context)) + if (info.roles->match(enabled.params.user_id, enabled.params.enabled_roles)) { - String key = info.calculateKey(*context); + String key = info.calculateKey(enabled); intervals = info.getOrBuildIntervals(key); break; } @@ -269,11 +279,11 @@ void QuotaContextFactory::chooseQuotaForContext(const std::shared_ptr(); /// No quota == no limits. - context->intervals.store(intervals); + enabled.intervals.store(intervals); } -std::vector QuotaContextFactory::getUsageInfo() const +std::vector QuotaCache::getUsageInfo() const { std::lock_guard lock{mutex}; std::vector all_infos; diff --git a/dbms/src/Access/QuotaContextFactory.h b/dbms/src/Access/QuotaCache.h similarity index 60% rename from dbms/src/Access/QuotaContextFactory.h rename to dbms/src/Access/QuotaCache.h index c130da4f2cd..81734f385c1 100644 --- a/dbms/src/Access/QuotaContextFactory.h +++ b/dbms/src/Access/QuotaCache.h @@ -1,11 +1,11 @@ #pragma once -#include +#include #include #include #include +#include #include -#include namespace DB @@ -14,47 +14,46 @@ class AccessControlManager; /// Stores information how much amount of resources have been consumed and how much are left. -class QuotaContextFactory +class QuotaCache { public: - QuotaContextFactory(const AccessControlManager & access_control_manager_); - ~QuotaContextFactory(); + QuotaCache(const AccessControlManager & access_control_manager_); + ~QuotaCache(); - QuotaContextPtr createContext(const String & user_name, const UUID & user_id, const std::vector & enabled_roles, const Poco::Net::IPAddress & address, const String & client_key); + std::shared_ptr getEnabledQuota(const UUID & user_id, const String & user_name, const std::vector & enabled_roles, const Poco::Net::IPAddress & address, const String & client_key); std::vector getUsageInfo() const; private: - using Interval = QuotaContext::Interval; - using Intervals = QuotaContext::Intervals; + using Interval = EnabledQuota::Interval; + using Intervals = EnabledQuota::Intervals; struct QuotaInfo { QuotaInfo(const QuotaPtr & quota_, const UUID & quota_id_) { setQuota(quota_, quota_id_); } void setQuota(const QuotaPtr & quota_, const UUID & quota_id_); - bool canUseWithContext(const QuotaContext & context) const; - String calculateKey(const QuotaContext & context) const; + String calculateKey(const EnabledQuota & enabled_quota) const; boost::shared_ptr getOrBuildIntervals(const String & key); boost::shared_ptr rebuildIntervals(const String & key); void rebuildAllIntervals(); QuotaPtr quota; UUID quota_id; - const GenericRoleSet * roles = nullptr; + const ExtendedRoleSet * roles = nullptr; std::unordered_map> key_to_intervals; }; void ensureAllQuotasRead(); void quotaAddedOrChanged(const UUID & quota_id, const std::shared_ptr & new_quota); void quotaRemoved(const UUID & quota_id); - void chooseQuotaForAllContexts(); - void chooseQuotaForContext(const std::shared_ptr & context); + void chooseQuotaToConsume(); + void chooseQuotaToConsumeFor(EnabledQuota & enabled_quota); const AccessControlManager & access_control_manager; mutable std::mutex mutex; std::unordered_map all_quotas; bool all_quotas_read = false; ext::scope_guard subscription; - std::vector> contexts; + std::map> enabled_quotas; }; } diff --git a/dbms/src/Access/QuotaUsageInfo.cpp b/dbms/src/Access/QuotaUsageInfo.cpp new file mode 100644 index 00000000000..bcdf2b50062 --- /dev/null +++ b/dbms/src/Access/QuotaUsageInfo.cpp @@ -0,0 +1,17 @@ +#include +#include + + +namespace DB +{ +QuotaUsageInfo::QuotaUsageInfo() : quota_id(UUID(UInt128(0))) +{ +} + + +QuotaUsageInfo::Interval::Interval() +{ + boost::range::fill(used, 0); + boost::range::fill(max, 0); +} +} diff --git a/dbms/src/Access/QuotaUsageInfo.h b/dbms/src/Access/QuotaUsageInfo.h new file mode 100644 index 00000000000..94e16fb9f69 --- /dev/null +++ b/dbms/src/Access/QuotaUsageInfo.h @@ -0,0 +1,32 @@ +#pragma once + +#include +#include + + +namespace DB +{ +/// The information about a quota consumption. +struct QuotaUsageInfo +{ + using ResourceType = Quota::ResourceType; + using ResourceAmount = Quota::ResourceAmount; + static constexpr size_t MAX_RESOURCE_TYPE = Quota::MAX_RESOURCE_TYPE; + + struct Interval + { + ResourceAmount used[MAX_RESOURCE_TYPE]; + ResourceAmount max[MAX_RESOURCE_TYPE]; + std::chrono::seconds duration = std::chrono::seconds::zero(); + bool randomize_interval = false; + std::chrono::system_clock::time_point end_of_interval; + Interval(); + }; + + std::vector intervals; + UUID quota_id; + String quota_name; + String quota_key; + QuotaUsageInfo(); +}; +} diff --git a/dbms/src/Access/RoleCache.cpp b/dbms/src/Access/RoleCache.cpp new file mode 100644 index 00000000000..107f3aa7577 --- /dev/null +++ b/dbms/src/Access/RoleCache.cpp @@ -0,0 +1,186 @@ +#include +#include +#include +#include +#include + + +namespace DB +{ +namespace +{ + struct CollectedRoleInfo + { + RolePtr role; + bool is_current_role = false; + bool with_admin_option = false; + }; + + + void collectRoles(boost::container::flat_map & collected_roles, + const std::function & get_role_function, + const UUID & role_id, + bool is_current_role, + bool with_admin_option) + { + auto it = collected_roles.find(role_id); + if (it != collected_roles.end()) + { + it->second.is_current_role |= is_current_role; + it->second.with_admin_option |= with_admin_option; + return; + } + + auto role = get_role_function(role_id); + collected_roles[role_id] = CollectedRoleInfo{role, is_current_role, with_admin_option}; + + if (!role) + return; + + for (const auto & granted_role : role->granted_roles) + collectRoles(collected_roles, get_role_function, granted_role, false, false); + + for (const auto & granted_role : role->granted_roles_with_admin_option) + collectRoles(collected_roles, get_role_function, granted_role, false, true); + } + + + std::shared_ptr collectInfoForRoles(const boost::container::flat_map & roles) + { + auto new_info = std::make_shared(); + for (const auto & [role_id, collect_info] : roles) + { + const auto & role = collect_info.role; + if (!role) + continue; + if (collect_info.is_current_role) + new_info->current_roles.emplace_back(role_id); + new_info->enabled_roles.emplace_back(role_id); + if (collect_info.with_admin_option) + new_info->enabled_roles_with_admin_option.emplace_back(role_id); + new_info->names_of_roles[role_id] = role->getName(); + new_info->access.merge(role->access); + new_info->access_with_grant_option.merge(role->access_with_grant_option); + } + return new_info; + } +} + + +RoleCache::RoleCache(const AccessControlManager & manager_) + : manager(manager_), cache(600000 /* 10 minutes */) {} + + +RoleCache::~RoleCache() = default; + + +std::shared_ptr RoleCache::getEnabledRoles( + const std::vector & roles, const std::vector & roles_with_admin_option) +{ + std::lock_guard lock{mutex}; + + EnabledRoles::Params params; + params.current_roles = roles; + params.current_roles_with_admin_option = roles_with_admin_option; + auto it = enabled_roles.find(params); + if (it != enabled_roles.end()) + { + auto from_cache = it->second.lock(); + if (from_cache) + return from_cache; + enabled_roles.erase(it); + } + + auto res = std::shared_ptr(new EnabledRoles(params)); + collectRolesInfoFor(*res); + enabled_roles.emplace(std::move(params), res); + return res; +} + + +void RoleCache::collectRolesInfo() +{ + /// `mutex` is already locked. + + std::erase_if( + enabled_roles, + [&](const std::pair> & pr) + { + auto elem = pr.second.lock(); + if (!elem) + return true; // remove from the `enabled_roles` map. + collectRolesInfoFor(*elem); + return false; // keep in the `enabled_roles` map. + }); +} + + +void RoleCache::collectRolesInfoFor(EnabledRoles & enabled) +{ + /// `mutex` is already locked. + + /// Collect roles in use. That includes the current roles, the roles granted to the current roles, and so on. + boost::container::flat_map collected_roles; + auto get_role_function = [this](const UUID & id) { return getRole(id); }; + for (const auto & current_role : enabled.params.current_roles) + collectRoles(collected_roles, get_role_function, current_role, true, false); + + for (const auto & current_role : enabled.params.current_roles_with_admin_option) + collectRoles(collected_roles, get_role_function, current_role, true, true); + + /// Collect data from the collected roles. + enabled.setRolesInfo(collectInfoForRoles(collected_roles)); +} + + +RolePtr RoleCache::getRole(const UUID & role_id) +{ + /// `mutex` is already locked. + + auto role_from_cache = cache.get(role_id); + if (role_from_cache) + return role_from_cache->first; + + auto subscription = manager.subscribeForChanges(role_id, + [this, role_id](const UUID &, const AccessEntityPtr & entity) + { + auto changed_role = entity ? typeid_cast(entity) : nullptr; + if (changed_role) + roleChanged(role_id, changed_role); + else + roleRemoved(role_id); + }); + + auto role = manager.tryRead(role_id); + if (role) + { + auto cache_value = Poco::SharedPtr>( + new std::pair{role, std::move(subscription)}); + cache.add(role_id, cache_value); + return role; + } + + return nullptr; +} + + +void RoleCache::roleChanged(const UUID & role_id, const RolePtr & changed_role) +{ + std::lock_guard lock{mutex}; + auto role_from_cache = cache.get(role_id); + if (!role_from_cache) + return; + role_from_cache->first = changed_role; + cache.update(role_id, role_from_cache); + collectRolesInfo(); +} + + +void RoleCache::roleRemoved(const UUID & role_id) +{ + std::lock_guard lock{mutex}; + cache.remove(role_id); + collectRolesInfo(); +} + +} diff --git a/dbms/src/Access/RoleCache.h b/dbms/src/Access/RoleCache.h new file mode 100644 index 00000000000..69f4cb2ebe8 --- /dev/null +++ b/dbms/src/Access/RoleCache.h @@ -0,0 +1,36 @@ +#pragma once + +#include +#include +#include +#include + + +namespace DB +{ +class AccessControlManager; +struct Role; +using RolePtr = std::shared_ptr; + +class RoleCache +{ +public: + RoleCache(const AccessControlManager & manager_); + ~RoleCache(); + + std::shared_ptr getEnabledRoles(const std::vector & current_roles, const std::vector & current_roles_with_admin_option); + +private: + void collectRolesInfo(); + void collectRolesInfoFor(EnabledRoles & enabled); + RolePtr getRole(const UUID & role_id); + void roleChanged(const UUID & role_id, const RolePtr & changed_role); + void roleRemoved(const UUID & role_id); + + const AccessControlManager & manager; + Poco::ExpireCache> cache; + std::map> enabled_roles; + mutable std::mutex mutex; +}; + +} diff --git a/dbms/src/Access/RoleContext.cpp b/dbms/src/Access/RoleContext.cpp deleted file mode 100644 index 291b44027d4..00000000000 --- a/dbms/src/Access/RoleContext.cpp +++ /dev/null @@ -1,200 +0,0 @@ -#include -#include -#include -#include -#include -#include -#include - - -namespace DB -{ -namespace -{ - void makeUnique(std::vector & vec) - { - boost::range::sort(vec); - vec.erase(std::unique(vec.begin(), vec.end()), vec.end()); - } -} - - -RoleContext::RoleContext(const AccessControlManager & manager_, const UUID & current_role_, bool with_admin_option_) - : manager(&manager_), current_role(current_role_), with_admin_option(with_admin_option_) -{ - update(); -} - - -RoleContext::RoleContext(std::vector && children_) - : children(std::move(children_)) -{ - update(); -} - - -RoleContext::~RoleContext() = default; - - -void RoleContext::update() -{ - std::vector handlers_to_notify; - CurrentRolesInfoPtr info_to_notify; - - { - std::lock_guard lock{mutex}; - auto old_info = info; - - updateImpl(); - - if (!handlers.empty() && (!old_info || (*old_info != *info))) - { - boost::range::copy(handlers, std::back_inserter(handlers_to_notify)); - info_to_notify = info; - } - } - - for (const auto & handler : handlers_to_notify) - handler(info_to_notify); -} - - -void RoleContext::updateImpl() -{ - if (!current_role && children.empty()) - { - info = std::make_shared(); - return; - } - - if (!children.empty()) - { - if (subscriptions_for_change_children.empty()) - { - for (const auto & child : children) - subscriptions_for_change_children.emplace_back( - child->subscribeForChanges([this](const CurrentRolesInfoPtr &) { update(); })); - } - - auto new_info = std::make_shared(); - auto & new_info_ref = *new_info; - - for (const auto & child : children) - { - auto child_info = child->getInfo(); - new_info_ref.access.merge(child_info->access); - new_info_ref.access_with_grant_option.merge(child_info->access_with_grant_option); - boost::range::copy(child_info->current_roles, std::back_inserter(new_info_ref.current_roles)); - boost::range::copy(child_info->enabled_roles, std::back_inserter(new_info_ref.enabled_roles)); - boost::range::copy(child_info->enabled_roles_with_admin_option, std::back_inserter(new_info_ref.enabled_roles_with_admin_option)); - boost::range::copy(child_info->names_of_roles, std::inserter(new_info_ref.names_of_roles, new_info_ref.names_of_roles.end())); - } - makeUnique(new_info_ref.current_roles); - makeUnique(new_info_ref.enabled_roles); - makeUnique(new_info_ref.enabled_roles_with_admin_option); - info = new_info; - return; - } - - assert(current_role); - traverseRoles(*current_role, with_admin_option); - - auto new_info = std::make_shared(); - auto & new_info_ref = *new_info; - - for (auto it = roles_map.begin(); it != roles_map.end();) - { - const auto & id = it->first; - auto & entry = it->second; - if (!entry.in_use) - { - it = roles_map.erase(it); - continue; - } - - if (id == *current_role) - new_info_ref.current_roles.push_back(id); - - new_info_ref.enabled_roles.push_back(id); - - if (entry.with_admin_option) - new_info_ref.enabled_roles_with_admin_option.push_back(id); - - new_info_ref.access.merge(entry.role->access); - new_info_ref.access_with_grant_option.merge(entry.role->access_with_grant_option); - new_info_ref.names_of_roles[id] = entry.role->getName(); - - entry.in_use = false; - entry.with_admin_option = false; - ++it; - } - - info = new_info; -} - - -void RoleContext::traverseRoles(const UUID & id_, bool with_admin_option_) -{ - auto it = roles_map.find(id_); - if (it == roles_map.end()) - { - assert(manager); - auto subscription = manager->subscribeForChanges(id_, [this, id_](const UUID &, const AccessEntityPtr & entity) - { - { - std::lock_guard lock{mutex}; - auto it2 = roles_map.find(id_); - if (it2 == roles_map.end()) - return; - if (entity) - it2->second.role = typeid_cast(entity); - else - roles_map.erase(it2); - } - update(); - }); - - auto role = manager->tryRead(id_); - if (!role) - return; - - RoleEntry new_entry; - new_entry.role = role; - new_entry.subscription_for_change_role = std::move(subscription); - it = roles_map.emplace(id_, std::move(new_entry)).first; - } - - RoleEntry & entry = it->second; - entry.with_admin_option |= with_admin_option_; - if (entry.in_use) - return; - - entry.in_use = true; - for (const auto & granted_role : entry.role->granted_roles) - traverseRoles(granted_role, false); - - for (const auto & granted_role : entry.role->granted_roles_with_admin_option) - traverseRoles(granted_role, true); -} - - -CurrentRolesInfoPtr RoleContext::getInfo() const -{ - std::lock_guard lock{mutex}; - return info; -} - - -ext::scope_guard RoleContext::subscribeForChanges(const OnChangeHandler & handler) const -{ - std::lock_guard lock{mutex}; - handlers.push_back(handler); - auto it = std::prev(handlers.end()); - - return [this, it] - { - std::lock_guard lock2{mutex}; - handlers.erase(it); - }; -} -} diff --git a/dbms/src/Access/RoleContext.h b/dbms/src/Access/RoleContext.h deleted file mode 100644 index 5f19adc56de..00000000000 --- a/dbms/src/Access/RoleContext.h +++ /dev/null @@ -1,64 +0,0 @@ -#pragma once - -#include -#include -#include -#include -#include -#include -#include - - -namespace DB -{ -struct Role; -using RolePtr = std::shared_ptr; -struct CurrentRolesInfo; -using CurrentRolesInfoPtr = std::shared_ptr; -class AccessControlManager; - - -class RoleContext -{ -public: - ~RoleContext(); - - /// Returns all the roles specified in the constructor. - CurrentRolesInfoPtr getInfo() const; - - using OnChangeHandler = std::function; - - /// Called when either the specified roles or the roles granted to the specified roles are changed. - ext::scope_guard subscribeForChanges(const OnChangeHandler & handler) const; - -private: - friend struct ext::shared_ptr_helper; - RoleContext(const AccessControlManager & manager_, const UUID & current_role_, bool with_admin_option_); - RoleContext(std::vector> && children_); - - void update(); - void updateImpl(); - - void traverseRoles(const UUID & id_, bool with_admin_option_); - - const AccessControlManager * manager = nullptr; - std::optional current_role; - bool with_admin_option = false; - std::vector> children; - std::vector subscriptions_for_change_children; - - struct RoleEntry - { - RolePtr role; - ext::scope_guard subscription_for_change_role; - bool with_admin_option = false; - bool in_use = false; - }; - mutable std::unordered_map roles_map; - mutable CurrentRolesInfoPtr info; - mutable std::list handlers; - mutable std::mutex mutex; -}; - -using RoleContextPtr = std::shared_ptr; -} diff --git a/dbms/src/Access/RoleContextFactory.cpp b/dbms/src/Access/RoleContextFactory.cpp deleted file mode 100644 index 3356bc238db..00000000000 --- a/dbms/src/Access/RoleContextFactory.cpp +++ /dev/null @@ -1,52 +0,0 @@ -#include -#include - - -namespace DB -{ - -RoleContextFactory::RoleContextFactory(const AccessControlManager & manager_) - : manager(manager_), cache(600000 /* 10 minutes */) {} - - -RoleContextFactory::~RoleContextFactory() = default; - - -RoleContextPtr RoleContextFactory::createContext( - const std::vector & roles, const std::vector & roles_with_admin_option) -{ - if (roles.size() == 1 && roles_with_admin_option.empty()) - return createContextImpl(roles[0], false); - - if (roles.size() == 1 && roles_with_admin_option == roles) - return createContextImpl(roles[0], true); - - std::vector children; - children.reserve(roles.size()); - for (const auto & role : roles_with_admin_option) - children.push_back(createContextImpl(role, true)); - - boost::container::flat_set roles_with_admin_option_set{roles_with_admin_option.begin(), roles_with_admin_option.end()}; - for (const auto & role : roles) - { - if (!roles_with_admin_option_set.contains(role)) - children.push_back(createContextImpl(role, false)); - } - - return ext::shared_ptr_helper::create(std::move(children)); -} - - -RoleContextPtr RoleContextFactory::createContextImpl(const UUID & id, bool with_admin_option) -{ - std::lock_guard lock{mutex}; - auto key = std::make_pair(id, with_admin_option); - auto x = cache.get(key); - if (x) - return *x; - auto res = ext::shared_ptr_helper::create(manager, id, with_admin_option); - cache.add(key, res); - return res; -} - -} diff --git a/dbms/src/Access/RoleContextFactory.h b/dbms/src/Access/RoleContextFactory.h deleted file mode 100644 index 659c9a218a1..00000000000 --- a/dbms/src/Access/RoleContextFactory.h +++ /dev/null @@ -1,29 +0,0 @@ -#pragma once - -#include -#include -#include - - -namespace DB -{ -class AccessControlManager; - - -class RoleContextFactory -{ -public: - RoleContextFactory(const AccessControlManager & manager_); - ~RoleContextFactory(); - - RoleContextPtr createContext(const std::vector & roles, const std::vector & roles_with_admin_option); - -private: - RoleContextPtr createContextImpl(const UUID & id, bool with_admin_option); - - const AccessControlManager & manager; - Poco::ExpireCache, RoleContextPtr> cache; - std::mutex mutex; -}; - -} diff --git a/dbms/src/Access/RowPolicy.cpp b/dbms/src/Access/RowPolicy.cpp index d5a28d14bb8..65b9451a453 100644 --- a/dbms/src/Access/RowPolicy.cpp +++ b/dbms/src/Access/RowPolicy.cpp @@ -77,11 +77,11 @@ bool RowPolicy::equal(const IAccessEntity & other) const const auto & other_policy = typeid_cast(other); return (database == other_policy.database) && (table_name == other_policy.table_name) && (policy_name == other_policy.policy_name) && boost::range::equal(conditions, other_policy.conditions) && restrictive == other_policy.restrictive - && (roles == other_policy.roles); + && (to_roles == other_policy.to_roles); } -const char * RowPolicy::conditionIndexToString(ConditionIndex index) +const char * RowPolicy::conditionTypeToString(ConditionType index) { switch (index) { @@ -95,7 +95,7 @@ const char * RowPolicy::conditionIndexToString(ConditionIndex index) } -const char * RowPolicy::conditionIndexToColumnName(ConditionIndex index) +const char * RowPolicy::conditionTypeToColumnName(ConditionType index) { switch (index) { diff --git a/dbms/src/Access/RowPolicy.h b/dbms/src/Access/RowPolicy.h index 6bc51a2481c..08219edb46b 100644 --- a/dbms/src/Access/RowPolicy.h +++ b/dbms/src/Access/RowPolicy.h @@ -1,7 +1,7 @@ #pragma once #include -#include +#include namespace DB @@ -37,7 +37,7 @@ struct RowPolicy : public IAccessEntity /// Check is a SQL condition expression used to check whether a row can be written into /// the table. If the expression returns NULL or false an exception is thrown. /// If a conditional expression here is empty it means no filtering is applied. - enum ConditionIndex + enum ConditionType { SELECT_FILTER, INSERT_CHECK, @@ -45,11 +45,11 @@ struct RowPolicy : public IAccessEntity UPDATE_CHECK, DELETE_FILTER, }; - static constexpr size_t MAX_CONDITION_INDEX = 5; - static const char * conditionIndexToString(ConditionIndex index); - static const char * conditionIndexToColumnName(ConditionIndex index); + static constexpr size_t MAX_CONDITION_TYPE = 5; + static const char * conditionTypeToString(ConditionType index); + static const char * conditionTypeToColumnName(ConditionType index); - String conditions[MAX_CONDITION_INDEX]; + String conditions[MAX_CONDITION_TYPE]; /// Sets that the policy is permissive. /// A row is only accessible if at least one of the permissive policies passes, @@ -67,7 +67,7 @@ struct RowPolicy : public IAccessEntity std::shared_ptr clone() const override { return cloneImpl(); } /// Which roles or users should use this row policy. - GenericRoleSet roles; + ExtendedRoleSet to_roles; private: String database; diff --git a/dbms/src/Access/RowPolicyContextFactory.cpp b/dbms/src/Access/RowPolicyCache.cpp similarity index 68% rename from dbms/src/Access/RowPolicyContextFactory.cpp rename to dbms/src/Access/RowPolicyCache.cpp index e891f43b5eb..9509923adbf 100644 --- a/dbms/src/Access/RowPolicyContextFactory.cpp +++ b/dbms/src/Access/RowPolicyCache.cpp @@ -1,5 +1,5 @@ -#include -#include +#include +#include #include #include #include @@ -92,8 +92,8 @@ namespace } - using ConditionIndex = RowPolicy::ConditionIndex; - constexpr size_t MAX_CONDITION_INDEX = RowPolicy::MAX_CONDITION_INDEX; + using ConditionType = RowPolicy::ConditionType; + constexpr size_t MAX_CONDITION_TYPE = RowPolicy::MAX_CONDITION_TYPE; /// Accumulates conditions from multiple row policies and joins them using the AND logical operation. @@ -124,24 +124,24 @@ namespace } -void RowPolicyContextFactory::PolicyInfo::setPolicy(const RowPolicyPtr & policy_) +void RowPolicyCache::PolicyInfo::setPolicy(const RowPolicyPtr & policy_) { policy = policy_; - roles = &policy->roles; + roles = &policy->to_roles; - for (auto index : ext::range_with_static_cast(0, MAX_CONDITION_INDEX)) + for (auto type : ext::range_with_static_cast(0, MAX_CONDITION_TYPE)) { - parsed_conditions[index] = nullptr; - const String & condition = policy->conditions[index]; + parsed_conditions[type] = nullptr; + const String & condition = policy->conditions[type]; if (condition.empty()) continue; - auto previous_range = std::pair(std::begin(policy->conditions), std::begin(policy->conditions) + index); + auto previous_range = std::pair(std::begin(policy->conditions), std::begin(policy->conditions) + type); auto previous_it = std::find(previous_range.first, previous_range.second, condition); if (previous_it != previous_range.second) { /// The condition is already parsed before. - parsed_conditions[index] = parsed_conditions[previous_it - previous_range.first]; + parsed_conditions[type] = parsed_conditions[previous_it - previous_range.first]; continue; } @@ -149,45 +149,52 @@ void RowPolicyContextFactory::PolicyInfo::setPolicy(const RowPolicyPtr & policy_ try { ParserExpression parser; - parsed_conditions[index] = parseQuery(parser, condition, 0); + parsed_conditions[type] = parseQuery(parser, condition, 0); } catch (...) { tryLogCurrentException( &Poco::Logger::get("RowPolicy"), - String("Could not parse the condition ") + RowPolicy::conditionIndexToString(index) + " of row policy " + String("Could not parse the condition ") + RowPolicy::conditionTypeToString(type) + " of row policy " + backQuote(policy->getFullName())); } } } -bool RowPolicyContextFactory::PolicyInfo::canUseWithContext(const RowPolicyContext & context) const -{ - return roles->match(context.user_id, context.enabled_roles); -} - - -RowPolicyContextFactory::RowPolicyContextFactory(const AccessControlManager & access_control_manager_) +RowPolicyCache::RowPolicyCache(const AccessControlManager & access_control_manager_) : access_control_manager(access_control_manager_) { } -RowPolicyContextFactory::~RowPolicyContextFactory() = default; +RowPolicyCache::~RowPolicyCache() = default; -RowPolicyContextPtr RowPolicyContextFactory::createContext(const UUID & user_id, const std::vector & enabled_roles) +std::shared_ptr RowPolicyCache::getEnabledRowPolicies(const UUID & user_id, const std::vector & enabled_roles) { std::lock_guard lock{mutex}; ensureAllRowPoliciesRead(); - auto context = ext::shared_ptr_helper::create(user_id, enabled_roles); - contexts.push_back(context); - mixConditionsForContext(*context); - return context; + + EnabledRowPolicies::Params params; + params.user_id = user_id; + params.enabled_roles = enabled_roles; + auto it = enabled_row_policies.find(params); + if (it != enabled_row_policies.end()) + { + auto from_cache = it->second.lock(); + if (from_cache) + return from_cache; + enabled_row_policies.erase(it); + } + + auto res = std::shared_ptr(new EnabledRowPolicies(params)); + enabled_row_policies.emplace(std::move(params), res); + mixConditionsFor(*res); + return res; } -void RowPolicyContextFactory::ensureAllRowPoliciesRead() +void RowPolicyCache::ensureAllRowPoliciesRead() { /// `mutex` is already locked. if (all_policies_read) @@ -212,7 +219,7 @@ void RowPolicyContextFactory::ensureAllRowPoliciesRead() } -void RowPolicyContextFactory::rowPolicyAddedOrChanged(const UUID & policy_id, const RowPolicyPtr & new_policy) +void RowPolicyCache::rowPolicyAddedOrChanged(const UUID & policy_id, const RowPolicyPtr & new_policy) { std::lock_guard lock{mutex}; auto it = all_policies.find(policy_id); @@ -228,46 +235,46 @@ void RowPolicyContextFactory::rowPolicyAddedOrChanged(const UUID & policy_id, co auto & info = it->second; info.setPolicy(new_policy); - mixConditionsForAllContexts(); + mixConditions(); } -void RowPolicyContextFactory::rowPolicyRemoved(const UUID & policy_id) +void RowPolicyCache::rowPolicyRemoved(const UUID & policy_id) { std::lock_guard lock{mutex}; all_policies.erase(policy_id); - mixConditionsForAllContexts(); + mixConditions(); } -void RowPolicyContextFactory::mixConditionsForAllContexts() +void RowPolicyCache::mixConditions() { /// `mutex` is already locked. - boost::range::remove_erase_if( - contexts, - [&](const std::weak_ptr & weak) + std::erase_if( + enabled_row_policies, + [&](const std::pair> & pr) { - auto context = weak.lock(); - if (!context) - return true; // remove from the `contexts` list. - mixConditionsForContext(*context); - return false; // keep in the `contexts` list. + auto elem = pr.second.lock(); + if (!elem) + return true; // remove from the `enabled_row_policies` map. + mixConditionsFor(*elem); + return false; // keep in the `enabled_row_policies` map. }); } -void RowPolicyContextFactory::mixConditionsForContext(RowPolicyContext & context) +void RowPolicyCache::mixConditionsFor(EnabledRowPolicies & enabled) { /// `mutex` is already locked. struct Mixers { - ConditionsMixer mixers[MAX_CONDITION_INDEX]; + ConditionsMixer mixers[MAX_CONDITION_TYPE]; std::vector policy_ids; }; - using MapOfMixedConditions = RowPolicyContext::MapOfMixedConditions; - using DatabaseAndTableName = RowPolicyContext::DatabaseAndTableName; - using DatabaseAndTableNameRef = RowPolicyContext::DatabaseAndTableNameRef; - using Hash = RowPolicyContext::Hash; + using MapOfMixedConditions = EnabledRowPolicies::MapOfMixedConditions; + using DatabaseAndTableName = EnabledRowPolicies::DatabaseAndTableName; + using DatabaseAndTableNameRef = EnabledRowPolicies::DatabaseAndTableNameRef; + using Hash = EnabledRowPolicies::Hash; std::unordered_map map_of_mixers; @@ -275,12 +282,12 @@ void RowPolicyContextFactory::mixConditionsForContext(RowPolicyContext & context { const auto & policy = *info.policy; auto & mixers = map_of_mixers[std::pair{policy.getDatabase(), policy.getTableName()}]; - if (info.canUseWithContext(context)) + if (info.roles->match(enabled.params.user_id, enabled.params.enabled_roles)) { mixers.policy_ids.push_back(policy_id); - for (auto index : ext::range(0, MAX_CONDITION_INDEX)) - if (info.parsed_conditions[index]) - mixers.mixers[index].add(info.parsed_conditions[index], policy.isRestrictive()); + for (auto type : ext::range(0, MAX_CONDITION_TYPE)) + if (info.parsed_conditions[type]) + mixers.mixers[type].add(info.parsed_conditions[type], policy.isRestrictive()); } } @@ -294,11 +301,11 @@ void RowPolicyContextFactory::mixConditionsForContext(RowPolicyContext & context database_and_table_name_keeper->second}]; mixed_conditions.database_and_table_name_keeper = std::move(database_and_table_name_keeper); mixed_conditions.policy_ids = std::move(mixers.policy_ids); - for (auto index : ext::range(0, MAX_CONDITION_INDEX)) - mixed_conditions.mixed_conditions[index] = std::move(mixers.mixers[index]).getResult(); + for (auto type : ext::range(0, MAX_CONDITION_TYPE)) + mixed_conditions.mixed_conditions[type] = std::move(mixers.mixers[type]).getResult(); } - context.map_of_mixed_conditions.store(map_of_mixed_conditions); + enabled.map_of_mixed_conditions.store(map_of_mixed_conditions); } } diff --git a/dbms/src/Access/RowPolicyContextFactory.h b/dbms/src/Access/RowPolicyCache.h similarity index 56% rename from dbms/src/Access/RowPolicyContextFactory.h rename to dbms/src/Access/RowPolicyCache.h index d93d1626b24..d0ec74b9ab8 100644 --- a/dbms/src/Access/RowPolicyContextFactory.h +++ b/dbms/src/Access/RowPolicyCache.h @@ -1,8 +1,9 @@ #pragma once -#include +#include #include #include +#include #include @@ -11,39 +12,38 @@ namespace DB class AccessControlManager; /// Stores read and parsed row policies. -class RowPolicyContextFactory +class RowPolicyCache { public: - RowPolicyContextFactory(const AccessControlManager & access_control_manager_); - ~RowPolicyContextFactory(); + RowPolicyCache(const AccessControlManager & access_control_manager_); + ~RowPolicyCache(); - RowPolicyContextPtr createContext(const UUID & user_id, const std::vector & enabled_roles); + std::shared_ptr getEnabledRowPolicies(const UUID & user_id, const std::vector & enabled_roles); private: - using ParsedConditions = RowPolicyContext::ParsedConditions; + using ParsedConditions = EnabledRowPolicies::ParsedConditions; struct PolicyInfo { PolicyInfo(const RowPolicyPtr & policy_) { setPolicy(policy_); } void setPolicy(const RowPolicyPtr & policy_); - bool canUseWithContext(const RowPolicyContext & context) const; RowPolicyPtr policy; - const GenericRoleSet * roles = nullptr; + const ExtendedRoleSet * roles = nullptr; ParsedConditions parsed_conditions; }; void ensureAllRowPoliciesRead(); void rowPolicyAddedOrChanged(const UUID & policy_id, const RowPolicyPtr & new_policy); void rowPolicyRemoved(const UUID & policy_id); - void mixConditionsForAllContexts(); - void mixConditionsForContext(RowPolicyContext & context); + void mixConditions(); + void mixConditionsFor(EnabledRowPolicies & enabled); const AccessControlManager & access_control_manager; std::unordered_map all_policies; bool all_policies_read = false; ext::scope_guard subscription; - std::vector> contexts; + std::map> enabled_row_policies; std::mutex mutex; }; diff --git a/dbms/src/Access/User.h b/dbms/src/Access/User.h index 3a9b3cd7014..a01e0332a2c 100644 --- a/dbms/src/Access/User.h +++ b/dbms/src/Access/User.h @@ -4,7 +4,7 @@ #include #include #include -#include +#include #include #include @@ -21,7 +21,7 @@ struct User : public IAccessEntity AccessRights access_with_grant_option; boost::container::flat_set granted_roles; boost::container::flat_set granted_roles_with_admin_option; - GenericRoleSet default_roles = GenericRoleSet::AllTag{}; + ExtendedRoleSet default_roles = ExtendedRoleSet::AllTag{}; String profile; bool equal(const IAccessEntity & other) const override; diff --git a/dbms/src/Access/UsersConfigAccessStorage.cpp b/dbms/src/Access/UsersConfigAccessStorage.cpp index fb3c3d5b80c..b32be09e5dc 100644 --- a/dbms/src/Access/UsersConfigAccessStorage.cpp +++ b/dbms/src/Access/UsersConfigAccessStorage.cpp @@ -224,7 +224,7 @@ namespace limits.max[ResourceType::EXECUTION_TIME] = Quota::secondsToExecutionTime(config.getUInt64(interval_config + ".execution_time", Quota::UNLIMITED)); } - quota->roles.add(user_ids); + quota->to_roles.add(user_ids); return quota; } @@ -324,7 +324,7 @@ namespace auto policy = std::make_shared(); policy->setFullName(database, table_name, user_name); policy->conditions[RowPolicy::SELECT_FILTER] = filter; - policy->roles.add(generateID(typeid(User), user_name)); + policy->to_roles.add(generateID(typeid(User), user_name)); policies.push_back(policy); } } diff --git a/dbms/src/DataStreams/IBlockInputStream.cpp b/dbms/src/DataStreams/IBlockInputStream.cpp index 4bccdff6848..733bafdcf71 100644 --- a/dbms/src/DataStreams/IBlockInputStream.cpp +++ b/dbms/src/DataStreams/IBlockInputStream.cpp @@ -2,7 +2,7 @@ #include #include -#include +#include #include #include diff --git a/dbms/src/DataStreams/IBlockInputStream.h b/dbms/src/DataStreams/IBlockInputStream.h index 6fe8be079d8..aacd12bacd9 100644 --- a/dbms/src/DataStreams/IBlockInputStream.h +++ b/dbms/src/DataStreams/IBlockInputStream.h @@ -21,8 +21,7 @@ namespace ErrorCodes } class ProcessListElement; -class QuotaContext; -using QuotaContextPtr = std::shared_ptr; +class EnabledQuota; class QueryStatus; struct SortColumnDescription; using SortDescription = std::vector; @@ -219,7 +218,7 @@ public: /** Set the quota. If you set a quota on the amount of raw data, * then you should also set mode = LIMITS_TOTAL to LocalLimits with setLimits. */ - virtual void setQuota(const QuotaContextPtr & quota_) + virtual void setQuota(const std::shared_ptr & quota_) { quota = quota_; } @@ -277,7 +276,7 @@ private: LocalLimits limits; - QuotaContextPtr quota; /// If nullptr - the quota is not used. + std::shared_ptr quota; /// If nullptr - the quota is not used. UInt64 prev_elapsed = 0; /// The approximate total number of rows to read. For progress bar. diff --git a/dbms/src/Functions/currentQuota.cpp b/dbms/src/Functions/currentQuota.cpp index d292627d1ca..b16a8a7c1ec 100644 --- a/dbms/src/Functions/currentQuota.cpp +++ b/dbms/src/Functions/currentQuota.cpp @@ -3,7 +3,8 @@ #include #include #include -#include +#include +#include #include diff --git a/dbms/src/Functions/currentRowPolicies.cpp b/dbms/src/Functions/currentRowPolicies.cpp index dfebf1552bc..0248f77c9b5 100644 --- a/dbms/src/Functions/currentRowPolicies.cpp +++ b/dbms/src/Functions/currentRowPolicies.cpp @@ -9,7 +9,7 @@ #include #include #include -#include +#include #include #include #include @@ -65,17 +65,20 @@ public: auto database_column = ColumnString::create(); auto table_name_column = ColumnString::create(); auto policy_name_column = ColumnString::create(); - for (const auto & policy_id : context.getRowPolicy()->getCurrentPolicyIDs()) + if (auto policies = context.getRowPolicies()) { - const auto policy = context.getAccessControlManager().tryRead(policy_id); - if (policy) + for (const auto & policy_id : policies->getCurrentPolicyIDs()) { - const String database = policy->getDatabase(); - const String table_name = policy->getTableName(); - const String policy_name = policy->getName(); - database_column->insertData(database.data(), database.length()); - table_name_column->insertData(table_name.data(), table_name.length()); - policy_name_column->insertData(policy_name.data(), policy_name.length()); + const auto policy = context.getAccessControlManager().tryRead(policy_id); + if (policy) + { + const String database = policy->getDatabase(); + const String table_name = policy->getTableName(); + const String policy_name = policy->getName(); + database_column->insertData(database.data(), database.length()); + table_name_column->insertData(table_name.data(), table_name.length()); + policy_name_column->insertData(policy_name.data(), policy_name.length()); + } } } auto offset_column = ColumnArray::ColumnOffsets::create(); @@ -113,13 +116,16 @@ public: { String database = database_column ? database_column->getDataAt(i).toString() : context.getCurrentDatabase(); String table_name = table_name_column->getDataAt(i).toString(); - for (const auto & policy_id : context.getRowPolicy()->getCurrentPolicyIDs(database, table_name)) + if (auto policies = context.getRowPolicies()) { - const auto policy = context.getAccessControlManager().tryRead(policy_id); - if (policy) + for (const auto & policy_id : policies->getCurrentPolicyIDs(database, table_name)) { - const String policy_name = policy->getName(); - policy_name_column->insertData(policy_name.data(), policy_name.length()); + const auto policy = context.getAccessControlManager().tryRead(policy_id); + if (policy) + { + const String policy_name = policy->getName(); + policy_name_column->insertData(policy_name.data(), policy_name.length()); + } } } offset_column->insertValue(policy_name_column->size()); @@ -169,8 +175,11 @@ public: if (arguments.empty()) { auto policy_id_column = ColumnVector::create(); - for (const auto & policy_id : context.getRowPolicy()->getCurrentPolicyIDs()) - policy_id_column->insertValue(policy_id); + if (auto policies = context.getRowPolicies()) + { + for (const auto & policy_id : policies->getCurrentPolicyIDs()) + policy_id_column->insertValue(policy_id); + } auto offset_column = ColumnArray::ColumnOffsets::create(); offset_column->insertValue(policy_id_column->size()); block.getByPosition(result_pos).column @@ -203,8 +212,11 @@ public: { String database = database_column ? database_column->getDataAt(i).toString() : context.getCurrentDatabase(); String table_name = table_name_column->getDataAt(i).toString(); - for (const auto & policy_id : context.getRowPolicy()->getCurrentPolicyIDs(database, table_name)) - policy_id_column->insertValue(policy_id); + if (auto policies = context.getRowPolicies()) + { + for (const auto & policy_id : policies->getCurrentPolicyIDs(database, table_name)) + policy_id_column->insertValue(policy_id); + } offset_column->insertValue(policy_id_column->size()); } diff --git a/dbms/src/Interpreters/Context.cpp b/dbms/src/Interpreters/Context.cpp index 81fdcae69c8..35c8d8a5f76 100644 --- a/dbms/src/Interpreters/Context.cpp +++ b/dbms/src/Interpreters/Context.cpp @@ -27,8 +27,8 @@ #include #include #include -#include -#include +#include +#include #include #include #include @@ -444,8 +444,6 @@ Context & Context::operator=(const Context &) = default; Context Context::createGlobal() { Context res; - res.access_rights = std::make_shared(); - res.initial_row_policy = std::make_shared(); res.shared = std::make_shared(); return res; } @@ -632,22 +630,22 @@ void Context::setUser(const String & name, const String & password, const Poco:: client_info.quota_key = quota_key; auto new_user_id = getAccessControlManager().find(name); - AccessRightsContextPtr new_access_rights; + std::shared_ptr new_access; if (new_user_id) { - new_access_rights = getAccessControlManager().getAccessRightsContext(*new_user_id, {}, true, settings, current_database, client_info); - if (!new_access_rights->isClientHostAllowed() || !new_access_rights->isCorrectPassword(password)) + new_access = getAccessControlManager().getContextAccess(*new_user_id, {}, true, settings, current_database, client_info); + if (!new_access->isClientHostAllowed() || !new_access->isCorrectPassword(password)) { new_user_id = {}; - new_access_rights = nullptr; + new_access = nullptr; } } - if (!new_user_id || !new_access_rights) + if (!new_user_id || !new_access) throw Exception(name + ": Authentication failed: password is incorrect or there is no user with such name", ErrorCodes::AUTHENTICATION_FAILED); user_id = new_user_id; - access_rights = std::move(new_access_rights); + access = std::move(new_access); current_roles.clear(); use_default_roles = true; @@ -657,13 +655,13 @@ void Context::setUser(const String & name, const String & password, const Poco:: std::shared_ptr Context::getUser() const { auto lock = getLock(); - return access_rights->getUser(); + return access->getUser(); } String Context::getUserName() const { auto lock = getLock(); - return access_rights->getUserName(); + return access->getUserName(); } UUID Context::getUserID() const @@ -697,22 +695,22 @@ void Context::setCurrentRolesDefault() std::vector Context::getCurrentRoles() const { - return getAccessRights()->getCurrentRoles(); + return getAccess()->getCurrentRoles(); } Strings Context::getCurrentRolesNames() const { - return getAccessRights()->getCurrentRolesNames(); + return getAccess()->getCurrentRolesNames(); } std::vector Context::getEnabledRoles() const { - return getAccessRights()->getEnabledRoles(); + return getAccess()->getEnabledRoles(); } Strings Context::getEnabledRolesNames() const { - return getAccessRights()->getEnabledRolesNames(); + return getAccess()->getEnabledRolesNames(); } @@ -720,59 +718,61 @@ void Context::calculateAccessRights() { auto lock = getLock(); if (user_id) - access_rights = getAccessControlManager().getAccessRightsContext(*user_id, current_roles, use_default_roles, settings, current_database, client_info); + access = getAccessControlManager().getContextAccess(*user_id, current_roles, use_default_roles, settings, current_database, client_info); } template void Context::checkAccessImpl(const Args &... args) const { - getAccessRights()->checkAccess(args...); + return getAccess()->checkAccess(args...); } -void Context::checkAccess(const AccessFlags & access) const { return checkAccessImpl(access); } -void Context::checkAccess(const AccessFlags & access, const std::string_view & database) const { return checkAccessImpl(access, database); } -void Context::checkAccess(const AccessFlags & access, const std::string_view & database, const std::string_view & table) const { return checkAccessImpl(access, database, table); } -void Context::checkAccess(const AccessFlags & access, const std::string_view & database, const std::string_view & table, const std::string_view & column) const { return checkAccessImpl(access, database, table, column); } -void Context::checkAccess(const AccessFlags & access, const std::string_view & database, const std::string_view & table, const std::vector & columns) const { return checkAccessImpl(access, database, table, columns); } -void Context::checkAccess(const AccessFlags & access, const std::string_view & database, const std::string_view & table, const Strings & columns) const { return checkAccessImpl(access, database, table, columns); } -void Context::checkAccess(const AccessRightsElement & access) const { return checkAccessImpl(access); } -void Context::checkAccess(const AccessRightsElements & access) const { return checkAccessImpl(access); } +void Context::checkAccess(const AccessFlags & flags) const { return checkAccessImpl(flags); } +void Context::checkAccess(const AccessFlags & flags, const std::string_view & database) const { return checkAccessImpl(flags, database); } +void Context::checkAccess(const AccessFlags & flags, const std::string_view & database, const std::string_view & table) const { return checkAccessImpl(flags, database, table); } +void Context::checkAccess(const AccessFlags & flags, const std::string_view & database, const std::string_view & table, const std::string_view & column) const { return checkAccessImpl(flags, database, table, column); } +void Context::checkAccess(const AccessFlags & flags, const std::string_view & database, const std::string_view & table, const std::vector & columns) const { return checkAccessImpl(flags, database, table, columns); } +void Context::checkAccess(const AccessFlags & flags, const std::string_view & database, const std::string_view & table, const Strings & columns) const { return checkAccessImpl(flags, database, table, columns); } +void Context::checkAccess(const AccessFlags & flags, const StorageID & table_id) const { checkAccessImpl(flags, table_id.getDatabaseName(), table_id.getTableName()); } +void Context::checkAccess(const AccessFlags & flags, const StorageID & table_id, const std::string_view & column) const { checkAccessImpl(flags, table_id.getDatabaseName(), table_id.getTableName(), column); } +void Context::checkAccess(const AccessFlags & flags, const StorageID & table_id, const std::vector & columns) const { checkAccessImpl(flags, table_id.getDatabaseName(), table_id.getTableName(), columns); } +void Context::checkAccess(const AccessFlags & flags, const StorageID & table_id, const Strings & columns) const { checkAccessImpl(flags, table_id.getDatabaseName(), table_id.getTableName(), columns); } +void Context::checkAccess(const AccessRightsElement & element) const { return checkAccessImpl(element); } +void Context::checkAccess(const AccessRightsElements & elements) const { return checkAccessImpl(elements); } -void Context::checkAccess(const AccessFlags & access, const StorageID & table_id) const { checkAccessImpl(access, table_id.getDatabaseName(), table_id.getTableName()); } -void Context::checkAccess(const AccessFlags & access, const StorageID & table_id, const std::string_view & column) const { checkAccessImpl(access, table_id.getDatabaseName(), table_id.getTableName(), column); } -void Context::checkAccess(const AccessFlags & access, const StorageID & table_id, const std::vector & columns) const { checkAccessImpl(access, table_id.getDatabaseName(), table_id.getTableName(), columns); } -void Context::checkAccess(const AccessFlags & access, const StorageID & table_id, const Strings & columns) const { checkAccessImpl(access, table_id.getDatabaseName(), table_id.getTableName(), columns); } -AccessRightsContextPtr Context::getAccessRights() const +std::shared_ptr Context::getAccess() const { auto lock = getLock(); - return access_rights; + return access ? access : ContextAccess::getFullAccess(); } -RowPolicyContextPtr Context::getRowPolicy() const +ASTPtr Context::getRowPolicyCondition(const String & database, const String & table_name, RowPolicy::ConditionType type) const { - return getAccessRights()->getRowPolicy(); + auto lock = getLock(); + auto initial_condition = initial_row_policy ? initial_row_policy->getCondition(database, table_name, type) : nullptr; + return getAccess()->getRowPolicyCondition(database, table_name, type, initial_condition); +} + +std::shared_ptr Context::getRowPolicies() const +{ + return getAccess()->getRowPolicies(); } void Context::setInitialRowPolicy() { auto lock = getLock(); auto initial_user_id = getAccessControlManager().find(client_info.initial_user); + initial_row_policy = nullptr; if (initial_user_id) - initial_row_policy = getAccessControlManager().getRowPolicyContext(*initial_user_id, {}); -} - -RowPolicyContextPtr Context::getInitialRowPolicy() const -{ - auto lock = getLock(); - return initial_row_policy; + initial_row_policy = getAccessControlManager().getEnabledRowPolicies(*initial_user_id, {}); } -QuotaContextPtr Context::getQuota() const +std::shared_ptr Context::getQuota() const { - return getAccessRights()->getQuota(); + return getAccess()->getQuota(); } diff --git a/dbms/src/Interpreters/Context.h b/dbms/src/Interpreters/Context.h index f21348a061c..14e1346dea1 100644 --- a/dbms/src/Interpreters/Context.h +++ b/dbms/src/Interpreters/Context.h @@ -9,6 +9,7 @@ #include #include #include +#include #include #include #include @@ -44,14 +45,11 @@ namespace DB struct ContextShared; class Context; -class AccessRightsContext; -using AccessRightsContextPtr = std::shared_ptr; +class ContextAccess; struct User; using UserPtr = std::shared_ptr; -class RowPolicyContext; -using RowPolicyContextPtr = std::shared_ptr; -class QuotaContext; -using QuotaContextPtr = std::shared_ptr; +class EnabledRowPolicies; +class EnabledQuota; class AccessFlags; struct AccessRightsElement; class AccessRightsElements; @@ -151,8 +149,8 @@ private: std::optional user_id; std::vector current_roles; bool use_default_roles = false; - AccessRightsContextPtr access_rights; - RowPolicyContextPtr initial_row_policy; + std::shared_ptr access; + std::shared_ptr initial_row_policy; String current_database; Settings settings; /// Setting for query execution. std::shared_ptr settings_constraints; @@ -246,31 +244,30 @@ public: /// Checks access rights. /// Empty database means the current database. - void checkAccess(const AccessFlags & access) const; - void checkAccess(const AccessFlags & access, const std::string_view & database) const; - void checkAccess(const AccessFlags & access, const std::string_view & database, const std::string_view & table) const; - void checkAccess(const AccessFlags & access, const std::string_view & database, const std::string_view & table, const std::string_view & column) const; - void checkAccess(const AccessFlags & access, const std::string_view & database, const std::string_view & table, const std::vector & columns) const; - void checkAccess(const AccessFlags & access, const std::string_view & database, const std::string_view & table, const Strings & columns) const; - void checkAccess(const AccessRightsElement & access) const; - void checkAccess(const AccessRightsElements & access) const; + void checkAccess(const AccessFlags & flags) const; + void checkAccess(const AccessFlags & flags, const std::string_view & database) const; + void checkAccess(const AccessFlags & flags, const std::string_view & database, const std::string_view & table) const; + void checkAccess(const AccessFlags & flags, const std::string_view & database, const std::string_view & table, const std::string_view & column) const; + void checkAccess(const AccessFlags & flags, const std::string_view & database, const std::string_view & table, const std::vector & columns) const; + void checkAccess(const AccessFlags & flags, const std::string_view & database, const std::string_view & table, const Strings & columns) const; + void checkAccess(const AccessFlags & flags, const StorageID & table_id) const; + void checkAccess(const AccessFlags & flags, const StorageID & table_id, const std::string_view & column) const; + void checkAccess(const AccessFlags & flags, const StorageID & table_id, const std::vector & columns) const; + void checkAccess(const AccessFlags & flags, const StorageID & table_id, const Strings & columns) const; + void checkAccess(const AccessRightsElement & element) const; + void checkAccess(const AccessRightsElements & elements) const; - void checkAccess(const AccessFlags & access, const StorageID & table_id) const; - void checkAccess(const AccessFlags & access, const StorageID & table_id, const std::string_view & column) const; - void checkAccess(const AccessFlags & access, const StorageID & table_id, const std::vector & columns) const; - void checkAccess(const AccessFlags & access, const StorageID & table_id, const Strings & columns) const; + std::shared_ptr getAccess() const; - AccessRightsContextPtr getAccessRights() const; - - RowPolicyContextPtr getRowPolicy() const; + std::shared_ptr getRowPolicies() const; + ASTPtr getRowPolicyCondition(const String & database, const String & table_name, RowPolicy::ConditionType type) const; /// Sets an extra row policy based on `client_info.initial_user`, if it exists. /// TODO: we need a better solution here. It seems we should pass the initial row policy /// because a shard is allowed to don't have the initial user or it may be another user with the same name. void setInitialRowPolicy(); - RowPolicyContextPtr getInitialRowPolicy() const; - QuotaContextPtr getQuota() const; + std::shared_ptr getQuota() const; /// We have to copy external tables inside executeQuery() to track limits. Therefore, set callback for it. Must set once. void setExternalTablesInitializer(ExternalTablesInitializer && initializer); diff --git a/dbms/src/Interpreters/InterpreterCreateQuotaQuery.cpp b/dbms/src/Interpreters/InterpreterCreateQuotaQuery.cpp index 10c52a5b4fb..4b64615dd36 100644 --- a/dbms/src/Interpreters/InterpreterCreateQuotaQuery.cpp +++ b/dbms/src/Interpreters/InterpreterCreateQuotaQuery.cpp @@ -1,6 +1,6 @@ #include #include -#include +#include #include #include #include @@ -14,7 +14,7 @@ namespace DB { namespace { -void updateQuotaFromQueryImpl(Quota & quota, const ASTCreateQuotaQuery & query, const std::optional & roles_from_query = {}) +void updateQuotaFromQueryImpl(Quota & quota, const ASTCreateQuotaQuery & query, const std::optional & roles_from_query = {}) { if (query.alter) { @@ -61,15 +61,15 @@ void updateQuotaFromQueryImpl(Quota & quota, const ASTCreateQuotaQuery & query, } } - const GenericRoleSet * roles = nullptr; - std::optional temp_role_set; + const ExtendedRoleSet * roles = nullptr; + std::optional temp_role_set; if (roles_from_query) roles = &*roles_from_query; else if (query.roles) roles = &temp_role_set.emplace(*query.roles); if (roles) - quota.roles = *roles; + quota.to_roles = *roles; } } @@ -80,9 +80,9 @@ BlockIO InterpreterCreateQuotaQuery::execute() auto & access_control = context.getAccessControlManager(); context.checkAccess(query.alter ? AccessType::ALTER_QUOTA : AccessType::CREATE_QUOTA); - std::optional roles_from_query; + std::optional roles_from_query; if (query.roles) - roles_from_query = GenericRoleSet{*query.roles, access_control, context.getUserID()}; + roles_from_query = ExtendedRoleSet{*query.roles, access_control, context.getUserID()}; if (query.alter) { diff --git a/dbms/src/Interpreters/InterpreterCreateRowPolicyQuery.cpp b/dbms/src/Interpreters/InterpreterCreateRowPolicyQuery.cpp index 2d0e23d284e..9ea47aba7bb 100644 --- a/dbms/src/Interpreters/InterpreterCreateRowPolicyQuery.cpp +++ b/dbms/src/Interpreters/InterpreterCreateRowPolicyQuery.cpp @@ -1,6 +1,6 @@ #include #include -#include +#include #include #include #include @@ -27,7 +27,7 @@ namespace void updateRowPolicyFromQueryImpl( RowPolicy & policy, const ASTCreateRowPolicyQuery & query, - const std::optional & roles_from_query = {}, + const std::optional & roles_from_query = {}, const String & current_database = {}) { if (query.alter) @@ -48,15 +48,15 @@ namespace for (const auto & [index, condition] : query.conditions) policy.conditions[index] = condition ? serializeAST(*condition) : String{}; - const GenericRoleSet * roles = nullptr; - std::optional temp_role_set; + const ExtendedRoleSet * roles = nullptr; + std::optional temp_role_set; if (roles_from_query) roles = &*roles_from_query; else if (query.roles) roles = &temp_role_set.emplace(*query.roles); if (roles) - policy.roles = *roles; + policy.to_roles = *roles; } } @@ -67,9 +67,9 @@ BlockIO InterpreterCreateRowPolicyQuery::execute() auto & access_control = context.getAccessControlManager(); context.checkAccess(query.alter ? AccessType::ALTER_POLICY : AccessType::CREATE_POLICY); - std::optional roles_from_query; + std::optional roles_from_query; if (query.roles) - roles_from_query = GenericRoleSet{*query.roles, access_control, context.getUserID()}; + roles_from_query = ExtendedRoleSet{*query.roles, access_control, context.getUserID()}; const String current_database = context.getCurrentDatabase(); diff --git a/dbms/src/Interpreters/InterpreterCreateUserQuery.cpp b/dbms/src/Interpreters/InterpreterCreateUserQuery.cpp index 6219a493b27..f01e2045a2a 100644 --- a/dbms/src/Interpreters/InterpreterCreateUserQuery.cpp +++ b/dbms/src/Interpreters/InterpreterCreateUserQuery.cpp @@ -4,8 +4,8 @@ #include #include #include -#include -#include +#include +#include #include @@ -13,7 +13,7 @@ namespace DB { namespace { - void updateUserFromQueryImpl(User & user, const ASTCreateUserQuery & query, const std::optional & default_roles_from_query = {}) + void updateUserFromQueryImpl(User & user, const ASTCreateUserQuery & query, const std::optional & default_roles_from_query = {}) { if (query.alter) { @@ -33,8 +33,8 @@ namespace if (query.add_hosts) user.allowed_client_hosts.add(*query.add_hosts); - const GenericRoleSet * default_roles = nullptr; - std::optional temp_role_set; + const ExtendedRoleSet * default_roles = nullptr; + std::optional temp_role_set; if (default_roles_from_query) default_roles = &*default_roles_from_query; else if (query.default_roles) @@ -58,16 +58,17 @@ BlockIO InterpreterCreateUserQuery::execute() { const auto & query = query_ptr->as(); auto & access_control = context.getAccessControlManager(); - context.checkAccess(query.alter ? AccessType::ALTER_USER : AccessType::CREATE_USER); + auto access = context.getAccess(); + access->checkAccess(query.alter ? AccessType::ALTER_USER : AccessType::CREATE_USER); - std::optional default_roles_from_query; + std::optional default_roles_from_query; if (query.default_roles) { - default_roles_from_query = GenericRoleSet{*query.default_roles, access_control}; + default_roles_from_query = ExtendedRoleSet{*query.default_roles, access_control}; if (!query.alter && !default_roles_from_query->all) { for (const UUID & role : default_roles_from_query->getMatchingIDs()) - context.getAccessRights()->checkAdminOption(role); + access->checkAdminOption(role); } } diff --git a/dbms/src/Interpreters/InterpreterGrantQuery.cpp b/dbms/src/Interpreters/InterpreterGrantQuery.cpp index fd38b1a8788..5d215ff3a93 100644 --- a/dbms/src/Interpreters/InterpreterGrantQuery.cpp +++ b/dbms/src/Interpreters/InterpreterGrantQuery.cpp @@ -2,8 +2,8 @@ #include #include #include -#include -#include +#include +#include #include #include #include @@ -61,17 +61,18 @@ BlockIO InterpreterGrantQuery::execute() { const auto & query = query_ptr->as(); auto & access_control = context.getAccessControlManager(); - context.getAccessRights()->checkGrantOption(query.access_rights_elements); + auto access = context.getAccess(); + access->checkGrantOption(query.access_rights_elements); std::vector roles_from_query; if (query.roles) { - roles_from_query = GenericRoleSet{*query.roles, access_control}.getMatchingRoles(access_control); + roles_from_query = ExtendedRoleSet{*query.roles, access_control}.getMatchingIDs(access_control); for (const UUID & role_from_query : roles_from_query) - context.getAccessRights()->checkAdminOption(role_from_query); + access->checkAdminOption(role_from_query); } - std::vector to_roles = GenericRoleSet{*query.to_roles, access_control, context.getUserID()}.getMatchingUsersAndRoles(access_control); + std::vector to_roles = ExtendedRoleSet{*query.to_roles, access_control, context.getUserID()}.getMatchingIDs(access_control); String current_database = context.getCurrentDatabase(); auto update_func = [&](const AccessEntityPtr & entity) -> AccessEntityPtr @@ -101,7 +102,7 @@ void InterpreterGrantQuery::updateUserFromQuery(User & user, const ASTGrantQuery { std::vector roles_from_query; if (query.roles) - roles_from_query = GenericRoleSet{*query.roles}.getMatchingIDs(); + roles_from_query = ExtendedRoleSet{*query.roles}.getMatchingIDs(); updateFromQueryImpl(user, query, roles_from_query, {}); } @@ -110,7 +111,7 @@ void InterpreterGrantQuery::updateRoleFromQuery(Role & role, const ASTGrantQuery { std::vector roles_from_query; if (query.roles) - roles_from_query = GenericRoleSet{*query.roles}.getMatchingIDs(); + roles_from_query = ExtendedRoleSet{*query.roles}.getMatchingIDs(); updateFromQueryImpl(role, query, roles_from_query, {}); } diff --git a/dbms/src/Interpreters/InterpreterKillQueryQuery.cpp b/dbms/src/Interpreters/InterpreterKillQueryQuery.cpp index dc365990794..196b2b4eef1 100644 --- a/dbms/src/Interpreters/InterpreterKillQueryQuery.cpp +++ b/dbms/src/Interpreters/InterpreterKillQueryQuery.cpp @@ -10,7 +10,7 @@ #include #include #include -#include +#include #include #include #include @@ -83,7 +83,18 @@ static QueryDescriptors extractQueriesExceptMeAndCheckAccess(const Block & proce const ColumnString & query_id_col = typeid_cast(*processes_block.getByName("query_id").column); const ColumnString & user_col = typeid_cast(*processes_block.getByName("user").column); const ClientInfo & my_client = context.getProcessListElement()->getClientInfo(); - std::optional can_kill_query_started_by_another_user; + + std::optional can_kill_query_started_by_another_user_cached; + auto can_kill_query_started_by_another_user = [&]() -> bool + { + if (!can_kill_query_started_by_another_user_cached) + { + can_kill_query_started_by_another_user_cached + = context.getAccess()->isGranted(&Poco::Logger::get("InterpreterKillQueryQuery"), AccessType::KILL_QUERY); + } + return *can_kill_query_started_by_another_user_cached; + }; + String query_user; bool access_denied = false; @@ -96,15 +107,10 @@ static QueryDescriptors extractQueriesExceptMeAndCheckAccess(const Block & proce auto query_id = query_id_col.getDataAt(i).toString(); query_user = user_col.getDataAt(i).toString(); - if (my_client.current_user != query_user) + if ((my_client.current_user != query_user) && !can_kill_query_started_by_another_user()) { - if (!can_kill_query_started_by_another_user) - can_kill_query_started_by_another_user = context.getAccessRights()->isGranted(&Poco::Logger::get("InterpreterKillQueryQuery"), AccessType::KILL_QUERY); - if (!*can_kill_query_started_by_another_user) - { - access_denied = true; - continue; - } + access_denied = true; + continue; } res.emplace_back(std::move(query_id), query_user, i, false); @@ -244,6 +250,7 @@ BlockIO InterpreterKillQueryQuery::execute() MutableColumns res_columns = header.cloneEmptyColumns(); auto table_id = StorageID::createEmpty(); AccessRightsElements required_access_rights; + auto access = context.getAccess(); bool access_denied = false; for (size_t i = 0; i < mutations_block.rows(); ++i) @@ -262,7 +269,7 @@ BlockIO InterpreterKillQueryQuery::execute() ParserAlterCommand parser; auto command_ast = parseQuery(parser, command_col.getDataAt(i).toString(), 0); required_access_rights = InterpreterAlterQuery::getRequiredAccessForCommand(command_ast->as(), table_id.database_name, table_id.table_name); - if (!context.getAccessRights()->isGranted(&Poco::Logger::get("InterpreterKillQueryQuery"), required_access_rights)) + if (!access->isGranted(&Poco::Logger::get("InterpreterKillQueryQuery"), required_access_rights)) { access_denied = true; continue; diff --git a/dbms/src/Interpreters/InterpreterSelectQuery.cpp b/dbms/src/Interpreters/InterpreterSelectQuery.cpp index 085ebe52557..c834b8cfaa2 100644 --- a/dbms/src/Interpreters/InterpreterSelectQuery.cpp +++ b/dbms/src/Interpreters/InterpreterSelectQuery.cpp @@ -38,7 +38,6 @@ #include #include -#include #include #include @@ -348,8 +347,7 @@ InterpreterSelectQuery::InterpreterSelectQuery( source_header = storage->getSampleBlockForColumns(required_columns); /// Fix source_header for filter actions. - auto row_policy_filter = context->getRowPolicy()->getCondition(table_id.getDatabaseName(), table_id.getTableName(), RowPolicy::SELECT_FILTER); - row_policy_filter = RowPolicyContext::combineConditionsUsingAnd(row_policy_filter, context->getInitialRowPolicy()->getCondition(table_id.getDatabaseName(), table_id.getTableName(), RowPolicy::SELECT_FILTER)); + auto row_policy_filter = context->getRowPolicyCondition(table_id.getDatabaseName(), table_id.getTableName(), RowPolicy::SELECT_FILTER); if (row_policy_filter) { filter_info = std::make_shared(); @@ -477,8 +475,7 @@ Block InterpreterSelectQuery::getSampleBlockImpl(bool try_move_to_prewhere) /// PREWHERE optimization. /// Turn off, if the table filter (row-level security) is applied. - if (!context->getRowPolicy()->getCondition(table_id.getDatabaseName(), table_id.getTableName(), RowPolicy::SELECT_FILTER) - && !context->getInitialRowPolicy()->getCondition(table_id.getDatabaseName(), table_id.getTableName(), RowPolicy::SELECT_FILTER)) + if (!context->getRowPolicyCondition(table_id.getDatabaseName(), table_id.getTableName(), RowPolicy::SELECT_FILTER)) { auto optimize_prewhere = [&](auto & merge_tree) { @@ -1115,7 +1112,7 @@ void InterpreterSelectQuery::executeFetchColumns( if (storage) { /// Append columns from the table filter to required - auto row_policy_filter = context->getRowPolicy()->getCondition(table_id.getDatabaseName(), table_id.getTableName(), RowPolicy::SELECT_FILTER); + auto row_policy_filter = context->getRowPolicyCondition(table_id.getDatabaseName(), table_id.getTableName(), RowPolicy::SELECT_FILTER); if (row_policy_filter) { auto initial_required_columns = required_columns; diff --git a/dbms/src/Interpreters/InterpreterSetRoleQuery.cpp b/dbms/src/Interpreters/InterpreterSetRoleQuery.cpp index 567c626cb90..2a6f2317a9c 100644 --- a/dbms/src/Interpreters/InterpreterSetRoleQuery.cpp +++ b/dbms/src/Interpreters/InterpreterSetRoleQuery.cpp @@ -1,8 +1,8 @@ #include #include -#include +#include #include -#include +#include #include #include @@ -38,7 +38,7 @@ void InterpreterSetRoleQuery::setRole(const ASTSetRoleQuery & query) } else { - GenericRoleSet roles_from_query{*query.roles, access_control}; + ExtendedRoleSet roles_from_query{*query.roles, access_control}; std::vector new_current_roles; if (roles_from_query.all) { @@ -65,8 +65,8 @@ void InterpreterSetRoleQuery::setDefaultRole(const ASTSetRoleQuery & query) context.checkAccess(AccessType::CREATE_USER | AccessType::DROP_USER); auto & access_control = context.getAccessControlManager(); - std::vector to_users = GenericRoleSet{*query.to_users, access_control, context.getUserID()}.getMatchingUsers(access_control); - GenericRoleSet roles_from_query{*query.roles, access_control}; + std::vector to_users = ExtendedRoleSet{*query.to_users, access_control, context.getUserID()}.getMatchingIDs(access_control); + ExtendedRoleSet roles_from_query{*query.roles, access_control}; auto update_func = [&](const AccessEntityPtr & entity) -> AccessEntityPtr { @@ -79,7 +79,7 @@ void InterpreterSetRoleQuery::setDefaultRole(const ASTSetRoleQuery & query) } -void InterpreterSetRoleQuery::updateUserSetDefaultRoles(User & user, const GenericRoleSet & roles_from_query) +void InterpreterSetRoleQuery::updateUserSetDefaultRoles(User & user, const ExtendedRoleSet & roles_from_query) { if (!roles_from_query.all) { diff --git a/dbms/src/Interpreters/InterpreterSetRoleQuery.h b/dbms/src/Interpreters/InterpreterSetRoleQuery.h index cace6b22c24..afb53014c87 100644 --- a/dbms/src/Interpreters/InterpreterSetRoleQuery.h +++ b/dbms/src/Interpreters/InterpreterSetRoleQuery.h @@ -7,7 +7,7 @@ namespace DB { class ASTSetRoleQuery; -struct GenericRoleSet; +struct ExtendedRoleSet; struct User; @@ -18,7 +18,7 @@ public: BlockIO execute() override; - static void updateUserSetDefaultRoles(User & user, const GenericRoleSet & roles_from_query); + static void updateUserSetDefaultRoles(User & user, const ExtendedRoleSet & roles_from_query); private: void setRole(const ASTSetRoleQuery & query); diff --git a/dbms/src/Interpreters/InterpreterShowCreateAccessEntityQuery.cpp b/dbms/src/Interpreters/InterpreterShowCreateAccessEntityQuery.cpp index 8c8658d820c..8f5aa4cc7d0 100644 --- a/dbms/src/Interpreters/InterpreterShowCreateAccessEntityQuery.cpp +++ b/dbms/src/Interpreters/InterpreterShowCreateAccessEntityQuery.cpp @@ -5,12 +5,13 @@ #include #include #include -#include +#include #include #include #include #include -#include +#include +#include #include #include #include @@ -45,12 +46,12 @@ namespace if (!user.profile.empty()) query->profile = user.profile; - if (user.default_roles != GenericRoleSet::AllTag{}) + if (user.default_roles != ExtendedRoleSet::AllTag{}) { if (attach_mode) - query->default_roles = GenericRoleSet{user.default_roles}.toAST(); + query->default_roles = ExtendedRoleSet{user.default_roles}.toAST(); else - query->default_roles = GenericRoleSet{user.default_roles}.toASTWithNames(*manager); + query->default_roles = ExtendedRoleSet{user.default_roles}.toASTWithNames(*manager); } if (attach_mode && (user.authentication.getType() != Authentication::NO_PASSWORD)) @@ -94,12 +95,12 @@ namespace query->all_limits.push_back(create_query_limits); } - if (!quota.roles.empty()) + if (!quota.to_roles.empty()) { if (attach_mode) - query->roles = quota.roles.toAST(); + query->roles = quota.to_roles.toAST(); else - query->roles = quota.roles.toASTWithNames(*manager); + query->roles = quota.to_roles.toASTWithNames(*manager); } return query; @@ -118,7 +119,7 @@ namespace if (policy.isRestrictive()) query->is_restrictive = policy.isRestrictive(); - for (auto index : ext::range_with_static_cast(RowPolicy::MAX_CONDITION_INDEX)) + for (auto index : ext::range_with_static_cast(RowPolicy::MAX_CONDITION_TYPE)) { const auto & condition = policy.conditions[index]; if (!condition.empty()) @@ -129,12 +130,12 @@ namespace } } - if (!policy.roles.empty()) + if (!policy.to_roles.empty()) { if (attach_mode) - query->roles = policy.roles.toAST(); + query->roles = policy.to_roles.toAST(); else - query->roles = policy.roles.toASTWithNames(*manager); + query->roles = policy.to_roles.toASTWithNames(*manager); } return query; diff --git a/dbms/src/Interpreters/InterpreterShowGrantsQuery.cpp b/dbms/src/Interpreters/InterpreterShowGrantsQuery.cpp index cbd4b3636ac..da1d46f0cab 100644 --- a/dbms/src/Interpreters/InterpreterShowGrantsQuery.cpp +++ b/dbms/src/Interpreters/InterpreterShowGrantsQuery.cpp @@ -1,7 +1,7 @@ #include #include #include -#include +#include #include #include #include @@ -62,7 +62,7 @@ namespace { ASTs res; - std::shared_ptr to_roles = std::make_shared(); + std::shared_ptr to_roles = std::make_shared(); to_roles->names.push_back(grantee.getName()); for (bool grant_option : {true, false}) @@ -104,9 +104,9 @@ namespace grant_query->admin_option = admin_option; grant_query->to_roles = to_roles; if (attach_mode) - grant_query->roles = GenericRoleSet{roles}.toAST(); + grant_query->roles = ExtendedRoleSet{roles}.toAST(); else - grant_query->roles = GenericRoleSet{roles}.toASTWithNames(*manager); + grant_query->roles = ExtendedRoleSet{roles}.toASTWithNames(*manager); res.push_back(std::move(grant_query)); } diff --git a/dbms/src/Interpreters/InterpreterSystemQuery.cpp b/dbms/src/Interpreters/InterpreterSystemQuery.cpp index e479a53cb03..87ed4a1f749 100644 --- a/dbms/src/Interpreters/InterpreterSystemQuery.cpp +++ b/dbms/src/Interpreters/InterpreterSystemQuery.cpp @@ -20,7 +20,7 @@ #include #include #include -#include +#include #include #include #include @@ -137,17 +137,17 @@ void InterpreterSystemQuery::startStopAction(StorageActionBlockType action_type, } else { + auto access = context.getAccess(); for (auto & elem : DatabaseCatalog::instance().getDatabases()) { for (auto iterator = elem.second->getTablesIterator(context); iterator->isValid(); iterator->next()) { - if (context.getAccessRights()->isGranted(log, getRequiredAccessType(action_type), elem.first, iterator->name())) - { - if (start) - manager->remove(iterator->table(), action_type); - else - manager->add(iterator->table(), action_type); - } + if (!access->isGranted(log, getRequiredAccessType(action_type), elem.first, iterator->name())) + continue; + if (start) + manager->remove(iterator->table(), action_type); + else + manager->add(iterator->table(), action_type); } } } diff --git a/dbms/src/Interpreters/executeQuery.cpp b/dbms/src/Interpreters/executeQuery.cpp index fefca6b580f..5c8c587fcc8 100644 --- a/dbms/src/Interpreters/executeQuery.cpp +++ b/dbms/src/Interpreters/executeQuery.cpp @@ -24,7 +24,7 @@ #include -#include +#include #include #include #include @@ -148,7 +148,8 @@ static void logException(Context & context, QueryLogElement & elem) static void onExceptionBeforeStart(const String & query_for_logging, Context & context, time_t current_time) { /// Exception before the query execution. - context.getQuota()->used(Quota::ERRORS, 1, /* check_exceeded = */ false); + if (auto quota = context.getQuota()) + quota->used(Quota::ERRORS, 1, /* check_exceeded = */ false); const Settings & settings = context.getSettingsRef(); @@ -307,12 +308,15 @@ static std::tuple executeQueryImpl( auto interpreter = InterpreterFactory::get(ast, context, stage); bool use_processors = settings.experimental_use_processors && allow_processors && interpreter->canExecuteWithProcessors(); - QuotaContextPtr quota; + std::shared_ptr quota; if (!interpreter->ignoreQuota()) { quota = context.getQuota(); - quota->used(Quota::QUERIES, 1); - quota->checkExceeded(Quota::ERRORS); + if (quota) + { + quota->used(Quota::QUERIES, 1); + quota->checkExceeded(Quota::ERRORS); + } } IBlockInputStream::LocalLimits limits; @@ -486,9 +490,10 @@ static std::tuple executeQueryImpl( } }; - auto exception_callback = [elem, &context, log_queries] () mutable + auto exception_callback = [elem, &context, log_queries, quota(quota)] () mutable { - context.getQuota()->used(Quota::ERRORS, 1, /* check_exceeded = */ false); + if (quota) + quota->used(Quota::ERRORS, 1, /* check_exceeded = */ false); elem.type = QueryLogElement::EXCEPTION_WHILE_PROCESSING; diff --git a/dbms/src/Parsers/ASTCreateQuotaQuery.cpp b/dbms/src/Parsers/ASTCreateQuotaQuery.cpp index 7e1017ae0c3..bb4a32f0556 100644 --- a/dbms/src/Parsers/ASTCreateQuotaQuery.cpp +++ b/dbms/src/Parsers/ASTCreateQuotaQuery.cpp @@ -1,5 +1,5 @@ #include -#include +#include #include #include #include @@ -94,7 +94,7 @@ namespace } } - void formatToRoles(const ASTGenericRoleSet & roles, const IAST::FormatSettings & settings) + void formatToRoles(const ASTExtendedRoleSet & roles, const IAST::FormatSettings & settings) { settings.ostr << (settings.hilite ? IAST::hilite_keyword : "") << " TO " << (settings.hilite ? IAST::hilite_none : ""); roles.format(settings); diff --git a/dbms/src/Parsers/ASTCreateQuotaQuery.h b/dbms/src/Parsers/ASTCreateQuotaQuery.h index 71b1b95d894..2968c2cc607 100644 --- a/dbms/src/Parsers/ASTCreateQuotaQuery.h +++ b/dbms/src/Parsers/ASTCreateQuotaQuery.h @@ -6,7 +6,7 @@ namespace DB { -class ASTGenericRoleSet; +class ASTExtendedRoleSet; /** CREATE QUOTA [IF NOT EXISTS | OR REPLACE] name @@ -53,7 +53,7 @@ public: }; std::vector all_limits; - std::shared_ptr roles; + std::shared_ptr roles; String getID(char) const override; ASTPtr clone() const override; diff --git a/dbms/src/Parsers/ASTCreateRowPolicyQuery.cpp b/dbms/src/Parsers/ASTCreateRowPolicyQuery.cpp index 0e3002d385f..8c7b3aa6e86 100644 --- a/dbms/src/Parsers/ASTCreateRowPolicyQuery.cpp +++ b/dbms/src/Parsers/ASTCreateRowPolicyQuery.cpp @@ -1,5 +1,5 @@ #include -#include +#include #include #include #include @@ -10,7 +10,7 @@ namespace DB { namespace { - using ConditionIndex = RowPolicy::ConditionIndex; + using ConditionType = RowPolicy::ConditionType; void formatRenameTo(const String & new_policy_name, const IAST::FormatSettings & settings) { @@ -37,13 +37,13 @@ namespace } - std::vector> - conditionalExpressionsToStrings(const std::vector> & exprs, const IAST::FormatSettings & settings) + std::vector> + conditionalExpressionsToStrings(const std::vector> & exprs, const IAST::FormatSettings & settings) { - std::vector> result; + std::vector> result; std::stringstream ss; IAST::FormatSettings temp_settings(ss, settings); - boost::range::transform(exprs, std::back_inserter(result), [&](const std::pair & in) + boost::range::transform(exprs, std::back_inserter(result), [&](const std::pair & in) { formatConditionalExpression(in.second, temp_settings); auto out = std::pair{in.first, ss.str()}; @@ -70,9 +70,9 @@ namespace } - void formatMultipleConditions(const std::vector> & conditions, bool alter, const IAST::FormatSettings & settings) + void formatMultipleConditions(const std::vector> & conditions, bool alter, const IAST::FormatSettings & settings) { - std::optional scond[RowPolicy::MAX_CONDITION_INDEX]; + std::optional scond[RowPolicy::MAX_CONDITION_TYPE]; for (const auto & [index, scondition] : conditionalExpressionsToStrings(conditions, settings)) scond[index] = scondition; @@ -112,7 +112,7 @@ namespace } } - void formatToRoles(const ASTGenericRoleSet & roles, const IAST::FormatSettings & settings) + void formatToRoles(const ASTExtendedRoleSet & roles, const IAST::FormatSettings & settings) { settings.ostr << (settings.hilite ? IAST::hilite_keyword : "") << " TO " << (settings.hilite ? IAST::hilite_none : ""); roles.format(settings); diff --git a/dbms/src/Parsers/ASTCreateRowPolicyQuery.h b/dbms/src/Parsers/ASTCreateRowPolicyQuery.h index 9c233799639..e58ed0ec46c 100644 --- a/dbms/src/Parsers/ASTCreateRowPolicyQuery.h +++ b/dbms/src/Parsers/ASTCreateRowPolicyQuery.h @@ -8,7 +8,7 @@ namespace DB { -class ASTGenericRoleSet; +class ASTExtendedRoleSet; /** CREATE [ROW] POLICY [IF NOT EXISTS | OR REPLACE] name ON [database.]table * [AS {PERMISSIVE | RESTRICTIVE}] @@ -39,10 +39,10 @@ public: String new_policy_name; std::optional is_restrictive; - using ConditionIndex = RowPolicy::ConditionIndex; - std::vector> conditions; + using ConditionType = RowPolicy::ConditionType; + std::vector> conditions; - std::shared_ptr roles; + std::shared_ptr roles; String getID(char) const override; ASTPtr clone() const override; diff --git a/dbms/src/Parsers/ASTCreateUserQuery.cpp b/dbms/src/Parsers/ASTCreateUserQuery.cpp index e848a5e0abb..94b2a35a314 100644 --- a/dbms/src/Parsers/ASTCreateUserQuery.cpp +++ b/dbms/src/Parsers/ASTCreateUserQuery.cpp @@ -1,5 +1,5 @@ #include -#include +#include #include @@ -135,7 +135,7 @@ namespace } - void formatDefaultRoles(const ASTGenericRoleSet & default_roles, const IAST::FormatSettings & settings) + void formatDefaultRoles(const ASTExtendedRoleSet & default_roles, const IAST::FormatSettings & settings) { settings.ostr << (settings.hilite ? IAST::hilite_keyword : "") << " DEFAULT ROLE " << (settings.hilite ? IAST::hilite_none : ""); default_roles.format(settings); diff --git a/dbms/src/Parsers/ASTCreateUserQuery.h b/dbms/src/Parsers/ASTCreateUserQuery.h index d6db56a408f..0357fb60720 100644 --- a/dbms/src/Parsers/ASTCreateUserQuery.h +++ b/dbms/src/Parsers/ASTCreateUserQuery.h @@ -7,7 +7,7 @@ namespace DB { -class ASTGenericRoleSet; +class ASTExtendedRoleSet; /** CREATE USER [IF NOT EXISTS | OR REPLACE] name * [IDENTIFIED [WITH {NO_PASSWORD|PLAINTEXT_PASSWORD|SHA256_PASSWORD|SHA256_HASH|DOUBLE_SHA1_PASSWORD|DOUBLE_SHA1_HASH}] BY {'password'|'hash'}] @@ -41,7 +41,7 @@ public: std::optional add_hosts; std::optional remove_hosts; - std::shared_ptr default_roles; + std::shared_ptr default_roles; std::optional profile; diff --git a/dbms/src/Parsers/ASTGenericRoleSet.cpp b/dbms/src/Parsers/ASTExtendedRoleSet.cpp similarity index 93% rename from dbms/src/Parsers/ASTGenericRoleSet.cpp rename to dbms/src/Parsers/ASTExtendedRoleSet.cpp index 50f2b0adc7e..3ac1052897d 100644 --- a/dbms/src/Parsers/ASTGenericRoleSet.cpp +++ b/dbms/src/Parsers/ASTExtendedRoleSet.cpp @@ -1,4 +1,4 @@ -#include +#include #include @@ -20,7 +20,7 @@ namespace } } -void ASTGenericRoleSet::formatImpl(const FormatSettings & settings, FormatState &, FormatStateStacked) const +void ASTExtendedRoleSet::formatImpl(const FormatSettings & settings, FormatState &, FormatStateStacked) const { if (empty()) { diff --git a/dbms/src/Parsers/ASTGenericRoleSet.h b/dbms/src/Parsers/ASTExtendedRoleSet.h similarity index 77% rename from dbms/src/Parsers/ASTGenericRoleSet.h rename to dbms/src/Parsers/ASTExtendedRoleSet.h index b9a1ab99248..17edbceed30 100644 --- a/dbms/src/Parsers/ASTGenericRoleSet.h +++ b/dbms/src/Parsers/ASTExtendedRoleSet.h @@ -8,7 +8,7 @@ namespace DB { /// Represents a set of users/roles like /// {user_name | role_name | CURRENT_USER} [,...] | NONE | ALL | ALL EXCEPT {user_name | role_name | CURRENT_USER} [,...] -class ASTGenericRoleSet : public IAST +class ASTExtendedRoleSet : public IAST { public: Strings names; @@ -20,8 +20,8 @@ public: bool empty() const { return names.empty() && !current_user && !all; } - String getID(char) const override { return "GenericRoleSet"; } - ASTPtr clone() const override { return std::make_shared(*this); } + String getID(char) const override { return "ExtendedRoleSet"; } + ASTPtr clone() const override { return std::make_shared(*this); } void formatImpl(const FormatSettings & settings, FormatState &, FormatStateStacked) const override; }; } diff --git a/dbms/src/Parsers/ASTGrantQuery.cpp b/dbms/src/Parsers/ASTGrantQuery.cpp index 9365e1b96b7..94521d790f2 100644 --- a/dbms/src/Parsers/ASTGrantQuery.cpp +++ b/dbms/src/Parsers/ASTGrantQuery.cpp @@ -1,5 +1,5 @@ #include -#include +#include #include #include #include @@ -97,7 +97,7 @@ namespace } - void formatToRoles(const ASTGenericRoleSet & to_roles, ASTGrantQuery::Kind kind, const IAST::FormatSettings & settings) + void formatToRoles(const ASTExtendedRoleSet & to_roles, ASTGrantQuery::Kind kind, const IAST::FormatSettings & settings) { using Kind = ASTGrantQuery::Kind; settings.ostr << (settings.hilite ? IAST::hilite_keyword : "") << ((kind == Kind::GRANT) ? " TO " : " FROM ") diff --git a/dbms/src/Parsers/ASTGrantQuery.h b/dbms/src/Parsers/ASTGrantQuery.h index 8ce3d9c20dc..95b5f0b8448 100644 --- a/dbms/src/Parsers/ASTGrantQuery.h +++ b/dbms/src/Parsers/ASTGrantQuery.h @@ -6,7 +6,7 @@ namespace DB { -class ASTGenericRoleSet; +class ASTExtendedRoleSet; /** GRANT access_type[(column_name [,...])] [,...] ON {db.table|db.*|*.*|table|*} TO {user_name | CURRENT_USER} [,...] [WITH GRANT OPTION] @@ -26,8 +26,8 @@ public: Kind kind = Kind::GRANT; bool attach = false; AccessRightsElements access_rights_elements; - std::shared_ptr roles; - std::shared_ptr to_roles; + std::shared_ptr roles; + std::shared_ptr to_roles; bool grant_option = false; bool admin_option = false; diff --git a/dbms/src/Parsers/ASTSetRoleQuery.cpp b/dbms/src/Parsers/ASTSetRoleQuery.cpp index de61f5a3113..0c8842fdac6 100644 --- a/dbms/src/Parsers/ASTSetRoleQuery.cpp +++ b/dbms/src/Parsers/ASTSetRoleQuery.cpp @@ -1,5 +1,5 @@ #include -#include +#include #include diff --git a/dbms/src/Parsers/ASTSetRoleQuery.h b/dbms/src/Parsers/ASTSetRoleQuery.h index ad22d30e287..8f1fb357d86 100644 --- a/dbms/src/Parsers/ASTSetRoleQuery.h +++ b/dbms/src/Parsers/ASTSetRoleQuery.h @@ -5,7 +5,7 @@ namespace DB { -class ASTGenericRoleSet; +class ASTExtendedRoleSet; /** SET ROLE {DEFAULT | NONE | role [,...] | ALL | ALL EXCEPT role [,...]} * SET DEFAULT ROLE {NONE | role [,...] | ALL | ALL EXCEPT role [,...]} TO {user|CURRENT_USER} [,...] @@ -21,8 +21,8 @@ public: }; Kind kind = Kind::SET_ROLE; - std::shared_ptr roles; - std::shared_ptr to_users; + std::shared_ptr roles; + std::shared_ptr to_users; String getID(char) const override; ASTPtr clone() const override; diff --git a/dbms/src/Parsers/ParserCreateQuotaQuery.cpp b/dbms/src/Parsers/ParserCreateQuotaQuery.cpp index c03fb14874c..d59c9832d79 100644 --- a/dbms/src/Parsers/ParserCreateQuotaQuery.cpp +++ b/dbms/src/Parsers/ParserCreateQuotaQuery.cpp @@ -3,10 +3,10 @@ #include #include #include -#include +#include #include #include -#include +#include #include #include @@ -187,15 +187,15 @@ namespace }); } - bool parseToRoles(IParserBase::Pos & pos, Expected & expected, bool id_mode, std::shared_ptr & roles) + bool parseToRoles(IParserBase::Pos & pos, Expected & expected, bool id_mode, std::shared_ptr & roles) { return IParserBase::wrapParseImpl(pos, [&] { ASTPtr node; - if (roles || !ParserKeyword{"TO"}.ignore(pos, expected) || !ParserGenericRoleSet{}.enableIDMode(id_mode).parse(pos, node, expected)) + if (roles || !ParserKeyword{"TO"}.ignore(pos, expected) || !ParserExtendedRoleSet{}.useIDMode(id_mode).parse(pos, node, expected)) return false; - roles = std::static_pointer_cast(node); + roles = std::static_pointer_cast(node); return true; }); } @@ -243,7 +243,7 @@ bool ParserCreateQuotaQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expe String new_name; std::optional key_type; std::vector all_limits; - std::shared_ptr roles; + std::shared_ptr roles; while (true) { diff --git a/dbms/src/Parsers/ParserCreateRowPolicyQuery.cpp b/dbms/src/Parsers/ParserCreateRowPolicyQuery.cpp index ff865f3644f..e277091bc0c 100644 --- a/dbms/src/Parsers/ParserCreateRowPolicyQuery.cpp +++ b/dbms/src/Parsers/ParserCreateRowPolicyQuery.cpp @@ -1,8 +1,8 @@ #include #include #include -#include -#include +#include +#include #include #include #include @@ -19,7 +19,7 @@ namespace ErrorCodes namespace { - using ConditionIndex = RowPolicy::ConditionIndex; + using ConditionType = RowPolicy::ConditionType; bool parseRenameTo(IParserBase::Pos & pos, Expected & expected, String & new_policy_name) { @@ -73,7 +73,7 @@ namespace }); } - bool parseConditions(IParserBase::Pos & pos, Expected & expected, bool alter, std::vector> & conditions) + bool parseConditions(IParserBase::Pos & pos, Expected & expected, bool alter, std::vector> & conditions) { return IParserBase::wrapParseImpl(pos, [&] { @@ -136,14 +136,14 @@ namespace if (filter && !check && !alter) check = filter; - auto set_condition = [&](ConditionIndex index, const ASTPtr & condition) + auto set_condition = [&](ConditionType index, const ASTPtr & condition) { - auto it = std::find_if(conditions.begin(), conditions.end(), [index](const std::pair & element) + auto it = std::find_if(conditions.begin(), conditions.end(), [index](const std::pair & element) { return element.first == index; }); if (it == conditions.end()) - it = conditions.insert(conditions.end(), std::pair{index, nullptr}); + it = conditions.insert(conditions.end(), std::pair{index, nullptr}); it->second = condition; }; @@ -170,11 +170,11 @@ namespace }); } - bool parseMultipleConditions(IParserBase::Pos & pos, Expected & expected, bool alter, std::vector> & conditions) + bool parseMultipleConditions(IParserBase::Pos & pos, Expected & expected, bool alter, std::vector> & conditions) { return IParserBase::wrapParseImpl(pos, [&] { - std::vector> res_conditions; + std::vector> res_conditions; do { if (!parseConditions(pos, expected, alter, res_conditions)) @@ -187,16 +187,16 @@ namespace }); } - bool parseToRoles(IParserBase::Pos & pos, Expected & expected, bool id_mode, std::shared_ptr & roles) + bool parseToRoles(IParserBase::Pos & pos, Expected & expected, bool id_mode, std::shared_ptr & roles) { return IParserBase::wrapParseImpl(pos, [&] { ASTPtr ast; if (roles || !ParserKeyword{"TO"}.ignore(pos, expected) - || !ParserGenericRoleSet{}.enableIDMode(id_mode).parse(pos, ast, expected)) + || !ParserExtendedRoleSet{}.useIDMode(id_mode).parse(pos, ast, expected)) return false; - roles = std::static_pointer_cast(ast); + roles = std::static_pointer_cast(ast); return true; }); } @@ -247,8 +247,8 @@ bool ParserCreateRowPolicyQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & String new_policy_name; std::optional is_restrictive; - std::vector> conditions; - std::shared_ptr roles; + std::vector> conditions; + std::shared_ptr roles; while (true) { diff --git a/dbms/src/Parsers/ParserCreateUserQuery.cpp b/dbms/src/Parsers/ParserCreateUserQuery.cpp index c0c4196acee..f82a592e773 100644 --- a/dbms/src/Parsers/ParserCreateUserQuery.cpp +++ b/dbms/src/Parsers/ParserCreateUserQuery.cpp @@ -5,8 +5,8 @@ #include #include #include -#include -#include +#include +#include #include #include @@ -208,7 +208,7 @@ namespace } - bool parseDefaultRoles(IParserBase::Pos & pos, Expected & expected, bool id_mode, std::shared_ptr & default_roles) + bool parseDefaultRoles(IParserBase::Pos & pos, Expected & expected, bool id_mode, std::shared_ptr & default_roles) { return IParserBase::wrapParseImpl(pos, [&] { @@ -216,10 +216,10 @@ namespace return false; ASTPtr ast; - if (!ParserGenericRoleSet{}.enableCurrentUserKeyword(false).enableIDMode(id_mode).parse(pos, ast, expected)) + if (!ParserExtendedRoleSet{}.enableCurrentUserKeyword(false).useIDMode(id_mode).parse(pos, ast, expected)) return false; - default_roles = typeid_cast>(ast); + default_roles = typeid_cast>(ast); return true; }); } @@ -289,7 +289,7 @@ bool ParserCreateUserQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expec std::optional hosts; std::optional add_hosts; std::optional remove_hosts; - std::shared_ptr default_roles; + std::shared_ptr default_roles; std::optional profile; while (true) diff --git a/dbms/src/Parsers/ParserGenericRoleSet.cpp b/dbms/src/Parsers/ParserExtendedRoleSet.cpp similarity index 94% rename from dbms/src/Parsers/ParserGenericRoleSet.cpp rename to dbms/src/Parsers/ParserExtendedRoleSet.cpp index a58c638e36d..80f05c45f5b 100644 --- a/dbms/src/Parsers/ParserGenericRoleSet.cpp +++ b/dbms/src/Parsers/ParserExtendedRoleSet.cpp @@ -1,8 +1,8 @@ -#include +#include #include #include #include -#include +#include #include #include @@ -109,7 +109,7 @@ namespace } -bool ParserGenericRoleSet::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) +bool ParserExtendedRoleSet::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) { Strings names; bool current_user = false; @@ -125,7 +125,7 @@ bool ParserGenericRoleSet::parseImpl(Pos & pos, ASTPtr & node, Expected & expect if (all) names.clear(); - auto result = std::make_shared(); + auto result = std::make_shared(); result->names = std::move(names); result->current_user = current_user; result->all = all; diff --git a/dbms/src/Parsers/ParserExtendedRoleSet.h b/dbms/src/Parsers/ParserExtendedRoleSet.h new file mode 100644 index 00000000000..df723786bd9 --- /dev/null +++ b/dbms/src/Parsers/ParserExtendedRoleSet.h @@ -0,0 +1,28 @@ +#pragma once + +#include + + +namespace DB +{ +/** Parses a string like this: + * {role|CURRENT_USER} [,...] | NONE | ALL | ALL EXCEPT {role|CURRENT_USER} [,...] + */ +class ParserExtendedRoleSet : public IParserBase +{ +public: + ParserExtendedRoleSet & enableAllKeyword(bool enable_) { all_keyword = enable_; return *this; } + ParserExtendedRoleSet & enableCurrentUserKeyword(bool enable_) { current_user_keyword = enable_; return *this; } + ParserExtendedRoleSet & useIDMode(bool enable_) { id_mode = enable_; return *this; } + +protected: + const char * getName() const override { return "ExtendedRoleSet"; } + bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override; + +private: + bool all_keyword = true; + bool current_user_keyword = true; + bool id_mode = false; +}; + +} diff --git a/dbms/src/Parsers/ParserGenericRoleSet.h b/dbms/src/Parsers/ParserGenericRoleSet.h deleted file mode 100644 index b209cb22350..00000000000 --- a/dbms/src/Parsers/ParserGenericRoleSet.h +++ /dev/null @@ -1,28 +0,0 @@ -#pragma once - -#include - - -namespace DB -{ -/** Parses a string like this: - * {role|CURRENT_USER} [,...] | NONE | ALL | ALL EXCEPT {role|CURRENT_USER} [,...] - */ -class ParserGenericRoleSet : public IParserBase -{ -public: - ParserGenericRoleSet & enableAllKeyword(bool enable_) { all_keyword = enable_; return *this; } - ParserGenericRoleSet & enableCurrentUserKeyword(bool enable_) { current_user_keyword = enable_; return *this; } - ParserGenericRoleSet & enableIDMode(bool enable_) { id_mode = enable_; return *this; } - -protected: - const char * getName() const override { return "GenericRoleSet"; } - bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override; - -private: - bool all_keyword = true; - bool current_user_keyword = true; - bool id_mode = false; -}; - -} diff --git a/dbms/src/Parsers/ParserGrantQuery.cpp b/dbms/src/Parsers/ParserGrantQuery.cpp index dc2fbc5f260..f8533c27d88 100644 --- a/dbms/src/Parsers/ParserGrantQuery.cpp +++ b/dbms/src/Parsers/ParserGrantQuery.cpp @@ -1,10 +1,10 @@ #include #include #include -#include +#include #include #include -#include +#include #include @@ -209,21 +209,21 @@ namespace } - bool parseRoles(IParser::Pos & pos, Expected & expected, bool id_mode, std::shared_ptr & roles) + bool parseRoles(IParser::Pos & pos, Expected & expected, bool id_mode, std::shared_ptr & roles) { return IParserBase::wrapParseImpl(pos, [&] { ASTPtr ast; - if (!ParserGenericRoleSet{}.enableAllKeyword(false).enableCurrentUserKeyword(false).enableIDMode(id_mode).parse(pos, ast, expected)) + if (!ParserExtendedRoleSet{}.enableAllKeyword(false).enableCurrentUserKeyword(false).useIDMode(id_mode).parse(pos, ast, expected)) return false; - roles = typeid_cast>(ast); + roles = typeid_cast>(ast); return true; }); } - bool parseToRoles(IParser::Pos & pos, Expected & expected, ASTGrantQuery::Kind kind, std::shared_ptr & to_roles) + bool parseToRoles(IParser::Pos & pos, Expected & expected, ASTGrantQuery::Kind kind, std::shared_ptr & to_roles) { return IParserBase::wrapParseImpl(pos, [&] { @@ -240,10 +240,10 @@ namespace } ASTPtr ast; - if (!ParserGenericRoleSet{}.enableAllKeyword(kind == Kind::REVOKE).parse(pos, ast, expected)) + if (!ParserExtendedRoleSet{}.enableAllKeyword(kind == Kind::REVOKE).parse(pos, ast, expected)) return false; - to_roles = typeid_cast>(ast); + to_roles = typeid_cast>(ast); return true; }); } @@ -280,11 +280,11 @@ bool ParserGrantQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) } AccessRightsElements elements; - std::shared_ptr roles; + std::shared_ptr roles; if (!parseAccessRightsElements(pos, expected, elements) && !parseRoles(pos, expected, attach, roles)) return false; - std::shared_ptr to_roles; + std::shared_ptr to_roles; if (!parseToRoles(pos, expected, kind, to_roles)) return false; diff --git a/dbms/src/Parsers/ParserSetRoleQuery.cpp b/dbms/src/Parsers/ParserSetRoleQuery.cpp index 3031bf8ad01..e6ff7893891 100644 --- a/dbms/src/Parsers/ParserSetRoleQuery.cpp +++ b/dbms/src/Parsers/ParserSetRoleQuery.cpp @@ -1,28 +1,28 @@ #include #include #include -#include -#include +#include +#include namespace DB { namespace { - bool parseRoles(IParserBase::Pos & pos, Expected & expected, std::shared_ptr & roles) + bool parseRoles(IParserBase::Pos & pos, Expected & expected, std::shared_ptr & roles) { return IParserBase::wrapParseImpl(pos, [&] { ASTPtr ast; - if (!ParserGenericRoleSet{}.enableCurrentUserKeyword(false).parse(pos, ast, expected)) + if (!ParserExtendedRoleSet{}.enableCurrentUserKeyword(false).parse(pos, ast, expected)) return false; - roles = typeid_cast>(ast); + roles = typeid_cast>(ast); return true; }); } - bool parseToUsers(IParserBase::Pos & pos, Expected & expected, std::shared_ptr & to_users) + bool parseToUsers(IParserBase::Pos & pos, Expected & expected, std::shared_ptr & to_users) { return IParserBase::wrapParseImpl(pos, [&] { @@ -30,10 +30,10 @@ namespace return false; ASTPtr ast; - if (!ParserGenericRoleSet{}.enableAllKeyword(false).parse(pos, ast, expected)) + if (!ParserExtendedRoleSet{}.enableAllKeyword(false).parse(pos, ast, expected)) return false; - to_users = typeid_cast>(ast); + to_users = typeid_cast>(ast); return true; }); } @@ -53,8 +53,8 @@ bool ParserSetRoleQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected else return false; - std::shared_ptr roles; - std::shared_ptr to_users; + std::shared_ptr roles; + std::shared_ptr to_users; if ((kind == Kind::SET_ROLE) || (kind == Kind::SET_DEFAULT_ROLE)) { diff --git a/dbms/src/Processors/Executors/TreeExecutorBlockInputStream.cpp b/dbms/src/Processors/Executors/TreeExecutorBlockInputStream.cpp index e0242533518..ee5b254ccf9 100644 --- a/dbms/src/Processors/Executors/TreeExecutorBlockInputStream.cpp +++ b/dbms/src/Processors/Executors/TreeExecutorBlockInputStream.cpp @@ -335,7 +335,7 @@ void TreeExecutorBlockInputStream::setLimits(const IBlockInputStream::LocalLimit source->setLimits(limits_); } -void TreeExecutorBlockInputStream::setQuota(const QuotaContextPtr & quota_) +void TreeExecutorBlockInputStream::setQuota(const std::shared_ptr & quota_) { for (auto & source : sources_with_progress) source->setQuota(quota_); diff --git a/dbms/src/Processors/Executors/TreeExecutorBlockInputStream.h b/dbms/src/Processors/Executors/TreeExecutorBlockInputStream.h index 3ab8dde6948..24cab387eb8 100644 --- a/dbms/src/Processors/Executors/TreeExecutorBlockInputStream.h +++ b/dbms/src/Processors/Executors/TreeExecutorBlockInputStream.h @@ -43,7 +43,7 @@ public: void setProgressCallback(const ProgressCallback & callback) final; void setProcessListElement(QueryStatus * elem) final; void setLimits(const LocalLimits & limits_) final; - void setQuota(const QuotaContextPtr & quota_) final; + void setQuota(const std::shared_ptr & quota_) final; void addTotalRowsApprox(size_t value) final; protected: diff --git a/dbms/src/Processors/Pipe.cpp b/dbms/src/Processors/Pipe.cpp index 7b35c351d2f..f3ffb6ee201 100644 --- a/dbms/src/Processors/Pipe.cpp +++ b/dbms/src/Processors/Pipe.cpp @@ -106,7 +106,7 @@ void Pipe::setLimits(const ISourceWithProgress::LocalLimits & limits) } } -void Pipe::setQuota(const QuotaContextPtr & quota) +void Pipe::setQuota(const std::shared_ptr & quota) { for (auto & processor : processors) { diff --git a/dbms/src/Processors/Pipe.h b/dbms/src/Processors/Pipe.h index 20f5eb038a3..f30eaef678f 100644 --- a/dbms/src/Processors/Pipe.h +++ b/dbms/src/Processors/Pipe.h @@ -40,7 +40,7 @@ public: /// Specify quotas and limits for every ISourceWithProgress. void setLimits(const SourceWithProgress::LocalLimits & limits); - void setQuota(const QuotaContextPtr & quota); + void setQuota(const std::shared_ptr & quota); /// Set information about preferred executor number for sources. void pinSources(size_t executor_number); diff --git a/dbms/src/Processors/Sources/SourceFromInputStream.h b/dbms/src/Processors/Sources/SourceFromInputStream.h index 83e7f9929c9..0fc92164059 100644 --- a/dbms/src/Processors/Sources/SourceFromInputStream.h +++ b/dbms/src/Processors/Sources/SourceFromInputStream.h @@ -28,7 +28,7 @@ public: /// Implementation for methods from ISourceWithProgress. void setLimits(const LocalLimits & limits_) final { stream->setLimits(limits_); } - void setQuota(const QuotaContextPtr & quota_) final { stream->setQuota(quota_); } + void setQuota(const std::shared_ptr & quota_) final { stream->setQuota(quota_); } void setProcessListElement(QueryStatus * elem) final { stream->setProcessListElement(elem); } void setProgressCallback(const ProgressCallback & callback) final { stream->setProgressCallback(callback); } void addTotalRowsApprox(size_t value) final { stream->addTotalRowsApprox(value); } diff --git a/dbms/src/Processors/Sources/SourceWithProgress.cpp b/dbms/src/Processors/Sources/SourceWithProgress.cpp index 0cac415aedb..80844da16cd 100644 --- a/dbms/src/Processors/Sources/SourceWithProgress.cpp +++ b/dbms/src/Processors/Sources/SourceWithProgress.cpp @@ -1,7 +1,7 @@ #include #include -#include +#include namespace DB { diff --git a/dbms/src/Processors/Sources/SourceWithProgress.h b/dbms/src/Processors/Sources/SourceWithProgress.h index d22a2bf087a..4778c50e49d 100644 --- a/dbms/src/Processors/Sources/SourceWithProgress.h +++ b/dbms/src/Processors/Sources/SourceWithProgress.h @@ -21,7 +21,7 @@ public: /// Set the quota. If you set a quota on the amount of raw data, /// then you should also set mode = LIMITS_TOTAL to LocalLimits with setLimits. - virtual void setQuota(const QuotaContextPtr & quota_) = 0; + virtual void setQuota(const std::shared_ptr & quota_) = 0; /// Set the pointer to the process list item. /// General information about the resources spent on the request will be written into it. @@ -49,7 +49,7 @@ public: using LimitsMode = IBlockInputStream::LimitsMode; void setLimits(const LocalLimits & limits_) final { limits = limits_; } - void setQuota(const QuotaContextPtr & quota_) final { quota = quota_; } + void setQuota(const std::shared_ptr & quota_) final { quota = quota_; } void setProcessListElement(QueryStatus * elem) final { process_list_elem = elem; } void setProgressCallback(const ProgressCallback & callback) final { progress_callback = callback; } void addTotalRowsApprox(size_t value) final { total_rows_approx += value; } @@ -62,7 +62,7 @@ protected: private: LocalLimits limits; - QuotaContextPtr quota; + std::shared_ptr quota; ProgressCallback progress_callback; QueryStatus * process_list_elem = nullptr; diff --git a/dbms/src/Processors/Transforms/LimitsCheckingTransform.cpp b/dbms/src/Processors/Transforms/LimitsCheckingTransform.cpp index 3ead146abc1..c3ac019f2b6 100644 --- a/dbms/src/Processors/Transforms/LimitsCheckingTransform.cpp +++ b/dbms/src/Processors/Transforms/LimitsCheckingTransform.cpp @@ -1,5 +1,5 @@ #include -#include +#include namespace DB { diff --git a/dbms/src/Processors/Transforms/LimitsCheckingTransform.h b/dbms/src/Processors/Transforms/LimitsCheckingTransform.h index bfc5c338da1..3014c259487 100644 --- a/dbms/src/Processors/Transforms/LimitsCheckingTransform.h +++ b/dbms/src/Processors/Transforms/LimitsCheckingTransform.h @@ -33,7 +33,7 @@ public: String getName() const override { return "LimitsCheckingTransform"; } - void setQuota(const QuotaContextPtr & quota_) { quota = quota_; } + void setQuota(const std::shared_ptr & quota_) { quota = quota_; } protected: void transform(Chunk & chunk) override; @@ -41,7 +41,7 @@ protected: private: LocalLimits limits; - QuotaContextPtr quota; + std::shared_ptr quota; UInt64 prev_elapsed = 0; ProcessorProfileInfo info; diff --git a/dbms/src/Storages/System/StorageSystemColumns.cpp b/dbms/src/Storages/System/StorageSystemColumns.cpp index 4e210f7935c..cbf6ada9ed3 100644 --- a/dbms/src/Storages/System/StorageSystemColumns.cpp +++ b/dbms/src/Storages/System/StorageSystemColumns.cpp @@ -9,7 +9,7 @@ #include #include #include -#include +#include #include #include @@ -62,12 +62,12 @@ public: ColumnPtr databases_, ColumnPtr tables_, Storages storages_, - const AccessRightsContextPtr & access_rights_, + const std::shared_ptr & access_, String query_id_) : SourceWithProgress(header_) , columns_mask(std::move(columns_mask_)), max_block_size(max_block_size_) , databases(std::move(databases_)), tables(std::move(tables_)), storages(std::move(storages_)) - , query_id(std::move(query_id_)), total_tables(tables->size()), access_rights(access_rights_) + , query_id(std::move(query_id_)), total_tables(tables->size()), access(access_) { } @@ -82,7 +82,7 @@ protected: MutableColumns res_columns = getPort().getHeader().cloneEmptyColumns(); size_t rows_count = 0; - const bool check_access_for_tables = !access_rights->isGranted(AccessType::SHOW_COLUMNS); + const bool check_access_for_tables = !access->isGranted(AccessType::SHOW_COLUMNS); while (rows_count < max_block_size && db_table_num < total_tables) { @@ -128,14 +128,14 @@ protected: column_sizes = storage->getColumnSizes(); } - bool check_access_for_columns = check_access_for_tables && !access_rights->isGranted(AccessType::SHOW_COLUMNS, database_name, table_name); + bool check_access_for_columns = check_access_for_tables && !access->isGranted(AccessType::SHOW_COLUMNS, database_name, table_name); for (const auto & column : columns) { if (column.is_virtual) continue; - if (check_access_for_columns && !access_rights->isGranted(AccessType::SHOW_COLUMNS, database_name, table_name, column.name)) + if (check_access_for_columns && !access->isGranted(AccessType::SHOW_COLUMNS, database_name, table_name, column.name)) continue; size_t src_index = 0; @@ -230,7 +230,7 @@ private: String query_id; size_t db_table_num = 0; size_t total_tables; - AccessRightsContextPtr access_rights; + std::shared_ptr access; }; @@ -332,7 +332,7 @@ Pipes StorageSystemColumns::read( pipes.emplace_back(std::make_shared( std::move(columns_mask), std::move(header), max_block_size, std::move(filtered_database_column), std::move(filtered_table_column), std::move(storages), - context.getAccessRights(), context.getCurrentQueryId())); + context.getAccess(), context.getCurrentQueryId())); return pipes; } diff --git a/dbms/src/Storages/System/StorageSystemDatabases.cpp b/dbms/src/Storages/System/StorageSystemDatabases.cpp index 835ed074a55..5a35e079a5b 100644 --- a/dbms/src/Storages/System/StorageSystemDatabases.cpp +++ b/dbms/src/Storages/System/StorageSystemDatabases.cpp @@ -1,7 +1,7 @@ #include #include #include -#include +#include #include @@ -20,13 +20,13 @@ NamesAndTypesList StorageSystemDatabases::getNamesAndTypes() void StorageSystemDatabases::fillData(MutableColumns & res_columns, const Context & context, const SelectQueryInfo &) const { - const auto access_rights = context.getAccessRights(); - const bool check_access_for_databases = !access_rights->isGranted(AccessType::SHOW_DATABASES); + const auto access = context.getAccess(); + const bool check_access_for_databases = !access->isGranted(AccessType::SHOW_DATABASES); auto databases = DatabaseCatalog::instance().getDatabases(); for (const auto & database : databases) { - if (check_access_for_databases && !access_rights->isGranted(AccessType::SHOW_DATABASES, database.first)) + if (check_access_for_databases && !access->isGranted(AccessType::SHOW_DATABASES, database.first)) continue; res_columns[0]->insert(database.first); diff --git a/dbms/src/Storages/System/StorageSystemDictionaries.cpp b/dbms/src/Storages/System/StorageSystemDictionaries.cpp index e3d4f7c773c..400b1074250 100644 --- a/dbms/src/Storages/System/StorageSystemDictionaries.cpp +++ b/dbms/src/Storages/System/StorageSystemDictionaries.cpp @@ -8,7 +8,7 @@ #include #include #include -#include +#include #include #include #include @@ -49,8 +49,8 @@ NamesAndTypesList StorageSystemDictionaries::getNamesAndTypes() void StorageSystemDictionaries::fillData(MutableColumns & res_columns, const Context & context, const SelectQueryInfo & /*query_info*/) const { - const auto access_rights = context.getAccessRights(); - const bool check_access_for_dictionaries = !access_rights->isGranted(AccessType::SHOW_DICTIONARIES); + const auto access = context.getAccess(); + const bool check_access_for_dictionaries = !access->isGranted(AccessType::SHOW_DICTIONARIES); const auto & external_dictionaries = context.getExternalDictionariesLoader(); for (const auto & load_result : external_dictionaries.getCurrentLoadResults()) @@ -74,7 +74,7 @@ void StorageSystemDictionaries::fillData(MutableColumns & res_columns, const Con } if (check_access_for_dictionaries - && !access_rights->isGranted(AccessType::SHOW_DICTIONARIES, database.empty() ? IDictionary::NO_DATABASE_TAG : database, short_name)) + && !access->isGranted(AccessType::SHOW_DICTIONARIES, database.empty() ? IDictionary::NO_DATABASE_TAG : database, short_name)) continue; size_t i = 0; diff --git a/dbms/src/Storages/System/StorageSystemMerges.cpp b/dbms/src/Storages/System/StorageSystemMerges.cpp index 04e05a31534..39d22bd00ca 100644 --- a/dbms/src/Storages/System/StorageSystemMerges.cpp +++ b/dbms/src/Storages/System/StorageSystemMerges.cpp @@ -1,7 +1,7 @@ #include #include #include -#include +#include namespace DB @@ -36,12 +36,12 @@ NamesAndTypesList StorageSystemMerges::getNamesAndTypes() void StorageSystemMerges::fillData(MutableColumns & res_columns, const Context & context, const SelectQueryInfo &) const { - const auto access_rights = context.getAccessRights(); - const bool check_access_for_tables = !access_rights->isGranted(AccessType::SHOW_TABLES); + const auto access = context.getAccess(); + const bool check_access_for_tables = !access->isGranted(AccessType::SHOW_TABLES); for (const auto & merge : context.getMergeList().get()) { - if (check_access_for_tables && !access_rights->isGranted(AccessType::SHOW_TABLES, merge.database, merge.table)) + if (check_access_for_tables && !access->isGranted(AccessType::SHOW_TABLES, merge.database, merge.table)) continue; size_t i = 0; diff --git a/dbms/src/Storages/System/StorageSystemMutations.cpp b/dbms/src/Storages/System/StorageSystemMutations.cpp index 4e324ae6bd2..e7d9cc38671 100644 --- a/dbms/src/Storages/System/StorageSystemMutations.cpp +++ b/dbms/src/Storages/System/StorageSystemMutations.cpp @@ -7,7 +7,7 @@ #include #include #include -#include +#include #include @@ -37,8 +37,8 @@ NamesAndTypesList StorageSystemMutations::getNamesAndTypes() void StorageSystemMutations::fillData(MutableColumns & res_columns, const Context & context, const SelectQueryInfo & query_info) const { - const auto access_rights = context.getAccessRights(); - const bool check_access_for_databases = !access_rights->isGranted(AccessType::SHOW_TABLES); + const auto access = context.getAccess(); + const bool check_access_for_databases = !access->isGranted(AccessType::SHOW_TABLES); /// Collect a set of *MergeTree tables. std::map> merge_tree_tables; @@ -48,14 +48,14 @@ void StorageSystemMutations::fillData(MutableColumns & res_columns, const Contex if (db.second->getEngineName() == "Lazy") continue; - const bool check_access_for_tables = check_access_for_databases && !access_rights->isGranted(AccessType::SHOW_TABLES, db.first); + const bool check_access_for_tables = check_access_for_databases && !access->isGranted(AccessType::SHOW_TABLES, db.first); for (auto iterator = db.second->getTablesIterator(context); iterator->isValid(); iterator->next()) { if (!dynamic_cast(iterator->table().get())) continue; - if (check_access_for_tables && !access_rights->isGranted(AccessType::SHOW_TABLES, db.first, iterator->name())) + if (check_access_for_tables && !access->isGranted(AccessType::SHOW_TABLES, db.first, iterator->name())) continue; merge_tree_tables[db.first][iterator->name()] = iterator->table(); diff --git a/dbms/src/Storages/System/StorageSystemPartsBase.cpp b/dbms/src/Storages/System/StorageSystemPartsBase.cpp index 235c17611fc..c5edde3e5d9 100644 --- a/dbms/src/Storages/System/StorageSystemPartsBase.cpp +++ b/dbms/src/Storages/System/StorageSystemPartsBase.cpp @@ -9,7 +9,7 @@ #include #include #include -#include +#include #include #include #include @@ -73,8 +73,8 @@ StoragesInfoStream::StoragesInfoStream(const SelectQueryInfo & query_info, const MutableColumnPtr engine_column_mut = ColumnString::create(); MutableColumnPtr active_column_mut = ColumnUInt8::create(); - const auto access_rights = context.getAccessRights(); - const bool check_access_for_tables = !access_rights->isGranted(AccessType::SHOW_TABLES); + const auto access = context.getAccess(); + const bool check_access_for_tables = !access->isGranted(AccessType::SHOW_TABLES); { Databases databases = DatabaseCatalog::instance().getDatabases(); @@ -119,7 +119,7 @@ StoragesInfoStream::StoragesInfoStream(const SelectQueryInfo & query_info, const if (!dynamic_cast(storage.get())) continue; - if (check_access_for_tables && !access_rights->isGranted(AccessType::SHOW_TABLES, database_name, table_name)) + if (check_access_for_tables && !access->isGranted(AccessType::SHOW_TABLES, database_name, table_name)) continue; storages[std::make_pair(database_name, iterator->name())] = storage; diff --git a/dbms/src/Storages/System/StorageSystemQuotaUsage.cpp b/dbms/src/Storages/System/StorageSystemQuotaUsage.cpp index 8835e77eeb5..53afb1d563a 100644 --- a/dbms/src/Storages/System/StorageSystemQuotaUsage.cpp +++ b/dbms/src/Storages/System/StorageSystemQuotaUsage.cpp @@ -6,7 +6,8 @@ #include #include #include -#include +#include +#include #include diff --git a/dbms/src/Storages/System/StorageSystemQuotas.cpp b/dbms/src/Storages/System/StorageSystemQuotas.cpp index 81969ab2364..228339ea305 100644 --- a/dbms/src/Storages/System/StorageSystemQuotas.cpp +++ b/dbms/src/Storages/System/StorageSystemQuotas.cpp @@ -87,7 +87,7 @@ void StorageSystemQuotas::fillData(MutableColumns & res_columns, const Context & storage_name_column.insert(storage_name); key_type_column.insert(static_cast(quota->key_type)); - for (const String & role : quota->roles.toStringsWithNames(access_control)) + for (const String & role : quota->to_roles.toStringsWithNames(access_control)) roles_data.insert(role); roles_offsets.push_back(roles_data.size()); diff --git a/dbms/src/Storages/System/StorageSystemReplicas.cpp b/dbms/src/Storages/System/StorageSystemReplicas.cpp index 02ae587d1b5..251b45e44b6 100644 --- a/dbms/src/Storages/System/StorageSystemReplicas.cpp +++ b/dbms/src/Storages/System/StorageSystemReplicas.cpp @@ -6,7 +6,7 @@ #include #include #include -#include +#include #include #include #include @@ -65,8 +65,8 @@ Pipes StorageSystemReplicas::read( { check(column_names); - const auto access_rights = context.getAccessRights(); - const bool check_access_for_databases = !access_rights->isGranted(AccessType::SHOW_TABLES); + const auto access = context.getAccess(); + const bool check_access_for_databases = !access->isGranted(AccessType::SHOW_TABLES); /// We collect a set of replicated tables. std::map> replicated_tables; @@ -75,12 +75,12 @@ Pipes StorageSystemReplicas::read( /// Lazy database can not contain replicated tables if (db.second->getEngineName() == "Lazy") continue; - const bool check_access_for_tables = check_access_for_databases && !access_rights->isGranted(AccessType::SHOW_TABLES, db.first); + const bool check_access_for_tables = check_access_for_databases && !access->isGranted(AccessType::SHOW_TABLES, db.first); for (auto iterator = db.second->getTablesIterator(context); iterator->isValid(); iterator->next()) { if (!dynamic_cast(iterator->table().get())) continue; - if (check_access_for_tables && !access_rights->isGranted(AccessType::SHOW_TABLES, db.first, iterator->name())) + if (check_access_for_tables && !access->isGranted(AccessType::SHOW_TABLES, db.first, iterator->name())) continue; replicated_tables[db.first][iterator->name()] = iterator->table(); } diff --git a/dbms/src/Storages/System/StorageSystemReplicationQueue.cpp b/dbms/src/Storages/System/StorageSystemReplicationQueue.cpp index 4519d735161..2c188cf3734 100644 --- a/dbms/src/Storages/System/StorageSystemReplicationQueue.cpp +++ b/dbms/src/Storages/System/StorageSystemReplicationQueue.cpp @@ -8,7 +8,7 @@ #include #include #include -#include +#include #include #include @@ -48,8 +48,8 @@ NamesAndTypesList StorageSystemReplicationQueue::getNamesAndTypes() void StorageSystemReplicationQueue::fillData(MutableColumns & res_columns, const Context & context, const SelectQueryInfo & query_info) const { - const auto access_rights = context.getAccessRights(); - const bool check_access_for_databases = !access_rights->isGranted(AccessType::SHOW_TABLES); + const auto access = context.getAccess(); + const bool check_access_for_databases = !access->isGranted(AccessType::SHOW_TABLES); std::map> replicated_tables; for (const auto & db : DatabaseCatalog::instance().getDatabases()) @@ -58,13 +58,13 @@ void StorageSystemReplicationQueue::fillData(MutableColumns & res_columns, const if (db.second->getEngineName() == "Lazy") continue; - const bool check_access_for_tables = check_access_for_databases && !access_rights->isGranted(AccessType::SHOW_TABLES, db.first); + const bool check_access_for_tables = check_access_for_databases && !access->isGranted(AccessType::SHOW_TABLES, db.first); for (auto iterator = db.second->getTablesIterator(context); iterator->isValid(); iterator->next()) { if (!dynamic_cast(iterator->table().get())) continue; - if (check_access_for_tables && !access_rights->isGranted(AccessType::SHOW_TABLES, db.first, iterator->name())) + if (check_access_for_tables && !access->isGranted(AccessType::SHOW_TABLES, db.first, iterator->name())) continue; replicated_tables[db.first][iterator->name()] = iterator->table(); } diff --git a/dbms/src/Storages/System/StorageSystemRowPolicies.cpp b/dbms/src/Storages/System/StorageSystemRowPolicies.cpp index 8ac4ac1b755..bd302cba3cf 100644 --- a/dbms/src/Storages/System/StorageSystemRowPolicies.cpp +++ b/dbms/src/Storages/System/StorageSystemRowPolicies.cpp @@ -24,8 +24,8 @@ NamesAndTypesList StorageSystemRowPolicies::getNamesAndTypes() {"restrictive", std::make_shared()}, }; - for (auto index : ext::range_with_static_cast(RowPolicy::MAX_CONDITION_INDEX)) - names_and_types.push_back({RowPolicy::conditionIndexToColumnName(index), std::make_shared()}); + for (auto index : ext::range_with_static_cast(RowPolicy::MAX_CONDITION_TYPE)) + names_and_types.push_back({RowPolicy::conditionTypeToColumnName(index), std::make_shared()}); return names_and_types; } @@ -52,7 +52,7 @@ void StorageSystemRowPolicies::fillData(MutableColumns & res_columns, const Cont res_columns[i++]->insert(storage ? storage->getStorageName() : ""); res_columns[i++]->insert(policy->isRestrictive()); - for (auto index : ext::range(RowPolicy::MAX_CONDITION_INDEX)) + for (auto index : ext::range(RowPolicy::MAX_CONDITION_TYPE)) res_columns[i++]->insert(policy->conditions[index]); } } diff --git a/dbms/src/Storages/System/StorageSystemTables.cpp b/dbms/src/Storages/System/StorageSystemTables.cpp index 2ada4a64b45..cb72d3408df 100644 --- a/dbms/src/Storages/System/StorageSystemTables.cpp +++ b/dbms/src/Storages/System/StorageSystemTables.cpp @@ -6,7 +6,7 @@ #include #include #include -#include +#include #include #include #include @@ -105,8 +105,8 @@ protected: MutableColumns res_columns = getPort().getHeader().cloneEmptyColumns(); - const auto access_rights = context.getAccessRights(); - const bool check_access_for_databases = !access_rights->isGranted(AccessType::SHOW_TABLES); + const auto access = context.getAccess(); + const bool check_access_for_databases = !access->isGranted(AccessType::SHOW_TABLES); size_t rows_count = 0; while (rows_count < max_block_size) @@ -196,7 +196,7 @@ protected: return Chunk(std::move(res_columns), num_rows); } - const bool check_access_for_tables = check_access_for_databases && !access_rights->isGranted(AccessType::SHOW_TABLES, database_name); + const bool check_access_for_tables = check_access_for_databases && !access->isGranted(AccessType::SHOW_TABLES, database_name); if (!tables_it || !tables_it->isValid()) tables_it = database->getTablesWithDictionaryTablesIterator(context); @@ -206,7 +206,7 @@ protected: for (; rows_count < max_block_size && tables_it->isValid(); tables_it->next()) { auto table_name = tables_it->name(); - if (check_access_for_tables && !access_rights->isGranted(AccessType::SHOW_TABLES, database_name, table_name)) + if (check_access_for_tables && !access->isGranted(AccessType::SHOW_TABLES, database_name, table_name)) continue; StoragePtr table = nullptr; From 49bf4ae37568a128fb40277e27f339f091c3bff8 Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Thu, 5 Mar 2020 01:27:03 +0300 Subject: [PATCH 0154/1355] Introduce SettingsProfile as a new access entity type. --- dbms/src/Access/AccessControlManager.cpp | 33 ++- dbms/src/Access/AccessControlManager.h | 14 ++ dbms/src/Access/ContextAccess.cpp | 17 ++ dbms/src/Access/ContextAccess.h | 6 + dbms/src/Access/EnabledRolesInfo.cpp | 3 +- dbms/src/Access/EnabledRolesInfo.h | 2 + dbms/src/Access/EnabledSettings.cpp | 36 +++ dbms/src/Access/EnabledSettings.h | 56 +++++ dbms/src/Access/Role.cpp | 3 +- dbms/src/Access/Role.h | 2 + dbms/src/Access/RoleCache.cpp | 1 + dbms/src/Access/SettingsProfile.cpp | 13 + dbms/src/Access/SettingsProfile.h | 24 ++ dbms/src/Access/SettingsProfileElement.cpp | 54 ++++ dbms/src/Access/SettingsProfileElement.h | 46 ++++ dbms/src/Access/SettingsProfilesCache.cpp | 234 ++++++++++++++++++ dbms/src/Access/SettingsProfilesCache.h | 55 ++++ dbms/src/Access/User.cpp | 2 +- dbms/src/Access/User.h | 3 +- dbms/src/Access/UsersConfigAccessStorage.cpp | 105 +++++++- dbms/src/Interpreters/Context.cpp | 61 ++--- dbms/src/Interpreters/Context.h | 4 +- .../InterpreterCreateUserQuery.cpp | 3 - ...InterpreterShowCreateAccessEntityQuery.cpp | 3 - 24 files changed, 722 insertions(+), 58 deletions(-) create mode 100644 dbms/src/Access/EnabledSettings.cpp create mode 100644 dbms/src/Access/EnabledSettings.h create mode 100644 dbms/src/Access/SettingsProfile.cpp create mode 100644 dbms/src/Access/SettingsProfile.h create mode 100644 dbms/src/Access/SettingsProfileElement.cpp create mode 100644 dbms/src/Access/SettingsProfileElement.h create mode 100644 dbms/src/Access/SettingsProfilesCache.cpp create mode 100644 dbms/src/Access/SettingsProfilesCache.h diff --git a/dbms/src/Access/AccessControlManager.cpp b/dbms/src/Access/AccessControlManager.cpp index 541400fe7a5..b5e06549c28 100644 --- a/dbms/src/Access/AccessControlManager.cpp +++ b/dbms/src/Access/AccessControlManager.cpp @@ -8,6 +8,7 @@ #include #include #include +#include #include #include #include @@ -36,7 +37,13 @@ class AccessControlManager::ContextAccessCache public: explicit ContextAccessCache(const AccessControlManager & manager_) : manager(manager_) {} - std::shared_ptr getContextAccess(const UUID & user_id, const std::vector & current_roles, bool use_default_roles, const Settings & settings, const String & current_database, const ClientInfo & client_info) + std::shared_ptr getContextAccess( + const UUID & user_id, + const std::vector & current_roles, + bool use_default_roles, + const Settings & settings, + const String & current_database, + const ClientInfo & client_info) { ContextAccess::Params params; params.user_id = user_id; @@ -72,7 +79,8 @@ AccessControlManager::AccessControlManager() context_access_cache(std::make_unique(*this)), role_cache(std::make_unique(*this)), row_policy_cache(std::make_unique(*this)), - quota_cache(std::make_unique(*this)) + quota_cache(std::make_unique(*this)), + settings_profiles_cache(std::make_unique(*this)) { } @@ -94,6 +102,12 @@ void AccessControlManager::setUsersConfig(const Poco::Util::AbstractConfiguratio } +void AccessControlManager::setDefaultProfileName(const String & default_profile_name) +{ + settings_profiles_cache->setDefaultProfileName(default_profile_name); +} + + std::shared_ptr AccessControlManager::getContextAccess( const UUID & user_id, const std::vector & current_roles, @@ -132,4 +146,19 @@ std::vector AccessControlManager::getQuotaUsageInfo() const return quota_cache->getUsageInfo(); } + +std::shared_ptr AccessControlManager::getEnabledSettings( + const UUID & user_id, + const SettingsProfileElements & settings_from_user, + const std::vector & enabled_roles, + const SettingsProfileElements & settings_from_enabled_roles) const +{ + return settings_profiles_cache->getEnabledSettings(user_id, settings_from_user, enabled_roles, settings_from_enabled_roles); +} + +std::shared_ptr AccessControlManager::getProfileSettings(const String & profile_name) const +{ + return settings_profiles_cache->getProfileSettings(profile_name); +} + } diff --git a/dbms/src/Access/AccessControlManager.h b/dbms/src/Access/AccessControlManager.h index 1e7c1e6df1d..810970a8379 100644 --- a/dbms/src/Access/AccessControlManager.h +++ b/dbms/src/Access/AccessControlManager.h @@ -29,6 +29,11 @@ class RowPolicyCache; class EnabledQuota; class QuotaCache; struct QuotaUsageInfo; +struct SettingsProfile; +using SettingsProfilePtr = std::shared_ptr; +class EnabledSettings; +class SettingsProfilesCache; +class SettingsProfileElements; class ClientInfo; struct Settings; @@ -42,6 +47,7 @@ public: void setLocalDirectory(const String & directory); void setUsersConfig(const Poco::Util::AbstractConfiguration & users_config); + void setDefaultProfileName(const String & default_profile_name); std::shared_ptr getContextAccess( const UUID & user_id, @@ -68,12 +74,20 @@ public: std::vector getQuotaUsageInfo() const; + std::shared_ptr getEnabledSettings(const UUID & user_id, + const SettingsProfileElements & settings_from_user, + const std::vector & enabled_roles, + const SettingsProfileElements & settings_from_enabled_roles) const; + + std::shared_ptr getProfileSettings(const String & profile_name) const; + private: class ContextAccessCache; std::unique_ptr context_access_cache; std::unique_ptr role_cache; std::unique_ptr row_policy_cache; std::unique_ptr quota_cache; + std::unique_ptr settings_profiles_cache; }; } diff --git a/dbms/src/Access/ContextAccess.cpp b/dbms/src/Access/ContextAccess.cpp index 4867694396c..f5f4ccfe6ac 100644 --- a/dbms/src/Access/ContextAccess.cpp +++ b/dbms/src/Access/ContextAccess.cpp @@ -5,6 +5,7 @@ #include #include #include +#include #include #include #include @@ -123,6 +124,7 @@ void ContextAccess::setUser(const UserPtr & user_) const roles_with_admin_option = nullptr; enabled_row_policies = nullptr; enabled_quota = nullptr; + enabled_settings = nullptr; return; } @@ -172,6 +174,7 @@ void ContextAccess::setRolesInfo(const std::shared_ptr & boost::range::fill(result_access, nullptr /* need recalculate */); enabled_row_policies = manager->getEnabledRowPolicies(*params.user_id, roles_info->enabled_roles); enabled_quota = manager->getEnabledQuota(*params.user_id, user_name, roles_info->enabled_roles, params.address, params.quota_key); + enabled_settings = manager->getEnabledSettings(*params.user_id, user->settings, roles_info->enabled_roles, roles_info->settings_from_enabled_roles); } @@ -532,4 +535,18 @@ std::shared_ptr ContextAccess::getFullAccess() return res; } + +std::shared_ptr ContextAccess::getDefaultSettings() const +{ + std::lock_guard lock{mutex}; + return enabled_settings->getSettings(); +} + + +std::shared_ptr ContextAccess::getSettingsConstraints() const +{ + std::lock_guard lock{mutex}; + return enabled_settings->getConstraints(); +} + } diff --git a/dbms/src/Access/ContextAccess.h b/dbms/src/Access/ContextAccess.h index dc84e51f9e6..bee63103793 100644 --- a/dbms/src/Access/ContextAccess.h +++ b/dbms/src/Access/ContextAccess.h @@ -21,7 +21,9 @@ struct EnabledRolesInfo; class EnabledRoles; class EnabledRowPolicies; class EnabledQuota; +class EnabledSettings; struct Settings; +class SettingsConstraints; class AccessControlManager; class IAST; using ASTPtr = std::shared_ptr; @@ -69,6 +71,8 @@ public: std::shared_ptr getRowPolicies() const; ASTPtr getRowPolicyCondition(const String & database, const String & table_name, RowPolicy::ConditionType index, const ASTPtr & extra_condition = nullptr) const; std::shared_ptr getQuota() const; + std::shared_ptr getDefaultSettings() const; + std::shared_ptr getSettingsConstraints() const; /// Checks if a specified access is granted, and throws an exception if not. /// Empty database means the current database. @@ -124,6 +128,7 @@ private: void setUser(const UserPtr & user_) const; void setRolesInfo(const std::shared_ptr & roles_info_) const; + void setSettingsAndConstraints() const; template bool checkAccessImpl(Poco::Logger * log_, const AccessFlags & flags, const Args &... args) const; @@ -150,6 +155,7 @@ private: mutable boost::atomic_shared_ptr result_access[7]; mutable std::shared_ptr enabled_row_policies; mutable std::shared_ptr enabled_quota; + mutable std::shared_ptr enabled_settings; mutable std::mutex mutex; }; diff --git a/dbms/src/Access/EnabledRolesInfo.cpp b/dbms/src/Access/EnabledRolesInfo.cpp index 7481e707033..01b90d6fa1e 100644 --- a/dbms/src/Access/EnabledRolesInfo.cpp +++ b/dbms/src/Access/EnabledRolesInfo.cpp @@ -28,7 +28,8 @@ bool operator==(const EnabledRolesInfo & lhs, const EnabledRolesInfo & rhs) { return (lhs.current_roles == rhs.current_roles) && (lhs.enabled_roles == rhs.enabled_roles) && (lhs.enabled_roles_with_admin_option == rhs.enabled_roles_with_admin_option) && (lhs.names_of_roles == rhs.names_of_roles) - && (lhs.access == rhs.access) && (lhs.access_with_grant_option == rhs.access_with_grant_option); + && (lhs.access == rhs.access) && (lhs.access_with_grant_option == rhs.access_with_grant_option) + && (lhs.settings_from_enabled_roles == rhs.settings_from_enabled_roles); } } diff --git a/dbms/src/Access/EnabledRolesInfo.h b/dbms/src/Access/EnabledRolesInfo.h index 1fb69e6e871..837d4b74ad5 100644 --- a/dbms/src/Access/EnabledRolesInfo.h +++ b/dbms/src/Access/EnabledRolesInfo.h @@ -1,6 +1,7 @@ #pragma once #include +#include #include #include #include @@ -18,6 +19,7 @@ struct EnabledRolesInfo std::unordered_map names_of_roles; AccessRights access; AccessRights access_with_grant_option; + SettingsProfileElements settings_from_enabled_roles; Strings getCurrentRolesNames() const; Strings getEnabledRolesNames() const; diff --git a/dbms/src/Access/EnabledSettings.cpp b/dbms/src/Access/EnabledSettings.cpp new file mode 100644 index 00000000000..65e38e4827f --- /dev/null +++ b/dbms/src/Access/EnabledSettings.cpp @@ -0,0 +1,36 @@ +#include + + +namespace DB +{ + +EnabledSettings::EnabledSettings(const Params & params_) : params(params_) +{ +} + +EnabledSettings::~EnabledSettings() = default; + + +std::shared_ptr EnabledSettings::getSettings() const +{ + std::lock_guard lock{mutex}; + return settings; +} + + +std::shared_ptr EnabledSettings::getConstraints() const +{ + std::lock_guard lock{mutex}; + return constraints; +} + + +void EnabledSettings::setSettingsAndConstraints( + const std::shared_ptr & settings_, const std::shared_ptr & constraints_) +{ + std::lock_guard lock{mutex}; + settings = settings_; + constraints = constraints_; +} + +} diff --git a/dbms/src/Access/EnabledSettings.h b/dbms/src/Access/EnabledSettings.h new file mode 100644 index 00000000000..d8e969d685d --- /dev/null +++ b/dbms/src/Access/EnabledSettings.h @@ -0,0 +1,56 @@ +#pragma once + +#include +#include +#include +#include +#include +#include + + +namespace DB +{ +/// Watches settings profiles for a specific user and roles. +class EnabledSettings +{ +public: + struct Params + { + UUID user_id; + std::vector enabled_roles; + SettingsProfileElements settings_from_enabled_roles; + SettingsProfileElements settings_from_user; + + auto toTuple() const { return std::tie(user_id, enabled_roles, settings_from_enabled_roles, settings_from_user); } + friend bool operator ==(const Params & lhs, const Params & rhs) { return lhs.toTuple() == rhs.toTuple(); } + friend bool operator !=(const Params & lhs, const Params & rhs) { return !(lhs == rhs); } + friend bool operator <(const Params & lhs, const Params & rhs) { return lhs.toTuple() < rhs.toTuple(); } + friend bool operator >(const Params & lhs, const Params & rhs) { return rhs < lhs; } + friend bool operator <=(const Params & lhs, const Params & rhs) { return !(rhs < lhs); } + friend bool operator >=(const Params & lhs, const Params & rhs) { return !(lhs < rhs); } + }; + + ~EnabledSettings(); + + /// Returns the default settings come from settings profiles defined for the user + /// and the roles passed in the constructor. + std::shared_ptr getSettings() const; + + /// Returns the constraints come from settings profiles defined for the user + /// and the roles passed in the constructor. + std::shared_ptr getConstraints() const; + +private: + friend class SettingsProfilesCache; + EnabledSettings(const Params & params_); + + void setSettingsAndConstraints( + const std::shared_ptr & settings_, const std::shared_ptr & constraints_); + + const Params params; + SettingsProfileElements settings_from_enabled; + std::shared_ptr settings; + std::shared_ptr constraints; + mutable std::mutex mutex; +}; +} diff --git a/dbms/src/Access/Role.cpp b/dbms/src/Access/Role.cpp index 7b1a395feec..f20ef9b9bfa 100644 --- a/dbms/src/Access/Role.cpp +++ b/dbms/src/Access/Role.cpp @@ -10,7 +10,8 @@ bool Role::equal(const IAccessEntity & other) const return false; const auto & other_role = typeid_cast(other); return (access == other_role.access) && (access_with_grant_option == other_role.access_with_grant_option) - && (granted_roles == other_role.granted_roles) && (granted_roles_with_admin_option == other_role.granted_roles_with_admin_option); + && (granted_roles == other_role.granted_roles) && (granted_roles_with_admin_option == other_role.granted_roles_with_admin_option) + && (settings == other_role.settings); } } diff --git a/dbms/src/Access/Role.h b/dbms/src/Access/Role.h index eaeb8debd3a..04330ba85f5 100644 --- a/dbms/src/Access/Role.h +++ b/dbms/src/Access/Role.h @@ -2,6 +2,7 @@ #include #include +#include #include #include @@ -15,6 +16,7 @@ struct Role : public IAccessEntity AccessRights access_with_grant_option; boost::container::flat_set granted_roles; boost::container::flat_set granted_roles_with_admin_option; + SettingsProfileElements settings; bool equal(const IAccessEntity & other) const override; std::shared_ptr clone() const override { return cloneImpl(); } diff --git a/dbms/src/Access/RoleCache.cpp b/dbms/src/Access/RoleCache.cpp index 107f3aa7577..63e19a3cb40 100644 --- a/dbms/src/Access/RoleCache.cpp +++ b/dbms/src/Access/RoleCache.cpp @@ -61,6 +61,7 @@ namespace new_info->names_of_roles[role_id] = role->getName(); new_info->access.merge(role->access); new_info->access_with_grant_option.merge(role->access_with_grant_option); + new_info->settings_from_enabled_roles.merge(role->settings); } return new_info; } diff --git a/dbms/src/Access/SettingsProfile.cpp b/dbms/src/Access/SettingsProfile.cpp new file mode 100644 index 00000000000..c2f868502c0 --- /dev/null +++ b/dbms/src/Access/SettingsProfile.cpp @@ -0,0 +1,13 @@ +#include + + +namespace DB +{ +bool SettingsProfile::equal(const IAccessEntity & other) const +{ + if (!IAccessEntity::equal(other)) + return false; + const auto & other_profile = typeid_cast(other); + return (elements == other_profile.elements) && (to_roles == other_profile.to_roles); +} +} diff --git a/dbms/src/Access/SettingsProfile.h b/dbms/src/Access/SettingsProfile.h new file mode 100644 index 00000000000..b73b45d57cf --- /dev/null +++ b/dbms/src/Access/SettingsProfile.h @@ -0,0 +1,24 @@ +#pragma once + +#include +#include +#include + + +namespace DB +{ +/// Represents a settings profile created by command +/// CREATE SETTINGS PROFILE name SETTINGS x=value MIN=min MAX=max READONLY... TO roles +struct SettingsProfile : public IAccessEntity +{ + SettingsProfileElements elements; + + /// Which roles or users should use this settings profile. + ExtendedRoleSet to_roles; + + bool equal(const IAccessEntity & other) const override; + std::shared_ptr clone() const override { return cloneImpl(); } +}; + +using SettingsProfilePtr = std::shared_ptr; +} diff --git a/dbms/src/Access/SettingsProfileElement.cpp b/dbms/src/Access/SettingsProfileElement.cpp new file mode 100644 index 00000000000..8ed6bbde88c --- /dev/null +++ b/dbms/src/Access/SettingsProfileElement.cpp @@ -0,0 +1,54 @@ +#include +#include +#include + + +namespace DB +{ +void SettingsProfileElements::merge(const SettingsProfileElements & other) +{ + insert(end(), other.begin(), other.end()); +} + + +Settings SettingsProfileElements::toSettings() const +{ + Settings res; + for (const auto & elem : *this) + { + if (!elem.name.empty() && !elem.value.isNull()) + res.set(elem.name, elem.value); + } + return res; +} + +SettingsChanges SettingsProfileElements::toSettingsChanges() const +{ + SettingsChanges res; + for (const auto & elem : *this) + { + if (!elem.name.empty() && !elem.value.isNull()) + res.push_back({elem.name, elem.value}); + } + return res; +} + +SettingsConstraints SettingsProfileElements::toSettingsConstraints() const +{ + SettingsConstraints res; + for (const auto & elem : *this) + { + if (!elem.name.empty()) + { + if (!elem.min_value.isNull()) + res.setMinValue(elem.name, elem.min_value); + if (!elem.max_value.isNull()) + res.setMaxValue(elem.name, elem.max_value); + if (elem.readonly) + res.setReadOnly(elem.name, *elem.readonly); + } + } + return res; +} + +} diff --git a/dbms/src/Access/SettingsProfileElement.h b/dbms/src/Access/SettingsProfileElement.h new file mode 100644 index 00000000000..0327cb70934 --- /dev/null +++ b/dbms/src/Access/SettingsProfileElement.h @@ -0,0 +1,46 @@ +#pragma once + +#include +#include +#include +#include + + +namespace DB +{ +struct Settings; +struct SettingChange; +using SettingsChanges = std::vector; +class SettingsConstraints; + + +struct SettingsProfileElement +{ + std::optional parent_profile; + String name; + Field value; + Field min_value; + Field max_value; + std::optional readonly; + + auto toTuple() const { return std::tie(parent_profile, name, value, min_value, max_value, readonly); } + friend bool operator==(const SettingsProfileElement & lhs, const SettingsProfileElement & rhs) { return lhs.toTuple() == rhs.toTuple(); } + friend bool operator!=(const SettingsProfileElement & lhs, const SettingsProfileElement & rhs) { return !(lhs == rhs); } + friend bool operator <(const SettingsProfileElement & lhs, const SettingsProfileElement & rhs) { return lhs.toTuple() < rhs.toTuple(); } + friend bool operator >(const SettingsProfileElement & lhs, const SettingsProfileElement & rhs) { return rhs < lhs; } + friend bool operator <=(const SettingsProfileElement & lhs, const SettingsProfileElement & rhs) { return !(rhs < lhs); } + friend bool operator >=(const SettingsProfileElement & lhs, const SettingsProfileElement & rhs) { return !(lhs < rhs); } +}; + + +class SettingsProfileElements : public std::vector +{ +public: + void merge(const SettingsProfileElements & other); + + Settings toSettings() const; + SettingsChanges toSettingsChanges() const; + SettingsConstraints toSettingsConstraints() const; +}; + +} diff --git a/dbms/src/Access/SettingsProfilesCache.cpp b/dbms/src/Access/SettingsProfilesCache.cpp new file mode 100644 index 00000000000..552ed324635 --- /dev/null +++ b/dbms/src/Access/SettingsProfilesCache.cpp @@ -0,0 +1,234 @@ +#include +#include +#include +#include +#include +#include + + +namespace DB +{ +namespace ErrorCodes +{ + extern const int THERE_IS_NO_PROFILE; +} + + +SettingsProfilesCache::SettingsProfilesCache(const AccessControlManager & manager_) + : manager(manager_) {} + +SettingsProfilesCache::~SettingsProfilesCache() = default; + + +void SettingsProfilesCache::ensureAllProfilesRead() +{ + /// `mutex` is already locked. + if (all_profiles_read) + return; + all_profiles_read = true; + + subscription = manager.subscribeForChanges( + [&](const UUID & id, const AccessEntityPtr & entity) + { + if (entity) + profileAddedOrChanged(id, typeid_cast(entity)); + else + profileRemoved(id); + }); + + for (const UUID & id : manager.findAll()) + { + auto profile = manager.tryRead(id); + if (profile) + { + all_profiles.emplace(id, profile); + profiles_by_name[profile->getName()] = id; + } + } +} + + +void SettingsProfilesCache::profileAddedOrChanged(const UUID & profile_id, const SettingsProfilePtr & new_profile) +{ + std::lock_guard lock{mutex}; + auto it = all_profiles.find(profile_id); + if (it == all_profiles.end()) + { + all_profiles.emplace(profile_id, new_profile); + profiles_by_name[new_profile->getName()] = profile_id; + } + else + { + auto old_profile = it->second; + it->second = new_profile; + if (old_profile->getName() != new_profile->getName()) + profiles_by_name.erase(old_profile->getName()); + profiles_by_name[new_profile->getName()] = profile_id; + } + settings_for_profiles.clear(); + mergeSettingsAndConstraints(); +} + + +void SettingsProfilesCache::profileRemoved(const UUID & profile_id) +{ + std::lock_guard lock{mutex}; + auto it = all_profiles.find(profile_id); + if (it == all_profiles.end()) + return; + profiles_by_name.erase(it->second->getName()); + all_profiles.erase(it); + settings_for_profiles.clear(); + mergeSettingsAndConstraints(); +} + + +void SettingsProfilesCache::setDefaultProfileName(const String & default_profile_name) +{ + std::lock_guard lock{mutex}; + ensureAllProfilesRead(); + + if (default_profile_name.empty()) + { + default_profile_id = {}; + return; + } + + auto it = profiles_by_name.find(default_profile_name); + if (it == profiles_by_name.end()) + throw Exception("Settings profile " + backQuote(default_profile_name) + " not found", ErrorCodes::THERE_IS_NO_PROFILE); + + default_profile_id = it->second; +} + +void SettingsProfilesCache::mergeSettingsAndConstraints() +{ + /// `mutex` is already locked. + std::erase_if( + enabled_settings, + [&](const std::pair> & pr) + { + auto enabled = pr.second.lock(); + if (!enabled) + return true; // remove from the `enabled_settings` list. + mergeSettingsAndConstraintsFor(*enabled); + return false; // keep in the `enabled_settings` list. + }); +} + + +void SettingsProfilesCache::mergeSettingsAndConstraintsFor(EnabledSettings & enabled) const +{ + SettingsProfileElements merged_settings; + if (default_profile_id) + { + SettingsProfileElement new_element; + new_element.parent_profile = *default_profile_id; + merged_settings.emplace_back(new_element); + } + + for (const auto & [profile_id, profile] : all_profiles) + if (profile->to_roles.match(enabled.params.user_id, enabled.params.enabled_roles)) + { + SettingsProfileElement new_element; + new_element.parent_profile = profile_id; + merged_settings.emplace_back(new_element); + } + + merged_settings.merge(enabled.params.settings_from_enabled_roles); + merged_settings.merge(enabled.params.settings_from_user); + + substituteProfiles(merged_settings); + + enabled.setSettingsAndConstraints( + std::make_shared(merged_settings.toSettings()), + std::make_shared(merged_settings.toSettingsConstraints())); +} + + +void SettingsProfilesCache::substituteProfiles(SettingsProfileElements & elements) const +{ + bool stop_substituting = false; + boost::container::flat_set already_substituted; + while (!stop_substituting) + { + stop_substituting = true; + for (size_t i = 0; i != elements.size(); ++i) + { + auto & element = elements[i]; + if (!element.parent_profile) + continue; + + auto parent_profile_id = *element.parent_profile; + element.parent_profile.reset(); + if (already_substituted.contains(parent_profile_id)) + continue; + + already_substituted.insert(parent_profile_id); + auto parent_profile = all_profiles.find(parent_profile_id); + if (parent_profile == all_profiles.end()) + continue; + + const auto & parent_profile_elements = parent_profile->second->elements; + elements.insert(elements.begin() + i + 1, parent_profile_elements.begin(), parent_profile_elements.end()); + i += parent_profile_elements.size(); + stop_substituting = false; + } + } +} + + +std::shared_ptr SettingsProfilesCache::getEnabledSettings( + const UUID & user_id, + const SettingsProfileElements & settings_from_user, + const std::vector & enabled_roles, + const SettingsProfileElements & settings_from_enabled_roles) +{ + std::lock_guard lock{mutex}; + ensureAllProfilesRead(); + + EnabledSettings::Params params; + params.user_id = user_id; + params.settings_from_user = settings_from_user; + params.enabled_roles = enabled_roles; + params.settings_from_enabled_roles = settings_from_enabled_roles; + + auto it = enabled_settings.find(params); + if (it != enabled_settings.end()) + { + auto from_cache = it->second.lock(); + if (from_cache) + return from_cache; + enabled_settings.erase(it); + } + + std::shared_ptr res(new EnabledSettings(params)); + enabled_settings.emplace(std::move(params), res); + mergeSettingsAndConstraintsFor(*res); + return res; +} + + +std::shared_ptr SettingsProfilesCache::getProfileSettings(const String & profile_name) +{ + std::lock_guard lock{mutex}; + ensureAllProfilesRead(); + + auto it = profiles_by_name.find(profile_name); + if (it == profiles_by_name.end()) + throw Exception("Settings profile " + backQuote(profile_name) + " not found", ErrorCodes::THERE_IS_NO_PROFILE); + const UUID profile_id = it->second; + + auto it2 = settings_for_profiles.find(profile_id); + if (it2 != settings_for_profiles.end()) + return it2->second; + + SettingsProfileElements elements = all_profiles[profile_id]->elements; + substituteProfiles(elements); + auto res = std::make_shared(elements.toSettingsChanges()); + settings_for_profiles.emplace(profile_id, res); + return res; +} + + +} diff --git a/dbms/src/Access/SettingsProfilesCache.h b/dbms/src/Access/SettingsProfilesCache.h new file mode 100644 index 00000000000..656ffc6fce6 --- /dev/null +++ b/dbms/src/Access/SettingsProfilesCache.h @@ -0,0 +1,55 @@ +#pragma once + +#include +#include +#include +#include +#include +#include + + +namespace DB +{ +class AccessControlManager; +struct SettingsProfile; +using SettingsProfilePtr = std::shared_ptr; +class SettingsProfileElements; +class EnabledSettings; + + +/// Reads and caches all the settings profiles. +class SettingsProfilesCache +{ +public: + SettingsProfilesCache(const AccessControlManager & manager_); + ~SettingsProfilesCache(); + + void setDefaultProfileName(const String & default_profile_name); + + std::shared_ptr getEnabledSettings( + const UUID & user_id, + const SettingsProfileElements & settings_from_user_, + const std::vector & enabled_roles, + const SettingsProfileElements & settings_from_enabled_roles_); + + std::shared_ptr getProfileSettings(const String & profile_name); + +private: + void ensureAllProfilesRead(); + void profileAddedOrChanged(const UUID & profile_id, const SettingsProfilePtr & new_profile); + void profileRemoved(const UUID & profile_id); + void mergeSettingsAndConstraints(); + void mergeSettingsAndConstraintsFor(EnabledSettings & enabled) const; + void substituteProfiles(SettingsProfileElements & elements) const; + + const AccessControlManager & manager; + std::unordered_map all_profiles; + std::unordered_map profiles_by_name; + bool all_profiles_read = false; + ext::scope_guard subscription; + std::map> enabled_settings; + std::optional default_profile_id; + std::unordered_map> settings_for_profiles; + mutable std::mutex mutex; +}; +} diff --git a/dbms/src/Access/User.cpp b/dbms/src/Access/User.cpp index bc5b062db6a..4a751c31e25 100644 --- a/dbms/src/Access/User.cpp +++ b/dbms/src/Access/User.cpp @@ -12,7 +12,7 @@ bool User::equal(const IAccessEntity & other) const return (authentication == other_user.authentication) && (allowed_client_hosts == other_user.allowed_client_hosts) && (access == other_user.access) && (access_with_grant_option == other_user.access_with_grant_option) && (granted_roles == other_user.granted_roles) && (granted_roles_with_admin_option == other_user.granted_roles_with_admin_option) - && (default_roles == other_user.default_roles) && (profile == other_user.profile); + && (default_roles == other_user.default_roles) && (settings == other_user.settings); } } diff --git a/dbms/src/Access/User.h b/dbms/src/Access/User.h index a01e0332a2c..6df3b3e4d3c 100644 --- a/dbms/src/Access/User.h +++ b/dbms/src/Access/User.h @@ -5,6 +5,7 @@ #include #include #include +#include #include #include @@ -22,7 +23,7 @@ struct User : public IAccessEntity boost::container::flat_set granted_roles; boost::container::flat_set granted_roles_with_admin_option; ExtendedRoleSet default_roles = ExtendedRoleSet::AllTag{}; - String profile; + SettingsProfileElements settings; bool equal(const IAccessEntity & other) const override; std::shared_ptr clone() const override { return cloneImpl(); } diff --git a/dbms/src/Access/UsersConfigAccessStorage.cpp b/dbms/src/Access/UsersConfigAccessStorage.cpp index b32be09e5dc..13102528108 100644 --- a/dbms/src/Access/UsersConfigAccessStorage.cpp +++ b/dbms/src/Access/UsersConfigAccessStorage.cpp @@ -2,11 +2,15 @@ #include #include #include +#include #include #include #include #include #include +#include +#include +#include #include @@ -16,6 +20,7 @@ namespace ErrorCodes { extern const int BAD_ARGUMENTS; extern const int UNKNOWN_ADDRESS_PATTERN_TYPE; + extern const int NOT_IMPLEMENTED; } @@ -29,6 +34,8 @@ namespace return 'Q'; if (type == typeid(RowPolicy)) return 'P'; + if (type == typeid(SettingsProfile)) + return 'S'; return 0; } @@ -82,7 +89,14 @@ namespace user->authentication.setPasswordHashHex(config.getString(user_config + ".password_double_sha1_hex")); } - user->profile = config.getString(user_config + ".profile"); + const auto profile_name_config = user_config + ".profile"; + if (config.has(profile_name_config)) + { + auto profile_name = config.getString(profile_name_config); + SettingsProfileElement profile_element; + profile_element.parent_profile = generateID(typeid(SettingsProfile), profile_name); + user->settings.push_back(std::move(profile_element)); + } /// Fill list of allowed hosts. const auto networks_config = user_config + ".networks"; @@ -330,6 +344,93 @@ namespace } return policies; } + + + SettingsProfileElements parseSettingsConstraints(const Poco::Util::AbstractConfiguration & config, + const String & path_to_constraints) + { + SettingsProfileElements profile_elements; + Poco::Util::AbstractConfiguration::Keys names; + config.keys(path_to_constraints, names); + for (const String & name : names) + { + SettingsProfileElement profile_element; + profile_element.name = name; + Poco::Util::AbstractConfiguration::Keys constraint_types; + String path_to_name = path_to_constraints + "." + name; + config.keys(path_to_name, constraint_types); + for (const String & constraint_type : constraint_types) + { + if (constraint_type == "min") + profile_element.min_value = config.getString(path_to_name + "." + constraint_type); + else if (constraint_type == "max") + profile_element.max_value = config.getString(path_to_name + "." + constraint_type); + else if (constraint_type == "readonly") + profile_element.readonly = true; + else + throw Exception("Setting " + constraint_type + " value for " + name + " isn't supported", ErrorCodes::NOT_IMPLEMENTED); + } + profile_elements.push_back(std::move(profile_element)); + } + return profile_elements; + } + + std::shared_ptr parseSettingsProfile( + const Poco::Util::AbstractConfiguration & config, + const String & profile_name) + { + auto profile = std::make_shared(); + profile->setName(profile_name); + String profile_config = "profiles." + profile_name; + + Poco::Util::AbstractConfiguration::Keys keys; + config.keys(profile_config, keys); + + for (const std::string & key : keys) + { + if (key == "profile" || key.starts_with("profile[")) + { + String parent_profile_name = config.getString(profile_config + "." + key); + SettingsProfileElement profile_element; + profile_element.parent_profile = generateID(typeid(SettingsProfile), parent_profile_name); + profile->elements.emplace_back(std::move(profile_element)); + continue; + } + + if (key == "constraints" || key.starts_with("constraints[")) + { + profile->elements.merge(parseSettingsConstraints(config, profile_config + "." + key)); + continue; + } + + SettingsProfileElement profile_element; + profile_element.name = key; + profile_element.value = config.getString(profile_config + "." + key); + profile->elements.emplace_back(std::move(profile_element)); + } + + return profile; + } + + + std::vector parseSettingsProfiles(const Poco::Util::AbstractConfiguration & config, Poco::Logger * log) + { + std::vector profiles; + Poco::Util::AbstractConfiguration::Keys profile_names; + config.keys("profiles", profile_names); + for (const auto & profile_name : profile_names) + { + try + { + profiles.push_back(parseSettingsProfile(config, profile_name)); + } + catch (...) + { + tryLogCurrentException(log, "Could not parse profile " + backQuote(profile_name)); + } + } + return profiles; + } } @@ -347,6 +448,8 @@ void UsersConfigAccessStorage::setConfiguration(const Poco::Util::AbstractConfig all_entities.emplace_back(generateID(*entity), entity); for (const auto & entity : parseRowPolicies(config, getLogger())) all_entities.emplace_back(generateID(*entity), entity); + for (const auto & entity : parseSettingsProfiles(config, getLogger())) + all_entities.emplace_back(generateID(*entity), entity); memory_storage.setAll(all_entities); } diff --git a/dbms/src/Interpreters/Context.cpp b/dbms/src/Interpreters/Context.cpp index 35c8d8a5f76..6ca4b4a0a2e 100644 --- a/dbms/src/Interpreters/Context.cpp +++ b/dbms/src/Interpreters/Context.cpp @@ -30,6 +30,7 @@ #include #include #include +#include #include #include #include @@ -633,7 +634,7 @@ void Context::setUser(const String & name, const String & password, const Poco:: std::shared_ptr new_access; if (new_user_id) { - new_access = getAccessControlManager().getContextAccess(*new_user_id, {}, true, settings, current_database, client_info); + new_access = getAccessControlManager().getContextAccess(*new_user_id, {}, true, {}, current_database, client_info); if (!new_access->isClientHostAllowed() || !new_access->isCorrectPassword(password)) { new_user_id = {}; @@ -649,7 +650,7 @@ void Context::setUser(const String & name, const String & password, const Poco:: current_roles.clear(); use_default_roles = true; - calculateUserSettings(); + setSettings(*access->getDefaultSettings()); } std::shared_ptr Context::getUser() const @@ -776,42 +777,9 @@ std::shared_ptr Context::getQuota() const } -void Context::calculateUserSettings() +void Context::setProfile(const String & profile_name) { - auto lock = getLock(); - String profile = getUser()->profile; - - bool old_readonly = settings.readonly; - bool old_allow_ddl = settings.allow_ddl; - bool old_allow_introspection_functions = settings.allow_introspection_functions; - - /// 1) Set default settings (hardcoded values) - /// NOTE: we ignore global_context settings (from which it is usually copied) - /// NOTE: global_context settings are immutable and not auto updated - settings = Settings(); - settings_constraints = nullptr; - - /// 2) Apply settings from default profile - auto default_profile_name = getDefaultProfileName(); - if (profile != default_profile_name) - setProfile(default_profile_name); - - /// 3) Apply settings from current user - setProfile(profile); - - /// 4) Recalculate access rights if it's necessary. - if ((settings.readonly != old_readonly) || (settings.allow_ddl != old_allow_ddl) || (settings.allow_introspection_functions != old_allow_introspection_functions)) - calculateAccessRights(); -} - -void Context::setProfile(const String & profile) -{ - settings.setProfile(profile, *shared->users_config); - - auto new_constraints - = settings_constraints ? std::make_shared(*settings_constraints) : std::make_shared(); - new_constraints->setProfile(profile, *shared->users_config); - settings_constraints = std::move(new_constraints); + applySettingsChanges(*getAccessControlManager().getProfileSettings(profile_name)); } @@ -993,30 +961,37 @@ void Context::applySettingsChanges(const SettingsChanges & changes) void Context::checkSettingsConstraints(const SettingChange & change) const { - if (settings_constraints) + if (auto settings_constraints = getSettingsConstraints()) settings_constraints->check(settings, change); } void Context::checkSettingsConstraints(const SettingsChanges & changes) const { - if (settings_constraints) + if (auto settings_constraints = getSettingsConstraints()) settings_constraints->check(settings, changes); } void Context::clampToSettingsConstraints(SettingChange & change) const { - if (settings_constraints) + if (auto settings_constraints = getSettingsConstraints()) settings_constraints->clamp(settings, change); } void Context::clampToSettingsConstraints(SettingsChanges & changes) const { - if (settings_constraints) + if (auto settings_constraints = getSettingsConstraints()) settings_constraints->clamp(settings, changes); } +std::shared_ptr Context::getSettingsConstraints() const +{ + auto lock = getLock(); + return access->getSettingsConstraints(); +} + + String Context::getCurrentDatabase() const { auto lock = getLock(); @@ -1877,8 +1852,10 @@ void Context::setApplicationType(ApplicationType type) void Context::setDefaultProfiles(const Poco::Util::AbstractConfiguration & config) { shared->default_profile_name = config.getString("default_profile", "default"); + getAccessControlManager().setDefaultProfileName(shared->default_profile_name); + shared->system_profile_name = config.getString("system_profile", shared->default_profile_name); - setSetting("profile", shared->system_profile_name); + setProfile(shared->system_profile_name); } String Context::getDefaultProfileName() const diff --git a/dbms/src/Interpreters/Context.h b/dbms/src/Interpreters/Context.h index 14e1346dea1..331c89294d0 100644 --- a/dbms/src/Interpreters/Context.h +++ b/dbms/src/Interpreters/Context.h @@ -153,7 +153,6 @@ private: std::shared_ptr initial_row_policy; String current_database; Settings settings; /// Setting for query execution. - std::shared_ptr settings_constraints; using ProgressCallback = std::function; ProgressCallback progress_callback; /// Callback for tracking progress of query execution. QueryStatus * process_list_elem = nullptr; /// For tracking total resource usage for query. @@ -353,7 +352,7 @@ public: void clampToSettingsConstraints(SettingsChanges & changes) const; /// Returns the current constraints (can return null). - std::shared_ptr getSettingsConstraints() const { return settings_constraints; } + std::shared_ptr getSettingsConstraints() const; const EmbeddedDictionaries & getEmbeddedDictionaries() const; const ExternalDictionariesLoader & getExternalDictionariesLoader() const; @@ -593,7 +592,6 @@ private: std::unique_lock getLock() const; /// Compute and set actual user settings, client_info.current_user should be set - void calculateUserSettings(); void calculateAccessRights(); template diff --git a/dbms/src/Interpreters/InterpreterCreateUserQuery.cpp b/dbms/src/Interpreters/InterpreterCreateUserQuery.cpp index f01e2045a2a..2d5b0687691 100644 --- a/dbms/src/Interpreters/InterpreterCreateUserQuery.cpp +++ b/dbms/src/Interpreters/InterpreterCreateUserQuery.cpp @@ -47,9 +47,6 @@ namespace InterpreterSetRoleQuery::updateUserSetDefaultRoles(user, *default_roles); } - - if (query.profile) - user.profile = *query.profile; } } diff --git a/dbms/src/Interpreters/InterpreterShowCreateAccessEntityQuery.cpp b/dbms/src/Interpreters/InterpreterShowCreateAccessEntityQuery.cpp index 8f5aa4cc7d0..dce1bbc7a87 100644 --- a/dbms/src/Interpreters/InterpreterShowCreateAccessEntityQuery.cpp +++ b/dbms/src/Interpreters/InterpreterShowCreateAccessEntityQuery.cpp @@ -43,9 +43,6 @@ namespace if (user.allowed_client_hosts != AllowedClientHosts::AnyHostTag{}) query->hosts = user.allowed_client_hosts; - if (!user.profile.empty()) - query->profile = user.profile; - if (user.default_roles != ExtendedRoleSet::AllTag{}) { if (attach_mode) From 9ca6af32ea6291f3b430c8673291870f5b5d19fa Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Wed, 18 Mar 2020 17:11:44 +0300 Subject: [PATCH 0155/1355] Implement SQL to manage settings profiles. --- dbms/src/Access/AccessFlags.h | 9 +- dbms/src/Access/AccessType.h | 8 + dbms/src/Access/DiskAccessStorage.cpp | 84 ++++++--- dbms/src/Access/DiskAccessStorage.h | 4 +- dbms/src/Access/IAccessEntity.cpp | 23 +++ dbms/src/Access/IAccessEntity.h | 2 + dbms/src/Access/SettingsProfileElement.cpp | 116 +++++++++++++ dbms/src/Access/SettingsProfileElement.h | 22 +++ .../InterpreterCreateRoleQuery.cpp | 46 +++-- .../InterpreterCreateSettingsProfileQuery.cpp | 104 +++++++++++ .../InterpreterCreateSettingsProfileQuery.h | 26 +++ .../InterpreterCreateUserQuery.cpp | 24 ++- .../InterpreterDropAccessEntityQuery.cpp | 97 ++++++----- dbms/src/Interpreters/InterpreterFactory.cpp | 6 + ...InterpreterShowCreateAccessEntityQuery.cpp | 117 +++++++++---- dbms/src/Parsers/ASTCreateQuotaQuery.cpp | 2 +- dbms/src/Parsers/ASTCreateRoleQuery.cpp | 29 +++- dbms/src/Parsers/ASTCreateRoleQuery.h | 11 +- dbms/src/Parsers/ASTCreateRowPolicyQuery.cpp | 2 +- .../Parsers/ASTCreateSettingsProfileQuery.cpp | 74 ++++++++ .../Parsers/ASTCreateSettingsProfileQuery.h | 40 +++++ dbms/src/Parsers/ASTCreateUserQuery.cpp | 39 +++-- dbms/src/Parsers/ASTCreateUserQuery.h | 15 +- dbms/src/Parsers/ASTDropAccessEntityQuery.cpp | 11 +- dbms/src/Parsers/ASTDropAccessEntityQuery.h | 4 +- dbms/src/Parsers/ASTExtendedRoleSet.h | 1 - .../src/Parsers/ASTSettingsProfileElement.cpp | 88 ++++++++++ dbms/src/Parsers/ASTSettingsProfileElement.h | 45 +++++ .../ASTShowCreateAccessEntityQuery.cpp | 16 +- .../Parsers/ASTShowCreateAccessEntityQuery.h | 7 +- dbms/src/Parsers/ParserCreateQuotaQuery.cpp | 9 +- dbms/src/Parsers/ParserCreateRoleQuery.cpp | 39 ++++- dbms/src/Parsers/ParserCreateRoleQuery.h | 4 +- .../Parsers/ParserCreateRowPolicyQuery.cpp | 11 +- .../ParserCreateSettingsProfileQuery.cpp | 130 ++++++++++++++ .../ParserCreateSettingsProfileQuery.h | 28 +++ dbms/src/Parsers/ParserCreateUserQuery.cpp | 30 ++-- dbms/src/Parsers/ParserCreateUserQuery.h | 14 +- .../Parsers/ParserDropAccessEntityQuery.cpp | 4 +- .../src/Parsers/ParserDropAccessEntityQuery.h | 3 +- dbms/src/Parsers/ParserQuery.cpp | 3 + .../Parsers/ParserSettingsProfileElement.cpp | 164 ++++++++++++++++++ .../Parsers/ParserSettingsProfileElement.h | 36 ++++ .../ParserShowCreateAccessEntityQuery.cpp | 17 +- 44 files changed, 1350 insertions(+), 214 deletions(-) create mode 100644 dbms/src/Interpreters/InterpreterCreateSettingsProfileQuery.cpp create mode 100644 dbms/src/Interpreters/InterpreterCreateSettingsProfileQuery.h create mode 100644 dbms/src/Parsers/ASTCreateSettingsProfileQuery.cpp create mode 100644 dbms/src/Parsers/ASTCreateSettingsProfileQuery.h create mode 100644 dbms/src/Parsers/ASTSettingsProfileElement.cpp create mode 100644 dbms/src/Parsers/ASTSettingsProfileElement.h create mode 100644 dbms/src/Parsers/ParserCreateSettingsProfileQuery.cpp create mode 100644 dbms/src/Parsers/ParserCreateSettingsProfileQuery.h create mode 100644 dbms/src/Parsers/ParserSettingsProfileElement.cpp create mode 100644 dbms/src/Parsers/ParserSettingsProfileElement.h diff --git a/dbms/src/Access/AccessFlags.h b/dbms/src/Access/AccessFlags.h index 61a160fc69a..f15e7d1e274 100644 --- a/dbms/src/Access/AccessFlags.h +++ b/dbms/src/Access/AccessFlags.h @@ -355,6 +355,7 @@ private: auto alter_user = std::make_unique("ALTER USER", next_flag++, GLOBAL); auto drop_user = std::make_unique("DROP USER", next_flag++, GLOBAL); auto create_role = std::make_unique("CREATE ROLE", next_flag++, GLOBAL); + auto alter_role = std::make_unique("ALTER ROLE", next_flag++, GLOBAL); auto drop_role = std::make_unique("DROP ROLE", next_flag++, GLOBAL); auto create_policy = std::make_unique("CREATE POLICY", next_flag++, GLOBAL); auto alter_policy = std::make_unique("ALTER POLICY", next_flag++, GLOBAL); @@ -362,8 +363,14 @@ private: auto create_quota = std::make_unique("CREATE QUOTA", next_flag++, GLOBAL); auto alter_quota = std::make_unique("ALTER QUOTA", next_flag++, GLOBAL); auto drop_quota = std::make_unique("DROP QUOTA", next_flag++, GLOBAL); + auto create_profile = std::make_unique("CREATE SETTINGS PROFILE", next_flag++, GLOBAL); + ext::push_back(create_profile->aliases, "CREATE PROFILE"); + auto alter_profile = std::make_unique("ALTER SETTINGS PROFILE", next_flag++, GLOBAL); + ext::push_back(alter_profile->aliases, "ALTER PROFILE"); + auto drop_profile = std::make_unique("DROP SETTINGS PROFILE", next_flag++, GLOBAL); + ext::push_back(drop_profile->aliases, "DROP PROFILE"); auto role_admin = std::make_unique("ROLE ADMIN", next_flag++, GLOBAL); - ext::push_back(all, std::move(create_user), std::move(alter_user), std::move(drop_user), std::move(create_role), std::move(drop_role), std::move(create_policy), std::move(alter_policy), std::move(drop_policy), std::move(create_quota), std::move(alter_quota), std::move(drop_quota), std::move(role_admin)); + ext::push_back(all, std::move(create_user), std::move(alter_user), std::move(drop_user), std::move(create_role), std::move(alter_role), std::move(drop_role), std::move(create_policy), std::move(alter_policy), std::move(drop_policy), std::move(create_quota), std::move(alter_quota), std::move(drop_quota), std::move(create_profile), std::move(alter_profile), std::move(drop_profile), std::move(role_admin)); auto shutdown = std::make_unique("SHUTDOWN", next_flag++, GLOBAL); ext::push_back(shutdown->aliases, "SYSTEM SHUTDOWN", "SYSTEM KILL"); diff --git a/dbms/src/Access/AccessType.h b/dbms/src/Access/AccessType.h index 4084d180013..27892076d59 100644 --- a/dbms/src/Access/AccessType.h +++ b/dbms/src/Access/AccessType.h @@ -84,6 +84,7 @@ enum class AccessType ALTER_USER, DROP_USER, CREATE_ROLE, + ALTER_ROLE, DROP_ROLE, CREATE_POLICY, ALTER_POLICY, @@ -91,6 +92,9 @@ enum class AccessType CREATE_QUOTA, ALTER_QUOTA, DROP_QUOTA, + CREATE_SETTINGS_PROFILE, + ALTER_SETTINGS_PROFILE, + DROP_SETTINGS_PROFILE, ROLE_ADMIN, /// allows to grant and revoke any roles. @@ -247,6 +251,7 @@ namespace impl ACCESS_TYPE_TO_KEYWORD_CASE(ALTER_USER); ACCESS_TYPE_TO_KEYWORD_CASE(DROP_USER); ACCESS_TYPE_TO_KEYWORD_CASE(CREATE_ROLE); + ACCESS_TYPE_TO_KEYWORD_CASE(ALTER_ROLE); ACCESS_TYPE_TO_KEYWORD_CASE(DROP_ROLE); ACCESS_TYPE_TO_KEYWORD_CASE(CREATE_POLICY); ACCESS_TYPE_TO_KEYWORD_CASE(ALTER_POLICY); @@ -254,6 +259,9 @@ namespace impl ACCESS_TYPE_TO_KEYWORD_CASE(CREATE_QUOTA); ACCESS_TYPE_TO_KEYWORD_CASE(ALTER_QUOTA); ACCESS_TYPE_TO_KEYWORD_CASE(DROP_QUOTA); + ACCESS_TYPE_TO_KEYWORD_CASE(CREATE_SETTINGS_PROFILE); + ACCESS_TYPE_TO_KEYWORD_CASE(ALTER_SETTINGS_PROFILE); + ACCESS_TYPE_TO_KEYWORD_CASE(DROP_SETTINGS_PROFILE); ACCESS_TYPE_TO_KEYWORD_CASE(ROLE_ADMIN); ACCESS_TYPE_TO_KEYWORD_CASE(SHUTDOWN); diff --git a/dbms/src/Access/DiskAccessStorage.cpp b/dbms/src/Access/DiskAccessStorage.cpp index f5f42e1ff80..12c65e7df1e 100644 --- a/dbms/src/Access/DiskAccessStorage.cpp +++ b/dbms/src/Access/DiskAccessStorage.cpp @@ -8,15 +8,18 @@ #include #include #include +#include #include #include #include #include +#include #include #include #include #include #include +#include #include #include #include @@ -24,6 +27,7 @@ #include #include #include +#include #include #include #include @@ -64,6 +68,8 @@ namespace return true; if (ParserCreateQuotaQuery{}.enableAttachMode(true).parse(pos, node, expected)) return true; + if (ParserCreateSettingsProfileQuery{}.enableAttachMode(true).parse(pos, node, expected)) + return true; if (ParserGrantQuery{}.enableAttachMode(true).parse(pos, node, expected)) return true; return false; @@ -97,6 +103,7 @@ namespace std::shared_ptr role; std::shared_ptr policy; std::shared_ptr quota; + std::shared_ptr profile; AccessEntityPtr res; for (const auto & query : queries) @@ -129,6 +136,13 @@ namespace res = quota = std::make_unique(); InterpreterCreateQuotaQuery::updateQuotaFromQuery(*quota, *create_quota_query); } + else if (auto create_profile_query = query->as()) + { + if (res) + throw Exception("Two access entities are attached in the same file " + file_path.string(), ErrorCodes::INCORRECT_ACCESS_ENTITY_DEFINITION); + res = profile = std::make_unique(); + InterpreterCreateSettingsProfileQuery::updateSettingsProfileFromQuery(*profile, *create_profile_query); + } else if (auto grant_query = query->as()) { if (!user && !role) @@ -139,7 +153,7 @@ namespace InterpreterGrantQuery::updateRoleFromQuery(*role, *grant_query); } else - throw Exception("Two access entities are attached in the same file " + file_path.string(), ErrorCodes::INCORRECT_ACCESS_ENTITY_DEFINITION); + throw Exception("No interpreter found for query " + query->getID(), ErrorCodes::INCORRECT_ACCESS_ENTITY_DEFINITION); } if (!res) @@ -149,6 +163,20 @@ namespace } + AccessEntityPtr tryReadAccessEntityFile(const std::filesystem::path & file_path, Poco::Logger & log) + { + try + { + return readAccessEntityFile(file_path); + } + catch (...) + { + tryLogCurrentException(&log, "Could not parse " + file_path.string()); + return nullptr; + } + } + + /// Writes ATTACH queries for building a specified access entity to a file. void writeAccessEntityFile(const std::filesystem::path & file_path, const IAccessEntity & entity) { @@ -238,6 +266,8 @@ namespace file_name = "quotas"; else if (type == typeid(RowPolicy)) file_name = "row_policies"; + else if (type == typeid(SettingsProfile)) + file_name = "settings_profiles"; else throw Exception("Unexpected type of access entity: " + IAccessEntity::getTypeName(type), ErrorCodes::LOGICAL_ERROR); @@ -254,13 +284,6 @@ namespace } - const std::vector & getAllAccessEntityTypes() - { - static const std::vector res = {typeid(User), typeid(Role), typeid(RowPolicy), typeid(Quota)}; - return res; - } - - bool tryParseUUID(const String & str, UUID & id) { try @@ -273,13 +296,20 @@ namespace return false; } } + + + const std::vector & getAllAccessEntityTypes() + { + static const std::vector res = {typeid(User), typeid(Role), typeid(RowPolicy), typeid(Quota), typeid(SettingsProfile)}; + return res; + } } DiskAccessStorage::DiskAccessStorage() : IAccessStorage("disk") { - for (const auto & type : getAllAccessEntityTypes()) + for (auto type : getAllAccessEntityTypes()) name_to_id_maps[type]; } @@ -340,10 +370,10 @@ void DiskAccessStorage::initialize(const String & directory_path_, Notifications bool DiskAccessStorage::readLists() { assert(id_to_entry_map.empty()); - assert(name_to_id_maps.size() == getAllAccessEntityTypes().size()); bool ok = true; - for (auto & [type, name_to_id_map] : name_to_id_maps) + for (auto type : getAllAccessEntityTypes()) { + auto & name_to_id_map = name_to_id_maps.at(type); auto file_path = getListFilePath(directory_path, type); if (!std::filesystem::exists(file_path)) { @@ -362,6 +392,7 @@ bool DiskAccessStorage::readLists() ok = false; break; } + for (const auto & [name, id] : name_to_id_map) id_to_entry_map.emplace(id, Entry{name, type}); } @@ -376,11 +407,14 @@ bool DiskAccessStorage::readLists() } -void DiskAccessStorage::writeLists() +bool DiskAccessStorage::writeLists() { - if (failed_to_write_lists || types_of_lists_to_write.empty()) - return; /// We don't try to write list files after the first fail. - /// The next restart of the server will invoke rebuilding of the list files. + if (failed_to_write_lists) + return false; /// We don't try to write list files after the first fail. + /// The next restart of the server will invoke rebuilding of the list files. + + if (types_of_lists_to_write.empty()) + return true; for (const auto & type : types_of_lists_to_write) { @@ -395,13 +429,14 @@ void DiskAccessStorage::writeLists() tryLogCurrentException(getLogger(), "Could not write " + file_path.string()); failed_to_write_lists = true; types_of_lists_to_write.clear(); - return; + return false; } } /// The list files was successfully written, we don't need the 'need_rebuild_lists.mark' file any longer. std::filesystem::remove(getNeedRebuildListsMarkFilePath(directory_path)); types_of_lists_to_write.clear(); + return true; } @@ -465,10 +500,11 @@ void DiskAccessStorage::listsWritingThreadFunc() /// Reads and parses all the ".sql" files from a specified directory /// and then saves the files "users.list", "roles.list", etc. to the same directory. -void DiskAccessStorage::rebuildLists() +bool DiskAccessStorage::rebuildLists() { LOG_WARNING(getLogger(), "Recovering lists in directory " + directory_path); assert(id_to_entry_map.empty()); + for (const auto & directory_entry : std::filesystem::directory_iterator(directory_path)) { if (!directory_entry.is_regular_file()) @@ -481,14 +517,21 @@ void DiskAccessStorage::rebuildLists() if (!tryParseUUID(path.stem(), id)) continue; - auto entity = readAccessEntityFile(getAccessEntityFilePath(directory_path, id)); + const auto access_entity_file_path = getAccessEntityFilePath(directory_path, id); + auto entity = tryReadAccessEntityFile(access_entity_file_path, *getLogger()); + if (!entity) + continue; + auto type = entity->getType(); - auto & name_to_id_map = name_to_id_maps[type]; + auto & name_to_id_map = name_to_id_maps.at(type); auto it_by_name = name_to_id_map.emplace(entity->getFullName(), id).first; id_to_entry_map.emplace(id, Entry{it_by_name->first, type}); } - boost::range::copy(getAllAccessEntityTypes(), std::inserter(types_of_lists_to_write, types_of_lists_to_write.end())); + for (auto type : getAllAccessEntityTypes()) + types_of_lists_to_write.insert(type); + + return true; } @@ -499,6 +542,7 @@ std::optional DiskAccessStorage::findImpl(std::type_index type, const Stri auto it = name_to_id_map.find(name); if (it == name_to_id_map.end()) return {}; + return it->second; } diff --git a/dbms/src/Access/DiskAccessStorage.h b/dbms/src/Access/DiskAccessStorage.h index 935cebfece9..104c0f1fa38 100644 --- a/dbms/src/Access/DiskAccessStorage.h +++ b/dbms/src/Access/DiskAccessStorage.h @@ -33,9 +33,9 @@ private: void initialize(const String & directory_path_, Notifications & notifications); bool readLists(); - void writeLists(); + bool writeLists(); void scheduleWriteLists(std::type_index type); - void rebuildLists(); + bool rebuildLists(); void startListsWritingThread(); void stopListsWritingThread(); diff --git a/dbms/src/Access/IAccessEntity.cpp b/dbms/src/Access/IAccessEntity.cpp index 361946863b2..5dbc056b71c 100644 --- a/dbms/src/Access/IAccessEntity.cpp +++ b/dbms/src/Access/IAccessEntity.cpp @@ -2,6 +2,8 @@ #include #include #include +#include +#include #include @@ -15,9 +17,30 @@ String IAccessEntity::getTypeName(std::type_index type) return "Quota"; if (type == typeid(RowPolicy)) return "Row policy"; + if (type == typeid(Role)) + return "Role"; + if (type == typeid(SettingsProfile)) + return "Settings profile"; return demangle(type.name()); } + +const char * IAccessEntity::getKeyword(std::type_index type) +{ + if (type == typeid(User)) + return "USER"; + if (type == typeid(Quota)) + return "QUOTA"; + if (type == typeid(RowPolicy)) + return "ROW POLICY"; + if (type == typeid(Role)) + return "ROLE"; + if (type == typeid(SettingsProfile)) + return "SETTINGS PROFILE"; + __builtin_unreachable(); +} + + bool IAccessEntity::equal(const IAccessEntity & other) const { return (full_name == other.full_name) && (getType() == other.getType()); diff --git a/dbms/src/Access/IAccessEntity.h b/dbms/src/Access/IAccessEntity.h index 272fde006ac..9214d64aa8c 100644 --- a/dbms/src/Access/IAccessEntity.h +++ b/dbms/src/Access/IAccessEntity.h @@ -20,6 +20,8 @@ struct IAccessEntity std::type_index getType() const { return typeid(*this); } static String getTypeName(std::type_index type); const String getTypeName() const { return getTypeName(getType()); } + static const char * getKeyword(std::type_index type); + const char * getKeyword() const { return getKeyword(getType()); } template bool isTypeOf() const { return isTypeOf(typeid(EntityType)); } diff --git a/dbms/src/Access/SettingsProfileElement.cpp b/dbms/src/Access/SettingsProfileElement.cpp index 8ed6bbde88c..b052f8b5e75 100644 --- a/dbms/src/Access/SettingsProfileElement.cpp +++ b/dbms/src/Access/SettingsProfileElement.cpp @@ -1,10 +1,126 @@ #include #include +#include +#include +#include #include +#include +#include namespace DB { +SettingsProfileElement::SettingsProfileElement(const ASTSettingsProfileElement & ast) +{ + init(ast, nullptr); +} + +SettingsProfileElement::SettingsProfileElement(const ASTSettingsProfileElement & ast, const AccessControlManager & manager) +{ + init(ast, &manager); +} + +void SettingsProfileElement::init(const ASTSettingsProfileElement & ast, const AccessControlManager * manager) +{ + auto name_to_id = [id_mode{ast.id_mode}, manager](const String & name_) -> UUID + { + if (id_mode) + return parse(name_); + assert(manager); + return manager->getID(name_); + }; + + if (!ast.parent_profile.empty()) + parent_profile = name_to_id(ast.parent_profile); + + if (!ast.name.empty()) + { + name = ast.name; + value = ast.value; + min_value = ast.min_value; + max_value = ast.max_value; + readonly = ast.readonly; + + size_t index = Settings::findIndexStrict(name); + if (!value.isNull()) + value = Settings::valueToCorrespondingType(index, value); + if (!min_value.isNull()) + min_value = Settings::valueToCorrespondingType(index, min_value); + if (!max_value.isNull()) + max_value = Settings::valueToCorrespondingType(index, max_value); + } +} + + +std::shared_ptr SettingsProfileElement::toAST() const +{ + auto ast = std::make_shared(); + ast->id_mode = true; + + if (parent_profile) + ast->parent_profile = ::DB::toString(*parent_profile); + + ast->name = name; + ast->value = value; + ast->min_value = min_value; + ast->max_value = max_value; + ast->readonly = readonly; + + return ast; +} + + +std::shared_ptr SettingsProfileElement::toASTWithNames(const AccessControlManager & manager) const +{ + auto ast = std::make_shared(); + + if (parent_profile) + { + auto parent_profile_name = manager.tryReadName(*parent_profile); + if (parent_profile_name) + ast->parent_profile = *parent_profile_name; + } + + ast->name = name; + ast->value = value; + ast->min_value = min_value; + ast->max_value = max_value; + ast->readonly = readonly; + + return ast; +} + + +SettingsProfileElements::SettingsProfileElements(const ASTSettingsProfileElements & ast) +{ + for (const auto & ast_element : ast.elements) + emplace_back(*ast_element); +} + +SettingsProfileElements::SettingsProfileElements(const ASTSettingsProfileElements & ast, const AccessControlManager & manager) +{ + for (const auto & ast_element : ast.elements) + emplace_back(*ast_element, manager); +} + + +std::shared_ptr SettingsProfileElements::toAST() const +{ + auto res = std::make_shared(); + for (const auto & element : *this) + res->elements.push_back(element.toAST()); + return res; +} + +std::shared_ptr SettingsProfileElements::toASTWithNames(const AccessControlManager & manager) const +{ + auto res = std::make_shared(); + for (const auto & element : *this) + res->elements.push_back(element.toASTWithNames(manager)); + return res; +} + + void SettingsProfileElements::merge(const SettingsProfileElements & other) { insert(end(), other.begin(), other.end()); diff --git a/dbms/src/Access/SettingsProfileElement.h b/dbms/src/Access/SettingsProfileElement.h index 0327cb70934..abcac2567c8 100644 --- a/dbms/src/Access/SettingsProfileElement.h +++ b/dbms/src/Access/SettingsProfileElement.h @@ -12,6 +12,9 @@ struct Settings; struct SettingChange; using SettingsChanges = std::vector; class SettingsConstraints; +class ASTSettingsProfileElement; +class ASTSettingsProfileElements; +class AccessControlManager; struct SettingsProfileElement @@ -30,12 +33,31 @@ struct SettingsProfileElement friend bool operator >(const SettingsProfileElement & lhs, const SettingsProfileElement & rhs) { return rhs < lhs; } friend bool operator <=(const SettingsProfileElement & lhs, const SettingsProfileElement & rhs) { return !(rhs < lhs); } friend bool operator >=(const SettingsProfileElement & lhs, const SettingsProfileElement & rhs) { return !(lhs < rhs); } + + SettingsProfileElement() {} + + /// The constructor from AST requires the AccessControlManager if `ast.id_mode == false`. + SettingsProfileElement(const ASTSettingsProfileElement & ast); + SettingsProfileElement(const ASTSettingsProfileElement & ast, const AccessControlManager & manager); + std::shared_ptr toAST() const; + std::shared_ptr toASTWithNames(const AccessControlManager & manager) const; + +private: + void init(const ASTSettingsProfileElement & ast, const AccessControlManager * manager); }; class SettingsProfileElements : public std::vector { public: + SettingsProfileElements() {} + + /// The constructor from AST requires the AccessControlManager if `ast.id_mode == false`. + SettingsProfileElements(const ASTSettingsProfileElements & ast); + SettingsProfileElements(const ASTSettingsProfileElements & ast, const AccessControlManager & manager); + std::shared_ptr toAST() const; + std::shared_ptr toASTWithNames(const AccessControlManager & manager) const; + void merge(const SettingsProfileElements & other); Settings toSettings() const; diff --git a/dbms/src/Interpreters/InterpreterCreateRoleQuery.cpp b/dbms/src/Interpreters/InterpreterCreateRoleQuery.cpp index f1c58f9d9bd..f64462d443b 100644 --- a/dbms/src/Interpreters/InterpreterCreateRoleQuery.cpp +++ b/dbms/src/Interpreters/InterpreterCreateRoleQuery.cpp @@ -7,21 +7,53 @@ namespace DB { +namespace +{ + void updateRoleFromQueryImpl( + Role & role, + const ASTCreateRoleQuery & query, + const std::optional & settings_from_query = {}) + { + if (query.alter) + { + if (!query.new_name.empty()) + role.setName(query.new_name); + } + else + role.setName(query.name); + + const SettingsProfileElements * settings = nullptr; + std::optional temp_settings; + if (settings_from_query) + settings = &*settings_from_query; + else if (query.settings) + settings = &temp_settings.emplace(*query.settings); + + if (settings) + role.settings = *settings; + } +} + + BlockIO InterpreterCreateRoleQuery::execute() { const auto & query = query_ptr->as(); auto & access_control = context.getAccessControlManager(); if (query.alter) - context.checkAccess(AccessType::CREATE_ROLE | AccessType::DROP_ROLE); + context.checkAccess(AccessType::ALTER_ROLE); else context.checkAccess(AccessType::CREATE_ROLE); + std::optional settings_from_query; + if (query.settings) + settings_from_query = SettingsProfileElements{*query.settings, access_control}; + if (query.alter) { auto update_func = [&](const AccessEntityPtr & entity) -> AccessEntityPtr { auto updated_role = typeid_cast>(entity->clone()); - updateRoleFromQuery(*updated_role, query); + updateRoleFromQueryImpl(*updated_role, query, settings_from_query); return updated_role; }; if (query.if_exists) @@ -35,7 +67,7 @@ BlockIO InterpreterCreateRoleQuery::execute() else { auto new_role = std::make_shared(); - updateRoleFromQuery(*new_role, query); + updateRoleFromQueryImpl(*new_role, query, settings_from_query); if (query.if_not_exists) access_control.tryInsert(new_role); @@ -51,12 +83,6 @@ BlockIO InterpreterCreateRoleQuery::execute() void InterpreterCreateRoleQuery::updateRoleFromQuery(Role & role, const ASTCreateRoleQuery & query) { - if (query.alter) - { - if (!query.new_name.empty()) - role.setName(query.new_name); - } - else - role.setName(query.name); + updateRoleFromQueryImpl(role, query); } } diff --git a/dbms/src/Interpreters/InterpreterCreateSettingsProfileQuery.cpp b/dbms/src/Interpreters/InterpreterCreateSettingsProfileQuery.cpp new file mode 100644 index 00000000000..9d110a69516 --- /dev/null +++ b/dbms/src/Interpreters/InterpreterCreateSettingsProfileQuery.cpp @@ -0,0 +1,104 @@ +#include +#include +#include +#include +#include +#include + + +namespace DB +{ +namespace +{ + void updateSettingsProfileFromQueryImpl( + SettingsProfile & profile, + const ASTCreateSettingsProfileQuery & query, + const std::optional & settings_from_query = {}, + const std::optional & roles_from_query = {}) + { + if (query.alter) + { + if (!query.new_name.empty()) + profile.setName(query.new_name); + } + else + profile.setName(query.name); + + const SettingsProfileElements * settings = nullptr; + std::optional temp_settings; + if (settings_from_query) + settings = &*settings_from_query; + else if (query.settings) + settings = &temp_settings.emplace(*query.settings); + + if (settings) + profile.elements = *settings; + + const ExtendedRoleSet * roles = nullptr; + std::optional temp_role_set; + if (roles_from_query) + roles = &*roles_from_query; + else if (query.to_roles) + roles = &temp_role_set.emplace(*query.to_roles); + + if (roles) + profile.to_roles = *roles; + } +} + + +BlockIO InterpreterCreateSettingsProfileQuery::execute() +{ + const auto & query = query_ptr->as(); + auto & access_control = context.getAccessControlManager(); + if (query.alter) + context.checkAccess(AccessType::ALTER_SETTINGS_PROFILE); + else + context.checkAccess(AccessType::CREATE_SETTINGS_PROFILE); + + std::optional settings_from_query; + if (query.settings) + settings_from_query = SettingsProfileElements{*query.settings, access_control}; + + std::optional roles_from_query; + if (query.to_roles) + roles_from_query = ExtendedRoleSet{*query.to_roles, access_control, context.getUserID()}; + + if (query.alter) + { + auto update_func = [&](const AccessEntityPtr & entity) -> AccessEntityPtr + { + auto updated_profile = typeid_cast>(entity->clone()); + updateSettingsProfileFromQueryImpl(*updated_profile, query, settings_from_query, roles_from_query); + return updated_profile; + }; + if (query.if_exists) + { + if (auto id = access_control.find(query.name)) + access_control.tryUpdate(*id, update_func); + } + else + access_control.update(access_control.getID(query.name), update_func); + } + else + { + auto new_profile = std::make_shared(); + updateSettingsProfileFromQueryImpl(*new_profile, query, settings_from_query, roles_from_query); + + if (query.if_not_exists) + access_control.tryInsert(new_profile); + else if (query.or_replace) + access_control.insertOrReplace(new_profile); + else + access_control.insert(new_profile); + } + + return {}; +} + + +void InterpreterCreateSettingsProfileQuery::updateSettingsProfileFromQuery(SettingsProfile & SettingsProfile, const ASTCreateSettingsProfileQuery & query) +{ + updateSettingsProfileFromQueryImpl(SettingsProfile, query); +} +} diff --git a/dbms/src/Interpreters/InterpreterCreateSettingsProfileQuery.h b/dbms/src/Interpreters/InterpreterCreateSettingsProfileQuery.h new file mode 100644 index 00000000000..fd420779cf4 --- /dev/null +++ b/dbms/src/Interpreters/InterpreterCreateSettingsProfileQuery.h @@ -0,0 +1,26 @@ +#pragma once + +#include +#include + + +namespace DB +{ +class ASTCreateSettingsProfileQuery; +struct SettingsProfile; + + +class InterpreterCreateSettingsProfileQuery : public IInterpreter +{ +public: + InterpreterCreateSettingsProfileQuery(const ASTPtr & query_ptr_, Context & context_) : query_ptr(query_ptr_), context(context_) {} + + BlockIO execute() override; + + static void updateSettingsProfileFromQuery(SettingsProfile & profile, const ASTCreateSettingsProfileQuery & query); + +private: + ASTPtr query_ptr; + Context & context; +}; +} diff --git a/dbms/src/Interpreters/InterpreterCreateUserQuery.cpp b/dbms/src/Interpreters/InterpreterCreateUserQuery.cpp index 2d5b0687691..5dba1fefc9c 100644 --- a/dbms/src/Interpreters/InterpreterCreateUserQuery.cpp +++ b/dbms/src/Interpreters/InterpreterCreateUserQuery.cpp @@ -13,7 +13,11 @@ namespace DB { namespace { - void updateUserFromQueryImpl(User & user, const ASTCreateUserQuery & query, const std::optional & default_roles_from_query = {}) + void updateUserFromQueryImpl( + User & user, + const ASTCreateUserQuery & query, + const std::optional & default_roles_from_query = {}, + const std::optional & settings_from_query = {}) { if (query.alter) { @@ -47,6 +51,16 @@ namespace InterpreterSetRoleQuery::updateUserSetDefaultRoles(user, *default_roles); } + + const SettingsProfileElements * settings = nullptr; + std::optional temp_settings; + if (settings_from_query) + settings = &*settings_from_query; + else if (query.settings) + settings = &temp_settings.emplace(*query.settings); + + if (settings) + user.settings = *settings; } } @@ -69,12 +83,16 @@ BlockIO InterpreterCreateUserQuery::execute() } } + std::optional settings_from_query; + if (query.settings) + settings_from_query = SettingsProfileElements{*query.settings, access_control}; + if (query.alter) { auto update_func = [&](const AccessEntityPtr & entity) -> AccessEntityPtr { auto updated_user = typeid_cast>(entity->clone()); - updateUserFromQueryImpl(*updated_user, query, default_roles_from_query); + updateUserFromQueryImpl(*updated_user, query, default_roles_from_query, settings_from_query); return updated_user; }; if (query.if_exists) @@ -88,7 +106,7 @@ BlockIO InterpreterCreateUserQuery::execute() else { auto new_user = std::make_shared(); - updateUserFromQueryImpl(*new_user, query, default_roles_from_query); + updateUserFromQueryImpl(*new_user, query, default_roles_from_query, settings_from_query); if (query.if_not_exists) access_control.tryInsert(new_user); diff --git a/dbms/src/Interpreters/InterpreterDropAccessEntityQuery.cpp b/dbms/src/Interpreters/InterpreterDropAccessEntityQuery.cpp index c69ce3ade45..12f33250188 100644 --- a/dbms/src/Interpreters/InterpreterDropAccessEntityQuery.cpp +++ b/dbms/src/Interpreters/InterpreterDropAccessEntityQuery.cpp @@ -7,64 +7,69 @@ #include #include #include +#include #include namespace DB { +namespace +{ + using Kind = ASTDropAccessEntityQuery::Kind; + + std::type_index getType(Kind kind) + { + switch (kind) + { + case Kind::USER: return typeid(User); + case Kind::ROLE: return typeid(Role); + case Kind::QUOTA: return typeid(Quota); + case Kind::ROW_POLICY: return typeid(RowPolicy); + case Kind::SETTINGS_PROFILE: return typeid(SettingsProfile); + } + __builtin_unreachable(); + } + + AccessType getRequiredAccessType(Kind kind) + { + switch (kind) + { + case Kind::USER: return AccessType::DROP_USER; + case Kind::ROLE: return AccessType::DROP_ROLE; + case Kind::QUOTA: return AccessType::DROP_QUOTA; + case Kind::ROW_POLICY: return AccessType::DROP_POLICY; + case Kind::SETTINGS_PROFILE: return AccessType::DROP_SETTINGS_PROFILE; + } + __builtin_unreachable(); + } +} + BlockIO InterpreterDropAccessEntityQuery::execute() { const auto & query = query_ptr->as(); auto & access_control = context.getAccessControlManager(); - using Kind = ASTDropAccessEntityQuery::Kind; - switch (query.kind) + std::type_index type = getType(query.kind); + context.checkAccess(getRequiredAccessType(query.kind)); + + if (query.kind == Kind::ROW_POLICY) { - case Kind::USER: - { - context.checkAccess(AccessType::DROP_USER); - if (query.if_exists) - access_control.tryRemove(access_control.find(query.names)); - else - access_control.remove(access_control.getIDs(query.names)); - return {}; - } - - case Kind::ROLE: - { - context.checkAccess(AccessType::DROP_ROLE); - if (query.if_exists) - access_control.tryRemove(access_control.find(query.names)); - else - access_control.remove(access_control.getIDs(query.names)); - return {}; - } - - case Kind::QUOTA: - { - context.checkAccess(AccessType::DROP_QUOTA); - if (query.if_exists) - access_control.tryRemove(access_control.find(query.names)); - else - access_control.remove(access_control.getIDs(query.names)); - return {}; - } - - case Kind::ROW_POLICY: - { - context.checkAccess(AccessType::DROP_POLICY); - Strings full_names; - boost::range::transform( - query.row_policies_names, std::back_inserter(full_names), - [this](const RowPolicy::FullNameParts & row_policy_name) { return row_policy_name.getFullName(context); }); - if (query.if_exists) - access_control.tryRemove(access_control.find(full_names)); - else - access_control.remove(access_control.getIDs(full_names)); - return {}; - } + Strings full_names; + boost::range::transform( + query.row_policies_names, std::back_inserter(full_names), + [this](const RowPolicy::FullNameParts & row_policy_name) { return row_policy_name.getFullName(context); }); + if (query.if_exists) + access_control.tryRemove(access_control.find(full_names)); + else + access_control.remove(access_control.getIDs(full_names)); + return {}; } - __builtin_unreachable(); + if (query.if_exists) + access_control.tryRemove(access_control.find(type, query.names)); + else + access_control.remove(access_control.getIDs(type, query.names)); + return {}; } + } diff --git a/dbms/src/Interpreters/InterpreterFactory.cpp b/dbms/src/Interpreters/InterpreterFactory.cpp index b3b1fd498db..0c34d6ed79f 100644 --- a/dbms/src/Interpreters/InterpreterFactory.cpp +++ b/dbms/src/Interpreters/InterpreterFactory.cpp @@ -5,6 +5,7 @@ #include #include #include +#include #include #include #include @@ -34,6 +35,7 @@ #include #include #include +#include #include #include #include @@ -204,6 +206,10 @@ std::unique_ptr InterpreterFactory::get(ASTPtr & query, Context & { return std::make_unique(query, context); } + else if (query->as()) + { + return std::make_unique(query, context); + } else if (query->as()) { return std::make_unique(query, context); diff --git a/dbms/src/Interpreters/InterpreterShowCreateAccessEntityQuery.cpp b/dbms/src/Interpreters/InterpreterShowCreateAccessEntityQuery.cpp index dce1bbc7a87..52126b0507e 100644 --- a/dbms/src/Interpreters/InterpreterShowCreateAccessEntityQuery.cpp +++ b/dbms/src/Interpreters/InterpreterShowCreateAccessEntityQuery.cpp @@ -4,6 +4,7 @@ #include #include #include +#include #include #include #include @@ -14,6 +15,7 @@ #include #include #include +#include #include #include #include @@ -46,9 +48,9 @@ namespace if (user.default_roles != ExtendedRoleSet::AllTag{}) { if (attach_mode) - query->default_roles = ExtendedRoleSet{user.default_roles}.toAST(); + query->default_roles = user.default_roles.toAST(); else - query->default_roles = ExtendedRoleSet{user.default_roles}.toASTWithNames(*manager); + query->default_roles = user.default_roles.toASTWithNames(*manager); } if (attach_mode && (user.authentication.getType() != Authentication::NO_PASSWORD)) @@ -56,15 +58,59 @@ namespace /// We don't show password unless it's an ATTACH statement. query->authentication = user.authentication; } + + if (!user.settings.empty()) + { + if (attach_mode) + query->settings = user.settings.toAST(); + else + query->settings = user.settings.toASTWithNames(*manager); + } + return query; } - ASTPtr getCreateQueryImpl(const Role & role, const AccessControlManager *, bool attach_mode = false) + ASTPtr getCreateQueryImpl(const Role & role, const AccessControlManager * manager, bool attach_mode = false) { auto query = std::make_shared(); query->name = role.getName(); query->attach = attach_mode; + + if (!role.settings.empty()) + { + if (attach_mode) + query->settings = role.settings.toAST(); + else + query->settings = role.settings.toASTWithNames(*manager); + } + + return query; + } + + + ASTPtr getCreateQueryImpl(const SettingsProfile & profile, const AccessControlManager * manager, bool attach_mode = false) + { + auto query = std::make_shared(); + query->name = profile.getName(); + query->attach = attach_mode; + + if (!profile.elements.empty()) + { + if (attach_mode) + query->settings = profile.elements.toAST(); + else + query->settings = profile.elements.toASTWithNames(*manager); + } + + if (!profile.to_roles.empty()) + { + if (attach_mode) + query->to_roles = profile.to_roles.toAST(); + else + query->to_roles = profile.to_roles.toASTWithNames(*manager); + } + return query; } @@ -151,8 +197,25 @@ namespace return getCreateQueryImpl(*policy, manager, attach_mode); if (const Quota * quota = typeid_cast(&entity)) return getCreateQueryImpl(*quota, manager, attach_mode); + if (const SettingsProfile * profile = typeid_cast(&entity)) + return getCreateQueryImpl(*profile, manager, attach_mode); throw Exception("Unexpected type of access entity: " + entity.getTypeName(), ErrorCodes::LOGICAL_ERROR); } + + using Kind = ASTShowCreateAccessEntityQuery::Kind; + + std::type_index getType(Kind kind) + { + switch (kind) + { + case Kind::USER: return typeid(User); + case Kind::ROLE: return typeid(Role); + case Kind::QUOTA: return typeid(Quota); + case Kind::ROW_POLICY: return typeid(RowPolicy); + case Kind::SETTINGS_PROFILE: return typeid(SettingsProfile); + } + __builtin_unreachable(); + } } @@ -193,36 +256,28 @@ BlockInputStreamPtr InterpreterShowCreateAccessEntityQuery::executeImpl() ASTPtr InterpreterShowCreateAccessEntityQuery::getCreateQuery(const ASTShowCreateAccessEntityQuery & show_query) const { const auto & access_control = context.getAccessControlManager(); - using Kind = ASTShowCreateAccessEntityQuery::Kind; - switch (show_query.kind) + + if (show_query.current_user) { - case Kind::USER: - { - UserPtr user; - if (show_query.current_user) - user = context.getUser(); - else - user = access_control.read(show_query.name); - return getCreateQueryImpl(*user, &access_control); - } - - case Kind::QUOTA: - { - QuotaPtr quota; - if (show_query.current_quota) - quota = access_control.read(context.getQuota()->getUsageInfo().quota_id); - else - quota = access_control.read(show_query.name); - return getCreateQueryImpl(*quota, &access_control); - } - - case Kind::ROW_POLICY: - { - RowPolicyPtr policy = access_control.read(show_query.row_policy_name.getFullName(context)); - return getCreateQueryImpl(*policy, &access_control); - } + auto user = context.getUser(); + return getCreateQueryImpl(*user, &access_control); } - __builtin_unreachable(); + + if (show_query.current_quota) + { + auto quota = access_control.read(context.getQuota()->getUsageInfo().quota_id); + return getCreateQueryImpl(*quota, &access_control); + } + + auto type = getType(show_query.kind); + if (show_query.kind == Kind::ROW_POLICY) + { + RowPolicyPtr policy = access_control.read(show_query.row_policy_name.getFullName(context)); + return getCreateQueryImpl(*policy, &access_control); + } + + auto entity = access_control.read(access_control.getID(type, show_query.name)); + return getCreateQueryImpl(*entity, &access_control); } diff --git a/dbms/src/Parsers/ASTCreateQuotaQuery.cpp b/dbms/src/Parsers/ASTCreateQuotaQuery.cpp index bb4a32f0556..7613fce6167 100644 --- a/dbms/src/Parsers/ASTCreateQuotaQuery.cpp +++ b/dbms/src/Parsers/ASTCreateQuotaQuery.cpp @@ -143,7 +143,7 @@ void ASTCreateQuotaQuery::formatImpl(const FormatSettings & settings, FormatStat formatAllLimits(all_limits, settings); - if (roles) + if (roles && (!roles->empty() || alter)) formatToRoles(*roles, settings); } } diff --git a/dbms/src/Parsers/ASTCreateRoleQuery.cpp b/dbms/src/Parsers/ASTCreateRoleQuery.cpp index b511a466d2f..3d69e4dac59 100644 --- a/dbms/src/Parsers/ASTCreateRoleQuery.cpp +++ b/dbms/src/Parsers/ASTCreateRoleQuery.cpp @@ -1,4 +1,5 @@ #include +#include #include @@ -11,6 +12,12 @@ namespace settings.ostr << (settings.hilite ? IAST::hilite_keyword : "") << " RENAME TO " << (settings.hilite ? IAST::hilite_none : "") << quoteString(new_name); } + + void formatSettings(const ASTSettingsProfileElements & settings, const IAST::FormatSettings & format) + { + format.ostr << (format.hilite ? IAST::hilite_keyword : "") << " SETTINGS " << (format.hilite ? IAST::hilite_none : ""); + settings.format(format); + } } @@ -26,28 +33,32 @@ ASTPtr ASTCreateRoleQuery::clone() const } -void ASTCreateRoleQuery::formatImpl(const FormatSettings & settings, FormatState &, FormatStateStacked) const +void ASTCreateRoleQuery::formatImpl(const FormatSettings & format, FormatState &, FormatStateStacked) const { if (attach) { - settings.ostr << (settings.hilite ? hilite_keyword : "") << "ATTACH ROLE" << (settings.hilite ? hilite_none : ""); + format.ostr << (format.hilite ? hilite_keyword : "") << "ATTACH ROLE" << (format.hilite ? hilite_none : ""); } else { - settings.ostr << (settings.hilite ? hilite_keyword : "") << (alter ? "ALTER ROLE" : "CREATE ROLE") - << (settings.hilite ? hilite_none : ""); + format.ostr << (format.hilite ? hilite_keyword : "") << (alter ? "ALTER ROLE" : "CREATE ROLE") + << (format.hilite ? hilite_none : ""); } if (if_exists) - settings.ostr << (settings.hilite ? hilite_keyword : "") << " IF EXISTS" << (settings.hilite ? hilite_none : ""); + format.ostr << (format.hilite ? hilite_keyword : "") << " IF EXISTS" << (format.hilite ? hilite_none : ""); else if (if_not_exists) - settings.ostr << (settings.hilite ? hilite_keyword : "") << " IF NOT EXISTS" << (settings.hilite ? hilite_none : ""); + format.ostr << (format.hilite ? hilite_keyword : "") << " IF NOT EXISTS" << (format.hilite ? hilite_none : ""); else if (or_replace) - settings.ostr << (settings.hilite ? hilite_keyword : "") << " OR REPLACE" << (settings.hilite ? hilite_none : ""); + format.ostr << (format.hilite ? hilite_keyword : "") << " OR REPLACE" << (format.hilite ? hilite_none : ""); - settings.ostr << " " << backQuoteIfNeed(name); + format.ostr << " " << backQuoteIfNeed(name); if (!new_name.empty()) - formatRenameTo(new_name, settings); + formatRenameTo(new_name, format); + + if (settings && (!settings->empty() || alter)) + formatSettings(*settings, format); } + } diff --git a/dbms/src/Parsers/ASTCreateRoleQuery.h b/dbms/src/Parsers/ASTCreateRoleQuery.h index 5109492fc9e..69bb9896fa3 100644 --- a/dbms/src/Parsers/ASTCreateRoleQuery.h +++ b/dbms/src/Parsers/ASTCreateRoleQuery.h @@ -5,10 +5,15 @@ namespace DB { +class ASTSettingsProfileElements; + + /** CREATE ROLE [IF NOT EXISTS | OR REPLACE] name + * [SETTINGS variable [= value] [MIN [=] min_value] [MAX [=] max_value] [READONLY|WRITABLE] | PROFILE 'profile_name'] [,...] * * ALTER ROLE [IF EXISTS] name - * [RENAME TO new_name] + * [RENAME TO new_name] + * [SETTINGS variable [= value] [MIN [=] min_value] [MAX [=] max_value] [READONLY|WRITABLE] | PROFILE 'profile_name'] [,...] */ class ASTCreateRoleQuery : public IAST { @@ -23,8 +28,10 @@ public: String name; String new_name; + std::shared_ptr settings; + String getID(char) const override; ASTPtr clone() const override; - void formatImpl(const FormatSettings & settings, FormatState &, FormatStateStacked) const override; + void formatImpl(const FormatSettings & format, FormatState &, FormatStateStacked) const override; }; } diff --git a/dbms/src/Parsers/ASTCreateRowPolicyQuery.cpp b/dbms/src/Parsers/ASTCreateRowPolicyQuery.cpp index 8c7b3aa6e86..ac3d859e66f 100644 --- a/dbms/src/Parsers/ASTCreateRowPolicyQuery.cpp +++ b/dbms/src/Parsers/ASTCreateRowPolicyQuery.cpp @@ -165,7 +165,7 @@ void ASTCreateRowPolicyQuery::formatImpl(const FormatSettings & settings, Format formatMultipleConditions(conditions, alter, settings); - if (roles) + if (roles && (!roles->empty() || alter)) formatToRoles(*roles, settings); } } diff --git a/dbms/src/Parsers/ASTCreateSettingsProfileQuery.cpp b/dbms/src/Parsers/ASTCreateSettingsProfileQuery.cpp new file mode 100644 index 00000000000..a5a5556baf3 --- /dev/null +++ b/dbms/src/Parsers/ASTCreateSettingsProfileQuery.cpp @@ -0,0 +1,74 @@ +#include +#include +#include +#include + + +namespace DB +{ +namespace +{ + void formatRenameTo(const String & new_name, const IAST::FormatSettings & settings) + { + settings.ostr << (settings.hilite ? IAST::hilite_keyword : "") << " RENAME TO " << (settings.hilite ? IAST::hilite_none : "") + << quoteString(new_name); + } + + void formatSettings(const ASTSettingsProfileElements & settings, const IAST::FormatSettings & format) + { + format.ostr << (format.hilite ? IAST::hilite_keyword : "") << " SETTINGS " << (format.hilite ? IAST::hilite_none : ""); + settings.format(format); + } + + void formatToRoles(const ASTExtendedRoleSet & roles, const IAST::FormatSettings & settings) + { + settings.ostr << (settings.hilite ? IAST::hilite_keyword : "") << " TO " << (settings.hilite ? IAST::hilite_none : ""); + roles.format(settings); + } +} + + +String ASTCreateSettingsProfileQuery::getID(char) const +{ + return "CreateSettingsProfileQuery"; +} + + +ASTPtr ASTCreateSettingsProfileQuery::clone() const +{ + return std::make_shared(*this); +} + + +void ASTCreateSettingsProfileQuery::formatImpl(const FormatSettings & format, FormatState &, FormatStateStacked) const +{ + if (attach) + { + format.ostr << (format.hilite ? hilite_keyword : "") << "ATTACH SETTINGS PROFILE" << (format.hilite ? hilite_none : ""); + } + else + { + format.ostr << (format.hilite ? hilite_keyword : "") << (alter ? "ALTER SETTINGS PROFILE" : "CREATE SETTINGS PROFILE") + << (format.hilite ? hilite_none : ""); + } + + if (if_exists) + format.ostr << (format.hilite ? hilite_keyword : "") << " IF EXISTS" << (format.hilite ? hilite_none : ""); + else if (if_not_exists) + format.ostr << (format.hilite ? hilite_keyword : "") << " IF NOT EXISTS" << (format.hilite ? hilite_none : ""); + else if (or_replace) + format.ostr << (format.hilite ? hilite_keyword : "") << " OR REPLACE" << (format.hilite ? hilite_none : ""); + + format.ostr << " " << backQuoteIfNeed(name); + + if (!new_name.empty()) + formatRenameTo(new_name, format); + + if (settings && (!settings->empty() || alter)) + formatSettings(*settings, format); + + if (to_roles && (!to_roles->empty() || alter)) + formatToRoles(*to_roles, format); +} + +} diff --git a/dbms/src/Parsers/ASTCreateSettingsProfileQuery.h b/dbms/src/Parsers/ASTCreateSettingsProfileQuery.h new file mode 100644 index 00000000000..b3a60853e57 --- /dev/null +++ b/dbms/src/Parsers/ASTCreateSettingsProfileQuery.h @@ -0,0 +1,40 @@ +#pragma once + +#include + + +namespace DB +{ +class ASTSettingsProfileElements; +class ASTExtendedRoleSet; + + +/** CREATE SETTINGS PROFILE [IF NOT EXISTS | OR REPLACE] name + * [SETTINGS variable [= value] [MIN [=] min_value] [MAX [=] max_value] [READONLY|WRITABLE] | PROFILE 'profile_name'] [,...] + * + * ALTER SETTINGS PROFILE [IF EXISTS] name + * [RENAME TO new_name] + * [SETTINGS variable [= value] [MIN [=] min_value] [MAX [=] max_value] [READONLY|WRITABLE] | PROFILE 'profile_name'] [,...] + */ +class ASTCreateSettingsProfileQuery : public IAST +{ +public: + bool alter = false; + bool attach = false; + + bool if_exists = false; + bool if_not_exists = false; + bool or_replace = false; + + String name; + String new_name; + + std::shared_ptr settings; + + std::shared_ptr to_roles; + + String getID(char) const override; + ASTPtr clone() const override; + void formatImpl(const FormatSettings & format, FormatState &, FormatStateStacked) const override; +}; +} diff --git a/dbms/src/Parsers/ASTCreateUserQuery.cpp b/dbms/src/Parsers/ASTCreateUserQuery.cpp index 94b2a35a314..0631d08ae74 100644 --- a/dbms/src/Parsers/ASTCreateUserQuery.cpp +++ b/dbms/src/Parsers/ASTCreateUserQuery.cpp @@ -1,5 +1,6 @@ #include #include +#include #include @@ -142,10 +143,10 @@ namespace } - void formatProfile(const String & profile_name, const IAST::FormatSettings & settings) + void formatSettings(const ASTSettingsProfileElements & settings, const IAST::FormatSettings & format) { - settings.ostr << (settings.hilite ? IAST::hilite_keyword : "") << " PROFILE " << (settings.hilite ? IAST::hilite_none : "") - << quoteString(profile_name); + format.ostr << (format.hilite ? IAST::hilite_keyword : "") << " SETTINGS " << (format.hilite ? IAST::hilite_none : ""); + settings.format(format); } } @@ -162,44 +163,44 @@ ASTPtr ASTCreateUserQuery::clone() const } -void ASTCreateUserQuery::formatImpl(const FormatSettings & settings, FormatState &, FormatStateStacked) const +void ASTCreateUserQuery::formatImpl(const FormatSettings & format, FormatState &, FormatStateStacked) const { if (attach) { - settings.ostr << (settings.hilite ? hilite_keyword : "") << "ATTACH USER" << (settings.hilite ? hilite_none : ""); + format.ostr << (format.hilite ? hilite_keyword : "") << "ATTACH USER" << (format.hilite ? hilite_none : ""); } else { - settings.ostr << (settings.hilite ? hilite_keyword : "") << (alter ? "ALTER USER" : "CREATE USER") - << (settings.hilite ? hilite_none : ""); + format.ostr << (format.hilite ? hilite_keyword : "") << (alter ? "ALTER USER" : "CREATE USER") + << (format.hilite ? hilite_none : ""); } if (if_exists) - settings.ostr << (settings.hilite ? hilite_keyword : "") << " IF EXISTS" << (settings.hilite ? hilite_none : ""); + format.ostr << (format.hilite ? hilite_keyword : "") << " IF EXISTS" << (format.hilite ? hilite_none : ""); else if (if_not_exists) - settings.ostr << (settings.hilite ? hilite_keyword : "") << " IF NOT EXISTS" << (settings.hilite ? hilite_none : ""); + format.ostr << (format.hilite ? hilite_keyword : "") << " IF NOT EXISTS" << (format.hilite ? hilite_none : ""); else if (or_replace) - settings.ostr << (settings.hilite ? hilite_keyword : "") << " OR REPLACE" << (settings.hilite ? hilite_none : ""); + format.ostr << (format.hilite ? hilite_keyword : "") << " OR REPLACE" << (format.hilite ? hilite_none : ""); - settings.ostr << " " << backQuoteIfNeed(name); + format.ostr << " " << backQuoteIfNeed(name); if (!new_name.empty()) - formatRenameTo(new_name, settings); + formatRenameTo(new_name, format); if (authentication) - formatAuthentication(*authentication, settings); + formatAuthentication(*authentication, format); if (hosts) - formatHosts(nullptr, *hosts, settings); + formatHosts(nullptr, *hosts, format); if (add_hosts) - formatHosts("ADD", *add_hosts, settings); + formatHosts("ADD", *add_hosts, format); if (remove_hosts) - formatHosts("REMOVE", *remove_hosts, settings); + formatHosts("REMOVE", *remove_hosts, format); if (default_roles) - formatDefaultRoles(*default_roles, settings); + formatDefaultRoles(*default_roles, format); - if (profile) - formatProfile(*profile, settings); + if (settings && (!settings->empty() || alter)) + formatSettings(*settings, format); } } diff --git a/dbms/src/Parsers/ASTCreateUserQuery.h b/dbms/src/Parsers/ASTCreateUserQuery.h index 0357fb60720..fc2aa0121ed 100644 --- a/dbms/src/Parsers/ASTCreateUserQuery.h +++ b/dbms/src/Parsers/ASTCreateUserQuery.h @@ -8,19 +8,20 @@ namespace DB { class ASTExtendedRoleSet; +class ASTSettingsProfileElements; /** CREATE USER [IF NOT EXISTS | OR REPLACE] name - * [IDENTIFIED [WITH {NO_PASSWORD|PLAINTEXT_PASSWORD|SHA256_PASSWORD|SHA256_HASH|DOUBLE_SHA1_PASSWORD|DOUBLE_SHA1_HASH}] BY {'password'|'hash'}] - * [HOST {LOCAL | NAME 'name' | NAME REGEXP 'name_regexp' | IP 'address' | LIKE 'pattern'} [,...] | ANY | NONE] - * [DEFAULT ROLE role [,...]] - * [PROFILE 'profile_name'] + * [IDENTIFIED [WITH {NO_PASSWORD|PLAINTEXT_PASSWORD|SHA256_PASSWORD|SHA256_HASH|DOUBLE_SHA1_PASSWORD|DOUBLE_SHA1_HASH}] BY {'password'|'hash'}] + * [HOST {LOCAL | NAME 'name' | NAME REGEXP 'name_regexp' | IP 'address' | LIKE 'pattern'} [,...] | ANY | NONE] + * [DEFAULT ROLE role [,...]] + * [SETTINGS variable [= value] [MIN [=] min_value] [MAX [=] max_value] [READONLY|WRITABLE] | PROFILE 'profile_name'] [,...] * * ALTER USER [IF EXISTS] name * [RENAME TO new_name] * [IDENTIFIED [WITH {PLAINTEXT_PASSWORD|SHA256_PASSWORD|DOUBLE_SHA1_PASSWORD}] BY {'password'|'hash'}] * [[ADD|REMOVE] HOST {LOCAL | NAME 'name' | NAME REGEXP 'name_regexp' | IP 'address' | LIKE 'pattern'} [,...] | ANY | NONE] * [DEFAULT ROLE role [,...] | ALL | ALL EXCEPT role [,...] ] - * [PROFILE 'profile_name'] + * [SETTINGS variable [= value] [MIN [=] min_value] [MAX [=] max_value] [READONLY|WRITABLE] | PROFILE 'profile_name'] [,...] */ class ASTCreateUserQuery : public IAST { @@ -43,10 +44,10 @@ public: std::shared_ptr default_roles; - std::optional profile; + std::shared_ptr settings; String getID(char) const override; ASTPtr clone() const override; - void formatImpl(const FormatSettings & settings, FormatState &, FormatStateStacked) const override; + void formatImpl(const FormatSettings & format, FormatState &, FormatStateStacked) const override; }; } diff --git a/dbms/src/Parsers/ASTDropAccessEntityQuery.cpp b/dbms/src/Parsers/ASTDropAccessEntityQuery.cpp index 0b6bae7575e..3896128ceb5 100644 --- a/dbms/src/Parsers/ASTDropAccessEntityQuery.cpp +++ b/dbms/src/Parsers/ASTDropAccessEntityQuery.cpp @@ -8,14 +8,15 @@ namespace { using Kind = ASTDropAccessEntityQuery::Kind; - const char * kindToKeyword(Kind kind) + const char * getKeyword(Kind kind) { switch (kind) { case Kind::USER: return "USER"; case Kind::ROLE: return "ROLE"; case Kind::QUOTA: return "QUOTA"; - case Kind::ROW_POLICY: return "POLICY"; + case Kind::ROW_POLICY: return "ROW POLICY"; + case Kind::SETTINGS_PROFILE: return "SETTINGS PROFILE"; } __builtin_unreachable(); } @@ -23,14 +24,14 @@ namespace ASTDropAccessEntityQuery::ASTDropAccessEntityQuery(Kind kind_) - : kind(kind_), keyword(kindToKeyword(kind_)) + : kind(kind_) { } String ASTDropAccessEntityQuery::getID(char) const { - return String("DROP ") + keyword + " query"; + return String("DROP ") + getKeyword(kind) + " query"; } @@ -43,7 +44,7 @@ ASTPtr ASTDropAccessEntityQuery::clone() const void ASTDropAccessEntityQuery::formatImpl(const FormatSettings & settings, FormatState &, FormatStateStacked) const { settings.ostr << (settings.hilite ? hilite_keyword : "") - << "DROP " << keyword + << "DROP " << getKeyword(kind) << (if_exists ? " IF EXISTS" : "") << (settings.hilite ? hilite_none : ""); diff --git a/dbms/src/Parsers/ASTDropAccessEntityQuery.h b/dbms/src/Parsers/ASTDropAccessEntityQuery.h index eea40fd5343..5f0b46bd896 100644 --- a/dbms/src/Parsers/ASTDropAccessEntityQuery.h +++ b/dbms/src/Parsers/ASTDropAccessEntityQuery.h @@ -11,6 +11,7 @@ namespace DB * DROP ROLE [IF EXISTS] name [,...] * DROP QUOTA [IF EXISTS] name [,...] * DROP [ROW] POLICY [IF EXISTS] name [,...] ON [database.]table [,...] + * DROP [SETTINGS] PROFILE [IF EXISTS] name [,...] */ class ASTDropAccessEntityQuery : public IAST { @@ -21,11 +22,10 @@ public: ROLE, QUOTA, ROW_POLICY, + SETTINGS_PROFILE, }; const Kind kind; - const char * const keyword; - bool if_exists = false; Strings names; std::vector row_policies_names; diff --git a/dbms/src/Parsers/ASTExtendedRoleSet.h b/dbms/src/Parsers/ASTExtendedRoleSet.h index 17edbceed30..84190211087 100644 --- a/dbms/src/Parsers/ASTExtendedRoleSet.h +++ b/dbms/src/Parsers/ASTExtendedRoleSet.h @@ -1,7 +1,6 @@ #pragma once #include -#include namespace DB diff --git a/dbms/src/Parsers/ASTSettingsProfileElement.cpp b/dbms/src/Parsers/ASTSettingsProfileElement.cpp new file mode 100644 index 00000000000..b3f4032d14c --- /dev/null +++ b/dbms/src/Parsers/ASTSettingsProfileElement.cpp @@ -0,0 +1,88 @@ +#include +#include +#include + + +namespace DB +{ +namespace +{ + void formatProfileNameOrID(const String & str, bool is_id, const IAST::FormatSettings & settings) + { + if (is_id) + { + settings.ostr << (settings.hilite ? IAST::hilite_keyword : "") << "ID" << (settings.hilite ? IAST::hilite_none : "") << "(" + << quoteString(str) << ")"; + } + else + { + settings.ostr << backQuoteIfNeed(str); + } + } +} + +void ASTSettingsProfileElement::formatImpl(const FormatSettings & settings, FormatState &, FormatStateStacked) const +{ + if (!parent_profile.empty()) + { + settings.ostr << (settings.hilite ? IAST::hilite_keyword : "") << "PROFILE " << (settings.hilite ? IAST::hilite_none : ""); + formatProfileNameOrID(parent_profile, id_mode, settings); + return; + } + + settings.ostr << name; + + if (!value.isNull()) + { + settings.ostr << " = " << applyVisitor(FieldVisitorToString{}, value); + } + + if (!min_value.isNull()) + { + settings.ostr << (settings.hilite ? IAST::hilite_keyword : "") << " MIN " << (settings.hilite ? IAST::hilite_none : "") + << applyVisitor(FieldVisitorToString{}, min_value); + } + + if (!max_value.isNull()) + { + settings.ostr << (settings.hilite ? IAST::hilite_keyword : "") << " MAX " << (settings.hilite ? IAST::hilite_none : "") + << applyVisitor(FieldVisitorToString{}, max_value); + } + + if (readonly) + { + settings.ostr << (settings.hilite ? IAST::hilite_keyword : "") << (*readonly ? " READONLY" : " WRITABLE") + << (settings.hilite ? IAST::hilite_none : ""); + } +} + + +bool ASTSettingsProfileElements::empty() const +{ + for (const auto & element : elements) + if (!element->empty()) + return false; + return true; +} + + +void ASTSettingsProfileElements::formatImpl(const FormatSettings & settings, FormatState &, FormatStateStacked) const +{ + if (empty()) + { + settings.ostr << (settings.hilite ? IAST::hilite_keyword : "") << "NONE" << (settings.hilite ? IAST::hilite_none : ""); + return; + } + + bool need_comma = false; + for (const auto & element : elements) + { + if (need_comma) + settings.ostr << ", "; + need_comma = true; + + element->format(settings); + } +} + +} diff --git a/dbms/src/Parsers/ASTSettingsProfileElement.h b/dbms/src/Parsers/ASTSettingsProfileElement.h new file mode 100644 index 00000000000..0470b51cf85 --- /dev/null +++ b/dbms/src/Parsers/ASTSettingsProfileElement.h @@ -0,0 +1,45 @@ +#pragma once + +#include +#include + + +namespace DB +{ +/** Represents a settings profile's element like the following + * {variable [= value] [MIN [=] min_value] [MAX [=] max_value] [READONLY|WRITABLE]} | PROFILE 'profile_name' + */ +class ASTSettingsProfileElement : public IAST +{ +public: + String parent_profile; + String name; + Field value; + Field min_value; + Field max_value; + std::optional readonly; + bool id_mode = false; /// If true then `parent_profile` keeps UUID, not a name. + + bool empty() const { return parent_profile.empty() && name.empty(); } + + String getID(char) const override { return "SettingsProfileElement"; } + ASTPtr clone() const override { return std::make_shared(*this); } + void formatImpl(const FormatSettings & settings, FormatState &, FormatStateStacked) const override; +}; + + +/** Represents settings profile's elements like the following + * {{variable [= value] [MIN [=] min_value] [MAX [=] max_value] [READONLY|WRITABLE]} | PROFILE 'profile_name'} [,...] + */ +class ASTSettingsProfileElements : public IAST +{ +public: + std::vector> elements; + + bool empty() const; + + String getID(char) const override { return "SettingsProfileElements"; } + ASTPtr clone() const override { return std::make_shared(*this); } + void formatImpl(const FormatSettings & settings, FormatState &, FormatStateStacked) const override; +}; +} diff --git a/dbms/src/Parsers/ASTShowCreateAccessEntityQuery.cpp b/dbms/src/Parsers/ASTShowCreateAccessEntityQuery.cpp index 4201a733f43..9e562043f09 100644 --- a/dbms/src/Parsers/ASTShowCreateAccessEntityQuery.cpp +++ b/dbms/src/Parsers/ASTShowCreateAccessEntityQuery.cpp @@ -8,13 +8,15 @@ namespace { using Kind = ASTShowCreateAccessEntityQuery::Kind; - const char * kindToKeyword(Kind kind) + const char * getKeyword(Kind kind) { switch (kind) { case Kind::USER: return "USER"; + case Kind::ROLE: return "ROLE"; case Kind::QUOTA: return "QUOTA"; - case Kind::ROW_POLICY: return "POLICY"; + case Kind::ROW_POLICY: return "ROW POLICY"; + case Kind::SETTINGS_PROFILE: return "SETTINGS PROFILE"; } __builtin_unreachable(); } @@ -22,14 +24,14 @@ namespace ASTShowCreateAccessEntityQuery::ASTShowCreateAccessEntityQuery(Kind kind_) - : kind(kind_), keyword(kindToKeyword(kind_)) + : kind(kind_) { } String ASTShowCreateAccessEntityQuery::getID(char) const { - return String("SHOW CREATE ") + keyword + " query"; + return String("SHOW CREATE ") + getKeyword(kind) + " query"; } @@ -42,13 +44,13 @@ ASTPtr ASTShowCreateAccessEntityQuery::clone() const void ASTShowCreateAccessEntityQuery::formatQueryImpl(const FormatSettings & settings, FormatState &, FormatStateStacked) const { settings.ostr << (settings.hilite ? hilite_keyword : "") - << "SHOW CREATE " << keyword + << "SHOW CREATE " << getKeyword(kind) << (settings.hilite ? hilite_none : ""); - if ((kind == Kind::USER) && current_user) + if (current_user) { } - else if ((kind == Kind::QUOTA) && current_quota) + else if (current_quota) settings.ostr << (settings.hilite ? hilite_keyword : "") << " CURRENT" << (settings.hilite ? hilite_none : ""); else if (kind == Kind::ROW_POLICY) { diff --git a/dbms/src/Parsers/ASTShowCreateAccessEntityQuery.h b/dbms/src/Parsers/ASTShowCreateAccessEntityQuery.h index 43fa215f64c..e76a9177979 100644 --- a/dbms/src/Parsers/ASTShowCreateAccessEntityQuery.h +++ b/dbms/src/Parsers/ASTShowCreateAccessEntityQuery.h @@ -9,6 +9,8 @@ namespace DB /** SHOW CREATE QUOTA [name | CURRENT] * SHOW CREATE [ROW] POLICY name ON [database.]table * SHOW CREATE USER [name | CURRENT_USER] + * SHOW CREATE ROLE name + * SHOW CREATE [SETTINGS] PROFILE name */ class ASTShowCreateAccessEntityQuery : public ASTQueryWithOutput { @@ -16,12 +18,13 @@ public: enum class Kind { USER, + ROLE, QUOTA, ROW_POLICY, + SETTINGS_PROFILE, }; - const Kind kind; - const char * const keyword; + const Kind kind; String name; bool current_quota = false; bool current_user = false; diff --git a/dbms/src/Parsers/ParserCreateQuotaQuery.cpp b/dbms/src/Parsers/ParserCreateQuotaQuery.cpp index d59c9832d79..9a6afec6941 100644 --- a/dbms/src/Parsers/ParserCreateQuotaQuery.cpp +++ b/dbms/src/Parsers/ParserCreateQuotaQuery.cpp @@ -205,12 +205,10 @@ namespace bool ParserCreateQuotaQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) { bool alter = false; - bool attach = false; if (attach_mode) { if (!ParserKeyword{"ATTACH QUOTA"}.ignore(pos, expected)) return false; - attach = true; } else { @@ -243,7 +241,6 @@ bool ParserCreateQuotaQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expe String new_name; std::optional key_type; std::vector all_limits; - std::shared_ptr roles; while (true) { @@ -256,12 +253,12 @@ bool ParserCreateQuotaQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expe if (parseAllLimits(pos, expected, alter, all_limits)) continue; - if (!roles && parseToRoles(pos, expected, attach, roles)) - continue; - break; } + std::shared_ptr roles; + parseToRoles(pos, expected, attach_mode, roles); + auto query = std::make_shared(); node = query; diff --git a/dbms/src/Parsers/ParserCreateRoleQuery.cpp b/dbms/src/Parsers/ParserCreateRoleQuery.cpp index 5a4ef016f77..e2b42c976b4 100644 --- a/dbms/src/Parsers/ParserCreateRoleQuery.cpp +++ b/dbms/src/Parsers/ParserCreateRoleQuery.cpp @@ -3,6 +3,8 @@ #include #include #include +#include +#include #include @@ -20,18 +22,35 @@ namespace return parseRoleName(pos, expected, new_name); }); } + + bool parseSettings(IParserBase::Pos & pos, Expected & expected, bool id_mode, std::shared_ptr & settings) + { + return IParserBase::wrapParseImpl(pos, [&] + { + if (!ParserKeyword{"SETTINGS"}.ignore(pos, expected)) + return false; + + ASTPtr new_settings_ast; + if (!ParserSettingsProfileElements{}.useIDMode(id_mode).parse(pos, new_settings_ast, expected)) + return false; + + if (!settings) + settings = std::make_shared(); + const auto & new_settings = new_settings_ast->as(); + settings->elements.insert(settings->elements.end(), new_settings.elements.begin(), new_settings.elements.end()); + return true; + }); + } } bool ParserCreateRoleQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) { - bool attach = false; bool alter = false; if (attach_mode) { if (!ParserKeyword{"ATTACH ROLE"}.ignore(pos, expected)) return false; - attach = true; } else { @@ -62,19 +81,29 @@ bool ParserCreateRoleQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expec return false; String new_name; - if (alter) - parseRenameTo(pos, expected, new_name); + std::shared_ptr settings; + while (true) + { + if (alter && parseRenameTo(pos, expected, new_name)) + continue; + + if (parseSettings(pos, expected, attach_mode, settings)) + continue; + + break; + } auto query = std::make_shared(); node = query; query->alter = alter; - query->attach = attach; + query->attach = attach_mode; query->if_exists = if_exists; query->if_not_exists = if_not_exists; query->or_replace = or_replace; query->name = std::move(name); query->new_name = std::move(new_name); + query->settings = std::move(settings); return true; } diff --git a/dbms/src/Parsers/ParserCreateRoleQuery.h b/dbms/src/Parsers/ParserCreateRoleQuery.h index a1690687282..2afeb7f7ec4 100644 --- a/dbms/src/Parsers/ParserCreateRoleQuery.h +++ b/dbms/src/Parsers/ParserCreateRoleQuery.h @@ -7,9 +7,11 @@ namespace DB { /** Parses queries like * CREATE ROLE [IF NOT EXISTS | OR REPLACE] name + * [SETTINGS variable [= value] [MIN [=] min_value] [MAX [=] max_value] [READONLY|WRITABLE] | PROFILE 'profile_name'] [,...] * * ALTER ROLE [IF EXISTS] name - * [RENAME TO new_name] + * [RENAME TO new_name] + * [SETTINGS variable [= value] [MIN [=] min_value] [MAX [=] max_value] [READONLY|WRITABLE] | PROFILE 'profile_name'] [,...] */ class ParserCreateRoleQuery : public IParserBase { diff --git a/dbms/src/Parsers/ParserCreateRowPolicyQuery.cpp b/dbms/src/Parsers/ParserCreateRowPolicyQuery.cpp index e277091bc0c..ab0fbc87e12 100644 --- a/dbms/src/Parsers/ParserCreateRowPolicyQuery.cpp +++ b/dbms/src/Parsers/ParserCreateRowPolicyQuery.cpp @@ -206,12 +206,10 @@ namespace bool ParserCreateRowPolicyQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) { bool alter = false; - bool attach = false; if (attach_mode) { if (!ParserKeyword{"ATTACH POLICY"}.ignore(pos, expected) && !ParserKeyword{"ATTACH ROW POLICY"}.ignore(pos, expected)) return false; - attach = true; } else { @@ -248,7 +246,6 @@ bool ParserCreateRowPolicyQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & String new_policy_name; std::optional is_restrictive; std::vector> conditions; - std::shared_ptr roles; while (true) { @@ -261,17 +258,17 @@ bool ParserCreateRowPolicyQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & if (parseMultipleConditions(pos, expected, alter, conditions)) continue; - if (!roles && parseToRoles(pos, expected, attach, roles)) - continue; - break; } + std::shared_ptr roles; + parseToRoles(pos, expected, attach_mode, roles); + auto query = std::make_shared(); node = query; query->alter = alter; - query->attach = attach; + query->attach = attach_mode; query->if_exists = if_exists; query->if_not_exists = if_not_exists; query->or_replace = or_replace; diff --git a/dbms/src/Parsers/ParserCreateSettingsProfileQuery.cpp b/dbms/src/Parsers/ParserCreateSettingsProfileQuery.cpp new file mode 100644 index 00000000000..c7c9e064f6c --- /dev/null +++ b/dbms/src/Parsers/ParserCreateSettingsProfileQuery.cpp @@ -0,0 +1,130 @@ +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + + +namespace DB +{ +namespace +{ + bool parseRenameTo(IParserBase::Pos & pos, Expected & expected, String & new_name) + { + return IParserBase::wrapParseImpl(pos, [&] + { + if (!ParserKeyword{"RENAME TO"}.ignore(pos, expected)) + return false; + + return parseIdentifierOrStringLiteral(pos, expected, new_name); + }); + } + + bool parseSettings(IParserBase::Pos & pos, Expected & expected, bool id_mode, std::shared_ptr & settings) + { + return IParserBase::wrapParseImpl(pos, [&] + { + if (!ParserKeyword{"SETTINGS"}.ignore(pos, expected)) + return false; + + ASTPtr new_settings_ast; + if (!ParserSettingsProfileElements{}.useIDMode(id_mode).parse(pos, new_settings_ast, expected)) + return false; + + if (!settings) + settings = std::make_shared(); + const auto & new_settings = new_settings_ast->as(); + settings->elements.insert(settings->elements.end(), new_settings.elements.begin(), new_settings.elements.end()); + return true; + }); + } + + bool parseToRoles(IParserBase::Pos & pos, Expected & expected, bool id_mode, std::shared_ptr & roles) + { + return IParserBase::wrapParseImpl(pos, [&] + { + ASTPtr ast; + if (roles || !ParserKeyword{"TO"}.ignore(pos, expected) + || !ParserExtendedRoleSet{}.useIDMode(id_mode).parse(pos, ast, expected)) + return false; + + roles = std::static_pointer_cast(ast); + return true; + }); + } +} + + +bool ParserCreateSettingsProfileQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) +{ + bool alter = false; + if (attach_mode) + { + if (!ParserKeyword{"ATTACH SETTINGS PROFILE"}.ignore(pos, expected) && !ParserKeyword{"ATTACH PROFILE"}.ignore(pos, expected)) + return false; + } + else + { + if (ParserKeyword{"ALTER SETTINGS PROFILE"}.ignore(pos, expected) || ParserKeyword{"ALTER PROFILE"}.ignore(pos, expected)) + alter = true; + else if (!ParserKeyword{"CREATE SETTINGS PROFILE"}.ignore(pos, expected) && !ParserKeyword{"CREATE PROFILE"}.ignore(pos, expected)) + return false; + } + + bool if_exists = false; + bool if_not_exists = false; + bool or_replace = false; + if (alter) + { + if (ParserKeyword{"IF EXISTS"}.ignore(pos, expected)) + if_exists = true; + } + else + { + if (ParserKeyword{"IF NOT EXISTS"}.ignore(pos, expected)) + if_not_exists = true; + else if (ParserKeyword{"OR REPLACE"}.ignore(pos, expected)) + or_replace = true; + } + + String name; + if (!parseIdentifierOrStringLiteral(pos, expected, name)) + return false; + + String new_name; + std::shared_ptr settings; + while (true) + { + if (alter && parseRenameTo(pos, expected, new_name)) + continue; + + if (parseSettings(pos, expected, attach_mode, settings)) + continue; + + break; + } + + std::shared_ptr to_roles; + parseToRoles(pos, expected, attach_mode, to_roles); + + auto query = std::make_shared(); + node = query; + + query->alter = alter; + query->attach = attach_mode; + query->if_exists = if_exists; + query->if_not_exists = if_not_exists; + query->or_replace = or_replace; + query->name = std::move(name); + query->new_name = std::move(new_name); + query->settings = std::move(settings); + query->to_roles = std::move(to_roles); + + return true; +} +} diff --git a/dbms/src/Parsers/ParserCreateSettingsProfileQuery.h b/dbms/src/Parsers/ParserCreateSettingsProfileQuery.h new file mode 100644 index 00000000000..6797fc884fa --- /dev/null +++ b/dbms/src/Parsers/ParserCreateSettingsProfileQuery.h @@ -0,0 +1,28 @@ +#pragma once + +#include + + +namespace DB +{ +/** Parses queries like + * CREATE SETTINGS PROFILE [IF NOT EXISTS | OR REPLACE] name + * [SETTINGS variable [= value] [MIN [=] min_value] [MAX [=] max_value] [READONLY|WRITABLE] | PROFILE 'profile_name'] [,...] + * + * ALTER SETTINGS PROFILE [IF EXISTS] name + * [RENAME TO new_name] + * [SETTINGS variable [= value] [MIN [=] min_value] [MAX [=] max_value] [READONLY|WRITABLE] | PROFILE 'profile_name'] [,...] + */ +class ParserCreateSettingsProfileQuery : public IParserBase +{ +public: + ParserCreateSettingsProfileQuery & enableAttachMode(bool enable) { attach_mode = enable; return *this; } + +protected: + const char * getName() const override { return "CREATE SETTINGS PROFILE or ALTER SETTINGS PROFILE query"; } + bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override; + +private: + bool attach_mode = false; +}; +} diff --git a/dbms/src/Parsers/ParserCreateUserQuery.cpp b/dbms/src/Parsers/ParserCreateUserQuery.cpp index f82a592e773..a7cc6550644 100644 --- a/dbms/src/Parsers/ParserCreateUserQuery.cpp +++ b/dbms/src/Parsers/ParserCreateUserQuery.cpp @@ -7,6 +7,8 @@ #include #include #include +#include +#include #include #include @@ -225,18 +227,21 @@ namespace } - bool parseProfileName(IParserBase::Pos & pos, Expected & expected, std::optional & profile) + bool parseSettings(IParserBase::Pos & pos, Expected & expected, bool id_mode, std::shared_ptr & settings) { return IParserBase::wrapParseImpl(pos, [&] { - if (!ParserKeyword{"PROFILE"}.ignore(pos, expected)) + if (!ParserKeyword{"SETTINGS"}.ignore(pos, expected)) return false; - ASTPtr ast; - if (!ParserStringLiteral{}.parse(pos, ast, expected)) + ASTPtr new_settings_ast; + if (!ParserSettingsProfileElements{}.useIDMode(id_mode).parse(pos, new_settings_ast, expected)) return false; - profile = ast->as().value.safeGet(); + if (!settings) + settings = std::make_shared(); + const auto & new_settings = new_settings_ast->as(); + settings->elements.insert(settings->elements.end(), new_settings.elements.begin(), new_settings.elements.end()); return true; }); } @@ -246,12 +251,9 @@ namespace bool ParserCreateUserQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) { bool alter = false; - bool attach = false; if (attach_mode) { - if (ParserKeyword{"ATTACH USER"}.ignore(pos, expected)) - attach = true; - else + if (!ParserKeyword{"ATTACH USER"}.ignore(pos, expected)) return false; } else @@ -290,7 +292,7 @@ bool ParserCreateUserQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expec std::optional add_hosts; std::optional remove_hosts; std::shared_ptr default_roles; - std::optional profile; + std::shared_ptr settings; while (true) { @@ -300,10 +302,10 @@ bool ParserCreateUserQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expec if (parseHosts(pos, expected, nullptr, hosts)) continue; - if (!profile && parseProfileName(pos, expected, profile)) + if (parseSettings(pos, expected, attach_mode, settings)) continue; - if (!default_roles && parseDefaultRoles(pos, expected, attach, default_roles)) + if (!default_roles && parseDefaultRoles(pos, expected, attach_mode, default_roles)) continue; if (alter) @@ -330,7 +332,7 @@ bool ParserCreateUserQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expec node = query; query->alter = alter; - query->attach = attach; + query->attach = attach_mode; query->if_exists = if_exists; query->if_not_exists = if_not_exists; query->or_replace = or_replace; @@ -341,7 +343,7 @@ bool ParserCreateUserQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expec query->add_hosts = std::move(add_hosts); query->remove_hosts = std::move(remove_hosts); query->default_roles = std::move(default_roles); - query->profile = std::move(profile); + query->settings = std::move(settings); return true; } diff --git a/dbms/src/Parsers/ParserCreateUserQuery.h b/dbms/src/Parsers/ParserCreateUserQuery.h index 85e0ada7cf6..bd6ab74d53f 100644 --- a/dbms/src/Parsers/ParserCreateUserQuery.h +++ b/dbms/src/Parsers/ParserCreateUserQuery.h @@ -7,15 +7,15 @@ namespace DB { /** Parses queries like * CREATE USER [IF NOT EXISTS | OR REPLACE] name - * [IDENTIFIED [WITH {NO_PASSWORD|PLAINTEXT_PASSWORD|SHA256_PASSWORD|SHA256_HASH|DOUBLE_SHA1_PASSWORD|DOUBLE_SHA1_HASH}] BY {'password'|'hash'}] - * [HOST {LOCAL | NAME 'name' | NAME REGEXP 'name_regexp' | IP 'address' | LIKE 'pattern'} [,...] | ANY | NONE] - * [PROFILE 'profile_name'] + * [IDENTIFIED [WITH {NO_PASSWORD|PLAINTEXT_PASSWORD|SHA256_PASSWORD|SHA256_HASH|DOUBLE_SHA1_PASSWORD|DOUBLE_SHA1_HASH}] BY {'password'|'hash'}] + * [HOST {LOCAL | NAME 'name' | NAME REGEXP 'name_regexp' | IP 'address' | LIKE 'pattern'} [,...] | ANY | NONE] + * [SETTINGS variable [= value] [MIN [=] min_value] [MAX [=] max_value] [READONLY|WRITABLE] | PROFILE 'profile_name'] [,...] * * ALTER USER [IF EXISTS] name - * [RENAME TO new_name] - * [IDENTIFIED [WITH {PLAINTEXT_PASSWORD|SHA256_PASSWORD|DOUBLE_SHA1_PASSWORD}] BY {'password'|'hash'}] - * [[ADD|REMOVE] HOST {LOCAL | NAME 'name' | NAME REGEXP 'name_regexp' | IP 'address' | LIKE 'pattern'} [,...] | ANY | NONE] - * [PROFILE 'profile_name'] + * [RENAME TO new_name] + * [IDENTIFIED [WITH {PLAINTEXT_PASSWORD|SHA256_PASSWORD|DOUBLE_SHA1_PASSWORD}] BY {'password'|'hash'}] + * [[ADD|REMOVE] HOST {LOCAL | NAME 'name' | NAME REGEXP 'name_regexp' | IP 'address' | LIKE 'pattern'} [,...] | ANY | NONE] + * [SETTINGS variable [= value] [MIN [=] min_value] [MAX [=] max_value] [READONLY|WRITABLE] | PROFILE 'profile_name'] [,...] */ class ParserCreateUserQuery : public IParserBase { diff --git a/dbms/src/Parsers/ParserDropAccessEntityQuery.cpp b/dbms/src/Parsers/ParserDropAccessEntityQuery.cpp index f257dc0fd64..23e18d7d32c 100644 --- a/dbms/src/Parsers/ParserDropAccessEntityQuery.cpp +++ b/dbms/src/Parsers/ParserDropAccessEntityQuery.cpp @@ -4,7 +4,6 @@ #include #include #include -#include namespace DB @@ -90,6 +89,8 @@ bool ParserDropAccessEntityQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & kind = Kind::QUOTA; else if (ParserKeyword{"POLICY"}.ignore(pos, expected) || ParserKeyword{"ROW POLICY"}.ignore(pos, expected)) kind = Kind::ROW_POLICY; + else if (ParserKeyword{"SETTINGS PROFILE"}.ignore(pos, expected) || ParserKeyword{"PROFILE"}.ignore(pos, expected)) + kind = Kind::SETTINGS_PROFILE; else return false; @@ -112,7 +113,6 @@ bool ParserDropAccessEntityQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & } else { - assert(kind == Kind::QUOTA); if (!parseNames(pos, expected, names)) return false; } diff --git a/dbms/src/Parsers/ParserDropAccessEntityQuery.h b/dbms/src/Parsers/ParserDropAccessEntityQuery.h index e4fb323d5f6..fd9149ba03a 100644 --- a/dbms/src/Parsers/ParserDropAccessEntityQuery.h +++ b/dbms/src/Parsers/ParserDropAccessEntityQuery.h @@ -9,12 +9,13 @@ namespace DB * DROP USER [IF EXISTS] name [,...] * DROP ROLE [IF EXISTS] name [,...] * DROP QUOTA [IF EXISTS] name [,...] + * DROP [SETTINGS] PROFILE [IF EXISTS] name [,...] * DROP [ROW] POLICY [IF EXISTS] name [,...] ON [database.]table [,...] */ class ParserDropAccessEntityQuery : public IParserBase { protected: - const char * getName() const override { return "DROP QUOTA query"; } + const char * getName() const override { return "DROP access entity query"; } bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override; }; } diff --git a/dbms/src/Parsers/ParserQuery.cpp b/dbms/src/Parsers/ParserQuery.cpp index a157a3ca354..144c309927b 100644 --- a/dbms/src/Parsers/ParserQuery.cpp +++ b/dbms/src/Parsers/ParserQuery.cpp @@ -14,6 +14,7 @@ #include #include #include +#include #include #include @@ -33,6 +34,7 @@ bool ParserQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) ParserCreateRoleQuery create_role_p; ParserCreateQuotaQuery create_quota_p; ParserCreateRowPolicyQuery create_row_policy_p; + ParserCreateSettingsProfileQuery create_settings_profile_p; ParserDropAccessEntityQuery drop_access_entity_p; ParserGrantQuery grant_p; ParserSetRoleQuery set_role_p; @@ -47,6 +49,7 @@ bool ParserQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) || create_role_p.parse(pos, node, expected) || create_quota_p.parse(pos, node, expected) || create_row_policy_p.parse(pos, node, expected) + || create_settings_profile_p.parse(pos, node, expected) || drop_access_entity_p.parse(pos, node, expected) || grant_p.parse(pos, node, expected); diff --git a/dbms/src/Parsers/ParserSettingsProfileElement.cpp b/dbms/src/Parsers/ParserSettingsProfileElement.cpp new file mode 100644 index 00000000000..06fa58fde4e --- /dev/null +++ b/dbms/src/Parsers/ParserSettingsProfileElement.cpp @@ -0,0 +1,164 @@ +#include +#include +#include +#include +#include +#include +#include + + +namespace DB +{ +namespace +{ + bool parseProfileNameOrID(IParserBase::Pos & pos, Expected & expected, bool parse_id, String & res) + { + return IParserBase::wrapParseImpl(pos, [&] + { + ASTPtr ast; + if (!parse_id) + return parseIdentifierOrStringLiteral(pos, expected, res); + + if (!ParserKeyword{"ID"}.ignore(pos, expected)) + return false; + if (!ParserToken(TokenType::OpeningRoundBracket).ignore(pos, expected)) + return false; + if (!ParserStringLiteral{}.parse(pos, ast, expected)) + return false; + String id = ast->as().value.safeGet(); + if (!ParserToken(TokenType::ClosingRoundBracket).ignore(pos, expected)) + return false; + + res = std::move(id); + return true; + }); + } + + + bool parseValue(IParserBase::Pos & pos, Expected & expected, Field & res) + { + return IParserBase::wrapParseImpl(pos, [&] + { + if (!ParserToken{TokenType::Equals}.ignore(pos, expected)) + return false; + + ASTPtr ast; + if (!ParserLiteral{}.parse(pos, ast, expected)) + return false; + + res = ast->as().value; + return true; + }); + } + + + bool parseMinMaxValue(IParserBase::Pos & pos, Expected & expected, Field & min_value, Field & max_value) + { + return IParserBase::wrapParseImpl(pos, [&] + { + bool is_min_value = ParserKeyword{"MIN"}.ignore(pos, expected); + bool is_max_value = !is_min_value && ParserKeyword{"MAX"}.ignore(pos, expected); + if (!is_min_value && !is_max_value) + return false; + + ParserToken{TokenType::Equals}.ignore(pos, expected); + + ASTPtr ast; + if (!ParserLiteral{}.parse(pos, ast, expected)) + return false; + + auto min_or_max_value = ast->as().value; + + if (is_min_value) + min_value = min_or_max_value; + else + max_value = min_or_max_value; + return true; + }); + } + + + bool parseReadonlyOrWritableKeyword(IParserBase::Pos & pos, Expected & expected, std::optional & readonly) + { + return IParserBase::wrapParseImpl(pos, [&] + { + if (ParserKeyword{"READONLY"}.ignore(pos, expected)) + { + readonly = true; + return true; + } + else if (ParserKeyword{"READONLY"}.ignore(pos, expected)) + { + readonly = false; + return true; + } + else + return false; + }); + } +} + + +bool ParserSettingsProfileElement::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) +{ + String parent_profile; + String name; + Field value; + Field min_value; + Field max_value; + std::optional readonly; + + if (ParserKeyword{"PROFILE"}.ignore(pos, expected)) + { + if (!parseProfileNameOrID(pos, expected, id_mode, parent_profile)) + return false; + } + else + { + ASTPtr name_ast; + if (!ParserIdentifier{}.parse(pos, name_ast, expected)) + return false; + name = getIdentifierName(name_ast); + + while (parseValue(pos, expected, value) || parseMinMaxValue(pos, expected, min_value, max_value) + || parseReadonlyOrWritableKeyword(pos, expected, readonly)) + ; + } + + auto result = std::make_shared(); + result->parent_profile = std::move(parent_profile); + result->name = std::move(name); + result->value = std::move(value); + result->min_value = std::move(min_value); + result->max_value = std::move(max_value); + result->readonly = readonly; + result->id_mode = id_mode; + node = result; + return true; +} + + +bool ParserSettingsProfileElements::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) +{ + std::vector> elements; + + if (!ParserKeyword{"NONE"}.ignore(pos, expected)) + { + do + { + ASTPtr ast; + if (!ParserSettingsProfileElement{}.useIDMode(id_mode).parse(pos, ast, expected)) + return false; + auto element = typeid_cast>(ast); + elements.push_back(std::move(element)); + } + while (ParserToken{TokenType::Comma}.ignore(pos, expected)); + } + + auto result = std::make_shared(); + result->elements = std::move(elements); + node = result; + return true; +} + +} diff --git a/dbms/src/Parsers/ParserSettingsProfileElement.h b/dbms/src/Parsers/ParserSettingsProfileElement.h new file mode 100644 index 00000000000..ec8e1abb5b5 --- /dev/null +++ b/dbms/src/Parsers/ParserSettingsProfileElement.h @@ -0,0 +1,36 @@ +#pragma once + +#include + + +namespace DB +{ +/** Parses a string like this: + * {variable [= value] [MIN [=] min_value] [MAX [=] max_value] [READONLY|WRITABLE]} | PROFILE 'profile_name' + */ +class ParserSettingsProfileElement : public IParserBase +{ +public: + ParserSettingsProfileElement & useIDMode(bool enable_) { id_mode = enable_; return *this; } + +protected: + const char * getName() const override { return "SettingsProfileElement"; } + bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override; + +private: + bool id_mode = false; +}; + + +class ParserSettingsProfileElements : public IParserBase +{ +public: + ParserSettingsProfileElements & useIDMode(bool enable_) { id_mode = enable_; return *this; } + +protected: + const char * getName() const override { return "SettingsProfileElements"; } + bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override; + +private: + bool id_mode = false; +};} diff --git a/dbms/src/Parsers/ParserShowCreateAccessEntityQuery.cpp b/dbms/src/Parsers/ParserShowCreateAccessEntityQuery.cpp index d1e6bc45478..faf9a0a1554 100644 --- a/dbms/src/Parsers/ParserShowCreateAccessEntityQuery.cpp +++ b/dbms/src/Parsers/ParserShowCreateAccessEntityQuery.cpp @@ -22,6 +22,10 @@ bool ParserShowCreateAccessEntityQuery::parseImpl(Pos & pos, ASTPtr & node, Expe kind = Kind::QUOTA; else if (ParserKeyword{"POLICY"}.ignore(pos, expected) || ParserKeyword{"ROW POLICY"}.ignore(pos, expected)) kind = Kind::ROW_POLICY; + else if (ParserKeyword{"ROLE"}.ignore(pos, expected)) + kind = Kind::ROLE; + else if (ParserKeyword{"SETTINGS PROFILE"}.ignore(pos, expected) || ParserKeyword{"PROFILE"}.ignore(pos, expected)) + kind = Kind::SETTINGS_PROFILE; else return false; @@ -35,6 +39,11 @@ bool ParserShowCreateAccessEntityQuery::parseImpl(Pos & pos, ASTPtr & node, Expe if (!parseUserNameOrCurrentUserTag(pos, expected, name, current_user)) current_user = true; } + else if (kind == Kind::ROLE) + { + if (!parseRoleName(pos, expected, name)) + return false; + } else if (kind == Kind::ROW_POLICY) { String & database = row_policy_name.database; @@ -44,9 +53,8 @@ bool ParserShowCreateAccessEntityQuery::parseImpl(Pos & pos, ASTPtr & node, Expe || !parseDatabaseAndTableName(pos, expected, database, table_name)) return false; } - else + else if (kind == Kind::QUOTA) { - assert(kind == Kind::QUOTA); if (ParserKeyword{"CURRENT"}.ignore(pos, expected)) { /// SHOW CREATE QUOTA CURRENT @@ -62,6 +70,11 @@ bool ParserShowCreateAccessEntityQuery::parseImpl(Pos & pos, ASTPtr & node, Expe current_quota = true; } } + else if (kind == Kind::SETTINGS_PROFILE) + { + if (!parseIdentifierOrStringLiteral(pos, expected, name)) + return false; + } auto query = std::make_shared(kind); node = query; From 6839e15b67844aac5cd52c189636e6c65886c09a Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Wed, 18 Mar 2020 17:12:09 +0300 Subject: [PATCH 0156/1355] Add tests for settings profiles. --- .../integration/test_authentication/test.py | 4 +- .../test_disk_access_storage/test.py | 22 +++- .../integration/test_grant_and_revoke/test.py | 20 ++-- .../test_settings_profile/__init__.py | 0 .../integration/test_settings_profile/test.py | 106 ++++++++++++++++++ 5 files changed, 136 insertions(+), 16 deletions(-) create mode 100644 dbms/tests/integration/test_settings_profile/__init__.py create mode 100644 dbms/tests/integration/test_settings_profile/test.py diff --git a/dbms/tests/integration/test_authentication/test.py b/dbms/tests/integration/test_authentication/test.py index 11ca967fbee..483b59813e5 100644 --- a/dbms/tests/integration/test_authentication/test.py +++ b/dbms/tests/integration/test_authentication/test.py @@ -10,8 +10,8 @@ def setup_nodes(): try: cluster.start() - instance.query("CREATE USER sasha PROFILE 'default'") - instance.query("CREATE USER masha IDENTIFIED BY 'qwerty' PROFILE 'default'") + instance.query("CREATE USER sasha") + instance.query("CREATE USER masha IDENTIFIED BY 'qwerty'") yield cluster diff --git a/dbms/tests/integration/test_disk_access_storage/test.py b/dbms/tests/integration/test_disk_access_storage/test.py index 169c0a35414..d5e1f283167 100644 --- a/dbms/tests/integration/test_disk_access_storage/test.py +++ b/dbms/tests/integration/test_disk_access_storage/test.py @@ -16,9 +16,11 @@ def started_cluster(): def create_entities(): - instance.query("CREATE USER u1") - instance.query("CREATE ROLE rx") + instance.query("CREATE SETTINGS PROFILE s1 SETTINGS max_memory_usage = 123456789 MIN 100000000 MAX 200000000") + instance.query("CREATE USER u1 SETTINGS PROFILE s1") + instance.query("CREATE ROLE rx SETTINGS PROFILE s1") instance.query("CREATE USER u2 IDENTIFIED BY 'qwerty' HOST LOCAL DEFAULT ROLE rx") + instance.query("CREATE SETTINGS PROFILE s2 SETTINGS PROFILE s1 TO u2") instance.query("CREATE ROW POLICY p ON mydb.mytable FOR SELECT USING a<1000 TO u1, u2") instance.query("CREATE QUOTA q FOR INTERVAL 1 HOUR SET MAX QUERIES = 100 TO ALL EXCEPT rx") @@ -29,19 +31,23 @@ def drop_entities(): instance.query("DROP ROLE IF EXISTS rx, ry") instance.query("DROP ROW POLICY IF EXISTS p ON mydb.mytable") instance.query("DROP QUOTA IF EXISTS q") + instance.query("DROP SETTINGS PROFILE IF EXISTS s1, s2") def test_create(): create_entities() def check(): - assert instance.query("SHOW CREATE USER u1") == "CREATE USER u1\n" + assert instance.query("SHOW CREATE USER u1") == "CREATE USER u1 SETTINGS PROFILE s1\n" assert instance.query("SHOW CREATE USER u2") == "CREATE USER u2 HOST LOCAL DEFAULT ROLE rx\n" assert instance.query("SHOW CREATE ROW POLICY p ON mydb.mytable") == "CREATE POLICY p ON mydb.mytable FOR SELECT USING a < 1000 TO u1, u2\n" assert instance.query("SHOW CREATE QUOTA q") == "CREATE QUOTA q KEYED BY \\'none\\' FOR INTERVAL 1 HOUR MAX QUERIES = 100 TO ALL EXCEPT rx\n" assert instance.query("SHOW GRANTS FOR u1") == "" assert instance.query("SHOW GRANTS FOR u2") == "GRANT rx TO u2\n" + assert instance.query("SHOW CREATE ROLE rx") == "CREATE ROLE rx SETTINGS PROFILE s1\n" assert instance.query("SHOW GRANTS FOR rx") == "" + assert instance.query("SHOW CREATE SETTINGS PROFILE s1") == "CREATE SETTINGS PROFILE s1 SETTINGS max_memory_usage = 123456789 MIN 100000000 MAX 200000000\n" + assert instance.query("SHOW CREATE SETTINGS PROFILE s2") == "CREATE SETTINGS PROFILE s2 SETTINGS PROFILE s1 TO u2\n" check() instance.restart_clickhouse() # Check persistency @@ -56,16 +62,22 @@ def test_alter(): instance.query("GRANT ry TO u2") instance.query("ALTER USER u2 DEFAULT ROLE ry") instance.query("GRANT rx TO ry WITH ADMIN OPTION") + instance.query("ALTER ROLE rx SETTINGS PROFILE s2") instance.query("GRANT SELECT ON mydb.mytable TO u1") instance.query("GRANT SELECT ON mydb.* TO rx WITH GRANT OPTION") + instance.query("ALTER SETTINGS PROFILE s1 SETTINGS max_memory_usage = 987654321 READONLY") def check(): - assert instance.query("SHOW CREATE USER u1") == "CREATE USER u1\n" + assert instance.query("SHOW CREATE USER u1") == "CREATE USER u1 SETTINGS PROFILE s1\n" assert instance.query("SHOW CREATE USER u2") == "CREATE USER u2 HOST LOCAL DEFAULT ROLE ry\n" assert instance.query("SHOW GRANTS FOR u1") == "GRANT SELECT ON mydb.mytable TO u1\n" assert instance.query("SHOW GRANTS FOR u2") == "GRANT rx, ry TO u2\n" + assert instance.query("SHOW CREATE ROLE rx") == "CREATE ROLE rx SETTINGS PROFILE s2\n" + assert instance.query("SHOW CREATE ROLE ry") == "CREATE ROLE ry\n" assert instance.query("SHOW GRANTS FOR rx") == "GRANT SELECT ON mydb.* TO rx WITH GRANT OPTION\n" assert instance.query("SHOW GRANTS FOR ry") == "GRANT rx TO ry WITH ADMIN OPTION\n" + assert instance.query("SHOW CREATE SETTINGS PROFILE s1") == "CREATE SETTINGS PROFILE s1 SETTINGS max_memory_usage = 987654321 READONLY\n" + assert instance.query("SHOW CREATE SETTINGS PROFILE s2") == "CREATE SETTINGS PROFILE s2 SETTINGS PROFILE s1 TO u2\n" check() instance.restart_clickhouse() # Check persistency @@ -80,9 +92,11 @@ def test_drop(): instance.query("DROP ROLE rx") instance.query("DROP ROW POLICY p ON mydb.mytable") instance.query("DROP QUOTA q") + instance.query("DROP SETTINGS PROFILE s1") def check(): assert instance.query("SHOW CREATE USER u1") == "CREATE USER u1\n" + assert instance.query("SHOW CREATE SETTINGS PROFILE s2") == "CREATE SETTINGS PROFILE s2\n" assert "User `u2` not found" in instance.query_and_get_error("SHOW CREATE USER u2") assert "Row policy `p ON mydb.mytable` not found" in instance.query_and_get_error("SHOW CREATE ROW POLICY p ON mydb.mytable") assert "Quota `q` not found" in instance.query_and_get_error("SHOW CREATE QUOTA q") diff --git a/dbms/tests/integration/test_grant_and_revoke/test.py b/dbms/tests/integration/test_grant_and_revoke/test.py index 132e62f3db0..25e0e9882de 100644 --- a/dbms/tests/integration/test_grant_and_revoke/test.py +++ b/dbms/tests/integration/test_grant_and_revoke/test.py @@ -30,14 +30,14 @@ def reset_users_and_roles(): def test_login(): - instance.query("CREATE USER A PROFILE 'default'") - instance.query("CREATE USER B PROFILE 'default'") + instance.query("CREATE USER A") + instance.query("CREATE USER B") assert instance.query("SELECT 1", user='A') == "1\n" assert instance.query("SELECT 1", user='B') == "1\n" def test_grant_and_revoke(): - instance.query("CREATE USER A PROFILE 'default'") + instance.query("CREATE USER A") assert "Not enough privileges" in instance.query_and_get_error("SELECT * FROM test_table", user='A') instance.query('GRANT SELECT ON test_table TO A') @@ -48,8 +48,8 @@ def test_grant_and_revoke(): def test_grant_option(): - instance.query("CREATE USER A PROFILE 'default'") - instance.query("CREATE USER B PROFILE 'default'") + instance.query("CREATE USER A") + instance.query("CREATE USER B") instance.query('GRANT SELECT ON test_table TO A') assert instance.query("SELECT * FROM test_table", user='A') == "1\t5\n2\t10\n" @@ -63,7 +63,7 @@ def test_grant_option(): def test_create_role(): - instance.query("CREATE USER A PROFILE 'default'") + instance.query("CREATE USER A") instance.query('CREATE ROLE R1') assert "Not enough privileges" in instance.query_and_get_error("SELECT * FROM test_table", user='A') @@ -79,7 +79,7 @@ def test_create_role(): def test_grant_role_to_role(): - instance.query("CREATE USER A PROFILE 'default'") + instance.query("CREATE USER A") instance.query('CREATE ROLE R1') instance.query('CREATE ROLE R2') @@ -96,7 +96,7 @@ def test_grant_role_to_role(): def test_combine_privileges(): - instance.query("CREATE USER A PROFILE 'default'") + instance.query("CREATE USER A ") instance.query('CREATE ROLE R1') instance.query('CREATE ROLE R2') @@ -113,8 +113,8 @@ def test_combine_privileges(): def test_admin_option(): - instance.query("CREATE USER A PROFILE 'default'") - instance.query("CREATE USER B PROFILE 'default'") + instance.query("CREATE USER A") + instance.query("CREATE USER B") instance.query('CREATE ROLE R1') instance.query('GRANT SELECT ON test_table TO R1') diff --git a/dbms/tests/integration/test_settings_profile/__init__.py b/dbms/tests/integration/test_settings_profile/__init__.py new file mode 100644 index 00000000000..e69de29bb2d diff --git a/dbms/tests/integration/test_settings_profile/test.py b/dbms/tests/integration/test_settings_profile/test.py new file mode 100644 index 00000000000..592ab5b92d6 --- /dev/null +++ b/dbms/tests/integration/test_settings_profile/test.py @@ -0,0 +1,106 @@ +import pytest +from helpers.cluster import ClickHouseCluster + +cluster = ClickHouseCluster(__file__) +instance = cluster.add_instance('instance') + + +@pytest.fixture(scope="module", autouse=True) +def setup_nodes(): + try: + cluster.start() + + instance.query("CREATE USER robin") + + yield cluster + + finally: + cluster.shutdown() + + +@pytest.fixture(autouse=True) +def reset_after_test(): + try: + yield + finally: + instance.query("CREATE USER OR REPLACE robin") + instance.query("DROP ROLE IF EXISTS worker") + instance.query("DROP SETTINGS PROFILE IF EXISTS xyz, alpha") + + +def test_settings_profile(): + # Set settings and constraints via CREATE SETTINGS PROFILE ... TO user + instance.query("CREATE SETTINGS PROFILE xyz SETTINGS max_memory_usage = 100000001 MIN 90000000 MAX 110000000 TO robin") + assert instance.query("SELECT value FROM system.settings WHERE name = 'max_memory_usage'", user="robin") == "100000001\n" + assert "Setting max_memory_usage shouldn't be less than 90000000" in instance.query_and_get_error("SET max_memory_usage = 80000000", user="robin") + assert "Setting max_memory_usage shouldn't be greater than 110000000" in instance.query_and_get_error("SET max_memory_usage = 120000000", user="robin") + + instance.query("ALTER SETTINGS PROFILE xyz TO NONE") + assert instance.query("SELECT value FROM system.settings WHERE name = 'max_memory_usage'", user="robin") == "10000000000\n" + instance.query("SET max_memory_usage = 80000000", user="robin") + instance.query("SET max_memory_usage = 120000000", user="robin") + + # Set settings and constraints via CREATE USER ... SETTINGS PROFILE + instance.query("ALTER USER robin SETTINGS PROFILE xyz") + assert instance.query("SELECT value FROM system.settings WHERE name = 'max_memory_usage'", user="robin") == "100000001\n" + assert "Setting max_memory_usage shouldn't be less than 90000000" in instance.query_and_get_error("SET max_memory_usage = 80000000", user="robin") + assert "Setting max_memory_usage shouldn't be greater than 110000000" in instance.query_and_get_error("SET max_memory_usage = 120000000", user="robin") + + instance.query("ALTER USER robin SETTINGS NONE") + assert instance.query("SELECT value FROM system.settings WHERE name = 'max_memory_usage'", user="robin") == "10000000000\n" + instance.query("SET max_memory_usage = 80000000", user="robin") + instance.query("SET max_memory_usage = 120000000", user="robin") + + +def test_settings_profile_from_granted_role(): + # Set settings and constraints via granted role + instance.query("CREATE SETTINGS PROFILE xyz SETTINGS max_memory_usage = 100000001 MIN 90000000 MAX 110000000") + instance.query("CREATE ROLE worker SETTINGS PROFILE xyz") + instance.query("GRANT worker TO robin") + assert instance.query("SELECT value FROM system.settings WHERE name = 'max_memory_usage'", user="robin") == "100000001\n" + assert "Setting max_memory_usage shouldn't be less than 90000000" in instance.query_and_get_error("SET max_memory_usage = 80000000", user="robin") + assert "Setting max_memory_usage shouldn't be greater than 110000000" in instance.query_and_get_error("SET max_memory_usage = 120000000", user="robin") + + instance.query("REVOKE worker FROM robin") + assert instance.query("SELECT value FROM system.settings WHERE name = 'max_memory_usage'", user="robin") == "10000000000\n" + instance.query("SET max_memory_usage = 80000000", user="robin") + instance.query("SET max_memory_usage = 120000000", user="robin") + + instance.query("ALTER ROLE worker SETTINGS NONE") + instance.query("GRANT worker TO robin") + assert instance.query("SELECT value FROM system.settings WHERE name = 'max_memory_usage'", user="robin") == "10000000000\n" + instance.query("SET max_memory_usage = 80000000", user="robin") + instance.query("SET max_memory_usage = 120000000", user="robin") + + # Set settings and constraints via CREATE SETTINGS PROFILE ... TO granted role + instance.query("ALTER SETTINGS PROFILE xyz TO worker") + assert instance.query("SELECT value FROM system.settings WHERE name = 'max_memory_usage'", user="robin") == "100000001\n" + assert "Setting max_memory_usage shouldn't be less than 90000000" in instance.query_and_get_error("SET max_memory_usage = 80000000", user="robin") + assert "Setting max_memory_usage shouldn't be greater than 110000000" in instance.query_and_get_error("SET max_memory_usage = 120000000", user="robin") + + instance.query("ALTER SETTINGS PROFILE xyz TO NONE") + assert instance.query("SELECT value FROM system.settings WHERE name = 'max_memory_usage'", user="robin") == "10000000000\n" + instance.query("SET max_memory_usage = 80000000", user="robin") + instance.query("SET max_memory_usage = 120000000", user="robin") + + +def test_inheritance_of_settings_profile(): + instance.query("CREATE SETTINGS PROFILE xyz SETTINGS max_memory_usage = 100000002 READONLY") + instance.query("CREATE SETTINGS PROFILE alpha SETTINGS PROFILE xyz TO robin") + assert instance.query("SELECT value FROM system.settings WHERE name = 'max_memory_usage'", user="robin") == "100000002\n" + assert "Setting max_memory_usage should not be changed" in instance.query_and_get_error("SET max_memory_usage = 80000000", user="robin") + + +def test_alter_and_drop(): + instance.query("CREATE SETTINGS PROFILE xyz SETTINGS max_memory_usage = 100000003 MIN 90000000 MAX 110000000 TO robin") + assert instance.query("SELECT value FROM system.settings WHERE name = 'max_memory_usage'", user="robin") == "100000003\n" + assert "Setting max_memory_usage shouldn't be less than 90000000" in instance.query_and_get_error("SET max_memory_usage = 80000000", user="robin") + assert "Setting max_memory_usage shouldn't be greater than 110000000" in instance.query_and_get_error("SET max_memory_usage = 120000000", user="robin") + + instance.query("ALTER SETTINGS PROFILE xyz SETTINGS readonly=1") + assert "Cannot modify 'max_memory_usage' setting in readonly mode" in instance.query_and_get_error("SET max_memory_usage = 80000000", user="robin") + + instance.query("DROP SETTINGS PROFILE xyz") + assert instance.query("SELECT value FROM system.settings WHERE name = 'max_memory_usage'", user="robin") == "10000000000\n" + instance.query("SET max_memory_usage = 80000000", user="robin") + instance.query("SET max_memory_usage = 120000000", user="robin") From 03d3332d434be4ea60967c498a8d5787a819742f Mon Sep 17 00:00:00 2001 From: Ivan Lezhankin Date: Sun, 22 Mar 2020 00:53:35 +0300 Subject: [PATCH 0157/1355] Fix cmake for unbundled build --- contrib/jemalloc-cmake/CMakeLists.txt | 8 +++++--- 1 file changed, 5 insertions(+), 3 deletions(-) diff --git a/contrib/jemalloc-cmake/CMakeLists.txt b/contrib/jemalloc-cmake/CMakeLists.txt index 4e3a817b481..26d42d63763 100644 --- a/contrib/jemalloc-cmake/CMakeLists.txt +++ b/contrib/jemalloc-cmake/CMakeLists.txt @@ -76,9 +76,11 @@ if (ENABLE_JEMALLOC) add_library (jemalloc STATIC IMPORTED) set_property (TARGET jemalloc PROPERTY IMPORTED_LOCATION ${LIBRARY_JEMALLOC}) - target_include_directories (jemalloc PUBLIC ${INCLUDE_JEMALLOC}) + set_property (TARGET jemalloc PROPERTY INTERFACE_INCLUDE_DIRECTORIES ${INCLUDE_JEMALLOC}) - set (CMAKE_REQUIRED_LIBRARIES jemalloc) + set(THREADS_PREFER_PTHREAD_FLAG ON) + find_package(Threads REQUIRED) + set (CMAKE_REQUIRED_LIBRARIES jemalloc Threads::Threads) check_cxx_source_compiles ( " #include @@ -95,7 +97,7 @@ if (ENABLE_JEMALLOC) endif () endif () - target_compile_definitions(jemalloc PUBLIC USE_JEMALLOC=1) + target_compile_definitions(jemalloc INTERFACE USE_JEMALLOC=1) message (STATUS "Using jemalloc") else () From d8eea84b1de943a4706ef32491c4e026539ed783 Mon Sep 17 00:00:00 2001 From: Ivan Blinkov Date: Sun, 22 Mar 2020 14:22:52 +0300 Subject: [PATCH 0158/1355] Do not do useless work on docs for old releases --- docs/tools/build.py | 31 ++++++++++++++++--------------- 1 file changed, 16 insertions(+), 15 deletions(-) diff --git a/docs/tools/build.py b/docs/tools/build.py index 4e0ad21083e..c42b9f90d03 100755 --- a/docs/tools/build.py +++ b/docs/tools/build.py @@ -196,21 +196,22 @@ def build_single_page_version(lang, args, cfg): with open(os.devnull, 'w') as devnull: subprocess.check_call(' '.join(create_pdf_command), shell=True, stderr=devnull) - with util.temp_dir() as test_dir: - cfg.load_dict({ - 'docs_dir': docs_temp_lang, - 'site_dir': test_dir, - 'extra': { - 'single_page': False - }, - 'nav': [ - {cfg.data.get('site_name'): 'single.md'} - ] - }) - mkdocs_build.build(cfg) - if args.save_raw_single_page: - shutil.copytree(test_dir, args.save_raw_single_page) - if not args.version_prefix: # maybe enable in future + if not args.version_prefix: # maybe enable in future + with util.temp_dir() as test_dir: + cfg.load_dict({ + 'docs_dir': docs_temp_lang, + 'site_dir': test_dir, + 'extra': { + 'single_page': False + }, + 'nav': [ + {cfg.data.get('site_name'): 'single.md'} + ] + }) + mkdocs_build.build(cfg) + if args.save_raw_single_page: + shutil.copytree(test_dir, args.save_raw_single_page) + test.test_single_page(os.path.join(test_dir, 'single', 'index.html'), lang) From 622131fc853a2bb1d002e1069f2b5c6ff808c4d3 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sun, 26 Jan 2020 23:31:51 +0300 Subject: [PATCH 0159/1355] Cover constant expression evaluation for optimize_skip_unused_shards v2: remove key=xxHash64() that is covered by #9804 --- ...ip_unused_shards_const_expr_eval.reference | 0 ...ize_skip_unused_shards_const_expr_eval.sql | 49 +++++++++++++++++++ 2 files changed, 49 insertions(+) create mode 100644 dbms/tests/queries/0_stateless/01072_optimize_skip_unused_shards_const_expr_eval.reference create mode 100644 dbms/tests/queries/0_stateless/01072_optimize_skip_unused_shards_const_expr_eval.sql diff --git a/dbms/tests/queries/0_stateless/01072_optimize_skip_unused_shards_const_expr_eval.reference b/dbms/tests/queries/0_stateless/01072_optimize_skip_unused_shards_const_expr_eval.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/dbms/tests/queries/0_stateless/01072_optimize_skip_unused_shards_const_expr_eval.sql b/dbms/tests/queries/0_stateless/01072_optimize_skip_unused_shards_const_expr_eval.sql new file mode 100644 index 00000000000..7e204159c73 --- /dev/null +++ b/dbms/tests/queries/0_stateless/01072_optimize_skip_unused_shards_const_expr_eval.sql @@ -0,0 +1,49 @@ +drop table if exists data_01072; +drop table if exists dist_01072; + +set optimize_skip_unused_shards=1; +set force_optimize_skip_unused_shards=1; + +create table data_01072 (key Int, value Int, str String) Engine=Null(); +create table dist_01072 (key Int, value Int, str String) Engine=Distributed(test_cluster_two_shards, currentDatabase(), data_01072, key%2); + +select * from dist_01072 where key=0 and length(str)=0; +select * from dist_01072 where key=0 and str=''; +select * from dist_01072 where xxHash64(0)==xxHash64(0) and key=0; +select * from dist_01072 where key=toInt32OrZero(toString(xxHash64(0))); +select * from dist_01072 where key=toInt32(xxHash32(0)); +select * from dist_01072 where key=toInt32(toInt32(xxHash32(0))); +select * from dist_01072 where key=toInt32(toInt32(toInt32(xxHash32(0)))); +select * from dist_01072 where key=value; -- { serverError 507; } +select * from dist_01072 where key=toInt32(value); -- { serverError 507; } +select * from dist_01072 where key=value settings force_optimize_skip_unused_shards=0; +select * from dist_01072 where key=toInt32(value) settings force_optimize_skip_unused_shards=0; + +drop table dist_01072; +create table dist_01072 (key Int, value Nullable(Int), str String) Engine=Distributed(test_cluster_two_shards, currentDatabase(), data_01072, key%2); +select * from dist_01072 where key=toInt32(xxHash32(0)); +select * from dist_01072 where key=value; -- { serverError 507; } +select * from dist_01072 where key=toInt32(value); -- { serverError 507; } +select * from dist_01072 where key=value settings force_optimize_skip_unused_shards=0; +select * from dist_01072 where key=toInt32(value) settings force_optimize_skip_unused_shards=0; + +set allow_suspicious_low_cardinality_types=1; + +drop table dist_01072; +create table dist_01072 (key Int, value LowCardinality(Int), str String) Engine=Distributed(test_cluster_two_shards, currentDatabase(), data_01072, key%2); +select * from dist_01072 where key=toInt32(xxHash32(0)); +select * from dist_01072 where key=value; -- { serverError 507; } +select * from dist_01072 where key=toInt32(value); -- { serverError 507; } +select * from dist_01072 where key=value settings force_optimize_skip_unused_shards=0; +select * from dist_01072 where key=toInt32(value) settings force_optimize_skip_unused_shards=0; + +drop table dist_01072; +create table dist_01072 (key Int, value LowCardinality(Nullable(Int)), str String) Engine=Distributed(test_cluster_two_shards, currentDatabase(), data_01072, key%2); +select * from dist_01072 where key=toInt32(xxHash32(0)); +select * from dist_01072 where key=value; -- { serverError 507; } +select * from dist_01072 where key=toInt32(value); -- { serverError 507; } +select * from dist_01072 where key=value settings force_optimize_skip_unused_shards=0; +select * from dist_01072 where key=toInt32(value) settings force_optimize_skip_unused_shards=0; + +drop table data_01072; +drop table dist_01072; From 9915506ad14ecbd307f23ab82c199ea43e72c1a2 Mon Sep 17 00:00:00 2001 From: Ivan Blinkov Date: Sun, 22 Mar 2020 15:08:47 +0300 Subject: [PATCH 0160/1355] Update docs/tools/README.md --- docs/tools/README.md | 20 ++++++++++---------- 1 file changed, 10 insertions(+), 10 deletions(-) diff --git a/docs/tools/README.md b/docs/tools/README.md index 985577dfe36..4de7a43e674 100644 --- a/docs/tools/README.md +++ b/docs/tools/README.md @@ -1,16 +1,20 @@ ## How ClickHouse documentation is generated? {#how-clickhouse-documentation-is-generated} -ClickHouse documentation is built using build.py script that uses [mkdocs](https://www.mkdocs.org) library and it’s dependencies to separately build all version of documentations (all languages in either single and multi page mode) as static HTMLs. The results are then put in correct directory structure. It can also generate PDF version. +ClickHouse documentation is built using [build.py](build.py) script that uses [mkdocs](https://www.mkdocs.org) library and it’s dependencies to separately build all version of documentations (all languages in either single and multi page mode) as static HTMLs and then a PDF for each single page version. The results are then put in the correct directory structure. It is recommended to use Python 3.7 to run this script. -release.sh also pulls static files needed for [official ClickHouse website](https://clickhouse.tech) from ../../website folder, packs them alongside docs into Docker container and tries to deploy it (possible only from Yandex private network). +[release.sh](release.sh) also pulls static files needed for [official ClickHouse website](https://clickhouse.tech) from [../../website](../../website) folder then pushes to specified GitHub repo to be served via [GitHub Pages](https://pages.github.com). ## How to check if the documentation will look fine? {#how-to-check-if-the-documentation-will-look-fine} There are few options that are all useful depending on how large or complex your edits are. +### Use GitHub web interface to edit + +GitHub has Markdown support with preview feature, but the details of GitHub Markdown dialect are a bit different in ClickHouse documentation. + ### Install Markdown editor or plugin for your IDE {#install-markdown-editor-or-plugin-for-your-ide} -Usually those have some way to preview how Markdown will look like, which allows to catch basic errors like unclosed tags very early. +Usually those also have some way to preview how Markdown will look like, which allows to catch basic errors like unclosed tags very early. ### Use build.py {#use-build-py} @@ -21,22 +25,18 @@ For the first time you’ll need to install [wkhtmltopdf](https://wkhtmltopdf.or ``` bash $ cd ClickHouse/docs/tools $ mkdir venv -$ virtualenv venv +$ virtualenv -p $(which python3) venv $ source venv/bin/activate -$ pip install -r requirements.txt +$ pip3 install -r requirements.txt ``` Then running `build.py` without args (there are some, check `build.py --help`) will generate `ClickHouse/docs/build` folder with complete static html website. You can just directly open those HTML files in browser, but usually it is more convenient to have some sort of HTTP server hosting them. For example, you can launch one by running `cd ClickHouse/docs/build && python -m SimpleHTTPServer 8888` and then go to http://localhost:8888 in browser. -### Commit blindly {#commit-blindly} - -Then push to GitHub so you can use it’s preview. It’s better to use previous methods too though. - ## How to subscribe on documentation changes? {#how-to-subscribe-on-documentation-changes} At the moment there’s no easy way to do just that, but you can consider: -- Hit the “Watch” button on top of GitHub web interface to know as early as possible, even during pull request. +- To hit the “Watch” button on top of GitHub web interface to know as early as possible, even during pull request. Alternative to this is `#github-activity` channel of [public ClickHouse Slack](https://join.slack.com/t/clickhousedb/shared_invite/enQtOTUzMjM4ODQwNTc5LWJmMjE3Yjc2YmI1ZDBlZmI4ZTc3OWY3ZTIwYTljYzY4MzBlODM3YzBjZTc1YmYyODRlZTJkYTgzYzBiNTA2Yjk). - Some search engines allow to subscribe on specific website changes via email and you can opt-in for that for https://clickhouse.tech. From d1851ecbd69dd84a411bac5c9d9848b2c300be29 Mon Sep 17 00:00:00 2001 From: Ivan Blinkov Date: Sun, 22 Mar 2020 15:12:15 +0300 Subject: [PATCH 0161/1355] Update README.md --- docs/tools/README.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/tools/README.md b/docs/tools/README.md index 4de7a43e674..56ca016ad9e 100644 --- a/docs/tools/README.md +++ b/docs/tools/README.md @@ -32,7 +32,7 @@ $ pip3 install -r requirements.txt Then running `build.py` without args (there are some, check `build.py --help`) will generate `ClickHouse/docs/build` folder with complete static html website. -You can just directly open those HTML files in browser, but usually it is more convenient to have some sort of HTTP server hosting them. For example, you can launch one by running `cd ClickHouse/docs/build && python -m SimpleHTTPServer 8888` and then go to http://localhost:8888 in browser. +The easiest way to see the result is to use `--livereload=8888` argument of build.py. Alternatively, you can manually launch a HTTP server to serve the docs, for example by running `cd ClickHouse/docs/build && python3 -m http.server 8888`. Then go to http://localhost:8888 in browser. Feel free to use any other port instead of 8888. ## How to subscribe on documentation changes? {#how-to-subscribe-on-documentation-changes} From dfa01a6358a65085381e3f6c831377a6e635f91b Mon Sep 17 00:00:00 2001 From: Ivan Blinkov Date: Sun, 22 Mar 2020 15:12:53 +0300 Subject: [PATCH 0162/1355] [docs] extra logging and minor refactoring (#9805) --- docs/tools/build.py | 23 ++++++++++++----------- docs/tools/website.py | 3 +++ 2 files changed, 15 insertions(+), 11 deletions(-) diff --git a/docs/tools/build.py b/docs/tools/build.py index c42b9f90d03..c6a294bf617 100755 --- a/docs/tools/build.py +++ b/docs/tools/build.py @@ -42,17 +42,17 @@ markdown.extensions.ClickHouseMarkdown = ClickHouseMarkdown def build_for_lang(lang, args): - logging.info('Building %s docs' % lang) + logging.info(f'Building {lang} docs') os.environ['SINGLE_PAGE'] = '0' - config_path = os.path.join(args.docs_dir, 'toc_%s.yml' % lang) + config_path = os.path.join(args.docs_dir, f'toc_{lang}.yml') if args.is_stable_release and not os.path.exists(config_path): - logging.warn('Skipping %s docs, because %s does not exist' % (lang, config_path)) + logging.warning(f'Skipping {lang} docs, because {config} does not exist') return try: theme_cfg = { - 'name': 'mkdocs', + 'name': None, 'custom_dir': os.path.join(os.path.dirname(__file__), args.theme_dir), 'language': lang, 'direction': 'rtl' if lang == 'fa' else 'ltr', @@ -97,7 +97,7 @@ def build_for_lang(lang, args): cfg = config.load_config( config_file=config_path, site_name=site_names.get(lang, site_names['en']) % args.version_prefix, - site_url='https://clickhouse.yandex/docs/%s/' % lang, + site_url=f'https://clickhouse.yandex/docs/{lang}/', docs_dir=os.path.join(args.docs_dir, lang), site_dir=site_dir, strict=not args.version_prefix, @@ -106,8 +106,8 @@ def build_for_lang(lang, args): use_directory_urls=True, repo_name='ClickHouse/ClickHouse', repo_url='https://github.com/ClickHouse/ClickHouse/', - edit_uri='edit/master/docs/%s' % lang, - extra_css=['assets/stylesheets/custom.css?%s' % args.rev_short], + edit_uri=f'edit/master/docs/{lang}', + extra_css=[f'assets/stylesheets/custom.css?{args.rev_short}'], markdown_extensions=[ 'mdx_clickhouse', 'admonition', @@ -190,9 +190,9 @@ def build_single_page_version(lang, args, cfg): if not args.skip_pdf: single_page_index_html = os.path.abspath(os.path.join(single_page_output_path, 'index.html')) - single_page_pdf = single_page_index_html.replace('index.html', 'clickhouse_%s.pdf' % lang) + single_page_pdf = single_page_index_html.replace('index.html', f'clickhouse_{lang}.pdf') create_pdf_command = ['wkhtmltopdf', '--print-media-type', single_page_index_html, single_page_pdf] - logging.debug(' '.join(create_pdf_command)) + logging.info(' '.join(create_pdf_command)) with open(os.devnull, 'w') as devnull: subprocess.check_call(' '.join(create_pdf_command), shell=True, stderr=devnull) @@ -242,7 +242,8 @@ def build_redirect_html(args, from_path, to_path): except OSError: pass version_prefix = args.version_prefix + '/' if args.version_prefix else '/' - to_url = '/docs%s%s/%s' % (version_prefix, lang, to_path.replace('.md', '/')) + target_path = to_path.replace('.md', '/') + to_url = f'/docs{version_prefix}{lang}/{target_path}' to_url = to_url.strip() write_redirect_html(out_path, to_url) @@ -336,7 +337,7 @@ if __name__ == '__main__': args.stable_releases = choose_latest_releases() if args.enable_stable_releases else [] args.rev = subprocess.check_output('git rev-parse HEAD', shell=True).decode('utf-8').strip() args.rev_short = subprocess.check_output('git rev-parse --short HEAD', shell=True).decode('utf-8').strip() - args.rev_url = 'https://github.com/ClickHouse/ClickHouse/commit/%s' % args.rev + args.rev_url = f'https://github.com/ClickHouse/ClickHouse/commit/{args.rev}' args.events = get_events(args) from build import build diff --git a/docs/tools/website.py b/docs/tools/website.py index 82bd0d2510f..60cc08a60f6 100644 --- a/docs/tools/website.py +++ b/docs/tools/website.py @@ -48,6 +48,9 @@ def build_website(args): for root, _, filenames in os.walk(args.output_dir): for filename in filenames: + if filename == 'main.html': + continue + path = os.path.join(root, filename) if not (filename.endswith('.html') or filename.endswith('.css') or filename.endswith('.js')): continue From b1ddbd247c5a4e95eb8d6d649ac8fe7089b8ac65 Mon Sep 17 00:00:00 2001 From: Denis Zhuravlev Date: Sun, 22 Mar 2020 09:13:19 -0300 Subject: [PATCH 0163/1355] Doc. Update `create table visits` with a new syntax. (#9801) --- docs/en/getting_started/example_datasets/metrica.md | 4 ++-- docs/es/getting_started/example_datasets/metrica.md | 4 ++-- docs/fa/getting_started/example_datasets/metrica.md | 4 ++-- docs/ru/getting_started/example_datasets/metrica.md | 2 +- 4 files changed, 7 insertions(+), 7 deletions(-) diff --git a/docs/en/getting_started/example_datasets/metrica.md b/docs/en/getting_started/example_datasets/metrica.md index fb6d3ae0a69..e855f4b98a3 100644 --- a/docs/en/getting_started/example_datasets/metrica.md +++ b/docs/en/getting_started/example_datasets/metrica.md @@ -2,7 +2,7 @@ Dataset consists of two tables containing anonymized data about hits (`hits_v1`) and visits (`visits_v1`) of Yandex.Metrica. You can read more about Yandex.Metrica in [ClickHouse history](../../introduction/history.md) section. -The dataset consists of two tables, either of them can be downloaded as a compressed `tsv.xz` file or as prepared partitions. In addition to that, an extended version of the `hits` table containing 100 million rows is available as TSV at https://clickhouse-datasets.s3.yandex.net/hits/tsv/hits\_100m\_obfuscated\_v1.tsv.xz and as prepared partitions at https://clickhouse-datasets.s3.yandex.net/hits/partitions/hits\_100m\_obfuscated\_v1.tar.xz. +The dataset consists of two tables, either of them can be downloaded as a compressed `tsv.xz` file or as prepared partitions. In addition to that, an extended version of the `hits` table containing 100 million rows is available as TSV at https://clickhouse-datasets.s3.yandex.net/hits/tsv/hits_100m_obfuscated_v1.tsv.xz and as prepared partitions at https://clickhouse-datasets.s3.yandex.net/hits/partitions/hits_100m_obfuscated_v1.tar.xz. ## Obtaining Tables from Prepared Partitions {#obtaining-tables-from-prepared-partitions} @@ -48,7 +48,7 @@ Download and import visits from compressed tsv-file: curl https://clickhouse-datasets.s3.yandex.net/visits/tsv/visits_v1.tsv.xz | unxz --threads=`nproc` > visits_v1.tsv # now create table clickhouse-client --query "CREATE DATABASE IF NOT EXISTS datasets" -clickhouse-client --query "CREATE TABLE datasets.visits_v1 ( CounterID UInt32, StartDate Date, Sign Int8, IsNew UInt8, VisitID UInt64, UserID UInt64, StartTime DateTime, Duration UInt32, UTCStartTime DateTime, PageViews Int32, Hits Int32, IsBounce UInt8, Referer String, StartURL String, RefererDomain String, StartURLDomain String, EndURL String, LinkURL String, IsDownload UInt8, TraficSourceID Int8, SearchEngineID UInt16, SearchPhrase String, AdvEngineID UInt8, PlaceID Int32, RefererCategories Array(UInt16), URLCategories Array(UInt16), URLRegions Array(UInt32), RefererRegions Array(UInt32), IsYandex UInt8, GoalReachesDepth Int32, GoalReachesURL Int32, GoalReachesAny Int32, SocialSourceNetworkID UInt8, SocialSourcePage String, MobilePhoneModel String, ClientEventTime DateTime, RegionID UInt32, ClientIP UInt32, ClientIP6 FixedString(16), RemoteIP UInt32, RemoteIP6 FixedString(16), IPNetworkID UInt32, SilverlightVersion3 UInt32, CodeVersion UInt32, ResolutionWidth UInt16, ResolutionHeight UInt16, UserAgentMajor UInt16, UserAgentMinor UInt16, WindowClientWidth UInt16, WindowClientHeight UInt16, SilverlightVersion2 UInt8, SilverlightVersion4 UInt16, FlashVersion3 UInt16, FlashVersion4 UInt16, ClientTimeZone Int16, OS UInt8, UserAgent UInt8, ResolutionDepth UInt8, FlashMajor UInt8, FlashMinor UInt8, NetMajor UInt8, NetMinor UInt8, MobilePhone UInt8, SilverlightVersion1 UInt8, Age UInt8, Sex UInt8, Income UInt8, JavaEnable UInt8, CookieEnable UInt8, JavascriptEnable UInt8, IsMobile UInt8, BrowserLanguage UInt16, BrowserCountry UInt16, Interests UInt16, Robotness UInt8, GeneralInterests Array(UInt16), Params Array(String), Goals Nested(ID UInt32, Serial UInt32, EventTime DateTime, Price Int64, OrderID String, CurrencyID UInt32), WatchIDs Array(UInt64), ParamSumPrice Int64, ParamCurrency FixedString(3), ParamCurrencyID UInt16, ClickLogID UInt64, ClickEventID Int32, ClickGoodEvent Int32, ClickEventTime DateTime, ClickPriorityID Int32, ClickPhraseID Int32, ClickPageID Int32, ClickPlaceID Int32, ClickTypeID Int32, ClickResourceID Int32, ClickCost UInt32, ClickClientIP UInt32, ClickDomainID UInt32, ClickURL String, ClickAttempt UInt8, ClickOrderID UInt32, ClickBannerID UInt32, ClickMarketCategoryID UInt32, ClickMarketPP UInt32, ClickMarketCategoryName String, ClickMarketPPName String, ClickAWAPSCampaignName String, ClickPageName String, ClickTargetType UInt16, ClickTargetPhraseID UInt64, ClickContextType UInt8, ClickSelectType Int8, ClickOptions String, ClickGroupBannerID Int32, OpenstatServiceName String, OpenstatCampaignID String, OpenstatAdID String, OpenstatSourceID String, UTMSource String, UTMMedium String, UTMCampaign String, UTMContent String, UTMTerm String, FromTag String, HasGCLID UInt8, FirstVisit DateTime, PredLastVisit Date, LastVisit Date, TotalVisits UInt32, TraficSource Nested(ID Int8, SearchEngineID UInt16, AdvEngineID UInt8, PlaceID UInt16, SocialSourceNetworkID UInt8, Domain String, SearchPhrase String, SocialSourcePage String), Attendance FixedString(16), CLID UInt32, YCLID UInt64, NormalizedRefererHash UInt64, SearchPhraseHash UInt64, RefererDomainHash UInt64, NormalizedStartURLHash UInt64, StartURLDomainHash UInt64, NormalizedEndURLHash UInt64, TopLevelDomain UInt64, URLScheme UInt64, OpenstatServiceNameHash UInt64, OpenstatCampaignIDHash UInt64, OpenstatAdIDHash UInt64, OpenstatSourceIDHash UInt64, UTMSourceHash UInt64, UTMMediumHash UInt64, UTMCampaignHash UInt64, UTMContentHash UInt64, UTMTermHash UInt64, FromHash UInt64, WebVisorEnabled UInt8, WebVisorActivity UInt32, ParsedParams Nested(Key1 String, Key2 String, Key3 String, Key4 String, Key5 String, ValueDouble Float64), Market Nested(Type UInt8, GoalID UInt32, OrderID String, OrderPrice Int64, PP UInt32, DirectPlaceID UInt32, DirectOrderID UInt32, DirectBannerID UInt32, GoodID String, GoodName String, GoodQuantity Int32, GoodPrice Int64), IslandID FixedString(16)) ENGINE = CollapsingMergeTree(StartDate, intHash32(UserID), (CounterID, StartDate, intHash32(UserID), VisitID), 8192, Sign)" +clickhouse-client --query "CREATE TABLE datasets.visits_v1 ( CounterID UInt32, StartDate Date, Sign Int8, IsNew UInt8, VisitID UInt64, UserID UInt64, StartTime DateTime, Duration UInt32, UTCStartTime DateTime, PageViews Int32, Hits Int32, IsBounce UInt8, Referer String, StartURL String, RefererDomain String, StartURLDomain String, EndURL String, LinkURL String, IsDownload UInt8, TraficSourceID Int8, SearchEngineID UInt16, SearchPhrase String, AdvEngineID UInt8, PlaceID Int32, RefererCategories Array(UInt16), URLCategories Array(UInt16), URLRegions Array(UInt32), RefererRegions Array(UInt32), IsYandex UInt8, GoalReachesDepth Int32, GoalReachesURL Int32, GoalReachesAny Int32, SocialSourceNetworkID UInt8, SocialSourcePage String, MobilePhoneModel String, ClientEventTime DateTime, RegionID UInt32, ClientIP UInt32, ClientIP6 FixedString(16), RemoteIP UInt32, RemoteIP6 FixedString(16), IPNetworkID UInt32, SilverlightVersion3 UInt32, CodeVersion UInt32, ResolutionWidth UInt16, ResolutionHeight UInt16, UserAgentMajor UInt16, UserAgentMinor UInt16, WindowClientWidth UInt16, WindowClientHeight UInt16, SilverlightVersion2 UInt8, SilverlightVersion4 UInt16, FlashVersion3 UInt16, FlashVersion4 UInt16, ClientTimeZone Int16, OS UInt8, UserAgent UInt8, ResolutionDepth UInt8, FlashMajor UInt8, FlashMinor UInt8, NetMajor UInt8, NetMinor UInt8, MobilePhone UInt8, SilverlightVersion1 UInt8, Age UInt8, Sex UInt8, Income UInt8, JavaEnable UInt8, CookieEnable UInt8, JavascriptEnable UInt8, IsMobile UInt8, BrowserLanguage UInt16, BrowserCountry UInt16, Interests UInt16, Robotness UInt8, GeneralInterests Array(UInt16), Params Array(String), Goals Nested(ID UInt32, Serial UInt32, EventTime DateTime, Price Int64, OrderID String, CurrencyID UInt32), WatchIDs Array(UInt64), ParamSumPrice Int64, ParamCurrency FixedString(3), ParamCurrencyID UInt16, ClickLogID UInt64, ClickEventID Int32, ClickGoodEvent Int32, ClickEventTime DateTime, ClickPriorityID Int32, ClickPhraseID Int32, ClickPageID Int32, ClickPlaceID Int32, ClickTypeID Int32, ClickResourceID Int32, ClickCost UInt32, ClickClientIP UInt32, ClickDomainID UInt32, ClickURL String, ClickAttempt UInt8, ClickOrderID UInt32, ClickBannerID UInt32, ClickMarketCategoryID UInt32, ClickMarketPP UInt32, ClickMarketCategoryName String, ClickMarketPPName String, ClickAWAPSCampaignName String, ClickPageName String, ClickTargetType UInt16, ClickTargetPhraseID UInt64, ClickContextType UInt8, ClickSelectType Int8, ClickOptions String, ClickGroupBannerID Int32, OpenstatServiceName String, OpenstatCampaignID String, OpenstatAdID String, OpenstatSourceID String, UTMSource String, UTMMedium String, UTMCampaign String, UTMContent String, UTMTerm String, FromTag String, HasGCLID UInt8, FirstVisit DateTime, PredLastVisit Date, LastVisit Date, TotalVisits UInt32, TraficSource Nested(ID Int8, SearchEngineID UInt16, AdvEngineID UInt8, PlaceID UInt16, SocialSourceNetworkID UInt8, Domain String, SearchPhrase String, SocialSourcePage String), Attendance FixedString(16), CLID UInt32, YCLID UInt64, NormalizedRefererHash UInt64, SearchPhraseHash UInt64, RefererDomainHash UInt64, NormalizedStartURLHash UInt64, StartURLDomainHash UInt64, NormalizedEndURLHash UInt64, TopLevelDomain UInt64, URLScheme UInt64, OpenstatServiceNameHash UInt64, OpenstatCampaignIDHash UInt64, OpenstatAdIDHash UInt64, OpenstatSourceIDHash UInt64, UTMSourceHash UInt64, UTMMediumHash UInt64, UTMCampaignHash UInt64, UTMContentHash UInt64, UTMTermHash UInt64, FromHash UInt64, WebVisorEnabled UInt8, WebVisorActivity UInt32, ParsedParams Nested(Key1 String, Key2 String, Key3 String, Key4 String, Key5 String, ValueDouble Float64), Market Nested(Type UInt8, GoalID UInt32, OrderID String, OrderPrice Int64, PP UInt32, DirectPlaceID UInt32, DirectOrderID UInt32, DirectBannerID UInt32, GoodID String, GoodName String, GoodQuantity Int32, GoodPrice Int64), IslandID FixedString(16)) ENGINE = CollapsingMergeTree(Sign) PARTITION BY toYYYYMM(StartDate) ORDER BY (CounterID, StartDate, intHash32(UserID), VisitID) SAMPLE BY intHash32(UserID) SETTINGS index_granularity = 8192" # import data cat visits_v1.tsv | clickhouse-client --query "INSERT INTO datasets.visits_v1 FORMAT TSV" --max_insert_block_size=100000 # optionally you can optimize table diff --git a/docs/es/getting_started/example_datasets/metrica.md b/docs/es/getting_started/example_datasets/metrica.md index bb359351043..187d0724862 100644 --- a/docs/es/getting_started/example_datasets/metrica.md +++ b/docs/es/getting_started/example_datasets/metrica.md @@ -2,7 +2,7 @@ El conjunto de datos consta de dos tablas que contienen datos anónimos sobre los hits (`hits_v1`) y visitas (`visits_v1`) el Yandex.Métrica. Puedes leer más sobre Yandex.Metrica en [Historial de ClickHouse](../../introduction/history.md) apartado. -El conjunto de datos consta de dos tablas, cualquiera de ellas se puede descargar como `tsv.xz` o como particiones preparadas. Además, una versión extendida de la `hits` La tabla que contiene 100 millones de filas está disponible como TSV en https://clickhouse-datasets.s3.yandex.net/hits/tsv/hits\_100m\_obfuscated\_v1.tsv.xz y como particiones preparadas en https://clickhouse-datasets.s3.yandex.net/hits/partitions/hits\_100m\_obfuscated\_v1.tar.xz. +El conjunto de datos consta de dos tablas, cualquiera de ellas se puede descargar como `tsv.xz` o como particiones preparadas. Además, una versión extendida de la `hits` La tabla que contiene 100 millones de filas está disponible como TSV en https://clickhouse-datasets.s3.yandex.net/hits/tsv/hits_100m_obfuscated_v1.tsv.xz y como particiones preparadas en https://clickhouse-datasets.s3.yandex.net/hits/partitions/hits_100m_obfuscated\_v1.tar.xz. ## Obtención de tablas a partir de particiones preparadas {#obtaining-tables-from-prepared-partitions} @@ -48,7 +48,7 @@ Descargue e importe visitas desde un archivo tsv comprimido: curl https://clickhouse-datasets.s3.yandex.net/visits/tsv/visits_v1.tsv.xz | unxz --threads=`nproc` > visits_v1.tsv # now create table clickhouse-client --query "CREATE DATABASE IF NOT EXISTS datasets" -clickhouse-client --query "CREATE TABLE datasets.visits_v1 ( CounterID UInt32, StartDate Date, Sign Int8, IsNew UInt8, VisitID UInt64, UserID UInt64, StartTime DateTime, Duration UInt32, UTCStartTime DateTime, PageViews Int32, Hits Int32, IsBounce UInt8, Referer String, StartURL String, RefererDomain String, StartURLDomain String, EndURL String, LinkURL String, IsDownload UInt8, TraficSourceID Int8, SearchEngineID UInt16, SearchPhrase String, AdvEngineID UInt8, PlaceID Int32, RefererCategories Array(UInt16), URLCategories Array(UInt16), URLRegions Array(UInt32), RefererRegions Array(UInt32), IsYandex UInt8, GoalReachesDepth Int32, GoalReachesURL Int32, GoalReachesAny Int32, SocialSourceNetworkID UInt8, SocialSourcePage String, MobilePhoneModel String, ClientEventTime DateTime, RegionID UInt32, ClientIP UInt32, ClientIP6 FixedString(16), RemoteIP UInt32, RemoteIP6 FixedString(16), IPNetworkID UInt32, SilverlightVersion3 UInt32, CodeVersion UInt32, ResolutionWidth UInt16, ResolutionHeight UInt16, UserAgentMajor UInt16, UserAgentMinor UInt16, WindowClientWidth UInt16, WindowClientHeight UInt16, SilverlightVersion2 UInt8, SilverlightVersion4 UInt16, FlashVersion3 UInt16, FlashVersion4 UInt16, ClientTimeZone Int16, OS UInt8, UserAgent UInt8, ResolutionDepth UInt8, FlashMajor UInt8, FlashMinor UInt8, NetMajor UInt8, NetMinor UInt8, MobilePhone UInt8, SilverlightVersion1 UInt8, Age UInt8, Sex UInt8, Income UInt8, JavaEnable UInt8, CookieEnable UInt8, JavascriptEnable UInt8, IsMobile UInt8, BrowserLanguage UInt16, BrowserCountry UInt16, Interests UInt16, Robotness UInt8, GeneralInterests Array(UInt16), Params Array(String), Goals Nested(ID UInt32, Serial UInt32, EventTime DateTime, Price Int64, OrderID String, CurrencyID UInt32), WatchIDs Array(UInt64), ParamSumPrice Int64, ParamCurrency FixedString(3), ParamCurrencyID UInt16, ClickLogID UInt64, ClickEventID Int32, ClickGoodEvent Int32, ClickEventTime DateTime, ClickPriorityID Int32, ClickPhraseID Int32, ClickPageID Int32, ClickPlaceID Int32, ClickTypeID Int32, ClickResourceID Int32, ClickCost UInt32, ClickClientIP UInt32, ClickDomainID UInt32, ClickURL String, ClickAttempt UInt8, ClickOrderID UInt32, ClickBannerID UInt32, ClickMarketCategoryID UInt32, ClickMarketPP UInt32, ClickMarketCategoryName String, ClickMarketPPName String, ClickAWAPSCampaignName String, ClickPageName String, ClickTargetType UInt16, ClickTargetPhraseID UInt64, ClickContextType UInt8, ClickSelectType Int8, ClickOptions String, ClickGroupBannerID Int32, OpenstatServiceName String, OpenstatCampaignID String, OpenstatAdID String, OpenstatSourceID String, UTMSource String, UTMMedium String, UTMCampaign String, UTMContent String, UTMTerm String, FromTag String, HasGCLID UInt8, FirstVisit DateTime, PredLastVisit Date, LastVisit Date, TotalVisits UInt32, TraficSource Nested(ID Int8, SearchEngineID UInt16, AdvEngineID UInt8, PlaceID UInt16, SocialSourceNetworkID UInt8, Domain String, SearchPhrase String, SocialSourcePage String), Attendance FixedString(16), CLID UInt32, YCLID UInt64, NormalizedRefererHash UInt64, SearchPhraseHash UInt64, RefererDomainHash UInt64, NormalizedStartURLHash UInt64, StartURLDomainHash UInt64, NormalizedEndURLHash UInt64, TopLevelDomain UInt64, URLScheme UInt64, OpenstatServiceNameHash UInt64, OpenstatCampaignIDHash UInt64, OpenstatAdIDHash UInt64, OpenstatSourceIDHash UInt64, UTMSourceHash UInt64, UTMMediumHash UInt64, UTMCampaignHash UInt64, UTMContentHash UInt64, UTMTermHash UInt64, FromHash UInt64, WebVisorEnabled UInt8, WebVisorActivity UInt32, ParsedParams Nested(Key1 String, Key2 String, Key3 String, Key4 String, Key5 String, ValueDouble Float64), Market Nested(Type UInt8, GoalID UInt32, OrderID String, OrderPrice Int64, PP UInt32, DirectPlaceID UInt32, DirectOrderID UInt32, DirectBannerID UInt32, GoodID String, GoodName String, GoodQuantity Int32, GoodPrice Int64), IslandID FixedString(16)) ENGINE = CollapsingMergeTree(StartDate, intHash32(UserID), (CounterID, StartDate, intHash32(UserID), VisitID), 8192, Sign)" +clickhouse-client --query "CREATE TABLE datasets.visits_v1 ( CounterID UInt32, StartDate Date, Sign Int8, IsNew UInt8, VisitID UInt64, UserID UInt64, StartTime DateTime, Duration UInt32, UTCStartTime DateTime, PageViews Int32, Hits Int32, IsBounce UInt8, Referer String, StartURL String, RefererDomain String, StartURLDomain String, EndURL String, LinkURL String, IsDownload UInt8, TraficSourceID Int8, SearchEngineID UInt16, SearchPhrase String, AdvEngineID UInt8, PlaceID Int32, RefererCategories Array(UInt16), URLCategories Array(UInt16), URLRegions Array(UInt32), RefererRegions Array(UInt32), IsYandex UInt8, GoalReachesDepth Int32, GoalReachesURL Int32, GoalReachesAny Int32, SocialSourceNetworkID UInt8, SocialSourcePage String, MobilePhoneModel String, ClientEventTime DateTime, RegionID UInt32, ClientIP UInt32, ClientIP6 FixedString(16), RemoteIP UInt32, RemoteIP6 FixedString(16), IPNetworkID UInt32, SilverlightVersion3 UInt32, CodeVersion UInt32, ResolutionWidth UInt16, ResolutionHeight UInt16, UserAgentMajor UInt16, UserAgentMinor UInt16, WindowClientWidth UInt16, WindowClientHeight UInt16, SilverlightVersion2 UInt8, SilverlightVersion4 UInt16, FlashVersion3 UInt16, FlashVersion4 UInt16, ClientTimeZone Int16, OS UInt8, UserAgent UInt8, ResolutionDepth UInt8, FlashMajor UInt8, FlashMinor UInt8, NetMajor UInt8, NetMinor UInt8, MobilePhone UInt8, SilverlightVersion1 UInt8, Age UInt8, Sex UInt8, Income UInt8, JavaEnable UInt8, CookieEnable UInt8, JavascriptEnable UInt8, IsMobile UInt8, BrowserLanguage UInt16, BrowserCountry UInt16, Interests UInt16, Robotness UInt8, GeneralInterests Array(UInt16), Params Array(String), Goals Nested(ID UInt32, Serial UInt32, EventTime DateTime, Price Int64, OrderID String, CurrencyID UInt32), WatchIDs Array(UInt64), ParamSumPrice Int64, ParamCurrency FixedString(3), ParamCurrencyID UInt16, ClickLogID UInt64, ClickEventID Int32, ClickGoodEvent Int32, ClickEventTime DateTime, ClickPriorityID Int32, ClickPhraseID Int32, ClickPageID Int32, ClickPlaceID Int32, ClickTypeID Int32, ClickResourceID Int32, ClickCost UInt32, ClickClientIP UInt32, ClickDomainID UInt32, ClickURL String, ClickAttempt UInt8, ClickOrderID UInt32, ClickBannerID UInt32, ClickMarketCategoryID UInt32, ClickMarketPP UInt32, ClickMarketCategoryName String, ClickMarketPPName String, ClickAWAPSCampaignName String, ClickPageName String, ClickTargetType UInt16, ClickTargetPhraseID UInt64, ClickContextType UInt8, ClickSelectType Int8, ClickOptions String, ClickGroupBannerID Int32, OpenstatServiceName String, OpenstatCampaignID String, OpenstatAdID String, OpenstatSourceID String, UTMSource String, UTMMedium String, UTMCampaign String, UTMContent String, UTMTerm String, FromTag String, HasGCLID UInt8, FirstVisit DateTime, PredLastVisit Date, LastVisit Date, TotalVisits UInt32, TraficSource Nested(ID Int8, SearchEngineID UInt16, AdvEngineID UInt8, PlaceID UInt16, SocialSourceNetworkID UInt8, Domain String, SearchPhrase String, SocialSourcePage String), Attendance FixedString(16), CLID UInt32, YCLID UInt64, NormalizedRefererHash UInt64, SearchPhraseHash UInt64, RefererDomainHash UInt64, NormalizedStartURLHash UInt64, StartURLDomainHash UInt64, NormalizedEndURLHash UInt64, TopLevelDomain UInt64, URLScheme UInt64, OpenstatServiceNameHash UInt64, OpenstatCampaignIDHash UInt64, OpenstatAdIDHash UInt64, OpenstatSourceIDHash UInt64, UTMSourceHash UInt64, UTMMediumHash UInt64, UTMCampaignHash UInt64, UTMContentHash UInt64, UTMTermHash UInt64, FromHash UInt64, WebVisorEnabled UInt8, WebVisorActivity UInt32, ParsedParams Nested(Key1 String, Key2 String, Key3 String, Key4 String, Key5 String, ValueDouble Float64), Market Nested(Type UInt8, GoalID UInt32, OrderID String, OrderPrice Int64, PP UInt32, DirectPlaceID UInt32, DirectOrderID UInt32, DirectBannerID UInt32, GoodID String, GoodName String, GoodQuantity Int32, GoodPrice Int64), IslandID FixedString(16)) ENGINE = CollapsingMergeTree(Sign) PARTITION BY toYYYYMM(StartDate) ORDER BY (CounterID, StartDate, intHash32(UserID), VisitID) SAMPLE BY intHash32(UserID) SETTINGS index_granularity = 8192" # import data cat visits_v1.tsv | clickhouse-client --query "INSERT INTO datasets.visits_v1 FORMAT TSV" --max_insert_block_size=100000 # optionally you can optimize table diff --git a/docs/fa/getting_started/example_datasets/metrica.md b/docs/fa/getting_started/example_datasets/metrica.md index 47b9751d513..1546e4f79d4 100644 --- a/docs/fa/getting_started/example_datasets/metrica.md +++ b/docs/fa/getting_started/example_datasets/metrica.md @@ -2,7 +2,7 @@ مجموعه داده شامل دو جدول حاوی داده های ناشناس در مورد بازدید (`hits_v1`) و بازدیدکننده داشته است (`visits_v1`) یاندکس . متریکا شما می توانید اطلاعات بیشتر در مورد یاندکس به عنوان خوانده شده.متریکا در [تاریخچه کلیک](../../introduction/history.md) بخش. -مجموعه داده ها شامل دو جدول است که هر کدام می توانند به عنوان یک فشرده دانلود شوند `tsv.xz` فایل و یا به عنوان پارتیشن تهیه شده است. علاوه بر این, یک نسخه طولانی از `hits` جدول حاوی 100 میلیون ردیف به عنوان تسو در دسترس است https://clickhouse-datasets.s3.yandex.net/hits/tsv/hits\_100m\_obfuscated\_v1.tsv.xz و به عنوان پارتیشن تهیه شده در https://clickhouse-datasets.s3.yandex.net/hits/partitions/hits\_100m\_obfuscated\_v1.tar.xz. +مجموعه داده ها شامل دو جدول است که هر کدام می توانند به عنوان یک فشرده دانلود شوند `tsv.xz` فایل و یا به عنوان پارتیشن تهیه شده است. علاوه بر این, یک نسخه طولانی از `hits` جدول حاوی 100 میلیون ردیف به عنوان تسو در دسترس است https://clickhouse-datasets.s3.yandex.net/hits/tsv/hits_100m_obfuscated_v1.tsv.xz و به عنوان پارتیشن تهیه شده در https://clickhouse-datasets.s3.yandex.net/hits/partitions/hits_100m_obfuscated_v1.tar.xz. ## اخذ جداول از پارتیشن های تهیه شده {#khdh-jdwl-z-prtyshn-hy-thyh-shdh} @@ -48,7 +48,7 @@ clickhouse-client --query "SELECT COUNT(*) FROM datasets.hits_v1" curl https://clickhouse-datasets.s3.yandex.net/visits/tsv/visits_v1.tsv.xz | unxz --threads=`nproc` > visits_v1.tsv # now create table clickhouse-client --query "CREATE DATABASE IF NOT EXISTS datasets" -clickhouse-client --query "CREATE TABLE datasets.visits_v1 ( CounterID UInt32, StartDate Date, Sign Int8, IsNew UInt8, VisitID UInt64, UserID UInt64, StartTime DateTime, Duration UInt32, UTCStartTime DateTime, PageViews Int32, Hits Int32, IsBounce UInt8, Referer String, StartURL String, RefererDomain String, StartURLDomain String, EndURL String, LinkURL String, IsDownload UInt8, TraficSourceID Int8, SearchEngineID UInt16, SearchPhrase String, AdvEngineID UInt8, PlaceID Int32, RefererCategories Array(UInt16), URLCategories Array(UInt16), URLRegions Array(UInt32), RefererRegions Array(UInt32), IsYandex UInt8, GoalReachesDepth Int32, GoalReachesURL Int32, GoalReachesAny Int32, SocialSourceNetworkID UInt8, SocialSourcePage String, MobilePhoneModel String, ClientEventTime DateTime, RegionID UInt32, ClientIP UInt32, ClientIP6 FixedString(16), RemoteIP UInt32, RemoteIP6 FixedString(16), IPNetworkID UInt32, SilverlightVersion3 UInt32, CodeVersion UInt32, ResolutionWidth UInt16, ResolutionHeight UInt16, UserAgentMajor UInt16, UserAgentMinor UInt16, WindowClientWidth UInt16, WindowClientHeight UInt16, SilverlightVersion2 UInt8, SilverlightVersion4 UInt16, FlashVersion3 UInt16, FlashVersion4 UInt16, ClientTimeZone Int16, OS UInt8, UserAgent UInt8, ResolutionDepth UInt8, FlashMajor UInt8, FlashMinor UInt8, NetMajor UInt8, NetMinor UInt8, MobilePhone UInt8, SilverlightVersion1 UInt8, Age UInt8, Sex UInt8, Income UInt8, JavaEnable UInt8, CookieEnable UInt8, JavascriptEnable UInt8, IsMobile UInt8, BrowserLanguage UInt16, BrowserCountry UInt16, Interests UInt16, Robotness UInt8, GeneralInterests Array(UInt16), Params Array(String), Goals Nested(ID UInt32, Serial UInt32, EventTime DateTime, Price Int64, OrderID String, CurrencyID UInt32), WatchIDs Array(UInt64), ParamSumPrice Int64, ParamCurrency FixedString(3), ParamCurrencyID UInt16, ClickLogID UInt64, ClickEventID Int32, ClickGoodEvent Int32, ClickEventTime DateTime, ClickPriorityID Int32, ClickPhraseID Int32, ClickPageID Int32, ClickPlaceID Int32, ClickTypeID Int32, ClickResourceID Int32, ClickCost UInt32, ClickClientIP UInt32, ClickDomainID UInt32, ClickURL String, ClickAttempt UInt8, ClickOrderID UInt32, ClickBannerID UInt32, ClickMarketCategoryID UInt32, ClickMarketPP UInt32, ClickMarketCategoryName String, ClickMarketPPName String, ClickAWAPSCampaignName String, ClickPageName String, ClickTargetType UInt16, ClickTargetPhraseID UInt64, ClickContextType UInt8, ClickSelectType Int8, ClickOptions String, ClickGroupBannerID Int32, OpenstatServiceName String, OpenstatCampaignID String, OpenstatAdID String, OpenstatSourceID String, UTMSource String, UTMMedium String, UTMCampaign String, UTMContent String, UTMTerm String, FromTag String, HasGCLID UInt8, FirstVisit DateTime, PredLastVisit Date, LastVisit Date, TotalVisits UInt32, TraficSource Nested(ID Int8, SearchEngineID UInt16, AdvEngineID UInt8, PlaceID UInt16, SocialSourceNetworkID UInt8, Domain String, SearchPhrase String, SocialSourcePage String), Attendance FixedString(16), CLID UInt32, YCLID UInt64, NormalizedRefererHash UInt64, SearchPhraseHash UInt64, RefererDomainHash UInt64, NormalizedStartURLHash UInt64, StartURLDomainHash UInt64, NormalizedEndURLHash UInt64, TopLevelDomain UInt64, URLScheme UInt64, OpenstatServiceNameHash UInt64, OpenstatCampaignIDHash UInt64, OpenstatAdIDHash UInt64, OpenstatSourceIDHash UInt64, UTMSourceHash UInt64, UTMMediumHash UInt64, UTMCampaignHash UInt64, UTMContentHash UInt64, UTMTermHash UInt64, FromHash UInt64, WebVisorEnabled UInt8, WebVisorActivity UInt32, ParsedParams Nested(Key1 String, Key2 String, Key3 String, Key4 String, Key5 String, ValueDouble Float64), Market Nested(Type UInt8, GoalID UInt32, OrderID String, OrderPrice Int64, PP UInt32, DirectPlaceID UInt32, DirectOrderID UInt32, DirectBannerID UInt32, GoodID String, GoodName String, GoodQuantity Int32, GoodPrice Int64), IslandID FixedString(16)) ENGINE = CollapsingMergeTree(StartDate, intHash32(UserID), (CounterID, StartDate, intHash32(UserID), VisitID), 8192, Sign)" +clickhouse-client --query "CREATE TABLE datasets.visits_v1 ( CounterID UInt32, StartDate Date, Sign Int8, IsNew UInt8, VisitID UInt64, UserID UInt64, StartTime DateTime, Duration UInt32, UTCStartTime DateTime, PageViews Int32, Hits Int32, IsBounce UInt8, Referer String, StartURL String, RefererDomain String, StartURLDomain String, EndURL String, LinkURL String, IsDownload UInt8, TraficSourceID Int8, SearchEngineID UInt16, SearchPhrase String, AdvEngineID UInt8, PlaceID Int32, RefererCategories Array(UInt16), URLCategories Array(UInt16), URLRegions Array(UInt32), RefererRegions Array(UInt32), IsYandex UInt8, GoalReachesDepth Int32, GoalReachesURL Int32, GoalReachesAny Int32, SocialSourceNetworkID UInt8, SocialSourcePage String, MobilePhoneModel String, ClientEventTime DateTime, RegionID UInt32, ClientIP UInt32, ClientIP6 FixedString(16), RemoteIP UInt32, RemoteIP6 FixedString(16), IPNetworkID UInt32, SilverlightVersion3 UInt32, CodeVersion UInt32, ResolutionWidth UInt16, ResolutionHeight UInt16, UserAgentMajor UInt16, UserAgentMinor UInt16, WindowClientWidth UInt16, WindowClientHeight UInt16, SilverlightVersion2 UInt8, SilverlightVersion4 UInt16, FlashVersion3 UInt16, FlashVersion4 UInt16, ClientTimeZone Int16, OS UInt8, UserAgent UInt8, ResolutionDepth UInt8, FlashMajor UInt8, FlashMinor UInt8, NetMajor UInt8, NetMinor UInt8, MobilePhone UInt8, SilverlightVersion1 UInt8, Age UInt8, Sex UInt8, Income UInt8, JavaEnable UInt8, CookieEnable UInt8, JavascriptEnable UInt8, IsMobile UInt8, BrowserLanguage UInt16, BrowserCountry UInt16, Interests UInt16, Robotness UInt8, GeneralInterests Array(UInt16), Params Array(String), Goals Nested(ID UInt32, Serial UInt32, EventTime DateTime, Price Int64, OrderID String, CurrencyID UInt32), WatchIDs Array(UInt64), ParamSumPrice Int64, ParamCurrency FixedString(3), ParamCurrencyID UInt16, ClickLogID UInt64, ClickEventID Int32, ClickGoodEvent Int32, ClickEventTime DateTime, ClickPriorityID Int32, ClickPhraseID Int32, ClickPageID Int32, ClickPlaceID Int32, ClickTypeID Int32, ClickResourceID Int32, ClickCost UInt32, ClickClientIP UInt32, ClickDomainID UInt32, ClickURL String, ClickAttempt UInt8, ClickOrderID UInt32, ClickBannerID UInt32, ClickMarketCategoryID UInt32, ClickMarketPP UInt32, ClickMarketCategoryName String, ClickMarketPPName String, ClickAWAPSCampaignName String, ClickPageName String, ClickTargetType UInt16, ClickTargetPhraseID UInt64, ClickContextType UInt8, ClickSelectType Int8, ClickOptions String, ClickGroupBannerID Int32, OpenstatServiceName String, OpenstatCampaignID String, OpenstatAdID String, OpenstatSourceID String, UTMSource String, UTMMedium String, UTMCampaign String, UTMContent String, UTMTerm String, FromTag String, HasGCLID UInt8, FirstVisit DateTime, PredLastVisit Date, LastVisit Date, TotalVisits UInt32, TraficSource Nested(ID Int8, SearchEngineID UInt16, AdvEngineID UInt8, PlaceID UInt16, SocialSourceNetworkID UInt8, Domain String, SearchPhrase String, SocialSourcePage String), Attendance FixedString(16), CLID UInt32, YCLID UInt64, NormalizedRefererHash UInt64, SearchPhraseHash UInt64, RefererDomainHash UInt64, NormalizedStartURLHash UInt64, StartURLDomainHash UInt64, NormalizedEndURLHash UInt64, TopLevelDomain UInt64, URLScheme UInt64, OpenstatServiceNameHash UInt64, OpenstatCampaignIDHash UInt64, OpenstatAdIDHash UInt64, OpenstatSourceIDHash UInt64, UTMSourceHash UInt64, UTMMediumHash UInt64, UTMCampaignHash UInt64, UTMContentHash UInt64, UTMTermHash UInt64, FromHash UInt64, WebVisorEnabled UInt8, WebVisorActivity UInt32, ParsedParams Nested(Key1 String, Key2 String, Key3 String, Key4 String, Key5 String, ValueDouble Float64), Market Nested(Type UInt8, GoalID UInt32, OrderID String, OrderPrice Int64, PP UInt32, DirectPlaceID UInt32, DirectOrderID UInt32, DirectBannerID UInt32, GoodID String, GoodName String, GoodQuantity Int32, GoodPrice Int64), IslandID FixedString(16)) ENGINE = CollapsingMergeTree(Sign) PARTITION BY toYYYYMM(StartDate) ORDER BY (CounterID, StartDate, intHash32(UserID), VisitID) SAMPLE BY intHash32(UserID) SETTINGS index_granularity = 8192" # import data cat visits_v1.tsv | clickhouse-client --query "INSERT INTO datasets.visits_v1 FORMAT TSV" --max_insert_block_size=100000 # optionally you can optimize table diff --git a/docs/ru/getting_started/example_datasets/metrica.md b/docs/ru/getting_started/example_datasets/metrica.md index 7980d2936a2..00404d77c83 100644 --- a/docs/ru/getting_started/example_datasets/metrica.md +++ b/docs/ru/getting_started/example_datasets/metrica.md @@ -46,7 +46,7 @@ $ clickhouse-client --query "SELECT COUNT(*) FROM datasets.hits_v1" $ curl https://clickhouse-datasets.s3.yandex.net/visits/tsv/visits_v1.tsv.xz | unxz --threads=`nproc` > visits_v1.tsv $ # теперь создадим таблицу $ clickhouse-client --query "CREATE DATABASE IF NOT EXISTS datasets" -$ clickhouse-client --query "CREATE TABLE datasets.visits_v1 ( CounterID UInt32, StartDate Date, Sign Int8, IsNew UInt8, VisitID UInt64, UserID UInt64, StartTime DateTime, Duration UInt32, UTCStartTime DateTime, PageViews Int32, Hits Int32, IsBounce UInt8, Referer String, StartURL String, RefererDomain String, StartURLDomain String, EndURL String, LinkURL String, IsDownload UInt8, TraficSourceID Int8, SearchEngineID UInt16, SearchPhrase String, AdvEngineID UInt8, PlaceID Int32, RefererCategories Array(UInt16), URLCategories Array(UInt16), URLRegions Array(UInt32), RefererRegions Array(UInt32), IsYandex UInt8, GoalReachesDepth Int32, GoalReachesURL Int32, GoalReachesAny Int32, SocialSourceNetworkID UInt8, SocialSourcePage String, MobilePhoneModel String, ClientEventTime DateTime, RegionID UInt32, ClientIP UInt32, ClientIP6 FixedString(16), RemoteIP UInt32, RemoteIP6 FixedString(16), IPNetworkID UInt32, SilverlightVersion3 UInt32, CodeVersion UInt32, ResolutionWidth UInt16, ResolutionHeight UInt16, UserAgentMajor UInt16, UserAgentMinor UInt16, WindowClientWidth UInt16, WindowClientHeight UInt16, SilverlightVersion2 UInt8, SilverlightVersion4 UInt16, FlashVersion3 UInt16, FlashVersion4 UInt16, ClientTimeZone Int16, OS UInt8, UserAgent UInt8, ResolutionDepth UInt8, FlashMajor UInt8, FlashMinor UInt8, NetMajor UInt8, NetMinor UInt8, MobilePhone UInt8, SilverlightVersion1 UInt8, Age UInt8, Sex UInt8, Income UInt8, JavaEnable UInt8, CookieEnable UInt8, JavascriptEnable UInt8, IsMobile UInt8, BrowserLanguage UInt16, BrowserCountry UInt16, Interests UInt16, Robotness UInt8, GeneralInterests Array(UInt16), Params Array(String), Goals Nested(ID UInt32, Serial UInt32, EventTime DateTime, Price Int64, OrderID String, CurrencyID UInt32), WatchIDs Array(UInt64), ParamSumPrice Int64, ParamCurrency FixedString(3), ParamCurrencyID UInt16, ClickLogID UInt64, ClickEventID Int32, ClickGoodEvent Int32, ClickEventTime DateTime, ClickPriorityID Int32, ClickPhraseID Int32, ClickPageID Int32, ClickPlaceID Int32, ClickTypeID Int32, ClickResourceID Int32, ClickCost UInt32, ClickClientIP UInt32, ClickDomainID UInt32, ClickURL String, ClickAttempt UInt8, ClickOrderID UInt32, ClickBannerID UInt32, ClickMarketCategoryID UInt32, ClickMarketPP UInt32, ClickMarketCategoryName String, ClickMarketPPName String, ClickAWAPSCampaignName String, ClickPageName String, ClickTargetType UInt16, ClickTargetPhraseID UInt64, ClickContextType UInt8, ClickSelectType Int8, ClickOptions String, ClickGroupBannerID Int32, OpenstatServiceName String, OpenstatCampaignID String, OpenstatAdID String, OpenstatSourceID String, UTMSource String, UTMMedium String, UTMCampaign String, UTMContent String, UTMTerm String, FromTag String, HasGCLID UInt8, FirstVisit DateTime, PredLastVisit Date, LastVisit Date, TotalVisits UInt32, TraficSource Nested(ID Int8, SearchEngineID UInt16, AdvEngineID UInt8, PlaceID UInt16, SocialSourceNetworkID UInt8, Domain String, SearchPhrase String, SocialSourcePage String), Attendance FixedString(16), CLID UInt32, YCLID UInt64, NormalizedRefererHash UInt64, SearchPhraseHash UInt64, RefererDomainHash UInt64, NormalizedStartURLHash UInt64, StartURLDomainHash UInt64, NormalizedEndURLHash UInt64, TopLevelDomain UInt64, URLScheme UInt64, OpenstatServiceNameHash UInt64, OpenstatCampaignIDHash UInt64, OpenstatAdIDHash UInt64, OpenstatSourceIDHash UInt64, UTMSourceHash UInt64, UTMMediumHash UInt64, UTMCampaignHash UInt64, UTMContentHash UInt64, UTMTermHash UInt64, FromHash UInt64, WebVisorEnabled UInt8, WebVisorActivity UInt32, ParsedParams Nested(Key1 String, Key2 String, Key3 String, Key4 String, Key5 String, ValueDouble Float64), Market Nested(Type UInt8, GoalID UInt32, OrderID String, OrderPrice Int64, PP UInt32, DirectPlaceID UInt32, DirectOrderID UInt32, DirectBannerID UInt32, GoodID String, GoodName String, GoodQuantity Int32, GoodPrice Int64), IslandID FixedString(16)) ENGINE = CollapsingMergeTree(StartDate, intHash32(UserID), (CounterID, StartDate, intHash32(UserID), VisitID), 8192, Sign)" +$ clickhouse-client --query "CREATE TABLE datasets.visits_v1 ( CounterID UInt32, StartDate Date, Sign Int8, IsNew UInt8, VisitID UInt64, UserID UInt64, StartTime DateTime, Duration UInt32, UTCStartTime DateTime, PageViews Int32, Hits Int32, IsBounce UInt8, Referer String, StartURL String, RefererDomain String, StartURLDomain String, EndURL String, LinkURL String, IsDownload UInt8, TraficSourceID Int8, SearchEngineID UInt16, SearchPhrase String, AdvEngineID UInt8, PlaceID Int32, RefererCategories Array(UInt16), URLCategories Array(UInt16), URLRegions Array(UInt32), RefererRegions Array(UInt32), IsYandex UInt8, GoalReachesDepth Int32, GoalReachesURL Int32, GoalReachesAny Int32, SocialSourceNetworkID UInt8, SocialSourcePage String, MobilePhoneModel String, ClientEventTime DateTime, RegionID UInt32, ClientIP UInt32, ClientIP6 FixedString(16), RemoteIP UInt32, RemoteIP6 FixedString(16), IPNetworkID UInt32, SilverlightVersion3 UInt32, CodeVersion UInt32, ResolutionWidth UInt16, ResolutionHeight UInt16, UserAgentMajor UInt16, UserAgentMinor UInt16, WindowClientWidth UInt16, WindowClientHeight UInt16, SilverlightVersion2 UInt8, SilverlightVersion4 UInt16, FlashVersion3 UInt16, FlashVersion4 UInt16, ClientTimeZone Int16, OS UInt8, UserAgent UInt8, ResolutionDepth UInt8, FlashMajor UInt8, FlashMinor UInt8, NetMajor UInt8, NetMinor UInt8, MobilePhone UInt8, SilverlightVersion1 UInt8, Age UInt8, Sex UInt8, Income UInt8, JavaEnable UInt8, CookieEnable UInt8, JavascriptEnable UInt8, IsMobile UInt8, BrowserLanguage UInt16, BrowserCountry UInt16, Interests UInt16, Robotness UInt8, GeneralInterests Array(UInt16), Params Array(String), Goals Nested(ID UInt32, Serial UInt32, EventTime DateTime, Price Int64, OrderID String, CurrencyID UInt32), WatchIDs Array(UInt64), ParamSumPrice Int64, ParamCurrency FixedString(3), ParamCurrencyID UInt16, ClickLogID UInt64, ClickEventID Int32, ClickGoodEvent Int32, ClickEventTime DateTime, ClickPriorityID Int32, ClickPhraseID Int32, ClickPageID Int32, ClickPlaceID Int32, ClickTypeID Int32, ClickResourceID Int32, ClickCost UInt32, ClickClientIP UInt32, ClickDomainID UInt32, ClickURL String, ClickAttempt UInt8, ClickOrderID UInt32, ClickBannerID UInt32, ClickMarketCategoryID UInt32, ClickMarketPP UInt32, ClickMarketCategoryName String, ClickMarketPPName String, ClickAWAPSCampaignName String, ClickPageName String, ClickTargetType UInt16, ClickTargetPhraseID UInt64, ClickContextType UInt8, ClickSelectType Int8, ClickOptions String, ClickGroupBannerID Int32, OpenstatServiceName String, OpenstatCampaignID String, OpenstatAdID String, OpenstatSourceID String, UTMSource String, UTMMedium String, UTMCampaign String, UTMContent String, UTMTerm String, FromTag String, HasGCLID UInt8, FirstVisit DateTime, PredLastVisit Date, LastVisit Date, TotalVisits UInt32, TraficSource Nested(ID Int8, SearchEngineID UInt16, AdvEngineID UInt8, PlaceID UInt16, SocialSourceNetworkID UInt8, Domain String, SearchPhrase String, SocialSourcePage String), Attendance FixedString(16), CLID UInt32, YCLID UInt64, NormalizedRefererHash UInt64, SearchPhraseHash UInt64, RefererDomainHash UInt64, NormalizedStartURLHash UInt64, StartURLDomainHash UInt64, NormalizedEndURLHash UInt64, TopLevelDomain UInt64, URLScheme UInt64, OpenstatServiceNameHash UInt64, OpenstatCampaignIDHash UInt64, OpenstatAdIDHash UInt64, OpenstatSourceIDHash UInt64, UTMSourceHash UInt64, UTMMediumHash UInt64, UTMCampaignHash UInt64, UTMContentHash UInt64, UTMTermHash UInt64, FromHash UInt64, WebVisorEnabled UInt8, WebVisorActivity UInt32, ParsedParams Nested(Key1 String, Key2 String, Key3 String, Key4 String, Key5 String, ValueDouble Float64), Market Nested(Type UInt8, GoalID UInt32, OrderID String, OrderPrice Int64, PP UInt32, DirectPlaceID UInt32, DirectOrderID UInt32, DirectBannerID UInt32, GoodID String, GoodName String, GoodQuantity Int32, GoodPrice Int64), IslandID FixedString(16)) ENGINE = CollapsingMergeTree(Sign) PARTITION BY toYYYYMM(StartDate) ORDER BY (CounterID, StartDate, intHash32(UserID), VisitID) SAMPLE BY intHash32(UserID) SETTINGS index_granularity = 8192" $ # импортируем данные $ cat visits_v1.tsv | clickhouse-client --query "INSERT INTO datasets.visits_v1 FORMAT TSV" --max_insert_block_size=100000 $ # опционально можно оптимизировать таблицу From fda8f6cb0e7331218a66f421684c7e3e6b9a3617 Mon Sep 17 00:00:00 2001 From: Ivan Blinkov Date: Sun, 22 Mar 2020 16:37:23 +0300 Subject: [PATCH 0164/1355] trigger ci --- docs/en/faq/general.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/faq/general.md b/docs/en/faq/general.md index bd573590190..058a2e5394f 100644 --- a/docs/en/faq/general.md +++ b/docs/en/faq/general.md @@ -50,4 +50,4 @@ $ clickhouse-client --query "SELECT * from table" --format FormatName > result.t See [clickhouse-client](../interfaces/cli.md). -[Original article](https://clickhouse.tech/docs/en/faq/general/) +{## [Original article](https://clickhouse.tech/docs/en/faq/general/) ##} From 18d659ed925a1b207e02085c171c45a9e8965313 Mon Sep 17 00:00:00 2001 From: Ivan Blinkov Date: Sun, 22 Mar 2020 16:53:46 +0300 Subject: [PATCH 0165/1355] Don't use a link to Russian book in English article --- docs/en/development/developer_instruction.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/development/developer_instruction.md b/docs/en/development/developer_instruction.md index 8f21b2b0dd9..c9cfd10a070 100644 --- a/docs/en/development/developer_instruction.md +++ b/docs/en/development/developer_instruction.md @@ -26,7 +26,7 @@ To do that in Ubuntu you would run in the command line terminal: sudo apt install git A brief manual on using Git can be found here: https://services.github.com/on-demand/downloads/github-git-cheat-sheet.pdf. -For a detailed manual on Git see https://git-scm.com/book/ru/v2. +For a detailed manual on Git see https://git-scm.com/book/en/v2. # Cloning a repository to your development machine {#cloning-a-repository-to-your-development-machine} From c2a73982a8be0976bb256b8ef1359f491b384e12 Mon Sep 17 00:00:00 2001 From: Ivan Blinkov Date: Sun, 22 Mar 2020 17:43:04 +0300 Subject: [PATCH 0166/1355] Keep some wkhtmltopdf logging --- docs/tools/build.py | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/docs/tools/build.py b/docs/tools/build.py index c6a294bf617..3c7ed9febf3 100755 --- a/docs/tools/build.py +++ b/docs/tools/build.py @@ -191,10 +191,9 @@ def build_single_page_version(lang, args, cfg): if not args.skip_pdf: single_page_index_html = os.path.abspath(os.path.join(single_page_output_path, 'index.html')) single_page_pdf = single_page_index_html.replace('index.html', f'clickhouse_{lang}.pdf') - create_pdf_command = ['wkhtmltopdf', '--print-media-type', single_page_index_html, single_page_pdf] + create_pdf_command = ['wkhtmltopdf', '--print-media-type', '--log-level', 'warn', single_page_index_html, single_page_pdf] logging.info(' '.join(create_pdf_command)) - with open(os.devnull, 'w') as devnull: - subprocess.check_call(' '.join(create_pdf_command), shell=True, stderr=devnull) + subprocess.check_call(' '.join(create_pdf_command), shell=True) if not args.version_prefix: # maybe enable in future with util.temp_dir() as test_dir: From 4cafab6fda4f612d4fad10151d8243ef3017d1e6 Mon Sep 17 00:00:00 2001 From: Ivan Blinkov Date: Sun, 22 Mar 2020 17:48:36 +0300 Subject: [PATCH 0167/1355] More docs build logging --- docs/tools/build.py | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/docs/tools/build.py b/docs/tools/build.py index 3c7ed9febf3..e8f52c44159 100755 --- a/docs/tools/build.py +++ b/docs/tools/build.py @@ -141,12 +141,14 @@ def build_for_lang(lang, args): if not args.skip_single_page: build_single_page_version(lang, args, cfg) + logging.info(f'Finished building {lang} docs') + except exceptions.ConfigurationError as e: raise SystemExit('\n' + str(e)) def build_single_page_version(lang, args, cfg): - logging.info('Building single page version for ' + lang) + logging.info(f'Building single page version for {lang}') os.environ['SINGLE_PAGE'] = '1' with util.autoremoved_file(os.path.join(args.docs_dir, lang, 'single.md')) as single_md: @@ -212,6 +214,7 @@ def build_single_page_version(lang, args, cfg): shutil.copytree(test_dir, args.save_raw_single_page) test.test_single_page(os.path.join(test_dir, 'single', 'index.html'), lang) + logging.info(f'Finished building single page version for {lang}') def write_redirect_html(out_path, to_url): From b8dc2a80c1f7f8eafd6a4dd38204986e67c67528 Mon Sep 17 00:00:00 2001 From: Ivan Blinkov Date: Sun, 22 Mar 2020 18:20:11 +0300 Subject: [PATCH 0168/1355] Update build.py --- docs/tools/build.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/tools/build.py b/docs/tools/build.py index e8f52c44159..de496a25cef 100755 --- a/docs/tools/build.py +++ b/docs/tools/build.py @@ -140,8 +140,8 @@ def build_for_lang(lang, args): if not args.skip_single_page: build_single_page_version(lang, args, cfg) - - logging.info(f'Finished building {lang} docs') + + logging.info(f'Finished building {lang} docs') except exceptions.ConfigurationError as e: raise SystemExit('\n' + str(e)) From bdb94fab45c47feee0fec379ec527d4490a5f089 Mon Sep 17 00:00:00 2001 From: Ivan Blinkov Date: Sun, 22 Mar 2020 18:55:22 +0300 Subject: [PATCH 0169/1355] make_links.sh run (#9807) --- docs/es/changelog/2018.md | 1 + docs/es/changelog/2019.md | 1 + docs/es/changelog/index.md | 1 + docs/es/interfaces/formats.md | 1 + docs/es/query_language/agg_functions/reference.md | 1 + docs/tools/translate/filter.py | 7 +++---- 6 files changed, 8 insertions(+), 4 deletions(-) create mode 120000 docs/es/changelog/2018.md create mode 120000 docs/es/changelog/2019.md create mode 120000 docs/es/changelog/index.md create mode 120000 docs/es/interfaces/formats.md create mode 120000 docs/es/query_language/agg_functions/reference.md diff --git a/docs/es/changelog/2018.md b/docs/es/changelog/2018.md new file mode 120000 index 00000000000..20799251f43 --- /dev/null +++ b/docs/es/changelog/2018.md @@ -0,0 +1 @@ +../../en/changelog/2018.md \ No newline at end of file diff --git a/docs/es/changelog/2019.md b/docs/es/changelog/2019.md new file mode 120000 index 00000000000..105ca144fca --- /dev/null +++ b/docs/es/changelog/2019.md @@ -0,0 +1 @@ +../../en/changelog/2019.md \ No newline at end of file diff --git a/docs/es/changelog/index.md b/docs/es/changelog/index.md new file mode 120000 index 00000000000..79b747aee1b --- /dev/null +++ b/docs/es/changelog/index.md @@ -0,0 +1 @@ +../../../CHANGELOG.md \ No newline at end of file diff --git a/docs/es/interfaces/formats.md b/docs/es/interfaces/formats.md new file mode 120000 index 00000000000..41a65ebe579 --- /dev/null +++ b/docs/es/interfaces/formats.md @@ -0,0 +1 @@ +../../en/interfaces/formats.md \ No newline at end of file diff --git a/docs/es/query_language/agg_functions/reference.md b/docs/es/query_language/agg_functions/reference.md new file mode 120000 index 00000000000..c5651cb0793 --- /dev/null +++ b/docs/es/query_language/agg_functions/reference.md @@ -0,0 +1 @@ +../../../en/query_language/agg_functions/reference.md \ No newline at end of file diff --git a/docs/tools/translate/filter.py b/docs/tools/translate/filter.py index 3e737587df9..cf673a75d63 100755 --- a/docs/tools/translate/filter.py +++ b/docs/tools/translate/filter.py @@ -30,7 +30,7 @@ def process_buffer(buffer, new_value, item=None): print('Failed to translate', str(e), file=sys.stderr) sys.exit(1) - debug('Translate', text, ' -> ', translated_text) + debug(f'Translate: "{text}" -> "{translated_text}"') if text and text[0].isupper() and not translated_text[0].isupper(): translated_text = translated_text[0].upper() + translated_text[1:] @@ -115,7 +115,6 @@ def translate_filter(key, value, _format, _): admonition_value.pop(-1) admonition_value += text else: - debug('>>>', ) text = admonition_value[-1].get('c') if text: text = translate(text[0].upper() + text[1:]) @@ -138,8 +137,8 @@ def translate_filter(key, value, _format, _): return cls(*value) elif key == 'Header': # TODO: title case header in en - value[1][0] = slugify.slugify(value[1][0], separator='-', word_boundary=True, save_order=True) - # TODO: title case header in en + if '_' not in value[1][0]: # Preserve some manually specified anchors + value[1][0] = slugify.slugify(value[1][0], separator='-', word_boundary=True, save_order=True) value[2] = process_sentence(value[2]) return cls(*value) elif key == 'SoftBreak': From b89de69345a88446a33cb90fed95b741e5bf95de Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sun, 22 Mar 2020 13:37:35 +0300 Subject: [PATCH 0170/1355] Evaluate constant expressions for optimize_skip_unused_shards --- dbms/src/Storages/StorageDistributed.cpp | 60 ++++++++++++++++--- dbms/src/Storages/StorageDistributed.h | 2 +- .../transformQueryForExternalDatabase.cpp | 5 +- 3 files changed, 55 insertions(+), 12 deletions(-) diff --git a/dbms/src/Storages/StorageDistributed.cpp b/dbms/src/Storages/StorageDistributed.cpp index 4ef8f39d1f7..f230ab6e306 100644 --- a/dbms/src/Storages/StorageDistributed.cpp +++ b/dbms/src/Storages/StorageDistributed.cpp @@ -193,19 +193,60 @@ std::string makeFormattedListOfShards(const ClusterPtr & cluster) return os.str(); } -} - - -/// For destruction of std::unique_ptr of type that is incomplete in class definition. -StorageDistributed::~StorageDistributed() = default; - -static ExpressionActionsPtr buildShardingKeyExpression(const ASTPtr & sharding_key, const Context & context, NamesAndTypesList columns, bool project) +ExpressionActionsPtr buildShardingKeyExpression(const ASTPtr & sharding_key, const Context & context, const NamesAndTypesList & columns, bool project) { ASTPtr query = sharding_key; auto syntax_result = SyntaxAnalyzer(context).analyze(query, columns); return ExpressionAnalyzer(query, syntax_result, context).getActions(project); } +class ReplacingConstantExpressionsMatcher +{ +public: + using Data = Block; + + static bool needChildVisit(ASTPtr &, const ASTPtr &) + { + return true; + } + + static void visit(ASTPtr & node, Block & block_with_constants) + { + if (!node->as()) + return; + + std::string name = node->getColumnName(); + if (block_with_constants.has(name)) + { + auto result = block_with_constants.getByName(name); + if (!isColumnConst(*result.column)) + return; + + if (result.column->isNullAt(0)) + { + node = std::make_shared(Field()); + } + else + { + node = std::make_shared(assert_cast(*result.column).getField()); + } + } + } +}; +void replaceConstantExpressions(ASTPtr & node, const Context & context, const NamesAndTypesList & columns) +{ + auto syntax_result = SyntaxAnalyzer(context).analyze(node, columns); + Block block_with_constants = KeyCondition::getBlockWithConstants(node, syntax_result, context); + + InDepthNodeVisitor visitor(block_with_constants); + visitor.visit(node); +} + +} // \anonymous + + +/// For destruction of std::unique_ptr of type that is incomplete in class definition. +StorageDistributed::~StorageDistributed() = default; StorageDistributed::StorageDistributed( const StorageID & id_, @@ -378,7 +419,7 @@ Pipes StorageDistributed::read( if (has_sharding_key) { - smaller_cluster = skipUnusedShards(cluster, query_info); + smaller_cluster = skipUnusedShards(cluster, query_info, context); if (smaller_cluster) { @@ -608,7 +649,7 @@ void StorageDistributed::ClusterNodeData::shutdownAndDropAllData() /// 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 SelectQueryInfo & query_info) +ClusterPtr StorageDistributed::skipUnusedShards(ClusterPtr cluster, const SelectQueryInfo & query_info, const Context & context) { const auto & select = query_info.query->as(); @@ -627,6 +668,7 @@ ClusterPtr StorageDistributed::skipUnusedShards(ClusterPtr cluster, const Select condition_ast = select.prewhere() ? select.prewhere()->clone() : select.where()->clone(); } + replaceConstantExpressions(condition_ast, context, getColumns().getAllPhysical() /** TODO: sharding_key_column_name */); const auto blocks = evaluateExpressionOverConstantCondition(condition_ast, sharding_key_expr); // Can't get definite answer if we can skip any shards diff --git a/dbms/src/Storages/StorageDistributed.h b/dbms/src/Storages/StorageDistributed.h index ee9b1e8f260..84ac005cda6 100644 --- a/dbms/src/Storages/StorageDistributed.h +++ b/dbms/src/Storages/StorageDistributed.h @@ -164,7 +164,7 @@ protected: const String & relative_data_path_, bool attach); - ClusterPtr skipUnusedShards(ClusterPtr cluster, const SelectQueryInfo & query_info); + ClusterPtr skipUnusedShards(ClusterPtr cluster, const SelectQueryInfo & query_info, const Context & context); void createStorage(); diff --git a/dbms/src/Storages/transformQueryForExternalDatabase.cpp b/dbms/src/Storages/transformQueryForExternalDatabase.cpp index f5ab7ca9026..5ba7deadb33 100644 --- a/dbms/src/Storages/transformQueryForExternalDatabase.cpp +++ b/dbms/src/Storages/transformQueryForExternalDatabase.cpp @@ -26,7 +26,8 @@ namespace ErrorCodes namespace { -class ReplacingConstantExpressionsMatcher +/// Everything except numbers is put as string literal. +class ReplacingConstantExpressionsMatcherNumOrStr { public: using Data = Block; @@ -75,7 +76,7 @@ void replaceConstantExpressions(ASTPtr & node, const Context & context, const Na auto syntax_result = SyntaxAnalyzer(context).analyze(node, all_columns); Block block_with_constants = KeyCondition::getBlockWithConstants(node, syntax_result, context); - InDepthNodeVisitor visitor(block_with_constants); + InDepthNodeVisitor visitor(block_with_constants); visitor.visit(node); } From 0204c3f9c71fba18f3ffb7dc6cd12eb1d65d11a1 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sun, 22 Mar 2020 14:29:09 +0300 Subject: [PATCH 0171/1355] Cover optimize_skip_unused_shards for types mismatch --- ...mize_skip_unused_shards_type_mismatch.reference | 0 ...0_optimize_skip_unused_shards_type_mismatch.sql | 14 ++++++++++++++ 2 files changed, 14 insertions(+) create mode 100644 dbms/tests/queries/0_stateless/02000_optimize_skip_unused_shards_type_mismatch.reference create mode 100644 dbms/tests/queries/0_stateless/02000_optimize_skip_unused_shards_type_mismatch.sql diff --git a/dbms/tests/queries/0_stateless/02000_optimize_skip_unused_shards_type_mismatch.reference b/dbms/tests/queries/0_stateless/02000_optimize_skip_unused_shards_type_mismatch.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/dbms/tests/queries/0_stateless/02000_optimize_skip_unused_shards_type_mismatch.sql b/dbms/tests/queries/0_stateless/02000_optimize_skip_unused_shards_type_mismatch.sql new file mode 100644 index 00000000000..d7e722fc9bc --- /dev/null +++ b/dbms/tests/queries/0_stateless/02000_optimize_skip_unused_shards_type_mismatch.sql @@ -0,0 +1,14 @@ +set optimize_skip_unused_shards=1; + +drop table if exists data_02000; +drop table if exists dist_02000; + +create table data_02000 (key Int) Engine=Null(); +create table dist_02000 as data_02000 Engine=Distributed(test_cluster_two_shards, currentDatabase(), data_02000, key); + +select * from data_02000 where key = 0xdeadbeafdeadbeaf; +select * from dist_02000 where key = 0xdeadbeafdeadbeaf settings force_optimize_skip_unused_shards=2; -- { serverError 507; } +select * from dist_02000 where key = 0xdeadbeafdeadbeaf; + +drop table data_02000; +drop table dist_02000; From 7caf8c006d8109f577a327170e9233ce92af4a07 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sun, 22 Mar 2020 14:11:57 +0300 Subject: [PATCH 0172/1355] Fix SIGSEGV with optimize_skip_unused_shards when type cannot be converted --- dbms/src/Interpreters/evaluateConstantExpression.cpp | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/dbms/src/Interpreters/evaluateConstantExpression.cpp b/dbms/src/Interpreters/evaluateConstantExpression.cpp index 37e990fd5fc..2834939407d 100644 --- a/dbms/src/Interpreters/evaluateConstantExpression.cpp +++ b/dbms/src/Interpreters/evaluateConstantExpression.cpp @@ -116,8 +116,10 @@ namespace if (name == identifier->name) { ColumnWithTypeAndName column; - // FIXME: what to do if field is not convertable? - column.column = type->createColumnConst(1, convertFieldToType(literal->value, *type)); + Field value = convertFieldToType(literal->value, *type); + if (!literal->value.isNull() && value.isNull()) + return {}; + column.column = type->createColumnConst(1, value); column.name = name; column.type = type; return {{std::move(column)}}; From 21f478e2c0dade5bf740a966d6bb60fc2ab88c5c Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 22 Mar 2020 23:44:17 +0300 Subject: [PATCH 0173/1355] Update "builder" docker image --- docker/builder/Dockerfile | 19 +++++++------------ 1 file changed, 7 insertions(+), 12 deletions(-) diff --git a/docker/builder/Dockerfile b/docker/builder/Dockerfile index c9e37dc75e5..e76d9c290c0 100644 --- a/docker/builder/Dockerfile +++ b/docker/builder/Dockerfile @@ -1,8 +1,6 @@ -FROM ubuntu:18.04 +FROM ubuntu:19.10 RUN apt-get update -y \ - && apt-get install -y software-properties-common \ - && add-apt-repository ppa:ubuntu-toolchain-r/test \ && env DEBIAN_FRONTEND=noninteractive \ apt-get install --yes --no-install-recommends \ bash \ @@ -10,9 +8,8 @@ RUN apt-get update -y \ cmake \ curl \ expect \ - g++-9 \ - gcc-9 \ - libreadline-dev \ + g++ \ + gcc \ ninja-build \ perl \ pkg-config \ @@ -22,12 +19,10 @@ RUN apt-get update -y \ python-termcolor \ sudo \ tzdata \ - gperf \ - libcctz-dev - -RUN apt install -y wget -RUN printf "deb http://apt.llvm.org/bionic/ llvm-toolchain-bionic-8 main\ndeb-src http://apt.llvm.org/bionic/ llvm-toolchain-bionic-8 main" >> /etc/apt/sources.list \ - && wget -O - https://apt.llvm.org/llvm-snapshot.gpg.key | apt-key add - && apt update && apt-get install -y clang-8 lldb-8 lld-8 + clang \ + clang-tidy \ + lld \ + lldb COPY build.sh / From 8df43f2f5043a677829282e77f04b4969dd9768b Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 23 Mar 2020 00:10:14 +0300 Subject: [PATCH 0174/1355] Update packager images --- docker/packager/binary/Dockerfile | 22 ++++++++-------------- docker/packager/deb/Dockerfile | 19 ++++++------------- 2 files changed, 14 insertions(+), 27 deletions(-) diff --git a/docker/packager/binary/Dockerfile b/docker/packager/binary/Dockerfile index a3d893619b5..878cc49a529 100644 --- a/docker/packager/binary/Dockerfile +++ b/docker/packager/binary/Dockerfile @@ -1,5 +1,5 @@ # docker build -t yandex/clickhouse-binary-builder . -FROM ubuntu:18.04 +FROM ubuntu:19.10 RUN apt-get --allow-unauthenticated update -y \ && env DEBIAN_FRONTEND=noninteractive \ @@ -13,9 +13,6 @@ RUN apt-get --allow-unauthenticated update -y \ apt-transport-https \ ca-certificates -RUN echo "deb [trusted=yes] http://apt.llvm.org/bionic/ llvm-toolchain-bionic-8 main" >> /etc/apt/sources.list -RUN add-apt-repository ppa:ubuntu-toolchain-r/test - RUN apt-get update -y \ && env DEBIAN_FRONTEND=noninteractive \ apt-get install --yes --no-install-recommends \ @@ -23,15 +20,11 @@ RUN apt-get update -y \ cmake \ ccache \ curl \ - gcc-8 \ - g++-8 \ gcc-9 \ g++-9 \ - clang-8 \ - lld-8 \ - libclang-8-dev \ - liblld-8-dev \ - clang-tidy-8 \ + clang-9 \ + lld-9 \ + clang-tidy-9 \ libicu-dev \ libreadline-dev \ ninja-build \ @@ -45,10 +38,10 @@ RUN apt-get update -y \ wget # This symlink required by gcc to find lld compiler -RUN ln -s /usr/bin/lld-8 /usr/bin/ld.lld +RUN ln -s /usr/bin/lld-9 /usr/bin/ld.lld -ENV CC=clang-8 -ENV CXX=clang++-8 +ENV CC=clang-9 +ENV CXX=clang++-9 # libtapi is required to support .tbh format from recent MacOS SDKs RUN git clone https://github.com/tpoechtrager/apple-libtapi.git @@ -64,6 +57,7 @@ RUN rm -rf cctools-port RUN wget https://github.com/phracker/MacOSX-SDKs/releases/download/10.14-beta4/MacOSX10.14.sdk.tar.xz # Download toolchain for ARM +# It contains all required headers and libraries. Note that it's named as "gcc" but actually we are using clang for cross compiling. RUN wget "https://developer.arm.com/-/media/Files/downloads/gnu-a/8.3-2019.03/binrel/gcc-arm-8.3-2019.03-x86_64-aarch64-linux-gnu.tar.xz?revision=2e88a73f-d233-4f96-b1f4-d8b36e9bb0b9&la=en" -O gcc-arm-8.3-2019.03-x86_64-aarch64-linux-gnu.tar.xz COPY build.sh / diff --git a/docker/packager/deb/Dockerfile b/docker/packager/deb/Dockerfile index 763fcd486b0..0ac953379f5 100644 --- a/docker/packager/deb/Dockerfile +++ b/docker/packager/deb/Dockerfile @@ -1,5 +1,5 @@ # docker build -t yandex/clickhouse-deb-builder . -FROM ubuntu:18.04 +FROM ubuntu:19.10 RUN apt-get --allow-unauthenticated update -y \ && env DEBIAN_FRONTEND=noninteractive \ @@ -13,22 +13,15 @@ RUN apt-get --allow-unauthenticated update -y \ apt-transport-https \ ca-certificates - -RUN echo "deb [trusted=yes] http://apt.llvm.org/bionic/ llvm-toolchain-bionic-8 main" >> /etc/apt/sources.list -RUN add-apt-repository ppa:ubuntu-toolchain-r/test - +# Libraries from OS are only needed to test the "unbundled" build (that is not used in production). RUN apt-get --allow-unauthenticated update -y \ && env DEBIAN_FRONTEND=noninteractive \ apt-get --allow-unauthenticated install --yes --no-install-recommends \ - gcc-8 \ - g++-8 \ gcc-9 \ g++-9 \ - clang-8 \ - lld-8 \ - libclang-8-dev \ - liblld-8-dev \ - clang-tidy-8 \ + clang-9 \ + lld-9 \ + clang-tidy-9 \ libicu-dev \ libreadline-dev \ gperf \ @@ -75,7 +68,7 @@ RUN chmod +x dpkg-deb RUN cp dpkg-deb /usr/bin # This symlink required by gcc to find lld compiler -RUN ln -s /usr/bin/lld-8 /usr/bin/ld.lld +RUN ln -s /usr/bin/lld-9 /usr/bin/ld.lld COPY build.sh / From c7863e1ac68435d7d30da842b2cd7b95e9a4dc13 Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Mon, 23 Mar 2020 05:12:31 +0300 Subject: [PATCH 0175/1355] Clang Tidy, part 7 (#9799) * Attempt to enable identifier naming check * Attempt to enable identifier naming check, continue * Progress on identifier names * Adopt identifier names check * Fixed error * Merge with master * Fixed build * Fixed build * Fixed build * Fixed build * Fixed build * Fixed error * Fixed error * Fixed error --- .clang-tidy | 41 ++++ base/common/shift10.cpp | 10 +- base/common/tests/date_lut4.cpp | 6 +- base/mysqlxx/Pool.cpp | 2 +- base/mysqlxx/Pool.h | 2 +- base/mysqlxx/PoolFactory.cpp | 6 +- base/mysqlxx/PoolFactory.h | 4 +- base/mysqlxx/PoolWithFailover.cpp | 6 +- base/mysqlxx/PoolWithFailover.h | 2 +- base/mysqlxx/Value.cpp | 4 +- dbms/programs/client/Client.cpp | 6 +- dbms/programs/copier/ClusterCopierApp.cpp | 10 +- dbms/programs/odbc-bridge/ODBCBridge.cpp | 10 +- dbms/programs/server/HTTPHandler.cpp | 6 +- dbms/programs/server/HTTPHandlerFactory.cpp | 4 +- dbms/programs/server/MySQLHandler.cpp | 4 +- dbms/programs/server/MySQLHandlerFactory.cpp | 24 +-- dbms/programs/server/Server.cpp | 6 +- dbms/src/Access/QuotaCache.cpp | 2 +- .../AggregateFunctionGroupArrayMoving.cpp | 18 +- .../AggregateFunctionGroupUniqArray.cpp | 28 +-- .../AggregateFunctionMLMethod.cpp | 14 +- .../AggregateFunctionMLMethod.h | 8 +- dbms/src/Columns/ColumnAggregateFunction.cpp | 8 +- dbms/src/Columns/ColumnArray.cpp | 35 ++-- dbms/src/Columns/ColumnFunction.cpp | 4 +- dbms/src/Columns/ColumnLowCardinality.cpp | 28 +-- dbms/src/Common/Dwarf.cpp | 180 +++++++++--------- dbms/src/Common/PoolBase.h | 2 +- dbms/src/Common/RWLock.cpp | 6 +- dbms/src/Common/ThreadFuzzer.cpp | 6 +- dbms/src/Common/ZooKeeper/ZooKeeperImpl.cpp | 6 +- .../src/Common/tests/parallel_aggregation.cpp | 2 +- .../Compression/CompressionCodecGorilla.cpp | 14 +- .../tests/gtest_compressionCodec.cpp | 14 +- dbms/src/Core/DecimalComparison.h | 12 +- dbms/src/DataTypes/DataTypeDateTime64.cpp | 12 +- dbms/src/DataTypes/DataTypeLowCardinality.cpp | 12 +- dbms/src/DataTypes/getMostSubtype.cpp | 21 +- dbms/src/Databases/DatabaseMySQL.cpp | 4 +- dbms/src/Databases/DatabaseOrdinary.cpp | 4 +- .../ClickHouseDictionarySource.cpp | 4 +- dbms/src/Dictionaries/DictionaryFactory.cpp | 4 +- .../ExecutableDictionarySource.cpp | 4 +- .../src/Dictionaries/FileDictionarySource.cpp | 4 +- .../src/Dictionaries/HTTPDictionarySource.cpp | 4 +- .../Dictionaries/LibraryDictionarySource.cpp | 40 ++-- .../Dictionaries/MongoDBDictionarySource.cpp | 12 +- .../Dictionaries/MySQLDictionarySource.cpp | 20 +- .../Dictionaries/RedisBlockInputStream.cpp | 12 +- .../Dictionaries/RedisDictionarySource.cpp | 4 +- dbms/src/Dictionaries/TrieDictionary.cpp | 12 +- .../src/Dictionaries/XDBCDictionarySource.cpp | 8 +- .../getDictionaryConfigurationFromAST.cpp | 6 +- dbms/src/Disks/DiskS3.cpp | 2 +- dbms/src/Disks/DiskSpaceMonitor.cpp | 2 +- dbms/src/Disks/tests/gtest_disk.cpp | 44 ++--- dbms/src/Functions/CRC.cpp | 16 +- dbms/src/Functions/EmptyImpl.h | 6 +- dbms/src/Functions/FunctionBinaryArithmetic.h | 70 +++---- .../FunctionDateOrDateTimeAddInterval.h | 20 +- dbms/src/Functions/FunctionHelpers.cpp | 12 +- dbms/src/Functions/FunctionStringOrArrayToT.h | 4 +- dbms/src/Functions/FunctionStringToString.h | 2 +- dbms/src/Functions/FunctionsComparison.h | 62 +++--- dbms/src/Functions/FunctionsLogical.cpp | 6 +- dbms/src/Functions/FunctionsStringRegex.cpp | 30 +-- dbms/src/Functions/FunctionsStringRegex.h | 4 +- dbms/src/Functions/FunctionsStringSearch.cpp | 28 +-- dbms/src/Functions/FunctionsStringSearch.h | 14 +- .../Functions/FunctionsStringSimilarity.cpp | 18 +- .../src/Functions/FunctionsStringSimilarity.h | 8 +- dbms/src/Functions/FunctionsVisitParam.h | 6 +- dbms/src/Functions/LowerUpperImpl.h | 2 +- dbms/src/Functions/LowerUpperUTF8Impl.h | 2 +- dbms/src/Functions/URL/FunctionsURL.h | 4 +- dbms/src/Functions/URL/decodeURLComponent.cpp | 2 +- .../Functions/URL/firstSignificantSubdomain.h | 2 +- .../src/Functions/URL/tldLookup.generated.cpp | 8 +- dbms/src/Functions/URL/tldLookup.h | 8 +- dbms/src/Functions/array/arrayIndex.h | 4 +- dbms/src/Functions/array/length.cpp | 4 +- dbms/src/Functions/concat.cpp | 6 +- dbms/src/Functions/dateDiff.cpp | 18 +- dbms/src/Functions/formatDateTime.cpp | 38 ++-- dbms/src/Functions/formatString.cpp | 6 +- dbms/src/Functions/if.cpp | 34 ++-- dbms/src/Functions/intDiv.cpp | 2 +- dbms/src/Functions/isValidUTF8.cpp | 4 +- dbms/src/Functions/lengthUTF8.cpp | 4 +- dbms/src/Functions/modulo.cpp | 2 +- dbms/src/Functions/pointInPolygon.cpp | 21 +- dbms/src/Functions/reverse.cpp | 4 +- dbms/src/Functions/reverseUTF8.cpp | 2 +- dbms/src/Functions/timeSlots.cpp | 12 +- dbms/src/Functions/toValidUTF8.cpp | 2 +- dbms/src/Functions/trim.cpp | 8 +- dbms/src/IO/HTTPCommon.cpp | 6 +- dbms/src/IO/ReadHelpers.cpp | 10 +- dbms/src/IO/S3Common.cpp | 4 +- dbms/src/IO/tests/gtest_DateTimeToString.cpp | 8 +- dbms/src/IO/tests/hashing_read_buffer.cpp | 4 +- .../src/IO/tests/o_direct_and_dirty_pages.cpp | 4 +- dbms/src/IO/tests/ryu_test.cpp | 18 +- dbms/src/Interpreters/ActionsVisitor.cpp | 12 +- .../Interpreters/ArrayJoinedColumnsVisitor.h | 2 +- dbms/src/Interpreters/CatBoostModel.cpp | 46 +++-- dbms/src/Interpreters/Context.cpp | 6 +- dbms/src/Interpreters/DatabaseCatalog.cpp | 7 +- dbms/src/Interpreters/ExpressionActions.cpp | 8 +- dbms/src/Interpreters/ExpressionAnalyzer.cpp | 12 +- dbms/src/Interpreters/ExpressionJIT.cpp | 20 +- dbms/src/Interpreters/ExternalLoader.cpp | 24 +-- .../Interpreters/InternalTextLogsQueue.cpp | 3 +- .../Interpreters/InterpreterSelectQuery.cpp | 4 +- .../JoinToSubqueryTransformVisitor.cpp | 2 +- .../PredicateExpressionsOptimizer.cpp | 2 +- dbms/src/Interpreters/SyntaxAnalyzer.cpp | 4 +- .../evaluateConstantExpression.cpp | 10 +- dbms/src/Interpreters/tests/hash_map.cpp | 4 +- dbms/src/Parsers/ASTSelectQuery.cpp | 10 +- dbms/src/Parsers/ASTSelectQuery.h | 8 +- dbms/src/Parsers/Lexer.cpp | 6 +- .../Processors/Executors/PipelineExecutor.cpp | 4 +- .../Formats/Impl/CapnProtoRowInputFormat.cpp | 14 +- .../Formats/Impl/PrettyBlockOutputFormat.cpp | 10 +- .../Transforms/CreatingSetsTransform.cpp | 6 +- .../tests/processors_test_aggregation.cpp | 4 +- .../tests/processors_test_chain.cpp | 6 +- .../tests/processors_test_expand_pipeline.cpp | 6 +- .../tests/processors_test_merge.cpp | 14 +- ...rocessors_test_merge_sorting_transform.cpp | 8 +- ...ocessors_test_merging_sorted_transform.cpp | 14 +- .../Storages/Kafka/KafkaBlockInputStream.cpp | 24 +-- .../Storages/Kafka/KafkaBlockOutputStream.cpp | 2 +- .../Kafka/ReadBufferFromKafkaConsumer.cpp | 6 +- .../Kafka/WriteBufferToKafkaProducer.cpp | 2 +- .../Kafka/WriteBufferToKafkaProducer.h | 2 +- .../src/Storages/LiveView/StorageLiveView.cpp | 4 +- .../MergeTree/IMergedBlockOutputStream.h | 2 +- .../MergeTreeBaseSelectProcessor.cpp | 4 +- .../MergeTree/MergeTreeDataMergerMutator.cpp | 8 +- .../MergeTree/MergeTreeDataMergerMutator.h | 10 +- .../MergeTree/MergeTreeDataPartChecksum.cpp | 38 ++-- .../MergeTree/MergeTreeDataPartChecksum.h | 6 +- .../MergeTree/MergeTreeDataSelectExecutor.cpp | 4 +- .../MergeTreeIndexConditionBloomFilter.cpp | 4 +- .../MergeTree/MergeTreeReaderCompact.cpp | 16 +- .../MergeTree/MergeTreeReaderCompact.h | 12 +- .../MergeTree/MergeTreeReaderWide.cpp | 14 +- .../Storages/MergeTree/MergeTreeReaderWide.h | 12 +- .../MergeTree/MergeTreeWhereOptimizer.cpp | 2 +- .../ReplicatedMergeTreeCleanupThread.cpp | 4 +- .../MergeTree/ReplicatedMergeTreeQueue.cpp | 6 +- .../ReplicatedMergeTreeQuorumAddedParts.h | 8 +- dbms/src/Storages/StorageFactory.cpp | 10 +- dbms/src/Storages/StorageFile.cpp | 5 +- dbms/src/Storages/StorageGenerateRandom.cpp | 4 +- dbms/src/Storages/StorageLog.cpp | 6 +- dbms/src/Storages/StorageMaterializedView.cpp | 2 +- dbms/src/Storages/StorageMerge.cpp | 2 +- dbms/src/Storages/StorageMySQL.cpp | 4 +- .../Storages/StorageReplicatedMergeTree.cpp | 4 +- .../System/StorageSystemPartsBase.cpp | 8 +- dbms/src/Storages/tests/gtest_storage_log.cpp | 16 +- .../src/TableFunctions/TableFunctionMySQL.cpp | 2 +- .../TableFunctions/TableFunctionRemote.cpp | 8 +- 167 files changed, 948 insertions(+), 923 deletions(-) diff --git a/.clang-tidy b/.clang-tidy index d7380a9a89b..3c222fbf8da 100644 --- a/.clang-tidy +++ b/.clang-tidy @@ -57,6 +57,7 @@ Checks: '-*, readability-simplify-subscript-expr, readability-simplify-boolean-expr, readability-inconsistent-declaration-parameter-name, + readability-identifier-naming, bugprone-undelegated-constructor, bugprone-argument-comment, @@ -162,3 +163,43 @@ Checks: '-*, boost-use-to-string, ' WarningsAsErrors: '*' + +CheckOptions: + - key: readability-identifier-naming.ClassCase + value: CamelCase + - key: readability-identifier-naming.EnumCase + value: CamelCase + - key: readability-identifier-naming.LocalVariableCase + value: lower_case + - key: readability-identifier-naming.StaticConstantCase + value: aNy_CasE + - key: readability-identifier-naming.MemberCase + value: lower_case + - key: readability-identifier-naming.PrivateMemberPrefix + value: '' + - key: readability-identifier-naming.ProtectedMemberPrefix + value: '' + - key: readability-identifier-naming.PublicMemberCase + value: lower_case + - key: readability-identifier-naming.MethodCase + value: camelBack + - key: readability-identifier-naming.PrivateMethodPrefix + value: '' + - key: readability-identifier-naming.ProtectedMethodPrefix + value: '' + - key: readability-identifier-naming.ParameterPackCase + value: lower_case + - key: readability-identifier-naming.StructCase + value: CamelCase + - key: readability-identifier-naming.TemplateTemplateParameterCase + value: CamelCase + - key: readability-identifier-naming.TemplateUsingCase + value: lower_case + - key: readability-identifier-naming.TypeTemplateParameterCase + value: CamelCase + - key: readability-identifier-naming.TypedefCase + value: CamelCase + - key: readability-identifier-naming.UnionCase + value: CamelCase + - key: readability-identifier-naming.UsingCase + value: CamelCase diff --git a/base/common/shift10.cpp b/base/common/shift10.cpp index 341ced6aa81..b7b39182145 100644 --- a/base/common/shift10.cpp +++ b/base/common/shift10.cpp @@ -8,8 +8,8 @@ template static T shift10Impl(T x, int exponent) { - static constexpr ssize_t MIN_EXPONENT = -323; - static constexpr ssize_t MAX_EXPONENT = 308; + static constexpr ssize_t min_exponent = -323; + static constexpr ssize_t max_exponent = 308; static const long double powers10[] = { @@ -47,12 +47,12 @@ static T shift10Impl(T x, int exponent) 1e291L,1e292L,1e293L,1e294L,1e295L,1e296L,1e297L,1e298L,1e299L,1e300L,1e301L,1e302L,1e303L,1e304L,1e305L,1e306L,1e307L,1e308L }; - if (unlikely(exponent < MIN_EXPONENT)) /// Note: there are some values below MIN_EXPONENT that is greater than zero. + if (unlikely(exponent < min_exponent)) /// Note: there are some values below MIN_EXPONENT that is greater than zero. x *= 0; /// Multiplying to keep the sign of zero. - else if (unlikely(exponent > MAX_EXPONENT)) + else if (unlikely(exponent > max_exponent)) x *= std::numeric_limits::infinity(); /// Multiplying to keep the sign of infinity. else - x *= powers10[exponent - MIN_EXPONENT]; + x *= powers10[exponent - min_exponent]; return x; } diff --git a/base/common/tests/date_lut4.cpp b/base/common/tests/date_lut4.cpp index a82ec25f183..86a4708dc79 100644 --- a/base/common/tests/date_lut4.cpp +++ b/base/common/tests/date_lut4.cpp @@ -5,12 +5,12 @@ int main(int, char **) { /** В DateLUT был глюк - для времён из дня 1970-01-01, возвращался номер часа больше 23. */ - static const time_t TIME = 66130; + static const time_t time = 66130; const auto & date_lut = DateLUT::instance(); - std::cerr << date_lut.toHour(TIME) << std::endl; - std::cerr << date_lut.toDayNum(TIME) << std::endl; + std::cerr << date_lut.toHour(time) << std::endl; + std::cerr << date_lut.toDayNum(time) << std::endl; const auto * values = reinterpret_cast(&date_lut); diff --git a/base/mysqlxx/Pool.cpp b/base/mysqlxx/Pool.cpp index b8216c254e6..99815363a56 100644 --- a/base/mysqlxx/Pool.cpp +++ b/base/mysqlxx/Pool.cpp @@ -119,7 +119,7 @@ Pool::~Pool() } -Pool::Entry Pool::Get() +Pool::Entry Pool::get() { std::unique_lock lock(mutex); diff --git a/base/mysqlxx/Pool.h b/base/mysqlxx/Pool.h index b5e0c71a5cc..bf9365a064a 100644 --- a/base/mysqlxx/Pool.h +++ b/base/mysqlxx/Pool.h @@ -185,7 +185,7 @@ public: ~Pool(); /// Allocates connection. - Entry Get(); + Entry get(); /// Allocates connection. /// If database is not accessible, returns empty Entry object. diff --git a/base/mysqlxx/PoolFactory.cpp b/base/mysqlxx/PoolFactory.cpp index d9c1ec1519d..f0a5543d723 100644 --- a/base/mysqlxx/PoolFactory.cpp +++ b/base/mysqlxx/PoolFactory.cpp @@ -17,10 +17,10 @@ struct PoolFactory::Impl std::mutex mutex; }; -PoolWithFailover PoolFactory::Get(const std::string & config_name, unsigned default_connections, +PoolWithFailover PoolFactory::get(const std::string & config_name, unsigned default_connections, unsigned max_connections, size_t max_tries) { - return Get(Poco::Util::Application::instance().config(), config_name, default_connections, max_connections, max_tries); + return get(Poco::Util::Application::instance().config(), config_name, default_connections, max_connections, max_tries); } /// Duplicate of code from StringUtils.h. Copied here for less dependencies. @@ -72,7 +72,7 @@ static std::string getPoolEntryName(const Poco::Util::AbstractConfiguration & co return entry_name; } -PoolWithFailover PoolFactory::Get(const Poco::Util::AbstractConfiguration & config, +PoolWithFailover PoolFactory::get(const Poco::Util::AbstractConfiguration & config, const std::string & config_name, unsigned default_connections, unsigned max_connections, size_t max_tries) { diff --git a/base/mysqlxx/PoolFactory.h b/base/mysqlxx/PoolFactory.h index a21c2baca2f..a7bb97cd0c6 100644 --- a/base/mysqlxx/PoolFactory.h +++ b/base/mysqlxx/PoolFactory.h @@ -27,13 +27,13 @@ public: PoolFactory(const PoolFactory &) = delete; /** Allocates a PoolWithFailover to connect to MySQL. */ - PoolWithFailover Get(const std::string & config_name, + PoolWithFailover get(const std::string & config_name, unsigned default_connections = MYSQLXX_POOL_WITH_FAILOVER_DEFAULT_START_CONNECTIONS, unsigned max_connections = MYSQLXX_POOL_WITH_FAILOVER_DEFAULT_MAX_CONNECTIONS, size_t max_tries = MYSQLXX_POOL_WITH_FAILOVER_DEFAULT_MAX_TRIES); /** Allocates a PoolWithFailover to connect to MySQL. */ - PoolWithFailover Get(const Poco::Util::AbstractConfiguration & config, + PoolWithFailover get(const Poco::Util::AbstractConfiguration & config, const std::string & config_name, unsigned default_connections = MYSQLXX_POOL_WITH_FAILOVER_DEFAULT_START_CONNECTIONS, unsigned max_connections = MYSQLXX_POOL_WITH_FAILOVER_DEFAULT_MAX_CONNECTIONS, diff --git a/base/mysqlxx/PoolWithFailover.cpp b/base/mysqlxx/PoolWithFailover.cpp index 2beb50d3da8..a763988331b 100644 --- a/base/mysqlxx/PoolWithFailover.cpp +++ b/base/mysqlxx/PoolWithFailover.cpp @@ -68,7 +68,7 @@ PoolWithFailover::PoolWithFailover(const PoolWithFailover & other) } } -PoolWithFailover::Entry PoolWithFailover::Get() +PoolWithFailover::Entry PoolWithFailover::get() { Poco::Util::Application & app = Poco::Util::Application::instance(); std::lock_guard locker(mutex); @@ -89,7 +89,7 @@ PoolWithFailover::Entry PoolWithFailover::Get() try { - Entry entry = shareable ? pool->Get() : pool->tryGet(); + Entry entry = shareable ? pool->get() : pool->tryGet(); if (!entry.isNull()) { @@ -121,7 +121,7 @@ PoolWithFailover::Entry PoolWithFailover::Get() if (full_pool) { app.logger().error("All connections failed, trying to wait on a full pool " + (*full_pool)->getDescription()); - return (*full_pool)->Get(); + return (*full_pool)->get(); } std::stringstream message; diff --git a/base/mysqlxx/PoolWithFailover.h b/base/mysqlxx/PoolWithFailover.h index fe151240fa5..9db29db5efa 100644 --- a/base/mysqlxx/PoolWithFailover.h +++ b/base/mysqlxx/PoolWithFailover.h @@ -105,6 +105,6 @@ namespace mysqlxx PoolWithFailover(const PoolWithFailover & other); /** Allocates a connection to use. */ - Entry Get(); + Entry get(); }; } diff --git a/base/mysqlxx/Value.cpp b/base/mysqlxx/Value.cpp index 4fed33c5da9..ed66167e8ea 100644 --- a/base/mysqlxx/Value.cpp +++ b/base/mysqlxx/Value.cpp @@ -154,7 +154,7 @@ double Value::readFloatText(const char * buf, size_t length) const void Value::throwException(const char * text) const { - static constexpr size_t MYSQLXX_QUERY_PREVIEW_LENGTH = 1000; + static constexpr size_t preview_length = 1000; std::stringstream info; info << text; @@ -166,7 +166,7 @@ void Value::throwException(const char * text) const } if (res && res->getQuery()) - info << ", query: " << res->getQuery()->str().substr(0, MYSQLXX_QUERY_PREVIEW_LENGTH); + info << ", query: " << res->getQuery()->str().substr(0, preview_length); throw CannotParseValue(info.str()); } diff --git a/dbms/programs/client/Client.cpp b/dbms/programs/client/Client.cpp index 21768911beb..42a06925c7d 100644 --- a/dbms/programs/client/Client.cpp +++ b/dbms/programs/client/Client.cpp @@ -1122,7 +1122,7 @@ private: /// to avoid losing sync. if (!cancelled) { - auto cancelQuery = [&] { + auto cancel_query = [&] { connection->sendCancel(); cancelled = true; if (is_interactive) @@ -1134,7 +1134,7 @@ private: if (interrupt_listener.check()) { - cancelQuery(); + cancel_query(); } else { @@ -1145,7 +1145,7 @@ private: << " Waited for " << static_cast(elapsed) << " seconds," << " timeout is " << receive_timeout.totalSeconds() << " seconds." << std::endl; - cancelQuery(); + cancel_query(); } } } diff --git a/dbms/programs/copier/ClusterCopierApp.cpp b/dbms/programs/copier/ClusterCopierApp.cpp index c70e79313b9..e49c0d8eca4 100644 --- a/dbms/programs/copier/ClusterCopierApp.cpp +++ b/dbms/programs/copier/ClusterCopierApp.cpp @@ -43,11 +43,11 @@ void ClusterCopierApp::initialize(Poco::Util::Application & self) void ClusterCopierApp::handleHelp(const std::string &, const std::string &) { - Poco::Util::HelpFormatter helpFormatter(options()); - helpFormatter.setCommand(commandName()); - helpFormatter.setHeader("Copies tables from one cluster to another"); - helpFormatter.setUsage("--config-file --task-path "); - helpFormatter.format(std::cerr); + Poco::Util::HelpFormatter help_formatter(options()); + help_formatter.setCommand(commandName()); + help_formatter.setHeader("Copies tables from one cluster to another"); + help_formatter.setUsage("--config-file --task-path "); + help_formatter.format(std::cerr); stopOptionsProcessing(); } diff --git a/dbms/programs/odbc-bridge/ODBCBridge.cpp b/dbms/programs/odbc-bridge/ODBCBridge.cpp index 9e1e8400cc1..e7e8aca7147 100644 --- a/dbms/programs/odbc-bridge/ODBCBridge.cpp +++ b/dbms/programs/odbc-bridge/ODBCBridge.cpp @@ -79,11 +79,11 @@ namespace void ODBCBridge::handleHelp(const std::string &, const std::string &) { - Poco::Util::HelpFormatter helpFormatter(options()); - helpFormatter.setCommand(commandName()); - helpFormatter.setHeader("HTTP-proxy for odbc requests"); - helpFormatter.setUsage("--http-port "); - helpFormatter.format(std::cerr); + Poco::Util::HelpFormatter help_formatter(options()); + help_formatter.setCommand(commandName()); + help_formatter.setHeader("HTTP-proxy for odbc requests"); + help_formatter.setUsage("--http-port "); + help_formatter.format(std::cerr); stopOptionsProcessing(); } diff --git a/dbms/programs/server/HTTPHandler.cpp b/dbms/programs/server/HTTPHandler.cpp index 65b605c993d..ec890c0a96d 100644 --- a/dbms/programs/server/HTTPHandler.cpp +++ b/dbms/programs/server/HTTPHandler.cpp @@ -546,7 +546,7 @@ void HTTPHandler::processQuery( client_info.http_method = http_method; client_info.http_user_agent = request.get("User-Agent", ""); - auto appendCallback = [&context] (ProgressCallback callback) + auto append_callback = [&context] (ProgressCallback callback) { auto prev = context.getProgressCallback(); @@ -561,13 +561,13 @@ void HTTPHandler::processQuery( /// While still no data has been sent, we will report about query execution progress by sending HTTP headers. if (settings.send_progress_in_http_headers) - appendCallback([&used_output] (const Progress & progress) { used_output.out->onProgress(progress); }); + append_callback([&used_output] (const Progress & progress) { used_output.out->onProgress(progress); }); if (settings.readonly > 0 && settings.cancel_http_readonly_queries_on_client_close) { Poco::Net::StreamSocket & socket = dynamic_cast(request).socket(); - appendCallback([&context, &socket](const Progress &) + append_callback([&context, &socket](const Progress &) { /// Assume that at the point this method is called no one is reading data from the socket any more. /// True for read-only queries. diff --git a/dbms/programs/server/HTTPHandlerFactory.cpp b/dbms/programs/server/HTTPHandlerFactory.cpp index ab8fb4f7336..7a1bcaf4fc6 100644 --- a/dbms/programs/server/HTTPHandlerFactory.cpp +++ b/dbms/programs/server/HTTPHandlerFactory.cpp @@ -23,9 +23,9 @@ Poco::Net::HTTPRequestHandler * HTTPRequestHandlerFactoryMain::createRequestHand << ", Content Type: " << request.getContentType() << ", Transfer Encoding: " << request.getTransferEncoding()); - for (auto & handlerFactory: child_handler_factories) + for (auto & handler_factory : child_handler_factories) { - auto handler = handlerFactory->createRequestHandler(request); + auto handler = handler_factory->createRequestHandler(request); if (handler != nullptr) return handler; } diff --git a/dbms/programs/server/MySQLHandler.cpp b/dbms/programs/server/MySQLHandler.cpp index aaa8cec90b2..3e1432dbfce 100644 --- a/dbms/programs/server/MySQLHandler.cpp +++ b/dbms/programs/server/MySQLHandler.cpp @@ -252,8 +252,8 @@ void MySQLHandler::comFieldList(ReadBuffer & payload) ComFieldList packet; packet.readPayload(payload); String database = connection_context.getCurrentDatabase(); - StoragePtr tablePtr = DatabaseCatalog::instance().getTable({database, packet.table}); - for (const NameAndTypePair & column: tablePtr->getColumns().getAll()) + StoragePtr table_ptr = DatabaseCatalog::instance().getTable({database, packet.table}); + for (const NameAndTypePair & column: table_ptr->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/dbms/programs/server/MySQLHandlerFactory.cpp b/dbms/programs/server/MySQLHandlerFactory.cpp index dceb3d2293e..925a1e0af5f 100644 --- a/dbms/programs/server/MySQLHandlerFactory.cpp +++ b/dbms/programs/server/MySQLHandlerFactory.cpp @@ -56,26 +56,26 @@ MySQLHandlerFactory::MySQLHandlerFactory(IServer & server_) void MySQLHandlerFactory::readRSAKeys() { const Poco::Util::LayeredConfiguration & config = Poco::Util::Application::instance().config(); - String certificateFileProperty = "openSSL.server.certificateFile"; - String privateKeyFileProperty = "openSSL.server.privateKeyFile"; + String certificate_file_property = "openSSL.server.certificateFile"; + String private_key_file_property = "openSSL.server.privateKeyFile"; - if (!config.has(certificateFileProperty)) + if (!config.has(certificate_file_property)) throw Exception("Certificate file is not set.", ErrorCodes::NO_ELEMENTS_IN_CONFIG); - if (!config.has(privateKeyFileProperty)) + if (!config.has(private_key_file_property)) throw Exception("Private key file is not set.", ErrorCodes::NO_ELEMENTS_IN_CONFIG); { - String certificateFile = config.getString(certificateFileProperty); - FILE * fp = fopen(certificateFile.data(), "r"); + String certificate_file = config.getString(certificate_file_property); + FILE * fp = fopen(certificate_file.data(), "r"); if (fp == nullptr) - throw Exception("Cannot open certificate file: " + certificateFile + ".", ErrorCodes::CANNOT_OPEN_FILE); + throw Exception("Cannot open certificate file: " + certificate_file + ".", ErrorCodes::CANNOT_OPEN_FILE); SCOPE_EXIT(fclose(fp)); X509 * x509 = PEM_read_X509(fp, nullptr, nullptr, nullptr); SCOPE_EXIT(X509_free(x509)); if (x509 == nullptr) - throw Exception("Failed to read PEM certificate from " + certificateFile + ". Error: " + getOpenSSLErrors(), ErrorCodes::OPENSSL_ERROR); + throw Exception("Failed to read PEM certificate from " + certificate_file + ". Error: " + getOpenSSLErrors(), ErrorCodes::OPENSSL_ERROR); EVP_PKEY * p = X509_get_pubkey(x509); if (p == nullptr) @@ -88,16 +88,16 @@ void MySQLHandlerFactory::readRSAKeys() } { - String privateKeyFile = config.getString(privateKeyFileProperty); + String private_key_file = config.getString(private_key_file_property); - FILE * fp = fopen(privateKeyFile.data(), "r"); + FILE * fp = fopen(private_key_file.data(), "r"); if (fp == nullptr) - throw Exception ("Cannot open private key file " + privateKeyFile + ".", ErrorCodes::CANNOT_OPEN_FILE); + throw Exception ("Cannot open private key file " + private_key_file + ".", ErrorCodes::CANNOT_OPEN_FILE); SCOPE_EXIT(fclose(fp)); private_key.reset(PEM_read_RSAPrivateKey(fp, nullptr, nullptr, nullptr)); if (!private_key) - throw Exception("Failed to read RSA private key from " + privateKeyFile + ". Error: " + getOpenSSLErrors(), ErrorCodes::OPENSSL_ERROR); + throw Exception("Failed to read RSA private key from " + private_key_file + ". Error: " + getOpenSSLErrors(), ErrorCodes::OPENSSL_ERROR); } } diff --git a/dbms/programs/server/Server.cpp b/dbms/programs/server/Server.cpp index 96ba2883480..5322514e7c2 100644 --- a/dbms/programs/server/Server.cpp +++ b/dbms/programs/server/Server.cpp @@ -162,12 +162,12 @@ int Server::run() { if (config().hasOption("help")) { - Poco::Util::HelpFormatter helpFormatter(Server::options()); + Poco::Util::HelpFormatter help_formatter(Server::options()); std::stringstream header; header << commandName() << " [OPTION] [-- [ARG]...]\n"; header << "positional arguments can be used to rewrite config.xml properties, for example, --http_port=8010"; - helpFormatter.setHeader(header.str()); - helpFormatter.format(std::cout); + help_formatter.setHeader(header.str()); + help_formatter.format(std::cout); return 0; } if (config().hasOption("version")) diff --git a/dbms/src/Access/QuotaCache.cpp b/dbms/src/Access/QuotaCache.cpp index cdf298d0e57..6db3eb66c5d 100644 --- a/dbms/src/Access/QuotaCache.cpp +++ b/dbms/src/Access/QuotaCache.cpp @@ -101,7 +101,7 @@ boost::shared_ptr QuotaCache::QuotaInfo::rebuildI new_intervals->quota_key = key; auto & intervals = new_intervals->intervals; intervals.reserve(quota->all_limits.size()); - constexpr size_t MAX_RESOURCE_TYPE = Quota::MAX_RESOURCE_TYPE; + static constexpr size_t MAX_RESOURCE_TYPE = Quota::MAX_RESOURCE_TYPE; for (const auto & limits : quota->all_limits) { intervals.emplace_back(); diff --git a/dbms/src/AggregateFunctions/AggregateFunctionGroupArrayMoving.cpp b/dbms/src/AggregateFunctions/AggregateFunctionGroupArrayMoving.cpp index 52fe70a30b5..31a921c3b2c 100644 --- a/dbms/src/AggregateFunctions/AggregateFunctionGroupArrayMoving.cpp +++ b/dbms/src/AggregateFunctions/AggregateFunctionGroupArrayMoving.cpp @@ -20,32 +20,32 @@ namespace ErrorCodes namespace { -template +template struct MovingSum { using DataType = MovingSumData; - using Function = MovingImpl; + using Function = MovingImpl; }; -template +template struct MovingAvg { using DataType = MovingAvgData; - using Function = MovingImpl; + using Function = MovingImpl; }; -template using MovingSumTemplate = typename MovingSum::Function; -template using MovingAvgTemplate = typename MovingAvg::Function; +template using MovingSumTemplate = typename MovingSum::Function; +template using MovingAvgTemplate = typename MovingAvg::Function; -template