From 11e86ed64f59f8d0ba379ba6bfd7267dc1409e66 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Wed, 7 Oct 2020 21:37:27 +0300 Subject: [PATCH 001/205] Refactor ExpressionActions [Part 1]. --- src/Interpreters/ExpressionActions.cpp | 101 +++++++++++++++++++++++++ src/Interpreters/ExpressionActions.h | 99 ++++++++++-------------- 2 files changed, 142 insertions(+), 58 deletions(-) diff --git a/src/Interpreters/ExpressionActions.cpp b/src/Interpreters/ExpressionActions.cpp index 36c08d945eb..b56be811a53 100644 --- a/src/Interpreters/ExpressionActions.cpp +++ b/src/Interpreters/ExpressionActions.cpp @@ -629,6 +629,107 @@ void ExpressionActions::execute(Block & block, bool dry_run) const } } +void ExpressionActions::executeAction(const Action & action, ExecutionContext & execution_context, bool dry_run) +{ + switch (action.node->type) + { + case ActionsDAG::Type::FUNCTION: + { + ColumnNumbers arguments(argument_names.size()); + for (size_t i = 0; i < argument_names.size(); ++i) + arguments[i] = block.getPositionByName(argument_names[i]); + + size_t num_columns_without_result = block.columns(); + block.insert({ nullptr, result_type, result_name}); + + ProfileEvents::increment(ProfileEvents::FunctionExecute); + if (is_function_compiled) + ProfileEvents::increment(ProfileEvents::CompiledFunctionExecute); + function->execute(block, arguments, num_columns_without_result, input_rows_count, dry_run); + + break; + } + + case ARRAY_JOIN: + { + auto source = block.getByName(source_name); + block.erase(source_name); + source.column = source.column->convertToFullColumnIfConst(); + + const ColumnArray * array = typeid_cast(source.column.get()); + if (!array) + throw Exception("ARRAY JOIN of not array: " + source_name, ErrorCodes::TYPE_MISMATCH); + + for (auto & column : block) + column.column = column.column->replicate(array->getOffsets()); + + source.column = array->getDataPtr(); + source.type = assert_cast(*source.type).getNestedType(); + source.name = result_name; + + block.insert(std::move(source)); + + break; + } + + case PROJECT: + { + Block new_block; + + for (const auto & elem : projection) + { + const std::string & name = elem.first; + const std::string & alias = elem.second; + ColumnWithTypeAndName column = block.getByName(name); + if (!alias.empty()) + column.name = alias; + new_block.insert(std::move(column)); + } + + block.swap(new_block); + + break; + } + + case ADD_ALIASES: + { + for (const auto & elem : projection) + { + const std::string & name = elem.first; + const std::string & alias = elem.second; + const ColumnWithTypeAndName & column = block.getByName(name); + if (!alias.empty() && !block.has(alias)) + block.insert({column.column, column.type, alias}); + } + break; + } + + case REMOVE_COLUMN: + block.erase(source_name); + break; + + case ADD_COLUMN: + block.insert({ added_column->cloneResized(input_rows_count), result_type, result_name }); + break; + + case COPY_COLUMN: + if (can_replace && block.has(result_name)) + { + auto & result = block.getByName(result_name); + const auto & source = block.getByName(source_name); + result.type = source.type; + result.column = source.column; + } + else + { + const auto & source_column = block.getByName(source_name); + block.insert({source_column.column, source_column.type, result_name}); + } + + break; + } +} + bool ExpressionActions::hasArrayJoin() const { for (const auto & action : actions) diff --git a/src/Interpreters/ExpressionActions.h b/src/Interpreters/ExpressionActions.h index b35f8972c97..7a6c050563c 100644 --- a/src/Interpreters/ExpressionActions.h +++ b/src/Interpreters/ExpressionActions.h @@ -193,8 +193,8 @@ public: ActionsDAG() = default; ActionsDAG(const ActionsDAG &) = delete; ActionsDAG & operator=(const ActionsDAG &) = delete; - ActionsDAG(const NamesAndTypesList & inputs); - ActionsDAG(const ColumnsWithTypeAndName & inputs); + explicit ActionsDAG(const NamesAndTypesList & inputs); + explicit ActionsDAG(const ColumnsWithTypeAndName & inputs); const Index & getIndex() const { return index; } @@ -227,29 +227,45 @@ using ActionsDAGPtr = std::shared_ptr; */ class ExpressionActions { +private: + using Node = ActionsDAG::Node; + using Index = ActionsDAG::Index; + + struct Argument + { + size_t position; + bool can_remove; + }; + + using Arguments = std::vector; + + struct Action + { + Node * node; + Arguments arguments; + size_t result_position; + }; + + using Actions = std::vector; + + struct ExecutionContext + { + ColumnsWithTypeAndName input_columns; + ColumnsWithTypeAndName columns; + size_t num_rows; + }; + + std::list nodes; + Index index; + Actions actions; + + NamesAndTypesList required_columns; + public: - using Actions = std::vector; - - ExpressionActions(const NamesAndTypesList & input_columns_, const Context & context_); - - /// For constant columns the columns themselves can be contained in `input_columns_`. - ExpressionActions(const ColumnsWithTypeAndName & input_columns_, const Context & context_); - ~ExpressionActions(); ExpressionActions(const ExpressionActions & other) = default; - /// Add the input column. - /// The name of the column must not match the names of the intermediate columns that occur when evaluating the expression. - /// The expression must not have any PROJECT actions. - void addInput(const ColumnWithTypeAndName & column); - void addInput(const NameAndTypePair & column); - - void add(const ExpressionAction & action); - - /// Adds new column names to out_new_columns (formed as a result of the added action). - void add(const ExpressionAction & action, Names & out_new_columns); - /// Adds to the beginning the removal of all extra columns. void prependProjectInput(); @@ -263,20 +279,11 @@ public: /// - Does not reorder the columns. /// - Does not remove "unexpected" columns (for example, added by functions). /// - If output_columns is empty, leaves one arbitrary column (so that the number of rows in the block is not lost). - void finalize(const Names & output_columns); - - const Actions & getActions() const { return actions; } + // void finalize(const Names & output_columns); /// Get a list of input columns. - Names getRequiredColumns() const - { - Names names; - for (const auto & input : input_columns) - names.push_back(input.name); - return names; - } - - const NamesAndTypesList & getRequiredColumnsWithTypes() const { return input_columns; } + Names getRequiredColumns() const; + const NamesAndTypesList & getRequiredColumnsWithTypes() const; /// Execute the expression on the block. The block must contain all the columns returned by getRequiredColumns. void execute(Block & block, bool dry_run = false) const; @@ -284,7 +291,7 @@ public: bool hasArrayJoin() const; /// Obtain a sample block that contains the names and types of result columns. - const Block & getSampleBlock() const { return sample_block; } + const Block & getSampleBlock() const; std::string dumpActions() const; @@ -296,28 +303,7 @@ public: /// Call it only after subqueries for sets were executed. bool checkColumnIsAlwaysFalse(const String & column_name) const; - struct ActionsHash - { - UInt128 operator()(const ExpressionActions::Actions & elems) const - { - SipHash hash; - for (const ExpressionAction & act : elems) - hash.update(ExpressionAction::ActionHash{}(act)); - UInt128 result; - hash.get128(result.low, result.high); - return result; - } - }; - private: - /// These columns have to be in input blocks (arguments of execute* methods) - NamesAndTypesList input_columns; - /// These actions will be executed on input blocks - Actions actions; - /// The example of result (output) block. - Block sample_block; - /// Columns which can't be used for constant folding. - NameSet names_not_for_constant_folding; Settings settings; #if USE_EMBEDDED_COMPILER @@ -326,10 +312,7 @@ private: void checkLimits(Block & block) const; - void addImpl(ExpressionAction action, Names & new_names); - - /// Move all arrayJoin as close as possible to the end. - void optimizeArrayJoin(); + void executeAction(const Action & action, ExecutionContext & execution_context, bool dry_run); }; From 07c27301690039573d447f3fbad4aac09b03b234 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 13 Oct 2020 11:16:47 +0300 Subject: [PATCH 002/205] Refactor ExpressionActions. --- src/Core/Block.h | 2 +- src/Interpreters/ExpressionActions.cpp | 296 ++++++++----------------- src/Interpreters/ExpressionActions.h | 29 ++- 3 files changed, 105 insertions(+), 222 deletions(-) diff --git a/src/Core/Block.h b/src/Core/Block.h index f05cc2b52eb..f55a75162c6 100644 --- a/src/Core/Block.h +++ b/src/Core/Block.h @@ -156,7 +156,7 @@ private: /// This is needed to allow function execution over data. /// It is safe because functions does not change column names, so index is unaffected. /// It is temporary. - friend struct ExpressionAction; + friend class ExpressionActions; friend class ActionsDAG; }; diff --git a/src/Interpreters/ExpressionActions.cpp b/src/Interpreters/ExpressionActions.cpp index 6767642388a..53fb0d7a22d 100644 --- a/src/Interpreters/ExpressionActions.cpp +++ b/src/Interpreters/ExpressionActions.cpp @@ -523,7 +523,7 @@ ExpressionActions::ExpressionActions(const ColumnsWithTypeAndName & input_column ExpressionActions::~ExpressionActions() = default; -void ExpressionActions::checkLimits(Block & block) const +void ExpressionActions::checkLimits(ExecutionContext & execution_context) const { if (settings.max_temporary_columns && block.columns() > settings.max_temporary_columns) throw Exception("Too many temporary columns: " + block.dumpNames() @@ -551,62 +551,6 @@ void ExpressionActions::checkLimits(Block & block) const } } -void ExpressionActions::addInput(const ColumnWithTypeAndName & column) -{ - input_columns.emplace_back(column.name, column.type); - sample_block.insert(column); -} - -void ExpressionActions::addInput(const NameAndTypePair & column) -{ - addInput(ColumnWithTypeAndName(nullptr, column.type, column.name)); -} - -void ExpressionActions::add(const ExpressionAction & action, Names & out_new_columns) -{ - addImpl(action, out_new_columns); -} - -void ExpressionActions::add(const ExpressionAction & action) -{ - Names new_names; - addImpl(action, new_names); -} - -void ExpressionActions::addImpl(ExpressionAction action, Names & new_names) -{ - if (!action.result_name.empty()) - new_names.push_back(action.result_name); - - /// Compiled functions are custom functions and they don't need building - if (action.type == ExpressionAction::APPLY_FUNCTION && !action.is_function_compiled) - { - if (sample_block.has(action.result_name)) - throw Exception("Column '" + action.result_name + "' already exists", ErrorCodes::DUPLICATE_COLUMN); - - ColumnsWithTypeAndName arguments(action.argument_names.size()); - for (size_t i = 0; i < action.argument_names.size(); ++i) - { - if (!sample_block.has(action.argument_names[i])) - throw Exception("Unknown identifier: '" + action.argument_names[i] + "'", ErrorCodes::UNKNOWN_IDENTIFIER); - arguments[i] = sample_block.getByName(action.argument_names[i]); - } - - if (!action.function_base) - { - action.function_base = action.function_builder->build(arguments); - action.result_type = action.function_base->getReturnType(); - } - } - - if (action.type == ExpressionAction::ADD_ALIASES) - for (const auto & name_with_alias : action.projection) - new_names.emplace_back(name_with_alias.second); - - action.prepare(sample_block, settings, names_not_for_constant_folding); - actions.push_back(action); -} - void ExpressionActions::prependProjectInput() { actions.insert(actions.begin(), ExpressionAction::project(getRequiredColumns())); @@ -614,12 +558,33 @@ void ExpressionActions::prependProjectInput() void ExpressionActions::execute(Block & block, bool dry_run) const { + ExecutionContext execution_context + { + .input_columns = block.data, + .num_rows = block.rows(), + }; + + execution_context.columns.reserve(num_columns); + + ColumnNumbers inputs_to_remove; + inputs_to_remove.reserve(required_columns.size()); + for (const auto & column : required_columns) + { + size_t pos = block.getPositionByName(column.name); + execution_context.columns.emplace_back(std::move(block.getByPosition(pos))); + + if (!sample_block.has(column.name)) + inputs_to_remove.emplace_back(pos); + } + + execution_context.columns.resize(num_columns); + for (const auto & action : actions) { try { - action.execute(block, dry_run); - checkLimits(block); + executeAction(action, execution_context, dry_run); + checkLimits(execution_context); } catch (Exception & e) { @@ -627,113 +592,112 @@ void ExpressionActions::execute(Block & block, bool dry_run) const throw; } } + + std::sort(inputs_to_remove.rbegin(), inputs_to_remove.rend()); + for (auto input : inputs_to_remove) + block.erase(input); + + for (const auto & action : actions) + { + if (!action.is_used_in_result) + continue; + + auto & column = execution_context.columns[action.result_position]; + column.name = action.node->result_name; + + if (block.has(action.node->result_name)) + block.getByName(action.node->result_name) = std::move(column); + else + block.insert(std::move(column)); + } } void ExpressionActions::executeAction(const Action & action, ExecutionContext & execution_context, bool dry_run) { + auto & columns = execution_context.columns; + auto & num_rows = execution_context.num_rows; + switch (action.node->type) { case ActionsDAG::Type::FUNCTION: { - ColumnNumbers arguments(argument_names.size()); - for (size_t i = 0; i < argument_names.size(); ++i) - arguments[i] = block.getPositionByName(argument_names[i]); + auto & res_column = columns[action.result_position]; + if (res_column.type || res_column.column) + throw Exception("Result column is not empty", ErrorCodes::LOGICAL_ERROR); - size_t num_columns_without_result = block.columns(); - block.insert({ nullptr, result_type, result_name}); + res_column.type = action.node->result_type; + /// Columns names are not used, avoid extra copy. + /// res_column.name = action.node->result_name; ProfileEvents::increment(ProfileEvents::FunctionExecute); - if (is_function_compiled) + if (action.node->is_function_compiled) ProfileEvents::increment(ProfileEvents::CompiledFunctionExecute); - function->execute(block, arguments, num_columns_without_result, input_rows_count, dry_run); + action.node->function->execute(columns, action.arguments, action.result_position, num_rows, dry_run); break; } - case ARRAY_JOIN: + case ActionsDAG::Type::ARRAY_JOIN: { - auto source = block.getByName(source_name); - block.erase(source_name); - source.column = source.column->convertToFullColumnIfConst(); + size_t array_join_key_pos = action.arguments.front(); + auto array_join_key = columns[array_join_key_pos]; - const ColumnArray * array = typeid_cast(source.column.get()); + /// Remove array join argument in advance if it is not needed. + if (!action.to_remove.empty()) + columns[array_join_key_pos] = {}; + + array_join_key.column = array_join_key.column->convertToFullColumnIfConst(); + + const ColumnArray * array = typeid_cast(array_join_key.column.get()); if (!array) - throw Exception("ARRAY JOIN of not array: " + source_name, ErrorCodes::TYPE_MISMATCH); + throw Exception("ARRAY JOIN of not array: " + action.node->result_name, ErrorCodes::TYPE_MISMATCH); - for (auto & column : block) - column.column = column.column->replicate(array->getOffsets()); + for (auto & column : columns) + if (column.column) + column.column = column.column->replicate(array->getOffsets()); - source.column = array->getDataPtr(); - source.type = assert_cast(*source.type).getNestedType(); - source.name = result_name; + for (auto & column : execution_context.input_columns) + if (column.column) + column.column = column.column->replicate(array->getOffsets()); - block.insert(std::move(source)); + auto & res_column = columns[action.result_position]; + res_column.column = array->getDataPtr(); + res_column.type = assert_cast(*array_join_key.type).getNestedType(); + + num_rows = res_column.column->size(); break; } - case PROJECT: + case ActionsDAG::Type::COLUMN: { - Block new_block; - - for (const auto & elem : projection) - { - const std::string & name = elem.first; - const std::string & alias = elem.second; - ColumnWithTypeAndName column = block.getByName(name); - if (!alias.empty()) - column.name = alias; - new_block.insert(std::move(column)); - } - - block.swap(new_block); - + auto & res_column = columns[action.result_position]; + res_column.column = action.node->column->cloneResized(num_rows); + res_column.type = action.node->result_type; break; } - case ADD_ALIASES: + case ActionsDAG::Type::ALIAS: { - for (const auto & elem : projection) - { - const std::string & name = elem.first; - const std::string & alias = elem.second; - const ColumnWithTypeAndName & column = block.getByName(name); - if (!alias.empty() && !block.has(alias)) - block.insert({column.column, column.type, alias}); - } + /// Do not care about names, they are empty. + columns[action.result_position] = columns[action.arguments.front()]; break; } - case REMOVE_COLUMN: - block.erase(source_name); - break; - - case ADD_COLUMN: - block.insert({ added_column->cloneResized(input_rows_count), result_type, result_name }); - break; - - case COPY_COLUMN: - if (can_replace && block.has(result_name)) - { - auto & result = block.getByName(result_name); - const auto & source = block.getByName(source_name); - result.type = source.type; - result.column = source.column; - } - else - { - const auto & source_column = block.getByName(source_name); - block.insert({source_column.column, source_column.type, result_name}); - } - - break; + case ActionsDAG::Type::INPUT: + { + throw Exception("Cannot execute INPUT action", ErrorCodes::LOGICAL_ERROR); + } } + + for (auto to_remove : action.to_remove) + columns[to_remove] = {}; } bool ExpressionActions::hasArrayJoin() const { for (const auto & action : actions) - if (action.type == ExpressionAction::ARRAY_JOIN) + if (action.node->type == ActionsDAG::Type::ARRAY_JOIN) return true; return false; @@ -1013,86 +977,6 @@ std::string ExpressionActions::dumpActions() const return ss.str(); } -void ExpressionActions::optimizeArrayJoin() -{ - const size_t none = actions.size(); - size_t first_array_join = none; - - /// Columns that need to be evaluated for arrayJoin. - /// Actions for adding them can not be moved to the left of the arrayJoin. - NameSet array_joined_columns; - - /// Columns needed to evaluate arrayJoin or those that depend on it. - /// Actions to delete them can not be moved to the left of the arrayJoin. - NameSet array_join_dependencies; - - for (size_t i = 0; i < actions.size(); ++i) - { - /// Do not move the action to the right of the projection (the more that they are not usually there). - if (actions[i].type == ExpressionAction::PROJECT) - break; - - bool depends_on_array_join = false; - Names needed; - - if (actions[i].type == ExpressionAction::ARRAY_JOIN) - { - depends_on_array_join = true; - needed = actions[i].getNeededColumns(); - } - else - { - if (first_array_join == none) - continue; - - needed = actions[i].getNeededColumns(); - - for (const auto & elem : needed) - { - if (array_joined_columns.count(elem)) - { - depends_on_array_join = true; - break; - } - } - } - - if (depends_on_array_join) - { - if (first_array_join == none) - first_array_join = i; - - if (!actions[i].result_name.empty()) - array_joined_columns.insert(actions[i].result_name); - - array_join_dependencies.insert(needed.begin(), needed.end()); - } - else - { - bool can_move = false; - - if (actions[i].type == ExpressionAction::REMOVE_COLUMN) - { - /// If you delete a column that is not needed for arrayJoin (and those who depend on it), you can delete it before arrayJoin. - can_move = !array_join_dependencies.count(actions[i].source_name); - } - else - { - /// If the action does not delete the columns and does not depend on the result of arrayJoin, you can make it until arrayJoin. - can_move = true; - } - - /// Move the current action to the position just before the first arrayJoin. - if (can_move) - { - /// Move the i-th element to the position `first_array_join`. - std::rotate(actions.begin() + first_array_join, actions.begin() + i, actions.begin() + i + 1); - ++first_array_join; - } - } - } -} - ExpressionActionsPtr ExpressionActions::splitActionsBeforeArrayJoin(const NameSet & array_joined_columns) { /// Create new actions. diff --git a/src/Interpreters/ExpressionActions.h b/src/Interpreters/ExpressionActions.h index 7a6c050563c..75edd4bf7ee 100644 --- a/src/Interpreters/ExpressionActions.h +++ b/src/Interpreters/ExpressionActions.h @@ -175,6 +175,8 @@ public: FunctionBasePtr function_base; /// Prepared function which is used in function execution. ExecutableFunctionPtr function; + /// If function is a compiled statement. + bool is_function_compiled = false; /// For COLUMN node and propagated constants. ColumnPtr column; @@ -231,35 +233,32 @@ private: using Node = ActionsDAG::Node; using Index = ActionsDAG::Index; - struct Argument - { - size_t position; - bool can_remove; - }; - - using Arguments = std::vector; - struct Action { Node * node; - Arguments arguments; + ColumnNumbers arguments; + /// Columns which will be removed after actions is executed. + /// It is always a subset of arguments. + ColumnNumbers to_remove; size_t result_position; + bool is_used_in_result; }; using Actions = std::vector; struct ExecutionContext { - ColumnsWithTypeAndName input_columns; + ColumnsWithTypeAndName & input_columns; ColumnsWithTypeAndName columns; size_t num_rows; }; std::list nodes; - Index index; Actions actions; + size_t num_columns; NamesAndTypesList required_columns; + Block sample_block; public: ~ExpressionActions(); @@ -283,7 +282,7 @@ public: /// Get a list of input columns. Names getRequiredColumns() const; - const NamesAndTypesList & getRequiredColumnsWithTypes() const; + const NamesAndTypesList & getRequiredColumnsWithTypes() const { return required_columns; } /// Execute the expression on the block. The block must contain all the columns returned by getRequiredColumns. void execute(Block & block, bool dry_run = false) const; @@ -291,7 +290,7 @@ public: bool hasArrayJoin() const; /// Obtain a sample block that contains the names and types of result columns. - const Block & getSampleBlock() const; + const Block & getSampleBlock() const { return sample_block; } std::string dumpActions() const; @@ -310,9 +309,9 @@ private: std::shared_ptr compilation_cache; #endif - void checkLimits(Block & block) const; + void checkLimits(ExecutionContext & execution_context) const; - void executeAction(const Action & action, ExecutionContext & execution_context, bool dry_run); + static void executeAction(const Action & action, ExecutionContext & execution_context, bool dry_run); }; From 47ebc9f74e7d201fbdfdb7678420ae63e9948a4d Mon Sep 17 00:00:00 2001 From: alesapin Date: Mon, 26 Oct 2020 14:02:47 +0300 Subject: [PATCH 003/205] Add separate pool for fetches --- .../MergeTree/BackgroundJobsExecutor.cpp | 4 ++- .../MergeTree/BackgroundJobsExecutor.h | 1 + .../MergeTree/ReplicatedMergeTreeQueue.cpp | 9 +++++ src/Storages/StorageReplicatedMergeTree.cpp | 33 ++++++++++++++++++- src/Storages/StorageReplicatedMergeTree.h | 4 +++ 5 files changed, 49 insertions(+), 2 deletions(-) diff --git a/src/Storages/MergeTree/BackgroundJobsExecutor.cpp b/src/Storages/MergeTree/BackgroundJobsExecutor.cpp index 5aba208a86e..a2eede8a0db 100644 --- a/src/Storages/MergeTree/BackgroundJobsExecutor.cpp +++ b/src/Storages/MergeTree/BackgroundJobsExecutor.cpp @@ -9,6 +9,7 @@ namespace CurrentMetrics { extern const Metric BackgroundPoolTask; extern const Metric BackgroundMovePoolTask; + extern const Metric BackgroundFetchesPoolTask; } namespace DB @@ -183,7 +184,8 @@ BackgroundJobsExecutor::BackgroundJobsExecutor( : IBackgroundJobExecutor( global_context_, global_context_.getBackgroundProcessingTaskSchedulingSettings(), - {PoolConfig{PoolType::MERGE_MUTATE, global_context_.getSettingsRef().background_pool_size, CurrentMetrics::BackgroundPoolTask}}) + {PoolConfig{PoolType::MERGE_MUTATE, global_context_.getSettingsRef().background_pool_size, CurrentMetrics::BackgroundPoolTask}, + PoolConfig{PoolType::FETCH, global_context_.getSettingsRef().background_fetches_pool_size, CurrentMetrics::BackgroundFetchesPoolTask}}) , data(data_) { } diff --git a/src/Storages/MergeTree/BackgroundJobsExecutor.h b/src/Storages/MergeTree/BackgroundJobsExecutor.h index 537a6064c86..de81472012e 100644 --- a/src/Storages/MergeTree/BackgroundJobsExecutor.h +++ b/src/Storages/MergeTree/BackgroundJobsExecutor.h @@ -33,6 +33,7 @@ enum class PoolType { MERGE_MUTATE, MOVE, + FETCH, }; /// Result from background job providers. Function which will be executed in pool and pool type. diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp index 95358ecee97..3f35c3694af 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp @@ -1033,6 +1033,15 @@ bool ReplicatedMergeTreeQueue::shouldExecuteLogEntry( } } + if (entry.type == LogEntry::GET_PART) + { + if (!storage.canExecuteFetch(entry, out_postpone_reason)) + { + LOG_TRACE(log, out_postpone_reason); + return false; + } + } + if (entry.type == LogEntry::MERGE_PARTS || entry.type == LogEntry::MUTATE_PART) { /** If any of the required parts are now fetched or in merge process, wait for the end of this operation. diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 53f2ff14b3b..243bf9d938d 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -76,6 +76,10 @@ namespace ProfileEvents extern const Event NotCreatedLogEntryForMutation; } +namespace CurrentMetrics +{ + extern const Metric BackgroundFetchesPoolTask; +} namespace DB { @@ -204,6 +208,7 @@ StorageReplicatedMergeTree::StorageReplicatedMergeTree( , part_check_thread(*this) , restarting_thread(*this) , allow_renaming(allow_renaming_) + , replicated_fetches_pool_size(global_context.getSettingsRef().background_fetches_pool_size) { queue_updating_task = global_context.getSchedulePool().createTask( getStorageID().getFullTableName() + " (StorageReplicatedMergeTree::queueUpdatingTask)", [this]{ queueUpdatingTask(); }); @@ -2614,10 +2619,36 @@ std::optional StorageReplicatedMergeTree::getDataProcessingJob() if (!selected_entry) return {}; + PoolType pool_type; + + if (selected_entry->log_entry->type == LogEntry::GET_PART) + pool_type = PoolType::FETCH; + else + pool_type = PoolType::MERGE_MUTATE; + return JobAndPool{[this, selected_entry] () mutable { processQueueEntry(selected_entry); - }, PoolType::MERGE_MUTATE}; + }, pool_type}; +} + + +bool StorageReplicatedMergeTree::canExecuteFetch(const ReplicatedMergeTreeLogEntry & entry, String & disable_reason) const +{ + if (fetcher.blocker.isCancelled()) + { + disable_reason = fmt::format("Not executing fetch of part {} because replicated fetches are cancelled now.", entry.new_part_name); + return false; + } + + size_t busy_threads_in_pool = CurrentMetrics::values[CurrentMetrics::BackgroundFetchesPoolTask].load(std::memory_order_relaxed); + if (busy_threads_in_pool >= replicated_fetches_pool_size) + { + disable_reason = fmt::format("Not executing fetch of part {} because {} fetches already executing, max {}.", entry.new_part_name, busy_threads_in_pool, replicated_fetches_pool_size); + return false; + } + + return true; } bool StorageReplicatedMergeTree::partIsAssignedToBackgroundOperation(const DataPartPtr & part) const diff --git a/src/Storages/StorageReplicatedMergeTree.h b/src/Storages/StorageReplicatedMergeTree.h index 92e17412ecc..2da3dd627ff 100644 --- a/src/Storages/StorageReplicatedMergeTree.h +++ b/src/Storages/StorageReplicatedMergeTree.h @@ -200,6 +200,8 @@ public: std::optional getDataProcessingJob() override; + bool canExecuteFetch(const ReplicatedMergeTreeLogEntry & entry, String & disable_reason) const; + private: /// Get a sequential consistent view of current parts. @@ -310,6 +312,8 @@ private: /// Do not allow RENAME TABLE if zookeeper_path contains {database} or {table} macro const bool allow_renaming; + const size_t replicated_fetches_pool_size; + template void foreachCommittedParts(const Func & func) const; From 77d350aaeb3e87143343f007a84e8455d7a4a130 Mon Sep 17 00:00:00 2001 From: alesapin Date: Mon, 26 Oct 2020 14:23:59 +0300 Subject: [PATCH 004/205] Bigger setting value --- src/Common/CurrentMetrics.cpp | 3 ++- src/Core/Settings.h | 1 + 2 files changed, 3 insertions(+), 1 deletion(-) diff --git a/src/Common/CurrentMetrics.cpp b/src/Common/CurrentMetrics.cpp index ef0b82666dd..7819ad042f3 100644 --- a/src/Common/CurrentMetrics.cpp +++ b/src/Common/CurrentMetrics.cpp @@ -9,7 +9,8 @@ M(ReplicatedFetch, "Number of data parts being fetched from replica") \ M(ReplicatedSend, "Number of data parts being sent to replicas") \ M(ReplicatedChecks, "Number of data parts checking for consistency") \ - M(BackgroundPoolTask, "Number of active tasks in BackgroundProcessingPool (merges, mutations, fetches, or replication queue bookkeeping)") \ + M(BackgroundPoolTask, "Number of active tasks in BackgroundProcessingPool (merges, mutations, or replication queue bookkeeping)") \ + M(BackgroundFetchesPoolTask, "Number of active tasks in BackgroundFetchesPool") \ M(BackgroundMovePoolTask, "Number of active tasks in BackgroundProcessingPool for moves") \ M(BackgroundSchedulePoolTask, "Number of active tasks in BackgroundSchedulePool. This pool is used for periodic ReplicatedMergeTree tasks, like cleaning old data parts, altering data parts, replica re-initialization, etc.") \ M(BackgroundBufferFlushSchedulePoolTask, "Number of active tasks in BackgroundBufferFlushSchedulePool. This pool is used for periodic Buffer flushes") \ diff --git a/src/Core/Settings.h b/src/Core/Settings.h index b94883ca871..73b687afaa8 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -71,6 +71,7 @@ class IColumn; M(UInt64, background_buffer_flush_schedule_pool_size, 16, "Number of threads performing background flush for tables with Buffer engine. Only has meaning at server startup.", 0) \ M(UInt64, background_pool_size, 16, "Number of threads performing background work for tables (for example, merging in merge tree). Only has meaning at server startup.", 0) \ M(UInt64, background_move_pool_size, 8, "Number of threads performing background moves for tables. Only has meaning at server startup.", 0) \ + M(UInt64, background_fetches_pool_size, 3, "Number of threads performing background fetches for replicated tables. Only has meaning at server startup.", 0) \ M(UInt64, background_schedule_pool_size, 16, "Number of threads performing background tasks for replicated tables, dns cache updates. Only has meaning at server startup.", 0) \ M(UInt64, background_message_broker_schedule_pool_size, 16, "Number of threads performing background tasks for message streaming. Only has meaning at server startup.", 0) \ M(UInt64, background_distributed_schedule_pool_size, 16, "Number of threads performing background tasks for distributed sends. Only has meaning at server startup.", 0) \ From d127c7d45ccca4a3c7956739fafa9ea4bc4dd535 Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 28 Oct 2020 14:54:13 +0300 Subject: [PATCH 005/205] Supress messages from inserts --- tests/queries/0_stateless/01459_manual_write_to_replicas.sh | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/01459_manual_write_to_replicas.sh b/tests/queries/0_stateless/01459_manual_write_to_replicas.sh index c402e19c3dc..1cf0ed56bc5 100755 --- a/tests/queries/0_stateless/01459_manual_write_to_replicas.sh +++ b/tests/queries/0_stateless/01459_manual_write_to_replicas.sh @@ -16,7 +16,9 @@ done function thread { for x in {0..99}; do - $CLICKHOUSE_CLIENT --query "INSERT INTO r$1 SELECT $x % $NUM_REPLICAS = $1 ? $x - 1 : $x" # Replace some records as duplicates so they will be written by other replicas + # sometimes we can try to commit obsolete part if fetches will be quite fast, + # so supress warning messages like "Tried to commit obsolete part ... covered by ..." + $CLICKHOUSE_CLIENT --query "INSERT INTO r$1 SELECT $x % $NUM_REPLICAS = $1 ? $x - 1 : $x" 2>/dev/null # Replace some records as duplicates so they will be written by other replicas done } From c1abf5d13ca29b1fcf4bee0c8531491e931b0616 Mon Sep 17 00:00:00 2001 From: "a.palagashvili" Date: Sun, 1 Nov 2020 02:56:41 +0300 Subject: [PATCH 006/205] *added adapters' boilerplate for Lzma buffers, *added submodule to gitmodules, *added cmake for xz --- .gitmodules | 6 + contrib/CMakeLists.txt | 1 + contrib/xz-cmake/CMakeLists.txt | 246 ++++++++++++++++++++++++++++++++ src/IO/CompressionMethod.cpp | 5 + src/IO/CompressionMethod.h | 6 +- src/IO/LzmaReadBuffer.cpp | 72 ++++++++++ src/IO/LzmaReadBuffer.h | 34 +++++ src/IO/LzmaWriteBuffer.cpp | 1 + src/IO/LzmaWriteBuffer.h | 10 ++ 9 files changed, 379 insertions(+), 2 deletions(-) create mode 100644 contrib/xz-cmake/CMakeLists.txt create mode 100644 src/IO/LzmaReadBuffer.cpp create mode 100644 src/IO/LzmaReadBuffer.h create mode 100644 src/IO/LzmaWriteBuffer.cpp create mode 100644 src/IO/LzmaWriteBuffer.h diff --git a/.gitmodules b/.gitmodules index 19f93ee8270..1545e92b54c 100644 --- a/.gitmodules +++ b/.gitmodules @@ -186,3 +186,9 @@ [submodule "contrib/cyrus-sasl"] path = contrib/cyrus-sasl url = https://github.com/cyrusimap/cyrus-sasl +[submodule "contrib/xz-mirror"] + path = contrib/xz-mirror + url = https://github.com/xz-mirror/xz +[submodule "contrib/xz"] + path = contrib/xz + url = https://github.com/xz-mirror/xz diff --git a/contrib/CMakeLists.txt b/contrib/CMakeLists.txt index 130e4b13c91..ec425c21239 100644 --- a/contrib/CMakeLists.txt +++ b/contrib/CMakeLists.txt @@ -32,6 +32,7 @@ add_subdirectory (murmurhash) add_subdirectory (replxx-cmake) add_subdirectory (ryu-cmake) add_subdirectory (unixodbc-cmake) +add_subdirectory (xz-cmake) add_subdirectory (poco-cmake) diff --git a/contrib/xz-cmake/CMakeLists.txt b/contrib/xz-cmake/CMakeLists.txt new file mode 100644 index 00000000000..d1295684d7f --- /dev/null +++ b/contrib/xz-cmake/CMakeLists.txt @@ -0,0 +1,246 @@ +############################################################################# +# +# Very limited CMake support for building some parts of XZ Utils +# +# For now, this is indented to be useful to build static or shared liblzma +# on Windows with MSVC (to avoid the need to maintain Visual Studio project +# files). Building liblzma on a few other platforms should work too but it +# is somewhat experimental and not as portable as using ./configure. +# +# On some platforms this builds also xz and xzdec, but these are +# highly experimental and meant for testing only: +# - No large file support on those 32-bit platforms that need it +# - No replacement getopt_long(), libc must have it +# - No sandboxing support +# - No translations +# - No xz symlinks are installed +# +# Other missing things: +# - No xzgrep or other scripts or their symlinks +# - No tests (no test failures either!) +# +# NOTE: Even if the code compiles without warnings, the end result may be +# different than via ./configure. Specifically, the list of #defines +# may be different (if so, probably this CMakeLists.txt got them wrong). +# +# This file provides the following installation components (if you only +# need liblzma, install only its components!): +# - liblzma_Runtime +# - liblzma_Development +# - xz (on some platforms only) +# - xzdec (on some platforms only) +# +# To find the target liblzma::liblzma from other packages, use the CONFIG +# option with find_package() to avoid a conflict with the FindLibLZMA module +# with case-insensitive file systems. For example, to require liblzma 5.2.5 +# or a newer compatible version: +# +# find_package(liblzma 5.2.5 REQUIRED CONFIG) +# target_link_libraries(my_application liblzma::liblzma) +# +############################################################################# +# +# Author: Lasse Collin +# +# This file has been put into the public domain. +# You can do whatever you want with this file. +# +############################################################################# + +# Define library directory, where sources and header files are located +SET(LIBRARY_DIR ${ClickHouse_SOURCE_DIR}/contrib/xz/src) + +# Read file with version +file(READ ${LIBRARY_DIR}/liblzma/api/lzma/version.h XZ_VERSION) +string(REGEX REPLACE +"^.*\n\ +#define LZMA_VERSION_MAJOR ([0-9]+)\n\ +#define LZMA_VERSION_MINOR ([0-9]+)\n\ +#define LZMA_VERSION_PATCH ([0-9]+)\n\ +.*$" + "\\1.\\2.\\3" XZ_VERSION "${XZ_VERSION}") + +# Parse version +MESSAGE(STATUS "LZMA VERSION ${XZ_VERSION}") + +# cd contrib/xz/src +# find . -name '*.c' | grep -vP 'deprecated|legacy|/xz/' | sort | sed 's/^\./ ${LIBRARY_DIR}/' +SET(Sources + ${LIBRARY_DIR}/common/tuklib_cpucores.c + ${LIBRARY_DIR}/common/tuklib_exit.c + ${LIBRARY_DIR}/common/tuklib_mbstr_fw.c + ${LIBRARY_DIR}/common/tuklib_mbstr_width.c + ${LIBRARY_DIR}/common/tuklib_open_stdxxx.c + ${LIBRARY_DIR}/common/tuklib_physmem.c + ${LIBRARY_DIR}/common/tuklib_progname.c + ${LIBRARY_DIR}/liblzma/check/check.c + ${LIBRARY_DIR}/liblzma/check/crc32_fast.c + ${LIBRARY_DIR}/liblzma/check/crc32_small.c + ${LIBRARY_DIR}/liblzma/check/crc32_table.c + ${LIBRARY_DIR}/liblzma/check/crc32_tablegen.c + ${LIBRARY_DIR}/liblzma/check/crc64_fast.c + ${LIBRARY_DIR}/liblzma/check/crc64_small.c + ${LIBRARY_DIR}/liblzma/check/crc64_table.c + ${LIBRARY_DIR}/liblzma/check/crc64_tablegen.c + ${LIBRARY_DIR}/liblzma/check/sha256.c + ${LIBRARY_DIR}/liblzma/common/alone_decoder.c + ${LIBRARY_DIR}/liblzma/common/alone_encoder.c + ${LIBRARY_DIR}/liblzma/common/auto_decoder.c + ${LIBRARY_DIR}/liblzma/common/block_buffer_decoder.c + ${LIBRARY_DIR}/liblzma/common/block_buffer_encoder.c + ${LIBRARY_DIR}/liblzma/common/block_decoder.c + ${LIBRARY_DIR}/liblzma/common/block_encoder.c + ${LIBRARY_DIR}/liblzma/common/block_header_decoder.c + ${LIBRARY_DIR}/liblzma/common/block_header_encoder.c + ${LIBRARY_DIR}/liblzma/common/block_util.c + ${LIBRARY_DIR}/liblzma/common/common.c + ${LIBRARY_DIR}/liblzma/common/easy_buffer_encoder.c + ${LIBRARY_DIR}/liblzma/common/easy_decoder_memusage.c + ${LIBRARY_DIR}/liblzma/common/easy_encoder.c + ${LIBRARY_DIR}/liblzma/common/easy_encoder_memusage.c + ${LIBRARY_DIR}/liblzma/common/easy_preset.c + ${LIBRARY_DIR}/liblzma/common/file_info.c + ${LIBRARY_DIR}/liblzma/common/filter_buffer_decoder.c + ${LIBRARY_DIR}/liblzma/common/filter_buffer_encoder.c + ${LIBRARY_DIR}/liblzma/common/filter_common.c + ${LIBRARY_DIR}/liblzma/common/filter_decoder.c + ${LIBRARY_DIR}/liblzma/common/filter_encoder.c + ${LIBRARY_DIR}/liblzma/common/filter_flags_decoder.c + ${LIBRARY_DIR}/liblzma/common/filter_flags_encoder.c + ${LIBRARY_DIR}/liblzma/common/hardware_cputhreads.c + ${LIBRARY_DIR}/liblzma/common/hardware_physmem.c + ${LIBRARY_DIR}/liblzma/common/index.c + ${LIBRARY_DIR}/liblzma/common/index_decoder.c + ${LIBRARY_DIR}/liblzma/common/index_encoder.c + ${LIBRARY_DIR}/liblzma/common/index_hash.c + ${LIBRARY_DIR}/liblzma/common/outqueue.c + ${LIBRARY_DIR}/liblzma/common/stream_buffer_decoder.c + ${LIBRARY_DIR}/liblzma/common/stream_buffer_encoder.c + ${LIBRARY_DIR}/liblzma/common/stream_decoder.c + ${LIBRARY_DIR}/liblzma/common/stream_encoder.c + ${LIBRARY_DIR}/liblzma/common/stream_encoder_mt.c + ${LIBRARY_DIR}/liblzma/common/stream_flags_common.c + ${LIBRARY_DIR}/liblzma/common/stream_flags_decoder.c + ${LIBRARY_DIR}/liblzma/common/stream_flags_encoder.c + ${LIBRARY_DIR}/liblzma/common/vli_decoder.c + ${LIBRARY_DIR}/liblzma/common/vli_encoder.c + ${LIBRARY_DIR}/liblzma/common/vli_size.c + ${LIBRARY_DIR}/liblzma/delta/delta_common.c + ${LIBRARY_DIR}/liblzma/delta/delta_decoder.c + ${LIBRARY_DIR}/liblzma/delta/delta_encoder.c + ${LIBRARY_DIR}/liblzma/lz/lz_decoder.c + ${LIBRARY_DIR}/liblzma/lz/lz_encoder.c + ${LIBRARY_DIR}/liblzma/lz/lz_encoder_mf.c + ${LIBRARY_DIR}/liblzma/lzma/fastpos_table.c + ${LIBRARY_DIR}/liblzma/lzma/fastpos_tablegen.c + ${LIBRARY_DIR}/liblzma/lzma/lzma2_decoder.c + ${LIBRARY_DIR}/liblzma/lzma/lzma2_encoder.c + ${LIBRARY_DIR}/liblzma/lzma/lzma_decoder.c + ${LIBRARY_DIR}/liblzma/lzma/lzma_encoder.c + ${LIBRARY_DIR}/liblzma/lzma/lzma_encoder_optimum_fast.c + ${LIBRARY_DIR}/liblzma/lzma/lzma_encoder_optimum_normal.c + ${LIBRARY_DIR}/liblzma/lzma/lzma_encoder_presets.c + ${LIBRARY_DIR}/liblzma/rangecoder/price_table.c + ${LIBRARY_DIR}/liblzma/rangecoder/price_tablegen.c + ${LIBRARY_DIR}/liblzma/simple/arm.c + ${LIBRARY_DIR}/liblzma/simple/armthumb.c + ${LIBRARY_DIR}/liblzma/simple/ia64.c + ${LIBRARY_DIR}/liblzma/simple/powerpc.c + ${LIBRARY_DIR}/liblzma/simple/simple_coder.c + ${LIBRARY_DIR}/liblzma/simple/simple_decoder.c + ${LIBRARY_DIR}/liblzma/simple/simple_encoder.c + ${LIBRARY_DIR}/liblzma/simple/sparc.c + ${LIBRARY_DIR}/liblzma/simple/x86.c + ${LIBRARY_DIR}/lzmainfo/lzmainfo.c +) + +# cd contrib/xz/src +# find . -name '*.h' | grep -vP 'deprecated|legacy|/xz/' | sort | sed 's/^\./ ${LIBRARY_DIR}/' +SET(Headers + ${LIBRARY_DIR}/common/mythread.h + ${LIBRARY_DIR}/common/sysdefs.h + ${LIBRARY_DIR}/common/tuklib_common.h + ${LIBRARY_DIR}/common/tuklib_config.h + ${LIBRARY_DIR}/common/tuklib_cpucores.h + ${LIBRARY_DIR}/common/tuklib_exit.h + ${LIBRARY_DIR}/common/tuklib_gettext.h + ${LIBRARY_DIR}/common/tuklib_integer.h + ${LIBRARY_DIR}/common/tuklib_mbstr.h + ${LIBRARY_DIR}/common/tuklib_open_stdxxx.h + ${LIBRARY_DIR}/common/tuklib_physmem.h + ${LIBRARY_DIR}/common/tuklib_progname.h + ${LIBRARY_DIR}/liblzma/api/lzma/base.h + ${LIBRARY_DIR}/liblzma/api/lzma/bcj.h + ${LIBRARY_DIR}/liblzma/api/lzma/block.h + ${LIBRARY_DIR}/liblzma/api/lzma/check.h + ${LIBRARY_DIR}/liblzma/api/lzma/container.h + ${LIBRARY_DIR}/liblzma/api/lzma/delta.h + ${LIBRARY_DIR}/liblzma/api/lzma/filter.h + ${LIBRARY_DIR}/liblzma/api/lzma.h + ${LIBRARY_DIR}/liblzma/api/lzma/hardware.h + ${LIBRARY_DIR}/liblzma/api/lzma/index.h + ${LIBRARY_DIR}/liblzma/api/lzma/index_hash.h + ${LIBRARY_DIR}/liblzma/api/lzma/lzma12.h + ${LIBRARY_DIR}/liblzma/api/lzma/stream_flags.h + ${LIBRARY_DIR}/liblzma/api/lzma/version.h + ${LIBRARY_DIR}/liblzma/api/lzma/vli.h + ${LIBRARY_DIR}/liblzma/check/check.h + ${LIBRARY_DIR}/liblzma/check/crc32_table_be.h + ${LIBRARY_DIR}/liblzma/check/crc32_table_le.h + ${LIBRARY_DIR}/liblzma/check/crc64_table_be.h + ${LIBRARY_DIR}/liblzma/check/crc64_table_le.h + ${LIBRARY_DIR}/liblzma/check/crc_macros.h + ${LIBRARY_DIR}/liblzma/common/alone_decoder.h + ${LIBRARY_DIR}/liblzma/common/block_buffer_encoder.h + ${LIBRARY_DIR}/liblzma/common/block_decoder.h + ${LIBRARY_DIR}/liblzma/common/block_encoder.h + ${LIBRARY_DIR}/liblzma/common/common.h + ${LIBRARY_DIR}/liblzma/common/easy_preset.h + ${LIBRARY_DIR}/liblzma/common/filter_common.h + ${LIBRARY_DIR}/liblzma/common/filter_decoder.h + ${LIBRARY_DIR}/liblzma/common/filter_encoder.h + ${LIBRARY_DIR}/liblzma/common/index_decoder.h + ${LIBRARY_DIR}/liblzma/common/index_encoder.h + ${LIBRARY_DIR}/liblzma/common/index.h + ${LIBRARY_DIR}/liblzma/common/memcmplen.h + ${LIBRARY_DIR}/liblzma/common/outqueue.h + ${LIBRARY_DIR}/liblzma/common/stream_decoder.h + ${LIBRARY_DIR}/liblzma/common/stream_flags_common.h + ${LIBRARY_DIR}/liblzma/delta/delta_common.h + ${LIBRARY_DIR}/liblzma/delta/delta_decoder.h + ${LIBRARY_DIR}/liblzma/delta/delta_encoder.h + ${LIBRARY_DIR}/liblzma/delta/delta_private.h + ${LIBRARY_DIR}/liblzma/lz/lz_decoder.h + ${LIBRARY_DIR}/liblzma/lz/lz_encoder.h + ${LIBRARY_DIR}/liblzma/lz/lz_encoder_hash.h + ${LIBRARY_DIR}/liblzma/lz/lz_encoder_hash_table.h + ${LIBRARY_DIR}/liblzma/lzma/fastpos.h + ${LIBRARY_DIR}/liblzma/lzma/lzma2_decoder.h + ${LIBRARY_DIR}/liblzma/lzma/lzma2_encoder.h + ${LIBRARY_DIR}/liblzma/lzma/lzma_common.h + ${LIBRARY_DIR}/liblzma/lzma/lzma_decoder.h + ${LIBRARY_DIR}/liblzma/lzma/lzma_encoder.h + ${LIBRARY_DIR}/liblzma/lzma/lzma_encoder_private.h + ${LIBRARY_DIR}/liblzma/rangecoder/price.h + ${LIBRARY_DIR}/liblzma/rangecoder/range_common.h + ${LIBRARY_DIR}/liblzma/rangecoder/range_decoder.h + ${LIBRARY_DIR}/liblzma/rangecoder/range_encoder.h + ${LIBRARY_DIR}/liblzma/simple/simple_coder.h + ${LIBRARY_DIR}/liblzma/simple/simple_decoder.h + ${LIBRARY_DIR}/liblzma/simple/simple_encoder.h + ${LIBRARY_DIR}/liblzma/simple/simple_private.h +) + +ADD_LIBRARY(liblzma ${Sources} ${Headers}) + +target_include_directories(liblzma PUBLIC + ${LIBRARY_DIR}/liblzma/api + ${LIBRARY_DIR}/liblzma/common + ${LIBRARY_DIR}/liblzma/check + ${LIBRARY_DIR}/liblzma/lz + ${LIBRARY_DIR}/liblzma/rangecoder + ${LIBRARY_DIR}/liblzma/lzma + ${LIBRARY_DIR}/liblzma/delta + ${LIBRARY_DIR}/liblzma/simple + ${LIBRARY_DIR}/common +) \ No newline at end of file diff --git a/src/IO/CompressionMethod.cpp b/src/IO/CompressionMethod.cpp index a0a5e19f4fa..0bf390d92ca 100644 --- a/src/IO/CompressionMethod.cpp +++ b/src/IO/CompressionMethod.cpp @@ -6,6 +6,8 @@ #include #include #include +#include +#include #if !defined(ARCADIA_BUILD) # include @@ -28,6 +30,7 @@ std::string toContentEncodingName(CompressionMethod method) case CompressionMethod::Gzip: return "gzip"; case CompressionMethod::Zlib: return "deflate"; case CompressionMethod::Brotli: return "br"; + case CompressionMethod::Xz: return "xz"; case CompressionMethod::None: return ""; } __builtin_unreachable(); @@ -73,6 +76,8 @@ std::unique_ptr wrapReadBufferWithCompressionMethod( if (method == CompressionMethod::Brotli) return std::make_unique(std::move(nested), buf_size, existing_memory, alignment); #endif + if (method == CompressionMethod::Xz) + return std::make_unique(std::move(nested), buf_size, existing_memory, alignment); if (method == CompressionMethod::None) return nested; diff --git a/src/IO/CompressionMethod.h b/src/IO/CompressionMethod.h index 64c2ba3341f..5b0d4330011 100644 --- a/src/IO/CompressionMethod.h +++ b/src/IO/CompressionMethod.h @@ -1,14 +1,13 @@ #pragma once -#include #include +#include #include namespace DB { - class ReadBuffer; class WriteBuffer; @@ -26,6 +25,9 @@ enum class CompressionMethod /// DEFLATE compression with zlib header and Adler32 checksum. /// This option corresponds to HTTP Content-Encoding: deflate. Zlib, + /// LZMA2-based content compression + /// This option corresponds to HTTP Content-Encoding: xz + Xz, Brotli }; diff --git a/src/IO/LzmaReadBuffer.cpp b/src/IO/LzmaReadBuffer.cpp new file mode 100644 index 00000000000..a241067f8bc --- /dev/null +++ b/src/IO/LzmaReadBuffer.cpp @@ -0,0 +1,72 @@ +#include + +namespace DB +{ +namespace ErrorCodes +{ + extern const int LZMA_STREAM_DECODER_FAILED; +} +LzmaReadBuffer::LzmaReadBuffer( + std::unique_ptr in_, size_t buf_size = DBMS_DEFAULT_BUFFER_SIZE, char * existing_memory = nullptr, size_t alignment = 0) + : BufferWithOwnMemory(buf_size, existing_memory, alignment), in(std::move(in_)) +{ + lstr.allocator = nullptr; + lstr.next_in = nullptr; + lstr.avail_in = 0; + lstr.next_out = nullptr; + lstr.avail_out = 0; + + // 500 mb + uint64_t memlimit = 500 << 30; + + lstr = LZMA_STREAM_INIT; + lzma_ret ret = lzma_stream_decoder(&lstr, memlimit, LZMA_CONCATENATED | LZMA_IGNORE_CHECK); + // lzma does not provide api for converting error code to string unlike zlib + if (ret != LZMA_OK) + throw Exception( + std::string("lzma_stream_decoder failed: error code: ") + std::to_string(ret) + "; lzma version: " + LZMA_VERSION_STRING, + ErrorCodes::LZMA_STREAM_DECODER_FAILED); +} + +LzmaReadBuffer::~LzmaReadBuffer() +{ + lzma_end(&lstr); +} + +bool LzmaReadBuffer::nextImpl() +{ + if (eof) + { + return false; + } + + if (!lstr.avail_in) + { + in->nextIfAtEnd(); + lstr.next_in = reinterpret_cast(in->position()); + lstr.avail_in = in->buffer().end() - in->position(); + } + lstr.next_out = reinterpret_cast(internal_buffer.begin()); + lstr.avail_out = internal_buffer.size(); + + lzma_ret ret = lzma_code(&lstr, LZMA_FINISH); + + in->position() = in->buffer().end() - lstr.avail_in; + + if (ret == LZMA_STREAM_END) + { + if (in->eof()) + { + eof = true; + return working_buffer.size() != 0; + } + } + + if (ret != LZMA_OK) + throw Exception( + std::string("lzma_stream_decoder failed: error code: ") + std::to_string(ret) + "; lzma version: " + LZMA_VERSION_STRING, + ErrorCodes::LZMA_STREAM_DECODER_FAILED); + + return true +} +} \ No newline at end of file diff --git a/src/IO/LzmaReadBuffer.h b/src/IO/LzmaReadBuffer.h new file mode 100644 index 00000000000..3ece8a46b7d --- /dev/null +++ b/src/IO/LzmaReadBuffer.h @@ -0,0 +1,34 @@ +#pragma once + +#include +#include +#include + +#include + +namespace DB +{ +namespace ErrorCodes +{ +} + +class LzmaReadBuffer : public BufferWithOwnMemory +{ +public: + LzmaReadBuffer( + std::unique_ptr in_, + size_t buf_size = DBMS_DEFAULT_BUFFER_SIZE, + char * existing_memory = nullptr, + size_t alignment = 0); + + ~LzmaReadBuffer() override; + +private: + bool nextImpl() override; + + std::unique_ptr in; + lzma_stream lstr; + bool eof; +}; + +} \ No newline at end of file diff --git a/src/IO/LzmaWriteBuffer.cpp b/src/IO/LzmaWriteBuffer.cpp new file mode 100644 index 00000000000..b9d8a3e1f2a --- /dev/null +++ b/src/IO/LzmaWriteBuffer.cpp @@ -0,0 +1 @@ +#include \ No newline at end of file diff --git a/src/IO/LzmaWriteBuffer.h b/src/IO/LzmaWriteBuffer.h new file mode 100644 index 00000000000..6824b88b21b --- /dev/null +++ b/src/IO/LzmaWriteBuffer.h @@ -0,0 +1,10 @@ +#pragma once + + +#include + +namespace DB { + + + +} \ No newline at end of file From 5982f9f7e56411f186f1ea4b9c1cf0f1480df1e4 Mon Sep 17 00:00:00 2001 From: "a.palagashvili" Date: Sun, 1 Nov 2020 03:00:36 +0300 Subject: [PATCH 007/205] removed extra record from cmake --- .gitmodules | 3 --- 1 file changed, 3 deletions(-) diff --git a/.gitmodules b/.gitmodules index 1545e92b54c..9ad672aa17f 100644 --- a/.gitmodules +++ b/.gitmodules @@ -186,9 +186,6 @@ [submodule "contrib/cyrus-sasl"] path = contrib/cyrus-sasl url = https://github.com/cyrusimap/cyrus-sasl -[submodule "contrib/xz-mirror"] - path = contrib/xz-mirror - url = https://github.com/xz-mirror/xz [submodule "contrib/xz"] path = contrib/xz url = https://github.com/xz-mirror/xz From 731e274cd0a464adc27ead4ceca974257dc1711f Mon Sep 17 00:00:00 2001 From: "a.palagashvili" Date: Sun, 1 Nov 2020 03:14:40 +0300 Subject: [PATCH 008/205] changed moment of stream object initialization --- src/IO/LzmaReadBuffer.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/IO/LzmaReadBuffer.cpp b/src/IO/LzmaReadBuffer.cpp index a241067f8bc..1150a8f48e6 100644 --- a/src/IO/LzmaReadBuffer.cpp +++ b/src/IO/LzmaReadBuffer.cpp @@ -10,6 +10,7 @@ LzmaReadBuffer::LzmaReadBuffer( std::unique_ptr in_, size_t buf_size = DBMS_DEFAULT_BUFFER_SIZE, char * existing_memory = nullptr, size_t alignment = 0) : BufferWithOwnMemory(buf_size, existing_memory, alignment), in(std::move(in_)) { + lstr = LZMA_STREAM_INIT; lstr.allocator = nullptr; lstr.next_in = nullptr; lstr.avail_in = 0; @@ -19,7 +20,6 @@ LzmaReadBuffer::LzmaReadBuffer( // 500 mb uint64_t memlimit = 500 << 30; - lstr = LZMA_STREAM_INIT; lzma_ret ret = lzma_stream_decoder(&lstr, memlimit, LZMA_CONCATENATED | LZMA_IGNORE_CHECK); // lzma does not provide api for converting error code to string unlike zlib if (ret != LZMA_OK) From 0f78dbc31d59fd614c606199e601affff2c275ac Mon Sep 17 00:00:00 2001 From: annvsh Date: Sun, 1 Nov 2020 21:41:10 +0700 Subject: [PATCH 009/205] Init commit --- docs/en/faq/integration/json-import.md | 3 +- docs/en/operations/settings/settings.md | 35 ++++++++++++++++++++++- docs/ru/operations/settings/settings.md | 37 +++++++++++++++++++++++++ 3 files changed, 73 insertions(+), 2 deletions(-) diff --git a/docs/en/faq/integration/json-import.md b/docs/en/faq/integration/json-import.md index fb94f226f2b..47255243f60 100644 --- a/docs/en/faq/integration/json-import.md +++ b/docs/en/faq/integration/json-import.md @@ -6,7 +6,8 @@ toc_priority: 11 # How to Import JSON Into ClickHouse? {#how-to-import-json-into-clickhouse} -ClickHouse supports a wide range of [data formats for input and output](../../interfaces/formats.md). There are multiple JSON variations among them, but the most commonly used for data ingestion is [JSONEachRow](../../interfaces/formats.md#jsoneachrow). It expects one JSON object per row, each object separated by a newline. +ClickHouse supports a wide range of [data formats for input and output](../../interfaces/formats.md). There are multiple JSON variations among them, but the most commonly used for data ingestion is [JSONEachRow](../../interfaces/formats.md#jsoneachrow). It expects one JSON object per row, each object separated by a newline. +In JSON format, the presence of quotation marks can determine whether the data is a number or a string. In the case of a number, you must always interpret it as an enum id. ## Examples {#examples} diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index a4bd7d77bfc..a89c5fa834e 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -1167,7 +1167,40 @@ For CSV input format enables or disables parsing of unquoted `NULL` as literal ( ## input_format_csv_enum_as_number {#settings-input_format_csv_enum_as_number} -For CSV input format switches to parsing enum values as enum ids. +Enables or disables to parsing enum values as enum ids for CSV input format. + +Possible values: + +- 0 — Disabled. +- 1 — Enabled. + +Default value: 0. + +**Example** + +Query: + +```sql +DROP TABLE IF EXISTS table_with_enum_column_for_csv_insert; + +CREATE TABLE table_with_enum_column_for_csv_insert (Id Int32,Value Enum('ef' = 1, 'es' = 2)) ENGINE=Memory(); + +SET input_format_csv_enum_as_number = 1; + +INSERT INTO table_with_enum_column_for_csv_insert FORMAT CSV 102,2; +SELECT * FROM table_with_enum_column_for_csv_insert; + +SET input_format_csv_enum_as_number = 0; +DROP TABLE IF EXISTS table_with_enum_column_for_csv_insert; +``` + +Result: + +```text +┌──Id─┬─Value─┐ +│ 102 │ es │ +└─────┴───────┘ +``` ## output_format_csv_crlf_end_of_line {#settings-output-format-csv-crlf-end-of-line} diff --git a/docs/ru/operations/settings/settings.md b/docs/ru/operations/settings/settings.md index 700203e36e1..70b90345fea 100644 --- a/docs/ru/operations/settings/settings.md +++ b/docs/ru/operations/settings/settings.md @@ -1127,6 +1127,43 @@ SELECT area/period FROM account_orders FORMAT JSON; Для формата CSV включает или выключает парсинг неэкранированной строки `NULL` как литерала (синоним для `\N`) +## input_format_csv_enum_as_number {#settings-input_format_csv_enum_as_number} + +Включает или отключает парсинг значений перечислений как идентификаторов перечислений для входного формата CSV. + +Возможные значения: + +- 0 — выключена. +- 1 — включена. + +Значение по умолчанию: 0. + +**Пример** + +Запрос: + +```sql +DROP TABLE IF EXISTS table_with_enum_column_for_csv_insert; + +CREATE TABLE table_with_enum_column_for_csv_insert (Id Int32,Value Enum('ef' = 1, 'es' = 2)) ENGINE=Memory(); + +SET input_format_csv_enum_as_number = 1; + +INSERT INTO table_with_enum_column_for_csv_insert FORMAT CSV 102,2; +SELECT * FROM table_with_enum_column_for_csv_insert; + +SET input_format_csv_enum_as_number = 0; +DROP TABLE IF EXISTS table_with_enum_column_for_csv_insert; +``` + +Результат: + +```text +┌──Id─┬─Value─┐ +│ 102 │ es │ +└─────┴───────┘ +``` + ## output_format_csv_crlf_end_of_line {#settings-output-format-csv-crlf-end-of-line} Использовать в качестве разделителя строк для CSV формата CRLF (DOS/Windows стиль) вместо LF (Unix стиль). From 13c49c723f8182187f9da06968ace9cec9d1bca0 Mon Sep 17 00:00:00 2001 From: annvsh Date: Sun, 1 Nov 2020 22:25:58 +0700 Subject: [PATCH 010/205] Added TSV format settings --- docs/en/operations/settings/settings.md | 39 ++++++++++++++++++++++- docs/ru/operations/settings/settings.md | 41 +++++++++++++++++++++++++ 2 files changed, 79 insertions(+), 1 deletion(-) diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index a89c5fa834e..229a7c90071 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -307,7 +307,44 @@ Disabled by default. ## input_format_tsv_enum_as_number {#settings-input_format_tsv_enum_as_number} -For TSV input format switches to parsing enum values as enum ids. +Enables or disables to parsing enum values as enum ids for TSV input format. + +Possible values: + +- 0 — Disabled. +- 1 — Enabled. + +Default value: 0. + +**Example** + +Query: + +```sql +DROP TABLE IF EXISTS table_with_enum_column_for_tsv_insert; + +CREATE TABLE table_with_enum_column_for_tsv_insert (Id Int32,Value Enum('ef' = 1, 'es' = 2)) ENGINE=Memory() FORMAT TabSeparatedRaw; + +SET input_format_tsv_enum_as_number = 1; + +INSERT INTO table_with_enum_column_for_tsv_insert VALUES (102, 2); +INSERT INTO table_with_enum_column_for_tsv_insert VALUES (103, 1); +SELECT * FROM table_with_enum_column_for_tsv_insert; + +SET input_format_tsv_enum_as_number = 0; +DROP TABLE IF EXISTS table_with_enum_column_for_tsv_insert; +``` + +Result: + +```text +┌──Id─┬─Value─┐ +│ 102 │ es │ +└─────┴───────┘ +┌──Id─┬─Value─┐ +│ 103 │ ef │ +└─────┴───────┘ +``` ## input_format_null_as_default {#settings-input-format-null-as-default} diff --git a/docs/ru/operations/settings/settings.md b/docs/ru/operations/settings/settings.md index 70b90345fea..ee4b9202cab 100644 --- a/docs/ru/operations/settings/settings.md +++ b/docs/ru/operations/settings/settings.md @@ -289,6 +289,47 @@ INSERT INTO test VALUES (lower('Hello')), (lower('world')), (lower('INSERT')), ( Disabled by default. +## input_format_tsv_enum_as_number {#settings-input_format_tsv_enum_as_number} + +Включает или отключает парсинг значений перечислений как идентификаторов перечислений для входного формата TSV. + +Возможные значения: + +- 0 — выключена. +- 1 — включена. + +Значение по умолчанию: 0. + +**Пример** + +Запрос: + +```sql +DROP TABLE IF EXISTS table_with_enum_column_for_tsv_insert; + +CREATE TABLE table_with_enum_column_for_tsv_insert (Id Int32,Value Enum('ef' = 1, 'es' = 2)) ENGINE=Memory() FORMAT TabSeparatedRaw; + +SET input_format_tsv_enum_as_number = 1; + +INSERT INTO table_with_enum_column_for_tsv_insert VALUES (102, 2); +INSERT INTO table_with_enum_column_for_tsv_insert VALUES (103, 1); +SELECT * FROM table_with_enum_column_for_tsv_insert; + +SET input_format_tsv_enum_as_number = 0; +DROP TABLE IF EXISTS table_with_enum_column_for_tsv_insert; +``` + +Результат: + +```text +┌──Id─┬─Value─┐ +│ 102 │ es │ +└─────┴───────┘ +┌──Id─┬─Value─┐ +│ 103 │ ef │ +└─────┴───────┘ +``` + ## input_format_null_as_default {#settings-input-format-null-as-default} Включает или отключает использование значений по умолчанию в случаях, когда во входных данных содержится `NULL`, но тип соответствующего столбца не `Nullable(T)` (для текстовых форматов). From be2b002ff49e74fb6d2931e3af511c6b0c818db1 Mon Sep 17 00:00:00 2001 From: "a.palagashvili" Date: Sun, 1 Nov 2020 21:40:05 +0300 Subject: [PATCH 011/205] fixed cmake for building, added test for compression method, added basic code for encoding --- contrib/CMakeLists.txt | 2 +- contrib/xz-cmake/CMakeLists.txt | 246 -------------------------------- src/CMakeLists.txt | 7 + src/Common/ErrorCodes.cpp | 3 + src/IO/LzmaReadBuffer.cpp | 7 +- src/IO/LzmaWriteBuffer.cpp | 119 ++++++++++++++- src/IO/LzmaWriteBuffer.h | 25 +++- src/IO/tests/CMakeLists.txt | 3 + src/IO/tests/lzma_buffers.cpp | 66 +++++++++ 9 files changed, 227 insertions(+), 251 deletions(-) delete mode 100644 contrib/xz-cmake/CMakeLists.txt create mode 100644 src/IO/tests/lzma_buffers.cpp diff --git a/contrib/CMakeLists.txt b/contrib/CMakeLists.txt index ec425c21239..695f0acf6db 100644 --- a/contrib/CMakeLists.txt +++ b/contrib/CMakeLists.txt @@ -32,7 +32,7 @@ add_subdirectory (murmurhash) add_subdirectory (replxx-cmake) add_subdirectory (ryu-cmake) add_subdirectory (unixodbc-cmake) -add_subdirectory (xz-cmake) +add_subdirectory (xz) add_subdirectory (poco-cmake) diff --git a/contrib/xz-cmake/CMakeLists.txt b/contrib/xz-cmake/CMakeLists.txt deleted file mode 100644 index d1295684d7f..00000000000 --- a/contrib/xz-cmake/CMakeLists.txt +++ /dev/null @@ -1,246 +0,0 @@ -############################################################################# -# -# Very limited CMake support for building some parts of XZ Utils -# -# For now, this is indented to be useful to build static or shared liblzma -# on Windows with MSVC (to avoid the need to maintain Visual Studio project -# files). Building liblzma on a few other platforms should work too but it -# is somewhat experimental and not as portable as using ./configure. -# -# On some platforms this builds also xz and xzdec, but these are -# highly experimental and meant for testing only: -# - No large file support on those 32-bit platforms that need it -# - No replacement getopt_long(), libc must have it -# - No sandboxing support -# - No translations -# - No xz symlinks are installed -# -# Other missing things: -# - No xzgrep or other scripts or their symlinks -# - No tests (no test failures either!) -# -# NOTE: Even if the code compiles without warnings, the end result may be -# different than via ./configure. Specifically, the list of #defines -# may be different (if so, probably this CMakeLists.txt got them wrong). -# -# This file provides the following installation components (if you only -# need liblzma, install only its components!): -# - liblzma_Runtime -# - liblzma_Development -# - xz (on some platforms only) -# - xzdec (on some platforms only) -# -# To find the target liblzma::liblzma from other packages, use the CONFIG -# option with find_package() to avoid a conflict with the FindLibLZMA module -# with case-insensitive file systems. For example, to require liblzma 5.2.5 -# or a newer compatible version: -# -# find_package(liblzma 5.2.5 REQUIRED CONFIG) -# target_link_libraries(my_application liblzma::liblzma) -# -############################################################################# -# -# Author: Lasse Collin -# -# This file has been put into the public domain. -# You can do whatever you want with this file. -# -############################################################################# - -# Define library directory, where sources and header files are located -SET(LIBRARY_DIR ${ClickHouse_SOURCE_DIR}/contrib/xz/src) - -# Read file with version -file(READ ${LIBRARY_DIR}/liblzma/api/lzma/version.h XZ_VERSION) -string(REGEX REPLACE -"^.*\n\ -#define LZMA_VERSION_MAJOR ([0-9]+)\n\ -#define LZMA_VERSION_MINOR ([0-9]+)\n\ -#define LZMA_VERSION_PATCH ([0-9]+)\n\ -.*$" - "\\1.\\2.\\3" XZ_VERSION "${XZ_VERSION}") - -# Parse version -MESSAGE(STATUS "LZMA VERSION ${XZ_VERSION}") - -# cd contrib/xz/src -# find . -name '*.c' | grep -vP 'deprecated|legacy|/xz/' | sort | sed 's/^\./ ${LIBRARY_DIR}/' -SET(Sources - ${LIBRARY_DIR}/common/tuklib_cpucores.c - ${LIBRARY_DIR}/common/tuklib_exit.c - ${LIBRARY_DIR}/common/tuklib_mbstr_fw.c - ${LIBRARY_DIR}/common/tuklib_mbstr_width.c - ${LIBRARY_DIR}/common/tuklib_open_stdxxx.c - ${LIBRARY_DIR}/common/tuklib_physmem.c - ${LIBRARY_DIR}/common/tuklib_progname.c - ${LIBRARY_DIR}/liblzma/check/check.c - ${LIBRARY_DIR}/liblzma/check/crc32_fast.c - ${LIBRARY_DIR}/liblzma/check/crc32_small.c - ${LIBRARY_DIR}/liblzma/check/crc32_table.c - ${LIBRARY_DIR}/liblzma/check/crc32_tablegen.c - ${LIBRARY_DIR}/liblzma/check/crc64_fast.c - ${LIBRARY_DIR}/liblzma/check/crc64_small.c - ${LIBRARY_DIR}/liblzma/check/crc64_table.c - ${LIBRARY_DIR}/liblzma/check/crc64_tablegen.c - ${LIBRARY_DIR}/liblzma/check/sha256.c - ${LIBRARY_DIR}/liblzma/common/alone_decoder.c - ${LIBRARY_DIR}/liblzma/common/alone_encoder.c - ${LIBRARY_DIR}/liblzma/common/auto_decoder.c - ${LIBRARY_DIR}/liblzma/common/block_buffer_decoder.c - ${LIBRARY_DIR}/liblzma/common/block_buffer_encoder.c - ${LIBRARY_DIR}/liblzma/common/block_decoder.c - ${LIBRARY_DIR}/liblzma/common/block_encoder.c - ${LIBRARY_DIR}/liblzma/common/block_header_decoder.c - ${LIBRARY_DIR}/liblzma/common/block_header_encoder.c - ${LIBRARY_DIR}/liblzma/common/block_util.c - ${LIBRARY_DIR}/liblzma/common/common.c - ${LIBRARY_DIR}/liblzma/common/easy_buffer_encoder.c - ${LIBRARY_DIR}/liblzma/common/easy_decoder_memusage.c - ${LIBRARY_DIR}/liblzma/common/easy_encoder.c - ${LIBRARY_DIR}/liblzma/common/easy_encoder_memusage.c - ${LIBRARY_DIR}/liblzma/common/easy_preset.c - ${LIBRARY_DIR}/liblzma/common/file_info.c - ${LIBRARY_DIR}/liblzma/common/filter_buffer_decoder.c - ${LIBRARY_DIR}/liblzma/common/filter_buffer_encoder.c - ${LIBRARY_DIR}/liblzma/common/filter_common.c - ${LIBRARY_DIR}/liblzma/common/filter_decoder.c - ${LIBRARY_DIR}/liblzma/common/filter_encoder.c - ${LIBRARY_DIR}/liblzma/common/filter_flags_decoder.c - ${LIBRARY_DIR}/liblzma/common/filter_flags_encoder.c - ${LIBRARY_DIR}/liblzma/common/hardware_cputhreads.c - ${LIBRARY_DIR}/liblzma/common/hardware_physmem.c - ${LIBRARY_DIR}/liblzma/common/index.c - ${LIBRARY_DIR}/liblzma/common/index_decoder.c - ${LIBRARY_DIR}/liblzma/common/index_encoder.c - ${LIBRARY_DIR}/liblzma/common/index_hash.c - ${LIBRARY_DIR}/liblzma/common/outqueue.c - ${LIBRARY_DIR}/liblzma/common/stream_buffer_decoder.c - ${LIBRARY_DIR}/liblzma/common/stream_buffer_encoder.c - ${LIBRARY_DIR}/liblzma/common/stream_decoder.c - ${LIBRARY_DIR}/liblzma/common/stream_encoder.c - ${LIBRARY_DIR}/liblzma/common/stream_encoder_mt.c - ${LIBRARY_DIR}/liblzma/common/stream_flags_common.c - ${LIBRARY_DIR}/liblzma/common/stream_flags_decoder.c - ${LIBRARY_DIR}/liblzma/common/stream_flags_encoder.c - ${LIBRARY_DIR}/liblzma/common/vli_decoder.c - ${LIBRARY_DIR}/liblzma/common/vli_encoder.c - ${LIBRARY_DIR}/liblzma/common/vli_size.c - ${LIBRARY_DIR}/liblzma/delta/delta_common.c - ${LIBRARY_DIR}/liblzma/delta/delta_decoder.c - ${LIBRARY_DIR}/liblzma/delta/delta_encoder.c - ${LIBRARY_DIR}/liblzma/lz/lz_decoder.c - ${LIBRARY_DIR}/liblzma/lz/lz_encoder.c - ${LIBRARY_DIR}/liblzma/lz/lz_encoder_mf.c - ${LIBRARY_DIR}/liblzma/lzma/fastpos_table.c - ${LIBRARY_DIR}/liblzma/lzma/fastpos_tablegen.c - ${LIBRARY_DIR}/liblzma/lzma/lzma2_decoder.c - ${LIBRARY_DIR}/liblzma/lzma/lzma2_encoder.c - ${LIBRARY_DIR}/liblzma/lzma/lzma_decoder.c - ${LIBRARY_DIR}/liblzma/lzma/lzma_encoder.c - ${LIBRARY_DIR}/liblzma/lzma/lzma_encoder_optimum_fast.c - ${LIBRARY_DIR}/liblzma/lzma/lzma_encoder_optimum_normal.c - ${LIBRARY_DIR}/liblzma/lzma/lzma_encoder_presets.c - ${LIBRARY_DIR}/liblzma/rangecoder/price_table.c - ${LIBRARY_DIR}/liblzma/rangecoder/price_tablegen.c - ${LIBRARY_DIR}/liblzma/simple/arm.c - ${LIBRARY_DIR}/liblzma/simple/armthumb.c - ${LIBRARY_DIR}/liblzma/simple/ia64.c - ${LIBRARY_DIR}/liblzma/simple/powerpc.c - ${LIBRARY_DIR}/liblzma/simple/simple_coder.c - ${LIBRARY_DIR}/liblzma/simple/simple_decoder.c - ${LIBRARY_DIR}/liblzma/simple/simple_encoder.c - ${LIBRARY_DIR}/liblzma/simple/sparc.c - ${LIBRARY_DIR}/liblzma/simple/x86.c - ${LIBRARY_DIR}/lzmainfo/lzmainfo.c -) - -# cd contrib/xz/src -# find . -name '*.h' | grep -vP 'deprecated|legacy|/xz/' | sort | sed 's/^\./ ${LIBRARY_DIR}/' -SET(Headers - ${LIBRARY_DIR}/common/mythread.h - ${LIBRARY_DIR}/common/sysdefs.h - ${LIBRARY_DIR}/common/tuklib_common.h - ${LIBRARY_DIR}/common/tuklib_config.h - ${LIBRARY_DIR}/common/tuklib_cpucores.h - ${LIBRARY_DIR}/common/tuklib_exit.h - ${LIBRARY_DIR}/common/tuklib_gettext.h - ${LIBRARY_DIR}/common/tuklib_integer.h - ${LIBRARY_DIR}/common/tuklib_mbstr.h - ${LIBRARY_DIR}/common/tuklib_open_stdxxx.h - ${LIBRARY_DIR}/common/tuklib_physmem.h - ${LIBRARY_DIR}/common/tuklib_progname.h - ${LIBRARY_DIR}/liblzma/api/lzma/base.h - ${LIBRARY_DIR}/liblzma/api/lzma/bcj.h - ${LIBRARY_DIR}/liblzma/api/lzma/block.h - ${LIBRARY_DIR}/liblzma/api/lzma/check.h - ${LIBRARY_DIR}/liblzma/api/lzma/container.h - ${LIBRARY_DIR}/liblzma/api/lzma/delta.h - ${LIBRARY_DIR}/liblzma/api/lzma/filter.h - ${LIBRARY_DIR}/liblzma/api/lzma.h - ${LIBRARY_DIR}/liblzma/api/lzma/hardware.h - ${LIBRARY_DIR}/liblzma/api/lzma/index.h - ${LIBRARY_DIR}/liblzma/api/lzma/index_hash.h - ${LIBRARY_DIR}/liblzma/api/lzma/lzma12.h - ${LIBRARY_DIR}/liblzma/api/lzma/stream_flags.h - ${LIBRARY_DIR}/liblzma/api/lzma/version.h - ${LIBRARY_DIR}/liblzma/api/lzma/vli.h - ${LIBRARY_DIR}/liblzma/check/check.h - ${LIBRARY_DIR}/liblzma/check/crc32_table_be.h - ${LIBRARY_DIR}/liblzma/check/crc32_table_le.h - ${LIBRARY_DIR}/liblzma/check/crc64_table_be.h - ${LIBRARY_DIR}/liblzma/check/crc64_table_le.h - ${LIBRARY_DIR}/liblzma/check/crc_macros.h - ${LIBRARY_DIR}/liblzma/common/alone_decoder.h - ${LIBRARY_DIR}/liblzma/common/block_buffer_encoder.h - ${LIBRARY_DIR}/liblzma/common/block_decoder.h - ${LIBRARY_DIR}/liblzma/common/block_encoder.h - ${LIBRARY_DIR}/liblzma/common/common.h - ${LIBRARY_DIR}/liblzma/common/easy_preset.h - ${LIBRARY_DIR}/liblzma/common/filter_common.h - ${LIBRARY_DIR}/liblzma/common/filter_decoder.h - ${LIBRARY_DIR}/liblzma/common/filter_encoder.h - ${LIBRARY_DIR}/liblzma/common/index_decoder.h - ${LIBRARY_DIR}/liblzma/common/index_encoder.h - ${LIBRARY_DIR}/liblzma/common/index.h - ${LIBRARY_DIR}/liblzma/common/memcmplen.h - ${LIBRARY_DIR}/liblzma/common/outqueue.h - ${LIBRARY_DIR}/liblzma/common/stream_decoder.h - ${LIBRARY_DIR}/liblzma/common/stream_flags_common.h - ${LIBRARY_DIR}/liblzma/delta/delta_common.h - ${LIBRARY_DIR}/liblzma/delta/delta_decoder.h - ${LIBRARY_DIR}/liblzma/delta/delta_encoder.h - ${LIBRARY_DIR}/liblzma/delta/delta_private.h - ${LIBRARY_DIR}/liblzma/lz/lz_decoder.h - ${LIBRARY_DIR}/liblzma/lz/lz_encoder.h - ${LIBRARY_DIR}/liblzma/lz/lz_encoder_hash.h - ${LIBRARY_DIR}/liblzma/lz/lz_encoder_hash_table.h - ${LIBRARY_DIR}/liblzma/lzma/fastpos.h - ${LIBRARY_DIR}/liblzma/lzma/lzma2_decoder.h - ${LIBRARY_DIR}/liblzma/lzma/lzma2_encoder.h - ${LIBRARY_DIR}/liblzma/lzma/lzma_common.h - ${LIBRARY_DIR}/liblzma/lzma/lzma_decoder.h - ${LIBRARY_DIR}/liblzma/lzma/lzma_encoder.h - ${LIBRARY_DIR}/liblzma/lzma/lzma_encoder_private.h - ${LIBRARY_DIR}/liblzma/rangecoder/price.h - ${LIBRARY_DIR}/liblzma/rangecoder/range_common.h - ${LIBRARY_DIR}/liblzma/rangecoder/range_decoder.h - ${LIBRARY_DIR}/liblzma/rangecoder/range_encoder.h - ${LIBRARY_DIR}/liblzma/simple/simple_coder.h - ${LIBRARY_DIR}/liblzma/simple/simple_decoder.h - ${LIBRARY_DIR}/liblzma/simple/simple_encoder.h - ${LIBRARY_DIR}/liblzma/simple/simple_private.h -) - -ADD_LIBRARY(liblzma ${Sources} ${Headers}) - -target_include_directories(liblzma PUBLIC - ${LIBRARY_DIR}/liblzma/api - ${LIBRARY_DIR}/liblzma/common - ${LIBRARY_DIR}/liblzma/check - ${LIBRARY_DIR}/liblzma/lz - ${LIBRARY_DIR}/liblzma/rangecoder - ${LIBRARY_DIR}/liblzma/lzma - ${LIBRARY_DIR}/liblzma/delta - ${LIBRARY_DIR}/liblzma/simple - ${LIBRARY_DIR}/common -) \ No newline at end of file diff --git a/src/CMakeLists.txt b/src/CMakeLists.txt index 843dd8c2615..7d76a910b17 100644 --- a/src/CMakeLists.txt +++ b/src/CMakeLists.txt @@ -318,6 +318,13 @@ if (ZSTD_LIBRARY) endif () endif() +set (LZMA_LIBRARY liblzma) +set (LZMA_INCLUDE_DIR ${ClickHouse_SOURCE_DIR}/contrib/xz/src/liblzma/api) +if (LZMA_LIBRARY) + target_link_libraries (clickhouse_common_io PUBLIC ${LZMA_LIBRARY}) + target_include_directories (clickhouse_common_io SYSTEM BEFORE PUBLIC ${LZMA_INCLUDE_DIR}) +endif() + if (USE_ICU) dbms_target_link_libraries (PRIVATE ${ICU_LIBRARIES}) dbms_target_include_directories (SYSTEM PRIVATE ${ICU_INCLUDE_DIRS}) diff --git a/src/Common/ErrorCodes.cpp b/src/Common/ErrorCodes.cpp index b75f885e559..1bb96fdb7a4 100644 --- a/src/Common/ErrorCodes.cpp +++ b/src/Common/ErrorCodes.cpp @@ -512,6 +512,9 @@ namespace ErrorCodes extern const int CONDITIONAL_TREE_PARENT_NOT_FOUND = 2001; extern const int ILLEGAL_PROJECTION_MANIPULATOR = 2002; extern const int UNRECOGNIZED_ARGUMENTS = 2003; + + extern const int LZMA_STREAM_ENCODER_FAILED = 2004; + extern const int LZMA_STREAM_DECODER_FAILED = 2005; } } diff --git a/src/IO/LzmaReadBuffer.cpp b/src/IO/LzmaReadBuffer.cpp index 1150a8f48e6..18cca25b4d5 100644 --- a/src/IO/LzmaReadBuffer.cpp +++ b/src/IO/LzmaReadBuffer.cpp @@ -7,7 +7,10 @@ namespace ErrorCodes extern const int LZMA_STREAM_DECODER_FAILED; } LzmaReadBuffer::LzmaReadBuffer( - std::unique_ptr in_, size_t buf_size = DBMS_DEFAULT_BUFFER_SIZE, char * existing_memory = nullptr, size_t alignment = 0) + std::unique_ptr in_, + size_t buf_size, + char * existing_memory, + size_t alignment) : BufferWithOwnMemory(buf_size, existing_memory, alignment), in(std::move(in_)) { lstr = LZMA_STREAM_INIT; @@ -67,6 +70,6 @@ bool LzmaReadBuffer::nextImpl() std::string("lzma_stream_decoder failed: error code: ") + std::to_string(ret) + "; lzma version: " + LZMA_VERSION_STRING, ErrorCodes::LZMA_STREAM_DECODER_FAILED); - return true + return true; } } \ No newline at end of file diff --git a/src/IO/LzmaWriteBuffer.cpp b/src/IO/LzmaWriteBuffer.cpp index b9d8a3e1f2a..6ca867d2af7 100644 --- a/src/IO/LzmaWriteBuffer.cpp +++ b/src/IO/LzmaWriteBuffer.cpp @@ -1 +1,118 @@ -#include \ No newline at end of file +#include + + +namespace DB { +namespace ErrorCodes +{ + extern const int LZMA_STREAM_ENCODER_FAILED; +} + +LzmaWriteBuffer::LzmaWriteBuffer( + std::unique_ptr out_, + int compression_level, + size_t buf_size, + char * existing_memory, + size_t alignment) + : BufferWithOwnMemory(buf_size, existing_memory, alignment) + , out(std::move(out_)) +{ + lstr = LZMA_STREAM_INIT; + lstr.allocator = nullptr; + lstr.next_in = nullptr; + lstr.avail_in = 0; + lstr.next_out = nullptr; + lstr.avail_out = 0; + + // options for further compression + lzma_options_lzma opt_lzma2; + if (lzma_lzma_preset(&opt_lzma2, compression_level)) { + throw Exception(std::string("lzma preset failed: ") + "; lzma version: " + LZMA_VERSION_STRING, ErrorCodes::LZMA_STREAM_ENCODER_FAILED); + } + + lzma_filter filters[] = { + { .id = LZMA_FILTER_X86, .options = NULL }, + { .id = LZMA_FILTER_LZMA2, .options = &opt_lzma2 }, + { .id = LZMA_VLI_UNKNOWN, .options = NULL }, + }; + lzma_ret ret = lzma_stream_encoder(&lstr, filters, LZMA_CHECK_CRC64); + + if (ret != LZMA_OK) + throw Exception(std::string("lzma stream encoder init failed: ") + std::to_string(ret) + "; lzma version: " + LZMA_VERSION_STRING, ErrorCodes::LZMA_STREAM_ENCODER_FAILED); + +} + +LzmaWriteBuffer::~LzmaWriteBuffer() +{ + try { + finish(); + + lzma_end(&lstr); + } catch (...) + { + tryLogCurrentException(__PRETTY_FUNCTION__); + } +} + +void LzmaWriteBuffer::nextImpl() +{ + if (!offset()) + return; + + lstr.next_in = reinterpret_cast(working_buffer.begin()); + lstr.avail_in = offset(); + std::cout << lstr.avail_in << std::endl; + + lzma_action action = LZMA_RUN; + do { + out->nextIfAtEnd(); + lstr.next_out = reinterpret_cast(out->position()); + lstr.avail_out = out->buffer().end() - out->position(); + + lzma_ret ret = lzma_code(&lstr, action); + out->position() = out->buffer().end() - lstr.avail_out; + + if (ret == LZMA_STREAM_END) + return; + + if (ret != LZMA_OK) + throw Exception(std::string("lzma stream encoding failed: ") + "; lzma version: " + LZMA_VERSION_STRING, ErrorCodes::LZMA_STREAM_ENCODER_FAILED); + + std::cout << lstr.avail_in << std::endl; + } while (lstr.avail_in > 0 || lstr.avail_out == 0); +} + + +void LzmaWriteBuffer::finish() +{ + if (finished) + return; + + next(); + + do { + out->nextIfAtEnd(); + lstr.next_out = reinterpret_cast(out->position()); + lstr.avail_out = out->buffer().end() - out->position(); + + lzma_ret ret = lzma_code(&lstr, LZMA_FINISH); + out->position() = out->buffer().end() - lstr.avail_out; + + if (ret == LZMA_STREAM_END) + return; + + if (ret != LZMA_OK) + throw Exception(std::string("lzma stream encoding failed: ") + "; lzma version: " + LZMA_VERSION_STRING, ErrorCodes::LZMA_STREAM_ENCODER_FAILED); + + std::cout << lstr.avail_in << std::endl; + } while (lstr.avail_out == 0); + + while (true) { + out->nextIfAtEnd(); + lstr.next_out = reinterpret_cast(out->position()); + lstr.avail_out = out->buffer().end() - out->position(); + + + } +} + +} \ No newline at end of file diff --git a/src/IO/LzmaWriteBuffer.h b/src/IO/LzmaWriteBuffer.h index 6824b88b21b..986dc065c8c 100644 --- a/src/IO/LzmaWriteBuffer.h +++ b/src/IO/LzmaWriteBuffer.h @@ -1,10 +1,33 @@ #pragma once +#include +#include #include namespace DB { +/// Performs compression using lzma library and writes compressed data to out_ WriteBuffer. +class LzmaWriteBuffer : public BufferWithOwnMemory +{ +public: + LzmaWriteBuffer( + std::unique_ptr out_, + int compression_level, + size_t buf_size = DBMS_DEFAULT_BUFFER_SIZE, + char * existing_memory = nullptr, + size_t alignment = 0); + + void finish(); + + ~LzmaWriteBuffer() override; + +private: +void nextImpl() override; + +std::unique_ptr out; +lzma_stream lstr; +bool finished = false; +}; - } \ No newline at end of file diff --git a/src/IO/tests/CMakeLists.txt b/src/IO/tests/CMakeLists.txt index dfbbfa77853..b54785bf370 100644 --- a/src/IO/tests/CMakeLists.txt +++ b/src/IO/tests/CMakeLists.txt @@ -65,6 +65,9 @@ endif () add_executable (zlib_buffers zlib_buffers.cpp) target_link_libraries (zlib_buffers PRIVATE clickhouse_common_io) +add_executable (lzma_buffers lzma_buffers.cpp) +target_link_libraries (lzma_buffers PRIVATE clickhouse_common_io) + add_executable (limit_read_buffer limit_read_buffer.cpp) target_link_libraries (limit_read_buffer PRIVATE clickhouse_common_io) diff --git a/src/IO/tests/lzma_buffers.cpp b/src/IO/tests/lzma_buffers.cpp new file mode 100644 index 00000000000..6d6b1a7f2df --- /dev/null +++ b/src/IO/tests/lzma_buffers.cpp @@ -0,0 +1,66 @@ +#include +#include + +#include +#include +#include +#include +#include +#include +#include + +int main(int, char **) +try +{ + std::cout << std::fixed << std::setprecision(2); + + size_t n = 100000; + Stopwatch stopwatch; + + { + auto buf = std::make_unique("test_lzma_buffers.xz", DBMS_DEFAULT_BUFFER_SIZE, O_WRONLY | O_CREAT | O_TRUNC); + DB::LzmaWriteBuffer lzma_buf(std::move(buf), /*compression level*/ 3); + + stopwatch.restart(); + for (size_t i = 0; i < n; ++i) + { + DB::writeIntText(i, lzma_buf); + DB::writeChar('\t', lzma_buf); + } + lzma_buf.finish(); + + stopwatch.stop(); + + std::cout << "Writing done. Elapsed: " << stopwatch.elapsedSeconds() << " s." + << ", " << (lzma_buf.count() / stopwatch.elapsedSeconds() / 1000000) << " MB/s" + << std::endl; + } +/* + { + auto buf = std::make_unique("test_lzma_buffers.xz"); + DB::LzmaReadBuffer lzma_buf(std::move(buf)); + + stopwatch.restart(); + for (size_t i = 0; i < n; ++i) + { + size_t x; + DB::readIntText(x, lzma_buf); + lzma_buf.ignore(); + + if (x != i) + throw DB::Exception("Failed!, read: " + std::to_string(x) + ", expected: " + std::to_string(i), 0); + } + stopwatch.stop(); + std::cout << "Reading done. Elapsed: " << stopwatch.elapsedSeconds() << " s." + << ", " << (lzma_buf.count() / stopwatch.elapsedSeconds() / 1000000) << " MB/s" + << std::endl; + + } +*/ + return 0; +} +catch (const DB::Exception & e) +{ + std::cerr << e.what() << ", " << e.displayText() << std::endl; + return 1; +} From 495cd47fd8f81878a148537d86353bde22978075 Mon Sep 17 00:00:00 2001 From: "a.palagashvili" Date: Mon, 2 Nov 2020 02:52:34 +0300 Subject: [PATCH 012/205] fixed compressor testing, added base logic for compressor and decompressor, tests passing --- src/IO/LzmaReadBuffer.cpp | 13 +++++++++---- src/IO/LzmaWriteBuffer.cpp | 33 ++++++++++++++++++--------------- src/IO/tests/lzma_buffers.cpp | 6 +++--- 3 files changed, 30 insertions(+), 22 deletions(-) diff --git a/src/IO/LzmaReadBuffer.cpp b/src/IO/LzmaReadBuffer.cpp index 18cca25b4d5..3f458de3880 100644 --- a/src/IO/LzmaReadBuffer.cpp +++ b/src/IO/LzmaReadBuffer.cpp @@ -21,13 +21,13 @@ LzmaReadBuffer::LzmaReadBuffer( lstr.avail_out = 0; // 500 mb - uint64_t memlimit = 500 << 30; + uint64_t memlimit = 500 << 20; - lzma_ret ret = lzma_stream_decoder(&lstr, memlimit, LZMA_CONCATENATED | LZMA_IGNORE_CHECK); + lzma_ret ret = lzma_stream_decoder(&lstr, memlimit, LZMA_CONCATENATED); // lzma does not provide api for converting error code to string unlike zlib if (ret != LZMA_OK) throw Exception( - std::string("lzma_stream_decoder failed: error code: ") + std::to_string(ret) + "; lzma version: " + LZMA_VERSION_STRING, + std::string("lzma_stream_decoder initialization failed: error code: ") + std::to_string(ret) + "; lzma version: " + LZMA_VERSION_STRING, ErrorCodes::LZMA_STREAM_DECODER_FAILED); } @@ -52,9 +52,10 @@ bool LzmaReadBuffer::nextImpl() lstr.next_out = reinterpret_cast(internal_buffer.begin()); lstr.avail_out = internal_buffer.size(); - lzma_ret ret = lzma_code(&lstr, LZMA_FINISH); + lzma_ret ret = lzma_code(&lstr, LZMA_RUN); in->position() = in->buffer().end() - lstr.avail_in; + working_buffer.resize(internal_buffer.size() - lstr.avail_out); if (ret == LZMA_STREAM_END) { @@ -62,6 +63,10 @@ bool LzmaReadBuffer::nextImpl() { eof = true; return working_buffer.size() != 0; + } else { + throw Exception( + std::string("lzma decoder finished, but stream is still alive: error code: ") + std::to_string(ret) + "; lzma version: " + LZMA_VERSION_STRING, + ErrorCodes::LZMA_STREAM_DECODER_FAILED); } } diff --git a/src/IO/LzmaWriteBuffer.cpp b/src/IO/LzmaWriteBuffer.cpp index 6ca867d2af7..332961a292f 100644 --- a/src/IO/LzmaWriteBuffer.cpp +++ b/src/IO/LzmaWriteBuffer.cpp @@ -60,24 +60,31 @@ void LzmaWriteBuffer::nextImpl() lstr.next_in = reinterpret_cast(working_buffer.begin()); lstr.avail_in = offset(); - std::cout << lstr.avail_in << std::endl; + //std::cout << lstr.avail_in << std::endl; lzma_action action = LZMA_RUN; do { out->nextIfAtEnd(); lstr.next_out = reinterpret_cast(out->position()); lstr.avail_out = out->buffer().end() - out->position(); + //std::cout << lstr.avail_out << " BEFOR" << std::endl; + lzma_ret ret = lzma_code(&lstr, action); out->position() = out->buffer().end() - lstr.avail_out; - if (ret == LZMA_STREAM_END) - return; + //std::cout << lstr.avail_out << " AFTER" << std::endl; + + //std::cout << ret << " RET IMPL" << std::endl; + + if (ret == LZMA_STREAM_END) { + return; + } if (ret != LZMA_OK) throw Exception(std::string("lzma stream encoding failed: ") + "; lzma version: " + LZMA_VERSION_STRING, ErrorCodes::LZMA_STREAM_ENCODER_FAILED); - std::cout << lstr.avail_in << std::endl; + //std::cout << lstr.avail_in << " " << lstr.avail_out << std::endl; } while (lstr.avail_in > 0 || lstr.avail_out == 0); } @@ -97,22 +104,18 @@ void LzmaWriteBuffer::finish() lzma_ret ret = lzma_code(&lstr, LZMA_FINISH); out->position() = out->buffer().end() - lstr.avail_out; - if (ret == LZMA_STREAM_END) - return; + //std::cout << ret << " RET FIN" << std::endl; + + if (ret == LZMA_STREAM_END) { + finished = true; + return; + } if (ret != LZMA_OK) throw Exception(std::string("lzma stream encoding failed: ") + "; lzma version: " + LZMA_VERSION_STRING, ErrorCodes::LZMA_STREAM_ENCODER_FAILED); - std::cout << lstr.avail_in << std::endl; + //std::cout << lstr.avail_in << std::endl; } while (lstr.avail_out == 0); - - while (true) { - out->nextIfAtEnd(); - lstr.next_out = reinterpret_cast(out->position()); - lstr.avail_out = out->buffer().end() - out->position(); - - - } } } \ No newline at end of file diff --git a/src/IO/tests/lzma_buffers.cpp b/src/IO/tests/lzma_buffers.cpp index 6d6b1a7f2df..1f691fa09f7 100644 --- a/src/IO/tests/lzma_buffers.cpp +++ b/src/IO/tests/lzma_buffers.cpp @@ -14,7 +14,7 @@ try { std::cout << std::fixed << std::setprecision(2); - size_t n = 100000; + size_t n = 10000000; Stopwatch stopwatch; { @@ -35,7 +35,7 @@ try << ", " << (lzma_buf.count() / stopwatch.elapsedSeconds() / 1000000) << " MB/s" << std::endl; } -/* + { auto buf = std::make_unique("test_lzma_buffers.xz"); DB::LzmaReadBuffer lzma_buf(std::move(buf)); @@ -56,7 +56,7 @@ try << std::endl; } -*/ + return 0; } catch (const DB::Exception & e) From 2ad01c59da53982c0b43299e272299d40928d87a Mon Sep 17 00:00:00 2001 From: "a.palagashvili" Date: Mon, 2 Nov 2020 16:17:25 +0300 Subject: [PATCH 013/205] fixed codestyle, added record to changelog --- CHANGELOG.md | 1 + src/IO/LzmaReadBuffer.cpp | 10 ++++------ src/IO/LzmaWriteBuffer.cpp | 30 +++++++++++++----------------- 3 files changed, 18 insertions(+), 23 deletions(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index 09ce72d20ed..cc4f07ffcaa 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -7,6 +7,7 @@ * Added column transformers `EXCEPT`, `REPLACE`, `APPLY`, which can be applied to the list of selected columns (after `*` or `COLUMNS(...)`). For example, you can write `SELECT * EXCEPT(URL) REPLACE(number + 1 AS number)`. Another example: `select * apply(length) apply(max) from wide_string_table` to find out the maxium length of all string columns. [#14233](https://github.com/ClickHouse/ClickHouse/pull/14233) ([Amos Bird](https://github.com/amosbird)). * Added an aggregate function `rankCorr` which computes a rank correlation coefficient. [#11769](https://github.com/ClickHouse/ClickHouse/pull/11769) ([antikvist](https://github.com/antikvist)) [#14411](https://github.com/ClickHouse/ClickHouse/pull/14411) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). * Added table function `view` which turns a subquery into a table object. This helps passing queries around. For instance, it can be used in remote/cluster table functions. [#12567](https://github.com/ClickHouse/ClickHouse/pull/12567) ([Amos Bird](https://github.com/amosbird)). +* Added support for `xz` compression format. This enables using `*.xz` files in `table()` function. [#16578](https://github.com/ClickHouse/ClickHouse/pull/16578) ([Abi Palagashvili](https://github.com/fibersel)) #### Bug Fix diff --git a/src/IO/LzmaReadBuffer.cpp b/src/IO/LzmaReadBuffer.cpp index 3f458de3880..7454df35a11 100644 --- a/src/IO/LzmaReadBuffer.cpp +++ b/src/IO/LzmaReadBuffer.cpp @@ -64,16 +64,14 @@ bool LzmaReadBuffer::nextImpl() eof = true; return working_buffer.size() != 0; } else { - throw Exception( - std::string("lzma decoder finished, but stream is still alive: error code: ") + std::to_string(ret) + "; lzma version: " + LZMA_VERSION_STRING, - ErrorCodes::LZMA_STREAM_DECODER_FAILED); + throw Exception(ErrorCodes::LZMA_STREAM_DECODER_FAILED, + "lzma decoder finished, but stream is still alive: error code: {}; lzma version: {}", ret, LZMA_VERSION_STRING); } } if (ret != LZMA_OK) - throw Exception( - std::string("lzma_stream_decoder failed: error code: ") + std::to_string(ret) + "; lzma version: " + LZMA_VERSION_STRING, - ErrorCodes::LZMA_STREAM_DECODER_FAILED); + throw Exception(ErrorCodes::LZMA_STREAM_DECODER_FAILED, + "lzma_stream_decoder failed: error code: error codeL {}; lzma version: {}", ret, LZMA_VERSION_STRING); return true; } diff --git a/src/IO/LzmaWriteBuffer.cpp b/src/IO/LzmaWriteBuffer.cpp index 332961a292f..59b50094334 100644 --- a/src/IO/LzmaWriteBuffer.cpp +++ b/src/IO/LzmaWriteBuffer.cpp @@ -25,9 +25,9 @@ LzmaWriteBuffer::LzmaWriteBuffer( // options for further compression lzma_options_lzma opt_lzma2; - if (lzma_lzma_preset(&opt_lzma2, compression_level)) { + if (lzma_lzma_preset(&opt_lzma2, compression_level)) throw Exception(std::string("lzma preset failed: ") + "; lzma version: " + LZMA_VERSION_STRING, ErrorCodes::LZMA_STREAM_ENCODER_FAILED); - } + lzma_filter filters[] = { { .id = LZMA_FILTER_X86, .options = NULL }, @@ -43,7 +43,8 @@ LzmaWriteBuffer::LzmaWriteBuffer( LzmaWriteBuffer::~LzmaWriteBuffer() { - try { + try + { finish(); lzma_end(&lstr); @@ -60,31 +61,26 @@ void LzmaWriteBuffer::nextImpl() lstr.next_in = reinterpret_cast(working_buffer.begin()); lstr.avail_in = offset(); - //std::cout << lstr.avail_in << std::endl; lzma_action action = LZMA_RUN; - do { + do + { out->nextIfAtEnd(); lstr.next_out = reinterpret_cast(out->position()); lstr.avail_out = out->buffer().end() - out->position(); - //std::cout << lstr.avail_out << " BEFOR" << std::endl; lzma_ret ret = lzma_code(&lstr, action); out->position() = out->buffer().end() - lstr.avail_out; - //std::cout << lstr.avail_out << " AFTER" << std::endl; - //std::cout << ret << " RET IMPL" << std::endl; - if (ret == LZMA_STREAM_END) { - return; - } + if (ret == LZMA_STREAM_END) + return; if (ret != LZMA_OK) throw Exception(std::string("lzma stream encoding failed: ") + "; lzma version: " + LZMA_VERSION_STRING, ErrorCodes::LZMA_STREAM_ENCODER_FAILED); - //std::cout << lstr.avail_in << " " << lstr.avail_out << std::endl; } while (lstr.avail_in > 0 || lstr.avail_out == 0); } @@ -96,7 +92,8 @@ void LzmaWriteBuffer::finish() next(); - do { + do + { out->nextIfAtEnd(); lstr.next_out = reinterpret_cast(out->position()); lstr.avail_out = out->buffer().end() - out->position(); @@ -104,9 +101,9 @@ void LzmaWriteBuffer::finish() lzma_ret ret = lzma_code(&lstr, LZMA_FINISH); out->position() = out->buffer().end() - lstr.avail_out; - //std::cout << ret << " RET FIN" << std::endl; - if (ret == LZMA_STREAM_END) { + if (ret == LZMA_STREAM_END) + { finished = true; return; } @@ -114,8 +111,7 @@ void LzmaWriteBuffer::finish() if (ret != LZMA_OK) throw Exception(std::string("lzma stream encoding failed: ") + "; lzma version: " + LZMA_VERSION_STRING, ErrorCodes::LZMA_STREAM_ENCODER_FAILED); - //std::cout << lstr.avail_in << std::endl; } while (lstr.avail_out == 0); } -} \ No newline at end of file +} From ba6fa5d82895e20a76c12717a8a8f9ea622bc58d Mon Sep 17 00:00:00 2001 From: "a.palagashvili" Date: Mon, 2 Nov 2020 23:04:49 +0300 Subject: [PATCH 014/205] fixed whitespaces, added hidden submodule file --- contrib/xz | 1 + src/IO/LzmaReadBuffer.cpp | 32 ++++++++++++---------- src/IO/LzmaReadBuffer.h | 1 - src/IO/LzmaWriteBuffer.cpp | 55 +++++++++++++++++++------------------- src/IO/LzmaWriteBuffer.h | 27 +++++++++---------- 5 files changed, 60 insertions(+), 56 deletions(-) create mode 160000 contrib/xz diff --git a/contrib/xz b/contrib/xz new file mode 160000 index 00000000000..869b9d1b4ed --- /dev/null +++ b/contrib/xz @@ -0,0 +1 @@ +Subproject commit 869b9d1b4edd6df07f819d360d306251f8147353 diff --git a/src/IO/LzmaReadBuffer.cpp b/src/IO/LzmaReadBuffer.cpp index 7454df35a11..377b68a4ae3 100644 --- a/src/IO/LzmaReadBuffer.cpp +++ b/src/IO/LzmaReadBuffer.cpp @@ -6,11 +6,7 @@ namespace ErrorCodes { extern const int LZMA_STREAM_DECODER_FAILED; } -LzmaReadBuffer::LzmaReadBuffer( - std::unique_ptr in_, - size_t buf_size, - char * existing_memory, - size_t alignment) +LzmaReadBuffer::LzmaReadBuffer(std::unique_ptr in_, size_t buf_size, char * existing_memory, size_t alignment) : BufferWithOwnMemory(buf_size, existing_memory, alignment), in(std::move(in_)) { lstr = LZMA_STREAM_INIT; @@ -27,7 +23,8 @@ LzmaReadBuffer::LzmaReadBuffer( // lzma does not provide api for converting error code to string unlike zlib if (ret != LZMA_OK) throw Exception( - std::string("lzma_stream_decoder initialization failed: error code: ") + std::to_string(ret) + "; lzma version: " + LZMA_VERSION_STRING, + std::string("lzma_stream_decoder initialization failed: error code: ") + std::to_string(ret) + + "; lzma version: " + LZMA_VERSION_STRING, ErrorCodes::LZMA_STREAM_DECODER_FAILED); } @@ -39,9 +36,8 @@ LzmaReadBuffer::~LzmaReadBuffer() bool LzmaReadBuffer::nextImpl() { if (eof) - { return false; - } + if (!lstr.avail_in) { @@ -63,16 +59,24 @@ bool LzmaReadBuffer::nextImpl() { eof = true; return working_buffer.size() != 0; - } else { - throw Exception(ErrorCodes::LZMA_STREAM_DECODER_FAILED, - "lzma decoder finished, but stream is still alive: error code: {}; lzma version: {}", ret, LZMA_VERSION_STRING); + } + else + { + throw Exception( + ErrorCodes::LZMA_STREAM_DECODER_FAILED, + "lzma decoder finished, but stream is still alive: error code: {}; lzma version: {}", + ret, + LZMA_VERSION_STRING); } } if (ret != LZMA_OK) - throw Exception(ErrorCodes::LZMA_STREAM_DECODER_FAILED, - "lzma_stream_decoder failed: error code: error codeL {}; lzma version: {}", ret, LZMA_VERSION_STRING); + throw Exception( + ErrorCodes::LZMA_STREAM_DECODER_FAILED, + "lzma_stream_decoder failed: error code: error codeL {}; lzma version: {}", + ret, + LZMA_VERSION_STRING); return true; } -} \ No newline at end of file +} diff --git a/src/IO/LzmaReadBuffer.h b/src/IO/LzmaReadBuffer.h index 3ece8a46b7d..ae4d4ccf749 100644 --- a/src/IO/LzmaReadBuffer.h +++ b/src/IO/LzmaReadBuffer.h @@ -30,5 +30,4 @@ private: lzma_stream lstr; bool eof; }; - } \ No newline at end of file diff --git a/src/IO/LzmaWriteBuffer.cpp b/src/IO/LzmaWriteBuffer.cpp index 59b50094334..395de00be1d 100644 --- a/src/IO/LzmaWriteBuffer.cpp +++ b/src/IO/LzmaWriteBuffer.cpp @@ -1,20 +1,16 @@ #include -namespace DB { +namespace DB +{ namespace ErrorCodes { extern const int LZMA_STREAM_ENCODER_FAILED; } LzmaWriteBuffer::LzmaWriteBuffer( - std::unique_ptr out_, - int compression_level, - size_t buf_size, - char * existing_memory, - size_t alignment) - : BufferWithOwnMemory(buf_size, existing_memory, alignment) - , out(std::move(out_)) + std::unique_ptr out_, int compression_level, size_t buf_size, char * existing_memory, size_t alignment) + : BufferWithOwnMemory(buf_size, existing_memory, alignment), out(std::move(out_)) { lstr = LZMA_STREAM_INIT; lstr.allocator = nullptr; @@ -22,23 +18,25 @@ LzmaWriteBuffer::LzmaWriteBuffer( lstr.avail_in = 0; lstr.next_out = nullptr; lstr.avail_out = 0; - + // options for further compression lzma_options_lzma opt_lzma2; if (lzma_lzma_preset(&opt_lzma2, compression_level)) - throw Exception(std::string("lzma preset failed: ") + "; lzma version: " + LZMA_VERSION_STRING, ErrorCodes::LZMA_STREAM_ENCODER_FAILED); + throw Exception( + std::string("lzma preset failed: ") + "; lzma version: " + LZMA_VERSION_STRING, ErrorCodes::LZMA_STREAM_ENCODER_FAILED); lzma_filter filters[] = { - { .id = LZMA_FILTER_X86, .options = NULL }, - { .id = LZMA_FILTER_LZMA2, .options = &opt_lzma2 }, - { .id = LZMA_VLI_UNKNOWN, .options = NULL }, + {.id = LZMA_FILTER_X86, .options = NULL}, + {.id = LZMA_FILTER_LZMA2, .options = &opt_lzma2}, + {.id = LZMA_VLI_UNKNOWN, .options = NULL}, }; lzma_ret ret = lzma_stream_encoder(&lstr, filters, LZMA_CHECK_CRC64); - if (ret != LZMA_OK) - throw Exception(std::string("lzma stream encoder init failed: ") + std::to_string(ret) + "; lzma version: " + LZMA_VERSION_STRING, ErrorCodes::LZMA_STREAM_ENCODER_FAILED); - + if (ret != LZMA_OK) + throw Exception( + std::string("lzma stream encoder init failed: ") + std::to_string(ret) + "; lzma version: " + LZMA_VERSION_STRING, + ErrorCodes::LZMA_STREAM_ENCODER_FAILED); } LzmaWriteBuffer::~LzmaWriteBuffer() @@ -48,7 +46,8 @@ LzmaWriteBuffer::~LzmaWriteBuffer() finish(); lzma_end(&lstr); - } catch (...) + } + catch (...) { tryLogCurrentException(__PRETTY_FUNCTION__); } @@ -74,13 +73,14 @@ void LzmaWriteBuffer::nextImpl() out->position() = out->buffer().end() - lstr.avail_out; - if (ret == LZMA_STREAM_END) return; - + if (ret != LZMA_OK) - throw Exception(std::string("lzma stream encoding failed: ") + "; lzma version: " + LZMA_VERSION_STRING, ErrorCodes::LZMA_STREAM_ENCODER_FAILED); - + throw Exception( + std::string("lzma stream encoding failed: ") + "; lzma version: " + LZMA_VERSION_STRING, + ErrorCodes::LZMA_STREAM_ENCODER_FAILED); + } while (lstr.avail_in > 0 || lstr.avail_out == 0); } @@ -102,16 +102,17 @@ void LzmaWriteBuffer::finish() out->position() = out->buffer().end() - lstr.avail_out; - if (ret == LZMA_STREAM_END) + if (ret == LZMA_STREAM_END) { finished = true; - return; + return; } - + if (ret != LZMA_OK) - throw Exception(std::string("lzma stream encoding failed: ") + "; lzma version: " + LZMA_VERSION_STRING, ErrorCodes::LZMA_STREAM_ENCODER_FAILED); - + throw Exception( + std::string("lzma stream encoding failed: ") + "; lzma version: " + LZMA_VERSION_STRING, + ErrorCodes::LZMA_STREAM_ENCODER_FAILED); + } while (lstr.avail_out == 0); } - } diff --git a/src/IO/LzmaWriteBuffer.h b/src/IO/LzmaWriteBuffer.h index 986dc065c8c..fa5e3456f79 100644 --- a/src/IO/LzmaWriteBuffer.h +++ b/src/IO/LzmaWriteBuffer.h @@ -1,33 +1,32 @@ #pragma once -#include #include +#include #include -namespace DB { - +namespace DB +{ /// Performs compression using lzma library and writes compressed data to out_ WriteBuffer. class LzmaWriteBuffer : public BufferWithOwnMemory -{ +{ public: LzmaWriteBuffer( - std::unique_ptr out_, - int compression_level, - size_t buf_size = DBMS_DEFAULT_BUFFER_SIZE, - char * existing_memory = nullptr, - size_t alignment = 0); + std::unique_ptr out_, + int compression_level, + size_t buf_size = DBMS_DEFAULT_BUFFER_SIZE, + char * existing_memory = nullptr, + size_t alignment = 0); void finish(); ~LzmaWriteBuffer() override; private: -void nextImpl() override; + void nextImpl() override; -std::unique_ptr out; -lzma_stream lstr; -bool finished = false; + std::unique_ptr out; + lzma_stream lstr; + bool finished = false; }; - } \ No newline at end of file From 8098f86e3ea5bcefb84601ed0ecd3de5096e022a Mon Sep 17 00:00:00 2001 From: "a.palagashvili" Date: Tue, 3 Nov 2020 01:05:50 +0300 Subject: [PATCH 015/205] added record for fasttest --- docker/test/fasttest/run.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docker/test/fasttest/run.sh b/docker/test/fasttest/run.sh index 8300c31681e..add8d2254f9 100755 --- a/docker/test/fasttest/run.sh +++ b/docker/test/fasttest/run.sh @@ -127,7 +127,7 @@ function clone_submodules ( cd "$FASTTEST_SOURCE" -SUBMODULES_TO_UPDATE=(contrib/boost contrib/zlib-ng contrib/libxml2 contrib/poco contrib/libunwind contrib/ryu contrib/fmtlib contrib/base64 contrib/cctz contrib/libcpuid contrib/double-conversion contrib/libcxx contrib/libcxxabi contrib/libc-headers contrib/lz4 contrib/zstd contrib/fastops contrib/rapidjson contrib/re2 contrib/sparsehash-c11 contrib/croaring) +SUBMODULES_TO_UPDATE=(contrib/boost contrib/zlib-ng contrib/libxml2 contrib/poco contrib/libunwind contrib/ryu contrib/fmtlib contrib/base64 contrib/cctz contrib/libcpuid contrib/double-conversion contrib/libcxx contrib/libcxxabi contrib/libc-headers contrib/lz4 contrib/zstd contrib/fastops contrib/rapidjson contrib/re2 contrib/sparsehash-c11 contrib/croaring contrib/xz) git submodule sync git submodule update --init --recursive "${SUBMODULES_TO_UPDATE[@]}" From 07a7c46b8979265cc0c240fa159586f2f80c0ca0 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 3 Nov 2020 14:28:28 +0300 Subject: [PATCH 016/205] Refactor ExpressionActions [Part 3] --- src/Core/Block.cpp | 16 +- src/Core/Block.h | 3 + src/Core/iostream_debug_helpers.cpp | 10 +- src/Core/iostream_debug_helpers.h | 4 +- .../CheckConstraintsBlockOutputStream.cpp | 2 +- src/Functions/FunctionsMiscellaneous.h | 2 +- src/Interpreters/ActionsVisitor.cpp | 37 +- src/Interpreters/ActionsVisitor.h | 11 +- src/Interpreters/Aggregator.cpp | 32 +- src/Interpreters/ExpressionActions.cpp | 1847 +++++++---------- src/Interpreters/ExpressionActions.h | 312 +-- src/Interpreters/ExpressionAnalyzer.cpp | 144 +- src/Interpreters/ExpressionAnalyzer.h | 33 +- src/Interpreters/ExpressionJIT.cpp | 312 +-- src/Interpreters/ExpressionJIT.h | 2 +- src/Interpreters/HashJoin.cpp | 8 + src/Interpreters/InterpreterSelectQuery.cpp | 66 +- src/Interpreters/InterpreterSelectQuery.h | 16 +- src/Interpreters/MutationsInterpreter.cpp | 8 +- .../getHeaderForProcessingStage.cpp | 2 +- src/Processors/QueryPlan/ExpressionStep.cpp | 27 +- src/Processors/QueryPlan/ExpressionStep.h | 10 +- src/Processors/QueryPlan/FillingStep.cpp | 2 +- src/Processors/QueryPlan/FilterStep.cpp | 14 +- src/Processors/QueryPlan/FilterStep.h | 10 +- src/Processors/QueryPlan/QueryPlan.cpp | 2 +- .../QueryPlan/ReadFromStorageStep.cpp | 10 +- src/Processors/QueryPlan/TotalsHavingStep.cpp | 9 +- src/Processors/QueryPlan/TotalsHavingStep.h | 8 +- .../Transforms/ExpressionTransform.cpp | 9 +- .../Transforms/FillingTransform.cpp | 15 +- src/Processors/Transforms/FillingTransform.h | 2 + src/Processors/Transforms/FilterTransform.cpp | 8 +- .../Transforms/TotalsHavingTransform.cpp | 18 +- src/Storages/ColumnsDescription.cpp | 2 +- src/Storages/IndicesDescription.cpp | 4 +- src/Storages/KeyDescription.cpp | 4 +- src/Storages/MergeTree/KeyCondition.cpp | 18 +- .../MergeTreeBaseSelectProcessor.cpp | 35 +- .../MergeTree/MergeTreeBaseSelectProcessor.h | 2 + .../MergeTree/MergeTreeBlockReadUtils.cpp | 4 +- src/Storages/MergeTree/MergeTreeData.cpp | 22 +- .../MergeTree/MergeTreeDataSelectExecutor.cpp | 27 +- .../MergeTree/MergeTreeDataSelectExecutor.h | 4 +- .../MergeTree/MergeTreeRangeReader.cpp | 45 +- src/Storages/MergeTree/MergeTreeRangeReader.h | 13 +- src/Storages/ReadInOrderOptimizer.cpp | 6 +- src/Storages/SelectQueryInfo.h | 13 +- src/Storages/StorageBuffer.cpp | 4 +- src/Storages/StorageDistributed.cpp | 4 +- src/Storages/TTLDescription.cpp | 16 +- ...ray_join_with_arrays_of_nullable.reference | 10 +- ...526_array_join_with_arrays_of_nullable.sql | 2 +- .../01508_explain_header.reference | 2 +- 54 files changed, 1570 insertions(+), 1678 deletions(-) diff --git a/src/Core/Block.cpp b/src/Core/Block.cpp index 07ec0810f96..cd2855739e2 100644 --- a/src/Core/Block.cpp +++ b/src/Core/Block.cpp @@ -40,7 +40,7 @@ Block::Block(const ColumnsWithTypeAndName & data_) : data{data_} void Block::initializeIndexByName() { for (size_t i = 0, size = data.size(); i < size; ++i) - index_by_name[data[i].name] = i; + index_by_name.emplace(data[i].name, i); } @@ -295,6 +295,20 @@ std::string Block::dumpStructure() const return out.str(); } +std::string Block::dumpIndex() const +{ + WriteBufferFromOwnString out; + bool first = true; + for (const auto & [name, pos] : index_by_name) + { + if (!first) + out << ", "; + first = false; + + out << name << ' ' << pos; + } + return out.str(); +} Block Block::cloneEmpty() const { diff --git a/src/Core/Block.h b/src/Core/Block.h index f55a75162c6..eef3c27363b 100644 --- a/src/Core/Block.h +++ b/src/Core/Block.h @@ -119,6 +119,9 @@ public: /** List of names, types and lengths of columns. Designed for debugging. */ std::string dumpStructure() const; + /** List of column names and positions from index */ + std::string dumpIndex() const; + /** Get the same block, but empty. */ Block cloneEmpty() const; diff --git a/src/Core/iostream_debug_helpers.cpp b/src/Core/iostream_debug_helpers.cpp index a6fc329c8eb..f64d1a5ae2d 100644 --- a/src/Core/iostream_debug_helpers.cpp +++ b/src/Core/iostream_debug_helpers.cpp @@ -106,11 +106,11 @@ std::ostream & operator<<(std::ostream & stream, const Packet & what) return stream; } -std::ostream & operator<<(std::ostream & stream, const ExpressionAction & what) -{ - stream << "ExpressionAction(" << what.toString() << ")"; - return stream; -} +//std::ostream & operator<<(std::ostream & stream, const ExpressionAction & what) +//{ +// stream << "ExpressionAction(" << what.toString() << ")"; +// return stream; +//} std::ostream & operator<<(std::ostream & stream, const ExpressionActions & what) { diff --git a/src/Core/iostream_debug_helpers.h b/src/Core/iostream_debug_helpers.h index ef195ed4abf..4005508e4b8 100644 --- a/src/Core/iostream_debug_helpers.h +++ b/src/Core/iostream_debug_helpers.h @@ -40,8 +40,8 @@ std::ostream & operator<<(std::ostream & stream, const IColumn & what); struct Packet; std::ostream & operator<<(std::ostream & stream, const Packet & what); -struct ExpressionAction; -std::ostream & operator<<(std::ostream & stream, const ExpressionAction & what); +//struct ExpressionAction; +//std::ostream & operator<<(std::ostream & stream, const ExpressionAction & what); class ExpressionActions; std::ostream & operator<<(std::ostream & stream, const ExpressionActions & what); diff --git a/src/DataStreams/CheckConstraintsBlockOutputStream.cpp b/src/DataStreams/CheckConstraintsBlockOutputStream.cpp index 7a67074dbdf..7c0253c6731 100644 --- a/src/DataStreams/CheckConstraintsBlockOutputStream.cpp +++ b/src/DataStreams/CheckConstraintsBlockOutputStream.cpp @@ -46,7 +46,7 @@ void CheckConstraintsBlockOutputStream::write(const Block & block) auto * constraint_ptr = constraints.constraints[i]->as(); - ColumnWithTypeAndName res_column = block_to_calculate.getByPosition(block_to_calculate.columns() - 1); + ColumnWithTypeAndName res_column = block_to_calculate.getByName(constraint_ptr->expr->getColumnName()); if (!isUInt8(res_column.type)) throw Exception(ErrorCodes::LOGICAL_ERROR, "Constraint {} does not return a value of type UInt8", diff --git a/src/Functions/FunctionsMiscellaneous.h b/src/Functions/FunctionsMiscellaneous.h index ce8a827b4cd..096f640b553 100644 --- a/src/Functions/FunctionsMiscellaneous.h +++ b/src/Functions/FunctionsMiscellaneous.h @@ -201,7 +201,7 @@ public: { /// Check that expression does not contain unusual actions that will break columnss structure. for (const auto & action : expression_actions->getActions()) - if (action.type == ExpressionAction::Type::ARRAY_JOIN) + if (action.node->type == ActionsDAG::Type::ARRAY_JOIN) throw Exception("Expression with arrayJoin or other unusual action cannot be captured", ErrorCodes::BAD_ARGUMENTS); std::unordered_map arguments_map; diff --git a/src/Interpreters/ActionsVisitor.cpp b/src/Interpreters/ActionsVisitor.cpp index e0e921b003b..fe7ccc2c6ea 100644 --- a/src/Interpreters/ActionsVisitor.cpp +++ b/src/Interpreters/ActionsVisitor.cpp @@ -350,7 +350,7 @@ SetPtr makeExplicitSet( auto it = index.find(left_arg->getColumnName()); if (it == index.end()) throw Exception("Unknown identifier: '" + left_arg->getColumnName() + "'", ErrorCodes::UNKNOWN_IDENTIFIER); - const DataTypePtr & left_arg_type = it->second->result_type; + const DataTypePtr & left_arg_type = (*it)->result_type; DataTypes set_element_types = {left_arg_type}; const auto * left_tuple_type = typeid_cast(left_arg_type.get()); @@ -404,7 +404,7 @@ ActionsMatcher::Data::Data( bool ActionsMatcher::Data::hasColumn(const String & column_name) const { - return actions_stack.getLastActions().getIndex().count(column_name) != 0; + return actions_stack.getLastActions().getIndex().contains(column_name); } ScopeStack::ScopeStack(ActionsDAGPtr actions, const Context & context_) @@ -413,9 +413,9 @@ ScopeStack::ScopeStack(ActionsDAGPtr actions, const Context & context_) auto & level = stack.emplace_back(); level.actions = std::move(actions); - for (const auto & [name, node] : level.actions->getIndex()) + for (const auto & node : level.actions->getIndex()) if (node->type == ActionsDAG::Type::INPUT) - level.inputs.emplace(name); + level.inputs.emplace(node->result_name); } void ScopeStack::pushLevel(const NamesAndTypesList & input_columns) @@ -432,9 +432,9 @@ void ScopeStack::pushLevel(const NamesAndTypesList & input_columns) const auto & index = level.actions->getIndex(); - for (const auto & [name, node] : prev.actions->getIndex()) + for (const auto & node : prev.actions->getIndex()) { - if (index.count(name) == 0) + if (!index.contains(node->result_name)) level.actions->addInput({node->column, node->result_type, node->result_name}); } } @@ -451,7 +451,7 @@ size_t ScopeStack::getColumnLevel(const std::string & name) const auto & index = stack[i].actions->getIndex(); auto it = index.find(name); - if (it != index.end() && it->second->type != ActionsDAG::Type::INPUT) + if (it != index.end() && (*it)->type != ActionsDAG::Type::INPUT) return i; } @@ -475,15 +475,15 @@ void ScopeStack::addAlias(const std::string & name, std::string alias) stack[j].actions->addInput({node.column, node.result_type, node.result_name}); } -void ScopeStack::addArrayJoin(const std::string & source_name, std::string result_name, std::string unique_column_name) +void ScopeStack::addArrayJoin(const std::string & source_name, std::string result_name) { getColumnLevel(source_name); - if (stack.front().actions->getIndex().count(source_name) == 0) + if (!stack.front().actions->getIndex().contains(source_name)) throw Exception("Expression with arrayJoin cannot depend on lambda argument: " + source_name, ErrorCodes::BAD_ARGUMENTS); - const auto & node = stack.front().actions->addArrayJoin(source_name, std::move(result_name), std::move(unique_column_name)); + const auto & node = stack.front().actions->addArrayJoin(source_name, std::move(result_name)); for (size_t j = 1; j < stack.size(); ++j) stack[j].actions->addInput({node.column, node.result_type, node.result_name}); @@ -492,14 +492,13 @@ void ScopeStack::addArrayJoin(const std::string & source_name, std::string resul void ScopeStack::addFunction( const FunctionOverloadResolverPtr & function, const Names & argument_names, - std::string result_name, - bool compile_expressions) + std::string result_name) { size_t level = 0; for (const auto & argument : argument_names) level = std::max(level, getColumnLevel(argument)); - const auto & node = stack[level].actions->addFunction(function, argument_names, std::move(result_name), compile_expressions); + const auto & node = stack[level].actions->addFunction(function, argument_names, std::move(result_name), context); for (size_t j = level + 1; j < stack.size(); ++j) stack[j].actions->addInput({node.column, node.result_type, node.result_name}); @@ -746,7 +745,7 @@ void ActionsMatcher::visit(const ASTFunction & node, const ASTPtr & ast, Data & auto it = index.find(child_column_name); if (it != index.end()) { - argument_types.push_back(it->second->result_type); + argument_types.push_back((*it)->result_type); argument_names.push_back(child_column_name); } else @@ -792,10 +791,12 @@ void ActionsMatcher::visit(const ASTFunction & node, const ASTPtr & ast, Data & data.actions_stack.pushLevel(lambda_arguments); visit(lambda->arguments->children.at(1), data); auto lambda_dag = data.actions_stack.popLevel(); - auto lambda_actions = lambda_dag->buildExpressions(data.context); String result_name = lambda->arguments->children.at(1)->getColumnName(); - lambda_actions->finalize(Names(1, result_name)); + lambda_dag->removeUnusedActions(Names(1, result_name)); + + auto lambda_actions = lambda_dag->buildExpressions(); + DataTypePtr result_type = lambda_actions->getSampleBlock().getByName(result_name).type; Names captured; @@ -853,7 +854,7 @@ void ActionsMatcher::visit(const ASTLiteral & literal, const ASTPtr & /* ast */, auto it = index.find(default_name); if (it != index.end()) - existing_column = it->second; + existing_column = *it; /* * To approximate CSE, bind all identical literals to a single temporary @@ -964,7 +965,7 @@ SetPtr ActionsMatcher::makeSet(const ASTFunction & node, Data & data, bool no_su { const auto & last_actions = data.actions_stack.getLastActions(); const auto & index = last_actions.getIndex(); - if (index.count(left_in_operand->getColumnName()) != 0) + if (index.contains(left_in_operand->getColumnName())) /// An explicit enumeration of values in parentheses. return makeExplicitSet(&node, last_actions, false, data.context, data.set_size_limit, data.prepared_sets); else diff --git a/src/Interpreters/ActionsVisitor.h b/src/Interpreters/ActionsVisitor.h index f4da9932163..2d53bfa6185 100644 --- a/src/Interpreters/ActionsVisitor.h +++ b/src/Interpreters/ActionsVisitor.h @@ -12,7 +12,6 @@ namespace DB class Context; class ASTFunction; -struct ExpressionAction; class ExpressionActions; using ExpressionActionsPtr = std::shared_ptr; @@ -83,12 +82,11 @@ struct ScopeStack void addColumn(ColumnWithTypeAndName column); void addAlias(const std::string & name, std::string alias); - void addArrayJoin(const std::string & source_name, std::string result_name, std::string unique_column_name); + void addArrayJoin(const std::string & source_name, std::string result_name); void addFunction( const FunctionOverloadResolverPtr & function, const Names & argument_names, - std::string result_name, - bool compile_expressions); + std::string result_name); ActionsDAGPtr popLevel(); @@ -147,15 +145,14 @@ public: void addArrayJoin(const std::string & source_name, std::string result_name) { - actions_stack.addArrayJoin(source_name, std::move(result_name), getUniqueName("_array_join_" + source_name)); + actions_stack.addArrayJoin(source_name, std::move(result_name)); } void addFunction(const FunctionOverloadResolverPtr & function, const Names & argument_names, std::string result_name) { - actions_stack.addFunction(function, argument_names, std::move(result_name), - context.getSettingsRef().compile_expressions); + actions_stack.addFunction(function, argument_names, std::move(result_name)); } ActionsDAGPtr getActions() diff --git a/src/Interpreters/Aggregator.cpp b/src/Interpreters/Aggregator.cpp index 5b9169a878b..87abca4d7cd 100644 --- a/src/Interpreters/Aggregator.cpp +++ b/src/Interpreters/Aggregator.cpp @@ -112,7 +112,22 @@ Block Aggregator::Params::getHeader( { Block res; - if (src_header) + if (intermediate_header) + { + res = intermediate_header.cloneEmpty(); + + if (final) + { + for (const auto & aggregate : aggregates) + { + auto & elem = res.getByName(aggregate.column_name); + + elem.type = aggregate.function->getReturnType(); + elem.column = elem.type->createColumn(); + } + } + } + else { for (const auto & key : keys) res.insert(src_header.safeGetByPosition(key).cloneEmpty()); @@ -133,21 +148,6 @@ Block Aggregator::Params::getHeader( res.insert({ type, aggregate.column_name }); } } - else if (intermediate_header) - { - res = intermediate_header.cloneEmpty(); - - if (final) - { - for (const auto & aggregate : aggregates) - { - auto & elem = res.getByName(aggregate.column_name); - - elem.type = aggregate.function->getReturnType(); - elem.column = elem.type->createColumn(); - } - } - } return materializeBlock(res); } diff --git a/src/Interpreters/ExpressionActions.cpp b/src/Interpreters/ExpressionActions.cpp index ae6a355f31e..bb30e5b3cda 100644 --- a/src/Interpreters/ExpressionActions.cpp +++ b/src/Interpreters/ExpressionActions.cpp @@ -17,6 +17,7 @@ #include #include #include +#include #if !defined(ARCADIA_BUILD) # include "config_core.h" @@ -44,537 +45,94 @@ namespace ErrorCodes } /// Read comment near usage -static constexpr auto DUMMY_COLUMN_NAME = "_dummy"; +/// static constexpr auto DUMMY_COLUMN_NAME = "_dummy"; - -Names ExpressionAction::getNeededColumns() const +static std::ostream & operator << (std::ostream & out, const ExpressionActions::Argument & argument) { - Names res = argument_names; - - if (table_join) - res.insert(res.end(), table_join->keyNamesLeft().begin(), table_join->keyNamesLeft().end()); - - for (const auto & column : projection) - res.push_back(column.first); - - if (!source_name.empty()) - res.push_back(source_name); - - return res; + return out << (argument.remove ? "*" : "") << argument.pos; } - -ExpressionAction ExpressionAction::applyFunction( - const FunctionOverloadResolverPtr & function_, - const std::vector & argument_names_, - std::string result_name_) +std::string ExpressionActions::Action::toString() const { - if (result_name_.empty()) + std::stringstream out; + switch (node->type) { - result_name_ = function_->getName() + "("; - for (size_t i = 0 ; i < argument_names_.size(); ++i) - { - if (i) - result_name_ += ", "; - result_name_ += argument_names_[i]; - } - result_name_ += ")"; - } - - ExpressionAction a; - a.type = APPLY_FUNCTION; - a.result_name = result_name_; - a.function_builder = function_; - a.argument_names = argument_names_; - return a; -} - -ExpressionAction ExpressionAction::addColumn( - const ColumnWithTypeAndName & added_column_) -{ - ExpressionAction a; - a.type = ADD_COLUMN; - a.result_name = added_column_.name; - a.result_type = added_column_.type; - a.added_column = added_column_.column; - return a; -} - -ExpressionAction ExpressionAction::removeColumn(const std::string & removed_name) -{ - ExpressionAction a; - a.type = REMOVE_COLUMN; - a.source_name = removed_name; - return a; -} - -ExpressionAction ExpressionAction::copyColumn(const std::string & from_name, const std::string & to_name, bool can_replace) -{ - ExpressionAction a; - a.type = COPY_COLUMN; - a.source_name = from_name; - a.result_name = to_name; - a.can_replace = can_replace; - return a; -} - -ExpressionAction ExpressionAction::project(const NamesWithAliases & projected_columns_) -{ - ExpressionAction a; - a.type = PROJECT; - a.projection = projected_columns_; - return a; -} - -ExpressionAction ExpressionAction::project(const Names & projected_columns_) -{ - ExpressionAction a; - a.type = PROJECT; - a.projection.resize(projected_columns_.size()); - for (size_t i = 0; i < projected_columns_.size(); ++i) - a.projection[i] = NameWithAlias(projected_columns_[i], ""); - return a; -} - -ExpressionAction ExpressionAction::addAliases(const NamesWithAliases & aliased_columns_) -{ - ExpressionAction a; - a.type = ADD_ALIASES; - a.projection = aliased_columns_; - return a; -} - -ExpressionAction ExpressionAction::arrayJoin(std::string source_name, std::string result_name) -{ - if (source_name == result_name) - throw Exception("ARRAY JOIN action should have different source and result names", ErrorCodes::LOGICAL_ERROR); - - ExpressionAction a; - a.type = ARRAY_JOIN; - a.source_name = std::move(source_name); - a.result_name = std::move(result_name); - return a; -} - - -void ExpressionAction::prepare(Block & sample_block, const Settings & settings, NameSet & names_not_for_constant_folding) -{ - // std::cerr << "preparing: " << toString() << std::endl; - - /** Constant expressions should be evaluated, and put the result in sample_block. - */ - - switch (type) - { - case APPLY_FUNCTION: - { - if (sample_block.has(result_name)) - throw Exception("Column '" + result_name + "' already exists", ErrorCodes::DUPLICATE_COLUMN); - - bool all_const = true; - bool all_suitable_for_constant_folding = true; - - ColumnsWithTypeAndName arguments(argument_names.size()); - for (size_t i = 0; i < argument_names.size(); ++i) - { - arguments[i] = sample_block.getByName(argument_names[i]); - ColumnPtr col = arguments[i].column; - if (!col || !isColumnConst(*col)) - all_const = false; - - if (names_not_for_constant_folding.count(argument_names[i])) - all_suitable_for_constant_folding = false; - } - - size_t result_position = sample_block.columns(); - sample_block.insert({nullptr, result_type, result_name}); - if (!function) - function = function_base->prepare(arguments); - function->createLowCardinalityResultCache(settings.max_threads); - - bool compile_expressions = false; -#if USE_EMBEDDED_COMPILER - compile_expressions = settings.compile_expressions; -#endif - /// If all arguments are constants, and function is suitable to be executed in 'prepare' stage - execute function. - /// But if we compile expressions compiled version of this function maybe placed in cache, - /// so we don't want to unfold non deterministic functions - if (all_const && function_base->isSuitableForConstantFolding() && (!compile_expressions || function_base->isDeterministic())) - { - if (added_column) - sample_block.getByPosition(result_position).column = added_column; - else - sample_block.getByPosition(result_position).column = function->execute(arguments, result_type, sample_block.rows(), true); - - /// If the result is not a constant, just in case, we will consider the result as unknown. - ColumnWithTypeAndName & col = sample_block.safeGetByPosition(result_position); - if (!isColumnConst(*col.column)) - { - col.column = nullptr; - } - else - { - /// All constant (literal) columns in block are added with size 1. - /// But if there was no columns in block before executing a function, the result has size 0. - /// Change the size to 1. - - if (col.column->empty()) - col.column = col.column->cloneResized(1); - - if (!all_suitable_for_constant_folding) - names_not_for_constant_folding.insert(result_name); - } - } - - /// Some functions like ignore() or getTypeName() always return constant result even if arguments are not constant. - /// We can't do constant folding, but can specify in sample block that function result is constant to avoid - /// unnecessary materialization. - auto & res = sample_block.getByPosition(result_position); - if (!res.column && function_base->isSuitableForConstantFolding()) - { - if (auto col = function_base->getResultIfAlwaysReturnsConstantAndHasArguments(arguments)) - { - res.column = std::move(col); - names_not_for_constant_folding.insert(result_name); - } - } - - break; - } - - case ARRAY_JOIN: - { - ColumnWithTypeAndName current = sample_block.getByName(source_name); - sample_block.erase(source_name); - - const DataTypeArray * array_type = typeid_cast(&*current.type); - if (!array_type) - throw Exception("ARRAY JOIN requires array argument", ErrorCodes::TYPE_MISMATCH); - - current.name = result_name; - current.type = array_type->getNestedType(); - current.column = nullptr; /// Result is never const - sample_block.insert(std::move(current)); - - break; - } - - case PROJECT: - { - Block new_block; - - for (const auto & elem : projection) - { - const std::string & name = elem.first; - const std::string & alias = elem.second; - ColumnWithTypeAndName column = sample_block.getByName(name); - if (!alias.empty()) - column.name = alias; - new_block.insert(std::move(column)); - } - - sample_block.swap(new_block); - break; - } - - case ADD_ALIASES: - { - for (const auto & elem : projection) - { - const std::string & name = elem.first; - const std::string & alias = elem.second; - const ColumnWithTypeAndName & column = sample_block.getByName(name); - if (!alias.empty() && !sample_block.has(alias)) - sample_block.insert({column.column, column.type, alias}); - } - break; - } - - case REMOVE_COLUMN: - { - sample_block.erase(source_name); - break; - } - - case ADD_COLUMN: - { - if (sample_block.has(result_name)) - throw Exception("Column '" + result_name + "' already exists", ErrorCodes::DUPLICATE_COLUMN); - - sample_block.insert(ColumnWithTypeAndName(added_column, result_type, result_name)); - break; - } - - case COPY_COLUMN: - { - const auto & source = sample_block.getByName(source_name); - result_type = source.type; - - if (sample_block.has(result_name)) - { - if (can_replace) - { - auto & result = sample_block.getByName(result_name); - result.type = result_type; - result.column = source.column; - } - else - throw Exception("Column '" + result_name + "' already exists", ErrorCodes::DUPLICATE_COLUMN); - } - else - sample_block.insert(ColumnWithTypeAndName(source.column, result_type, result_name)); - - break; - } - } -} - -void ExpressionAction::execute(Block & block, bool dry_run) const -{ - size_t input_rows_count = block.rows(); - - if (type == REMOVE_COLUMN || type == COPY_COLUMN) - if (!block.has(source_name)) - throw Exception("Not found column '" + source_name + "'. There are columns: " + block.dumpNames(), ErrorCodes::NOT_FOUND_COLUMN_IN_BLOCK); - - if (type == ADD_COLUMN || (type == COPY_COLUMN && !can_replace) || type == APPLY_FUNCTION) - if (block.has(result_name)) - throw Exception("Column '" + result_name + "' already exists", ErrorCodes::DUPLICATE_COLUMN); - - switch (type) - { - case APPLY_FUNCTION: - { - ColumnsWithTypeAndName arguments(argument_names.size()); - for (size_t i = 0; i < argument_names.size(); ++i) - arguments[i] = block.getByName(argument_names[i]); - - size_t num_columns_without_result = block.columns(); - block.insert({ nullptr, result_type, result_name}); - - ProfileEvents::increment(ProfileEvents::FunctionExecute); - if (is_function_compiled) - ProfileEvents::increment(ProfileEvents::CompiledFunctionExecute); - block.getByPosition(num_columns_without_result).column = function->execute(arguments, result_type, input_rows_count, dry_run); - - break; - } - - case ARRAY_JOIN: - { - auto source = block.getByName(source_name); - block.erase(source_name); - source.column = source.column->convertToFullColumnIfConst(); - - const ColumnArray * array = typeid_cast(source.column.get()); - if (!array) - throw Exception("ARRAY JOIN of not array: " + source_name, ErrorCodes::TYPE_MISMATCH); - - for (auto & column : block) - column.column = column.column->replicate(array->getOffsets()); - - source.column = array->getDataPtr(); - source.type = assert_cast(*source.type).getNestedType(); - source.name = result_name; - - block.insert(std::move(source)); - - break; - } - - case PROJECT: - { - Block new_block; - - for (const auto & elem : projection) - { - const std::string & name = elem.first; - const std::string & alias = elem.second; - ColumnWithTypeAndName column = block.getByName(name); - if (!alias.empty()) - column.name = alias; - new_block.insert(std::move(column)); - } - - block.swap(new_block); - - break; - } - - case ADD_ALIASES: - { - for (const auto & elem : projection) - { - const std::string & name = elem.first; - const std::string & alias = elem.second; - const ColumnWithTypeAndName & column = block.getByName(name); - if (!alias.empty() && !block.has(alias)) - block.insert({column.column, column.type, alias}); - } - break; - } - - case REMOVE_COLUMN: - block.erase(source_name); + case ActionsDAG::Type::COLUMN: + out << "COLUMN " + << (node->column ? node->column->getName() : "(no column)"); break; - case ADD_COLUMN: - block.insert({ added_column->cloneResized(input_rows_count), result_type, result_name }); + case ActionsDAG::Type::ALIAS: + out << "ALIAS " << node->children.front()->result_name << " " << arguments.front(); break; - case COPY_COLUMN: - if (can_replace && block.has(result_name)) - { - auto & result = block.getByName(result_name); - const auto & source = block.getByName(source_name); - result.type = source.type; - result.column = source.column; - } - else - { - const auto & source_column = block.getByName(source_name); - block.insert({source_column.column, source_column.type, result_name}); - } - - break; - } -} - - -std::string ExpressionAction::toString() const -{ - std::stringstream ss; - switch (type) - { - case ADD_COLUMN: - ss << "ADD " << result_name << " " - << (result_type ? result_type->getName() : "(no type)") << " " - << (added_column ? added_column->getName() : "(no column)"); - break; - - case REMOVE_COLUMN: - ss << "REMOVE " << source_name; - break; - - case COPY_COLUMN: - ss << "COPY " << result_name << " = " << source_name; - if (can_replace) - ss << " (can replace)"; - break; - - case APPLY_FUNCTION: - ss << "FUNCTION " << result_name << " " << (is_function_compiled ? "[compiled] " : "") - << (result_type ? result_type->getName() : "(no type)") << " = " - << (function_base ? function_base->getName() : "(no function)") << "("; - for (size_t i = 0; i < argument_names.size(); ++i) + case ActionsDAG::Type::FUNCTION: + out << "FUNCTION " << (node->is_function_compiled ? "[compiled] " : "") + << (node->function_base ? node->function_base->getName() : "(no function)") << "("; + for (size_t i = 0; i < node->children.size(); ++i) { if (i) - ss << ", "; - ss << argument_names[i]; + out << ", "; + out << node->children[i]->result_name << " " << arguments[i]; } - ss << ")"; + out << ")"; break; - case ARRAY_JOIN: - ss << "ARRAY JOIN " << source_name << " -> " << result_name; + case ActionsDAG::Type::ARRAY_JOIN: + out << "ARRAY JOIN " << node->children.front()->result_name << " " << arguments.front(); break; - case PROJECT: [[fallthrough]]; - case ADD_ALIASES: - ss << (type == PROJECT ? "PROJECT " : "ADD_ALIASES "); - for (size_t i = 0; i < projection.size(); ++i) - { - if (i) - ss << ", "; - ss << projection[i].first; - if (!projection[i].second.empty() && projection[i].second != projection[i].first) - ss << " AS " << projection[i].second; - } + case ActionsDAG::Type::INPUT: + out << "INPUT " << arguments.front(); break; } - return ss.str(); -} - -ExpressionActions::ExpressionActions(const NamesAndTypesList & input_columns_, const Context & context_) - : input_columns(input_columns_), settings(context_.getSettingsRef()) -{ - for (const auto & input_elem : input_columns) - sample_block.insert(ColumnWithTypeAndName(nullptr, input_elem.type, input_elem.name)); - -#if USE_EMBEDDED_COMPILER -compilation_cache = context_.getCompiledExpressionCache(); -#endif -} - -/// For constant columns the columns themselves can be contained in `input_columns_`. -ExpressionActions::ExpressionActions(const ColumnsWithTypeAndName & input_columns_, const Context & context_) - : settings(context_.getSettingsRef()) -{ - for (const auto & input_elem : input_columns_) - { - input_columns.emplace_back(input_elem.name, input_elem.type); - sample_block.insert(input_elem); - } -#if USE_EMBEDDED_COMPILER - compilation_cache = context_.getCompiledExpressionCache(); -#endif + out << " -> " << node->result_name + << " " << (node->result_type ? node->result_type->getName() : "(no type)") << " " << result_position; + return out.str(); } ExpressionActions::~ExpressionActions() = default; void ExpressionActions::checkLimits(ExecutionContext & execution_context) const { - if (settings.max_temporary_columns && block.columns() > settings.max_temporary_columns) - throw Exception("Too many temporary columns: " + block.dumpNames() - + ". Maximum: " + settings.max_temporary_columns.toString(), - ErrorCodes::TOO_MANY_TEMPORARY_COLUMNS); - - if (settings.max_temporary_non_const_columns) + if (max_temporary_non_const_columns) { size_t non_const_columns = 0; - for (size_t i = 0, size = block.columns(); i < size; ++i) - if (block.safeGetByPosition(i).column && !isColumnConst(*block.safeGetByPosition(i).column)) + for (const auto & column : execution_context.columns) + if (column.column && !isColumnConst(*column.column)) ++non_const_columns; - if (non_const_columns > settings.max_temporary_non_const_columns) + if (non_const_columns > max_temporary_non_const_columns) { std::stringstream list_of_non_const_columns; - for (size_t i = 0, size = block.columns(); i < size; ++i) - if (block.safeGetByPosition(i).column && !isColumnConst(*block.safeGetByPosition(i).column)) - list_of_non_const_columns << "\n" << block.safeGetByPosition(i).name; + for (const auto & column : execution_context.columns) + if (column.column && !isColumnConst(*column.column)) + list_of_non_const_columns << "\n" << column.name; throw Exception("Too many temporary non-const columns:" + list_of_non_const_columns.str() - + ". Maximum: " + settings.max_temporary_non_const_columns.toString(), + + ". Maximum: " + std::to_string(max_temporary_non_const_columns), ErrorCodes::TOO_MANY_TEMPORARY_NON_CONST_COLUMNS); } } } -void ExpressionActions::prependProjectInput() -{ - actions.insert(actions.begin(), ExpressionAction::project(getRequiredColumns())); -} - -void ExpressionActions::execute(Block & block, bool dry_run) const +void ExpressionActions::execute(Block & block, size_t & num_rows, bool dry_run) const { ExecutionContext execution_context { - .input_columns = block.data, - .num_rows = block.rows(), + .inputs = block.data, + .num_rows = num_rows, }; - execution_context.columns.reserve(num_columns); + execution_context.inputs_pos.reserve(required_columns.size()); - ColumnNumbers inputs_to_remove; - inputs_to_remove.reserve(required_columns.size()); for (const auto & column : required_columns) { - size_t pos = block.getPositionByName(column.name); - execution_context.columns.emplace_back(std::move(block.getByPosition(pos))); - - if (!sample_block.has(column.name)) - inputs_to_remove.emplace_back(pos); + ssize_t pos = -1; + if (block.has(column.name)) + pos = block.getPositionByName(column.name); + execution_context.inputs_pos.push_back(pos); } execution_context.columns.resize(num_columns); @@ -585,6 +143,10 @@ void ExpressionActions::execute(Block & block, bool dry_run) const { executeAction(action, execution_context, dry_run); checkLimits(execution_context); + + //std::cerr << "Action: " << action.toString() << std::endl; + //for (const auto & col : execution_context.columns) + // std::cerr << col.dumpStructure() << std::endl; } catch (Exception & e) { @@ -593,27 +155,38 @@ void ExpressionActions::execute(Block & block, bool dry_run) const } } - std::sort(inputs_to_remove.rbegin(), inputs_to_remove.rend()); - for (auto input : inputs_to_remove) - block.erase(input); - - for (const auto & action : actions) + if (project_input) { - if (!action.is_used_in_result) - continue; - - auto & column = execution_context.columns[action.result_position]; - column.name = action.node->result_name; - - if (block.has(action.node->result_name)) - block.getByName(action.node->result_name) = std::move(column); - else - block.insert(std::move(column)); + block.clear(); } + else + { + std::sort(execution_context.inputs_pos.rbegin(), execution_context.inputs_pos.rend()); + for (auto input : execution_context.inputs_pos) + if (input >= 0) + block.erase(input); + } + + for (auto pos : result_positions) + if (execution_context.columns[pos].column) + block.insert(execution_context.columns[pos]); + + num_rows = execution_context.num_rows; +} + +void ExpressionActions::execute(Block & block, bool dry_run) const +{ + size_t num_rows = block.rows(); + + execute(block, num_rows, dry_run); + + if (!block) + block.insert({DataTypeUInt8().createColumnConst(num_rows, 0), std::make_shared(), "_dummy"}); } void ExpressionActions::executeAction(const Action & action, ExecutionContext & execution_context, bool dry_run) { + auto & inputs = execution_context.inputs; auto & columns = execution_context.columns; auto & num_rows = execution_context.num_rows; @@ -626,24 +199,32 @@ void ExpressionActions::executeAction(const Action & action, ExecutionContext & throw Exception("Result column is not empty", ErrorCodes::LOGICAL_ERROR); res_column.type = action.node->result_type; - /// Columns names are not used, avoid extra copy. - /// res_column.name = action.node->result_name; + res_column.name = action.node->result_name; + + ColumnsWithTypeAndName arguments(action.arguments.size()); + for (size_t i = 0; i < arguments.size(); ++i) + { + if (action.arguments[i].remove) + arguments[i] = std::move(columns[action.arguments[i].pos]); + else + arguments[i] = columns[action.arguments[i].pos]; + } ProfileEvents::increment(ProfileEvents::FunctionExecute); if (action.node->is_function_compiled) ProfileEvents::increment(ProfileEvents::CompiledFunctionExecute); - action.node->function->execute(columns, action.arguments, action.result_position, num_rows, dry_run); + res_column.column = action.node->function->execute(arguments, res_column.type, num_rows, dry_run); break; } case ActionsDAG::Type::ARRAY_JOIN: { - size_t array_join_key_pos = action.arguments.front(); + size_t array_join_key_pos = action.arguments.front().pos; auto array_join_key = columns[array_join_key_pos]; /// Remove array join argument in advance if it is not needed. - if (!action.to_remove.empty()) + if (action.arguments.front().remove) columns[array_join_key_pos] = {}; array_join_key.column = array_join_key.column->convertToFullColumnIfConst(); @@ -656,7 +237,7 @@ void ExpressionActions::executeAction(const Action & action, ExecutionContext & if (column.column) column.column = column.column->replicate(array->getOffsets()); - for (auto & column : execution_context.input_columns) + for (auto & column : inputs) if (column.column) column.column = column.column->replicate(array->getOffsets()); @@ -664,6 +245,7 @@ void ExpressionActions::executeAction(const Action & action, ExecutionContext & res_column.column = array->getDataPtr(); res_column.type = assert_cast(*array_join_key.type).getNestedType(); + res_column.name = action.node->result_name; num_rows = res_column.column->size(); break; @@ -674,24 +256,51 @@ void ExpressionActions::executeAction(const Action & action, ExecutionContext & auto & res_column = columns[action.result_position]; res_column.column = action.node->column->cloneResized(num_rows); res_column.type = action.node->result_type; + res_column.name = action.node->result_name; break; } case ActionsDAG::Type::ALIAS: { - /// Do not care about names, they are empty. - columns[action.result_position] = columns[action.arguments.front()]; + const auto & arg = action.arguments.front(); + if (action.result_position != arg.pos) + { + columns[action.result_position].column = columns[arg.pos].column; + columns[action.result_position].type = columns[arg.pos].type; + + if (arg.remove) + columns[arg.pos] = {}; + } + + columns[action.result_position].name = action.node->result_name; + break; } case ActionsDAG::Type::INPUT: { - throw Exception("Cannot execute INPUT action", ErrorCodes::LOGICAL_ERROR); + auto pos = execution_context.inputs_pos[action.arguments.front().pos]; + if (pos < 0) + { + if (!action.arguments.front().remove) + throw Exception(ErrorCodes::NOT_FOUND_COLUMN_IN_BLOCK, + "Not found column {} in block", + action.node->result_name); + } + else + columns[action.result_position] = std::move(inputs[pos]); + + break; } } +} - for (auto to_remove : action.to_remove) - columns[to_remove] = {}; +Names ExpressionActions::getRequiredColumns() const +{ + Names names; + for (const auto & input : required_columns) + names.push_back(input.name); + return names; } bool ExpressionActions::hasArrayJoin() const @@ -727,242 +336,12 @@ std::string ExpressionActions::getSmallestColumn(const NamesAndTypesList & colum return res; } -void ExpressionActions::finalize(const Names & output_columns) -{ - NameSet final_columns; - for (const auto & name : output_columns) - { - if (!sample_block.has(name)) - throw Exception("Unknown column: " + name + ", there are only columns " - + sample_block.dumpNames(), ErrorCodes::UNKNOWN_IDENTIFIER); - final_columns.insert(name); - } - -#if USE_EMBEDDED_COMPILER - /// This has to be done before removing redundant actions and inserting REMOVE_COLUMNs - /// because inlining may change dependency sets. - if (settings.compile_expressions) - compileFunctions(actions, output_columns, sample_block, compilation_cache, settings.min_count_to_compile_expression); -#endif - - /// Which columns are needed to perform actions from the current to the last. - NameSet needed_columns = final_columns; - /// Which columns nobody will touch from the current action to the last. - NameSet unmodified_columns; - - { - NamesAndTypesList sample_columns = sample_block.getNamesAndTypesList(); - for (const auto & sample_column : sample_columns) - unmodified_columns.insert(sample_column.name); - } - - /// Let's go from the end and maintain set of required columns at this stage. - /// We will throw out unnecessary actions, although usually they are absent by construction. - for (int i = static_cast(actions.size()) - 1; i >= 0; --i) - { - ExpressionAction & action = actions[i]; - Names in = action.getNeededColumns(); - - if (action.type == ExpressionAction::PROJECT) - { - needed_columns = NameSet(in.begin(), in.end()); - unmodified_columns.clear(); - } - else if (action.type == ExpressionAction::ADD_ALIASES) - { - needed_columns.insert(in.begin(), in.end()); - for (auto & name_wit_alias : action.projection) - { - auto it = unmodified_columns.find(name_wit_alias.second); - if (it != unmodified_columns.end()) - unmodified_columns.erase(it); - } - } - else if (action.type == ExpressionAction::ARRAY_JOIN) - { - /// We need source anyway, in order to calculate number of rows correctly. - needed_columns.insert(action.source_name); - unmodified_columns.erase(action.result_name); - needed_columns.erase(action.result_name); - - /// Note: technically, if result of arrayJoin is not needed, - /// we may remove all the columns and loose the number of rows here. - /// However, I cannot imagine how it is possible. - /// For "big" ARRAY JOIN it could have happened in query like - /// SELECT count() FROM table ARRAY JOIN x - /// Now, "big" ARRAY JOIN is moved to separate pipeline step, - /// and arrayJoin(x) is an expression which result can't be lost. - } - else - { - std::string out = action.result_name; - if (!out.empty()) - { - /// If the result is not used and there are no side effects, throw out the action. - if (!needed_columns.count(out) && - (action.type == ExpressionAction::APPLY_FUNCTION - || action.type == ExpressionAction::ADD_COLUMN - || action.type == ExpressionAction::COPY_COLUMN)) - { - actions.erase(actions.begin() + i); - - if (unmodified_columns.count(out)) - { - sample_block.erase(out); - unmodified_columns.erase(out); - } - - continue; - } - - unmodified_columns.erase(out); - needed_columns.erase(out); - - /** If the function is a constant expression, then replace the action by adding a column-constant - result. - * That is, we perform constant folding. - */ - if (action.type == ExpressionAction::APPLY_FUNCTION && sample_block.has(out)) - { - auto & result = sample_block.getByName(out); - if (result.column && names_not_for_constant_folding.count(result.name) == 0) - { - action.type = ExpressionAction::ADD_COLUMN; - action.result_type = result.type; - action.added_column = result.column; - action.function_builder = nullptr; - action.function_base = nullptr; - action.function = nullptr; - action.argument_names.clear(); - in.clear(); - } - } - } - - needed_columns.insert(in.begin(), in.end()); - } - } - - - /// 1) Sometimes we don't need any columns to perform actions and sometimes actions doesn't produce any columns as result. - /// But Block class doesn't store any information about structure itself, it uses information from column. - /// If we remove all columns from input or output block we will lose information about amount of rows in it. - /// To avoid this situation we always leaving one of the columns in required columns (input) - /// and output column. We choose that "redundant" column by size with help of getSmallestColumn. - /// - /// 2) Sometimes we have to read data from different Storages to execute query. - /// For example in 'remote' function which requires to read data from local table (for example MergeTree) and - /// remote table (doesn't know anything about it). - /// - /// If we have combination of two previous cases, our heuristic from (1) can choose absolutely different columns, - /// so generated streams with these actions will have different headers. To avoid this we additionally rename our "redundant" column - /// to DUMMY_COLUMN_NAME with help of COPY_COLUMN action and consequent remove of original column. - /// It doesn't affect any logic, but all streams will have same "redundant" column in header called "_dummy". - - /// Also, it seems like we will always have same type (UInt8) of "redundant" column, but it's not obvious. - - bool dummy_column_copied = false; - - - /// We will not throw out all the input columns, so as not to lose the number of rows in the block. - if (needed_columns.empty() && !input_columns.empty()) - { - auto colname = getSmallestColumn(input_columns); - needed_columns.insert(colname); - actions.insert(actions.begin(), ExpressionAction::copyColumn(colname, DUMMY_COLUMN_NAME, true)); - dummy_column_copied = true; - } - - /// We will not leave the block empty so as not to lose the number of rows in it. - if (final_columns.empty() && !input_columns.empty()) - { - auto colname = getSmallestColumn(input_columns); - final_columns.insert(DUMMY_COLUMN_NAME); - if (!dummy_column_copied) /// otherwise we already have this column - actions.insert(actions.begin(), ExpressionAction::copyColumn(colname, DUMMY_COLUMN_NAME, true)); - } - - for (NamesAndTypesList::iterator it = input_columns.begin(); it != input_columns.end();) - { - NamesAndTypesList::iterator it0 = it; - ++it; - if (!needed_columns.count(it0->name)) - { - if (unmodified_columns.count(it0->name)) - sample_block.erase(it0->name); - input_columns.erase(it0); - } - } - -/* std::cerr << "\n"; - for (const auto & action : actions) - std::cerr << action.toString() << "\n"; - std::cerr << "\n";*/ - - /// Deletes unnecessary temporary columns. - - /// If the column after performing the function `refcount = 0`, it can be deleted. - std::map columns_refcount; - - for (const auto & name : final_columns) - ++columns_refcount[name]; - - for (const auto & action : actions) - { - if (!action.source_name.empty()) - ++columns_refcount[action.source_name]; - - for (const auto & name : action.argument_names) - ++columns_refcount[name]; - - for (const auto & name_alias : action.projection) - ++columns_refcount[name_alias.first]; - } - - Actions new_actions; - new_actions.reserve(actions.size()); - - for (const auto & action : actions) - { - new_actions.push_back(action); - - auto process = [&] (const String & name) - { - auto refcount = --columns_refcount[name]; - if (refcount <= 0 && action.type != ExpressionAction::ARRAY_JOIN) - { - new_actions.push_back(ExpressionAction::removeColumn(name)); - if (sample_block.has(name)) - sample_block.erase(name); - } - }; - - if (!action.source_name.empty()) - process(action.source_name); - - for (const auto & name : action.argument_names) - process(name); - - /// For `projection`, there is no reduction in `refcount`, because the `project` action replaces the names of the columns, in effect, already deleting them under the old names. - } - - actions.swap(new_actions); - -/* std::cerr << "\n"; - for (const auto & action : actions) - std::cerr << action.toString() << "\n"; - std::cerr << "\n";*/ - - optimizeArrayJoin(); - checkLimits(sample_block); -} - - std::string ExpressionActions::dumpActions() const { std::stringstream ss; ss << "input:\n"; - for (const auto & input_column : input_columns) + for (const auto & input_column : required_columns) ss << input_column.name << " " << input_column.type->getName() << "\n"; ss << "\nactions:\n"; @@ -974,130 +353,197 @@ std::string ExpressionActions::dumpActions() const for (const auto & output_column : output_columns) ss << output_column.name << " " << output_column.type->getName() << "\n"; + ss << "\nproject input: " << project_input << "\noutput positions:"; + for (auto pos : result_positions) + ss << " " << pos; + ss << "\n"; + return ss.str(); } -ExpressionActionsPtr ExpressionActions::splitActionsBeforeArrayJoin(const NameSet & array_joined_columns) +//static std::string getUniqueNameForIndex(ActionsDAG::Index & index, std::string name) +//{ +// if (index.contains(name)) +// return name; +// +// size_t next_id = 0; +// std::string res; +// do +// res = name + "_" + std::to_string(next_id); +// while (index.contains(res)); +// +// return res; +//} + +bool ActionsDAG::hasArrayJoin() const { - /// Create new actions. - /// Copy from this because we don't have context. - /// TODO: remove context from constructor? - auto split_actions = std::make_shared(*this); - split_actions->actions.clear(); - split_actions->sample_block.clear(); - split_actions->input_columns.clear(); + for (const auto & node : nodes) + if (node.type == Type::ARRAY_JOIN) + return true; - /// Expected chain: - /// Expression (this) -> ArrayJoin (array_joined_columns) -> Expression (split_actions) + return false; +} - /// We are going to move as many actions as we can from this to split_actions. - /// We can move all inputs which are not depend on array_joined_columns - /// (with some exceptions to PROJECT and REMOVE_COLUMN +bool ActionsDAG::empty() const +{ + for (const auto & node : nodes) + if (node.type != Type::INPUT) + return false; - /// Use the same inputs for split_actions, except array_joined_columns. - for (const auto & input_column : input_columns) + return true; +} + +ActionsDAGPtr ActionsDAG::splitActionsBeforeArrayJoin(const NameSet & array_joined_columns) +{ + /// Split DAG into two parts. + /// (this_nodes, this_index) is a part which depends on ARRAY JOIN and stays here. + /// (split_nodes, split_index) is a part which will be moved before ARRAY JOIN. + std::list this_nodes; + std::list split_nodes; + Index this_index; + Index split_index; + + struct Frame { - if (array_joined_columns.count(input_column.name) == 0) - { - split_actions->input_columns.emplace_back(input_column); - split_actions->sample_block.insert(ColumnWithTypeAndName(nullptr, input_column.type, input_column.name)); - } - } + Node * node; + size_t next_child_to_visit = 0; + }; - /// Do not split action if input depends only on array joined columns. - if (split_actions->input_columns.empty()) - return nullptr; - - /// Actions which depend on ARRAY JOIN result. - NameSet array_join_dependent_columns = array_joined_columns; - /// Arguments of actions which depend on ARRAY JOIN result. - /// This columns can't be deleted in split_actions. - NameSet array_join_dependent_columns_arguments; - - /// We create new_actions list for `this`. Current actions are moved to new_actions nor added to split_actions. - Actions new_actions; - for (const auto & action : actions) + struct Data { - /// Exception for PROJECT. - /// It removes columns, so it will remove split_actions output which may be needed for actions from `this`. - /// So, we replace it ADD_ALIASES. - /// Usually, PROJECT is added to begin of actions in order to remove unused output of prev actions. - /// We skip it now, but will prependProjectInput at the end. - if (action.type == ExpressionAction::PROJECT) + bool depend_on_array_join = false; + bool visited = false; + bool used_in_result = false; + + /// Copies of node in one of the DAGs. + /// For COLUMN and INPUT both copies may exist. + Node * to_this = nullptr; + Node * to_split = nullptr; + }; + + std::stack stack; + std::unordered_map data; + + for (const auto & node : index) + data[node].used_in_result = true; + + /// DFS. Decide if node depends on ARRAY JOIN and move it to one of the DAGs. + for (auto & node : nodes) + { + if (!data[&node].visited) + stack.push({.node = &node}); + + while (!stack.empty()) { - /// Each alias has separate dependencies, so we split this action into two parts. - NamesWithAliases split_aliases; - NamesWithAliases depend_aliases; - for (const auto & pair : action.projection) + auto & cur = stack.top(); + auto & cur_data = data[cur.node]; + + /// At first, visit all children. We depend on ARRAY JOIN if any child does. + while (cur.next_child_to_visit < cur.node->children.size()) { - /// Skip if is not alias. - if (pair.second.empty()) - continue; + auto * child = cur.node->children[cur.next_child_to_visit]; + auto & child_data = data[child]; - if (array_join_dependent_columns.count(pair.first)) + if (!child_data.visited) { - array_join_dependent_columns.insert(pair.second); - depend_aliases.emplace_back(std::move(pair)); + stack.push({.node = child}); + break; } - else - split_aliases.emplace_back(std::move(pair)); + + ++cur.next_child_to_visit; + if (child_data.depend_on_array_join) + cur_data.depend_on_array_join = true; } - if (!split_aliases.empty()) - split_actions->add(ExpressionAction::addAliases(split_aliases)); + /// Make a copy part. + if (cur.next_child_to_visit == cur.node->children.size()) + { + if (cur.node->type == Type::INPUT && array_joined_columns.count(cur.node->result_name)) + cur_data.depend_on_array_join = true; - if (!depend_aliases.empty()) - new_actions.emplace_back(ExpressionAction::addAliases(depend_aliases)); + cur_data.visited = true; + stack.pop(); - continue; + if (cur_data.depend_on_array_join) + { + auto & copy = this_nodes.emplace_back(*cur.node); + cur_data.to_this = © + + /// Replace children to newly created nodes. + for (auto & child : copy.children) + { + auto & child_data = data[child]; + + /// If children is not created, int may be from split part. + if (!child_data.to_this) + { + if (child->type == Type::COLUMN) /// Just create new node for COLUMN action. + { + child_data.to_this = &this_nodes.emplace_back(*child); + } + else + { + /// Node from split part is added as new input. + Node input_node; + input_node.type = Type::INPUT; + input_node.result_type = child->result_type; + input_node.result_name = child->result_name; // getUniqueNameForIndex(index, child->result_name); + child_data.to_this = &this_nodes.emplace_back(std::move(input_node)); + + /// This node is needed for current action, so put it to index also. + split_index[child_data.to_split->result_name] = child_data.to_split; + } + } + + child = child_data.to_this; + } + } + else + { + auto & copy = split_nodes.emplace_back(*cur.node); + cur_data.to_split = © + + /// Replace children to newly created nodes. + for (auto & child : copy.children) + { + child = data[child].to_split; + assert(child != nullptr); + } + + if (cur_data.used_in_result) + { + split_index[copy.result_name] = © + + /// If this node is needed in result, add it as input. + Node input_node; + input_node.type = Type::INPUT; + input_node.result_type = node.result_type; + input_node.result_name = node.result_name; + cur_data.to_this = &this_nodes.emplace_back(std::move(input_node)); + } + } + } } - - bool depends_on_array_join = false; - for (auto & column : action.getNeededColumns()) - if (array_join_dependent_columns.count(column) != 0) - depends_on_array_join = true; - - if (depends_on_array_join) - { - /// Add result of this action to array_join_dependent_columns too. - if (!action.result_name.empty()) - array_join_dependent_columns.insert(action.result_name); - - /// Add arguments of this action to array_join_dependent_columns_arguments. - auto needed = action.getNeededColumns(); - array_join_dependent_columns_arguments.insert(needed.begin(), needed.end()); - - new_actions.emplace_back(action); - } - else if (action.type == ExpressionAction::REMOVE_COLUMN) - { - /// Exception for REMOVE_COLUMN. - /// We cannot move it to split_actions if any argument from `this` needed that column. - if (array_join_dependent_columns_arguments.count(action.source_name)) - new_actions.emplace_back(action); - else - split_actions->add(action); - } - else - split_actions->add(action); } - /// Return empty actions if nothing was separated. Keep `this` unchanged. - if (split_actions->getActions().empty()) - return nullptr; + for (auto * node : index) + this_index.insert(data[node].to_this); - std::swap(actions, new_actions); + /// Consider actions are empty if all nodes are constants or inputs. + bool split_actions_are_empty = true; + for (const auto & node : split_nodes) + if (!node.children.empty()) + split_actions_are_empty = false; - /// Collect inputs from ARRAY JOIN. - NamesAndTypesList inputs_from_array_join; - for (auto & column : input_columns) - if (array_joined_columns.count(column.name)) - inputs_from_array_join.emplace_back(std::move(column)); + if (split_actions_are_empty) + return {}; - /// Fix inputs for `this`. - /// It is output of split_actions + inputs from ARRAY JOIN. - input_columns = split_actions->getSampleBlock().getNamesAndTypesList(); - input_columns.insert(input_columns.end(), inputs_from_array_join.begin(), inputs_from_array_join.end()); + index.swap(this_index); + nodes.swap(this_nodes); + + auto split_actions = cloneEmpty(); + split_actions->nodes.swap(split_nodes); + split_actions->index.swap(split_index); return split_actions; } @@ -1111,15 +557,16 @@ bool ExpressionActions::checkColumnIsAlwaysFalse(const String & column_name) con for (auto it = actions.rbegin(); it != actions.rend(); ++it) { const auto & action = *it; - if (action.type == action.APPLY_FUNCTION && action.function_base) + if (action.node->type == ActionsDAG::Type::FUNCTION && action.node->function_base) { - auto name = action.function_base->getName(); - if ((name == "in" || name == "globalIn") - && action.result_name == column_name - && action.argument_names.size() > 1) + if (action.node->result_name == column_name && action.node->children.size() > 1) { - set_to_check = action.argument_names[1]; - break; + auto name = action.node->function_base->getName(); + if ((name == "in" || name == "globalIn")) + { + set_to_check = action.node->children[1]->result_name; + break; + } } } } @@ -1128,10 +575,10 @@ bool ExpressionActions::checkColumnIsAlwaysFalse(const String & column_name) con { for (const auto & action : actions) { - if (action.type == action.ADD_COLUMN && action.result_name == set_to_check) + if (action.node->type == ActionsDAG::Type::COLUMN && action.node->result_name == set_to_check) { // Constant ColumnSet cannot be empty, so we only need to check non-constant ones. - if (const auto * column_set = checkAndGetColumn(action.added_column.get())) + if (const auto * column_set = checkAndGetColumn(action.node->column.get())) { if (column_set->getData()->isCreated() && column_set->getData()->getTotalRowCount() == 0) return true; @@ -1144,115 +591,113 @@ bool ExpressionActions::checkColumnIsAlwaysFalse(const String & column_name) con } -/// It is not important to calculate the hash of individual strings or their concatenation -UInt128 ExpressionAction::ActionHash::operator()(const ExpressionAction & action) const -{ - SipHash hash; - hash.update(action.type); - hash.update(action.is_function_compiled); - switch (action.type) - { - case ADD_COLUMN: - hash.update(action.result_name); - if (action.result_type) - hash.update(action.result_type->getName()); - if (action.added_column) - hash.update(action.added_column->getName()); - break; - case REMOVE_COLUMN: - hash.update(action.source_name); - break; - case COPY_COLUMN: - hash.update(action.result_name); - hash.update(action.source_name); - break; - case APPLY_FUNCTION: - hash.update(action.result_name); - if (action.result_type) - hash.update(action.result_type->getName()); - if (action.function_base) - { - hash.update(action.function_base->getName()); - for (const auto & arg_type : action.function_base->getArgumentTypes()) - hash.update(arg_type->getName()); - } - for (const auto & arg_name : action.argument_names) - hash.update(arg_name); - break; - case ARRAY_JOIN: - hash.update(action.result_name); - hash.update(action.source_name); - break; - case PROJECT: - for (const auto & pair_of_strs : action.projection) - { - hash.update(pair_of_strs.first); - hash.update(pair_of_strs.second); - } - break; - case ADD_ALIASES: - break; - } - UInt128 result; - hash.get128(result.low, result.high); - return result; -} +///// It is not important to calculate the hash of individual strings or their concatenation +//UInt128 ExpressionAction::ActionHash::operator()(const ExpressionAction & action) const +//{ +// SipHash hash; +// hash.update(action.type); +// hash.update(action.is_function_compiled); +// switch (action.type) +// { +// case ADD_COLUMN: +// hash.update(action.result_name); +// if (action.result_type) +// hash.update(action.result_type->getName()); +// if (action.added_column) +// hash.update(action.added_column->getName()); +// break; +// case REMOVE_COLUMN: +// hash.update(action.source_name); +// break; +// case COPY_COLUMN: +// hash.update(action.result_name); +// hash.update(action.source_name); +// break; +// case APPLY_FUNCTION: +// hash.update(action.result_name); +// if (action.result_type) +// hash.update(action.result_type->getName()); +// if (action.function_base) +// { +// hash.update(action.function_base->getName()); +// for (const auto & arg_type : action.function_base->getArgumentTypes()) +// hash.update(arg_type->getName()); +// } +// for (const auto & arg_name : action.argument_names) +// hash.update(arg_name); +// break; +// case ARRAY_JOIN: +// hash.update(action.result_name); +// hash.update(action.source_name); +// break; +// case PROJECT: +// for (const auto & pair_of_strs : action.projection) +// { +// hash.update(pair_of_strs.first); +// hash.update(pair_of_strs.second); +// } +// break; +// case ADD_ALIASES: +// break; +// } +// UInt128 result; +// hash.get128(result.low, result.high); +// return result; +//} +// +//bool ExpressionAction::operator==(const ExpressionAction & other) const +//{ +// if (result_type != other.result_type) +// { +// if (result_type == nullptr || other.result_type == nullptr) +// return false; +// else if (!result_type->equals(*other.result_type)) +// return false; +// } +// +// if (function_base != other.function_base) +// { +// if (function_base == nullptr || other.function_base == nullptr) +// return false; +// else if (function_base->getName() != other.function_base->getName()) +// return false; +// +// const auto & my_arg_types = function_base->getArgumentTypes(); +// const auto & other_arg_types = other.function_base->getArgumentTypes(); +// if (my_arg_types.size() != other_arg_types.size()) +// return false; +// +// for (size_t i = 0; i < my_arg_types.size(); ++i) +// if (!my_arg_types[i]->equals(*other_arg_types[i])) +// return false; +// } +// +// if (added_column != other.added_column) +// { +// if (added_column == nullptr || other.added_column == nullptr) +// return false; +// else if (added_column->getName() != other.added_column->getName()) +// return false; +// } +// +// return source_name == other.source_name +// && result_name == other.result_name +// && argument_names == other.argument_names +// && TableJoin::sameJoin(table_join.get(), other.table_join.get()) +// && projection == other.projection +// && is_function_compiled == other.is_function_compiled; +//} -bool ExpressionAction::operator==(const ExpressionAction & other) const -{ - if (result_type != other.result_type) - { - if (result_type == nullptr || other.result_type == nullptr) - return false; - else if (!result_type->equals(*other.result_type)) - return false; - } - - if (function_base != other.function_base) - { - if (function_base == nullptr || other.function_base == nullptr) - return false; - else if (function_base->getName() != other.function_base->getName()) - return false; - - const auto & my_arg_types = function_base->getArgumentTypes(); - const auto & other_arg_types = other.function_base->getArgumentTypes(); - if (my_arg_types.size() != other_arg_types.size()) - return false; - - for (size_t i = 0; i < my_arg_types.size(); ++i) - if (!my_arg_types[i]->equals(*other_arg_types[i])) - return false; - } - - if (added_column != other.added_column) - { - if (added_column == nullptr || other.added_column == nullptr) - return false; - else if (added_column->getName() != other.added_column->getName()) - return false; - } - - return source_name == other.source_name - && result_name == other.result_name - && argument_names == other.argument_names - && TableJoin::sameJoin(table_join.get(), other.table_join.get()) - && projection == other.projection - && is_function_compiled == other.is_function_compiled; -} - -void ExpressionActionsChain::addStep() +void ExpressionActionsChain::addStep(NameSet non_constant_inputs) { if (steps.empty()) throw Exception("Cannot add action to empty ExpressionActionsChain", ErrorCodes::LOGICAL_ERROR); - if (auto * step = typeid_cast(steps.back().get())) - { - if (!step->actions) - step->actions = step->actions_dag->buildExpressions(context); - } - ColumnsWithTypeAndName columns = steps.back()->getResultColumns(); + for (auto & column : columns) + if (column.column && isColumnConst(*column.column) && non_constant_inputs.count(column.name)) + column.column = nullptr; + steps.push_back(std::make_unique(std::make_shared(columns))); } @@ -1402,15 +847,10 @@ void ExpressionActionsChain::JoinStep::finalize(const Names & required_output_) ActionsDAGPtr & ExpressionActionsChain::Step::actions() { - return typeid_cast(this)->actions_dag; + return typeid_cast(this)->actions; } const ActionsDAGPtr & ExpressionActionsChain::Step::actions() const -{ - return typeid_cast(this)->actions_dag; -} - -ExpressionActionsPtr ExpressionActionsChain::Step::getExpression() const { return typeid_cast(this)->actions; } @@ -1424,7 +864,12 @@ ActionsDAG::ActionsDAG(const NamesAndTypesList & inputs) ActionsDAG::ActionsDAG(const ColumnsWithTypeAndName & inputs) { for (const auto & input : inputs) - addInput(input); + { + if (input.column && isColumnConst(*input.column)) + addInput(input); + else + addInput(input.name, input.type); + } } ActionsDAG::Node & ActionsDAG::addNode(Node node, bool can_replace) @@ -1435,9 +880,6 @@ ActionsDAG::Node & ActionsDAG::addNode(Node node, bool can_replace) auto & res = nodes.emplace_back(std::move(node)); - if (it != index.end()) - it->second->renaming_parent = &res; - index[res.result_name] = &res; return res; } @@ -1448,7 +890,7 @@ ActionsDAG::Node & ActionsDAG::getNode(const std::string & name) if (it == index.end()) throw Exception("Unknown identifier: '" + name + "'", ErrorCodes::UNKNOWN_IDENTIFIER); - return *it->second; + return **it; } const ActionsDAG::Node & ActionsDAG::addInput(std::string name, DataTypePtr type) @@ -1475,7 +917,7 @@ const ActionsDAG::Node & ActionsDAG::addInput(ColumnWithTypeAndName column) const ActionsDAG::Node & ActionsDAG::addColumn(ColumnWithTypeAndName column) { if (!column.column) - throw Exception("Cannot add column " + column.name + " because it is nullptr", ErrorCodes::LOGICAL_ERROR); + throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot add column {} because it is nullptr", column.name); Node node; node.type = Type::COLUMN; @@ -1501,8 +943,7 @@ const ActionsDAG::Node & ActionsDAG::addAlias(const std::string & name, std::str return addNode(std::move(node), can_replace); } -const ActionsDAG::Node & ActionsDAG::addArrayJoin( - const std::string & source_name, std::string result_name, std::string unique_column_name) +const ActionsDAG::Node & ActionsDAG::addArrayJoin(const std::string & source_name, std::string result_name) { auto & child = getNode(source_name); @@ -1514,7 +955,6 @@ const ActionsDAG::Node & ActionsDAG::addArrayJoin( node.type = Type::ARRAY_JOIN; node.result_type = array_type->getNestedType(); node.result_name = std::move(result_name); - node.unique_column_name_for_array_join = std::move(unique_column_name); node.children.emplace_back(&child); return addNode(std::move(node)); @@ -1524,8 +964,20 @@ const ActionsDAG::Node & ActionsDAG::addFunction( const FunctionOverloadResolverPtr & function, const Names & argument_names, std::string result_name, - bool compile_expressions [[maybe_unused]]) + const Context & context [[maybe_unused]]) { + const auto & settings = context.getSettingsRef(); + max_temporary_columns = settings.max_temporary_columns; + max_temporary_non_const_columns = settings.max_temporary_non_const_columns; + + bool do_compile_expressions = false; +#if USE_EMBEDDED_COMPILER + do_compile_expressions = settings.compile_expressions; + + if (!compilation_cache) + compilation_cache = context.getCompiledExpressionCache(); +#endif + size_t num_arguments = argument_names.size(); Node node; @@ -1556,10 +1008,6 @@ const ActionsDAG::Node & ActionsDAG::addFunction( node.result_type = node.function_base->getResultType(); node.function = node.function_base->prepare(arguments); - bool do_compile_expressions = false; -#if USE_EMBEDDED_COMPILER - do_compile_expressions = compile_expressions; -#endif /// If all arguments are constants, and function is suitable to be executed in 'prepare' stage - execute function. /// But if we compile expressions compiled version of this function maybe placed in cache, /// so we don't want to unfold non deterministic functions @@ -1611,13 +1059,22 @@ const ActionsDAG::Node & ActionsDAG::addFunction( return addNode(std::move(node)); } +NamesAndTypesList ActionsDAG::getRequiredColumns() const +{ + NamesAndTypesList result; + for (const auto & node : nodes) + if (node.type == Type::INPUT) + result.push_back({node.result_name, node.result_type}); + + return result; +} + ColumnsWithTypeAndName ActionsDAG::getResultColumns() const { ColumnsWithTypeAndName result; result.reserve(index.size()); - for (const auto & node : nodes) - if (!node.renaming_parent) - result.emplace_back(node.column, node.result_type, node.result_name); + for (const auto & node : index) + result.emplace_back(node->column, node->result_type, node->result_name); return result; } @@ -1625,9 +1082,8 @@ ColumnsWithTypeAndName ActionsDAG::getResultColumns() const NamesAndTypesList ActionsDAG::getNamesAndTypesList() const { NamesAndTypesList result; - for (const auto & node : nodes) - if (!node.renaming_parent) - result.emplace_back(node.result_name, node.result_type); + for (const auto & node : index) + result.emplace_back(node->result_name, node->result_type); return result; } @@ -1636,9 +1092,8 @@ Names ActionsDAG::getNames() const { Names names; names.reserve(index.size()); - for (const auto & node : nodes) - if (!node.renaming_parent) - names.emplace_back(node.result_name); + for (const auto & node : index) + names.emplace_back(node->result_name); return names; } @@ -1655,70 +1110,243 @@ std::string ActionsDAG::dumpNames() const return out.str(); } -ExpressionActionsPtr ActionsDAG::buildExpressions(const Context & context) +void ActionsDAG::removeUnusedActions(const Names & required_names) +{ + std::unordered_set nodes_set; + std::vector required_nodes; + required_nodes.reserve(required_names.size()); + + for (const auto & name : required_names) + { + auto it = index.find(name); + if (it == index.end()) + throw Exception(ErrorCodes::UNKNOWN_IDENTIFIER, + "Unknown column: {}, there are only columns {}", name, dumpNames()); + + if (nodes_set.insert(*it).second) + required_nodes.push_back(*it); + } + + removeUnusedActions(required_nodes); +} + +void ActionsDAG::removeUnusedActions(const std::vector & required_nodes) +{ + std::unordered_set visited_nodes; + std::stack stack; + + { + Index new_index; + + for (auto * node : required_nodes) + { + new_index.insert(node); + visited_nodes.insert(node); + stack.push(node); + } + + index.swap(new_index); + } + + while (!stack.empty()) + { + auto * node = stack.top(); + stack.pop(); + + if (!node->children.empty() && node->column && isColumnConst(*node->column) && node->allow_constant_folding) + { + /// Constant folding. + node->type = ActionsDAG::Type::COLUMN; + node->children.clear(); + } + + for (auto * child : node->children) + { + if (visited_nodes.count(child) == 0) + { + stack.push(child); + visited_nodes.insert(child); + } + } + } + + nodes.remove_if([&](const Node & node) { return visited_nodes.count(&node) == 0; }); +} + +void ActionsDAG::addAliases(const NamesWithAliases & aliases, std::vector & result_nodes) +{ + std::vector required_nodes; + + for (const auto & item : aliases) + { + auto & child = getNode(item.first); + required_nodes.push_back(&child); + } + + result_nodes.reserve(aliases.size()); + + for (size_t i = 0; i < aliases.size(); ++i) + { + const auto & item = aliases[i]; + auto * child = required_nodes[i]; + + if (!item.second.empty() && item.first != item.second) + { + Node node; + node.type = Type::ALIAS; + node.result_type = child->result_type; + node.result_name = std::move(item.second); + node.column = child->column; + node.allow_constant_folding = child->allow_constant_folding; + node.children.emplace_back(child); + + auto & alias = addNode(std::move(node), true); + result_nodes.push_back(&alias); + } + else + result_nodes.push_back(child); + } +} + +void ActionsDAG::addAliases(const NamesWithAliases & aliases) +{ + std::vector result_nodes; + addAliases(aliases, result_nodes); +} + +void ActionsDAG::project(const NamesWithAliases & projection) +{ + std::vector result_nodes; + addAliases(projection, result_nodes); + removeUnusedActions(result_nodes); + projectInput(); + projected_output = true; +} + +void ActionsDAG::removeColumn(const std::string & column_name) +{ + auto & node = getNode(column_name); + index.remove(&node); +} + +bool ActionsDAG::tryRestoreColumn(const std::string & column_name) +{ + if (index.contains(column_name)) + return true; + + for (auto it = nodes.rbegin(); it != nodes.rend(); ++it) + { + auto & node = *it; + if (node.result_name == column_name) + { + index[node.result_name] = &node; + return true; + } + } + + return false; +} + +ActionsDAGPtr ActionsDAG::clone() const +{ + auto actions = cloneEmpty(); + + std::unordered_map copy_map; + + for (const auto & node : nodes) + { + auto & copy_node = actions->nodes.emplace_back(node); + copy_map[&node] = ©_node; + } + + for (auto & node : actions->nodes) + for (auto & child : node.children) + child = copy_map[child]; + + for (const auto & node : index) + actions->index.insert(copy_map[node]); + + return actions; +} + +ExpressionActionsPtr ExpressionActions::clone() const +{ + auto expressions = std::make_shared(); + + expressions->actions = actions; + expressions->num_columns = num_columns; + expressions->required_columns = required_columns; + expressions->result_positions = result_positions; + expressions->sample_block = sample_block; + expressions->project_input = project_input; + expressions->max_temporary_non_const_columns = max_temporary_non_const_columns; + + std::unordered_map copy_map; + for (const auto & node : nodes) + { + auto & copy_node = expressions->nodes.emplace_back(node); + copy_map[&node] = ©_node; + } + + for (auto & node : expressions->nodes) + for (auto & child : node.children) + child = copy_map[child]; + + for (auto & action : expressions->actions) + action.node = copy_map[action.node]; + + return expressions; +} + + +ExpressionActionsPtr ActionsDAG::linearizeActions() const { struct Data { - Node * node = nullptr; + const Node * node = nullptr; size_t num_created_children = 0; - size_t num_expected_children = 0; - std::vector parents; - Node * renamed_child = nullptr; + std::vector parents; + + ssize_t position = -1; + size_t num_created_parents = 0; + bool used_in_result = false; }; std::vector data(nodes.size()); - std::unordered_map reverse_index; + std::unordered_map reverse_index; - for (auto & node : nodes) + for (const auto & node : nodes) { size_t id = reverse_index.size(); data[id].node = &node; reverse_index[&node] = id; } - std::queue ready_nodes; - std::queue ready_array_joins; + std::queue ready_nodes; + std::queue ready_array_joins; - for (auto & node : nodes) + for (const auto * node : index) + data[reverse_index[node]].used_in_result = true; + + for (const auto & node : nodes) { - data[reverse_index[&node]].num_expected_children += node.children.size(); - for (const auto & child : node.children) data[reverse_index[child]].parents.emplace_back(&node); - - if (node.renaming_parent) - { - - auto & cur = data[reverse_index[node.renaming_parent]]; - cur.renamed_child = &node; - cur.num_expected_children += 1; - } } - for (auto & node : nodes) + for (const auto & node : nodes) { - if (node.children.empty() && data[reverse_index[&node]].renamed_child == nullptr) + if (node.children.empty()) ready_nodes.emplace(&node); } - auto update_parent = [&](Node * parent) - { - auto & cur = data[reverse_index[parent]]; - ++cur.num_created_children; - - if (cur.num_created_children == cur.num_expected_children) - { - auto & push_stack = parent->type == Type::ARRAY_JOIN ? ready_array_joins : ready_nodes; - push_stack.push(parent); - } - }; - - auto expressions = std::make_shared(NamesAndTypesList(), context); + auto expressions = std::make_shared(); + std::stack free_positions; while (!ready_nodes.empty() || !ready_array_joins.empty()) { auto & stack = ready_nodes.empty() ? ready_array_joins : ready_nodes; - Node * node = stack.front(); + const Node * node = stack.front(); stack.pop(); Names argument_names; @@ -1727,51 +1355,160 @@ ExpressionActionsPtr ActionsDAG::buildExpressions(const Context & context) auto & cur = data[reverse_index[node]]; - switch (node->type) + size_t free_position = expressions->num_columns; + if (free_positions.empty()) + ++expressions->num_columns; + else { - case Type::INPUT: - expressions->addInput({node->column, node->result_type, node->result_name}); - break; - case Type::COLUMN: - expressions->add(ExpressionAction::addColumn({node->column, node->result_type, node->result_name})); - break; - case Type::ALIAS: - expressions->add(ExpressionAction::copyColumn(argument_names.at(0), node->result_name, cur.renamed_child != nullptr)); - break; - case Type::ARRAY_JOIN: - /// Here we copy argument because arrayJoin removes source column. - /// It makes possible to remove source column before arrayJoin if it won't be needed anymore. - - /// It could have been possible to implement arrayJoin which keeps source column, - /// but in this case it will always be replicated (as many arrays), which is expensive. - expressions->add(ExpressionAction::copyColumn(argument_names.at(0), node->unique_column_name_for_array_join)); - expressions->add(ExpressionAction::arrayJoin(node->unique_column_name_for_array_join, node->result_name)); - break; - case Type::FUNCTION: - { - ExpressionAction action; - action.type = ExpressionAction::APPLY_FUNCTION; - action.result_name = node->result_name; - action.result_type = node->result_type; - action.function_builder = node->function_builder; - action.function_base = node->function_base; - action.function = node->function; - action.argument_names = std::move(argument_names); - action.added_column = node->column; - - expressions->add(action); - break; - } + free_position = free_positions.top(); + free_positions.pop(); } - for (const auto & parent : cur.parents) - update_parent(parent); + cur.position = free_position; - if (node->renaming_parent) - update_parent(node->renaming_parent); + ExpressionActions::Arguments arguments; + arguments.reserve(cur.node->children.size()); + for (auto * child : cur.node->children) + { + auto & arg = data[reverse_index[child]]; + + if (arg.position < 0) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Argument was not calculated for {}", child->result_name); + + ++arg.num_created_parents; + + ExpressionActions::Argument argument; + argument.pos = arg.position; + argument.remove = !arg.used_in_result && arg.num_created_parents == arg.parents.size(); + + if (argument.remove) + free_positions.push(argument.pos); + + arguments.emplace_back(argument); + } + + if (node->type == Type::INPUT) + { + /// Argument for input is special. It contains the position from required columns. + ExpressionActions::Argument argument; + argument.pos = expressions->required_columns.size(); + argument.remove = cur.parents.empty(); + arguments.emplace_back(argument); + + expressions->required_columns.push_back({node->result_name, node->result_type}); + } + + expressions->actions.push_back({node, arguments, free_position}); + + for (const auto & parent : cur.parents) + { + auto & parent_data = data[reverse_index[parent]]; + ++parent_data.num_created_children; + + if (parent_data.num_created_children == parent->children.size()) + { + auto & push_stack = parent->type == Type::ARRAY_JOIN ? ready_array_joins : ready_nodes; + push_stack.push(parent); + } + } + } + + expressions->result_positions.reserve(index.size()); + + for (const auto & node : index) + { + auto pos = data[reverse_index[node]].position; + + if (pos < 0) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Action for {} was not calculated", node->result_name); + + expressions->result_positions.push_back(pos); + + ColumnWithTypeAndName col{node->column, node->result_type, node->result_name}; + expressions->sample_block.insert(std::move(col)); } return expressions; } +ExpressionActionsPtr ActionsDAG::buildExpressions() +{ + auto cloned = clone(); + auto expressions = cloned->linearizeActions(); + + expressions->nodes.swap(cloned->nodes); + + if (max_temporary_columns && expressions->num_columns > max_temporary_columns) + throw Exception(ErrorCodes::TOO_MANY_TEMPORARY_COLUMNS, + "Too many temporary columns: {}. Maximum: {}", + dumpNames(), std::to_string(max_temporary_columns)); + + expressions->max_temporary_non_const_columns = max_temporary_non_const_columns; + expressions->project_input = project_input; + + return expressions; +} + +std::string ActionsDAG::dump() const +{ + return linearizeActions()->dumpActions(); +} + +std::string ActionsDAG::dumpDAG() const +{ + std::unordered_map map; + for (const auto & node : nodes) + { + size_t idx = map.size(); + map[&node] = idx; + } + + std::stringstream out; + for (const auto & node : nodes) + { + out << map[&node] << " : "; + switch (node.type) + { + case ActionsDAG::Type::COLUMN: + out << "COLUMN "; + break; + + case ActionsDAG::Type::ALIAS: + out << "ALIAS "; + break; + + case ActionsDAG::Type::FUNCTION: + out << "FUNCTION "; + break; + + case ActionsDAG::Type::ARRAY_JOIN: + out << "ARRAY JOIN "; + break; + + case ActionsDAG::Type::INPUT: + out << "INPUT "; + break; + } + + out << "("; + for (size_t i = 0; i < node.children.size(); ++i) + { + if (i) + out << ", "; + out << map[node.children[i]]; + } + out << ")"; + + out << " " << (node.column ? node.column->getName() : "(no column)"); + out << " " << (node.result_type ? node.result_type->getName() : "(no type)"); + out << " " << (!node.result_name.empty() ? node.result_name : "(no name)"); + if (node.function_base) + out << " [" << node.function_base->getName() << "]"; + + out << "\n"; + } + + return out.str(); +} + } diff --git a/src/Interpreters/ExpressionActions.h b/src/Interpreters/ExpressionActions.h index 75edd4bf7ee..c7011dd7d86 100644 --- a/src/Interpreters/ExpressionActions.h +++ b/src/Interpreters/ExpressionActions.h @@ -4,6 +4,7 @@ #include #include #include +#include #include #include #include @@ -11,6 +12,11 @@ #include #include +#include +#include +#include +#include + #include #if !defined(ARCADIA_BUILD) @@ -49,97 +55,12 @@ class CompiledExpressionCache; class ArrayJoinAction; using ArrayJoinActionPtr = std::shared_ptr; -/** Action on the block. - */ -struct ExpressionAction -{ -private: - using ExpressionActionsPtr = std::shared_ptr; -public: - enum Type - { - ADD_COLUMN, - REMOVE_COLUMN, - COPY_COLUMN, - - APPLY_FUNCTION, - - /// Replaces the source column with array into column with elements. - /// Duplicates the values in the remaining columns by the number of elements in the arrays. - /// Source column is removed from block. - ARRAY_JOIN, - - /// Reorder and rename the columns, delete the extra ones. The same column names are allowed in the result. - PROJECT, - /// Add columns with alias names. This columns are the same as non-aliased. PROJECT columns if you need to modify them. - ADD_ALIASES, - }; - - Type type{}; - - /// For ADD/REMOVE/ARRAY_JOIN/COPY_COLUMN. - std::string source_name; - std::string result_name; - DataTypePtr result_type; - - /// If COPY_COLUMN can replace the result column. - bool can_replace = false; - - /// For ADD_COLUMN. - ColumnPtr added_column; - - /// For APPLY_FUNCTION. - /// OverloadResolver is used before action was added to ExpressionActions (when we don't know types of arguments). - FunctionOverloadResolverPtr function_builder; - - /// Can be used after action was added to ExpressionActions if we want to get function signature or properties like monotonicity. - FunctionBasePtr function_base; - /// Prepared function which is used in function execution. - ExecutableFunctionPtr function; - Names argument_names; - bool is_function_compiled = false; - - /// For JOIN - std::shared_ptr table_join; - JoinPtr join; - - /// For PROJECT. - NamesWithAliases projection; - - /// If result_name_ == "", as name "function_name(arguments separated by commas) is used". - static ExpressionAction applyFunction( - const FunctionOverloadResolverPtr & function_, const std::vector & argument_names_, std::string result_name_ = ""); - - static ExpressionAction addColumn(const ColumnWithTypeAndName & added_column_); - static ExpressionAction removeColumn(const std::string & removed_name); - static ExpressionAction copyColumn(const std::string & from_name, const std::string & to_name, bool can_replace = false); - static ExpressionAction project(const NamesWithAliases & projected_columns_); - static ExpressionAction project(const Names & projected_columns_); - static ExpressionAction addAliases(const NamesWithAliases & aliased_columns_); - static ExpressionAction arrayJoin(std::string source_name, std::string result_name); - - /// Which columns necessary to perform this action. - Names getNeededColumns() const; - - std::string toString() const; - - bool operator==(const ExpressionAction & other) const; - - struct ActionHash - { - UInt128 operator()(const ExpressionAction & action) const; - }; - -private: - friend class ExpressionActions; - - void prepare(Block & sample_block, const Settings & settings, NameSet & names_not_for_constant_folding); - void execute(Block & block, bool dry_run) const; -}; - class ExpressionActions; using ExpressionActionsPtr = std::shared_ptr; +class ActionsDAG; +using ActionsDAGPtr = std::shared_ptr; + class ActionsDAG { public: @@ -160,16 +81,12 @@ public: struct Node { std::vector children; - /// This field is filled if current node is replaced by existing node with the same name. - Node * renaming_parent = nullptr; Type type; std::string result_name; DataTypePtr result_type; - std::string unique_column_name_for_array_join; - FunctionOverloadResolverPtr function_builder; /// Can be used after action was added to ExpressionActions if we want to get function signature or properties like monotonicity. FunctionBasePtr function_base; @@ -185,12 +102,75 @@ public: bool allow_constant_folding = true; }; - using Index = std::unordered_map; + class Index + { + public: + Node *& operator[](std::string_view key) + { + auto res = map.emplace(key, list.end()); + if (res.second) + res.first->second = list.emplace(list.end(), nullptr); + + return *res.first->second; + } + + void swap(Index & other) + { + list.swap(other.list); + map.swap(other.map); + } + + auto size() const { return list.size(); } + bool contains(std::string_view key) const { return map.count(key) != 0; } + + std::list::iterator begin() { return list.begin(); } + std::list::iterator end() { return list.end(); } + std::list::const_iterator begin() const { return list.begin(); } + std::list::const_iterator end() const { return list.end(); } + std::list::const_iterator find(std::string_view key) const + { + auto it = map.find(key); + if (it == map.end()) + return list.end(); + + return it->second; + } + + /// Insert method doesn't check if map already have node with the same name. + /// If node with the same name exists, it is removed from map, but not list. + /// It is expected and used for project(), when result may have several columns with the same name. + void insert(Node * node) { map[node->result_name] = list.emplace(list.end(), node); } + void remove(Node * node) + { + auto it = map.find(node->result_name); + if (it != map.end()) + return; + + list.erase(it->second); + map.erase(it); + } + + private: + std::list list; + std::unordered_map::iterator> map; + }; + + using Nodes = std::list; private: - std::list nodes; + Nodes nodes; Index index; + size_t max_temporary_columns = 0; + size_t max_temporary_non_const_columns = 0; + +#if USE_EMBEDDED_COMPILER + std::shared_ptr compilation_cache; +#endif + + bool project_input = false; + bool projected_output = false; + public: ActionsDAG() = default; ActionsDAG(const ActionsDAG &) = delete; @@ -198,58 +178,108 @@ public: explicit ActionsDAG(const NamesAndTypesList & inputs); explicit ActionsDAG(const ColumnsWithTypeAndName & inputs); + const Nodes & getNodes() const { return nodes; } const Index & getIndex() const { return index; } + NamesAndTypesList getRequiredColumns() const; ColumnsWithTypeAndName getResultColumns() const; NamesAndTypesList getNamesAndTypesList() const; + Names getNames() const; std::string dumpNames() const; + std::string dump() const; + std::string dumpDAG() const; const Node & addInput(std::string name, DataTypePtr type); const Node & addInput(ColumnWithTypeAndName column); const Node & addColumn(ColumnWithTypeAndName column); const Node & addAlias(const std::string & name, std::string alias, bool can_replace = false); - const Node & addArrayJoin(const std::string & source_name, std::string result_name, std::string unique_column_name); + const Node & addArrayJoin(const std::string & source_name, std::string result_name); const Node & addFunction( const FunctionOverloadResolverPtr & function, const Names & argument_names, std::string result_name, - bool compile_expressions); + const Context & context); - ExpressionActionsPtr buildExpressions(const Context & context); + /// Call addAlias several times. + void addAliases(const NamesWithAliases & aliases); + /// Adds alias actions and removes unused columns from index. + void project(const NamesWithAliases & projection); + + /// Removes column from index. + void removeColumn(const std::string & column_name); + /// If column is not in index, try to find it in nodes and insert back into index. + bool tryRestoreColumn(const std::string & column_name); + + void projectInput() { project_input = true; } + void removeUnusedActions(const Names & required_names); + ExpressionActionsPtr buildExpressions(); + + /// Splits actions into two parts. Returned half may be swapped with ARRAY JOIN. + /// Returns nullptr if no actions may be moved before ARRAY JOIN. + ActionsDAGPtr splitActionsBeforeArrayJoin(const NameSet & array_joined_columns); + + bool hasArrayJoin() const; + bool empty() const; + bool projectedOutput() const { return projected_output; } + + ActionsDAGPtr clone() const; private: Node & addNode(Node node, bool can_replace = false); Node & getNode(const std::string & name); + + ActionsDAGPtr cloneEmpty() const + { + auto actions = std::make_shared(); + actions->max_temporary_columns = max_temporary_columns; + actions->max_temporary_non_const_columns = max_temporary_non_const_columns; + +#if USE_EMBEDDED_COMPILER + actions->compilation_cache = compilation_cache; +#endif + return actions; + } + + ExpressionActionsPtr linearizeActions() const; + void removeUnusedActions(const std::vector & required_nodes); + void addAliases(const NamesWithAliases & aliases, std::vector & result_nodes); }; -using ActionsDAGPtr = std::shared_ptr; /** Contains a sequence of actions on the block. */ class ExpressionActions { -private: +public: using Node = ActionsDAG::Node; using Index = ActionsDAG::Index; + struct Argument + { + size_t pos; + bool remove; + }; + + using Arguments = std::vector; + struct Action { - Node * node; - ColumnNumbers arguments; - /// Columns which will be removed after actions is executed. - /// It is always a subset of arguments. - ColumnNumbers to_remove; + const Node * node; + Arguments arguments; size_t result_position; - bool is_used_in_result; + + std::string toString() const; }; using Actions = std::vector; +private: struct ExecutionContext { - ColumnsWithTypeAndName & input_columns; + ColumnsWithTypeAndName & inputs; ColumnsWithTypeAndName columns; + std::vector inputs_pos; size_t num_rows; }; @@ -258,19 +288,26 @@ private: size_t num_columns; NamesAndTypesList required_columns; + ColumnNumbers result_positions; Block sample_block; + /// This flag means that all columns except input will be removed from block before execution. + bool project_input = false; + + size_t max_temporary_non_const_columns = 0; + + friend class ActionsDAG; + public: ~ExpressionActions(); + ExpressionActions() = default; + ExpressionActions(const ExpressionActions &) = delete; + ExpressionActions & operator=(const ExpressionActions &) = delete; - ExpressionActions(const ExpressionActions & other) = default; + const Actions & getActions() const { return actions; } /// Adds to the beginning the removal of all extra columns. - void prependProjectInput(); - - /// Splits actions into two parts. Returned half may be swapped with ARRAY JOIN. - /// Returns nullptr if no actions may be moved before ARRAY JOIN. - ExpressionActionsPtr splitActionsBeforeArrayJoin(const NameSet & array_joined_columns); + void projectInput() { project_input = true; } /// - Adds actions to delete all but the specified columns. /// - Removes unused input columns. @@ -286,6 +323,7 @@ public: /// Execute the expression on the block. The block must contain all the columns returned by getRequiredColumns. void execute(Block & block, bool dry_run = false) const; + void execute(Block & block, size_t & num_rows, bool dry_run = false) const; bool hasArrayJoin() const; @@ -296,18 +334,13 @@ public: static std::string getSmallestColumn(const NamesAndTypesList & columns); - const Settings & getSettings() const { return settings; } - /// Check if column is always zero. True if it's definite, false if we can't say for sure. /// Call it only after subqueries for sets were executed. bool checkColumnIsAlwaysFalse(const String & column_name) const; -private: + ExpressionActionsPtr clone() const; - Settings settings; -#if USE_EMBEDDED_COMPILER - std::shared_ptr compilation_cache; -#endif +private: void checkLimits(ExecutionContext & execution_context) const; @@ -343,8 +376,8 @@ struct ExpressionActionsChain /// If not empty, has the same size with required_output; is filled in finalize(). std::vector can_remove_required_output; - virtual const NamesAndTypesList & getRequiredColumns() const = 0; - virtual const ColumnsWithTypeAndName & getResultColumns() const = 0; + virtual NamesAndTypesList getRequiredColumns() const = 0; + virtual ColumnsWithTypeAndName getResultColumns() const = 0; /// Remove unused result and update required columns virtual void finalize(const Names & required_output_) = 0; /// Add projections to expression @@ -354,43 +387,42 @@ struct ExpressionActionsChain /// Only for ExpressionActionsStep ActionsDAGPtr & actions(); const ActionsDAGPtr & actions() const; - ExpressionActionsPtr getExpression() const; }; struct ExpressionActionsStep : public Step { - ActionsDAGPtr actions_dag; - ExpressionActionsPtr actions; + ActionsDAGPtr actions; explicit ExpressionActionsStep(ActionsDAGPtr actions_, Names required_output_ = Names()) : Step(std::move(required_output_)) - , actions_dag(std::move(actions_)) + , actions(std::move(actions_)) { } - const NamesAndTypesList & getRequiredColumns() const override + NamesAndTypesList getRequiredColumns() const override { - return actions->getRequiredColumnsWithTypes(); + return actions->getRequiredColumns(); } - const ColumnsWithTypeAndName & getResultColumns() const override + ColumnsWithTypeAndName getResultColumns() const override { - return actions->getSampleBlock().getColumnsWithTypeAndName(); + return actions->getResultColumns(); } void finalize(const Names & required_output_) override { - actions->finalize(required_output_); + if (!actions->projectedOutput()) + actions->removeUnusedActions(required_output_); } void prependProjectInput() const override { - actions->prependProjectInput(); + actions->projectInput(); } std::string dump() const override { - return actions->dumpActions(); + return actions->dump(); } }; @@ -402,8 +434,8 @@ struct ExpressionActionsChain ArrayJoinStep(ArrayJoinActionPtr array_join_, ColumnsWithTypeAndName required_columns_); - const NamesAndTypesList & getRequiredColumns() const override { return required_columns; } - const ColumnsWithTypeAndName & getResultColumns() const override { return result_columns; } + NamesAndTypesList getRequiredColumns() const override { return required_columns; } + ColumnsWithTypeAndName getResultColumns() const override { return result_columns; } void finalize(const Names & required_output_) override; void prependProjectInput() const override {} /// TODO: remove unused columns before ARRAY JOIN ? std::string dump() const override { return "ARRAY JOIN"; } @@ -418,8 +450,8 @@ struct ExpressionActionsChain ColumnsWithTypeAndName result_columns; JoinStep(std::shared_ptr analyzed_join_, JoinPtr join_, ColumnsWithTypeAndName required_columns_); - const NamesAndTypesList & getRequiredColumns() const override { return required_columns; } - const ColumnsWithTypeAndName & getResultColumns() const override { return result_columns; } + NamesAndTypesList getRequiredColumns() const override { return required_columns; } + ColumnsWithTypeAndName getResultColumns() const override { return result_columns; } void finalize(const Names & required_output_) override; void prependProjectInput() const override {} /// TODO: remove unused columns before JOIN ? std::string dump() const override { return "JOIN"; } @@ -431,7 +463,7 @@ struct ExpressionActionsChain const Context & context; Steps steps; - void addStep(); + void addStep(NameSet non_constant_inputs = {}); void finalize(); @@ -440,7 +472,7 @@ struct ExpressionActionsChain steps.clear(); } - ExpressionActionsPtr getLastActions(bool allow_empty = false) + ActionsDAGPtr getLastActions(bool allow_empty = false) { if (steps.empty()) { @@ -449,9 +481,7 @@ struct ExpressionActionsChain throw Exception("Empty ExpressionActionsChain", ErrorCodes::LOGICAL_ERROR); } - auto * step = typeid_cast(steps.back().get()); - step->actions = step->actions_dag->buildExpressions(context); - return step->actions; + return typeid_cast(steps.back().get())->actions; } Step & getLastStep() diff --git a/src/Interpreters/ExpressionAnalyzer.cpp b/src/Interpreters/ExpressionAnalyzer.cpp index f79bb36ec46..49eaa21cc3d 100644 --- a/src/Interpreters/ExpressionAnalyzer.cpp +++ b/src/Interpreters/ExpressionAnalyzer.cpp @@ -70,16 +70,16 @@ namespace /// Check if there is an ignore function. It's used for disabling constant folding in query /// predicates because some performance tests use ignore function as a non-optimize guard. -bool allowEarlyConstantFolding(const ExpressionActions & actions, const Settings & settings) +bool allowEarlyConstantFolding(const ActionsDAG & actions, const Settings & settings) { if (!settings.enable_early_constant_folding) return false; - for (const auto & action : actions.getActions()) + for (const auto & node : actions.getNodes()) { - if (action.type == action.APPLY_FUNCTION && action.function_base) + if (node.type == ActionsDAG::Type::FUNCTION && node.function_base) { - auto name = action.function_base->getName(); + auto name = node.function_base->getName(); if (name == "ignore") return false; } @@ -234,7 +234,7 @@ void ExpressionAnalyzer::analyzeAggregation() if (it == index.end()) throw Exception("Unknown identifier (in GROUP BY): " + column_name, ErrorCodes::UNKNOWN_IDENTIFIER); - const auto & node = it->second; + const auto & node = *it; /// Constant expressions have non-null column pointer at this stage. if (node->column && isColumnConst(*node->column)) @@ -382,7 +382,7 @@ void SelectQueryExpressionAnalyzer::makeSetsForIndex(const ASTPtr & node) auto temp_actions = std::make_shared(columns_after_join); getRootActions(left_in_operand, true, temp_actions); - if (temp_actions->getIndex().count(left_in_operand->getColumnName()) != 0) + if (temp_actions->getIndex().contains(left_in_operand->getColumnName())) makeExplicitSet(func, *temp_actions, true, context, settings.size_limits_for_set, prepared_sets); } @@ -434,7 +434,7 @@ bool ExpressionAnalyzer::makeAggregateDescriptions(ActionsDAGPtr & actions) if (it == index.end()) throw Exception(ErrorCodes::UNKNOWN_IDENTIFIER, "Unknown identifier (in aggregate function '{}'): {}", node->name, name); - types[i] = it->second->result_type; + types[i] = (*it)->result_type; aggregate.argument_names[i] = name; } @@ -481,7 +481,7 @@ ArrayJoinActionPtr ExpressionAnalyzer::addMultipleArrayJoinAction(ActionsDAGPtr return std::make_shared(result_columns, array_join_is_left, context); } -ArrayJoinActionPtr SelectQueryExpressionAnalyzer::appendArrayJoin(ExpressionActionsChain & chain, ExpressionActionsPtr & before_array_join, bool only_types) +ArrayJoinActionPtr SelectQueryExpressionAnalyzer::appendArrayJoin(ExpressionActionsChain & chain, ActionsDAGPtr & before_array_join, bool only_types) { const auto * select_query = getSelectQuery(); @@ -637,11 +637,11 @@ JoinPtr SelectQueryExpressionAnalyzer::makeTableJoin(const ASTTablesInSelectQuer return subquery_for_join.join; } -ExpressionActionsPtr SelectQueryExpressionAnalyzer::appendPrewhere( +ActionsDAGPtr SelectQueryExpressionAnalyzer::appendPrewhere( ExpressionActionsChain & chain, bool only_types, const Names & additional_required_columns) { const auto * select_query = getSelectQuery(); - ExpressionActionsPtr prewhere_actions; + ActionsDAGPtr prewhere_actions; if (!select_query->prewhere()) return prewhere_actions; @@ -652,7 +652,7 @@ ExpressionActionsPtr SelectQueryExpressionAnalyzer::appendPrewhere( step.required_output.push_back(prewhere_column_name); step.can_remove_required_output.push_back(true); - auto filter_type = step.actions()->getIndex().find(prewhere_column_name)->second->result_type; + auto filter_type = (*step.actions()->getIndex().find(prewhere_column_name))->result_type; if (!filter_type->canBeUsedInBooleanContext()) throw Exception("Invalid type for filter in PREWHERE: " + filter_type->getName(), ErrorCodes::ILLEGAL_TYPE_OF_COLUMN_FOR_FILTER); @@ -661,8 +661,8 @@ ExpressionActionsPtr SelectQueryExpressionAnalyzer::appendPrewhere( /// Remove unused source_columns from prewhere actions. auto tmp_actions_dag = std::make_shared(sourceColumns()); getRootActions(select_query->prewhere(), only_types, tmp_actions_dag); - auto tmp_actions = tmp_actions_dag->buildExpressions(context); - tmp_actions->finalize({prewhere_column_name}); + tmp_actions_dag->removeUnusedActions({prewhere_column_name}); + auto tmp_actions = tmp_actions_dag->buildExpressions(); auto required_columns = tmp_actions->getRequiredColumns(); NameSet required_source_columns(required_columns.begin(), required_columns.end()); @@ -686,7 +686,7 @@ ExpressionActionsPtr SelectQueryExpressionAnalyzer::appendPrewhere( Names required_output(name_set.begin(), name_set.end()); prewhere_actions = chain.getLastActions(); - prewhere_actions->finalize(required_output); + prewhere_actions->removeUnusedActions(required_output); } { @@ -697,11 +697,14 @@ ExpressionActionsPtr SelectQueryExpressionAnalyzer::appendPrewhere( /// 2. Store side columns which were calculated during prewhere actions execution if they are used. /// Example: select F(A) prewhere F(A) > 0. F(A) can be saved from prewhere step. /// 3. Check if we can remove filter column at prewhere step. If we can, action will store single REMOVE_COLUMN. - ColumnsWithTypeAndName columns = prewhere_actions->getSampleBlock().getColumnsWithTypeAndName(); + ColumnsWithTypeAndName columns = prewhere_actions->getResultColumns(); auto required_columns = prewhere_actions->getRequiredColumns(); - NameSet prewhere_input_names(required_columns.begin(), required_columns.end()); + NameSet prewhere_input_names; NameSet unused_source_columns; + for (const auto & col : required_columns) + prewhere_input_names.insert(col.name); + for (const auto & column : sourceColumns()) { if (prewhere_input_names.count(column.name) == 0) @@ -721,7 +724,7 @@ ExpressionActionsPtr SelectQueryExpressionAnalyzer::appendPrewhere( return prewhere_actions; } -void SelectQueryExpressionAnalyzer::appendPreliminaryFilter(ExpressionActionsChain & chain, ExpressionActionsPtr actions, String column_name) +void SelectQueryExpressionAnalyzer::appendPreliminaryFilter(ExpressionActionsChain & chain, ActionsDAGPtr actions, String column_name) { ExpressionActionsChain::Step & step = chain.lastStep(sourceColumns()); @@ -749,7 +752,7 @@ bool SelectQueryExpressionAnalyzer::appendWhere(ExpressionActionsChain & chain, getRootActions(select_query->where(), only_types, step.actions()); - auto filter_type = step.actions()->getIndex().find(where_column_name)->second->result_type; + auto filter_type = (*step.actions()->getIndex().find(where_column_name))->result_type; if (!filter_type->canBeUsedInBooleanContext()) throw Exception("Invalid type for filter in WHERE: " + filter_type->getName(), ErrorCodes::ILLEGAL_TYPE_OF_COLUMN_FOR_FILTER); @@ -780,7 +783,7 @@ bool SelectQueryExpressionAnalyzer::appendGroupBy(ExpressionActionsChain & chain { auto actions_dag = std::make_shared(columns_after_join); getRootActions(child, only_types, actions_dag); - group_by_elements_actions.emplace_back(actions_dag->buildExpressions(context)); + group_by_elements_actions.emplace_back(actions_dag->buildExpressions()); } } @@ -842,18 +845,24 @@ void SelectQueryExpressionAnalyzer::appendSelect(ExpressionActionsChain & chain, step.required_output.push_back(child->getColumnName()); } -bool SelectQueryExpressionAnalyzer::appendOrderBy(ExpressionActionsChain & chain, bool only_types, bool optimize_read_in_order, +ActionsDAGPtr SelectQueryExpressionAnalyzer::appendOrderBy(ExpressionActionsChain & chain, bool only_types, bool optimize_read_in_order, ManyExpressionActions & order_by_elements_actions) { const auto * select_query = getSelectQuery(); if (!select_query->orderBy()) - return false; + { + auto actions = chain.getLastActions(); + chain.addStep(); + return actions; + } ExpressionActionsChain::Step & step = chain.lastStep(aggregated_columns); getRootActions(select_query->orderBy(), only_types, step.actions()); + bool with_fill = false; + NameSet order_by_keys; for (auto & child : select_query->orderBy()->children) { const auto * ast = child->as(); @@ -861,6 +870,9 @@ bool SelectQueryExpressionAnalyzer::appendOrderBy(ExpressionActionsChain & chain throw Exception("Bad order expression AST", ErrorCodes::UNKNOWN_TYPE_OF_AST_NODE); ASTPtr order_expression = ast->children.at(0); step.required_output.push_back(order_expression->getColumnName()); + + if (ast->with_fill) + with_fill = true; } if (optimize_read_in_order) @@ -869,10 +881,21 @@ bool SelectQueryExpressionAnalyzer::appendOrderBy(ExpressionActionsChain & chain { auto actions_dag = std::make_shared(columns_after_join); getRootActions(child, only_types, actions_dag); - order_by_elements_actions.emplace_back(actions_dag->buildExpressions(context)); + order_by_elements_actions.emplace_back(actions_dag->buildExpressions()); } } - return true; + + NameSet non_constant_inputs; + if (with_fill) + { + for (const auto & column : step.getResultColumns()) + if (!order_by_keys.count(column.name)) + non_constant_inputs.insert(column.name); + } + + auto actions = chain.getLastActions(); + chain.addStep(non_constant_inputs); + return actions; } bool SelectQueryExpressionAnalyzer::appendLimitBy(ExpressionActionsChain & chain, bool only_types) @@ -903,7 +926,7 @@ bool SelectQueryExpressionAnalyzer::appendLimitBy(ExpressionActionsChain & chain return true; } -ExpressionActionsPtr SelectQueryExpressionAnalyzer::appendProjectResult(ExpressionActionsChain & chain) const +ActionsDAGPtr SelectQueryExpressionAnalyzer::appendProjectResult(ExpressionActionsChain & chain) const { const auto * select_query = getSelectQuery(); @@ -950,7 +973,7 @@ ExpressionActionsPtr SelectQueryExpressionAnalyzer::appendProjectResult(Expressi } auto actions = chain.getLastActions(); - actions->add(ExpressionAction::project(result_columns)); + actions->project(result_columns); return actions; } @@ -963,7 +986,7 @@ void ExpressionAnalyzer::appendExpression(ExpressionActionsChain & chain, const } -ExpressionActionsPtr ExpressionAnalyzer::getActions(bool add_aliases, bool project_result) +ActionsDAGPtr ExpressionAnalyzer::getActionsDAG(bool add_aliases, bool project_result) { auto actions_dag = std::make_shared(aggregated_columns); NamesWithAliases result_columns; @@ -989,14 +1012,12 @@ ExpressionActionsPtr ExpressionAnalyzer::getActions(bool add_aliases, bool proje getRootActions(ast, false, actions_dag); } - auto actions = actions_dag->buildExpressions(context); - if (add_aliases) { if (project_result) - actions->add(ExpressionAction::project(result_columns)); + actions_dag->project(result_columns); else - actions->add(ExpressionAction::addAliases(result_columns)); + actions_dag->addAliases(result_columns); } if (!(add_aliases && project_result)) @@ -1006,9 +1027,13 @@ ExpressionActionsPtr ExpressionAnalyzer::getActions(bool add_aliases, bool proje result_names.push_back(column_name_type.name); } - actions->finalize(result_names); + actions_dag->removeUnusedActions(result_names); + return actions_dag; +} - return actions; +ExpressionActionsPtr ExpressionAnalyzer::getActions(bool add_aliases, bool project_result) +{ + return getActionsDAG(add_aliases, project_result)->buildExpressions(); } @@ -1017,10 +1042,10 @@ ExpressionActionsPtr ExpressionAnalyzer::getConstActions() auto actions = std::make_shared(NamesAndTypesList()); getRootActions(query, true, actions, true); - return actions->buildExpressions(context); + return actions->buildExpressions(); } -ExpressionActionsPtr SelectQueryExpressionAnalyzer::simpleSelectActions() +ActionsDAGPtr SelectQueryExpressionAnalyzer::simpleSelectActions() { ExpressionActionsChain new_chain(context); appendSelect(new_chain, false); @@ -1061,7 +1086,7 @@ ExpressionAnalysisResult::ExpressionAnalysisResult( if (!finalized) { - finalize(chain, context, where_step_num); + finalize(chain, where_step_num); finalized = true; } @@ -1107,7 +1132,7 @@ ExpressionAnalysisResult::ExpressionAnalysisResult( Block before_prewhere_sample = source_header; if (sanitizeBlock(before_prewhere_sample)) { - prewhere_info->prewhere_actions->execute(before_prewhere_sample); + prewhere_info->prewhere_actions->buildExpressions()->execute(before_prewhere_sample); auto & column_elem = before_prewhere_sample.getByName(query.prewhere()->getColumnName()); /// If the filter column is a constant, record it. if (column_elem.column) @@ -1140,7 +1165,7 @@ ExpressionAnalysisResult::ExpressionAnalysisResult( before_where_sample = source_header; if (sanitizeBlock(before_where_sample)) { - before_where->execute(before_where_sample); + before_where->buildExpressions()->execute(before_where_sample); auto & column_elem = before_where_sample.getByName(query.where()->getColumnName()); /// If the filter column is a constant, record it. if (column_elem.column) @@ -1188,10 +1213,12 @@ ExpressionAnalysisResult::ExpressionAnalysisResult( /// If there is aggregation, we execute expressions in SELECT and ORDER BY on the initiating server, otherwise on the source servers. query_analyzer.appendSelect(chain, only_types || (need_aggregate ? !second_stage : !first_stage)); selected_columns = chain.getLastStep().required_output; - has_order_by = query_analyzer.appendOrderBy(chain, only_types || (need_aggregate ? !second_stage : !first_stage), - optimize_read_in_order, order_by_elements_actions); - before_order_and_select = chain.getLastActions(); - chain.addStep(); + has_order_by = query.orderBy() != nullptr; + before_order_and_select = query_analyzer.appendOrderBy( + chain, + only_types || (need_aggregate ? !second_stage : !first_stage), + optimize_read_in_order, + order_by_elements_actions); if (query_analyzer.appendLimitBy(chain, only_types || !second_stage)) { @@ -1210,28 +1237,35 @@ ExpressionAnalysisResult::ExpressionAnalysisResult( checkActions(); } -void ExpressionAnalysisResult::finalize(const ExpressionActionsChain & chain, const Context & context_, size_t where_step_num) +void ExpressionAnalysisResult::finalize(const ExpressionActionsChain & chain, size_t where_step_num) { if (hasPrewhere()) { const ExpressionActionsChain::Step & step = *chain.steps.at(0); prewhere_info->remove_prewhere_column = step.can_remove_required_output.at(0); - Names columns_to_remove; + NameSet columns_to_remove; for (size_t i = 1; i < step.required_output.size(); ++i) { if (step.can_remove_required_output[i]) - columns_to_remove.push_back(step.required_output[i]); + columns_to_remove.insert(step.required_output[i]); } if (!columns_to_remove.empty()) { - auto columns = prewhere_info->prewhere_actions->getSampleBlock().getNamesAndTypesList(); - ExpressionActionsPtr actions = std::make_shared(columns, context_); - for (const auto & column : columns_to_remove) - actions->add(ExpressionAction::removeColumn(column)); + auto columns = prewhere_info->prewhere_actions->getResultColumns(); - prewhere_info->remove_columns_actions = std::move(actions); + auto remove_actions = std::make_shared(); + for (const auto & column : columns) + { + if (columns_to_remove.count(column.name)) + { + remove_actions->addInput(column); + remove_actions->removeColumn(column.name); + } + } + + prewhere_info->remove_columns_actions = std::move(remove_actions); } columns_to_remove_after_prewhere = std::move(columns_to_remove); @@ -1248,11 +1282,11 @@ void ExpressionAnalysisResult::finalize(const ExpressionActionsChain & chain, co void ExpressionAnalysisResult::removeExtraColumns() const { if (hasFilter()) - filter_info->actions->prependProjectInput(); + filter_info->actions->projectInput(); if (hasWhere()) - before_where->prependProjectInput(); + before_where->projectInput(); if (hasHaving()) - before_having->prependProjectInput(); + before_having->projectInput(); } void ExpressionAnalysisResult::checkActions() const @@ -1260,11 +1294,11 @@ void ExpressionAnalysisResult::checkActions() const /// Check that PREWHERE doesn't contain unusual actions. Unusual actions are that can change number of rows. if (hasPrewhere()) { - auto check_actions = [](const ExpressionActionsPtr & actions) + auto check_actions = [](const ActionsDAGPtr & actions) { if (actions) - for (const auto & action : actions->getActions()) - if (action.type == ExpressionAction::Type::ARRAY_JOIN) + for (const auto & node : actions->getNodes()) + if (node.type == ActionsDAG::Type::ARRAY_JOIN) throw Exception("PREWHERE cannot contain ARRAY JOIN action", ErrorCodes::ILLEGAL_PREWHERE); }; diff --git a/src/Interpreters/ExpressionAnalyzer.h b/src/Interpreters/ExpressionAnalyzer.h index 6389d8a142c..95a65ee2bfa 100644 --- a/src/Interpreters/ExpressionAnalyzer.h +++ b/src/Interpreters/ExpressionAnalyzer.h @@ -102,6 +102,7 @@ public: /// If add_aliases, only the calculated values in the desired order and add aliases. /// If also project_result, than only aliases remain in the output block. /// Otherwise, only temporary columns will be deleted from the block. + ActionsDAGPtr getActionsDAG(bool add_aliases, bool project_result = true); ExpressionActionsPtr getActions(bool add_aliases, bool project_result = true); /// Actions that can be performed on an empty block: adding constants and applying functions that depend only on constants. @@ -182,22 +183,22 @@ struct ExpressionAnalysisResult bool optimize_aggregation_in_order = false; bool join_has_delayed_stream = false; - ExpressionActionsPtr before_array_join; + ActionsDAGPtr before_array_join; ArrayJoinActionPtr array_join; - ExpressionActionsPtr before_join; + ActionsDAGPtr before_join; JoinPtr join; - ExpressionActionsPtr before_where; - ExpressionActionsPtr before_aggregation; - ExpressionActionsPtr before_having; - ExpressionActionsPtr before_order_and_select; - ExpressionActionsPtr before_limit_by; - ExpressionActionsPtr final_projection; + ActionsDAGPtr before_where; + ActionsDAGPtr before_aggregation; + ActionsDAGPtr before_having; + ActionsDAGPtr before_order_and_select; + ActionsDAGPtr before_limit_by; + ActionsDAGPtr final_projection; /// Columns from the SELECT list, before renaming them to aliases. Names selected_columns; /// Columns will be removed after prewhere actions execution. - Names columns_to_remove_after_prewhere; + NameSet columns_to_remove_after_prewhere; PrewhereInfoPtr prewhere_info; FilterInfoPtr filter_info; @@ -229,7 +230,7 @@ struct ExpressionAnalysisResult void removeExtraColumns() const; void checkActions() const; - void finalize(const ExpressionActionsChain & chain, const Context & context, size_t where_step_num); + void finalize(const ExpressionActionsChain & chain, size_t where_step_num); }; /// SelectQuery specific ExpressionAnalyzer part. @@ -267,12 +268,12 @@ public: /// Tables that will need to be sent to remote servers for distributed query processing. const TemporaryTablesMapping & getExternalTables() const { return external_tables; } - ExpressionActionsPtr simpleSelectActions(); + ActionsDAGPtr simpleSelectActions(); /// These appends are public only for tests void appendSelect(ExpressionActionsChain & chain, bool only_types); /// Deletes all columns except mentioned by SELECT, arranges the remaining columns and renames them to aliases. - ExpressionActionsPtr appendProjectResult(ExpressionActionsChain & chain) const; + ActionsDAGPtr appendProjectResult(ExpressionActionsChain & chain) const; private: StorageMetadataPtr metadata_snapshot; @@ -315,14 +316,14 @@ private: */ /// Before aggregation: - ArrayJoinActionPtr appendArrayJoin(ExpressionActionsChain & chain, ExpressionActionsPtr & before_array_join, bool only_types); + ArrayJoinActionPtr appendArrayJoin(ExpressionActionsChain & chain, ActionsDAGPtr & before_array_join, bool only_types); bool appendJoinLeftKeys(ExpressionActionsChain & chain, bool only_types); JoinPtr appendJoin(ExpressionActionsChain & chain); /// Add preliminary rows filtration. Actions are created in other expression analyzer to prevent any possible alias injection. - void appendPreliminaryFilter(ExpressionActionsChain & chain, ExpressionActionsPtr actions, String column_name); + void appendPreliminaryFilter(ExpressionActionsChain & chain, ActionsDAGPtr actions, String column_name); /// remove_filter is set in ExpressionActionsChain::finalize(); /// Columns in `additional_required_columns` will not be removed (they can be used for e.g. sampling or FINAL modifier). - ExpressionActionsPtr appendPrewhere(ExpressionActionsChain & chain, bool only_types, const Names & additional_required_columns); + ActionsDAGPtr appendPrewhere(ExpressionActionsChain & chain, bool only_types, const Names & additional_required_columns); bool appendWhere(ExpressionActionsChain & chain, bool only_types); bool appendGroupBy(ExpressionActionsChain & chain, bool only_types, bool optimize_aggregation_in_order, ManyExpressionActions &); void appendAggregateFunctionsArguments(ExpressionActionsChain & chain, bool only_types); @@ -330,7 +331,7 @@ private: /// After aggregation: bool appendHaving(ExpressionActionsChain & chain, bool only_types); /// appendSelect - bool appendOrderBy(ExpressionActionsChain & chain, bool only_types, bool optimize_read_in_order, ManyExpressionActions &); + ActionsDAGPtr appendOrderBy(ExpressionActionsChain & chain, bool only_types, bool optimize_read_in_order, ManyExpressionActions &); bool appendLimitBy(ExpressionActionsChain & chain, bool only_types); /// appendProjectResult }; diff --git a/src/Interpreters/ExpressionJIT.cpp b/src/Interpreters/ExpressionJIT.cpp index 056d23299b7..27554e8a4d8 100644 --- a/src/Interpreters/ExpressionJIT.cpp +++ b/src/Interpreters/ExpressionJIT.cpp @@ -442,7 +442,7 @@ struct LLVMModuleState }; LLVMFunction::LLVMFunction(const ExpressionActions::Actions & actions, const DB::Block & sample_block) - : name(actions.back().result_name) + : name(actions.back().node->result_name) , module_state(std::make_unique()) { LLVMContext context; @@ -452,21 +452,21 @@ LLVMFunction::LLVMFunction(const ExpressionActions::Actions & actions, const DB: subexpressions[c.name] = subexpression(c.column, c.type); for (const auto & action : actions) { - const auto & names = action.argument_names; - const auto & types = action.function_base->getArgumentTypes(); + const auto & children = action.node->children; + const auto & types = action.node->function_base->getArgumentTypes(); std::vector args; - for (size_t i = 0; i < names.size(); ++i) + for (size_t i = 0; i < children.size(); ++i) { - auto inserted = subexpressions.emplace(names[i], subexpression(arg_names.size())); + auto inserted = subexpressions.emplace(children[i]->result_name, subexpression(arg_names.size())); if (inserted.second) { - arg_names.push_back(names[i]); + arg_names.push_back(children[i]->result_name); arg_types.push_back(types[i]); } args.push_back(inserted.first->second); } - subexpressions[action.result_name] = subexpression(*action.function_base, std::move(args)); - originals.push_back(action.function_base); + subexpressions[action.node->result_name] = subexpression(*action.node->function_base, std::move(args)); + originals.push_back(action.node->function_base); } compileFunctionToLLVMByteCode(context, *this); context.compileAllFunctionsToNativeCode(); @@ -555,155 +555,155 @@ LLVMFunction::Monotonicity LLVMFunction::getMonotonicityForRange(const IDataType } -static bool isCompilable(const IFunctionBase & function) -{ - if (!canBeNativeType(*function.getResultType())) - return false; - for (const auto & type : function.getArgumentTypes()) - if (!canBeNativeType(*type)) - return false; - return function.isCompilable(); -} +//static bool isCompilable(const IFunctionBase & function) +//{ +// if (!canBeNativeType(*function.getResultType())) +// return false; +// for (const auto & type : function.getArgumentTypes()) +// if (!canBeNativeType(*type)) +// return false; +// return function.isCompilable(); +//} -static std::vector>> getActionsDependents(const ExpressionActions::Actions & actions, const Names & output_columns) -{ - /// an empty optional is a poisoned value prohibiting the column's producer from being removed - /// (which it could be, if it was inlined into every dependent function). - std::unordered_map>> current_dependents; - for (const auto & name : output_columns) - current_dependents[name].emplace(); - /// a snapshot of each compilable function's dependents at the time of its execution. - std::vector>> dependents(actions.size()); - for (size_t i = actions.size(); i--;) - { - switch (actions[i].type) - { - case ExpressionAction::REMOVE_COLUMN: - current_dependents.erase(actions[i].source_name); - /// poison every other column used after this point so that inlining chains do not cross it. - for (auto & dep : current_dependents) - dep.second.emplace(); - break; - - case ExpressionAction::PROJECT: - current_dependents.clear(); - for (const auto & proj : actions[i].projection) - current_dependents[proj.first].emplace(); - break; - - case ExpressionAction::ADD_ALIASES: - for (const auto & proj : actions[i].projection) - current_dependents[proj.first].emplace(); - break; - - case ExpressionAction::ADD_COLUMN: - case ExpressionAction::COPY_COLUMN: - case ExpressionAction::ARRAY_JOIN: - { - Names columns = actions[i].getNeededColumns(); - for (const auto & column : columns) - current_dependents[column].emplace(); - break; - } - - case ExpressionAction::APPLY_FUNCTION: - { - dependents[i] = current_dependents[actions[i].result_name]; - const bool compilable = isCompilable(*actions[i].function_base); - for (const auto & name : actions[i].argument_names) - { - if (compilable) - current_dependents[name].emplace(i); - else - current_dependents[name].emplace(); - } - break; - } - } - } - return dependents; -} - -void compileFunctions( - ExpressionActions::Actions & actions, - const Names & output_columns, - const Block & sample_block, - std::shared_ptr compilation_cache, - size_t min_count_to_compile_expression) -{ - static std::unordered_map counter; - static std::mutex mutex; - - struct LLVMTargetInitializer - { - LLVMTargetInitializer() - { - llvm::InitializeNativeTarget(); - llvm::InitializeNativeTargetAsmPrinter(); - llvm::sys::DynamicLibrary::LoadLibraryPermanently(nullptr); - } - }; - - static LLVMTargetInitializer initializer; - - auto dependents = getActionsDependents(actions, output_columns); - std::vector fused(actions.size()); - for (size_t i = 0; i < actions.size(); ++i) - { - if (actions[i].type != ExpressionAction::APPLY_FUNCTION || !isCompilable(*actions[i].function_base)) - continue; - - fused[i].push_back(actions[i]); - if (dependents[i].find({}) != dependents[i].end()) - { - /// the result of compiling one function in isolation is pretty much the same as its `execute` method. - if (fused[i].size() == 1) - continue; - - auto hash_key = ExpressionActions::ActionsHash{}(fused[i]); - { - std::lock_guard lock(mutex); - if (counter[hash_key]++ < min_count_to_compile_expression) - continue; - } - - FunctionBasePtr fn; - if (compilation_cache) - { - std::tie(fn, std::ignore) = compilation_cache->getOrSet(hash_key, [&inlined_func=std::as_const(fused[i]), &sample_block] () - { - Stopwatch watch; - FunctionBasePtr result_fn; - result_fn = std::make_shared(std::make_unique(inlined_func, sample_block)); - ProfileEvents::increment(ProfileEvents::CompileExpressionsMicroseconds, watch.elapsedMicroseconds()); - return result_fn; - }); - } - else - { - Stopwatch watch; - fn = std::make_shared(std::make_unique(fused[i], sample_block)); - ProfileEvents::increment(ProfileEvents::CompileExpressionsMicroseconds, watch.elapsedMicroseconds()); - } - - actions[i].function_base = fn; - actions[i].argument_names = typeid_cast(typeid_cast(fn.get())->getImpl())->getArgumentNames(); - actions[i].is_function_compiled = true; - - continue; - } - - /// TODO: determine whether it's profitable to inline the function if there's more than one dependent. - for (const auto & dep : dependents[i]) - fused[*dep].insert(fused[*dep].end(), fused[i].begin(), fused[i].end()); - } - - for (auto & action : actions) - { - if (action.type == ExpressionAction::APPLY_FUNCTION && action.is_function_compiled) - action.function = action.function_base->prepare({}); /// Arguments are not used for LLVMFunction. - } -} +//static std::vector>> getActionsDependents(const ExpressionActions::Actions & actions, const Names & output_columns) +//{ +// /// an empty optional is a poisoned value prohibiting the column's producer from being removed +// /// (which it could be, if it was inlined into every dependent function). +// std::unordered_map>> current_dependents; +// for (const auto & name : output_columns) +// current_dependents[name].emplace(); +// /// a snapshot of each compilable function's dependents at the time of its execution. +// std::vector>> dependents(actions.size()); +// for (size_t i = actions.size(); i--;) +// { +// switch (actions[i].type) +// { +// case ExpressionAction::REMOVE_COLUMN: +// current_dependents.erase(actions[i].source_name); +// /// poison every other column used after this point so that inlining chains do not cross it. +// for (auto & dep : current_dependents) +// dep.second.emplace(); +// break; +// +// case ExpressionAction::PROJECT: +// current_dependents.clear(); +// for (const auto & proj : actions[i].projection) +// current_dependents[proj.first].emplace(); +// break; +// +// case ExpressionAction::ADD_ALIASES: +// for (const auto & proj : actions[i].projection) +// current_dependents[proj.first].emplace(); +// break; +// +// case ExpressionAction::ADD_COLUMN: +// case ExpressionAction::COPY_COLUMN: +// case ExpressionAction::ARRAY_JOIN: +// { +// Names columns = actions[i].getNeededColumns(); +// for (const auto & column : columns) +// current_dependents[column].emplace(); +// break; +// } +// +// case ExpressionAction::APPLY_FUNCTION: +// { +// dependents[i] = current_dependents[actions[i].result_name]; +// const bool compilable = isCompilable(*actions[i].function_base); +// for (const auto & name : actions[i].argument_names) +// { +// if (compilable) +// current_dependents[name].emplace(i); +// else +// current_dependents[name].emplace(); +// } +// break; +// } +// } +// } +// return dependents; +//} +// +//void compileFunctions( +// ExpressionActions::Actions & actions, +// const Names & output_columns, +// const Block & sample_block, +// std::shared_ptr compilation_cache, +// size_t min_count_to_compile_expression) +//{ +// static std::unordered_map counter; +// static std::mutex mutex; +// +// struct LLVMTargetInitializer +// { +// LLVMTargetInitializer() +// { +// llvm::InitializeNativeTarget(); +// llvm::InitializeNativeTargetAsmPrinter(); +// llvm::sys::DynamicLibrary::LoadLibraryPermanently(nullptr); +// } +// }; +// +// static LLVMTargetInitializer initializer; +// +// auto dependents = getActionsDependents(actions, output_columns); +// std::vector fused(actions.size()); +// for (size_t i = 0; i < actions.size(); ++i) +// { +// if (actions[i].type != ExpressionAction::APPLY_FUNCTION || !isCompilable(*actions[i].function_base)) +// continue; +// +// fused[i].push_back(actions[i]); +// if (dependents[i].find({}) != dependents[i].end()) +// { +// /// the result of compiling one function in isolation is pretty much the same as its `execute` method. +// if (fused[i].size() == 1) +// continue; +// +// auto hash_key = ExpressionActions::ActionsHash{}(fused[i]); +// { +// std::lock_guard lock(mutex); +// if (counter[hash_key]++ < min_count_to_compile_expression) +// continue; +// } +// +// FunctionBasePtr fn; +// if (compilation_cache) +// { +// std::tie(fn, std::ignore) = compilation_cache->getOrSet(hash_key, [&inlined_func=std::as_const(fused[i]), &sample_block] () +// { +// Stopwatch watch; +// FunctionBasePtr result_fn; +// result_fn = std::make_shared(std::make_unique(inlined_func, sample_block)); +// ProfileEvents::increment(ProfileEvents::CompileExpressionsMicroseconds, watch.elapsedMicroseconds()); +// return result_fn; +// }); +// } +// else +// { +// Stopwatch watch; +// fn = std::make_shared(std::make_unique(fused[i], sample_block)); +// ProfileEvents::increment(ProfileEvents::CompileExpressionsMicroseconds, watch.elapsedMicroseconds()); +// } +// +// actions[i].function_base = fn; +// actions[i].argument_names = typeid_cast(typeid_cast(fn.get())->getImpl())->getArgumentNames(); +// actions[i].is_function_compiled = true; +// +// continue; +// } +// +// /// TODO: determine whether it's profitable to inline the function if there's more than one dependent. +// for (const auto & dep : dependents[i]) +// fused[*dep].insert(fused[*dep].end(), fused[i].begin(), fused[i].end()); +// } +// +// for (auto & action : actions) +// { +// if (action.type == ExpressionAction::APPLY_FUNCTION && action.is_function_compiled) +// action.function = action.function_base->prepare({}); /// Arguments are not used for LLVMFunction. +// } +//} } diff --git a/src/Interpreters/ExpressionJIT.h b/src/Interpreters/ExpressionJIT.h index b2226aad638..bf015478215 100644 --- a/src/Interpreters/ExpressionJIT.h +++ b/src/Interpreters/ExpressionJIT.h @@ -74,7 +74,7 @@ public: /// For each APPLY_FUNCTION action, try to compile the function to native code; if the only uses of a compilable /// function's result are as arguments to other compilable functions, inline it and leave the now-redundant action as-is. -void compileFunctions(ExpressionActions::Actions & actions, const Names & output_columns, const Block & sample_block, std::shared_ptr compilation_cache, size_t min_count_to_compile_expression); +// void compileFunctions(ExpressionActions::Actions & actions, const Names & output_columns, const Block & sample_block, std::shared_ptr compilation_cache, size_t min_count_to_compile_expression); } diff --git a/src/Interpreters/HashJoin.cpp b/src/Interpreters/HashJoin.cpp index 1c3ffd4db1c..cf73581c6d8 100644 --- a/src/Interpreters/HashJoin.cpp +++ b/src/Interpreters/HashJoin.cpp @@ -988,6 +988,10 @@ void HashJoin::joinBlockImpl( const auto & right_key = required_right_keys.getByPosition(i); const auto & left_name = required_right_keys_sources[i]; + /// asof column is already in block. + if (is_asof_join && right_key.name == key_names_right.back()) + continue; + const auto & col = block.getByName(left_name); bool is_nullable = nullable_right_side || right_key.type->isNullable(); block.insert(correctNullability({col.column, col.type, right_key.name}, is_nullable)); @@ -1007,6 +1011,10 @@ void HashJoin::joinBlockImpl( const auto & right_key = required_right_keys.getByPosition(i); const auto & left_name = required_right_keys_sources[i]; + /// asof column is already in block. + if (is_asof_join && right_key.name == key_names_right.back()) + continue; + const auto & col = block.getByName(left_name); bool is_nullable = nullable_right_side || right_key.type->isNullable(); diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index d9821be4e4e..349fd926402 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -98,7 +98,7 @@ namespace ErrorCodes /// Assumes `storage` is set and the table filter (row-level security) is not empty. String InterpreterSelectQuery::generateFilterActions( - ExpressionActionsPtr & actions, const ASTPtr & row_policy_filter, const Names & prerequisite_columns) const + ActionsDAGPtr & actions, const ASTPtr & row_policy_filter, const Names & prerequisite_columns) const { const auto & db_name = table_id.getDatabaseName(); const auto & table_name = table_id.getTableName(); @@ -393,7 +393,7 @@ InterpreterSelectQuery::InterpreterSelectQuery( filter_info = std::make_shared(); filter_info->column_name = generateFilterActions(filter_info->actions, row_policy_filter, required_columns); source_header = metadata_snapshot->getSampleBlockForColumns( - filter_info->actions->getRequiredColumns(), storage->getVirtuals(), storage->getStorageID()); + filter_info->actions->getRequiredColumns().getNames(), storage->getVirtuals(), storage->getStorageID()); } } @@ -520,7 +520,7 @@ Block InterpreterSelectQuery::getSampleBlockImpl() if (analysis_result.prewhere_info) { - analysis_result.prewhere_info->prewhere_actions->execute(header); + analysis_result.prewhere_info->prewhere_actions->buildExpressions()->execute(header); header = materializeBlock(header); if (analysis_result.prewhere_info->remove_prewhere_column) header.erase(analysis_result.prewhere_info->prewhere_column_name); @@ -531,9 +531,9 @@ Block InterpreterSelectQuery::getSampleBlockImpl() if (options.to_stage == QueryProcessingStage::Enum::WithMergeableState) { if (!analysis_result.need_aggregate) - return analysis_result.before_order_and_select->getSampleBlock(); + return analysis_result.before_order_and_select->getResultColumns(); - auto header = analysis_result.before_aggregation->getSampleBlock(); + Block header = analysis_result.before_aggregation->getResultColumns(); Block res; @@ -557,10 +557,10 @@ Block InterpreterSelectQuery::getSampleBlockImpl() if (options.to_stage == QueryProcessingStage::Enum::WithMergeableStateAfterAggregation) { - return analysis_result.before_order_and_select->getSampleBlock(); + return analysis_result.before_order_and_select->getResultColumns(); } - return analysis_result.final_projection->getSampleBlock(); + return analysis_result.final_projection->getResultColumns(); } static Field getWithFillFieldValue(const ASTPtr & node, const Context & context) @@ -1108,7 +1108,7 @@ static StreamLocalLimits getLimitsForStorage(const Settings & settings, const Se void InterpreterSelectQuery::executeFetchColumns( QueryProcessingStage::Enum processing_stage, QueryPlan & query_plan, - const PrewhereInfoPtr & prewhere_info, const Names & columns_to_remove_after_prewhere) + const PrewhereInfoPtr & prewhere_info, const NameSet & columns_to_remove_after_prewhere) { auto & query = getSelectQuery(); const Settings & settings = context->getSettingsRef(); @@ -1156,7 +1156,7 @@ void InterpreterSelectQuery::executeFetchColumns( auto column = ColumnAggregateFunction::create(func); column->insertFrom(place); - auto header = analysis_result.before_aggregation->getSampleBlock(); + Block header = analysis_result.before_aggregation->getResultColumns(); size_t arguments_size = desc.argument_names.size(); DataTypes argument_types(arguments_size); for (size_t j = 0; j < arguments_size; ++j) @@ -1176,7 +1176,7 @@ void InterpreterSelectQuery::executeFetchColumns( } /// Actions to calculate ALIAS if required. - ExpressionActionsPtr alias_actions; + ActionsDAGPtr alias_actions; if (storage) { @@ -1185,14 +1185,14 @@ void InterpreterSelectQuery::executeFetchColumns( if (row_policy_filter) { auto initial_required_columns = required_columns; - ExpressionActionsPtr actions; + ActionsDAGPtr actions; generateFilterActions(actions, row_policy_filter, initial_required_columns); auto required_columns_from_filter = actions->getRequiredColumns(); for (const auto & column : required_columns_from_filter) { - if (required_columns.end() == std::find(required_columns.begin(), required_columns.end(), column)) - required_columns.push_back(column); + if (required_columns.end() == std::find(required_columns.begin(), required_columns.end(), column.name)) + required_columns.push_back(column.name); } } @@ -1224,7 +1224,7 @@ void InterpreterSelectQuery::executeFetchColumns( if (prewhere_info) { /// Get some columns directly from PREWHERE expression actions - auto prewhere_required_columns = prewhere_info->prewhere_actions->getRequiredColumns(); + auto prewhere_required_columns = prewhere_info->prewhere_actions->getRequiredColumns().getNames(); required_columns_from_prewhere.insert(prewhere_required_columns.begin(), prewhere_required_columns.end()); } @@ -1270,7 +1270,7 @@ void InterpreterSelectQuery::executeFetchColumns( if (prewhere_info) { NameSet columns_to_remove(columns_to_remove_after_prewhere.begin(), columns_to_remove_after_prewhere.end()); - Block prewhere_actions_result = prewhere_info->prewhere_actions->getSampleBlock(); + Block prewhere_actions_result = prewhere_info->prewhere_actions->getResultColumns(); /// Populate required columns with the columns, added by PREWHERE actions and not removed afterwards. /// XXX: looks hacky that we already know which columns after PREWHERE we won't need for sure. @@ -1291,10 +1291,10 @@ void InterpreterSelectQuery::executeFetchColumns( } auto syntax_result = TreeRewriter(*context).analyze(required_columns_all_expr, required_columns_after_prewhere, storage, metadata_snapshot); - alias_actions = ExpressionAnalyzer(required_columns_all_expr, syntax_result, *context).getActions(true); + alias_actions = ExpressionAnalyzer(required_columns_all_expr, syntax_result, *context).getActionsDAG(true); /// The set of required columns could be added as a result of adding an action to calculate ALIAS. - required_columns = alias_actions->getRequiredColumns(); + required_columns = alias_actions->getRequiredColumns().getNames(); /// Do not remove prewhere filter if it is a column which is used as alias. if (prewhere_info && prewhere_info->remove_prewhere_column) @@ -1311,27 +1311,21 @@ void InterpreterSelectQuery::executeFetchColumns( if (prewhere_info) { /// Don't remove columns which are needed to be aliased. - auto new_actions = std::make_shared(prewhere_info->prewhere_actions->getRequiredColumnsWithTypes(), *context); - for (const auto & action : prewhere_info->prewhere_actions->getActions()) - { - if (action.type != ExpressionAction::REMOVE_COLUMN - || required_columns.end() == std::find(required_columns.begin(), required_columns.end(), action.source_name)) - new_actions->add(action); - } - prewhere_info->prewhere_actions = std::move(new_actions); + for (const auto & name : required_columns) + prewhere_info->prewhere_actions->tryRestoreColumn(name); auto analyzed_result = TreeRewriter(*context).analyze(required_columns_from_prewhere_expr, metadata_snapshot->getColumns().getAllPhysical()); prewhere_info->alias_actions - = ExpressionAnalyzer(required_columns_from_prewhere_expr, analyzed_result, *context).getActions(true, false); + = ExpressionAnalyzer(required_columns_from_prewhere_expr, analyzed_result, *context).getActionsDAG(true, false); /// Add (physical?) columns required by alias actions. auto required_columns_from_alias = prewhere_info->alias_actions->getRequiredColumns(); - Block prewhere_actions_result = prewhere_info->prewhere_actions->getSampleBlock(); + Block prewhere_actions_result = prewhere_info->prewhere_actions->getResultColumns(); for (auto & column : required_columns_from_alias) - if (!prewhere_actions_result.has(column)) - if (required_columns.end() == std::find(required_columns.begin(), required_columns.end(), column)) - required_columns.push_back(column); + if (!prewhere_actions_result.has(column.name)) + if (required_columns.end() == std::find(required_columns.begin(), required_columns.end(), column.name)) + required_columns.push_back(column.name); /// Add physical columns required by prewhere actions. for (const auto & column : required_columns_from_prewhere) @@ -1488,7 +1482,7 @@ void InterpreterSelectQuery::executeFetchColumns( } -void InterpreterSelectQuery::executeWhere(QueryPlan & query_plan, const ExpressionActionsPtr & expression, bool remove_filter) +void InterpreterSelectQuery::executeWhere(QueryPlan & query_plan, const ActionsDAGPtr & expression, bool remove_filter) { auto where_step = std::make_unique( query_plan.getCurrentDataStream(), @@ -1501,7 +1495,7 @@ void InterpreterSelectQuery::executeWhere(QueryPlan & query_plan, const Expressi } -void InterpreterSelectQuery::executeAggregation(QueryPlan & query_plan, const ExpressionActionsPtr & expression, bool overflow_row, bool final, InputOrderInfoPtr group_by_info) +void InterpreterSelectQuery::executeAggregation(QueryPlan & query_plan, const ActionsDAGPtr & expression, bool overflow_row, bool final, InputOrderInfoPtr group_by_info) { auto expression_before_aggregation = std::make_unique(query_plan.getCurrentDataStream(), expression); expression_before_aggregation->setStepDescription("Before GROUP BY"); @@ -1598,7 +1592,7 @@ void InterpreterSelectQuery::executeMergeAggregated(QueryPlan & query_plan, bool } -void InterpreterSelectQuery::executeHaving(QueryPlan & query_plan, const ExpressionActionsPtr & expression) +void InterpreterSelectQuery::executeHaving(QueryPlan & query_plan, const ActionsDAGPtr & expression) { auto having_step = std::make_unique( query_plan.getCurrentDataStream(), @@ -1609,7 +1603,7 @@ void InterpreterSelectQuery::executeHaving(QueryPlan & query_plan, const Express } -void InterpreterSelectQuery::executeTotalsAndHaving(QueryPlan & query_plan, bool has_having, const ExpressionActionsPtr & expression, bool overflow_row, bool final) +void InterpreterSelectQuery::executeTotalsAndHaving(QueryPlan & query_plan, bool has_having, const ActionsDAGPtr & expression, bool overflow_row, bool final) { const Settings & settings = context->getSettingsRef(); @@ -1651,7 +1645,7 @@ void InterpreterSelectQuery::executeRollupOrCube(QueryPlan & query_plan, Modific } -void InterpreterSelectQuery::executeExpression(QueryPlan & query_plan, const ExpressionActionsPtr & expression, const std::string & description) +void InterpreterSelectQuery::executeExpression(QueryPlan & query_plan, const ActionsDAGPtr & expression, const std::string & description) { auto expression_step = std::make_unique(query_plan.getCurrentDataStream(), expression); @@ -1742,7 +1736,7 @@ void InterpreterSelectQuery::executeMergeSorted(QueryPlan & query_plan, const So } -void InterpreterSelectQuery::executeProjection(QueryPlan & query_plan, const ExpressionActionsPtr & expression) +void InterpreterSelectQuery::executeProjection(QueryPlan & query_plan, const ActionsDAGPtr & expression) { auto projection_step = std::make_unique(query_plan.getCurrentDataStream(), expression); projection_step->setStepDescription("Projection"); diff --git a/src/Interpreters/InterpreterSelectQuery.h b/src/Interpreters/InterpreterSelectQuery.h index 455b1a1e623..9623f1eb5a0 100644 --- a/src/Interpreters/InterpreterSelectQuery.h +++ b/src/Interpreters/InterpreterSelectQuery.h @@ -117,14 +117,14 @@ private: QueryProcessingStage::Enum processing_stage, QueryPlan & query_plan, const PrewhereInfoPtr & prewhere_info, - const Names & columns_to_remove_after_prewhere); + const NameSet & columns_to_remove_after_prewhere); - void executeWhere(QueryPlan & query_plan, const ExpressionActionsPtr & expression, bool remove_filter); - void executeAggregation(QueryPlan & query_plan, const ExpressionActionsPtr & expression, bool overflow_row, bool final, InputOrderInfoPtr group_by_info); + void executeWhere(QueryPlan & query_plan, const ActionsDAGPtr & expression, bool remove_filter); + void executeAggregation(QueryPlan & query_plan, const ActionsDAGPtr & expression, bool overflow_row, bool final, InputOrderInfoPtr group_by_info); void executeMergeAggregated(QueryPlan & query_plan, bool overflow_row, bool final); - void executeTotalsAndHaving(QueryPlan & query_plan, bool has_having, const ExpressionActionsPtr & expression, bool overflow_row, bool final); - void executeHaving(QueryPlan & query_plan, const ExpressionActionsPtr & expression); - static void executeExpression(QueryPlan & query_plan, const ExpressionActionsPtr & expression, const std::string & description); + void executeTotalsAndHaving(QueryPlan & query_plan, bool has_having, const ActionsDAGPtr & expression, bool overflow_row, bool final); + void executeHaving(QueryPlan & query_plan, const ActionsDAGPtr & expression); + static void executeExpression(QueryPlan & query_plan, const ActionsDAGPtr & expression, const std::string & description); void executeOrder(QueryPlan & query_plan, InputOrderInfoPtr sorting_info); void executeOrderOptimized(QueryPlan & query_plan, InputOrderInfoPtr sorting_info, UInt64 limit, SortDescription & output_order_descr); void executeWithFill(QueryPlan & query_plan); @@ -133,14 +133,14 @@ private: void executeLimitBy(QueryPlan & query_plan); void executeLimit(QueryPlan & query_plan); void executeOffset(QueryPlan & query_plan); - static void executeProjection(QueryPlan & query_plan, const ExpressionActionsPtr & expression); + static void executeProjection(QueryPlan & query_plan, const ActionsDAGPtr & expression); void executeDistinct(QueryPlan & query_plan, bool before_order, Names columns, bool pre_distinct); void executeExtremes(QueryPlan & query_plan); void executeSubqueriesInSetsAndJoins(QueryPlan & query_plan, std::unordered_map & subqueries_for_sets); void executeMergeSorted(QueryPlan & query_plan, const SortDescription & sort_description, UInt64 limit, const std::string & description); String generateFilterActions( - ExpressionActionsPtr & actions, const ASTPtr & row_policy_filter, const Names & prerequisite_columns = {}) const; + ActionsDAGPtr & actions, const ASTPtr & row_policy_filter, const Names & prerequisite_columns = {}) const; enum class Modificator { diff --git a/src/Interpreters/MutationsInterpreter.cpp b/src/Interpreters/MutationsInterpreter.cpp index 3e7ebfec139..3bfa05c1d8a 100644 --- a/src/Interpreters/MutationsInterpreter.cpp +++ b/src/Interpreters/MutationsInterpreter.cpp @@ -644,6 +644,10 @@ ASTPtr MutationsInterpreter::prepareInterpreterSelectQuery(std::vector & for (const auto & column_name : prepared_stages[0].output_columns) select->select()->children.push_back(std::make_shared(column_name)); + /// Don't let select list be empty. + if (select->select()->children.empty()) + select->select()->children.push_back(std::make_shared(Field(0))); + if (!prepared_stages[0].filters.empty()) { ASTPtr where_expression; @@ -676,12 +680,12 @@ QueryPipelinePtr MutationsInterpreter::addStreamsForLaterStages(const std::vecto if (i < stage.filter_column_names.size()) { /// Execute DELETEs. - plan.addStep(std::make_unique(plan.getCurrentDataStream(), step->getExpression(), stage.filter_column_names[i], false)); + plan.addStep(std::make_unique(plan.getCurrentDataStream(), step->actions(), stage.filter_column_names[i], false)); } else { /// Execute UPDATE or final projection. - plan.addStep(std::make_unique(plan.getCurrentDataStream(), step->getExpression())); + plan.addStep(std::make_unique(plan.getCurrentDataStream(), step->actions())); } } diff --git a/src/Interpreters/getHeaderForProcessingStage.cpp b/src/Interpreters/getHeaderForProcessingStage.cpp index db7008a1779..3a98109fdd9 100644 --- a/src/Interpreters/getHeaderForProcessingStage.cpp +++ b/src/Interpreters/getHeaderForProcessingStage.cpp @@ -43,7 +43,7 @@ Block getHeaderForProcessingStage( Block header = metadata_snapshot->getSampleBlockForColumns(column_names, storage.getVirtuals(), storage.getStorageID()); if (query_info.prewhere_info) { - query_info.prewhere_info->prewhere_actions->execute(header); + query_info.prewhere_info->prewhere_actions->buildExpressions()->execute(header); if (query_info.prewhere_info->remove_prewhere_column) header.erase(query_info.prewhere_info->prewhere_column_name); } diff --git a/src/Processors/QueryPlan/ExpressionStep.cpp b/src/Processors/QueryPlan/ExpressionStep.cpp index 61e4f2ffebc..293583ef550 100644 --- a/src/Processors/QueryPlan/ExpressionStep.cpp +++ b/src/Processors/QueryPlan/ExpressionStep.cpp @@ -9,18 +9,18 @@ namespace DB { -static ITransformingStep::Traits getTraits(const ExpressionActionsPtr & expression) +static ITransformingStep::Traits getTraits(const ActionsDAGPtr & actions) { return ITransformingStep::Traits { { - .preserves_distinct_columns = !expression->hasArrayJoin(), + .preserves_distinct_columns = !actions->hasArrayJoin(), .returns_single_stream = false, .preserves_number_of_streams = true, - .preserves_sorting = !expression->hasArrayJoin(), + .preserves_sorting = !actions->hasArrayJoin(), }, { - .preserves_number_of_rows = !expression->hasArrayJoin(), + .preserves_number_of_rows = !actions->hasArrayJoin(), } }; } @@ -41,12 +41,12 @@ static ITransformingStep::Traits getJoinTraits() }; } -ExpressionStep::ExpressionStep(const DataStream & input_stream_, ExpressionActionsPtr expression_) +ExpressionStep::ExpressionStep(const DataStream & input_stream_, ActionsDAGPtr actions_) : ITransformingStep( input_stream_, - Transform::transformHeader(input_stream_.header, expression_), - getTraits(expression_)) - , expression(std::move(expression_)) + Transform::transformHeader(input_stream_.header, actions_->buildExpressions()), + getTraits(actions_)) + , actions(std::move(actions_)) { /// Some columns may be removed by expression. updateDistinctColumns(output_stream->header, output_stream->distinct_columns); @@ -55,7 +55,7 @@ ExpressionStep::ExpressionStep(const DataStream & input_stream_, ExpressionActio void ExpressionStep::updateInputStream(DataStream input_stream, bool keep_header) { Block out_header = keep_header ? std::move(output_stream->header) - : Transform::transformHeader(input_stream.header, expression); + : Transform::transformHeader(input_stream.header, actions->buildExpressions()); output_stream = createOutputStream( input_stream, std::move(out_header), @@ -67,6 +67,7 @@ void ExpressionStep::updateInputStream(DataStream input_stream, bool keep_header void ExpressionStep::transformPipeline(QueryPipeline & pipeline) { + auto expression = actions->buildExpressions(); pipeline.addSimpleTransform([&](const Block & header) { return std::make_shared(header, expression); @@ -82,11 +83,12 @@ void ExpressionStep::transformPipeline(QueryPipeline & pipeline) } } -static void doDescribeActions(const ExpressionActionsPtr & expression, IQueryPlanStep::FormatSettings & settings) +void ExpressionStep::describeActions(FormatSettings & settings) const { String prefix(settings.offset, ' '); bool first = true; + auto expression = actions->buildExpressions(); for (const auto & action : expression->getActions()) { settings.out << prefix << (first ? "Actions: " @@ -96,11 +98,6 @@ static void doDescribeActions(const ExpressionActionsPtr & expression, IQueryPla } } -void ExpressionStep::describeActions(FormatSettings & settings) const -{ - doDescribeActions(expression, settings); -} - JoinStep::JoinStep(const DataStream & input_stream_, JoinPtr join_) : ITransformingStep( input_stream_, diff --git a/src/Processors/QueryPlan/ExpressionStep.h b/src/Processors/QueryPlan/ExpressionStep.h index 45aaa010121..c002de8deb1 100644 --- a/src/Processors/QueryPlan/ExpressionStep.h +++ b/src/Processors/QueryPlan/ExpressionStep.h @@ -4,8 +4,8 @@ namespace DB { -class ExpressionActions; -using ExpressionActionsPtr = std::shared_ptr; +class ActionsDAG; +using ActionsDAGPtr = std::shared_ptr; class IJoin; using JoinPtr = std::shared_ptr; @@ -19,7 +19,7 @@ class ExpressionStep : public ITransformingStep public: using Transform = ExpressionTransform; - explicit ExpressionStep(const DataStream & input_stream_, ExpressionActionsPtr expression_); + explicit ExpressionStep(const DataStream & input_stream_, ActionsDAGPtr actions_); String getName() const override { return "Expression"; } void transformPipeline(QueryPipeline & pipeline) override; @@ -28,10 +28,10 @@ public: void describeActions(FormatSettings & settings) const override; - const ExpressionActionsPtr & getExpression() const { return expression; } + const ActionsDAGPtr & getExpression() const { return actions; } private: - ExpressionActionsPtr expression; + ActionsDAGPtr actions; }; /// TODO: add separate step for join. diff --git a/src/Processors/QueryPlan/FillingStep.cpp b/src/Processors/QueryPlan/FillingStep.cpp index 015b5224054..1a8fba97ee2 100644 --- a/src/Processors/QueryPlan/FillingStep.cpp +++ b/src/Processors/QueryPlan/FillingStep.cpp @@ -28,7 +28,7 @@ static ITransformingStep::Traits getTraits() } FillingStep::FillingStep(const DataStream & input_stream_, SortDescription sort_description_) - : ITransformingStep(input_stream_, input_stream_.header, getTraits()) + : ITransformingStep(input_stream_, FillingTransform::transformHeader(input_stream_.header, sort_description_), getTraits()) , sort_description(std::move(sort_description_)) { if (!input_stream_.has_single_port) diff --git a/src/Processors/QueryPlan/FilterStep.cpp b/src/Processors/QueryPlan/FilterStep.cpp index 2bbbc0373da..ce6522cccc8 100644 --- a/src/Processors/QueryPlan/FilterStep.cpp +++ b/src/Processors/QueryPlan/FilterStep.cpp @@ -8,7 +8,7 @@ namespace DB { -static ITransformingStep::Traits getTraits(const ExpressionActionsPtr & expression) +static ITransformingStep::Traits getTraits(const ActionsDAGPtr & expression) { return ITransformingStep::Traits { @@ -26,14 +26,14 @@ static ITransformingStep::Traits getTraits(const ExpressionActionsPtr & expressi FilterStep::FilterStep( const DataStream & input_stream_, - ExpressionActionsPtr expression_, + ActionsDAGPtr actions_, String filter_column_name_, bool remove_filter_column_) : ITransformingStep( input_stream_, - FilterTransform::transformHeader(input_stream_.header, expression_, filter_column_name_, remove_filter_column_), - getTraits(expression_)) - , expression(std::move(expression_)) + FilterTransform::transformHeader(input_stream_.header, actions_->buildExpressions(), filter_column_name_, remove_filter_column_), + getTraits(actions_)) + , actions(std::move(actions_)) , filter_column_name(std::move(filter_column_name_)) , remove_filter_column(remove_filter_column_) { @@ -45,7 +45,7 @@ void FilterStep::updateInputStream(DataStream input_stream, bool keep_header) { Block out_header = std::move(output_stream->header); if (keep_header) - out_header = FilterTransform::transformHeader(input_stream.header, expression, filter_column_name, remove_filter_column); + out_header = FilterTransform::transformHeader(input_stream.header, actions->buildExpressions(), filter_column_name, remove_filter_column); output_stream = createOutputStream( input_stream, @@ -58,6 +58,7 @@ void FilterStep::updateInputStream(DataStream input_stream, bool keep_header) void FilterStep::transformPipeline(QueryPipeline & pipeline) { + auto expression = actions->buildExpressions(); pipeline.addSimpleTransform([&](const Block & header, QueryPipeline::StreamType stream_type) { bool on_totals = stream_type == QueryPipeline::StreamType::Totals; @@ -79,6 +80,7 @@ void FilterStep::describeActions(FormatSettings & settings) const settings.out << prefix << "Filter column: " << filter_column_name << '\n'; bool first = true; + auto expression = actions->buildExpressions(); for (const auto & action : expression->getActions()) { settings.out << prefix << (first ? "Actions: " diff --git a/src/Processors/QueryPlan/FilterStep.h b/src/Processors/QueryPlan/FilterStep.h index d827fe920eb..efd91637c1f 100644 --- a/src/Processors/QueryPlan/FilterStep.h +++ b/src/Processors/QueryPlan/FilterStep.h @@ -4,8 +4,8 @@ namespace DB { -class ExpressionActions; -using ExpressionActionsPtr = std::shared_ptr; +class ActionsDAG; +using ActionsDAGPtr = std::shared_ptr; /// Implements WHERE, HAVING operations. See FilterTransform. class FilterStep : public ITransformingStep @@ -13,7 +13,7 @@ class FilterStep : public ITransformingStep public: FilterStep( const DataStream & input_stream_, - ExpressionActionsPtr expression_, + ActionsDAGPtr actions_, String filter_column_name_, bool remove_filter_column_); @@ -24,12 +24,12 @@ public: void describeActions(FormatSettings & settings) const override; - const ExpressionActionsPtr & getExpression() const { return expression; } + const ActionsDAGPtr & getExpression() const { return actions; } const String & getFilterColumnName() const { return filter_column_name; } bool removesFilterColumn() const { return remove_filter_column; } private: - ExpressionActionsPtr expression; + ActionsDAGPtr actions; String filter_column_name; bool remove_filter_column; }; diff --git a/src/Processors/QueryPlan/QueryPlan.cpp b/src/Processors/QueryPlan/QueryPlan.cpp index 1ff844480a9..040dc27518e 100644 --- a/src/Processors/QueryPlan/QueryPlan.cpp +++ b/src/Processors/QueryPlan/QueryPlan.cpp @@ -438,7 +438,7 @@ static void tryLiftUpArrayJoin(QueryPlan::Node * parent_node, QueryPlan::Node * return; /// All actions was moved before ARRAY JOIN. Swap Expression and ArrayJoin. - if (expression->getActions().empty()) + if (expression->empty()) { auto expected_header = parent->getOutputStream().header; diff --git a/src/Processors/QueryPlan/ReadFromStorageStep.cpp b/src/Processors/QueryPlan/ReadFromStorageStep.cpp index fd38dd9218b..88248c0d194 100644 --- a/src/Processors/QueryPlan/ReadFromStorageStep.cpp +++ b/src/Processors/QueryPlan/ReadFromStorageStep.cpp @@ -38,17 +38,19 @@ ReadFromStorageStep::ReadFromStorageStep( { if (query_info.prewhere_info->alias_actions) { + auto alias_actions = query_info.prewhere_info->alias_actions->buildExpressions(); pipe.addSimpleTransform([&](const Block & header) { - return std::make_shared(header, query_info.prewhere_info->alias_actions); + return std::make_shared(header, alias_actions); }); } + auto prewhere_actions = query_info.prewhere_info->prewhere_actions->buildExpressions(); pipe.addSimpleTransform([&](const Block & header) { return std::make_shared( header, - query_info.prewhere_info->prewhere_actions, + prewhere_actions, query_info.prewhere_info->prewhere_column_name, query_info.prewhere_info->remove_prewhere_column); }); @@ -59,10 +61,10 @@ ReadFromStorageStep::ReadFromStorageStep( // This leads to mismatched header in distributed table if (query_info.prewhere_info->remove_columns_actions) { + auto remove_actions = query_info.prewhere_info->remove_columns_actions->buildExpressions(); pipe.addSimpleTransform([&](const Block & header) { - return std::make_shared( - header, query_info.prewhere_info->remove_columns_actions); + return std::make_shared(header, remove_actions); }); } } diff --git a/src/Processors/QueryPlan/TotalsHavingStep.cpp b/src/Processors/QueryPlan/TotalsHavingStep.cpp index 823db356f7b..fd27f67f70e 100644 --- a/src/Processors/QueryPlan/TotalsHavingStep.cpp +++ b/src/Processors/QueryPlan/TotalsHavingStep.cpp @@ -27,17 +27,17 @@ static ITransformingStep::Traits getTraits(bool has_filter) TotalsHavingStep::TotalsHavingStep( const DataStream & input_stream_, bool overflow_row_, - const ExpressionActionsPtr & expression_, + const ActionsDAGPtr & actions_, const std::string & filter_column_, TotalsMode totals_mode_, double auto_include_threshold_, bool final_) : ITransformingStep( input_stream_, - TotalsHavingTransform::transformHeader(input_stream_.header, expression_, final_), + TotalsHavingTransform::transformHeader(input_stream_.header, (actions_ ? actions_->buildExpressions() : nullptr), final_), getTraits(!filter_column_.empty())) , overflow_row(overflow_row_) - , expression(expression_) + , actions(actions_) , filter_column_name(filter_column_) , totals_mode(totals_mode_) , auto_include_threshold(auto_include_threshold_) @@ -48,7 +48,7 @@ TotalsHavingStep::TotalsHavingStep( void TotalsHavingStep::transformPipeline(QueryPipeline & pipeline) { auto totals_having = std::make_shared( - pipeline.getHeader(), overflow_row, expression, + pipeline.getHeader(), overflow_row, (actions ? actions->buildExpressions() : nullptr), filter_column_name, totals_mode, auto_include_threshold, final); pipeline.addTotalsHavingTransform(std::move(totals_having)); @@ -78,6 +78,7 @@ void TotalsHavingStep::describeActions(FormatSettings & settings) const settings.out << prefix << "Mode: " << totalsModeToString(totals_mode, auto_include_threshold) << '\n'; bool first = true; + auto expression = actions->buildExpressions(); for (const auto & action : expression->getActions()) { settings.out << prefix << (first ? "Actions: " diff --git a/src/Processors/QueryPlan/TotalsHavingStep.h b/src/Processors/QueryPlan/TotalsHavingStep.h index c9c73985126..792b96c9162 100644 --- a/src/Processors/QueryPlan/TotalsHavingStep.h +++ b/src/Processors/QueryPlan/TotalsHavingStep.h @@ -4,8 +4,8 @@ namespace DB { -class ExpressionActions; -using ExpressionActionsPtr = std::shared_ptr; +class ActionsDAG; +using ActionsDAGPtr = std::shared_ptr; enum class TotalsMode; @@ -16,7 +16,7 @@ public: TotalsHavingStep( const DataStream & input_stream_, bool overflow_row_, - const ExpressionActionsPtr & expression_, + const ActionsDAGPtr & actions_, const std::string & filter_column_, TotalsMode totals_mode_, double auto_include_threshold_, @@ -30,7 +30,7 @@ public: private: bool overflow_row; - ExpressionActionsPtr expression; + ActionsDAGPtr actions; String filter_column_name; TotalsMode totals_mode; double auto_include_threshold; diff --git a/src/Processors/Transforms/ExpressionTransform.cpp b/src/Processors/Transforms/ExpressionTransform.cpp index 501a01906ff..e60fc65e96e 100644 --- a/src/Processors/Transforms/ExpressionTransform.cpp +++ b/src/Processors/Transforms/ExpressionTransform.cpp @@ -1,13 +1,12 @@ #include -#include #include - namespace DB { Block ExpressionTransform::transformHeader(Block header, const ExpressionActionsPtr & expression) { - expression->execute(header, true); + size_t num_rows = header.rows(); + expression->execute(header, num_rows, true); return header; } @@ -20,11 +19,11 @@ ExpressionTransform::ExpressionTransform(const Block & header_, ExpressionAction void ExpressionTransform::transform(Chunk & chunk) { + size_t num_rows = chunk.getNumRows(); auto block = getInputPort().getHeader().cloneWithColumns(chunk.detachColumns()); - expression->execute(block); + expression->execute(block, num_rows); - auto num_rows = block.rows(); chunk.setColumns(block.getColumns(), num_rows); } diff --git a/src/Processors/Transforms/FillingTransform.cpp b/src/Processors/Transforms/FillingTransform.cpp index 76c4d7ddc16..3f3a0e75223 100644 --- a/src/Processors/Transforms/FillingTransform.cpp +++ b/src/Processors/Transforms/FillingTransform.cpp @@ -10,10 +10,23 @@ namespace ErrorCodes extern const int INVALID_WITH_FILL_EXPRESSION; } +Block FillingTransform::transformHeader(Block header, const SortDescription & sort_description) +{ + NameSet sort_keys; + for (const auto & key : sort_description) + sort_keys.insert(key.column_name); + + /// Columns which are not from sorting key may not be constant anymore. + for (auto & column : header) + if (column.column && isColumnConst(*column.column) && !sort_keys.count(column.name)) + column.column = column.type->createColumn(); + + return header; +} FillingTransform::FillingTransform( const Block & header_, const SortDescription & sort_description_) - : ISimpleTransform(header_, header_, true) + : ISimpleTransform(header_, transformHeader(header_, sort_description_), true) , sort_description(sort_description_) , filling_row(sort_description_) , next_row(sort_description_) diff --git a/src/Processors/Transforms/FillingTransform.h b/src/Processors/Transforms/FillingTransform.h index 3df981634ca..33717b079a0 100644 --- a/src/Processors/Transforms/FillingTransform.h +++ b/src/Processors/Transforms/FillingTransform.h @@ -19,6 +19,8 @@ public: Status prepare() override; + static Block transformHeader(Block header, const SortDescription & sort_description); + protected: void transform(Chunk & Chunk) override; diff --git a/src/Processors/Transforms/FilterTransform.cpp b/src/Processors/Transforms/FilterTransform.cpp index 197e0ac7595..23b1bf8a984 100644 --- a/src/Processors/Transforms/FilterTransform.cpp +++ b/src/Processors/Transforms/FilterTransform.cpp @@ -33,7 +33,8 @@ Block FilterTransform::transformHeader( const String & filter_column_name, bool remove_filter_column) { - expression->execute(header); + size_t num_rows = header.rows(); + expression->execute(header, num_rows); if (remove_filter_column) header.erase(filter_column_name); @@ -96,16 +97,15 @@ void FilterTransform::removeFilterIfNeed(Chunk & chunk) const void FilterTransform::transform(Chunk & chunk) { - size_t num_rows_before_filtration; + size_t num_rows_before_filtration = chunk.getNumRows(); auto columns = chunk.detachColumns(); { Block block = getInputPort().getHeader().cloneWithColumns(columns); columns.clear(); - expression->execute(block); + expression->execute(block, num_rows_before_filtration); - num_rows_before_filtration = block.rows(); columns = block.getColumns(); } diff --git a/src/Processors/Transforms/TotalsHavingTransform.cpp b/src/Processors/Transforms/TotalsHavingTransform.cpp index 65043f65e1a..fefe96aaa7c 100644 --- a/src/Processors/Transforms/TotalsHavingTransform.cpp +++ b/src/Processors/Transforms/TotalsHavingTransform.cpp @@ -32,8 +32,10 @@ Block TotalsHavingTransform::transformHeader(Block block, const ExpressionAction if (final) finalizeBlock(block); + size_t num_rows = block.rows(); + if (expression) - expression->execute(block); + expression->execute(block, num_rows); return block; } @@ -64,7 +66,8 @@ TotalsHavingTransform::TotalsHavingTransform( if (expression) { auto totals_header = finalized_header; - expression->execute(totals_header); + size_t num_rows = totals_header.rows(); + expression->execute(totals_header, num_rows); outputs.emplace_back(totals_header, this); } else @@ -155,8 +158,9 @@ void TotalsHavingTransform::transform(Chunk & chunk) { /// Compute the expression in HAVING. const auto & cur_header = final ? finalized_header : getInputPort().getHeader(); + size_t num_rows = finalized.getNumRows(); auto finalized_block = cur_header.cloneWithColumns(finalized.detachColumns()); - expression->execute(finalized_block); + expression->execute(finalized_block, num_rows); auto columns = finalized_block.getColumns(); ColumnPtr filter_column_ptr = columns[filter_column_pos]; @@ -165,7 +169,6 @@ void TotalsHavingTransform::transform(Chunk & chunk) if (const_filter_description.always_true) { addToTotals(chunk, nullptr); - auto num_rows = columns.front()->size(); chunk.setColumns(std::move(columns), num_rows); return; } @@ -198,7 +201,7 @@ void TotalsHavingTransform::transform(Chunk & chunk) } } - auto num_rows = columns.front()->size(); + num_rows = columns.front()->size(); chunk.setColumns(std::move(columns), num_rows); } @@ -255,10 +258,11 @@ void TotalsHavingTransform::prepareTotals() if (expression) { + size_t num_rows = totals.getNumRows(); auto block = finalized_header.cloneWithColumns(totals.detachColumns()); - expression->execute(block); + expression->execute(block, num_rows); /// Note: after expression totals may have several rows if `arrayJoin` was used in expression. - totals = Chunk(block.getColumns(), block.rows()); + totals = Chunk(block.getColumns(), num_rows); } } diff --git a/src/Storages/ColumnsDescription.cpp b/src/Storages/ColumnsDescription.cpp index 9bbc1653848..c55d608110a 100644 --- a/src/Storages/ColumnsDescription.cpp +++ b/src/Storages/ColumnsDescription.cpp @@ -502,7 +502,7 @@ Block validateColumnsDefaultsAndGetSampleBlock(ASTPtr default_expr_list, const N auto syntax_analyzer_result = TreeRewriter(context).analyze(default_expr_list, all_columns); const auto actions = ExpressionAnalyzer(default_expr_list, syntax_analyzer_result, context).getActions(true); for (const auto & action : actions->getActions()) - if (action.type == ExpressionAction::Type::ARRAY_JOIN) + if (action.node->type == ActionsDAG::Type::ARRAY_JOIN) throw Exception("Unsupported default value that requires ARRAY JOIN action", ErrorCodes::THERE_IS_NO_DEFAULT_VALUE); return actions->getSampleBlock(); diff --git a/src/Storages/IndicesDescription.cpp b/src/Storages/IndicesDescription.cpp index 143d97cdc15..8adf2be1bd4 100644 --- a/src/Storages/IndicesDescription.cpp +++ b/src/Storages/IndicesDescription.cpp @@ -31,7 +31,7 @@ IndexDescription::IndexDescription(const IndexDescription & other) , granularity(other.granularity) { if (other.expression) - expression = std::make_shared(*other.expression); + expression = other.expression->clone(); } @@ -54,7 +54,7 @@ IndexDescription & IndexDescription::operator=(const IndexDescription & other) type = other.type; if (other.expression) - expression = std::make_shared(*other.expression); + expression = other.expression->clone(); else expression.reset(); diff --git a/src/Storages/KeyDescription.cpp b/src/Storages/KeyDescription.cpp index 533736d19ed..e6cd72e51b3 100644 --- a/src/Storages/KeyDescription.cpp +++ b/src/Storages/KeyDescription.cpp @@ -25,7 +25,7 @@ KeyDescription::KeyDescription(const KeyDescription & other) , additional_column(other.additional_column) { if (other.expression) - expression = std::make_shared(*other.expression); + expression = other.expression->clone(); } KeyDescription & KeyDescription::operator=(const KeyDescription & other) @@ -45,7 +45,7 @@ KeyDescription & KeyDescription::operator=(const KeyDescription & other) if (other.expression) - expression = std::make_shared(*other.expression); + expression = other.expression->clone(); else expression.reset(); diff --git a/src/Storages/MergeTree/KeyCondition.cpp b/src/Storages/MergeTree/KeyCondition.cpp index 7cb872f174a..39ee4684af9 100644 --- a/src/Storages/MergeTree/KeyCondition.cpp +++ b/src/Storages/MergeTree/KeyCondition.cpp @@ -569,7 +569,7 @@ bool KeyCondition::canConstantBeWrappedByMonotonicFunctions( return false; bool found_transformation = false; - for (const ExpressionAction & action : key_expr->getActions()) + for (const auto & action : key_expr->getActions()) { /** The key functional expression constraint may be inferred from a plain column in the expression. * For example, if the key contains `toStartOfHour(Timestamp)` and query contains `WHERE Timestamp >= now()`, @@ -581,25 +581,25 @@ bool KeyCondition::canConstantBeWrappedByMonotonicFunctions( * Instead, we can qualify only functions that do not transform the range (for example rounding), * which while not strictly monotonic, are monotonic everywhere on the input range. */ - const auto & argument_names = action.argument_names; - if (action.type == ExpressionAction::Type::APPLY_FUNCTION - && argument_names.size() == 1 - && argument_names[0] == expr_name) + const auto & children = action.node->children; + if (action.node->type == ActionsDAG::Type::FUNCTION + && children.size() == 1 + && children[0]->result_name == expr_name) { - if (!action.function_base->hasInformationAboutMonotonicity()) + if (!action.node->function_base->hasInformationAboutMonotonicity()) return false; /// Range is irrelevant in this case. - IFunction::Monotonicity monotonicity = action.function_base->getMonotonicityForRange(*out_type, Field(), Field()); + IFunction::Monotonicity monotonicity = action.node->function_base->getMonotonicityForRange(*out_type, Field(), Field()); if (!monotonicity.is_always_monotonic) return false; /// Apply the next transformation step. std::tie(out_value, out_type) = applyFunctionForFieldOfUnknownType( - action.function_builder, + action.node->function_builder, out_type, out_value); - expr_name = action.result_name; + expr_name = action.node->result_name; /// Transformation results in a key expression, accept. auto it = key_columns.find(expr_name); diff --git a/src/Storages/MergeTree/MergeTreeBaseSelectProcessor.cpp b/src/Storages/MergeTree/MergeTreeBaseSelectProcessor.cpp index c852151f27d..8f0ad643811 100644 --- a/src/Storages/MergeTree/MergeTreeBaseSelectProcessor.cpp +++ b/src/Storages/MergeTree/MergeTreeBaseSelectProcessor.cpp @@ -40,6 +40,13 @@ MergeTreeBaseSelectProcessor::MergeTreeBaseSelectProcessor( , use_uncompressed_cache(use_uncompressed_cache_) , virt_column_names(virt_column_names_) { + if (prewhere_info) + { + if (prewhere_info->alias_actions) + prewhere_alias_actions = prewhere_info->alias_actions->buildExpressions(); + prewhere_actions = prewhere_info->prewhere_actions->buildExpressions(); + } + header_without_virtual_columns = getPort().getHeader(); for (auto it = virt_column_names.rbegin(); it != virt_column_names.rend(); ++it) @@ -74,23 +81,39 @@ void MergeTreeBaseSelectProcessor::initializeRangeReaders(MergeTreeReadTask & cu { if (reader->getColumns().empty()) { - current_task.range_reader = MergeTreeRangeReader(pre_reader.get(), nullptr, prewhere_info, true); + current_task.range_reader = MergeTreeRangeReader( + pre_reader.get(), nullptr, + prewhere_alias_actions, + prewhere_actions, + prewhere_info->prewhere_column_name, + prewhere_info->remove_prewhere_column, + prewhere_info->need_filter, + true); } else { MergeTreeRangeReader * pre_reader_ptr = nullptr; if (pre_reader != nullptr) { - current_task.pre_range_reader = MergeTreeRangeReader(pre_reader.get(), nullptr, prewhere_info, false); + current_task.pre_range_reader = MergeTreeRangeReader( + pre_reader.get(), nullptr, + prewhere_alias_actions, + prewhere_actions, + prewhere_info->prewhere_column_name, + prewhere_info->remove_prewhere_column, + prewhere_info->need_filter, + false); pre_reader_ptr = ¤t_task.pre_range_reader; } - current_task.range_reader = MergeTreeRangeReader(reader.get(), pre_reader_ptr, nullptr, true); + current_task.range_reader = MergeTreeRangeReader( + reader.get(), pre_reader_ptr, nullptr, nullptr, {}, false, false, true); } } else { - current_task.range_reader = MergeTreeRangeReader(reader.get(), nullptr, nullptr, true); + current_task.range_reader = MergeTreeRangeReader( + reader.get(), nullptr, nullptr, nullptr, {}, false, false, true); } } @@ -314,9 +337,9 @@ void MergeTreeBaseSelectProcessor::executePrewhereActions(Block & block, const P if (prewhere_info) { if (prewhere_info->alias_actions) - prewhere_info->alias_actions->execute(block); + prewhere_info->alias_actions->buildExpressions()->execute(block); - prewhere_info->prewhere_actions->execute(block); + prewhere_info->prewhere_actions->buildExpressions()->execute(block); auto & prewhere_column = block.getByName(prewhere_info->prewhere_column_name); if (!prewhere_column.type->canBeUsedInBooleanContext()) diff --git a/src/Storages/MergeTree/MergeTreeBaseSelectProcessor.h b/src/Storages/MergeTree/MergeTreeBaseSelectProcessor.h index 00ef131ae45..b44fe709f78 100644 --- a/src/Storages/MergeTree/MergeTreeBaseSelectProcessor.h +++ b/src/Storages/MergeTree/MergeTreeBaseSelectProcessor.h @@ -58,6 +58,8 @@ protected: StorageMetadataPtr metadata_snapshot; PrewhereInfoPtr prewhere_info; + ExpressionActionsPtr prewhere_alias_actions; + ExpressionActionsPtr prewhere_actions; UInt64 max_block_size_rows; UInt64 preferred_block_size_bytes; diff --git a/src/Storages/MergeTree/MergeTreeBlockReadUtils.cpp b/src/Storages/MergeTree/MergeTreeBlockReadUtils.cpp index 739dfedfde4..24f626e51df 100644 --- a/src/Storages/MergeTree/MergeTreeBlockReadUtils.cpp +++ b/src/Storages/MergeTree/MergeTreeBlockReadUtils.cpp @@ -260,9 +260,9 @@ MergeTreeReadTaskColumns getReadTaskColumns( if (prewhere_info) { if (prewhere_info->alias_actions) - pre_column_names = prewhere_info->alias_actions->getRequiredColumns(); + pre_column_names = prewhere_info->alias_actions->getRequiredColumns().getNames(); else - pre_column_names = prewhere_info->prewhere_actions->getRequiredColumns(); + pre_column_names = prewhere_info->prewhere_actions->getRequiredColumns().getNames(); if (pre_column_names.empty()) pre_column_names.push_back(column_names[0]); diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index cd776a661ed..c57d71dd31b 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -256,14 +256,14 @@ StoragePolicyPtr MergeTreeData::getStoragePolicy() const static void checkKeyExpression(const ExpressionActions & expr, const Block & sample_block, const String & key_name, bool allow_nullable_key) { - for (const ExpressionAction & action : expr.getActions()) + for (const auto & action : expr.getActions()) { - if (action.type == ExpressionAction::ARRAY_JOIN) + if (action.node->type == ActionsDAG::Type::ARRAY_JOIN) throw Exception(key_name + " key cannot contain array joins", ErrorCodes::ILLEGAL_COLUMN); - if (action.type == ExpressionAction::APPLY_FUNCTION) + if (action.node->type == ActionsDAG::Type::FUNCTION) { - IFunctionBase & func = *action.function_base; + IFunctionBase & func = *action.node->function_base; if (!func.isDeterministic()) throw Exception(key_name + " key cannot contain non-deterministic functions, " "but contains function " + func.getName(), @@ -437,7 +437,7 @@ void MergeTreeData::checkPartitionKeyAndInitMinMax(const KeyDescription & new_pa /// Add all columns used in the partition key to the min-max index. const NamesAndTypesList & minmax_idx_columns_with_types = new_partition_key.expression->getRequiredColumnsWithTypes(); - minmax_idx_expr = std::make_shared(minmax_idx_columns_with_types, global_context); + minmax_idx_expr = std::make_shared(minmax_idx_columns_with_types)->buildExpressions(); for (const NameAndTypePair & column : minmax_idx_columns_with_types) { minmax_idx_columns.emplace_back(column.name); @@ -1401,10 +1401,10 @@ void MergeTreeData::checkAlterIsPossible(const AlterCommands & commands, const S { /// Forbid altering columns inside partition key expressions because it can change partition ID format. auto partition_key_expr = old_metadata.getPartitionKey().expression; - for (const ExpressionAction & action : partition_key_expr->getActions()) + for (const auto & action : partition_key_expr->getActions()) { - auto action_columns = action.getNeededColumns(); - columns_alter_type_forbidden.insert(action_columns.begin(), action_columns.end()); + for (const auto * child : action.node->children) + columns_alter_type_forbidden.insert(child->result_name); } /// But allow to alter columns without expressions under certain condition. @@ -1421,10 +1421,10 @@ void MergeTreeData::checkAlterIsPossible(const AlterCommands & commands, const S if (old_metadata.hasSortingKey()) { auto sorting_key_expr = old_metadata.getSortingKey().expression; - for (const ExpressionAction & action : sorting_key_expr->getActions()) + for (const auto & action : sorting_key_expr->getActions()) { - auto action_columns = action.getNeededColumns(); - columns_alter_type_forbidden.insert(action_columns.begin(), action_columns.end()); + for (const auto * child : action.node->children) + columns_alter_type_forbidden.insert(child->result_name); } for (const String & col : sorting_key_expr->getRequiredColumns()) columns_alter_type_metadata_only.insert(col); diff --git a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp index 8c1dc845d26..b4d08b2b718 100644 --- a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp +++ b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp @@ -706,7 +706,7 @@ Pipe MergeTreeDataSelectExecutor::readFromParts( /// Projection, that needed to drop columns, which have appeared by execution /// of some extra expressions, and to allow execute the same expressions later. /// NOTE: It may lead to double computation of expressions. - ExpressionActionsPtr result_projection; + ActionsDAGPtr result_projection; if (select.final()) { @@ -784,9 +784,10 @@ Pipe MergeTreeDataSelectExecutor::readFromParts( if (result_projection) { - res.addSimpleTransform([&result_projection](const Block & header) + auto result_projection_actions = result_projection->buildExpressions(); + res.addSimpleTransform([&result_projection_actions](const Block & header) { - return std::make_shared(header, result_projection); + return std::make_shared(header, result_projection_actions); }); } @@ -802,9 +803,10 @@ Pipe MergeTreeDataSelectExecutor::readFromParts( if (query_info.prewhere_info && query_info.prewhere_info->remove_columns_actions) { - res.addSimpleTransform([&query_info](const Block & header) + auto remove_actions = query_info.prewhere_info->remove_columns_actions->buildExpressions(); + res.addSimpleTransform([&remove_actions](const Block & header) { - return std::make_shared(header, query_info.prewhere_info->remove_columns_actions); + return std::make_shared(header, remove_actions); }); } @@ -956,11 +958,12 @@ Pipe MergeTreeDataSelectExecutor::spreadMarkRangesAmongStreams( } } -static ExpressionActionsPtr createProjection(const Pipe & pipe, const MergeTreeData & data) +static ActionsDAGPtr createProjection(const Pipe & pipe) { const auto & header = pipe.getHeader(); - auto projection = std::make_shared(header.getNamesAndTypesList(), data.global_context); - projection->add(ExpressionAction::project(header.getNames())); + auto projection = std::make_shared(header.getNamesAndTypesList()); + projection->removeUnusedActions(header.getNames()); + projection->projectInput(); return projection; } @@ -976,7 +979,7 @@ Pipe MergeTreeDataSelectExecutor::spreadMarkRangesAmongStreamsWithOrder( const Names & virt_columns, const Settings & settings, const MergeTreeReaderSettings & reader_settings, - ExpressionActionsPtr & out_projection) const + ActionsDAGPtr & out_projection) const { size_t sum_marks = 0; const InputOrderInfoPtr & input_order_info = query_info.input_order_info; @@ -1182,7 +1185,7 @@ Pipe MergeTreeDataSelectExecutor::spreadMarkRangesAmongStreamsWithOrder( input_order_info->direction, 1); /// Drop temporary columns, added by 'sorting_key_prefix_expr' - out_projection = createProjection(pipe, data); + out_projection = createProjection(pipe); pipe.addSimpleTransform([sorting_key_prefix_expr](const Block & header) { return std::make_shared(header, sorting_key_prefix_expr); @@ -1210,7 +1213,7 @@ Pipe MergeTreeDataSelectExecutor::spreadMarkRangesAmongStreamsFinal( const Names & virt_columns, const Settings & settings, const MergeTreeReaderSettings & reader_settings, - ExpressionActionsPtr & out_projection) const + ActionsDAGPtr & out_projection) const { const auto data_settings = data.getSettings(); size_t sum_marks = 0; @@ -1259,7 +1262,7 @@ Pipe MergeTreeDataSelectExecutor::spreadMarkRangesAmongStreamsFinal( /// Drop temporary columns, added by 'sorting_key_expr' if (!out_projection) - out_projection = createProjection(pipe, data); + out_projection = createProjection(pipe); pipe.addSimpleTransform([&metadata_snapshot](const Block & header) { diff --git a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.h b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.h index 5894d6e044b..2217d570a56 100644 --- a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.h +++ b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.h @@ -73,7 +73,7 @@ private: const Names & virt_columns, const Settings & settings, const MergeTreeReaderSettings & reader_settings, - ExpressionActionsPtr & out_projection) const; + ActionsDAGPtr & out_projection) const; Pipe spreadMarkRangesAmongStreamsFinal( RangesInDataParts && parts, @@ -86,7 +86,7 @@ private: const Names & virt_columns, const Settings & settings, const MergeTreeReaderSettings & reader_settings, - ExpressionActionsPtr & out_projection) const; + ActionsDAGPtr & out_projection) const; /// Get the approximate value (bottom estimate - only by full marks) of the number of rows falling under the index. size_t getApproximateTotalRowsToRead( diff --git a/src/Storages/MergeTree/MergeTreeRangeReader.cpp b/src/Storages/MergeTree/MergeTreeRangeReader.cpp index 726e405b3e8..8dcd7fa688f 100644 --- a/src/Storages/MergeTree/MergeTreeRangeReader.cpp +++ b/src/Storages/MergeTree/MergeTreeRangeReader.cpp @@ -489,11 +489,20 @@ size_t MergeTreeRangeReader::ReadResult::countBytesInResultFilter(const IColumn: MergeTreeRangeReader::MergeTreeRangeReader( IMergeTreeReader * merge_tree_reader_, MergeTreeRangeReader * prev_reader_, - const PrewhereInfoPtr & prewhere_, + ExpressionActionsPtr prewhere_alias_actions_, + ExpressionActionsPtr prewhere_actions_, + String prewhere_column_name_, + bool remove_prewhere_column_, + bool prewhere_need_filter_, bool last_reader_in_chain_) : merge_tree_reader(merge_tree_reader_) , index_granularity(&(merge_tree_reader->data_part->index_granularity)), prev_reader(prev_reader_) - , prewhere(prewhere_), last_reader_in_chain(last_reader_in_chain_), is_initialized(true) + , prewhere_alias_actions(std::move(prewhere_alias_actions_)) + , prewhere_actions(std::move(prewhere_actions_)) + , prewhere_column_name(std::move(prewhere_column_name_)) + , remove_prewhere_column(remove_prewhere_column_) + , prewhere_need_filter(prewhere_need_filter_) + , last_reader_in_chain(last_reader_in_chain_), is_initialized(true) { if (prev_reader) sample_block = prev_reader->getSampleBlock(); @@ -501,16 +510,16 @@ MergeTreeRangeReader::MergeTreeRangeReader( for (const auto & name_and_type : merge_tree_reader->getColumns()) sample_block.insert({name_and_type.type->createColumn(), name_and_type.type, name_and_type.name}); - if (prewhere) + if (prewhere_actions) { - if (prewhere->alias_actions) - prewhere->alias_actions->execute(sample_block, true); + if (prewhere_alias_actions) + prewhere_alias_actions->execute(sample_block, true); - if (prewhere->prewhere_actions) - prewhere->prewhere_actions->execute(sample_block, true); + if (prewhere_actions) + prewhere_actions->execute(sample_block, true); - if (prewhere->remove_prewhere_column) - sample_block.erase(prewhere->prewhere_column_name); + if (remove_prewhere_column) + sample_block.erase(prewhere_column_name); } } @@ -794,7 +803,7 @@ Columns MergeTreeRangeReader::continueReadingChain(ReadResult & result, size_t & void MergeTreeRangeReader::executePrewhereActionsAndFilterColumns(ReadResult & result) { - if (!prewhere) + if (!prewhere_actions) return; const auto & header = merge_tree_reader->getColumns(); @@ -825,14 +834,14 @@ void MergeTreeRangeReader::executePrewhereActionsAndFilterColumns(ReadResult & r for (auto name_and_type = header.begin(); pos < num_columns; ++pos, ++name_and_type) block.insert({result.columns[pos], name_and_type->type, name_and_type->name}); - if (prewhere->alias_actions) - prewhere->alias_actions->execute(block); + if (prewhere_alias_actions) + prewhere_alias_actions->execute(block); /// Columns might be projected out. We need to store them here so that default columns can be evaluated later. result.block_before_prewhere = block; - prewhere->prewhere_actions->execute(block); + prewhere_actions->execute(block); - prewhere_column_pos = block.getPositionByName(prewhere->prewhere_column_name); + prewhere_column_pos = block.getPositionByName(prewhere_column_name); result.columns.clear(); result.columns.reserve(block.columns()); @@ -860,7 +869,7 @@ void MergeTreeRangeReader::executePrewhereActionsAndFilterColumns(ReadResult & r if (result.totalRowsPerGranule() == 0) result.setFilterConstFalse(); /// If we need to filter in PREWHERE - else if (prewhere->need_filter || result.need_filter) + else if (prewhere_need_filter || result.need_filter) { /// If there is a filter and without optimized if (result.getFilter() && last_reader_in_chain) @@ -901,11 +910,11 @@ void MergeTreeRangeReader::executePrewhereActionsAndFilterColumns(ReadResult & r /// Check if the PREWHERE column is needed if (!result.columns.empty()) { - if (prewhere->remove_prewhere_column) + if (remove_prewhere_column) result.columns.erase(result.columns.begin() + prewhere_column_pos); else result.columns[prewhere_column_pos] = - getSampleBlock().getByName(prewhere->prewhere_column_name).type-> + getSampleBlock().getByName(prewhere_column_name).type-> createColumnConst(result.num_rows, 1u)->convertToFullColumnIfConst(); } } @@ -913,7 +922,7 @@ void MergeTreeRangeReader::executePrewhereActionsAndFilterColumns(ReadResult & r else { result.columns[prewhere_column_pos] = result.getFilterHolder()->convertToFullColumnIfConst(); - if (getSampleBlock().getByName(prewhere->prewhere_column_name).type->isNullable()) + if (getSampleBlock().getByName(prewhere_column_name).type->isNullable()) result.columns[prewhere_column_pos] = makeNullable(std::move(result.columns[prewhere_column_pos])); result.clearFilter(); // Acting as a flag to not filter in PREWHERE } diff --git a/src/Storages/MergeTree/MergeTreeRangeReader.h b/src/Storages/MergeTree/MergeTreeRangeReader.h index 381b87ecffd..d41ca6b56eb 100644 --- a/src/Storages/MergeTree/MergeTreeRangeReader.h +++ b/src/Storages/MergeTree/MergeTreeRangeReader.h @@ -24,7 +24,11 @@ public: MergeTreeRangeReader( IMergeTreeReader * merge_tree_reader_, MergeTreeRangeReader * prev_reader_, - const PrewhereInfoPtr & prewhere_, + ExpressionActionsPtr prewhere_alias_actions_, + ExpressionActionsPtr prewhere_actions_, + String prewhere_column_name_, + bool remove_prewhere_column_, + bool prewhere_need_filter_, bool last_reader_in_chain_); MergeTreeRangeReader() = default; @@ -217,7 +221,12 @@ private: IMergeTreeReader * merge_tree_reader = nullptr; const MergeTreeIndexGranularity * index_granularity = nullptr; MergeTreeRangeReader * prev_reader = nullptr; /// If not nullptr, read from prev_reader firstly. - PrewhereInfoPtr prewhere; + + ExpressionActionsPtr prewhere_alias_actions; + ExpressionActionsPtr prewhere_actions; + String prewhere_column_name; + bool remove_prewhere_column; + bool prewhere_need_filter; Stream stream; diff --git a/src/Storages/ReadInOrderOptimizer.cpp b/src/Storages/ReadInOrderOptimizer.cpp index 37f07ad1876..48412f3dfa0 100644 --- a/src/Storages/ReadInOrderOptimizer.cpp +++ b/src/Storages/ReadInOrderOptimizer.cpp @@ -72,7 +72,7 @@ InputOrderInfoPtr ReadInOrderOptimizer::getInputOrder(const StoragePtr & storage bool found_function = false; for (const auto & action : elements_actions[i]->getActions()) { - if (action.type != ExpressionAction::APPLY_FUNCTION) + if (action.node->type != ActionsDAG::Type::FUNCTION) continue; if (found_function) @@ -83,13 +83,13 @@ InputOrderInfoPtr ReadInOrderOptimizer::getInputOrder(const StoragePtr & storage else found_function = true; - if (action.argument_names.size() != 1 || action.argument_names.at(0) != sorting_key_columns[i]) + if (action.node->children.size() != 1 || action.node->children.at(0)->result_name != sorting_key_columns[i]) { current_direction = 0; break; } - const auto & func = *action.function_base; + const auto & func = *action.node->function_base; if (!func.hasInformationAboutMonotonicity()) { current_direction = 0; diff --git a/src/Storages/SelectQueryInfo.h b/src/Storages/SelectQueryInfo.h index 9a5d0cc6338..cbf2c816d87 100644 --- a/src/Storages/SelectQueryInfo.h +++ b/src/Storages/SelectQueryInfo.h @@ -12,27 +12,30 @@ namespace DB class ExpressionActions; using ExpressionActionsPtr = std::shared_ptr; +class ActionsDAG; +using ActionsDAGPtr = std::shared_ptr; + struct PrewhereInfo { /// Actions which are executed in order to alias columns are used for prewhere actions. - ExpressionActionsPtr alias_actions; + ActionsDAGPtr alias_actions; /// Actions which are executed on block in order to get filter column for prewhere step. - ExpressionActionsPtr prewhere_actions; + ActionsDAGPtr prewhere_actions; /// Actions which are executed after reading from storage in order to remove unused columns. - ExpressionActionsPtr remove_columns_actions; + ActionsDAGPtr remove_columns_actions; String prewhere_column_name; bool remove_prewhere_column = false; bool need_filter = false; PrewhereInfo() = default; - explicit PrewhereInfo(ExpressionActionsPtr prewhere_actions_, String prewhere_column_name_) + explicit PrewhereInfo(ActionsDAGPtr prewhere_actions_, String prewhere_column_name_) : prewhere_actions(std::move(prewhere_actions_)), prewhere_column_name(std::move(prewhere_column_name_)) {} }; /// Helper struct to store all the information about the filter expression. struct FilterInfo { - ExpressionActionsPtr actions; + ActionsDAGPtr actions; String column_name; bool do_remove_column = false; }; diff --git a/src/Storages/StorageBuffer.cpp b/src/Storages/StorageBuffer.cpp index cf1eaa1f46d..a4d78865f4f 100644 --- a/src/Storages/StorageBuffer.cpp +++ b/src/Storages/StorageBuffer.cpp @@ -278,7 +278,7 @@ Pipe StorageBuffer::read( pipe_from_buffers.addSimpleTransform([&](const Block & header) { return std::make_shared( - header, query_info.prewhere_info->prewhere_actions, + header, query_info.prewhere_info->prewhere_actions->buildExpressions(), query_info.prewhere_info->prewhere_column_name, query_info.prewhere_info->remove_prewhere_column); }); @@ -286,7 +286,7 @@ Pipe StorageBuffer::read( { pipe_from_buffers.addSimpleTransform([&](const Block & header) { - return std::make_shared(header, query_info.prewhere_info->alias_actions); + return std::make_shared(header, query_info.prewhere_info->alias_actions->buildExpressions()); }); } } diff --git a/src/Storages/StorageDistributed.cpp b/src/Storages/StorageDistributed.cpp index b858239d637..be3b70a5e44 100644 --- a/src/Storages/StorageDistributed.cpp +++ b/src/Storages/StorageDistributed.cpp @@ -201,9 +201,9 @@ bool isExpressionActionsDeterministics(const ExpressionActionsPtr & actions) { for (const auto & action : actions->getActions()) { - if (action.type != ExpressionAction::APPLY_FUNCTION) + if (action.node->type != ActionsDAG::Type::FUNCTION) continue; - if (!action.function_base->isDeterministic()) + if (!action.node->function_base->isDeterministic()) return false; } return true; diff --git a/src/Storages/TTLDescription.cpp b/src/Storages/TTLDescription.cpp index 7f55badf819..50a74ddf7f0 100644 --- a/src/Storages/TTLDescription.cpp +++ b/src/Storages/TTLDescription.cpp @@ -30,7 +30,7 @@ TTLAggregateDescription::TTLAggregateDescription(const TTLAggregateDescription & , expression_result_column_name(other.expression_result_column_name) { if (other.expression) - expression = std::make_shared(*other.expression); + expression = other.expression->clone(); } TTLAggregateDescription & TTLAggregateDescription::operator=(const TTLAggregateDescription & other) @@ -41,7 +41,7 @@ TTLAggregateDescription & TTLAggregateDescription::operator=(const TTLAggregateD column_name = other.column_name; expression_result_column_name = other.expression_result_column_name; if (other.expression) - expression = std::make_shared(*other.expression); + expression = other.expression->clone(); else expression.reset(); return *this; @@ -54,9 +54,9 @@ void checkTTLExpression(const ExpressionActionsPtr & ttl_expression, const Strin { for (const auto & action : ttl_expression->getActions()) { - if (action.type == ExpressionAction::APPLY_FUNCTION) + if (action.node->type == ActionsDAG::Type::FUNCTION) { - IFunctionBase & func = *action.function_base; + IFunctionBase & func = *action.node->function_base; if (!func.isDeterministic()) throw Exception( "TTL expression cannot contain non-deterministic functions, " @@ -92,10 +92,10 @@ TTLDescription::TTLDescription(const TTLDescription & other) , recompression_codec(other.recompression_codec) { if (other.expression) - expression = std::make_shared(*other.expression); + expression = other.expression->clone(); if (other.where_expression) - where_expression = std::make_shared(*other.where_expression); + where_expression = other.where_expression->clone(); } TTLDescription & TTLDescription::operator=(const TTLDescription & other) @@ -110,13 +110,13 @@ TTLDescription & TTLDescription::operator=(const TTLDescription & other) expression_ast.reset(); if (other.expression) - expression = std::make_shared(*other.expression); + expression = other.expression->clone(); else expression.reset(); result_column = other.result_column; if (other.where_expression) - where_expression = std::make_shared(*other.where_expression); + where_expression = other.where_expression->clone(); else where_expression.reset(); diff --git a/tests/queries/0_stateless/00526_array_join_with_arrays_of_nullable.reference b/tests/queries/0_stateless/00526_array_join_with_arrays_of_nullable.reference index e48bca390db..b0c7614fe30 100644 --- a/tests/queries/0_stateless/00526_array_join_with_arrays_of_nullable.reference +++ b/tests/queries/0_stateless/00526_array_join_with_arrays_of_nullable.reference @@ -1,9 +1,9 @@ 1 (NULL,'') a -1 (NULL,'') \N 1 (NULL,'') b -\N (123,'Hello') a -\N (123,'Hello') \N -\N (123,'Hello') b +1 (NULL,'') \N 3 (456,NULL) a -3 (456,NULL) \N 3 (456,NULL) b +3 (456,NULL) \N +\N (123,'Hello') a +\N (123,'Hello') b +\N (123,'Hello') \N diff --git a/tests/queries/0_stateless/00526_array_join_with_arrays_of_nullable.sql b/tests/queries/0_stateless/00526_array_join_with_arrays_of_nullable.sql index ce0894ba107..50a8e2b7d74 100644 --- a/tests/queries/0_stateless/00526_array_join_with_arrays_of_nullable.sql +++ b/tests/queries/0_stateless/00526_array_join_with_arrays_of_nullable.sql @@ -1 +1 @@ -SELECT x, y, arrayJoin(['a', NULL, 'b']) AS z FROM system.one ARRAY JOIN [1, NULL, 3] AS x, [(NULL, ''), (123, 'Hello'), (456, NULL)] AS y; +SELECT x, y, arrayJoin(['a', NULL, 'b']) AS z FROM system.one ARRAY JOIN [1, NULL, 3] AS x, [(NULL, ''), (123, 'Hello'), (456, NULL)] AS y order by x, y, z; diff --git a/tests/queries/0_stateless/01508_explain_header.reference b/tests/queries/0_stateless/01508_explain_header.reference index 50216432e14..2b818ca9547 100644 --- a/tests/queries/0_stateless/01508_explain_header.reference +++ b/tests/queries/0_stateless/01508_explain_header.reference @@ -1,7 +1,7 @@ Expression (Projection) Header: x UInt8 Expression (Before ORDER BY and SELECT) - Header: _dummy UInt8 + Header: dummy UInt8 1 UInt8 ReadFromStorage (Read from SystemOne) Header: dummy UInt8 From feccf47f2741107ab47b03c6e3c90b85788bc7a1 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 3 Nov 2020 16:30:55 +0300 Subject: [PATCH 017/205] Fxi build. --- src/Interpreters/ExpressionActions.cpp | 2 +- src/Interpreters/ExpressionActions.h | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Interpreters/ExpressionActions.cpp b/src/Interpreters/ExpressionActions.cpp index 13f30ce678c..4a448e9f463 100644 --- a/src/Interpreters/ExpressionActions.cpp +++ b/src/Interpreters/ExpressionActions.cpp @@ -1442,7 +1442,7 @@ ExpressionActionsPtr ActionsDAG::linearizeActions() const return expressions; } -ExpressionActionsPtr ActionsDAG::buildExpressions() +ExpressionActionsPtr ActionsDAG::buildExpressions() const { auto cloned = clone(); auto expressions = cloned->linearizeActions(); diff --git a/src/Interpreters/ExpressionActions.h b/src/Interpreters/ExpressionActions.h index c7011dd7d86..fefe64c8374 100644 --- a/src/Interpreters/ExpressionActions.h +++ b/src/Interpreters/ExpressionActions.h @@ -213,7 +213,7 @@ public: void projectInput() { project_input = true; } void removeUnusedActions(const Names & required_names); - ExpressionActionsPtr buildExpressions(); + ExpressionActionsPtr buildExpressions() const; /// Splits actions into two parts. Returned half may be swapped with ARRAY JOIN. /// Returns nullptr if no actions may be moved before ARRAY JOIN. From 9f7de6a1a4f5c4405d4a0755037865fe96ab762e Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 3 Nov 2020 17:26:47 +0300 Subject: [PATCH 018/205] Try fix header for TTLBlockInputStream. --- src/DataStreams/TTLBlockInputStream.cpp | 11 ++++++++++- 1 file changed, 10 insertions(+), 1 deletion(-) diff --git a/src/DataStreams/TTLBlockInputStream.cpp b/src/DataStreams/TTLBlockInputStream.cpp index c1c482b1a73..38479409f84 100644 --- a/src/DataStreams/TTLBlockInputStream.cpp +++ b/src/DataStreams/TTLBlockInputStream.cpp @@ -103,6 +103,15 @@ bool TTLBlockInputStream::isTTLExpired(time_t ttl) const return (ttl && (ttl <= current_time)); } +Block reorderColumns(Block block, const Block & header) +{ + Block res; + for (const auto & col : header) + res.insert(block.getByName(col.name)); + + return res; +} + Block TTLBlockInputStream::readImpl() { /// Skip all data if table ttl is expired for part @@ -136,7 +145,7 @@ Block TTLBlockInputStream::readImpl() updateMovesTTL(block); updateRecompressionTTL(block); - return block; + return reorderColumns(std::move(block), header); } void TTLBlockInputStream::readSuffixImpl() From 36f9bcdb52ed2868506168b3bc4a50ec04d7c3f1 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 3 Nov 2020 19:00:34 +0300 Subject: [PATCH 019/205] Fix build. --- src/Interpreters/ExpressionActions.h | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Interpreters/ExpressionActions.h b/src/Interpreters/ExpressionActions.h index fefe64c8374..4dd6efbead9 100644 --- a/src/Interpreters/ExpressionActions.h +++ b/src/Interpreters/ExpressionActions.h @@ -278,8 +278,8 @@ private: struct ExecutionContext { ColumnsWithTypeAndName & inputs; - ColumnsWithTypeAndName columns; - std::vector inputs_pos; + ColumnsWithTypeAndName columns = {}; + std::vector inputs_pos = {}; size_t num_rows; }; From 7b61f5d64151882df79e7254e0ef77f0438eb001 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 3 Nov 2020 22:05:47 +0300 Subject: [PATCH 020/205] Refactor more --- src/Interpreters/ExpressionAnalyzer.cpp | 2 +- src/Interpreters/ExpressionAnalyzer.h | 2 +- src/Interpreters/InterpreterSelectQuery.cpp | 18 +++++++- src/Interpreters/InterpreterSelectQuery.h | 2 +- .../MergeTreeBaseSelectProcessor.cpp | 35 +++------------ .../MergeTree/MergeTreeBaseSelectProcessor.h | 2 - .../MergeTree/MergeTreeRangeReader.cpp | 45 ++++++++----------- src/Storages/MergeTree/MergeTreeRangeReader.h | 13 +----- src/Storages/SelectQueryInfo.h | 24 ++++++++-- 9 files changed, 65 insertions(+), 78 deletions(-) diff --git a/src/Interpreters/ExpressionAnalyzer.cpp b/src/Interpreters/ExpressionAnalyzer.cpp index a5d57a87609..9c7db6c1a20 100644 --- a/src/Interpreters/ExpressionAnalyzer.cpp +++ b/src/Interpreters/ExpressionAnalyzer.cpp @@ -1126,7 +1126,7 @@ ExpressionAnalysisResult::ExpressionAnalysisResult( if (auto actions = query_analyzer.appendPrewhere(chain, !first_stage, additional_required_columns_after_prewhere)) { - prewhere_info = std::make_shared(actions, query.prewhere()->getColumnName()); + prewhere_info = std::make_shared(actions, query.prewhere()->getColumnName()); if (allowEarlyConstantFolding(*prewhere_info->prewhere_actions, settings)) { diff --git a/src/Interpreters/ExpressionAnalyzer.h b/src/Interpreters/ExpressionAnalyzer.h index 95a65ee2bfa..aae1a2766d8 100644 --- a/src/Interpreters/ExpressionAnalyzer.h +++ b/src/Interpreters/ExpressionAnalyzer.h @@ -200,7 +200,7 @@ struct ExpressionAnalysisResult /// Columns will be removed after prewhere actions execution. NameSet columns_to_remove_after_prewhere; - PrewhereInfoPtr prewhere_info; + PrewhereDAGInfoPtr prewhere_info; FilterInfoPtr filter_info; ConstantFilterDescription prewhere_constant_filter_description; ConstantFilterDescription where_constant_filter_description; diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index 349fd926402..4b9463f8c67 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -1108,7 +1108,7 @@ static StreamLocalLimits getLimitsForStorage(const Settings & settings, const Se void InterpreterSelectQuery::executeFetchColumns( QueryProcessingStage::Enum processing_stage, QueryPlan & query_plan, - const PrewhereInfoPtr & prewhere_info, const NameSet & columns_to_remove_after_prewhere) + const PrewhereDAGInfoPtr & prewhere_info, const NameSet & columns_to_remove_after_prewhere) { auto & query = getSelectQuery(); const Settings & settings = context->getSettingsRef(); @@ -1430,7 +1430,21 @@ void InterpreterSelectQuery::executeFetchColumns( query_info.query = query_ptr; query_info.syntax_analyzer_result = syntax_analyzer_result; query_info.sets = query_analyzer->getPreparedSets(); - query_info.prewhere_info = prewhere_info; + + if (prewhere_info) + { + query_info.prewhere_info = std::make_shared( + prewhere_info->prewhere_actions->buildExpressions(), + prewhere_info->prewhere_column_name); + + if (prewhere_info->alias_actions) + query_info.prewhere_info->alias_actions = prewhere_info->alias_actions->buildExpressions(); + if (prewhere_info->remove_columns_actions) + query_info.prewhere_info->remove_columns_actions = prewhere_info->remove_columns_actions->buildExpressions(); + + query_info.prewhere_info->remove_prewhere_column = prewhere_info->remove_prewhere_column; + query_info.prewhere_info->need_filter = prewhere_info->need_filter; + } /// Create optimizer with prepared actions. /// Maybe we will need to calc input_order_info later, e.g. while reading from StorageMerge. diff --git a/src/Interpreters/InterpreterSelectQuery.h b/src/Interpreters/InterpreterSelectQuery.h index 9623f1eb5a0..4d56bfdbc34 100644 --- a/src/Interpreters/InterpreterSelectQuery.h +++ b/src/Interpreters/InterpreterSelectQuery.h @@ -116,7 +116,7 @@ private: void executeFetchColumns( QueryProcessingStage::Enum processing_stage, QueryPlan & query_plan, - const PrewhereInfoPtr & prewhere_info, + const PrewhereDAGInfoPtr & prewhere_info, const NameSet & columns_to_remove_after_prewhere); void executeWhere(QueryPlan & query_plan, const ActionsDAGPtr & expression, bool remove_filter); diff --git a/src/Storages/MergeTree/MergeTreeBaseSelectProcessor.cpp b/src/Storages/MergeTree/MergeTreeBaseSelectProcessor.cpp index 8f0ad643811..c852151f27d 100644 --- a/src/Storages/MergeTree/MergeTreeBaseSelectProcessor.cpp +++ b/src/Storages/MergeTree/MergeTreeBaseSelectProcessor.cpp @@ -40,13 +40,6 @@ MergeTreeBaseSelectProcessor::MergeTreeBaseSelectProcessor( , use_uncompressed_cache(use_uncompressed_cache_) , virt_column_names(virt_column_names_) { - if (prewhere_info) - { - if (prewhere_info->alias_actions) - prewhere_alias_actions = prewhere_info->alias_actions->buildExpressions(); - prewhere_actions = prewhere_info->prewhere_actions->buildExpressions(); - } - header_without_virtual_columns = getPort().getHeader(); for (auto it = virt_column_names.rbegin(); it != virt_column_names.rend(); ++it) @@ -81,39 +74,23 @@ void MergeTreeBaseSelectProcessor::initializeRangeReaders(MergeTreeReadTask & cu { if (reader->getColumns().empty()) { - current_task.range_reader = MergeTreeRangeReader( - pre_reader.get(), nullptr, - prewhere_alias_actions, - prewhere_actions, - prewhere_info->prewhere_column_name, - prewhere_info->remove_prewhere_column, - prewhere_info->need_filter, - true); + current_task.range_reader = MergeTreeRangeReader(pre_reader.get(), nullptr, prewhere_info, true); } else { MergeTreeRangeReader * pre_reader_ptr = nullptr; if (pre_reader != nullptr) { - current_task.pre_range_reader = MergeTreeRangeReader( - pre_reader.get(), nullptr, - prewhere_alias_actions, - prewhere_actions, - prewhere_info->prewhere_column_name, - prewhere_info->remove_prewhere_column, - prewhere_info->need_filter, - false); + current_task.pre_range_reader = MergeTreeRangeReader(pre_reader.get(), nullptr, prewhere_info, false); pre_reader_ptr = ¤t_task.pre_range_reader; } - current_task.range_reader = MergeTreeRangeReader( - reader.get(), pre_reader_ptr, nullptr, nullptr, {}, false, false, true); + current_task.range_reader = MergeTreeRangeReader(reader.get(), pre_reader_ptr, nullptr, true); } } else { - current_task.range_reader = MergeTreeRangeReader( - reader.get(), nullptr, nullptr, nullptr, {}, false, false, true); + current_task.range_reader = MergeTreeRangeReader(reader.get(), nullptr, nullptr, true); } } @@ -337,9 +314,9 @@ void MergeTreeBaseSelectProcessor::executePrewhereActions(Block & block, const P if (prewhere_info) { if (prewhere_info->alias_actions) - prewhere_info->alias_actions->buildExpressions()->execute(block); + prewhere_info->alias_actions->execute(block); - prewhere_info->prewhere_actions->buildExpressions()->execute(block); + prewhere_info->prewhere_actions->execute(block); auto & prewhere_column = block.getByName(prewhere_info->prewhere_column_name); if (!prewhere_column.type->canBeUsedInBooleanContext()) diff --git a/src/Storages/MergeTree/MergeTreeBaseSelectProcessor.h b/src/Storages/MergeTree/MergeTreeBaseSelectProcessor.h index b44fe709f78..00ef131ae45 100644 --- a/src/Storages/MergeTree/MergeTreeBaseSelectProcessor.h +++ b/src/Storages/MergeTree/MergeTreeBaseSelectProcessor.h @@ -58,8 +58,6 @@ protected: StorageMetadataPtr metadata_snapshot; PrewhereInfoPtr prewhere_info; - ExpressionActionsPtr prewhere_alias_actions; - ExpressionActionsPtr prewhere_actions; UInt64 max_block_size_rows; UInt64 preferred_block_size_bytes; diff --git a/src/Storages/MergeTree/MergeTreeRangeReader.cpp b/src/Storages/MergeTree/MergeTreeRangeReader.cpp index 8dcd7fa688f..726e405b3e8 100644 --- a/src/Storages/MergeTree/MergeTreeRangeReader.cpp +++ b/src/Storages/MergeTree/MergeTreeRangeReader.cpp @@ -489,20 +489,11 @@ size_t MergeTreeRangeReader::ReadResult::countBytesInResultFilter(const IColumn: MergeTreeRangeReader::MergeTreeRangeReader( IMergeTreeReader * merge_tree_reader_, MergeTreeRangeReader * prev_reader_, - ExpressionActionsPtr prewhere_alias_actions_, - ExpressionActionsPtr prewhere_actions_, - String prewhere_column_name_, - bool remove_prewhere_column_, - bool prewhere_need_filter_, + const PrewhereInfoPtr & prewhere_, bool last_reader_in_chain_) : merge_tree_reader(merge_tree_reader_) , index_granularity(&(merge_tree_reader->data_part->index_granularity)), prev_reader(prev_reader_) - , prewhere_alias_actions(std::move(prewhere_alias_actions_)) - , prewhere_actions(std::move(prewhere_actions_)) - , prewhere_column_name(std::move(prewhere_column_name_)) - , remove_prewhere_column(remove_prewhere_column_) - , prewhere_need_filter(prewhere_need_filter_) - , last_reader_in_chain(last_reader_in_chain_), is_initialized(true) + , prewhere(prewhere_), last_reader_in_chain(last_reader_in_chain_), is_initialized(true) { if (prev_reader) sample_block = prev_reader->getSampleBlock(); @@ -510,16 +501,16 @@ MergeTreeRangeReader::MergeTreeRangeReader( for (const auto & name_and_type : merge_tree_reader->getColumns()) sample_block.insert({name_and_type.type->createColumn(), name_and_type.type, name_and_type.name}); - if (prewhere_actions) + if (prewhere) { - if (prewhere_alias_actions) - prewhere_alias_actions->execute(sample_block, true); + if (prewhere->alias_actions) + prewhere->alias_actions->execute(sample_block, true); - if (prewhere_actions) - prewhere_actions->execute(sample_block, true); + if (prewhere->prewhere_actions) + prewhere->prewhere_actions->execute(sample_block, true); - if (remove_prewhere_column) - sample_block.erase(prewhere_column_name); + if (prewhere->remove_prewhere_column) + sample_block.erase(prewhere->prewhere_column_name); } } @@ -803,7 +794,7 @@ Columns MergeTreeRangeReader::continueReadingChain(ReadResult & result, size_t & void MergeTreeRangeReader::executePrewhereActionsAndFilterColumns(ReadResult & result) { - if (!prewhere_actions) + if (!prewhere) return; const auto & header = merge_tree_reader->getColumns(); @@ -834,14 +825,14 @@ void MergeTreeRangeReader::executePrewhereActionsAndFilterColumns(ReadResult & r for (auto name_and_type = header.begin(); pos < num_columns; ++pos, ++name_and_type) block.insert({result.columns[pos], name_and_type->type, name_and_type->name}); - if (prewhere_alias_actions) - prewhere_alias_actions->execute(block); + if (prewhere->alias_actions) + prewhere->alias_actions->execute(block); /// Columns might be projected out. We need to store them here so that default columns can be evaluated later. result.block_before_prewhere = block; - prewhere_actions->execute(block); + prewhere->prewhere_actions->execute(block); - prewhere_column_pos = block.getPositionByName(prewhere_column_name); + prewhere_column_pos = block.getPositionByName(prewhere->prewhere_column_name); result.columns.clear(); result.columns.reserve(block.columns()); @@ -869,7 +860,7 @@ void MergeTreeRangeReader::executePrewhereActionsAndFilterColumns(ReadResult & r if (result.totalRowsPerGranule() == 0) result.setFilterConstFalse(); /// If we need to filter in PREWHERE - else if (prewhere_need_filter || result.need_filter) + else if (prewhere->need_filter || result.need_filter) { /// If there is a filter and without optimized if (result.getFilter() && last_reader_in_chain) @@ -910,11 +901,11 @@ void MergeTreeRangeReader::executePrewhereActionsAndFilterColumns(ReadResult & r /// Check if the PREWHERE column is needed if (!result.columns.empty()) { - if (remove_prewhere_column) + if (prewhere->remove_prewhere_column) result.columns.erase(result.columns.begin() + prewhere_column_pos); else result.columns[prewhere_column_pos] = - getSampleBlock().getByName(prewhere_column_name).type-> + getSampleBlock().getByName(prewhere->prewhere_column_name).type-> createColumnConst(result.num_rows, 1u)->convertToFullColumnIfConst(); } } @@ -922,7 +913,7 @@ void MergeTreeRangeReader::executePrewhereActionsAndFilterColumns(ReadResult & r else { result.columns[prewhere_column_pos] = result.getFilterHolder()->convertToFullColumnIfConst(); - if (getSampleBlock().getByName(prewhere_column_name).type->isNullable()) + if (getSampleBlock().getByName(prewhere->prewhere_column_name).type->isNullable()) result.columns[prewhere_column_pos] = makeNullable(std::move(result.columns[prewhere_column_pos])); result.clearFilter(); // Acting as a flag to not filter in PREWHERE } diff --git a/src/Storages/MergeTree/MergeTreeRangeReader.h b/src/Storages/MergeTree/MergeTreeRangeReader.h index d41ca6b56eb..381b87ecffd 100644 --- a/src/Storages/MergeTree/MergeTreeRangeReader.h +++ b/src/Storages/MergeTree/MergeTreeRangeReader.h @@ -24,11 +24,7 @@ public: MergeTreeRangeReader( IMergeTreeReader * merge_tree_reader_, MergeTreeRangeReader * prev_reader_, - ExpressionActionsPtr prewhere_alias_actions_, - ExpressionActionsPtr prewhere_actions_, - String prewhere_column_name_, - bool remove_prewhere_column_, - bool prewhere_need_filter_, + const PrewhereInfoPtr & prewhere_, bool last_reader_in_chain_); MergeTreeRangeReader() = default; @@ -221,12 +217,7 @@ private: IMergeTreeReader * merge_tree_reader = nullptr; const MergeTreeIndexGranularity * index_granularity = nullptr; MergeTreeRangeReader * prev_reader = nullptr; /// If not nullptr, read from prev_reader firstly. - - ExpressionActionsPtr prewhere_alias_actions; - ExpressionActionsPtr prewhere_actions; - String prewhere_column_name; - bool remove_prewhere_column; - bool prewhere_need_filter; + PrewhereInfoPtr prewhere; Stream stream; diff --git a/src/Storages/SelectQueryInfo.h b/src/Storages/SelectQueryInfo.h index cbf2c816d87..9f1d1c10a20 100644 --- a/src/Storages/SelectQueryInfo.h +++ b/src/Storages/SelectQueryInfo.h @@ -18,20 +18,35 @@ using ActionsDAGPtr = std::shared_ptr; struct PrewhereInfo { /// Actions which are executed in order to alias columns are used for prewhere actions. - ActionsDAGPtr alias_actions; + ExpressionActionsPtr alias_actions; /// Actions which are executed on block in order to get filter column for prewhere step. - ActionsDAGPtr prewhere_actions; + ExpressionActionsPtr prewhere_actions; /// Actions which are executed after reading from storage in order to remove unused columns. - ActionsDAGPtr remove_columns_actions; + ExpressionActionsPtr remove_columns_actions; String prewhere_column_name; bool remove_prewhere_column = false; bool need_filter = false; PrewhereInfo() = default; - explicit PrewhereInfo(ActionsDAGPtr prewhere_actions_, String prewhere_column_name_) + explicit PrewhereInfo(ExpressionActionsPtr prewhere_actions_, String prewhere_column_name_) : prewhere_actions(std::move(prewhere_actions_)), prewhere_column_name(std::move(prewhere_column_name_)) {} }; +/// Same as PrewhereInfo, but with ActionsDAG +struct PrewhereDAGInfo +{ + ActionsDAGPtr alias_actions; + ActionsDAGPtr prewhere_actions; + ActionsDAGPtr remove_columns_actions; + String prewhere_column_name; + bool remove_prewhere_column = false; + bool need_filter = false; + + PrewhereDAGInfo() = default; + explicit PrewhereDAGInfo(ActionsDAGPtr prewhere_actions_, String prewhere_column_name_) + : prewhere_actions(std::move(prewhere_actions_)), prewhere_column_name(std::move(prewhere_column_name_)) {} +}; + /// Helper struct to store all the information about the filter expression. struct FilterInfo { @@ -57,6 +72,7 @@ struct InputOrderInfo }; using PrewhereInfoPtr = std::shared_ptr; +using PrewhereDAGInfoPtr = std::shared_ptr; using FilterInfoPtr = std::shared_ptr; using InputOrderInfoPtr = std::shared_ptr; From 3b6806ea39402115ce0be95a5d9261cc03b3f1b9 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 3 Nov 2020 22:21:24 +0300 Subject: [PATCH 021/205] Fix build --- src/Interpreters/getHeaderForProcessingStage.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Interpreters/getHeaderForProcessingStage.cpp b/src/Interpreters/getHeaderForProcessingStage.cpp index 3a98109fdd9..db7008a1779 100644 --- a/src/Interpreters/getHeaderForProcessingStage.cpp +++ b/src/Interpreters/getHeaderForProcessingStage.cpp @@ -43,7 +43,7 @@ Block getHeaderForProcessingStage( Block header = metadata_snapshot->getSampleBlockForColumns(column_names, storage.getVirtuals(), storage.getStorageID()); if (query_info.prewhere_info) { - query_info.prewhere_info->prewhere_actions->buildExpressions()->execute(header); + query_info.prewhere_info->prewhere_actions->execute(header); if (query_info.prewhere_info->remove_prewhere_column) header.erase(query_info.prewhere_info->prewhere_column_name); } From 3e012265d3d401e183b349c25cc39e5db1db965f Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 3 Nov 2020 22:26:55 +0300 Subject: [PATCH 022/205] Fix build --- src/Storages/StorageBuffer.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Storages/StorageBuffer.cpp b/src/Storages/StorageBuffer.cpp index a4d78865f4f..cf1eaa1f46d 100644 --- a/src/Storages/StorageBuffer.cpp +++ b/src/Storages/StorageBuffer.cpp @@ -278,7 +278,7 @@ Pipe StorageBuffer::read( pipe_from_buffers.addSimpleTransform([&](const Block & header) { return std::make_shared( - header, query_info.prewhere_info->prewhere_actions->buildExpressions(), + header, query_info.prewhere_info->prewhere_actions, query_info.prewhere_info->prewhere_column_name, query_info.prewhere_info->remove_prewhere_column); }); @@ -286,7 +286,7 @@ Pipe StorageBuffer::read( { pipe_from_buffers.addSimpleTransform([&](const Block & header) { - return std::make_shared(header, query_info.prewhere_info->alias_actions->buildExpressions()); + return std::make_shared(header, query_info.prewhere_info->alias_actions); }); } } From 4984fdc693c3c79a26137a4fccecf7e88b7ee65a Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 3 Nov 2020 22:28:50 +0300 Subject: [PATCH 023/205] Fix build --- src/Storages/MergeTree/MergeTreeBlockReadUtils.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeBlockReadUtils.cpp b/src/Storages/MergeTree/MergeTreeBlockReadUtils.cpp index 31578cc8f4f..ad10a437b1e 100644 --- a/src/Storages/MergeTree/MergeTreeBlockReadUtils.cpp +++ b/src/Storages/MergeTree/MergeTreeBlockReadUtils.cpp @@ -260,9 +260,9 @@ MergeTreeReadTaskColumns getReadTaskColumns( if (prewhere_info) { if (prewhere_info->alias_actions) - pre_column_names = prewhere_info->alias_actions->getRequiredColumns().getNames(); + pre_column_names = prewhere_info->alias_actions->getRequiredColumns(); else - pre_column_names = prewhere_info->prewhere_actions->getRequiredColumns().getNames(); + pre_column_names = prewhere_info->prewhere_actions->getRequiredColumns(); if (pre_column_names.empty()) pre_column_names.push_back(column_names[0]); From 54a9b80a114d7d0df1178bef13c489112c1e8e84 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 3 Nov 2020 22:30:58 +0300 Subject: [PATCH 024/205] Fix build --- src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp index 6b7a6b66605..978724137f8 100644 --- a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp +++ b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp @@ -803,10 +803,9 @@ Pipe MergeTreeDataSelectExecutor::readFromParts( if (query_info.prewhere_info && query_info.prewhere_info->remove_columns_actions) { - auto remove_actions = query_info.prewhere_info->remove_columns_actions->buildExpressions(); - res.addSimpleTransform([&remove_actions](const Block & header) + res.addSimpleTransform([&query_info](const Block & header) { - return std::make_shared(header, remove_actions); + return std::make_shared(header, query_info.prewhere_info->remove_columns_actions); }); } From 4ce88298d73d42f6b9563f978dbedf2232fbd4b0 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 3 Nov 2020 22:33:29 +0300 Subject: [PATCH 025/205] Fix build --- src/Processors/QueryPlan/ReadFromStorageStep.cpp | 10 ++++------ 1 file changed, 4 insertions(+), 6 deletions(-) diff --git a/src/Processors/QueryPlan/ReadFromStorageStep.cpp b/src/Processors/QueryPlan/ReadFromStorageStep.cpp index 88248c0d194..fd38dd9218b 100644 --- a/src/Processors/QueryPlan/ReadFromStorageStep.cpp +++ b/src/Processors/QueryPlan/ReadFromStorageStep.cpp @@ -38,19 +38,17 @@ ReadFromStorageStep::ReadFromStorageStep( { if (query_info.prewhere_info->alias_actions) { - auto alias_actions = query_info.prewhere_info->alias_actions->buildExpressions(); pipe.addSimpleTransform([&](const Block & header) { - return std::make_shared(header, alias_actions); + return std::make_shared(header, query_info.prewhere_info->alias_actions); }); } - auto prewhere_actions = query_info.prewhere_info->prewhere_actions->buildExpressions(); pipe.addSimpleTransform([&](const Block & header) { return std::make_shared( header, - prewhere_actions, + query_info.prewhere_info->prewhere_actions, query_info.prewhere_info->prewhere_column_name, query_info.prewhere_info->remove_prewhere_column); }); @@ -61,10 +59,10 @@ ReadFromStorageStep::ReadFromStorageStep( // This leads to mismatched header in distributed table if (query_info.prewhere_info->remove_columns_actions) { - auto remove_actions = query_info.prewhere_info->remove_columns_actions->buildExpressions(); pipe.addSimpleTransform([&](const Block & header) { - return std::make_shared(header, remove_actions); + return std::make_shared( + header, query_info.prewhere_info->remove_columns_actions); }); } } From 8ecf1d07c1a1791485ca9a49d01f7327db4f0b9a Mon Sep 17 00:00:00 2001 From: "a.palagashvili" Date: Wed, 4 Nov 2020 01:37:05 +0300 Subject: [PATCH 026/205] attempt to update fasttest version --- docker/test/fasttest/Dockerfile | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docker/test/fasttest/Dockerfile b/docker/test/fasttest/Dockerfile index f6c665ff3fd..ac22a9dfaf0 100644 --- a/docker/test/fasttest/Dockerfile +++ b/docker/test/fasttest/Dockerfile @@ -1,5 +1,5 @@ # docker build -t yandex/clickhouse-fasttest . -FROM ubuntu:19.10 +FROM ubuntu:20.04 ENV DEBIAN_FRONTEND=noninteractive LLVM_VERSION=10 From f9cebbfa81b6bb50a9f67d35574b08680f76a7e7 Mon Sep 17 00:00:00 2001 From: "a.palagashvili" Date: Wed, 4 Nov 2020 02:47:39 +0300 Subject: [PATCH 027/205] added newline for files --- src/IO/LzmaReadBuffer.h | 2 +- src/IO/LzmaWriteBuffer.h | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/src/IO/LzmaReadBuffer.h b/src/IO/LzmaReadBuffer.h index ae4d4ccf749..5f936475ee1 100644 --- a/src/IO/LzmaReadBuffer.h +++ b/src/IO/LzmaReadBuffer.h @@ -30,4 +30,4 @@ private: lzma_stream lstr; bool eof; }; -} \ No newline at end of file +} diff --git a/src/IO/LzmaWriteBuffer.h b/src/IO/LzmaWriteBuffer.h index fa5e3456f79..d59595dab23 100644 --- a/src/IO/LzmaWriteBuffer.h +++ b/src/IO/LzmaWriteBuffer.h @@ -29,4 +29,4 @@ private: lzma_stream lstr; bool finished = false; }; -} \ No newline at end of file +} From 986d13df3a7df63065d8eff084b174072673ac22 Mon Sep 17 00:00:00 2001 From: "a.palagashvili" Date: Wed, 4 Nov 2020 03:35:16 +0300 Subject: [PATCH 028/205] replaced null with nullptr --- src/IO/LzmaWriteBuffer.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/IO/LzmaWriteBuffer.cpp b/src/IO/LzmaWriteBuffer.cpp index 395de00be1d..8439fc624d4 100644 --- a/src/IO/LzmaWriteBuffer.cpp +++ b/src/IO/LzmaWriteBuffer.cpp @@ -27,9 +27,9 @@ LzmaWriteBuffer::LzmaWriteBuffer( lzma_filter filters[] = { - {.id = LZMA_FILTER_X86, .options = NULL}, + {.id = LZMA_FILTER_X86, .options = nullptr}, {.id = LZMA_FILTER_LZMA2, .options = &opt_lzma2}, - {.id = LZMA_VLI_UNKNOWN, .options = NULL}, + {.id = LZMA_VLI_UNKNOWN, .options = nullptr}, }; lzma_ret ret = lzma_stream_encoder(&lstr, filters, LZMA_CHECK_CRC64); From e7b4dce258b8faf5b8e3ceb0c54b550a4e9f9a8c Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Wed, 4 Nov 2020 13:20:32 +0300 Subject: [PATCH 029/205] Remove inappropriate enum parsing mention --- docs/en/faq/integration/json-import.md | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/docs/en/faq/integration/json-import.md b/docs/en/faq/integration/json-import.md index 47255243f60..7038cc539d2 100644 --- a/docs/en/faq/integration/json-import.md +++ b/docs/en/faq/integration/json-import.md @@ -6,8 +6,7 @@ toc_priority: 11 # How to Import JSON Into ClickHouse? {#how-to-import-json-into-clickhouse} -ClickHouse supports a wide range of [data formats for input and output](../../interfaces/formats.md). There are multiple JSON variations among them, but the most commonly used for data ingestion is [JSONEachRow](../../interfaces/formats.md#jsoneachrow). It expects one JSON object per row, each object separated by a newline. -In JSON format, the presence of quotation marks can determine whether the data is a number or a string. In the case of a number, you must always interpret it as an enum id. +ClickHouse supports a wide range of [data formats for input and output](../../interfaces/formats.md). There are multiple JSON variations among them, but the most commonly used for data ingestion is [JSONEachRow](../../interfaces/formats.md#jsoneachrow). It expects one JSON object per row, each object separated by a newline. ## Examples {#examples} @@ -31,4 +30,4 @@ Instead of inserting data manually, you might consider to use one of [client lib - `input_format_import_nested_json` allows to insert nested JSON objects into columns of [Nested](../../sql-reference/data-types/nested-data-structures/nested.md) type. !!! note "Note" - Settings are specified as `GET` parameters for the HTTP interface or as additional command-line arguments prefixed with `--` for the `CLI` interface. \ No newline at end of file + Settings are specified as `GET` parameters for the HTTP interface or as additional command-line arguments prefixed with `--` for the `CLI` interface. From 53a064b6e38737154f2eccab3336e1db0b7e36ad Mon Sep 17 00:00:00 2001 From: "a.palagashvili" Date: Wed, 4 Nov 2020 15:45:37 +0300 Subject: [PATCH 030/205] added eof initializing in constuctor --- src/IO/LzmaReadBuffer.cpp | 53 ++++++++++++++++----------------------- 1 file changed, 21 insertions(+), 32 deletions(-) diff --git a/src/IO/LzmaReadBuffer.cpp b/src/IO/LzmaReadBuffer.cpp index 377b68a4ae3..1763a101694 100644 --- a/src/IO/LzmaReadBuffer.cpp +++ b/src/IO/LzmaReadBuffer.cpp @@ -1,13 +1,11 @@ #include -namespace DB -{ -namespace ErrorCodes -{ +namespace DB { +namespace ErrorCodes { extern const int LZMA_STREAM_DECODER_FAILED; } -LzmaReadBuffer::LzmaReadBuffer(std::unique_ptr in_, size_t buf_size, char * existing_memory, size_t alignment) - : BufferWithOwnMemory(buf_size, existing_memory, alignment), in(std::move(in_)) +LzmaReadBuffer::LzmaReadBuffer(std::unique_ptr in_, size_t buf_size, char* existing_memory, size_t alignment) + : BufferWithOwnMemory(buf_size, existing_memory, alignment), in(std::move(in_)), eof(false) { lstr = LZMA_STREAM_INIT; lstr.allocator = nullptr; @@ -22,10 +20,9 @@ LzmaReadBuffer::LzmaReadBuffer(std::unique_ptr in_, size_t buf_size, lzma_ret ret = lzma_stream_decoder(&lstr, memlimit, LZMA_CONCATENATED); // lzma does not provide api for converting error code to string unlike zlib if (ret != LZMA_OK) - throw Exception( - std::string("lzma_stream_decoder initialization failed: error code: ") + std::to_string(ret) - + "; lzma version: " + LZMA_VERSION_STRING, - ErrorCodes::LZMA_STREAM_DECODER_FAILED); + throw Exception(std::string("lzma_stream_decoder initialization failed: error code: ") + std::to_string(ret) + "; lzma version: " + + LZMA_VERSION_STRING, + ErrorCodes::LZMA_STREAM_DECODER_FAILED); } LzmaReadBuffer::~LzmaReadBuffer() @@ -38,14 +35,12 @@ bool LzmaReadBuffer::nextImpl() if (eof) return false; - - if (!lstr.avail_in) - { + if (!lstr.avail_in) { in->nextIfAtEnd(); - lstr.next_in = reinterpret_cast(in->position()); + lstr.next_in = reinterpret_cast(in->position()); lstr.avail_in = in->buffer().end() - in->position(); } - lstr.next_out = reinterpret_cast(internal_buffer.begin()); + lstr.next_out = reinterpret_cast(internal_buffer.begin()); lstr.avail_out = internal_buffer.size(); lzma_ret ret = lzma_code(&lstr, LZMA_RUN); @@ -53,29 +48,23 @@ bool LzmaReadBuffer::nextImpl() in->position() = in->buffer().end() - lstr.avail_in; working_buffer.resize(internal_buffer.size() - lstr.avail_out); - if (ret == LZMA_STREAM_END) - { - if (in->eof()) - { + if (ret == LZMA_STREAM_END) { + if (in->eof()) { eof = true; return working_buffer.size() != 0; - } - else - { - throw Exception( - ErrorCodes::LZMA_STREAM_DECODER_FAILED, - "lzma decoder finished, but stream is still alive: error code: {}; lzma version: {}", - ret, - LZMA_VERSION_STRING); + } else { + throw Exception(ErrorCodes::LZMA_STREAM_DECODER_FAILED, + "lzma decoder finished, but stream is still alive: error code: {}; lzma version: {}", + ret, + LZMA_VERSION_STRING); } } if (ret != LZMA_OK) - throw Exception( - ErrorCodes::LZMA_STREAM_DECODER_FAILED, - "lzma_stream_decoder failed: error code: error codeL {}; lzma version: {}", - ret, - LZMA_VERSION_STRING); + throw Exception(ErrorCodes::LZMA_STREAM_DECODER_FAILED, + "lzma_stream_decoder failed: error code: error codeL {}; lzma version: {}", + ret, + LZMA_VERSION_STRING); return true; } From 87cc3546caa601930f67a463c5abccf53551a021 Mon Sep 17 00:00:00 2001 From: "a.palagashvili" Date: Wed, 4 Nov 2020 19:39:26 +0300 Subject: [PATCH 031/205] fix codestyle --- src/IO/LzmaReadBuffer.cpp | 50 +++++++++++++++++++++++---------------- 1 file changed, 30 insertions(+), 20 deletions(-) diff --git a/src/IO/LzmaReadBuffer.cpp b/src/IO/LzmaReadBuffer.cpp index 1763a101694..22fda48b3c6 100644 --- a/src/IO/LzmaReadBuffer.cpp +++ b/src/IO/LzmaReadBuffer.cpp @@ -1,10 +1,12 @@ #include -namespace DB { -namespace ErrorCodes { +namespace DB +{ +namespace ErrorCodes +{ extern const int LZMA_STREAM_DECODER_FAILED; } -LzmaReadBuffer::LzmaReadBuffer(std::unique_ptr in_, size_t buf_size, char* existing_memory, size_t alignment) +LzmaReadBuffer::LzmaReadBuffer(std::unique_ptr in_, size_t buf_size, char * existing_memory, size_t alignment) : BufferWithOwnMemory(buf_size, existing_memory, alignment), in(std::move(in_)), eof(false) { lstr = LZMA_STREAM_INIT; @@ -20,9 +22,10 @@ LzmaReadBuffer::LzmaReadBuffer(std::unique_ptr in_, size_t buf_size, lzma_ret ret = lzma_stream_decoder(&lstr, memlimit, LZMA_CONCATENATED); // lzma does not provide api for converting error code to string unlike zlib if (ret != LZMA_OK) - throw Exception(std::string("lzma_stream_decoder initialization failed: error code: ") + std::to_string(ret) + "; lzma version: " - + LZMA_VERSION_STRING, - ErrorCodes::LZMA_STREAM_DECODER_FAILED); + throw Exception( + std::string("lzma_stream_decoder initialization failed: error code: ") + std::to_string(ret) + + "; lzma version: " + LZMA_VERSION_STRING, + ErrorCodes::LZMA_STREAM_DECODER_FAILED); } LzmaReadBuffer::~LzmaReadBuffer() @@ -35,12 +38,13 @@ bool LzmaReadBuffer::nextImpl() if (eof) return false; - if (!lstr.avail_in) { + if (!lstr.avail_in) + { in->nextIfAtEnd(); - lstr.next_in = reinterpret_cast(in->position()); + lstr.next_in = reinterpret_cast(in->position()); lstr.avail_in = in->buffer().end() - in->position(); } - lstr.next_out = reinterpret_cast(internal_buffer.begin()); + lstr.next_out = reinterpret_cast(internal_buffer.begin()); lstr.avail_out = internal_buffer.size(); lzma_ret ret = lzma_code(&lstr, LZMA_RUN); @@ -48,23 +52,29 @@ bool LzmaReadBuffer::nextImpl() in->position() = in->buffer().end() - lstr.avail_in; working_buffer.resize(internal_buffer.size() - lstr.avail_out); - if (ret == LZMA_STREAM_END) { - if (in->eof()) { + if (ret == LZMA_STREAM_END) + { + if (in->eof()) + { eof = true; return working_buffer.size() != 0; - } else { - throw Exception(ErrorCodes::LZMA_STREAM_DECODER_FAILED, - "lzma decoder finished, but stream is still alive: error code: {}; lzma version: {}", - ret, - LZMA_VERSION_STRING); + } + else + { + throw Exception( + ErrorCodes::LZMA_STREAM_DECODER_FAILED, + "lzma decoder finished, but stream is still alive: error code: {}; lzma version: {}", + ret, + LZMA_VERSION_STRING); } } if (ret != LZMA_OK) - throw Exception(ErrorCodes::LZMA_STREAM_DECODER_FAILED, - "lzma_stream_decoder failed: error code: error codeL {}; lzma version: {}", - ret, - LZMA_VERSION_STRING); + throw Exception( + ErrorCodes::LZMA_STREAM_DECODER_FAILED, + "lzma_stream_decoder failed: error code: error codeL {}; lzma version: {}", + ret, + LZMA_VERSION_STRING); return true; } From 28fb1affac3e6112c2c8495e30693e599239c7d2 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 5 Nov 2020 19:11:14 +0300 Subject: [PATCH 032/205] Update complie expressions. --- src/Interpreters/ExpressionActions.cpp | 136 ++----- src/Interpreters/ExpressionActions.h | 7 + src/Interpreters/ExpressionJIT.cpp | 520 +++++++++++++++++-------- src/Interpreters/ExpressionJIT.h | 35 +- 4 files changed, 413 insertions(+), 285 deletions(-) diff --git a/src/Interpreters/ExpressionActions.cpp b/src/Interpreters/ExpressionActions.cpp index 4a448e9f463..4d006165767 100644 --- a/src/Interpreters/ExpressionActions.cpp +++ b/src/Interpreters/ExpressionActions.cpp @@ -599,104 +599,6 @@ bool ExpressionActions::checkColumnIsAlwaysFalse(const String & column_name) con return false; } - -///// It is not important to calculate the hash of individual strings or their concatenation -//UInt128 ExpressionAction::ActionHash::operator()(const ExpressionAction & action) const -//{ -// SipHash hash; -// hash.update(action.type); -// hash.update(action.is_function_compiled); -// switch (action.type) -// { -// case ADD_COLUMN: -// hash.update(action.result_name); -// if (action.result_type) -// hash.update(action.result_type->getName()); -// if (action.added_column) -// hash.update(action.added_column->getName()); -// break; -// case REMOVE_COLUMN: -// hash.update(action.source_name); -// break; -// case COPY_COLUMN: -// hash.update(action.result_name); -// hash.update(action.source_name); -// break; -// case APPLY_FUNCTION: -// hash.update(action.result_name); -// if (action.result_type) -// hash.update(action.result_type->getName()); -// if (action.function_base) -// { -// hash.update(action.function_base->getName()); -// for (const auto & arg_type : action.function_base->getArgumentTypes()) -// hash.update(arg_type->getName()); -// } -// for (const auto & arg_name : action.argument_names) -// hash.update(arg_name); -// break; -// case ARRAY_JOIN: -// hash.update(action.result_name); -// hash.update(action.source_name); -// break; -// case PROJECT: -// for (const auto & pair_of_strs : action.projection) -// { -// hash.update(pair_of_strs.first); -// hash.update(pair_of_strs.second); -// } -// break; -// case ADD_ALIASES: -// break; -// } -// UInt128 result; -// hash.get128(result.low, result.high); -// return result; -//} -// -//bool ExpressionAction::operator==(const ExpressionAction & other) const -//{ -// if (result_type != other.result_type) -// { -// if (result_type == nullptr || other.result_type == nullptr) -// return false; -// else if (!result_type->equals(*other.result_type)) -// return false; -// } -// -// if (function_base != other.function_base) -// { -// if (function_base == nullptr || other.function_base == nullptr) -// return false; -// else if (function_base->getName() != other.function_base->getName()) -// return false; -// -// const auto & my_arg_types = function_base->getArgumentTypes(); -// const auto & other_arg_types = other.function_base->getArgumentTypes(); -// if (my_arg_types.size() != other_arg_types.size()) -// return false; -// -// for (size_t i = 0; i < my_arg_types.size(); ++i) -// if (!my_arg_types[i]->equals(*other_arg_types[i])) -// return false; -// } -// -// if (added_column != other.added_column) -// { -// if (added_column == nullptr || other.added_column == nullptr) -// return false; -// else if (added_column->getName() != other.added_column->getName()) -// return false; -// } -// -// return source_name == other.source_name -// && result_name == other.result_name -// && argument_names == other.argument_names -// && TableJoin::sameJoin(table_join.get(), other.table_join.get()) -// && projection == other.projection -// && is_function_compiled == other.is_function_compiled; -//} - void ExpressionActionsChain::addStep(NameSet non_constant_inputs) { if (steps.empty()) @@ -979,9 +881,9 @@ const ActionsDAG::Node & ActionsDAG::addFunction( max_temporary_columns = settings.max_temporary_columns; max_temporary_non_const_columns = settings.max_temporary_non_const_columns; - bool do_compile_expressions = false; #if USE_EMBEDDED_COMPILER - do_compile_expressions = settings.compile_expressions; + compile_expressions = settings.compile_expressions; + min_count_to_compile_expression = settings.min_count_to_compile_expression; if (!compilation_cache) compilation_cache = context.getCompiledExpressionCache(); @@ -1004,7 +906,6 @@ const ActionsDAG::Node & ActionsDAG::addFunction( node.allow_constant_folding = node.allow_constant_folding && child.allow_constant_folding; ColumnWithTypeAndName argument; - argument.name = argument_names[i]; argument.column = child.column; argument.type = child.result_type; argument.name = child.result_name; @@ -1022,7 +923,7 @@ const ActionsDAG::Node & ActionsDAG::addFunction( /// If all arguments are constants, and function is suitable to be executed in 'prepare' stage - execute function. /// But if we compile expressions compiled version of this function maybe placed in cache, /// so we don't want to unfold non deterministic functions - if (all_const && node.function_base->isSuitableForConstantFolding() && (!do_compile_expressions || node.function_base->isDeterministic())) + if (all_const && node.function_base->isSuitableForConstantFolding() && (!compile_expressions || node.function_base->isDeterministic())) { size_t num_rows = arguments.empty() ? 0 : arguments.front().column->size(); auto col = node.function->execute(arguments, node.result_type, num_rows, true); @@ -1143,22 +1044,29 @@ void ActionsDAG::removeUnusedActions(const Names & required_names) void ActionsDAG::removeUnusedActions(const std::vector & required_nodes) { - std::unordered_set visited_nodes; - std::stack stack; - { Index new_index; for (auto * node : required_nodes) - { new_index.insert(node); - visited_nodes.insert(node); - stack.push(node); - } index.swap(new_index); } + removeUnusedActions(); +} + +void ActionsDAG::removeUnusedActions() +{ + std::unordered_set visited_nodes; + std::stack stack; + + for (auto * node : index) + { + visited_nodes.insert(node); + stack.push(node); + } + while (!stack.empty()) { auto * node = stack.top(); @@ -1445,8 +1353,16 @@ ExpressionActionsPtr ActionsDAG::linearizeActions() const ExpressionActionsPtr ActionsDAG::buildExpressions() const { auto cloned = clone(); - auto expressions = cloned->linearizeActions(); +#if USE_EMBEDDED_COMPILER + if (compile_expressions) + { + cloned->compileFunctions(); + cloned->removeUnusedActions(); + } +#endif + + auto expressions = cloned->linearizeActions(); expressions->nodes.swap(cloned->nodes); if (max_temporary_columns && expressions->num_columns > max_temporary_columns) diff --git a/src/Interpreters/ExpressionActions.h b/src/Interpreters/ExpressionActions.h index 4dd6efbead9..8fff1dc372d 100644 --- a/src/Interpreters/ExpressionActions.h +++ b/src/Interpreters/ExpressionActions.h @@ -163,6 +163,8 @@ private: size_t max_temporary_columns = 0; size_t max_temporary_non_const_columns = 0; + size_t min_count_to_compile_expression = 0; + bool compile_expressions = false; #if USE_EMBEDDED_COMPILER std::shared_ptr compilation_cache; @@ -234,6 +236,8 @@ private: auto actions = std::make_shared(); actions->max_temporary_columns = max_temporary_columns; actions->max_temporary_non_const_columns = max_temporary_non_const_columns; + actions->min_count_to_compile_expression = min_count_to_compile_expression; + actions->compile_expressions = compile_expressions; #if USE_EMBEDDED_COMPILER actions->compilation_cache = compilation_cache; @@ -243,7 +247,10 @@ private: ExpressionActionsPtr linearizeActions() const; void removeUnusedActions(const std::vector & required_nodes); + void removeUnusedActions(); void addAliases(const NamesWithAliases & aliases, std::vector & result_nodes); + + void compileFunctions(); }; diff --git a/src/Interpreters/ExpressionJIT.cpp b/src/Interpreters/ExpressionJIT.cpp index 27554e8a4d8..c78e270a7f5 100644 --- a/src/Interpreters/ExpressionJIT.cpp +++ b/src/Interpreters/ExpressionJIT.cpp @@ -16,6 +16,8 @@ #include #include #include +#include +#include #pragma GCC diagnostic push #pragma GCC diagnostic ignored "-Wunused-parameter" @@ -441,33 +443,55 @@ struct LLVMModuleState std::shared_ptr memory_manager; }; -LLVMFunction::LLVMFunction(const ExpressionActions::Actions & actions, const DB::Block & sample_block) - : name(actions.back().node->result_name) +LLVMFunction::LLVMFunction(const CompileDAG & dag) + : name(dag.dump()) , module_state(std::make_unique()) { LLVMContext context; - for (const auto & c : sample_block) - /// TODO: implement `getNativeValue` for all types & replace the check with `c.column && toNativeType(...)` - if (c.column && getNativeValue(toNativeType(context.builder, c.type), *c.column, 0)) - subexpressions[c.name] = subexpression(c.column, c.type); - for (const auto & action : actions) + std::vector expressions; + expressions.reserve(dag.size()); + + for (const auto & node : dag) { - const auto & children = action.node->children; - const auto & types = action.node->function_base->getArgumentTypes(); - std::vector args; - for (size_t i = 0; i < children.size(); ++i) + switch (node.type) { - auto inserted = subexpressions.emplace(children[i]->result_name, subexpression(arg_names.size())); - if (inserted.second) + case CompileNode::NodeType::CONSTANT: { - arg_names.push_back(children[i]->result_name); - arg_types.push_back(types[i]); + const auto * col = typeid_cast(node.column.get()); + + /// TODO: implement `getNativeValue` for all types & replace the check with `c.column && toNativeType(...)` + if (!getNativeValue(toNativeType(context.builder, node.result_type), col->getDataColumn(), 0)) + throw Exception(ErrorCodes::LOGICAL_ERROR, + "Cannot compile constant of type {} = {}", + node.result_type->getName(), + applyVisitor(FieldVisitorToString(), col->getDataColumn()[0])); + + expressions.emplace_back(subexpression(col->getDataColumnPtr(), node.result_type)); + break; + } + case CompileNode::NodeType::FUNCTION: + { + std::vector args; + args.reserve(node.arguments.size()); + + for (auto arg : node.arguments) + args.emplace_back(expressions[arg]); + + originals.push_back(node.function); + expressions.emplace_back(subexpression(*node.function, std::move(args))); + break; + } + case CompileNode::NodeType::INPUT: + { + arg_types.push_back(node.result_type); + expressions.emplace_back(subexpression(arg_types.size())); + break; } - args.push_back(inserted.first->second); } - subexpressions[action.node->result_name] = subexpression(*action.node->function_base, std::move(args)); - originals.push_back(action.node->function_base); } + + expression = std::move(expressions.back()); + compileFunctionToLLVMByteCode(context, *this); context.compileAllFunctionsToNativeCode(); @@ -478,10 +502,7 @@ LLVMFunction::LLVMFunction(const ExpressionActions::Actions & actions, const DB: llvm::Value * LLVMFunction::compile(llvm::IRBuilderBase & builder, ValuePlaceholders values) const { - auto it = subexpressions.find(name); - if (subexpressions.end() == it) - throw Exception("Cannot find subexpression " + name + " in LLVMFunction", ErrorCodes::LOGICAL_ERROR); - return it->second(builder, values); + return expression(builder, values); } ExecutableFunctionImplPtr LLVMFunction::prepare(const ColumnsWithTypeAndName &) const { return std::make_unique(name, module_state->symbols); } @@ -555,155 +576,314 @@ LLVMFunction::Monotonicity LLVMFunction::getMonotonicityForRange(const IDataType } -//static bool isCompilable(const IFunctionBase & function) -//{ -// if (!canBeNativeType(*function.getResultType())) -// return false; -// for (const auto & type : function.getArgumentTypes()) -// if (!canBeNativeType(*type)) -// return false; -// return function.isCompilable(); -//} +static bool isCompilable(const IFunctionBase & function) +{ + if (!canBeNativeType(*function.getResultType())) + return false; + for (const auto & type : function.getArgumentTypes()) + if (!canBeNativeType(*type)) + return false; + return function.isCompilable(); +} -//static std::vector>> getActionsDependents(const ExpressionActions::Actions & actions, const Names & output_columns) -//{ -// /// an empty optional is a poisoned value prohibiting the column's producer from being removed -// /// (which it could be, if it was inlined into every dependent function). -// std::unordered_map>> current_dependents; -// for (const auto & name : output_columns) -// current_dependents[name].emplace(); -// /// a snapshot of each compilable function's dependents at the time of its execution. -// std::vector>> dependents(actions.size()); -// for (size_t i = actions.size(); i--;) -// { -// switch (actions[i].type) -// { -// case ExpressionAction::REMOVE_COLUMN: -// current_dependents.erase(actions[i].source_name); -// /// poison every other column used after this point so that inlining chains do not cross it. -// for (auto & dep : current_dependents) -// dep.second.emplace(); -// break; -// -// case ExpressionAction::PROJECT: -// current_dependents.clear(); -// for (const auto & proj : actions[i].projection) -// current_dependents[proj.first].emplace(); -// break; -// -// case ExpressionAction::ADD_ALIASES: -// for (const auto & proj : actions[i].projection) -// current_dependents[proj.first].emplace(); -// break; -// -// case ExpressionAction::ADD_COLUMN: -// case ExpressionAction::COPY_COLUMN: -// case ExpressionAction::ARRAY_JOIN: -// { -// Names columns = actions[i].getNeededColumns(); -// for (const auto & column : columns) -// current_dependents[column].emplace(); -// break; -// } -// -// case ExpressionAction::APPLY_FUNCTION: -// { -// dependents[i] = current_dependents[actions[i].result_name]; -// const bool compilable = isCompilable(*actions[i].function_base); -// for (const auto & name : actions[i].argument_names) -// { -// if (compilable) -// current_dependents[name].emplace(i); -// else -// current_dependents[name].emplace(); -// } -// break; -// } -// } -// } -// return dependents; -//} -// -//void compileFunctions( -// ExpressionActions::Actions & actions, -// const Names & output_columns, -// const Block & sample_block, -// std::shared_ptr compilation_cache, -// size_t min_count_to_compile_expression) -//{ -// static std::unordered_map counter; -// static std::mutex mutex; -// -// struct LLVMTargetInitializer -// { -// LLVMTargetInitializer() -// { -// llvm::InitializeNativeTarget(); -// llvm::InitializeNativeTargetAsmPrinter(); -// llvm::sys::DynamicLibrary::LoadLibraryPermanently(nullptr); -// } -// }; -// -// static LLVMTargetInitializer initializer; -// -// auto dependents = getActionsDependents(actions, output_columns); -// std::vector fused(actions.size()); -// for (size_t i = 0; i < actions.size(); ++i) -// { -// if (actions[i].type != ExpressionAction::APPLY_FUNCTION || !isCompilable(*actions[i].function_base)) -// continue; -// -// fused[i].push_back(actions[i]); -// if (dependents[i].find({}) != dependents[i].end()) -// { -// /// the result of compiling one function in isolation is pretty much the same as its `execute` method. -// if (fused[i].size() == 1) -// continue; -// -// auto hash_key = ExpressionActions::ActionsHash{}(fused[i]); -// { -// std::lock_guard lock(mutex); -// if (counter[hash_key]++ < min_count_to_compile_expression) -// continue; -// } -// -// FunctionBasePtr fn; -// if (compilation_cache) -// { -// std::tie(fn, std::ignore) = compilation_cache->getOrSet(hash_key, [&inlined_func=std::as_const(fused[i]), &sample_block] () -// { -// Stopwatch watch; -// FunctionBasePtr result_fn; -// result_fn = std::make_shared(std::make_unique(inlined_func, sample_block)); -// ProfileEvents::increment(ProfileEvents::CompileExpressionsMicroseconds, watch.elapsedMicroseconds()); -// return result_fn; -// }); -// } -// else -// { -// Stopwatch watch; -// fn = std::make_shared(std::make_unique(fused[i], sample_block)); -// ProfileEvents::increment(ProfileEvents::CompileExpressionsMicroseconds, watch.elapsedMicroseconds()); -// } -// -// actions[i].function_base = fn; -// actions[i].argument_names = typeid_cast(typeid_cast(fn.get())->getImpl())->getArgumentNames(); -// actions[i].is_function_compiled = true; -// -// continue; -// } -// -// /// TODO: determine whether it's profitable to inline the function if there's more than one dependent. -// for (const auto & dep : dependents[i]) -// fused[*dep].insert(fused[*dep].end(), fused[i].begin(), fused[i].end()); -// } -// -// for (auto & action : actions) -// { -// if (action.type == ExpressionAction::APPLY_FUNCTION && action.is_function_compiled) -// action.function = action.function_base->prepare({}); /// Arguments are not used for LLVMFunction. -// } -//} +static bool isCompilableConstant(const ActionsDAG::Node & node) +{ + return node.column && isColumnConst(*node.column) && canBeNativeType(*node.result_type); +} + +static bool isCompilableFunction(const ActionsDAG::Node & node) +{ + return node.type == ActionsDAG::Type::FUNCTION && isCompilable(*node.function_base); +} + +static LLVMFunction::CompileDAG getCompilableDAG(ActionsDAG::Node * root, std::vector & children) +{ + LLVMFunction::CompileDAG dag; + + std::unordered_map positions; + struct Frame + { + ActionsDAG::Node * node; + size_t next_child_to_visit = 0; + }; + + std::stack stack; + stack.push(Frame{.node = root}); + + while (!stack.empty()) + { + auto & frame = stack.top(); + bool is_const = isCompilableConstant(*frame.node); + bool is_compilable_function = !is_const && isCompilableFunction(*frame.node); + + while (is_compilable_function && frame.next_child_to_visit < frame.node->children.size()) + { + auto * child = frame.node->children[frame.next_child_to_visit]; + + if (positions.count(child)) + ++frame.next_child_to_visit; + else + stack.emplace(Frame{.node = child}); + } + + if (!is_compilable_function || frame.next_child_to_visit == frame.node->children.size()) + { + LLVMFunction::CompileNode node; + node.function = frame.node->function_base; + node.result_type = frame.node->result_type; + node.type = is_const ? LLVMFunction::CompileNode::NodeType::CONSTANT + : (is_compilable_function ? LLVMFunction::CompileNode::NodeType::FUNCTION + : LLVMFunction::CompileNode::NodeType::INPUT); + + for (const auto * child : frame.node->children) + node.arguments.push_back(positions[child]); + + if (node.type == LLVMFunction::CompileNode::NodeType::INPUT) + children.emplace_back(frame.node); + + positions[frame.node] = dag.size(); + dag.push_back(std::move(node)); + stack.pop(); + } + } + + return dag; +} + +std::string LLVMFunction::CompileDAG::dump() const +{ + WriteBufferFromOwnString out; + bool first = true; + for (const auto & node : *this) + { + if (!first) + out << " ; "; + first = false; + + switch (node.type) + { + case CompileNode::NodeType::CONSTANT: + { + const auto * column = typeid_cast(node.column.get()); + const auto & data = column->getDataColumn(); + out << node.result_type->getName() << " = " << applyVisitor(FieldVisitorToString(), data[0]); + break; + } + case CompileNode::NodeType::FUNCTION: + { + out << node.result_type->getName() << " = "; + out << node.function->getName() << "("; + + for (size_t i = 0; i < node.arguments.size(); ++i) + { + if (i) + out << ", "; + + out << node.arguments[i]; + } + + out << ")"; + break; + } + case CompileNode::NodeType::INPUT: + { + out << node.result_type->getName(); + break; + } + } + } + + return out.str(); +} + +UInt128 LLVMFunction::CompileDAG::hash() const +{ + SipHash hash; + for (const auto & node : *this) + { + hash.update(node.type); + hash.update(node.result_type->getName()); + + switch (node.type) + { + case CompileNode::NodeType::CONSTANT: + { + typeid_cast(node.column.get())->getDataColumn().updateHashWithValue(0, hash); + break; + } + case CompileNode::NodeType::FUNCTION: + { + hash.update(node.function->getName()); + for (size_t arg : node.arguments) + hash.update(arg); + + break; + } + case CompileNode::NodeType::INPUT: + { + break; + } + } + } + + UInt128 result; + hash.get128(result.low, result.high); + return result; +} + +static FunctionBasePtr compile( + const LLVMFunction::CompileDAG & dag, + size_t min_count_to_compile_expression, + const std::shared_ptr & compilation_cache) +{ + static std::unordered_map counter; + static std::mutex mutex; + + struct LLVMTargetInitializer + { + LLVMTargetInitializer() + { + llvm::InitializeNativeTarget(); + llvm::InitializeNativeTargetAsmPrinter(); + llvm::sys::DynamicLibrary::LoadLibraryPermanently(nullptr); + } + }; + + static LLVMTargetInitializer initializer; + + auto hash_key = dag.hash(); + { + std::lock_guard lock(mutex); + if (counter[hash_key]++ < min_count_to_compile_expression) + return nullptr; + } + + FunctionBasePtr fn; + if (compilation_cache) + { + std::tie(fn, std::ignore) = compilation_cache->getOrSet(hash_key, [&dag] () + { + Stopwatch watch; + FunctionBasePtr result_fn; + result_fn = std::make_shared(std::make_unique(dag)); + ProfileEvents::increment(ProfileEvents::CompileExpressionsMicroseconds, watch.elapsedMicroseconds()); + return result_fn; + }); + } + else + { + Stopwatch watch; + fn = std::make_shared(std::make_unique(dag)); + ProfileEvents::increment(ProfileEvents::CompileExpressionsMicroseconds, watch.elapsedMicroseconds()); + } + + return fn; +} + +void ActionsDAG::compileFunctions() +{ + struct Data + { + bool used_in_result = false; + bool is_compilable = false; + bool all_parents_compilable = false; + size_t num_inlineable_nodes = 0; + }; + + std::unordered_map data; + std::unordered_map constants; + + for (const auto & node : nodes) + data[&node].is_compilable = isCompilableConstant(node) || isCompilableFunction(node); + + for (const auto & node : nodes) + if (!data[&node].is_compilable) + for (const auto * child : node.children) + data[child].all_parents_compilable = false; + + for (const auto * node : index) + data[node].used_in_result = true; + + struct Frame + { + Node * node; + size_t next_child_to_visit = 0; + }; + + std::stack stack; + std::unordered_set visited; + + for (auto & node : nodes) + { + if (visited.count(&node)) + continue; + + stack.emplace(Frame{.node = &node}); + while (!stack.empty()) + { + auto & frame = stack.top(); + + while (frame.next_child_to_visit < frame.node->children.size()) + { + auto * child = frame.node->children[frame.next_child_to_visit]; + + if (visited.count(child)) + ++frame.next_child_to_visit; + else + stack.emplace(Frame{.node = child}); + } + + if (frame.next_child_to_visit == frame.node->children.size()) + { + auto & cur = data[frame.node]; + if (cur.is_compilable) + { + cur.num_inlineable_nodes = 1; + for (const auto * child : frame.node->children) + cur.num_inlineable_nodes += data[child].num_inlineable_nodes; + + /// Check if we should inline current node. + + /// Inline parents instead of node is possible. + if (!cur.used_in_result && cur.all_parents_compilable) + continue; + + /// There is not reason to inline single node. + /// The result of compiling function in isolation is pretty much the same as its `execute` method. + if (cur.num_inlineable_nodes <= 1) + continue; + + /// Compile. + std::vector new_children; + auto dag = getCompilableDAG(frame.node, new_children); + if (dag.size() != cur.num_inlineable_nodes) + throw Exception(ErrorCodes::LOGICAL_ERROR, + "Unexpected number of nodes in compile expression DAG. " + "Expected {}, got {}. Chain: {}", + cur.num_inlineable_nodes, dag.size(), dag.dump()); + + if (auto fn = compile(dag, min_count_to_compile_expression, compilation_cache)) + { + /// Replace current node to compilable function. + + ColumnsWithTypeAndName arguments; + arguments.reserve(new_children.size()); + for (const auto * child : new_children) + arguments.emplace_back( + ColumnWithTypeAndName{child->column, child->result_type, child->result_name}); + + frame.node->type = ActionsDAG::Type::FUNCTION; + frame.node->function_base = fn; + frame.node->function = fn->prepare(arguments); + frame.node->children.swap(new_children); + frame.node->is_function_compiled = true; + frame.node->column = nullptr; /// Just in case. + } + } + } + } + } +} } diff --git a/src/Interpreters/ExpressionJIT.h b/src/Interpreters/ExpressionJIT.h index bf015478215..6ae6a7ac4f2 100644 --- a/src/Interpreters/ExpressionJIT.h +++ b/src/Interpreters/ExpressionJIT.h @@ -22,16 +22,43 @@ struct LLVMModuleState; class LLVMFunction : public IFunctionBaseImpl { std::string name; - Names arg_names; DataTypes arg_types; std::vector originals; - std::unordered_map subexpressions; + CompilableExpression expression; std::unique_ptr module_state; public: - LLVMFunction(const ExpressionActions::Actions & actions, const DB::Block & sample_block); + + struct CompileNode + { + enum class NodeType + { + INPUT = 0, + CONSTANT = 1, + FUNCTION = 2, + }; + + NodeType type; + DataTypePtr result_type; + + /// For CONSTANT + ColumnPtr column; + llvm::Value * value = nullptr; + + /// For FUNCTION + FunctionBasePtr function; + std::vector arguments; + }; + + struct CompileDAG : public std::vector + { + std::string dump() const; + UInt128 hash() const; + }; + + LLVMFunction(const CompileDAG & dag); bool isCompilable() const override { return true; } @@ -39,8 +66,6 @@ public: String getName() const override { return name; } - const Names & getArgumentNames() const { return arg_names; } - const DataTypes & getArgumentTypes() const override { return arg_types; } const DataTypePtr & getResultType() const override { return originals.back()->getResultType(); } From f07c5ffefdc2aba8ad86b553dc91d4096f3a73f8 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 5 Nov 2020 19:29:03 +0300 Subject: [PATCH 033/205] Update complie expressions. --- src/Interpreters/ExpressionJIT.cpp | 2 ++ 1 file changed, 2 insertions(+) diff --git a/src/Interpreters/ExpressionJIT.cpp b/src/Interpreters/ExpressionJIT.cpp index c78e270a7f5..de37c3ed691 100644 --- a/src/Interpreters/ExpressionJIT.cpp +++ b/src/Interpreters/ExpressionJIT.cpp @@ -880,6 +880,8 @@ void ActionsDAG::compileFunctions() frame.node->column = nullptr; /// Just in case. } } + + stack.pop(); } } } From 8eab6b7694a2b90a45e53cba2655602585b66fc9 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 5 Nov 2020 19:34:17 +0300 Subject: [PATCH 034/205] Update complie expressions. --- src/Interpreters/ExpressionJIT.cpp | 8 ++++++-- src/Interpreters/ExpressionJIT.h | 1 - 2 files changed, 6 insertions(+), 3 deletions(-) diff --git a/src/Interpreters/ExpressionJIT.cpp b/src/Interpreters/ExpressionJIT.cpp index de37c3ed691..2cd77dd9eea 100644 --- a/src/Interpreters/ExpressionJIT.cpp +++ b/src/Interpreters/ExpressionJIT.cpp @@ -635,8 +635,12 @@ static LLVMFunction::CompileDAG getCompilableDAG(ActionsDAG::Node * root, std::v : (is_compilable_function ? LLVMFunction::CompileNode::NodeType::FUNCTION : LLVMFunction::CompileNode::NodeType::INPUT); - for (const auto * child : frame.node->children) - node.arguments.push_back(positions[child]); + if (node.type == LLVMFunction::CompileNode::NodeType::FUNCTION) + for (const auto * child : frame.node->children) + node.arguments.push_back(positions[child]); + + if (node.type == LLVMFunction::CompileNode::NodeType::CONSTANT) + node.column = frame.node->column; if (node.type == LLVMFunction::CompileNode::NodeType::INPUT) children.emplace_back(frame.node); diff --git a/src/Interpreters/ExpressionJIT.h b/src/Interpreters/ExpressionJIT.h index 6ae6a7ac4f2..ba16194a6bb 100644 --- a/src/Interpreters/ExpressionJIT.h +++ b/src/Interpreters/ExpressionJIT.h @@ -45,7 +45,6 @@ public: /// For CONSTANT ColumnPtr column; - llvm::Value * value = nullptr; /// For FUNCTION FunctionBasePtr function; From 7af517775be3614c35a96d560bd86e649461d51b Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 5 Nov 2020 19:45:09 +0300 Subject: [PATCH 035/205] Update complie expressions. --- src/Interpreters/ExpressionJIT.cpp | 1 + src/Interpreters/ExpressionJIT.h | 2 +- 2 files changed, 2 insertions(+), 1 deletion(-) diff --git a/src/Interpreters/ExpressionJIT.cpp b/src/Interpreters/ExpressionJIT.cpp index 2cd77dd9eea..b63e45eaee3 100644 --- a/src/Interpreters/ExpressionJIT.cpp +++ b/src/Interpreters/ExpressionJIT.cpp @@ -885,6 +885,7 @@ void ActionsDAG::compileFunctions() } } + visited.insert(frame.node); stack.pop(); } } diff --git a/src/Interpreters/ExpressionJIT.h b/src/Interpreters/ExpressionJIT.h index ba16194a6bb..3c885aca72e 100644 --- a/src/Interpreters/ExpressionJIT.h +++ b/src/Interpreters/ExpressionJIT.h @@ -57,7 +57,7 @@ public: UInt128 hash() const; }; - LLVMFunction(const CompileDAG & dag); + explicit LLVMFunction(const CompileDAG & dag); bool isCompilable() const override { return true; } From a77596b36252a41d8026b83265d97513d0002fbb Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 5 Nov 2020 20:14:18 +0300 Subject: [PATCH 036/205] Update complie expressions. --- src/Interpreters/ExpressionJIT.cpp | 51 +++++++++++++++--------------- 1 file changed, 26 insertions(+), 25 deletions(-) diff --git a/src/Interpreters/ExpressionJIT.cpp b/src/Interpreters/ExpressionJIT.cpp index b63e45eaee3..d932e4f2a1f 100644 --- a/src/Interpreters/ExpressionJIT.cpp +++ b/src/Interpreters/ExpressionJIT.cpp @@ -795,7 +795,6 @@ void ActionsDAG::compileFunctions() }; std::unordered_map data; - std::unordered_map constants; for (const auto & node : nodes) data[&node].is_compilable = isCompilableConstant(node) || isCompilableFunction(node); @@ -847,41 +846,43 @@ void ActionsDAG::compileFunctions() cur.num_inlineable_nodes += data[child].num_inlineable_nodes; /// Check if we should inline current node. + bool should_compile = true; /// Inline parents instead of node is possible. if (!cur.used_in_result && cur.all_parents_compilable) - continue; + should_compile = false; /// There is not reason to inline single node. /// The result of compiling function in isolation is pretty much the same as its `execute` method. if (cur.num_inlineable_nodes <= 1) - continue; + should_compile = false;; - /// Compile. - std::vector new_children; - auto dag = getCompilableDAG(frame.node, new_children); - if (dag.size() != cur.num_inlineable_nodes) - throw Exception(ErrorCodes::LOGICAL_ERROR, - "Unexpected number of nodes in compile expression DAG. " - "Expected {}, got {}. Chain: {}", - cur.num_inlineable_nodes, dag.size(), dag.dump()); - - if (auto fn = compile(dag, min_count_to_compile_expression, compilation_cache)) + if (should_compile) { - /// Replace current node to compilable function. + std::vector new_children; + auto dag = getCompilableDAG(frame.node, new_children); + if (dag.size() != cur.num_inlineable_nodes) + throw Exception(ErrorCodes::LOGICAL_ERROR, + "Unexpected number of nodes in compile expression DAG. " + "Expected {}, got {}. Chain: {}", + cur.num_inlineable_nodes, dag.size(), dag.dump()); - ColumnsWithTypeAndName arguments; - arguments.reserve(new_children.size()); - for (const auto * child : new_children) - arguments.emplace_back( - ColumnWithTypeAndName{child->column, child->result_type, child->result_name}); + if (auto fn = compile(dag, min_count_to_compile_expression, compilation_cache)) + { + /// Replace current node to compilable function. - frame.node->type = ActionsDAG::Type::FUNCTION; - frame.node->function_base = fn; - frame.node->function = fn->prepare(arguments); - frame.node->children.swap(new_children); - frame.node->is_function_compiled = true; - frame.node->column = nullptr; /// Just in case. + ColumnsWithTypeAndName arguments; + arguments.reserve(new_children.size()); + for (const auto * child : new_children) + arguments.emplace_back(child->column, child->result_type, child->result_name); + + frame.node->type = ActionsDAG::Type::FUNCTION; + frame.node->function_base = fn; + frame.node->function = fn->prepare(arguments); + frame.node->children.swap(new_children); + frame.node->is_function_compiled = true; + frame.node->column = nullptr; /// Just in case. + } } } From 46e0a53d82e24871beb3ab43818c0ec058edc909 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 5 Nov 2020 20:15:16 +0300 Subject: [PATCH 037/205] Update complie expressions. --- src/Interpreters/ExpressionJIT.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Interpreters/ExpressionJIT.cpp b/src/Interpreters/ExpressionJIT.cpp index d932e4f2a1f..4b0e248643a 100644 --- a/src/Interpreters/ExpressionJIT.cpp +++ b/src/Interpreters/ExpressionJIT.cpp @@ -855,7 +855,7 @@ void ActionsDAG::compileFunctions() /// There is not reason to inline single node. /// The result of compiling function in isolation is pretty much the same as its `execute` method. if (cur.num_inlineable_nodes <= 1) - should_compile = false;; + should_compile = false; if (should_compile) { From b2155c55ffdb13b36e775fab96864ee4eb41ff8a Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 5 Nov 2020 21:08:23 +0300 Subject: [PATCH 038/205] Update complie expressions. --- src/Interpreters/ExpressionJIT.cpp | 26 +++++++++++++++++++------- 1 file changed, 19 insertions(+), 7 deletions(-) diff --git a/src/Interpreters/ExpressionJIT.cpp b/src/Interpreters/ExpressionJIT.cpp index 4b0e248643a..ecf424b2a39 100644 --- a/src/Interpreters/ExpressionJIT.cpp +++ b/src/Interpreters/ExpressionJIT.cpp @@ -596,7 +596,10 @@ static bool isCompilableFunction(const ActionsDAG::Node & node) return node.type == ActionsDAG::Type::FUNCTION && isCompilable(*node.function_base); } -static LLVMFunction::CompileDAG getCompilableDAG(ActionsDAG::Node * root, std::vector & children) +static LLVMFunction::CompileDAG getCompilableDAG( + ActionsDAG::Node * root, + std::vector & children, + const std::unordered_set & used_in_result) { LLVMFunction::CompileDAG dag; @@ -614,7 +617,7 @@ static LLVMFunction::CompileDAG getCompilableDAG(ActionsDAG::Node * root, std::v { auto & frame = stack.top(); bool is_const = isCompilableConstant(*frame.node); - bool is_compilable_function = !is_const && isCompilableFunction(*frame.node); + bool is_compilable_function = !is_const && !used_in_result.count(frame.node) && isCompilableFunction(*frame.node); while (is_compilable_function && frame.next_child_to_visit < frame.node->children.size()) { @@ -623,7 +626,10 @@ static LLVMFunction::CompileDAG getCompilableDAG(ActionsDAG::Node * root, std::v if (positions.count(child)) ++frame.next_child_to_visit; else + { stack.emplace(Frame{.node = child}); + break; + } } if (!is_compilable_function || frame.next_child_to_visit == frame.node->children.size()) @@ -788,13 +794,13 @@ void ActionsDAG::compileFunctions() { struct Data { - bool used_in_result = false; bool is_compilable = false; bool all_parents_compilable = false; size_t num_inlineable_nodes = 0; }; std::unordered_map data; + std::unordered_set used_in_result; for (const auto & node : nodes) data[&node].is_compilable = isCompilableConstant(node) || isCompilableFunction(node); @@ -805,7 +811,7 @@ void ActionsDAG::compileFunctions() data[child].all_parents_compilable = false; for (const auto * node : index) - data[node].used_in_result = true; + used_in_result.insert(node); struct Frame { @@ -833,7 +839,10 @@ void ActionsDAG::compileFunctions() if (visited.count(child)) ++frame.next_child_to_visit; else + { stack.emplace(Frame{.node = child}); + break; + } } if (frame.next_child_to_visit == frame.node->children.size()) @@ -842,14 +851,17 @@ void ActionsDAG::compileFunctions() if (cur.is_compilable) { cur.num_inlineable_nodes = 1; - for (const auto * child : frame.node->children) - cur.num_inlineable_nodes += data[child].num_inlineable_nodes; + + if (!isCompilableConstant(*frame.node)) + for (const auto * child : frame.node->children) + if (!used_in_result.count(child)) + cur.num_inlineable_nodes += data[child].num_inlineable_nodes; /// Check if we should inline current node. bool should_compile = true; /// Inline parents instead of node is possible. - if (!cur.used_in_result && cur.all_parents_compilable) + if (!used_in_result.count(frame.node) && cur.all_parents_compilable) should_compile = false; /// There is not reason to inline single node. From 513779afb08e15a2520518fbab7e1161d63c3345 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 5 Nov 2020 21:09:58 +0300 Subject: [PATCH 039/205] Update complie expressions. --- src/Interpreters/ExpressionJIT.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Interpreters/ExpressionJIT.cpp b/src/Interpreters/ExpressionJIT.cpp index ecf424b2a39..962f1ad2330 100644 --- a/src/Interpreters/ExpressionJIT.cpp +++ b/src/Interpreters/ExpressionJIT.cpp @@ -872,7 +872,7 @@ void ActionsDAG::compileFunctions() if (should_compile) { std::vector new_children; - auto dag = getCompilableDAG(frame.node, new_children); + auto dag = getCompilableDAG(frame.node, new_children, used_in_result); if (dag.size() != cur.num_inlineable_nodes) throw Exception(ErrorCodes::LOGICAL_ERROR, "Unexpected number of nodes in compile expression DAG. " From 0a4bb37dcbb4caa94277cb4ae7f3d9829c711fab Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 5 Nov 2020 21:26:30 +0300 Subject: [PATCH 040/205] Update complie expressions. --- src/Interpreters/ExpressionJIT.cpp | 5 ----- 1 file changed, 5 deletions(-) diff --git a/src/Interpreters/ExpressionJIT.cpp b/src/Interpreters/ExpressionJIT.cpp index 962f1ad2330..e4cca59cdf6 100644 --- a/src/Interpreters/ExpressionJIT.cpp +++ b/src/Interpreters/ExpressionJIT.cpp @@ -873,11 +873,6 @@ void ActionsDAG::compileFunctions() { std::vector new_children; auto dag = getCompilableDAG(frame.node, new_children, used_in_result); - if (dag.size() != cur.num_inlineable_nodes) - throw Exception(ErrorCodes::LOGICAL_ERROR, - "Unexpected number of nodes in compile expression DAG. " - "Expected {}, got {}. Chain: {}", - cur.num_inlineable_nodes, dag.size(), dag.dump()); if (auto fn = compile(dag, min_count_to_compile_expression, compilation_cache)) { From 63fa2b15fdf65bc8fe2dd1b9f516afd20b763a4d Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 5 Nov 2020 23:36:37 +0300 Subject: [PATCH 041/205] Update complie expressions. --- src/Interpreters/ExpressionJIT.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Interpreters/ExpressionJIT.cpp b/src/Interpreters/ExpressionJIT.cpp index e4cca59cdf6..32ad402f6f2 100644 --- a/src/Interpreters/ExpressionJIT.cpp +++ b/src/Interpreters/ExpressionJIT.cpp @@ -483,8 +483,8 @@ LLVMFunction::LLVMFunction(const CompileDAG & dag) } case CompileNode::NodeType::INPUT: { - arg_types.push_back(node.result_type); expressions.emplace_back(subexpression(arg_types.size())); + arg_types.push_back(node.result_type); break; } } @@ -795,7 +795,7 @@ void ActionsDAG::compileFunctions() struct Data { bool is_compilable = false; - bool all_parents_compilable = false; + bool all_parents_compilable = true; size_t num_inlineable_nodes = 0; }; From 54d6b476083658b31f590bb08b66e2cda50f8b3f Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 6 Nov 2020 00:13:36 +0300 Subject: [PATCH 042/205] Update complie expressions. --- src/Interpreters/ExpressionJIT.cpp | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/Interpreters/ExpressionJIT.cpp b/src/Interpreters/ExpressionJIT.cpp index 32ad402f6f2..48f3f36a06c 100644 --- a/src/Interpreters/ExpressionJIT.cpp +++ b/src/Interpreters/ExpressionJIT.cpp @@ -617,7 +617,8 @@ static LLVMFunction::CompileDAG getCompilableDAG( { auto & frame = stack.top(); bool is_const = isCompilableConstant(*frame.node); - bool is_compilable_function = !is_const && !used_in_result.count(frame.node) && isCompilableFunction(*frame.node); + bool can_inline = stack.size() == 1 || !used_in_result.count(frame.node); + bool is_compilable_function = !is_const && can_inline && isCompilableFunction(*frame.node); while (is_compilable_function && frame.next_child_to_visit < frame.node->children.size()) { From 11191d932a969f62ea849fa939410100c6358db4 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 6 Nov 2020 11:06:06 +0300 Subject: [PATCH 043/205] Update complie expressions. --- src/Interpreters/ExpressionJIT.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Interpreters/ExpressionJIT.cpp b/src/Interpreters/ExpressionJIT.cpp index 48f3f36a06c..d36b08bcd74 100644 --- a/src/Interpreters/ExpressionJIT.cpp +++ b/src/Interpreters/ExpressionJIT.cpp @@ -588,7 +588,7 @@ static bool isCompilable(const IFunctionBase & function) static bool isCompilableConstant(const ActionsDAG::Node & node) { - return node.column && isColumnConst(*node.column) && canBeNativeType(*node.result_type); + return node.column && isColumnConst(*node.column) && canBeNativeType(*node.result_type) && node.allow_constant_folding; } static bool isCompilableFunction(const ActionsDAG::Node & node) From 983aaa3b835cf68ede39697860e0423d68a0ad5b Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 6 Nov 2020 11:08:47 +0300 Subject: [PATCH 044/205] Try fix integration test. --- src/Functions/version.cpp | 3 +++ 1 file changed, 3 insertions(+) diff --git a/src/Functions/version.cpp b/src/Functions/version.cpp index 73c31ae59a2..91c8c69d55c 100644 --- a/src/Functions/version.cpp +++ b/src/Functions/version.cpp @@ -26,6 +26,9 @@ public: return name; } + bool isDeterministic() const override { return false; } + bool isDeterministicInScopeOfQuery() const override { return false; } + size_t getNumberOfArguments() const override { return 0; From 93e4802fa4919894b07699f8f8d4596fe2400d8a Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 6 Nov 2020 17:07:28 +0300 Subject: [PATCH 045/205] Try fix integration test. --- src/Functions/hostName.cpp | 2 ++ src/Functions/version.cpp | 1 + 2 files changed, 3 insertions(+) diff --git a/src/Functions/hostName.cpp b/src/Functions/hostName.cpp index ee3715f51bd..0316a5395b8 100644 --- a/src/Functions/hostName.cpp +++ b/src/Functions/hostName.cpp @@ -32,6 +32,8 @@ public: return false; } + bool isSuitableForConstantFolding() const override { return false; } + size_t getNumberOfArguments() const override { return 0; diff --git a/src/Functions/version.cpp b/src/Functions/version.cpp index 91c8c69d55c..c4668c21fa8 100644 --- a/src/Functions/version.cpp +++ b/src/Functions/version.cpp @@ -28,6 +28,7 @@ public: bool isDeterministic() const override { return false; } bool isDeterministicInScopeOfQuery() const override { return false; } + bool isSuitableForConstantFolding() const override { return false; } size_t getNumberOfArguments() const override { From e474bcca8ea6ed0d207f5ed7977bf9873afd8c84 Mon Sep 17 00:00:00 2001 From: root <@bigo.sg> Date: Wed, 2 Sep 2020 09:13:59 +0000 Subject: [PATCH 046/205] Add StorageEmbeddedRocksdb Engine --- .gitmodules | 4 + CMakeLists.txt | 2 + cmake/find/rocksdb.cmake | 22 ++ contrib/CMakeLists.txt | 7 + contrib/rocksdb | 1 + src/CMakeLists.txt | 12 + src/Core/config_core.h.in | 2 + .../Rocksdb/StorageEmbeddedRocksdb.cpp | 297 ++++++++++++++++++ src/Storages/Rocksdb/StorageEmbeddedRocksdb.h | 67 ++++ src/Storages/registerStorages.cpp | 4 + src/Storages/registerStorages.h | 4 + .../0_stateless/01504_rocksdb.reference | 6 + tests/queries/0_stateless/01504_rocksdb.sql | 22 ++ 13 files changed, 450 insertions(+) create mode 100644 cmake/find/rocksdb.cmake create mode 160000 contrib/rocksdb create mode 100644 src/Storages/Rocksdb/StorageEmbeddedRocksdb.cpp create mode 100644 src/Storages/Rocksdb/StorageEmbeddedRocksdb.h create mode 100644 tests/queries/0_stateless/01504_rocksdb.reference create mode 100644 tests/queries/0_stateless/01504_rocksdb.sql diff --git a/.gitmodules b/.gitmodules index fdd48fcce01..d1b8b5ac0c4 100644 --- a/.gitmodules +++ b/.gitmodules @@ -190,3 +190,7 @@ path = contrib/croaring url = https://github.com/RoaringBitmap/CRoaring branch = v0.2.66 +[submodule "contrib/rocksdb"] + path = contrib/rocksdb + url = https://github.com/facebook/rocksdb + branch = v6.11.4 diff --git a/CMakeLists.txt b/CMakeLists.txt index 783a9f80b66..0506ef10ba0 100644 --- a/CMakeLists.txt +++ b/CMakeLists.txt @@ -455,6 +455,8 @@ include (cmake/find/simdjson.cmake) include (cmake/find/rapidjson.cmake) include (cmake/find/fastops.cmake) include (cmake/find/odbc.cmake) +include (cmake/find/rocksdb.cmake) + if(NOT USE_INTERNAL_PARQUET_LIBRARY) set (ENABLE_ORC OFF CACHE INTERNAL "") diff --git a/cmake/find/rocksdb.cmake b/cmake/find/rocksdb.cmake new file mode 100644 index 00000000000..8155449df41 --- /dev/null +++ b/cmake/find/rocksdb.cmake @@ -0,0 +1,22 @@ +option(ENABLE_ROCKSDB "Enable ROCKSDB" ${ENABLE_LIBRARIES}) +option(USE_INTERNAL_ROCKSDB_LIBRARY "Set to FALSE to use system ROCKSDB library instead of bundled" ${NOT_UNBUNDLED}) + +if(ENABLE_ROCKSDB) + if (NOT EXISTS "${ClickHouse_SOURCE_DIR}/contrib/rocksdb") + message (WARNING "submodule contrib is missing. to fix try run: \n git submodule update --init --recursive") + set (MISSING_ROCKSDB 1) + endif () + + if (USE_INTERNAL_ROCKSDB_LIBRARY AND NOT MISSING_ROCKSDB) + set (ROCKSDB_INCLUDE_DIR "${ClickHouse_SOURCE_DIR}/contrib/rocksdb/include") + set (ROCKSDB_CORE_INCLUDE_DIR "${ClickHouse_SOURCE_DIR}/contrib/rocksdb/include") + set (ROCKSDB_LIBRARY "rocksdb") + set (USE_INTERNAL_ROCKSDB_LIBRARY 1) + set (USE_ROCKSDB 1) + else() + find_package(ROCKSDB) + endif () + +endif() + +message (STATUS "Using ROCKSDB=${USE_ROCKSDB}: ${ROCKSDB_INCLUDE_DIR} : ${ROCKSDB_LIBRARY}") \ No newline at end of file diff --git a/contrib/CMakeLists.txt b/contrib/CMakeLists.txt index 7d6b9c0e374..41008adce90 100644 --- a/contrib/CMakeLists.txt +++ b/contrib/CMakeLists.txt @@ -318,3 +318,10 @@ if (USE_KRB5) add_subdirectory (cyrus-sasl-cmake) endif() endif() + +if (USE_INTERNAL_ROCKSDB_LIBRARY) + set(WITH_TESTS OFF) + set(WITH_BENCHMARK_TOOLS OFF) + set(WITH_TOOLS OFF) + add_subdirectory (rocksdb) +endif() diff --git a/contrib/rocksdb b/contrib/rocksdb new file mode 160000 index 00000000000..963314ffd68 --- /dev/null +++ b/contrib/rocksdb @@ -0,0 +1 @@ +Subproject commit 963314ffd681596ef2738a95249fe4c1163ef87a diff --git a/src/CMakeLists.txt b/src/CMakeLists.txt index 085269847e4..9518810a703 100644 --- a/src/CMakeLists.txt +++ b/src/CMakeLists.txt @@ -78,6 +78,10 @@ if (USE_AMQPCPP) add_headers_and_sources(dbms Storages/RabbitMQ) endif() +if (USE_ROCKSDB) + add_headers_and_sources(dbms Storages/Rocksdb) +endif() + if (USE_AWS_S3) add_headers_and_sources(dbms Common/S3) add_headers_and_sources(dbms Disks/S3) @@ -294,6 +298,7 @@ if (USE_KRB5) dbms_target_link_libraries(PRIVATE ${KRB5_LIBRARY}) endif() + if(RE2_INCLUDE_DIR) target_include_directories(clickhouse_common_io SYSTEM BEFORE PUBLIC ${RE2_INCLUDE_DIR}) endif() @@ -401,6 +406,13 @@ if (USE_ORC) dbms_target_include_directories(SYSTEM BEFORE PUBLIC ${ORC_INCLUDE_DIR} ${CMAKE_BINARY_DIR}/contrib/orc/c++/include) endif () +if (USE_ROCKSDB) + dbms_target_link_libraries(PUBLIC ${ROCKSDB_LIBRARY}) + dbms_target_include_directories(SYSTEM BEFORE PUBLIC ${ROCKSDB_INCLUDE_DIR}) + # target_link_libraries (clickhouse_common_io PUBLIC ${ROCKSDB_LIBRARY}) + # target_include_directories (clickhouse_common_io SYSTEM BEFORE PUBLIC ${ROCKSDB_INCLUDE_DIR}) +endif() + if (ENABLE_TESTS AND USE_GTEST) macro (grep_gtest_sources BASE_DIR DST_VAR) # Cold match files that are not in tests/ directories diff --git a/src/Core/config_core.h.in b/src/Core/config_core.h.in index bf52bf59975..5dbac2f69bf 100644 --- a/src/Core/config_core.h.in +++ b/src/Core/config_core.h.in @@ -11,3 +11,5 @@ #cmakedefine01 USE_SSL #cmakedefine01 USE_OPENCL #cmakedefine01 USE_LDAP +#cmakedefine01 USE_ROCKSDB + diff --git a/src/Storages/Rocksdb/StorageEmbeddedRocksdb.cpp b/src/Storages/Rocksdb/StorageEmbeddedRocksdb.cpp new file mode 100644 index 00000000000..760fd756b16 --- /dev/null +++ b/src/Storages/Rocksdb/StorageEmbeddedRocksdb.cpp @@ -0,0 +1,297 @@ +#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 BAD_ARGUMENTS; + extern const int SYSTEM_ERROR; + extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; +} + + +static bool extractKeyImpl(const IAST & elem, Strings & res) +{ + const auto * function = elem.as(); + if (!function) + return false; + + if (function->name == "equals" || function->name == "in") + { + const auto & args = function->arguments->as(); + const IAST * value; + + if (args.children.size() != 2) + return false; + + const ASTIdentifier * ident; + if ((ident = args.children.at(0)->as())) + value = args.children.at(1).get(); + else if ((ident = args.children.at(1)->as())) + value = args.children.at(0).get(); + else + return false; + + if (ident->name != "key") + return false; + + if (const auto * literal = value->as()) + { + if (literal->value.getType() == Field::Types::String) + { + res.push_back(literal->value.safeGet()); + return true; + } + else if (literal->value.getType() == Field::Types::Tuple) + { + auto tuple = literal->value.safeGet(); + for (const auto & f : tuple) + { + res.push_back(f.safeGet()); + } + return true; + } + else return false; + } + } + return false; +} + + +/** Retrieve from the query a condition of the form `key = 'key'` or `key in ('xxx_') or `key like 'xxx%'`, from conjunctions in the WHERE clause. + */ +static std::pair extractKey(const ASTPtr & query) +{ + const auto & select = query->as(); + if (!select.where()) + { + return std::make_pair(Strings{}, true); + } + Strings res; + extractKeyImpl(*select.where(), res); + return std::make_pair(res, false); +} + +class EmbeddedRocksdbBlockOutputStream : public IBlockOutputStream +{ +public: + explicit EmbeddedRocksdbBlockOutputStream( + StorageEmbeddedRocksdb & storage_, + const StorageMetadataPtr & metadata_snapshot_) + : storage(storage_) + , metadata_snapshot(metadata_snapshot_) + {} + + Block getHeader() const override { return metadata_snapshot->getSampleBlock(); } + + void write(const Block & block) override + { + metadata_snapshot->check(block, true); + auto rows = block.rows(); + auto key_col = block.getByName("key"); + auto val_col = block.getByName("value"); + + const ColumnString * keys = checkAndGetColumn(key_col.column.get()); + + WriteBufferFromOwnString wb_value; + + for (size_t i = 0; i < rows; i++) + { + StringRef key = keys->getDataAt(i); + val_col.type->serializeBinary(*val_col.column, i, wb_value); + + auto status = storage.rocksdb_ptr->rocksdb->Put(rocksdb::WriteOptions(), key.toString(), wb_value.str()); + if (!status.ok()) + throw Exception("Rocksdb write error: " + status.ToString(), ErrorCodes::SYSTEM_ERROR); + + wb_value.restart(); + } + + rocksdb::Iterator* it = storage.rocksdb_ptr->rocksdb->NewIterator(rocksdb::ReadOptions()); + for (it->SeekToFirst(); it->Valid(); it->Next()) + { + LOG_DEBUG(&Poco::Logger::get("StorageEmbeddedRocksdb"), "Iterator `{}` returns `{}`, {}", + it->key().ToString(), it->value().ToString(), it->key().size()); + } + // Check for any errors found during the scan + assert(it->status().ok()); + delete it; + } + +private: + StorageEmbeddedRocksdb & storage; + StorageMetadataPtr metadata_snapshot; +}; + +StorageEmbeddedRocksdb::StorageEmbeddedRocksdb(const StorageFactory::Arguments & args) + : IStorage(args.table_id) +{ + //must contains two columns, key and value + if (args.columns.size() != 2) + throw Exception("Storage " + getName() + " requires exactly 2 columns", ErrorCodes::BAD_ARGUMENTS); + + if (!args.columns.has("key") || !args.columns.has("value")) + throw Exception("Storage " + getName() + " requires columns are: key and value", ErrorCodes::BAD_ARGUMENTS); + + StorageInMemoryMetadata storage_metadata; + storage_metadata.setColumns(args.columns); + storage_metadata.setConstraints(args.constraints); + setInMemoryMetadata(storage_metadata); + + rocksdb_dir = args.context.getPath() + args.relative_data_path + "/rocksdb"; + if (!args.attach) + { + Poco::File(rocksdb_dir).createDirectories(); + } + initDb(); +} + +void StorageEmbeddedRocksdb::truncate(const ASTPtr &, const StorageMetadataPtr & , const Context &, TableExclusiveLockHolder &) +{ + if (rocksdb_ptr) + { + rocksdb_ptr->shutdown(); + } + Poco::File(rocksdb_dir).remove(true); + Poco::File(rocksdb_dir).createDirectories(); + initDb(); +} + +void StorageEmbeddedRocksdb::initDb() +{ + rocksdb::Options options; + rocksdb::BlockBasedTableOptions table_options; + + rocksdb::DB * db; + options.create_if_missing = true; + options.statistics = rocksdb::CreateDBStatistics(); + auto cache = rocksdb::NewLRUCache(256 << 20); + table_options.block_cache = cache; + options.table_factory.reset(rocksdb::NewBlockBasedTableFactory(table_options)); + + rocksdb::Status status = rocksdb::DB::Open(options, rocksdb_dir, &db); + + if (status != rocksdb::Status::OK()) + throw Exception("Fail to open rocksdb path at: " + rocksdb_dir, ErrorCodes::SYSTEM_ERROR); + rocksdb_ptr = std::make_shared(db); +} + + +Pipe StorageEmbeddedRocksdb::read( + const Names & column_names, + const StorageMetadataPtr & metadata_snapshot, + const SelectQueryInfo & query_info, + const Context & /*context*/, + QueryProcessingStage::Enum /*processed_stage*/, + size_t /*max_block_size*/, + unsigned /*num_streams*/) +{ + metadata_snapshot->check(column_names, getVirtuals(), getStorageID()); + Block sample_block = metadata_snapshot->getSampleBlock(); + + size_t key_pos = 0; + size_t value_pos = 1; + if (sample_block.getByPosition(0).name != "key") + std::swap(key_pos, value_pos); + + MutableColumns columns = sample_block.cloneEmptyColumns(); + + Strings keys; + bool all_scan = false; + std::tie(keys, all_scan) = extractKey(query_info.query); + if (keys.empty() && !all_scan) + throw Exception("StorageEmbeddedRocksdb engine must contain condition like key = 'key' or key in tuple(String) or key like 'xxx%' in WHERE clause or empty WHERE clause for all key value scan.", ErrorCodes::BAD_ARGUMENTS); + + // TODO pipline + if (all_scan) + { + auto it = rocksdb_ptr->rocksdb->NewIterator(rocksdb::ReadOptions()); + for (it->SeekToFirst(); it->Valid(); it->Next()) + { + ReadBufferFromString rvalue(it->value().ToString()); + columns[key_pos]->insert(it->key().ToString()); + sample_block.getByName("value").type->deserializeBinary(*columns[value_pos], rvalue); + } + assert(it->status().ok()); + delete it; + } + else + { + Strings values; + std::vector slices_keys; + for (auto & key : keys) + { + slices_keys.push_back(key); + } + auto statuses = rocksdb_ptr->rocksdb->MultiGet(rocksdb::ReadOptions(), slices_keys, &values); + for (size_t i = 0; i < statuses.size(); ++i) + { + if (statuses[i].ok()) + { + ReadBufferFromString rvalue(values[i]); + columns[key_pos]->insert(keys[i]); + sample_block.getByName("value").type->deserializeBinary(*columns[value_pos], rvalue); + } + } + } + + UInt64 num_rows = columns.at(0)->size(); + Chunk chunk(std::move(columns), num_rows); + return Pipe(std::make_shared(sample_block, std::move(chunk))); +} + +BlockOutputStreamPtr StorageEmbeddedRocksdb::write(const ASTPtr & /*query*/, const StorageMetadataPtr & metadata_snapshot, const Context & /*context*/) +{ + return std::make_shared(*this, metadata_snapshot); +} + +StorageEmbeddedRocksdb::~StorageEmbeddedRocksdb() +{ + if (rocksdb_ptr) + { + rocksdb_ptr->shutdown(); + } +} + + +void registerStorageEmbeddedRocksdb(StorageFactory & factory) +{ + factory.registerStorage("EmbeddedRocksdb", [](const StorageFactory::Arguments & args) + { + if (!args.engine_args.empty()) + throw Exception( + "Engine " + args.engine_name + " doesn't support any arguments (" + toString(args.engine_args.size()) + " given)", + ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); + return StorageEmbeddedRocksdb::create(args); + }); +} + + +} diff --git a/src/Storages/Rocksdb/StorageEmbeddedRocksdb.h b/src/Storages/Rocksdb/StorageEmbeddedRocksdb.h new file mode 100644 index 00000000000..6fbc6c04018 --- /dev/null +++ b/src/Storages/Rocksdb/StorageEmbeddedRocksdb.h @@ -0,0 +1,67 @@ +#pragma once + +#include +#include +#include +#include + +#include + +#include "rocksdb/db.h" +#include "rocksdb/table.h" + +namespace DB +{ + +class Context; + +struct Rocksdb +{ + rocksdb::DB * rocksdb; + explicit Rocksdb(rocksdb::DB * rocksdb_) : rocksdb{rocksdb_} {} + Rocksdb(const Rocksdb &) = delete; + Rocksdb & operator=(const Rocksdb &) = delete; + + void shutdown() + { + if (rocksdb) + { + rocksdb->Close(); + delete rocksdb; + } + } + }; + +class StorageEmbeddedRocksdb final : public ext::shared_ptr_helper, public IStorage +{ + friend struct ext::shared_ptr_helper; + friend class EmbeddedRocksdbBlockOutputStream; +public: + std::string getName() const override { return "EmbeddedRocksdb"; } + + ~StorageEmbeddedRocksdb() override; + + Pipe read( + const Names & column_names, + const StorageMetadataPtr & metadata_snapshot, + const SelectQueryInfo & query_info, + const Context & context, + QueryProcessingStage::Enum processed_stage, + size_t max_block_size, + unsigned num_streams) override; + + BlockOutputStreamPtr write(const ASTPtr & query, const StorageMetadataPtr & /*metadata_snapshot*/, const Context & context) override; + void truncate(const ASTPtr &, const StorageMetadataPtr & metadata_snapshot, const Context &, TableExclusiveLockHolder &) override; + +protected: + StorageEmbeddedRocksdb(const StorageFactory::Arguments & args); + +private: + using RocksdbPtr = std::shared_ptr; + String rocksdb_dir; + RocksdbPtr rocksdb_ptr; + mutable std::shared_mutex rwlock; + + void initDb(); +}; +} diff --git a/src/Storages/registerStorages.cpp b/src/Storages/registerStorages.cpp index 0c7a36c011f..f1e38c4336d 100644 --- a/src/Storages/registerStorages.cpp +++ b/src/Storages/registerStorages.cpp @@ -53,6 +53,10 @@ void registerStorages() #if USE_AMQPCPP registerStorageRabbitMQ(factory); #endif + + #if USE_ROCKSDB + registerStorageEmbeddedRocksdb(factory); + #endif } } diff --git a/src/Storages/registerStorages.h b/src/Storages/registerStorages.h index 592bdcf383f..f74eefe1c00 100644 --- a/src/Storages/registerStorages.h +++ b/src/Storages/registerStorages.h @@ -54,6 +54,10 @@ void registerStorageKafka(StorageFactory & factory); void registerStorageRabbitMQ(StorageFactory & factory); #endif +#if USE_ROCKSDB +void registerStorageEmbeddedRocksdb(StorageFactory & factory); +#endif + void registerStorages(); } diff --git a/tests/queries/0_stateless/01504_rocksdb.reference b/tests/queries/0_stateless/01504_rocksdb.reference new file mode 100644 index 00000000000..2723ca095f1 --- /dev/null +++ b/tests/queries/0_stateless/01504_rocksdb.reference @@ -0,0 +1,6 @@ +1 +1000 +0 435761734006 +1 435761734006 +2 435761734006 +0 diff --git a/tests/queries/0_stateless/01504_rocksdb.sql b/tests/queries/0_stateless/01504_rocksdb.sql new file mode 100644 index 00000000000..84add73b15c --- /dev/null +++ b/tests/queries/0_stateless/01504_rocksdb.sql @@ -0,0 +1,22 @@ +DROP TABLE IF EXISTS test; +create table test (key String, value UInt32) Engine=EmbeddedRocksdb; + +insert into test select '1_1', number from numbers(10000); +select count(1) from test; + +insert into test select concat(toString(number), '_1'), number from numbers(10000); +select sum(value) from test where key in ('1_1', '99_1', '900_1'); + + +DROP TABLE IF EXISTS test; +create table test (key String, value UInt64) Engine=EmbeddedRocksdb; + +insert into test select toString(number%3) as key, sum(number) as value from numbers(1000000) group by key; + +select key, sum(value) from test group by key order by key; + +truncate table test; +select count(1) from test; + +DROP TABLE IF EXISTS test; + From 2f5e903c10ccbe39c9bbf96abc6589e97c44fbb9 Mon Sep 17 00:00:00 2001 From: sundy-li <543950155@qq.com> Date: Mon, 21 Sep 2020 21:24:28 +0800 Subject: [PATCH 047/205] fix styles --- src/Storages/Rocksdb/StorageEmbeddedRocksdb.cpp | 3 +-- src/Storages/Rocksdb/StorageEmbeddedRocksdb.h | 3 ++- src/Storages/ya.make | 1 + 3 files changed, 4 insertions(+), 3 deletions(-) diff --git a/src/Storages/Rocksdb/StorageEmbeddedRocksdb.cpp b/src/Storages/Rocksdb/StorageEmbeddedRocksdb.cpp index 760fd756b16..5a99ed716b6 100644 --- a/src/Storages/Rocksdb/StorageEmbeddedRocksdb.cpp +++ b/src/Storages/Rocksdb/StorageEmbeddedRocksdb.cpp @@ -9,7 +9,6 @@ #include #include #include -#include #include #include @@ -85,7 +84,7 @@ static bool extractKeyImpl(const IAST & elem, Strings & res) } -/** Retrieve from the query a condition of the form `key = 'key'` or `key in ('xxx_') or `key like 'xxx%'`, from conjunctions in the WHERE clause. +/** Retrieve from the query a condition of the form `key = 'key'` or `key in ('xxx_'), from conjunctions in the WHERE clause. */ static std::pair extractKey(const ASTPtr & query) { diff --git a/src/Storages/Rocksdb/StorageEmbeddedRocksdb.h b/src/Storages/Rocksdb/StorageEmbeddedRocksdb.h index 6fbc6c04018..546e7e47dc6 100644 --- a/src/Storages/Rocksdb/StorageEmbeddedRocksdb.h +++ b/src/Storages/Rocksdb/StorageEmbeddedRocksdb.h @@ -30,7 +30,8 @@ struct Rocksdb delete rocksdb; } } - }; +}; + class StorageEmbeddedRocksdb final : public ext::shared_ptr_helper, public IStorage { diff --git a/src/Storages/ya.make b/src/Storages/ya.make index 0f14826d859..f45767d42f2 100644 --- a/src/Storages/ya.make +++ b/src/Storages/ya.make @@ -105,6 +105,7 @@ SRCS( MutationCommands.cpp PartitionCommands.cpp ReadInOrderOptimizer.cpp + Rocksdb/StorageEmbeddedRocksdb.cpp SelectQueryDescription.cpp SetSettings.cpp StorageBuffer.cpp From 1ea900a2f307888d7140484a0437b2ccc126cf67 Mon Sep 17 00:00:00 2001 From: sundy-li <543950155@qq.com> Date: Tue, 22 Sep 2020 21:23:07 +0800 Subject: [PATCH 048/205] update cmake --- cmake/find/rocksdb.cmake | 1 + 1 file changed, 1 insertion(+) diff --git a/cmake/find/rocksdb.cmake b/cmake/find/rocksdb.cmake index 8155449df41..c476b86c9d1 100644 --- a/cmake/find/rocksdb.cmake +++ b/cmake/find/rocksdb.cmake @@ -4,6 +4,7 @@ option(USE_INTERNAL_ROCKSDB_LIBRARY "Set to FALSE to use system ROCKSDB library if(ENABLE_ROCKSDB) if (NOT EXISTS "${ClickHouse_SOURCE_DIR}/contrib/rocksdb") message (WARNING "submodule contrib is missing. to fix try run: \n git submodule update --init --recursive") + set (USE_INTERNAL_ROCKSDB_LIBRARY 0) set (MISSING_ROCKSDB 1) endif () From 546d7da12e167fbb5f73af1af3d6a2d50e16a8e4 Mon Sep 17 00:00:00 2001 From: sundy-li <543950155@qq.com> Date: Tue, 22 Sep 2020 21:48:12 +0800 Subject: [PATCH 049/205] update cmake && remove useless codes --- cmake/find/rocksdb.cmake | 14 ++++++-------- src/Storages/Rocksdb/StorageEmbeddedRocksdb.cpp | 10 ---------- 2 files changed, 6 insertions(+), 18 deletions(-) diff --git a/cmake/find/rocksdb.cmake b/cmake/find/rocksdb.cmake index c476b86c9d1..1c055f7f54a 100644 --- a/cmake/find/rocksdb.cmake +++ b/cmake/find/rocksdb.cmake @@ -2,22 +2,20 @@ option(ENABLE_ROCKSDB "Enable ROCKSDB" ${ENABLE_LIBRARIES}) option(USE_INTERNAL_ROCKSDB_LIBRARY "Set to FALSE to use system ROCKSDB library instead of bundled" ${NOT_UNBUNDLED}) if(ENABLE_ROCKSDB) - if (NOT EXISTS "${ClickHouse_SOURCE_DIR}/contrib/rocksdb") - message (WARNING "submodule contrib is missing. to fix try run: \n git submodule update --init --recursive") - set (USE_INTERNAL_ROCKSDB_LIBRARY 0) - set (MISSING_ROCKSDB 1) - endif () - if (USE_INTERNAL_ROCKSDB_LIBRARY AND NOT MISSING_ROCKSDB) set (ROCKSDB_INCLUDE_DIR "${ClickHouse_SOURCE_DIR}/contrib/rocksdb/include") set (ROCKSDB_CORE_INCLUDE_DIR "${ClickHouse_SOURCE_DIR}/contrib/rocksdb/include") set (ROCKSDB_LIBRARY "rocksdb") - set (USE_INTERNAL_ROCKSDB_LIBRARY 1) set (USE_ROCKSDB 1) else() find_package(ROCKSDB) endif () - endif() +if (NOT EXISTS "${ClickHouse_SOURCE_DIR}/contrib/rocksdb/CMakeLists.txt") + message (WARNING "submodule contrib is missing. to fix try run: \n git submodule update --init --recursive") + set (USE_INTERNAL_ROCKSDB_LIBRARY 0) + set (MISSING_ROCKSDB 1) +endif () + message (STATUS "Using ROCKSDB=${USE_ROCKSDB}: ${ROCKSDB_INCLUDE_DIR} : ${ROCKSDB_LIBRARY}") \ No newline at end of file diff --git a/src/Storages/Rocksdb/StorageEmbeddedRocksdb.cpp b/src/Storages/Rocksdb/StorageEmbeddedRocksdb.cpp index 5a99ed716b6..a4832ec8edb 100644 --- a/src/Storages/Rocksdb/StorageEmbeddedRocksdb.cpp +++ b/src/Storages/Rocksdb/StorageEmbeddedRocksdb.cpp @@ -132,16 +132,6 @@ public: wb_value.restart(); } - - rocksdb::Iterator* it = storage.rocksdb_ptr->rocksdb->NewIterator(rocksdb::ReadOptions()); - for (it->SeekToFirst(); it->Valid(); it->Next()) - { - LOG_DEBUG(&Poco::Logger::get("StorageEmbeddedRocksdb"), "Iterator `{}` returns `{}`, {}", - it->key().ToString(), it->value().ToString(), it->key().size()); - } - // Check for any errors found during the scan - assert(it->status().ok()); - delete it; } private: From e3ec5224ab1cd68acdba62b0e18eedfd8497d451 Mon Sep 17 00:00:00 2001 From: sundy-li <543950155@qq.com> Date: Tue, 22 Sep 2020 22:41:26 +0800 Subject: [PATCH 050/205] disable WITH_GFLAGS --- contrib/CMakeLists.txt | 1 + tests/queries/0_stateless/arcadia_skip_list.txt | 1 + 2 files changed, 2 insertions(+) diff --git a/contrib/CMakeLists.txt b/contrib/CMakeLists.txt index 41008adce90..182057dddcf 100644 --- a/contrib/CMakeLists.txt +++ b/contrib/CMakeLists.txt @@ -323,5 +323,6 @@ if (USE_INTERNAL_ROCKSDB_LIBRARY) set(WITH_TESTS OFF) set(WITH_BENCHMARK_TOOLS OFF) set(WITH_TOOLS OFF) + set (WITH_GFLAGS OFF) add_subdirectory (rocksdb) endif() diff --git a/tests/queries/0_stateless/arcadia_skip_list.txt b/tests/queries/0_stateless/arcadia_skip_list.txt index 900cc82b33f..66a8bb3bab0 100644 --- a/tests/queries/0_stateless/arcadia_skip_list.txt +++ b/tests/queries/0_stateless/arcadia_skip_list.txt @@ -150,6 +150,7 @@ 00609_mv_index_in_in 00510_materizlized_view_and_deduplication_zookeeper 00738_lock_for_inner_table +01504_rocksdb 01515_force_data_skipping_indices 01526_complex_key_dict_direct_layout 01509_dictionary_preallocate From 09bcc166696a6d450970eecbcc6790fbae5dee5e Mon Sep 17 00:00:00 2001 From: sundy-li <543950155@qq.com> Date: Thu, 1 Oct 2020 18:45:53 +0800 Subject: [PATCH 051/205] add primary key for StorageEmbeddedRocksdb --- .../Rocksdb/StorageEmbeddedRocksdb.cpp | 206 ++++++++++-------- src/Storages/Rocksdb/StorageEmbeddedRocksdb.h | 30 +-- .../0_stateless/01504_rocksdb.reference | 11 +- tests/queries/0_stateless/01504_rocksdb.sql | 33 ++- 4 files changed, 159 insertions(+), 121 deletions(-) diff --git a/src/Storages/Rocksdb/StorageEmbeddedRocksdb.cpp b/src/Storages/Rocksdb/StorageEmbeddedRocksdb.cpp index a4832ec8edb..55a2b863680 100644 --- a/src/Storages/Rocksdb/StorageEmbeddedRocksdb.cpp +++ b/src/Storages/Rocksdb/StorageEmbeddedRocksdb.cpp @@ -8,6 +8,7 @@ #include #include #include +#include #include #include #include @@ -21,9 +22,12 @@ #include #include #include +#include #include #include +#include +#include namespace DB { @@ -36,13 +40,21 @@ namespace ErrorCodes } -static bool extractKeyImpl(const IAST & elem, Strings & res) +// returns keys may be filter by condition +static bool traverseASTFilter(const String & primary_key, const ASTPtr & elem, FieldVector & res) { - const auto * function = elem.as(); + const auto * function = elem->as(); if (!function) return false; - if (function->name == "equals" || function->name == "in") + if (function->name == "and") + { + for (const auto & child : function->arguments->children) + if (traverseASTFilter(primary_key, child, res)) + return true; + return false; + } + else if (function->name == "equals" || function->name == "in") { const auto & args = function->arguments->as(); const IAST * value; @@ -58,22 +70,22 @@ static bool extractKeyImpl(const IAST & elem, Strings & res) else return false; - if (ident->name != "key") + if (ident->name != primary_key) return false; if (const auto * literal = value->as()) { - if (literal->value.getType() == Field::Types::String) + if (function->name == "equals") { - res.push_back(literal->value.safeGet()); + res.push_back(literal->value); return true; } - else if (literal->value.getType() == Field::Types::Tuple) + else if (function->name == "in" && literal->value.getType() == Field::Types::Tuple) { auto tuple = literal->value.safeGet(); for (const auto & f : tuple) { - res.push_back(f.safeGet()); + res.push_back(f); } return true; } @@ -84,20 +96,22 @@ static bool extractKeyImpl(const IAST & elem, Strings & res) } -/** Retrieve from the query a condition of the form `key = 'key'` or `key in ('xxx_'), from conjunctions in the WHERE clause. +/** Retrieve from the query a condition of the form `key = 'key'`, `key in ('xxx_'), from conjunctions in the WHERE clause. + * TODO support key like search */ -static std::pair extractKey(const ASTPtr & query) +static std::pair getFilterKeys(const String & primary_key, const ASTPtr & query) { const auto & select = query->as(); if (!select.where()) { - return std::make_pair(Strings{}, true); + return std::make_pair(FieldVector{}, true); } - Strings res; - extractKeyImpl(*select.where(), res); - return std::make_pair(res, false); + FieldVector res; + auto matched_keys = traverseASTFilter(primary_key, select.where(), res); + return std::make_pair(res, !matched_keys); } + class EmbeddedRocksdbBlockOutputStream : public IBlockOutputStream { public: @@ -114,24 +128,32 @@ public: { metadata_snapshot->check(block, true); auto rows = block.rows(); - auto key_col = block.getByName("key"); - auto val_col = block.getByName("value"); - - const ColumnString * keys = checkAndGetColumn(key_col.column.get()); + WriteBufferFromOwnString wb_key; WriteBufferFromOwnString wb_value; + rocksdb::WriteBatch batch; + auto columns = metadata_snapshot->getColumns(); + for (size_t i = 0; i < rows; i++) { - StringRef key = keys->getDataAt(i); - val_col.type->serializeBinary(*val_col.column, i, wb_value); - - auto status = storage.rocksdb_ptr->rocksdb->Put(rocksdb::WriteOptions(), key.toString(), wb_value.str()); - if (!status.ok()) - throw Exception("Rocksdb write error: " + status.ToString(), ErrorCodes::SYSTEM_ERROR); - + wb_key.restart(); wb_value.restart(); + + for (const auto & col : columns) + { + const auto & type = block.getByName(col.name).type; + const auto & column = block.getByName(col.name).column; + if (col.name == storage.primary_key) + type->serializeBinary(*column, i, wb_key); + else + type->serializeBinary(*column, i, wb_value); + } + batch.Put(wb_key.str(), wb_value.str()); } + auto status = storage.rocksdb_ptr->Write(rocksdb::WriteOptions(), &batch); + if (!status.ok()) + throw Exception("Rocksdb write error: " + status.ToString(), ErrorCodes::SYSTEM_ERROR); } private: @@ -139,23 +161,17 @@ private: StorageMetadataPtr metadata_snapshot; }; -StorageEmbeddedRocksdb::StorageEmbeddedRocksdb(const StorageFactory::Arguments & args) - : IStorage(args.table_id) +StorageEmbeddedRocksdb::StorageEmbeddedRocksdb(const StorageID & table_id_, + const String & relative_data_path_, + const StorageInMemoryMetadata & metadata_, + bool attach, + Context & context_, + const String & primary_key_) + : IStorage(table_id_), primary_key{primary_key_} { - //must contains two columns, key and value - if (args.columns.size() != 2) - throw Exception("Storage " + getName() + " requires exactly 2 columns", ErrorCodes::BAD_ARGUMENTS); - - if (!args.columns.has("key") || !args.columns.has("value")) - throw Exception("Storage " + getName() + " requires columns are: key and value", ErrorCodes::BAD_ARGUMENTS); - - StorageInMemoryMetadata storage_metadata; - storage_metadata.setColumns(args.columns); - storage_metadata.setConstraints(args.constraints); - setInMemoryMetadata(storage_metadata); - - rocksdb_dir = args.context.getPath() + args.relative_data_path + "/rocksdb"; - if (!args.attach) + setInMemoryMetadata(metadata_); + rocksdb_dir = context_.getPath() + relative_data_path_ + "/rocksdb"; + if (!attach) { Poco::File(rocksdb_dir).createDirectories(); } @@ -164,10 +180,7 @@ StorageEmbeddedRocksdb::StorageEmbeddedRocksdb(const StorageFactory::Arguments & void StorageEmbeddedRocksdb::truncate(const ASTPtr &, const StorageMetadataPtr & , const Context &, TableExclusiveLockHolder &) { - if (rocksdb_ptr) - { - rocksdb_ptr->shutdown(); - } + rocksdb_ptr->Close(); Poco::File(rocksdb_dir).remove(true); Poco::File(rocksdb_dir).createDirectories(); initDb(); @@ -176,20 +189,13 @@ void StorageEmbeddedRocksdb::truncate(const ASTPtr &, const StorageMetadataPtr & void StorageEmbeddedRocksdb::initDb() { rocksdb::Options options; - rocksdb::BlockBasedTableOptions table_options; - rocksdb::DB * db; options.create_if_missing = true; - options.statistics = rocksdb::CreateDBStatistics(); - auto cache = rocksdb::NewLRUCache(256 << 20); - table_options.block_cache = cache; - options.table_factory.reset(rocksdb::NewBlockBasedTableFactory(table_options)); - rocksdb::Status status = rocksdb::DB::Open(options, rocksdb_dir, &db); if (status != rocksdb::Status::OK()) throw Exception("Fail to open rocksdb path at: " + rocksdb_dir, ErrorCodes::SYSTEM_ERROR); - rocksdb_ptr = std::make_shared(db); + rocksdb_ptr = std::unique_ptr(db); } @@ -204,49 +210,61 @@ Pipe StorageEmbeddedRocksdb::read( { metadata_snapshot->check(column_names, getVirtuals(), getStorageID()); Block sample_block = metadata_snapshot->getSampleBlock(); - - size_t key_pos = 0; - size_t value_pos = 1; - if (sample_block.getByPosition(0).name != "key") - std::swap(key_pos, value_pos); - MutableColumns columns = sample_block.cloneEmptyColumns(); - Strings keys; + FieldVector keys; bool all_scan = false; - std::tie(keys, all_scan) = extractKey(query_info.query); - if (keys.empty() && !all_scan) - throw Exception("StorageEmbeddedRocksdb engine must contain condition like key = 'key' or key in tuple(String) or key like 'xxx%' in WHERE clause or empty WHERE clause for all key value scan.", ErrorCodes::BAD_ARGUMENTS); + std::tie(keys, all_scan) = getFilterKeys(primary_key, query_info.query); // TODO pipline if (all_scan) { - auto it = rocksdb_ptr->rocksdb->NewIterator(rocksdb::ReadOptions()); + auto it = std::unique_ptr(rocksdb_ptr->NewIterator(rocksdb::ReadOptions())); for (it->SeekToFirst(); it->Valid(); it->Next()) { - ReadBufferFromString rvalue(it->value().ToString()); - columns[key_pos]->insert(it->key().ToString()); - sample_block.getByName("value").type->deserializeBinary(*columns[value_pos], rvalue); + ReadBufferFromString key_buffer(it->key()); + ReadBufferFromString value_buffer(it->value()); + + for (const auto & item : ext::enumerate(sample_block.getColumnsWithTypeAndName())) + { + if (item.second.name == primary_key) + item.second.type->deserializeBinary(*columns[item.first], key_buffer); + else + item.second.type->deserializeBinary(*columns[item.first], value_buffer); + } } assert(it->status().ok()); - delete it; } else { - Strings values; std::vector slices_keys; - for (auto & key : keys) + std::vector values; + + WriteBufferFromOwnString wb; + UInt64 offset = 0; + for (const auto & key : keys) { - slices_keys.push_back(key); + sample_block.getByName(primary_key).type->serializeBinary(key, wb); + auto str_ref = wb.stringRef(); + slices_keys.emplace_back(str_ref.data + offset, str_ref.size - offset); + offset = str_ref.size; } - auto statuses = rocksdb_ptr->rocksdb->MultiGet(rocksdb::ReadOptions(), slices_keys, &values); + + auto statuses = rocksdb_ptr->MultiGet(rocksdb::ReadOptions(), slices_keys, &values); for (size_t i = 0; i < statuses.size(); ++i) { if (statuses[i].ok()) { - ReadBufferFromString rvalue(values[i]); - columns[key_pos]->insert(keys[i]); - sample_block.getByName("value").type->deserializeBinary(*columns[value_pos], rvalue); + ReadBufferFromString key_buffer(slices_keys[i]); + ReadBufferFromString value_buffer(values[i]); + + for (const auto & item : ext::enumerate(sample_block.getColumnsWithTypeAndName())) + { + if (item.second.name == primary_key) + item.second.type->deserializeBinary(*columns[item.first], key_buffer); + else + item.second.type->deserializeBinary(*columns[item.first], value_buffer); + } } } } @@ -263,23 +281,41 @@ BlockOutputStreamPtr StorageEmbeddedRocksdb::write(const ASTPtr & /*query*/, con StorageEmbeddedRocksdb::~StorageEmbeddedRocksdb() { - if (rocksdb_ptr) +} + + +static StoragePtr create(const StorageFactory::Arguments & args) +{ + // TODO RocksdbSettings + if (!args.engine_args.empty()) + throw Exception( + "Engine " + args.engine_name + " doesn't support any arguments (" + toString(args.engine_args.size()) + " given)", + ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); + + StorageInMemoryMetadata metadata; + metadata.setColumns(args.columns); + metadata.setConstraints(args.constraints); + + if (!args.storage_def->primary_key) + throw Exception("StorageEmbeddedRocksdb must require one primary key", ErrorCodes::BAD_ARGUMENTS); + + metadata.primary_key = KeyDescription::getKeyFromAST(args.storage_def->primary_key->ptr(), metadata.columns, args.context); + auto primary_key_names = metadata.getColumnsRequiredForPrimaryKey(); + if (primary_key_names.size() != 1) { - rocksdb_ptr->shutdown(); + throw Exception("StorageEmbeddedRocksdb must require one primary key", ErrorCodes::BAD_ARGUMENTS); } + return StorageEmbeddedRocksdb::create(args.table_id, args.relative_data_path, metadata, args.attach, args.context, primary_key_names[0]); } void registerStorageEmbeddedRocksdb(StorageFactory & factory) { - factory.registerStorage("EmbeddedRocksdb", [](const StorageFactory::Arguments & args) - { - if (!args.engine_args.empty()) - throw Exception( - "Engine " + args.engine_name + " doesn't support any arguments (" + toString(args.engine_args.size()) + " given)", - ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); - return StorageEmbeddedRocksdb::create(args); - }); + StorageFactory::StorageFeatures features{ + .supports_sort_order = true, + }; + + factory.registerStorage("EmbeddedRocksdb", create, features); } diff --git a/src/Storages/Rocksdb/StorageEmbeddedRocksdb.h b/src/Storages/Rocksdb/StorageEmbeddedRocksdb.h index 546e7e47dc6..8e050867929 100644 --- a/src/Storages/Rocksdb/StorageEmbeddedRocksdb.h +++ b/src/Storages/Rocksdb/StorageEmbeddedRocksdb.h @@ -15,24 +15,6 @@ namespace DB class Context; -struct Rocksdb -{ - rocksdb::DB * rocksdb; - explicit Rocksdb(rocksdb::DB * rocksdb_) : rocksdb{rocksdb_} {} - Rocksdb(const Rocksdb &) = delete; - Rocksdb & operator=(const Rocksdb &) = delete; - - void shutdown() - { - if (rocksdb) - { - rocksdb->Close(); - delete rocksdb; - } - } -}; - - class StorageEmbeddedRocksdb final : public ext::shared_ptr_helper, public IStorage { friend struct ext::shared_ptr_helper; @@ -55,12 +37,18 @@ public: void truncate(const ASTPtr &, const StorageMetadataPtr & metadata_snapshot, const Context &, TableExclusiveLockHolder &) override; protected: - StorageEmbeddedRocksdb(const StorageFactory::Arguments & args); + StorageEmbeddedRocksdb(const StorageID & table_id_, + const String & relative_data_path_, + const StorageInMemoryMetadata & metadata, + bool attach, + Context & context_, + const String & primary_key_); private: - using RocksdbPtr = std::shared_ptr; - String rocksdb_dir; + const String primary_key; + using RocksdbPtr = std::unique_ptr; RocksdbPtr rocksdb_ptr; + String rocksdb_dir; mutable std::shared_mutex rwlock; void initDb(); diff --git a/tests/queries/0_stateless/01504_rocksdb.reference b/tests/queries/0_stateless/01504_rocksdb.reference index 2723ca095f1..ae092fcbc30 100644 --- a/tests/queries/0_stateless/01504_rocksdb.reference +++ b/tests/queries/0_stateless/01504_rocksdb.reference @@ -1,6 +1,7 @@ 1 -1000 -0 435761734006 -1 435761734006 -2 435761734006 -0 +1 +0 0 0 0 0 +1 +1 +1 +1 diff --git a/tests/queries/0_stateless/01504_rocksdb.sql b/tests/queries/0_stateless/01504_rocksdb.sql index 84add73b15c..f78f6e2f658 100644 --- a/tests/queries/0_stateless/01504_rocksdb.sql +++ b/tests/queries/0_stateless/01504_rocksdb.sql @@ -1,22 +1,35 @@ DROP TABLE IF EXISTS test; -create table test (key String, value UInt32) Engine=EmbeddedRocksdb; +CREATE TABLE test (key String, value UInt32) Engine=EmbeddedRocksdb primary key(key); -insert into test select '1_1', number from numbers(10000); -select count(1) from test; +INSERT INTO test SELECT '1_1', number FROM numbers(10000); +SELECT count(1) == 1 FROM test; -insert into test select concat(toString(number), '_1'), number from numbers(10000); -select sum(value) from test where key in ('1_1', '99_1', '900_1'); +INSERT INTO test SELECT concat(toString(number), '_1'), number FROM numbers(10000); +SELECT SUM(value) == 1 + 99 + 900 FROM test WHERE key in ('1_1', '99_1', '900_1'); DROP TABLE IF EXISTS test; -create table test (key String, value UInt64) Engine=EmbeddedRocksdb; +DROP TABLE IF EXISTS test_memory; -insert into test select toString(number%3) as key, sum(number) as value from numbers(1000000) group by key; +CREATE TABLE test (k UInt32, value UInt64, dummy Tuple(UInt32, Float64), bm AggregateFunction(groupBitmap, UInt64)) Engine=EmbeddedRocksdb primary key(k); +CREATE TABLE test_memory AS test Engine = Memory; -select key, sum(value) from test group by key order by key; +INSERT INTO test SELECT number % 77 AS k, SUM(number) AS value, (1, 1.2), bitmapBuild(groupArray(number)) FROM numbers(10000000) group by k; -truncate table test; -select count(1) from test; +INSERT INTO test_memory SELECT number % 77 AS k, SUM(number) AS value, (1, 1.2), bitmapBuild(groupArray(number)) FROM numbers(10000000) group by k; + + +SELECT A.a - B.a, A.b - B.b, A.c - B.c, A.d - B.d, A.e - B.e FROM ( SELECT 0 AS a, groupBitmapMerge(bm) AS b , SUM(k) AS c, SUM(value) AS d, SUM(dummy.1) AS e FROM test) A ANY LEFT JOIN (SELECT 0 AS a, groupBitmapMerge(bm) AS b , SUM(k) AS c, SUM(value) AS d, SUM(dummy.1) AS e FROM test_memory) B USING a ORDER BY a; + + +SET max_rows_to_read = 2; +SELECT dummy == (1,1.2) FROM test WHERE k IN (1, 3); +SELECT k == 4 FROM test WHERE k = 4; +SELECT k, value FROM test WHERE k = 0 OR value > 0; -- { serverError 158 } + +TRUNCATE TABLE test; +SELECT 0 == count(1) FROM test; DROP TABLE IF EXISTS test; +DROP TABLE IF EXISTS test_memory; From 87eb69da46fb207d1324712333483467b966607d Mon Sep 17 00:00:00 2001 From: sundy-li <543950155@qq.com> Date: Thu, 1 Oct 2020 18:59:51 +0800 Subject: [PATCH 052/205] add docs --- .../integrations/embedded-rocksdb.md | 44 +++++++++++++++++++ 1 file changed, 44 insertions(+) create mode 100644 docs/en/engines/table-engines/integrations/embedded-rocksdb.md diff --git a/docs/en/engines/table-engines/integrations/embedded-rocksdb.md b/docs/en/engines/table-engines/integrations/embedded-rocksdb.md new file mode 100644 index 00000000000..0bd4123c0cf --- /dev/null +++ b/docs/en/engines/table-engines/integrations/embedded-rocksdb.md @@ -0,0 +1,44 @@ +--- +toc_priority: 6 +toc_title: EmbeddedRocksdb +--- + +# EmbeddedRocksdb Engine {#EmbeddedRocksdb-engine} + +This engine allows integrating ClickHouse with [rocksdb](http://rocksdb.org/). + +`EmbeddedRocksdb` lets you: + +## Creating a Table {#table_engine-EmbeddedRocksdb-creating-a-table} + +``` sql +CREATE TABLE [IF NOT EXISTS] [db.]table_name [ON CLUSTER cluster] +( + name1 [type1] [DEFAULT|MATERIALIZED|ALIAS expr1], + name2 [type2] [DEFAULT|MATERIALIZED|ALIAS expr2], + ... +) ENGINE = EmbeddedRocksdb PRIMARY KEY(primary_key_name) +``` + +Required parameters: + +- `primary_key_name` – any column name in the column list. + +Example: + +``` sql +CREATE TABLE test +( + `key` String, + `v1` UInt32, + `v2` String, + `v3` Float32, +) +ENGINE = EmbeddedRocksdb +PRIMARY KEY key +``` + +## Description {#description} + +- `primary key` must be specified, only support one primary key. The primary key will serializeBinary as rocksdb key. +- Columns other than the primary key will be serializeBinary as rocksdb value in corresponding order. From df9a003cf78aae319029e1872c2772467ccea722 Mon Sep 17 00:00:00 2001 From: sundy-li <543950155@qq.com> Date: Thu, 1 Oct 2020 19:00:03 +0800 Subject: [PATCH 053/205] add docs --- docs/en/engines/table-engines/integrations/embedded-rocksdb.md | 1 + 1 file changed, 1 insertion(+) diff --git a/docs/en/engines/table-engines/integrations/embedded-rocksdb.md b/docs/en/engines/table-engines/integrations/embedded-rocksdb.md index 0bd4123c0cf..88a95ae7de7 100644 --- a/docs/en/engines/table-engines/integrations/embedded-rocksdb.md +++ b/docs/en/engines/table-engines/integrations/embedded-rocksdb.md @@ -42,3 +42,4 @@ PRIMARY KEY key - `primary key` must be specified, only support one primary key. The primary key will serializeBinary as rocksdb key. - Columns other than the primary key will be serializeBinary as rocksdb value in corresponding order. +- Queries with key `equals` or `in` filtering will be optimized to multi keys look up from rocksdb. From 1f4a63b6c6a168562eea95859139f6e5600aeec3 Mon Sep 17 00:00:00 2001 From: sundy-li <543950155@qq.com> Date: Thu, 1 Oct 2020 22:14:52 +0800 Subject: [PATCH 054/205] skip rocksdb test in fasttest --- docker/test/fasttest/run.sh | 3 +++ .../Rocksdb/StorageEmbeddedRocksdb.cpp | 20 ++++++++----------- src/Storages/Rocksdb/StorageEmbeddedRocksdb.h | 2 -- 3 files changed, 11 insertions(+), 14 deletions(-) diff --git a/docker/test/fasttest/run.sh b/docker/test/fasttest/run.sh index bcba50e9d81..c1c76b93829 100755 --- a/docker/test/fasttest/run.sh +++ b/docker/test/fasttest/run.sh @@ -270,6 +270,9 @@ TESTS_TO_SKIP=( 00646_url_engine 00974_query_profiler + # Rocksdb is not enabled by default + 01504_rocksdb + # Look at DistributedFilesToInsert, so cannot run in parallel. 01460_DistributedFilesToInsert diff --git a/src/Storages/Rocksdb/StorageEmbeddedRocksdb.cpp b/src/Storages/Rocksdb/StorageEmbeddedRocksdb.cpp index 55a2b863680..72602de68e9 100644 --- a/src/Storages/Rocksdb/StorageEmbeddedRocksdb.cpp +++ b/src/Storages/Rocksdb/StorageEmbeddedRocksdb.cpp @@ -225,12 +225,12 @@ Pipe StorageEmbeddedRocksdb::read( ReadBufferFromString key_buffer(it->key()); ReadBufferFromString value_buffer(it->value()); - for (const auto & item : ext::enumerate(sample_block.getColumnsWithTypeAndName())) + for (const auto [idx, column_type] : ext::enumerate(sample_block.getColumnsWithTypeAndName())) { - if (item.second.name == primary_key) - item.second.type->deserializeBinary(*columns[item.first], key_buffer); + if (column_type.name == primary_key) + column_type.type->deserializeBinary(*columns[idx], key_buffer); else - item.second.type->deserializeBinary(*columns[item.first], value_buffer); + column_type.type->deserializeBinary(*columns[idx], value_buffer); } } assert(it->status().ok()); @@ -258,12 +258,12 @@ Pipe StorageEmbeddedRocksdb::read( ReadBufferFromString key_buffer(slices_keys[i]); ReadBufferFromString value_buffer(values[i]); - for (const auto & item : ext::enumerate(sample_block.getColumnsWithTypeAndName())) + for (const auto [idx, column_type] : ext::enumerate(sample_block.getColumnsWithTypeAndName())) { - if (item.second.name == primary_key) - item.second.type->deserializeBinary(*columns[item.first], key_buffer); + if (column_type.name == primary_key) + column_type.type->deserializeBinary(*columns[idx], key_buffer); else - item.second.type->deserializeBinary(*columns[item.first], value_buffer); + column_type.type->deserializeBinary(*columns[idx], value_buffer); } } } @@ -279,10 +279,6 @@ BlockOutputStreamPtr StorageEmbeddedRocksdb::write(const ASTPtr & /*query*/, con return std::make_shared(*this, metadata_snapshot); } -StorageEmbeddedRocksdb::~StorageEmbeddedRocksdb() -{ -} - static StoragePtr create(const StorageFactory::Arguments & args) { diff --git a/src/Storages/Rocksdb/StorageEmbeddedRocksdb.h b/src/Storages/Rocksdb/StorageEmbeddedRocksdb.h index 8e050867929..7c8d1f9c47e 100644 --- a/src/Storages/Rocksdb/StorageEmbeddedRocksdb.h +++ b/src/Storages/Rocksdb/StorageEmbeddedRocksdb.h @@ -22,8 +22,6 @@ class StorageEmbeddedRocksdb final : public ext::shared_ptr_helper Date: Fri, 2 Oct 2020 10:26:59 +0800 Subject: [PATCH 055/205] USE PORTABLE to avoid Illegal instruction, add SANITIZE for rocksdb cmake build options --- contrib/CMakeLists.txt | 13 ++++++++++++- 1 file changed, 12 insertions(+), 1 deletion(-) diff --git a/contrib/CMakeLists.txt b/contrib/CMakeLists.txt index 182057dddcf..6ccea277558 100644 --- a/contrib/CMakeLists.txt +++ b/contrib/CMakeLists.txt @@ -323,6 +323,17 @@ if (USE_INTERNAL_ROCKSDB_LIBRARY) set(WITH_TESTS OFF) set(WITH_BENCHMARK_TOOLS OFF) set(WITH_TOOLS OFF) - set (WITH_GFLAGS OFF) + set(WITH_GFLAGS OFF) + set(PORTABLE ON) + set(FORCE_SSE42 ON) + + if (SANITIZE STREQUAL "undefined") + set(WITH_UBSAN ON) + elseif (SANITIZE STREQUAL "address") + set(WITH_ASAN ON) + elseif (SANITIZE STREQUAL "thread") + set(WITH_TSAN ON) + endif() + add_subdirectory (rocksdb) endif() From 88250fec5d121b0d494c74c5443f309b5f3a9c35 Mon Sep 17 00:00:00 2001 From: sundy-li <543950155@qq.com> Date: Sun, 4 Oct 2020 10:24:08 +0800 Subject: [PATCH 056/205] add sync_file_range for glibc-compatibility --- .../musl/sync_file_range.c | 21 ++++++++++ cmake/find/rocksdb.cmake | 41 +++++++++++++------ docker/packager/unbundled/Dockerfile | 1 + src/CMakeLists.txt | 2 - src/Storages/Rocksdb/StorageEmbeddedRocksdb.h | 1 - 5 files changed, 51 insertions(+), 15 deletions(-) create mode 100644 base/glibc-compatibility/musl/sync_file_range.c diff --git a/base/glibc-compatibility/musl/sync_file_range.c b/base/glibc-compatibility/musl/sync_file_range.c new file mode 100644 index 00000000000..610b11c8c96 --- /dev/null +++ b/base/glibc-compatibility/musl/sync_file_range.c @@ -0,0 +1,21 @@ +#define _GNU_SOURCE +#include +#include +#include "syscall.h" + +// works same in x86_64 && aarch64 +#define __SYSCALL_LL_E(x) (x) +#define __SYSCALL_LL_O(x) (x) + +int sync_file_range(int fd, off_t pos, off_t len, unsigned flags) +{ +#if defined(SYS_sync_file_range2) + return syscall(SYS_sync_file_range2, fd, flags, + __SYSCALL_LL_E(pos), __SYSCALL_LL_E(len)); +#elif defined(SYS_sync_file_range) + return __syscall(SYS_sync_file_range, fd, + __SYSCALL_LL_O(pos), __SYSCALL_LL_E(len), flags); +#else + return __syscall_ret(-ENOSYS); +#endif +} \ No newline at end of file diff --git a/cmake/find/rocksdb.cmake b/cmake/find/rocksdb.cmake index 1c055f7f54a..d53febee762 100644 --- a/cmake/find/rocksdb.cmake +++ b/cmake/find/rocksdb.cmake @@ -1,21 +1,38 @@ option(ENABLE_ROCKSDB "Enable ROCKSDB" ${ENABLE_LIBRARIES}) -option(USE_INTERNAL_ROCKSDB_LIBRARY "Set to FALSE to use system ROCKSDB library instead of bundled" ${NOT_UNBUNDLED}) -if(ENABLE_ROCKSDB) - if (USE_INTERNAL_ROCKSDB_LIBRARY AND NOT MISSING_ROCKSDB) - set (ROCKSDB_INCLUDE_DIR "${ClickHouse_SOURCE_DIR}/contrib/rocksdb/include") - set (ROCKSDB_CORE_INCLUDE_DIR "${ClickHouse_SOURCE_DIR}/contrib/rocksdb/include") - set (ROCKSDB_LIBRARY "rocksdb") - set (USE_ROCKSDB 1) - else() - find_package(ROCKSDB) - endif () +if (NOT ENABLE_ROCKSDB) + if (USE_INTERNAL_ROCKSDB_LIBRARY) + message (${RECONFIGURE_MESSAGE_LEVEL} "Can't use internal rocksdb library with ENABLE_ROCKSDB=OFF") + endif() + return() endif() +option(USE_INTERNAL_ROCKSDB_LIBRARY "Set to FALSE to use system ROCKSDB library instead of bundled" ${NOT_UNBUNDLED}) + if (NOT EXISTS "${ClickHouse_SOURCE_DIR}/contrib/rocksdb/CMakeLists.txt") - message (WARNING "submodule contrib is missing. to fix try run: \n git submodule update --init --recursive") - set (USE_INTERNAL_ROCKSDB_LIBRARY 0) + if (USE_INTERNAL_ROCKSDB_LIBRARY) + message (WARNING "submodule contrib is missing. to fix try run: \n git submodule update --init --recursive") + message(${RECONFIGURE_MESSAGE_LEVEL} "cannot find internal rocksdb") + set (USE_INTERNAL_ROCKSDB_LIBRARY 0) + endif() set (MISSING_ROCKSDB 1) endif () +if (NOT USE_INTERNAL_ROCKSDB_LIBRARY) + find_library (ROCKSDB_LIBRARY rocksdb) + find_path (ROCKSDB_INCLUDE_DIR NAMES rocksdb/db.h PATHS ${ROCKSDB_INCLUDE_PATHS}) + if (NOT ROCKSDB_LIBRARY OR NOT ROCKSDB_INCLUDE_DIR) + message (${RECONFIGURE_MESSAGE_LEVEL} "Can't find system rocksdb library") + endif() +endif () + +if (NOT ROCKSDB_LIBRARY AND NOT MISSING_ROCKSDB) + set (USE_INTERNAL_ROCKSDB_LIBRARY 1) + + set (ROCKSDB_INCLUDE_DIR "${ClickHouse_SOURCE_DIR}/contrib/rocksdb/include") + set (ROCKSDB_CORE_INCLUDE_DIR "${ClickHouse_SOURCE_DIR}/contrib/rocksdb/include") + set (ROCKSDB_LIBRARY "rocksdb") + set (USE_ROCKSDB 1) +endif () + message (STATUS "Using ROCKSDB=${USE_ROCKSDB}: ${ROCKSDB_INCLUDE_DIR} : ${ROCKSDB_LIBRARY}") \ No newline at end of file diff --git a/docker/packager/unbundled/Dockerfile b/docker/packager/unbundled/Dockerfile index 50671011a23..698a7e2621b 100644 --- a/docker/packager/unbundled/Dockerfile +++ b/docker/packager/unbundled/Dockerfile @@ -64,6 +64,7 @@ RUN apt-get update \ libbz2-dev \ libavro-dev \ libfarmhash-dev \ + librocksdb-dev \ libmysqlclient-dev \ --yes --no-install-recommends diff --git a/src/CMakeLists.txt b/src/CMakeLists.txt index 9518810a703..3af10090f20 100644 --- a/src/CMakeLists.txt +++ b/src/CMakeLists.txt @@ -409,8 +409,6 @@ endif () if (USE_ROCKSDB) dbms_target_link_libraries(PUBLIC ${ROCKSDB_LIBRARY}) dbms_target_include_directories(SYSTEM BEFORE PUBLIC ${ROCKSDB_INCLUDE_DIR}) - # target_link_libraries (clickhouse_common_io PUBLIC ${ROCKSDB_LIBRARY}) - # target_include_directories (clickhouse_common_io SYSTEM BEFORE PUBLIC ${ROCKSDB_INCLUDE_DIR}) endif() if (ENABLE_TESTS AND USE_GTEST) diff --git a/src/Storages/Rocksdb/StorageEmbeddedRocksdb.h b/src/Storages/Rocksdb/StorageEmbeddedRocksdb.h index 7c8d1f9c47e..79e7d1678e0 100644 --- a/src/Storages/Rocksdb/StorageEmbeddedRocksdb.h +++ b/src/Storages/Rocksdb/StorageEmbeddedRocksdb.h @@ -47,7 +47,6 @@ private: using RocksdbPtr = std::unique_ptr; RocksdbPtr rocksdb_ptr; String rocksdb_dir; - mutable std::shared_mutex rwlock; void initDb(); }; From 70ce9fb877a924eda958ce6401ee05acd304429e Mon Sep 17 00:00:00 2001 From: sundy-li <543950155@qq.com> Date: Tue, 6 Oct 2020 09:26:29 +0800 Subject: [PATCH 057/205] trigger --- contrib/CMakeLists.txt | 1 - 1 file changed, 1 deletion(-) diff --git a/contrib/CMakeLists.txt b/contrib/CMakeLists.txt index 6ccea277558..e5014f3c229 100644 --- a/contrib/CMakeLists.txt +++ b/contrib/CMakeLists.txt @@ -325,7 +325,6 @@ if (USE_INTERNAL_ROCKSDB_LIBRARY) set(WITH_TOOLS OFF) set(WITH_GFLAGS OFF) set(PORTABLE ON) - set(FORCE_SSE42 ON) if (SANITIZE STREQUAL "undefined") set(WITH_UBSAN ON) From 8dd9d166dc6944a68db4c163ba98a179931bbb65 Mon Sep 17 00:00:00 2001 From: sundy-li <543950155@qq.com> Date: Fri, 16 Oct 2020 09:02:16 +0800 Subject: [PATCH 058/205] fix extract keys --- src/Storages/Rocksdb/StorageEmbeddedRocksdb.cpp | 12 +++++------- 1 file changed, 5 insertions(+), 7 deletions(-) diff --git a/src/Storages/Rocksdb/StorageEmbeddedRocksdb.cpp b/src/Storages/Rocksdb/StorageEmbeddedRocksdb.cpp index 72602de68e9..1f97d8512a5 100644 --- a/src/Storages/Rocksdb/StorageEmbeddedRocksdb.cpp +++ b/src/Storages/Rocksdb/StorageEmbeddedRocksdb.cpp @@ -239,15 +239,13 @@ Pipe StorageEmbeddedRocksdb::read( { std::vector slices_keys; std::vector values; + std::vector wbs(keys.size()); - WriteBufferFromOwnString wb; - UInt64 offset = 0; - for (const auto & key : keys) + for (size_t i = 0; i < keys.size(); ++i) { - sample_block.getByName(primary_key).type->serializeBinary(key, wb); - auto str_ref = wb.stringRef(); - slices_keys.emplace_back(str_ref.data + offset, str_ref.size - offset); - offset = str_ref.size; + sample_block.getByName(primary_key).type->serializeBinary(keys[i], wbs[i]); + auto str_ref = wbs[i].stringRef(); + slices_keys.emplace_back(str_ref.data, str_ref.size); } auto statuses = rocksdb_ptr->MultiGet(rocksdb::ReadOptions(), slices_keys, &values); From 4fe13ff404113e19bd2d8d8e201c4bca6a81291c Mon Sep 17 00:00:00 2001 From: sundy-li <543950155@qq.com> Date: Fri, 16 Oct 2020 10:27:56 +0800 Subject: [PATCH 059/205] add rocksdb perf tests --- tests/performance/rocksdb.xml | 18 ++++++++++++++++++ 1 file changed, 18 insertions(+) create mode 100644 tests/performance/rocksdb.xml diff --git a/tests/performance/rocksdb.xml b/tests/performance/rocksdb.xml new file mode 100644 index 00000000000..e0867eece65 --- /dev/null +++ b/tests/performance/rocksdb.xml @@ -0,0 +1,18 @@ + + + + hits_100m_single + + + + 30000000000 + + + CREATE TABLE ma_kv (key String, value AggregateFunction(groupBitmap, UInt64)) EmbeddedRocksdb primary key(key) + + INSERT INTO ma_kv SELECT concat('CodeVersion=', CodeVersion) as key, bitmapBuild(groupArray(UserID)) AS value FROM hits_100m_single GROUP BY key + + SELECT groupBitmapOr(value) FROM ma_kv WHERE key IN ('CodeVersion=1657', 'CodeVersion=1', 'CodeVersion=275') + + DROP TABLE IF EXISTS ma_kv + From c88ede4f973f33bfbb8a11d00f2da6bd9f7efbaf Mon Sep 17 00:00:00 2001 From: sundy-li <543950155@qq.com> Date: Mon, 19 Oct 2020 01:09:00 +0800 Subject: [PATCH 060/205] add supportsIndexForIn && supportsParallelInsert, support parallel reading --- .../Rocksdb/StorageEmbeddedRocksdb.cpp | 207 ++++++++++++++---- src/Storages/Rocksdb/StorageEmbeddedRocksdb.h | 8 + tests/performance/rocksdb.xml | 18 -- .../0_stateless/01504_rocksdb.reference | 1 + tests/queries/0_stateless/01504_rocksdb.sql | 2 + 5 files changed, 170 insertions(+), 66 deletions(-) delete mode 100644 tests/performance/rocksdb.xml diff --git a/src/Storages/Rocksdb/StorageEmbeddedRocksdb.cpp b/src/Storages/Rocksdb/StorageEmbeddedRocksdb.cpp index 1f97d8512a5..127d1349a2c 100644 --- a/src/Storages/Rocksdb/StorageEmbeddedRocksdb.cpp +++ b/src/Storages/Rocksdb/StorageEmbeddedRocksdb.cpp @@ -6,6 +6,7 @@ #include #include #include +#include #include #include #include @@ -18,6 +19,8 @@ #include #include +#include +#include #include #include #include @@ -36,12 +39,13 @@ namespace ErrorCodes { extern const int BAD_ARGUMENTS; extern const int SYSTEM_ERROR; + extern const int LOGICAL_ERROR; extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; } // returns keys may be filter by condition -static bool traverseASTFilter(const String & primary_key, const ASTPtr & elem, FieldVector & res) +static bool traverseASTFilter(const String & primary_key, const ASTPtr & elem, const PreparedSets & sets, FieldVector & res) { const auto * function = elem->as(); if (!function) @@ -50,7 +54,7 @@ static bool traverseASTFilter(const String & primary_key, const ASTPtr & elem, F if (function->name == "and") { for (const auto & child : function->arguments->children) - if (traverseASTFilter(primary_key, child, res)) + if (traverseASTFilter(primary_key, child, sets, res)) return true; return false; } @@ -73,6 +77,27 @@ static bool traverseASTFilter(const String & primary_key, const ASTPtr & elem, F if (ident->name != primary_key) return false; + + if (function->name == "in" && ((value->as() || value->as()))) + { + auto set_it = sets.find(PreparedSetKey::forSubquery(*value)); + if (set_it == sets.end()) + return false; + SetPtr prepared_set = set_it->second; + + if (!prepared_set->hasExplicitSetElements()) + return false; + + prepared_set->checkColumnsNumber(1); + + const auto & set_column = *prepared_set->getSetElements()[0]; + for (size_t row = 0; row < set_column.size(); ++row) + { + res.push_back(set_column[row]); + } + return true; + } + if (const auto * literal = value->as()) { if (function->name == "equals") @@ -80,14 +105,17 @@ static bool traverseASTFilter(const String & primary_key, const ASTPtr & elem, F res.push_back(literal->value); return true; } - else if (function->name == "in" && literal->value.getType() == Field::Types::Tuple) + else if (function->name == "in") { - auto tuple = literal->value.safeGet(); - for (const auto & f : tuple) + if (literal->value.getType() == Field::Types::Tuple) { - res.push_back(f); + auto tuple = literal->value.safeGet(); + for (const auto & f : tuple) + { + res.push_back(f); + } + return true; } - return true; } else return false; } @@ -99,19 +127,104 @@ static bool traverseASTFilter(const String & primary_key, const ASTPtr & elem, F /** Retrieve from the query a condition of the form `key = 'key'`, `key in ('xxx_'), from conjunctions in the WHERE clause. * TODO support key like search */ -static std::pair getFilterKeys(const String & primary_key, const ASTPtr & query) +static std::pair getFilterKeys(const String & primary_key, const SelectQueryInfo & query_info) { - const auto & select = query->as(); + const auto & select = query_info.query->as(); if (!select.where()) { return std::make_pair(FieldVector{}, true); } FieldVector res; - auto matched_keys = traverseASTFilter(primary_key, select.where(), res); + auto matched_keys = traverseASTFilter(primary_key, select.where(), query_info.sets, res); return std::make_pair(res, !matched_keys); } +class EmbeddedRocksdbSource : public SourceWithProgress +{ +public: + EmbeddedRocksdbSource( + const StorageEmbeddedRocksdb & storage_, + const StorageMetadataPtr & metadata_snapshot_, + const FieldVector & keys_, + const size_t start_, + const size_t end_, + const size_t rocksdb_batch_read_size_) + : SourceWithProgress(metadata_snapshot_->getSampleBlock()) + , storage(storage_) + , metadata_snapshot(metadata_snapshot_) + , start(start_) + , end(end_) + , rocksdb_batch_read_size(rocksdb_batch_read_size_) + { + // slice the keys + if (end > start) + { + keys.resize(end - start); + std::copy(keys_.begin() + start, keys_.begin() + end, keys.begin()); + } + } + + String getName() const override + { + return storage.getName(); + } + + Chunk generate() override + { + if (processed_keys >= keys.size() || (start == end)) + return {}; + + std::vector slices_keys; + slices_keys.reserve(keys.size()); + std::vector values; + std::vector wbs(keys.size()); + + const auto & sample_block = metadata_snapshot->getSampleBlock(); + const auto & key_column = sample_block.getByName(storage.primary_key); + auto columns = sample_block.cloneEmptyColumns(); + size_t primary_key_pos = sample_block.getPositionByName(storage.primary_key); + + for (size_t i = processed_keys; i < std::min(keys.size(), processed_keys + size_t(rocksdb_batch_read_size)); ++i) + { + key_column.type->serializeBinary(keys[i], wbs[i]); + auto str_ref = wbs[i].stringRef(); + slices_keys.emplace_back(str_ref.data, str_ref.size); + } + + auto statuses = storage.rocksdb_ptr->MultiGet(rocksdb::ReadOptions(), slices_keys, &values); + for (size_t i = 0; i < statuses.size(); ++i) + { + if (statuses[i].ok()) + { + ReadBufferFromString key_buffer(slices_keys[i]); + ReadBufferFromString value_buffer(values[i]); + + for (const auto [idx, column_type] : ext::enumerate(sample_block.getColumnsWithTypeAndName())) + { + column_type.type->deserializeBinary(*columns[idx], idx == primary_key_pos? key_buffer: value_buffer); + } + } + } + processed_keys += rocksdb_batch_read_size; + + UInt64 num_rows = columns.at(0)->size(); + return Chunk(std::move(columns), num_rows); + } + +private: + const StorageEmbeddedRocksdb & storage; + + const StorageMetadataPtr metadata_snapshot; + const size_t start; + const size_t end; + const size_t rocksdb_batch_read_size; + FieldVector keys; + + size_t processed_keys = 0; +}; + + class EmbeddedRocksdbBlockOutputStream : public IBlockOutputStream { public: @@ -206,19 +319,19 @@ Pipe StorageEmbeddedRocksdb::read( const Context & /*context*/, QueryProcessingStage::Enum /*processed_stage*/, size_t /*max_block_size*/, - unsigned /*num_streams*/) + unsigned num_streams) { metadata_snapshot->check(column_names, getVirtuals(), getStorageID()); Block sample_block = metadata_snapshot->getSampleBlock(); - MutableColumns columns = sample_block.cloneEmptyColumns(); + size_t primary_key_pos = sample_block.getPositionByName(primary_key); FieldVector keys; bool all_scan = false; - std::tie(keys, all_scan) = getFilterKeys(primary_key, query_info.query); - // TODO pipline + std::tie(keys, all_scan) = getFilterKeys(primary_key, query_info); if (all_scan) { + MutableColumns columns = sample_block.cloneEmptyColumns(); auto it = std::unique_ptr(rocksdb_ptr->NewIterator(rocksdb::ReadOptions())); for (it->SeekToFirst(); it->Valid(); it->Next()) { @@ -227,49 +340,47 @@ Pipe StorageEmbeddedRocksdb::read( for (const auto [idx, column_type] : ext::enumerate(sample_block.getColumnsWithTypeAndName())) { - if (column_type.name == primary_key) - column_type.type->deserializeBinary(*columns[idx], key_buffer); - else - column_type.type->deserializeBinary(*columns[idx], value_buffer); + column_type.type->deserializeBinary(*columns[idx], idx == primary_key_pos? key_buffer: value_buffer); } } - assert(it->status().ok()); + + if (!it->status().ok()) + { + throw Exception("Engine " + getName() + " got error while seeking key value datas: " + it->status().ToString(), + ErrorCodes::LOGICAL_ERROR); + } + UInt64 num_rows = columns.at(0)->size(); + Chunk chunk(std::move(columns), num_rows); + return Pipe(std::make_shared(sample_block, std::move(chunk))); } else { - std::vector slices_keys; - std::vector values; - std::vector wbs(keys.size()); + if (keys.empty()) + return {}; - for (size_t i = 0; i < keys.size(); ++i) + Pipes pipes; + size_t start = 0; + size_t end; + + const size_t num_threads = std::min(size_t(num_streams), keys.size()); + const size_t batch_per_size = ceil(keys.size() * 1.0 / num_threads); + + // TODO settings + static constexpr size_t rocksdb_batch_read_size = 81920; + + for (size_t t = 0; t < num_threads; ++t) { - sample_block.getByName(primary_key).type->serializeBinary(keys[i], wbs[i]); - auto str_ref = wbs[i].stringRef(); - slices_keys.emplace_back(str_ref.data, str_ref.size); - } + if (start >= keys.size()) + start = end = 0; + else + end = start + batch_per_size > keys.size() ? keys.size() : start + batch_per_size; - auto statuses = rocksdb_ptr->MultiGet(rocksdb::ReadOptions(), slices_keys, &values); - for (size_t i = 0; i < statuses.size(); ++i) - { - if (statuses[i].ok()) - { - ReadBufferFromString key_buffer(slices_keys[i]); - ReadBufferFromString value_buffer(values[i]); - - for (const auto [idx, column_type] : ext::enumerate(sample_block.getColumnsWithTypeAndName())) - { - if (column_type.name == primary_key) - column_type.type->deserializeBinary(*columns[idx], key_buffer); - else - column_type.type->deserializeBinary(*columns[idx], value_buffer); - } - } + pipes.emplace_back( + std::make_shared(*this, metadata_snapshot, keys, start, end, rocksdb_batch_read_size)); + start += batch_per_size; } + return Pipe::unitePipes(std::move(pipes)); } - - UInt64 num_rows = columns.at(0)->size(); - Chunk chunk(std::move(columns), num_rows); - return Pipe(std::make_shared(sample_block, std::move(chunk))); } BlockOutputStreamPtr StorageEmbeddedRocksdb::write(const ASTPtr & /*query*/, const StorageMetadataPtr & metadata_snapshot, const Context & /*context*/) @@ -280,7 +391,7 @@ BlockOutputStreamPtr StorageEmbeddedRocksdb::write(const ASTPtr & /*query*/, con static StoragePtr create(const StorageFactory::Arguments & args) { - // TODO RocksdbSettings + // TODO custom RocksdbSettings if (!args.engine_args.empty()) throw Exception( "Engine " + args.engine_name + " doesn't support any arguments (" + toString(args.engine_args.size()) + " given)", diff --git a/src/Storages/Rocksdb/StorageEmbeddedRocksdb.h b/src/Storages/Rocksdb/StorageEmbeddedRocksdb.h index 79e7d1678e0..bc70e795cba 100644 --- a/src/Storages/Rocksdb/StorageEmbeddedRocksdb.h +++ b/src/Storages/Rocksdb/StorageEmbeddedRocksdb.h @@ -18,6 +18,7 @@ class Context; class StorageEmbeddedRocksdb final : public ext::shared_ptr_helper, public IStorage { friend struct ext::shared_ptr_helper; + friend class EmbeddedRocksdbSource; friend class EmbeddedRocksdbBlockOutputStream; public: std::string getName() const override { return "EmbeddedRocksdb"; } @@ -34,6 +35,13 @@ public: BlockOutputStreamPtr write(const ASTPtr & query, const StorageMetadataPtr & /*metadata_snapshot*/, const Context & context) override; void truncate(const ASTPtr &, const StorageMetadataPtr & metadata_snapshot, const Context &, TableExclusiveLockHolder &) override; + bool supportsParallelInsert() const override { return true; } + bool supportsIndexForIn() const override { return true; } + bool mayBenefitFromIndexForIn(const ASTPtr & node, const Context & /*query_context*/, const StorageMetadataPtr & /*metadata_snapshot*/) const override + { + return node->getColumnName() == primary_key; + } + protected: StorageEmbeddedRocksdb(const StorageID & table_id_, const String & relative_data_path_, diff --git a/tests/performance/rocksdb.xml b/tests/performance/rocksdb.xml deleted file mode 100644 index e0867eece65..00000000000 --- a/tests/performance/rocksdb.xml +++ /dev/null @@ -1,18 +0,0 @@ - - - - hits_100m_single - - - - 30000000000 - - - CREATE TABLE ma_kv (key String, value AggregateFunction(groupBitmap, UInt64)) EmbeddedRocksdb primary key(key) - - INSERT INTO ma_kv SELECT concat('CodeVersion=', CodeVersion) as key, bitmapBuild(groupArray(UserID)) AS value FROM hits_100m_single GROUP BY key - - SELECT groupBitmapOr(value) FROM ma_kv WHERE key IN ('CodeVersion=1657', 'CodeVersion=1', 'CodeVersion=275') - - DROP TABLE IF EXISTS ma_kv - diff --git a/tests/queries/0_stateless/01504_rocksdb.reference b/tests/queries/0_stateless/01504_rocksdb.reference index ae092fcbc30..f98edab5dbf 100644 --- a/tests/queries/0_stateless/01504_rocksdb.reference +++ b/tests/queries/0_stateless/01504_rocksdb.reference @@ -5,3 +5,4 @@ 1 1 1 +1 diff --git a/tests/queries/0_stateless/01504_rocksdb.sql b/tests/queries/0_stateless/01504_rocksdb.sql index f78f6e2f658..8b01aafa38a 100644 --- a/tests/queries/0_stateless/01504_rocksdb.sql +++ b/tests/queries/0_stateless/01504_rocksdb.sql @@ -21,10 +21,12 @@ INSERT INTO test_memory SELECT number % 77 AS k, SUM(number) AS value, (1, 1.2), SELECT A.a - B.a, A.b - B.b, A.c - B.c, A.d - B.d, A.e - B.e FROM ( SELECT 0 AS a, groupBitmapMerge(bm) AS b , SUM(k) AS c, SUM(value) AS d, SUM(dummy.1) AS e FROM test) A ANY LEFT JOIN (SELECT 0 AS a, groupBitmapMerge(bm) AS b , SUM(k) AS c, SUM(value) AS d, SUM(dummy.1) AS e FROM test_memory) B USING a ORDER BY a; +CREATE TEMPORARY TABLE keys AS SELECT * FROM numbers(1000); SET max_rows_to_read = 2; SELECT dummy == (1,1.2) FROM test WHERE k IN (1, 3); SELECT k == 4 FROM test WHERE k = 4; +SELECT k == 4 FROM test WHERE k IN (SELECT toUInt32(number) FROM keys WHERE number = 4); SELECT k, value FROM test WHERE k = 0 OR value > 0; -- { serverError 158 } TRUNCATE TABLE test; From 94cb974f42ce6e1fccde7e93ab9fee70669db771 Mon Sep 17 00:00:00 2001 From: sundy-li <543950155@qq.com> Date: Tue, 20 Oct 2020 21:26:09 +0800 Subject: [PATCH 061/205] * fix fasttest and cmake && pipline for all_scan * unique the keys * add inputstream && outputstream --- cmake/find/rocksdb.cmake | 7 +- docker/test/fasttest/run.sh | 2 +- .../EmbeddedRocksdbBlockInputStream.cpp | 65 ++++++ .../Rocksdb/EmbeddedRocksdbBlockInputStream.h | 33 +++ .../EmbeddedRocksdbBlockOutputStream.cpp | 50 +++++ .../EmbeddedRocksdbBlockOutputStream.h | 29 +++ .../Rocksdb/StorageEmbeddedRocksdb.cpp | 192 +++++++----------- src/Storages/Rocksdb/StorageEmbeddedRocksdb.h | 1 + src/Storages/ya.make | 6 + tests/queries/0_stateless/01504_rocksdb.sql | 4 +- 10 files changed, 260 insertions(+), 129 deletions(-) create mode 100644 src/Storages/Rocksdb/EmbeddedRocksdbBlockInputStream.cpp create mode 100644 src/Storages/Rocksdb/EmbeddedRocksdbBlockInputStream.h create mode 100644 src/Storages/Rocksdb/EmbeddedRocksdbBlockOutputStream.cpp create mode 100644 src/Storages/Rocksdb/EmbeddedRocksdbBlockOutputStream.h diff --git a/cmake/find/rocksdb.cmake b/cmake/find/rocksdb.cmake index d53febee762..24d959eb074 100644 --- a/cmake/find/rocksdb.cmake +++ b/cmake/find/rocksdb.cmake @@ -13,9 +13,8 @@ if (NOT EXISTS "${ClickHouse_SOURCE_DIR}/contrib/rocksdb/CMakeLists.txt") if (USE_INTERNAL_ROCKSDB_LIBRARY) message (WARNING "submodule contrib is missing. to fix try run: \n git submodule update --init --recursive") message(${RECONFIGURE_MESSAGE_LEVEL} "cannot find internal rocksdb") - set (USE_INTERNAL_ROCKSDB_LIBRARY 0) endif() - set (MISSING_ROCKSDB 1) + set (MISSING_INTERNAL_ROCKSDB 1) endif () if (NOT USE_INTERNAL_ROCKSDB_LIBRARY) @@ -26,7 +25,9 @@ if (NOT USE_INTERNAL_ROCKSDB_LIBRARY) endif() endif () -if (NOT ROCKSDB_LIBRARY AND NOT MISSING_ROCKSDB) +if(ROCKSDB_LIBRARY AND ROCKSDB_INCLUDE_DIR) + set(USE_ROCKSDB 1) +elseif (NOT MISSING_INTERNAL_ROCKSDB) set (USE_INTERNAL_ROCKSDB_LIBRARY 1) set (ROCKSDB_INCLUDE_DIR "${ClickHouse_SOURCE_DIR}/contrib/rocksdb/include") diff --git a/docker/test/fasttest/run.sh b/docker/test/fasttest/run.sh index c1c76b93829..9d84274c297 100755 --- a/docker/test/fasttest/run.sh +++ b/docker/test/fasttest/run.sh @@ -270,7 +270,7 @@ TESTS_TO_SKIP=( 00646_url_engine 00974_query_profiler - # Rocksdb is not enabled by default + # In fasttest, ENABLE_LIBRARIES=0, so rocksdb engine is not enabled by default 01504_rocksdb # Look at DistributedFilesToInsert, so cannot run in parallel. diff --git a/src/Storages/Rocksdb/EmbeddedRocksdbBlockInputStream.cpp b/src/Storages/Rocksdb/EmbeddedRocksdbBlockInputStream.cpp new file mode 100644 index 00000000000..0f75f41c7dd --- /dev/null +++ b/src/Storages/Rocksdb/EmbeddedRocksdbBlockInputStream.cpp @@ -0,0 +1,65 @@ +#include +#include +#include +#include + +#include + + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int LOGICAL_ERROR; +} + +EmbeddedRocksdbBlockInputStream::EmbeddedRocksdbBlockInputStream( + StorageEmbeddedRocksdb & storage_, + const StorageMetadataPtr & metadata_snapshot_, + size_t max_block_size_) + : storage(storage_) + , metadata_snapshot(metadata_snapshot_) + , max_block_size(max_block_size_) +{ + sample_block = metadata_snapshot->getSampleBlock(); + primary_key_pos = sample_block.getPositionByName(storage.primary_key); +} + +Block EmbeddedRocksdbBlockInputStream::readImpl() +{ + if (finished) + return {}; + + if (!iterator) + { + iterator = std::unique_ptr(storage.rocksdb_ptr->NewIterator(rocksdb::ReadOptions())); + iterator->SeekToFirst(); + } + + MutableColumns columns = sample_block.cloneEmptyColumns(); + size_t rows = 0; + for (; iterator->Valid(); iterator->Next()) + { + ReadBufferFromString key_buffer(iterator->key()); + ReadBufferFromString value_buffer(iterator->value()); + + for (const auto [idx, column_type] : ext::enumerate(sample_block.getColumnsWithTypeAndName())) + { + column_type.type->deserializeBinary(*columns[idx], idx == primary_key_pos? key_buffer: value_buffer); + } + ++rows; + if (rows >= max_block_size) + break; + } + + finished = !iterator->Valid(); + if (!iterator->status().ok()) + { + throw Exception("Engine " + getName() + " got error while seeking key value datas: " + iterator->status().ToString(), + ErrorCodes::LOGICAL_ERROR); + } + return sample_block.cloneWithColumns(std::move(columns)); +} + +} diff --git a/src/Storages/Rocksdb/EmbeddedRocksdbBlockInputStream.h b/src/Storages/Rocksdb/EmbeddedRocksdbBlockInputStream.h new file mode 100644 index 00000000000..817aace58d0 --- /dev/null +++ b/src/Storages/Rocksdb/EmbeddedRocksdbBlockInputStream.h @@ -0,0 +1,33 @@ +#pragma once + +#include +#include +#include + +#include + +namespace DB +{ + +class EmbeddedRocksdbBlockInputStream : public IBlockInputStream +{ + +public: + EmbeddedRocksdbBlockInputStream( + StorageEmbeddedRocksdb & storage_, const StorageMetadataPtr & metadata_snapshot_, size_t max_block_size_); + + String getName() const override { return storage.getName(); } + Block getHeader() const override { return sample_block; } + Block readImpl() override; + +private: + StorageEmbeddedRocksdb & storage; + StorageMetadataPtr metadata_snapshot; + const size_t max_block_size; + + Block sample_block; + std::unique_ptr iterator; + size_t primary_key_pos; + bool finished = false; +}; +} diff --git a/src/Storages/Rocksdb/EmbeddedRocksdbBlockOutputStream.cpp b/src/Storages/Rocksdb/EmbeddedRocksdbBlockOutputStream.cpp new file mode 100644 index 00000000000..aebc41addda --- /dev/null +++ b/src/Storages/Rocksdb/EmbeddedRocksdbBlockOutputStream.cpp @@ -0,0 +1,50 @@ + +#include +#include + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int SYSTEM_ERROR; +} + +Block EmbeddedRocksdbBlockOutputStream::getHeader() const +{ + return metadata_snapshot->getSampleBlock(); +} + +void EmbeddedRocksdbBlockOutputStream::write(const Block & block) +{ + metadata_snapshot->check(block, true); + auto rows = block.rows(); + + WriteBufferFromOwnString wb_key; + WriteBufferFromOwnString wb_value; + + rocksdb::WriteBatch batch; + auto columns = metadata_snapshot->getColumns(); + + for (size_t i = 0; i < rows; i++) + { + wb_key.restart(); + wb_value.restart(); + + for (const auto & col : columns) + { + const auto & type = block.getByName(col.name).type; + const auto & column = block.getByName(col.name).column; + if (col.name == storage.primary_key) + type->serializeBinary(*column, i, wb_key); + else + type->serializeBinary(*column, i, wb_value); + } + batch.Put(wb_key.str(), wb_value.str()); + } + auto status = storage.rocksdb_ptr->Write(rocksdb::WriteOptions(), &batch); + if (!status.ok()) + throw Exception("Rocksdb write error: " + status.ToString(), ErrorCodes::SYSTEM_ERROR); +} + +} diff --git a/src/Storages/Rocksdb/EmbeddedRocksdbBlockOutputStream.h b/src/Storages/Rocksdb/EmbeddedRocksdbBlockOutputStream.h new file mode 100644 index 00000000000..d92e15d553a --- /dev/null +++ b/src/Storages/Rocksdb/EmbeddedRocksdbBlockOutputStream.h @@ -0,0 +1,29 @@ +#pragma once + +#include +#include +#include +#include + +namespace DB +{ + +class EmbeddedRocksdbBlockOutputStream : public IBlockOutputStream +{ +public: + explicit EmbeddedRocksdbBlockOutputStream( + StorageEmbeddedRocksdb & storage_, + const StorageMetadataPtr & metadata_snapshot_) + : storage(storage_) + , metadata_snapshot(metadata_snapshot_) + {} + + Block getHeader() const override; + void write(const Block & block) override; + +private: + StorageEmbeddedRocksdb & storage; + StorageMetadataPtr metadata_snapshot; +}; + +} diff --git a/src/Storages/Rocksdb/StorageEmbeddedRocksdb.cpp b/src/Storages/Rocksdb/StorageEmbeddedRocksdb.cpp index 127d1349a2c..86d2f6ae406 100644 --- a/src/Storages/Rocksdb/StorageEmbeddedRocksdb.cpp +++ b/src/Storages/Rocksdb/StorageEmbeddedRocksdb.cpp @@ -3,6 +3,8 @@ #include #include #include +#include +#include #include #include #include @@ -17,7 +19,7 @@ #include #include -#include +#include #include #include #include @@ -39,13 +41,12 @@ namespace ErrorCodes { extern const int BAD_ARGUMENTS; extern const int SYSTEM_ERROR; - extern const int LOGICAL_ERROR; extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; } // returns keys may be filter by condition -static bool traverseASTFilter(const String & primary_key, const ASTPtr & elem, const PreparedSets & sets, FieldVector & res) +static bool traverseASTFilter(const String & primary_key, const DataTypePtr & primary_key_type, const ASTPtr & elem, const PreparedSets & sets, FieldVector & res) { const auto * function = elem->as(); if (!function) @@ -53,34 +54,50 @@ static bool traverseASTFilter(const String & primary_key, const ASTPtr & elem, c if (function->name == "and") { + // one child has the key filter condition is ok for (const auto & child : function->arguments->children) - if (traverseASTFilter(primary_key, child, sets, res)) + if (traverseASTFilter(primary_key, primary_key_type, child, sets, res)) return true; return false; } + else if (function->name == "or") + { + // make sure every child has the key filter condition + FieldVector child_res; + for (const auto & child : function->arguments->children) + { + if (!traverseASTFilter(primary_key, primary_key_type, child, sets, child_res)) + return false; + } + res.insert(res.end(), child_res.begin(), child_res.end()); + return true; + } else if (function->name == "equals" || function->name == "in") { const auto & args = function->arguments->as(); + const ASTIdentifier * ident; const IAST * value; if (args.children.size() != 2) - return false; + return false; - const ASTIdentifier * ident; - if ((ident = args.children.at(0)->as())) - value = args.children.at(1).get(); - else if ((ident = args.children.at(1)->as())) - value = args.children.at(0).get(); - else - return false; - - if (ident->name != primary_key) - return false; - - - if (function->name == "in" && ((value->as() || value->as()))) + if (function->name == "in") { - auto set_it = sets.find(PreparedSetKey::forSubquery(*value)); + ident = args.children.at(0)->as(); + if (!ident) + return false; + + if (ident->name != primary_key) + return false; + value = args.children.at(1).get(); + + PreparedSetKey set_key; + if ((value->as() || value->as())) + set_key = PreparedSetKey::forSubquery(*value); + else + set_key = PreparedSetKey::forLiteral(*value, {primary_key_type}); + + auto set_it = sets.find(set_key); if (set_it == sets.end()) return false; SetPtr prepared_set = set_it->second; @@ -89,7 +106,6 @@ static bool traverseASTFilter(const String & primary_key, const ASTPtr & elem, c return false; prepared_set->checkColumnsNumber(1); - const auto & set_column = *prepared_set->getSetElements()[0]; for (size_t row = 0; row < set_column.size(); ++row) { @@ -97,27 +113,24 @@ static bool traverseASTFilter(const String & primary_key, const ASTPtr & elem, c } return true; } - - if (const auto * literal = value->as()) + else { - if (function->name == "equals") + if ((ident = args.children.at(0)->as())) + value = args.children.at(1).get(); + else if ((ident = args.children.at(1)->as())) + value = args.children.at(0).get(); + else + return false; + + if (ident->name != primary_key) + return false; + + //function->name == "equals" + if (const auto * literal = value->as()) { res.push_back(literal->value); return true; } - else if (function->name == "in") - { - if (literal->value.getType() == Field::Types::Tuple) - { - auto tuple = literal->value.safeGet(); - for (const auto & f : tuple) - { - res.push_back(f); - } - return true; - } - } - else return false; } } return false; @@ -127,7 +140,7 @@ static bool traverseASTFilter(const String & primary_key, const ASTPtr & elem, c /** Retrieve from the query a condition of the form `key = 'key'`, `key in ('xxx_'), from conjunctions in the WHERE clause. * TODO support key like search */ -static std::pair getFilterKeys(const String & primary_key, const SelectQueryInfo & query_info) +static std::pair getFilterKeys(const String & primary_key, const DataTypePtr & primary_key_type, const SelectQueryInfo & query_info) { const auto & select = query_info.query->as(); if (!select.where()) @@ -135,7 +148,7 @@ static std::pair getFilterKeys(const String & primary_key, co return std::make_pair(FieldVector{}, true); } FieldVector res; - auto matched_keys = traverseASTFilter(primary_key, select.where(), query_info.sets, res); + auto matched_keys = traverseASTFilter(primary_key, primary_key_type, select.where(), query_info.sets, res); return std::make_pair(res, !matched_keys); } @@ -149,13 +162,13 @@ public: const FieldVector & keys_, const size_t start_, const size_t end_, - const size_t rocksdb_batch_read_size_) + const size_t max_block_size_) : SourceWithProgress(metadata_snapshot_->getSampleBlock()) , storage(storage_) , metadata_snapshot(metadata_snapshot_) , start(start_) , end(end_) - , rocksdb_batch_read_size(rocksdb_batch_read_size_) + , max_block_size(max_block_size_) { // slice the keys if (end > start) @@ -185,7 +198,7 @@ public: auto columns = sample_block.cloneEmptyColumns(); size_t primary_key_pos = sample_block.getPositionByName(storage.primary_key); - for (size_t i = processed_keys; i < std::min(keys.size(), processed_keys + size_t(rocksdb_batch_read_size)); ++i) + for (size_t i = processed_keys; i < std::min(keys.size(), processed_keys + max_block_size); ++i) { key_column.type->serializeBinary(keys[i], wbs[i]); auto str_ref = wbs[i].stringRef(); @@ -206,7 +219,7 @@ public: } } } - processed_keys += rocksdb_batch_read_size; + processed_keys += max_block_size; UInt64 num_rows = columns.at(0)->size(); return Chunk(std::move(columns), num_rows); @@ -218,62 +231,13 @@ private: const StorageMetadataPtr metadata_snapshot; const size_t start; const size_t end; - const size_t rocksdb_batch_read_size; + const size_t max_block_size; FieldVector keys; size_t processed_keys = 0; }; -class EmbeddedRocksdbBlockOutputStream : public IBlockOutputStream -{ -public: - explicit EmbeddedRocksdbBlockOutputStream( - StorageEmbeddedRocksdb & storage_, - const StorageMetadataPtr & metadata_snapshot_) - : storage(storage_) - , metadata_snapshot(metadata_snapshot_) - {} - - Block getHeader() const override { return metadata_snapshot->getSampleBlock(); } - - void write(const Block & block) override - { - metadata_snapshot->check(block, true); - auto rows = block.rows(); - - WriteBufferFromOwnString wb_key; - WriteBufferFromOwnString wb_value; - - rocksdb::WriteBatch batch; - auto columns = metadata_snapshot->getColumns(); - - for (size_t i = 0; i < rows; i++) - { - wb_key.restart(); - wb_value.restart(); - - for (const auto & col : columns) - { - const auto & type = block.getByName(col.name).type; - const auto & column = block.getByName(col.name).column; - if (col.name == storage.primary_key) - type->serializeBinary(*column, i, wb_key); - else - type->serializeBinary(*column, i, wb_value); - } - batch.Put(wb_key.str(), wb_value.str()); - } - auto status = storage.rocksdb_ptr->Write(rocksdb::WriteOptions(), &batch); - if (!status.ok()) - throw Exception("Rocksdb write error: " + status.ToString(), ErrorCodes::SYSTEM_ERROR); - } - -private: - StorageEmbeddedRocksdb & storage; - StorageMetadataPtr metadata_snapshot; -}; - StorageEmbeddedRocksdb::StorageEmbeddedRocksdb(const StorageID & table_id_, const String & relative_data_path_, const StorageInMemoryMetadata & metadata_, @@ -318,46 +282,31 @@ Pipe StorageEmbeddedRocksdb::read( const SelectQueryInfo & query_info, const Context & /*context*/, QueryProcessingStage::Enum /*processed_stage*/, - size_t /*max_block_size*/, + size_t max_block_size, unsigned num_streams) { metadata_snapshot->check(column_names, getVirtuals(), getStorageID()); - Block sample_block = metadata_snapshot->getSampleBlock(); - size_t primary_key_pos = sample_block.getPositionByName(primary_key); - FieldVector keys; bool all_scan = false; - std::tie(keys, all_scan) = getFilterKeys(primary_key, query_info); + auto primary_key_data_type = metadata_snapshot->getSampleBlock().getByName(primary_key).type; + std::tie(keys, all_scan) = getFilterKeys(primary_key, primary_key_data_type, query_info); if (all_scan) { - MutableColumns columns = sample_block.cloneEmptyColumns(); - auto it = std::unique_ptr(rocksdb_ptr->NewIterator(rocksdb::ReadOptions())); - for (it->SeekToFirst(); it->Valid(); it->Next()) - { - ReadBufferFromString key_buffer(it->key()); - ReadBufferFromString value_buffer(it->value()); - - for (const auto [idx, column_type] : ext::enumerate(sample_block.getColumnsWithTypeAndName())) - { - column_type.type->deserializeBinary(*columns[idx], idx == primary_key_pos? key_buffer: value_buffer); - } - } - - if (!it->status().ok()) - { - throw Exception("Engine " + getName() + " got error while seeking key value datas: " + it->status().ToString(), - ErrorCodes::LOGICAL_ERROR); - } - UInt64 num_rows = columns.at(0)->size(); - Chunk chunk(std::move(columns), num_rows); - return Pipe(std::make_shared(sample_block, std::move(chunk))); + auto reader = std::make_shared( + *this, metadata_snapshot, max_block_size); + return Pipe(std::make_shared(reader)); } else { if (keys.empty()) return {}; + std::sort(keys.begin(), keys.end()); + auto unique_iter = std::unique(keys.begin(), keys.end()); + if (unique_iter != keys.end()) + keys.erase(unique_iter, keys.end()); + Pipes pipes; size_t start = 0; size_t end; @@ -365,9 +314,6 @@ Pipe StorageEmbeddedRocksdb::read( const size_t num_threads = std::min(size_t(num_streams), keys.size()); const size_t batch_per_size = ceil(keys.size() * 1.0 / num_threads); - // TODO settings - static constexpr size_t rocksdb_batch_read_size = 81920; - for (size_t t = 0; t < num_threads; ++t) { if (start >= keys.size()) @@ -376,7 +322,7 @@ Pipe StorageEmbeddedRocksdb::read( end = start + batch_per_size > keys.size() ? keys.size() : start + batch_per_size; pipes.emplace_back( - std::make_shared(*this, metadata_snapshot, keys, start, end, rocksdb_batch_read_size)); + std::make_shared(*this, metadata_snapshot, keys, start, end, max_block_size)); start += batch_per_size; } return Pipe::unitePipes(std::move(pipes)); @@ -391,7 +337,7 @@ BlockOutputStreamPtr StorageEmbeddedRocksdb::write(const ASTPtr & /*query*/, con static StoragePtr create(const StorageFactory::Arguments & args) { - // TODO custom RocksdbSettings + // TODO custom RocksdbSettings, table function if (!args.engine_args.empty()) throw Exception( "Engine " + args.engine_name + " doesn't support any arguments (" + toString(args.engine_args.size()) + " given)", diff --git a/src/Storages/Rocksdb/StorageEmbeddedRocksdb.h b/src/Storages/Rocksdb/StorageEmbeddedRocksdb.h index bc70e795cba..c803055223b 100644 --- a/src/Storages/Rocksdb/StorageEmbeddedRocksdb.h +++ b/src/Storages/Rocksdb/StorageEmbeddedRocksdb.h @@ -20,6 +20,7 @@ class StorageEmbeddedRocksdb final : public ext::shared_ptr_helper; friend class EmbeddedRocksdbSource; friend class EmbeddedRocksdbBlockOutputStream; + friend class EmbeddedRocksdbBlockInputStream; public: std::string getName() const override { return "EmbeddedRocksdb"; } diff --git a/src/Storages/ya.make b/src/Storages/ya.make index f45767d42f2..d9fb44e69f1 100644 --- a/src/Storages/ya.make +++ b/src/Storages/ya.make @@ -105,6 +105,12 @@ SRCS( MutationCommands.cpp PartitionCommands.cpp ReadInOrderOptimizer.cpp +<<<<<<< HEAD +======= + registerStorages.cpp + Rocksdb/EmbeddedRocksdbBlockInputStream.cpp + Rocksdb/EmbeddedRocksdbBlockOutputStream.cpp +>>>>>>> * fix fasttest and cmake && pipline for all_scan Rocksdb/StorageEmbeddedRocksdb.cpp SelectQueryDescription.cpp SetSettings.cpp diff --git a/tests/queries/0_stateless/01504_rocksdb.sql b/tests/queries/0_stateless/01504_rocksdb.sql index 8b01aafa38a..2ad00f18cd8 100644 --- a/tests/queries/0_stateless/01504_rocksdb.sql +++ b/tests/queries/0_stateless/01504_rocksdb.sql @@ -24,8 +24,8 @@ SELECT A.a - B.a, A.b - B.b, A.c - B.c, A.d - B.d, A.e - B.e FROM ( SELECT 0 AS CREATE TEMPORARY TABLE keys AS SELECT * FROM numbers(1000); SET max_rows_to_read = 2; -SELECT dummy == (1,1.2) FROM test WHERE k IN (1, 3); -SELECT k == 4 FROM test WHERE k = 4; +SELECT dummy == (1,1.2) FROM test WHERE k IN (1, 3) OR k IN (1) OR k IN (3, 1) OR k IN [1] OR k IN [1, 3] ; +SELECT k == 4 FROM test WHERE k = 4 OR k IN [4]; SELECT k == 4 FROM test WHERE k IN (SELECT toUInt32(number) FROM keys WHERE number = 4); SELECT k, value FROM test WHERE k = 0 OR value > 0; -- { serverError 158 } From e55824c3df727aa0104b68ade6c42bb63300e0fd Mon Sep 17 00:00:00 2001 From: sundy-li <543950155@qq.com> Date: Thu, 22 Oct 2020 16:48:57 +0800 Subject: [PATCH 062/205] improve unbundle build --- cmake/find/rocksdb.cmake | 32 +++++++++++++++++-- docker/packager/unbundled/Dockerfile | 1 + docker/test/stateless_unbundled/Dockerfile | 2 ++ .../Rocksdb/EmbeddedRocksdbBlockInputStream.h | 1 - .../EmbeddedRocksdbBlockOutputStream.h | 1 - .../Rocksdb/StorageEmbeddedRocksdb.cpp | 6 ++-- src/Storages/ya.make | 4 --- tests/queries/0_stateless/01504_rocksdb.sql | 3 +- 8 files changed, 38 insertions(+), 12 deletions(-) diff --git a/cmake/find/rocksdb.cmake b/cmake/find/rocksdb.cmake index 24d959eb074..968cdb52407 100644 --- a/cmake/find/rocksdb.cmake +++ b/cmake/find/rocksdb.cmake @@ -23,6 +23,35 @@ if (NOT USE_INTERNAL_ROCKSDB_LIBRARY) if (NOT ROCKSDB_LIBRARY OR NOT ROCKSDB_INCLUDE_DIR) message (${RECONFIGURE_MESSAGE_LEVEL} "Can't find system rocksdb library") endif() + + if (NOT SNAPPY_LIBRARY) + include(cmake/find/snappy.cmake) + endif() + if (NOT ZLIB_LIBRARY) + include(cmake/find/zlib.cmake) + endif() + + find_package(BZip2) + find_library(ZSTD_LIBRARY zstd) + find_library(LZ4_LIBRARY lz4) + find_library(GFLAGS_LIBRARY gflags) + + if(SNAPPY_LIBRARY AND ZLIB_LIBRARY AND LZ4_LIBRARY AND BZIP2_FOUND AND ZSTD_LIBRARY AND GFLAGS_LIBRARY) + list (APPEND ROCKSDB_LIBRARY ${SNAPPY_LIBRARY}) + list (APPEND ROCKSDB_LIBRARY ${ZLIB_LIBRARY}) + list (APPEND ROCKSDB_LIBRARY ${LZ4_LIBRARY}) + list (APPEND ROCKSDB_LIBRARY ${BZIP2_LIBRARY}) + list (APPEND ROCKSDB_LIBRARY ${ZSTD_LIBRARY}) + list (APPEND ROCKSDB_LIBRARY ${GFLAGS_LIBRARY}) + else() + message (${RECONFIGURE_MESSAGE_LEVEL} + "Can't find system rocksdb: snappy=${SNAPPY_LIBRARY} ;" + " zlib=${ZLIB_LIBRARY} ;" + " lz4=${LZ4_LIBRARY} ;" + " bz2=${BZIP2_LIBRARY} ;" + " zstd=${ZSTD_LIBRARY} ;" + " gflags=${GFLAGS_LIBRARY} ;") + endif() endif () if(ROCKSDB_LIBRARY AND ROCKSDB_INCLUDE_DIR) @@ -31,9 +60,8 @@ elseif (NOT MISSING_INTERNAL_ROCKSDB) set (USE_INTERNAL_ROCKSDB_LIBRARY 1) set (ROCKSDB_INCLUDE_DIR "${ClickHouse_SOURCE_DIR}/contrib/rocksdb/include") - set (ROCKSDB_CORE_INCLUDE_DIR "${ClickHouse_SOURCE_DIR}/contrib/rocksdb/include") set (ROCKSDB_LIBRARY "rocksdb") set (USE_ROCKSDB 1) endif () -message (STATUS "Using ROCKSDB=${USE_ROCKSDB}: ${ROCKSDB_INCLUDE_DIR} : ${ROCKSDB_LIBRARY}") \ No newline at end of file +message (STATUS "Using ROCKSDB=${USE_ROCKSDB}: ${ROCKSDB_INCLUDE_DIR} : ${ROCKSDB_LIBRARY}") diff --git a/docker/packager/unbundled/Dockerfile b/docker/packager/unbundled/Dockerfile index 698a7e2621b..261edf1a86c 100644 --- a/docker/packager/unbundled/Dockerfile +++ b/docker/packager/unbundled/Dockerfile @@ -65,6 +65,7 @@ RUN apt-get update \ libavro-dev \ libfarmhash-dev \ librocksdb-dev \ + libgflags-dev \ libmysqlclient-dev \ --yes --no-install-recommends diff --git a/docker/test/stateless_unbundled/Dockerfile b/docker/test/stateless_unbundled/Dockerfile index 345ba905412..1c9f9510d7e 100644 --- a/docker/test/stateless_unbundled/Dockerfile +++ b/docker/test/stateless_unbundled/Dockerfile @@ -43,6 +43,8 @@ RUN apt-get --allow-unauthenticated update -y \ libreadline-dev \ libsasl2-dev \ libzstd-dev \ + librocksdb-dev \ + libgflags-dev \ lsof \ moreutils \ ncdu \ diff --git a/src/Storages/Rocksdb/EmbeddedRocksdbBlockInputStream.h b/src/Storages/Rocksdb/EmbeddedRocksdbBlockInputStream.h index 817aace58d0..2caf434ffd8 100644 --- a/src/Storages/Rocksdb/EmbeddedRocksdbBlockInputStream.h +++ b/src/Storages/Rocksdb/EmbeddedRocksdbBlockInputStream.h @@ -1,7 +1,6 @@ #pragma once #include -#include #include #include diff --git a/src/Storages/Rocksdb/EmbeddedRocksdbBlockOutputStream.h b/src/Storages/Rocksdb/EmbeddedRocksdbBlockOutputStream.h index d92e15d553a..5ead88216b1 100644 --- a/src/Storages/Rocksdb/EmbeddedRocksdbBlockOutputStream.h +++ b/src/Storages/Rocksdb/EmbeddedRocksdbBlockOutputStream.h @@ -1,7 +1,6 @@ #pragma once #include -#include #include #include diff --git a/src/Storages/Rocksdb/StorageEmbeddedRocksdb.cpp b/src/Storages/Rocksdb/StorageEmbeddedRocksdb.cpp index 86d2f6ae406..d47cc408d39 100644 --- a/src/Storages/Rocksdb/StorageEmbeddedRocksdb.cpp +++ b/src/Storages/Rocksdb/StorageEmbeddedRocksdb.cpp @@ -79,7 +79,7 @@ static bool traverseASTFilter(const String & primary_key, const DataTypePtr & pr const IAST * value; if (args.children.size() != 2) - return false; + return false; if (function->name == "in") { @@ -87,7 +87,7 @@ static bool traverseASTFilter(const String & primary_key, const DataTypePtr & pr if (!ident) return false; - if (ident->name != primary_key) + if (ident->name() != primary_key) return false; value = args.children.at(1).get(); @@ -122,7 +122,7 @@ static bool traverseASTFilter(const String & primary_key, const DataTypePtr & pr else return false; - if (ident->name != primary_key) + if (ident->name() != primary_key) return false; //function->name == "equals" diff --git a/src/Storages/ya.make b/src/Storages/ya.make index d9fb44e69f1..3f0114a5d48 100644 --- a/src/Storages/ya.make +++ b/src/Storages/ya.make @@ -105,12 +105,8 @@ SRCS( MutationCommands.cpp PartitionCommands.cpp ReadInOrderOptimizer.cpp -<<<<<<< HEAD -======= - registerStorages.cpp Rocksdb/EmbeddedRocksdbBlockInputStream.cpp Rocksdb/EmbeddedRocksdbBlockOutputStream.cpp ->>>>>>> * fix fasttest and cmake && pipline for all_scan Rocksdb/StorageEmbeddedRocksdb.cpp SelectQueryDescription.cpp SetSettings.cpp diff --git a/tests/queries/0_stateless/01504_rocksdb.sql b/tests/queries/0_stateless/01504_rocksdb.sql index 2ad00f18cd8..eaaecca96b5 100644 --- a/tests/queries/0_stateless/01504_rocksdb.sql +++ b/tests/queries/0_stateless/01504_rocksdb.sql @@ -25,9 +25,10 @@ CREATE TEMPORARY TABLE keys AS SELECT * FROM numbers(1000); SET max_rows_to_read = 2; SELECT dummy == (1,1.2) FROM test WHERE k IN (1, 3) OR k IN (1) OR k IN (3, 1) OR k IN [1] OR k IN [1, 3] ; -SELECT k == 4 FROM test WHERE k = 4 OR k IN [4]; +SELECT k == 4 FROM test WHERE k = 4 OR k IN [4] OR k in (4, 10000001, 10000002) AND value > 0; SELECT k == 4 FROM test WHERE k IN (SELECT toUInt32(number) FROM keys WHERE number = 4); SELECT k, value FROM test WHERE k = 0 OR value > 0; -- { serverError 158 } +SELECT k, value FROM test WHERE k = 0 AND k IN (1, 3) OR k > 8; -- { serverError 158 } TRUNCATE TABLE test; SELECT 0 == count(1) FROM test; From 3342da1d774221e056eb58b48670d88a8b799305 Mon Sep 17 00:00:00 2001 From: feng lv Date: Sat, 7 Nov 2020 04:14:50 +0000 Subject: [PATCH 063/205] fix #16123 --- src/Interpreters/TreeRewriter.cpp | 2 +- ...559_aggregate_null_for_empty_fix.reference | 8 ++ .../01559_aggregate_null_for_empty_fix.sql | 91 +++++++++++++++++++ 3 files changed, 100 insertions(+), 1 deletion(-) create mode 100644 tests/queries/0_stateless/01559_aggregate_null_for_empty_fix.reference create mode 100644 tests/queries/0_stateless/01559_aggregate_null_for_empty_fix.sql diff --git a/src/Interpreters/TreeRewriter.cpp b/src/Interpreters/TreeRewriter.cpp index c8691c25f1b..b4d4e030028 100644 --- a/src/Interpreters/TreeRewriter.cpp +++ b/src/Interpreters/TreeRewriter.cpp @@ -125,7 +125,7 @@ struct CustomizeAggregateFunctionsSuffixData { auto properties = instance.tryGetProperties(func.name); if (properties && !properties->returns_default_when_only_null) - func.name = func.name + customized_func_suffix; + func.name = Poco::toLower(func.name) + customized_func_suffix; } } }; diff --git a/tests/queries/0_stateless/01559_aggregate_null_for_empty_fix.reference b/tests/queries/0_stateless/01559_aggregate_null_for_empty_fix.reference new file mode 100644 index 00000000000..929e088bb7a --- /dev/null +++ b/tests/queries/0_stateless/01559_aggregate_null_for_empty_fix.reference @@ -0,0 +1,8 @@ +0 +0 +0 +0 +-1 +-1 +-1 +\N diff --git a/tests/queries/0_stateless/01559_aggregate_null_for_empty_fix.sql b/tests/queries/0_stateless/01559_aggregate_null_for_empty_fix.sql new file mode 100644 index 00000000000..5955dee37f8 --- /dev/null +++ b/tests/queries/0_stateless/01559_aggregate_null_for_empty_fix.sql @@ -0,0 +1,91 @@ +SELECT MAX(aggr) +FROM +( + SELECT MAX(-1) AS aggr + FROM system.one + WHERE NOT 1 + UNION ALL + SELECT MAX(-1) AS aggr + FROM system.one + WHERE 1 + +); +SELECT MaX(aggr) +FROM +( + SELECT mAX(-1) AS aggr + FROM system.one + WHERE NOT 1 + UNION ALL + SELECT MAx(-1) AS aggr + FROM system.one + WHERE 1 +); +SELECT MaX(aggr) +FROM +( + SELECT mAX(-1) AS aggr + FROM system.one + WHERE NOT 1 + UNION ALL + SELECT max(-1) AS aggr + FROM system.one + WHERE 1 +); +SELECT MaX(aggr) +FROM +( + SELECT mAX(-1) AS aggr + FROM system.one + WHERE NOT 1 + UNION ALL + SELECT max(-1) AS aggr + FROM system.one + WHERE not 1 +); +SET aggregate_functions_null_for_empty=1; +SELECT MAX(aggr) +FROM +( + SELECT MAX(-1) AS aggr + FROM system.one + WHERE NOT 1 + UNION ALL + SELECT MAX(-1) AS aggr + FROM system.one + WHERE 1 + +); +SELECT MaX(aggr) +FROM +( + SELECT mAX(-1) AS aggr + FROM system.one + WHERE NOT 1 + UNION ALL + SELECT MAx(-1) AS aggr + FROM system.one + WHERE 1 +); +SELECT MaX(aggr) +FROM +( + SELECT mAX(-1) AS aggr + FROM system.one + WHERE NOT 1 + UNION ALL + SELECT max(-1) AS aggr + FROM system.one + WHERE 1 +); +SELECT MaX(aggr) +FROM +( + SELECT mAX(-1) AS aggr + FROM system.one + WHERE NOT 1 + UNION ALL + SELECT max(-1) AS aggr + FROM system.one + WHERE not 1 +); From ca3a1b15c29fcd97238161d0a3cdaafb8214bd2b Mon Sep 17 00:00:00 2001 From: annvsh Date: Sat, 7 Nov 2020 12:37:34 +0700 Subject: [PATCH 064/205] Fixed --- docs/en/operations/settings/settings.md | 81 +++++++++++++++---------- docs/ru/operations/settings/settings.md | 75 ++++++++++++++--------- 2 files changed, 95 insertions(+), 61 deletions(-) diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index 229a7c90071..74120fc5459 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -307,45 +307,54 @@ Disabled by default. ## input_format_tsv_enum_as_number {#settings-input_format_tsv_enum_as_number} -Enables or disables to parsing enum values as enum ids for TSV input format. +Enables or disables parsing enum values as enum ids for TSV input format. Possible values: -- 0 — Disabled. -- 1 — Enabled. +- 0 — Enum values are parsed as values. +- 1 — Enum values are parsed as enum IDs Default value: 0. **Example** -Query: +Consider the table: ```sql -DROP TABLE IF EXISTS table_with_enum_column_for_tsv_insert; +CREATE TABLE table_with_enum_column_for_tsv_insert (Id Int32,Value Enum('first' = 1, 'second' = 2)) ENGINE=Memory() FORMAT TabSeparatedRaw; +``` -CREATE TABLE table_with_enum_column_for_tsv_insert (Id Int32,Value Enum('ef' = 1, 'es' = 2)) ENGINE=Memory() FORMAT TabSeparatedRaw; +When the `input_format_tsv_enum_as_number` setting is enabled: +```sql SET input_format_tsv_enum_as_number = 1; - -INSERT INTO table_with_enum_column_for_tsv_insert VALUES (102, 2); -INSERT INTO table_with_enum_column_for_tsv_insert VALUES (103, 1); +INSERT INTO table_with_enum_column_for_tsv_insert FORMAT TSV 102 2; +INSERT INTO table_with_enum_column_for_tsv_insert FORMAT TSV 103 1; SELECT * FROM table_with_enum_column_for_tsv_insert; - -SET input_format_tsv_enum_as_number = 0; -DROP TABLE IF EXISTS table_with_enum_column_for_tsv_insert; ``` Result: ```text -┌──Id─┬─Value─┐ -│ 102 │ es │ -└─────┴───────┘ -┌──Id─┬─Value─┐ -│ 103 │ ef │ -└─────┴───────┘ +┌──Id─┬─Value──┐ +│ 102 │ second │ +└─────┴────────┘ +┌──Id─┬─Value──┐ +│ 103 │ first │ +└─────┴────────┘ ``` +When the `input_format_tsv_enum_as_number` setting is disabled, the `INSERT` query: + +```sql +SET input_format_tsv_enum_as_number = 0; +INSERT INTO table_with_enum_column_for_tsv_insert FORMAT TSV 102 2; +INSERT INTO table_with_enum_column_for_tsv_insert FORMAT TSV 103 1; +SELECT * FROM table_with_enum_column_for_tsv_insert; +``` + +throws an exception. + ## input_format_null_as_default {#settings-input-format-null-as-default} Enables or disables using default values if input data contain `NULL`, but the data type of the corresponding column in not `Nullable(T)` (for text input formats). @@ -1204,41 +1213,49 @@ For CSV input format enables or disables parsing of unquoted `NULL` as literal ( ## input_format_csv_enum_as_number {#settings-input_format_csv_enum_as_number} -Enables or disables to parsing enum values as enum ids for CSV input format. +Enables or disables parsing enum values as enum ids for CSV input format. Possible values: -- 0 — Disabled. -- 1 — Enabled. +- 0 — Enum values are parsed as values. +- 1 — Enum values are parsed as enum IDs. Default value: 0. -**Example** +**Examples** -Query: +Consider the table: ```sql -DROP TABLE IF EXISTS table_with_enum_column_for_csv_insert; +CREATE TABLE table_with_enum_column_for_csv_insert (Id Int32,Value Enum('first' = 1, 'second' = 2)) ENGINE=Memory(); +``` -CREATE TABLE table_with_enum_column_for_csv_insert (Id Int32,Value Enum('ef' = 1, 'es' = 2)) ENGINE=Memory(); +When the `input_format_csv_enum_as_number` setting is enabled: +```sql SET input_format_csv_enum_as_number = 1; - INSERT INTO table_with_enum_column_for_csv_insert FORMAT CSV 102,2; SELECT * FROM table_with_enum_column_for_csv_insert; - -SET input_format_csv_enum_as_number = 0; -DROP TABLE IF EXISTS table_with_enum_column_for_csv_insert; ``` Result: ```text -┌──Id─┬─Value─┐ -│ 102 │ es │ -└─────┴───────┘ +┌──Id─┬─Value─────┐ +│ 102 │ second │ +└─────┴───────────┘ ``` +When the `input_format_csv_enum_as_number` setting is disabled, the `INSERT` query: + +```sql +SET input_format_csv_enum_as_number = 0; +INSERT INTO table_with_enum_column_for_csv_insert FORMAT CSV 102,2; +SELECT * FROM table_with_enum_column_for_csv_insert; +``` + +throws an exception. + ## output_format_csv_crlf_end_of_line {#settings-output-format-csv-crlf-end-of-line} Use DOS/Windows-style line separator (CRLF) in CSV instead of Unix style (LF). diff --git a/docs/ru/operations/settings/settings.md b/docs/ru/operations/settings/settings.md index ee4b9202cab..2ea65e89579 100644 --- a/docs/ru/operations/settings/settings.md +++ b/docs/ru/operations/settings/settings.md @@ -295,41 +295,50 @@ Disabled by default. Возможные значения: -- 0 — выключена. -- 1 — включена. +- 0 — парсинг значений перечисления как значений. +- 1 — парсинг значений перечисления как идентификаторов перечисления. Значение по умолчанию: 0. **Пример** -Запрос: +Рассмотрим таблицу: ```sql -DROP TABLE IF EXISTS table_with_enum_column_for_tsv_insert; +CREATE TABLE table_with_enum_column_for_tsv_insert (Id Int32,Value Enum('first' = 1, 'second' = 2)) ENGINE=Memory() FORMAT TabSeparatedRaw; +``` -CREATE TABLE table_with_enum_column_for_tsv_insert (Id Int32,Value Enum('ef' = 1, 'es' = 2)) ENGINE=Memory() FORMAT TabSeparatedRaw; +При включенной настройке `input_format_tsv_enum_as_number`: +```sql SET input_format_tsv_enum_as_number = 1; - -INSERT INTO table_with_enum_column_for_tsv_insert VALUES (102, 2); -INSERT INTO table_with_enum_column_for_tsv_insert VALUES (103, 1); +INSERT INTO table_with_enum_column_for_tsv_insert FORMAT TSV 102 2; +INSERT INTO table_with_enum_column_for_tsv_insert FORMAT TSV 103 1; SELECT * FROM table_with_enum_column_for_tsv_insert; - -SET input_format_tsv_enum_as_number = 0; -DROP TABLE IF EXISTS table_with_enum_column_for_tsv_insert; ``` Результат: ```text -┌──Id─┬─Value─┐ -│ 102 │ es │ -└─────┴───────┘ -┌──Id─┬─Value─┐ -│ 103 │ ef │ -└─────┴───────┘ +┌──Id─┬─Value──┐ +│ 102 │ second │ +└─────┴────────┘ +┌──Id─┬─Value──┐ +│ 103 │ first │ +└─────┴────────┘ ``` +При отключенной настройке `input_format_tsv_enum_as_number` запрос `INSERT`: + +```sql +SET input_format_tsv_enum_as_number = 0; +INSERT INTO table_with_enum_column_for_tsv_insert FORMAT TSV 102 2; +INSERT INTO table_with_enum_column_for_tsv_insert FORMAT TSV 103 1; +SELECT * FROM table_with_enum_column_for_tsv_insert; +``` + +сгенерирует исключение. + ## input_format_null_as_default {#settings-input-format-null-as-default} Включает или отключает использование значений по умолчанию в случаях, когда во входных данных содержится `NULL`, но тип соответствующего столбца не `Nullable(T)` (для текстовых форматов). @@ -1174,37 +1183,45 @@ SELECT area/period FROM account_orders FORMAT JSON; Возможные значения: -- 0 — выключена. -- 1 — включена. +- 0 — парсинг значений перечисления как значений. +- 1 — парсинг значений перечисления как идентификаторов перечисления. Значение по умолчанию: 0. **Пример** -Запрос: +Рассмотрим таблицу: ```sql -DROP TABLE IF EXISTS table_with_enum_column_for_csv_insert; +CREATE TABLE table_with_enum_column_for_csv_insert (Id Int32,Value Enum('first' = 1, 'second' = 2)) ENGINE=Memory(); +``` -CREATE TABLE table_with_enum_column_for_csv_insert (Id Int32,Value Enum('ef' = 1, 'es' = 2)) ENGINE=Memory(); +При включенной настройке `input_format_csv_enum_as_number`: +```sql SET input_format_csv_enum_as_number = 1; - INSERT INTO table_with_enum_column_for_csv_insert FORMAT CSV 102,2; SELECT * FROM table_with_enum_column_for_csv_insert; - -SET input_format_csv_enum_as_number = 0; -DROP TABLE IF EXISTS table_with_enum_column_for_csv_insert; ``` Результат: ```text -┌──Id─┬─Value─┐ -│ 102 │ es │ -└─────┴───────┘ +┌──Id─┬─Value──┐ +│ 102 │ second │ +└─────┴────────┘ ``` +При отключенной настройке `input_format_csv_enum_as_number` запрос `INSERT`: + +```sql +SET input_format_csv_enum_as_number = 0; +INSERT INTO table_with_enum_column_for_csv_insert FORMAT CSV 102,2; +SELECT * FROM table_with_enum_column_for_csv_insert; +``` + +сгенерирует исключение. + ## output_format_csv_crlf_end_of_line {#settings-output-format-csv-crlf-end-of-line} Использовать в качестве разделителя строк для CSV формата CRLF (DOS/Windows стиль) вместо LF (Unix стиль). From 73e5d283aa472ce897e9df57be32265ff686cc9d Mon Sep 17 00:00:00 2001 From: "a.palagashvili" Date: Sat, 7 Nov 2020 16:24:21 +0300 Subject: [PATCH 065/205] regenerated ya.make --- src/IO/ya.make | 2 ++ 1 file changed, 2 insertions(+) diff --git a/src/IO/ya.make b/src/IO/ya.make index 3796494ff33..f3b14a94a94 100644 --- a/src/IO/ya.make +++ b/src/IO/ya.make @@ -24,6 +24,8 @@ SRCS( HexWriteBuffer.cpp HTTPCommon.cpp LimitReadBuffer.cpp + LzmaReadBuffer.cpp + LzmaWriteBuffer.cpp MemoryReadWriteBuffer.cpp MMapReadBufferFromFile.cpp MMapReadBufferFromFileDescriptor.cpp From a1fee4ddc59cbe52489966cd4d3de2b4187bfa3a Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Sun, 8 Nov 2020 18:35:22 +0300 Subject: [PATCH 066/205] Update embedded-rocksdb.md --- .../engines/table-engines/integrations/embedded-rocksdb.md | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/docs/en/engines/table-engines/integrations/embedded-rocksdb.md b/docs/en/engines/table-engines/integrations/embedded-rocksdb.md index 88a95ae7de7..3b23fa63b72 100644 --- a/docs/en/engines/table-engines/integrations/embedded-rocksdb.md +++ b/docs/en/engines/table-engines/integrations/embedded-rocksdb.md @@ -40,6 +40,6 @@ PRIMARY KEY key ## Description {#description} -- `primary key` must be specified, only support one primary key. The primary key will serializeBinary as rocksdb key. -- Columns other than the primary key will be serializeBinary as rocksdb value in corresponding order. -- Queries with key `equals` or `in` filtering will be optimized to multi keys look up from rocksdb. +- `primary key` must be specified, it only supports one column in primary key. The primary key will serialized in binary as rocksdb key. +- columns other than the primary key will be serialized in binary as rocksdb value in corresponding order. +- queries with key `equals` or `in` filtering will be optimized to multi keys lookup from rocksdb. From dd2f31c0b473dba887e144fad2c2878bc64c8980 Mon Sep 17 00:00:00 2001 From: feng lv Date: Sun, 8 Nov 2020 15:36:20 +0000 Subject: [PATCH 067/205] cancel combinators native names constraint --- .../AggregateFunctionFactory.cpp | 47 +++++++------------ .../AggregateFunctionFactory.h | 14 +++--- src/Interpreters/TreeRewriter.cpp | 2 +- ...ombinator_native_name_constraint.reference | 5 ++ ...func_combinator_native_name_constraint.sql | 5 ++ 5 files changed, 34 insertions(+), 39 deletions(-) create mode 100644 tests/queries/0_stateless/01560_cancel_agg_func_combinator_native_name_constraint.reference create mode 100644 tests/queries/0_stateless/01560_cancel_agg_func_combinator_native_name_constraint.sql diff --git a/src/AggregateFunctions/AggregateFunctionFactory.cpp b/src/AggregateFunctions/AggregateFunctionFactory.cpp index f7c6fe9da14..1c66650be45 100644 --- a/src/AggregateFunctions/AggregateFunctionFactory.cpp +++ b/src/AggregateFunctions/AggregateFunctionFactory.cpp @@ -56,11 +56,7 @@ static DataTypes convertLowCardinalityTypesToNested(const DataTypes & types) } AggregateFunctionPtr AggregateFunctionFactory::get( - const String & name, - const DataTypes & argument_types, - const Array & parameters, - AggregateFunctionProperties & out_properties, - int recursion_level) const + const String & name, const DataTypes & argument_types, const Array & parameters, AggregateFunctionProperties & out_properties) const { auto type_without_low_cardinality = convertLowCardinalityTypesToNested(argument_types); @@ -81,11 +77,11 @@ AggregateFunctionPtr AggregateFunctionFactory::get( [](const auto & type) { return type->onlyNull(); }); AggregateFunctionPtr nested_function = getImpl( - name, nested_types, nested_parameters, out_properties, has_null_arguments, recursion_level); + name, nested_types, nested_parameters, out_properties, has_null_arguments); return combinator->transformAggregateFunction(nested_function, out_properties, type_without_low_cardinality, parameters); } - auto res = getImpl(name, type_without_low_cardinality, parameters, out_properties, false, recursion_level); + auto res = getImpl(name, type_without_low_cardinality, parameters, out_properties, false); if (!res) throw Exception("Logical error: AggregateFunctionFactory returned nullptr", ErrorCodes::LOGICAL_ERROR); return res; @@ -97,8 +93,7 @@ AggregateFunctionPtr AggregateFunctionFactory::getImpl( const DataTypes & argument_types, const Array & parameters, AggregateFunctionProperties & out_properties, - bool has_null_arguments, - int recursion_level) const + bool has_null_arguments) const { String name = getAliasToOrName(name_param); Value found; @@ -108,13 +103,9 @@ AggregateFunctionPtr AggregateFunctionFactory::getImpl( { found = it->second; } - /// Find by case-insensitive name. - /// Combinators cannot apply for case insensitive (SQL-style) aggregate function names. Only for native names. - else if (recursion_level == 0) - { - if (auto jt = case_insensitive_aggregate_functions.find(Poco::toLower(name)); jt != case_insensitive_aggregate_functions.end()) - found = jt->second; - } + + if (auto jt = case_insensitive_aggregate_functions.find(Poco::toLower(name)); jt != case_insensitive_aggregate_functions.end()) + found = jt->second; if (found.creator) { @@ -140,7 +131,7 @@ AggregateFunctionPtr AggregateFunctionFactory::getImpl( DataTypes nested_types = combinator->transformArguments(argument_types); Array nested_parameters = combinator->transformParameters(parameters); - AggregateFunctionPtr nested_function = get(nested_name, nested_types, nested_parameters, out_properties, recursion_level + 1); + AggregateFunctionPtr nested_function = get(nested_name, nested_types, nested_parameters, out_properties); return combinator->transformAggregateFunction(nested_function, out_properties, argument_types, parameters); } @@ -162,7 +153,7 @@ AggregateFunctionPtr AggregateFunctionFactory::tryGet( } -std::optional AggregateFunctionFactory::tryGetPropertiesImpl(const String & name_param, int recursion_level) const +std::optional AggregateFunctionFactory::tryGetPropertiesImpl(const String & name_param) const { String name = getAliasToOrName(name_param); Value found; @@ -172,13 +163,9 @@ std::optional AggregateFunctionFactory::tryGetPrope { found = it->second; } - /// Find by case-insensitive name. - /// Combinators cannot apply for case insensitive (SQL-style) aggregate function names. Only for native names. - else if (recursion_level == 0) - { - if (auto jt = case_insensitive_aggregate_functions.find(Poco::toLower(name)); jt != case_insensitive_aggregate_functions.end()) - found = jt->second; - } + + if (auto jt = case_insensitive_aggregate_functions.find(Poco::toLower(name)); jt != case_insensitive_aggregate_functions.end()) + found = jt->second; if (found.creator) return found.properties; @@ -195,7 +182,7 @@ std::optional AggregateFunctionFactory::tryGetPrope String nested_name = name.substr(0, name.size() - combinator->getName().size()); /// NOTE: It's reasonable to also allow to transform properties by combinator. - return tryGetPropertiesImpl(nested_name, recursion_level + 1); + return tryGetPropertiesImpl(nested_name); } return {}; @@ -204,21 +191,21 @@ std::optional AggregateFunctionFactory::tryGetPrope std::optional AggregateFunctionFactory::tryGetProperties(const String & name) const { - return tryGetPropertiesImpl(name, 0); + return tryGetPropertiesImpl(name); } -bool AggregateFunctionFactory::isAggregateFunctionName(const String & name, int recursion_level) const +bool AggregateFunctionFactory::isAggregateFunctionName(const String & name) const { if (aggregate_functions.count(name) || isAlias(name)) return true; String name_lowercase = Poco::toLower(name); - if (recursion_level == 0 && (case_insensitive_aggregate_functions.count(name_lowercase) || isAlias(name_lowercase))) + if (case_insensitive_aggregate_functions.count(name_lowercase) || isAlias(name_lowercase)) return true; if (AggregateFunctionCombinatorPtr combinator = AggregateFunctionCombinatorFactory::instance().tryFindSuffix(name)) - return isAggregateFunctionName(name.substr(0, name.size() - combinator->getName().size()), recursion_level + 1); + return isAggregateFunctionName(name.substr(0, name.size() - combinator->getName().size())); return false; } diff --git a/src/AggregateFunctions/AggregateFunctionFactory.h b/src/AggregateFunctions/AggregateFunctionFactory.h index 143e6562a30..7bbe04aff70 100644 --- a/src/AggregateFunctions/AggregateFunctionFactory.h +++ b/src/AggregateFunctions/AggregateFunctionFactory.h @@ -59,12 +59,11 @@ public: CaseSensitiveness case_sensitiveness = CaseSensitive); /// Throws an exception if not found. - AggregateFunctionPtr get( - const String & name, + AggregateFunctionPtr + get(const String & name, const DataTypes & argument_types, const Array & parameters, - AggregateFunctionProperties & out_properties, - int recursion_level = 0) const; + AggregateFunctionProperties & out_properties) const; /// Returns nullptr if not found. AggregateFunctionPtr tryGet( @@ -76,7 +75,7 @@ public: /// Get properties if the aggregate function exists. std::optional tryGetProperties(const String & name) const; - bool isAggregateFunctionName(const String & name, int recursion_level = 0) const; + bool isAggregateFunctionName(const String & name) const; private: AggregateFunctionPtr getImpl( @@ -84,10 +83,9 @@ private: const DataTypes & argument_types, const Array & parameters, AggregateFunctionProperties & out_properties, - bool has_null_arguments, - int recursion_level) const; + bool has_null_arguments) const; - std::optional tryGetPropertiesImpl(const String & name, int recursion_level) const; + std::optional tryGetPropertiesImpl(const String & name) const; private: using AggregateFunctions = std::unordered_map; diff --git a/src/Interpreters/TreeRewriter.cpp b/src/Interpreters/TreeRewriter.cpp index b4d4e030028..c8691c25f1b 100644 --- a/src/Interpreters/TreeRewriter.cpp +++ b/src/Interpreters/TreeRewriter.cpp @@ -125,7 +125,7 @@ struct CustomizeAggregateFunctionsSuffixData { auto properties = instance.tryGetProperties(func.name); if (properties && !properties->returns_default_when_only_null) - func.name = Poco::toLower(func.name) + customized_func_suffix; + func.name = func.name + customized_func_suffix; } } }; diff --git a/tests/queries/0_stateless/01560_cancel_agg_func_combinator_native_name_constraint.reference b/tests/queries/0_stateless/01560_cancel_agg_func_combinator_native_name_constraint.reference new file mode 100644 index 00000000000..6fd1fc6c572 --- /dev/null +++ b/tests/queries/0_stateless/01560_cancel_agg_func_combinator_native_name_constraint.reference @@ -0,0 +1,5 @@ +1 +1 +1 +1 +\N diff --git a/tests/queries/0_stateless/01560_cancel_agg_func_combinator_native_name_constraint.sql b/tests/queries/0_stateless/01560_cancel_agg_func_combinator_native_name_constraint.sql new file mode 100644 index 00000000000..096ba663923 --- /dev/null +++ b/tests/queries/0_stateless/01560_cancel_agg_func_combinator_native_name_constraint.sql @@ -0,0 +1,5 @@ +SELECT Sum(1); +SELECT SumOrNull(1); +SELECT SUMOrNull(1); +SELECT SUMOrNullIf(1, 1); +SELECT SUMOrNullIf(1, 0); From c89a3639807c8ce9ec03c64f6a06789ee6cb9add Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 8 Nov 2020 18:41:16 +0300 Subject: [PATCH 068/205] Rename Rocksdb to RocksDB --- .../integrations/embedded-rocksdb.md | 12 ++--- src/CMakeLists.txt | 2 +- .../EmbeddedRocksDBBlockInputStream.cpp} | 10 ++--- .../EmbeddedRocksDBBlockInputStream.h} | 10 ++--- .../EmbeddedRocksDBBlockOutputStream.cpp} | 8 ++-- .../EmbeddedRocksDBBlockOutputStream.h} | 10 ++--- .../StorageEmbeddedRocksDB.cpp} | 44 +++++++++---------- .../StorageEmbeddedRocksDB.h} | 20 ++++----- src/Storages/registerStorages.cpp | 2 +- src/Storages/registerStorages.h | 2 +- tests/queries/0_stateless/01504_rocksdb.sql | 4 +- 11 files changed, 62 insertions(+), 62 deletions(-) rename src/Storages/{Rocksdb/EmbeddedRocksdbBlockInputStream.cpp => RocksDB/EmbeddedRocksDBBlockInputStream.cpp} (85%) rename src/Storages/{Rocksdb/EmbeddedRocksdbBlockInputStream.h => RocksDB/EmbeddedRocksDBBlockInputStream.h} (67%) rename src/Storages/{Rocksdb/EmbeddedRocksdbBlockOutputStream.cpp => RocksDB/EmbeddedRocksDBBlockOutputStream.cpp} (81%) rename src/Storages/{Rocksdb/EmbeddedRocksdbBlockOutputStream.h => RocksDB/EmbeddedRocksDBBlockOutputStream.h} (62%) rename src/Storages/{Rocksdb/StorageEmbeddedRocksdb.cpp => RocksDB/StorageEmbeddedRocksDB.cpp} (90%) rename src/Storages/{Rocksdb/StorageEmbeddedRocksdb.h => RocksDB/StorageEmbeddedRocksDB.h} (74%) diff --git a/docs/en/engines/table-engines/integrations/embedded-rocksdb.md b/docs/en/engines/table-engines/integrations/embedded-rocksdb.md index 3b23fa63b72..857e148277c 100644 --- a/docs/en/engines/table-engines/integrations/embedded-rocksdb.md +++ b/docs/en/engines/table-engines/integrations/embedded-rocksdb.md @@ -1,15 +1,15 @@ --- toc_priority: 6 -toc_title: EmbeddedRocksdb +toc_title: EmbeddedRocksDB --- -# EmbeddedRocksdb Engine {#EmbeddedRocksdb-engine} +# EmbeddedRocksDB Engine {#EmbeddedRocksDB-engine} This engine allows integrating ClickHouse with [rocksdb](http://rocksdb.org/). -`EmbeddedRocksdb` lets you: +`EmbeddedRocksDB` lets you: -## Creating a Table {#table_engine-EmbeddedRocksdb-creating-a-table} +## Creating a Table {#table_engine-EmbeddedRocksDB-creating-a-table} ``` sql CREATE TABLE [IF NOT EXISTS] [db.]table_name [ON CLUSTER cluster] @@ -17,7 +17,7 @@ CREATE TABLE [IF NOT EXISTS] [db.]table_name [ON CLUSTER cluster] name1 [type1] [DEFAULT|MATERIALIZED|ALIAS expr1], name2 [type2] [DEFAULT|MATERIALIZED|ALIAS expr2], ... -) ENGINE = EmbeddedRocksdb PRIMARY KEY(primary_key_name) +) ENGINE = EmbeddedRocksDB PRIMARY KEY(primary_key_name) ``` Required parameters: @@ -34,7 +34,7 @@ CREATE TABLE test `v2` String, `v3` Float32, ) -ENGINE = EmbeddedRocksdb +ENGINE = EmbeddedRocksDB PRIMARY KEY key ``` diff --git a/src/CMakeLists.txt b/src/CMakeLists.txt index 3af10090f20..c472622b5a7 100644 --- a/src/CMakeLists.txt +++ b/src/CMakeLists.txt @@ -79,7 +79,7 @@ if (USE_AMQPCPP) endif() if (USE_ROCKSDB) - add_headers_and_sources(dbms Storages/Rocksdb) + add_headers_and_sources(dbms Storages/RocksDB) endif() if (USE_AWS_S3) diff --git a/src/Storages/Rocksdb/EmbeddedRocksdbBlockInputStream.cpp b/src/Storages/RocksDB/EmbeddedRocksDBBlockInputStream.cpp similarity index 85% rename from src/Storages/Rocksdb/EmbeddedRocksdbBlockInputStream.cpp rename to src/Storages/RocksDB/EmbeddedRocksDBBlockInputStream.cpp index 0f75f41c7dd..11944c7e295 100644 --- a/src/Storages/Rocksdb/EmbeddedRocksdbBlockInputStream.cpp +++ b/src/Storages/RocksDB/EmbeddedRocksDBBlockInputStream.cpp @@ -1,7 +1,7 @@ #include #include -#include -#include +#include +#include #include @@ -14,8 +14,8 @@ namespace ErrorCodes extern const int LOGICAL_ERROR; } -EmbeddedRocksdbBlockInputStream::EmbeddedRocksdbBlockInputStream( - StorageEmbeddedRocksdb & storage_, +EmbeddedRocksDBBlockInputStream::EmbeddedRocksDBBlockInputStream( + StorageEmbeddedRocksDB & storage_, const StorageMetadataPtr & metadata_snapshot_, size_t max_block_size_) : storage(storage_) @@ -26,7 +26,7 @@ EmbeddedRocksdbBlockInputStream::EmbeddedRocksdbBlockInputStream( primary_key_pos = sample_block.getPositionByName(storage.primary_key); } -Block EmbeddedRocksdbBlockInputStream::readImpl() +Block EmbeddedRocksDBBlockInputStream::readImpl() { if (finished) return {}; diff --git a/src/Storages/Rocksdb/EmbeddedRocksdbBlockInputStream.h b/src/Storages/RocksDB/EmbeddedRocksDBBlockInputStream.h similarity index 67% rename from src/Storages/Rocksdb/EmbeddedRocksdbBlockInputStream.h rename to src/Storages/RocksDB/EmbeddedRocksDBBlockInputStream.h index 2caf434ffd8..cc7911e0087 100644 --- a/src/Storages/Rocksdb/EmbeddedRocksdbBlockInputStream.h +++ b/src/Storages/RocksDB/EmbeddedRocksDBBlockInputStream.h @@ -1,26 +1,26 @@ #pragma once #include -#include +#include #include namespace DB { -class EmbeddedRocksdbBlockInputStream : public IBlockInputStream +class EmbeddedRocksDBBlockInputStream : public IBlockInputStream { public: - EmbeddedRocksdbBlockInputStream( - StorageEmbeddedRocksdb & storage_, const StorageMetadataPtr & metadata_snapshot_, size_t max_block_size_); + EmbeddedRocksDBBlockInputStream( + StorageEmbeddedRocksDB & storage_, const StorageMetadataPtr & metadata_snapshot_, size_t max_block_size_); String getName() const override { return storage.getName(); } Block getHeader() const override { return sample_block; } Block readImpl() override; private: - StorageEmbeddedRocksdb & storage; + StorageEmbeddedRocksDB & storage; StorageMetadataPtr metadata_snapshot; const size_t max_block_size; diff --git a/src/Storages/Rocksdb/EmbeddedRocksdbBlockOutputStream.cpp b/src/Storages/RocksDB/EmbeddedRocksDBBlockOutputStream.cpp similarity index 81% rename from src/Storages/Rocksdb/EmbeddedRocksdbBlockOutputStream.cpp rename to src/Storages/RocksDB/EmbeddedRocksDBBlockOutputStream.cpp index aebc41addda..e7180c18bf4 100644 --- a/src/Storages/Rocksdb/EmbeddedRocksdbBlockOutputStream.cpp +++ b/src/Storages/RocksDB/EmbeddedRocksDBBlockOutputStream.cpp @@ -1,5 +1,5 @@ -#include +#include #include namespace DB @@ -10,12 +10,12 @@ namespace ErrorCodes extern const int SYSTEM_ERROR; } -Block EmbeddedRocksdbBlockOutputStream::getHeader() const +Block EmbeddedRocksDBBlockOutputStream::getHeader() const { return metadata_snapshot->getSampleBlock(); } -void EmbeddedRocksdbBlockOutputStream::write(const Block & block) +void EmbeddedRocksDBBlockOutputStream::write(const Block & block) { metadata_snapshot->check(block, true); auto rows = block.rows(); @@ -44,7 +44,7 @@ void EmbeddedRocksdbBlockOutputStream::write(const Block & block) } auto status = storage.rocksdb_ptr->Write(rocksdb::WriteOptions(), &batch); if (!status.ok()) - throw Exception("Rocksdb write error: " + status.ToString(), ErrorCodes::SYSTEM_ERROR); + throw Exception("RocksDB write error: " + status.ToString(), ErrorCodes::SYSTEM_ERROR); } } diff --git a/src/Storages/Rocksdb/EmbeddedRocksdbBlockOutputStream.h b/src/Storages/RocksDB/EmbeddedRocksDBBlockOutputStream.h similarity index 62% rename from src/Storages/Rocksdb/EmbeddedRocksdbBlockOutputStream.h rename to src/Storages/RocksDB/EmbeddedRocksDBBlockOutputStream.h index 5ead88216b1..a1412b90856 100644 --- a/src/Storages/Rocksdb/EmbeddedRocksdbBlockOutputStream.h +++ b/src/Storages/RocksDB/EmbeddedRocksDBBlockOutputStream.h @@ -1,17 +1,17 @@ #pragma once #include -#include +#include #include namespace DB { -class EmbeddedRocksdbBlockOutputStream : public IBlockOutputStream +class EmbeddedRocksDBBlockOutputStream : public IBlockOutputStream { public: - explicit EmbeddedRocksdbBlockOutputStream( - StorageEmbeddedRocksdb & storage_, + explicit EmbeddedRocksDBBlockOutputStream( + StorageEmbeddedRocksDB & storage_, const StorageMetadataPtr & metadata_snapshot_) : storage(storage_) , metadata_snapshot(metadata_snapshot_) @@ -21,7 +21,7 @@ public: void write(const Block & block) override; private: - StorageEmbeddedRocksdb & storage; + StorageEmbeddedRocksDB & storage; StorageMetadataPtr metadata_snapshot; }; diff --git a/src/Storages/Rocksdb/StorageEmbeddedRocksdb.cpp b/src/Storages/RocksDB/StorageEmbeddedRocksDB.cpp similarity index 90% rename from src/Storages/Rocksdb/StorageEmbeddedRocksdb.cpp rename to src/Storages/RocksDB/StorageEmbeddedRocksDB.cpp index d47cc408d39..cad33de5c25 100644 --- a/src/Storages/Rocksdb/StorageEmbeddedRocksdb.cpp +++ b/src/Storages/RocksDB/StorageEmbeddedRocksDB.cpp @@ -2,9 +2,9 @@ #include #include #include -#include -#include -#include +#include +#include +#include #include #include #include @@ -153,11 +153,11 @@ static std::pair getFilterKeys(const String & primary_key, co } -class EmbeddedRocksdbSource : public SourceWithProgress +class EmbeddedRocksDBSource : public SourceWithProgress { public: - EmbeddedRocksdbSource( - const StorageEmbeddedRocksdb & storage_, + EmbeddedRocksDBSource( + const StorageEmbeddedRocksDB & storage_, const StorageMetadataPtr & metadata_snapshot_, const FieldVector & keys_, const size_t start_, @@ -226,7 +226,7 @@ public: } private: - const StorageEmbeddedRocksdb & storage; + const StorageEmbeddedRocksDB & storage; const StorageMetadataPtr metadata_snapshot; const size_t start; @@ -238,7 +238,7 @@ private: }; -StorageEmbeddedRocksdb::StorageEmbeddedRocksdb(const StorageID & table_id_, +StorageEmbeddedRocksDB::StorageEmbeddedRocksDB(const StorageID & table_id_, const String & relative_data_path_, const StorageInMemoryMetadata & metadata_, bool attach, @@ -255,7 +255,7 @@ StorageEmbeddedRocksdb::StorageEmbeddedRocksdb(const StorageID & table_id_, initDb(); } -void StorageEmbeddedRocksdb::truncate(const ASTPtr &, const StorageMetadataPtr & , const Context &, TableExclusiveLockHolder &) +void StorageEmbeddedRocksDB::truncate(const ASTPtr &, const StorageMetadataPtr & , const Context &, TableExclusiveLockHolder &) { rocksdb_ptr->Close(); Poco::File(rocksdb_dir).remove(true); @@ -263,7 +263,7 @@ void StorageEmbeddedRocksdb::truncate(const ASTPtr &, const StorageMetadataPtr & initDb(); } -void StorageEmbeddedRocksdb::initDb() +void StorageEmbeddedRocksDB::initDb() { rocksdb::Options options; rocksdb::DB * db; @@ -276,10 +276,10 @@ void StorageEmbeddedRocksdb::initDb() } -Pipe StorageEmbeddedRocksdb::read( +Pipe StorageEmbeddedRocksDB::read( const Names & column_names, const StorageMetadataPtr & metadata_snapshot, - const SelectQueryInfo & query_info, + SelectQueryInfo & query_info, const Context & /*context*/, QueryProcessingStage::Enum /*processed_stage*/, size_t max_block_size, @@ -293,7 +293,7 @@ Pipe StorageEmbeddedRocksdb::read( std::tie(keys, all_scan) = getFilterKeys(primary_key, primary_key_data_type, query_info); if (all_scan) { - auto reader = std::make_shared( + auto reader = std::make_shared( *this, metadata_snapshot, max_block_size); return Pipe(std::make_shared(reader)); } @@ -322,22 +322,22 @@ Pipe StorageEmbeddedRocksdb::read( end = start + batch_per_size > keys.size() ? keys.size() : start + batch_per_size; pipes.emplace_back( - std::make_shared(*this, metadata_snapshot, keys, start, end, max_block_size)); + std::make_shared(*this, metadata_snapshot, keys, start, end, max_block_size)); start += batch_per_size; } return Pipe::unitePipes(std::move(pipes)); } } -BlockOutputStreamPtr StorageEmbeddedRocksdb::write(const ASTPtr & /*query*/, const StorageMetadataPtr & metadata_snapshot, const Context & /*context*/) +BlockOutputStreamPtr StorageEmbeddedRocksDB::write(const ASTPtr & /*query*/, const StorageMetadataPtr & metadata_snapshot, const Context & /*context*/) { - return std::make_shared(*this, metadata_snapshot); + return std::make_shared(*this, metadata_snapshot); } static StoragePtr create(const StorageFactory::Arguments & args) { - // TODO custom RocksdbSettings, table function + // TODO custom RocksDBSettings, table function if (!args.engine_args.empty()) throw Exception( "Engine " + args.engine_name + " doesn't support any arguments (" + toString(args.engine_args.size()) + " given)", @@ -348,25 +348,25 @@ static StoragePtr create(const StorageFactory::Arguments & args) metadata.setConstraints(args.constraints); if (!args.storage_def->primary_key) - throw Exception("StorageEmbeddedRocksdb must require one primary key", ErrorCodes::BAD_ARGUMENTS); + throw Exception("StorageEmbeddedRocksDB must require one primary key", ErrorCodes::BAD_ARGUMENTS); metadata.primary_key = KeyDescription::getKeyFromAST(args.storage_def->primary_key->ptr(), metadata.columns, args.context); auto primary_key_names = metadata.getColumnsRequiredForPrimaryKey(); if (primary_key_names.size() != 1) { - throw Exception("StorageEmbeddedRocksdb must require one primary key", ErrorCodes::BAD_ARGUMENTS); + throw Exception("StorageEmbeddedRocksDB must require one primary key", ErrorCodes::BAD_ARGUMENTS); } - return StorageEmbeddedRocksdb::create(args.table_id, args.relative_data_path, metadata, args.attach, args.context, primary_key_names[0]); + return StorageEmbeddedRocksDB::create(args.table_id, args.relative_data_path, metadata, args.attach, args.context, primary_key_names[0]); } -void registerStorageEmbeddedRocksdb(StorageFactory & factory) +void registerStorageEmbeddedRocksDB(StorageFactory & factory) { StorageFactory::StorageFeatures features{ .supports_sort_order = true, }; - factory.registerStorage("EmbeddedRocksdb", create, features); + factory.registerStorage("EmbeddedRocksDB", create, features); } diff --git a/src/Storages/Rocksdb/StorageEmbeddedRocksdb.h b/src/Storages/RocksDB/StorageEmbeddedRocksDB.h similarity index 74% rename from src/Storages/Rocksdb/StorageEmbeddedRocksdb.h rename to src/Storages/RocksDB/StorageEmbeddedRocksDB.h index c803055223b..209f70ceb52 100644 --- a/src/Storages/Rocksdb/StorageEmbeddedRocksdb.h +++ b/src/Storages/RocksDB/StorageEmbeddedRocksDB.h @@ -15,19 +15,19 @@ namespace DB class Context; -class StorageEmbeddedRocksdb final : public ext::shared_ptr_helper, public IStorage +class StorageEmbeddedRocksDB final : public ext::shared_ptr_helper, public IStorage { - friend struct ext::shared_ptr_helper; - friend class EmbeddedRocksdbSource; - friend class EmbeddedRocksdbBlockOutputStream; - friend class EmbeddedRocksdbBlockInputStream; + friend struct ext::shared_ptr_helper; + friend class EmbeddedRocksDBSource; + friend class EmbeddedRocksDBBlockOutputStream; + friend class EmbeddedRocksDBBlockInputStream; public: - std::string getName() const override { return "EmbeddedRocksdb"; } + std::string getName() const override { return "EmbeddedRocksDB"; } Pipe read( const Names & column_names, const StorageMetadataPtr & metadata_snapshot, - const SelectQueryInfo & query_info, + SelectQueryInfo & query_info, const Context & context, QueryProcessingStage::Enum processed_stage, size_t max_block_size, @@ -44,7 +44,7 @@ public: } protected: - StorageEmbeddedRocksdb(const StorageID & table_id_, + StorageEmbeddedRocksDB(const StorageID & table_id_, const String & relative_data_path_, const StorageInMemoryMetadata & metadata, bool attach, @@ -53,8 +53,8 @@ protected: private: const String primary_key; - using RocksdbPtr = std::unique_ptr; - RocksdbPtr rocksdb_ptr; + using RocksDBPtr = std::unique_ptr; + RocksDBPtr rocksdb_ptr; String rocksdb_dir; void initDb(); diff --git a/src/Storages/registerStorages.cpp b/src/Storages/registerStorages.cpp index f1e38c4336d..13c20fcda8d 100644 --- a/src/Storages/registerStorages.cpp +++ b/src/Storages/registerStorages.cpp @@ -55,7 +55,7 @@ void registerStorages() #endif #if USE_ROCKSDB - registerStorageEmbeddedRocksdb(factory); + registerStorageEmbeddedRocksDB(factory); #endif } diff --git a/src/Storages/registerStorages.h b/src/Storages/registerStorages.h index f74eefe1c00..df7249a64af 100644 --- a/src/Storages/registerStorages.h +++ b/src/Storages/registerStorages.h @@ -55,7 +55,7 @@ void registerStorageRabbitMQ(StorageFactory & factory); #endif #if USE_ROCKSDB -void registerStorageEmbeddedRocksdb(StorageFactory & factory); +void registerStorageEmbeddedRocksDB(StorageFactory & factory); #endif void registerStorages(); diff --git a/tests/queries/0_stateless/01504_rocksdb.sql b/tests/queries/0_stateless/01504_rocksdb.sql index eaaecca96b5..123c6c2d9c6 100644 --- a/tests/queries/0_stateless/01504_rocksdb.sql +++ b/tests/queries/0_stateless/01504_rocksdb.sql @@ -1,5 +1,5 @@ DROP TABLE IF EXISTS test; -CREATE TABLE test (key String, value UInt32) Engine=EmbeddedRocksdb primary key(key); +CREATE TABLE test (key String, value UInt32) Engine=EmbeddedRocksDB primary key(key); INSERT INTO test SELECT '1_1', number FROM numbers(10000); SELECT count(1) == 1 FROM test; @@ -11,7 +11,7 @@ SELECT SUM(value) == 1 + 99 + 900 FROM test WHERE key in ('1_1', '99_1', '900_1' DROP TABLE IF EXISTS test; DROP TABLE IF EXISTS test_memory; -CREATE TABLE test (k UInt32, value UInt64, dummy Tuple(UInt32, Float64), bm AggregateFunction(groupBitmap, UInt64)) Engine=EmbeddedRocksdb primary key(k); +CREATE TABLE test (k UInt32, value UInt64, dummy Tuple(UInt32, Float64), bm AggregateFunction(groupBitmap, UInt64)) Engine=EmbeddedRocksDB primary key(k); CREATE TABLE test_memory AS test Engine = Memory; INSERT INTO test SELECT number % 77 AS k, SUM(number) AS value, (1, 1.2), bitmapBuild(groupArray(number)) FROM numbers(10000000) group by k; From 7d0f1d33c6fdb7b6e7ebc2e5e27ec94c3293698c Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 8 Nov 2020 18:43:05 +0300 Subject: [PATCH 069/205] Update ya.make --- src/Storages/ya.make | 3 --- src/Storages/ya.make.in | 2 +- 2 files changed, 1 insertion(+), 4 deletions(-) diff --git a/src/Storages/ya.make b/src/Storages/ya.make index 3f0114a5d48..0f14826d859 100644 --- a/src/Storages/ya.make +++ b/src/Storages/ya.make @@ -105,9 +105,6 @@ SRCS( MutationCommands.cpp PartitionCommands.cpp ReadInOrderOptimizer.cpp - Rocksdb/EmbeddedRocksdbBlockInputStream.cpp - Rocksdb/EmbeddedRocksdbBlockOutputStream.cpp - Rocksdb/StorageEmbeddedRocksdb.cpp SelectQueryDescription.cpp SetSettings.cpp StorageBuffer.cpp diff --git a/src/Storages/ya.make.in b/src/Storages/ya.make.in index dbae43aa3fc..ad772eb5c50 100644 --- a/src/Storages/ya.make.in +++ b/src/Storages/ya.make.in @@ -8,7 +8,7 @@ PEERDIR( SRCS( - + ) END() From 4353a567900d1b160a9b0125c98803193a7d093e Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 8 Nov 2020 18:45:24 +0300 Subject: [PATCH 070/205] Make code slightly better --- src/Storages/registerStorages.cpp | 56 +++++++++++++++++++++++++++++- src/Storages/registerStorages.h | 57 ------------------------------- 2 files changed, 55 insertions(+), 58 deletions(-) diff --git a/src/Storages/registerStorages.cpp b/src/Storages/registerStorages.cpp index 13c20fcda8d..33c1b6245ac 100644 --- a/src/Storages/registerStorages.cpp +++ b/src/Storages/registerStorages.cpp @@ -1,10 +1,64 @@ #include #include +#if !defined(ARCADIA_BUILD) +# include +# include "config_core.h" +#endif namespace DB { +void registerStorageLog(StorageFactory & factory); +void registerStorageTinyLog(StorageFactory & factory); +void registerStorageStripeLog(StorageFactory & factory); +void registerStorageMergeTree(StorageFactory & factory); +void registerStorageNull(StorageFactory & factory); +void registerStorageMerge(StorageFactory & factory); +void registerStorageBuffer(StorageFactory & factory); +void registerStorageDistributed(StorageFactory & factory); +void registerStorageMemory(StorageFactory & factory); +void registerStorageFile(StorageFactory & factory); +void registerStorageURL(StorageFactory & factory); +void registerStorageDictionary(StorageFactory & factory); +void registerStorageSet(StorageFactory & factory); +void registerStorageJoin(StorageFactory & factory); +void registerStorageView(StorageFactory & factory); +void registerStorageMaterializedView(StorageFactory & factory); +void registerStorageLiveView(StorageFactory & factory); +void registerStorageGenerateRandom(StorageFactory & factory); + +#if USE_AWS_S3 +void registerStorageS3(StorageFactory & factory); +void registerStorageCOS(StorageFactory & factory); +#endif + +#if USE_HDFS +void registerStorageHDFS(StorageFactory & factory); +#endif + +void registerStorageODBC(StorageFactory & factory); +void registerStorageJDBC(StorageFactory & factory); + +#if USE_MYSQL +void registerStorageMySQL(StorageFactory & factory); +#endif + +void registerStorageMongoDB(StorageFactory & factory); + +#if USE_RDKAFKA +void registerStorageKafka(StorageFactory & factory); +#endif + +#if USE_AMQPCPP +void registerStorageRabbitMQ(StorageFactory & factory); +#endif + +#if USE_ROCKSDB +void registerStorageEmbeddedRocksDB(StorageFactory & factory); +#endif + + void registerStorages() { auto & factory = StorageFactory::instance(); @@ -28,7 +82,7 @@ void registerStorages() registerStorageLiveView(factory); registerStorageGenerateRandom(factory); -#if USE_AWS_S3 + #if USE_AWS_S3 registerStorageS3(factory); registerStorageCOS(factory); #endif diff --git a/src/Storages/registerStorages.h b/src/Storages/registerStorages.h index df7249a64af..d44b934ff9f 100644 --- a/src/Storages/registerStorages.h +++ b/src/Storages/registerStorages.h @@ -1,63 +1,6 @@ #pragma once -#if !defined(ARCADIA_BUILD) -# include -# include "config_core.h" -#endif - namespace DB { -class StorageFactory; - -void registerStorageLog(StorageFactory & factory); -void registerStorageTinyLog(StorageFactory & factory); -void registerStorageStripeLog(StorageFactory & factory); -void registerStorageMergeTree(StorageFactory & factory); -void registerStorageNull(StorageFactory & factory); -void registerStorageMerge(StorageFactory & factory); -void registerStorageBuffer(StorageFactory & factory); -void registerStorageDistributed(StorageFactory & factory); -void registerStorageMemory(StorageFactory & factory); -void registerStorageFile(StorageFactory & factory); -void registerStorageURL(StorageFactory & factory); -void registerStorageDictionary(StorageFactory & factory); -void registerStorageSet(StorageFactory & factory); -void registerStorageJoin(StorageFactory & factory); -void registerStorageView(StorageFactory & factory); -void registerStorageMaterializedView(StorageFactory & factory); -void registerStorageLiveView(StorageFactory & factory); -void registerStorageGenerateRandom(StorageFactory & factory); - -#if USE_AWS_S3 -void registerStorageS3(StorageFactory & factory); -void registerStorageCOS(StorageFactory & factory); -#endif - -#if USE_HDFS -void registerStorageHDFS(StorageFactory & factory); -#endif - -void registerStorageODBC(StorageFactory & factory); -void registerStorageJDBC(StorageFactory & factory); - -#if USE_MYSQL -void registerStorageMySQL(StorageFactory & factory); -#endif - -void registerStorageMongoDB(StorageFactory & factory); - -#if USE_RDKAFKA -void registerStorageKafka(StorageFactory & factory); -#endif - -#if USE_AMQPCPP -void registerStorageRabbitMQ(StorageFactory & factory); -#endif - -#if USE_ROCKSDB -void registerStorageEmbeddedRocksDB(StorageFactory & factory); -#endif - void registerStorages(); - } From 2ea93f4b67dbf6e1ba475a99c6218452e7e90931 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 8 Nov 2020 18:52:55 +0300 Subject: [PATCH 071/205] Fix bad wording --- src/Storages/RocksDB/StorageEmbeddedRocksDB.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Storages/RocksDB/StorageEmbeddedRocksDB.cpp b/src/Storages/RocksDB/StorageEmbeddedRocksDB.cpp index cad33de5c25..0aef9ccd705 100644 --- a/src/Storages/RocksDB/StorageEmbeddedRocksDB.cpp +++ b/src/Storages/RocksDB/StorageEmbeddedRocksDB.cpp @@ -348,13 +348,13 @@ static StoragePtr create(const StorageFactory::Arguments & args) metadata.setConstraints(args.constraints); if (!args.storage_def->primary_key) - throw Exception("StorageEmbeddedRocksDB must require one primary key", ErrorCodes::BAD_ARGUMENTS); + throw Exception("StorageEmbeddedRocksDB must require one column in primary key", ErrorCodes::BAD_ARGUMENTS); metadata.primary_key = KeyDescription::getKeyFromAST(args.storage_def->primary_key->ptr(), metadata.columns, args.context); auto primary_key_names = metadata.getColumnsRequiredForPrimaryKey(); if (primary_key_names.size() != 1) { - throw Exception("StorageEmbeddedRocksDB must require one primary key", ErrorCodes::BAD_ARGUMENTS); + throw Exception("StorageEmbeddedRocksDB must require one column in primary key", ErrorCodes::BAD_ARGUMENTS); } return StorageEmbeddedRocksDB::create(args.table_id, args.relative_data_path, metadata, args.attach, args.context, primary_key_names[0]); } From 2d9f07e6017b271505e5f76f0c4651d097b21a45 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 8 Nov 2020 19:24:58 +0300 Subject: [PATCH 072/205] Better code --- src/Common/ErrorCodes.cpp | 1 + .../RocksDB/EmbeddedRocksDBBlockInputStream.cpp | 1 + .../RocksDB/EmbeddedRocksDBBlockInputStream.h | 11 ++++++++--- .../RocksDB/EmbeddedRocksDBBlockOutputStream.cpp | 5 +++-- .../RocksDB/EmbeddedRocksDBBlockOutputStream.h | 4 +++- 5 files changed, 16 insertions(+), 6 deletions(-) diff --git a/src/Common/ErrorCodes.cpp b/src/Common/ErrorCodes.cpp index f069b27827e..74daa4ebbed 100644 --- a/src/Common/ErrorCodes.cpp +++ b/src/Common/ErrorCodes.cpp @@ -519,6 +519,7 @@ M(550, CONDITIONAL_TREE_PARENT_NOT_FOUND) \ M(551, ILLEGAL_PROJECTION_MANIPULATOR) \ M(552, UNRECOGNIZED_ARGUMENTS) \ + M(553, ROCKSDB_ERROR) \ \ M(999, KEEPER_EXCEPTION) \ M(1000, POCO_EXCEPTION) \ diff --git a/src/Storages/RocksDB/EmbeddedRocksDBBlockInputStream.cpp b/src/Storages/RocksDB/EmbeddedRocksDBBlockInputStream.cpp index 11944c7e295..e1a5452661f 100644 --- a/src/Storages/RocksDB/EmbeddedRocksDBBlockInputStream.cpp +++ b/src/Storages/RocksDB/EmbeddedRocksDBBlockInputStream.cpp @@ -2,6 +2,7 @@ #include #include #include +#include #include diff --git a/src/Storages/RocksDB/EmbeddedRocksDBBlockInputStream.h b/src/Storages/RocksDB/EmbeddedRocksDBBlockInputStream.h index cc7911e0087..3de04d70f42 100644 --- a/src/Storages/RocksDB/EmbeddedRocksDBBlockInputStream.h +++ b/src/Storages/RocksDB/EmbeddedRocksDBBlockInputStream.h @@ -1,13 +1,18 @@ #pragma once #include -#include -#include + +namespace rocksdb +{ + class Iterator; +} namespace DB { +class StorageEmbeddedRocksDB; + class EmbeddedRocksDBBlockInputStream : public IBlockInputStream { @@ -15,7 +20,7 @@ public: EmbeddedRocksDBBlockInputStream( StorageEmbeddedRocksDB & storage_, const StorageMetadataPtr & metadata_snapshot_, size_t max_block_size_); - String getName() const override { return storage.getName(); } + String getName() const override { return "EmbeddedRocksDB"; } Block getHeader() const override { return sample_block; } Block readImpl() override; diff --git a/src/Storages/RocksDB/EmbeddedRocksDBBlockOutputStream.cpp b/src/Storages/RocksDB/EmbeddedRocksDBBlockOutputStream.cpp index e7180c18bf4..a6c118d84d3 100644 --- a/src/Storages/RocksDB/EmbeddedRocksDBBlockOutputStream.cpp +++ b/src/Storages/RocksDB/EmbeddedRocksDBBlockOutputStream.cpp @@ -1,5 +1,6 @@ #include +#include #include namespace DB @@ -7,7 +8,7 @@ namespace DB namespace ErrorCodes { - extern const int SYSTEM_ERROR; + extern const int ROCKSDB_ERROR; } Block EmbeddedRocksDBBlockOutputStream::getHeader() const @@ -44,7 +45,7 @@ void EmbeddedRocksDBBlockOutputStream::write(const Block & block) } auto status = storage.rocksdb_ptr->Write(rocksdb::WriteOptions(), &batch); if (!status.ok()) - throw Exception("RocksDB write error: " + status.ToString(), ErrorCodes::SYSTEM_ERROR); + throw Exception("RocksDB write error: " + status.ToString(), ErrorCodes::ROCKSDB_ERROR); } } diff --git a/src/Storages/RocksDB/EmbeddedRocksDBBlockOutputStream.h b/src/Storages/RocksDB/EmbeddedRocksDBBlockOutputStream.h index a1412b90856..4ed5867cb72 100644 --- a/src/Storages/RocksDB/EmbeddedRocksDBBlockOutputStream.h +++ b/src/Storages/RocksDB/EmbeddedRocksDBBlockOutputStream.h @@ -1,12 +1,14 @@ #pragma once #include -#include #include + namespace DB { +class StorageEmbeddedRocksDB; + class EmbeddedRocksDBBlockOutputStream : public IBlockOutputStream { public: From 1250006e4efca681126cca631de7b1c6a29480c5 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 8 Nov 2020 20:09:41 +0300 Subject: [PATCH 073/205] Improve performance two times --- .../EmbeddedRocksDBBlockInputStream.cpp | 11 ++++--- .../EmbeddedRocksDBBlockOutputStream.cpp | 29 +++++++++++++------ .../EmbeddedRocksDBBlockOutputStream.h | 8 ++--- 3 files changed, 30 insertions(+), 18 deletions(-) diff --git a/src/Storages/RocksDB/EmbeddedRocksDBBlockInputStream.cpp b/src/Storages/RocksDB/EmbeddedRocksDBBlockInputStream.cpp index e1a5452661f..767c183f8d5 100644 --- a/src/Storages/RocksDB/EmbeddedRocksDBBlockInputStream.cpp +++ b/src/Storages/RocksDB/EmbeddedRocksDBBlockInputStream.cpp @@ -12,7 +12,7 @@ namespace DB namespace ErrorCodes { - extern const int LOGICAL_ERROR; + extern const int ROCKSDB_ERROR; } EmbeddedRocksDBBlockInputStream::EmbeddedRocksDBBlockInputStream( @@ -39,15 +39,18 @@ Block EmbeddedRocksDBBlockInputStream::readImpl() } MutableColumns columns = sample_block.cloneEmptyColumns(); + size_t rows = 0; for (; iterator->Valid(); iterator->Next()) { ReadBufferFromString key_buffer(iterator->key()); ReadBufferFromString value_buffer(iterator->value()); - for (const auto [idx, column_type] : ext::enumerate(sample_block.getColumnsWithTypeAndName())) + size_t idx = 0; + for (const auto & elem : sample_block) { - column_type.type->deserializeBinary(*columns[idx], idx == primary_key_pos? key_buffer: value_buffer); + elem.type->deserializeBinary(*columns[idx], idx == primary_key_pos ? key_buffer : value_buffer); + ++idx; } ++rows; if (rows >= max_block_size) @@ -58,7 +61,7 @@ Block EmbeddedRocksDBBlockInputStream::readImpl() if (!iterator->status().ok()) { throw Exception("Engine " + getName() + " got error while seeking key value datas: " + iterator->status().ToString(), - ErrorCodes::LOGICAL_ERROR); + ErrorCodes::ROCKSDB_ERROR); } return sample_block.cloneWithColumns(std::move(columns)); } diff --git a/src/Storages/RocksDB/EmbeddedRocksDBBlockOutputStream.cpp b/src/Storages/RocksDB/EmbeddedRocksDBBlockOutputStream.cpp index a6c118d84d3..b00901d6033 100644 --- a/src/Storages/RocksDB/EmbeddedRocksDBBlockOutputStream.cpp +++ b/src/Storages/RocksDB/EmbeddedRocksDBBlockOutputStream.cpp @@ -11,6 +11,21 @@ namespace ErrorCodes extern const int ROCKSDB_ERROR; } +EmbeddedRocksDBBlockOutputStream::EmbeddedRocksDBBlockOutputStream( + StorageEmbeddedRocksDB & storage_, + const StorageMetadataPtr & metadata_snapshot_) + : storage(storage_) + , metadata_snapshot(metadata_snapshot_) +{ + Block sample_block = metadata_snapshot->getSampleBlock(); + for (const auto & elem : sample_block) + { + if (elem.name == storage.primary_key) + break; + ++primary_key_pos; + } +} + Block EmbeddedRocksDBBlockOutputStream::getHeader() const { return metadata_snapshot->getSampleBlock(); @@ -25,24 +40,20 @@ void EmbeddedRocksDBBlockOutputStream::write(const Block & block) WriteBufferFromOwnString wb_value; rocksdb::WriteBatch batch; - auto columns = metadata_snapshot->getColumns(); - for (size_t i = 0; i < rows; i++) { wb_key.restart(); wb_value.restart(); - for (const auto & col : columns) + size_t idx = 0; + for (const auto & elem : block) { - const auto & type = block.getByName(col.name).type; - const auto & column = block.getByName(col.name).column; - if (col.name == storage.primary_key) - type->serializeBinary(*column, i, wb_key); - else - type->serializeBinary(*column, i, wb_value); + elem.type->serializeBinary(*elem.column, i, idx == primary_key_pos ? wb_key : wb_value); + ++idx; } batch.Put(wb_key.str(), wb_value.str()); } + auto status = storage.rocksdb_ptr->Write(rocksdb::WriteOptions(), &batch); if (!status.ok()) throw Exception("RocksDB write error: " + status.ToString(), ErrorCodes::ROCKSDB_ERROR); diff --git a/src/Storages/RocksDB/EmbeddedRocksDBBlockOutputStream.h b/src/Storages/RocksDB/EmbeddedRocksDBBlockOutputStream.h index 4ed5867cb72..51768df11ac 100644 --- a/src/Storages/RocksDB/EmbeddedRocksDBBlockOutputStream.h +++ b/src/Storages/RocksDB/EmbeddedRocksDBBlockOutputStream.h @@ -12,12 +12,9 @@ class StorageEmbeddedRocksDB; class EmbeddedRocksDBBlockOutputStream : public IBlockOutputStream { public: - explicit EmbeddedRocksDBBlockOutputStream( + EmbeddedRocksDBBlockOutputStream( StorageEmbeddedRocksDB & storage_, - const StorageMetadataPtr & metadata_snapshot_) - : storage(storage_) - , metadata_snapshot(metadata_snapshot_) - {} + const StorageMetadataPtr & metadata_snapshot_); Block getHeader() const override; void write(const Block & block) override; @@ -25,6 +22,7 @@ public: private: StorageEmbeddedRocksDB & storage; StorageMetadataPtr metadata_snapshot; + size_t primary_key_pos = 0; }; } From e072d18a83799b432cec94927187aded1947fde3 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 8 Nov 2020 20:13:17 +0300 Subject: [PATCH 074/205] Remove unneeded dir --- src/Storages/RocksDB/StorageEmbeddedRocksDB.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/RocksDB/StorageEmbeddedRocksDB.cpp b/src/Storages/RocksDB/StorageEmbeddedRocksDB.cpp index 0aef9ccd705..38025a997d2 100644 --- a/src/Storages/RocksDB/StorageEmbeddedRocksDB.cpp +++ b/src/Storages/RocksDB/StorageEmbeddedRocksDB.cpp @@ -247,7 +247,7 @@ StorageEmbeddedRocksDB::StorageEmbeddedRocksDB(const StorageID & table_id_, : IStorage(table_id_), primary_key{primary_key_} { setInMemoryMetadata(metadata_); - rocksdb_dir = context_.getPath() + relative_data_path_ + "/rocksdb"; + rocksdb_dir = context_.getPath() + relative_data_path_; if (!attach) { Poco::File(rocksdb_dir).createDirectories(); From fa29c2f00d53d272007640bf691f1c5facc64860 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 8 Nov 2020 20:22:21 +0300 Subject: [PATCH 075/205] More normal --- .../EmbeddedRocksDBBlockInputStream.cpp | 3 +- .../EmbeddedRocksDBBlockOutputStream.cpp | 4 ++- .../RocksDB/StorageEmbeddedRocksDB.cpp | 28 +++++++++---------- src/Storages/RocksDB/StorageEmbeddedRocksDB.h | 14 ++++++---- 4 files changed, 26 insertions(+), 23 deletions(-) diff --git a/src/Storages/RocksDB/EmbeddedRocksDBBlockInputStream.cpp b/src/Storages/RocksDB/EmbeddedRocksDBBlockInputStream.cpp index 767c183f8d5..b7602f8bedb 100644 --- a/src/Storages/RocksDB/EmbeddedRocksDBBlockInputStream.cpp +++ b/src/Storages/RocksDB/EmbeddedRocksDBBlockInputStream.cpp @@ -2,9 +2,8 @@ #include #include #include -#include -#include +#include namespace DB diff --git a/src/Storages/RocksDB/EmbeddedRocksDBBlockOutputStream.cpp b/src/Storages/RocksDB/EmbeddedRocksDBBlockOutputStream.cpp index b00901d6033..a3a0ddb4348 100644 --- a/src/Storages/RocksDB/EmbeddedRocksDBBlockOutputStream.cpp +++ b/src/Storages/RocksDB/EmbeddedRocksDBBlockOutputStream.cpp @@ -1,8 +1,10 @@ - #include #include #include +#include + + namespace DB { diff --git a/src/Storages/RocksDB/StorageEmbeddedRocksDB.cpp b/src/Storages/RocksDB/StorageEmbeddedRocksDB.cpp index 38025a997d2..977b80bc348 100644 --- a/src/Storages/RocksDB/StorageEmbeddedRocksDB.cpp +++ b/src/Storages/RocksDB/StorageEmbeddedRocksDB.cpp @@ -1,10 +1,10 @@ -#include -#include -#include -#include #include #include #include + +#include +#include + #include #include #include @@ -12,28 +12,26 @@ #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 { @@ -213,9 +211,11 @@ public: ReadBufferFromString key_buffer(slices_keys[i]); ReadBufferFromString value_buffer(values[i]); - for (const auto [idx, column_type] : ext::enumerate(sample_block.getColumnsWithTypeAndName())) + size_t idx = 0; + for (const auto & elem : sample_block) { - column_type.type->deserializeBinary(*columns[idx], idx == primary_key_pos? key_buffer: value_buffer); + elem.type->deserializeBinary(*columns[idx], idx == primary_key_pos ? key_buffer : value_buffer); + ++idx; } } } diff --git a/src/Storages/RocksDB/StorageEmbeddedRocksDB.h b/src/Storages/RocksDB/StorageEmbeddedRocksDB.h index 209f70ceb52..bd700a35809 100644 --- a/src/Storages/RocksDB/StorageEmbeddedRocksDB.h +++ b/src/Storages/RocksDB/StorageEmbeddedRocksDB.h @@ -1,14 +1,15 @@ #pragma once +#include #include #include -#include -#include -#include -#include "rocksdb/db.h" -#include "rocksdb/table.h" +namespace rocksdb +{ + class DB; +} + namespace DB { @@ -38,7 +39,8 @@ public: bool supportsParallelInsert() const override { return true; } bool supportsIndexForIn() const override { return true; } - bool mayBenefitFromIndexForIn(const ASTPtr & node, const Context & /*query_context*/, const StorageMetadataPtr & /*metadata_snapshot*/) const override + bool mayBenefitFromIndexForIn( + const ASTPtr & node, const Context & /*query_context*/, const StorageMetadataPtr & /*metadata_snapshot*/) const override { return node->getColumnName() == primary_key; } From 8bb10ac3d84c222f2fc295a97b84db727bd6e11e Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 8 Nov 2020 20:29:16 +0300 Subject: [PATCH 076/205] Fix improper error handling --- src/Storages/RocksDB/StorageEmbeddedRocksDB.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Storages/RocksDB/StorageEmbeddedRocksDB.cpp b/src/Storages/RocksDB/StorageEmbeddedRocksDB.cpp index 977b80bc348..84de695d63b 100644 --- a/src/Storages/RocksDB/StorageEmbeddedRocksDB.cpp +++ b/src/Storages/RocksDB/StorageEmbeddedRocksDB.cpp @@ -38,7 +38,7 @@ namespace DB namespace ErrorCodes { extern const int BAD_ARGUMENTS; - extern const int SYSTEM_ERROR; + extern const int ROCKSDB_ERROR; extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; } @@ -271,7 +271,7 @@ void StorageEmbeddedRocksDB::initDb() rocksdb::Status status = rocksdb::DB::Open(options, rocksdb_dir, &db); if (status != rocksdb::Status::OK()) - throw Exception("Fail to open rocksdb path at: " + rocksdb_dir, ErrorCodes::SYSTEM_ERROR); + throw Exception("Fail to open rocksdb path at: " + rocksdb_dir + ": " + status.ToString(), ErrorCodes::ROCKSDB_ERROR); rocksdb_ptr = std::unique_ptr(db); } From cedaa938f36f7ad7c73e09b4ef0a483f76262e64 Mon Sep 17 00:00:00 2001 From: alesapin Date: Mon, 9 Nov 2020 12:10:32 +0300 Subject: [PATCH 077/205] Add simple integration test --- .../__init__.py | 58 +++++++++++++++ .../test_limited_replicated_fetches/test.py | 71 +++++++++++++++++++ 2 files changed, 129 insertions(+) create mode 100644 tests/integration/test_limited_replicated_fetches/__init__.py create mode 100644 tests/integration/test_limited_replicated_fetches/test.py diff --git a/tests/integration/test_limited_replicated_fetches/__init__.py b/tests/integration/test_limited_replicated_fetches/__init__.py new file mode 100644 index 00000000000..900e89d8bcf --- /dev/null +++ b/tests/integration/test_limited_replicated_fetches/__init__.py @@ -0,0 +1,58 @@ +#!/usr/bin/env python3 + +import pytest +import time +from helpers.cluster import ClickHouseCluster +from helpers.network import PartitionManager +import random +import string + +cluster = ClickHouseCluster(__file__) +node1 = cluster.add_instance('node1', with_zookeeper=True) +node2 = cluster.add_instance('node2', with_zookeeper=True) + +DEFAULT_MAX_THREADS_FOR_FETCH = 3 + +@pytest.fixture(scope="module") +def started_cluster(): + try: + cluster.start() + + yield cluster + + finally: + cluster.shutdown() + + +def get_random_string(length): + return ''.join(random.choice(string.ascii_uppercase + string.digits) for _ in range(length)) + +def test_limited_fetches(started_cluster): + node1.query("CREATE TABLE t (key UInt64, data String) ENGINE = ReplicatedMergeTree('/clickhouse/test/t', '1') ORDER BY tuple() PARTITION BY key") + node2.query("CREATE TABLE t (key UInt64, data String) ENGINE = ReplicatedMergeTree('/clickhouse/test/t', '2') ORDER BY tuple() PARTITION BY key") + + with PartitionManager() as pm: + node2.query("SYSTEM STOP FETCHES t") + node1.query("INSERT INTO t SELECT 1, '{}' FROM numbers(5000)".format(get_random_string(104857))) + node1.query("INSERT INTO t SELECT 2, '{}' FROM numbers(5000)".format(get_random_string(104857))) + node1.query("INSERT INTO t SELECT 3, '{}' FROM numbers(5000)".format(get_random_string(104857))) + node1.query("INSERT INTO t SELECT 4, '{}' FROM numbers(5000)".format(get_random_string(104857))) + node1.query("INSERT INTO t SELECT 5, '{}' FROM numbers(5000)".format(get_random_string(104857))) + node1.query("INSERT INTO t SELECT 6, '{}' FROM numbers(5000)".format(get_random_string(104857))) + pm.add_network_delay(node1, 80) + node2.query("SYSTEM START FETCHES t") + fetches_result = [] + for _ in range(1000): + result = node2.query("SELECT result_part_name FROM system.replicated_fetches") + if not result: + if fetches_result: + break + time.sleep(0.1) + else: + fetches_result.append(result.split('\n')) + print(fetches_result[-1]) + time.sleep(0.1) + + for concurrently_fetching_parts in fetches_result: + if len(concurrently_fetching_parts) > DEFAULT_MAX_THREADS_FOR_FETCH: + assert False, "Found more than {} concurrently fetching parts: {}".format(DEFAULT_MAX_THREADS_FOR_FETCH, ', '.join(concurrently_fetching_parts)) diff --git a/tests/integration/test_limited_replicated_fetches/test.py b/tests/integration/test_limited_replicated_fetches/test.py new file mode 100644 index 00000000000..2091c65857e --- /dev/null +++ b/tests/integration/test_limited_replicated_fetches/test.py @@ -0,0 +1,71 @@ +#!/usr/bin/env python3 + +import pytest +import time +from helpers.cluster import ClickHouseCluster +from helpers.network import PartitionManager +import random +import string + +cluster = ClickHouseCluster(__file__) +node1 = cluster.add_instance('node1', with_zookeeper=True) +node2 = cluster.add_instance('node2', with_zookeeper=True) + +DEFAULT_MAX_THREADS_FOR_FETCH = 3 + +@pytest.fixture(scope="module") +def started_cluster(): + try: + cluster.start() + + yield cluster + + finally: + cluster.shutdown() + + +def get_random_string(length): + return ''.join(random.choice(string.ascii_uppercase + string.digits) for _ in range(length)) + + +def test_limited_fetches(started_cluster): + """ + Test checks that that we utilize all available threads for fetches + """ + node1.query("CREATE TABLE t (key UInt64, data String) ENGINE = ReplicatedMergeTree('/clickhouse/test/t', '1') ORDER BY tuple() PARTITION BY key") + node2.query("CREATE TABLE t (key UInt64, data String) ENGINE = ReplicatedMergeTree('/clickhouse/test/t', '2') ORDER BY tuple() PARTITION BY key") + + with PartitionManager() as pm: + node2.query("SYSTEM STOP FETCHES t") + node1.query("INSERT INTO t SELECT 1, '{}' FROM numbers(5000)".format(get_random_string(104857))) + node1.query("INSERT INTO t SELECT 2, '{}' FROM numbers(5000)".format(get_random_string(104857))) + node1.query("INSERT INTO t SELECT 3, '{}' FROM numbers(5000)".format(get_random_string(104857))) + node1.query("INSERT INTO t SELECT 4, '{}' FROM numbers(5000)".format(get_random_string(104857))) + node1.query("INSERT INTO t SELECT 5, '{}' FROM numbers(5000)".format(get_random_string(104857))) + node1.query("INSERT INTO t SELECT 6, '{}' FROM numbers(5000)".format(get_random_string(104857))) + pm.add_network_delay(node1, 80) + node2.query("SYSTEM START FETCHES t") + fetches_result = [] + background_fetches_metric = [] + fetched_parts = set([]) + for _ in range(1000): + result = node2.query("SELECT result_part_name FROM system.replicated_fetches").strip().split() + background_fetches_metric.append(int(node2.query("select value from system.metrics where metric = 'BackgroundFetchesPoolTask'").strip())) + if not result: + if len(fetched_parts) == 6: + break + time.sleep(0.1) + else: + for part in result: + fetched_parts.add(part) + fetches_result.append(result) + print(fetches_result[-1]) + print(background_fetches_metric[-1]) + time.sleep(0.1) + + for concurrently_fetching_parts in fetches_result: + if len(concurrently_fetching_parts) > DEFAULT_MAX_THREADS_FOR_FETCH: + assert False, "Found more than {} concurrently fetching parts: {}".format(DEFAULT_MAX_THREADS_FOR_FETCH, ', '.join(concurrently_fetching_parts)) + + assert max([len(parts) for parts in fetches_result]) == 3, "Strange, but we don't utilize max concurrent threads for fetches" + assert(max(background_fetches_metric)) == 3, "Just checking metric consistent with table" From e347f4b72fc93c292f8328a737831810a1970d60 Mon Sep 17 00:00:00 2001 From: alesapin Date: Mon, 9 Nov 2020 12:14:20 +0300 Subject: [PATCH 078/205] Add some comments --- src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp | 1 + src/Storages/StorageReplicatedMergeTree.cpp | 1 + src/Storages/StorageReplicatedMergeTree.h | 3 +++ 3 files changed, 5 insertions(+) diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp index a32ccf496d5..543ce813a14 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp @@ -1040,6 +1040,7 @@ bool ReplicatedMergeTreeQueue::shouldExecuteLogEntry( } } + /// Check that fetches pool is not overloaded if (entry.type == LogEntry::GET_PART) { if (!storage.canExecuteFetch(entry, out_postpone_reason)) diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 439b1c5dcfc..42acc35ee30 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -2626,6 +2626,7 @@ std::optional StorageReplicatedMergeTree::getDataProcessingJob() PoolType pool_type; + /// Depending on entry type execute in fetches (small) pool or big merge_mutate pool if (selected_entry->log_entry->type == LogEntry::GET_PART) pool_type = PoolType::FETCH; else diff --git a/src/Storages/StorageReplicatedMergeTree.h b/src/Storages/StorageReplicatedMergeTree.h index d0e328966ac..bd9c05346be 100644 --- a/src/Storages/StorageReplicatedMergeTree.h +++ b/src/Storages/StorageReplicatedMergeTree.h @@ -198,8 +198,11 @@ public: */ static void dropReplica(zkutil::ZooKeeperPtr zookeeper, const String & zookeeper_path, const String & replica, Poco::Logger * logger); + /// Get job to execute in background pool (merge, mutate, drop range and so on) std::optional getDataProcessingJob() override; + /// Checks that fetches are not disabled with action blocker and pool for fetches + /// is not overloaded bool canExecuteFetch(const ReplicatedMergeTreeLogEntry & entry, String & disable_reason) const; private: From a056379ec1028eb1de6da0bb78c6aefb9d912bff Mon Sep 17 00:00:00 2001 From: alesapin Date: Mon, 9 Nov 2020 12:25:50 +0300 Subject: [PATCH 079/205] Remove duplicate code --- .../__init__.py | 58 ------------------- 1 file changed, 58 deletions(-) diff --git a/tests/integration/test_limited_replicated_fetches/__init__.py b/tests/integration/test_limited_replicated_fetches/__init__.py index 900e89d8bcf..e69de29bb2d 100644 --- a/tests/integration/test_limited_replicated_fetches/__init__.py +++ b/tests/integration/test_limited_replicated_fetches/__init__.py @@ -1,58 +0,0 @@ -#!/usr/bin/env python3 - -import pytest -import time -from helpers.cluster import ClickHouseCluster -from helpers.network import PartitionManager -import random -import string - -cluster = ClickHouseCluster(__file__) -node1 = cluster.add_instance('node1', with_zookeeper=True) -node2 = cluster.add_instance('node2', with_zookeeper=True) - -DEFAULT_MAX_THREADS_FOR_FETCH = 3 - -@pytest.fixture(scope="module") -def started_cluster(): - try: - cluster.start() - - yield cluster - - finally: - cluster.shutdown() - - -def get_random_string(length): - return ''.join(random.choice(string.ascii_uppercase + string.digits) for _ in range(length)) - -def test_limited_fetches(started_cluster): - node1.query("CREATE TABLE t (key UInt64, data String) ENGINE = ReplicatedMergeTree('/clickhouse/test/t', '1') ORDER BY tuple() PARTITION BY key") - node2.query("CREATE TABLE t (key UInt64, data String) ENGINE = ReplicatedMergeTree('/clickhouse/test/t', '2') ORDER BY tuple() PARTITION BY key") - - with PartitionManager() as pm: - node2.query("SYSTEM STOP FETCHES t") - node1.query("INSERT INTO t SELECT 1, '{}' FROM numbers(5000)".format(get_random_string(104857))) - node1.query("INSERT INTO t SELECT 2, '{}' FROM numbers(5000)".format(get_random_string(104857))) - node1.query("INSERT INTO t SELECT 3, '{}' FROM numbers(5000)".format(get_random_string(104857))) - node1.query("INSERT INTO t SELECT 4, '{}' FROM numbers(5000)".format(get_random_string(104857))) - node1.query("INSERT INTO t SELECT 5, '{}' FROM numbers(5000)".format(get_random_string(104857))) - node1.query("INSERT INTO t SELECT 6, '{}' FROM numbers(5000)".format(get_random_string(104857))) - pm.add_network_delay(node1, 80) - node2.query("SYSTEM START FETCHES t") - fetches_result = [] - for _ in range(1000): - result = node2.query("SELECT result_part_name FROM system.replicated_fetches") - if not result: - if fetches_result: - break - time.sleep(0.1) - else: - fetches_result.append(result.split('\n')) - print(fetches_result[-1]) - time.sleep(0.1) - - for concurrently_fetching_parts in fetches_result: - if len(concurrently_fetching_parts) > DEFAULT_MAX_THREADS_FOR_FETCH: - assert False, "Found more than {} concurrently fetching parts: {}".format(DEFAULT_MAX_THREADS_FOR_FETCH, ', '.join(concurrently_fetching_parts)) From 8c4db34f9d873b47fae511f00c716c7e3607ba62 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Mon, 9 Nov 2020 14:58:11 +0300 Subject: [PATCH 080/205] Update after merge. --- src/Interpreters/ExpressionActions.h | 1 + src/Storages/MergeTree/KeyCondition.cpp | 121 ++++++++++++++++-------- 2 files changed, 82 insertions(+), 40 deletions(-) diff --git a/src/Interpreters/ExpressionActions.h b/src/Interpreters/ExpressionActions.h index 8fff1dc372d..2a6c8f7c70b 100644 --- a/src/Interpreters/ExpressionActions.h +++ b/src/Interpreters/ExpressionActions.h @@ -312,6 +312,7 @@ public: ExpressionActions & operator=(const ExpressionActions &) = delete; const Actions & getActions() const { return actions; } + const std::list & getNodes() const { return nodes; } /// Adds to the beginning the removal of all extra columns. void projectInput() { project_input = true; } diff --git a/src/Storages/MergeTree/KeyCondition.cpp b/src/Storages/MergeTree/KeyCondition.cpp index e6880250c64..b575da46fcb 100644 --- a/src/Storages/MergeTree/KeyCondition.cpp +++ b/src/Storages/MergeTree/KeyCondition.cpp @@ -19,6 +19,7 @@ #include #include +#include namespace DB @@ -651,12 +652,12 @@ bool KeyCondition::canConstantBeWrappedByMonotonicFunctions( /// Looking for possible transformation of `column = constant` into `partition_expr = function(constant)` bool KeyCondition::canConstantBeWrappedByFunctions( - const ASTPtr & node, size_t & out_key_column_num, DataTypePtr & out_key_column_type, Field & out_value, DataTypePtr & out_type) + const ASTPtr & ast, size_t & out_key_column_num, DataTypePtr & out_key_column_type, Field & out_value, DataTypePtr & out_type) { if (strict) return false; - String expr_name = node->getColumnName(); + String expr_name = ast->getColumnName(); const auto & sample_block = key_expr->getSampleBlock(); if (!sample_block.has(expr_name)) return false; @@ -665,58 +666,98 @@ bool KeyCondition::canConstantBeWrappedByFunctions( if (out_value.isNull()) return false; - bool found_transformation = false; - auto input_column = sample_block.getByName(expr_name); - auto const_column = out_type->createColumnConst(1, out_value); - out_value = (*castColumn({const_column, out_type, "c"}, input_column.type))[0]; - out_type = input_column.type; - Block transform({{input_column.type->createColumn(), input_column.type, input_column.name}}); - for (const auto & action : key_expr->getActions()) + for (const auto & node : key_expr->getNodes()) { - if (action.node->type == ActionsDAG::Type::FUNCTION) + auto it = key_columns.find(node.result_name); + if (it != key_columns.end()) { - if (!action.node->function_base->isDeterministic()) - return false; - if (action.node->children.size() == 1 && action.node->children[0]->result_name == expr_name) + std::stack chain; + + const auto * cur_node = &node; + bool is_valid_chain = true; + + while (is_valid_chain) { - std::tie(out_value, out_type) = applyFunctionForFieldOfUnknownType(action.node->function_builder, out_type, out_value); - } - else if (action.node->children.size() == 2) - { - if (!transform.has(action.node->children[0]->result_name) || !transform.has(action.node->children[1]->result_name)) - return false; - auto left = transform.getByName(action.node->children[0]->result_name); - auto right = transform.getByName(action.node->children[1]->result_name); - if (isColumnConst(*left.column)) + if (cur_node->result_name == expr_name) + break; + + chain.push(cur_node); + + if (cur_node->type == ActionsDAG::Type::FUNCTION && cur_node->children.size() <= 2) { - auto left_arg_type = left.type; - auto left_arg_value = (*left.column)[0]; - std::tie(out_value, out_type) = applyBinaryFunctionForFieldOfUnknownType( - action.node->function_builder, left_arg_type, left_arg_value, out_type, out_value); - } - else if (isColumnConst(*right.column)) - { - auto right_arg_type = right.type; - auto right_arg_value = (*right.column)[0]; - std::tie(out_value, out_type) = applyBinaryFunctionForFieldOfUnknownType( - action.node->function_builder, out_type, out_value, right_arg_type, right_arg_value); + if (!cur_node->function_base->isDeterministic()) + is_valid_chain = false; + + const ActionsDAG::Node * next_node = nullptr; + for (const auto * arg : cur_node->children) + { + if (arg->column && isColumnConst(*arg->column)) + continue; + + if (next_node) + is_valid_chain = false; + + next_node = arg; + } + + cur_node = next_node; } + else if (cur_node->type == ActionsDAG::Type::ALIAS) + cur_node = cur_node->children.front(); + else + is_valid_chain = false; } - expr_name = action.node->result_name; - auto it = key_columns.find(expr_name); - if (key_columns.end() != it) + if (is_valid_chain) { + { + auto input_column = sample_block.getByName(expr_name); + auto const_column = out_type->createColumnConst(1, out_value); + out_value = (*castColumn({const_column, out_type, "c"}, input_column.type))[0]; + out_type = input_column.type; + } + + while (!chain.empty()) + { + const auto * func = chain.top(); + chain.pop(); + + if (func->type != ActionsDAG::Type::FUNCTION) + continue; + + if (func->children.size() == 1) + { + std::tie(out_value, out_type) = applyFunctionForFieldOfUnknownType(func->function_builder, out_type, out_value); + } + else if (func->children.size() == 2) + { + const auto * left = func->children[0]; + const auto * right = func->children[1]; + if (left->column && isColumnConst(*left->column)) + { + auto left_arg_type = left->result_type; + auto left_arg_value = (*left->column)[0]; + std::tie(out_value, out_type) = applyBinaryFunctionForFieldOfUnknownType( + func->function_builder, left_arg_type, left_arg_value, out_type, out_value); + } + else + { + auto right_arg_type = right->result_type; + auto right_arg_value = (*right->column)[0]; + std::tie(out_value, out_type) = applyBinaryFunctionForFieldOfUnknownType( + func->function_builder, out_type, out_value, right_arg_type, right_arg_value); + } + } + } + out_key_column_num = it->second; out_key_column_type = sample_block.getByName(it->first).type; - found_transformation = true; - break; + return true; } } - action.execute(transform, true); } - return found_transformation; + return false; } bool KeyCondition::tryPrepareSetIndex( From 99cc9b1ec0f94c5e0eed80f32adb3106f4bc1df2 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Mon, 9 Nov 2020 16:20:56 +0300 Subject: [PATCH 081/205] Fix build --- src/Storages/MergeTree/KeyCondition.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/MergeTree/KeyCondition.h b/src/Storages/MergeTree/KeyCondition.h index 265bc01be49..a5587124826 100644 --- a/src/Storages/MergeTree/KeyCondition.h +++ b/src/Storages/MergeTree/KeyCondition.h @@ -403,7 +403,7 @@ private: DataTypePtr & out_type); bool canConstantBeWrappedByFunctions( - const ASTPtr & node, size_t & out_key_column_num, DataTypePtr & out_key_column_type, Field & out_value, DataTypePtr & out_type); + const ASTPtr & ast, size_t & out_key_column_num, DataTypePtr & out_key_column_type, Field & out_value, DataTypePtr & out_type); /// If it's possible to make an RPNElement /// that will filter values (possibly tuples) by the content of 'prepared_set', From 6543b32221490438d0424097bd090586dfea3404 Mon Sep 17 00:00:00 2001 From: annvsh Date: Mon, 9 Nov 2020 22:00:18 +0700 Subject: [PATCH 082/205] Fixed --- docs/en/operations/settings/settings.md | 3 --- docs/ru/operations/settings/settings.md | 3 --- 2 files changed, 6 deletions(-) diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index 74120fc5459..196402b3f62 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -349,8 +349,6 @@ When the `input_format_tsv_enum_as_number` setting is disabled, the `INSERT` que ```sql SET input_format_tsv_enum_as_number = 0; INSERT INTO table_with_enum_column_for_tsv_insert FORMAT TSV 102 2; -INSERT INTO table_with_enum_column_for_tsv_insert FORMAT TSV 103 1; -SELECT * FROM table_with_enum_column_for_tsv_insert; ``` throws an exception. @@ -1251,7 +1249,6 @@ When the `input_format_csv_enum_as_number` setting is disabled, the `INSERT` que ```sql SET input_format_csv_enum_as_number = 0; INSERT INTO table_with_enum_column_for_csv_insert FORMAT CSV 102,2; -SELECT * FROM table_with_enum_column_for_csv_insert; ``` throws an exception. diff --git a/docs/ru/operations/settings/settings.md b/docs/ru/operations/settings/settings.md index 2ea65e89579..aa4f181ea7b 100644 --- a/docs/ru/operations/settings/settings.md +++ b/docs/ru/operations/settings/settings.md @@ -333,8 +333,6 @@ SELECT * FROM table_with_enum_column_for_tsv_insert; ```sql SET input_format_tsv_enum_as_number = 0; INSERT INTO table_with_enum_column_for_tsv_insert FORMAT TSV 102 2; -INSERT INTO table_with_enum_column_for_tsv_insert FORMAT TSV 103 1; -SELECT * FROM table_with_enum_column_for_tsv_insert; ``` сгенерирует исключение. @@ -1217,7 +1215,6 @@ SELECT * FROM table_with_enum_column_for_csv_insert; ```sql SET input_format_csv_enum_as_number = 0; INSERT INTO table_with_enum_column_for_csv_insert FORMAT CSV 102,2; -SELECT * FROM table_with_enum_column_for_csv_insert; ``` сгенерирует исключение. From 74f7f661cad3c3a3c2726d8f62225287a565f885 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Mon, 9 Nov 2020 18:01:08 +0300 Subject: [PATCH 083/205] More comments. --- src/Interpreters/ExpressionActions.cpp | 18 ++++++++-------- src/Interpreters/ExpressionActions.h | 30 ++++++++++++-------------- 2 files changed, 23 insertions(+), 25 deletions(-) diff --git a/src/Interpreters/ExpressionActions.cpp b/src/Interpreters/ExpressionActions.cpp index aebf713a4ca..5e389b9b5bd 100644 --- a/src/Interpreters/ExpressionActions.cpp +++ b/src/Interpreters/ExpressionActions.cpp @@ -58,7 +58,7 @@ namespace ErrorCodes static std::ostream & operator << (std::ostream & out, const ExpressionActions::Argument & argument) { - return out << (argument.remove ? "*" : "") << argument.pos; + return out << (argument.needed_later ? ": " : ":: ") << argument.pos; } std::string ExpressionActions::Action::toString() const @@ -97,7 +97,7 @@ std::string ExpressionActions::Action::toString() const } out << " -> " << node->result_name - << " " << (node->result_type ? node->result_type->getName() : "(no type)") << " " << result_position; + << " " << (node->result_type ? node->result_type->getName() : "(no type)") << " : " << result_position; return out.str(); } @@ -213,7 +213,7 @@ void ExpressionActions::executeAction(const Action & action, ExecutionContext & ColumnsWithTypeAndName arguments(action.arguments.size()); for (size_t i = 0; i < arguments.size(); ++i) { - if (action.arguments[i].remove) + if (!action.arguments[i].needed_later) arguments[i] = std::move(columns[action.arguments[i].pos]); else arguments[i] = columns[action.arguments[i].pos]; @@ -233,7 +233,7 @@ void ExpressionActions::executeAction(const Action & action, ExecutionContext & auto array_join_key = columns[array_join_key_pos]; /// Remove array join argument in advance if it is not needed. - if (action.arguments.front().remove) + if (!action.arguments.front().needed_later) columns[array_join_key_pos] = {}; array_join_key.column = array_join_key.column->convertToFullColumnIfConst(); @@ -277,7 +277,7 @@ void ExpressionActions::executeAction(const Action & action, ExecutionContext & columns[action.result_position].column = columns[arg.pos].column; columns[action.result_position].type = columns[arg.pos].type; - if (arg.remove) + if (!arg.needed_later) columns[arg.pos] = {}; } @@ -291,7 +291,7 @@ void ExpressionActions::executeAction(const Action & action, ExecutionContext & auto pos = execution_context.inputs_pos[action.arguments.front().pos]; if (pos < 0) { - if (!action.arguments.front().remove) + if (action.arguments.front().needed_later) throw Exception(ErrorCodes::NOT_FOUND_COLUMN_IN_BLOCK, "Not found column {} in block", action.node->result_name); @@ -1300,9 +1300,9 @@ ExpressionActionsPtr ActionsDAG::linearizeActions() const ExpressionActions::Argument argument; argument.pos = arg.position; - argument.remove = !arg.used_in_result && arg.num_created_parents == arg.parents.size(); + argument.needed_later = arg.used_in_result || arg.num_created_parents != arg.parents.size(); - if (argument.remove) + if (!argument.needed_later) free_positions.push(argument.pos); arguments.emplace_back(argument); @@ -1313,7 +1313,7 @@ ExpressionActionsPtr ActionsDAG::linearizeActions() const /// Argument for input is special. It contains the position from required columns. ExpressionActions::Argument argument; argument.pos = expressions->required_columns.size(); - argument.remove = cur.parents.empty(); + argument.needed_later = !cur.parents.empty(); arguments.emplace_back(argument); expressions->required_columns.push_back({node->result_name, node->result_type}); diff --git a/src/Interpreters/ExpressionActions.h b/src/Interpreters/ExpressionActions.h index 2a6c8f7c70b..51a3fc9e384 100644 --- a/src/Interpreters/ExpressionActions.h +++ b/src/Interpreters/ExpressionActions.h @@ -102,8 +102,14 @@ public: bool allow_constant_folding = true; }; + /// Index is a list of nodes + hash table: name -> list_elem. + /// List is ordered, may contain nodes with same names, or one node several times. class Index { + private: + std::list list; + std::unordered_map::iterator> map; + public: Node *& operator[](std::string_view key) { @@ -149,10 +155,6 @@ public: list.erase(it->second); map.erase(it); } - - private: - std::list list; - std::unordered_map::iterator> map; }; using Nodes = std::list; @@ -222,8 +224,8 @@ public: ActionsDAGPtr splitActionsBeforeArrayJoin(const NameSet & array_joined_columns); bool hasArrayJoin() const; - bool empty() const; - bool projectedOutput() const { return projected_output; } + bool empty() const; /// If actions only contain inputs. + bool projectedOutput() const { return projected_output; } /// Remove all columns which are not in inputs from block. ActionsDAGPtr clone() const; @@ -264,8 +266,11 @@ public: struct Argument { + /// Position in ExecutionContext::columns size_t pos; - bool remove; + /// True if there is another action which will use this column. + /// Otherwise column will be removed. + bool needed_later; }; using Arguments = std::vector; @@ -317,21 +322,14 @@ public: /// Adds to the beginning the removal of all extra columns. void projectInput() { project_input = true; } - /// - Adds actions to delete all but the specified columns. - /// - Removes unused input columns. - /// - Can somehow optimize the expression. - /// - Does not reorder the columns. - /// - Does not remove "unexpected" columns (for example, added by functions). - /// - If output_columns is empty, leaves one arbitrary column (so that the number of rows in the block is not lost). - // void finalize(const Names & output_columns); - /// Get a list of input columns. Names getRequiredColumns() const; const NamesAndTypesList & getRequiredColumnsWithTypes() const { return required_columns; } /// Execute the expression on the block. The block must contain all the columns returned by getRequiredColumns. - void execute(Block & block, bool dry_run = false) const; void execute(Block & block, size_t & num_rows, bool dry_run = false) const; + /// The same, but without `num_rows`. If result block is empty, adds `_dummy` column to keep block size. + void execute(Block & block, bool dry_run = false) const; bool hasArrayJoin() const; From 030419db909c7e6f013b6775dd497b09b24c9b99 Mon Sep 17 00:00:00 2001 From: Alexander Kuzmenkov Date: Mon, 9 Nov 2020 19:14:05 +0300 Subject: [PATCH 084/205] Try 12 threads max in perf test. This will allow us to use more machines, not only Gold 6320. --- .../users.d/perf-comparison-tweaks-users.xml | 2 +- tests/performance/constant_column_search.xml | 12 ++-- tests/performance/encrypt_decrypt_slow.xml | 25 ++++++-- tests/performance/generate_table_function.xml | 6 +- tests/performance/parse_engine_file.xml | 57 ++++++++++--------- tests/performance/rand.xml | 6 +- tests/performance/string_sort.xml | 2 +- .../synthetic_hardware_benchmark.xml | 2 +- tests/performance/trim_urls.xml | 4 +- 9 files changed, 64 insertions(+), 52 deletions(-) diff --git a/docker/test/performance-comparison/config/users.d/perf-comparison-tweaks-users.xml b/docker/test/performance-comparison/config/users.d/perf-comparison-tweaks-users.xml index ce7a6ae094a..f3609bcfcdb 100644 --- a/docker/test/performance-comparison/config/users.d/perf-comparison-tweaks-users.xml +++ b/docker/test/performance-comparison/config/users.d/perf-comparison-tweaks-users.xml @@ -16,7 +16,7 @@ 300 - 20 + 12 diff --git a/tests/performance/constant_column_search.xml b/tests/performance/constant_column_search.xml index 2f90844cb4a..6914aef705e 100644 --- a/tests/performance/constant_column_search.xml +++ b/tests/performance/constant_column_search.xml @@ -20,7 +20,7 @@ - + @@ -30,23 +30,23 @@ - + - + - - + + - + diff --git a/tests/performance/encrypt_decrypt_slow.xml b/tests/performance/encrypt_decrypt_slow.xml index 57712da9086..8c94dba7d90 100644 --- a/tests/performance/encrypt_decrypt_slow.xml +++ b/tests/performance/encrypt_decrypt_slow.xml @@ -13,24 +13,37 @@ + - table + table_fast numbers(50000) + - plaintext + table_slow + + numbers(25000) + + + + + plaintext_fast number 'paintext' + + + + + plaintext_slow + '\x12\x2B\xF9\x16\x93\xA4\xD6\x74\x22\xD9\x17\x5E\x38\xCD\x1D\x7B\xB0\x12\xEC\x43\x6B\xC7\x76\xFD\xA1\xA2\x4E\xFC\xBC\x19\x92\x3A\x12\x8B\xD4\xB3\x62\xA8\x9D\xBB\x3E\x0C\x08\x12\x67\x20\x7D\x02\x58\xCF\xE7\xD6\x06\xB8\xB0\x14\x0A\x70\xA1\x81\x94\x14\x24\x74' - - WITH {plaintext} as plaintext, repeat('k', 32) as key32, substring(key32, 1, 24) as key24, substring(key32, 1, 16) as key16, repeat('iv', 8) as iv16, substring(iv16, 1, 12) as iv12 SELECT count() FROM {table} WHERE NOT ignore({func}) LIMIT 1 - - WITH {plaintext} as plaintext, repeat('k', 32) as key32, substring(key32, 1, 24) as key24, substring(key32, 1, 16) as key16, repeat('iv', 8) as iv16, substring(iv16, 1, 12) as iv12 SELECT count() FROM {table} WHERE NOT ignore({func}) + WITH {plaintext_fast} as plaintext, repeat('k', 32) as key32, substring(key32, 1, 24) as key24, substring(key32, 1, 16) as key16, repeat('iv', 8) as iv16, substring(iv16, 1, 12) as iv12 SELECT count() FROM {table_fast} WHERE NOT ignore({func}) + WITH {plaintext_slow} as plaintext, repeat('k', 32) as key32, substring(key32, 1, 24) as key24, substring(key32, 1, 16) as key16, repeat('iv', 8) as iv16, substring(iv16, 1, 12) as iv12 SELECT count() FROM {table_slow} WHERE NOT ignore({func}) diff --git a/tests/performance/generate_table_function.xml b/tests/performance/generate_table_function.xml index ebc674f54b2..0628e120966 100644 --- a/tests/performance/generate_table_function.xml +++ b/tests/performance/generate_table_function.xml @@ -5,17 +5,17 @@ SELECT sum(NOT ignore(*)) FROM (SELECT * FROM generateRandom('i Array(Nullable(Enum8(\'hello\' = 1, \'world\' = 5)))', 0, 10, 10) LIMIT 100000000); SELECT sum(NOT ignore(*)) FROM (SELECT * FROM generateRandom('i Nullable(Enum16(\'h\' = 1, \'w\' = 5 , \'o\' = -200))', 0, 10, 10) LIMIT 1000000000); SELECT sum(NOT ignore(*)) FROM (SELECT * FROM generateRandom('d Date, dt DateTime, dtm DateTime(\'Europe/Moscow\')', 0, 10, 10) LIMIT 1000000000); - SELECT sum(NOT ignore(*)) FROM (SELECT * FROM generateRandom('dt64 DateTime64, dts64 DateTime64(6), dtms64 DateTime64(6 ,\'Europe/Moscow\')', 0, 10, 10) LIMIT 1000000000); + SELECT sum(NOT ignore(*)) FROM (SELECT * FROM generateRandom('dt64 DateTime64, dts64 DateTime64(6), dtms64 DateTime64(6 ,\'Europe/Moscow\')', 0, 10, 10) LIMIT 100000000); SELECT sum(NOT ignore(*)) FROM (SELECT * FROM generateRandom('f32 Float32, f64 Float64', 0, 10, 10) LIMIT 1000000000); SELECT sum(NOT ignore(*)) FROM (SELECT * FROM generateRandom('d32 Decimal32(4), d64 Decimal64(8), d128 Decimal128(16)', 0, 10, 10) LIMIT 1000000000); SELECT sum(NOT ignore(*)) FROM (SELECT * FROM generateRandom('i Tuple(Int32, Int64)', 0, 10, 10) LIMIT 1000000000); SELECT sum(NOT ignore(*)) FROM (SELECT * FROM generateRandom('i Array(Int8)', 0, 10, 10) LIMIT 100000000); SELECT sum(NOT ignore(*)) FROM (SELECT * FROM generateRandom('i Array(Nullable(Int32))', 0, 10, 10) LIMIT 100000000); SELECT sum(NOT ignore(*)) FROM (SELECT * FROM generateRandom('i Tuple(Int32, Array(Int64))', 0, 10, 10) LIMIT 1000000000); - SELECT sum(NOT ignore(*)) FROM (SELECT * FROM generateRandom('i Nullable(String)', 0, 10, 10) LIMIT 1000000000); + SELECT sum(NOT ignore(*)) FROM (SELECT * FROM generateRandom('i Nullable(String)', 0, 10, 10) LIMIT 100000000); SELECT sum(NOT ignore(*)) FROM (SELECT * FROM generateRandom('i Array(String)', 0, 10, 10) LIMIT 100000000); SELECT sum(NOT ignore(*)) FROM (SELECT * FROM generateRandom('i UUID', 0, 10, 10) LIMIT 1000000000); SELECT sum(NOT ignore(*)) FROM (SELECT * FROM generateRandom('i Array(Nullable(UUID))', 0, 10, 10) LIMIT 100000000); SELECT sum(NOT ignore(*)) FROM (SELECT * FROM generateRandom('i FixedString(4)', 0, 10, 10) LIMIT 1000000000); - SELECT sum(NOT ignore(*)) FROM (SELECT * FROM generateRandom('i String', 0, 10, 10) LIMIT 1000000000); + SELECT sum(NOT ignore(*)) FROM (SELECT * FROM generateRandom('i String', 0, 10, 10) LIMIT 100000000); diff --git a/tests/performance/parse_engine_file.xml b/tests/performance/parse_engine_file.xml index f876f314a90..2459ed084cd 100644 --- a/tests/performance/parse_engine_file.xml +++ b/tests/performance/parse_engine_file.xml @@ -1,36 +1,37 @@ + + + test.hits + + + + + format + + TabSeparated + TabSeparatedWithNames + TabSeparatedWithNamesAndTypes + CSV + CSVWithNames + Values + JSONEachRow + JSONCompactEachRow + JSONCompactEachRowWithNamesAndTypes + TSKV + RowBinary + Native + Avro + MsgPack + + + - CREATE TABLE IF NOT EXISTS table_{format} ENGINE = File({format}) AS test.hits +CREATE TABLE IF NOT EXISTS table_{format} ENGINE = File({format}) AS test.hits - INSERT INTO table_{format} SELECT * FROM test.hits LIMIT 200000 - - - - format - - TabSeparated - TabSeparatedWithNames - TabSeparatedWithNamesAndTypes - CSV - CSVWithNames - Values - JSONEachRow - JSONCompactEachRow - JSONCompactEachRowWithNamesAndTypes - TSKV - RowBinary - Native - Avro - MsgPack - - - - - - test.hits - +INSERT INTO table_{format} SELECT * FROM test.hits LIMIT 100000 SELECT * FROM table_{format} FORMAT Null DROP TABLE IF EXISTS table_{format} + diff --git a/tests/performance/rand.xml b/tests/performance/rand.xml index 3f46d152206..32c97bb77d6 100644 --- a/tests/performance/rand.xml +++ b/tests/performance/rand.xml @@ -8,7 +8,7 @@ - SELECT count() FROM (SELECT rand() FROM {table}) - SELECT count() FROM (SELECT rand64() FROM {table}) - SELECT count() FROM (SELECT generateUUIDv4() FROM {table}) + SELECT count() FROM (SELECT rand() FROM zeros(1000000000)) + SELECT count() FROM (SELECT rand64() FROM zeros(1000000000)) + SELECT count() FROM (SELECT generateUUIDv4() FROM zeros( 100000000)) diff --git a/tests/performance/string_sort.xml b/tests/performance/string_sort.xml index 5d859398ece..82a91892af4 100644 --- a/tests/performance/string_sort.xml +++ b/tests/performance/string_sort.xml @@ -48,5 +48,5 @@ - + diff --git a/tests/performance/synthetic_hardware_benchmark.xml b/tests/performance/synthetic_hardware_benchmark.xml index 5de363b6e20..d4d93447265 100644 --- a/tests/performance/synthetic_hardware_benchmark.xml +++ b/tests/performance/synthetic_hardware_benchmark.xml @@ -77,7 +77,7 @@ -SELECT number % toUInt32(1e8) AS k, count() FROM numbers_mt(toUInt32(1e8)) GROUP BY k FORMAT Null +SELECT number % toUInt32(0.5e8) AS k, count() FROM numbers_mt(toUInt32(0.5e8)) GROUP BY k FORMAT Null diff --git a/tests/performance/trim_urls.xml b/tests/performance/trim_urls.xml index 075984682f9..5ca565da7e8 100644 --- a/tests/performance/trim_urls.xml +++ b/tests/performance/trim_urls.xml @@ -4,8 +4,6 @@ hits_100m_single - - func @@ -20,5 +18,5 @@ - SELECT ignore({func}URL)) FROM hits_100m_single LIMIT 50000000 FORMAT Null + SELECT ignore({func}URL)) FROM hits_100m_single LIMIT 220000000 FORMAT Null From e41b1ae52b9bc60670188bcae0bc823801ad994d Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Mon, 9 Nov 2020 19:35:43 +0300 Subject: [PATCH 085/205] Empty commit. --- src/Storages/MergeTree/KeyCondition.cpp | 3 +++ 1 file changed, 3 insertions(+) diff --git a/src/Storages/MergeTree/KeyCondition.cpp b/src/Storages/MergeTree/KeyCondition.cpp index b575da46fcb..6771d99c100 100644 --- a/src/Storages/MergeTree/KeyCondition.cpp +++ b/src/Storages/MergeTree/KeyCondition.cpp @@ -700,6 +700,9 @@ bool KeyCondition::canConstantBeWrappedByFunctions( next_node = arg; } + if (!next_node) + is_valid_chain = false; + cur_node = next_node; } else if (cur_node->type == ActionsDAG::Type::ALIAS) From f999ea2a6f123a962e2fa2e7edd38bb6b0f6e1c9 Mon Sep 17 00:00:00 2001 From: "a.palagashvili" Date: Tue, 10 Nov 2020 01:52:22 +0300 Subject: [PATCH 086/205] renamed files, added new library, changed error codes, added tests for file() function --- .gitmodules | 4 ++ CHANGELOG.md | 1 - contrib/fast-lzma2 | 1 + src/CMakeLists.txt | 7 +++ src/Common/ErrorCodes.cpp | 6 +-- src/IO/CompressionMethod.cpp | 50 +++++++++---------- ...uffer.cpp => LZMADeflatingWriteBuffer.cpp} | 23 ++++++--- ...iteBuffer.h => LZMADeflatingWriteBuffer.h} | 9 ++-- ...Buffer.cpp => LZMAInflatingReadBuffer.cpp} | 18 +++++-- ...ReadBuffer.h => LZMAInflatingReadBuffer.h} | 11 ++-- src/IO/tests/lzma_buffers.cpp | 26 +++++----- .../01059_storage_file_brotli.reference | 4 +- .../0_stateless/01059_storage_file_brotli.sql | 11 +++- 13 files changed, 105 insertions(+), 66 deletions(-) create mode 160000 contrib/fast-lzma2 rename src/IO/{LzmaWriteBuffer.cpp => LZMADeflatingWriteBuffer.cpp} (85%) rename src/IO/{LzmaWriteBuffer.h => LZMADeflatingWriteBuffer.h} (73%) rename src/IO/{LzmaReadBuffer.cpp => LZMAInflatingReadBuffer.cpp} (81%) rename src/IO/{LzmaReadBuffer.h => LZMAInflatingReadBuffer.h} (68%) diff --git a/.gitmodules b/.gitmodules index ecaf806067c..e489444d46c 100644 --- a/.gitmodules +++ b/.gitmodules @@ -17,6 +17,7 @@ [submodule "contrib/zlib-ng"] path = contrib/zlib-ng url = https://github.com/ClickHouse-Extras/zlib-ng.git + branch = clickhouse [submodule "contrib/googletest"] path = contrib/googletest url = https://github.com/google/googletest.git @@ -193,3 +194,6 @@ [submodule "contrib/xz"] path = contrib/xz url = https://github.com/xz-mirror/xz +[submodule "contrib/fast-lzma2"] + path = contrib/fast-lzma2 + url = https://github.com/conor42/fast-lzma2 diff --git a/CHANGELOG.md b/CHANGELOG.md index 1242bd2e7b4..457346aff9a 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -219,7 +219,6 @@ * Added column transformers `EXCEPT`, `REPLACE`, `APPLY`, which can be applied to the list of selected columns (after `*` or `COLUMNS(...)`). For example, you can write `SELECT * EXCEPT(URL) REPLACE(number + 1 AS number)`. Another example: `select * apply(length) apply(max) from wide_string_table` to find out the maxium length of all string columns. [#14233](https://github.com/ClickHouse/ClickHouse/pull/14233) ([Amos Bird](https://github.com/amosbird)). * Added an aggregate function `rankCorr` which computes a rank correlation coefficient. [#11769](https://github.com/ClickHouse/ClickHouse/pull/11769) ([antikvist](https://github.com/antikvist)) [#14411](https://github.com/ClickHouse/ClickHouse/pull/14411) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). * Added table function `view` which turns a subquery into a table object. This helps passing queries around. For instance, it can be used in remote/cluster table functions. [#12567](https://github.com/ClickHouse/ClickHouse/pull/12567) ([Amos Bird](https://github.com/amosbird)). -* Added support for `xz` compression format. This enables using `*.xz` files in `table()` function. [#16578](https://github.com/ClickHouse/ClickHouse/pull/16578) ([Abi Palagashvili](https://github.com/fibersel)) #### Bug Fix diff --git a/contrib/fast-lzma2 b/contrib/fast-lzma2 new file mode 160000 index 00000000000..ded964d203c --- /dev/null +++ b/contrib/fast-lzma2 @@ -0,0 +1 @@ +Subproject commit ded964d203cabe1a572d2c813c55e8a94b4eda48 diff --git a/src/CMakeLists.txt b/src/CMakeLists.txt index 7a8fe586e11..6839b06677d 100644 --- a/src/CMakeLists.txt +++ b/src/CMakeLists.txt @@ -336,6 +336,13 @@ if (LZMA_LIBRARY) target_include_directories (clickhouse_common_io SYSTEM BEFORE PUBLIC ${LZMA_INCLUDE_DIR}) endif() +set (FAST_LZMA_LIBRARY fast-lzma2) +set (FAST_LZMA_INCLUDE_DIR ${ClickHouse_SOURCE_DIR}/contrib/fast-lzma2/) +if (FAST_LZMA_LIBRARY) + target_link_libraries (clickhouse_common_io PUBLIC ${FAST_LZMA_LIBRARY}) + target_include_directories (clickhouse_common_io SYSTEM BEFORE PUBLIC ${FAST_LZMA_INCLUDE_DIR}) +endif() + if (USE_ICU) dbms_target_link_libraries (PRIVATE ${ICU_LIBRARIES}) dbms_target_include_directories (SYSTEM PRIVATE ${ICU_INCLUDE_DIRS}) diff --git a/src/Common/ErrorCodes.cpp b/src/Common/ErrorCodes.cpp index b12623c73ff..82084976a82 100644 --- a/src/Common/ErrorCodes.cpp +++ b/src/Common/ErrorCodes.cpp @@ -519,13 +519,13 @@ M(550, CONDITIONAL_TREE_PARENT_NOT_FOUND) \ M(551, ILLEGAL_PROJECTION_MANIPULATOR) \ M(552, UNRECOGNIZED_ARGUMENTS) \ + M(553, LZMA_STREAM_ENCODER_FAILED) \ + M(554, LZMA_STREAM_DECODER_FAILED) \ \ M(999, KEEPER_EXCEPTION) \ M(1000, POCO_EXCEPTION) \ M(1001, STD_EXCEPTION) \ - M(1002, UNKNOWN_EXCEPTION) \ - M(1003, LZMA_STREAM_ENCODER_FAILED) \ - M(1004, LZMA_STREAM_DECODER_FAILED) + M(1002, UNKNOWN_EXCEPTION) /* See END */ diff --git a/src/IO/CompressionMethod.cpp b/src/IO/CompressionMethod.cpp index 0bf390d92ca..ec278b5d71f 100644 --- a/src/IO/CompressionMethod.cpp +++ b/src/IO/CompressionMethod.cpp @@ -1,13 +1,13 @@ #include -#include -#include -#include -#include #include #include -#include -#include +#include +#include +#include +#include +#include +#include #if !defined(ARCADIA_BUILD) # include @@ -16,7 +16,6 @@ namespace DB { - namespace ErrorCodes { extern const int NOT_IMPLEMENTED; @@ -27,11 +26,16 @@ std::string toContentEncodingName(CompressionMethod method) { switch (method) { - case CompressionMethod::Gzip: return "gzip"; - case CompressionMethod::Zlib: return "deflate"; - case CompressionMethod::Brotli: return "br"; - case CompressionMethod::Xz: return "xz"; - case CompressionMethod::None: return ""; + case CompressionMethod::Gzip: + return "gzip"; + case CompressionMethod::Zlib: + return "deflate"; + case CompressionMethod::Brotli: + return "br"; + case CompressionMethod::Xz: + return "xz"; + case CompressionMethod::None: + return ""; } __builtin_unreachable(); } @@ -55,20 +59,19 @@ CompressionMethod chooseCompressionMethod(const std::string & path, const std::s return CompressionMethod::Zlib; if (*method_str == "brotli" || *method_str == "br") return CompressionMethod::Brotli; + if (*method_str == "LZMA" || *method_str == "xz") + return CompressionMethod::Xz; if (hint.empty() || hint == "auto" || hint == "none") return CompressionMethod::None; - throw Exception("Unknown compression method " + hint + ". Only 'auto', 'none', 'gzip', 'br' are supported as compression methods", + throw Exception( + "Unknown compression method " + hint + ". Only 'auto', 'none', 'gzip', 'br', 'xz' are supported as compression methods", ErrorCodes::NOT_IMPLEMENTED); } std::unique_ptr wrapReadBufferWithCompressionMethod( - std::unique_ptr nested, - CompressionMethod method, - size_t buf_size, - char * existing_memory, - size_t alignment) + std::unique_ptr nested, CompressionMethod method, size_t buf_size, char * existing_memory, size_t alignment) { if (method == CompressionMethod::Gzip || method == CompressionMethod::Zlib) return std::make_unique(std::move(nested), method, buf_size, existing_memory, alignment); @@ -77,7 +80,7 @@ std::unique_ptr wrapReadBufferWithCompressionMethod( return std::make_unique(std::move(nested), buf_size, existing_memory, alignment); #endif if (method == CompressionMethod::Xz) - return std::make_unique(std::move(nested), buf_size, existing_memory, alignment); + return std::make_unique(std::move(nested), buf_size, existing_memory, alignment); if (method == CompressionMethod::None) return nested; @@ -87,12 +90,7 @@ std::unique_ptr wrapReadBufferWithCompressionMethod( std::unique_ptr wrapWriteBufferWithCompressionMethod( - std::unique_ptr nested, - CompressionMethod method, - int level, - size_t buf_size, - char * existing_memory, - size_t alignment) + std::unique_ptr nested, CompressionMethod method, int level, size_t buf_size, char * existing_memory, size_t alignment) { if (method == DB::CompressionMethod::Gzip || method == CompressionMethod::Zlib) return std::make_unique(std::move(nested), method, level, buf_size, existing_memory, alignment); @@ -101,6 +99,8 @@ std::unique_ptr wrapWriteBufferWithCompressionMethod( if (method == DB::CompressionMethod::Brotli) return std::make_unique(std::move(nested), level, buf_size, existing_memory, alignment); #endif + if (method == CompressionMethod::Xz) + return std::make_unique(std::move(nested), level, buf_size, existing_memory, alignment); if (method == CompressionMethod::None) return nested; diff --git a/src/IO/LzmaWriteBuffer.cpp b/src/IO/LZMADeflatingWriteBuffer.cpp similarity index 85% rename from src/IO/LzmaWriteBuffer.cpp rename to src/IO/LZMADeflatingWriteBuffer.cpp index 8439fc624d4..66cd11c13d6 100644 --- a/src/IO/LzmaWriteBuffer.cpp +++ b/src/IO/LZMADeflatingWriteBuffer.cpp @@ -1,4 +1,4 @@ -#include +#include namespace DB @@ -8,11 +8,15 @@ namespace ErrorCodes extern const int LZMA_STREAM_ENCODER_FAILED; } -LzmaWriteBuffer::LzmaWriteBuffer( +LZMADeflatingWriteBuffer::LZMADeflatingWriteBuffer( std::unique_ptr out_, int compression_level, size_t buf_size, char * existing_memory, size_t alignment) : BufferWithOwnMemory(buf_size, existing_memory, alignment), out(std::move(out_)) { - lstr = LZMA_STREAM_INIT; + // FL2_createCStreamMt(number of threads, flag of two dictionaries usage) + lstr = FL2_createCStreamMt(2, 0); + /* size_t res = */ FL2_initCStream(lstr, compression_level); + + /*lstr = LZMA_STREAM_INIT; lstr.allocator = nullptr; lstr.next_in = nullptr; lstr.avail_in = 0; @@ -37,15 +41,16 @@ LzmaWriteBuffer::LzmaWriteBuffer( throw Exception( std::string("lzma stream encoder init failed: ") + std::to_string(ret) + "; lzma version: " + LZMA_VERSION_STRING, ErrorCodes::LZMA_STREAM_ENCODER_FAILED); + */ } -LzmaWriteBuffer::~LzmaWriteBuffer() +LZMADeflatingWriteBuffer::~LZMADeflatingWriteBuffer() { try { finish(); - lzma_end(&lstr); + //lzma_end(&lstr); } catch (...) { @@ -53,8 +58,9 @@ LzmaWriteBuffer::~LzmaWriteBuffer() } } -void LzmaWriteBuffer::nextImpl() +void LZMADeflatingWriteBuffer::nextImpl() { + /* if (!offset()) return; @@ -82,11 +88,13 @@ void LzmaWriteBuffer::nextImpl() ErrorCodes::LZMA_STREAM_ENCODER_FAILED); } while (lstr.avail_in > 0 || lstr.avail_out == 0); + */ } -void LzmaWriteBuffer::finish() +void LZMADeflatingWriteBuffer::finish() { + /* if (finished) return; @@ -114,5 +122,6 @@ void LzmaWriteBuffer::finish() ErrorCodes::LZMA_STREAM_ENCODER_FAILED); } while (lstr.avail_out == 0); + */ } } diff --git a/src/IO/LzmaWriteBuffer.h b/src/IO/LZMADeflatingWriteBuffer.h similarity index 73% rename from src/IO/LzmaWriteBuffer.h rename to src/IO/LZMADeflatingWriteBuffer.h index d59595dab23..aadf15ec6dd 100644 --- a/src/IO/LzmaWriteBuffer.h +++ b/src/IO/LZMADeflatingWriteBuffer.h @@ -4,14 +4,15 @@ #include #include +#include namespace DB { /// Performs compression using lzma library and writes compressed data to out_ WriteBuffer. -class LzmaWriteBuffer : public BufferWithOwnMemory +class LZMADeflatingWriteBuffer : public BufferWithOwnMemory { public: - LzmaWriteBuffer( + LZMADeflatingWriteBuffer( std::unique_ptr out_, int compression_level, size_t buf_size = DBMS_DEFAULT_BUFFER_SIZE, @@ -20,13 +21,13 @@ public: void finish(); - ~LzmaWriteBuffer() override; + ~LZMADeflatingWriteBuffer() override; private: void nextImpl() override; std::unique_ptr out; - lzma_stream lstr; + FL2_CStream * lstr; bool finished = false; }; } diff --git a/src/IO/LzmaReadBuffer.cpp b/src/IO/LZMAInflatingReadBuffer.cpp similarity index 81% rename from src/IO/LzmaReadBuffer.cpp rename to src/IO/LZMAInflatingReadBuffer.cpp index 22fda48b3c6..09aa31b7f43 100644 --- a/src/IO/LzmaReadBuffer.cpp +++ b/src/IO/LZMAInflatingReadBuffer.cpp @@ -1,4 +1,4 @@ -#include +#include namespace DB { @@ -6,9 +6,13 @@ namespace ErrorCodes { extern const int LZMA_STREAM_DECODER_FAILED; } -LzmaReadBuffer::LzmaReadBuffer(std::unique_ptr in_, size_t buf_size, char * existing_memory, size_t alignment) +LZMAInflatingReadBuffer::LZMAInflatingReadBuffer(std::unique_ptr in_, size_t buf_size, char * existing_memory, size_t alignment) : BufferWithOwnMemory(buf_size, existing_memory, alignment), in(std::move(in_)), eof(false) { + // FL2_createDStreamMt(number of threads) + lstr = FL2_createDStreamMt(2); + /* size_t res = */ FL2_initDStream(lstr); + /* lstr = LZMA_STREAM_INIT; lstr.allocator = nullptr; lstr.next_in = nullptr; @@ -26,15 +30,17 @@ LzmaReadBuffer::LzmaReadBuffer(std::unique_ptr in_, size_t buf_size, std::string("lzma_stream_decoder initialization failed: error code: ") + std::to_string(ret) + "; lzma version: " + LZMA_VERSION_STRING, ErrorCodes::LZMA_STREAM_DECODER_FAILED); + */ } -LzmaReadBuffer::~LzmaReadBuffer() +LZMAInflatingReadBuffer::~LZMAInflatingReadBuffer() { - lzma_end(&lstr); + //lzma_end(&lstr); } -bool LzmaReadBuffer::nextImpl() +bool LZMAInflatingReadBuffer::nextImpl() { + /* if (eof) return false; @@ -77,5 +83,7 @@ bool LzmaReadBuffer::nextImpl() LZMA_VERSION_STRING); return true; + */ + return true; } } diff --git a/src/IO/LzmaReadBuffer.h b/src/IO/LZMAInflatingReadBuffer.h similarity index 68% rename from src/IO/LzmaReadBuffer.h rename to src/IO/LZMAInflatingReadBuffer.h index 5f936475ee1..4a9893e5b4c 100644 --- a/src/IO/LzmaReadBuffer.h +++ b/src/IO/LZMAInflatingReadBuffer.h @@ -1,10 +1,10 @@ #pragma once #include -#include #include #include +#include namespace DB { @@ -12,22 +12,23 @@ namespace ErrorCodes { } -class LzmaReadBuffer : public BufferWithOwnMemory +class LZMAInflatingReadBuffer : public BufferWithOwnMemory { public: - LzmaReadBuffer( + LZMAInflatingReadBuffer( std::unique_ptr in_, size_t buf_size = DBMS_DEFAULT_BUFFER_SIZE, char * existing_memory = nullptr, size_t alignment = 0); - ~LzmaReadBuffer() override; + ~LZMAInflatingReadBuffer() override; private: bool nextImpl() override; std::unique_ptr in; - lzma_stream lstr; + FL2_DStream * lstr; + bool eof; }; } diff --git a/src/IO/tests/lzma_buffers.cpp b/src/IO/tests/lzma_buffers.cpp index 1f691fa09f7..7eb6bf8b81c 100644 --- a/src/IO/tests/lzma_buffers.cpp +++ b/src/IO/tests/lzma_buffers.cpp @@ -1,13 +1,13 @@ -#include #include +#include -#include -#include +#include +#include #include -#include -#include -#include #include +#include +#include +#include int main(int, char **) try @@ -18,8 +18,9 @@ try Stopwatch stopwatch; { - auto buf = std::make_unique("test_lzma_buffers.xz", DBMS_DEFAULT_BUFFER_SIZE, O_WRONLY | O_CREAT | O_TRUNC); - DB::LzmaWriteBuffer lzma_buf(std::move(buf), /*compression level*/ 3); + auto buf + = std::make_unique("test_lzma_buffers.xz", DBMS_DEFAULT_BUFFER_SIZE, O_WRONLY | O_CREAT | O_TRUNC); + DB::LZMADeflatingWriteBuffer lzma_buf(std::move(buf), /*compression level*/ 3); stopwatch.restart(); for (size_t i = 0; i < n; ++i) @@ -32,13 +33,12 @@ try stopwatch.stop(); std::cout << "Writing done. Elapsed: " << stopwatch.elapsedSeconds() << " s." - << ", " << (lzma_buf.count() / stopwatch.elapsedSeconds() / 1000000) << " MB/s" - << std::endl; + << ", " << (lzma_buf.count() / stopwatch.elapsedSeconds() / 1000000) << " MB/s" << std::endl; } { auto buf = std::make_unique("test_lzma_buffers.xz"); - DB::LzmaReadBuffer lzma_buf(std::move(buf)); + DB::LZMAInflatingReadBuffer lzma_buf(std::move(buf)); stopwatch.restart(); for (size_t i = 0; i < n; ++i) @@ -52,9 +52,7 @@ try } stopwatch.stop(); std::cout << "Reading done. Elapsed: " << stopwatch.elapsedSeconds() << " s." - << ", " << (lzma_buf.count() / stopwatch.elapsedSeconds() / 1000000) << " MB/s" - << std::endl; - + << ", " << (lzma_buf.count() / stopwatch.elapsedSeconds() / 1000000) << " MB/s" << std::endl; } return 0; diff --git a/tests/queries/0_stateless/01059_storage_file_brotli.reference b/tests/queries/0_stateless/01059_storage_file_brotli.reference index 6c545e9faec..aae55b2873c 100644 --- a/tests/queries/0_stateless/01059_storage_file_brotli.reference +++ b/tests/queries/0_stateless/01059_storage_file_brotli.reference @@ -1,5 +1,7 @@ 1000000 999999 1000000 999999 -2000000 999999 +1000000 999999 +3000000 999999 1 255 1 255 +1 255 \ No newline at end of file diff --git a/tests/queries/0_stateless/01059_storage_file_brotli.sql b/tests/queries/0_stateless/01059_storage_file_brotli.sql index e7d5a87b2af..eba61e4450f 100644 --- a/tests/queries/0_stateless/01059_storage_file_brotli.sql +++ b/tests/queries/0_stateless/01059_storage_file_brotli.sql @@ -15,8 +15,17 @@ SELECT count(), max(x) FROM file; DROP TABLE file; -SELECT count(), max(x) FROM file('data{1,2}.tsv.{gz,br}', TSV, 'x UInt64'); +CREATE TABLE file (x UInt64) ENGINE = File(TSV, 'data3.tsv.xz'); +TRUNCATE TABLE file; + +INSERT INTO file SELECT * FROM numbers(1000000); +SELECT count(), max(x) FROM file; + +DROP TABLE file; + +SELECT count(), max(x) FROM file('data{1,2,3}.tsv.{gz,br,xz}', TSV, 'x UInt64'); -- check that they are compressed SELECT count() < 1000000, max(x) FROM file('data1.tsv.br', RowBinary, 'x UInt8', 'none'); SELECT count() < 3000000, max(x) FROM file('data2.tsv.gz', RowBinary, 'x UInt8', 'none'); +SELECT count() < 1000000, max(x) FROM file('data3.tsv.xz', RowBinary, 'x UInt8', 'none'); From 84fb76bad40a1d68438da71364a1c8f9b53060aa Mon Sep 17 00:00:00 2001 From: Alexander Kuzmenkov Date: Tue, 10 Nov 2020 07:19:43 +0300 Subject: [PATCH 087/205] fixes --- tests/performance/constant_column_search.xml | 4 +--- tests/performance/quantile_merge.xml | 2 +- tests/performance/trim_urls.xml | 15 ++++++++++----- 3 files changed, 12 insertions(+), 9 deletions(-) diff --git a/tests/performance/constant_column_search.xml b/tests/performance/constant_column_search.xml index 6914aef705e..94b41fbf907 100644 --- a/tests/performance/constant_column_search.xml +++ b/tests/performance/constant_column_search.xml @@ -8,13 +8,11 @@ hits_10m_single - - - + diff --git a/tests/performance/quantile_merge.xml b/tests/performance/quantile_merge.xml index 0ddb688d8eb..36c4e583aa9 100644 --- a/tests/performance/quantile_merge.xml +++ b/tests/performance/quantile_merge.xml @@ -1,3 +1,3 @@ - SELECT quantileMerge(arrayJoin(arrayMap(x -> state, range(500000)))) FROM (SELECT quantileState(rand()) AS state FROM numbers(10000)) + SELECT quantileMerge(arrayJoin(arrayMap(x -> state, range(500000)))) FROM (SELECT quantileState(rand()) AS state FROM numbers(100000)) diff --git a/tests/performance/trim_urls.xml b/tests/performance/trim_urls.xml index 5ca565da7e8..f7308f15c04 100644 --- a/tests/performance/trim_urls.xml +++ b/tests/performance/trim_urls.xml @@ -1,22 +1,27 @@ - hits_100m_single + hits_10m_single - func + func_fast trim( ltrim( rtrim( trim(LEADING 'htpsw:/' FROM - trim(TRAILING '/' FROM - trim(BOTH 'htpsw:/' FROM + + func_slow + + trim(TRAILING '/' FROM + trim(BOTH 'htpsw:/' FROM + - SELECT ignore({func}URL)) FROM hits_100m_single LIMIT 220000000 FORMAT Null + SELECT ignore({func_fast}URL)) FROM hits_10m_single FORMAT Null + SELECT ignore({func_slow}URL)) FROM hits_100m_single FORMAT Null From ca4088b400d28b39165588c5596b80ecc555e744 Mon Sep 17 00:00:00 2001 From: Pavel Kruglov Date: Thu, 5 Nov 2020 16:53:56 +0300 Subject: [PATCH 088/205] Try to treat input values as enum id in tsv/scv --- src/DataTypes/DataTypeEnum.cpp | 6 ++--- src/DataTypes/DataTypeEnum.h | 23 +++++++++++++++---- src/IO/ReadHelpers.cpp | 8 +++++++ src/IO/ReadHelpers.h | 2 ++ ...558_enum_as_num_in_tsv_csv_input.reference | 10 ++++++++ .../01558_enum_as_num_in_tsv_csv_input.sql | 23 +++++++++++++++++++ 6 files changed, 64 insertions(+), 8 deletions(-) create mode 100644 tests/queries/0_stateless/01558_enum_as_num_in_tsv_csv_input.reference create mode 100644 tests/queries/0_stateless/01558_enum_as_num_in_tsv_csv_input.sql diff --git a/src/DataTypes/DataTypeEnum.cpp b/src/DataTypes/DataTypeEnum.cpp index ce61794facd..53b309e1db7 100644 --- a/src/DataTypes/DataTypeEnum.cpp +++ b/src/DataTypes/DataTypeEnum.cpp @@ -155,7 +155,7 @@ void DataTypeEnum::deserializeTextEscaped(IColumn & column, ReadBuffer & i /// NOTE It would be nice to do without creating a temporary object - at least extract std::string out. std::string field_name; readEscapedString(field_name, istr); - assert_cast(column).getData().push_back(getValue(StringRef(field_name))); + assert_cast(column).getData().push_back(getValue(StringRef(field_name), true)); } } @@ -182,7 +182,7 @@ void DataTypeEnum::deserializeWholeText(IColumn & column, ReadBuffer & ist { std::string field_name; readString(field_name, istr); - assert_cast(column).getData().push_back(getValue(StringRef(field_name))); + assert_cast(column).getData().push_back(getValue(StringRef(field_name), true)); } } @@ -226,7 +226,7 @@ void DataTypeEnum::deserializeTextCSV(IColumn & column, ReadBuffer & istr, { std::string field_name; readCSVString(field_name, istr, settings.csv); - assert_cast(column).getData().push_back(getValue(StringRef(field_name))); + assert_cast(column).getData().push_back(getValue(StringRef(field_name), true)); } } diff --git a/src/DataTypes/DataTypeEnum.h b/src/DataTypes/DataTypeEnum.h index a66b36c6a8d..c6850e022d3 100644 --- a/src/DataTypes/DataTypeEnum.h +++ b/src/DataTypes/DataTypeEnum.h @@ -4,6 +4,7 @@ #include #include #include +#include #include #include @@ -80,13 +81,25 @@ public: return findByValue(value)->second; } - FieldType getValue(StringRef field_name) const + FieldType getValue(StringRef field_name, bool try_treat_as_id=false) const { - const auto it = name_to_value_map.find(field_name); - if (!it) + const auto value_it = name_to_value_map.find(field_name); + if (!value_it) + { + /// It is used in CSV and TSV input formats. If we fail to find given string in + /// enum names and this string is number, we will try to treat it as enum id. + if (try_treat_as_id && isStringNumber(field_name)) + { + FieldType x; + ReadBufferFromMemory tmp_buf(field_name.data, field_name.size); + readText(x, tmp_buf); + const auto name_it = value_to_name_map.find(x); + if (name_it != value_to_name_map.end()) + return x; + } throw Exception{"Unknown element '" + field_name.toString() + "' for type " + getName(), ErrorCodes::BAD_ARGUMENTS}; - - return it->getMapped(); + } + return value_it->getMapped(); } FieldType readValue(ReadBuffer & istr) const diff --git a/src/IO/ReadHelpers.cpp b/src/IO/ReadHelpers.cpp index bf41de3959a..73b4738b078 100644 --- a/src/IO/ReadHelpers.cpp +++ b/src/IO/ReadHelpers.cpp @@ -1111,4 +1111,12 @@ bool loadAtPosition(ReadBuffer & in, DB::Memory<> & memory, char * & current) return loaded_more; } +bool isStringNumber(StringRef str) +{ + for (size_t i = 0; i != str.size; ++i) + if (!isdigit(str.data[i])) + return false; + return true; +} + } diff --git a/src/IO/ReadHelpers.h b/src/IO/ReadHelpers.h index 9ff1858c723..5cb4ec680c7 100644 --- a/src/IO/ReadHelpers.h +++ b/src/IO/ReadHelpers.h @@ -1228,4 +1228,6 @@ void saveUpToPosition(ReadBuffer & in, Memory<> & memory, char * current); */ bool loadAtPosition(ReadBuffer & in, Memory<> & memory, char * & current); +bool isStringNumber(StringRef str); + } diff --git a/tests/queries/0_stateless/01558_enum_as_num_in_tsv_csv_input.reference b/tests/queries/0_stateless/01558_enum_as_num_in_tsv_csv_input.reference new file mode 100644 index 00000000000..103709cf498 --- /dev/null +++ b/tests/queries/0_stateless/01558_enum_as_num_in_tsv_csv_input.reference @@ -0,0 +1,10 @@ +1 a +2 3 +3 3 +4 a +5 b +6 a +7 3 +8 3 +9 a +10 b diff --git a/tests/queries/0_stateless/01558_enum_as_num_in_tsv_csv_input.sql b/tests/queries/0_stateless/01558_enum_as_num_in_tsv_csv_input.sql new file mode 100644 index 00000000000..6a0f2a97b4f --- /dev/null +++ b/tests/queries/0_stateless/01558_enum_as_num_in_tsv_csv_input.sql @@ -0,0 +1,23 @@ +DROP TABLE IF EXISTS enum_as_num; + +CREATE TABLE enum_as_num ( + Id Int32, + Value Enum('a' = 1, '3' = 2, 'b' = 3) +) ENGINE=Memory(); + +INSERT INTO enum_as_num FORMAT TSV 1 1 +INSERT INTO enum_as_num FORMAT TSV 2 2 +INSERT INTO enum_as_num FORMAT TSV 3 3 +INSERT INTO enum_as_num FORMAT TSV 4 a +INSERT INTO enum_as_num FORMAT TSV 5 b + +INSERT INTO enum_as_num FORMAT CSV 6,1 +INSERT INTO enum_as_num FORMAT CSV 7,2 +INSERT INTO enum_as_num FORMAT CSV 8,3 +INSERT INTO enum_as_num FORMAT CSV 9,a +INSERT INTO enum_as_num FORMAT CSV 10,b + +SELECT * FROM enum_as_num ORDER BY Id; + + +DROP TABLE IF EXISTS enum_as_num; From 1db8e773710c3d2058912988a5c0ada2d1cf3115 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 10 Nov 2020 17:54:59 +0300 Subject: [PATCH 089/205] Add comments. Update ActionsDAG::Index --- src/Functions/FunctionsMiscellaneous.h | 2 +- src/Interpreters/ActionsVisitor.cpp | 4 +- src/Interpreters/ExpressionActions.cpp | 78 +++++++++---------- src/Interpreters/ExpressionActions.h | 50 ++++++++---- src/Interpreters/ExpressionAnalyzer.cpp | 4 +- src/Interpreters/ExpressionJIT.cpp | 4 +- .../Transforms/TotalsHavingTransform.cpp | 2 +- src/Storages/ColumnsDescription.cpp | 2 +- src/Storages/MergeTree/KeyCondition.cpp | 8 +- src/Storages/MergeTree/MergeTreeData.cpp | 4 +- src/Storages/ReadInOrderOptimizer.cpp | 2 +- src/Storages/StorageDistributed.cpp | 2 +- src/Storages/TTLDescription.cpp | 2 +- 13 files changed, 90 insertions(+), 74 deletions(-) diff --git a/src/Functions/FunctionsMiscellaneous.h b/src/Functions/FunctionsMiscellaneous.h index 096f640b553..064ddcc3f31 100644 --- a/src/Functions/FunctionsMiscellaneous.h +++ b/src/Functions/FunctionsMiscellaneous.h @@ -201,7 +201,7 @@ public: { /// Check that expression does not contain unusual actions that will break columnss structure. for (const auto & action : expression_actions->getActions()) - if (action.node->type == ActionsDAG::Type::ARRAY_JOIN) + if (action.node->type == ActionsDAG::ActionType::ARRAY_JOIN) throw Exception("Expression with arrayJoin or other unusual action cannot be captured", ErrorCodes::BAD_ARGUMENTS); std::unordered_map arguments_map; diff --git a/src/Interpreters/ActionsVisitor.cpp b/src/Interpreters/ActionsVisitor.cpp index b24cee9ff71..b47667efdbd 100644 --- a/src/Interpreters/ActionsVisitor.cpp +++ b/src/Interpreters/ActionsVisitor.cpp @@ -414,7 +414,7 @@ ScopeStack::ScopeStack(ActionsDAGPtr actions, const Context & context_) level.actions = std::move(actions); for (const auto & node : level.actions->getIndex()) - if (node->type == ActionsDAG::Type::INPUT) + if (node->type == ActionsDAG::ActionType::INPUT) level.inputs.emplace(node->result_name); } @@ -451,7 +451,7 @@ size_t ScopeStack::getColumnLevel(const std::string & name) const auto & index = stack[i].actions->getIndex(); auto it = index.find(name); - if (it != index.end() && (*it)->type != ActionsDAG::Type::INPUT) + if (it != index.end() && (*it)->type != ActionsDAG::ActionType::INPUT) return i; } diff --git a/src/Interpreters/ExpressionActions.cpp b/src/Interpreters/ExpressionActions.cpp index 5e389b9b5bd..82d50d63b0f 100644 --- a/src/Interpreters/ExpressionActions.cpp +++ b/src/Interpreters/ExpressionActions.cpp @@ -66,16 +66,16 @@ std::string ExpressionActions::Action::toString() const std::stringstream out; switch (node->type) { - case ActionsDAG::Type::COLUMN: + case ActionsDAG::ActionType::COLUMN: out << "COLUMN " << (node->column ? node->column->getName() : "(no column)"); break; - case ActionsDAG::Type::ALIAS: + case ActionsDAG::ActionType::ALIAS: out << "ALIAS " << node->children.front()->result_name << " " << arguments.front(); break; - case ActionsDAG::Type::FUNCTION: + case ActionsDAG::ActionType::FUNCTION: out << "FUNCTION " << (node->is_function_compiled ? "[compiled] " : "") << (node->function_base ? node->function_base->getName() : "(no function)") << "("; for (size_t i = 0; i < node->children.size(); ++i) @@ -87,11 +87,11 @@ std::string ExpressionActions::Action::toString() const out << ")"; break; - case ActionsDAG::Type::ARRAY_JOIN: + case ActionsDAG::ActionType::ARRAY_JOIN: out << "ARRAY JOIN " << node->children.front()->result_name << " " << arguments.front(); break; - case ActionsDAG::Type::INPUT: + case ActionsDAG::ActionType::INPUT: out << "INPUT " << arguments.front(); break; } @@ -201,7 +201,7 @@ void ExpressionActions::executeAction(const Action & action, ExecutionContext & switch (action.node->type) { - case ActionsDAG::Type::FUNCTION: + case ActionsDAG::ActionType::FUNCTION: { auto & res_column = columns[action.result_position]; if (res_column.type || res_column.column) @@ -227,7 +227,7 @@ void ExpressionActions::executeAction(const Action & action, ExecutionContext & break; } - case ActionsDAG::Type::ARRAY_JOIN: + case ActionsDAG::ActionType::ARRAY_JOIN: { size_t array_join_key_pos = action.arguments.front().pos; auto array_join_key = columns[array_join_key_pos]; @@ -260,7 +260,7 @@ void ExpressionActions::executeAction(const Action & action, ExecutionContext & break; } - case ActionsDAG::Type::COLUMN: + case ActionsDAG::ActionType::COLUMN: { auto & res_column = columns[action.result_position]; res_column.column = action.node->column->cloneResized(num_rows); @@ -269,7 +269,7 @@ void ExpressionActions::executeAction(const Action & action, ExecutionContext & break; } - case ActionsDAG::Type::ALIAS: + case ActionsDAG::ActionType::ALIAS: { const auto & arg = action.arguments.front(); if (action.result_position != arg.pos) @@ -286,7 +286,7 @@ void ExpressionActions::executeAction(const Action & action, ExecutionContext & break; } - case ActionsDAG::Type::INPUT: + case ActionsDAG::ActionType::INPUT: { auto pos = execution_context.inputs_pos[action.arguments.front().pos]; if (pos < 0) @@ -315,7 +315,7 @@ Names ExpressionActions::getRequiredColumns() const bool ExpressionActions::hasArrayJoin() const { for (const auto & action : actions) - if (action.node->type == ActionsDAG::Type::ARRAY_JOIN) + if (action.node->type == ActionsDAG::ActionType::ARRAY_JOIN) return true; return false; @@ -388,7 +388,7 @@ std::string ExpressionActions::dumpActions() const bool ActionsDAG::hasArrayJoin() const { for (const auto & node : nodes) - if (node.type == Type::ARRAY_JOIN) + if (node.type == ActionType::ARRAY_JOIN) return true; return false; @@ -397,7 +397,7 @@ bool ActionsDAG::hasArrayJoin() const bool ActionsDAG::empty() const { for (const auto & node : nodes) - if (node.type != Type::INPUT) + if (node.type != ActionType::INPUT) return false; return true; @@ -468,7 +468,7 @@ ActionsDAGPtr ActionsDAG::splitActionsBeforeArrayJoin(const NameSet & array_join /// Make a copy part. if (cur.next_child_to_visit == cur.node->children.size()) { - if (cur.node->type == Type::INPUT && array_joined_columns.count(cur.node->result_name)) + if (cur.node->type == ActionType::INPUT && array_joined_columns.count(cur.node->result_name)) cur_data.depend_on_array_join = true; cur_data.visited = true; @@ -487,7 +487,7 @@ ActionsDAGPtr ActionsDAG::splitActionsBeforeArrayJoin(const NameSet & array_join /// If children is not created, int may be from split part. if (!child_data.to_this) { - if (child->type == Type::COLUMN) /// Just create new node for COLUMN action. + if (child->type == ActionType::COLUMN) /// Just create new node for COLUMN action. { child_data.to_this = &this_nodes.emplace_back(*child); } @@ -495,13 +495,13 @@ ActionsDAGPtr ActionsDAG::splitActionsBeforeArrayJoin(const NameSet & array_join { /// Node from split part is added as new input. Node input_node; - input_node.type = Type::INPUT; + input_node.type = ActionType::INPUT; input_node.result_type = child->result_type; input_node.result_name = child->result_name; // getUniqueNameForIndex(index, child->result_name); child_data.to_this = &this_nodes.emplace_back(std::move(input_node)); /// This node is needed for current action, so put it to index also. - split_index[child_data.to_split->result_name] = child_data.to_split; + split_index.replace(child_data.to_split); } } @@ -522,11 +522,11 @@ ActionsDAGPtr ActionsDAG::splitActionsBeforeArrayJoin(const NameSet & array_join if (cur_data.used_in_result) { - split_index[copy.result_name] = © + split_index.replace(©); /// If this node is needed in result, add it as input. Node input_node; - input_node.type = Type::INPUT; + input_node.type = ActionType::INPUT; input_node.result_type = node.result_type; input_node.result_name = node.result_name; cur_data.to_this = &this_nodes.emplace_back(std::move(input_node)); @@ -567,7 +567,7 @@ bool ExpressionActions::checkColumnIsAlwaysFalse(const String & column_name) con for (auto it = actions.rbegin(); it != actions.rend(); ++it) { const auto & action = *it; - if (action.node->type == ActionsDAG::Type::FUNCTION && action.node->function_base) + if (action.node->type == ActionsDAG::ActionType::FUNCTION && action.node->function_base) { if (action.node->result_name == column_name && action.node->children.size() > 1) { @@ -585,7 +585,7 @@ bool ExpressionActions::checkColumnIsAlwaysFalse(const String & column_name) con { for (const auto & action : actions) { - if (action.node->type == ActionsDAG::Type::COLUMN && action.node->result_name == set_to_check) + if (action.node->type == ActionsDAG::ActionType::COLUMN && action.node->result_name == set_to_check) { // Constant ColumnSet cannot be empty, so we only need to check non-constant ones. if (const auto * column_set = checkAndGetColumn(action.node->column.get())) @@ -793,7 +793,7 @@ ActionsDAG::Node & ActionsDAG::addNode(Node node, bool can_replace) auto & res = nodes.emplace_back(std::move(node)); - index[res.result_name] = &res; + index.replace(&res); return res; } @@ -809,7 +809,7 @@ ActionsDAG::Node & ActionsDAG::getNode(const std::string & name) const ActionsDAG::Node & ActionsDAG::addInput(std::string name, DataTypePtr type) { Node node; - node.type = Type::INPUT; + node.type = ActionType::INPUT; node.result_type = std::move(type); node.result_name = std::move(name); @@ -819,7 +819,7 @@ const ActionsDAG::Node & ActionsDAG::addInput(std::string name, DataTypePtr type const ActionsDAG::Node & ActionsDAG::addInput(ColumnWithTypeAndName column) { Node node; - node.type = Type::INPUT; + node.type = ActionType::INPUT; node.result_type = std::move(column.type); node.result_name = std::move(column.name); node.column = std::move(column.column); @@ -833,7 +833,7 @@ const ActionsDAG::Node & ActionsDAG::addColumn(ColumnWithTypeAndName column) throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot add column {} because it is nullptr", column.name); Node node; - node.type = Type::COLUMN; + node.type = ActionType::COLUMN; node.result_type = std::move(column.type); node.result_name = std::move(column.name); node.column = std::move(column.column); @@ -846,7 +846,7 @@ const ActionsDAG::Node & ActionsDAG::addAlias(const std::string & name, std::str auto & child = getNode(name); Node node; - node.type = Type::ALIAS; + node.type = ActionType::ALIAS; node.result_type = child.result_type; node.result_name = std::move(alias); node.column = child.column; @@ -865,7 +865,7 @@ const ActionsDAG::Node & ActionsDAG::addArrayJoin(const std::string & source_nam throw Exception("ARRAY JOIN requires array argument", ErrorCodes::TYPE_MISMATCH); Node node; - node.type = Type::ARRAY_JOIN; + node.type = ActionType::ARRAY_JOIN; node.result_type = array_type->getNestedType(); node.result_name = std::move(result_name); node.children.emplace_back(&child); @@ -894,7 +894,7 @@ const ActionsDAG::Node & ActionsDAG::addFunction( size_t num_arguments = argument_names.size(); Node node; - node.type = Type::FUNCTION; + node.type = ActionType::FUNCTION; node.function_builder = function; node.children.reserve(num_arguments); @@ -977,7 +977,7 @@ NamesAndTypesList ActionsDAG::getRequiredColumns() const { NamesAndTypesList result; for (const auto & node : nodes) - if (node.type == Type::INPUT) + if (node.type == ActionType::INPUT) result.push_back({node.result_name, node.result_type}); return result; @@ -1077,7 +1077,7 @@ void ActionsDAG::removeUnusedActions() if (!node->children.empty() && node->column && isColumnConst(*node->column) && node->allow_constant_folding) { /// Constant folding. - node->type = ActionsDAG::Type::COLUMN; + node->type = ActionsDAG::ActionType::COLUMN; node->children.clear(); } @@ -1114,7 +1114,7 @@ void ActionsDAG::addAliases(const NamesWithAliases & aliases, std::vectorresult_type; node.result_name = std::move(item.second); node.column = child->column; @@ -1160,7 +1160,7 @@ bool ActionsDAG::tryRestoreColumn(const std::string & column_name) auto & node = *it; if (node.result_name == column_name) { - index[node.result_name] = &node; + index.replace(&node); return true; } } @@ -1308,7 +1308,7 @@ ExpressionActionsPtr ActionsDAG::linearizeActions() const arguments.emplace_back(argument); } - if (node->type == Type::INPUT) + if (node->type == ActionType::INPUT) { /// Argument for input is special. It contains the position from required columns. ExpressionActions::Argument argument; @@ -1328,7 +1328,7 @@ ExpressionActionsPtr ActionsDAG::linearizeActions() const if (parent_data.num_created_children == parent->children.size()) { - auto & push_stack = parent->type == Type::ARRAY_JOIN ? ready_array_joins : ready_nodes; + auto & push_stack = parent->type == ActionType::ARRAY_JOIN ? ready_array_joins : ready_nodes; push_stack.push(parent); } } @@ -1398,23 +1398,23 @@ std::string ActionsDAG::dumpDAG() const out << map[&node] << " : "; switch (node.type) { - case ActionsDAG::Type::COLUMN: + case ActionsDAG::ActionType::COLUMN: out << "COLUMN "; break; - case ActionsDAG::Type::ALIAS: + case ActionsDAG::ActionType::ALIAS: out << "ALIAS "; break; - case ActionsDAG::Type::FUNCTION: + case ActionsDAG::ActionType::FUNCTION: out << "FUNCTION "; break; - case ActionsDAG::Type::ARRAY_JOIN: + case ActionsDAG::ActionType::ARRAY_JOIN: out << "ARRAY JOIN "; break; - case ActionsDAG::Type::INPUT: + case ActionsDAG::ActionType::INPUT: out << "INPUT "; break; } diff --git a/src/Interpreters/ExpressionActions.h b/src/Interpreters/ExpressionActions.h index 51a3fc9e384..bdc35c8e0f2 100644 --- a/src/Interpreters/ExpressionActions.h +++ b/src/Interpreters/ExpressionActions.h @@ -61,11 +61,20 @@ using ExpressionActionsPtr = std::shared_ptr; class ActionsDAG; using ActionsDAGPtr = std::shared_ptr; +/// Directed acyclic graph of expressions. +/// This is an intermediate representation of actions which is usually built from expression list AST. +/// Node of DAG describe calculation of a single column with known type, name, and constant value (if applicable). +/// +/// DAG representation is useful in case we need to know explicit dependencies between actions. +/// It is helpful when it is needed to optimize actions, remove unused expressions, compile subexpressions, +/// split or merge parts of graph, calculate expressions on partial input. +/// +/// Built DAG is used by ExpressionActions, which calculates expressions on block. class ActionsDAG { public: - enum class Type + enum class ActionType { /// Column which must be in input. INPUT, @@ -82,7 +91,7 @@ public: { std::vector children; - Type type; + ActionType type; std::string result_name; DataTypePtr result_type; @@ -108,24 +117,11 @@ public: { private: std::list list; + /// Map key is a string_view to Node::result_name for node from value. + /// Map always point to existing node, so key always valid (nodes live longer then index). std::unordered_map::iterator> map; public: - Node *& operator[](std::string_view key) - { - auto res = map.emplace(key, list.end()); - if (res.second) - res.first->second = list.emplace(list.end(), nullptr); - - return *res.first->second; - } - - void swap(Index & other) - { - list.swap(other.list); - map.swap(other.map); - } - auto size() const { return list.size(); } bool contains(std::string_view key) const { return map.count(key) != 0; } @@ -146,6 +142,20 @@ public: /// If node with the same name exists, it is removed from map, but not list. /// It is expected and used for project(), when result may have several columns with the same name. void insert(Node * node) { map[node->result_name] = list.emplace(list.end(), node); } + + /// If node with same name exists in index, replace it. Otherwise insert new node to index. + void replace(Node * node) + { + if (auto handle = map.extract(node->result_name)) + { + handle.key() = node->result_name; /// Change string_view + *handle.mapped() = node; + map.insert(std::move(handle)); + } + else + insert(node); + } + void remove(Node * node) { auto it = map.find(node->result_name); @@ -155,6 +165,12 @@ public: list.erase(it->second); map.erase(it); } + + void swap(Index & other) + { + list.swap(other.list); + map.swap(other.map); + } }; using Nodes = std::list; diff --git a/src/Interpreters/ExpressionAnalyzer.cpp b/src/Interpreters/ExpressionAnalyzer.cpp index 46d15705ece..60975b75196 100644 --- a/src/Interpreters/ExpressionAnalyzer.cpp +++ b/src/Interpreters/ExpressionAnalyzer.cpp @@ -77,7 +77,7 @@ bool allowEarlyConstantFolding(const ActionsDAG & actions, const Settings & sett for (const auto & node : actions.getNodes()) { - if (node.type == ActionsDAG::Type::FUNCTION && node.function_base) + if (node.type == ActionsDAG::ActionType::FUNCTION && node.function_base) { auto name = node.function_base->getName(); if (name == "ignore") @@ -1309,7 +1309,7 @@ void ExpressionAnalysisResult::checkActions() const { if (actions) for (const auto & node : actions->getNodes()) - if (node.type == ActionsDAG::Type::ARRAY_JOIN) + if (node.type == ActionsDAG::ActionType::ARRAY_JOIN) throw Exception("PREWHERE cannot contain ARRAY JOIN action", ErrorCodes::ILLEGAL_PREWHERE); }; diff --git a/src/Interpreters/ExpressionJIT.cpp b/src/Interpreters/ExpressionJIT.cpp index d36b08bcd74..05aebbd1f40 100644 --- a/src/Interpreters/ExpressionJIT.cpp +++ b/src/Interpreters/ExpressionJIT.cpp @@ -593,7 +593,7 @@ static bool isCompilableConstant(const ActionsDAG::Node & node) static bool isCompilableFunction(const ActionsDAG::Node & node) { - return node.type == ActionsDAG::Type::FUNCTION && isCompilable(*node.function_base); + return node.type == ActionsDAG::ActionType::FUNCTION && isCompilable(*node.function_base); } static LLVMFunction::CompileDAG getCompilableDAG( @@ -884,7 +884,7 @@ void ActionsDAG::compileFunctions() for (const auto * child : new_children) arguments.emplace_back(child->column, child->result_type, child->result_name); - frame.node->type = ActionsDAG::Type::FUNCTION; + frame.node->type = ActionsDAG::ActionType::FUNCTION; frame.node->function_base = fn; frame.node->function = fn->prepare(arguments); frame.node->children.swap(new_children); diff --git a/src/Processors/Transforms/TotalsHavingTransform.cpp b/src/Processors/Transforms/TotalsHavingTransform.cpp index 109ead164a5..c81c389b6fa 100644 --- a/src/Processors/Transforms/TotalsHavingTransform.cpp +++ b/src/Processors/Transforms/TotalsHavingTransform.cpp @@ -164,7 +164,7 @@ void TotalsHavingTransform::transform(Chunk & chunk) for (const auto & action : expression->getActions()) { - if (action.node->type == ActionsDAG::Type::ARRAY_JOIN) + if (action.node->type == ActionsDAG::ActionType::ARRAY_JOIN) throw Exception("Having clause cannot contain arrayJoin", ErrorCodes::ILLEGAL_COLUMN); } diff --git a/src/Storages/ColumnsDescription.cpp b/src/Storages/ColumnsDescription.cpp index c55d608110a..dbe0b0247f6 100644 --- a/src/Storages/ColumnsDescription.cpp +++ b/src/Storages/ColumnsDescription.cpp @@ -502,7 +502,7 @@ Block validateColumnsDefaultsAndGetSampleBlock(ASTPtr default_expr_list, const N auto syntax_analyzer_result = TreeRewriter(context).analyze(default_expr_list, all_columns); const auto actions = ExpressionAnalyzer(default_expr_list, syntax_analyzer_result, context).getActions(true); for (const auto & action : actions->getActions()) - if (action.node->type == ActionsDAG::Type::ARRAY_JOIN) + if (action.node->type == ActionsDAG::ActionType::ARRAY_JOIN) throw Exception("Unsupported default value that requires ARRAY JOIN action", ErrorCodes::THERE_IS_NO_DEFAULT_VALUE); return actions->getSampleBlock(); diff --git a/src/Storages/MergeTree/KeyCondition.cpp b/src/Storages/MergeTree/KeyCondition.cpp index 6771d99c100..80b65f73d99 100644 --- a/src/Storages/MergeTree/KeyCondition.cpp +++ b/src/Storages/MergeTree/KeyCondition.cpp @@ -616,7 +616,7 @@ bool KeyCondition::canConstantBeWrappedByMonotonicFunctions( * which while not strictly monotonic, are monotonic everywhere on the input range. */ const auto & children = action.node->children; - if (action.node->type == ActionsDAG::Type::FUNCTION + if (action.node->type == ActionsDAG::ActionType::FUNCTION && children.size() == 1 && children[0]->result_name == expr_name) { @@ -683,7 +683,7 @@ bool KeyCondition::canConstantBeWrappedByFunctions( chain.push(cur_node); - if (cur_node->type == ActionsDAG::Type::FUNCTION && cur_node->children.size() <= 2) + if (cur_node->type == ActionsDAG::ActionType::FUNCTION && cur_node->children.size() <= 2) { if (!cur_node->function_base->isDeterministic()) is_valid_chain = false; @@ -705,7 +705,7 @@ bool KeyCondition::canConstantBeWrappedByFunctions( cur_node = next_node; } - else if (cur_node->type == ActionsDAG::Type::ALIAS) + else if (cur_node->type == ActionsDAG::ActionType::ALIAS) cur_node = cur_node->children.front(); else is_valid_chain = false; @@ -725,7 +725,7 @@ bool KeyCondition::canConstantBeWrappedByFunctions( const auto * func = chain.top(); chain.pop(); - if (func->type != ActionsDAG::Type::FUNCTION) + if (func->type != ActionsDAG::ActionType::FUNCTION) continue; if (func->children.size() == 1) diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 43dfdfe7beb..4bfe97f2dc3 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -259,10 +259,10 @@ static void checkKeyExpression(const ExpressionActions & expr, const Block & sam { for (const auto & action : expr.getActions()) { - if (action.node->type == ActionsDAG::Type::ARRAY_JOIN) + if (action.node->type == ActionsDAG::ActionType::ARRAY_JOIN) throw Exception(key_name + " key cannot contain array joins", ErrorCodes::ILLEGAL_COLUMN); - if (action.node->type == ActionsDAG::Type::FUNCTION) + if (action.node->type == ActionsDAG::ActionType::FUNCTION) { IFunctionBase & func = *action.node->function_base; if (!func.isDeterministic()) diff --git a/src/Storages/ReadInOrderOptimizer.cpp b/src/Storages/ReadInOrderOptimizer.cpp index 6761e0b967a..3613fbff5cf 100644 --- a/src/Storages/ReadInOrderOptimizer.cpp +++ b/src/Storages/ReadInOrderOptimizer.cpp @@ -57,7 +57,7 @@ InputOrderInfoPtr ReadInOrderOptimizer::getInputOrder(const StorageMetadataPtr & bool found_function = false; for (const auto & action : elements_actions[i]->getActions()) { - if (action.node->type != ActionsDAG::Type::FUNCTION) + if (action.node->type != ActionsDAG::ActionType::FUNCTION) continue; if (found_function) diff --git a/src/Storages/StorageDistributed.cpp b/src/Storages/StorageDistributed.cpp index 4a98bc4673f..977df904dca 100644 --- a/src/Storages/StorageDistributed.cpp +++ b/src/Storages/StorageDistributed.cpp @@ -201,7 +201,7 @@ bool isExpressionActionsDeterministics(const ExpressionActionsPtr & actions) { for (const auto & action : actions->getActions()) { - if (action.node->type != ActionsDAG::Type::FUNCTION) + if (action.node->type != ActionsDAG::ActionType::FUNCTION) continue; if (!action.node->function_base->isDeterministic()) return false; diff --git a/src/Storages/TTLDescription.cpp b/src/Storages/TTLDescription.cpp index 50a74ddf7f0..b1c6a033d8b 100644 --- a/src/Storages/TTLDescription.cpp +++ b/src/Storages/TTLDescription.cpp @@ -54,7 +54,7 @@ void checkTTLExpression(const ExpressionActionsPtr & ttl_expression, const Strin { for (const auto & action : ttl_expression->getActions()) { - if (action.node->type == ActionsDAG::Type::FUNCTION) + if (action.node->type == ActionsDAG::ActionType::FUNCTION) { IFunctionBase & func = *action.node->function_base; if (!func.isDeterministic()) From c6575c90325de698b3207e0fc11e042121703ba0 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 10 Nov 2020 19:27:55 +0300 Subject: [PATCH 090/205] Update ExpressionActions constructur --- src/Interpreters/ActionsVisitor.cpp | 2 +- src/Interpreters/ExpressionActions.cpp | 380 ++++++++---------- src/Interpreters/ExpressionActions.h | 51 +-- src/Interpreters/ExpressionAnalyzer.cpp | 14 +- src/Interpreters/InterpreterSelectQuery.cpp | 8 +- src/Processors/QueryPlan/ExpressionStep.cpp | 8 +- src/Processors/QueryPlan/FilterStep.cpp | 8 +- src/Processors/QueryPlan/TotalsHavingStep.cpp | 9 +- src/Storages/MergeTree/MergeTreeData.cpp | 2 +- .../MergeTree/MergeTreeDataSelectExecutor.cpp | 2 +- 10 files changed, 226 insertions(+), 258 deletions(-) diff --git a/src/Interpreters/ActionsVisitor.cpp b/src/Interpreters/ActionsVisitor.cpp index b47667efdbd..29fb9ed336f 100644 --- a/src/Interpreters/ActionsVisitor.cpp +++ b/src/Interpreters/ActionsVisitor.cpp @@ -895,7 +895,7 @@ void ActionsMatcher::visit(const ASTFunction & node, const ASTPtr & ast, Data & String result_name = lambda->arguments->children.at(1)->getColumnName(); lambda_dag->removeUnusedActions(Names(1, result_name)); - auto lambda_actions = lambda_dag->buildExpressions(); + auto lambda_actions = std::make_shared(lambda_dag); DataTypePtr result_type = lambda_actions->getSampleBlock().getByName(result_name).type; diff --git a/src/Interpreters/ExpressionActions.cpp b/src/Interpreters/ExpressionActions.cpp index 82d50d63b0f..73bc54a0473 100644 --- a/src/Interpreters/ExpressionActions.cpp +++ b/src/Interpreters/ExpressionActions.cpp @@ -53,8 +53,164 @@ namespace ErrorCodes extern const int TYPE_MISMATCH; } -/// Read comment near usage -/// static constexpr auto DUMMY_COLUMN_NAME = "_dummy"; +ExpressionActions::~ExpressionActions() = default; + +ExpressionActions::ExpressionActions(ActionsDAGPtr actions_dag_) +{ + actions_dag = actions_dag_->clone(); + + actions_dag->compileExpressions(); + + linearizeActions(); + + const auto & settings = actions_dag->getSettings(); + + if (settings.max_temporary_columns && num_columns > settings.max_temporary_columns) + throw Exception(ErrorCodes::TOO_MANY_TEMPORARY_COLUMNS, + "Too many temporary columns: {}. Maximum: {}", + actions_dag->dumpNames(), std::to_string(settings.max_temporary_columns)); + + max_temporary_non_const_columns = settings.max_temporary_non_const_columns; + project_input = settings.project_input; +} + +ExpressionActionsPtr ExpressionActions::clone() const +{ + auto expressions = std::make_shared(*this); +} + +void ExpressionActions::linearizeActions() +{ + struct Data + { + const Node * node = nullptr; + size_t num_created_children = 0; + std::vector parents; + + ssize_t position = -1; + size_t num_created_parents = 0; + bool used_in_result = false; + }; + + const auto & nodes = getNodes(); + const auto & index = actions_dag->getIndex(); + + std::vector data(nodes.size()); + std::unordered_map reverse_index; + + for (const auto & node : nodes) + { + size_t id = reverse_index.size(); + data[id].node = &node; + reverse_index[&node] = id; + } + + std::queue ready_nodes; + std::queue ready_array_joins; + + for (const auto * node : index) + data[reverse_index[node]].used_in_result = true; + + for (const auto & node : nodes) + { + for (const auto & child : node.children) + data[reverse_index[child]].parents.emplace_back(&node); + } + + for (const auto & node : nodes) + { + if (node.children.empty()) + ready_nodes.emplace(&node); + } + + std::stack free_positions; + + while (!ready_nodes.empty() || !ready_array_joins.empty()) + { + auto & stack = ready_nodes.empty() ? ready_array_joins : ready_nodes; + const Node * node = stack.front(); + stack.pop(); + + Names argument_names; + for (const auto & child : node->children) + argument_names.emplace_back(child->result_name); + + auto & cur = data[reverse_index[node]]; + + size_t free_position = num_columns; + if (free_positions.empty()) + ++num_columns; + else + { + free_position = free_positions.top(); + free_positions.pop(); + } + + cur.position = free_position; + + ExpressionActions::Arguments arguments; + arguments.reserve(cur.node->children.size()); + for (auto * child : cur.node->children) + { + auto & arg = data[reverse_index[child]]; + + if (arg.position < 0) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Argument was not calculated for {}", child->result_name); + + ++arg.num_created_parents; + + ExpressionActions::Argument argument; + argument.pos = arg.position; + argument.needed_later = arg.used_in_result || arg.num_created_parents != arg.parents.size(); + + if (!argument.needed_later) + free_positions.push(argument.pos); + + arguments.emplace_back(argument); + } + + if (node->type == ActionsDAG::ActionType::INPUT) + { + /// Argument for input is special. It contains the position from required columns. + ExpressionActions::Argument argument; + argument.pos = required_columns.size(); + argument.needed_later = !cur.parents.empty(); + arguments.emplace_back(argument); + + required_columns.push_back({node->result_name, node->result_type}); + } + + actions.push_back({node, arguments, free_position}); + + for (const auto & parent : cur.parents) + { + auto & parent_data = data[reverse_index[parent]]; + ++parent_data.num_created_children; + + if (parent_data.num_created_children == parent->children.size()) + { + auto & push_stack = parent->type == ActionsDAG::ActionType::ARRAY_JOIN ? ready_array_joins : ready_nodes; + push_stack.push(parent); + } + } + } + + result_positions.reserve(index.size()); + + for (const auto & node : index) + { + auto pos = data[reverse_index[node]].position; + + if (pos < 0) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Action for {} was not calculated", node->result_name); + + result_positions.push_back(pos); + + ColumnWithTypeAndName col{node->column, node->result_type, node->result_name}; + sample_block.insert(std::move(col)); + } +} + static std::ostream & operator << (std::ostream & out, const ExpressionActions::Argument & argument) { @@ -101,8 +257,6 @@ std::string ExpressionActions::Action::toString() const return out.str(); } -ExpressionActions::~ExpressionActions() = default; - void ExpressionActions::checkLimits(ExecutionContext & execution_context) const { if (max_temporary_non_const_columns) @@ -371,19 +525,6 @@ std::string ExpressionActions::dumpActions() const return ss.str(); } -//static std::string getUniqueNameForIndex(ActionsDAG::Index & index, std::string name) -//{ -// if (index.contains(name)) -// return name; -// -// size_t next_id = 0; -// std::string res; -// do -// res = name + "_" + std::to_string(next_id); -// while (index.contains(res)); -// -// return res; -//} bool ActionsDAG::hasArrayJoin() const { @@ -879,13 +1020,13 @@ const ActionsDAG::Node & ActionsDAG::addFunction( std::string result_name, const Context & context [[maybe_unused]]) { - const auto & settings = context.getSettingsRef(); - max_temporary_columns = settings.max_temporary_columns; - max_temporary_non_const_columns = settings.max_temporary_non_const_columns; + const auto & all_settings = context.getSettingsRef(); + settings.max_temporary_columns = all_settings.max_temporary_columns; + settings.max_temporary_non_const_columns = all_settings.max_temporary_non_const_columns; #if USE_EMBEDDED_COMPILER - compile_expressions = settings.compile_expressions; - min_count_to_compile_expression = settings.min_count_to_compile_expression; + settings.compile_expressions = settings.compile_expressions; + settings.min_count_to_compile_expression = settings.min_count_to_compile_expression; if (!compilation_cache) compilation_cache = context.getCompiledExpressionCache(); @@ -925,7 +1066,8 @@ const ActionsDAG::Node & ActionsDAG::addFunction( /// If all arguments are constants, and function is suitable to be executed in 'prepare' stage - execute function. /// But if we compile expressions compiled version of this function maybe placed in cache, /// so we don't want to unfold non deterministic functions - if (all_const && node.function_base->isSuitableForConstantFolding() && (!compile_expressions || node.function_base->isDeterministic())) + if (all_const && node.function_base->isSuitableForConstantFolding() + && (!settings.compile_expressions || node.function_base->isDeterministic())) { size_t num_rows = arguments.empty() ? 0 : arguments.front().column->size(); auto col = node.function->execute(arguments, node.result_type, num_rows, true); @@ -1141,7 +1283,7 @@ void ActionsDAG::project(const NamesWithAliases & projection) addAliases(projection, result_nodes); removeUnusedActions(result_nodes); projectInput(); - projected_output = true; + settings.projected_output = true; } void ActionsDAG::removeColumn(const std::string & column_name) @@ -1190,197 +1332,15 @@ ActionsDAGPtr ActionsDAG::clone() const return actions; } -ExpressionActionsPtr ExpressionActions::clone() const +void ActionsDAG::compileExpressions() { - auto expressions = std::make_shared(); - - expressions->actions = actions; - expressions->num_columns = num_columns; - expressions->required_columns = required_columns; - expressions->result_positions = result_positions; - expressions->sample_block = sample_block; - expressions->project_input = project_input; - expressions->max_temporary_non_const_columns = max_temporary_non_const_columns; - - std::unordered_map copy_map; - for (const auto & node : nodes) - { - auto & copy_node = expressions->nodes.emplace_back(node); - copy_map[&node] = ©_node; - } - - for (auto & node : expressions->nodes) - for (auto & child : node.children) - child = copy_map[child]; - - for (auto & action : expressions->actions) - action.node = copy_map[action.node]; - - return expressions; -} - - -ExpressionActionsPtr ActionsDAG::linearizeActions() const -{ - struct Data - { - const Node * node = nullptr; - size_t num_created_children = 0; - std::vector parents; - - ssize_t position = -1; - size_t num_created_parents = 0; - bool used_in_result = false; - }; - - std::vector data(nodes.size()); - std::unordered_map reverse_index; - - for (const auto & node : nodes) - { - size_t id = reverse_index.size(); - data[id].node = &node; - reverse_index[&node] = id; - } - - std::queue ready_nodes; - std::queue ready_array_joins; - - for (const auto * node : index) - data[reverse_index[node]].used_in_result = true; - - for (const auto & node : nodes) - { - for (const auto & child : node.children) - data[reverse_index[child]].parents.emplace_back(&node); - } - - for (const auto & node : nodes) - { - if (node.children.empty()) - ready_nodes.emplace(&node); - } - - auto expressions = std::make_shared(); - std::stack free_positions; - - while (!ready_nodes.empty() || !ready_array_joins.empty()) - { - auto & stack = ready_nodes.empty() ? ready_array_joins : ready_nodes; - const Node * node = stack.front(); - stack.pop(); - - Names argument_names; - for (const auto & child : node->children) - argument_names.emplace_back(child->result_name); - - auto & cur = data[reverse_index[node]]; - - size_t free_position = expressions->num_columns; - if (free_positions.empty()) - ++expressions->num_columns; - else - { - free_position = free_positions.top(); - free_positions.pop(); - } - - cur.position = free_position; - - ExpressionActions::Arguments arguments; - arguments.reserve(cur.node->children.size()); - for (auto * child : cur.node->children) - { - auto & arg = data[reverse_index[child]]; - - if (arg.position < 0) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Argument was not calculated for {}", child->result_name); - - ++arg.num_created_parents; - - ExpressionActions::Argument argument; - argument.pos = arg.position; - argument.needed_later = arg.used_in_result || arg.num_created_parents != arg.parents.size(); - - if (!argument.needed_later) - free_positions.push(argument.pos); - - arguments.emplace_back(argument); - } - - if (node->type == ActionType::INPUT) - { - /// Argument for input is special. It contains the position from required columns. - ExpressionActions::Argument argument; - argument.pos = expressions->required_columns.size(); - argument.needed_later = !cur.parents.empty(); - arguments.emplace_back(argument); - - expressions->required_columns.push_back({node->result_name, node->result_type}); - } - - expressions->actions.push_back({node, arguments, free_position}); - - for (const auto & parent : cur.parents) - { - auto & parent_data = data[reverse_index[parent]]; - ++parent_data.num_created_children; - - if (parent_data.num_created_children == parent->children.size()) - { - auto & push_stack = parent->type == ActionType::ARRAY_JOIN ? ready_array_joins : ready_nodes; - push_stack.push(parent); - } - } - } - - expressions->result_positions.reserve(index.size()); - - for (const auto & node : index) - { - auto pos = data[reverse_index[node]].position; - - if (pos < 0) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Action for {} was not calculated", node->result_name); - - expressions->result_positions.push_back(pos); - - ColumnWithTypeAndName col{node->column, node->result_type, node->result_name}; - expressions->sample_block.insert(std::move(col)); - } - - return expressions; -} - -ExpressionActionsPtr ActionsDAG::buildExpressions() const -{ - auto cloned = clone(); - #if USE_EMBEDDED_COMPILER - if (compile_expressions) + if (settings.compile_expressions) { - cloned->compileFunctions(); - cloned->removeUnusedActions(); + compileFunctions(); + removeUnusedActions(); } #endif - - auto expressions = cloned->linearizeActions(); - expressions->nodes.swap(cloned->nodes); - - if (max_temporary_columns && expressions->num_columns > max_temporary_columns) - throw Exception(ErrorCodes::TOO_MANY_TEMPORARY_COLUMNS, - "Too many temporary columns: {}. Maximum: {}", - dumpNames(), std::to_string(max_temporary_columns)); - - expressions->max_temporary_non_const_columns = max_temporary_non_const_columns; - expressions->project_input = project_input; - - return expressions; -} - -std::string ActionsDAG::dump() const -{ - return linearizeActions()->dumpActions(); } std::string ActionsDAG::dumpDAG() const diff --git a/src/Interpreters/ExpressionActions.h b/src/Interpreters/ExpressionActions.h index bdc35c8e0f2..da45454fff2 100644 --- a/src/Interpreters/ExpressionActions.h +++ b/src/Interpreters/ExpressionActions.h @@ -175,22 +175,26 @@ public: using Nodes = std::list; + struct ActionsSettings + { + size_t max_temporary_columns = 0; + size_t max_temporary_non_const_columns = 0; + size_t min_count_to_compile_expression = 0; + bool compile_expressions = false; + bool project_input = false; + bool projected_output = false; + }; + private: Nodes nodes; Index index; - size_t max_temporary_columns = 0; - size_t max_temporary_non_const_columns = 0; - size_t min_count_to_compile_expression = 0; - bool compile_expressions = false; + ActionsSettings settings; #if USE_EMBEDDED_COMPILER std::shared_ptr compilation_cache; #endif - bool project_input = false; - bool projected_output = false; - public: ActionsDAG() = default; ActionsDAG(const ActionsDAG &) = delete; @@ -207,7 +211,6 @@ public: Names getNames() const; std::string dumpNames() const; - std::string dump() const; std::string dumpDAG() const; const Node & addInput(std::string name, DataTypePtr type); @@ -231,9 +234,8 @@ public: /// If column is not in index, try to find it in nodes and insert back into index. bool tryRestoreColumn(const std::string & column_name); - void projectInput() { project_input = true; } + void projectInput() { settings.project_input = true; } void removeUnusedActions(const Names & required_names); - ExpressionActionsPtr buildExpressions() const; /// Splits actions into two parts. Returned half may be swapped with ARRAY JOIN. /// Returns nullptr if no actions may be moved before ARRAY JOIN. @@ -241,7 +243,10 @@ public: bool hasArrayJoin() const; bool empty() const; /// If actions only contain inputs. - bool projectedOutput() const { return projected_output; } /// Remove all columns which are not in inputs from block. + + const ActionsSettings & getSettings() const { return settings; } + + void compileExpressions(); ActionsDAGPtr clone() const; @@ -252,10 +257,7 @@ private: ActionsDAGPtr cloneEmpty() const { auto actions = std::make_shared(); - actions->max_temporary_columns = max_temporary_columns; - actions->max_temporary_non_const_columns = max_temporary_non_const_columns; - actions->min_count_to_compile_expression = min_count_to_compile_expression; - actions->compile_expressions = compile_expressions; + actions->settings = settings; #if USE_EMBEDDED_COMPILER actions->compilation_cache = compilation_cache; @@ -263,7 +265,6 @@ private: return actions; } - ExpressionActionsPtr linearizeActions() const; void removeUnusedActions(const std::vector & required_nodes); void removeUnusedActions(); void addAliases(const NamesWithAliases & aliases, std::vector & result_nodes); @@ -311,7 +312,7 @@ private: size_t num_rows; }; - std::list nodes; + ActionsDAGPtr actions_dag; Actions actions; size_t num_columns; @@ -328,12 +329,13 @@ private: public: ~ExpressionActions(); - ExpressionActions() = default; - ExpressionActions(const ExpressionActions &) = delete; - ExpressionActions & operator=(const ExpressionActions &) = delete; + explicit ExpressionActions(ActionsDAGPtr actions_dag_); + ExpressionActions(const ExpressionActions &) = default; + ExpressionActions & operator=(const ExpressionActions &) = default; const Actions & getActions() const { return actions; } - const std::list & getNodes() const { return nodes; } + const std::list & getNodes() const { return actions_dag->getNodes(); } + const ActionsDAG & getActionsDAG() const { return *actions_dag; } /// Adds to the beginning the removal of all extra columns. void projectInput() { project_input = true; } @@ -363,10 +365,13 @@ public: ExpressionActionsPtr clone() const; private: + ExpressionActions() = default; void checkLimits(ExecutionContext & execution_context) const; static void executeAction(const Action & action, ExecutionContext & execution_context, bool dry_run); + + void linearizeActions(); }; @@ -433,7 +438,7 @@ struct ExpressionActionsChain void finalize(const Names & required_output_) override { - if (!actions->projectedOutput()) + if (!actions->getSettings().projected_output) actions->removeUnusedActions(required_output_); } @@ -444,7 +449,7 @@ struct ExpressionActionsChain std::string dump() const override { - return actions->dump(); + return actions->dumpDAG(); } }; diff --git a/src/Interpreters/ExpressionAnalyzer.cpp b/src/Interpreters/ExpressionAnalyzer.cpp index 60975b75196..7a8d7770bfc 100644 --- a/src/Interpreters/ExpressionAnalyzer.cpp +++ b/src/Interpreters/ExpressionAnalyzer.cpp @@ -673,7 +673,7 @@ ActionsDAGPtr SelectQueryExpressionAnalyzer::appendPrewhere( auto tmp_actions_dag = std::make_shared(sourceColumns()); getRootActions(select_query->prewhere(), only_types, tmp_actions_dag); tmp_actions_dag->removeUnusedActions({prewhere_column_name}); - auto tmp_actions = tmp_actions_dag->buildExpressions(); + auto tmp_actions = std::make_shared(tmp_actions_dag); auto required_columns = tmp_actions->getRequiredColumns(); NameSet required_source_columns(required_columns.begin(), required_columns.end()); @@ -794,7 +794,7 @@ bool SelectQueryExpressionAnalyzer::appendGroupBy(ExpressionActionsChain & chain { auto actions_dag = std::make_shared(columns_after_join); getRootActions(child, only_types, actions_dag); - group_by_elements_actions.emplace_back(actions_dag->buildExpressions()); + group_by_elements_actions.emplace_back(std::make_shared(actions_dag)); } } @@ -892,7 +892,7 @@ ActionsDAGPtr SelectQueryExpressionAnalyzer::appendOrderBy(ExpressionActionsChai { auto actions_dag = std::make_shared(columns_after_join); getRootActions(child, only_types, actions_dag); - order_by_elements_actions.emplace_back(actions_dag->buildExpressions()); + order_by_elements_actions.emplace_back(std::make_shared(actions_dag)); } } @@ -1044,7 +1044,7 @@ ActionsDAGPtr ExpressionAnalyzer::getActionsDAG(bool add_aliases, bool project_r ExpressionActionsPtr ExpressionAnalyzer::getActions(bool add_aliases, bool project_result) { - return getActionsDAG(add_aliases, project_result)->buildExpressions(); + return std::make_shared(getActionsDAG(add_aliases, project_result)); } @@ -1053,7 +1053,7 @@ ExpressionActionsPtr ExpressionAnalyzer::getConstActions() auto actions = std::make_shared(NamesAndTypesList()); getRootActions(query, true, actions, true); - return actions->buildExpressions(); + return std::make_shared(actions); } ActionsDAGPtr SelectQueryExpressionAnalyzer::simpleSelectActions() @@ -1143,7 +1143,7 @@ ExpressionAnalysisResult::ExpressionAnalysisResult( Block before_prewhere_sample = source_header; if (sanitizeBlock(before_prewhere_sample)) { - prewhere_info->prewhere_actions->buildExpressions()->execute(before_prewhere_sample); + ExpressionActions(prewhere_info->prewhere_actions).execute(before_prewhere_sample); auto & column_elem = before_prewhere_sample.getByName(query.prewhere()->getColumnName()); /// If the filter column is a constant, record it. if (column_elem.column) @@ -1176,7 +1176,7 @@ ExpressionAnalysisResult::ExpressionAnalysisResult( before_where_sample = source_header; if (sanitizeBlock(before_where_sample)) { - before_where->buildExpressions()->execute(before_where_sample); + ExpressionActions(before_where).execute(before_where_sample); auto & column_elem = before_where_sample.getByName(query.where()->getColumnName()); /// If the filter column is a constant, record it. if (column_elem.column) diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index f7cc0ffb927..40357ec3319 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -522,7 +522,7 @@ Block InterpreterSelectQuery::getSampleBlockImpl() if (analysis_result.prewhere_info) { - analysis_result.prewhere_info->prewhere_actions->buildExpressions()->execute(header); + ExpressionActions(analysis_result.prewhere_info->prewhere_actions).execute(header); header = materializeBlock(header); if (analysis_result.prewhere_info->remove_prewhere_column) header.erase(analysis_result.prewhere_info->prewhere_column_name); @@ -1435,13 +1435,13 @@ void InterpreterSelectQuery::executeFetchColumns( if (prewhere_info) { query_info.prewhere_info = std::make_shared( - prewhere_info->prewhere_actions->buildExpressions(), + std::make_shared(prewhere_info->prewhere_actions), prewhere_info->prewhere_column_name); if (prewhere_info->alias_actions) - query_info.prewhere_info->alias_actions = prewhere_info->alias_actions->buildExpressions(); + query_info.prewhere_info->alias_actions = std::make_shared(prewhere_info->alias_actions); if (prewhere_info->remove_columns_actions) - query_info.prewhere_info->remove_columns_actions = prewhere_info->remove_columns_actions->buildExpressions(); + query_info.prewhere_info->remove_columns_actions = std::make_shared(prewhere_info->remove_columns_actions); query_info.prewhere_info->remove_prewhere_column = prewhere_info->remove_prewhere_column; query_info.prewhere_info->need_filter = prewhere_info->need_filter; diff --git a/src/Processors/QueryPlan/ExpressionStep.cpp b/src/Processors/QueryPlan/ExpressionStep.cpp index 293583ef550..7e35fcb79f5 100644 --- a/src/Processors/QueryPlan/ExpressionStep.cpp +++ b/src/Processors/QueryPlan/ExpressionStep.cpp @@ -44,7 +44,7 @@ static ITransformingStep::Traits getJoinTraits() ExpressionStep::ExpressionStep(const DataStream & input_stream_, ActionsDAGPtr actions_) : ITransformingStep( input_stream_, - Transform::transformHeader(input_stream_.header, actions_->buildExpressions()), + Transform::transformHeader(input_stream_.header, std::make_shared(actions_)), getTraits(actions_)) , actions(std::move(actions_)) { @@ -55,7 +55,7 @@ ExpressionStep::ExpressionStep(const DataStream & input_stream_, ActionsDAGPtr a void ExpressionStep::updateInputStream(DataStream input_stream, bool keep_header) { Block out_header = keep_header ? std::move(output_stream->header) - : Transform::transformHeader(input_stream.header, actions->buildExpressions()); + : Transform::transformHeader(input_stream.header, std::make_shared(actions)); output_stream = createOutputStream( input_stream, std::move(out_header), @@ -67,7 +67,7 @@ void ExpressionStep::updateInputStream(DataStream input_stream, bool keep_header void ExpressionStep::transformPipeline(QueryPipeline & pipeline) { - auto expression = actions->buildExpressions(); + auto expression = std::make_shared(actions); pipeline.addSimpleTransform([&](const Block & header) { return std::make_shared(header, expression); @@ -88,7 +88,7 @@ void ExpressionStep::describeActions(FormatSettings & settings) const String prefix(settings.offset, ' '); bool first = true; - auto expression = actions->buildExpressions(); + auto expression = std::make_shared(actions); for (const auto & action : expression->getActions()) { settings.out << prefix << (first ? "Actions: " diff --git a/src/Processors/QueryPlan/FilterStep.cpp b/src/Processors/QueryPlan/FilterStep.cpp index ce6522cccc8..5a77d01017b 100644 --- a/src/Processors/QueryPlan/FilterStep.cpp +++ b/src/Processors/QueryPlan/FilterStep.cpp @@ -31,7 +31,7 @@ FilterStep::FilterStep( bool remove_filter_column_) : ITransformingStep( input_stream_, - FilterTransform::transformHeader(input_stream_.header, actions_->buildExpressions(), filter_column_name_, remove_filter_column_), + FilterTransform::transformHeader(input_stream_.header, std::make_shared(actions_), filter_column_name_, remove_filter_column_), getTraits(actions_)) , actions(std::move(actions_)) , filter_column_name(std::move(filter_column_name_)) @@ -45,7 +45,7 @@ void FilterStep::updateInputStream(DataStream input_stream, bool keep_header) { Block out_header = std::move(output_stream->header); if (keep_header) - out_header = FilterTransform::transformHeader(input_stream.header, actions->buildExpressions(), filter_column_name, remove_filter_column); + out_header = FilterTransform::transformHeader(input_stream.header, std::make_shared(actions), filter_column_name, remove_filter_column); output_stream = createOutputStream( input_stream, @@ -58,7 +58,7 @@ void FilterStep::updateInputStream(DataStream input_stream, bool keep_header) void FilterStep::transformPipeline(QueryPipeline & pipeline) { - auto expression = actions->buildExpressions(); + auto expression = std::make_shared(actions); pipeline.addSimpleTransform([&](const Block & header, QueryPipeline::StreamType stream_type) { bool on_totals = stream_type == QueryPipeline::StreamType::Totals; @@ -80,7 +80,7 @@ void FilterStep::describeActions(FormatSettings & settings) const settings.out << prefix << "Filter column: " << filter_column_name << '\n'; bool first = true; - auto expression = actions->buildExpressions(); + auto expression = std::make_shared(actions); for (const auto & action : expression->getActions()) { settings.out << prefix << (first ? "Actions: " diff --git a/src/Processors/QueryPlan/TotalsHavingStep.cpp b/src/Processors/QueryPlan/TotalsHavingStep.cpp index fd27f67f70e..07c834ec27e 100644 --- a/src/Processors/QueryPlan/TotalsHavingStep.cpp +++ b/src/Processors/QueryPlan/TotalsHavingStep.cpp @@ -34,7 +34,10 @@ TotalsHavingStep::TotalsHavingStep( bool final_) : ITransformingStep( input_stream_, - TotalsHavingTransform::transformHeader(input_stream_.header, (actions_ ? actions_->buildExpressions() : nullptr), final_), + TotalsHavingTransform::transformHeader( + input_stream_.header, + (actions_ ? std::make_shared(actions_) : nullptr), + final_), getTraits(!filter_column_.empty())) , overflow_row(overflow_row_) , actions(actions_) @@ -48,7 +51,7 @@ TotalsHavingStep::TotalsHavingStep( void TotalsHavingStep::transformPipeline(QueryPipeline & pipeline) { auto totals_having = std::make_shared( - pipeline.getHeader(), overflow_row, (actions ? actions->buildExpressions() : nullptr), + pipeline.getHeader(), overflow_row, (actions ? std::make_shared(actions) : nullptr), filter_column_name, totals_mode, auto_include_threshold, final); pipeline.addTotalsHavingTransform(std::move(totals_having)); @@ -78,7 +81,7 @@ void TotalsHavingStep::describeActions(FormatSettings & settings) const settings.out << prefix << "Mode: " << totalsModeToString(totals_mode, auto_include_threshold) << '\n'; bool first = true; - auto expression = actions->buildExpressions(); + auto expression = std::make_shared(actions); for (const auto & action : expression->getActions()) { settings.out << prefix << (first ? "Actions: " diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 4bfe97f2dc3..e01d8160214 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -438,7 +438,7 @@ void MergeTreeData::checkPartitionKeyAndInitMinMax(const KeyDescription & new_pa /// Add all columns used in the partition key to the min-max index. const NamesAndTypesList & minmax_idx_columns_with_types = new_partition_key.expression->getRequiredColumnsWithTypes(); - minmax_idx_expr = std::make_shared(minmax_idx_columns_with_types)->buildExpressions(); + minmax_idx_expr = std::make_shared(std::make_shared(minmax_idx_columns_with_types)); for (const NameAndTypePair & column : minmax_idx_columns_with_types) { minmax_idx_columns.emplace_back(column.name); diff --git a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp index de7d06a7471..17a5e576b8b 100644 --- a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp +++ b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp @@ -814,7 +814,7 @@ Pipe MergeTreeDataSelectExecutor::readFromParts( if (result_projection) { - auto result_projection_actions = result_projection->buildExpressions(); + auto result_projection_actions = std::make_shared(result_projection); res.addSimpleTransform([&result_projection_actions](const Block & header) { return std::make_shared(header, result_projection_actions); From 9f5386e5603301f17af47f180fdb489355382c03 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 10 Nov 2020 19:52:28 +0300 Subject: [PATCH 091/205] Update ExpressionActions constructur --- src/Interpreters/ExpressionJIT.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Interpreters/ExpressionJIT.cpp b/src/Interpreters/ExpressionJIT.cpp index 05aebbd1f40..c52032807d2 100644 --- a/src/Interpreters/ExpressionJIT.cpp +++ b/src/Interpreters/ExpressionJIT.cpp @@ -875,7 +875,7 @@ void ActionsDAG::compileFunctions() std::vector new_children; auto dag = getCompilableDAG(frame.node, new_children, used_in_result); - if (auto fn = compile(dag, min_count_to_compile_expression, compilation_cache)) + if (auto fn = compile(dag, settings.min_count_to_compile_expression, compilation_cache)) { /// Replace current node to compilable function. From 5b494546222b9fa15fd03462a89a6b35ab640b96 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 10 Nov 2020 20:05:56 +0300 Subject: [PATCH 092/205] Update ExpressionActions constructur --- src/Interpreters/ExpressionActions.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Interpreters/ExpressionActions.cpp b/src/Interpreters/ExpressionActions.cpp index 73bc54a0473..c7d07017fcb 100644 --- a/src/Interpreters/ExpressionActions.cpp +++ b/src/Interpreters/ExpressionActions.cpp @@ -76,7 +76,7 @@ ExpressionActions::ExpressionActions(ActionsDAGPtr actions_dag_) ExpressionActionsPtr ExpressionActions::clone() const { - auto expressions = std::make_shared(*this); + return std::make_shared(*this); } void ExpressionActions::linearizeActions() From 821f6bcb4d5ae9eb905e3b548bf42f3776f162e5 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 10 Nov 2020 22:09:18 +0300 Subject: [PATCH 093/205] Update ExpressionActions constructur --- src/Interpreters/ExpressionActions.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Interpreters/ExpressionActions.h b/src/Interpreters/ExpressionActions.h index da45454fff2..e1134f6b079 100644 --- a/src/Interpreters/ExpressionActions.h +++ b/src/Interpreters/ExpressionActions.h @@ -314,7 +314,7 @@ private: ActionsDAGPtr actions_dag; Actions actions; - size_t num_columns; + size_t num_columns = 0; NamesAndTypesList required_columns; ColumnNumbers result_positions; From 2e6ed2a88012dbb706db12e614443dc831a03e51 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 10 Nov 2020 22:21:43 +0300 Subject: [PATCH 094/205] Higher mmap threshold in debug build --- src/Common/Allocator.cpp | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/src/Common/Allocator.cpp b/src/Common/Allocator.cpp index 9de52a6e3f5..08c275abfc2 100644 --- a/src/Common/Allocator.cpp +++ b/src/Common/Allocator.cpp @@ -14,6 +14,8 @@ * In debug build, use small mmap threshold to reproduce more memory * stomping bugs. Along with ASLR it will hopefully detect more issues than * ASan. The program may fail due to the limit on number of memory mappings. + * + * Not too small to avoid too quick exhaust of memory mappings. */ - __attribute__((__weak__)) extern const size_t MMAP_THRESHOLD = 4096; + __attribute__((__weak__)) extern const size_t MMAP_THRESHOLD = 16384; #endif From d70f9f4175d23ca7bab7ee4b7fac54ccd90a9fb8 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 10 Nov 2020 23:36:38 +0300 Subject: [PATCH 095/205] Move ExecutionContext --- src/Interpreters/ExpressionActions.cpp | 143 +++++++++++++------------ src/Interpreters/ExpressionActions.h | 11 +- 2 files changed, 78 insertions(+), 76 deletions(-) diff --git a/src/Interpreters/ExpressionActions.cpp b/src/Interpreters/ExpressionActions.cpp index c7d07017fcb..73e4bf9fde5 100644 --- a/src/Interpreters/ExpressionActions.cpp +++ b/src/Interpreters/ExpressionActions.cpp @@ -257,19 +257,19 @@ std::string ExpressionActions::Action::toString() const return out.str(); } -void ExpressionActions::checkLimits(ExecutionContext & execution_context) const +void ExpressionActions::checkLimits(const ColumnsWithTypeAndName & columns) const { if (max_temporary_non_const_columns) { size_t non_const_columns = 0; - for (const auto & column : execution_context.columns) + for (const auto & column : columns) if (column.column && !isColumnConst(*column.column)) ++non_const_columns; if (non_const_columns > max_temporary_non_const_columns) { std::stringstream list_of_non_const_columns; - for (const auto & column : execution_context.columns) + for (const auto & column : columns) if (column.column && !isColumnConst(*column.column)) list_of_non_const_columns << "\n" << column.name; @@ -280,74 +280,18 @@ void ExpressionActions::checkLimits(ExecutionContext & execution_context) const } } -void ExpressionActions::execute(Block & block, size_t & num_rows, bool dry_run) const +namespace { - ExecutionContext execution_context + struct ExecutionContext { - .inputs = block.data, - .num_rows = num_rows, + ColumnsWithTypeAndName & inputs; + ColumnsWithTypeAndName columns = {}; + std::vector inputs_pos = {}; + size_t num_rows; }; - - execution_context.inputs_pos.reserve(required_columns.size()); - - for (const auto & column : required_columns) - { - ssize_t pos = -1; - if (block.has(column.name)) - pos = block.getPositionByName(column.name); - execution_context.inputs_pos.push_back(pos); - } - - execution_context.columns.resize(num_columns); - - for (const auto & action : actions) - { - try - { - executeAction(action, execution_context, dry_run); - checkLimits(execution_context); - - //std::cerr << "Action: " << action.toString() << std::endl; - //for (const auto & col : execution_context.columns) - // std::cerr << col.dumpStructure() << std::endl; - } - catch (Exception & e) - { - e.addMessage(fmt::format("while executing '{}'", action.toString())); - throw; - } - } - - if (project_input) - { - block.clear(); - } - else - { - std::sort(execution_context.inputs_pos.rbegin(), execution_context.inputs_pos.rend()); - for (auto input : execution_context.inputs_pos) - if (input >= 0) - block.erase(input); - } - - for (auto pos : result_positions) - if (execution_context.columns[pos].column) - block.insert(execution_context.columns[pos]); - - num_rows = execution_context.num_rows; } -void ExpressionActions::execute(Block & block, bool dry_run) const -{ - size_t num_rows = block.rows(); - - execute(block, num_rows, dry_run); - - if (!block) - block.insert({DataTypeUInt8().createColumnConst(num_rows, 0), std::make_shared(), "_dummy"}); -} - -void ExpressionActions::executeAction(const Action & action, ExecutionContext & execution_context, bool dry_run) +static void executeAction(const ExpressionActions::Action & action, ExecutionContext & execution_context, bool dry_run) { auto & inputs = execution_context.inputs; auto & columns = execution_context.columns; @@ -458,6 +402,73 @@ void ExpressionActions::executeAction(const Action & action, ExecutionContext & } } +void ExpressionActions::execute(Block & block, size_t & num_rows, bool dry_run) const +{ + ExecutionContext execution_context + { + .inputs = block.data, + .num_rows = num_rows, + }; + + execution_context.inputs_pos.reserve(required_columns.size()); + + for (const auto & column : required_columns) + { + ssize_t pos = -1; + if (block.has(column.name)) + pos = block.getPositionByName(column.name); + execution_context.inputs_pos.push_back(pos); + } + + execution_context.columns.resize(num_columns); + + for (const auto & action : actions) + { + try + { + executeAction(action, execution_context, dry_run); + checkLimits(execution_context.columns); + + //std::cerr << "Action: " << action.toString() << std::endl; + //for (const auto & col : execution_context.columns) + // std::cerr << col.dumpStructure() << std::endl; + } + catch (Exception & e) + { + e.addMessage(fmt::format("while executing '{}'", action.toString())); + throw; + } + } + + if (project_input) + { + block.clear(); + } + else + { + std::sort(execution_context.inputs_pos.rbegin(), execution_context.inputs_pos.rend()); + for (auto input : execution_context.inputs_pos) + if (input >= 0) + block.erase(input); + } + + for (auto pos : result_positions) + if (execution_context.columns[pos].column) + block.insert(execution_context.columns[pos]); + + num_rows = execution_context.num_rows; +} + +void ExpressionActions::execute(Block & block, bool dry_run) const +{ + size_t num_rows = block.rows(); + + execute(block, num_rows, dry_run); + + if (!block) + block.insert({DataTypeUInt8().createColumnConst(num_rows, 0), std::make_shared(), "_dummy"}); +} + Names ExpressionActions::getRequiredColumns() const { Names names; diff --git a/src/Interpreters/ExpressionActions.h b/src/Interpreters/ExpressionActions.h index e1134f6b079..493e07b8bdf 100644 --- a/src/Interpreters/ExpressionActions.h +++ b/src/Interpreters/ExpressionActions.h @@ -304,13 +304,6 @@ public: using Actions = std::vector; private: - struct ExecutionContext - { - ColumnsWithTypeAndName & inputs; - ColumnsWithTypeAndName columns = {}; - std::vector inputs_pos = {}; - size_t num_rows; - }; ActionsDAGPtr actions_dag; Actions actions; @@ -367,9 +360,7 @@ public: private: ExpressionActions() = default; - void checkLimits(ExecutionContext & execution_context) const; - - static void executeAction(const Action & action, ExecutionContext & execution_context, bool dry_run); + void checkLimits(const ColumnsWithTypeAndName & columns) const; void linearizeActions(); }; From ab8f13b885baa61e6819edec22d0688e9f371202 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Tue, 10 Nov 2020 22:37:44 +0300 Subject: [PATCH 096/205] Add --stage option for clickhouse-client It is sometimes useful to process queries not up to the Complete stage but intermediate some (i.e. for Distributed queries debugging and similar). --- programs/benchmark/Benchmark.cpp | 13 +------ programs/client/Client.cpp | 8 +++-- src/Core/QueryProcessingStage.cpp | 35 +++++++++++++++++++ src/Core/QueryProcessingStage.h | 8 +++++ src/Core/ya.make | 1 + .../01561_clickhouse_client_stage.reference | 15 ++++++++ .../01561_clickhouse_client_stage.sh | 20 +++++++++++ 7 files changed, 86 insertions(+), 14 deletions(-) create mode 100644 src/Core/QueryProcessingStage.cpp create mode 100644 tests/queries/0_stateless/01561_clickhouse_client_stage.reference create mode 100755 tests/queries/0_stateless/01561_clickhouse_client_stage.sh diff --git a/programs/benchmark/Benchmark.cpp b/programs/benchmark/Benchmark.cpp index 08ded9ed870..a383a06ade0 100644 --- a/programs/benchmark/Benchmark.cpp +++ b/programs/benchmark/Benchmark.cpp @@ -49,7 +49,6 @@ using Ports = std::vector; namespace ErrorCodes { extern const int CANNOT_BLOCK_SIGNAL; - extern const int BAD_ARGUMENTS; extern const int EMPTY_DATA_PASSED; } @@ -103,17 +102,7 @@ public: /// (example: when using stage = 'with_mergeable_state') registerAggregateFunctions(); - if (stage == "complete") - query_processing_stage = QueryProcessingStage::Complete; - else if (stage == "fetch_columns") - query_processing_stage = QueryProcessingStage::FetchColumns; - else if (stage == "with_mergeable_state") - query_processing_stage = QueryProcessingStage::WithMergeableState; - else if (stage == "with_mergeable_state_after_aggregation") - query_processing_stage = QueryProcessingStage::WithMergeableStateAfterAggregation; - else - throw Exception("Unknown query processing stage: " + stage, ErrorCodes::BAD_ARGUMENTS); - + query_processing_stage = QueryProcessingStage::fromString(stage); } void initialize(Poco::Util::Application & self [[maybe_unused]]) override diff --git a/programs/client/Client.cpp b/programs/client/Client.cpp index b940c59dcaa..802c4470ede 100644 --- a/programs/client/Client.cpp +++ b/programs/client/Client.cpp @@ -223,6 +223,7 @@ private: /// We will format query_id in interactive mode in various ways, the default is just to print Query id: ... std::vector> query_id_formats; + QueryProcessingStage::Enum query_processing_stage; void initialize(Poco::Util::Application & self) override { @@ -1441,7 +1442,7 @@ private: connection_parameters.timeouts, query_to_send, context.getCurrentQueryId(), - QueryProcessingStage::Complete, + query_processing_stage, &context.getSettingsRef(), &context.getClientInfo(), true); @@ -1482,7 +1483,7 @@ private: connection_parameters.timeouts, query_to_send, context.getCurrentQueryId(), - QueryProcessingStage::Complete, + query_processing_stage, &context.getSettingsRef(), &context.getClientInfo(), true); @@ -2304,6 +2305,7 @@ public: ("password", po::value()->implicit_value("\n", ""), "password") ("ask-password", "ask-password") ("quota_key", po::value(), "A string to differentiate quotas when the user have keyed quotas configured on server") + ("stage", po::value()->default_value("complete"), "Request query processing up to specified stage: complete,fetch_columns,with_mergeable_state,with_mergeable_state_after_aggregation") ("query_id", po::value(), "query_id") ("query,q", po::value(), "query") ("database,d", po::value(), "database") @@ -2427,6 +2429,8 @@ public: if (options.count("config-file") && options.count("config")) throw Exception("Two or more configuration files referenced in arguments", ErrorCodes::BAD_ARGUMENTS); + query_processing_stage = QueryProcessingStage::fromString(options["stage"].as()); + /// Save received data into the internal config. if (options.count("config-file")) config().setString("config-file", options["config-file"].as()); diff --git a/src/Core/QueryProcessingStage.cpp b/src/Core/QueryProcessingStage.cpp new file mode 100644 index 00000000000..14bde0e548d --- /dev/null +++ b/src/Core/QueryProcessingStage.cpp @@ -0,0 +1,35 @@ +#include +#include + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int BAD_ARGUMENTS; +} + +namespace QueryProcessingStage +{ + + Enum fromString(const std::string & stage_string) + { + Enum stage; + + if (stage_string == "complete") + stage = Complete; + else if (stage_string == "fetch_columns") + stage = FetchColumns; + else if (stage_string == "with_mergeable_state") + stage = WithMergeableState; + else if (stage_string == "with_mergeable_state_after_aggregation") + stage = WithMergeableStateAfterAggregation; + else + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Unknown query processing stage: {}", stage_string); + + return stage; + } + +} + +} diff --git a/src/Core/QueryProcessingStage.h b/src/Core/QueryProcessingStage.h index b1ed4709df2..360cf70ebdd 100644 --- a/src/Core/QueryProcessingStage.h +++ b/src/Core/QueryProcessingStage.h @@ -43,6 +43,14 @@ namespace QueryProcessingStage ? data[stage] : "Unknown stage"; } + + /// This methid is used for the program options, + /// hence it accept under_score notation for stage: + /// - complete + /// - fetch_columns + /// - with_mergeable_state + /// - with_mergeable_state_after_aggregation + Enum fromString(const std::string & stage_string); } } diff --git a/src/Core/ya.make b/src/Core/ya.make index 6bef761a193..d7ba5f8dab9 100644 --- a/src/Core/ya.make +++ b/src/Core/ya.make @@ -30,6 +30,7 @@ SRCS( MySQL/PacketsReplication.cpp NamesAndTypes.cpp PostgreSQLProtocol.cpp + QueryProcessingStage.cpp Settings.cpp SettingsEnums.cpp SettingsFields.cpp diff --git a/tests/queries/0_stateless/01561_clickhouse_client_stage.reference b/tests/queries/0_stateless/01561_clickhouse_client_stage.reference new file mode 100644 index 00000000000..44c39f2a444 --- /dev/null +++ b/tests/queries/0_stateless/01561_clickhouse_client_stage.reference @@ -0,0 +1,15 @@ +execute: default +"foo" +1 +execute: --stage fetch_columns +"dummy" +0 +execute: --stage with_mergeable_state +"1" +1 +execute: --stage with_mergeable_state_after_aggregation +"1" +1 +execute: --stage complete +"foo" +1 diff --git a/tests/queries/0_stateless/01561_clickhouse_client_stage.sh b/tests/queries/0_stateless/01561_clickhouse_client_stage.sh new file mode 100755 index 00000000000..afe3703f4f3 --- /dev/null +++ b/tests/queries/0_stateless/01561_clickhouse_client_stage.sh @@ -0,0 +1,20 @@ +#!/usr/bin/env bash + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +. "$CURDIR"/../shell_config.sh + +function execute_query() +{ + if [ $# -eq 0 ]; then + echo "execute: default" + else + echo "execute: $*" + fi + ${CLICKHOUSE_CLIENT} "$@" --format CSVWithNames -q "SELECT 1 AS foo" +} + +execute_query # default -- complete +execute_query --stage fetch_columns +execute_query --stage with_mergeable_state +execute_query --stage with_mergeable_state_after_aggregation +execute_query --stage complete From e9de5b6ad4e6257124dd1db2f86a6d734c4295ab Mon Sep 17 00:00:00 2001 From: root <@bigo.sg> Date: Wed, 2 Sep 2020 09:13:59 +0000 Subject: [PATCH 097/205] Add StorageEmbeddedRocksdb Engine --- .gitmodules | 4 + CMakeLists.txt | 2 + cmake/find/rocksdb.cmake | 22 ++ contrib/CMakeLists.txt | 7 + contrib/rocksdb | 1 + src/CMakeLists.txt | 12 + src/Core/config_core.h.in | 2 + .../Rocksdb/StorageEmbeddedRocksdb.cpp | 297 ++++++++++++++++++ src/Storages/Rocksdb/StorageEmbeddedRocksdb.h | 67 ++++ src/Storages/registerStorages.cpp | 4 + src/Storages/registerStorages.h | 4 + .../0_stateless/01504_rocksdb.reference | 6 + tests/queries/0_stateless/01504_rocksdb.sql | 22 ++ 13 files changed, 450 insertions(+) create mode 100644 cmake/find/rocksdb.cmake create mode 160000 contrib/rocksdb create mode 100644 src/Storages/Rocksdb/StorageEmbeddedRocksdb.cpp create mode 100644 src/Storages/Rocksdb/StorageEmbeddedRocksdb.h create mode 100644 tests/queries/0_stateless/01504_rocksdb.reference create mode 100644 tests/queries/0_stateless/01504_rocksdb.sql diff --git a/.gitmodules b/.gitmodules index 0e4291eac9e..eaef4cb01ec 100644 --- a/.gitmodules +++ b/.gitmodules @@ -193,3 +193,7 @@ [submodule "contrib/miniselect"] path = contrib/miniselect url = https://github.com/danlark1/miniselect +[submodule "contrib/rocksdb"] + path = contrib/rocksdb + url = https://github.com/facebook/rocksdb + branch = v6.11.4 diff --git a/CMakeLists.txt b/CMakeLists.txt index 182d9989dc2..cababc083fa 100644 --- a/CMakeLists.txt +++ b/CMakeLists.txt @@ -456,6 +456,8 @@ include (cmake/find/simdjson.cmake) include (cmake/find/rapidjson.cmake) include (cmake/find/fastops.cmake) include (cmake/find/odbc.cmake) +include (cmake/find/rocksdb.cmake) + if(NOT USE_INTERNAL_PARQUET_LIBRARY) set (ENABLE_ORC OFF CACHE INTERNAL "") diff --git a/cmake/find/rocksdb.cmake b/cmake/find/rocksdb.cmake new file mode 100644 index 00000000000..8155449df41 --- /dev/null +++ b/cmake/find/rocksdb.cmake @@ -0,0 +1,22 @@ +option(ENABLE_ROCKSDB "Enable ROCKSDB" ${ENABLE_LIBRARIES}) +option(USE_INTERNAL_ROCKSDB_LIBRARY "Set to FALSE to use system ROCKSDB library instead of bundled" ${NOT_UNBUNDLED}) + +if(ENABLE_ROCKSDB) + if (NOT EXISTS "${ClickHouse_SOURCE_DIR}/contrib/rocksdb") + message (WARNING "submodule contrib is missing. to fix try run: \n git submodule update --init --recursive") + set (MISSING_ROCKSDB 1) + endif () + + if (USE_INTERNAL_ROCKSDB_LIBRARY AND NOT MISSING_ROCKSDB) + set (ROCKSDB_INCLUDE_DIR "${ClickHouse_SOURCE_DIR}/contrib/rocksdb/include") + set (ROCKSDB_CORE_INCLUDE_DIR "${ClickHouse_SOURCE_DIR}/contrib/rocksdb/include") + set (ROCKSDB_LIBRARY "rocksdb") + set (USE_INTERNAL_ROCKSDB_LIBRARY 1) + set (USE_ROCKSDB 1) + else() + find_package(ROCKSDB) + endif () + +endif() + +message (STATUS "Using ROCKSDB=${USE_ROCKSDB}: ${ROCKSDB_INCLUDE_DIR} : ${ROCKSDB_LIBRARY}") \ No newline at end of file diff --git a/contrib/CMakeLists.txt b/contrib/CMakeLists.txt index d15715c7b8f..69f6e23d436 100644 --- a/contrib/CMakeLists.txt +++ b/contrib/CMakeLists.txt @@ -320,3 +320,10 @@ if (USE_KRB5) add_subdirectory (cyrus-sasl-cmake) endif() endif() + +if (USE_INTERNAL_ROCKSDB_LIBRARY) + set(WITH_TESTS OFF) + set(WITH_BENCHMARK_TOOLS OFF) + set(WITH_TOOLS OFF) + add_subdirectory (rocksdb) +endif() diff --git a/contrib/rocksdb b/contrib/rocksdb new file mode 160000 index 00000000000..963314ffd68 --- /dev/null +++ b/contrib/rocksdb @@ -0,0 +1 @@ +Subproject commit 963314ffd681596ef2738a95249fe4c1163ef87a diff --git a/src/CMakeLists.txt b/src/CMakeLists.txt index 7b9c05a1adc..1f8731874f7 100644 --- a/src/CMakeLists.txt +++ b/src/CMakeLists.txt @@ -78,6 +78,10 @@ if (USE_AMQPCPP) add_headers_and_sources(dbms Storages/RabbitMQ) endif() +if (USE_ROCKSDB) + add_headers_and_sources(dbms Storages/Rocksdb) +endif() + if (USE_AWS_S3) add_headers_and_sources(dbms Common/S3) add_headers_and_sources(dbms Disks/S3) @@ -294,6 +298,7 @@ if (USE_KRB5) dbms_target_link_libraries(PRIVATE ${KRB5_LIBRARY}) endif() + if(RE2_INCLUDE_DIR) target_include_directories(clickhouse_common_io SYSTEM BEFORE PUBLIC ${RE2_INCLUDE_DIR}) endif() @@ -402,6 +407,13 @@ if (USE_ORC) dbms_target_include_directories(SYSTEM BEFORE PUBLIC ${ORC_INCLUDE_DIR} ${CMAKE_BINARY_DIR}/contrib/orc/c++/include) endif () +if (USE_ROCKSDB) + dbms_target_link_libraries(PUBLIC ${ROCKSDB_LIBRARY}) + dbms_target_include_directories(SYSTEM BEFORE PUBLIC ${ROCKSDB_INCLUDE_DIR}) + # target_link_libraries (clickhouse_common_io PUBLIC ${ROCKSDB_LIBRARY}) + # target_include_directories (clickhouse_common_io SYSTEM BEFORE PUBLIC ${ROCKSDB_INCLUDE_DIR}) +endif() + if (ENABLE_TESTS AND USE_GTEST) macro (grep_gtest_sources BASE_DIR DST_VAR) # Cold match files that are not in tests/ directories diff --git a/src/Core/config_core.h.in b/src/Core/config_core.h.in index bf52bf59975..5dbac2f69bf 100644 --- a/src/Core/config_core.h.in +++ b/src/Core/config_core.h.in @@ -11,3 +11,5 @@ #cmakedefine01 USE_SSL #cmakedefine01 USE_OPENCL #cmakedefine01 USE_LDAP +#cmakedefine01 USE_ROCKSDB + diff --git a/src/Storages/Rocksdb/StorageEmbeddedRocksdb.cpp b/src/Storages/Rocksdb/StorageEmbeddedRocksdb.cpp new file mode 100644 index 00000000000..760fd756b16 --- /dev/null +++ b/src/Storages/Rocksdb/StorageEmbeddedRocksdb.cpp @@ -0,0 +1,297 @@ +#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 BAD_ARGUMENTS; + extern const int SYSTEM_ERROR; + extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; +} + + +static bool extractKeyImpl(const IAST & elem, Strings & res) +{ + const auto * function = elem.as(); + if (!function) + return false; + + if (function->name == "equals" || function->name == "in") + { + const auto & args = function->arguments->as(); + const IAST * value; + + if (args.children.size() != 2) + return false; + + const ASTIdentifier * ident; + if ((ident = args.children.at(0)->as())) + value = args.children.at(1).get(); + else if ((ident = args.children.at(1)->as())) + value = args.children.at(0).get(); + else + return false; + + if (ident->name != "key") + return false; + + if (const auto * literal = value->as()) + { + if (literal->value.getType() == Field::Types::String) + { + res.push_back(literal->value.safeGet()); + return true; + } + else if (literal->value.getType() == Field::Types::Tuple) + { + auto tuple = literal->value.safeGet(); + for (const auto & f : tuple) + { + res.push_back(f.safeGet()); + } + return true; + } + else return false; + } + } + return false; +} + + +/** Retrieve from the query a condition of the form `key = 'key'` or `key in ('xxx_') or `key like 'xxx%'`, from conjunctions in the WHERE clause. + */ +static std::pair extractKey(const ASTPtr & query) +{ + const auto & select = query->as(); + if (!select.where()) + { + return std::make_pair(Strings{}, true); + } + Strings res; + extractKeyImpl(*select.where(), res); + return std::make_pair(res, false); +} + +class EmbeddedRocksdbBlockOutputStream : public IBlockOutputStream +{ +public: + explicit EmbeddedRocksdbBlockOutputStream( + StorageEmbeddedRocksdb & storage_, + const StorageMetadataPtr & metadata_snapshot_) + : storage(storage_) + , metadata_snapshot(metadata_snapshot_) + {} + + Block getHeader() const override { return metadata_snapshot->getSampleBlock(); } + + void write(const Block & block) override + { + metadata_snapshot->check(block, true); + auto rows = block.rows(); + auto key_col = block.getByName("key"); + auto val_col = block.getByName("value"); + + const ColumnString * keys = checkAndGetColumn(key_col.column.get()); + + WriteBufferFromOwnString wb_value; + + for (size_t i = 0; i < rows; i++) + { + StringRef key = keys->getDataAt(i); + val_col.type->serializeBinary(*val_col.column, i, wb_value); + + auto status = storage.rocksdb_ptr->rocksdb->Put(rocksdb::WriteOptions(), key.toString(), wb_value.str()); + if (!status.ok()) + throw Exception("Rocksdb write error: " + status.ToString(), ErrorCodes::SYSTEM_ERROR); + + wb_value.restart(); + } + + rocksdb::Iterator* it = storage.rocksdb_ptr->rocksdb->NewIterator(rocksdb::ReadOptions()); + for (it->SeekToFirst(); it->Valid(); it->Next()) + { + LOG_DEBUG(&Poco::Logger::get("StorageEmbeddedRocksdb"), "Iterator `{}` returns `{}`, {}", + it->key().ToString(), it->value().ToString(), it->key().size()); + } + // Check for any errors found during the scan + assert(it->status().ok()); + delete it; + } + +private: + StorageEmbeddedRocksdb & storage; + StorageMetadataPtr metadata_snapshot; +}; + +StorageEmbeddedRocksdb::StorageEmbeddedRocksdb(const StorageFactory::Arguments & args) + : IStorage(args.table_id) +{ + //must contains two columns, key and value + if (args.columns.size() != 2) + throw Exception("Storage " + getName() + " requires exactly 2 columns", ErrorCodes::BAD_ARGUMENTS); + + if (!args.columns.has("key") || !args.columns.has("value")) + throw Exception("Storage " + getName() + " requires columns are: key and value", ErrorCodes::BAD_ARGUMENTS); + + StorageInMemoryMetadata storage_metadata; + storage_metadata.setColumns(args.columns); + storage_metadata.setConstraints(args.constraints); + setInMemoryMetadata(storage_metadata); + + rocksdb_dir = args.context.getPath() + args.relative_data_path + "/rocksdb"; + if (!args.attach) + { + Poco::File(rocksdb_dir).createDirectories(); + } + initDb(); +} + +void StorageEmbeddedRocksdb::truncate(const ASTPtr &, const StorageMetadataPtr & , const Context &, TableExclusiveLockHolder &) +{ + if (rocksdb_ptr) + { + rocksdb_ptr->shutdown(); + } + Poco::File(rocksdb_dir).remove(true); + Poco::File(rocksdb_dir).createDirectories(); + initDb(); +} + +void StorageEmbeddedRocksdb::initDb() +{ + rocksdb::Options options; + rocksdb::BlockBasedTableOptions table_options; + + rocksdb::DB * db; + options.create_if_missing = true; + options.statistics = rocksdb::CreateDBStatistics(); + auto cache = rocksdb::NewLRUCache(256 << 20); + table_options.block_cache = cache; + options.table_factory.reset(rocksdb::NewBlockBasedTableFactory(table_options)); + + rocksdb::Status status = rocksdb::DB::Open(options, rocksdb_dir, &db); + + if (status != rocksdb::Status::OK()) + throw Exception("Fail to open rocksdb path at: " + rocksdb_dir, ErrorCodes::SYSTEM_ERROR); + rocksdb_ptr = std::make_shared(db); +} + + +Pipe StorageEmbeddedRocksdb::read( + const Names & column_names, + const StorageMetadataPtr & metadata_snapshot, + const SelectQueryInfo & query_info, + const Context & /*context*/, + QueryProcessingStage::Enum /*processed_stage*/, + size_t /*max_block_size*/, + unsigned /*num_streams*/) +{ + metadata_snapshot->check(column_names, getVirtuals(), getStorageID()); + Block sample_block = metadata_snapshot->getSampleBlock(); + + size_t key_pos = 0; + size_t value_pos = 1; + if (sample_block.getByPosition(0).name != "key") + std::swap(key_pos, value_pos); + + MutableColumns columns = sample_block.cloneEmptyColumns(); + + Strings keys; + bool all_scan = false; + std::tie(keys, all_scan) = extractKey(query_info.query); + if (keys.empty() && !all_scan) + throw Exception("StorageEmbeddedRocksdb engine must contain condition like key = 'key' or key in tuple(String) or key like 'xxx%' in WHERE clause or empty WHERE clause for all key value scan.", ErrorCodes::BAD_ARGUMENTS); + + // TODO pipline + if (all_scan) + { + auto it = rocksdb_ptr->rocksdb->NewIterator(rocksdb::ReadOptions()); + for (it->SeekToFirst(); it->Valid(); it->Next()) + { + ReadBufferFromString rvalue(it->value().ToString()); + columns[key_pos]->insert(it->key().ToString()); + sample_block.getByName("value").type->deserializeBinary(*columns[value_pos], rvalue); + } + assert(it->status().ok()); + delete it; + } + else + { + Strings values; + std::vector slices_keys; + for (auto & key : keys) + { + slices_keys.push_back(key); + } + auto statuses = rocksdb_ptr->rocksdb->MultiGet(rocksdb::ReadOptions(), slices_keys, &values); + for (size_t i = 0; i < statuses.size(); ++i) + { + if (statuses[i].ok()) + { + ReadBufferFromString rvalue(values[i]); + columns[key_pos]->insert(keys[i]); + sample_block.getByName("value").type->deserializeBinary(*columns[value_pos], rvalue); + } + } + } + + UInt64 num_rows = columns.at(0)->size(); + Chunk chunk(std::move(columns), num_rows); + return Pipe(std::make_shared(sample_block, std::move(chunk))); +} + +BlockOutputStreamPtr StorageEmbeddedRocksdb::write(const ASTPtr & /*query*/, const StorageMetadataPtr & metadata_snapshot, const Context & /*context*/) +{ + return std::make_shared(*this, metadata_snapshot); +} + +StorageEmbeddedRocksdb::~StorageEmbeddedRocksdb() +{ + if (rocksdb_ptr) + { + rocksdb_ptr->shutdown(); + } +} + + +void registerStorageEmbeddedRocksdb(StorageFactory & factory) +{ + factory.registerStorage("EmbeddedRocksdb", [](const StorageFactory::Arguments & args) + { + if (!args.engine_args.empty()) + throw Exception( + "Engine " + args.engine_name + " doesn't support any arguments (" + toString(args.engine_args.size()) + " given)", + ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); + return StorageEmbeddedRocksdb::create(args); + }); +} + + +} diff --git a/src/Storages/Rocksdb/StorageEmbeddedRocksdb.h b/src/Storages/Rocksdb/StorageEmbeddedRocksdb.h new file mode 100644 index 00000000000..6fbc6c04018 --- /dev/null +++ b/src/Storages/Rocksdb/StorageEmbeddedRocksdb.h @@ -0,0 +1,67 @@ +#pragma once + +#include +#include +#include +#include + +#include + +#include "rocksdb/db.h" +#include "rocksdb/table.h" + +namespace DB +{ + +class Context; + +struct Rocksdb +{ + rocksdb::DB * rocksdb; + explicit Rocksdb(rocksdb::DB * rocksdb_) : rocksdb{rocksdb_} {} + Rocksdb(const Rocksdb &) = delete; + Rocksdb & operator=(const Rocksdb &) = delete; + + void shutdown() + { + if (rocksdb) + { + rocksdb->Close(); + delete rocksdb; + } + } + }; + +class StorageEmbeddedRocksdb final : public ext::shared_ptr_helper, public IStorage +{ + friend struct ext::shared_ptr_helper; + friend class EmbeddedRocksdbBlockOutputStream; +public: + std::string getName() const override { return "EmbeddedRocksdb"; } + + ~StorageEmbeddedRocksdb() override; + + Pipe read( + const Names & column_names, + const StorageMetadataPtr & metadata_snapshot, + const SelectQueryInfo & query_info, + const Context & context, + QueryProcessingStage::Enum processed_stage, + size_t max_block_size, + unsigned num_streams) override; + + BlockOutputStreamPtr write(const ASTPtr & query, const StorageMetadataPtr & /*metadata_snapshot*/, const Context & context) override; + void truncate(const ASTPtr &, const StorageMetadataPtr & metadata_snapshot, const Context &, TableExclusiveLockHolder &) override; + +protected: + StorageEmbeddedRocksdb(const StorageFactory::Arguments & args); + +private: + using RocksdbPtr = std::shared_ptr; + String rocksdb_dir; + RocksdbPtr rocksdb_ptr; + mutable std::shared_mutex rwlock; + + void initDb(); +}; +} diff --git a/src/Storages/registerStorages.cpp b/src/Storages/registerStorages.cpp index 0c7a36c011f..f1e38c4336d 100644 --- a/src/Storages/registerStorages.cpp +++ b/src/Storages/registerStorages.cpp @@ -53,6 +53,10 @@ void registerStorages() #if USE_AMQPCPP registerStorageRabbitMQ(factory); #endif + + #if USE_ROCKSDB + registerStorageEmbeddedRocksdb(factory); + #endif } } diff --git a/src/Storages/registerStorages.h b/src/Storages/registerStorages.h index 592bdcf383f..f74eefe1c00 100644 --- a/src/Storages/registerStorages.h +++ b/src/Storages/registerStorages.h @@ -54,6 +54,10 @@ void registerStorageKafka(StorageFactory & factory); void registerStorageRabbitMQ(StorageFactory & factory); #endif +#if USE_ROCKSDB +void registerStorageEmbeddedRocksdb(StorageFactory & factory); +#endif + void registerStorages(); } diff --git a/tests/queries/0_stateless/01504_rocksdb.reference b/tests/queries/0_stateless/01504_rocksdb.reference new file mode 100644 index 00000000000..2723ca095f1 --- /dev/null +++ b/tests/queries/0_stateless/01504_rocksdb.reference @@ -0,0 +1,6 @@ +1 +1000 +0 435761734006 +1 435761734006 +2 435761734006 +0 diff --git a/tests/queries/0_stateless/01504_rocksdb.sql b/tests/queries/0_stateless/01504_rocksdb.sql new file mode 100644 index 00000000000..84add73b15c --- /dev/null +++ b/tests/queries/0_stateless/01504_rocksdb.sql @@ -0,0 +1,22 @@ +DROP TABLE IF EXISTS test; +create table test (key String, value UInt32) Engine=EmbeddedRocksdb; + +insert into test select '1_1', number from numbers(10000); +select count(1) from test; + +insert into test select concat(toString(number), '_1'), number from numbers(10000); +select sum(value) from test where key in ('1_1', '99_1', '900_1'); + + +DROP TABLE IF EXISTS test; +create table test (key String, value UInt64) Engine=EmbeddedRocksdb; + +insert into test select toString(number%3) as key, sum(number) as value from numbers(1000000) group by key; + +select key, sum(value) from test group by key order by key; + +truncate table test; +select count(1) from test; + +DROP TABLE IF EXISTS test; + From 6f98a9e1c8973230b3f1d73eacb98c440e906abc Mon Sep 17 00:00:00 2001 From: sundy-li <543950155@qq.com> Date: Mon, 21 Sep 2020 21:24:28 +0800 Subject: [PATCH 098/205] fix styles --- src/Storages/Rocksdb/StorageEmbeddedRocksdb.cpp | 3 +-- src/Storages/Rocksdb/StorageEmbeddedRocksdb.h | 3 ++- src/Storages/ya.make | 1 + 3 files changed, 4 insertions(+), 3 deletions(-) diff --git a/src/Storages/Rocksdb/StorageEmbeddedRocksdb.cpp b/src/Storages/Rocksdb/StorageEmbeddedRocksdb.cpp index 760fd756b16..5a99ed716b6 100644 --- a/src/Storages/Rocksdb/StorageEmbeddedRocksdb.cpp +++ b/src/Storages/Rocksdb/StorageEmbeddedRocksdb.cpp @@ -9,7 +9,6 @@ #include #include #include -#include #include #include @@ -85,7 +84,7 @@ static bool extractKeyImpl(const IAST & elem, Strings & res) } -/** Retrieve from the query a condition of the form `key = 'key'` or `key in ('xxx_') or `key like 'xxx%'`, from conjunctions in the WHERE clause. +/** Retrieve from the query a condition of the form `key = 'key'` or `key in ('xxx_'), from conjunctions in the WHERE clause. */ static std::pair extractKey(const ASTPtr & query) { diff --git a/src/Storages/Rocksdb/StorageEmbeddedRocksdb.h b/src/Storages/Rocksdb/StorageEmbeddedRocksdb.h index 6fbc6c04018..546e7e47dc6 100644 --- a/src/Storages/Rocksdb/StorageEmbeddedRocksdb.h +++ b/src/Storages/Rocksdb/StorageEmbeddedRocksdb.h @@ -30,7 +30,8 @@ struct Rocksdb delete rocksdb; } } - }; +}; + class StorageEmbeddedRocksdb final : public ext::shared_ptr_helper, public IStorage { diff --git a/src/Storages/ya.make b/src/Storages/ya.make index e0c6cab602f..c0a4d82c5bf 100644 --- a/src/Storages/ya.make +++ b/src/Storages/ya.make @@ -106,6 +106,7 @@ SRCS( MutationCommands.cpp PartitionCommands.cpp ReadInOrderOptimizer.cpp + Rocksdb/StorageEmbeddedRocksdb.cpp SelectQueryDescription.cpp SetSettings.cpp StorageBuffer.cpp From 20315edae807a5e6a3988669086adc9d7276986a Mon Sep 17 00:00:00 2001 From: sundy-li <543950155@qq.com> Date: Tue, 22 Sep 2020 21:23:07 +0800 Subject: [PATCH 099/205] update cmake --- cmake/find/rocksdb.cmake | 1 + 1 file changed, 1 insertion(+) diff --git a/cmake/find/rocksdb.cmake b/cmake/find/rocksdb.cmake index 8155449df41..c476b86c9d1 100644 --- a/cmake/find/rocksdb.cmake +++ b/cmake/find/rocksdb.cmake @@ -4,6 +4,7 @@ option(USE_INTERNAL_ROCKSDB_LIBRARY "Set to FALSE to use system ROCKSDB library if(ENABLE_ROCKSDB) if (NOT EXISTS "${ClickHouse_SOURCE_DIR}/contrib/rocksdb") message (WARNING "submodule contrib is missing. to fix try run: \n git submodule update --init --recursive") + set (USE_INTERNAL_ROCKSDB_LIBRARY 0) set (MISSING_ROCKSDB 1) endif () From 1fa4906dd2e64be56cdb514567ed9e1f6b03b98b Mon Sep 17 00:00:00 2001 From: sundy-li <543950155@qq.com> Date: Tue, 22 Sep 2020 21:48:12 +0800 Subject: [PATCH 100/205] update cmake && remove useless codes --- cmake/find/rocksdb.cmake | 14 ++++++-------- src/Storages/Rocksdb/StorageEmbeddedRocksdb.cpp | 10 ---------- 2 files changed, 6 insertions(+), 18 deletions(-) diff --git a/cmake/find/rocksdb.cmake b/cmake/find/rocksdb.cmake index c476b86c9d1..1c055f7f54a 100644 --- a/cmake/find/rocksdb.cmake +++ b/cmake/find/rocksdb.cmake @@ -2,22 +2,20 @@ option(ENABLE_ROCKSDB "Enable ROCKSDB" ${ENABLE_LIBRARIES}) option(USE_INTERNAL_ROCKSDB_LIBRARY "Set to FALSE to use system ROCKSDB library instead of bundled" ${NOT_UNBUNDLED}) if(ENABLE_ROCKSDB) - if (NOT EXISTS "${ClickHouse_SOURCE_DIR}/contrib/rocksdb") - message (WARNING "submodule contrib is missing. to fix try run: \n git submodule update --init --recursive") - set (USE_INTERNAL_ROCKSDB_LIBRARY 0) - set (MISSING_ROCKSDB 1) - endif () - if (USE_INTERNAL_ROCKSDB_LIBRARY AND NOT MISSING_ROCKSDB) set (ROCKSDB_INCLUDE_DIR "${ClickHouse_SOURCE_DIR}/contrib/rocksdb/include") set (ROCKSDB_CORE_INCLUDE_DIR "${ClickHouse_SOURCE_DIR}/contrib/rocksdb/include") set (ROCKSDB_LIBRARY "rocksdb") - set (USE_INTERNAL_ROCKSDB_LIBRARY 1) set (USE_ROCKSDB 1) else() find_package(ROCKSDB) endif () - endif() +if (NOT EXISTS "${ClickHouse_SOURCE_DIR}/contrib/rocksdb/CMakeLists.txt") + message (WARNING "submodule contrib is missing. to fix try run: \n git submodule update --init --recursive") + set (USE_INTERNAL_ROCKSDB_LIBRARY 0) + set (MISSING_ROCKSDB 1) +endif () + message (STATUS "Using ROCKSDB=${USE_ROCKSDB}: ${ROCKSDB_INCLUDE_DIR} : ${ROCKSDB_LIBRARY}") \ No newline at end of file diff --git a/src/Storages/Rocksdb/StorageEmbeddedRocksdb.cpp b/src/Storages/Rocksdb/StorageEmbeddedRocksdb.cpp index 5a99ed716b6..a4832ec8edb 100644 --- a/src/Storages/Rocksdb/StorageEmbeddedRocksdb.cpp +++ b/src/Storages/Rocksdb/StorageEmbeddedRocksdb.cpp @@ -132,16 +132,6 @@ public: wb_value.restart(); } - - rocksdb::Iterator* it = storage.rocksdb_ptr->rocksdb->NewIterator(rocksdb::ReadOptions()); - for (it->SeekToFirst(); it->Valid(); it->Next()) - { - LOG_DEBUG(&Poco::Logger::get("StorageEmbeddedRocksdb"), "Iterator `{}` returns `{}`, {}", - it->key().ToString(), it->value().ToString(), it->key().size()); - } - // Check for any errors found during the scan - assert(it->status().ok()); - delete it; } private: From 03ad32a3fa05d04a3a0d5dbe6e0b2a3d5c6b36cd Mon Sep 17 00:00:00 2001 From: sundy-li <543950155@qq.com> Date: Tue, 22 Sep 2020 22:41:26 +0800 Subject: [PATCH 101/205] disable WITH_GFLAGS --- contrib/CMakeLists.txt | 1 + tests/queries/0_stateless/arcadia_skip_list.txt | 1 + 2 files changed, 2 insertions(+) diff --git a/contrib/CMakeLists.txt b/contrib/CMakeLists.txt index 69f6e23d436..dc3a9e05b1d 100644 --- a/contrib/CMakeLists.txt +++ b/contrib/CMakeLists.txt @@ -325,5 +325,6 @@ if (USE_INTERNAL_ROCKSDB_LIBRARY) set(WITH_TESTS OFF) set(WITH_BENCHMARK_TOOLS OFF) set(WITH_TOOLS OFF) + set (WITH_GFLAGS OFF) add_subdirectory (rocksdb) endif() diff --git a/tests/queries/0_stateless/arcadia_skip_list.txt b/tests/queries/0_stateless/arcadia_skip_list.txt index 6420eadfc09..66812f6e16b 100644 --- a/tests/queries/0_stateless/arcadia_skip_list.txt +++ b/tests/queries/0_stateless/arcadia_skip_list.txt @@ -150,6 +150,7 @@ 00609_mv_index_in_in 00510_materizlized_view_and_deduplication_zookeeper 00738_lock_for_inner_table +01504_rocksdb 01515_force_data_skipping_indices 01526_complex_key_dict_direct_layout 01509_dictionary_preallocate From b092ebe40d5bd043d2f577374b9d8981fbe490d5 Mon Sep 17 00:00:00 2001 From: sundy-li <543950155@qq.com> Date: Thu, 1 Oct 2020 18:45:53 +0800 Subject: [PATCH 102/205] add primary key for StorageEmbeddedRocksdb --- .../Rocksdb/StorageEmbeddedRocksdb.cpp | 206 ++++++++++-------- src/Storages/Rocksdb/StorageEmbeddedRocksdb.h | 30 +-- .../0_stateless/01504_rocksdb.reference | 11 +- tests/queries/0_stateless/01504_rocksdb.sql | 33 ++- 4 files changed, 159 insertions(+), 121 deletions(-) diff --git a/src/Storages/Rocksdb/StorageEmbeddedRocksdb.cpp b/src/Storages/Rocksdb/StorageEmbeddedRocksdb.cpp index a4832ec8edb..55a2b863680 100644 --- a/src/Storages/Rocksdb/StorageEmbeddedRocksdb.cpp +++ b/src/Storages/Rocksdb/StorageEmbeddedRocksdb.cpp @@ -8,6 +8,7 @@ #include #include #include +#include #include #include #include @@ -21,9 +22,12 @@ #include #include #include +#include #include #include +#include +#include namespace DB { @@ -36,13 +40,21 @@ namespace ErrorCodes } -static bool extractKeyImpl(const IAST & elem, Strings & res) +// returns keys may be filter by condition +static bool traverseASTFilter(const String & primary_key, const ASTPtr & elem, FieldVector & res) { - const auto * function = elem.as(); + const auto * function = elem->as(); if (!function) return false; - if (function->name == "equals" || function->name == "in") + if (function->name == "and") + { + for (const auto & child : function->arguments->children) + if (traverseASTFilter(primary_key, child, res)) + return true; + return false; + } + else if (function->name == "equals" || function->name == "in") { const auto & args = function->arguments->as(); const IAST * value; @@ -58,22 +70,22 @@ static bool extractKeyImpl(const IAST & elem, Strings & res) else return false; - if (ident->name != "key") + if (ident->name != primary_key) return false; if (const auto * literal = value->as()) { - if (literal->value.getType() == Field::Types::String) + if (function->name == "equals") { - res.push_back(literal->value.safeGet()); + res.push_back(literal->value); return true; } - else if (literal->value.getType() == Field::Types::Tuple) + else if (function->name == "in" && literal->value.getType() == Field::Types::Tuple) { auto tuple = literal->value.safeGet(); for (const auto & f : tuple) { - res.push_back(f.safeGet()); + res.push_back(f); } return true; } @@ -84,20 +96,22 @@ static bool extractKeyImpl(const IAST & elem, Strings & res) } -/** Retrieve from the query a condition of the form `key = 'key'` or `key in ('xxx_'), from conjunctions in the WHERE clause. +/** Retrieve from the query a condition of the form `key = 'key'`, `key in ('xxx_'), from conjunctions in the WHERE clause. + * TODO support key like search */ -static std::pair extractKey(const ASTPtr & query) +static std::pair getFilterKeys(const String & primary_key, const ASTPtr & query) { const auto & select = query->as(); if (!select.where()) { - return std::make_pair(Strings{}, true); + return std::make_pair(FieldVector{}, true); } - Strings res; - extractKeyImpl(*select.where(), res); - return std::make_pair(res, false); + FieldVector res; + auto matched_keys = traverseASTFilter(primary_key, select.where(), res); + return std::make_pair(res, !matched_keys); } + class EmbeddedRocksdbBlockOutputStream : public IBlockOutputStream { public: @@ -114,24 +128,32 @@ public: { metadata_snapshot->check(block, true); auto rows = block.rows(); - auto key_col = block.getByName("key"); - auto val_col = block.getByName("value"); - - const ColumnString * keys = checkAndGetColumn(key_col.column.get()); + WriteBufferFromOwnString wb_key; WriteBufferFromOwnString wb_value; + rocksdb::WriteBatch batch; + auto columns = metadata_snapshot->getColumns(); + for (size_t i = 0; i < rows; i++) { - StringRef key = keys->getDataAt(i); - val_col.type->serializeBinary(*val_col.column, i, wb_value); - - auto status = storage.rocksdb_ptr->rocksdb->Put(rocksdb::WriteOptions(), key.toString(), wb_value.str()); - if (!status.ok()) - throw Exception("Rocksdb write error: " + status.ToString(), ErrorCodes::SYSTEM_ERROR); - + wb_key.restart(); wb_value.restart(); + + for (const auto & col : columns) + { + const auto & type = block.getByName(col.name).type; + const auto & column = block.getByName(col.name).column; + if (col.name == storage.primary_key) + type->serializeBinary(*column, i, wb_key); + else + type->serializeBinary(*column, i, wb_value); + } + batch.Put(wb_key.str(), wb_value.str()); } + auto status = storage.rocksdb_ptr->Write(rocksdb::WriteOptions(), &batch); + if (!status.ok()) + throw Exception("Rocksdb write error: " + status.ToString(), ErrorCodes::SYSTEM_ERROR); } private: @@ -139,23 +161,17 @@ private: StorageMetadataPtr metadata_snapshot; }; -StorageEmbeddedRocksdb::StorageEmbeddedRocksdb(const StorageFactory::Arguments & args) - : IStorage(args.table_id) +StorageEmbeddedRocksdb::StorageEmbeddedRocksdb(const StorageID & table_id_, + const String & relative_data_path_, + const StorageInMemoryMetadata & metadata_, + bool attach, + Context & context_, + const String & primary_key_) + : IStorage(table_id_), primary_key{primary_key_} { - //must contains two columns, key and value - if (args.columns.size() != 2) - throw Exception("Storage " + getName() + " requires exactly 2 columns", ErrorCodes::BAD_ARGUMENTS); - - if (!args.columns.has("key") || !args.columns.has("value")) - throw Exception("Storage " + getName() + " requires columns are: key and value", ErrorCodes::BAD_ARGUMENTS); - - StorageInMemoryMetadata storage_metadata; - storage_metadata.setColumns(args.columns); - storage_metadata.setConstraints(args.constraints); - setInMemoryMetadata(storage_metadata); - - rocksdb_dir = args.context.getPath() + args.relative_data_path + "/rocksdb"; - if (!args.attach) + setInMemoryMetadata(metadata_); + rocksdb_dir = context_.getPath() + relative_data_path_ + "/rocksdb"; + if (!attach) { Poco::File(rocksdb_dir).createDirectories(); } @@ -164,10 +180,7 @@ StorageEmbeddedRocksdb::StorageEmbeddedRocksdb(const StorageFactory::Arguments & void StorageEmbeddedRocksdb::truncate(const ASTPtr &, const StorageMetadataPtr & , const Context &, TableExclusiveLockHolder &) { - if (rocksdb_ptr) - { - rocksdb_ptr->shutdown(); - } + rocksdb_ptr->Close(); Poco::File(rocksdb_dir).remove(true); Poco::File(rocksdb_dir).createDirectories(); initDb(); @@ -176,20 +189,13 @@ void StorageEmbeddedRocksdb::truncate(const ASTPtr &, const StorageMetadataPtr & void StorageEmbeddedRocksdb::initDb() { rocksdb::Options options; - rocksdb::BlockBasedTableOptions table_options; - rocksdb::DB * db; options.create_if_missing = true; - options.statistics = rocksdb::CreateDBStatistics(); - auto cache = rocksdb::NewLRUCache(256 << 20); - table_options.block_cache = cache; - options.table_factory.reset(rocksdb::NewBlockBasedTableFactory(table_options)); - rocksdb::Status status = rocksdb::DB::Open(options, rocksdb_dir, &db); if (status != rocksdb::Status::OK()) throw Exception("Fail to open rocksdb path at: " + rocksdb_dir, ErrorCodes::SYSTEM_ERROR); - rocksdb_ptr = std::make_shared(db); + rocksdb_ptr = std::unique_ptr(db); } @@ -204,49 +210,61 @@ Pipe StorageEmbeddedRocksdb::read( { metadata_snapshot->check(column_names, getVirtuals(), getStorageID()); Block sample_block = metadata_snapshot->getSampleBlock(); - - size_t key_pos = 0; - size_t value_pos = 1; - if (sample_block.getByPosition(0).name != "key") - std::swap(key_pos, value_pos); - MutableColumns columns = sample_block.cloneEmptyColumns(); - Strings keys; + FieldVector keys; bool all_scan = false; - std::tie(keys, all_scan) = extractKey(query_info.query); - if (keys.empty() && !all_scan) - throw Exception("StorageEmbeddedRocksdb engine must contain condition like key = 'key' or key in tuple(String) or key like 'xxx%' in WHERE clause or empty WHERE clause for all key value scan.", ErrorCodes::BAD_ARGUMENTS); + std::tie(keys, all_scan) = getFilterKeys(primary_key, query_info.query); // TODO pipline if (all_scan) { - auto it = rocksdb_ptr->rocksdb->NewIterator(rocksdb::ReadOptions()); + auto it = std::unique_ptr(rocksdb_ptr->NewIterator(rocksdb::ReadOptions())); for (it->SeekToFirst(); it->Valid(); it->Next()) { - ReadBufferFromString rvalue(it->value().ToString()); - columns[key_pos]->insert(it->key().ToString()); - sample_block.getByName("value").type->deserializeBinary(*columns[value_pos], rvalue); + ReadBufferFromString key_buffer(it->key()); + ReadBufferFromString value_buffer(it->value()); + + for (const auto & item : ext::enumerate(sample_block.getColumnsWithTypeAndName())) + { + if (item.second.name == primary_key) + item.second.type->deserializeBinary(*columns[item.first], key_buffer); + else + item.second.type->deserializeBinary(*columns[item.first], value_buffer); + } } assert(it->status().ok()); - delete it; } else { - Strings values; std::vector slices_keys; - for (auto & key : keys) + std::vector values; + + WriteBufferFromOwnString wb; + UInt64 offset = 0; + for (const auto & key : keys) { - slices_keys.push_back(key); + sample_block.getByName(primary_key).type->serializeBinary(key, wb); + auto str_ref = wb.stringRef(); + slices_keys.emplace_back(str_ref.data + offset, str_ref.size - offset); + offset = str_ref.size; } - auto statuses = rocksdb_ptr->rocksdb->MultiGet(rocksdb::ReadOptions(), slices_keys, &values); + + auto statuses = rocksdb_ptr->MultiGet(rocksdb::ReadOptions(), slices_keys, &values); for (size_t i = 0; i < statuses.size(); ++i) { if (statuses[i].ok()) { - ReadBufferFromString rvalue(values[i]); - columns[key_pos]->insert(keys[i]); - sample_block.getByName("value").type->deserializeBinary(*columns[value_pos], rvalue); + ReadBufferFromString key_buffer(slices_keys[i]); + ReadBufferFromString value_buffer(values[i]); + + for (const auto & item : ext::enumerate(sample_block.getColumnsWithTypeAndName())) + { + if (item.second.name == primary_key) + item.second.type->deserializeBinary(*columns[item.first], key_buffer); + else + item.second.type->deserializeBinary(*columns[item.first], value_buffer); + } } } } @@ -263,23 +281,41 @@ BlockOutputStreamPtr StorageEmbeddedRocksdb::write(const ASTPtr & /*query*/, con StorageEmbeddedRocksdb::~StorageEmbeddedRocksdb() { - if (rocksdb_ptr) +} + + +static StoragePtr create(const StorageFactory::Arguments & args) +{ + // TODO RocksdbSettings + if (!args.engine_args.empty()) + throw Exception( + "Engine " + args.engine_name + " doesn't support any arguments (" + toString(args.engine_args.size()) + " given)", + ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); + + StorageInMemoryMetadata metadata; + metadata.setColumns(args.columns); + metadata.setConstraints(args.constraints); + + if (!args.storage_def->primary_key) + throw Exception("StorageEmbeddedRocksdb must require one primary key", ErrorCodes::BAD_ARGUMENTS); + + metadata.primary_key = KeyDescription::getKeyFromAST(args.storage_def->primary_key->ptr(), metadata.columns, args.context); + auto primary_key_names = metadata.getColumnsRequiredForPrimaryKey(); + if (primary_key_names.size() != 1) { - rocksdb_ptr->shutdown(); + throw Exception("StorageEmbeddedRocksdb must require one primary key", ErrorCodes::BAD_ARGUMENTS); } + return StorageEmbeddedRocksdb::create(args.table_id, args.relative_data_path, metadata, args.attach, args.context, primary_key_names[0]); } void registerStorageEmbeddedRocksdb(StorageFactory & factory) { - factory.registerStorage("EmbeddedRocksdb", [](const StorageFactory::Arguments & args) - { - if (!args.engine_args.empty()) - throw Exception( - "Engine " + args.engine_name + " doesn't support any arguments (" + toString(args.engine_args.size()) + " given)", - ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); - return StorageEmbeddedRocksdb::create(args); - }); + StorageFactory::StorageFeatures features{ + .supports_sort_order = true, + }; + + factory.registerStorage("EmbeddedRocksdb", create, features); } diff --git a/src/Storages/Rocksdb/StorageEmbeddedRocksdb.h b/src/Storages/Rocksdb/StorageEmbeddedRocksdb.h index 546e7e47dc6..8e050867929 100644 --- a/src/Storages/Rocksdb/StorageEmbeddedRocksdb.h +++ b/src/Storages/Rocksdb/StorageEmbeddedRocksdb.h @@ -15,24 +15,6 @@ namespace DB class Context; -struct Rocksdb -{ - rocksdb::DB * rocksdb; - explicit Rocksdb(rocksdb::DB * rocksdb_) : rocksdb{rocksdb_} {} - Rocksdb(const Rocksdb &) = delete; - Rocksdb & operator=(const Rocksdb &) = delete; - - void shutdown() - { - if (rocksdb) - { - rocksdb->Close(); - delete rocksdb; - } - } -}; - - class StorageEmbeddedRocksdb final : public ext::shared_ptr_helper, public IStorage { friend struct ext::shared_ptr_helper; @@ -55,12 +37,18 @@ public: void truncate(const ASTPtr &, const StorageMetadataPtr & metadata_snapshot, const Context &, TableExclusiveLockHolder &) override; protected: - StorageEmbeddedRocksdb(const StorageFactory::Arguments & args); + StorageEmbeddedRocksdb(const StorageID & table_id_, + const String & relative_data_path_, + const StorageInMemoryMetadata & metadata, + bool attach, + Context & context_, + const String & primary_key_); private: - using RocksdbPtr = std::shared_ptr; - String rocksdb_dir; + const String primary_key; + using RocksdbPtr = std::unique_ptr; RocksdbPtr rocksdb_ptr; + String rocksdb_dir; mutable std::shared_mutex rwlock; void initDb(); diff --git a/tests/queries/0_stateless/01504_rocksdb.reference b/tests/queries/0_stateless/01504_rocksdb.reference index 2723ca095f1..ae092fcbc30 100644 --- a/tests/queries/0_stateless/01504_rocksdb.reference +++ b/tests/queries/0_stateless/01504_rocksdb.reference @@ -1,6 +1,7 @@ 1 -1000 -0 435761734006 -1 435761734006 -2 435761734006 -0 +1 +0 0 0 0 0 +1 +1 +1 +1 diff --git a/tests/queries/0_stateless/01504_rocksdb.sql b/tests/queries/0_stateless/01504_rocksdb.sql index 84add73b15c..f78f6e2f658 100644 --- a/tests/queries/0_stateless/01504_rocksdb.sql +++ b/tests/queries/0_stateless/01504_rocksdb.sql @@ -1,22 +1,35 @@ DROP TABLE IF EXISTS test; -create table test (key String, value UInt32) Engine=EmbeddedRocksdb; +CREATE TABLE test (key String, value UInt32) Engine=EmbeddedRocksdb primary key(key); -insert into test select '1_1', number from numbers(10000); -select count(1) from test; +INSERT INTO test SELECT '1_1', number FROM numbers(10000); +SELECT count(1) == 1 FROM test; -insert into test select concat(toString(number), '_1'), number from numbers(10000); -select sum(value) from test where key in ('1_1', '99_1', '900_1'); +INSERT INTO test SELECT concat(toString(number), '_1'), number FROM numbers(10000); +SELECT SUM(value) == 1 + 99 + 900 FROM test WHERE key in ('1_1', '99_1', '900_1'); DROP TABLE IF EXISTS test; -create table test (key String, value UInt64) Engine=EmbeddedRocksdb; +DROP TABLE IF EXISTS test_memory; -insert into test select toString(number%3) as key, sum(number) as value from numbers(1000000) group by key; +CREATE TABLE test (k UInt32, value UInt64, dummy Tuple(UInt32, Float64), bm AggregateFunction(groupBitmap, UInt64)) Engine=EmbeddedRocksdb primary key(k); +CREATE TABLE test_memory AS test Engine = Memory; -select key, sum(value) from test group by key order by key; +INSERT INTO test SELECT number % 77 AS k, SUM(number) AS value, (1, 1.2), bitmapBuild(groupArray(number)) FROM numbers(10000000) group by k; -truncate table test; -select count(1) from test; +INSERT INTO test_memory SELECT number % 77 AS k, SUM(number) AS value, (1, 1.2), bitmapBuild(groupArray(number)) FROM numbers(10000000) group by k; + + +SELECT A.a - B.a, A.b - B.b, A.c - B.c, A.d - B.d, A.e - B.e FROM ( SELECT 0 AS a, groupBitmapMerge(bm) AS b , SUM(k) AS c, SUM(value) AS d, SUM(dummy.1) AS e FROM test) A ANY LEFT JOIN (SELECT 0 AS a, groupBitmapMerge(bm) AS b , SUM(k) AS c, SUM(value) AS d, SUM(dummy.1) AS e FROM test_memory) B USING a ORDER BY a; + + +SET max_rows_to_read = 2; +SELECT dummy == (1,1.2) FROM test WHERE k IN (1, 3); +SELECT k == 4 FROM test WHERE k = 4; +SELECT k, value FROM test WHERE k = 0 OR value > 0; -- { serverError 158 } + +TRUNCATE TABLE test; +SELECT 0 == count(1) FROM test; DROP TABLE IF EXISTS test; +DROP TABLE IF EXISTS test_memory; From 8cb3ec5aea0a120187ecc5cf8aa546e4abb812ac Mon Sep 17 00:00:00 2001 From: sundy-li <543950155@qq.com> Date: Thu, 1 Oct 2020 18:59:51 +0800 Subject: [PATCH 103/205] add docs --- .../integrations/embedded-rocksdb.md | 44 +++++++++++++++++++ 1 file changed, 44 insertions(+) create mode 100644 docs/en/engines/table-engines/integrations/embedded-rocksdb.md diff --git a/docs/en/engines/table-engines/integrations/embedded-rocksdb.md b/docs/en/engines/table-engines/integrations/embedded-rocksdb.md new file mode 100644 index 00000000000..0bd4123c0cf --- /dev/null +++ b/docs/en/engines/table-engines/integrations/embedded-rocksdb.md @@ -0,0 +1,44 @@ +--- +toc_priority: 6 +toc_title: EmbeddedRocksdb +--- + +# EmbeddedRocksdb Engine {#EmbeddedRocksdb-engine} + +This engine allows integrating ClickHouse with [rocksdb](http://rocksdb.org/). + +`EmbeddedRocksdb` lets you: + +## Creating a Table {#table_engine-EmbeddedRocksdb-creating-a-table} + +``` sql +CREATE TABLE [IF NOT EXISTS] [db.]table_name [ON CLUSTER cluster] +( + name1 [type1] [DEFAULT|MATERIALIZED|ALIAS expr1], + name2 [type2] [DEFAULT|MATERIALIZED|ALIAS expr2], + ... +) ENGINE = EmbeddedRocksdb PRIMARY KEY(primary_key_name) +``` + +Required parameters: + +- `primary_key_name` – any column name in the column list. + +Example: + +``` sql +CREATE TABLE test +( + `key` String, + `v1` UInt32, + `v2` String, + `v3` Float32, +) +ENGINE = EmbeddedRocksdb +PRIMARY KEY key +``` + +## Description {#description} + +- `primary key` must be specified, only support one primary key. The primary key will serializeBinary as rocksdb key. +- Columns other than the primary key will be serializeBinary as rocksdb value in corresponding order. From a5fc4d69a56ac6733d4d6fd2f5d4dc1abf24c369 Mon Sep 17 00:00:00 2001 From: sundy-li <543950155@qq.com> Date: Thu, 1 Oct 2020 19:00:03 +0800 Subject: [PATCH 104/205] add docs --- docs/en/engines/table-engines/integrations/embedded-rocksdb.md | 1 + 1 file changed, 1 insertion(+) diff --git a/docs/en/engines/table-engines/integrations/embedded-rocksdb.md b/docs/en/engines/table-engines/integrations/embedded-rocksdb.md index 0bd4123c0cf..88a95ae7de7 100644 --- a/docs/en/engines/table-engines/integrations/embedded-rocksdb.md +++ b/docs/en/engines/table-engines/integrations/embedded-rocksdb.md @@ -42,3 +42,4 @@ PRIMARY KEY key - `primary key` must be specified, only support one primary key. The primary key will serializeBinary as rocksdb key. - Columns other than the primary key will be serializeBinary as rocksdb value in corresponding order. +- Queries with key `equals` or `in` filtering will be optimized to multi keys look up from rocksdb. From 84dab52e458e3ccd94d920d416e6ed675260e70a Mon Sep 17 00:00:00 2001 From: sundy-li <543950155@qq.com> Date: Thu, 1 Oct 2020 22:14:52 +0800 Subject: [PATCH 105/205] skip rocksdb test in fasttest --- docker/test/fasttest/run.sh | 3 +++ .../Rocksdb/StorageEmbeddedRocksdb.cpp | 20 ++++++++----------- src/Storages/Rocksdb/StorageEmbeddedRocksdb.h | 2 -- 3 files changed, 11 insertions(+), 14 deletions(-) diff --git a/docker/test/fasttest/run.sh b/docker/test/fasttest/run.sh index 36aa8baf252..d6f656754ba 100755 --- a/docker/test/fasttest/run.sh +++ b/docker/test/fasttest/run.sh @@ -274,6 +274,9 @@ TESTS_TO_SKIP=( 00646_url_engine 00974_query_profiler + # Rocksdb is not enabled by default + 01504_rocksdb + # Look at DistributedFilesToInsert, so cannot run in parallel. 01460_DistributedFilesToInsert diff --git a/src/Storages/Rocksdb/StorageEmbeddedRocksdb.cpp b/src/Storages/Rocksdb/StorageEmbeddedRocksdb.cpp index 55a2b863680..72602de68e9 100644 --- a/src/Storages/Rocksdb/StorageEmbeddedRocksdb.cpp +++ b/src/Storages/Rocksdb/StorageEmbeddedRocksdb.cpp @@ -225,12 +225,12 @@ Pipe StorageEmbeddedRocksdb::read( ReadBufferFromString key_buffer(it->key()); ReadBufferFromString value_buffer(it->value()); - for (const auto & item : ext::enumerate(sample_block.getColumnsWithTypeAndName())) + for (const auto [idx, column_type] : ext::enumerate(sample_block.getColumnsWithTypeAndName())) { - if (item.second.name == primary_key) - item.second.type->deserializeBinary(*columns[item.first], key_buffer); + if (column_type.name == primary_key) + column_type.type->deserializeBinary(*columns[idx], key_buffer); else - item.second.type->deserializeBinary(*columns[item.first], value_buffer); + column_type.type->deserializeBinary(*columns[idx], value_buffer); } } assert(it->status().ok()); @@ -258,12 +258,12 @@ Pipe StorageEmbeddedRocksdb::read( ReadBufferFromString key_buffer(slices_keys[i]); ReadBufferFromString value_buffer(values[i]); - for (const auto & item : ext::enumerate(sample_block.getColumnsWithTypeAndName())) + for (const auto [idx, column_type] : ext::enumerate(sample_block.getColumnsWithTypeAndName())) { - if (item.second.name == primary_key) - item.second.type->deserializeBinary(*columns[item.first], key_buffer); + if (column_type.name == primary_key) + column_type.type->deserializeBinary(*columns[idx], key_buffer); else - item.second.type->deserializeBinary(*columns[item.first], value_buffer); + column_type.type->deserializeBinary(*columns[idx], value_buffer); } } } @@ -279,10 +279,6 @@ BlockOutputStreamPtr StorageEmbeddedRocksdb::write(const ASTPtr & /*query*/, con return std::make_shared(*this, metadata_snapshot); } -StorageEmbeddedRocksdb::~StorageEmbeddedRocksdb() -{ -} - static StoragePtr create(const StorageFactory::Arguments & args) { diff --git a/src/Storages/Rocksdb/StorageEmbeddedRocksdb.h b/src/Storages/Rocksdb/StorageEmbeddedRocksdb.h index 8e050867929..7c8d1f9c47e 100644 --- a/src/Storages/Rocksdb/StorageEmbeddedRocksdb.h +++ b/src/Storages/Rocksdb/StorageEmbeddedRocksdb.h @@ -22,8 +22,6 @@ class StorageEmbeddedRocksdb final : public ext::shared_ptr_helper Date: Fri, 2 Oct 2020 10:26:59 +0800 Subject: [PATCH 106/205] USE PORTABLE to avoid Illegal instruction, add SANITIZE for rocksdb cmake build options --- contrib/CMakeLists.txt | 13 ++++++++++++- 1 file changed, 12 insertions(+), 1 deletion(-) diff --git a/contrib/CMakeLists.txt b/contrib/CMakeLists.txt index dc3a9e05b1d..02355f5db5f 100644 --- a/contrib/CMakeLists.txt +++ b/contrib/CMakeLists.txt @@ -325,6 +325,17 @@ if (USE_INTERNAL_ROCKSDB_LIBRARY) set(WITH_TESTS OFF) set(WITH_BENCHMARK_TOOLS OFF) set(WITH_TOOLS OFF) - set (WITH_GFLAGS OFF) + set(WITH_GFLAGS OFF) + set(PORTABLE ON) + set(FORCE_SSE42 ON) + + if (SANITIZE STREQUAL "undefined") + set(WITH_UBSAN ON) + elseif (SANITIZE STREQUAL "address") + set(WITH_ASAN ON) + elseif (SANITIZE STREQUAL "thread") + set(WITH_TSAN ON) + endif() + add_subdirectory (rocksdb) endif() From c7617173c4d632cb63c5b8a3ee607392a3084d11 Mon Sep 17 00:00:00 2001 From: sundy-li <543950155@qq.com> Date: Sun, 4 Oct 2020 10:24:08 +0800 Subject: [PATCH 107/205] add sync_file_range for glibc-compatibility --- .../musl/sync_file_range.c | 21 ++++++++++ cmake/find/rocksdb.cmake | 41 +++++++++++++------ docker/packager/unbundled/Dockerfile | 1 + src/CMakeLists.txt | 2 - src/Storages/Rocksdb/StorageEmbeddedRocksdb.h | 1 - 5 files changed, 51 insertions(+), 15 deletions(-) create mode 100644 base/glibc-compatibility/musl/sync_file_range.c diff --git a/base/glibc-compatibility/musl/sync_file_range.c b/base/glibc-compatibility/musl/sync_file_range.c new file mode 100644 index 00000000000..610b11c8c96 --- /dev/null +++ b/base/glibc-compatibility/musl/sync_file_range.c @@ -0,0 +1,21 @@ +#define _GNU_SOURCE +#include +#include +#include "syscall.h" + +// works same in x86_64 && aarch64 +#define __SYSCALL_LL_E(x) (x) +#define __SYSCALL_LL_O(x) (x) + +int sync_file_range(int fd, off_t pos, off_t len, unsigned flags) +{ +#if defined(SYS_sync_file_range2) + return syscall(SYS_sync_file_range2, fd, flags, + __SYSCALL_LL_E(pos), __SYSCALL_LL_E(len)); +#elif defined(SYS_sync_file_range) + return __syscall(SYS_sync_file_range, fd, + __SYSCALL_LL_O(pos), __SYSCALL_LL_E(len), flags); +#else + return __syscall_ret(-ENOSYS); +#endif +} \ No newline at end of file diff --git a/cmake/find/rocksdb.cmake b/cmake/find/rocksdb.cmake index 1c055f7f54a..d53febee762 100644 --- a/cmake/find/rocksdb.cmake +++ b/cmake/find/rocksdb.cmake @@ -1,21 +1,38 @@ option(ENABLE_ROCKSDB "Enable ROCKSDB" ${ENABLE_LIBRARIES}) -option(USE_INTERNAL_ROCKSDB_LIBRARY "Set to FALSE to use system ROCKSDB library instead of bundled" ${NOT_UNBUNDLED}) -if(ENABLE_ROCKSDB) - if (USE_INTERNAL_ROCKSDB_LIBRARY AND NOT MISSING_ROCKSDB) - set (ROCKSDB_INCLUDE_DIR "${ClickHouse_SOURCE_DIR}/contrib/rocksdb/include") - set (ROCKSDB_CORE_INCLUDE_DIR "${ClickHouse_SOURCE_DIR}/contrib/rocksdb/include") - set (ROCKSDB_LIBRARY "rocksdb") - set (USE_ROCKSDB 1) - else() - find_package(ROCKSDB) - endif () +if (NOT ENABLE_ROCKSDB) + if (USE_INTERNAL_ROCKSDB_LIBRARY) + message (${RECONFIGURE_MESSAGE_LEVEL} "Can't use internal rocksdb library with ENABLE_ROCKSDB=OFF") + endif() + return() endif() +option(USE_INTERNAL_ROCKSDB_LIBRARY "Set to FALSE to use system ROCKSDB library instead of bundled" ${NOT_UNBUNDLED}) + if (NOT EXISTS "${ClickHouse_SOURCE_DIR}/contrib/rocksdb/CMakeLists.txt") - message (WARNING "submodule contrib is missing. to fix try run: \n git submodule update --init --recursive") - set (USE_INTERNAL_ROCKSDB_LIBRARY 0) + if (USE_INTERNAL_ROCKSDB_LIBRARY) + message (WARNING "submodule contrib is missing. to fix try run: \n git submodule update --init --recursive") + message(${RECONFIGURE_MESSAGE_LEVEL} "cannot find internal rocksdb") + set (USE_INTERNAL_ROCKSDB_LIBRARY 0) + endif() set (MISSING_ROCKSDB 1) endif () +if (NOT USE_INTERNAL_ROCKSDB_LIBRARY) + find_library (ROCKSDB_LIBRARY rocksdb) + find_path (ROCKSDB_INCLUDE_DIR NAMES rocksdb/db.h PATHS ${ROCKSDB_INCLUDE_PATHS}) + if (NOT ROCKSDB_LIBRARY OR NOT ROCKSDB_INCLUDE_DIR) + message (${RECONFIGURE_MESSAGE_LEVEL} "Can't find system rocksdb library") + endif() +endif () + +if (NOT ROCKSDB_LIBRARY AND NOT MISSING_ROCKSDB) + set (USE_INTERNAL_ROCKSDB_LIBRARY 1) + + set (ROCKSDB_INCLUDE_DIR "${ClickHouse_SOURCE_DIR}/contrib/rocksdb/include") + set (ROCKSDB_CORE_INCLUDE_DIR "${ClickHouse_SOURCE_DIR}/contrib/rocksdb/include") + set (ROCKSDB_LIBRARY "rocksdb") + set (USE_ROCKSDB 1) +endif () + message (STATUS "Using ROCKSDB=${USE_ROCKSDB}: ${ROCKSDB_INCLUDE_DIR} : ${ROCKSDB_LIBRARY}") \ No newline at end of file diff --git a/docker/packager/unbundled/Dockerfile b/docker/packager/unbundled/Dockerfile index 50671011a23..698a7e2621b 100644 --- a/docker/packager/unbundled/Dockerfile +++ b/docker/packager/unbundled/Dockerfile @@ -64,6 +64,7 @@ RUN apt-get update \ libbz2-dev \ libavro-dev \ libfarmhash-dev \ + librocksdb-dev \ libmysqlclient-dev \ --yes --no-install-recommends diff --git a/src/CMakeLists.txt b/src/CMakeLists.txt index 1f8731874f7..4acb8090b6b 100644 --- a/src/CMakeLists.txt +++ b/src/CMakeLists.txt @@ -410,8 +410,6 @@ endif () if (USE_ROCKSDB) dbms_target_link_libraries(PUBLIC ${ROCKSDB_LIBRARY}) dbms_target_include_directories(SYSTEM BEFORE PUBLIC ${ROCKSDB_INCLUDE_DIR}) - # target_link_libraries (clickhouse_common_io PUBLIC ${ROCKSDB_LIBRARY}) - # target_include_directories (clickhouse_common_io SYSTEM BEFORE PUBLIC ${ROCKSDB_INCLUDE_DIR}) endif() if (ENABLE_TESTS AND USE_GTEST) diff --git a/src/Storages/Rocksdb/StorageEmbeddedRocksdb.h b/src/Storages/Rocksdb/StorageEmbeddedRocksdb.h index 7c8d1f9c47e..79e7d1678e0 100644 --- a/src/Storages/Rocksdb/StorageEmbeddedRocksdb.h +++ b/src/Storages/Rocksdb/StorageEmbeddedRocksdb.h @@ -47,7 +47,6 @@ private: using RocksdbPtr = std::unique_ptr; RocksdbPtr rocksdb_ptr; String rocksdb_dir; - mutable std::shared_mutex rwlock; void initDb(); }; From ad78e0b3f56b325ca44fd72bba09fca00f969370 Mon Sep 17 00:00:00 2001 From: sundy-li <543950155@qq.com> Date: Tue, 6 Oct 2020 09:26:29 +0800 Subject: [PATCH 108/205] trigger --- contrib/CMakeLists.txt | 1 - 1 file changed, 1 deletion(-) diff --git a/contrib/CMakeLists.txt b/contrib/CMakeLists.txt index 02355f5db5f..15194b8a2a8 100644 --- a/contrib/CMakeLists.txt +++ b/contrib/CMakeLists.txt @@ -327,7 +327,6 @@ if (USE_INTERNAL_ROCKSDB_LIBRARY) set(WITH_TOOLS OFF) set(WITH_GFLAGS OFF) set(PORTABLE ON) - set(FORCE_SSE42 ON) if (SANITIZE STREQUAL "undefined") set(WITH_UBSAN ON) From fced1a0e28a91470a55c10698b49ced0d6e9bb4a Mon Sep 17 00:00:00 2001 From: sundy-li <543950155@qq.com> Date: Fri, 16 Oct 2020 09:02:16 +0800 Subject: [PATCH 109/205] fix extract keys --- src/Storages/Rocksdb/StorageEmbeddedRocksdb.cpp | 12 +++++------- 1 file changed, 5 insertions(+), 7 deletions(-) diff --git a/src/Storages/Rocksdb/StorageEmbeddedRocksdb.cpp b/src/Storages/Rocksdb/StorageEmbeddedRocksdb.cpp index 72602de68e9..1f97d8512a5 100644 --- a/src/Storages/Rocksdb/StorageEmbeddedRocksdb.cpp +++ b/src/Storages/Rocksdb/StorageEmbeddedRocksdb.cpp @@ -239,15 +239,13 @@ Pipe StorageEmbeddedRocksdb::read( { std::vector slices_keys; std::vector values; + std::vector wbs(keys.size()); - WriteBufferFromOwnString wb; - UInt64 offset = 0; - for (const auto & key : keys) + for (size_t i = 0; i < keys.size(); ++i) { - sample_block.getByName(primary_key).type->serializeBinary(key, wb); - auto str_ref = wb.stringRef(); - slices_keys.emplace_back(str_ref.data + offset, str_ref.size - offset); - offset = str_ref.size; + sample_block.getByName(primary_key).type->serializeBinary(keys[i], wbs[i]); + auto str_ref = wbs[i].stringRef(); + slices_keys.emplace_back(str_ref.data, str_ref.size); } auto statuses = rocksdb_ptr->MultiGet(rocksdb::ReadOptions(), slices_keys, &values); From b9e663af25ed285a03c12b09c2e2f883856f49e3 Mon Sep 17 00:00:00 2001 From: sundy-li <543950155@qq.com> Date: Fri, 16 Oct 2020 10:27:56 +0800 Subject: [PATCH 110/205] add rocksdb perf tests --- tests/performance/rocksdb.xml | 18 ++++++++++++++++++ 1 file changed, 18 insertions(+) create mode 100644 tests/performance/rocksdb.xml diff --git a/tests/performance/rocksdb.xml b/tests/performance/rocksdb.xml new file mode 100644 index 00000000000..e0867eece65 --- /dev/null +++ b/tests/performance/rocksdb.xml @@ -0,0 +1,18 @@ + + + + hits_100m_single + + + + 30000000000 + + + CREATE TABLE ma_kv (key String, value AggregateFunction(groupBitmap, UInt64)) EmbeddedRocksdb primary key(key) + + INSERT INTO ma_kv SELECT concat('CodeVersion=', CodeVersion) as key, bitmapBuild(groupArray(UserID)) AS value FROM hits_100m_single GROUP BY key + + SELECT groupBitmapOr(value) FROM ma_kv WHERE key IN ('CodeVersion=1657', 'CodeVersion=1', 'CodeVersion=275') + + DROP TABLE IF EXISTS ma_kv + From f95a66b5845ede5c33148836299869c4c0d4e1a1 Mon Sep 17 00:00:00 2001 From: sundy-li <543950155@qq.com> Date: Mon, 19 Oct 2020 01:09:00 +0800 Subject: [PATCH 111/205] add supportsIndexForIn && supportsParallelInsert, support parallel reading --- .../Rocksdb/StorageEmbeddedRocksdb.cpp | 207 ++++++++++++++---- src/Storages/Rocksdb/StorageEmbeddedRocksdb.h | 8 + tests/performance/rocksdb.xml | 18 -- .../0_stateless/01504_rocksdb.reference | 1 + tests/queries/0_stateless/01504_rocksdb.sql | 2 + 5 files changed, 170 insertions(+), 66 deletions(-) delete mode 100644 tests/performance/rocksdb.xml diff --git a/src/Storages/Rocksdb/StorageEmbeddedRocksdb.cpp b/src/Storages/Rocksdb/StorageEmbeddedRocksdb.cpp index 1f97d8512a5..127d1349a2c 100644 --- a/src/Storages/Rocksdb/StorageEmbeddedRocksdb.cpp +++ b/src/Storages/Rocksdb/StorageEmbeddedRocksdb.cpp @@ -6,6 +6,7 @@ #include #include #include +#include #include #include #include @@ -18,6 +19,8 @@ #include #include +#include +#include #include #include #include @@ -36,12 +39,13 @@ namespace ErrorCodes { extern const int BAD_ARGUMENTS; extern const int SYSTEM_ERROR; + extern const int LOGICAL_ERROR; extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; } // returns keys may be filter by condition -static bool traverseASTFilter(const String & primary_key, const ASTPtr & elem, FieldVector & res) +static bool traverseASTFilter(const String & primary_key, const ASTPtr & elem, const PreparedSets & sets, FieldVector & res) { const auto * function = elem->as(); if (!function) @@ -50,7 +54,7 @@ static bool traverseASTFilter(const String & primary_key, const ASTPtr & elem, F if (function->name == "and") { for (const auto & child : function->arguments->children) - if (traverseASTFilter(primary_key, child, res)) + if (traverseASTFilter(primary_key, child, sets, res)) return true; return false; } @@ -73,6 +77,27 @@ static bool traverseASTFilter(const String & primary_key, const ASTPtr & elem, F if (ident->name != primary_key) return false; + + if (function->name == "in" && ((value->as() || value->as()))) + { + auto set_it = sets.find(PreparedSetKey::forSubquery(*value)); + if (set_it == sets.end()) + return false; + SetPtr prepared_set = set_it->second; + + if (!prepared_set->hasExplicitSetElements()) + return false; + + prepared_set->checkColumnsNumber(1); + + const auto & set_column = *prepared_set->getSetElements()[0]; + for (size_t row = 0; row < set_column.size(); ++row) + { + res.push_back(set_column[row]); + } + return true; + } + if (const auto * literal = value->as()) { if (function->name == "equals") @@ -80,14 +105,17 @@ static bool traverseASTFilter(const String & primary_key, const ASTPtr & elem, F res.push_back(literal->value); return true; } - else if (function->name == "in" && literal->value.getType() == Field::Types::Tuple) + else if (function->name == "in") { - auto tuple = literal->value.safeGet(); - for (const auto & f : tuple) + if (literal->value.getType() == Field::Types::Tuple) { - res.push_back(f); + auto tuple = literal->value.safeGet(); + for (const auto & f : tuple) + { + res.push_back(f); + } + return true; } - return true; } else return false; } @@ -99,19 +127,104 @@ static bool traverseASTFilter(const String & primary_key, const ASTPtr & elem, F /** Retrieve from the query a condition of the form `key = 'key'`, `key in ('xxx_'), from conjunctions in the WHERE clause. * TODO support key like search */ -static std::pair getFilterKeys(const String & primary_key, const ASTPtr & query) +static std::pair getFilterKeys(const String & primary_key, const SelectQueryInfo & query_info) { - const auto & select = query->as(); + const auto & select = query_info.query->as(); if (!select.where()) { return std::make_pair(FieldVector{}, true); } FieldVector res; - auto matched_keys = traverseASTFilter(primary_key, select.where(), res); + auto matched_keys = traverseASTFilter(primary_key, select.where(), query_info.sets, res); return std::make_pair(res, !matched_keys); } +class EmbeddedRocksdbSource : public SourceWithProgress +{ +public: + EmbeddedRocksdbSource( + const StorageEmbeddedRocksdb & storage_, + const StorageMetadataPtr & metadata_snapshot_, + const FieldVector & keys_, + const size_t start_, + const size_t end_, + const size_t rocksdb_batch_read_size_) + : SourceWithProgress(metadata_snapshot_->getSampleBlock()) + , storage(storage_) + , metadata_snapshot(metadata_snapshot_) + , start(start_) + , end(end_) + , rocksdb_batch_read_size(rocksdb_batch_read_size_) + { + // slice the keys + if (end > start) + { + keys.resize(end - start); + std::copy(keys_.begin() + start, keys_.begin() + end, keys.begin()); + } + } + + String getName() const override + { + return storage.getName(); + } + + Chunk generate() override + { + if (processed_keys >= keys.size() || (start == end)) + return {}; + + std::vector slices_keys; + slices_keys.reserve(keys.size()); + std::vector values; + std::vector wbs(keys.size()); + + const auto & sample_block = metadata_snapshot->getSampleBlock(); + const auto & key_column = sample_block.getByName(storage.primary_key); + auto columns = sample_block.cloneEmptyColumns(); + size_t primary_key_pos = sample_block.getPositionByName(storage.primary_key); + + for (size_t i = processed_keys; i < std::min(keys.size(), processed_keys + size_t(rocksdb_batch_read_size)); ++i) + { + key_column.type->serializeBinary(keys[i], wbs[i]); + auto str_ref = wbs[i].stringRef(); + slices_keys.emplace_back(str_ref.data, str_ref.size); + } + + auto statuses = storage.rocksdb_ptr->MultiGet(rocksdb::ReadOptions(), slices_keys, &values); + for (size_t i = 0; i < statuses.size(); ++i) + { + if (statuses[i].ok()) + { + ReadBufferFromString key_buffer(slices_keys[i]); + ReadBufferFromString value_buffer(values[i]); + + for (const auto [idx, column_type] : ext::enumerate(sample_block.getColumnsWithTypeAndName())) + { + column_type.type->deserializeBinary(*columns[idx], idx == primary_key_pos? key_buffer: value_buffer); + } + } + } + processed_keys += rocksdb_batch_read_size; + + UInt64 num_rows = columns.at(0)->size(); + return Chunk(std::move(columns), num_rows); + } + +private: + const StorageEmbeddedRocksdb & storage; + + const StorageMetadataPtr metadata_snapshot; + const size_t start; + const size_t end; + const size_t rocksdb_batch_read_size; + FieldVector keys; + + size_t processed_keys = 0; +}; + + class EmbeddedRocksdbBlockOutputStream : public IBlockOutputStream { public: @@ -206,19 +319,19 @@ Pipe StorageEmbeddedRocksdb::read( const Context & /*context*/, QueryProcessingStage::Enum /*processed_stage*/, size_t /*max_block_size*/, - unsigned /*num_streams*/) + unsigned num_streams) { metadata_snapshot->check(column_names, getVirtuals(), getStorageID()); Block sample_block = metadata_snapshot->getSampleBlock(); - MutableColumns columns = sample_block.cloneEmptyColumns(); + size_t primary_key_pos = sample_block.getPositionByName(primary_key); FieldVector keys; bool all_scan = false; - std::tie(keys, all_scan) = getFilterKeys(primary_key, query_info.query); - // TODO pipline + std::tie(keys, all_scan) = getFilterKeys(primary_key, query_info); if (all_scan) { + MutableColumns columns = sample_block.cloneEmptyColumns(); auto it = std::unique_ptr(rocksdb_ptr->NewIterator(rocksdb::ReadOptions())); for (it->SeekToFirst(); it->Valid(); it->Next()) { @@ -227,49 +340,47 @@ Pipe StorageEmbeddedRocksdb::read( for (const auto [idx, column_type] : ext::enumerate(sample_block.getColumnsWithTypeAndName())) { - if (column_type.name == primary_key) - column_type.type->deserializeBinary(*columns[idx], key_buffer); - else - column_type.type->deserializeBinary(*columns[idx], value_buffer); + column_type.type->deserializeBinary(*columns[idx], idx == primary_key_pos? key_buffer: value_buffer); } } - assert(it->status().ok()); + + if (!it->status().ok()) + { + throw Exception("Engine " + getName() + " got error while seeking key value datas: " + it->status().ToString(), + ErrorCodes::LOGICAL_ERROR); + } + UInt64 num_rows = columns.at(0)->size(); + Chunk chunk(std::move(columns), num_rows); + return Pipe(std::make_shared(sample_block, std::move(chunk))); } else { - std::vector slices_keys; - std::vector values; - std::vector wbs(keys.size()); + if (keys.empty()) + return {}; - for (size_t i = 0; i < keys.size(); ++i) + Pipes pipes; + size_t start = 0; + size_t end; + + const size_t num_threads = std::min(size_t(num_streams), keys.size()); + const size_t batch_per_size = ceil(keys.size() * 1.0 / num_threads); + + // TODO settings + static constexpr size_t rocksdb_batch_read_size = 81920; + + for (size_t t = 0; t < num_threads; ++t) { - sample_block.getByName(primary_key).type->serializeBinary(keys[i], wbs[i]); - auto str_ref = wbs[i].stringRef(); - slices_keys.emplace_back(str_ref.data, str_ref.size); - } + if (start >= keys.size()) + start = end = 0; + else + end = start + batch_per_size > keys.size() ? keys.size() : start + batch_per_size; - auto statuses = rocksdb_ptr->MultiGet(rocksdb::ReadOptions(), slices_keys, &values); - for (size_t i = 0; i < statuses.size(); ++i) - { - if (statuses[i].ok()) - { - ReadBufferFromString key_buffer(slices_keys[i]); - ReadBufferFromString value_buffer(values[i]); - - for (const auto [idx, column_type] : ext::enumerate(sample_block.getColumnsWithTypeAndName())) - { - if (column_type.name == primary_key) - column_type.type->deserializeBinary(*columns[idx], key_buffer); - else - column_type.type->deserializeBinary(*columns[idx], value_buffer); - } - } + pipes.emplace_back( + std::make_shared(*this, metadata_snapshot, keys, start, end, rocksdb_batch_read_size)); + start += batch_per_size; } + return Pipe::unitePipes(std::move(pipes)); } - - UInt64 num_rows = columns.at(0)->size(); - Chunk chunk(std::move(columns), num_rows); - return Pipe(std::make_shared(sample_block, std::move(chunk))); } BlockOutputStreamPtr StorageEmbeddedRocksdb::write(const ASTPtr & /*query*/, const StorageMetadataPtr & metadata_snapshot, const Context & /*context*/) @@ -280,7 +391,7 @@ BlockOutputStreamPtr StorageEmbeddedRocksdb::write(const ASTPtr & /*query*/, con static StoragePtr create(const StorageFactory::Arguments & args) { - // TODO RocksdbSettings + // TODO custom RocksdbSettings if (!args.engine_args.empty()) throw Exception( "Engine " + args.engine_name + " doesn't support any arguments (" + toString(args.engine_args.size()) + " given)", diff --git a/src/Storages/Rocksdb/StorageEmbeddedRocksdb.h b/src/Storages/Rocksdb/StorageEmbeddedRocksdb.h index 79e7d1678e0..bc70e795cba 100644 --- a/src/Storages/Rocksdb/StorageEmbeddedRocksdb.h +++ b/src/Storages/Rocksdb/StorageEmbeddedRocksdb.h @@ -18,6 +18,7 @@ class Context; class StorageEmbeddedRocksdb final : public ext::shared_ptr_helper, public IStorage { friend struct ext::shared_ptr_helper; + friend class EmbeddedRocksdbSource; friend class EmbeddedRocksdbBlockOutputStream; public: std::string getName() const override { return "EmbeddedRocksdb"; } @@ -34,6 +35,13 @@ public: BlockOutputStreamPtr write(const ASTPtr & query, const StorageMetadataPtr & /*metadata_snapshot*/, const Context & context) override; void truncate(const ASTPtr &, const StorageMetadataPtr & metadata_snapshot, const Context &, TableExclusiveLockHolder &) override; + bool supportsParallelInsert() const override { return true; } + bool supportsIndexForIn() const override { return true; } + bool mayBenefitFromIndexForIn(const ASTPtr & node, const Context & /*query_context*/, const StorageMetadataPtr & /*metadata_snapshot*/) const override + { + return node->getColumnName() == primary_key; + } + protected: StorageEmbeddedRocksdb(const StorageID & table_id_, const String & relative_data_path_, diff --git a/tests/performance/rocksdb.xml b/tests/performance/rocksdb.xml deleted file mode 100644 index e0867eece65..00000000000 --- a/tests/performance/rocksdb.xml +++ /dev/null @@ -1,18 +0,0 @@ - - - - hits_100m_single - - - - 30000000000 - - - CREATE TABLE ma_kv (key String, value AggregateFunction(groupBitmap, UInt64)) EmbeddedRocksdb primary key(key) - - INSERT INTO ma_kv SELECT concat('CodeVersion=', CodeVersion) as key, bitmapBuild(groupArray(UserID)) AS value FROM hits_100m_single GROUP BY key - - SELECT groupBitmapOr(value) FROM ma_kv WHERE key IN ('CodeVersion=1657', 'CodeVersion=1', 'CodeVersion=275') - - DROP TABLE IF EXISTS ma_kv - diff --git a/tests/queries/0_stateless/01504_rocksdb.reference b/tests/queries/0_stateless/01504_rocksdb.reference index ae092fcbc30..f98edab5dbf 100644 --- a/tests/queries/0_stateless/01504_rocksdb.reference +++ b/tests/queries/0_stateless/01504_rocksdb.reference @@ -5,3 +5,4 @@ 1 1 1 +1 diff --git a/tests/queries/0_stateless/01504_rocksdb.sql b/tests/queries/0_stateless/01504_rocksdb.sql index f78f6e2f658..8b01aafa38a 100644 --- a/tests/queries/0_stateless/01504_rocksdb.sql +++ b/tests/queries/0_stateless/01504_rocksdb.sql @@ -21,10 +21,12 @@ INSERT INTO test_memory SELECT number % 77 AS k, SUM(number) AS value, (1, 1.2), SELECT A.a - B.a, A.b - B.b, A.c - B.c, A.d - B.d, A.e - B.e FROM ( SELECT 0 AS a, groupBitmapMerge(bm) AS b , SUM(k) AS c, SUM(value) AS d, SUM(dummy.1) AS e FROM test) A ANY LEFT JOIN (SELECT 0 AS a, groupBitmapMerge(bm) AS b , SUM(k) AS c, SUM(value) AS d, SUM(dummy.1) AS e FROM test_memory) B USING a ORDER BY a; +CREATE TEMPORARY TABLE keys AS SELECT * FROM numbers(1000); SET max_rows_to_read = 2; SELECT dummy == (1,1.2) FROM test WHERE k IN (1, 3); SELECT k == 4 FROM test WHERE k = 4; +SELECT k == 4 FROM test WHERE k IN (SELECT toUInt32(number) FROM keys WHERE number = 4); SELECT k, value FROM test WHERE k = 0 OR value > 0; -- { serverError 158 } TRUNCATE TABLE test; From fb7066d73d33949c70e20a3d16347514d2c415c6 Mon Sep 17 00:00:00 2001 From: sundy-li <543950155@qq.com> Date: Tue, 20 Oct 2020 21:26:09 +0800 Subject: [PATCH 112/205] * fix fasttest and cmake && pipline for all_scan * unique the keys * add inputstream && outputstream --- cmake/find/rocksdb.cmake | 7 +- docker/test/fasttest/run.sh | 2 +- .../EmbeddedRocksdbBlockInputStream.cpp | 65 ++++++ .../Rocksdb/EmbeddedRocksdbBlockInputStream.h | 33 +++ .../EmbeddedRocksdbBlockOutputStream.cpp | 50 +++++ .../EmbeddedRocksdbBlockOutputStream.h | 29 +++ .../Rocksdb/StorageEmbeddedRocksdb.cpp | 192 +++++++----------- src/Storages/Rocksdb/StorageEmbeddedRocksdb.h | 1 + src/Storages/ya.make | 6 + tests/queries/0_stateless/01504_rocksdb.sql | 4 +- 10 files changed, 260 insertions(+), 129 deletions(-) create mode 100644 src/Storages/Rocksdb/EmbeddedRocksdbBlockInputStream.cpp create mode 100644 src/Storages/Rocksdb/EmbeddedRocksdbBlockInputStream.h create mode 100644 src/Storages/Rocksdb/EmbeddedRocksdbBlockOutputStream.cpp create mode 100644 src/Storages/Rocksdb/EmbeddedRocksdbBlockOutputStream.h diff --git a/cmake/find/rocksdb.cmake b/cmake/find/rocksdb.cmake index d53febee762..24d959eb074 100644 --- a/cmake/find/rocksdb.cmake +++ b/cmake/find/rocksdb.cmake @@ -13,9 +13,8 @@ if (NOT EXISTS "${ClickHouse_SOURCE_DIR}/contrib/rocksdb/CMakeLists.txt") if (USE_INTERNAL_ROCKSDB_LIBRARY) message (WARNING "submodule contrib is missing. to fix try run: \n git submodule update --init --recursive") message(${RECONFIGURE_MESSAGE_LEVEL} "cannot find internal rocksdb") - set (USE_INTERNAL_ROCKSDB_LIBRARY 0) endif() - set (MISSING_ROCKSDB 1) + set (MISSING_INTERNAL_ROCKSDB 1) endif () if (NOT USE_INTERNAL_ROCKSDB_LIBRARY) @@ -26,7 +25,9 @@ if (NOT USE_INTERNAL_ROCKSDB_LIBRARY) endif() endif () -if (NOT ROCKSDB_LIBRARY AND NOT MISSING_ROCKSDB) +if(ROCKSDB_LIBRARY AND ROCKSDB_INCLUDE_DIR) + set(USE_ROCKSDB 1) +elseif (NOT MISSING_INTERNAL_ROCKSDB) set (USE_INTERNAL_ROCKSDB_LIBRARY 1) set (ROCKSDB_INCLUDE_DIR "${ClickHouse_SOURCE_DIR}/contrib/rocksdb/include") diff --git a/docker/test/fasttest/run.sh b/docker/test/fasttest/run.sh index d6f656754ba..bb35489a2cc 100755 --- a/docker/test/fasttest/run.sh +++ b/docker/test/fasttest/run.sh @@ -274,7 +274,7 @@ TESTS_TO_SKIP=( 00646_url_engine 00974_query_profiler - # Rocksdb is not enabled by default + # In fasttest, ENABLE_LIBRARIES=0, so rocksdb engine is not enabled by default 01504_rocksdb # Look at DistributedFilesToInsert, so cannot run in parallel. diff --git a/src/Storages/Rocksdb/EmbeddedRocksdbBlockInputStream.cpp b/src/Storages/Rocksdb/EmbeddedRocksdbBlockInputStream.cpp new file mode 100644 index 00000000000..0f75f41c7dd --- /dev/null +++ b/src/Storages/Rocksdb/EmbeddedRocksdbBlockInputStream.cpp @@ -0,0 +1,65 @@ +#include +#include +#include +#include + +#include + + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int LOGICAL_ERROR; +} + +EmbeddedRocksdbBlockInputStream::EmbeddedRocksdbBlockInputStream( + StorageEmbeddedRocksdb & storage_, + const StorageMetadataPtr & metadata_snapshot_, + size_t max_block_size_) + : storage(storage_) + , metadata_snapshot(metadata_snapshot_) + , max_block_size(max_block_size_) +{ + sample_block = metadata_snapshot->getSampleBlock(); + primary_key_pos = sample_block.getPositionByName(storage.primary_key); +} + +Block EmbeddedRocksdbBlockInputStream::readImpl() +{ + if (finished) + return {}; + + if (!iterator) + { + iterator = std::unique_ptr(storage.rocksdb_ptr->NewIterator(rocksdb::ReadOptions())); + iterator->SeekToFirst(); + } + + MutableColumns columns = sample_block.cloneEmptyColumns(); + size_t rows = 0; + for (; iterator->Valid(); iterator->Next()) + { + ReadBufferFromString key_buffer(iterator->key()); + ReadBufferFromString value_buffer(iterator->value()); + + for (const auto [idx, column_type] : ext::enumerate(sample_block.getColumnsWithTypeAndName())) + { + column_type.type->deserializeBinary(*columns[idx], idx == primary_key_pos? key_buffer: value_buffer); + } + ++rows; + if (rows >= max_block_size) + break; + } + + finished = !iterator->Valid(); + if (!iterator->status().ok()) + { + throw Exception("Engine " + getName() + " got error while seeking key value datas: " + iterator->status().ToString(), + ErrorCodes::LOGICAL_ERROR); + } + return sample_block.cloneWithColumns(std::move(columns)); +} + +} diff --git a/src/Storages/Rocksdb/EmbeddedRocksdbBlockInputStream.h b/src/Storages/Rocksdb/EmbeddedRocksdbBlockInputStream.h new file mode 100644 index 00000000000..817aace58d0 --- /dev/null +++ b/src/Storages/Rocksdb/EmbeddedRocksdbBlockInputStream.h @@ -0,0 +1,33 @@ +#pragma once + +#include +#include +#include + +#include + +namespace DB +{ + +class EmbeddedRocksdbBlockInputStream : public IBlockInputStream +{ + +public: + EmbeddedRocksdbBlockInputStream( + StorageEmbeddedRocksdb & storage_, const StorageMetadataPtr & metadata_snapshot_, size_t max_block_size_); + + String getName() const override { return storage.getName(); } + Block getHeader() const override { return sample_block; } + Block readImpl() override; + +private: + StorageEmbeddedRocksdb & storage; + StorageMetadataPtr metadata_snapshot; + const size_t max_block_size; + + Block sample_block; + std::unique_ptr iterator; + size_t primary_key_pos; + bool finished = false; +}; +} diff --git a/src/Storages/Rocksdb/EmbeddedRocksdbBlockOutputStream.cpp b/src/Storages/Rocksdb/EmbeddedRocksdbBlockOutputStream.cpp new file mode 100644 index 00000000000..aebc41addda --- /dev/null +++ b/src/Storages/Rocksdb/EmbeddedRocksdbBlockOutputStream.cpp @@ -0,0 +1,50 @@ + +#include +#include + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int SYSTEM_ERROR; +} + +Block EmbeddedRocksdbBlockOutputStream::getHeader() const +{ + return metadata_snapshot->getSampleBlock(); +} + +void EmbeddedRocksdbBlockOutputStream::write(const Block & block) +{ + metadata_snapshot->check(block, true); + auto rows = block.rows(); + + WriteBufferFromOwnString wb_key; + WriteBufferFromOwnString wb_value; + + rocksdb::WriteBatch batch; + auto columns = metadata_snapshot->getColumns(); + + for (size_t i = 0; i < rows; i++) + { + wb_key.restart(); + wb_value.restart(); + + for (const auto & col : columns) + { + const auto & type = block.getByName(col.name).type; + const auto & column = block.getByName(col.name).column; + if (col.name == storage.primary_key) + type->serializeBinary(*column, i, wb_key); + else + type->serializeBinary(*column, i, wb_value); + } + batch.Put(wb_key.str(), wb_value.str()); + } + auto status = storage.rocksdb_ptr->Write(rocksdb::WriteOptions(), &batch); + if (!status.ok()) + throw Exception("Rocksdb write error: " + status.ToString(), ErrorCodes::SYSTEM_ERROR); +} + +} diff --git a/src/Storages/Rocksdb/EmbeddedRocksdbBlockOutputStream.h b/src/Storages/Rocksdb/EmbeddedRocksdbBlockOutputStream.h new file mode 100644 index 00000000000..d92e15d553a --- /dev/null +++ b/src/Storages/Rocksdb/EmbeddedRocksdbBlockOutputStream.h @@ -0,0 +1,29 @@ +#pragma once + +#include +#include +#include +#include + +namespace DB +{ + +class EmbeddedRocksdbBlockOutputStream : public IBlockOutputStream +{ +public: + explicit EmbeddedRocksdbBlockOutputStream( + StorageEmbeddedRocksdb & storage_, + const StorageMetadataPtr & metadata_snapshot_) + : storage(storage_) + , metadata_snapshot(metadata_snapshot_) + {} + + Block getHeader() const override; + void write(const Block & block) override; + +private: + StorageEmbeddedRocksdb & storage; + StorageMetadataPtr metadata_snapshot; +}; + +} diff --git a/src/Storages/Rocksdb/StorageEmbeddedRocksdb.cpp b/src/Storages/Rocksdb/StorageEmbeddedRocksdb.cpp index 127d1349a2c..86d2f6ae406 100644 --- a/src/Storages/Rocksdb/StorageEmbeddedRocksdb.cpp +++ b/src/Storages/Rocksdb/StorageEmbeddedRocksdb.cpp @@ -3,6 +3,8 @@ #include #include #include +#include +#include #include #include #include @@ -17,7 +19,7 @@ #include #include -#include +#include #include #include #include @@ -39,13 +41,12 @@ namespace ErrorCodes { extern const int BAD_ARGUMENTS; extern const int SYSTEM_ERROR; - extern const int LOGICAL_ERROR; extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; } // returns keys may be filter by condition -static bool traverseASTFilter(const String & primary_key, const ASTPtr & elem, const PreparedSets & sets, FieldVector & res) +static bool traverseASTFilter(const String & primary_key, const DataTypePtr & primary_key_type, const ASTPtr & elem, const PreparedSets & sets, FieldVector & res) { const auto * function = elem->as(); if (!function) @@ -53,34 +54,50 @@ static bool traverseASTFilter(const String & primary_key, const ASTPtr & elem, c if (function->name == "and") { + // one child has the key filter condition is ok for (const auto & child : function->arguments->children) - if (traverseASTFilter(primary_key, child, sets, res)) + if (traverseASTFilter(primary_key, primary_key_type, child, sets, res)) return true; return false; } + else if (function->name == "or") + { + // make sure every child has the key filter condition + FieldVector child_res; + for (const auto & child : function->arguments->children) + { + if (!traverseASTFilter(primary_key, primary_key_type, child, sets, child_res)) + return false; + } + res.insert(res.end(), child_res.begin(), child_res.end()); + return true; + } else if (function->name == "equals" || function->name == "in") { const auto & args = function->arguments->as(); + const ASTIdentifier * ident; const IAST * value; if (args.children.size() != 2) - return false; + return false; - const ASTIdentifier * ident; - if ((ident = args.children.at(0)->as())) - value = args.children.at(1).get(); - else if ((ident = args.children.at(1)->as())) - value = args.children.at(0).get(); - else - return false; - - if (ident->name != primary_key) - return false; - - - if (function->name == "in" && ((value->as() || value->as()))) + if (function->name == "in") { - auto set_it = sets.find(PreparedSetKey::forSubquery(*value)); + ident = args.children.at(0)->as(); + if (!ident) + return false; + + if (ident->name != primary_key) + return false; + value = args.children.at(1).get(); + + PreparedSetKey set_key; + if ((value->as() || value->as())) + set_key = PreparedSetKey::forSubquery(*value); + else + set_key = PreparedSetKey::forLiteral(*value, {primary_key_type}); + + auto set_it = sets.find(set_key); if (set_it == sets.end()) return false; SetPtr prepared_set = set_it->second; @@ -89,7 +106,6 @@ static bool traverseASTFilter(const String & primary_key, const ASTPtr & elem, c return false; prepared_set->checkColumnsNumber(1); - const auto & set_column = *prepared_set->getSetElements()[0]; for (size_t row = 0; row < set_column.size(); ++row) { @@ -97,27 +113,24 @@ static bool traverseASTFilter(const String & primary_key, const ASTPtr & elem, c } return true; } - - if (const auto * literal = value->as()) + else { - if (function->name == "equals") + if ((ident = args.children.at(0)->as())) + value = args.children.at(1).get(); + else if ((ident = args.children.at(1)->as())) + value = args.children.at(0).get(); + else + return false; + + if (ident->name != primary_key) + return false; + + //function->name == "equals" + if (const auto * literal = value->as()) { res.push_back(literal->value); return true; } - else if (function->name == "in") - { - if (literal->value.getType() == Field::Types::Tuple) - { - auto tuple = literal->value.safeGet(); - for (const auto & f : tuple) - { - res.push_back(f); - } - return true; - } - } - else return false; } } return false; @@ -127,7 +140,7 @@ static bool traverseASTFilter(const String & primary_key, const ASTPtr & elem, c /** Retrieve from the query a condition of the form `key = 'key'`, `key in ('xxx_'), from conjunctions in the WHERE clause. * TODO support key like search */ -static std::pair getFilterKeys(const String & primary_key, const SelectQueryInfo & query_info) +static std::pair getFilterKeys(const String & primary_key, const DataTypePtr & primary_key_type, const SelectQueryInfo & query_info) { const auto & select = query_info.query->as(); if (!select.where()) @@ -135,7 +148,7 @@ static std::pair getFilterKeys(const String & primary_key, co return std::make_pair(FieldVector{}, true); } FieldVector res; - auto matched_keys = traverseASTFilter(primary_key, select.where(), query_info.sets, res); + auto matched_keys = traverseASTFilter(primary_key, primary_key_type, select.where(), query_info.sets, res); return std::make_pair(res, !matched_keys); } @@ -149,13 +162,13 @@ public: const FieldVector & keys_, const size_t start_, const size_t end_, - const size_t rocksdb_batch_read_size_) + const size_t max_block_size_) : SourceWithProgress(metadata_snapshot_->getSampleBlock()) , storage(storage_) , metadata_snapshot(metadata_snapshot_) , start(start_) , end(end_) - , rocksdb_batch_read_size(rocksdb_batch_read_size_) + , max_block_size(max_block_size_) { // slice the keys if (end > start) @@ -185,7 +198,7 @@ public: auto columns = sample_block.cloneEmptyColumns(); size_t primary_key_pos = sample_block.getPositionByName(storage.primary_key); - for (size_t i = processed_keys; i < std::min(keys.size(), processed_keys + size_t(rocksdb_batch_read_size)); ++i) + for (size_t i = processed_keys; i < std::min(keys.size(), processed_keys + max_block_size); ++i) { key_column.type->serializeBinary(keys[i], wbs[i]); auto str_ref = wbs[i].stringRef(); @@ -206,7 +219,7 @@ public: } } } - processed_keys += rocksdb_batch_read_size; + processed_keys += max_block_size; UInt64 num_rows = columns.at(0)->size(); return Chunk(std::move(columns), num_rows); @@ -218,62 +231,13 @@ private: const StorageMetadataPtr metadata_snapshot; const size_t start; const size_t end; - const size_t rocksdb_batch_read_size; + const size_t max_block_size; FieldVector keys; size_t processed_keys = 0; }; -class EmbeddedRocksdbBlockOutputStream : public IBlockOutputStream -{ -public: - explicit EmbeddedRocksdbBlockOutputStream( - StorageEmbeddedRocksdb & storage_, - const StorageMetadataPtr & metadata_snapshot_) - : storage(storage_) - , metadata_snapshot(metadata_snapshot_) - {} - - Block getHeader() const override { return metadata_snapshot->getSampleBlock(); } - - void write(const Block & block) override - { - metadata_snapshot->check(block, true); - auto rows = block.rows(); - - WriteBufferFromOwnString wb_key; - WriteBufferFromOwnString wb_value; - - rocksdb::WriteBatch batch; - auto columns = metadata_snapshot->getColumns(); - - for (size_t i = 0; i < rows; i++) - { - wb_key.restart(); - wb_value.restart(); - - for (const auto & col : columns) - { - const auto & type = block.getByName(col.name).type; - const auto & column = block.getByName(col.name).column; - if (col.name == storage.primary_key) - type->serializeBinary(*column, i, wb_key); - else - type->serializeBinary(*column, i, wb_value); - } - batch.Put(wb_key.str(), wb_value.str()); - } - auto status = storage.rocksdb_ptr->Write(rocksdb::WriteOptions(), &batch); - if (!status.ok()) - throw Exception("Rocksdb write error: " + status.ToString(), ErrorCodes::SYSTEM_ERROR); - } - -private: - StorageEmbeddedRocksdb & storage; - StorageMetadataPtr metadata_snapshot; -}; - StorageEmbeddedRocksdb::StorageEmbeddedRocksdb(const StorageID & table_id_, const String & relative_data_path_, const StorageInMemoryMetadata & metadata_, @@ -318,46 +282,31 @@ Pipe StorageEmbeddedRocksdb::read( const SelectQueryInfo & query_info, const Context & /*context*/, QueryProcessingStage::Enum /*processed_stage*/, - size_t /*max_block_size*/, + size_t max_block_size, unsigned num_streams) { metadata_snapshot->check(column_names, getVirtuals(), getStorageID()); - Block sample_block = metadata_snapshot->getSampleBlock(); - size_t primary_key_pos = sample_block.getPositionByName(primary_key); - FieldVector keys; bool all_scan = false; - std::tie(keys, all_scan) = getFilterKeys(primary_key, query_info); + auto primary_key_data_type = metadata_snapshot->getSampleBlock().getByName(primary_key).type; + std::tie(keys, all_scan) = getFilterKeys(primary_key, primary_key_data_type, query_info); if (all_scan) { - MutableColumns columns = sample_block.cloneEmptyColumns(); - auto it = std::unique_ptr(rocksdb_ptr->NewIterator(rocksdb::ReadOptions())); - for (it->SeekToFirst(); it->Valid(); it->Next()) - { - ReadBufferFromString key_buffer(it->key()); - ReadBufferFromString value_buffer(it->value()); - - for (const auto [idx, column_type] : ext::enumerate(sample_block.getColumnsWithTypeAndName())) - { - column_type.type->deserializeBinary(*columns[idx], idx == primary_key_pos? key_buffer: value_buffer); - } - } - - if (!it->status().ok()) - { - throw Exception("Engine " + getName() + " got error while seeking key value datas: " + it->status().ToString(), - ErrorCodes::LOGICAL_ERROR); - } - UInt64 num_rows = columns.at(0)->size(); - Chunk chunk(std::move(columns), num_rows); - return Pipe(std::make_shared(sample_block, std::move(chunk))); + auto reader = std::make_shared( + *this, metadata_snapshot, max_block_size); + return Pipe(std::make_shared(reader)); } else { if (keys.empty()) return {}; + std::sort(keys.begin(), keys.end()); + auto unique_iter = std::unique(keys.begin(), keys.end()); + if (unique_iter != keys.end()) + keys.erase(unique_iter, keys.end()); + Pipes pipes; size_t start = 0; size_t end; @@ -365,9 +314,6 @@ Pipe StorageEmbeddedRocksdb::read( const size_t num_threads = std::min(size_t(num_streams), keys.size()); const size_t batch_per_size = ceil(keys.size() * 1.0 / num_threads); - // TODO settings - static constexpr size_t rocksdb_batch_read_size = 81920; - for (size_t t = 0; t < num_threads; ++t) { if (start >= keys.size()) @@ -376,7 +322,7 @@ Pipe StorageEmbeddedRocksdb::read( end = start + batch_per_size > keys.size() ? keys.size() : start + batch_per_size; pipes.emplace_back( - std::make_shared(*this, metadata_snapshot, keys, start, end, rocksdb_batch_read_size)); + std::make_shared(*this, metadata_snapshot, keys, start, end, max_block_size)); start += batch_per_size; } return Pipe::unitePipes(std::move(pipes)); @@ -391,7 +337,7 @@ BlockOutputStreamPtr StorageEmbeddedRocksdb::write(const ASTPtr & /*query*/, con static StoragePtr create(const StorageFactory::Arguments & args) { - // TODO custom RocksdbSettings + // TODO custom RocksdbSettings, table function if (!args.engine_args.empty()) throw Exception( "Engine " + args.engine_name + " doesn't support any arguments (" + toString(args.engine_args.size()) + " given)", diff --git a/src/Storages/Rocksdb/StorageEmbeddedRocksdb.h b/src/Storages/Rocksdb/StorageEmbeddedRocksdb.h index bc70e795cba..c803055223b 100644 --- a/src/Storages/Rocksdb/StorageEmbeddedRocksdb.h +++ b/src/Storages/Rocksdb/StorageEmbeddedRocksdb.h @@ -20,6 +20,7 @@ class StorageEmbeddedRocksdb final : public ext::shared_ptr_helper; friend class EmbeddedRocksdbSource; friend class EmbeddedRocksdbBlockOutputStream; + friend class EmbeddedRocksdbBlockInputStream; public: std::string getName() const override { return "EmbeddedRocksdb"; } diff --git a/src/Storages/ya.make b/src/Storages/ya.make index c0a4d82c5bf..fd3f3392892 100644 --- a/src/Storages/ya.make +++ b/src/Storages/ya.make @@ -106,6 +106,12 @@ SRCS( MutationCommands.cpp PartitionCommands.cpp ReadInOrderOptimizer.cpp +<<<<<<< HEAD +======= + registerStorages.cpp + Rocksdb/EmbeddedRocksdbBlockInputStream.cpp + Rocksdb/EmbeddedRocksdbBlockOutputStream.cpp +>>>>>>> * fix fasttest and cmake && pipline for all_scan Rocksdb/StorageEmbeddedRocksdb.cpp SelectQueryDescription.cpp SetSettings.cpp diff --git a/tests/queries/0_stateless/01504_rocksdb.sql b/tests/queries/0_stateless/01504_rocksdb.sql index 8b01aafa38a..2ad00f18cd8 100644 --- a/tests/queries/0_stateless/01504_rocksdb.sql +++ b/tests/queries/0_stateless/01504_rocksdb.sql @@ -24,8 +24,8 @@ SELECT A.a - B.a, A.b - B.b, A.c - B.c, A.d - B.d, A.e - B.e FROM ( SELECT 0 AS CREATE TEMPORARY TABLE keys AS SELECT * FROM numbers(1000); SET max_rows_to_read = 2; -SELECT dummy == (1,1.2) FROM test WHERE k IN (1, 3); -SELECT k == 4 FROM test WHERE k = 4; +SELECT dummy == (1,1.2) FROM test WHERE k IN (1, 3) OR k IN (1) OR k IN (3, 1) OR k IN [1] OR k IN [1, 3] ; +SELECT k == 4 FROM test WHERE k = 4 OR k IN [4]; SELECT k == 4 FROM test WHERE k IN (SELECT toUInt32(number) FROM keys WHERE number = 4); SELECT k, value FROM test WHERE k = 0 OR value > 0; -- { serverError 158 } From 46c01eb7a28918c4b6419878adba6aad74cc7c75 Mon Sep 17 00:00:00 2001 From: sundy-li <543950155@qq.com> Date: Thu, 22 Oct 2020 16:48:57 +0800 Subject: [PATCH 113/205] improve unbundle build --- cmake/find/rocksdb.cmake | 32 +++++++++++++++++-- docker/packager/unbundled/Dockerfile | 1 + docker/test/stateless_unbundled/Dockerfile | 2 ++ .../Rocksdb/EmbeddedRocksdbBlockInputStream.h | 1 - .../EmbeddedRocksdbBlockOutputStream.h | 1 - .../Rocksdb/StorageEmbeddedRocksdb.cpp | 6 ++-- src/Storages/ya.make | 4 --- tests/queries/0_stateless/01504_rocksdb.sql | 3 +- 8 files changed, 38 insertions(+), 12 deletions(-) diff --git a/cmake/find/rocksdb.cmake b/cmake/find/rocksdb.cmake index 24d959eb074..968cdb52407 100644 --- a/cmake/find/rocksdb.cmake +++ b/cmake/find/rocksdb.cmake @@ -23,6 +23,35 @@ if (NOT USE_INTERNAL_ROCKSDB_LIBRARY) if (NOT ROCKSDB_LIBRARY OR NOT ROCKSDB_INCLUDE_DIR) message (${RECONFIGURE_MESSAGE_LEVEL} "Can't find system rocksdb library") endif() + + if (NOT SNAPPY_LIBRARY) + include(cmake/find/snappy.cmake) + endif() + if (NOT ZLIB_LIBRARY) + include(cmake/find/zlib.cmake) + endif() + + find_package(BZip2) + find_library(ZSTD_LIBRARY zstd) + find_library(LZ4_LIBRARY lz4) + find_library(GFLAGS_LIBRARY gflags) + + if(SNAPPY_LIBRARY AND ZLIB_LIBRARY AND LZ4_LIBRARY AND BZIP2_FOUND AND ZSTD_LIBRARY AND GFLAGS_LIBRARY) + list (APPEND ROCKSDB_LIBRARY ${SNAPPY_LIBRARY}) + list (APPEND ROCKSDB_LIBRARY ${ZLIB_LIBRARY}) + list (APPEND ROCKSDB_LIBRARY ${LZ4_LIBRARY}) + list (APPEND ROCKSDB_LIBRARY ${BZIP2_LIBRARY}) + list (APPEND ROCKSDB_LIBRARY ${ZSTD_LIBRARY}) + list (APPEND ROCKSDB_LIBRARY ${GFLAGS_LIBRARY}) + else() + message (${RECONFIGURE_MESSAGE_LEVEL} + "Can't find system rocksdb: snappy=${SNAPPY_LIBRARY} ;" + " zlib=${ZLIB_LIBRARY} ;" + " lz4=${LZ4_LIBRARY} ;" + " bz2=${BZIP2_LIBRARY} ;" + " zstd=${ZSTD_LIBRARY} ;" + " gflags=${GFLAGS_LIBRARY} ;") + endif() endif () if(ROCKSDB_LIBRARY AND ROCKSDB_INCLUDE_DIR) @@ -31,9 +60,8 @@ elseif (NOT MISSING_INTERNAL_ROCKSDB) set (USE_INTERNAL_ROCKSDB_LIBRARY 1) set (ROCKSDB_INCLUDE_DIR "${ClickHouse_SOURCE_DIR}/contrib/rocksdb/include") - set (ROCKSDB_CORE_INCLUDE_DIR "${ClickHouse_SOURCE_DIR}/contrib/rocksdb/include") set (ROCKSDB_LIBRARY "rocksdb") set (USE_ROCKSDB 1) endif () -message (STATUS "Using ROCKSDB=${USE_ROCKSDB}: ${ROCKSDB_INCLUDE_DIR} : ${ROCKSDB_LIBRARY}") \ No newline at end of file +message (STATUS "Using ROCKSDB=${USE_ROCKSDB}: ${ROCKSDB_INCLUDE_DIR} : ${ROCKSDB_LIBRARY}") diff --git a/docker/packager/unbundled/Dockerfile b/docker/packager/unbundled/Dockerfile index 698a7e2621b..261edf1a86c 100644 --- a/docker/packager/unbundled/Dockerfile +++ b/docker/packager/unbundled/Dockerfile @@ -65,6 +65,7 @@ RUN apt-get update \ libavro-dev \ libfarmhash-dev \ librocksdb-dev \ + libgflags-dev \ libmysqlclient-dev \ --yes --no-install-recommends diff --git a/docker/test/stateless_unbundled/Dockerfile b/docker/test/stateless_unbundled/Dockerfile index 345ba905412..1c9f9510d7e 100644 --- a/docker/test/stateless_unbundled/Dockerfile +++ b/docker/test/stateless_unbundled/Dockerfile @@ -43,6 +43,8 @@ RUN apt-get --allow-unauthenticated update -y \ libreadline-dev \ libsasl2-dev \ libzstd-dev \ + librocksdb-dev \ + libgflags-dev \ lsof \ moreutils \ ncdu \ diff --git a/src/Storages/Rocksdb/EmbeddedRocksdbBlockInputStream.h b/src/Storages/Rocksdb/EmbeddedRocksdbBlockInputStream.h index 817aace58d0..2caf434ffd8 100644 --- a/src/Storages/Rocksdb/EmbeddedRocksdbBlockInputStream.h +++ b/src/Storages/Rocksdb/EmbeddedRocksdbBlockInputStream.h @@ -1,7 +1,6 @@ #pragma once #include -#include #include #include diff --git a/src/Storages/Rocksdb/EmbeddedRocksdbBlockOutputStream.h b/src/Storages/Rocksdb/EmbeddedRocksdbBlockOutputStream.h index d92e15d553a..5ead88216b1 100644 --- a/src/Storages/Rocksdb/EmbeddedRocksdbBlockOutputStream.h +++ b/src/Storages/Rocksdb/EmbeddedRocksdbBlockOutputStream.h @@ -1,7 +1,6 @@ #pragma once #include -#include #include #include diff --git a/src/Storages/Rocksdb/StorageEmbeddedRocksdb.cpp b/src/Storages/Rocksdb/StorageEmbeddedRocksdb.cpp index 86d2f6ae406..d47cc408d39 100644 --- a/src/Storages/Rocksdb/StorageEmbeddedRocksdb.cpp +++ b/src/Storages/Rocksdb/StorageEmbeddedRocksdb.cpp @@ -79,7 +79,7 @@ static bool traverseASTFilter(const String & primary_key, const DataTypePtr & pr const IAST * value; if (args.children.size() != 2) - return false; + return false; if (function->name == "in") { @@ -87,7 +87,7 @@ static bool traverseASTFilter(const String & primary_key, const DataTypePtr & pr if (!ident) return false; - if (ident->name != primary_key) + if (ident->name() != primary_key) return false; value = args.children.at(1).get(); @@ -122,7 +122,7 @@ static bool traverseASTFilter(const String & primary_key, const DataTypePtr & pr else return false; - if (ident->name != primary_key) + if (ident->name() != primary_key) return false; //function->name == "equals" diff --git a/src/Storages/ya.make b/src/Storages/ya.make index fd3f3392892..3254f28be13 100644 --- a/src/Storages/ya.make +++ b/src/Storages/ya.make @@ -106,12 +106,8 @@ SRCS( MutationCommands.cpp PartitionCommands.cpp ReadInOrderOptimizer.cpp -<<<<<<< HEAD -======= - registerStorages.cpp Rocksdb/EmbeddedRocksdbBlockInputStream.cpp Rocksdb/EmbeddedRocksdbBlockOutputStream.cpp ->>>>>>> * fix fasttest and cmake && pipline for all_scan Rocksdb/StorageEmbeddedRocksdb.cpp SelectQueryDescription.cpp SetSettings.cpp diff --git a/tests/queries/0_stateless/01504_rocksdb.sql b/tests/queries/0_stateless/01504_rocksdb.sql index 2ad00f18cd8..eaaecca96b5 100644 --- a/tests/queries/0_stateless/01504_rocksdb.sql +++ b/tests/queries/0_stateless/01504_rocksdb.sql @@ -25,9 +25,10 @@ CREATE TEMPORARY TABLE keys AS SELECT * FROM numbers(1000); SET max_rows_to_read = 2; SELECT dummy == (1,1.2) FROM test WHERE k IN (1, 3) OR k IN (1) OR k IN (3, 1) OR k IN [1] OR k IN [1, 3] ; -SELECT k == 4 FROM test WHERE k = 4 OR k IN [4]; +SELECT k == 4 FROM test WHERE k = 4 OR k IN [4] OR k in (4, 10000001, 10000002) AND value > 0; SELECT k == 4 FROM test WHERE k IN (SELECT toUInt32(number) FROM keys WHERE number = 4); SELECT k, value FROM test WHERE k = 0 OR value > 0; -- { serverError 158 } +SELECT k, value FROM test WHERE k = 0 AND k IN (1, 3) OR k > 8; -- { serverError 158 } TRUNCATE TABLE test; SELECT 0 == count(1) FROM test; From 75c994ad872999a4e2572baa5e20ddd4c21ee643 Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Sun, 8 Nov 2020 18:35:22 +0300 Subject: [PATCH 114/205] Update embedded-rocksdb.md --- .../engines/table-engines/integrations/embedded-rocksdb.md | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/docs/en/engines/table-engines/integrations/embedded-rocksdb.md b/docs/en/engines/table-engines/integrations/embedded-rocksdb.md index 88a95ae7de7..3b23fa63b72 100644 --- a/docs/en/engines/table-engines/integrations/embedded-rocksdb.md +++ b/docs/en/engines/table-engines/integrations/embedded-rocksdb.md @@ -40,6 +40,6 @@ PRIMARY KEY key ## Description {#description} -- `primary key` must be specified, only support one primary key. The primary key will serializeBinary as rocksdb key. -- Columns other than the primary key will be serializeBinary as rocksdb value in corresponding order. -- Queries with key `equals` or `in` filtering will be optimized to multi keys look up from rocksdb. +- `primary key` must be specified, it only supports one column in primary key. The primary key will serialized in binary as rocksdb key. +- columns other than the primary key will be serialized in binary as rocksdb value in corresponding order. +- queries with key `equals` or `in` filtering will be optimized to multi keys lookup from rocksdb. From 630aa9b0c3536b5597d70615d4bd8a5ccb42559a Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 8 Nov 2020 18:41:16 +0300 Subject: [PATCH 115/205] Rename Rocksdb to RocksDB --- .../integrations/embedded-rocksdb.md | 12 ++--- src/CMakeLists.txt | 2 +- .../EmbeddedRocksDBBlockInputStream.cpp} | 10 ++--- .../EmbeddedRocksDBBlockInputStream.h} | 10 ++--- .../EmbeddedRocksDBBlockOutputStream.cpp} | 8 ++-- .../EmbeddedRocksDBBlockOutputStream.h} | 10 ++--- .../StorageEmbeddedRocksDB.cpp} | 44 +++++++++---------- .../StorageEmbeddedRocksDB.h} | 20 ++++----- src/Storages/registerStorages.cpp | 2 +- src/Storages/registerStorages.h | 2 +- tests/queries/0_stateless/01504_rocksdb.sql | 4 +- 11 files changed, 62 insertions(+), 62 deletions(-) rename src/Storages/{Rocksdb/EmbeddedRocksdbBlockInputStream.cpp => RocksDB/EmbeddedRocksDBBlockInputStream.cpp} (85%) rename src/Storages/{Rocksdb/EmbeddedRocksdbBlockInputStream.h => RocksDB/EmbeddedRocksDBBlockInputStream.h} (67%) rename src/Storages/{Rocksdb/EmbeddedRocksdbBlockOutputStream.cpp => RocksDB/EmbeddedRocksDBBlockOutputStream.cpp} (81%) rename src/Storages/{Rocksdb/EmbeddedRocksdbBlockOutputStream.h => RocksDB/EmbeddedRocksDBBlockOutputStream.h} (62%) rename src/Storages/{Rocksdb/StorageEmbeddedRocksdb.cpp => RocksDB/StorageEmbeddedRocksDB.cpp} (90%) rename src/Storages/{Rocksdb/StorageEmbeddedRocksdb.h => RocksDB/StorageEmbeddedRocksDB.h} (74%) diff --git a/docs/en/engines/table-engines/integrations/embedded-rocksdb.md b/docs/en/engines/table-engines/integrations/embedded-rocksdb.md index 3b23fa63b72..857e148277c 100644 --- a/docs/en/engines/table-engines/integrations/embedded-rocksdb.md +++ b/docs/en/engines/table-engines/integrations/embedded-rocksdb.md @@ -1,15 +1,15 @@ --- toc_priority: 6 -toc_title: EmbeddedRocksdb +toc_title: EmbeddedRocksDB --- -# EmbeddedRocksdb Engine {#EmbeddedRocksdb-engine} +# EmbeddedRocksDB Engine {#EmbeddedRocksDB-engine} This engine allows integrating ClickHouse with [rocksdb](http://rocksdb.org/). -`EmbeddedRocksdb` lets you: +`EmbeddedRocksDB` lets you: -## Creating a Table {#table_engine-EmbeddedRocksdb-creating-a-table} +## Creating a Table {#table_engine-EmbeddedRocksDB-creating-a-table} ``` sql CREATE TABLE [IF NOT EXISTS] [db.]table_name [ON CLUSTER cluster] @@ -17,7 +17,7 @@ CREATE TABLE [IF NOT EXISTS] [db.]table_name [ON CLUSTER cluster] name1 [type1] [DEFAULT|MATERIALIZED|ALIAS expr1], name2 [type2] [DEFAULT|MATERIALIZED|ALIAS expr2], ... -) ENGINE = EmbeddedRocksdb PRIMARY KEY(primary_key_name) +) ENGINE = EmbeddedRocksDB PRIMARY KEY(primary_key_name) ``` Required parameters: @@ -34,7 +34,7 @@ CREATE TABLE test `v2` String, `v3` Float32, ) -ENGINE = EmbeddedRocksdb +ENGINE = EmbeddedRocksDB PRIMARY KEY key ``` diff --git a/src/CMakeLists.txt b/src/CMakeLists.txt index 4acb8090b6b..2ac990acfff 100644 --- a/src/CMakeLists.txt +++ b/src/CMakeLists.txt @@ -79,7 +79,7 @@ if (USE_AMQPCPP) endif() if (USE_ROCKSDB) - add_headers_and_sources(dbms Storages/Rocksdb) + add_headers_and_sources(dbms Storages/RocksDB) endif() if (USE_AWS_S3) diff --git a/src/Storages/Rocksdb/EmbeddedRocksdbBlockInputStream.cpp b/src/Storages/RocksDB/EmbeddedRocksDBBlockInputStream.cpp similarity index 85% rename from src/Storages/Rocksdb/EmbeddedRocksdbBlockInputStream.cpp rename to src/Storages/RocksDB/EmbeddedRocksDBBlockInputStream.cpp index 0f75f41c7dd..11944c7e295 100644 --- a/src/Storages/Rocksdb/EmbeddedRocksdbBlockInputStream.cpp +++ b/src/Storages/RocksDB/EmbeddedRocksDBBlockInputStream.cpp @@ -1,7 +1,7 @@ #include #include -#include -#include +#include +#include #include @@ -14,8 +14,8 @@ namespace ErrorCodes extern const int LOGICAL_ERROR; } -EmbeddedRocksdbBlockInputStream::EmbeddedRocksdbBlockInputStream( - StorageEmbeddedRocksdb & storage_, +EmbeddedRocksDBBlockInputStream::EmbeddedRocksDBBlockInputStream( + StorageEmbeddedRocksDB & storage_, const StorageMetadataPtr & metadata_snapshot_, size_t max_block_size_) : storage(storage_) @@ -26,7 +26,7 @@ EmbeddedRocksdbBlockInputStream::EmbeddedRocksdbBlockInputStream( primary_key_pos = sample_block.getPositionByName(storage.primary_key); } -Block EmbeddedRocksdbBlockInputStream::readImpl() +Block EmbeddedRocksDBBlockInputStream::readImpl() { if (finished) return {}; diff --git a/src/Storages/Rocksdb/EmbeddedRocksdbBlockInputStream.h b/src/Storages/RocksDB/EmbeddedRocksDBBlockInputStream.h similarity index 67% rename from src/Storages/Rocksdb/EmbeddedRocksdbBlockInputStream.h rename to src/Storages/RocksDB/EmbeddedRocksDBBlockInputStream.h index 2caf434ffd8..cc7911e0087 100644 --- a/src/Storages/Rocksdb/EmbeddedRocksdbBlockInputStream.h +++ b/src/Storages/RocksDB/EmbeddedRocksDBBlockInputStream.h @@ -1,26 +1,26 @@ #pragma once #include -#include +#include #include namespace DB { -class EmbeddedRocksdbBlockInputStream : public IBlockInputStream +class EmbeddedRocksDBBlockInputStream : public IBlockInputStream { public: - EmbeddedRocksdbBlockInputStream( - StorageEmbeddedRocksdb & storage_, const StorageMetadataPtr & metadata_snapshot_, size_t max_block_size_); + EmbeddedRocksDBBlockInputStream( + StorageEmbeddedRocksDB & storage_, const StorageMetadataPtr & metadata_snapshot_, size_t max_block_size_); String getName() const override { return storage.getName(); } Block getHeader() const override { return sample_block; } Block readImpl() override; private: - StorageEmbeddedRocksdb & storage; + StorageEmbeddedRocksDB & storage; StorageMetadataPtr metadata_snapshot; const size_t max_block_size; diff --git a/src/Storages/Rocksdb/EmbeddedRocksdbBlockOutputStream.cpp b/src/Storages/RocksDB/EmbeddedRocksDBBlockOutputStream.cpp similarity index 81% rename from src/Storages/Rocksdb/EmbeddedRocksdbBlockOutputStream.cpp rename to src/Storages/RocksDB/EmbeddedRocksDBBlockOutputStream.cpp index aebc41addda..e7180c18bf4 100644 --- a/src/Storages/Rocksdb/EmbeddedRocksdbBlockOutputStream.cpp +++ b/src/Storages/RocksDB/EmbeddedRocksDBBlockOutputStream.cpp @@ -1,5 +1,5 @@ -#include +#include #include namespace DB @@ -10,12 +10,12 @@ namespace ErrorCodes extern const int SYSTEM_ERROR; } -Block EmbeddedRocksdbBlockOutputStream::getHeader() const +Block EmbeddedRocksDBBlockOutputStream::getHeader() const { return metadata_snapshot->getSampleBlock(); } -void EmbeddedRocksdbBlockOutputStream::write(const Block & block) +void EmbeddedRocksDBBlockOutputStream::write(const Block & block) { metadata_snapshot->check(block, true); auto rows = block.rows(); @@ -44,7 +44,7 @@ void EmbeddedRocksdbBlockOutputStream::write(const Block & block) } auto status = storage.rocksdb_ptr->Write(rocksdb::WriteOptions(), &batch); if (!status.ok()) - throw Exception("Rocksdb write error: " + status.ToString(), ErrorCodes::SYSTEM_ERROR); + throw Exception("RocksDB write error: " + status.ToString(), ErrorCodes::SYSTEM_ERROR); } } diff --git a/src/Storages/Rocksdb/EmbeddedRocksdbBlockOutputStream.h b/src/Storages/RocksDB/EmbeddedRocksDBBlockOutputStream.h similarity index 62% rename from src/Storages/Rocksdb/EmbeddedRocksdbBlockOutputStream.h rename to src/Storages/RocksDB/EmbeddedRocksDBBlockOutputStream.h index 5ead88216b1..a1412b90856 100644 --- a/src/Storages/Rocksdb/EmbeddedRocksdbBlockOutputStream.h +++ b/src/Storages/RocksDB/EmbeddedRocksDBBlockOutputStream.h @@ -1,17 +1,17 @@ #pragma once #include -#include +#include #include namespace DB { -class EmbeddedRocksdbBlockOutputStream : public IBlockOutputStream +class EmbeddedRocksDBBlockOutputStream : public IBlockOutputStream { public: - explicit EmbeddedRocksdbBlockOutputStream( - StorageEmbeddedRocksdb & storage_, + explicit EmbeddedRocksDBBlockOutputStream( + StorageEmbeddedRocksDB & storage_, const StorageMetadataPtr & metadata_snapshot_) : storage(storage_) , metadata_snapshot(metadata_snapshot_) @@ -21,7 +21,7 @@ public: void write(const Block & block) override; private: - StorageEmbeddedRocksdb & storage; + StorageEmbeddedRocksDB & storage; StorageMetadataPtr metadata_snapshot; }; diff --git a/src/Storages/Rocksdb/StorageEmbeddedRocksdb.cpp b/src/Storages/RocksDB/StorageEmbeddedRocksDB.cpp similarity index 90% rename from src/Storages/Rocksdb/StorageEmbeddedRocksdb.cpp rename to src/Storages/RocksDB/StorageEmbeddedRocksDB.cpp index d47cc408d39..cad33de5c25 100644 --- a/src/Storages/Rocksdb/StorageEmbeddedRocksdb.cpp +++ b/src/Storages/RocksDB/StorageEmbeddedRocksDB.cpp @@ -2,9 +2,9 @@ #include #include #include -#include -#include -#include +#include +#include +#include #include #include #include @@ -153,11 +153,11 @@ static std::pair getFilterKeys(const String & primary_key, co } -class EmbeddedRocksdbSource : public SourceWithProgress +class EmbeddedRocksDBSource : public SourceWithProgress { public: - EmbeddedRocksdbSource( - const StorageEmbeddedRocksdb & storage_, + EmbeddedRocksDBSource( + const StorageEmbeddedRocksDB & storage_, const StorageMetadataPtr & metadata_snapshot_, const FieldVector & keys_, const size_t start_, @@ -226,7 +226,7 @@ public: } private: - const StorageEmbeddedRocksdb & storage; + const StorageEmbeddedRocksDB & storage; const StorageMetadataPtr metadata_snapshot; const size_t start; @@ -238,7 +238,7 @@ private: }; -StorageEmbeddedRocksdb::StorageEmbeddedRocksdb(const StorageID & table_id_, +StorageEmbeddedRocksDB::StorageEmbeddedRocksDB(const StorageID & table_id_, const String & relative_data_path_, const StorageInMemoryMetadata & metadata_, bool attach, @@ -255,7 +255,7 @@ StorageEmbeddedRocksdb::StorageEmbeddedRocksdb(const StorageID & table_id_, initDb(); } -void StorageEmbeddedRocksdb::truncate(const ASTPtr &, const StorageMetadataPtr & , const Context &, TableExclusiveLockHolder &) +void StorageEmbeddedRocksDB::truncate(const ASTPtr &, const StorageMetadataPtr & , const Context &, TableExclusiveLockHolder &) { rocksdb_ptr->Close(); Poco::File(rocksdb_dir).remove(true); @@ -263,7 +263,7 @@ void StorageEmbeddedRocksdb::truncate(const ASTPtr &, const StorageMetadataPtr & initDb(); } -void StorageEmbeddedRocksdb::initDb() +void StorageEmbeddedRocksDB::initDb() { rocksdb::Options options; rocksdb::DB * db; @@ -276,10 +276,10 @@ void StorageEmbeddedRocksdb::initDb() } -Pipe StorageEmbeddedRocksdb::read( +Pipe StorageEmbeddedRocksDB::read( const Names & column_names, const StorageMetadataPtr & metadata_snapshot, - const SelectQueryInfo & query_info, + SelectQueryInfo & query_info, const Context & /*context*/, QueryProcessingStage::Enum /*processed_stage*/, size_t max_block_size, @@ -293,7 +293,7 @@ Pipe StorageEmbeddedRocksdb::read( std::tie(keys, all_scan) = getFilterKeys(primary_key, primary_key_data_type, query_info); if (all_scan) { - auto reader = std::make_shared( + auto reader = std::make_shared( *this, metadata_snapshot, max_block_size); return Pipe(std::make_shared(reader)); } @@ -322,22 +322,22 @@ Pipe StorageEmbeddedRocksdb::read( end = start + batch_per_size > keys.size() ? keys.size() : start + batch_per_size; pipes.emplace_back( - std::make_shared(*this, metadata_snapshot, keys, start, end, max_block_size)); + std::make_shared(*this, metadata_snapshot, keys, start, end, max_block_size)); start += batch_per_size; } return Pipe::unitePipes(std::move(pipes)); } } -BlockOutputStreamPtr StorageEmbeddedRocksdb::write(const ASTPtr & /*query*/, const StorageMetadataPtr & metadata_snapshot, const Context & /*context*/) +BlockOutputStreamPtr StorageEmbeddedRocksDB::write(const ASTPtr & /*query*/, const StorageMetadataPtr & metadata_snapshot, const Context & /*context*/) { - return std::make_shared(*this, metadata_snapshot); + return std::make_shared(*this, metadata_snapshot); } static StoragePtr create(const StorageFactory::Arguments & args) { - // TODO custom RocksdbSettings, table function + // TODO custom RocksDBSettings, table function if (!args.engine_args.empty()) throw Exception( "Engine " + args.engine_name + " doesn't support any arguments (" + toString(args.engine_args.size()) + " given)", @@ -348,25 +348,25 @@ static StoragePtr create(const StorageFactory::Arguments & args) metadata.setConstraints(args.constraints); if (!args.storage_def->primary_key) - throw Exception("StorageEmbeddedRocksdb must require one primary key", ErrorCodes::BAD_ARGUMENTS); + throw Exception("StorageEmbeddedRocksDB must require one primary key", ErrorCodes::BAD_ARGUMENTS); metadata.primary_key = KeyDescription::getKeyFromAST(args.storage_def->primary_key->ptr(), metadata.columns, args.context); auto primary_key_names = metadata.getColumnsRequiredForPrimaryKey(); if (primary_key_names.size() != 1) { - throw Exception("StorageEmbeddedRocksdb must require one primary key", ErrorCodes::BAD_ARGUMENTS); + throw Exception("StorageEmbeddedRocksDB must require one primary key", ErrorCodes::BAD_ARGUMENTS); } - return StorageEmbeddedRocksdb::create(args.table_id, args.relative_data_path, metadata, args.attach, args.context, primary_key_names[0]); + return StorageEmbeddedRocksDB::create(args.table_id, args.relative_data_path, metadata, args.attach, args.context, primary_key_names[0]); } -void registerStorageEmbeddedRocksdb(StorageFactory & factory) +void registerStorageEmbeddedRocksDB(StorageFactory & factory) { StorageFactory::StorageFeatures features{ .supports_sort_order = true, }; - factory.registerStorage("EmbeddedRocksdb", create, features); + factory.registerStorage("EmbeddedRocksDB", create, features); } diff --git a/src/Storages/Rocksdb/StorageEmbeddedRocksdb.h b/src/Storages/RocksDB/StorageEmbeddedRocksDB.h similarity index 74% rename from src/Storages/Rocksdb/StorageEmbeddedRocksdb.h rename to src/Storages/RocksDB/StorageEmbeddedRocksDB.h index c803055223b..209f70ceb52 100644 --- a/src/Storages/Rocksdb/StorageEmbeddedRocksdb.h +++ b/src/Storages/RocksDB/StorageEmbeddedRocksDB.h @@ -15,19 +15,19 @@ namespace DB class Context; -class StorageEmbeddedRocksdb final : public ext::shared_ptr_helper, public IStorage +class StorageEmbeddedRocksDB final : public ext::shared_ptr_helper, public IStorage { - friend struct ext::shared_ptr_helper; - friend class EmbeddedRocksdbSource; - friend class EmbeddedRocksdbBlockOutputStream; - friend class EmbeddedRocksdbBlockInputStream; + friend struct ext::shared_ptr_helper; + friend class EmbeddedRocksDBSource; + friend class EmbeddedRocksDBBlockOutputStream; + friend class EmbeddedRocksDBBlockInputStream; public: - std::string getName() const override { return "EmbeddedRocksdb"; } + std::string getName() const override { return "EmbeddedRocksDB"; } Pipe read( const Names & column_names, const StorageMetadataPtr & metadata_snapshot, - const SelectQueryInfo & query_info, + SelectQueryInfo & query_info, const Context & context, QueryProcessingStage::Enum processed_stage, size_t max_block_size, @@ -44,7 +44,7 @@ public: } protected: - StorageEmbeddedRocksdb(const StorageID & table_id_, + StorageEmbeddedRocksDB(const StorageID & table_id_, const String & relative_data_path_, const StorageInMemoryMetadata & metadata, bool attach, @@ -53,8 +53,8 @@ protected: private: const String primary_key; - using RocksdbPtr = std::unique_ptr; - RocksdbPtr rocksdb_ptr; + using RocksDBPtr = std::unique_ptr; + RocksDBPtr rocksdb_ptr; String rocksdb_dir; void initDb(); diff --git a/src/Storages/registerStorages.cpp b/src/Storages/registerStorages.cpp index f1e38c4336d..13c20fcda8d 100644 --- a/src/Storages/registerStorages.cpp +++ b/src/Storages/registerStorages.cpp @@ -55,7 +55,7 @@ void registerStorages() #endif #if USE_ROCKSDB - registerStorageEmbeddedRocksdb(factory); + registerStorageEmbeddedRocksDB(factory); #endif } diff --git a/src/Storages/registerStorages.h b/src/Storages/registerStorages.h index f74eefe1c00..df7249a64af 100644 --- a/src/Storages/registerStorages.h +++ b/src/Storages/registerStorages.h @@ -55,7 +55,7 @@ void registerStorageRabbitMQ(StorageFactory & factory); #endif #if USE_ROCKSDB -void registerStorageEmbeddedRocksdb(StorageFactory & factory); +void registerStorageEmbeddedRocksDB(StorageFactory & factory); #endif void registerStorages(); diff --git a/tests/queries/0_stateless/01504_rocksdb.sql b/tests/queries/0_stateless/01504_rocksdb.sql index eaaecca96b5..123c6c2d9c6 100644 --- a/tests/queries/0_stateless/01504_rocksdb.sql +++ b/tests/queries/0_stateless/01504_rocksdb.sql @@ -1,5 +1,5 @@ DROP TABLE IF EXISTS test; -CREATE TABLE test (key String, value UInt32) Engine=EmbeddedRocksdb primary key(key); +CREATE TABLE test (key String, value UInt32) Engine=EmbeddedRocksDB primary key(key); INSERT INTO test SELECT '1_1', number FROM numbers(10000); SELECT count(1) == 1 FROM test; @@ -11,7 +11,7 @@ SELECT SUM(value) == 1 + 99 + 900 FROM test WHERE key in ('1_1', '99_1', '900_1' DROP TABLE IF EXISTS test; DROP TABLE IF EXISTS test_memory; -CREATE TABLE test (k UInt32, value UInt64, dummy Tuple(UInt32, Float64), bm AggregateFunction(groupBitmap, UInt64)) Engine=EmbeddedRocksdb primary key(k); +CREATE TABLE test (k UInt32, value UInt64, dummy Tuple(UInt32, Float64), bm AggregateFunction(groupBitmap, UInt64)) Engine=EmbeddedRocksDB primary key(k); CREATE TABLE test_memory AS test Engine = Memory; INSERT INTO test SELECT number % 77 AS k, SUM(number) AS value, (1, 1.2), bitmapBuild(groupArray(number)) FROM numbers(10000000) group by k; From 742ffdd145449435709cd3bb50dd4489697d0fc9 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 8 Nov 2020 18:43:05 +0300 Subject: [PATCH 116/205] Update ya.make --- src/Storages/ya.make | 3 --- src/Storages/ya.make.in | 2 +- 2 files changed, 1 insertion(+), 4 deletions(-) diff --git a/src/Storages/ya.make b/src/Storages/ya.make index 3254f28be13..e0c6cab602f 100644 --- a/src/Storages/ya.make +++ b/src/Storages/ya.make @@ -106,9 +106,6 @@ SRCS( MutationCommands.cpp PartitionCommands.cpp ReadInOrderOptimizer.cpp - Rocksdb/EmbeddedRocksdbBlockInputStream.cpp - Rocksdb/EmbeddedRocksdbBlockOutputStream.cpp - Rocksdb/StorageEmbeddedRocksdb.cpp SelectQueryDescription.cpp SetSettings.cpp StorageBuffer.cpp diff --git a/src/Storages/ya.make.in b/src/Storages/ya.make.in index dbae43aa3fc..ad772eb5c50 100644 --- a/src/Storages/ya.make.in +++ b/src/Storages/ya.make.in @@ -8,7 +8,7 @@ PEERDIR( SRCS( - + ) END() From 6376c8db7e3f442f141d0e7d42edebc1ef0d3c38 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 8 Nov 2020 18:45:24 +0300 Subject: [PATCH 117/205] Make code slightly better --- src/Storages/registerStorages.cpp | 56 +++++++++++++++++++++++++++++- src/Storages/registerStorages.h | 57 ------------------------------- 2 files changed, 55 insertions(+), 58 deletions(-) diff --git a/src/Storages/registerStorages.cpp b/src/Storages/registerStorages.cpp index 13c20fcda8d..33c1b6245ac 100644 --- a/src/Storages/registerStorages.cpp +++ b/src/Storages/registerStorages.cpp @@ -1,10 +1,64 @@ #include #include +#if !defined(ARCADIA_BUILD) +# include +# include "config_core.h" +#endif namespace DB { +void registerStorageLog(StorageFactory & factory); +void registerStorageTinyLog(StorageFactory & factory); +void registerStorageStripeLog(StorageFactory & factory); +void registerStorageMergeTree(StorageFactory & factory); +void registerStorageNull(StorageFactory & factory); +void registerStorageMerge(StorageFactory & factory); +void registerStorageBuffer(StorageFactory & factory); +void registerStorageDistributed(StorageFactory & factory); +void registerStorageMemory(StorageFactory & factory); +void registerStorageFile(StorageFactory & factory); +void registerStorageURL(StorageFactory & factory); +void registerStorageDictionary(StorageFactory & factory); +void registerStorageSet(StorageFactory & factory); +void registerStorageJoin(StorageFactory & factory); +void registerStorageView(StorageFactory & factory); +void registerStorageMaterializedView(StorageFactory & factory); +void registerStorageLiveView(StorageFactory & factory); +void registerStorageGenerateRandom(StorageFactory & factory); + +#if USE_AWS_S3 +void registerStorageS3(StorageFactory & factory); +void registerStorageCOS(StorageFactory & factory); +#endif + +#if USE_HDFS +void registerStorageHDFS(StorageFactory & factory); +#endif + +void registerStorageODBC(StorageFactory & factory); +void registerStorageJDBC(StorageFactory & factory); + +#if USE_MYSQL +void registerStorageMySQL(StorageFactory & factory); +#endif + +void registerStorageMongoDB(StorageFactory & factory); + +#if USE_RDKAFKA +void registerStorageKafka(StorageFactory & factory); +#endif + +#if USE_AMQPCPP +void registerStorageRabbitMQ(StorageFactory & factory); +#endif + +#if USE_ROCKSDB +void registerStorageEmbeddedRocksDB(StorageFactory & factory); +#endif + + void registerStorages() { auto & factory = StorageFactory::instance(); @@ -28,7 +82,7 @@ void registerStorages() registerStorageLiveView(factory); registerStorageGenerateRandom(factory); -#if USE_AWS_S3 + #if USE_AWS_S3 registerStorageS3(factory); registerStorageCOS(factory); #endif diff --git a/src/Storages/registerStorages.h b/src/Storages/registerStorages.h index df7249a64af..d44b934ff9f 100644 --- a/src/Storages/registerStorages.h +++ b/src/Storages/registerStorages.h @@ -1,63 +1,6 @@ #pragma once -#if !defined(ARCADIA_BUILD) -# include -# include "config_core.h" -#endif - namespace DB { -class StorageFactory; - -void registerStorageLog(StorageFactory & factory); -void registerStorageTinyLog(StorageFactory & factory); -void registerStorageStripeLog(StorageFactory & factory); -void registerStorageMergeTree(StorageFactory & factory); -void registerStorageNull(StorageFactory & factory); -void registerStorageMerge(StorageFactory & factory); -void registerStorageBuffer(StorageFactory & factory); -void registerStorageDistributed(StorageFactory & factory); -void registerStorageMemory(StorageFactory & factory); -void registerStorageFile(StorageFactory & factory); -void registerStorageURL(StorageFactory & factory); -void registerStorageDictionary(StorageFactory & factory); -void registerStorageSet(StorageFactory & factory); -void registerStorageJoin(StorageFactory & factory); -void registerStorageView(StorageFactory & factory); -void registerStorageMaterializedView(StorageFactory & factory); -void registerStorageLiveView(StorageFactory & factory); -void registerStorageGenerateRandom(StorageFactory & factory); - -#if USE_AWS_S3 -void registerStorageS3(StorageFactory & factory); -void registerStorageCOS(StorageFactory & factory); -#endif - -#if USE_HDFS -void registerStorageHDFS(StorageFactory & factory); -#endif - -void registerStorageODBC(StorageFactory & factory); -void registerStorageJDBC(StorageFactory & factory); - -#if USE_MYSQL -void registerStorageMySQL(StorageFactory & factory); -#endif - -void registerStorageMongoDB(StorageFactory & factory); - -#if USE_RDKAFKA -void registerStorageKafka(StorageFactory & factory); -#endif - -#if USE_AMQPCPP -void registerStorageRabbitMQ(StorageFactory & factory); -#endif - -#if USE_ROCKSDB -void registerStorageEmbeddedRocksDB(StorageFactory & factory); -#endif - void registerStorages(); - } From 3915d65c2fa309478acfef65d9db97401549e4e6 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 8 Nov 2020 18:52:55 +0300 Subject: [PATCH 118/205] Fix bad wording --- src/Storages/RocksDB/StorageEmbeddedRocksDB.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Storages/RocksDB/StorageEmbeddedRocksDB.cpp b/src/Storages/RocksDB/StorageEmbeddedRocksDB.cpp index cad33de5c25..0aef9ccd705 100644 --- a/src/Storages/RocksDB/StorageEmbeddedRocksDB.cpp +++ b/src/Storages/RocksDB/StorageEmbeddedRocksDB.cpp @@ -348,13 +348,13 @@ static StoragePtr create(const StorageFactory::Arguments & args) metadata.setConstraints(args.constraints); if (!args.storage_def->primary_key) - throw Exception("StorageEmbeddedRocksDB must require one primary key", ErrorCodes::BAD_ARGUMENTS); + throw Exception("StorageEmbeddedRocksDB must require one column in primary key", ErrorCodes::BAD_ARGUMENTS); metadata.primary_key = KeyDescription::getKeyFromAST(args.storage_def->primary_key->ptr(), metadata.columns, args.context); auto primary_key_names = metadata.getColumnsRequiredForPrimaryKey(); if (primary_key_names.size() != 1) { - throw Exception("StorageEmbeddedRocksDB must require one primary key", ErrorCodes::BAD_ARGUMENTS); + throw Exception("StorageEmbeddedRocksDB must require one column in primary key", ErrorCodes::BAD_ARGUMENTS); } return StorageEmbeddedRocksDB::create(args.table_id, args.relative_data_path, metadata, args.attach, args.context, primary_key_names[0]); } From 0001433b82852e3e488322e193f01d2b6945e1b9 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 8 Nov 2020 19:24:58 +0300 Subject: [PATCH 119/205] Better code --- src/Common/ErrorCodes.cpp | 1 + .../RocksDB/EmbeddedRocksDBBlockInputStream.cpp | 1 + .../RocksDB/EmbeddedRocksDBBlockInputStream.h | 11 ++++++++--- .../RocksDB/EmbeddedRocksDBBlockOutputStream.cpp | 5 +++-- .../RocksDB/EmbeddedRocksDBBlockOutputStream.h | 4 +++- 5 files changed, 16 insertions(+), 6 deletions(-) diff --git a/src/Common/ErrorCodes.cpp b/src/Common/ErrorCodes.cpp index f069b27827e..74daa4ebbed 100644 --- a/src/Common/ErrorCodes.cpp +++ b/src/Common/ErrorCodes.cpp @@ -519,6 +519,7 @@ M(550, CONDITIONAL_TREE_PARENT_NOT_FOUND) \ M(551, ILLEGAL_PROJECTION_MANIPULATOR) \ M(552, UNRECOGNIZED_ARGUMENTS) \ + M(553, ROCKSDB_ERROR) \ \ M(999, KEEPER_EXCEPTION) \ M(1000, POCO_EXCEPTION) \ diff --git a/src/Storages/RocksDB/EmbeddedRocksDBBlockInputStream.cpp b/src/Storages/RocksDB/EmbeddedRocksDBBlockInputStream.cpp index 11944c7e295..e1a5452661f 100644 --- a/src/Storages/RocksDB/EmbeddedRocksDBBlockInputStream.cpp +++ b/src/Storages/RocksDB/EmbeddedRocksDBBlockInputStream.cpp @@ -2,6 +2,7 @@ #include #include #include +#include #include diff --git a/src/Storages/RocksDB/EmbeddedRocksDBBlockInputStream.h b/src/Storages/RocksDB/EmbeddedRocksDBBlockInputStream.h index cc7911e0087..3de04d70f42 100644 --- a/src/Storages/RocksDB/EmbeddedRocksDBBlockInputStream.h +++ b/src/Storages/RocksDB/EmbeddedRocksDBBlockInputStream.h @@ -1,13 +1,18 @@ #pragma once #include -#include -#include + +namespace rocksdb +{ + class Iterator; +} namespace DB { +class StorageEmbeddedRocksDB; + class EmbeddedRocksDBBlockInputStream : public IBlockInputStream { @@ -15,7 +20,7 @@ public: EmbeddedRocksDBBlockInputStream( StorageEmbeddedRocksDB & storage_, const StorageMetadataPtr & metadata_snapshot_, size_t max_block_size_); - String getName() const override { return storage.getName(); } + String getName() const override { return "EmbeddedRocksDB"; } Block getHeader() const override { return sample_block; } Block readImpl() override; diff --git a/src/Storages/RocksDB/EmbeddedRocksDBBlockOutputStream.cpp b/src/Storages/RocksDB/EmbeddedRocksDBBlockOutputStream.cpp index e7180c18bf4..a6c118d84d3 100644 --- a/src/Storages/RocksDB/EmbeddedRocksDBBlockOutputStream.cpp +++ b/src/Storages/RocksDB/EmbeddedRocksDBBlockOutputStream.cpp @@ -1,5 +1,6 @@ #include +#include #include namespace DB @@ -7,7 +8,7 @@ namespace DB namespace ErrorCodes { - extern const int SYSTEM_ERROR; + extern const int ROCKSDB_ERROR; } Block EmbeddedRocksDBBlockOutputStream::getHeader() const @@ -44,7 +45,7 @@ void EmbeddedRocksDBBlockOutputStream::write(const Block & block) } auto status = storage.rocksdb_ptr->Write(rocksdb::WriteOptions(), &batch); if (!status.ok()) - throw Exception("RocksDB write error: " + status.ToString(), ErrorCodes::SYSTEM_ERROR); + throw Exception("RocksDB write error: " + status.ToString(), ErrorCodes::ROCKSDB_ERROR); } } diff --git a/src/Storages/RocksDB/EmbeddedRocksDBBlockOutputStream.h b/src/Storages/RocksDB/EmbeddedRocksDBBlockOutputStream.h index a1412b90856..4ed5867cb72 100644 --- a/src/Storages/RocksDB/EmbeddedRocksDBBlockOutputStream.h +++ b/src/Storages/RocksDB/EmbeddedRocksDBBlockOutputStream.h @@ -1,12 +1,14 @@ #pragma once #include -#include #include + namespace DB { +class StorageEmbeddedRocksDB; + class EmbeddedRocksDBBlockOutputStream : public IBlockOutputStream { public: From 73e2f1b7edb0d4b9d7d96a036762015d1e8880ee Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 8 Nov 2020 20:09:41 +0300 Subject: [PATCH 120/205] Improve performance two times --- .../EmbeddedRocksDBBlockInputStream.cpp | 11 ++++--- .../EmbeddedRocksDBBlockOutputStream.cpp | 29 +++++++++++++------ .../EmbeddedRocksDBBlockOutputStream.h | 8 ++--- 3 files changed, 30 insertions(+), 18 deletions(-) diff --git a/src/Storages/RocksDB/EmbeddedRocksDBBlockInputStream.cpp b/src/Storages/RocksDB/EmbeddedRocksDBBlockInputStream.cpp index e1a5452661f..767c183f8d5 100644 --- a/src/Storages/RocksDB/EmbeddedRocksDBBlockInputStream.cpp +++ b/src/Storages/RocksDB/EmbeddedRocksDBBlockInputStream.cpp @@ -12,7 +12,7 @@ namespace DB namespace ErrorCodes { - extern const int LOGICAL_ERROR; + extern const int ROCKSDB_ERROR; } EmbeddedRocksDBBlockInputStream::EmbeddedRocksDBBlockInputStream( @@ -39,15 +39,18 @@ Block EmbeddedRocksDBBlockInputStream::readImpl() } MutableColumns columns = sample_block.cloneEmptyColumns(); + size_t rows = 0; for (; iterator->Valid(); iterator->Next()) { ReadBufferFromString key_buffer(iterator->key()); ReadBufferFromString value_buffer(iterator->value()); - for (const auto [idx, column_type] : ext::enumerate(sample_block.getColumnsWithTypeAndName())) + size_t idx = 0; + for (const auto & elem : sample_block) { - column_type.type->deserializeBinary(*columns[idx], idx == primary_key_pos? key_buffer: value_buffer); + elem.type->deserializeBinary(*columns[idx], idx == primary_key_pos ? key_buffer : value_buffer); + ++idx; } ++rows; if (rows >= max_block_size) @@ -58,7 +61,7 @@ Block EmbeddedRocksDBBlockInputStream::readImpl() if (!iterator->status().ok()) { throw Exception("Engine " + getName() + " got error while seeking key value datas: " + iterator->status().ToString(), - ErrorCodes::LOGICAL_ERROR); + ErrorCodes::ROCKSDB_ERROR); } return sample_block.cloneWithColumns(std::move(columns)); } diff --git a/src/Storages/RocksDB/EmbeddedRocksDBBlockOutputStream.cpp b/src/Storages/RocksDB/EmbeddedRocksDBBlockOutputStream.cpp index a6c118d84d3..b00901d6033 100644 --- a/src/Storages/RocksDB/EmbeddedRocksDBBlockOutputStream.cpp +++ b/src/Storages/RocksDB/EmbeddedRocksDBBlockOutputStream.cpp @@ -11,6 +11,21 @@ namespace ErrorCodes extern const int ROCKSDB_ERROR; } +EmbeddedRocksDBBlockOutputStream::EmbeddedRocksDBBlockOutputStream( + StorageEmbeddedRocksDB & storage_, + const StorageMetadataPtr & metadata_snapshot_) + : storage(storage_) + , metadata_snapshot(metadata_snapshot_) +{ + Block sample_block = metadata_snapshot->getSampleBlock(); + for (const auto & elem : sample_block) + { + if (elem.name == storage.primary_key) + break; + ++primary_key_pos; + } +} + Block EmbeddedRocksDBBlockOutputStream::getHeader() const { return metadata_snapshot->getSampleBlock(); @@ -25,24 +40,20 @@ void EmbeddedRocksDBBlockOutputStream::write(const Block & block) WriteBufferFromOwnString wb_value; rocksdb::WriteBatch batch; - auto columns = metadata_snapshot->getColumns(); - for (size_t i = 0; i < rows; i++) { wb_key.restart(); wb_value.restart(); - for (const auto & col : columns) + size_t idx = 0; + for (const auto & elem : block) { - const auto & type = block.getByName(col.name).type; - const auto & column = block.getByName(col.name).column; - if (col.name == storage.primary_key) - type->serializeBinary(*column, i, wb_key); - else - type->serializeBinary(*column, i, wb_value); + elem.type->serializeBinary(*elem.column, i, idx == primary_key_pos ? wb_key : wb_value); + ++idx; } batch.Put(wb_key.str(), wb_value.str()); } + auto status = storage.rocksdb_ptr->Write(rocksdb::WriteOptions(), &batch); if (!status.ok()) throw Exception("RocksDB write error: " + status.ToString(), ErrorCodes::ROCKSDB_ERROR); diff --git a/src/Storages/RocksDB/EmbeddedRocksDBBlockOutputStream.h b/src/Storages/RocksDB/EmbeddedRocksDBBlockOutputStream.h index 4ed5867cb72..51768df11ac 100644 --- a/src/Storages/RocksDB/EmbeddedRocksDBBlockOutputStream.h +++ b/src/Storages/RocksDB/EmbeddedRocksDBBlockOutputStream.h @@ -12,12 +12,9 @@ class StorageEmbeddedRocksDB; class EmbeddedRocksDBBlockOutputStream : public IBlockOutputStream { public: - explicit EmbeddedRocksDBBlockOutputStream( + EmbeddedRocksDBBlockOutputStream( StorageEmbeddedRocksDB & storage_, - const StorageMetadataPtr & metadata_snapshot_) - : storage(storage_) - , metadata_snapshot(metadata_snapshot_) - {} + const StorageMetadataPtr & metadata_snapshot_); Block getHeader() const override; void write(const Block & block) override; @@ -25,6 +22,7 @@ public: private: StorageEmbeddedRocksDB & storage; StorageMetadataPtr metadata_snapshot; + size_t primary_key_pos = 0; }; } From 33599ae9b0255490ee195bcd9051bdb3bd0b053d Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 8 Nov 2020 20:13:17 +0300 Subject: [PATCH 121/205] Remove unneeded dir --- src/Storages/RocksDB/StorageEmbeddedRocksDB.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/RocksDB/StorageEmbeddedRocksDB.cpp b/src/Storages/RocksDB/StorageEmbeddedRocksDB.cpp index 0aef9ccd705..38025a997d2 100644 --- a/src/Storages/RocksDB/StorageEmbeddedRocksDB.cpp +++ b/src/Storages/RocksDB/StorageEmbeddedRocksDB.cpp @@ -247,7 +247,7 @@ StorageEmbeddedRocksDB::StorageEmbeddedRocksDB(const StorageID & table_id_, : IStorage(table_id_), primary_key{primary_key_} { setInMemoryMetadata(metadata_); - rocksdb_dir = context_.getPath() + relative_data_path_ + "/rocksdb"; + rocksdb_dir = context_.getPath() + relative_data_path_; if (!attach) { Poco::File(rocksdb_dir).createDirectories(); From 07fbab9645bb37c367cf1b69cbec7fb39ef37c48 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 8 Nov 2020 20:22:21 +0300 Subject: [PATCH 122/205] More normal --- .../EmbeddedRocksDBBlockInputStream.cpp | 3 +- .../EmbeddedRocksDBBlockOutputStream.cpp | 4 ++- .../RocksDB/StorageEmbeddedRocksDB.cpp | 28 +++++++++---------- src/Storages/RocksDB/StorageEmbeddedRocksDB.h | 14 ++++++---- 4 files changed, 26 insertions(+), 23 deletions(-) diff --git a/src/Storages/RocksDB/EmbeddedRocksDBBlockInputStream.cpp b/src/Storages/RocksDB/EmbeddedRocksDBBlockInputStream.cpp index 767c183f8d5..b7602f8bedb 100644 --- a/src/Storages/RocksDB/EmbeddedRocksDBBlockInputStream.cpp +++ b/src/Storages/RocksDB/EmbeddedRocksDBBlockInputStream.cpp @@ -2,9 +2,8 @@ #include #include #include -#include -#include +#include namespace DB diff --git a/src/Storages/RocksDB/EmbeddedRocksDBBlockOutputStream.cpp b/src/Storages/RocksDB/EmbeddedRocksDBBlockOutputStream.cpp index b00901d6033..a3a0ddb4348 100644 --- a/src/Storages/RocksDB/EmbeddedRocksDBBlockOutputStream.cpp +++ b/src/Storages/RocksDB/EmbeddedRocksDBBlockOutputStream.cpp @@ -1,8 +1,10 @@ - #include #include #include +#include + + namespace DB { diff --git a/src/Storages/RocksDB/StorageEmbeddedRocksDB.cpp b/src/Storages/RocksDB/StorageEmbeddedRocksDB.cpp index 38025a997d2..977b80bc348 100644 --- a/src/Storages/RocksDB/StorageEmbeddedRocksDB.cpp +++ b/src/Storages/RocksDB/StorageEmbeddedRocksDB.cpp @@ -1,10 +1,10 @@ -#include -#include -#include -#include #include #include #include + +#include +#include + #include #include #include @@ -12,28 +12,26 @@ #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 { @@ -213,9 +211,11 @@ public: ReadBufferFromString key_buffer(slices_keys[i]); ReadBufferFromString value_buffer(values[i]); - for (const auto [idx, column_type] : ext::enumerate(sample_block.getColumnsWithTypeAndName())) + size_t idx = 0; + for (const auto & elem : sample_block) { - column_type.type->deserializeBinary(*columns[idx], idx == primary_key_pos? key_buffer: value_buffer); + elem.type->deserializeBinary(*columns[idx], idx == primary_key_pos ? key_buffer : value_buffer); + ++idx; } } } diff --git a/src/Storages/RocksDB/StorageEmbeddedRocksDB.h b/src/Storages/RocksDB/StorageEmbeddedRocksDB.h index 209f70ceb52..bd700a35809 100644 --- a/src/Storages/RocksDB/StorageEmbeddedRocksDB.h +++ b/src/Storages/RocksDB/StorageEmbeddedRocksDB.h @@ -1,14 +1,15 @@ #pragma once +#include #include #include -#include -#include -#include -#include "rocksdb/db.h" -#include "rocksdb/table.h" +namespace rocksdb +{ + class DB; +} + namespace DB { @@ -38,7 +39,8 @@ public: bool supportsParallelInsert() const override { return true; } bool supportsIndexForIn() const override { return true; } - bool mayBenefitFromIndexForIn(const ASTPtr & node, const Context & /*query_context*/, const StorageMetadataPtr & /*metadata_snapshot*/) const override + bool mayBenefitFromIndexForIn( + const ASTPtr & node, const Context & /*query_context*/, const StorageMetadataPtr & /*metadata_snapshot*/) const override { return node->getColumnName() == primary_key; } From 765c9eaeed22cac74a10ded7af81ba4adf725852 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 8 Nov 2020 20:29:16 +0300 Subject: [PATCH 123/205] Fix improper error handling --- src/Storages/RocksDB/StorageEmbeddedRocksDB.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Storages/RocksDB/StorageEmbeddedRocksDB.cpp b/src/Storages/RocksDB/StorageEmbeddedRocksDB.cpp index 977b80bc348..84de695d63b 100644 --- a/src/Storages/RocksDB/StorageEmbeddedRocksDB.cpp +++ b/src/Storages/RocksDB/StorageEmbeddedRocksDB.cpp @@ -38,7 +38,7 @@ namespace DB namespace ErrorCodes { extern const int BAD_ARGUMENTS; - extern const int SYSTEM_ERROR; + extern const int ROCKSDB_ERROR; extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; } @@ -271,7 +271,7 @@ void StorageEmbeddedRocksDB::initDb() rocksdb::Status status = rocksdb::DB::Open(options, rocksdb_dir, &db); if (status != rocksdb::Status::OK()) - throw Exception("Fail to open rocksdb path at: " + rocksdb_dir, ErrorCodes::SYSTEM_ERROR); + throw Exception("Fail to open rocksdb path at: " + rocksdb_dir + ": " + status.ToString(), ErrorCodes::ROCKSDB_ERROR); rocksdb_ptr = std::unique_ptr(db); } From c56d1212a259007f5cc0d95087742935e2a26fa1 Mon Sep 17 00:00:00 2001 From: sundy-li <543950155@qq.com> Date: Wed, 11 Nov 2020 09:39:09 +0800 Subject: [PATCH 124/205] rocksdb-cmake && fix iteration bug && improve tests --- contrib/CMakeLists.txt | 16 +- contrib/rocksdb-cmake/CMakeLists.txt | 732 ++++++++++++++++++ .../EmbeddedRocksDBBlockInputStream.cpp | 17 +- .../RocksDB/EmbeddedRocksDBBlockInputStream.h | 3 +- .../EmbeddedRocksDBBlockOutputStream.cpp | 11 +- .../EmbeddedRocksDBBlockOutputStream.h | 3 +- .../RocksDB/StorageEmbeddedRocksDB.cpp | 1 + .../0_stateless/01504_rocksdb.reference | 4 +- tests/queries/0_stateless/01504_rocksdb.sql | 21 +- 9 files changed, 766 insertions(+), 42 deletions(-) create mode 100644 contrib/rocksdb-cmake/CMakeLists.txt diff --git a/contrib/CMakeLists.txt b/contrib/CMakeLists.txt index 15194b8a2a8..0e60102bd04 100644 --- a/contrib/CMakeLists.txt +++ b/contrib/CMakeLists.txt @@ -322,19 +322,5 @@ if (USE_KRB5) endif() if (USE_INTERNAL_ROCKSDB_LIBRARY) - set(WITH_TESTS OFF) - set(WITH_BENCHMARK_TOOLS OFF) - set(WITH_TOOLS OFF) - set(WITH_GFLAGS OFF) - set(PORTABLE ON) - - if (SANITIZE STREQUAL "undefined") - set(WITH_UBSAN ON) - elseif (SANITIZE STREQUAL "address") - set(WITH_ASAN ON) - elseif (SANITIZE STREQUAL "thread") - set(WITH_TSAN ON) - endif() - - add_subdirectory (rocksdb) + add_subdirectory(rocksdb-cmake) endif() diff --git a/contrib/rocksdb-cmake/CMakeLists.txt b/contrib/rocksdb-cmake/CMakeLists.txt new file mode 100644 index 00000000000..2a6d95e5bc5 --- /dev/null +++ b/contrib/rocksdb-cmake/CMakeLists.txt @@ -0,0 +1,732 @@ +## this file is extracted from `contrib/rocksdb/CMakeLists.txt` +set(ROCKSDB_SOURCE_DIR "${ClickHouse_SOURCE_DIR}/contrib/rocksdb") +list(APPEND CMAKE_MODULE_PATH "${ROCKSDB_SOURCE_DIR}/cmake/modules/") + +find_program(CCACHE_FOUND ccache) +if(CCACHE_FOUND) + set_property(GLOBAL PROPERTY RULE_LAUNCH_COMPILE ccache) + set_property(GLOBAL PROPERTY RULE_LAUNCH_LINK ccache) +endif(CCACHE_FOUND) + +if (SANITIZE STREQUAL "undefined") + set(WITH_UBSAN ON) +elseif (SANITIZE STREQUAL "address") + set(WITH_ASAN ON) +elseif (SANITIZE STREQUAL "thread") + set(WITH_TSAN ON) +endif() + +set(PORTABLE ON) +option(WITH_JEMALLOC "build with JeMalloc" ${ENABLE_JEMALLOC}) +option(WITH_SNAPPY "build with SNAPPY" ON) +option(WITH_LZ4 "build with lz4" ON) +option(WITH_ZLIB "build with zlib" ON) +option(WITH_ZSTD "build with zstd" ON) + +# third-party/folly is only validated to work on Linux and Windows for now. +# So only turn it on there by default. +if(CMAKE_SYSTEM_NAME MATCHES "Linux|Windows") + if(MSVC AND MSVC_VERSION LESS 1910) + # Folly does not compile with MSVC older than VS2017 + option(WITH_FOLLY_DISTRIBUTED_MUTEX "build with folly::DistributedMutex" OFF) + else() + option(WITH_FOLLY_DISTRIBUTED_MUTEX "build with folly::DistributedMutex" ON) + endif() +else() + option(WITH_FOLLY_DISTRIBUTED_MUTEX "build with folly::DistributedMutex" OFF) +endif() + +if( NOT DEFINED CMAKE_CXX_STANDARD ) + set(CMAKE_CXX_STANDARD 11) +endif() + +if(MSVC) + option(WITH_XPRESS "build with windows built in compression" OFF) + include(${ROCKSDB_SOURCE_DIR}/thirdparty.inc) +else() + if(CMAKE_SYSTEM_NAME MATCHES "FreeBSD" AND NOT CMAKE_SYSTEM_NAME MATCHES "kFreeBSD") + # FreeBSD has jemalloc as default malloc + # but it does not have all the jemalloc files in include/... + set(WITH_JEMALLOC ON) + else() + if(WITH_JEMALLOC) + find_package(JeMalloc REQUIRED) + add_definitions(-DROCKSDB_JEMALLOC -DJEMALLOC_NO_DEMANGLE) + list(APPEND THIRDPARTY_LIBS JeMalloc::JeMalloc) + endif() + endif() + + if(WITH_SNAPPY) + find_package(Snappy CONFIG) + if(NOT Snappy_FOUND) + find_package(Snappy REQUIRED) + endif() + add_definitions(-DSNAPPY) + list(APPEND THIRDPARTY_LIBS Snappy::snappy) + endif() + + if(WITH_ZLIB) + find_package(ZLIB REQUIRED) + add_definitions(-DZLIB) + list(APPEND THIRDPARTY_LIBS ZLIB::ZLIB) + endif() + + option(WITH_BZ2 "build with bzip2" OFF) + if(WITH_BZ2) + find_package(BZip2 REQUIRED) + add_definitions(-DBZIP2) + if(BZIP2_INCLUDE_DIRS) + include_directories(${BZIP2_INCLUDE_DIRS}) + else() + include_directories(${BZIP2_INCLUDE_DIR}) + endif() + list(APPEND THIRDPARTY_LIBS ${BZIP2_LIBRARIES}) + endif() + + if(WITH_LZ4) + find_package(lz4 REQUIRED) + add_definitions(-DLZ4) + list(APPEND THIRDPARTY_LIBS lz4::lz4) + endif() + + if(WITH_ZSTD) + find_package(zstd REQUIRED) + add_definitions(-DZSTD) + include_directories(${ZSTD_INCLUDE_DIR}) + list(APPEND THIRDPARTY_LIBS zstd::zstd) + endif() +endif() + +string(TIMESTAMP TS "%Y/%m/%d %H:%M:%S" UTC) +set(GIT_DATE_TIME "${TS}" CACHE STRING "the time we first built rocksdb") + +find_package(Git) + +if(GIT_FOUND AND EXISTS "${ROCKSDB_SOURCE_DIR}/.git") + if(WIN32) + execute_process(COMMAND $ENV{COMSPEC} /C ${GIT_EXECUTABLE} -C ${ROCKSDB_SOURCE_DIR} rev-parse HEAD OUTPUT_VARIABLE GIT_SHA) + else() + execute_process(COMMAND ${GIT_EXECUTABLE} -C ${ROCKSDB_SOURCE_DIR} rev-parse HEAD OUTPUT_VARIABLE GIT_SHA) + endif() +else() + set(GIT_SHA 0) +endif() + +string(REGEX REPLACE "[^0-9a-f]+" "" GIT_SHA "${GIT_SHA}") + +set(BUILD_VERSION_CC ${CMAKE_BINARY_DIR}/rocksdb_build_version.cc) +configure_file(${ROCKSDB_SOURCE_DIR}/util/build_version.cc.in ${BUILD_VERSION_CC} @ONLY) +add_library(rocksdb_build_version OBJECT ${BUILD_VERSION_CC}) +target_include_directories(rocksdb_build_version PRIVATE + ${ROCKSDB_SOURCE_DIR}/util) +if(MSVC) + set(CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} /Zi /nologo /EHsc /GS /Gd /GR /GF /fp:precise /Zc:wchar_t /Zc:forScope /errorReport:queue") + set(CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} /FC /d2Zi+ /W4 /wd4127 /wd4800 /wd4996 /wd4351 /wd4100 /wd4204 /wd4324") +else() + set(CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} -W -Wextra -Wall") + set(CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} -Wsign-compare -Wshadow -Wno-unused-parameter -Wno-unused-variable -Woverloaded-virtual -Wnon-virtual-dtor -Wno-missing-field-initializers -Wno-strict-aliasing") + if(MINGW) + set(CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} -Wno-format -fno-asynchronous-unwind-tables") + add_definitions(-D_POSIX_C_SOURCE=1) + endif() + if(NOT CMAKE_BUILD_TYPE STREQUAL "Debug") + set(CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} -fno-omit-frame-pointer") + include(CheckCXXCompilerFlag) + CHECK_CXX_COMPILER_FLAG("-momit-leaf-frame-pointer" HAVE_OMIT_LEAF_FRAME_POINTER) + if(HAVE_OMIT_LEAF_FRAME_POINTER) + set(CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} -momit-leaf-frame-pointer") + endif() + endif() +endif() + +include(CheckCCompilerFlag) +if(CMAKE_SYSTEM_PROCESSOR MATCHES "^(powerpc|ppc)64") + CHECK_C_COMPILER_FLAG("-mcpu=power9" HAS_POWER9) + if(HAS_POWER9) + set(CMAKE_C_FLAGS "${CMAKE_C_FLAGS} -mcpu=power9 -mtune=power9") + set(CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} -mcpu=power9 -mtune=power9") + else() + CHECK_C_COMPILER_FLAG("-mcpu=power8" HAS_POWER8) + if(HAS_POWER8) + set(CMAKE_C_FLAGS "${CMAKE_C_FLAGS} -mcpu=power8 -mtune=power8") + set(CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} -mcpu=power8 -mtune=power8") + endif(HAS_POWER8) + endif(HAS_POWER9) + CHECK_C_COMPILER_FLAG("-maltivec" HAS_ALTIVEC) + if(HAS_ALTIVEC) + message(STATUS " HAS_ALTIVEC yes") + set(CMAKE_C_FLAGS "${CMAKE_C_FLAGS} -maltivec") + set(CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} -maltivec") + endif(HAS_ALTIVEC) +endif(CMAKE_SYSTEM_PROCESSOR MATCHES "^(powerpc|ppc)64") + +if(CMAKE_SYSTEM_PROCESSOR MATCHES "aarch64|AARCH64") + CHECK_C_COMPILER_FLAG("-march=armv8-a+crc+crypto" HAS_ARMV8_CRC) + if(HAS_ARMV8_CRC) + message(STATUS " HAS_ARMV8_CRC yes") + set(CMAKE_C_FLAGS "${CMAKE_C_FLAGS} -march=armv8-a+crc+crypto -Wno-unused-function") + set(CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} -march=armv8-a+crc+crypto -Wno-unused-function") + endif(HAS_ARMV8_CRC) +endif(CMAKE_SYSTEM_PROCESSOR MATCHES "aarch64|AARCH64") + + +include(CheckCXXSourceCompiles) +if(NOT MSVC) + set(CMAKE_REQUIRED_FLAGS "-msse4.2 -mpclmul") +endif() + +CHECK_CXX_SOURCE_COMPILES(" +#include +#include +#include +int main() { + volatile uint32_t x = _mm_crc32_u32(0, 0); + const auto a = _mm_set_epi64x(0, 0); + const auto b = _mm_set_epi64x(0, 0); + const auto c = _mm_clmulepi64_si128(a, b, 0x00); + auto d = _mm_cvtsi128_si64(c); +} +" HAVE_SSE42) +unset(CMAKE_REQUIRED_FLAGS) +if(HAVE_SSE42) + add_definitions(-DHAVE_SSE42) + add_definitions(-DHAVE_PCLMUL) +elseif(FORCE_SSE42) + message(FATAL_ERROR "FORCE_SSE42=ON but unable to compile with SSE4.2 enabled") +endif() + +CHECK_CXX_SOURCE_COMPILES(" +#if defined(_MSC_VER) && !defined(__thread) +#define __thread __declspec(thread) +#endif +int main() { + static __thread int tls; +} +" HAVE_THREAD_LOCAL) +if(HAVE_THREAD_LOCAL) + add_definitions(-DROCKSDB_SUPPORT_THREAD_LOCAL) +endif() + +option(FAIL_ON_WARNINGS "Treat compile warnings as errors" ON) +if(FAIL_ON_WARNINGS) + if(MSVC) + set(CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} /WX") + else() # assume GCC + set(CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} -Werror") + endif() +endif() + +option(WITH_ASAN "build with ASAN" OFF) +if(WITH_ASAN) + set(CMAKE_EXE_LINKER_FLAGS "${CMAKE_EXE_LINKER_FLAGS} -fsanitize=address") + set(CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} -fsanitize=address") + set(CMAKE_C_FLAGS "${CMAKE_C_FLAGS} -fsanitize=address") + if(WITH_JEMALLOC) + message(FATAL "ASAN does not work well with JeMalloc") + endif() +endif() + +option(WITH_TSAN "build with TSAN" OFF) +if(WITH_TSAN) + set(CMAKE_EXE_LINKER_FLAGS "${CMAKE_EXE_LINKER_FLAGS} -fsanitize=thread -pie") + set(CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} -fsanitize=thread -fPIC") + set(CMAKE_C_FLAGS "${CMAKE_C_FLAGS} -fsanitize=thread -fPIC") + if(WITH_JEMALLOC) + message(FATAL "TSAN does not work well with JeMalloc") + endif() +endif() + +option(WITH_UBSAN "build with UBSAN" OFF) +if(WITH_UBSAN) + add_definitions(-DROCKSDB_UBSAN_RUN) + set(CMAKE_EXE_LINKER_FLAGS "${CMAKE_EXE_LINKER_FLAGS} -fsanitize=undefined") + set(CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} -fsanitize=undefined") + set(CMAKE_C_FLAGS "${CMAKE_C_FLAGS} -fsanitize=undefined") + if(WITH_JEMALLOC) + message(FATAL "UBSAN does not work well with JeMalloc") + endif() +endif() + +option(WITH_NUMA "build with NUMA policy support" OFF) +if(WITH_NUMA) + find_package(NUMA REQUIRED) + add_definitions(-DNUMA) + include_directories(${NUMA_INCLUDE_DIR}) + list(APPEND THIRDPARTY_LIBS NUMA::NUMA) +endif() + +option(WITH_TBB "build with Threading Building Blocks (TBB)" OFF) +if(WITH_TBB) + find_package(TBB REQUIRED) + add_definitions(-DTBB) + list(APPEND THIRDPARTY_LIBS TBB::TBB) +endif() + +# Stall notifications eat some performance from inserts +option(DISABLE_STALL_NOTIF "Build with stall notifications" OFF) +if(DISABLE_STALL_NOTIF) + add_definitions(-DROCKSDB_DISABLE_STALL_NOTIFICATION) +endif() + +option(WITH_DYNAMIC_EXTENSION "build with dynamic extension support" OFF) +if(NOT WITH_DYNAMIC_EXTENSION) + add_definitions(-DROCKSDB_NO_DYNAMIC_EXTENSION) +endif() + +if(DEFINED USE_RTTI) + if(USE_RTTI) + message(STATUS "Enabling RTTI") + set(CMAKE_CXX_FLAGS_DEBUG "${CMAKE_CXX_FLAGS_DEBUG} -DROCKSDB_USE_RTTI") + set(CMAKE_CXX_FLAGS_RELEASE "${CMAKE_CXX_FLAGS_RELEASE} -DROCKSDB_USE_RTTI") + else() + if(MSVC) + message(STATUS "Disabling RTTI in Release builds. Always on in Debug.") + set(CMAKE_CXX_FLAGS_DEBUG "${CMAKE_CXX_FLAGS_DEBUG} -DROCKSDB_USE_RTTI") + set(CMAKE_CXX_FLAGS_RELEASE "${CMAKE_CXX_FLAGS_RELEASE} /GR-") + else() + message(STATUS "Disabling RTTI in Release builds") + set(CMAKE_CXX_FLAGS_DEBUG "${CMAKE_CXX_FLAGS_DEBUG} -fno-rtti") + set(CMAKE_CXX_FLAGS_RELEASE "${CMAKE_CXX_FLAGS_RELEASE} -fno-rtti") + endif() + endif() +else() + message(STATUS "Enabling RTTI in Debug builds only (default)") + set(CMAKE_CXX_FLAGS_DEBUG "${CMAKE_CXX_FLAGS_DEBUG} -DROCKSDB_USE_RTTI") + if(MSVC) + set(CMAKE_CXX_FLAGS_RELEASE "${CMAKE_CXX_FLAGS_RELEASE} /GR-") + else() + set(CMAKE_CXX_FLAGS_RELEASE "${CMAKE_CXX_FLAGS_RELEASE} -fno-rtti") + endif() +endif() + + +if(CMAKE_SYSTEM_NAME MATCHES "Cygwin") + add_definitions(-fno-builtin-memcmp -DCYGWIN) +elseif(CMAKE_SYSTEM_NAME MATCHES "Darwin") + add_definitions(-DOS_MACOSX) + if(CMAKE_SYSTEM_PROCESSOR MATCHES arm) + add_definitions(-DIOS_CROSS_COMPILE -DROCKSDB_LITE) + # no debug info for IOS, that will make our library big + add_definitions(-DNDEBUG) + endif() +elseif(CMAKE_SYSTEM_NAME MATCHES "Linux") + add_definitions(-DOS_LINUX) +elseif(CMAKE_SYSTEM_NAME MATCHES "SunOS") + add_definitions(-DOS_SOLARIS) +elseif(CMAKE_SYSTEM_NAME MATCHES "kFreeBSD") + add_definitions(-DOS_GNU_KFREEBSD) +elseif(CMAKE_SYSTEM_NAME MATCHES "FreeBSD") + add_definitions(-DOS_FREEBSD) +elseif(CMAKE_SYSTEM_NAME MATCHES "NetBSD") + add_definitions(-DOS_NETBSD) +elseif(CMAKE_SYSTEM_NAME MATCHES "OpenBSD") + add_definitions(-DOS_OPENBSD) +elseif(CMAKE_SYSTEM_NAME MATCHES "DragonFly") + add_definitions(-DOS_DRAGONFLYBSD) +elseif(CMAKE_SYSTEM_NAME MATCHES "Android") + add_definitions(-DOS_ANDROID) +elseif(CMAKE_SYSTEM_NAME MATCHES "Windows") + add_definitions(-DWIN32 -DOS_WIN -D_MBCS -DWIN64 -DNOMINMAX) + if(MINGW) + add_definitions(-D_WIN32_WINNT=_WIN32_WINNT_VISTA) + endif() +endif() + +if(NOT WIN32) + add_definitions(-DROCKSDB_PLATFORM_POSIX -DROCKSDB_LIB_IO_POSIX) +endif() + +option(WITH_FALLOCATE "build with fallocate" ON) +if(WITH_FALLOCATE) + CHECK_CXX_SOURCE_COMPILES(" +#include +#include +int main() { + int fd = open(\"/dev/null\", 0); + fallocate(fd, FALLOC_FL_KEEP_SIZE, 0, 1024); +} +" HAVE_FALLOCATE) + if(HAVE_FALLOCATE) + add_definitions(-DROCKSDB_FALLOCATE_PRESENT) + endif() +endif() + +CHECK_CXX_SOURCE_COMPILES(" +#include +int main() { + int fd = open(\"/dev/null\", 0); + sync_file_range(fd, 0, 1024, SYNC_FILE_RANGE_WRITE); +} +" HAVE_SYNC_FILE_RANGE_WRITE) +if(HAVE_SYNC_FILE_RANGE_WRITE) + add_definitions(-DROCKSDB_RANGESYNC_PRESENT) +endif() + +CHECK_CXX_SOURCE_COMPILES(" +#include +int main() { + (void) PTHREAD_MUTEX_ADAPTIVE_NP; +} +" HAVE_PTHREAD_MUTEX_ADAPTIVE_NP) +if(HAVE_PTHREAD_MUTEX_ADAPTIVE_NP) + add_definitions(-DROCKSDB_PTHREAD_ADAPTIVE_MUTEX) +endif() + +include(CheckCXXSymbolExists) +if(CMAKE_SYSTEM_NAME MATCHES "^FreeBSD") + check_cxx_symbol_exists(malloc_usable_size ${ROCKSDB_SOURCE_DIR}/malloc_np.h HAVE_MALLOC_USABLE_SIZE) +else() + check_cxx_symbol_exists(malloc_usable_size ${ROCKSDB_SOURCE_DIR}/malloc.h HAVE_MALLOC_USABLE_SIZE) +endif() +if(HAVE_MALLOC_USABLE_SIZE) + add_definitions(-DROCKSDB_MALLOC_USABLE_SIZE) +endif() + +check_cxx_symbol_exists(sched_getcpu sched.h HAVE_SCHED_GETCPU) +if(HAVE_SCHED_GETCPU) + add_definitions(-DROCKSDB_SCHED_GETCPU_PRESENT) +endif() + +check_cxx_symbol_exists(getauxval auvx.h HAVE_AUXV_GETAUXVAL) +if(HAVE_AUXV_GETAUXVAL) + add_definitions(-DROCKSDB_AUXV_GETAUXVAL_PRESENT) +endif() + +include_directories(${ROCKSDB_SOURCE_DIR}) +include_directories(${ROCKSDB_SOURCE_DIR}/include) +if(WITH_FOLLY_DISTRIBUTED_MUTEX) + include_directories(${ROCKSDB_SOURCE_DIR}/third-party/folly) +endif() +find_package(Threads REQUIRED) + +# Main library source code + +set(SOURCES + ${ROCKSDB_SOURCE_DIR}/cache/cache.cc + ${ROCKSDB_SOURCE_DIR}/cache/clock_cache.cc + ${ROCKSDB_SOURCE_DIR}/cache/lru_cache.cc + ${ROCKSDB_SOURCE_DIR}/cache/sharded_cache.cc + ${ROCKSDB_SOURCE_DIR}/db/arena_wrapped_db_iter.cc + ${ROCKSDB_SOURCE_DIR}/db/blob/blob_file_addition.cc + ${ROCKSDB_SOURCE_DIR}/db/blob/blob_file_builder.cc + ${ROCKSDB_SOURCE_DIR}/db/blob/blob_file_garbage.cc + ${ROCKSDB_SOURCE_DIR}/db/blob/blob_file_meta.cc + ${ROCKSDB_SOURCE_DIR}/db/blob/blob_log_format.cc + ${ROCKSDB_SOURCE_DIR}/db/blob/blob_log_reader.cc + ${ROCKSDB_SOURCE_DIR}/db/blob/blob_log_writer.cc + ${ROCKSDB_SOURCE_DIR}/db/builder.cc + ${ROCKSDB_SOURCE_DIR}/db/c.cc + ${ROCKSDB_SOURCE_DIR}/db/column_family.cc + ${ROCKSDB_SOURCE_DIR}/db/compacted_db_impl.cc + ${ROCKSDB_SOURCE_DIR}/db/compaction/compaction.cc + ${ROCKSDB_SOURCE_DIR}/db/compaction/compaction_iterator.cc + ${ROCKSDB_SOURCE_DIR}/db/compaction/compaction_picker.cc + ${ROCKSDB_SOURCE_DIR}/db/compaction/compaction_job.cc + ${ROCKSDB_SOURCE_DIR}/db/compaction/compaction_picker_fifo.cc + ${ROCKSDB_SOURCE_DIR}/db/compaction/compaction_picker_level.cc + ${ROCKSDB_SOURCE_DIR}/db/compaction/compaction_picker_universal.cc + ${ROCKSDB_SOURCE_DIR}/db/compaction/sst_partitioner.cc + ${ROCKSDB_SOURCE_DIR}/db/convenience.cc + ${ROCKSDB_SOURCE_DIR}/db/db_filesnapshot.cc + ${ROCKSDB_SOURCE_DIR}/db/db_impl/db_impl.cc + ${ROCKSDB_SOURCE_DIR}/db/db_impl/db_impl_write.cc + ${ROCKSDB_SOURCE_DIR}/db/db_impl/db_impl_compaction_flush.cc + ${ROCKSDB_SOURCE_DIR}/db/db_impl/db_impl_files.cc + ${ROCKSDB_SOURCE_DIR}/db/db_impl/db_impl_open.cc + ${ROCKSDB_SOURCE_DIR}/db/db_impl/db_impl_debug.cc + ${ROCKSDB_SOURCE_DIR}/db/db_impl/db_impl_experimental.cc + ${ROCKSDB_SOURCE_DIR}/db/db_impl/db_impl_readonly.cc + ${ROCKSDB_SOURCE_DIR}/db/db_impl/db_impl_secondary.cc + ${ROCKSDB_SOURCE_DIR}/db/db_info_dumper.cc + ${ROCKSDB_SOURCE_DIR}/db/db_iter.cc + ${ROCKSDB_SOURCE_DIR}/db/dbformat.cc + ${ROCKSDB_SOURCE_DIR}/db/error_handler.cc + ${ROCKSDB_SOURCE_DIR}/db/event_helpers.cc + ${ROCKSDB_SOURCE_DIR}/db/experimental.cc + ${ROCKSDB_SOURCE_DIR}/db/external_sst_file_ingestion_job.cc + ${ROCKSDB_SOURCE_DIR}/db/file_indexer.cc + ${ROCKSDB_SOURCE_DIR}/db/flush_job.cc + ${ROCKSDB_SOURCE_DIR}/db/flush_scheduler.cc + ${ROCKSDB_SOURCE_DIR}/db/forward_iterator.cc + ${ROCKSDB_SOURCE_DIR}/db/import_column_family_job.cc + ${ROCKSDB_SOURCE_DIR}/db/internal_stats.cc + ${ROCKSDB_SOURCE_DIR}/db/logs_with_prep_tracker.cc + ${ROCKSDB_SOURCE_DIR}/db/log_reader.cc + ${ROCKSDB_SOURCE_DIR}/db/log_writer.cc + ${ROCKSDB_SOURCE_DIR}/db/malloc_stats.cc + ${ROCKSDB_SOURCE_DIR}/db/memtable.cc + ${ROCKSDB_SOURCE_DIR}/db/memtable_list.cc + ${ROCKSDB_SOURCE_DIR}/db/merge_helper.cc + ${ROCKSDB_SOURCE_DIR}/db/merge_operator.cc + ${ROCKSDB_SOURCE_DIR}/db/range_del_aggregator.cc + ${ROCKSDB_SOURCE_DIR}/db/range_tombstone_fragmenter.cc + ${ROCKSDB_SOURCE_DIR}/db/repair.cc + ${ROCKSDB_SOURCE_DIR}/db/snapshot_impl.cc + ${ROCKSDB_SOURCE_DIR}/db/table_cache.cc + ${ROCKSDB_SOURCE_DIR}/db/table_properties_collector.cc + ${ROCKSDB_SOURCE_DIR}/db/transaction_log_impl.cc + ${ROCKSDB_SOURCE_DIR}/db/trim_history_scheduler.cc + ${ROCKSDB_SOURCE_DIR}/db/version_builder.cc + ${ROCKSDB_SOURCE_DIR}/db/version_edit.cc + ${ROCKSDB_SOURCE_DIR}/db/version_edit_handler.cc + ${ROCKSDB_SOURCE_DIR}/db/version_set.cc + ${ROCKSDB_SOURCE_DIR}/db/wal_edit.cc + ${ROCKSDB_SOURCE_DIR}/db/wal_manager.cc + ${ROCKSDB_SOURCE_DIR}/db/write_batch.cc + ${ROCKSDB_SOURCE_DIR}/db/write_batch_base.cc + ${ROCKSDB_SOURCE_DIR}/db/write_controller.cc + ${ROCKSDB_SOURCE_DIR}/db/write_thread.cc + ${ROCKSDB_SOURCE_DIR}/env/env.cc + ${ROCKSDB_SOURCE_DIR}/env/env_chroot.cc + ${ROCKSDB_SOURCE_DIR}/env/env_encryption.cc + ${ROCKSDB_SOURCE_DIR}/env/env_hdfs.cc + ${ROCKSDB_SOURCE_DIR}/env/file_system.cc + ${ROCKSDB_SOURCE_DIR}/env/file_system_tracer.cc + ${ROCKSDB_SOURCE_DIR}/env/mock_env.cc + ${ROCKSDB_SOURCE_DIR}/file/delete_scheduler.cc + ${ROCKSDB_SOURCE_DIR}/file/file_prefetch_buffer.cc + ${ROCKSDB_SOURCE_DIR}/file/file_util.cc + ${ROCKSDB_SOURCE_DIR}/file/filename.cc + ${ROCKSDB_SOURCE_DIR}/file/random_access_file_reader.cc + ${ROCKSDB_SOURCE_DIR}/file/read_write_util.cc + ${ROCKSDB_SOURCE_DIR}/file/readahead_raf.cc + ${ROCKSDB_SOURCE_DIR}/file/sequence_file_reader.cc + ${ROCKSDB_SOURCE_DIR}/file/sst_file_manager_impl.cc + ${ROCKSDB_SOURCE_DIR}/file/writable_file_writer.cc + ${ROCKSDB_SOURCE_DIR}/logging/auto_roll_logger.cc + ${ROCKSDB_SOURCE_DIR}/logging/event_logger.cc + ${ROCKSDB_SOURCE_DIR}/logging/log_buffer.cc + ${ROCKSDB_SOURCE_DIR}/memory/arena.cc + ${ROCKSDB_SOURCE_DIR}/memory/concurrent_arena.cc + ${ROCKSDB_SOURCE_DIR}/memory/jemalloc_nodump_allocator.cc + ${ROCKSDB_SOURCE_DIR}/memory/memkind_kmem_allocator.cc + ${ROCKSDB_SOURCE_DIR}/memtable/alloc_tracker.cc + ${ROCKSDB_SOURCE_DIR}/memtable/hash_linklist_rep.cc + ${ROCKSDB_SOURCE_DIR}/memtable/hash_skiplist_rep.cc + ${ROCKSDB_SOURCE_DIR}/memtable/skiplistrep.cc + ${ROCKSDB_SOURCE_DIR}/memtable/vectorrep.cc + ${ROCKSDB_SOURCE_DIR}/memtable/write_buffer_manager.cc + ${ROCKSDB_SOURCE_DIR}/monitoring/histogram.cc + ${ROCKSDB_SOURCE_DIR}/monitoring/histogram_windowing.cc + ${ROCKSDB_SOURCE_DIR}/monitoring/in_memory_stats_history.cc + ${ROCKSDB_SOURCE_DIR}/monitoring/instrumented_mutex.cc + ${ROCKSDB_SOURCE_DIR}/monitoring/iostats_context.cc + ${ROCKSDB_SOURCE_DIR}/monitoring/perf_context.cc + ${ROCKSDB_SOURCE_DIR}/monitoring/perf_level.cc + ${ROCKSDB_SOURCE_DIR}/monitoring/persistent_stats_history.cc + ${ROCKSDB_SOURCE_DIR}/monitoring/statistics.cc + ${ROCKSDB_SOURCE_DIR}/monitoring/stats_dump_scheduler.cc + ${ROCKSDB_SOURCE_DIR}/monitoring/thread_status_impl.cc + ${ROCKSDB_SOURCE_DIR}/monitoring/thread_status_updater.cc + ${ROCKSDB_SOURCE_DIR}/monitoring/thread_status_util.cc + ${ROCKSDB_SOURCE_DIR}/monitoring/thread_status_util_debug.cc + ${ROCKSDB_SOURCE_DIR}/options/cf_options.cc + ${ROCKSDB_SOURCE_DIR}/options/db_options.cc + ${ROCKSDB_SOURCE_DIR}/options/options.cc + ${ROCKSDB_SOURCE_DIR}/options/options_helper.cc + ${ROCKSDB_SOURCE_DIR}/options/options_parser.cc + ${ROCKSDB_SOURCE_DIR}/port/stack_trace.cc + ${ROCKSDB_SOURCE_DIR}/table/adaptive/adaptive_table_factory.cc + ${ROCKSDB_SOURCE_DIR}/table/block_based/binary_search_index_reader.cc + ${ROCKSDB_SOURCE_DIR}/table/block_based/block.cc + ${ROCKSDB_SOURCE_DIR}/table/block_based/block_based_filter_block.cc + ${ROCKSDB_SOURCE_DIR}/table/block_based/block_based_table_builder.cc + ${ROCKSDB_SOURCE_DIR}/table/block_based/block_based_table_factory.cc + ${ROCKSDB_SOURCE_DIR}/table/block_based/block_based_table_iterator.cc + ${ROCKSDB_SOURCE_DIR}/table/block_based/block_based_table_reader.cc + ${ROCKSDB_SOURCE_DIR}/table/block_based/block_builder.cc + ${ROCKSDB_SOURCE_DIR}/table/block_based/block_prefetcher.cc + ${ROCKSDB_SOURCE_DIR}/table/block_based/block_prefix_index.cc + ${ROCKSDB_SOURCE_DIR}/table/block_based/data_block_hash_index.cc + ${ROCKSDB_SOURCE_DIR}/table/block_based/data_block_footer.cc + ${ROCKSDB_SOURCE_DIR}/table/block_based/filter_block_reader_common.cc + ${ROCKSDB_SOURCE_DIR}/table/block_based/filter_policy.cc + ${ROCKSDB_SOURCE_DIR}/table/block_based/flush_block_policy.cc + ${ROCKSDB_SOURCE_DIR}/table/block_based/full_filter_block.cc + ${ROCKSDB_SOURCE_DIR}/table/block_based/hash_index_reader.cc + ${ROCKSDB_SOURCE_DIR}/table/block_based/index_builder.cc + ${ROCKSDB_SOURCE_DIR}/table/block_based/index_reader_common.cc + ${ROCKSDB_SOURCE_DIR}/table/block_based/parsed_full_filter_block.cc + ${ROCKSDB_SOURCE_DIR}/table/block_based/partitioned_filter_block.cc + ${ROCKSDB_SOURCE_DIR}/table/block_based/partitioned_index_iterator.cc + ${ROCKSDB_SOURCE_DIR}/table/block_based/partitioned_index_reader.cc + ${ROCKSDB_SOURCE_DIR}/table/block_based/reader_common.cc + ${ROCKSDB_SOURCE_DIR}/table/block_based/uncompression_dict_reader.cc + ${ROCKSDB_SOURCE_DIR}/table/block_fetcher.cc + ${ROCKSDB_SOURCE_DIR}/table/cuckoo/cuckoo_table_builder.cc + ${ROCKSDB_SOURCE_DIR}/table/cuckoo/cuckoo_table_factory.cc + ${ROCKSDB_SOURCE_DIR}/table/cuckoo/cuckoo_table_reader.cc + ${ROCKSDB_SOURCE_DIR}/table/format.cc + ${ROCKSDB_SOURCE_DIR}/table/get_context.cc + ${ROCKSDB_SOURCE_DIR}/table/iterator.cc + ${ROCKSDB_SOURCE_DIR}/table/merging_iterator.cc + ${ROCKSDB_SOURCE_DIR}/table/meta_blocks.cc + ${ROCKSDB_SOURCE_DIR}/table/persistent_cache_helper.cc + ${ROCKSDB_SOURCE_DIR}/table/plain/plain_table_bloom.cc + ${ROCKSDB_SOURCE_DIR}/table/plain/plain_table_builder.cc + ${ROCKSDB_SOURCE_DIR}/table/plain/plain_table_factory.cc + ${ROCKSDB_SOURCE_DIR}/table/plain/plain_table_index.cc + ${ROCKSDB_SOURCE_DIR}/table/plain/plain_table_key_coding.cc + ${ROCKSDB_SOURCE_DIR}/table/plain/plain_table_reader.cc + ${ROCKSDB_SOURCE_DIR}/table/sst_file_dumper.cc + ${ROCKSDB_SOURCE_DIR}/table/sst_file_reader.cc + ${ROCKSDB_SOURCE_DIR}/table/sst_file_writer.cc + ${ROCKSDB_SOURCE_DIR}/table/table_properties.cc + ${ROCKSDB_SOURCE_DIR}/table/two_level_iterator.cc + ${ROCKSDB_SOURCE_DIR}/test_util/sync_point.cc + ${ROCKSDB_SOURCE_DIR}/test_util/sync_point_impl.cc + ${ROCKSDB_SOURCE_DIR}/test_util/testutil.cc + ${ROCKSDB_SOURCE_DIR}/test_util/transaction_test_util.cc + ${ROCKSDB_SOURCE_DIR}/tools/block_cache_analyzer/block_cache_trace_analyzer.cc + ${ROCKSDB_SOURCE_DIR}/tools/dump/db_dump_tool.cc + ${ROCKSDB_SOURCE_DIR}/tools/ldb_cmd.cc + ${ROCKSDB_SOURCE_DIR}/tools/ldb_tool.cc + ${ROCKSDB_SOURCE_DIR}/tools/sst_dump_tool.cc + ${ROCKSDB_SOURCE_DIR}/tools/trace_analyzer_tool.cc + ${ROCKSDB_SOURCE_DIR}/trace_replay/trace_replay.cc + ${ROCKSDB_SOURCE_DIR}/trace_replay/block_cache_tracer.cc + ${ROCKSDB_SOURCE_DIR}/trace_replay/io_tracer.cc + ${ROCKSDB_SOURCE_DIR}/util/coding.cc + ${ROCKSDB_SOURCE_DIR}/util/compaction_job_stats_impl.cc + ${ROCKSDB_SOURCE_DIR}/util/comparator.cc + ${ROCKSDB_SOURCE_DIR}/util/compression_context_cache.cc + ${ROCKSDB_SOURCE_DIR}/util/concurrent_task_limiter_impl.cc + ${ROCKSDB_SOURCE_DIR}/util/crc32c.cc + ${ROCKSDB_SOURCE_DIR}/util/dynamic_bloom.cc + ${ROCKSDB_SOURCE_DIR}/util/hash.cc + ${ROCKSDB_SOURCE_DIR}/util/murmurhash.cc + ${ROCKSDB_SOURCE_DIR}/util/random.cc + ${ROCKSDB_SOURCE_DIR}/util/rate_limiter.cc + ${ROCKSDB_SOURCE_DIR}/util/slice.cc + ${ROCKSDB_SOURCE_DIR}/util/file_checksum_helper.cc + ${ROCKSDB_SOURCE_DIR}/util/status.cc + ${ROCKSDB_SOURCE_DIR}/util/string_util.cc + ${ROCKSDB_SOURCE_DIR}/util/thread_local.cc + ${ROCKSDB_SOURCE_DIR}/util/threadpool_imp.cc + ${ROCKSDB_SOURCE_DIR}/util/xxhash.cc + ${ROCKSDB_SOURCE_DIR}/utilities/backupable/backupable_db.cc + ${ROCKSDB_SOURCE_DIR}/utilities/blob_db/blob_compaction_filter.cc + ${ROCKSDB_SOURCE_DIR}/utilities/blob_db/blob_db.cc + ${ROCKSDB_SOURCE_DIR}/utilities/blob_db/blob_db_impl.cc + ${ROCKSDB_SOURCE_DIR}/utilities/blob_db/blob_db_impl_filesnapshot.cc + ${ROCKSDB_SOURCE_DIR}/utilities/blob_db/blob_dump_tool.cc + ${ROCKSDB_SOURCE_DIR}/utilities/blob_db/blob_file.cc + ${ROCKSDB_SOURCE_DIR}/utilities/cassandra/cassandra_compaction_filter.cc + ${ROCKSDB_SOURCE_DIR}/utilities/cassandra/format.cc + ${ROCKSDB_SOURCE_DIR}/utilities/cassandra/merge_operator.cc + ${ROCKSDB_SOURCE_DIR}/utilities/checkpoint/checkpoint_impl.cc + ${ROCKSDB_SOURCE_DIR}/utilities/compaction_filters/remove_emptyvalue_compactionfilter.cc + ${ROCKSDB_SOURCE_DIR}/utilities/debug.cc + ${ROCKSDB_SOURCE_DIR}/utilities/env_mirror.cc + ${ROCKSDB_SOURCE_DIR}/utilities/env_timed.cc + ${ROCKSDB_SOURCE_DIR}/utilities/fault_injection_env.cc + ${ROCKSDB_SOURCE_DIR}/utilities/fault_injection_fs.cc + ${ROCKSDB_SOURCE_DIR}/utilities/leveldb_options/leveldb_options.cc + ${ROCKSDB_SOURCE_DIR}/utilities/memory/memory_util.cc + ${ROCKSDB_SOURCE_DIR}/utilities/merge_operators/bytesxor.cc + ${ROCKSDB_SOURCE_DIR}/utilities/merge_operators/max.cc + ${ROCKSDB_SOURCE_DIR}/utilities/merge_operators/put.cc + ${ROCKSDB_SOURCE_DIR}/utilities/merge_operators/sortlist.cc + ${ROCKSDB_SOURCE_DIR}/utilities/merge_operators/string_append/stringappend.cc + ${ROCKSDB_SOURCE_DIR}/utilities/merge_operators/string_append/stringappend2.cc + ${ROCKSDB_SOURCE_DIR}/utilities/merge_operators/uint64add.cc + ${ROCKSDB_SOURCE_DIR}/utilities/object_registry.cc + ${ROCKSDB_SOURCE_DIR}/utilities/option_change_migration/option_change_migration.cc + ${ROCKSDB_SOURCE_DIR}/utilities/options/options_util.cc + ${ROCKSDB_SOURCE_DIR}/utilities/persistent_cache/block_cache_tier.cc + ${ROCKSDB_SOURCE_DIR}/utilities/persistent_cache/block_cache_tier_file.cc + ${ROCKSDB_SOURCE_DIR}/utilities/persistent_cache/block_cache_tier_metadata.cc + ${ROCKSDB_SOURCE_DIR}/utilities/persistent_cache/persistent_cache_tier.cc + ${ROCKSDB_SOURCE_DIR}/utilities/persistent_cache/volatile_tier_impl.cc + ${ROCKSDB_SOURCE_DIR}/utilities/simulator_cache/cache_simulator.cc + ${ROCKSDB_SOURCE_DIR}/utilities/simulator_cache/sim_cache.cc + ${ROCKSDB_SOURCE_DIR}/utilities/table_properties_collectors/compact_on_deletion_collector.cc + ${ROCKSDB_SOURCE_DIR}/utilities/trace/file_trace_reader_writer.cc + ${ROCKSDB_SOURCE_DIR}/utilities/transactions/lock/lock_tracker.cc + ${ROCKSDB_SOURCE_DIR}/utilities/transactions/lock/point_lock_tracker.cc + ${ROCKSDB_SOURCE_DIR}/utilities/transactions/optimistic_transaction_db_impl.cc + ${ROCKSDB_SOURCE_DIR}/utilities/transactions/optimistic_transaction.cc + ${ROCKSDB_SOURCE_DIR}/utilities/transactions/pessimistic_transaction.cc + ${ROCKSDB_SOURCE_DIR}/utilities/transactions/pessimistic_transaction_db.cc + ${ROCKSDB_SOURCE_DIR}/utilities/transactions/snapshot_checker.cc + ${ROCKSDB_SOURCE_DIR}/utilities/transactions/transaction_base.cc + ${ROCKSDB_SOURCE_DIR}/utilities/transactions/transaction_db_mutex_impl.cc + ${ROCKSDB_SOURCE_DIR}/utilities/transactions/transaction_lock_mgr.cc + ${ROCKSDB_SOURCE_DIR}/utilities/transactions/transaction_util.cc + ${ROCKSDB_SOURCE_DIR}/utilities/transactions/write_prepared_txn.cc + ${ROCKSDB_SOURCE_DIR}/utilities/transactions/write_prepared_txn_db.cc + ${ROCKSDB_SOURCE_DIR}/utilities/transactions/write_unprepared_txn.cc + ${ROCKSDB_SOURCE_DIR}/utilities/transactions/write_unprepared_txn_db.cc + ${ROCKSDB_SOURCE_DIR}/utilities/ttl/db_ttl_impl.cc + ${ROCKSDB_SOURCE_DIR}/utilities/write_batch_with_index/write_batch_with_index.cc + ${ROCKSDB_SOURCE_DIR}/utilities/write_batch_with_index/write_batch_with_index_internal.cc + $) + +if(HAVE_SSE42 AND NOT MSVC) + set_source_files_properties( + ${ROCKSDB_SOURCE_DIR}/util/crc32c.cc + PROPERTIES COMPILE_FLAGS "-msse4.2 -mpclmul") +endif() + +if(CMAKE_SYSTEM_PROCESSOR MATCHES "^(powerpc|ppc)64") + list(APPEND SOURCES + ${ROCKSDB_SOURCE_DIR}/util/crc32c_ppc.c + ${ROCKSDB_SOURCE_DIR}/util/crc32c_ppc_asm.S) +endif(CMAKE_SYSTEM_PROCESSOR MATCHES "^(powerpc|ppc)64") + +if(HAS_ARMV8_CRC) + list(APPEND SOURCES + ${ROCKSDB_SOURCE_DIR}/util/crc32c_arm64.cc) +endif(HAS_ARMV8_CRC) + +if(WIN32) + list(APPEND SOURCES + ${ROCKSDB_SOURCE_DIR}/port/win/io_win.cc + ${ROCKSDB_SOURCE_DIR}/port/win/env_win.cc + ${ROCKSDB_SOURCE_DIR}/port/win/env_default.cc + ${ROCKSDB_SOURCE_DIR}/port/win/port_win.cc + ${ROCKSDB_SOURCE_DIR}/port/win/win_logger.cc) + if(NOT MINGW) + # Mingw only supports std::thread when using + # posix threads. + list(APPEND SOURCES + ${ROCKSDB_SOURCE_DIR}/port/win/win_thread.cc) + endif() +if(WITH_XPRESS) + list(APPEND SOURCES + ${ROCKSDB_SOURCE_DIR}/port/win/xpress_win.cc) +endif() + +if(WITH_JEMALLOC) + list(APPEND SOURCES + ${ROCKSDB_SOURCE_DIR}/port/win/win_jemalloc.cc) +endif() + +else() + list(APPEND SOURCES + ${ROCKSDB_SOURCE_DIR}/port/port_posix.cc + ${ROCKSDB_SOURCE_DIR}/env/env_posix.cc + ${ROCKSDB_SOURCE_DIR}/env/fs_posix.cc + ${ROCKSDB_SOURCE_DIR}/env/io_posix.cc) +endif() + +if(WITH_FOLLY_DISTRIBUTED_MUTEX) + list(APPEND SOURCES + ${ROCKSDB_SOURCE_DIR}/third-party/folly/folly/detail/Futex.cpp + ${ROCKSDB_SOURCE_DIR}/third-party/folly/folly/synchronization/AtomicNotification.cpp + ${ROCKSDB_SOURCE_DIR}/third-party/folly/folly/synchronization/DistributedMutex.cpp + ${ROCKSDB_SOURCE_DIR}/third-party/folly/folly/synchronization/ParkingLot.cpp + ${ROCKSDB_SOURCE_DIR}/third-party/folly/folly/synchronization/WaitOptions.cpp) +endif() + +set(ROCKSDB_STATIC_LIB rocksdb) + +if(WIN32) + set(SYSTEM_LIBS ${SYSTEM_LIBS} shlwapi.lib rpcrt4.lib) +else() + set(SYSTEM_LIBS ${CMAKE_THREAD_LIBS_INIT}) +endif() + +add_library(${ROCKSDB_STATIC_LIB} STATIC ${SOURCES}) +target_link_libraries(${ROCKSDB_STATIC_LIB} PRIVATE + ${THIRDPARTY_LIBS} ${SYSTEM_LIBS}) diff --git a/src/Storages/RocksDB/EmbeddedRocksDBBlockInputStream.cpp b/src/Storages/RocksDB/EmbeddedRocksDBBlockInputStream.cpp index b7602f8bedb..a8e6b51654c 100644 --- a/src/Storages/RocksDB/EmbeddedRocksDBBlockInputStream.cpp +++ b/src/Storages/RocksDB/EmbeddedRocksDBBlockInputStream.cpp @@ -1,6 +1,5 @@ #include #include -#include #include #include @@ -14,13 +13,11 @@ namespace ErrorCodes extern const int ROCKSDB_ERROR; } +class StorageEmbeddedRocksDB; + EmbeddedRocksDBBlockInputStream::EmbeddedRocksDBBlockInputStream( - StorageEmbeddedRocksDB & storage_, - const StorageMetadataPtr & metadata_snapshot_, - size_t max_block_size_) - : storage(storage_) - , metadata_snapshot(metadata_snapshot_) - , max_block_size(max_block_size_) + StorageEmbeddedRocksDB & storage_, const StorageMetadataPtr & metadata_snapshot_, size_t max_block_size_) + : storage(storage_), metadata_snapshot(metadata_snapshot_), max_block_size(max_block_size_) { sample_block = metadata_snapshot->getSampleBlock(); primary_key_pos = sample_block.getPositionByName(storage.primary_key); @@ -39,8 +36,7 @@ Block EmbeddedRocksDBBlockInputStream::readImpl() MutableColumns columns = sample_block.cloneEmptyColumns(); - size_t rows = 0; - for (; iterator->Valid(); iterator->Next()) + for (size_t rows = 0; iterator->Valid() && rows < max_block_size; ++rows, iterator->Next()) { ReadBufferFromString key_buffer(iterator->key()); ReadBufferFromString value_buffer(iterator->value()); @@ -51,9 +47,6 @@ Block EmbeddedRocksDBBlockInputStream::readImpl() elem.type->deserializeBinary(*columns[idx], idx == primary_key_pos ? key_buffer : value_buffer); ++idx; } - ++rows; - if (rows >= max_block_size) - break; } finished = !iterator->Valid(); diff --git a/src/Storages/RocksDB/EmbeddedRocksDBBlockInputStream.h b/src/Storages/RocksDB/EmbeddedRocksDBBlockInputStream.h index 3de04d70f42..9491355fef2 100644 --- a/src/Storages/RocksDB/EmbeddedRocksDBBlockInputStream.h +++ b/src/Storages/RocksDB/EmbeddedRocksDBBlockInputStream.h @@ -1,6 +1,7 @@ #pragma once #include +#include namespace rocksdb @@ -11,8 +12,6 @@ namespace rocksdb namespace DB { -class StorageEmbeddedRocksDB; - class EmbeddedRocksDBBlockInputStream : public IBlockInputStream { diff --git a/src/Storages/RocksDB/EmbeddedRocksDBBlockOutputStream.cpp b/src/Storages/RocksDB/EmbeddedRocksDBBlockOutputStream.cpp index a3a0ddb4348..180becde80e 100644 --- a/src/Storages/RocksDB/EmbeddedRocksDBBlockOutputStream.cpp +++ b/src/Storages/RocksDB/EmbeddedRocksDBBlockOutputStream.cpp @@ -1,5 +1,4 @@ #include -#include #include #include @@ -7,12 +6,13 @@ namespace DB { - namespace ErrorCodes { extern const int ROCKSDB_ERROR; } +class StorageEmbeddedRocksDB; + EmbeddedRocksDBBlockOutputStream::EmbeddedRocksDBBlockOutputStream( StorageEmbeddedRocksDB & storage_, const StorageMetadataPtr & metadata_snapshot_) @@ -42,6 +42,7 @@ void EmbeddedRocksDBBlockOutputStream::write(const Block & block) WriteBufferFromOwnString wb_value; rocksdb::WriteBatch batch; + rocksdb::Status status; for (size_t i = 0; i < rows; i++) { wb_key.restart(); @@ -53,10 +54,12 @@ void EmbeddedRocksDBBlockOutputStream::write(const Block & block) elem.type->serializeBinary(*elem.column, i, idx == primary_key_pos ? wb_key : wb_value); ++idx; } - batch.Put(wb_key.str(), wb_value.str()); + status = batch.Put(wb_key.str(), wb_value.str()); + if (!status.ok()) + throw Exception("RocksDB write error: " + status.ToString(), ErrorCodes::ROCKSDB_ERROR); } - auto status = storage.rocksdb_ptr->Write(rocksdb::WriteOptions(), &batch); + status = storage.rocksdb_ptr->Write(rocksdb::WriteOptions(), &batch); if (!status.ok()) throw Exception("RocksDB write error: " + status.ToString(), ErrorCodes::ROCKSDB_ERROR); } diff --git a/src/Storages/RocksDB/EmbeddedRocksDBBlockOutputStream.h b/src/Storages/RocksDB/EmbeddedRocksDBBlockOutputStream.h index 51768df11ac..717d08579bb 100644 --- a/src/Storages/RocksDB/EmbeddedRocksDBBlockOutputStream.h +++ b/src/Storages/RocksDB/EmbeddedRocksDBBlockOutputStream.h @@ -2,13 +2,12 @@ #include #include +#include namespace DB { -class StorageEmbeddedRocksDB; - class EmbeddedRocksDBBlockOutputStream : public IBlockOutputStream { public: diff --git a/src/Storages/RocksDB/StorageEmbeddedRocksDB.cpp b/src/Storages/RocksDB/StorageEmbeddedRocksDB.cpp index 84de695d63b..ae28205d07b 100644 --- a/src/Storages/RocksDB/StorageEmbeddedRocksDB.cpp +++ b/src/Storages/RocksDB/StorageEmbeddedRocksDB.cpp @@ -268,6 +268,7 @@ void StorageEmbeddedRocksDB::initDb() rocksdb::Options options; rocksdb::DB * db; options.create_if_missing = true; + options.compression = rocksdb::CompressionType::kZSTD; rocksdb::Status status = rocksdb::DB::Open(options, rocksdb_dir, &db); if (status != rocksdb::Status::OK()) diff --git a/tests/queries/0_stateless/01504_rocksdb.reference b/tests/queries/0_stateless/01504_rocksdb.reference index f98edab5dbf..ba9665685fe 100644 --- a/tests/queries/0_stateless/01504_rocksdb.reference +++ b/tests/queries/0_stateless/01504_rocksdb.reference @@ -1,6 +1,8 @@ 1 1 -0 0 0 0 0 +1 +1 +1 1 1 1 1 1 1 1 diff --git a/tests/queries/0_stateless/01504_rocksdb.sql b/tests/queries/0_stateless/01504_rocksdb.sql index 123c6c2d9c6..0e1917a95f2 100644 --- a/tests/queries/0_stateless/01504_rocksdb.sql +++ b/tests/queries/0_stateless/01504_rocksdb.sql @@ -1,17 +1,26 @@ DROP TABLE IF EXISTS test; -CREATE TABLE test (key String, value UInt32) Engine=EmbeddedRocksDB primary key(key); + +CREATE TABLE test (key String, value UInt32) Engine=EmbeddedRocksDB; -- { serverError 36 } +CREATE TABLE test (key String, value UInt32) Engine=EmbeddedRocksDB PRIMARY KEY(key2); -- { serverError 47 } +CREATE TABLE test (key String, value UInt32) Engine=EmbeddedRocksDB PRIMARY KEY(key, value); -- { serverError 36 } +CREATE TABLE test (key Tuple(String, UInt32), value UInt64) Engine=EmbeddedRocksDB PRIMARY KEY(key); + +DROP TABLE IF EXISTS test; +CREATE TABLE test (key String, value UInt32) Engine=EmbeddedRocksDB PRIMARY KEY(key); INSERT INTO test SELECT '1_1', number FROM numbers(10000); -SELECT count(1) == 1 FROM test; +SELECT COUNT(1) == 1 FROM test; INSERT INTO test SELECT concat(toString(number), '_1'), number FROM numbers(10000); -SELECT SUM(value) == 1 + 99 + 900 FROM test WHERE key in ('1_1', '99_1', '900_1'); +SELECT COUNT(1) == 10000 FROM test; +SELECT uniqExact(key) == 32 FROM (SELECT * FROM test LIMIT 32 SETTINGS max_block_size = 1); +SELECT SUM(value) == 1 + 99 + 900 FROM test WHERE key IN ('1_1', '99_1', '900_1'); DROP TABLE IF EXISTS test; DROP TABLE IF EXISTS test_memory; -CREATE TABLE test (k UInt32, value UInt64, dummy Tuple(UInt32, Float64), bm AggregateFunction(groupBitmap, UInt64)) Engine=EmbeddedRocksDB primary key(k); +CREATE TABLE test (k UInt32, value UInt64, dummy Tuple(UInt32, Float64), bm AggregateFunction(groupBitmap, UInt64)) Engine=EmbeddedRocksDB PRIMARY KEY(k); CREATE TABLE test_memory AS test Engine = Memory; INSERT INTO test SELECT number % 77 AS k, SUM(number) AS value, (1, 1.2), bitmapBuild(groupArray(number)) FROM numbers(10000000) group by k; @@ -19,7 +28,7 @@ INSERT INTO test SELECT number % 77 AS k, SUM(number) AS value, (1, 1.2), bitmap INSERT INTO test_memory SELECT number % 77 AS k, SUM(number) AS value, (1, 1.2), bitmapBuild(groupArray(number)) FROM numbers(10000000) group by k; -SELECT A.a - B.a, A.b - B.b, A.c - B.c, A.d - B.d, A.e - B.e FROM ( SELECT 0 AS a, groupBitmapMerge(bm) AS b , SUM(k) AS c, SUM(value) AS d, SUM(dummy.1) AS e FROM test) A ANY LEFT JOIN (SELECT 0 AS a, groupBitmapMerge(bm) AS b , SUM(k) AS c, SUM(value) AS d, SUM(dummy.1) AS e FROM test_memory) B USING a ORDER BY a; +SELECT A.a = B.a, A.b = B.b, A.c = B.c, A.d = B.d, A.e = B.e FROM ( SELECT 0 AS a, groupBitmapMerge(bm) AS b , SUM(k) AS c, SUM(value) AS d, SUM(dummy.1) AS e FROM test) A ANY LEFT JOIN (SELECT 0 AS a, groupBitmapMerge(bm) AS b , SUM(k) AS c, SUM(value) AS d, SUM(dummy.1) AS e FROM test_memory) B USING a ORDER BY a; CREATE TEMPORARY TABLE keys AS SELECT * FROM numbers(1000); @@ -31,7 +40,7 @@ SELECT k, value FROM test WHERE k = 0 OR value > 0; -- { serverError 158 } SELECT k, value FROM test WHERE k = 0 AND k IN (1, 3) OR k > 8; -- { serverError 158 } TRUNCATE TABLE test; -SELECT 0 == count(1) FROM test; +SELECT 0 == COUNT(1) FROM test; DROP TABLE IF EXISTS test; DROP TABLE IF EXISTS test_memory; From c07f55bd56ae7961a78361ea41ec08a73c0f182d Mon Sep 17 00:00:00 2001 From: OuO Date: Wed, 11 Nov 2020 11:55:08 +0800 Subject: [PATCH 125/205] update backup.md to make it more human readable --- docs/zh/operations/backup.md | 18 ++++++++++-------- 1 file changed, 10 insertions(+), 8 deletions(-) diff --git a/docs/zh/operations/backup.md b/docs/zh/operations/backup.md index 0ce138c764e..72491bb53ff 100644 --- a/docs/zh/operations/backup.md +++ b/docs/zh/operations/backup.md @@ -7,35 +7,37 @@ toc_title: "\u6570\u636E\u5907\u4EFD" # 数据备份 {#data-backup} -碌莽禄While: [复制](../engines/table-engines/mergetree-family/replication.md) provides protection from hardware failures, it does not protect against human errors: accidental deletion of data, deletion of the wrong table or a table on the wrong cluster, and software bugs that result in incorrect data processing or data corruption. In many cases mistakes like these will affect all replicas. ClickHouse has built-in safeguards to prevent some types of mistakes — for example, by default [您不能使用类似MergeTree的引擎删除包含超过50Gb数据的表](https://github.com/ClickHouse/ClickHouse/blob/v18.14.18-stable/programs/server/config.xml#L322-L330). 但是,这些保障措施并不涵盖所有可能的情况,可以规避。 +尽管[副本](../engines/table-engines/mergetree-family/replication.md) 可以预防硬件错误带来的数据丢失, 但是它不能防止人为操作的错误: 意外删除数据, 删除错误的 table 或者删除错误 cluster 上的 table, 可以导致错误数据处理错误或者数据损坏的 bugs. 这类意外可能会影响所有的副本. ClickHouse 有内建的保障措施可以预防一些错误 — 例如, 默认情况下[您不能使用类似MergeTree的引擎删除包含超过50Gb数据的表](https://github.com/ClickHouse/ClickHouse/blob/v18.14.18-stable/programs/server/config.xml#L322-L330). 但是,这些保障措施不能涵盖所有可能的情况,并且可以规避。 -为了有效地减少可能的人为错误,您应该仔细准备备份和还原数据的策略 **提前**. +为了有效地减少可能的人为错误,您应该 **提前**准备备份和还原数据的策略. -每家公司都有不同的可用资源和业务需求,因此没有适合各种情况的ClickHouse备份和恢复通用解决方案。 什么适用于一千兆字节的数据可能不会为几十pb的工作。 有多种可能的方法有自己的优点和缺点,这将在下面讨论。 这是一个好主意,使用几种方法,而不是只是一个,以弥补其各种缺点。 +不同公司有不同的可用资源和业务需求,因此没有适合各种情况的ClickHouse备份和恢复通用解决方案。 适用于 1GB 的数据的方案可能并不适用于几十 PB 数据的情况。 有多种可能的并有自己优缺点的方法,这将在下面讨论。 好的主意是同时结合使用多种方法而不是仅使用一种,这样可以弥补不同方法各自的缺点。 !!! note "注" 请记住,如果您备份了某些内容并且从未尝试过还原它,那么当您实际需要它时(或者至少需要比业务能够容忍的时间更长),恢复可能无法正常工作。 因此,无论您选择哪种备份方法,请确保自动还原过程,并定期在备用ClickHouse群集上练习。 ## 将源数据复制到其他地方 {#duplicating-source-data-somewhere-else} -通常被摄入到ClickHouse的数据是通过某种持久队列传递的,例如 [Apache Kafka](https://kafka.apache.org). 在这种情况下,可以配置一组额外的订阅服务器,这些订阅服务器将在写入ClickHouse时读取相同的数据流,并将其存储在冷存储中。 大多数公司已经有一些默认的推荐冷存储,可能是对象存储或分布式文件系统,如 [HDFS](https://hadoop.apache.org/docs/stable/hadoop-project-dist/hadoop-hdfs/HdfsDesign.html). +通常被聚集到ClickHouse的数据是通过某种持久队列传递的,例如 [Apache Kafka](https://kafka.apache.org). 在这种情况下,可以配置一组额外的订阅服务器,这些订阅服务器将在写入ClickHouse时读取相同的数据流,并将其存储在冷存储中。 大多数公司已经有一些默认的推荐冷存储,可能是对象存储或分布式文件系统,如 [HDFS](https://hadoop.apache.org/docs/stable/hadoop-project-dist/hadoop-hdfs/HdfsDesign.html). ## 文件系统快照 {#filesystem-snapshots} 某些本地文件系统提供快照功能(例如, [ZFS](https://en.wikipedia.org/wiki/ZFS)),但它们可能不是提供实时查询的最佳选择。 一个可能的解决方案是使用这种文件系统创建额外的副本,并将它们从 [分布](../engines/table-engines/special/distributed.md) 用于以下目的的表 `SELECT` 查询。 任何修改数据的查询都无法访问此类副本上的快照。 作为奖励,这些副本可能具有特殊的硬件配置,每个服务器附加更多的磁盘,这将是经济高效的。 -## ツ环板-ョツ嘉ッツ偲 {#clickhouse-copier} +## clickhouse-copier {#clickhouse-copier} -[ツ环板-ョツ嘉ッツ偲](utilities/clickhouse-copier.md) 是一个多功能工具,最初创建用于重新分片pb大小的表。 它还可用于备份和还原目的,因为它可以在ClickHouse表和集群之间可靠地复制数据。 +[clickhouse-copier](utilities/clickhouse-copier.md) 是一个多功能工具,最初创建用于重新分片pb大小的表。 因为它可以在ClickHouse表和集群之间可靠地复制数据,所以它还可用于备份和还原数据。 对于较小的数据量,一个简单的 `INSERT INTO ... SELECT ...` 到远程表也可以工作。 ## 部件操作 {#manipulations-with-parts} -ClickHouse允许使用 `ALTER TABLE ... FREEZE PARTITION ...` 查询以创建表分区的本地副本。 这是使用硬链接来实现 `/var/lib/clickhouse/shadow/` 文件夹中,所以它通常不会占用旧数据的额外磁盘空间。 创建的文件副本不由ClickHouse服务器处理,所以你可以把它们留在那里:你将有一个简单的备份,不需要任何额外的外部系统,但它仍然会容易出现硬件问题。 出于这个原因,最好将它们远程复制到另一个位置,然后删除本地副本。 分布式文件系统和对象存储仍然是一个不错的选择,但是具有足够大容量的正常附加文件服务器也可以工作(在这种情况下,传输将通过网络文件系统 [rsync](https://en.wikipedia.org/wiki/Rsync)). +ClickHouse允许使用 `ALTER TABLE ... FREEZE PARTITION ...` 查询以创建表分区的本地副本。 这是利用硬链接(hardlink)到 `/var/lib/clickhouse/shadow/` 文件夹中实现的,所以它通常不会占用旧数据的额外磁盘空间。 创建的文件副本不由ClickHouse服务器处理,所以你可以把它们留在那里:你将有一个简单的备份,不需要任何额外的外部系统,但它仍然会容易出现硬件问题。 出于这个原因,最好将它们远程复制到另一个位置,然后删除本地副本。 分布式文件系统和对象存储仍然是一个不错的选择,但是具有足够大容量的正常附加文件服务器也可以工作(在这种情况下,传输将通过网络文件系统 [rsync](https://en.wikipedia.org/wiki/Rsync)). + +数据可以使用 `ALTER TABLE ... ATTACH PARTITION ...` 从备份中恢复。 有关与分区操作相关的查询的详细信息,请参阅 [更改文档](../sql-reference/statements/alter.md#alter_manipulations-with-partitions). -第三方工具可用于自动化此方法: [ツ环板backupョツ嘉ッツ偲](https://github.com/AlexAkulov/clickhouse-backup). +第三方工具可用于自动化此方法: [clickhouse-backup](https://github.com/AlexAkulov/clickhouse-backup). [原始文章](https://clickhouse.tech/docs/en/operations/backup/) From e31753b4db7aa0a72a85757dc11fc403962e30db Mon Sep 17 00:00:00 2001 From: su-houzhen Date: Wed, 11 Nov 2020 17:12:22 +0800 Subject: [PATCH 126/205] Fix build err --- utils/db-generator/query_db_generator.cpp | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/utils/db-generator/query_db_generator.cpp b/utils/db-generator/query_db_generator.cpp index c8aae4a56f3..08389ce4f6b 100644 --- a/utils/db-generator/query_db_generator.cpp +++ b/utils/db-generator/query_db_generator.cpp @@ -1292,9 +1292,11 @@ int main(int argc, const char *argv[]) return 1; } if (vm.count("input")) - freopen(vm["input"].as().c_str(), "r", stdin); + if (!freopen(vm["input"].as().c_str(), "r", stdin)) + std::cout << "Error while input." << std::endl; if (vm.count("output")) - freopen(vm["output"].as().c_str(), "w", stdout); + if (!freopen(vm["output"].as().c_str(), "w", stdout)) + std::cout << "Error while output." << std::endl; if (vm.empty()) std::cout << "Copy your queries (with semicolons) here, press Enter and Ctrl+D." << std::endl; } From 579f4c50ed52118e3f592cd3e626269f60df308a Mon Sep 17 00:00:00 2001 From: sundy-li <543950155@qq.com> Date: Wed, 11 Nov 2020 17:39:23 +0800 Subject: [PATCH 127/205] improve rocksdb-cmake build --- contrib/rocksdb-cmake/CMakeLists.txt | 85 ++----------------- .../EmbeddedRocksDBBlockOutputStream.h | 1 - 2 files changed, 7 insertions(+), 79 deletions(-) diff --git a/contrib/rocksdb-cmake/CMakeLists.txt b/contrib/rocksdb-cmake/CMakeLists.txt index 2a6d95e5bc5..205b5187da3 100644 --- a/contrib/rocksdb-cmake/CMakeLists.txt +++ b/contrib/rocksdb-cmake/CMakeLists.txt @@ -16,9 +16,10 @@ elseif (SANITIZE STREQUAL "thread") set(WITH_TSAN ON) endif() + set(PORTABLE ON) option(WITH_JEMALLOC "build with JeMalloc" ${ENABLE_JEMALLOC}) -option(WITH_SNAPPY "build with SNAPPY" ON) +option(WITH_SNAPPY "build with SNAPPY" ${USE_SNAPPY}) option(WITH_LZ4 "build with lz4" ON) option(WITH_ZLIB "build with zlib" ON) option(WITH_ZSTD "build with zstd" ON) @@ -50,50 +51,30 @@ else() set(WITH_JEMALLOC ON) else() if(WITH_JEMALLOC) - find_package(JeMalloc REQUIRED) add_definitions(-DROCKSDB_JEMALLOC -DJEMALLOC_NO_DEMANGLE) - list(APPEND THIRDPARTY_LIBS JeMalloc::JeMalloc) + list(APPEND THIRDPARTY_LIBS jemalloc) endif() endif() if(WITH_SNAPPY) - find_package(Snappy CONFIG) - if(NOT Snappy_FOUND) - find_package(Snappy REQUIRED) - endif() add_definitions(-DSNAPPY) - list(APPEND THIRDPARTY_LIBS Snappy::snappy) + list(APPEND THIRDPARTY_LIBS snappy) endif() if(WITH_ZLIB) - find_package(ZLIB REQUIRED) add_definitions(-DZLIB) - list(APPEND THIRDPARTY_LIBS ZLIB::ZLIB) - endif() - - option(WITH_BZ2 "build with bzip2" OFF) - if(WITH_BZ2) - find_package(BZip2 REQUIRED) - add_definitions(-DBZIP2) - if(BZIP2_INCLUDE_DIRS) - include_directories(${BZIP2_INCLUDE_DIRS}) - else() - include_directories(${BZIP2_INCLUDE_DIR}) - endif() - list(APPEND THIRDPARTY_LIBS ${BZIP2_LIBRARIES}) + list(APPEND THIRDPARTY_LIBS zlib) endif() if(WITH_LZ4) - find_package(lz4 REQUIRED) add_definitions(-DLZ4) - list(APPEND THIRDPARTY_LIBS lz4::lz4) + list(APPEND THIRDPARTY_LIBS lz4) endif() if(WITH_ZSTD) - find_package(zstd REQUIRED) add_definitions(-DZSTD) include_directories(${ZSTD_INCLUDE_DIR}) - list(APPEND THIRDPARTY_LIBS zstd::zstd) + list(APPEND THIRDPARTY_LIBS zstd) endif() endif() @@ -247,58 +228,6 @@ if(WITH_UBSAN) endif() endif() -option(WITH_NUMA "build with NUMA policy support" OFF) -if(WITH_NUMA) - find_package(NUMA REQUIRED) - add_definitions(-DNUMA) - include_directories(${NUMA_INCLUDE_DIR}) - list(APPEND THIRDPARTY_LIBS NUMA::NUMA) -endif() - -option(WITH_TBB "build with Threading Building Blocks (TBB)" OFF) -if(WITH_TBB) - find_package(TBB REQUIRED) - add_definitions(-DTBB) - list(APPEND THIRDPARTY_LIBS TBB::TBB) -endif() - -# Stall notifications eat some performance from inserts -option(DISABLE_STALL_NOTIF "Build with stall notifications" OFF) -if(DISABLE_STALL_NOTIF) - add_definitions(-DROCKSDB_DISABLE_STALL_NOTIFICATION) -endif() - -option(WITH_DYNAMIC_EXTENSION "build with dynamic extension support" OFF) -if(NOT WITH_DYNAMIC_EXTENSION) - add_definitions(-DROCKSDB_NO_DYNAMIC_EXTENSION) -endif() - -if(DEFINED USE_RTTI) - if(USE_RTTI) - message(STATUS "Enabling RTTI") - set(CMAKE_CXX_FLAGS_DEBUG "${CMAKE_CXX_FLAGS_DEBUG} -DROCKSDB_USE_RTTI") - set(CMAKE_CXX_FLAGS_RELEASE "${CMAKE_CXX_FLAGS_RELEASE} -DROCKSDB_USE_RTTI") - else() - if(MSVC) - message(STATUS "Disabling RTTI in Release builds. Always on in Debug.") - set(CMAKE_CXX_FLAGS_DEBUG "${CMAKE_CXX_FLAGS_DEBUG} -DROCKSDB_USE_RTTI") - set(CMAKE_CXX_FLAGS_RELEASE "${CMAKE_CXX_FLAGS_RELEASE} /GR-") - else() - message(STATUS "Disabling RTTI in Release builds") - set(CMAKE_CXX_FLAGS_DEBUG "${CMAKE_CXX_FLAGS_DEBUG} -fno-rtti") - set(CMAKE_CXX_FLAGS_RELEASE "${CMAKE_CXX_FLAGS_RELEASE} -fno-rtti") - endif() - endif() -else() - message(STATUS "Enabling RTTI in Debug builds only (default)") - set(CMAKE_CXX_FLAGS_DEBUG "${CMAKE_CXX_FLAGS_DEBUG} -DROCKSDB_USE_RTTI") - if(MSVC) - set(CMAKE_CXX_FLAGS_RELEASE "${CMAKE_CXX_FLAGS_RELEASE} /GR-") - else() - set(CMAKE_CXX_FLAGS_RELEASE "${CMAKE_CXX_FLAGS_RELEASE} -fno-rtti") - endif() -endif() - if(CMAKE_SYSTEM_NAME MATCHES "Cygwin") add_definitions(-fno-builtin-memcmp -DCYGWIN) diff --git a/src/Storages/RocksDB/EmbeddedRocksDBBlockOutputStream.h b/src/Storages/RocksDB/EmbeddedRocksDBBlockOutputStream.h index f6710f98d3d..e6229782505 100644 --- a/src/Storages/RocksDB/EmbeddedRocksDBBlockOutputStream.h +++ b/src/Storages/RocksDB/EmbeddedRocksDBBlockOutputStream.h @@ -1,7 +1,6 @@ #pragma once #include -#include namespace DB From e400d97b12f510be7b07ee9de97a62c5f3988290 Mon Sep 17 00:00:00 2001 From: Mikhail Filimonov Date: Wed, 11 Nov 2020 11:17:21 +0100 Subject: [PATCH 128/205] Test for issue #16862 --- ...1560_timeseriesgroupsum_segfault.reference | 3 +++ .../01560_timeseriesgroupsum_segfault.sql | 23 +++++++++++++++++++ 2 files changed, 26 insertions(+) create mode 100644 tests/queries/0_stateless/01560_timeseriesgroupsum_segfault.reference create mode 100644 tests/queries/0_stateless/01560_timeseriesgroupsum_segfault.sql diff --git a/tests/queries/0_stateless/01560_timeseriesgroupsum_segfault.reference b/tests/queries/0_stateless/01560_timeseriesgroupsum_segfault.reference new file mode 100644 index 00000000000..814095e7818 --- /dev/null +++ b/tests/queries/0_stateless/01560_timeseriesgroupsum_segfault.reference @@ -0,0 +1,3 @@ +[] +1 +server is still alive diff --git a/tests/queries/0_stateless/01560_timeseriesgroupsum_segfault.sql b/tests/queries/0_stateless/01560_timeseriesgroupsum_segfault.sql new file mode 100644 index 00000000000..5aeae422f1b --- /dev/null +++ b/tests/queries/0_stateless/01560_timeseriesgroupsum_segfault.sql @@ -0,0 +1,23 @@ +DROP TABLE IF EXISTS tsgs_local; +DROP TABLE IF EXISTS tsgs; + +CREATE TABLE tsgs_local ENGINE = MergeTree ORDER BY tuple() AS +SELECT + toUInt64(13820745146630357293) AS a, + toInt64(1604422500000000000) AS b, + toFloat64(0) AS c +FROM numbers(100); + +-- the issue (https://github.com/ClickHouse/ClickHouse/issues/16862) happens during serialization of the state +-- so happens only when Distributed tables are used or with -State modifier. + +CREATE TABLE tsgs AS tsgs_local ENGINE = Distributed(test_cluster_two_shards, currentDatabase(), tsgs_local); + +SELECT timeSeriesGroupSum(a, b, c) FROM tsgs; + +SELECT count() FROM ( SELECT timeSeriesGroupSumState(a, b, c) FROM tsgs_local) WHERE NOT ignore(*); + +SELECT 'server is still alive'; + +DROP TABLE tsgs_local; +DROP TABLE tsgs; From 4b97c95d6e16bd5f420fcfabf06d79e66c03481e Mon Sep 17 00:00:00 2001 From: Pavel Kruglov Date: Wed, 11 Nov 2020 13:55:20 +0300 Subject: [PATCH 129/205] Better implementation --- src/DataTypes/DataTypeEnum.h | 16 +++++++--------- src/IO/ReadHelpers.cpp | 8 -------- src/IO/ReadHelpers.h | 2 -- 3 files changed, 7 insertions(+), 19 deletions(-) diff --git a/src/DataTypes/DataTypeEnum.h b/src/DataTypes/DataTypeEnum.h index c6850e022d3..9c38cbddd83 100644 --- a/src/DataTypes/DataTypeEnum.h +++ b/src/DataTypes/DataTypeEnum.h @@ -4,7 +4,6 @@ #include #include #include -#include #include #include @@ -81,25 +80,24 @@ public: return findByValue(value)->second; } - FieldType getValue(StringRef field_name, bool try_treat_as_id=false) const + FieldType getValue(StringRef field_name, bool try_treat_as_id = false) const { - const auto value_it = name_to_value_map.find(field_name); - if (!value_it) + const auto it = name_to_value_map.find(field_name); + if (!it) { /// It is used in CSV and TSV input formats. If we fail to find given string in - /// enum names and this string is number, we will try to treat it as enum id. - if (try_treat_as_id && isStringNumber(field_name)) + /// enum names, we will try to treat it as enum id. + if (try_treat_as_id) { FieldType x; ReadBufferFromMemory tmp_buf(field_name.data, field_name.size); readText(x, tmp_buf); - const auto name_it = value_to_name_map.find(x); - if (name_it != value_to_name_map.end()) + if (tmp_buf.eof() && value_to_name_map.find(x) != value_to_name_map.end()) return x; } throw Exception{"Unknown element '" + field_name.toString() + "' for type " + getName(), ErrorCodes::BAD_ARGUMENTS}; } - return value_it->getMapped(); + return it->getMapped(); } FieldType readValue(ReadBuffer & istr) const diff --git a/src/IO/ReadHelpers.cpp b/src/IO/ReadHelpers.cpp index 73b4738b078..bf41de3959a 100644 --- a/src/IO/ReadHelpers.cpp +++ b/src/IO/ReadHelpers.cpp @@ -1111,12 +1111,4 @@ bool loadAtPosition(ReadBuffer & in, DB::Memory<> & memory, char * & current) return loaded_more; } -bool isStringNumber(StringRef str) -{ - for (size_t i = 0; i != str.size; ++i) - if (!isdigit(str.data[i])) - return false; - return true; -} - } diff --git a/src/IO/ReadHelpers.h b/src/IO/ReadHelpers.h index 5cb4ec680c7..9ff1858c723 100644 --- a/src/IO/ReadHelpers.h +++ b/src/IO/ReadHelpers.h @@ -1228,6 +1228,4 @@ void saveUpToPosition(ReadBuffer & in, Memory<> & memory, char * current); */ bool loadAtPosition(ReadBuffer & in, Memory<> & memory, char * & current); -bool isStringNumber(StringRef str); - } From c4e0d403883bff3bb87cb5840536444c49133080 Mon Sep 17 00:00:00 2001 From: Pavel Kruglov Date: Wed, 11 Nov 2020 14:01:46 +0300 Subject: [PATCH 130/205] Add a comment --- src/DataTypes/DataTypeEnum.h | 2 ++ 1 file changed, 2 insertions(+) diff --git a/src/DataTypes/DataTypeEnum.h b/src/DataTypes/DataTypeEnum.h index 9c38cbddd83..c75d348f15c 100644 --- a/src/DataTypes/DataTypeEnum.h +++ b/src/DataTypes/DataTypeEnum.h @@ -92,6 +92,8 @@ public: FieldType x; ReadBufferFromMemory tmp_buf(field_name.data, field_name.size); readText(x, tmp_buf); + /// Check if we reached end of the tmp_buf (otherwise field_name is not a number) + /// and try to find it in enum ids if (tmp_buf.eof() && value_to_name_map.find(x) != value_to_name_map.end()) return x; } From bd15b4e77f2ee4d918be9f7cbce4909b4a77d792 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Wed, 11 Nov 2020 14:15:25 +0300 Subject: [PATCH 131/205] Fix tests. --- src/Interpreters/ExpressionActions.cpp | 9 ++++----- src/Interpreters/ExpressionActions.h | 14 ++++---------- 2 files changed, 8 insertions(+), 15 deletions(-) diff --git a/src/Interpreters/ExpressionActions.cpp b/src/Interpreters/ExpressionActions.cpp index 73e4bf9fde5..a51e8fe485c 100644 --- a/src/Interpreters/ExpressionActions.cpp +++ b/src/Interpreters/ExpressionActions.cpp @@ -69,9 +69,6 @@ ExpressionActions::ExpressionActions(ActionsDAGPtr actions_dag_) throw Exception(ErrorCodes::TOO_MANY_TEMPORARY_COLUMNS, "Too many temporary columns: {}. Maximum: {}", actions_dag->dumpNames(), std::to_string(settings.max_temporary_columns)); - - max_temporary_non_const_columns = settings.max_temporary_non_const_columns; - project_input = settings.project_input; } ExpressionActionsPtr ExpressionActions::clone() const @@ -259,6 +256,7 @@ std::string ExpressionActions::Action::toString() const void ExpressionActions::checkLimits(const ColumnsWithTypeAndName & columns) const { + auto max_temporary_non_const_columns = actions_dag->getSettings().max_temporary_non_const_columns; if (max_temporary_non_const_columns) { size_t non_const_columns = 0; @@ -440,7 +438,7 @@ void ExpressionActions::execute(Block & block, size_t & num_rows, bool dry_run) } } - if (project_input) + if (actions_dag->getSettings().project_input) { block.clear(); } @@ -528,7 +526,7 @@ std::string ExpressionActions::dumpActions() const for (const auto & output_column : output_columns) ss << output_column.name << " " << output_column.type->getName() << "\n"; - ss << "\nproject input: " << project_input << "\noutput positions:"; + ss << "\nproject input: " << actions_dag->getSettings().project_input << "\noutput positions:"; for (auto pos : result_positions) ss << " " << pos; ss << "\n"; @@ -706,6 +704,7 @@ ActionsDAGPtr ActionsDAG::splitActionsBeforeArrayJoin(const NameSet & array_join auto split_actions = cloneEmpty(); split_actions->nodes.swap(split_nodes); split_actions->index.swap(split_index); + split_actions->settings.project_input = false; return split_actions; } diff --git a/src/Interpreters/ExpressionActions.h b/src/Interpreters/ExpressionActions.h index 493e07b8bdf..e62f79614b4 100644 --- a/src/Interpreters/ExpressionActions.h +++ b/src/Interpreters/ExpressionActions.h @@ -273,8 +273,10 @@ private: }; -/** Contains a sequence of actions on the block. - */ +/// Sequence of actions on the block. +/// Is used to calculate expressions. +/// +/// Takes ActionsDAG and orders actions using top-sort. class ExpressionActions { public: @@ -313,11 +315,6 @@ private: ColumnNumbers result_positions; Block sample_block; - /// This flag means that all columns except input will be removed from block before execution. - bool project_input = false; - - size_t max_temporary_non_const_columns = 0; - friend class ActionsDAG; public: @@ -330,9 +327,6 @@ public: const std::list & getNodes() const { return actions_dag->getNodes(); } const ActionsDAG & getActionsDAG() const { return *actions_dag; } - /// Adds to the beginning the removal of all extra columns. - void projectInput() { project_input = true; } - /// Get a list of input columns. Names getRequiredColumns() const; const NamesAndTypesList & getRequiredColumnsWithTypes() const { return required_columns; } From ed93ec673f8f18ed47ec0a2f60456a20acb4fef1 Mon Sep 17 00:00:00 2001 From: sundy-li <543950155@qq.com> Date: Wed, 11 Nov 2020 19:41:54 +0800 Subject: [PATCH 132/205] add zstd include directory for rocksdb --- contrib/rocksdb-cmake/CMakeLists.txt | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/contrib/rocksdb-cmake/CMakeLists.txt b/contrib/rocksdb-cmake/CMakeLists.txt index 205b5187da3..dc5d1acc819 100644 --- a/contrib/rocksdb-cmake/CMakeLists.txt +++ b/contrib/rocksdb-cmake/CMakeLists.txt @@ -74,6 +74,10 @@ else() if(WITH_ZSTD) add_definitions(-DZSTD) include_directories(${ZSTD_INCLUDE_DIR}) + include_directories(${ZSTD_INCLUDE_DIR}/common) + include_directories(${ZSTD_INCLUDE_DIR}/dictBuilder) + include_directories(${ZSTD_INCLUDE_DIR}/deprecated) + list(APPEND THIRDPARTY_LIBS zstd) endif() endif() From 906b338a1d6dbac9bf5202cd0e4cb1297a0f72c2 Mon Sep 17 00:00:00 2001 From: Alexander Kuzmenkov Date: Wed, 11 Nov 2020 14:51:21 +0300 Subject: [PATCH 133/205] fixes --- tests/performance/quantile_merge.xml | 2 +- tests/performance/trim_urls.xml | 1 + 2 files changed, 2 insertions(+), 1 deletion(-) diff --git a/tests/performance/quantile_merge.xml b/tests/performance/quantile_merge.xml index 36c4e583aa9..218b946bdde 100644 --- a/tests/performance/quantile_merge.xml +++ b/tests/performance/quantile_merge.xml @@ -1,3 +1,3 @@ - SELECT quantileMerge(arrayJoin(arrayMap(x -> state, range(500000)))) FROM (SELECT quantileState(rand()) AS state FROM numbers(100000)) + SELECT quantileMerge(arrayJoin(arrayMap(x -> state, range(5000000)))) FROM (SELECT quantileState(rand()) AS state FROM numbers(10000)) diff --git a/tests/performance/trim_urls.xml b/tests/performance/trim_urls.xml index f7308f15c04..a257267d070 100644 --- a/tests/performance/trim_urls.xml +++ b/tests/performance/trim_urls.xml @@ -20,6 +20,7 @@ trim(TRAILING '/' FROM trim(BOTH 'htpsw:/' FROM + SELECT ignore({func_fast}URL)) FROM hits_10m_single FORMAT Null From 3b2c681ac66f87e86b91590a3477a5dd34fc9733 Mon Sep 17 00:00:00 2001 From: Mikhail Filimonov Date: Wed, 11 Nov 2020 13:25:05 +0100 Subject: [PATCH 134/205] blind fix --- .../AggregateFunctionTimeSeriesGroupSum.h | 10 ++++++++-- 1 file changed, 8 insertions(+), 2 deletions(-) diff --git a/src/AggregateFunctions/AggregateFunctionTimeSeriesGroupSum.h b/src/AggregateFunctions/AggregateFunctionTimeSeriesGroupSum.h index b755fbf081b..7505ac54169 100644 --- a/src/AggregateFunctions/AggregateFunctionTimeSeriesGroupSum.h +++ b/src/AggregateFunctions/AggregateFunctionTimeSeriesGroupSum.h @@ -189,7 +189,10 @@ struct AggregateFunctionTimeSeriesGroupSumData { size_t size = result.size(); writeVarUInt(size, buf); - buf.write(reinterpret_cast(result.data()), sizeof(result[0])); + if (size > 0) + { + buf.write(reinterpret_cast(result.data()), sizeof(result[0])); + } } void deserialize(ReadBuffer & buf) @@ -197,7 +200,10 @@ struct AggregateFunctionTimeSeriesGroupSumData size_t size = 0; readVarUInt(size, buf); result.resize(size); - buf.read(reinterpret_cast(result.data()), size * sizeof(result[0])); + if (size > 0) + { + buf.read(reinterpret_cast(result.data()), size * sizeof(result[0])); + } } }; template From 55d05c95bfcd43c9b094da5ff389d7be1f0ad365 Mon Sep 17 00:00:00 2001 From: "a.palagashvili" Date: Wed, 11 Nov 2020 15:34:28 +0300 Subject: [PATCH 135/205] fixed style, xz check fasttest skipped, removed fast-lzma2 --- contrib/fast-lzma2 | 1 - docker/test/fasttest/run.sh | 1 + src/Server/HTTPHandler.cpp | 277 ++++++++++++++++++------------------ 3 files changed, 137 insertions(+), 142 deletions(-) delete mode 160000 contrib/fast-lzma2 diff --git a/contrib/fast-lzma2 b/contrib/fast-lzma2 deleted file mode 160000 index ded964d203c..00000000000 --- a/contrib/fast-lzma2 +++ /dev/null @@ -1 +0,0 @@ -Subproject commit ded964d203cabe1a572d2c813c55e8a94b4eda48 diff --git a/docker/test/fasttest/run.sh b/docker/test/fasttest/run.sh index 36aa8baf252..5b2f324e588 100755 --- a/docker/test/fasttest/run.sh +++ b/docker/test/fasttest/run.sh @@ -268,6 +268,7 @@ TESTS_TO_SKIP=( protobuf secure sha256 + xz # Not sure why these two fail even in sequential mode. Disabled for now # to make some progress. diff --git a/src/Server/HTTPHandler.cpp b/src/Server/HTTPHandler.cpp index d7a1484d51d..94d66d44af0 100644 --- a/src/Server/HTTPHandler.cpp +++ b/src/Server/HTTPHandler.cpp @@ -5,39 +5,39 @@ #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 #if !defined(ARCADIA_BUILD) # include @@ -46,8 +46,10 @@ namespace DB { + namespace ErrorCodes { + extern const int LOGICAL_ERROR; extern const int CANNOT_PARSE_TEXT; extern const int CANNOT_PARSE_ESCAPE_SEQUENCE; @@ -107,25 +109,36 @@ static Poco::Net::HTTPResponse::HTTPStatus exceptionCodeToHTTPStatus(int excepti { return HTTPResponse::HTTP_UNAUTHORIZED; } - else if ( - exception_code == ErrorCodes::CANNOT_PARSE_TEXT || exception_code == ErrorCodes::CANNOT_PARSE_ESCAPE_SEQUENCE - || exception_code == ErrorCodes::CANNOT_PARSE_QUOTED_STRING || exception_code == ErrorCodes::CANNOT_PARSE_DATE - || exception_code == ErrorCodes::CANNOT_PARSE_DATETIME || exception_code == ErrorCodes::CANNOT_PARSE_NUMBER - || exception_code == ErrorCodes::CANNOT_PARSE_INPUT_ASSERTION_FAILED || exception_code == ErrorCodes::UNKNOWN_ELEMENT_IN_AST - || exception_code == ErrorCodes::UNKNOWN_TYPE_OF_AST_NODE || exception_code == ErrorCodes::TOO_DEEP_AST - || exception_code == ErrorCodes::TOO_BIG_AST || exception_code == ErrorCodes::UNEXPECTED_AST_STRUCTURE - || exception_code == ErrorCodes::SYNTAX_ERROR || exception_code == ErrorCodes::INCORRECT_DATA - || exception_code == ErrorCodes::TYPE_MISMATCH) + else if (exception_code == ErrorCodes::CANNOT_PARSE_TEXT || + exception_code == ErrorCodes::CANNOT_PARSE_ESCAPE_SEQUENCE || + exception_code == ErrorCodes::CANNOT_PARSE_QUOTED_STRING || + exception_code == ErrorCodes::CANNOT_PARSE_DATE || + exception_code == ErrorCodes::CANNOT_PARSE_DATETIME || + exception_code == ErrorCodes::CANNOT_PARSE_NUMBER || + exception_code == ErrorCodes::CANNOT_PARSE_INPUT_ASSERTION_FAILED || + exception_code == ErrorCodes::UNKNOWN_ELEMENT_IN_AST || + exception_code == ErrorCodes::UNKNOWN_TYPE_OF_AST_NODE || + exception_code == ErrorCodes::TOO_DEEP_AST || + exception_code == ErrorCodes::TOO_BIG_AST || + exception_code == ErrorCodes::UNEXPECTED_AST_STRUCTURE || + exception_code == ErrorCodes::SYNTAX_ERROR || + exception_code == ErrorCodes::INCORRECT_DATA || + exception_code == ErrorCodes::TYPE_MISMATCH) { return HTTPResponse::HTTP_BAD_REQUEST; } - else if ( - exception_code == ErrorCodes::UNKNOWN_TABLE || exception_code == ErrorCodes::UNKNOWN_FUNCTION - || exception_code == ErrorCodes::UNKNOWN_IDENTIFIER || exception_code == ErrorCodes::UNKNOWN_TYPE - || exception_code == ErrorCodes::UNKNOWN_STORAGE || exception_code == ErrorCodes::UNKNOWN_DATABASE - || exception_code == ErrorCodes::UNKNOWN_SETTING || exception_code == ErrorCodes::UNKNOWN_DIRECTION_OF_SORTING - || exception_code == ErrorCodes::UNKNOWN_AGGREGATE_FUNCTION || exception_code == ErrorCodes::UNKNOWN_FORMAT - || exception_code == ErrorCodes::UNKNOWN_DATABASE_ENGINE || exception_code == ErrorCodes::UNKNOWN_TYPE_OF_QUERY) + else if (exception_code == ErrorCodes::UNKNOWN_TABLE || + exception_code == ErrorCodes::UNKNOWN_FUNCTION || + exception_code == ErrorCodes::UNKNOWN_IDENTIFIER || + exception_code == ErrorCodes::UNKNOWN_TYPE || + exception_code == ErrorCodes::UNKNOWN_STORAGE || + exception_code == ErrorCodes::UNKNOWN_DATABASE || + exception_code == ErrorCodes::UNKNOWN_SETTING || + exception_code == ErrorCodes::UNKNOWN_DIRECTION_OF_SORTING || + exception_code == ErrorCodes::UNKNOWN_AGGREGATE_FUNCTION || + exception_code == ErrorCodes::UNKNOWN_FORMAT || + exception_code == ErrorCodes::UNKNOWN_DATABASE_ENGINE || + exception_code == ErrorCodes::UNKNOWN_TYPE_OF_QUERY) { return HTTPResponse::HTTP_NOT_FOUND; } @@ -137,7 +150,8 @@ static Poco::Net::HTTPResponse::HTTPStatus exceptionCodeToHTTPStatus(int excepti { return HTTPResponse::HTTP_NOT_IMPLEMENTED; } - else if (exception_code == ErrorCodes::SOCKET_TIMEOUT || exception_code == ErrorCodes::CANNOT_OPEN_FILE) + else if (exception_code == ErrorCodes::SOCKET_TIMEOUT || + exception_code == ErrorCodes::CANNOT_OPEN_FILE) { return HTTPResponse::HTTP_SERVICE_UNAVAILABLE; } @@ -150,7 +164,9 @@ static Poco::Net::HTTPResponse::HTTPStatus exceptionCodeToHTTPStatus(int excepti } -static std::chrono::steady_clock::duration parseSessionTimeout(const Poco::Util::AbstractConfiguration & config, const HTMLForm & params) +static std::chrono::steady_clock::duration parseSessionTimeout( + const Poco::Util::AbstractConfiguration & config, + const HTMLForm & params) { unsigned session_timeout = config.getInt("default_session_timeout", 60); @@ -164,9 +180,8 @@ static std::chrono::steady_clock::duration parseSessionTimeout(const Poco::Util: throw Exception("Invalid session timeout: '" + session_timeout_str + "'", ErrorCodes::INVALID_SESSION_TIMEOUT); if (session_timeout > max_session_timeout) - throw Exception( - "Session timeout '" + session_timeout_str + "' is larger than max_session_timeout: " + toString(max_session_timeout) - + ". Maximum session timeout could be modified in configuration file.", + throw Exception("Session timeout '" + session_timeout_str + "' is larger than max_session_timeout: " + toString(max_session_timeout) + + ". Maximum session timeout could be modified in configuration file.", ErrorCodes::INVALID_SESSION_TIMEOUT); } @@ -194,7 +209,8 @@ void HTTPHandler::pushDelayedResults(Output & used_output) IReadableWriteBuffer * write_buf_concrete; ReadBufferPtr reread_buf; - if (write_buf && (write_buf_concrete = dynamic_cast(write_buf.get())) + if (write_buf + && (write_buf_concrete = dynamic_cast(write_buf.get())) && (reread_buf = write_buf_concrete->tryGetReadBuffer())) { read_buffers.emplace_back(reread_buf); @@ -207,7 +223,9 @@ void HTTPHandler::pushDelayedResults(Output & used_output) } -HTTPHandler::HTTPHandler(IServer & server_, const std::string & name) : server(server_), log(&Poco::Logger::get(name)) +HTTPHandler::HTTPHandler(IServer & server_, const std::string & name) + : server(server_) + , log(&Poco::Logger::get(name)) { server_display_name = server.config().getString("display_name", getFQDNOrHostName()); } @@ -253,12 +271,12 @@ void HTTPHandler::processQuery( else { /// It is prohibited to mix different authorization schemes. - if (request.hasCredentials() || params.has("user") || params.has("password") || params.has("quota_key")) + if (request.hasCredentials() + || params.has("user") + || params.has("password") + || params.has("quota_key")) { - throw Exception( - "Invalid authentication: it is not allowed to use X-ClickHouse HTTP headers and other authentication methods " - "simultaneously", - ErrorCodes::REQUIRED_PASSWORD); + throw Exception("Invalid authentication: it is not allowed to use X-ClickHouse HTTP headers and other authentication methods simultaneously", ErrorCodes::REQUIRED_PASSWORD); } } @@ -300,13 +318,12 @@ void HTTPHandler::processQuery( { std::string opentelemetry_traceparent = request.get("traceparent"); std::string error; - if (!context.getClientInfo().parseTraceparentHeader(opentelemetry_traceparent, error)) + if (!context.getClientInfo().parseTraceparentHeader( + opentelemetry_traceparent, error)) { - throw Exception( - ErrorCodes::BAD_REQUEST_PARAMETER, + throw Exception(ErrorCodes::BAD_REQUEST_PARAMETER, "Failed to parse OpenTelemetry traceparent header '{}': {}", - opentelemetry_traceparent, - error); + opentelemetry_traceparent, error); } context.getClientInfo().opentelemetry_tracestate = request.get("tracestate", ""); @@ -315,7 +332,8 @@ void HTTPHandler::processQuery( // Set the query id supplied by the user, if any, and also update the // OpenTelemetry fields. - context.setCurrentQueryId(params.get("query_id", request.get("X-ClickHouse-Query-Id", ""))); + context.setCurrentQueryId(params.get("query_id", + request.get("X-ClickHouse-Query-Id", ""))); /// The client can pass a HTTP header indicating supported compression method (gzip or deflate). String http_response_compression_methods = request.get("Accept-Encoding", ""); @@ -344,8 +362,8 @@ void HTTPHandler::processQuery( bool internal_compression = params.getParsed("compress", false); /// At least, we should postpone sending of first buffer_size result bytes - size_t buffer_size_total - = std::max(params.getParsed("buffer_size", DBMS_DEFAULT_BUFFER_SIZE), static_cast(DBMS_DEFAULT_BUFFER_SIZE)); + size_t buffer_size_total = std::max( + params.getParsed("buffer_size", DBMS_DEFAULT_BUFFER_SIZE), static_cast(DBMS_DEFAULT_BUFFER_SIZE)); /// If it is specified, the whole result will be buffered. /// First ~buffer_size bytes will be buffered in memory, the remaining bytes will be stored in temporary file. @@ -377,20 +395,23 @@ void HTTPHandler::processQuery( const std::string tmp_path(context.getTemporaryVolume()->getDisk()->getPath()); const std::string tmp_path_template(tmp_path + "http_buffers/"); - auto create_tmp_disk_buffer - = [tmp_path_template](const WriteBufferPtr &) { return WriteBufferFromTemporaryFile::create(tmp_path_template); }; + auto create_tmp_disk_buffer = [tmp_path_template] (const WriteBufferPtr &) + { + return WriteBufferFromTemporaryFile::create(tmp_path_template); + }; cascade_buffer2.emplace_back(std::move(create_tmp_disk_buffer)); } else { - auto push_memory_buffer_and_continue = [next_buffer = used_output.out_maybe_compressed](const WriteBufferPtr & prev_buf) { + auto push_memory_buffer_and_continue = [next_buffer = used_output.out_maybe_compressed] (const WriteBufferPtr & prev_buf) + { auto * prev_memory_buffer = typeid_cast(prev_buf.get()); if (!prev_memory_buffer) throw Exception("Expected MemoryWriteBuffer", ErrorCodes::LOGICAL_ERROR); auto rdbuf = prev_memory_buffer->tryGetReadBuffer(); - copyData(*rdbuf, *next_buffer); + copyData(*rdbuf , *next_buffer); return next_buffer; }; @@ -398,8 +419,8 @@ void HTTPHandler::processQuery( cascade_buffer2.emplace_back(push_memory_buffer_and_continue); } - used_output.out_maybe_delayed_and_compressed - = std::make_shared(std::move(cascade_buffer1), std::move(cascade_buffer2)); + used_output.out_maybe_delayed_and_compressed = std::make_shared( + std::move(cascade_buffer1), std::move(cascade_buffer2)); } else { @@ -425,23 +446,13 @@ void HTTPHandler::processQuery( std::unique_ptr in; - static const NameSet reserved_param_names{ - "compress", - "decompress", - "user", - "password", - "quota_key", - "query_id", - "stacktrace", - "buffer_size", - "wait_end_of_query", - "session_id", - "session_timeout", - "session_check"}; + static const NameSet reserved_param_names{"compress", "decompress", "user", "password", "quota_key", "query_id", "stacktrace", + "buffer_size", "wait_end_of_query", "session_id", "session_timeout", "session_check"}; Names reserved_param_suffixes; - auto param_could_be_skipped = [&](const String & name) { + auto param_could_be_skipped = [&] (const String & name) + { /// Empty parameter appears when URL like ?&a=b or a=b&&c=d. Just skip them for user's convenience. if (name.empty()) return true; @@ -566,10 +577,12 @@ void HTTPHandler::processQuery( client_info.http_method = http_method; client_info.http_user_agent = request.get("User-Agent", ""); - auto append_callback = [&context](ProgressCallback callback) { + auto append_callback = [&context] (ProgressCallback callback) + { auto prev = context.getProgressCallback(); - context.setProgressCallback([prev, callback](const Progress & progress) { + context.setProgressCallback([prev, callback] (const Progress & progress) + { if (prev) prev(progress); @@ -579,13 +592,14 @@ 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) - append_callback([&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(); - append_callback([&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. try @@ -609,17 +623,15 @@ void HTTPHandler::processQuery( query_scope.emplace(context); - executeQuery( - *in, - *used_output.out_maybe_delayed_and_compressed, - /* allow_into_outfile = */ false, - context, - [&response](const String & current_query_id, const String & content_type, const String & format, const String & timezone) { + executeQuery(*in, *used_output.out_maybe_delayed_and_compressed, /* allow_into_outfile = */ false, context, + [&response] (const String & current_query_id, const String & content_type, const String & format, const String & timezone) + { response.setContentType(content_type); response.add("X-ClickHouse-Query-Id", current_query_id); response.add("X-ClickHouse-Format", format); response.add("X-ClickHouse-Timezone", timezone); - }); + } + ); if (used_output.hasDelayed()) { @@ -632,11 +644,8 @@ void HTTPHandler::processQuery( used_output.out->finalize(); } -void HTTPHandler::trySendExceptionToClient( - const std::string & s, - int exception_code, - Poco::Net::HTTPServerRequest & request, - Poco::Net::HTTPServerResponse & response, +void HTTPHandler::trySendExceptionToClient(const std::string & s, int exception_code, + Poco::Net::HTTPServerRequest & request, Poco::Net::HTTPServerResponse & response, Output & used_output) { try @@ -645,14 +654,17 @@ void HTTPHandler::trySendExceptionToClient( /// If HTTP method is POST and Keep-Alive is turned on, we should read the whole request body /// to avoid reading part of the current request body in the next request. - if (request.getMethod() == Poco::Net::HTTPRequest::HTTP_POST && response.getKeepAlive() && !request.stream().eof() + if (request.getMethod() == Poco::Net::HTTPRequest::HTTP_POST + && response.getKeepAlive() + && !request.stream().eof() && exception_code != ErrorCodes::HTTP_LENGTH_REQUIRED) { request.stream().ignore(std::numeric_limits::max()); } - bool auth_fail = exception_code == ErrorCodes::UNKNOWN_USER || exception_code == ErrorCodes::WRONG_PASSWORD - || exception_code == ErrorCodes::REQUIRED_PASSWORD; + bool auth_fail = exception_code == ErrorCodes::UNKNOWN_USER || + exception_code == ErrorCodes::WRONG_PASSWORD || + exception_code == ErrorCodes::REQUIRED_PASSWORD; if (auth_fail) { @@ -730,12 +742,10 @@ void HTTPHandler::handleRequest(Poco::Net::HTTPServerRequest & request, Poco::Ne with_stacktrace = params.getParsed("stacktrace", false); /// Workaround. Poco does not detect 411 Length Required case. - if (request.getMethod() == Poco::Net::HTTPRequest::HTTP_POST && !request.getChunkedTransferEncoding() - && !request.hasContentLength()) + if (request.getMethod() == Poco::Net::HTTPRequest::HTTP_POST && !request.getChunkedTransferEncoding() && + !request.hasContentLength()) { - throw Exception( - "The Transfer-Encoding is not chunked and there is no Content-Length header for POST request", - ErrorCodes::HTTP_LENGTH_REQUIRED); + throw Exception("The Transfer-Encoding is not chunked and there is no Content-Length header for POST request", ErrorCodes::HTTP_LENGTH_REQUIRED); } processQuery(context, request, params, response, used_output, query_scope); @@ -763,7 +773,7 @@ DynamicQueryHandler::DynamicQueryHandler(IServer & server_, const std::string & bool DynamicQueryHandler::customizeQueryParam(Context & context, const std::string & key, const std::string & value) { if (key == param_name) - return true; /// do nothing + return true; /// do nothing if (startsWith(key, "param_")) { @@ -778,6 +788,7 @@ bool DynamicQueryHandler::customizeQueryParam(Context & context, const std::stri std::string DynamicQueryHandler::getQuery(Poco::Net::HTTPServerRequest & request, HTMLForm & params, Context & context) { + if (likely(!startsWith(request.getContentType(), "multipart/form-data"))) { /// Part of the query can be passed in the 'query' parameter and the rest in the request body @@ -802,16 +813,10 @@ std::string DynamicQueryHandler::getQuery(Poco::Net::HTTPServerRequest & request } PredefinedQueryHandler::PredefinedQueryHandler( - IServer & server_, - const NameSet & receive_params_, - const std::string & predefined_query_, - const CompiledRegexPtr & url_regex_, - const std::unordered_map & header_name_with_regex_) - : HTTPHandler(server_, "PredefinedQueryHandler") - , receive_params(receive_params_) - , predefined_query(predefined_query_) - , url_regex(url_regex_) - , header_name_with_capture_regex(header_name_with_regex_) + IServer & server_, const NameSet & receive_params_, const std::string & predefined_query_ + , const CompiledRegexPtr & url_regex_, const std::unordered_map & header_name_with_regex_) + : HTTPHandler(server_, "PredefinedQueryHandler"), receive_params(receive_params_), predefined_query(predefined_query_) + , url_regex(url_regex_), header_name_with_capture_regex(header_name_with_regex_) { } @@ -831,7 +836,8 @@ void PredefinedQueryHandler::customizeContext(Poco::Net::HTTPServerRequest & req /// If in the configuration file, the handler's header is regex and contains named capture group /// We will extract regex named capture groups as query parameters - const auto & set_query_params = [&](const char * begin, const char * end, const CompiledRegexPtr & compiled_regex) { + const auto & set_query_params = [&](const char * begin, const char * end, const CompiledRegexPtr & compiled_regex) + { int num_captures = compiled_regex->NumberOfCapturingGroups() + 1; re2::StringPiece matches[num_captures]; @@ -876,16 +882,16 @@ std::string PredefinedQueryHandler::getQuery(Poco::Net::HTTPServerRequest & requ Poco::Net::HTTPRequestHandlerFactory * createDynamicHandlerFactory(IServer & server, const std::string & config_prefix) { std::string query_param_name = server.config().getString(config_prefix + ".handler.query_param_name", "query"); - return addFiltersFromConfig( - new HandlingRuleHTTPHandlerFactory(server, std::move(query_param_name)), server.config(), config_prefix); + return addFiltersFromConfig(new HandlingRuleHTTPHandlerFactory(server, std::move(query_param_name)), server.config(), config_prefix); } static inline bool capturingNamedQueryParam(NameSet receive_params, const CompiledRegexPtr & compiled_regex) { const auto & capturing_names = compiled_regex->NamedCapturingGroups(); - return std::count_if(capturing_names.begin(), capturing_names.end(), [&](const auto & iterator) { - return std::count_if( - receive_params.begin(), receive_params.end(), [&](const auto & param_name) { return param_name == iterator.first; }); + return std::count_if(capturing_names.begin(), capturing_names.end(), [&](const auto & iterator) + { + return std::count_if(receive_params.begin(), receive_params.end(), + [&](const auto & param_name) { return param_name == iterator.first; }); }); } @@ -894,10 +900,8 @@ static inline CompiledRegexPtr getCompiledRegex(const std::string & expression) auto compiled_regex = std::make_shared(expression); if (!compiled_regex->ok()) - throw Exception( - "Cannot compile re2: " + expression + " for http handling rule, error: " + compiled_regex->error() - + ". Look at https://github.com/google/re2/wiki/Syntax for reference.", - ErrorCodes::CANNOT_COMPILE_REGEXP); + throw Exception("Cannot compile re2: " + expression + " for http handling rule, error: " + + compiled_regex->error() + ". Look at https://github.com/google/re2/wiki/Syntax for reference.", ErrorCodes::CANNOT_COMPILE_REGEXP); return compiled_regex; } @@ -907,8 +911,7 @@ Poco::Net::HTTPRequestHandlerFactory * createPredefinedHandlerFactory(IServer & Poco::Util::AbstractConfiguration & configuration = server.config(); if (!configuration.has(config_prefix + ".handler.query")) - throw Exception( - "There is no path '" + config_prefix + ".handler.query" + "' in configuration file.", ErrorCodes::NO_ELEMENTS_IN_CONFIG); + throw Exception("There is no path '" + config_prefix + ".handler.query" + "' in configuration file.", ErrorCodes::NO_ELEMENTS_IN_CONFIG); std::string predefined_query = configuration.getString(config_prefix + ".handler.query"); NameSet analyze_receive_params = analyzeReceiveQueryParams(predefined_query); @@ -939,22 +942,14 @@ Poco::Net::HTTPRequestHandlerFactory * createPredefinedHandlerFactory(IServer & auto regex = getCompiledRegex(url_expression); if (capturingNamedQueryParam(analyze_receive_params, regex)) - return addFiltersFromConfig( - new HandlingRuleHTTPHandlerFactory( - server, - std::move(analyze_receive_params), - std::move(predefined_query), - std::move(regex), - std::move(headers_name_with_regex)), - configuration, - config_prefix); + return addFiltersFromConfig(new HandlingRuleHTTPHandlerFactory( + server, std::move(analyze_receive_params), std::move(predefined_query), std::move(regex), + std::move(headers_name_with_regex)), configuration, config_prefix); } - return addFiltersFromConfig( - new HandlingRuleHTTPHandlerFactory( - server, std::move(analyze_receive_params), std::move(predefined_query), CompiledRegexPtr{}, std::move(headers_name_with_regex)), - configuration, - config_prefix); + return addFiltersFromConfig(new HandlingRuleHTTPHandlerFactory( + server, std::move(analyze_receive_params), std::move(predefined_query), CompiledRegexPtr{} ,std::move(headers_name_with_regex)), + configuration, config_prefix); } } From 1b06fd9373c6cbaf16e082f0ffdd8d1e4700628f Mon Sep 17 00:00:00 2001 From: "a.palagashvili" Date: Wed, 11 Nov 2020 15:57:35 +0300 Subject: [PATCH 136/205] regenerated ya.make --- src/IO/ya.make | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/IO/ya.make b/src/IO/ya.make index 33b735fafa5..a4d406d73ce 100644 --- a/src/IO/ya.make +++ b/src/IO/ya.make @@ -20,9 +20,9 @@ SRCS( HTTPCommon.cpp HashingWriteBuffer.cpp HexWriteBuffer.cpp + LZMADeflatingWriteBuffer.cpp + LZMAInflatingReadBuffer.cpp LimitReadBuffer.cpp - LzmaReadBuffer.cpp - LzmaWriteBuffer.cpp MMapReadBufferFromFile.cpp MMapReadBufferFromFileDescriptor.cpp MemoryReadWriteBuffer.cpp From 79954051096773a9e9c12cabbd77642f66d7287e Mon Sep 17 00:00:00 2001 From: filimonov <1549571+filimonov@users.noreply.github.com> Date: Wed, 11 Nov 2020 14:27:21 +0100 Subject: [PATCH 137/205] Update 01560_timeseriesgroupsum_segfault.sql --- tests/queries/0_stateless/01560_timeseriesgroupsum_segfault.sql | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/01560_timeseriesgroupsum_segfault.sql b/tests/queries/0_stateless/01560_timeseriesgroupsum_segfault.sql index 5aeae422f1b..eb7c91967e5 100644 --- a/tests/queries/0_stateless/01560_timeseriesgroupsum_segfault.sql +++ b/tests/queries/0_stateless/01560_timeseriesgroupsum_segfault.sql @@ -15,7 +15,7 @@ CREATE TABLE tsgs AS tsgs_local ENGINE = Distributed(test_cluster_two_shards, cu SELECT timeSeriesGroupSum(a, b, c) FROM tsgs; -SELECT count() FROM ( SELECT timeSeriesGroupSumState(a, b, c) FROM tsgs_local) WHERE NOT ignore(*); +SELECT count() FROM ( SELECT timeSeriesGroupSumState(a, b, c) as x FROM tsgs_local) WHERE NOT ignore(*); SELECT 'server is still alive'; From 6f0dc08ba7084ff2ab760812d343dc6a11ecbdb2 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Wed, 11 Nov 2020 16:34:07 +0300 Subject: [PATCH 138/205] avoid code duplication in alterPartition --- src/Storages/MergeTree/MergeTreeData.cpp | 90 ++++++++++++++++++++ src/Storages/MergeTree/MergeTreeData.h | 12 +++ src/Storages/StorageMergeTree.cpp | 92 +-------------------- src/Storages/StorageMergeTree.h | 13 +-- src/Storages/StorageReplicatedMergeTree.cpp | 87 ------------------- src/Storages/StorageReplicatedMergeTree.h | 15 ++-- 6 files changed, 113 insertions(+), 196 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 175bc72b08d..441c95fec4e 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -2753,6 +2753,96 @@ void MergeTreeData::movePartitionToVolume(const ASTPtr & partition, const String throw Exception("Cannot move parts because moves are manually disabled", ErrorCodes::ABORTED); } +void MergeTreeData::fetchPartition(const ASTPtr & /*partition*/, const StorageMetadataPtr & /*metadata_snapshot*/, const String & /*from*/, const Context & /*query_context*/) +{ + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "FETCH PARTITION is not supported by storage {}", getName()); +} + +Pipe MergeTreeData::alterPartition( + const StorageMetadataPtr & metadata_snapshot, + const PartitionCommands & commands, + const Context & query_context) +{ + PartitionCommandsResultInfo result; + for (const PartitionCommand & command : commands) + { + PartitionCommandsResultInfo current_command_results; + switch (command.type) + { + case PartitionCommand::DROP_PARTITION: + if (command.part) + checkPartCanBeDropped(command.partition); + else + checkPartitionCanBeDropped(command.partition); + dropPartition(command.partition, command.detach, command.part, query_context); + break; + + case PartitionCommand::DROP_DETACHED_PARTITION: + dropDetached(command.partition, command.part, query_context); + break; + + case PartitionCommand::ATTACH_PARTITION: + current_command_results = attachPartition(command.partition, metadata_snapshot, command.part, query_context); + break; + case PartitionCommand::MOVE_PARTITION: + { + switch (*command.move_destination_type) + { + case PartitionCommand::MoveDestinationType::DISK: + movePartitionToDisk(command.partition, command.move_destination_name, command.part, query_context); + break; + + case PartitionCommand::MoveDestinationType::VOLUME: + movePartitionToVolume(command.partition, command.move_destination_name, command.part, query_context); + break; + + case PartitionCommand::MoveDestinationType::TABLE: + checkPartitionCanBeDropped(command.partition); + String dest_database = query_context.resolveDatabase(command.to_database); + auto dest_storage = DatabaseCatalog::instance().getTable({dest_database, command.to_table}, query_context); + movePartitionToTable(dest_storage, command.partition, query_context); + break; + } + } + break; + + case PartitionCommand::REPLACE_PARTITION: + { + checkPartitionCanBeDropped(command.partition); + String from_database = query_context.resolveDatabase(command.from_database); + auto from_storage = DatabaseCatalog::instance().getTable({from_database, command.from_table}, query_context); + replacePartitionFrom(from_storage, command.partition, command.replace, query_context); + } + break; + + case PartitionCommand::FETCH_PARTITION: + fetchPartition(command.partition, metadata_snapshot, command.from_zookeeper_path, query_context); + break; + + case PartitionCommand::FREEZE_PARTITION: + { + auto lock = lockForShare(query_context.getCurrentQueryId(), query_context.getSettingsRef().lock_acquire_timeout); + current_command_results = freezePartition(command.partition, metadata_snapshot, command.with_name, query_context, lock); + } + break; + + case PartitionCommand::FREEZE_ALL_PARTITIONS: + { + auto lock = lockForShare(query_context.getCurrentQueryId(), query_context.getSettingsRef().lock_acquire_timeout); + current_command_results = freezeAll(command.with_name, metadata_snapshot, query_context, lock); + } + break; + } + for (auto & command_result : current_command_results) + command_result.command_type = command.typeToString(); + result.insert(result.end(), current_command_results.begin(), current_command_results.end()); + } + + if (query_context.getSettingsRef().alter_partition_verbose_result) + return convertCommandsResultToSource(result); + + return {}; +} String MergeTreeData::getPartitionIDFromQuery(const ASTPtr & ast, const Context & context) const { diff --git a/src/Storages/MergeTree/MergeTreeData.h b/src/Storages/MergeTree/MergeTreeData.h index 6bfb1613754..a2c8aa55e99 100644 --- a/src/Storages/MergeTree/MergeTreeData.h +++ b/src/Storages/MergeTree/MergeTreeData.h @@ -563,6 +563,11 @@ public: void checkPartCanBeDropped(const ASTPtr & part); + Pipe alterPartition( + const StorageMetadataPtr & metadata_snapshot, + const PartitionCommands & commands, + const Context & query_context) override; + size_t getColumnCompressedSize(const std::string & name) const { auto lock = lockParts(); @@ -868,8 +873,15 @@ protected: using MatcherFn = std::function; PartitionCommandsResultInfo freezePartitionsByMatcher(MatcherFn matcher, const StorageMetadataPtr & metadata_snapshot, const String & with_name, const Context & context); + // Partition helpers bool canReplacePartition(const DataPartPtr & src_part) const; + virtual void dropPartition(const ASTPtr & partition, bool detach, bool drop_part, const Context & context) = 0; + virtual PartitionCommandsResultInfo attachPartition(const ASTPtr & partition, const StorageMetadataPtr & metadata_snapshot, bool part, const Context & context) = 0; + virtual void replacePartitionFrom(const StoragePtr & source_table, const ASTPtr & partition, bool replace, const Context & context) = 0; + virtual void movePartitionToTable(const StoragePtr & dest_table, const ASTPtr & partition, const Context & context) = 0; + virtual void fetchPartition(const ASTPtr & partition, const StorageMetadataPtr & metadata_snapshot, const String & from, const Context & query_context); + void writePartLog( PartLogElement::Type type, const ExecutionStatus & execution_status, diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index cc889d76c9c..fa4698af637 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -1056,95 +1056,6 @@ bool StorageMergeTree::optimize( return true; } -Pipe StorageMergeTree::alterPartition( - const StorageMetadataPtr & metadata_snapshot, - const PartitionCommands & commands, - const Context & query_context) -{ - PartitionCommandsResultInfo result; - for (const PartitionCommand & command : commands) - { - PartitionCommandsResultInfo current_command_results; - switch (command.type) - { - case PartitionCommand::DROP_PARTITION: - if (command.part) - checkPartCanBeDropped(command.partition); - else - checkPartitionCanBeDropped(command.partition); - dropPartition(command.partition, command.detach, command.part, query_context); - break; - - case PartitionCommand::DROP_DETACHED_PARTITION: - dropDetached(command.partition, command.part, query_context); - break; - - case PartitionCommand::ATTACH_PARTITION: - current_command_results = attachPartition(command.partition, command.part, query_context); - break; - - case PartitionCommand::MOVE_PARTITION: - { - switch (*command.move_destination_type) - { - case PartitionCommand::MoveDestinationType::DISK: - movePartitionToDisk(command.partition, command.move_destination_name, command.part, query_context); - break; - - case PartitionCommand::MoveDestinationType::VOLUME: - movePartitionToVolume(command.partition, command.move_destination_name, command.part, query_context); - break; - - case PartitionCommand::MoveDestinationType::TABLE: - checkPartitionCanBeDropped(command.partition); - String dest_database = query_context.resolveDatabase(command.to_database); - auto dest_storage = DatabaseCatalog::instance().getTable({dest_database, command.to_table}, query_context); - movePartitionToTable(dest_storage, command.partition, query_context); - break; - } - - } - break; - - case PartitionCommand::REPLACE_PARTITION: - { - checkPartitionCanBeDropped(command.partition); - String from_database = query_context.resolveDatabase(command.from_database); - auto from_storage = DatabaseCatalog::instance().getTable({from_database, command.from_table}, query_context); - replacePartitionFrom(from_storage, command.partition, command.replace, query_context); - } - break; - - case PartitionCommand::FREEZE_PARTITION: - { - auto lock = lockForShare(query_context.getCurrentQueryId(), query_context.getSettingsRef().lock_acquire_timeout); - current_command_results = freezePartition(command.partition, metadata_snapshot, command.with_name, query_context, lock); - } - break; - - case PartitionCommand::FREEZE_ALL_PARTITIONS: - { - auto lock = lockForShare(query_context.getCurrentQueryId(), query_context.getSettingsRef().lock_acquire_timeout); - current_command_results = freezeAll(command.with_name, metadata_snapshot, query_context, lock); - } - break; - - default: - IStorage::alterPartition(metadata_snapshot, commands, query_context); // should throw an exception. - } - - for (auto & command_result : current_command_results) - command_result.command_type = command.typeToString(); - result.insert(result.end(), current_command_results.begin(), current_command_results.end()); - } - - if (query_context.getSettingsRef().alter_partition_verbose_result) - return convertCommandsResultToSource(result); - - return {}; -} - - ActionLock StorageMergeTree::stopMergesAndWait() { /// Asks to complete merges and does not allow them to start. @@ -1221,7 +1132,8 @@ void StorageMergeTree::dropPartition(const ASTPtr & partition, bool detach, bool PartitionCommandsResultInfo StorageMergeTree::attachPartition( - const ASTPtr & partition, bool attach_part, const Context & context) + const ASTPtr & partition, const StorageMetadataPtr & /* metadata_snapshot */, + bool attach_part, const Context & context) { PartitionCommandsResultInfo results; PartsTemporaryRename renamed_parts(*this, "detached/"); diff --git a/src/Storages/StorageMergeTree.h b/src/Storages/StorageMergeTree.h index d765b8bb92a..5d57d104338 100644 --- a/src/Storages/StorageMergeTree.h +++ b/src/Storages/StorageMergeTree.h @@ -62,11 +62,6 @@ public: bool deduplicate, const Context & context) override; - Pipe alterPartition( - const StorageMetadataPtr & /* metadata_snapshot */, - const PartitionCommands & commands, - const Context & context) override; - void mutate(const MutationCommands & commands, const Context & context) override; /// Return introspection information about currently processing or recently processed mutations. @@ -182,11 +177,11 @@ private: void clearOldMutations(bool truncate = false); // Partition helpers - void dropPartition(const ASTPtr & partition, bool detach, bool drop_part, const Context & context); - PartitionCommandsResultInfo attachPartition(const ASTPtr & partition, bool part, const Context & context); + void dropPartition(const ASTPtr & partition, bool detach, bool drop_part, const Context & context) override; + PartitionCommandsResultInfo attachPartition(const ASTPtr & partition, const StorageMetadataPtr & metadata_snapshot, bool part, const Context & context) override; - void replacePartitionFrom(const StoragePtr & source_table, const ASTPtr & partition, bool replace, const Context & context); - void movePartitionToTable(const StoragePtr & dest_table, const ASTPtr & partition, const Context & context); + void replacePartitionFrom(const StoragePtr & source_table, const ASTPtr & partition, bool replace, const Context & context) override; + void movePartitionToTable(const StoragePtr & dest_table, const ASTPtr & partition, const Context & context) override; bool partIsAssignedToBackgroundOperation(const DataPartPtr & part) const override; /// Update mutation entries after part mutation execution. May reset old /// errors if mutation was successful. Otherwise update last_failed* fields diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 8c48febca1b..2a998f79e26 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -4170,93 +4170,6 @@ void StorageReplicatedMergeTree::alter( } } -Pipe StorageReplicatedMergeTree::alterPartition( - const StorageMetadataPtr & metadata_snapshot, - const PartitionCommands & commands, - const Context & query_context) -{ - PartitionCommandsResultInfo result; - for (const PartitionCommand & command : commands) - { - PartitionCommandsResultInfo current_command_results; - switch (command.type) - { - case PartitionCommand::DROP_PARTITION: - if (command.part) - checkPartCanBeDropped(command.partition); - else - checkPartitionCanBeDropped(command.partition); - dropPartition(command.partition, command.detach, command.part, query_context); - break; - - case PartitionCommand::DROP_DETACHED_PARTITION: - dropDetached(command.partition, command.part, query_context); - break; - - case PartitionCommand::ATTACH_PARTITION: - current_command_results = attachPartition(command.partition, metadata_snapshot, command.part, query_context); - break; - case PartitionCommand::MOVE_PARTITION: - { - switch (*command.move_destination_type) - { - case PartitionCommand::MoveDestinationType::DISK: - movePartitionToDisk(command.partition, command.move_destination_name, command.part, query_context); - break; - - case PartitionCommand::MoveDestinationType::VOLUME: - movePartitionToVolume(command.partition, command.move_destination_name, command.part, query_context); - break; - - case PartitionCommand::MoveDestinationType::TABLE: - checkPartitionCanBeDropped(command.partition); - String dest_database = query_context.resolveDatabase(command.to_database); - auto dest_storage = DatabaseCatalog::instance().getTable({dest_database, command.to_table}, query_context); - movePartitionToTable(dest_storage, command.partition, query_context); - break; - } - } - break; - - case PartitionCommand::REPLACE_PARTITION: - { - checkPartitionCanBeDropped(command.partition); - String from_database = query_context.resolveDatabase(command.from_database); - auto from_storage = DatabaseCatalog::instance().getTable({from_database, command.from_table}, query_context); - replacePartitionFrom(from_storage, command.partition, command.replace, query_context); - } - break; - - case PartitionCommand::FETCH_PARTITION: - fetchPartition(command.partition, metadata_snapshot, command.from_zookeeper_path, query_context); - break; - - case PartitionCommand::FREEZE_PARTITION: - { - auto lock = lockForShare(query_context.getCurrentQueryId(), query_context.getSettingsRef().lock_acquire_timeout); - current_command_results = freezePartition(command.partition, metadata_snapshot, command.with_name, query_context, lock); - } - break; - - case PartitionCommand::FREEZE_ALL_PARTITIONS: - { - auto lock = lockForShare(query_context.getCurrentQueryId(), query_context.getSettingsRef().lock_acquire_timeout); - current_command_results = freezeAll(command.with_name, metadata_snapshot, query_context, lock); - } - break; - } - for (auto & command_result : current_command_results) - command_result.command_type = command.typeToString(); - result.insert(result.end(), current_command_results.begin(), current_command_results.end()); - } - - if (query_context.getSettingsRef().alter_partition_verbose_result) - return convertCommandsResultToSource(result); - - return {}; -} - - /// If new version returns ordinary name, else returns part name containing the first and last month of the month /// NOTE: use it in pair with getFakePartCoveringAllPartsInPartition(...) static String getPartNamePossiblyFake(MergeTreeDataFormatVersion format_version, const MergeTreePartInfo & part_info) diff --git a/src/Storages/StorageReplicatedMergeTree.h b/src/Storages/StorageReplicatedMergeTree.h index c9f94427112..2d92a033e6f 100644 --- a/src/Storages/StorageReplicatedMergeTree.h +++ b/src/Storages/StorageReplicatedMergeTree.h @@ -112,11 +112,6 @@ public: void alter(const AlterCommands & commands, const Context & query_context, TableLockHolder & table_lock_holder) override; - Pipe alterPartition( - const StorageMetadataPtr & metadata_snapshot, - const PartitionCommands & commands, - const Context & query_context) override; - void mutate(const MutationCommands & commands, const Context & context) override; void waitMutation(const String & znode_name, size_t mutations_sync) const; std::vector getMutationsStatus() const override; @@ -548,11 +543,11 @@ private: zkutil::ZooKeeper & zookeeper, String & partition_id, LogEntry & entry, bool detach); // Partition helpers - void dropPartition(const ASTPtr & partition, bool detach, bool drop_part, const Context & query_context); - PartitionCommandsResultInfo attachPartition(const ASTPtr & partition, const StorageMetadataPtr & metadata_snapshot, bool part, const Context & query_context); - void replacePartitionFrom(const StoragePtr & source_table, const ASTPtr & partition, bool replace, const Context & query_context); - void movePartitionToTable(const StoragePtr & dest_table, const ASTPtr & partition, const Context & query_context); - void fetchPartition(const ASTPtr & partition, const StorageMetadataPtr & metadata_snapshot, const String & from, const Context & query_context); + void dropPartition(const ASTPtr & partition, bool detach, bool drop_part, const Context & query_context) override; + PartitionCommandsResultInfo attachPartition(const ASTPtr & partition, const StorageMetadataPtr & metadata_snapshot, bool part, const Context & query_context) override; + void replacePartitionFrom(const StoragePtr & source_table, const ASTPtr & partition, bool replace, const Context & query_context) override; + void movePartitionToTable(const StoragePtr & dest_table, const ASTPtr & partition, const Context & query_context) override; + void fetchPartition(const ASTPtr & partition, const StorageMetadataPtr & metadata_snapshot, const String & from, const Context & query_context) override; /// Check granularity of already existing replicated table in zookeeper if it exists /// return true if it's fixed From b78657656cb2470a6d26fb343ba1a196608e4e85 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Wed, 11 Nov 2020 16:50:29 +0300 Subject: [PATCH 139/205] Add ActionsDAG::updateHeader() --- src/Interpreters/ExpressionActions.cpp | 15 +++++++++++++ src/Interpreters/ExpressionActions.h | 4 ++++ src/Interpreters/ExpressionAnalyzer.cpp | 7 +++--- src/Interpreters/InterpreterSelectQuery.cpp | 2 +- src/Processors/QueryPlan/ExpressionStep.cpp | 4 ++-- src/Processors/QueryPlan/FilterStep.cpp | 2 +- src/Processors/QueryPlan/TotalsHavingStep.cpp | 22 +++++++++---------- .../Transforms/ExpressionTransform.cpp | 7 +++--- .../Transforms/ExpressionTransform.h | 4 +++- src/Processors/Transforms/FilterTransform.cpp | 7 +++--- src/Processors/Transforms/FilterTransform.h | 4 +++- .../Transforms/TotalsHavingTransform.cpp | 11 +++++----- .../Transforms/TotalsHavingTransform.h | 4 +++- 13 files changed, 58 insertions(+), 35 deletions(-) diff --git a/src/Interpreters/ExpressionActions.cpp b/src/Interpreters/ExpressionActions.cpp index a51e8fe485c..430e93edb31 100644 --- a/src/Interpreters/ExpressionActions.cpp +++ b/src/Interpreters/ExpressionActions.cpp @@ -467,6 +467,21 @@ void ExpressionActions::execute(Block & block, bool dry_run) const block.insert({DataTypeUInt8().createColumnConst(num_rows, 0), std::make_shared(), "_dummy"}); } +void ActionsDAG::updateHeader(Block & block) const +{ + if (settings.project_input) + block.clear(); + else + { + for (const auto & node : nodes) + if (node.type == ActionType::INPUT && block.has(node.result_name)) + block.erase(node.result_name); + } + + for (const auto & node : index) + block.insert({node->column, node->result_type, node->result_name}); +} + Names ExpressionActions::getRequiredColumns() const { Names names; diff --git a/src/Interpreters/ExpressionActions.h b/src/Interpreters/ExpressionActions.h index e62f79614b4..d701633cdf8 100644 --- a/src/Interpreters/ExpressionActions.h +++ b/src/Interpreters/ExpressionActions.h @@ -237,6 +237,10 @@ public: void projectInput() { settings.project_input = true; } void removeUnusedActions(const Names & required_names); + /// Changed structure of empty block as if actions are executed. + /// Does not actually execute actions, but only creates empty columns. + void updateHeader(Block & block) const; + /// Splits actions into two parts. Returned half may be swapped with ARRAY JOIN. /// Returns nullptr if no actions may be moved before ARRAY JOIN. ActionsDAGPtr splitActionsBeforeArrayJoin(const NameSet & array_joined_columns); diff --git a/src/Interpreters/ExpressionAnalyzer.cpp b/src/Interpreters/ExpressionAnalyzer.cpp index 7a8d7770bfc..d706199f94e 100644 --- a/src/Interpreters/ExpressionAnalyzer.cpp +++ b/src/Interpreters/ExpressionAnalyzer.cpp @@ -673,8 +673,7 @@ ActionsDAGPtr SelectQueryExpressionAnalyzer::appendPrewhere( auto tmp_actions_dag = std::make_shared(sourceColumns()); getRootActions(select_query->prewhere(), only_types, tmp_actions_dag); tmp_actions_dag->removeUnusedActions({prewhere_column_name}); - auto tmp_actions = std::make_shared(tmp_actions_dag); - auto required_columns = tmp_actions->getRequiredColumns(); + auto required_columns = tmp_actions_dag->getRequiredColumns(); NameSet required_source_columns(required_columns.begin(), required_columns.end()); /// Add required columns to required output in order not to remove them after prewhere execution. @@ -1143,7 +1142,7 @@ ExpressionAnalysisResult::ExpressionAnalysisResult( Block before_prewhere_sample = source_header; if (sanitizeBlock(before_prewhere_sample)) { - ExpressionActions(prewhere_info->prewhere_actions).execute(before_prewhere_sample); + prewhere_info->prewhere_actions->updateHeader(before_prewhere_sample); auto & column_elem = before_prewhere_sample.getByName(query.prewhere()->getColumnName()); /// If the filter column is a constant, record it. if (column_elem.column) @@ -1176,7 +1175,7 @@ ExpressionAnalysisResult::ExpressionAnalysisResult( before_where_sample = source_header; if (sanitizeBlock(before_where_sample)) { - ExpressionActions(before_where).execute(before_where_sample); + before_where->updateHeader(before_where_sample); auto & column_elem = before_where_sample.getByName(query.where()->getColumnName()); /// If the filter column is a constant, record it. if (column_elem.column) diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index 3fafcb438b6..9d8e950cfcc 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -524,7 +524,7 @@ Block InterpreterSelectQuery::getSampleBlockImpl() if (analysis_result.prewhere_info) { - ExpressionActions(analysis_result.prewhere_info->prewhere_actions).execute(header); + analysis_result.prewhere_info->prewhere_actions->updateHeader(header); header = materializeBlock(header); if (analysis_result.prewhere_info->remove_prewhere_column) header.erase(analysis_result.prewhere_info->prewhere_column_name); diff --git a/src/Processors/QueryPlan/ExpressionStep.cpp b/src/Processors/QueryPlan/ExpressionStep.cpp index 7e35fcb79f5..87ebfa2526d 100644 --- a/src/Processors/QueryPlan/ExpressionStep.cpp +++ b/src/Processors/QueryPlan/ExpressionStep.cpp @@ -44,7 +44,7 @@ static ITransformingStep::Traits getJoinTraits() ExpressionStep::ExpressionStep(const DataStream & input_stream_, ActionsDAGPtr actions_) : ITransformingStep( input_stream_, - Transform::transformHeader(input_stream_.header, std::make_shared(actions_)), + Transform::transformHeader(input_stream_.header, *actions_), getTraits(actions_)) , actions(std::move(actions_)) { @@ -55,7 +55,7 @@ ExpressionStep::ExpressionStep(const DataStream & input_stream_, ActionsDAGPtr a void ExpressionStep::updateInputStream(DataStream input_stream, bool keep_header) { Block out_header = keep_header ? std::move(output_stream->header) - : Transform::transformHeader(input_stream.header, std::make_shared(actions)); + : Transform::transformHeader(input_stream.header, *actions); output_stream = createOutputStream( input_stream, std::move(out_header), diff --git a/src/Processors/QueryPlan/FilterStep.cpp b/src/Processors/QueryPlan/FilterStep.cpp index 5a77d01017b..55dc1812ab4 100644 --- a/src/Processors/QueryPlan/FilterStep.cpp +++ b/src/Processors/QueryPlan/FilterStep.cpp @@ -31,7 +31,7 @@ FilterStep::FilterStep( bool remove_filter_column_) : ITransformingStep( input_stream_, - FilterTransform::transformHeader(input_stream_.header, std::make_shared(actions_), filter_column_name_, remove_filter_column_), + FilterTransform::transformHeader(input_stream_.header, *actions_, filter_column_name_, remove_filter_column_), getTraits(actions_)) , actions(std::move(actions_)) , filter_column_name(std::move(filter_column_name_)) diff --git a/src/Processors/QueryPlan/TotalsHavingStep.cpp b/src/Processors/QueryPlan/TotalsHavingStep.cpp index 07c834ec27e..3fe2f906d02 100644 --- a/src/Processors/QueryPlan/TotalsHavingStep.cpp +++ b/src/Processors/QueryPlan/TotalsHavingStep.cpp @@ -34,10 +34,7 @@ TotalsHavingStep::TotalsHavingStep( bool final_) : ITransformingStep( input_stream_, - TotalsHavingTransform::transformHeader( - input_stream_.header, - (actions_ ? std::make_shared(actions_) : nullptr), - final_), + TotalsHavingTransform::transformHeader(input_stream_.header, actions_.get(), final_), getTraits(!filter_column_.empty())) , overflow_row(overflow_row_) , actions(actions_) @@ -80,14 +77,17 @@ void TotalsHavingStep::describeActions(FormatSettings & settings) const settings.out << prefix << "Filter column: " << filter_column_name << '\n'; settings.out << prefix << "Mode: " << totalsModeToString(totals_mode, auto_include_threshold) << '\n'; - bool first = true; - auto expression = std::make_shared(actions); - for (const auto & action : expression->getActions()) + if (actions) { - settings.out << prefix << (first ? "Actions: " - : " "); - first = false; - settings.out << action.toString() << '\n'; + bool first = true; + auto expression = std::make_shared(actions); + for (const auto & action : expression->getActions()) + { + settings.out << prefix << (first ? "Actions: " + : " "); + first = false; + settings.out << action.toString() << '\n'; + } } } diff --git a/src/Processors/Transforms/ExpressionTransform.cpp b/src/Processors/Transforms/ExpressionTransform.cpp index e60fc65e96e..efcdaa888bb 100644 --- a/src/Processors/Transforms/ExpressionTransform.cpp +++ b/src/Processors/Transforms/ExpressionTransform.cpp @@ -3,16 +3,15 @@ namespace DB { -Block ExpressionTransform::transformHeader(Block header, const ExpressionActionsPtr & expression) +Block ExpressionTransform::transformHeader(Block header, const ActionsDAG & expression) { - size_t num_rows = header.rows(); - expression->execute(header, num_rows, true); + expression.updateHeader(header); return header; } ExpressionTransform::ExpressionTransform(const Block & header_, ExpressionActionsPtr expression_) - : ISimpleTransform(header_, transformHeader(header_, expression_), false) + : ISimpleTransform(header_, transformHeader(header_, expression_->getActionsDAG()), false) , expression(std::move(expression_)) { } diff --git a/src/Processors/Transforms/ExpressionTransform.h b/src/Processors/Transforms/ExpressionTransform.h index 525273bad81..d4cc1c8f78a 100644 --- a/src/Processors/Transforms/ExpressionTransform.h +++ b/src/Processors/Transforms/ExpressionTransform.h @@ -7,6 +7,8 @@ namespace DB class ExpressionActions; using ExpressionActionsPtr = std::shared_ptr; +class ActionsDAG; + /** Executes a certain expression over the block. * The expression consists of column identifiers from the block, constants, common functions. * For example: hits * 2 + 3, url LIKE '%yandex%' @@ -21,7 +23,7 @@ public: String getName() const override { return "ExpressionTransform"; } - static Block transformHeader(Block header, const ExpressionActionsPtr & expression); + static Block transformHeader(Block header, const ActionsDAG & expression); protected: void transform(Chunk & chunk) override; diff --git a/src/Processors/Transforms/FilterTransform.cpp b/src/Processors/Transforms/FilterTransform.cpp index 23b1bf8a984..734a4995ceb 100644 --- a/src/Processors/Transforms/FilterTransform.cpp +++ b/src/Processors/Transforms/FilterTransform.cpp @@ -29,12 +29,11 @@ static void replaceFilterToConstant(Block & block, const String & filter_column_ Block FilterTransform::transformHeader( Block header, - const ExpressionActionsPtr & expression, + const ActionsDAG & expression, const String & filter_column_name, bool remove_filter_column) { - size_t num_rows = header.rows(); - expression->execute(header, num_rows); + expression.updateHeader(header); if (remove_filter_column) header.erase(filter_column_name); @@ -50,7 +49,7 @@ FilterTransform::FilterTransform( String filter_column_name_, bool remove_filter_column_, bool on_totals_) - : ISimpleTransform(header_, transformHeader(header_, expression_, filter_column_name_, remove_filter_column_), true) + : ISimpleTransform(header_, transformHeader(header_, expression_->getActionsDAG(), filter_column_name_, remove_filter_column_), true) , expression(std::move(expression_)) , filter_column_name(std::move(filter_column_name_)) , remove_filter_column(remove_filter_column_) diff --git a/src/Processors/Transforms/FilterTransform.h b/src/Processors/Transforms/FilterTransform.h index c0ccf0fd072..39f1f1c42db 100644 --- a/src/Processors/Transforms/FilterTransform.h +++ b/src/Processors/Transforms/FilterTransform.h @@ -8,6 +8,8 @@ namespace DB class ExpressionActions; using ExpressionActionsPtr = std::shared_ptr; +class ActionsDAG; + /** Implements WHERE, HAVING operations. * Takes an expression, which adds to the block one ColumnUInt8 column containing the filtering conditions. * The expression is evaluated and result chunks contain only the filtered rows. @@ -22,7 +24,7 @@ public: static Block transformHeader( Block header, - const ExpressionActionsPtr & expression, + const ActionsDAG & expression, const String & filter_column_name, bool remove_filter_column); diff --git a/src/Processors/Transforms/TotalsHavingTransform.cpp b/src/Processors/Transforms/TotalsHavingTransform.cpp index c81c389b6fa..dad4002eb69 100644 --- a/src/Processors/Transforms/TotalsHavingTransform.cpp +++ b/src/Processors/Transforms/TotalsHavingTransform.cpp @@ -28,15 +28,13 @@ void finalizeChunk(Chunk & chunk) chunk.setColumns(std::move(columns), num_rows); } -Block TotalsHavingTransform::transformHeader(Block block, const ExpressionActionsPtr & expression, bool final) +Block TotalsHavingTransform::transformHeader(Block block, const ActionsDAG * expression, bool final) { if (final) finalizeBlock(block); - size_t num_rows = block.rows(); - if (expression) - expression->execute(block, num_rows); + expression->updateHeader(block); return block; } @@ -49,7 +47,10 @@ TotalsHavingTransform::TotalsHavingTransform( TotalsMode totals_mode_, double auto_include_threshold_, bool final_) - : ISimpleTransform(header, transformHeader(header, expression_, final_), true) + : ISimpleTransform( + header, + transformHeader(header, expression_ ? &expression_->getActionsDAG() : nullptr, final_), + true) , overflow_row(overflow_row_) , expression(expression_) , filter_column_name(filter_column_) diff --git a/src/Processors/Transforms/TotalsHavingTransform.h b/src/Processors/Transforms/TotalsHavingTransform.h index 8191ec763b0..5809f382e0e 100644 --- a/src/Processors/Transforms/TotalsHavingTransform.h +++ b/src/Processors/Transforms/TotalsHavingTransform.h @@ -12,6 +12,8 @@ using ArenaPtr = std::shared_ptr; class ExpressionActions; using ExpressionActionsPtr = std::shared_ptr; +class ActionsDAG; + enum class TotalsMode; /** Takes blocks after grouping, with non-finalized aggregate functions. @@ -37,7 +39,7 @@ public: Status prepare() override; void work() override; - static Block transformHeader(Block block, const ExpressionActionsPtr & expression, bool final); + static Block transformHeader(Block block, const ActionsDAG * expression, bool final); protected: void transform(Chunk & chunk) override; From d72c7dfacb7c92c0374e9ff25e4446357ce2bd2a Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Wed, 11 Nov 2020 17:07:51 +0300 Subject: [PATCH 140/205] Add more comments. --- src/Interpreters/ExpressionActions.h | 11 ++++++++++- src/Interpreters/ExpressionAnalyzer.cpp | 2 +- 2 files changed, 11 insertions(+), 2 deletions(-) diff --git a/src/Interpreters/ExpressionActions.h b/src/Interpreters/ExpressionActions.h index d701633cdf8..c06a09d0063 100644 --- a/src/Interpreters/ExpressionActions.h +++ b/src/Interpreters/ExpressionActions.h @@ -111,7 +111,16 @@ public: bool allow_constant_folding = true; }; - /// Index is a list of nodes + hash table: name -> list_elem. + /// Index is used to: + /// * find Node buy it's result_name + /// * specify order of columns in result + /// It represents a set of available columns. + /// Removing of column from index is equivalent to removing of column from final result. + /// + /// DAG allows actions with duplicating result names. In this case index will point to last added Node. + /// It does not cause any problems as long as execution of actions does not depend on action names anymore. + /// + /// Index is a list of nodes + [map: name -> list::iterator]. /// List is ordered, may contain nodes with same names, or one node several times. class Index { diff --git a/src/Interpreters/ExpressionAnalyzer.cpp b/src/Interpreters/ExpressionAnalyzer.cpp index d706199f94e..c6b16d77a38 100644 --- a/src/Interpreters/ExpressionAnalyzer.cpp +++ b/src/Interpreters/ExpressionAnalyzer.cpp @@ -673,7 +673,7 @@ ActionsDAGPtr SelectQueryExpressionAnalyzer::appendPrewhere( auto tmp_actions_dag = std::make_shared(sourceColumns()); getRootActions(select_query->prewhere(), only_types, tmp_actions_dag); tmp_actions_dag->removeUnusedActions({prewhere_column_name}); - auto required_columns = tmp_actions_dag->getRequiredColumns(); + auto required_columns = tmp_actions_dag->getRequiredColumns().getNames(); NameSet required_source_columns(required_columns.begin(), required_columns.end()); /// Add required columns to required output in order not to remove them after prewhere execution. From 4abcd1bcb3366623c78f273faa3e350782665977 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Wed, 11 Nov 2020 17:26:18 +0300 Subject: [PATCH 141/205] Add more comments. --- src/Interpreters/ExpressionActions.cpp | 11 ++++++++++- src/Interpreters/ExpressionActions.h | 2 +- src/Processors/QueryPlan/FilterStep.cpp | 2 +- 3 files changed, 12 insertions(+), 3 deletions(-) diff --git a/src/Interpreters/ExpressionActions.cpp b/src/Interpreters/ExpressionActions.cpp index 430e93edb31..5cf235224b7 100644 --- a/src/Interpreters/ExpressionActions.cpp +++ b/src/Interpreters/ExpressionActions.cpp @@ -280,6 +280,15 @@ void ExpressionActions::checkLimits(const ColumnsWithTypeAndName & columns) cons namespace { + /// This struct stores context needed to execute actions. + /// + /// Execution model is following: + /// * execution is performed over list of columns (with fixed size = ExpressionActions::num_columns) + /// * every argument has fixed position in columns list, every action has fixed position for result + /// * if argument is not needed anymore (Argument::needed_later == false), it is removed from list + /// * argument for INPUT is in inputs[inputs_pos[argument.pos]] + /// + /// Columns on positions `ExpressionActions::result_positions` are inserted back into block. struct ExecutionContext { ColumnsWithTypeAndName & inputs; @@ -1145,7 +1154,7 @@ NamesAndTypesList ActionsDAG::getRequiredColumns() const NamesAndTypesList result; for (const auto & node : nodes) if (node.type == ActionType::INPUT) - result.push_back({node.result_name, node.result_type}); + result.emplace_back(node.result_name, node.result_type); return result; } diff --git a/src/Interpreters/ExpressionActions.h b/src/Interpreters/ExpressionActions.h index c06a09d0063..e3702200070 100644 --- a/src/Interpreters/ExpressionActions.h +++ b/src/Interpreters/ExpressionActions.h @@ -235,7 +235,7 @@ public: /// Call addAlias several times. void addAliases(const NamesWithAliases & aliases); - /// Adds alias actions and removes unused columns from index. + /// Add alias actions and remove unused columns from index. Also specify result columns order in index. void project(const NamesWithAliases & projection); /// Removes column from index. diff --git a/src/Processors/QueryPlan/FilterStep.cpp b/src/Processors/QueryPlan/FilterStep.cpp index 55dc1812ab4..6f69a00fb27 100644 --- a/src/Processors/QueryPlan/FilterStep.cpp +++ b/src/Processors/QueryPlan/FilterStep.cpp @@ -45,7 +45,7 @@ void FilterStep::updateInputStream(DataStream input_stream, bool keep_header) { Block out_header = std::move(output_stream->header); if (keep_header) - out_header = FilterTransform::transformHeader(input_stream.header, std::make_shared(actions), filter_column_name, remove_filter_column); + out_header = FilterTransform::transformHeader(input_stream.header, *actions, filter_column_name, remove_filter_column); output_stream = createOutputStream( input_stream, From 5a38e80b00c0fe117aa60f2a967f8eea98fc2546 Mon Sep 17 00:00:00 2001 From: sundy-li <543950155@qq.com> Date: Wed, 11 Nov 2020 22:53:42 +0800 Subject: [PATCH 142/205] disable JeMalloc in rocksdb contrib --- contrib/rocksdb-cmake/CMakeLists.txt | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/contrib/rocksdb-cmake/CMakeLists.txt b/contrib/rocksdb-cmake/CMakeLists.txt index dc5d1acc819..07c6861b406 100644 --- a/contrib/rocksdb-cmake/CMakeLists.txt +++ b/contrib/rocksdb-cmake/CMakeLists.txt @@ -18,8 +18,11 @@ endif() set(PORTABLE ON) -option(WITH_JEMALLOC "build with JeMalloc" ${ENABLE_JEMALLOC}) +## always disable jemalloc for rocksdb by default +## because it introduces non-standard jemalloc APIs +option(WITH_JEMALLOC "build with JeMalloc" OFF) option(WITH_SNAPPY "build with SNAPPY" ${USE_SNAPPY}) +## lz4, zlib, zstd is enabled in ClickHouse by default option(WITH_LZ4 "build with lz4" ON) option(WITH_ZLIB "build with zlib" ON) option(WITH_ZSTD "build with zstd" ON) From c7f93140b47976996b0c0273f6ed2ea59d239c72 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Wed, 11 Nov 2020 17:56:56 +0300 Subject: [PATCH 143/205] Add more comments. --- src/Interpreters/ExpressionActions.cpp | 14 ++++++++++---- src/Interpreters/ExpressionJIT.h | 4 ++++ 2 files changed, 14 insertions(+), 4 deletions(-) diff --git a/src/Interpreters/ExpressionActions.cpp b/src/Interpreters/ExpressionActions.cpp index 5cf235224b7..71b31a57f17 100644 --- a/src/Interpreters/ExpressionActions.cpp +++ b/src/Interpreters/ExpressionActions.cpp @@ -78,6 +78,9 @@ ExpressionActionsPtr ExpressionActions::clone() const void ExpressionActions::linearizeActions() { + /// This function does the topological sort or DAG and fills all the fields of ExpressionActions. + /// Algorithm traverses DAG starting from nodes without children. + /// For every node we support the number of created children, and if all children are created, put node into queue. struct Data { const Node * node = nullptr; @@ -102,6 +105,8 @@ void ExpressionActions::linearizeActions() reverse_index[&node] = id; } + /// There are independent queues for arrayJoin and other actions. + /// We delay creation of arrayJoin as long as we can, so that they will be executed closer to end. std::queue ready_nodes; std::queue ready_array_joins; @@ -120,6 +125,8 @@ void ExpressionActions::linearizeActions() ready_nodes.emplace(&node); } + /// Every argument will have fixed position in columns list. + /// If argument is removed, it's position may be reused by other action. std::stack free_positions; while (!ready_nodes.empty() || !ready_array_joins.empty()) @@ -128,12 +135,9 @@ void ExpressionActions::linearizeActions() const Node * node = stack.front(); stack.pop(); - Names argument_names; - for (const auto & child : node->children) - argument_names.emplace_back(child->result_name); - auto & cur = data[reverse_index[node]]; + /// Select position for action result. size_t free_position = num_columns; if (free_positions.empty()) ++num_columns; @@ -396,6 +400,8 @@ static void executeAction(const ExpressionActions::Action & action, ExecutionCon auto pos = execution_context.inputs_pos[action.arguments.front().pos]; if (pos < 0) { + /// Here we allow to skip input if it is not in block (in case it is not needed). + /// It may be unusual, but some code depend on such behaviour. if (action.arguments.front().needed_later) throw Exception(ErrorCodes::NOT_FOUND_COLUMN_IN_BLOCK, "Not found column {} in block", diff --git a/src/Interpreters/ExpressionJIT.h b/src/Interpreters/ExpressionJIT.h index 3c885aca72e..023599f98f0 100644 --- a/src/Interpreters/ExpressionJIT.h +++ b/src/Interpreters/ExpressionJIT.h @@ -31,6 +31,8 @@ class LLVMFunction : public IFunctionBaseImpl public: + /// LLVMFunction is a compiled part of ActionsDAG. + /// We store this part as independent DAG with minial required information to compile it. struct CompileNode { enum class NodeType @@ -51,6 +53,8 @@ public: std::vector arguments; }; + /// DAG is represented as list of nodes stored in in-order traverse order. + /// Expression (a + 1) + (b + 1) will be represented like chain: a, 1, a + 1, b, b + 1, (a + 1) + (b + 1). struct CompileDAG : public std::vector { std::string dump() const; From 134786afd525d7bac39a3dd097b2b0992d38f2e2 Mon Sep 17 00:00:00 2001 From: Amos Bird Date: Wed, 11 Nov 2020 22:34:58 +0800 Subject: [PATCH 144/205] Disable ATTACH/DETACH TABLE --- src/Interpreters/InterpreterCreateQuery.cpp | 4 +++ src/Interpreters/InterpreterDropQuery.cpp | 1 + src/Storages/IStorage.h | 2 ++ src/Storages/StorageDictionary.cpp | 5 ++++ src/Storages/StorageDictionary.h | 1 + ...sable_detach_table_of_dictionary.reference | 0 ...575_disable_detach_table_of_dictionary.sql | 26 +++++++++++++++++++ 7 files changed, 39 insertions(+) create mode 100644 tests/queries/0_stateless/01575_disable_detach_table_of_dictionary.reference create mode 100644 tests/queries/0_stateless/01575_disable_detach_table_of_dictionary.sql diff --git a/src/Interpreters/InterpreterCreateQuery.cpp b/src/Interpreters/InterpreterCreateQuery.cpp index 286d5269a64..8e6e6170f06 100644 --- a/src/Interpreters/InterpreterCreateQuery.cpp +++ b/src/Interpreters/InterpreterCreateQuery.cpp @@ -684,6 +684,10 @@ BlockIO InterpreterCreateQuery::createTable(ASTCreateQuery & create) // Table SQL definition is available even if the table is detached auto query = database->getCreateTableQuery(create.table, context); create = query->as(); // Copy the saved create query, but use ATTACH instead of CREATE + if (create.is_dictionary) + throw Exception( + "Cannot ATTACH TABLE " + backQuoteIfNeed(database_name) + "." + backQuoteIfNeed(create.table) + ", it is a Dictionary", + ErrorCodes::INCORRECT_QUERY); create.attach = true; create.attach_short_syntax = true; create.if_not_exists = if_not_exists; diff --git a/src/Interpreters/InterpreterDropQuery.cpp b/src/Interpreters/InterpreterDropQuery.cpp index 144e045ecee..5f7f70fdda4 100644 --- a/src/Interpreters/InterpreterDropQuery.cpp +++ b/src/Interpreters/InterpreterDropQuery.cpp @@ -119,6 +119,7 @@ BlockIO InterpreterDropQuery::executeToTableImpl(const ASTDropQuery & query, Dat if (query.kind == ASTDropQuery::Kind::Detach) { context.checkAccess(table->isView() ? AccessType::DROP_VIEW : AccessType::DROP_TABLE, table_id); + table->checkTableCanBeDetached(); table->shutdown(); TableExclusiveLockHolder table_lock; if (database->getEngineName() != "Atomic") diff --git a/src/Storages/IStorage.h b/src/Storages/IStorage.h index 5fa2a915978..7c27282e4f3 100644 --- a/src/Storages/IStorage.h +++ b/src/Storages/IStorage.h @@ -443,6 +443,8 @@ public: /// Otherwise - throws an exception with detailed information. /// We do not use mutex because it is not very important that the size could change during the operation. virtual void checkTableCanBeDropped() const {} + /// Similar to above but checks for DETACH. It's only used for DICTIONARIES. + virtual void checkTableCanBeDetached() const {} /// Checks that Partition could be dropped right now /// Otherwise - throws an exception with detailed information. diff --git a/src/Storages/StorageDictionary.cpp b/src/Storages/StorageDictionary.cpp index ee08dd5a824..70c3ce8094e 100644 --- a/src/Storages/StorageDictionary.cpp +++ b/src/Storages/StorageDictionary.cpp @@ -130,6 +130,11 @@ void StorageDictionary::checkTableCanBeDropped() const throw Exception("Cannot detach table " + getStorageID().getFullTableName() + " from a database with DICTIONARY engine", ErrorCodes::CANNOT_DETACH_DICTIONARY_AS_TABLE); } +void StorageDictionary::checkTableCanBeDetached() const +{ + checkTableCanBeDropped(); +} + Pipe StorageDictionary::read( const Names & column_names, const StorageMetadataPtr & /*metadata_snapshot*/, diff --git a/src/Storages/StorageDictionary.h b/src/Storages/StorageDictionary.h index 576cc2de064..589ff7d4654 100644 --- a/src/Storages/StorageDictionary.h +++ b/src/Storages/StorageDictionary.h @@ -15,6 +15,7 @@ public: std::string getName() const override { return "Dictionary"; } void checkTableCanBeDropped() const override; + void checkTableCanBeDetached() const override; Pipe read( const Names & column_names, diff --git a/tests/queries/0_stateless/01575_disable_detach_table_of_dictionary.reference b/tests/queries/0_stateless/01575_disable_detach_table_of_dictionary.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/01575_disable_detach_table_of_dictionary.sql b/tests/queries/0_stateless/01575_disable_detach_table_of_dictionary.sql new file mode 100644 index 00000000000..44c0c149e3c --- /dev/null +++ b/tests/queries/0_stateless/01575_disable_detach_table_of_dictionary.sql @@ -0,0 +1,26 @@ +DROP DATABASE IF EXISTS database_for_dict; + +CREATE DATABASE database_for_dict; + +CREATE TABLE database_for_dict.table_for_dict (k UInt64, v UInt8) ENGINE = MergeTree ORDER BY k; + +DROP DICTIONARY IF EXISTS database_for_dict.dict1; + +CREATE DICTIONARY database_for_dict.dict1 (k UInt64 DEFAULT 0, v UInt8 DEFAULT 1) PRIMARY KEY k +SOURCE(CLICKHOUSE(HOST 'localhost' PORT 9000 USER 'default' TABLE 'table_for_dict' PASSWORD '' DB 'database_for_dict')) +LIFETIME(MIN 1 MAX 10) +LAYOUT(FLAT()); + +DETACH TABLE database_for_dict.dict1; -- { serverError 520 } + +DETACH DICTIONARY database_for_dict.dict1; + +ATTACH TABLE database_for_dict.dict1; -- { serverError 80 } + +ATTACH DICTIONARY database_for_dict.dict1; + +DROP DICTIONARY database_for_dict.dict1; + +DROP TABLE database_for_dict.table_for_dict; + +DROP DATABASE IF EXISTS database_for_dict; From 4618df5915e9bc51305943f51f651917e3a20775 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Wed, 11 Nov 2020 18:16:46 +0300 Subject: [PATCH 145/205] Try fix tests. --- src/Interpreters/ExpressionActions.cpp | 7 +++++++ 1 file changed, 7 insertions(+) diff --git a/src/Interpreters/ExpressionActions.cpp b/src/Interpreters/ExpressionActions.cpp index 71b31a57f17..46d1f9cbb93 100644 --- a/src/Interpreters/ExpressionActions.cpp +++ b/src/Interpreters/ExpressionActions.cpp @@ -494,7 +494,14 @@ void ActionsDAG::updateHeader(Block & block) const } for (const auto & node : index) + { + ColumnPtr col; + + if (node->column && isColumnConst(*node->column)) + col = node->column->cloneResized(0); + block.insert({node->column, node->result_type, node->result_name}); + } } Names ExpressionActions::getRequiredColumns() const From aefbc2b48b7139df79707e8467b4f1d2dbd322c4 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Wed, 11 Nov 2020 18:19:27 +0300 Subject: [PATCH 146/205] Try fix tests. --- src/Interpreters/ExpressionActions.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Interpreters/ExpressionActions.cpp b/src/Interpreters/ExpressionActions.cpp index 46d1f9cbb93..c987bda5cc1 100644 --- a/src/Interpreters/ExpressionActions.cpp +++ b/src/Interpreters/ExpressionActions.cpp @@ -500,7 +500,7 @@ void ActionsDAG::updateHeader(Block & block) const if (node->column && isColumnConst(*node->column)) col = node->column->cloneResized(0); - block.insert({node->column, node->result_type, node->result_name}); + block.insert({col, node->result_type, node->result_name}); } } From 02770ae216731e9562ab0de74b4453aac66e805f Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Wed, 11 Nov 2020 19:05:00 +0300 Subject: [PATCH 147/205] Remove comment. --- src/Interpreters/ExpressionJIT.h | 4 ---- 1 file changed, 4 deletions(-) diff --git a/src/Interpreters/ExpressionJIT.h b/src/Interpreters/ExpressionJIT.h index 023599f98f0..30e6e03854d 100644 --- a/src/Interpreters/ExpressionJIT.h +++ b/src/Interpreters/ExpressionJIT.h @@ -100,10 +100,6 @@ public: using Base::Base; }; -/// For each APPLY_FUNCTION action, try to compile the function to native code; if the only uses of a compilable -/// function's result are as arguments to other compilable functions, inline it and leave the now-redundant action as-is. -// void compileFunctions(ExpressionActions::Actions & actions, const Names & output_columns, const Block & sample_block, std::shared_ptr compilation_cache, size_t min_count_to_compile_expression); - } #endif From a8ddb27c00c26c2120671d11d4102a5f5bb43dd7 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Wed, 11 Nov 2020 19:08:48 +0300 Subject: [PATCH 148/205] Remove comment. --- src/Interpreters/ExpressionActions.cpp | 2 ++ 1 file changed, 2 insertions(+) diff --git a/src/Interpreters/ExpressionActions.cpp b/src/Interpreters/ExpressionActions.cpp index c987bda5cc1..5fb6f68fa2d 100644 --- a/src/Interpreters/ExpressionActions.cpp +++ b/src/Interpreters/ExpressionActions.cpp @@ -499,6 +499,8 @@ void ActionsDAG::updateHeader(Block & block) const if (node->column && isColumnConst(*node->column)) col = node->column->cloneResized(0); + else + col = node->result_type->createColumn(); block.insert({col, node->result_type, node->result_name}); } From c01870076dfb16bd9f1e0e88941c1c8a54f05b43 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Wed, 11 Nov 2020 19:09:01 +0300 Subject: [PATCH 149/205] Update MergeTreeData.h --- src/Storages/MergeTree/MergeTreeData.h | 2 ++ 1 file changed, 2 insertions(+) diff --git a/src/Storages/MergeTree/MergeTreeData.h b/src/Storages/MergeTree/MergeTreeData.h index a2c8aa55e99..c5cd21ef597 100644 --- a/src/Storages/MergeTree/MergeTreeData.h +++ b/src/Storages/MergeTree/MergeTreeData.h @@ -880,6 +880,8 @@ protected: virtual PartitionCommandsResultInfo attachPartition(const ASTPtr & partition, const StorageMetadataPtr & metadata_snapshot, bool part, const Context & context) = 0; virtual void replacePartitionFrom(const StoragePtr & source_table, const ASTPtr & partition, bool replace, const Context & context) = 0; virtual void movePartitionToTable(const StoragePtr & dest_table, const ASTPtr & partition, const Context & context) = 0; + + /// Makes sense only for replicated tables virtual void fetchPartition(const ASTPtr & partition, const StorageMetadataPtr & metadata_snapshot, const String & from, const Context & query_context); void writePartLog( From 2497598a49c508cbcc39f74ead7f41877f6c0838 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Wed, 11 Nov 2020 19:18:21 +0300 Subject: [PATCH 150/205] Remove empty parts after they were pruned by TTL --- src/Storages/MergeTree/MergeTreeData.cpp | 13 ++++ src/Storages/MergeTree/MergeTreeData.h | 2 + .../ReplicatedMergeTreeCleanupThread.cpp | 1 + src/Storages/StorageMergeTree.cpp | 2 + tests/integration/test_ttl_replicated/test.py | 62 ++++++++++++++++--- .../01560_ttl_remove_empty_parts.reference | 4 ++ .../01560_ttl_remove_empty_parts.sql | 20 ++++++ 7 files changed, 94 insertions(+), 10 deletions(-) create mode 100644 tests/queries/0_stateless/01560_ttl_remove_empty_parts.reference create mode 100644 tests/queries/0_stateless/01560_ttl_remove_empty_parts.sql diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 441c95fec4e..641f11d2dec 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -1215,6 +1215,19 @@ void MergeTreeData::clearOldWriteAheadLogs() } } +void MergeTreeData::clearEmptyParts() +{ + auto parts = getDataPartsVector(); + for (const auto & part : parts) + { + if (part->rows_count == 0) + { + ASTPtr literal = std::make_shared(part->name); + dropPartition(literal, /* detach = */ false, /*drop_part = */ true, global_context); + } + } +} + void MergeTreeData::rename(const String & new_table_path, const StorageID & new_table_id) { auto disks = getStoragePolicy()->getDisks(); diff --git a/src/Storages/MergeTree/MergeTreeData.h b/src/Storages/MergeTree/MergeTreeData.h index a2c8aa55e99..55967777b8a 100644 --- a/src/Storages/MergeTree/MergeTreeData.h +++ b/src/Storages/MergeTree/MergeTreeData.h @@ -498,6 +498,8 @@ public: /// Must be called with locked lockForShare() because use relative_data_path. void clearOldTemporaryDirectories(ssize_t custom_directories_lifetime_seconds = -1); + void clearEmptyParts(); + /// After the call to dropAllData() no method can be called. /// Deletes the data directory and flushes the uncompressed blocks cache and the marks cache. void dropAllData(); diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeCleanupThread.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeCleanupThread.cpp index 5de0a79b2c3..701cb2fa1ed 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeCleanupThread.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeCleanupThread.cpp @@ -72,6 +72,7 @@ void ReplicatedMergeTreeCleanupThread::iterate() clearOldLogs(); clearOldBlocks(); clearOldMutations(); + storage.clearEmptyParts(); } } diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index fa4698af637..02b7c7b7e42 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -99,6 +99,7 @@ void StorageMergeTree::startup() { clearOldPartsFromFilesystem(); clearOldWriteAheadLogs(); + clearEmptyParts(); /// Temporary directories contain incomplete results of merges (after forced restart) /// and don't allow to reinitialize them, so delete each of them immediately @@ -933,6 +934,7 @@ std::optional StorageMergeTree::getDataProcessingJob() clearOldTemporaryDirectories(); clearOldWriteAheadLogs(); clearOldMutations(); + clearEmptyParts(); }, PoolType::MERGE_MUTATE}; } return {}; diff --git a/tests/integration/test_ttl_replicated/test.py b/tests/integration/test_ttl_replicated/test.py index cbf13c20220..2cb384d10a0 100644 --- a/tests/integration/test_ttl_replicated/test.py +++ b/tests/integration/test_ttl_replicated/test.py @@ -213,6 +213,14 @@ def test_ttl_double_delete_rule_returns_error(started_cluster): assert False +def optimize_with_retry(node, table_name, retry=20): + for i in range(retry): + try: + node.query("OPTIMIZE TABLE {name} FINAL SETTINGS optimize_throw_if_noop = 1".format(name=table_name), settings={"optimize_throw_if_noop": "1"}) + break + except e: + time.sleep(0.5) + @pytest.mark.parametrize("name,engine", [ ("test_ttl_alter_delete", "MergeTree()"), ("test_replicated_ttl_alter_delete", "ReplicatedMergeTree('/clickhouse/test_replicated_ttl_alter_delete', '1')"), @@ -238,14 +246,6 @@ limitations under the License.""" """ drop_table([node1], name) - def optimize_with_retry(retry=20): - for i in range(retry): - try: - node1.query("OPTIMIZE TABLE {name} FINAL".format(name=name), settings={"optimize_throw_if_noop": "1"}) - break - except: - time.sleep(0.5) - node1.query( """ CREATE TABLE {name} ( @@ -267,7 +267,7 @@ limitations under the License.""" time.sleep(1) - optimize_with_retry() + optimize_with_retry(node1, name) r = node1.query("SELECT s1, b1 FROM {name} ORDER BY b1, s1".format(name=name)).splitlines() assert r == ["\t1", "hello2\t2"] @@ -277,7 +277,49 @@ limitations under the License.""" time.sleep(1) - optimize_with_retry() + optimize_with_retry(node1, name) r = node1.query("SELECT s1, b1 FROM {name} ORDER BY b1, s1".format(name=name)).splitlines() assert r == ["\t0", "\t0", "hello2\t2"] + +def test_ttl_empty_parts(started_cluster): + drop_table([node1, node2], "test_ttl_empty_parts") + for node in [node1, node2]: + node.query( + ''' + CREATE TABLE test_ttl_empty_parts(date Date, id UInt32) + ENGINE = ReplicatedMergeTree('/clickhouse/tables/test/test_ttl', '{replica}') + ORDER BY id + SETTINGS max_bytes_to_merge_at_min_space_in_pool = 1, max_bytes_to_merge_at_max_space_in_pool = 1, + cleanup_delay_period = 1, cleanup_delay_period_random_add = 0 + '''.format(replica=node.name)) + + for i in range (1, 7): + node1.query("INSERT INTO test_ttl_empty_parts SELECT '2{}00-01-0{}', number FROM numbers(1000)".format(i % 2, i)) + + assert node1.query("SELECT count() FROM test_ttl_empty_parts") == "6000\n" + assert node1.query("SELECT name FROM system.parts WHERE table = 'test_ttl_empty_parts' AND active ORDER BY name") == \ + "all_0_0_0\nall_1_1_0\nall_2_2_0\nall_3_3_0\nall_4_4_0\nall_5_5_0\n" + + node1.query("ALTER TABLE test_ttl_empty_parts MODIFY TTL date") + + assert node1.query("SELECT count() FROM test_ttl_empty_parts") == "3000\n" + + time.sleep(3) # Wait for cleanup thread + assert node1.query("SELECT name FROM system.parts WHERE table = 'test_ttl_empty_parts' AND active ORDER BY name") == \ + "all_0_0_0_6\nall_2_2_0_6\nall_4_4_0_6\n" + + for node in [node1, node2]: + node.query("ALTER TABLE test_ttl_empty_parts MODIFY SETTING max_bytes_to_merge_at_min_space_in_pool = 1000000000") + node.query("ALTER TABLE test_ttl_empty_parts MODIFY SETTING max_bytes_to_merge_at_max_space_in_pool = 1000000000") + + optimize_with_retry(node1, 'test_ttl_empty_parts') + assert node1.query("SELECT name FROM system.parts WHERE table = 'test_ttl_empty_parts' AND active ORDER BY name") == "all_0_4_1_6\n" + + # Check that after removing empty parts mutations and merges works + node1.query("INSERT INTO test_ttl_empty_parts SELECT '2100-01-20', number FROM numbers(1000)") + node1.query("ALTER TABLE test_ttl_empty_parts DELETE WHERE id % 2 = 0 SETTINGS mutations_sync = 2") + assert node1.query("SELECT count() FROM test_ttl_empty_parts") == "2000\n" + + optimize_with_retry(node1, 'test_ttl_empty_parts') + assert node1.query("SELECT name FROM system.parts WHERE table = 'test_ttl_empty_parts' AND active ORDER BY name") == "all_0_7_2_8\n" diff --git a/tests/queries/0_stateless/01560_ttl_remove_empty_parts.reference b/tests/queries/0_stateless/01560_ttl_remove_empty_parts.reference new file mode 100644 index 00000000000..526c8b842be --- /dev/null +++ b/tests/queries/0_stateless/01560_ttl_remove_empty_parts.reference @@ -0,0 +1,4 @@ +1000 +2 +500 +1 diff --git a/tests/queries/0_stateless/01560_ttl_remove_empty_parts.sql b/tests/queries/0_stateless/01560_ttl_remove_empty_parts.sql new file mode 100644 index 00000000000..f40ed70caef --- /dev/null +++ b/tests/queries/0_stateless/01560_ttl_remove_empty_parts.sql @@ -0,0 +1,20 @@ +DROP TABLE IF EXISTS ttl_empty_parts; + +CREATE TABLE ttl_empty_parts (id UInt32, d Date) ENGINE = MergeTree ORDER BY tuple() PARTITION BY id; + +INSERT INTO ttl_empty_parts SELECT 0, toDate('2005-01-01') + number from numbers(500); +INSERT INTO ttl_empty_parts SELECT 1, toDate('2050-01-01') + number from numbers(500); + +SELECT count() FROM ttl_empty_parts; +SELECT count() FROM system.parts WHERE table = 'ttl_empty_parts' AND database = currentDatabase() AND active; + +ALTER TABLE ttl_empty_parts MODIFY TTL d; + +-- To be sure, that task, which clears outdated parts executed. +DETACH TABLE ttl_empty_parts; +ATTACH TABLE ttl_empty_parts; + +SELECT count() FROM ttl_empty_parts; +SELECT count() FROM system.parts WHERE table = 'ttl_empty_parts' AND database = currentDatabase() AND active; + +DROP TABLE ttl_empty_parts; From 58c4a7b89055e46b9cfc1dfc1cc3ba7da3129134 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Wed, 11 Nov 2020 19:52:27 +0300 Subject: [PATCH 151/205] Some renames. --- src/Interpreters/ActionsVisitor.cpp | 46 +++++++++---------- src/Interpreters/ActionsVisitor.h | 6 +-- src/Interpreters/ExpressionActions.cpp | 4 +- src/Interpreters/ExpressionActions.h | 20 ++++---- src/Interpreters/ExpressionAnalyzer.cpp | 8 ++-- src/Interpreters/ExpressionAnalyzer.h | 2 +- src/Interpreters/InterpreterSelectQuery.cpp | 12 ++--- src/Processors/QueryPlan/ExpressionStep.cpp | 14 +++--- src/Processors/QueryPlan/ExpressionStep.h | 6 +-- src/Processors/QueryPlan/FilterStep.cpp | 14 +++--- src/Processors/QueryPlan/FilterStep.h | 6 +-- src/Processors/QueryPlan/TotalsHavingStep.cpp | 12 ++--- src/Processors/QueryPlan/TotalsHavingStep.h | 4 +- src/Storages/SelectQueryInfo.h | 2 +- 14 files changed, 78 insertions(+), 78 deletions(-) diff --git a/src/Interpreters/ActionsVisitor.cpp b/src/Interpreters/ActionsVisitor.cpp index 29fb9ed336f..2ee474b1f69 100644 --- a/src/Interpreters/ActionsVisitor.cpp +++ b/src/Interpreters/ActionsVisitor.cpp @@ -383,7 +383,7 @@ SetPtr makeExplicitSet( ActionsMatcher::Data::Data( const Context & context_, SizeLimits set_size_limit_, size_t subquery_depth_, - const NamesAndTypesList & source_columns_, ActionsDAGPtr actions, + const NamesAndTypesList & source_columns_, ActionsDAGPtr actions_dag, PreparedSets & prepared_sets_, SubqueriesForSets & subqueries_for_sets_, bool no_subqueries_, bool no_makeset_, bool only_consts_, bool create_source_for_in_) : context(context_) @@ -397,7 +397,7 @@ ActionsMatcher::Data::Data( , only_consts(only_consts_) , create_source_for_in(create_source_for_in_) , visit_depth(0) - , actions_stack(std::move(actions), context) + , actions_stack(std::move(actions_dag), context) , next_unique_suffix(actions_stack.getLastActions().getIndex().size() + 1) { } @@ -407,13 +407,13 @@ bool ActionsMatcher::Data::hasColumn(const String & column_name) const return actions_stack.getLastActions().getIndex().contains(column_name); } -ScopeStack::ScopeStack(ActionsDAGPtr actions, const Context & context_) +ScopeStack::ScopeStack(ActionsDAGPtr actions_dag, const Context & context_) : context(context_) { auto & level = stack.emplace_back(); - level.actions = std::move(actions); + level.actions_dag = std::move(actions_dag); - for (const auto & node : level.actions->getIndex()) + for (const auto & node : level.actions_dag->getIndex()) if (node->type == ActionsDAG::ActionType::INPUT) level.inputs.emplace(node->result_name); } @@ -421,21 +421,21 @@ ScopeStack::ScopeStack(ActionsDAGPtr actions, const Context & context_) void ScopeStack::pushLevel(const NamesAndTypesList & input_columns) { auto & level = stack.emplace_back(); - level.actions = std::make_shared(); + level.actions_dag = std::make_shared(); const auto & prev = stack[stack.size() - 2]; for (const auto & input_column : input_columns) { - level.actions->addInput(input_column.name, input_column.type); + level.actions_dag->addInput(input_column.name, input_column.type); level.inputs.emplace(input_column.name); } - const auto & index = level.actions->getIndex(); + const auto & index = level.actions_dag->getIndex(); - for (const auto & node : prev.actions->getIndex()) + for (const auto & node : prev.actions_dag->getIndex()) { if (!index.contains(node->result_name)) - level.actions->addInput({node->column, node->result_type, node->result_name}); + level.actions_dag->addInput({node->column, node->result_type, node->result_name}); } } @@ -448,7 +448,7 @@ size_t ScopeStack::getColumnLevel(const std::string & name) if (stack[i].inputs.count(name)) return i; - const auto & index = stack[i].actions->getIndex(); + const auto & index = stack[i].actions_dag->getIndex(); auto it = index.find(name); if (it != index.end() && (*it)->type != ActionsDAG::ActionType::INPUT) @@ -460,33 +460,33 @@ size_t ScopeStack::getColumnLevel(const std::string & name) void ScopeStack::addColumn(ColumnWithTypeAndName column) { - const auto & node = stack[0].actions->addColumn(std::move(column)); + const auto & node = stack[0].actions_dag->addColumn(std::move(column)); for (size_t j = 1; j < stack.size(); ++j) - stack[j].actions->addInput({node.column, node.result_type, node.result_name}); + stack[j].actions_dag->addInput({node.column, node.result_type, node.result_name}); } void ScopeStack::addAlias(const std::string & name, std::string alias) { auto level = getColumnLevel(name); - const auto & node = stack[level].actions->addAlias(name, std::move(alias)); + const auto & node = stack[level].actions_dag->addAlias(name, std::move(alias)); for (size_t j = level + 1; j < stack.size(); ++j) - stack[j].actions->addInput({node.column, node.result_type, node.result_name}); + stack[j].actions_dag->addInput({node.column, node.result_type, node.result_name}); } void ScopeStack::addArrayJoin(const std::string & source_name, std::string result_name) { getColumnLevel(source_name); - if (!stack.front().actions->getIndex().contains(source_name)) + if (!stack.front().actions_dag->getIndex().contains(source_name)) throw Exception("Expression with arrayJoin cannot depend on lambda argument: " + source_name, ErrorCodes::BAD_ARGUMENTS); - const auto & node = stack.front().actions->addArrayJoin(source_name, std::move(result_name)); + const auto & node = stack.front().actions_dag->addArrayJoin(source_name, std::move(result_name)); for (size_t j = 1; j < stack.size(); ++j) - stack[j].actions->addInput({node.column, node.result_type, node.result_name}); + stack[j].actions_dag->addInput({node.column, node.result_type, node.result_name}); } void ScopeStack::addFunction( @@ -498,27 +498,27 @@ void ScopeStack::addFunction( for (const auto & argument : argument_names) level = std::max(level, getColumnLevel(argument)); - const auto & node = stack[level].actions->addFunction(function, argument_names, std::move(result_name), context); + const auto & node = stack[level].actions_dag->addFunction(function, argument_names, std::move(result_name), context); for (size_t j = level + 1; j < stack.size(); ++j) - stack[j].actions->addInput({node.column, node.result_type, node.result_name}); + stack[j].actions_dag->addInput({node.column, node.result_type, node.result_name}); } ActionsDAGPtr ScopeStack::popLevel() { auto res = std::move(stack.back()); stack.pop_back(); - return res.actions; + return res.actions_dag; } std::string ScopeStack::dumpNames() const { - return stack.back().actions->dumpNames(); + return stack.back().actions_dag->dumpNames(); } const ActionsDAG & ScopeStack::getLastActions() const { - return *stack.back().actions; + return *stack.back().actions_dag; } bool ActionsMatcher::needChildVisit(const ASTPtr & node, const ASTPtr & child) diff --git a/src/Interpreters/ActionsVisitor.h b/src/Interpreters/ActionsVisitor.h index cede7e08cac..8e186f06fee 100644 --- a/src/Interpreters/ActionsVisitor.h +++ b/src/Interpreters/ActionsVisitor.h @@ -64,7 +64,7 @@ struct ScopeStack { struct Level { - ActionsDAGPtr actions; + ActionsDAGPtr actions_dag; NameSet inputs; }; @@ -74,7 +74,7 @@ struct ScopeStack const Context & context; - ScopeStack(ActionsDAGPtr actions, const Context & context_); + ScopeStack(ActionsDAGPtr actions_dag, const Context & context_); void pushLevel(const NamesAndTypesList & input_columns); @@ -127,7 +127,7 @@ public: int next_unique_suffix; Data(const Context & context_, SizeLimits set_size_limit_, size_t subquery_depth_, - const NamesAndTypesList & source_columns_, ActionsDAGPtr actions, + const NamesAndTypesList & source_columns_, ActionsDAGPtr actions_dag, PreparedSets & prepared_sets_, SubqueriesForSets & subqueries_for_sets_, bool no_subqueries_, bool no_makeset_, bool only_consts_, bool create_source_for_in_); diff --git a/src/Interpreters/ExpressionActions.cpp b/src/Interpreters/ExpressionActions.cpp index 5fb6f68fa2d..4fee0b44ab3 100644 --- a/src/Interpreters/ExpressionActions.cpp +++ b/src/Interpreters/ExpressionActions.cpp @@ -950,12 +950,12 @@ void ExpressionActionsChain::JoinStep::finalize(const Names & required_output_) ActionsDAGPtr & ExpressionActionsChain::Step::actions() { - return typeid_cast(this)->actions; + return typeid_cast(this)->actions_dag; } const ActionsDAGPtr & ExpressionActionsChain::Step::actions() const { - return typeid_cast(this)->actions; + return typeid_cast(this)->actions_dag; } ActionsDAG::ActionsDAG(const NamesAndTypesList & inputs) diff --git a/src/Interpreters/ExpressionActions.h b/src/Interpreters/ExpressionActions.h index e3702200070..e7953a52c6d 100644 --- a/src/Interpreters/ExpressionActions.h +++ b/src/Interpreters/ExpressionActions.h @@ -416,38 +416,38 @@ struct ExpressionActionsChain struct ExpressionActionsStep : public Step { - ActionsDAGPtr actions; + ActionsDAGPtr actions_dag; - explicit ExpressionActionsStep(ActionsDAGPtr actions_, Names required_output_ = Names()) + explicit ExpressionActionsStep(ActionsDAGPtr actions_dag_, Names required_output_ = Names()) : Step(std::move(required_output_)) - , actions(std::move(actions_)) + , actions_dag(std::move(actions_dag_)) { } NamesAndTypesList getRequiredColumns() const override { - return actions->getRequiredColumns(); + return actions_dag->getRequiredColumns(); } ColumnsWithTypeAndName getResultColumns() const override { - return actions->getResultColumns(); + return actions_dag->getResultColumns(); } void finalize(const Names & required_output_) override { - if (!actions->getSettings().projected_output) - actions->removeUnusedActions(required_output_); + if (!actions_dag->getSettings().projected_output) + actions_dag->removeUnusedActions(required_output_); } void prependProjectInput() const override { - actions->projectInput(); + actions_dag->projectInput(); } std::string dump() const override { - return actions->dumpDAG(); + return actions_dag->dumpDAG(); } }; @@ -506,7 +506,7 @@ struct ExpressionActionsChain throw Exception("Empty ExpressionActionsChain", ErrorCodes::LOGICAL_ERROR); } - return typeid_cast(steps.back().get())->actions; + return typeid_cast(steps.back().get())->actions_dag; } Step & getLastStep() diff --git a/src/Interpreters/ExpressionAnalyzer.cpp b/src/Interpreters/ExpressionAnalyzer.cpp index c6b16d77a38..f9308c46143 100644 --- a/src/Interpreters/ExpressionAnalyzer.cpp +++ b/src/Interpreters/ExpressionAnalyzer.cpp @@ -734,13 +734,13 @@ ActionsDAGPtr SelectQueryExpressionAnalyzer::appendPrewhere( return prewhere_actions; } -void SelectQueryExpressionAnalyzer::appendPreliminaryFilter(ExpressionActionsChain & chain, ActionsDAGPtr actions, String column_name) +void SelectQueryExpressionAnalyzer::appendPreliminaryFilter(ExpressionActionsChain & chain, ActionsDAGPtr actions_dag, String column_name) { ExpressionActionsChain::Step & step = chain.lastStep(sourceColumns()); // FIXME: assert(filter_info); auto * expression_step = typeid_cast(&step); - expression_step->actions = std::move(actions); + expression_step->actions_dag = std::move(actions_dag); step.required_output.push_back(std::move(column_name)); step.can_remove_required_output = {true}; @@ -1130,7 +1130,7 @@ ExpressionAnalysisResult::ExpressionAnalysisResult( if (storage && filter_info_) { filter_info = filter_info_; - query_analyzer.appendPreliminaryFilter(chain, filter_info->actions, filter_info->column_name); + query_analyzer.appendPreliminaryFilter(chain, filter_info->actions_dag, filter_info->column_name); } if (auto actions = query_analyzer.appendPrewhere(chain, !first_stage, additional_required_columns_after_prewhere)) @@ -1292,7 +1292,7 @@ void ExpressionAnalysisResult::finalize(const ExpressionActionsChain & chain, si void ExpressionAnalysisResult::removeExtraColumns() const { if (hasFilter()) - filter_info->actions->projectInput(); + filter_info->actions_dag->projectInput(); if (hasWhere()) before_where->projectInput(); if (hasHaving()) diff --git a/src/Interpreters/ExpressionAnalyzer.h b/src/Interpreters/ExpressionAnalyzer.h index c13344049db..a50fe4de266 100644 --- a/src/Interpreters/ExpressionAnalyzer.h +++ b/src/Interpreters/ExpressionAnalyzer.h @@ -322,7 +322,7 @@ private: bool appendJoinLeftKeys(ExpressionActionsChain & chain, bool only_types); JoinPtr appendJoin(ExpressionActionsChain & chain); /// Add preliminary rows filtration. Actions are created in other expression analyzer to prevent any possible alias injection. - void appendPreliminaryFilter(ExpressionActionsChain & chain, ActionsDAGPtr actions, String column_name); + void appendPreliminaryFilter(ExpressionActionsChain & chain, ActionsDAGPtr actions_dag, String column_name); /// remove_filter is set in ExpressionActionsChain::finalize(); /// Columns in `additional_required_columns` will not be removed (they can be used for e.g. sampling or FINAL modifier). ActionsDAGPtr appendPrewhere(ExpressionActionsChain & chain, bool only_types, const Names & additional_required_columns); diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index 9d8e950cfcc..e16b51adb56 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -393,9 +393,9 @@ InterpreterSelectQuery::InterpreterSelectQuery( if (row_policy_filter) { filter_info = std::make_shared(); - filter_info->column_name = generateFilterActions(filter_info->actions, row_policy_filter, required_columns); + filter_info->column_name = generateFilterActions(filter_info->actions_dag, row_policy_filter, required_columns); source_header = metadata_snapshot->getSampleBlockForColumns( - filter_info->actions->getRequiredColumns().getNames(), storage->getVirtuals(), storage->getStorageID()); + filter_info->actions_dag->getRequiredColumns().getNames(), storage->getVirtuals(), storage->getStorageID()); } } @@ -881,7 +881,7 @@ void InterpreterSelectQuery::executeImpl(QueryPlan & query_plan, const BlockInpu { auto row_level_security_step = std::make_unique( query_plan.getCurrentDataStream(), - expressions.filter_info->actions, + expressions.filter_info->actions_dag, expressions.filter_info->column_name, expressions.filter_info->do_remove_column); @@ -1231,9 +1231,9 @@ void InterpreterSelectQuery::executeFetchColumns( if (row_policy_filter) { auto initial_required_columns = required_columns; - ActionsDAGPtr actions; - generateFilterActions(actions, row_policy_filter, initial_required_columns); - auto required_columns_from_filter = actions->getRequiredColumns(); + ActionsDAGPtr actions_dag; + generateFilterActions(actions_dag, row_policy_filter, initial_required_columns); + auto required_columns_from_filter = actions_dag->getRequiredColumns(); for (const auto & column : required_columns_from_filter) { diff --git a/src/Processors/QueryPlan/ExpressionStep.cpp b/src/Processors/QueryPlan/ExpressionStep.cpp index 87ebfa2526d..076f6125e28 100644 --- a/src/Processors/QueryPlan/ExpressionStep.cpp +++ b/src/Processors/QueryPlan/ExpressionStep.cpp @@ -41,12 +41,12 @@ static ITransformingStep::Traits getJoinTraits() }; } -ExpressionStep::ExpressionStep(const DataStream & input_stream_, ActionsDAGPtr actions_) +ExpressionStep::ExpressionStep(const DataStream & input_stream_, ActionsDAGPtr actions_dag_) : ITransformingStep( input_stream_, - Transform::transformHeader(input_stream_.header, *actions_), - getTraits(actions_)) - , actions(std::move(actions_)) + Transform::transformHeader(input_stream_.header, *actions_dag_), + getTraits(actions_dag_)) + , actions_dag(std::move(actions_dag_)) { /// Some columns may be removed by expression. updateDistinctColumns(output_stream->header, output_stream->distinct_columns); @@ -55,7 +55,7 @@ ExpressionStep::ExpressionStep(const DataStream & input_stream_, ActionsDAGPtr a void ExpressionStep::updateInputStream(DataStream input_stream, bool keep_header) { Block out_header = keep_header ? std::move(output_stream->header) - : Transform::transformHeader(input_stream.header, *actions); + : Transform::transformHeader(input_stream.header, *actions_dag); output_stream = createOutputStream( input_stream, std::move(out_header), @@ -67,7 +67,7 @@ void ExpressionStep::updateInputStream(DataStream input_stream, bool keep_header void ExpressionStep::transformPipeline(QueryPipeline & pipeline) { - auto expression = std::make_shared(actions); + auto expression = std::make_shared(actions_dag); pipeline.addSimpleTransform([&](const Block & header) { return std::make_shared(header, expression); @@ -88,7 +88,7 @@ void ExpressionStep::describeActions(FormatSettings & settings) const String prefix(settings.offset, ' '); bool first = true; - auto expression = std::make_shared(actions); + auto expression = std::make_shared(actions_dag); for (const auto & action : expression->getActions()) { settings.out << prefix << (first ? "Actions: " diff --git a/src/Processors/QueryPlan/ExpressionStep.h b/src/Processors/QueryPlan/ExpressionStep.h index c002de8deb1..60f186688b0 100644 --- a/src/Processors/QueryPlan/ExpressionStep.h +++ b/src/Processors/QueryPlan/ExpressionStep.h @@ -19,7 +19,7 @@ class ExpressionStep : public ITransformingStep public: using Transform = ExpressionTransform; - explicit ExpressionStep(const DataStream & input_stream_, ActionsDAGPtr actions_); + explicit ExpressionStep(const DataStream & input_stream_, ActionsDAGPtr actions_dag_); String getName() const override { return "Expression"; } void transformPipeline(QueryPipeline & pipeline) override; @@ -28,10 +28,10 @@ public: void describeActions(FormatSettings & settings) const override; - const ActionsDAGPtr & getExpression() const { return actions; } + const ActionsDAGPtr & getExpression() const { return actions_dag; } private: - ActionsDAGPtr actions; + ActionsDAGPtr actions_dag; }; /// TODO: add separate step for join. diff --git a/src/Processors/QueryPlan/FilterStep.cpp b/src/Processors/QueryPlan/FilterStep.cpp index 6f69a00fb27..c4bf97071e4 100644 --- a/src/Processors/QueryPlan/FilterStep.cpp +++ b/src/Processors/QueryPlan/FilterStep.cpp @@ -26,14 +26,14 @@ static ITransformingStep::Traits getTraits(const ActionsDAGPtr & expression) FilterStep::FilterStep( const DataStream & input_stream_, - ActionsDAGPtr actions_, + ActionsDAGPtr actions_dag_, String filter_column_name_, bool remove_filter_column_) : ITransformingStep( input_stream_, - FilterTransform::transformHeader(input_stream_.header, *actions_, filter_column_name_, remove_filter_column_), - getTraits(actions_)) - , actions(std::move(actions_)) + FilterTransform::transformHeader(input_stream_.header, *actions_dag_, filter_column_name_, remove_filter_column_), + getTraits(actions_dag_)) + , actions_dag(std::move(actions_dag_)) , filter_column_name(std::move(filter_column_name_)) , remove_filter_column(remove_filter_column_) { @@ -45,7 +45,7 @@ void FilterStep::updateInputStream(DataStream input_stream, bool keep_header) { Block out_header = std::move(output_stream->header); if (keep_header) - out_header = FilterTransform::transformHeader(input_stream.header, *actions, filter_column_name, remove_filter_column); + out_header = FilterTransform::transformHeader(input_stream.header, *actions_dag, filter_column_name, remove_filter_column); output_stream = createOutputStream( input_stream, @@ -58,7 +58,7 @@ void FilterStep::updateInputStream(DataStream input_stream, bool keep_header) void FilterStep::transformPipeline(QueryPipeline & pipeline) { - auto expression = std::make_shared(actions); + auto expression = std::make_shared(actions_dag); pipeline.addSimpleTransform([&](const Block & header, QueryPipeline::StreamType stream_type) { bool on_totals = stream_type == QueryPipeline::StreamType::Totals; @@ -80,7 +80,7 @@ void FilterStep::describeActions(FormatSettings & settings) const settings.out << prefix << "Filter column: " << filter_column_name << '\n'; bool first = true; - auto expression = std::make_shared(actions); + auto expression = std::make_shared(actions_dag); for (const auto & action : expression->getActions()) { settings.out << prefix << (first ? "Actions: " diff --git a/src/Processors/QueryPlan/FilterStep.h b/src/Processors/QueryPlan/FilterStep.h index efd91637c1f..72b02624faa 100644 --- a/src/Processors/QueryPlan/FilterStep.h +++ b/src/Processors/QueryPlan/FilterStep.h @@ -13,7 +13,7 @@ class FilterStep : public ITransformingStep public: FilterStep( const DataStream & input_stream_, - ActionsDAGPtr actions_, + ActionsDAGPtr actions_dag_, String filter_column_name_, bool remove_filter_column_); @@ -24,12 +24,12 @@ public: void describeActions(FormatSettings & settings) const override; - const ActionsDAGPtr & getExpression() const { return actions; } + const ActionsDAGPtr & getExpression() const { return actions_dag; } const String & getFilterColumnName() const { return filter_column_name; } bool removesFilterColumn() const { return remove_filter_column; } private: - ActionsDAGPtr actions; + ActionsDAGPtr actions_dag; String filter_column_name; bool remove_filter_column; }; diff --git a/src/Processors/QueryPlan/TotalsHavingStep.cpp b/src/Processors/QueryPlan/TotalsHavingStep.cpp index 3fe2f906d02..87e6994d60f 100644 --- a/src/Processors/QueryPlan/TotalsHavingStep.cpp +++ b/src/Processors/QueryPlan/TotalsHavingStep.cpp @@ -27,17 +27,17 @@ static ITransformingStep::Traits getTraits(bool has_filter) TotalsHavingStep::TotalsHavingStep( const DataStream & input_stream_, bool overflow_row_, - const ActionsDAGPtr & actions_, + const ActionsDAGPtr & actions_dag_, const std::string & filter_column_, TotalsMode totals_mode_, double auto_include_threshold_, bool final_) : ITransformingStep( input_stream_, - TotalsHavingTransform::transformHeader(input_stream_.header, actions_.get(), final_), + TotalsHavingTransform::transformHeader(input_stream_.header, actions_dag_.get(), final_), getTraits(!filter_column_.empty())) , overflow_row(overflow_row_) - , actions(actions_) + , actions_dag(actions_dag_) , filter_column_name(filter_column_) , totals_mode(totals_mode_) , auto_include_threshold(auto_include_threshold_) @@ -48,7 +48,7 @@ TotalsHavingStep::TotalsHavingStep( void TotalsHavingStep::transformPipeline(QueryPipeline & pipeline) { auto totals_having = std::make_shared( - pipeline.getHeader(), overflow_row, (actions ? std::make_shared(actions) : nullptr), + pipeline.getHeader(), overflow_row, (actions_dag ? std::make_shared(actions_dag) : nullptr), filter_column_name, totals_mode, auto_include_threshold, final); pipeline.addTotalsHavingTransform(std::move(totals_having)); @@ -77,10 +77,10 @@ void TotalsHavingStep::describeActions(FormatSettings & settings) const settings.out << prefix << "Filter column: " << filter_column_name << '\n'; settings.out << prefix << "Mode: " << totalsModeToString(totals_mode, auto_include_threshold) << '\n'; - if (actions) + if (actions_dag) { bool first = true; - auto expression = std::make_shared(actions); + auto expression = std::make_shared(actions_dag); for (const auto & action : expression->getActions()) { settings.out << prefix << (first ? "Actions: " diff --git a/src/Processors/QueryPlan/TotalsHavingStep.h b/src/Processors/QueryPlan/TotalsHavingStep.h index 792b96c9162..7c1638013e5 100644 --- a/src/Processors/QueryPlan/TotalsHavingStep.h +++ b/src/Processors/QueryPlan/TotalsHavingStep.h @@ -16,7 +16,7 @@ public: TotalsHavingStep( const DataStream & input_stream_, bool overflow_row_, - const ActionsDAGPtr & actions_, + const ActionsDAGPtr & actions_dag_, const std::string & filter_column_, TotalsMode totals_mode_, double auto_include_threshold_, @@ -30,7 +30,7 @@ public: private: bool overflow_row; - ActionsDAGPtr actions; + ActionsDAGPtr actions_dag; String filter_column_name; TotalsMode totals_mode; double auto_include_threshold; diff --git a/src/Storages/SelectQueryInfo.h b/src/Storages/SelectQueryInfo.h index 00f20ac3893..683b2463d1f 100644 --- a/src/Storages/SelectQueryInfo.h +++ b/src/Storages/SelectQueryInfo.h @@ -50,7 +50,7 @@ struct PrewhereDAGInfo /// Helper struct to store all the information about the filter expression. struct FilterInfo { - ActionsDAGPtr actions; + ActionsDAGPtr actions_dag; String column_name; bool do_remove_column = false; }; From 9b13d0e0ebfad8b038c9edbe670ea5a9c14a9c6f Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Wed, 11 Nov 2020 20:17:26 +0300 Subject: [PATCH 152/205] Revert "Add ActionsDAG::updateHeader()" This reverts commit b78657656cb2470a6d26fb343ba1a196608e4e85. --- src/Interpreters/ExpressionActions.cpp | 24 ------------------- src/Interpreters/ExpressionActions.h | 4 ---- src/Interpreters/ExpressionAnalyzer.cpp | 7 +++--- src/Interpreters/InterpreterSelectQuery.cpp | 2 +- src/Processors/QueryPlan/ExpressionStep.cpp | 4 ++-- src/Processors/QueryPlan/FilterStep.cpp | 4 ++-- src/Processors/QueryPlan/TotalsHavingStep.cpp | 5 +++- .../Transforms/ExpressionTransform.cpp | 7 +++--- .../Transforms/ExpressionTransform.h | 4 +--- src/Processors/Transforms/FilterTransform.cpp | 7 +++--- src/Processors/Transforms/FilterTransform.h | 4 +--- .../Transforms/TotalsHavingTransform.cpp | 11 ++++----- .../Transforms/TotalsHavingTransform.h | 4 +--- 13 files changed, 29 insertions(+), 58 deletions(-) diff --git a/src/Interpreters/ExpressionActions.cpp b/src/Interpreters/ExpressionActions.cpp index 4fee0b44ab3..5c22d77d11b 100644 --- a/src/Interpreters/ExpressionActions.cpp +++ b/src/Interpreters/ExpressionActions.cpp @@ -482,30 +482,6 @@ void ExpressionActions::execute(Block & block, bool dry_run) const block.insert({DataTypeUInt8().createColumnConst(num_rows, 0), std::make_shared(), "_dummy"}); } -void ActionsDAG::updateHeader(Block & block) const -{ - if (settings.project_input) - block.clear(); - else - { - for (const auto & node : nodes) - if (node.type == ActionType::INPUT && block.has(node.result_name)) - block.erase(node.result_name); - } - - for (const auto & node : index) - { - ColumnPtr col; - - if (node->column && isColumnConst(*node->column)) - col = node->column->cloneResized(0); - else - col = node->result_type->createColumn(); - - block.insert({col, node->result_type, node->result_name}); - } -} - Names ExpressionActions::getRequiredColumns() const { Names names; diff --git a/src/Interpreters/ExpressionActions.h b/src/Interpreters/ExpressionActions.h index e7953a52c6d..90d8596889a 100644 --- a/src/Interpreters/ExpressionActions.h +++ b/src/Interpreters/ExpressionActions.h @@ -246,10 +246,6 @@ public: void projectInput() { settings.project_input = true; } void removeUnusedActions(const Names & required_names); - /// Changed structure of empty block as if actions are executed. - /// Does not actually execute actions, but only creates empty columns. - void updateHeader(Block & block) const; - /// Splits actions into two parts. Returned half may be swapped with ARRAY JOIN. /// Returns nullptr if no actions may be moved before ARRAY JOIN. ActionsDAGPtr splitActionsBeforeArrayJoin(const NameSet & array_joined_columns); diff --git a/src/Interpreters/ExpressionAnalyzer.cpp b/src/Interpreters/ExpressionAnalyzer.cpp index f9308c46143..de82a9b9645 100644 --- a/src/Interpreters/ExpressionAnalyzer.cpp +++ b/src/Interpreters/ExpressionAnalyzer.cpp @@ -673,7 +673,8 @@ ActionsDAGPtr SelectQueryExpressionAnalyzer::appendPrewhere( auto tmp_actions_dag = std::make_shared(sourceColumns()); getRootActions(select_query->prewhere(), only_types, tmp_actions_dag); tmp_actions_dag->removeUnusedActions({prewhere_column_name}); - auto required_columns = tmp_actions_dag->getRequiredColumns().getNames(); + auto tmp_actions = std::make_shared(tmp_actions_dag); + auto required_columns = tmp_actions->getRequiredColumns(); NameSet required_source_columns(required_columns.begin(), required_columns.end()); /// Add required columns to required output in order not to remove them after prewhere execution. @@ -1142,7 +1143,7 @@ ExpressionAnalysisResult::ExpressionAnalysisResult( Block before_prewhere_sample = source_header; if (sanitizeBlock(before_prewhere_sample)) { - prewhere_info->prewhere_actions->updateHeader(before_prewhere_sample); + ExpressionActions(prewhere_info->prewhere_actions).execute(before_prewhere_sample); auto & column_elem = before_prewhere_sample.getByName(query.prewhere()->getColumnName()); /// If the filter column is a constant, record it. if (column_elem.column) @@ -1175,7 +1176,7 @@ ExpressionAnalysisResult::ExpressionAnalysisResult( before_where_sample = source_header; if (sanitizeBlock(before_where_sample)) { - before_where->updateHeader(before_where_sample); + ExpressionActions(before_where).execute(before_where_sample); auto & column_elem = before_where_sample.getByName(query.where()->getColumnName()); /// If the filter column is a constant, record it. if (column_elem.column) diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index e16b51adb56..2eee269efe1 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -524,7 +524,7 @@ Block InterpreterSelectQuery::getSampleBlockImpl() if (analysis_result.prewhere_info) { - analysis_result.prewhere_info->prewhere_actions->updateHeader(header); + ExpressionActions(analysis_result.prewhere_info->prewhere_actions).execute(header); header = materializeBlock(header); if (analysis_result.prewhere_info->remove_prewhere_column) header.erase(analysis_result.prewhere_info->prewhere_column_name); diff --git a/src/Processors/QueryPlan/ExpressionStep.cpp b/src/Processors/QueryPlan/ExpressionStep.cpp index 076f6125e28..6c7c5867b7f 100644 --- a/src/Processors/QueryPlan/ExpressionStep.cpp +++ b/src/Processors/QueryPlan/ExpressionStep.cpp @@ -44,7 +44,7 @@ static ITransformingStep::Traits getJoinTraits() ExpressionStep::ExpressionStep(const DataStream & input_stream_, ActionsDAGPtr actions_dag_) : ITransformingStep( input_stream_, - Transform::transformHeader(input_stream_.header, *actions_dag_), + Transform::transformHeader(input_stream_.header, std::make_shared(actions_dag_)), getTraits(actions_dag_)) , actions_dag(std::move(actions_dag_)) { @@ -55,7 +55,7 @@ ExpressionStep::ExpressionStep(const DataStream & input_stream_, ActionsDAGPtr a void ExpressionStep::updateInputStream(DataStream input_stream, bool keep_header) { Block out_header = keep_header ? std::move(output_stream->header) - : Transform::transformHeader(input_stream.header, *actions_dag); + : Transform::transformHeader(input_stream.header, std::make_shared(actions_dag)); output_stream = createOutputStream( input_stream, std::move(out_header), diff --git a/src/Processors/QueryPlan/FilterStep.cpp b/src/Processors/QueryPlan/FilterStep.cpp index c4bf97071e4..63dbc2b63b4 100644 --- a/src/Processors/QueryPlan/FilterStep.cpp +++ b/src/Processors/QueryPlan/FilterStep.cpp @@ -31,7 +31,7 @@ FilterStep::FilterStep( bool remove_filter_column_) : ITransformingStep( input_stream_, - FilterTransform::transformHeader(input_stream_.header, *actions_dag_, filter_column_name_, remove_filter_column_), + FilterTransform::transformHeader(input_stream_.header, std::make_shared(actions_dag_), filter_column_name_, remove_filter_column_), getTraits(actions_dag_)) , actions_dag(std::move(actions_dag_)) , filter_column_name(std::move(filter_column_name_)) @@ -45,7 +45,7 @@ void FilterStep::updateInputStream(DataStream input_stream, bool keep_header) { Block out_header = std::move(output_stream->header); if (keep_header) - out_header = FilterTransform::transformHeader(input_stream.header, *actions_dag, filter_column_name, remove_filter_column); + out_header = FilterTransform::transformHeader(input_stream.header, std::make_shared(actions_dag), filter_column_name, remove_filter_column); output_stream = createOutputStream( input_stream, diff --git a/src/Processors/QueryPlan/TotalsHavingStep.cpp b/src/Processors/QueryPlan/TotalsHavingStep.cpp index 87e6994d60f..9947875e679 100644 --- a/src/Processors/QueryPlan/TotalsHavingStep.cpp +++ b/src/Processors/QueryPlan/TotalsHavingStep.cpp @@ -34,7 +34,10 @@ TotalsHavingStep::TotalsHavingStep( bool final_) : ITransformingStep( input_stream_, - TotalsHavingTransform::transformHeader(input_stream_.header, actions_dag_.get(), final_), + TotalsHavingTransform::transformHeader( + input_stream_.header, + (actions_dag_ ? std::make_shared(actions_dag_) : nullptr), + final_), getTraits(!filter_column_.empty())) , overflow_row(overflow_row_) , actions_dag(actions_dag_) diff --git a/src/Processors/Transforms/ExpressionTransform.cpp b/src/Processors/Transforms/ExpressionTransform.cpp index efcdaa888bb..e60fc65e96e 100644 --- a/src/Processors/Transforms/ExpressionTransform.cpp +++ b/src/Processors/Transforms/ExpressionTransform.cpp @@ -3,15 +3,16 @@ namespace DB { -Block ExpressionTransform::transformHeader(Block header, const ActionsDAG & expression) +Block ExpressionTransform::transformHeader(Block header, const ExpressionActionsPtr & expression) { - expression.updateHeader(header); + size_t num_rows = header.rows(); + expression->execute(header, num_rows, true); return header; } ExpressionTransform::ExpressionTransform(const Block & header_, ExpressionActionsPtr expression_) - : ISimpleTransform(header_, transformHeader(header_, expression_->getActionsDAG()), false) + : ISimpleTransform(header_, transformHeader(header_, expression_), false) , expression(std::move(expression_)) { } diff --git a/src/Processors/Transforms/ExpressionTransform.h b/src/Processors/Transforms/ExpressionTransform.h index d4cc1c8f78a..525273bad81 100644 --- a/src/Processors/Transforms/ExpressionTransform.h +++ b/src/Processors/Transforms/ExpressionTransform.h @@ -7,8 +7,6 @@ namespace DB class ExpressionActions; using ExpressionActionsPtr = std::shared_ptr; -class ActionsDAG; - /** Executes a certain expression over the block. * The expression consists of column identifiers from the block, constants, common functions. * For example: hits * 2 + 3, url LIKE '%yandex%' @@ -23,7 +21,7 @@ public: String getName() const override { return "ExpressionTransform"; } - static Block transformHeader(Block header, const ActionsDAG & expression); + static Block transformHeader(Block header, const ExpressionActionsPtr & expression); protected: void transform(Chunk & chunk) override; diff --git a/src/Processors/Transforms/FilterTransform.cpp b/src/Processors/Transforms/FilterTransform.cpp index 734a4995ceb..23b1bf8a984 100644 --- a/src/Processors/Transforms/FilterTransform.cpp +++ b/src/Processors/Transforms/FilterTransform.cpp @@ -29,11 +29,12 @@ static void replaceFilterToConstant(Block & block, const String & filter_column_ Block FilterTransform::transformHeader( Block header, - const ActionsDAG & expression, + const ExpressionActionsPtr & expression, const String & filter_column_name, bool remove_filter_column) { - expression.updateHeader(header); + size_t num_rows = header.rows(); + expression->execute(header, num_rows); if (remove_filter_column) header.erase(filter_column_name); @@ -49,7 +50,7 @@ FilterTransform::FilterTransform( String filter_column_name_, bool remove_filter_column_, bool on_totals_) - : ISimpleTransform(header_, transformHeader(header_, expression_->getActionsDAG(), filter_column_name_, remove_filter_column_), true) + : ISimpleTransform(header_, transformHeader(header_, expression_, filter_column_name_, remove_filter_column_), true) , expression(std::move(expression_)) , filter_column_name(std::move(filter_column_name_)) , remove_filter_column(remove_filter_column_) diff --git a/src/Processors/Transforms/FilterTransform.h b/src/Processors/Transforms/FilterTransform.h index 39f1f1c42db..c0ccf0fd072 100644 --- a/src/Processors/Transforms/FilterTransform.h +++ b/src/Processors/Transforms/FilterTransform.h @@ -8,8 +8,6 @@ namespace DB class ExpressionActions; using ExpressionActionsPtr = std::shared_ptr; -class ActionsDAG; - /** Implements WHERE, HAVING operations. * Takes an expression, which adds to the block one ColumnUInt8 column containing the filtering conditions. * The expression is evaluated and result chunks contain only the filtered rows. @@ -24,7 +22,7 @@ public: static Block transformHeader( Block header, - const ActionsDAG & expression, + const ExpressionActionsPtr & expression, const String & filter_column_name, bool remove_filter_column); diff --git a/src/Processors/Transforms/TotalsHavingTransform.cpp b/src/Processors/Transforms/TotalsHavingTransform.cpp index dad4002eb69..c81c389b6fa 100644 --- a/src/Processors/Transforms/TotalsHavingTransform.cpp +++ b/src/Processors/Transforms/TotalsHavingTransform.cpp @@ -28,13 +28,15 @@ void finalizeChunk(Chunk & chunk) chunk.setColumns(std::move(columns), num_rows); } -Block TotalsHavingTransform::transformHeader(Block block, const ActionsDAG * expression, bool final) +Block TotalsHavingTransform::transformHeader(Block block, const ExpressionActionsPtr & expression, bool final) { if (final) finalizeBlock(block); + size_t num_rows = block.rows(); + if (expression) - expression->updateHeader(block); + expression->execute(block, num_rows); return block; } @@ -47,10 +49,7 @@ TotalsHavingTransform::TotalsHavingTransform( TotalsMode totals_mode_, double auto_include_threshold_, bool final_) - : ISimpleTransform( - header, - transformHeader(header, expression_ ? &expression_->getActionsDAG() : nullptr, final_), - true) + : ISimpleTransform(header, transformHeader(header, expression_, final_), true) , overflow_row(overflow_row_) , expression(expression_) , filter_column_name(filter_column_) diff --git a/src/Processors/Transforms/TotalsHavingTransform.h b/src/Processors/Transforms/TotalsHavingTransform.h index 5809f382e0e..8191ec763b0 100644 --- a/src/Processors/Transforms/TotalsHavingTransform.h +++ b/src/Processors/Transforms/TotalsHavingTransform.h @@ -12,8 +12,6 @@ using ArenaPtr = std::shared_ptr; class ExpressionActions; using ExpressionActionsPtr = std::shared_ptr; -class ActionsDAG; - enum class TotalsMode; /** Takes blocks after grouping, with non-finalized aggregate functions. @@ -39,7 +37,7 @@ public: Status prepare() override; void work() override; - static Block transformHeader(Block block, const ActionsDAG * expression, bool final); + static Block transformHeader(Block block, const ExpressionActionsPtr & expression, bool final); protected: void transform(Chunk & chunk) override; From f4e44e63b016d15de8fb84383a5bd18daeb6b475 Mon Sep 17 00:00:00 2001 From: Mikhail Filimonov Date: Wed, 11 Nov 2020 18:35:48 +0100 Subject: [PATCH 153/205] Remove timeSeriesGroupRateSum from docs --- .../aggregate-functions/reference/index.md | 2 - .../reference/timeseriesgroupratesum.md | 16 ----- .../reference/timeseriesgroupsum.md | 57 ----------------- .../aggregate-functions/reference.md | 63 ------------------- .../aggregate-functions/reference.md | 63 ------------------- .../aggregate-functions/reference.md | 63 ------------------- .../aggregate-functions/reference.md | 63 ------------------- .../aggregate-functions/reference/index.md | 2 - .../reference/timeseriesgroupratesum.md | 18 ------ .../reference/timeseriesgroupsum.md | 59 ----------------- .../aggregate-functions/reference.md | 63 ------------------- .../aggregate-functions/reference.md | 63 ------------------- 12 files changed, 532 deletions(-) delete mode 100644 docs/en/sql-reference/aggregate-functions/reference/timeseriesgroupratesum.md delete mode 100644 docs/en/sql-reference/aggregate-functions/reference/timeseriesgroupsum.md delete mode 100644 docs/ru/sql-reference/aggregate-functions/reference/timeseriesgroupratesum.md delete mode 100644 docs/ru/sql-reference/aggregate-functions/reference/timeseriesgroupsum.md diff --git a/docs/en/sql-reference/aggregate-functions/reference/index.md b/docs/en/sql-reference/aggregate-functions/reference/index.md index 6e00ad8d991..b96fa887279 100644 --- a/docs/en/sql-reference/aggregate-functions/reference/index.md +++ b/docs/en/sql-reference/aggregate-functions/reference/index.md @@ -50,8 +50,6 @@ ClickHouse-specific aggregate functions: - [skewPop](../../../sql-reference/aggregate-functions/reference/skewpop.md) - [kurtSamp](../../../sql-reference/aggregate-functions/reference/kurtsamp.md) - [kurtPop](../../../sql-reference/aggregate-functions/reference/kurtpop.md) -- [timeSeriesGroupSum](../../../sql-reference/aggregate-functions/reference/timeseriesgroupsum.md) -- [timeSeriesGroupRateSum](../../../sql-reference/aggregate-functions/reference/timeseriesgroupratesum.md) - [uniq](../../../sql-reference/aggregate-functions/reference/uniq.md) - [uniqExact](../../../sql-reference/aggregate-functions/reference/uniqexact.md) - [uniqCombined](../../../sql-reference/aggregate-functions/reference/uniqcombined.md) diff --git a/docs/en/sql-reference/aggregate-functions/reference/timeseriesgroupratesum.md b/docs/en/sql-reference/aggregate-functions/reference/timeseriesgroupratesum.md deleted file mode 100644 index d9b519a9cde..00000000000 --- a/docs/en/sql-reference/aggregate-functions/reference/timeseriesgroupratesum.md +++ /dev/null @@ -1,16 +0,0 @@ ---- -toc_priority: 171 ---- - -# timeSeriesGroupRateSum {#agg-function-timeseriesgroupratesum} - -Syntax: `timeSeriesGroupRateSum(uid, ts, val)` - -Similarly to [timeSeriesGroupSum](../../../sql-reference/aggregate-functions/reference/timeseriesgroupsum.md), `timeSeriesGroupRateSum` calculates the rate of time-series and then sum rates together. -Also, timestamp should be in ascend order before use this function. - -Applying this function to the data from the `timeSeriesGroupSum` example, you get the following result: - -``` text -[(2,0),(3,0.1),(7,0.3),(8,0.3),(12,0.3),(17,0.3),(18,0.3),(24,0.3),(25,0.1)] -``` diff --git a/docs/en/sql-reference/aggregate-functions/reference/timeseriesgroupsum.md b/docs/en/sql-reference/aggregate-functions/reference/timeseriesgroupsum.md deleted file mode 100644 index aa90c7956df..00000000000 --- a/docs/en/sql-reference/aggregate-functions/reference/timeseriesgroupsum.md +++ /dev/null @@ -1,57 +0,0 @@ ---- -toc_priority: 170 ---- - -# timeSeriesGroupSum {#agg-function-timeseriesgroupsum} - -Syntax: `timeSeriesGroupSum(uid, timestamp, value)` - -`timeSeriesGroupSum` can aggregate different time series that sample timestamp not alignment. -It will use linear interpolation between two sample timestamp and then sum time-series together. - -- `uid` is the time series unique id, `UInt64`. -- `timestamp` is Int64 type in order to support millisecond or microsecond. -- `value` is the metric. - -The function returns array of tuples with `(timestamp, aggregated_value)` pairs. - -Before using this function make sure `timestamp` is in ascending order. - -Example: - -``` text -┌─uid─┬─timestamp─┬─value─┐ -│ 1 │ 2 │ 0.2 │ -│ 1 │ 7 │ 0.7 │ -│ 1 │ 12 │ 1.2 │ -│ 1 │ 17 │ 1.7 │ -│ 1 │ 25 │ 2.5 │ -│ 2 │ 3 │ 0.6 │ -│ 2 │ 8 │ 1.6 │ -│ 2 │ 12 │ 2.4 │ -│ 2 │ 18 │ 3.6 │ -│ 2 │ 24 │ 4.8 │ -└─────┴───────────┴───────┘ -``` - -``` sql -CREATE TABLE time_series( - uid UInt64, - timestamp Int64, - value Float64 -) ENGINE = Memory; -INSERT INTO time_series VALUES - (1,2,0.2),(1,7,0.7),(1,12,1.2),(1,17,1.7),(1,25,2.5), - (2,3,0.6),(2,8,1.6),(2,12,2.4),(2,18,3.6),(2,24,4.8); - -SELECT timeSeriesGroupSum(uid, timestamp, value) -FROM ( - SELECT * FROM time_series order by timestamp ASC -); -``` - -And the result will be: - -``` text -[(2,0.2),(3,0.9),(7,2.1),(8,2.4),(12,3.6),(17,5.1),(18,5.4),(24,7.2),(25,2.5)] -``` diff --git a/docs/es/sql-reference/aggregate-functions/reference.md b/docs/es/sql-reference/aggregate-functions/reference.md index 927d16bc748..572c4d01051 100644 --- a/docs/es/sql-reference/aggregate-functions/reference.md +++ b/docs/es/sql-reference/aggregate-functions/reference.md @@ -464,69 +464,6 @@ The kurtosis of the given distribution. Type — [Float64](../../sql-reference/d SELECT kurtSamp(value) FROM series_with_value_column ``` -## Para obtener más información, consulta nuestra Política de privacidad y nuestras Condiciones de uso) {#agg-function-timeseriesgroupsum} - -`timeSeriesGroupSum` puede agregar diferentes series de tiempo que muestran la marca de tiempo no la alineación. -Utilizará la interpolación lineal entre dos marcas de tiempo de muestra y luego sumará series temporales juntas. - -- `uid` es la identificación única de la serie temporal, `UInt64`. -- `timestamp` es el tipo Int64 para admitir milisegundos o microsegundos. -- `value` es la métrica. - -La función devuelve una matriz de tuplas con `(timestamp, aggregated_value)` par. - -Antes de utilizar esta función, asegúrese de `timestamp` está en orden ascendente. - -Ejemplo: - -``` text -┌─uid─┬─timestamp─┬─value─┐ -│ 1 │ 2 │ 0.2 │ -│ 1 │ 7 │ 0.7 │ -│ 1 │ 12 │ 1.2 │ -│ 1 │ 17 │ 1.7 │ -│ 1 │ 25 │ 2.5 │ -│ 2 │ 3 │ 0.6 │ -│ 2 │ 8 │ 1.6 │ -│ 2 │ 12 │ 2.4 │ -│ 2 │ 18 │ 3.6 │ -│ 2 │ 24 │ 4.8 │ -└─────┴───────────┴───────┘ -``` - -``` sql -CREATE TABLE time_series( - uid UInt64, - timestamp Int64, - value Float64 -) ENGINE = Memory; -INSERT INTO time_series VALUES - (1,2,0.2),(1,7,0.7),(1,12,1.2),(1,17,1.7),(1,25,2.5), - (2,3,0.6),(2,8,1.6),(2,12,2.4),(2,18,3.6),(2,24,4.8); - -SELECT timeSeriesGroupSum(uid, timestamp, value) -FROM ( - SELECT * FROM time_series order by timestamp ASC -); -``` - -Y el resultado será: - -``` text -[(2,0.2),(3,0.9),(7,2.1),(8,2.4),(12,3.6),(17,5.1),(18,5.4),(24,7.2),(25,2.5)] -``` - -## También puede utilizar el siguiente ejemplo:) {#agg-function-timeseriesgroupratesum} - -De manera similar a `timeSeriesGroupSum`, `timeSeriesGroupRateSum` calcula la tasa de series temporales y luego suma las tasas juntas. -Además, la marca de tiempo debe estar en orden ascendente antes de usar esta función. - -Aplicando esta función a los datos del `timeSeriesGroupSum` ejemplo, se obtiene el siguiente resultado: - -``` text -[(2,0),(3,0.1),(7,0.3),(8,0.3),(12,0.3),(17,0.3),(18,0.3),(24,0.3),(25,0.1)] -``` - ## Acerca de) {#agg_function-avg} Calcula el promedio. diff --git a/docs/fa/sql-reference/aggregate-functions/reference.md b/docs/fa/sql-reference/aggregate-functions/reference.md index 0933bf50b20..f18dc9e6bf6 100644 --- a/docs/fa/sql-reference/aggregate-functions/reference.md +++ b/docs/fa/sql-reference/aggregate-functions/reference.md @@ -464,69 +464,6 @@ The kurtosis of the given distribution. Type — [جسم شناور64](../../sql SELECT kurtSamp(value) FROM series_with_value_column ``` -## هشدار داده می شود) {#agg-function-timeseriesgroupsum} - -`timeSeriesGroupSum` می توانید سری های زمانی مختلف که برچسب زمان نمونه هم ترازی جمع نمی. -این برون یابی خطی بین دو برچسب زمان نمونه و سپس مجموع زمان سری با هم استفاده کنید. - -- `uid` سری زمان شناسه منحصر به فرد است, `UInt64`. -- `timestamp` است نوع درون64 به منظور حمایت میلی ثانیه یا میکروثانیه. -- `value` متریک است. - -تابع گرداند مجموعه ای از تاپل با `(timestamp, aggregated_value)` جفت - -قبل از استفاده از این تابع اطمینان حاصل کنید `timestamp` به ترتیب صعودی است. - -مثال: - -``` text -┌─uid─┬─timestamp─┬─value─┐ -│ 1 │ 2 │ 0.2 │ -│ 1 │ 7 │ 0.7 │ -│ 1 │ 12 │ 1.2 │ -│ 1 │ 17 │ 1.7 │ -│ 1 │ 25 │ 2.5 │ -│ 2 │ 3 │ 0.6 │ -│ 2 │ 8 │ 1.6 │ -│ 2 │ 12 │ 2.4 │ -│ 2 │ 18 │ 3.6 │ -│ 2 │ 24 │ 4.8 │ -└─────┴───────────┴───────┘ -``` - -``` sql -CREATE TABLE time_series( - uid UInt64, - timestamp Int64, - value Float64 -) ENGINE = Memory; -INSERT INTO time_series VALUES - (1,2,0.2),(1,7,0.7),(1,12,1.2),(1,17,1.7),(1,25,2.5), - (2,3,0.6),(2,8,1.6),(2,12,2.4),(2,18,3.6),(2,24,4.8); - -SELECT timeSeriesGroupSum(uid, timestamp, value) -FROM ( - SELECT * FROM time_series order by timestamp ASC -); -``` - -و نتیجه خواهد بود: - -``` text -[(2,0.2),(3,0.9),(7,2.1),(8,2.4),(12,3.6),(17,5.1),(18,5.4),(24,7.2),(25,2.5)] -``` - -## هشدار داده می شود) {#agg-function-timeseriesgroupratesum} - -به طور مشابه به `timeSeriesGroupSum`, `timeSeriesGroupRateSum` محاسبه نرخ زمان سری و سپس مجموع نرخ با هم. -همچنین, برچسب زمان باید در جهت صعود قبل از استفاده از این تابع باشد. - -استفاده از این تابع به داده ها از `timeSeriesGroupSum` مثال, شما نتیجه زیر را دریافت کنید: - -``` text -[(2,0),(3,0.1),(7,0.3),(8,0.3),(12,0.3),(17,0.3),(18,0.3),(24,0.3),(25,0.1)] -``` - ## میانگین) {#agg_function-avg} محاسبه متوسط. diff --git a/docs/fr/sql-reference/aggregate-functions/reference.md b/docs/fr/sql-reference/aggregate-functions/reference.md index dcbf3bdd95a..37d150a6f68 100644 --- a/docs/fr/sql-reference/aggregate-functions/reference.md +++ b/docs/fr/sql-reference/aggregate-functions/reference.md @@ -464,69 +464,6 @@ The kurtosis of the given distribution. Type — [Float64](../../sql-reference/d SELECT kurtSamp(value) FROM series_with_value_column ``` -## timeSeriesGroupSum(uid, horodatage, valeur) {#agg-function-timeseriesgroupsum} - -`timeSeriesGroupSum` peut agréger différentes séries temporelles qui échantillonnent l'horodatage et non l'alignement. -Il utilisera une interpolation linéaire entre deux échantillons d'horodatage, puis additionnera les séries temporelles ensemble. - -- `uid` la série temporelle est elle unique, `UInt64`. -- `timestamp` est de type Int64 afin de prendre en charge la milliseconde ou la microseconde. -- `value` est la métrique. - -La fonction renvoie un tableau de tuples avec `(timestamp, aggregated_value)` pair. - -Avant d'utiliser cette fonction, assurez-vous `timestamp` est dans l'ordre croissant. - -Exemple: - -``` text -┌─uid─┬─timestamp─┬─value─┐ -│ 1 │ 2 │ 0.2 │ -│ 1 │ 7 │ 0.7 │ -│ 1 │ 12 │ 1.2 │ -│ 1 │ 17 │ 1.7 │ -│ 1 │ 25 │ 2.5 │ -│ 2 │ 3 │ 0.6 │ -│ 2 │ 8 │ 1.6 │ -│ 2 │ 12 │ 2.4 │ -│ 2 │ 18 │ 3.6 │ -│ 2 │ 24 │ 4.8 │ -└─────┴───────────┴───────┘ -``` - -``` sql -CREATE TABLE time_series( - uid UInt64, - timestamp Int64, - value Float64 -) ENGINE = Memory; -INSERT INTO time_series VALUES - (1,2,0.2),(1,7,0.7),(1,12,1.2),(1,17,1.7),(1,25,2.5), - (2,3,0.6),(2,8,1.6),(2,12,2.4),(2,18,3.6),(2,24,4.8); - -SELECT timeSeriesGroupSum(uid, timestamp, value) -FROM ( - SELECT * FROM time_series order by timestamp ASC -); -``` - -Et le résultat sera: - -``` text -[(2,0.2),(3,0.9),(7,2.1),(8,2.4),(12,3.6),(17,5.1),(18,5.4),(24,7.2),(25,2.5)] -``` - -## timeSeriesGroupRateSum(uid, ts, val) {#agg-function-timeseriesgroupratesum} - -De la même manière à `timeSeriesGroupSum`, `timeSeriesGroupRateSum` calcule le taux de séries chronologiques, puis additionne les taux ensemble. -En outre, l'horodatage doit être dans l'ordre croissant avant d'utiliser cette fonction. - -Application de cette fonction aux données du `timeSeriesGroupSum` exemple, vous obtenez le résultat suivant: - -``` text -[(2,0),(3,0.1),(7,0.3),(8,0.3),(12,0.3),(17,0.3),(18,0.3),(24,0.3),(25,0.1)] -``` - ## avg (x) {#agg_function-avg} Calcule la moyenne. diff --git a/docs/ja/sql-reference/aggregate-functions/reference.md b/docs/ja/sql-reference/aggregate-functions/reference.md index 298de5a75b5..465f36179da 100644 --- a/docs/ja/sql-reference/aggregate-functions/reference.md +++ b/docs/ja/sql-reference/aggregate-functions/reference.md @@ -464,69 +464,6 @@ The kurtosis of the given distribution. Type — [Float64](../../sql-reference/d SELECT kurtSamp(value) FROM series_with_value_column ``` -## timeSeriesGroupSum(uid,タイムスタンプ,値) {#agg-function-timeseriesgroupsum} - -`timeSeriesGroupSum` 総異なる時系列のサンプルのタイムスタンプなアライメントを実施します。 -これは、二つのサンプルタイムスタンプ間の線形補間を使用して、一緒に時系列を合計します。 - -- `uid` 時系列は一意のidですか, `UInt64`. -- `timestamp` ミリ秒またはマイクロ秒をサポートするためにInt64型です。 -- `value` は指標です。 - -この関数は、次のような組の配列を返します `(timestamp, aggregated_value)` ペア。 - -この関数を使用する前に、必ず `timestamp` 昇順です。 - -例: - -``` text -┌─uid─┬─timestamp─┬─value─┐ -│ 1 │ 2 │ 0.2 │ -│ 1 │ 7 │ 0.7 │ -│ 1 │ 12 │ 1.2 │ -│ 1 │ 17 │ 1.7 │ -│ 1 │ 25 │ 2.5 │ -│ 2 │ 3 │ 0.6 │ -│ 2 │ 8 │ 1.6 │ -│ 2 │ 12 │ 2.4 │ -│ 2 │ 18 │ 3.6 │ -│ 2 │ 24 │ 4.8 │ -└─────┴───────────┴───────┘ -``` - -``` sql -CREATE TABLE time_series( - uid UInt64, - timestamp Int64, - value Float64 -) ENGINE = Memory; -INSERT INTO time_series VALUES - (1,2,0.2),(1,7,0.7),(1,12,1.2),(1,17,1.7),(1,25,2.5), - (2,3,0.6),(2,8,1.6),(2,12,2.4),(2,18,3.6),(2,24,4.8); - -SELECT timeSeriesGroupSum(uid, timestamp, value) -FROM ( - SELECT * FROM time_series order by timestamp ASC -); -``` - -結果は次のようになります: - -``` text -[(2,0.2),(3,0.9),(7,2.1),(8,2.4),(12,3.6),(17,5.1),(18,5.4),(24,7.2),(25,2.5)] -``` - -## タイムセリエスグロプラテスム(uid,ts,val) {#agg-function-timeseriesgroupratesum} - -同様に `timeSeriesGroupSum`, `timeSeriesGroupRateSum` 時系列のレートを計算し、レートを合計します。 -また、timestampはこの関数を使用する前に上昇順にする必要があります。 - -のデータにこの関数を適用します。 `timeSeriesGroupSum` 例では、次の結果が得られます: - -``` text -[(2,0),(3,0.1),(7,0.3),(8,0.3),(12,0.3),(17,0.3),(18,0.3),(24,0.3),(25,0.1)] -``` - ## avg(x) {#agg_function-avg} 平均を計算します。 diff --git a/docs/ru/sql-reference/aggregate-functions/reference/index.md b/docs/ru/sql-reference/aggregate-functions/reference/index.md index 4cbe0a0fba4..4c0060581fd 100644 --- a/docs/ru/sql-reference/aggregate-functions/reference/index.md +++ b/docs/ru/sql-reference/aggregate-functions/reference/index.md @@ -45,8 +45,6 @@ toc_hidden: true - [skewPop](../../../sql-reference/aggregate-functions/reference/skewpop.md) - [kurtSamp](../../../sql-reference/aggregate-functions/reference/kurtsamp.md) - [kurtPop](../../../sql-reference/aggregate-functions/reference/kurtpop.md) -- [timeSeriesGroupSum](../../../sql-reference/aggregate-functions/reference/timeseriesgroupsum.md) -- [timeSeriesGroupRateSum](../../../sql-reference/aggregate-functions/reference/timeseriesgroupratesum.md) - [uniq](../../../sql-reference/aggregate-functions/reference/uniq.md) - [uniqExact](../../../sql-reference/aggregate-functions/reference/uniqexact.md) - [uniqCombined](../../../sql-reference/aggregate-functions/reference/uniqcombined.md) diff --git a/docs/ru/sql-reference/aggregate-functions/reference/timeseriesgroupratesum.md b/docs/ru/sql-reference/aggregate-functions/reference/timeseriesgroupratesum.md deleted file mode 100644 index da5935c8f61..00000000000 --- a/docs/ru/sql-reference/aggregate-functions/reference/timeseriesgroupratesum.md +++ /dev/null @@ -1,18 +0,0 @@ ---- -toc_priority: 171 ---- - -# timeSeriesGroupRateSum {#agg-function-timeseriesgroupratesum} - -Синтаксис: `timeSeriesGroupRateSum(uid, ts, val)` - -Аналогично timeSeriesGroupSum, timeSeriesGroupRateSum будет вычислять производные по timestamp для рядов, а затем суммировать полученные производные для всех рядов для одного значения timestamp. -Также ряды должны быть отсортированы по возрастанию timestamp. - -Для пример из описания timeSeriesGroupSum результат будет следующим: - -``` text -[(2,0),(3,0.1),(7,0.3),(8,0.3),(12,0.3),(17,0.3),(18,0.3),(24,0.3),(25,0.1)] -``` - -[Оригинальная статья](https://clickhouse.tech/docs/en/sql-reference/aggregate-functions/reference/timeseriesgroupratesum/) diff --git a/docs/ru/sql-reference/aggregate-functions/reference/timeseriesgroupsum.md b/docs/ru/sql-reference/aggregate-functions/reference/timeseriesgroupsum.md deleted file mode 100644 index 6a34c08c8c5..00000000000 --- a/docs/ru/sql-reference/aggregate-functions/reference/timeseriesgroupsum.md +++ /dev/null @@ -1,59 +0,0 @@ ---- -toc_priority: 170 ---- - -# timeSeriesGroupSum {#agg-function-timeseriesgroupsum} - -Синтаксис: `timeSeriesGroupSum(uid, timestamp, value)` - -`timeSeriesGroupSum` агрегирует временные ряды в которых не совпадают моменты. -Функция использует линейную интерполяцию между двумя значениями времени, а затем суммирует значения для одного и того же момента (как измеренные так и интерполированные) по всем рядам. - -- `uid` уникальный идентификатор временного ряда, `UInt64`. -- `timestamp` имеет тип `Int64` чтобы можно было учитывать милли и микросекунды. -- `value` представляет собой значение метрики. - -Функция возвращает массив кортежей с парами `(timestamp, aggregated_value)`. - -Временные ряды должны быть отсортированы по возрастанию `timestamp`. - -Пример: - -``` text -┌─uid─┬─timestamp─┬─value─┐ -│ 1 │ 2 │ 0.2 │ -│ 1 │ 7 │ 0.7 │ -│ 1 │ 12 │ 1.2 │ -│ 1 │ 17 │ 1.7 │ -│ 1 │ 25 │ 2.5 │ -│ 2 │ 3 │ 0.6 │ -│ 2 │ 8 │ 1.6 │ -│ 2 │ 12 │ 2.4 │ -│ 2 │ 18 │ 3.6 │ -│ 2 │ 24 │ 4.8 │ -└─────┴───────────┴───────┘ -``` - -``` sql -CREATE TABLE time_series( - uid UInt64, - timestamp Int64, - value Float64 -) ENGINE = Memory; -INSERT INTO time_series VALUES - (1,2,0.2),(1,7,0.7),(1,12,1.2),(1,17,1.7),(1,25,2.5), - (2,3,0.6),(2,8,1.6),(2,12,2.4),(2,18,3.6),(2,24,4.8); - -SELECT timeSeriesGroupSum(uid, timestamp, value) -FROM ( - SELECT * FROM time_series order by timestamp ASC -); -``` - -И результат будет: - -``` text -[(2,0.2),(3,0.9),(7,2.1),(8,2.4),(12,3.6),(17,5.1),(18,5.4),(24,7.2),(25,2.5)] -``` - -[Оригинальная статья](https://clickhouse.tech/docs/en/sql-reference/aggregate-functions/reference/timeseriesgroupsum/) diff --git a/docs/tr/sql-reference/aggregate-functions/reference.md b/docs/tr/sql-reference/aggregate-functions/reference.md index 32706b325a9..b606d395cc7 100644 --- a/docs/tr/sql-reference/aggregate-functions/reference.md +++ b/docs/tr/sql-reference/aggregate-functions/reference.md @@ -464,69 +464,6 @@ The kurtosis of the given distribution. Type — [Float64](../../sql-reference/d SELECT kurtSamp(value) FROM series_with_value_column ``` -## timeSeriesGroupSum(uıd, zaman damgası, değer) {#agg-function-timeseriesgroupsum} - -`timeSeriesGroupSum` örnek zaman damgası değil hizalama farklı zaman serileri toplayabilir. -İki örnek zaman damgası arasında doğrusal enterpolasyon kullanacak ve daha sonra zaman serilerini birlikte toplayacaktır. - -- `uid` zaman serisi benzersiz kimliği mi, `UInt64`. -- `timestamp` milisaniye veya mikrosaniye desteklemek için Int64 türüdür. -- `value` metr .iktir. - -İşlev, tuples dizisini döndürür `(timestamp, aggregated_value)` çiftliler. - -Bu işlevi kullanmadan önce emin olun `timestamp` artan düzende. - -Örnek: - -``` text -┌─uid─┬─timestamp─┬─value─┐ -│ 1 │ 2 │ 0.2 │ -│ 1 │ 7 │ 0.7 │ -│ 1 │ 12 │ 1.2 │ -│ 1 │ 17 │ 1.7 │ -│ 1 │ 25 │ 2.5 │ -│ 2 │ 3 │ 0.6 │ -│ 2 │ 8 │ 1.6 │ -│ 2 │ 12 │ 2.4 │ -│ 2 │ 18 │ 3.6 │ -│ 2 │ 24 │ 4.8 │ -└─────┴───────────┴───────┘ -``` - -``` sql -CREATE TABLE time_series( - uid UInt64, - timestamp Int64, - value Float64 -) ENGINE = Memory; -INSERT INTO time_series VALUES - (1,2,0.2),(1,7,0.7),(1,12,1.2),(1,17,1.7),(1,25,2.5), - (2,3,0.6),(2,8,1.6),(2,12,2.4),(2,18,3.6),(2,24,4.8); - -SELECT timeSeriesGroupSum(uid, timestamp, value) -FROM ( - SELECT * FROM time_series order by timestamp ASC -); -``` - -Ve sonuç olacak: - -``` text -[(2,0.2),(3,0.9),(7,2.1),(8,2.4),(12,3.6),(17,5.1),(18,5.4),(24,7.2),(25,2.5)] -``` - -## timeSeriesGroupRateSum(uıd, ts, val) {#agg-function-timeseriesgroupratesum} - -Benzer şekilde `timeSeriesGroupSum`, `timeSeriesGroupRateSum` zaman serilerinin oranını hesaplar ve daha sonra toplam oranları birlikte hesaplar. -Ayrıca, bu işlevi kullanmadan önce zaman damgası yükseliş sırasına göre olmalıdır. - -Bu fonksiyon dataun ver theiye uygulanması `timeSeriesGroupSum` örnek, aşağıdaki sonucu alırsınız: - -``` text -[(2,0),(3,0.1),(7,0.3),(8,0.3),(12,0.3),(17,0.3),(18,0.3),(24,0.3),(25,0.1)] -``` - ## avg (x) {#agg_function-avg} Ortalama hesaplar. diff --git a/docs/zh/sql-reference/aggregate-functions/reference.md b/docs/zh/sql-reference/aggregate-functions/reference.md index 8d1dcda4d83..cf7dddb9b7e 100644 --- a/docs/zh/sql-reference/aggregate-functions/reference.md +++ b/docs/zh/sql-reference/aggregate-functions/reference.md @@ -462,69 +462,6 @@ kurtSamp(expr) SELECT kurtSamp(value) FROM series_with_value_column ``` -## timeSeriesGroupSum(uid,timestamp,value) {#agg-function-timeseriesgroupsum} - -`timeSeriesGroupSum` 可以聚合不同的时间序列,即采样时间戳不对齐。 -它将在两个采样时间戳之间使用线性插值,然后将时间序列和在一起。 - -- `uid` 是时间序列唯一id, `UInt64`. -- `timestamp` 是Int64型,以支持毫秒或微秒。 -- `value` 是指标。 - -函数返回元组数组 `(timestamp, aggregated_value)` 对。 - -在使用此功能之前,请确保 `timestamp` 按升序排列 - -示例: - -``` text -┌─uid─┬─timestamp─┬─value─┐ -│ 1 │ 2 │ 0.2 │ -│ 1 │ 7 │ 0.7 │ -│ 1 │ 12 │ 1.2 │ -│ 1 │ 17 │ 1.7 │ -│ 1 │ 25 │ 2.5 │ -│ 2 │ 3 │ 0.6 │ -│ 2 │ 8 │ 1.6 │ -│ 2 │ 12 │ 2.4 │ -│ 2 │ 18 │ 3.6 │ -│ 2 │ 24 │ 4.8 │ -└─────┴───────────┴───────┘ -``` - -``` sql -CREATE TABLE time_series( - uid UInt64, - timestamp Int64, - value Float64 -) ENGINE = Memory; -INSERT INTO time_series VALUES - (1,2,0.2),(1,7,0.7),(1,12,1.2),(1,17,1.7),(1,25,2.5), - (2,3,0.6),(2,8,1.6),(2,12,2.4),(2,18,3.6),(2,24,4.8); - -SELECT timeSeriesGroupSum(uid, timestamp, value) -FROM ( - SELECT * FROM time_series order by timestamp ASC -); -``` - -其结果将是: - -``` text -[(2,0.2),(3,0.9),(7,2.1),(8,2.4),(12,3.6),(17,5.1),(18,5.4),(24,7.2),(25,2.5)] -``` - -## timeSeriesGroupRateSum(uid,ts,val) {#agg-function-timeseriesgroupratesum} - -同样 `timeSeriesGroupSum`, `timeSeriesGroupRateSum` 计算时间序列的速率,然后将速率总和在一起。 -此外,使用此函数之前,时间戳应该是上升顺序。 - -应用此功能从数据 `timeSeriesGroupSum` 例如,您将得到以下结果: - -``` text -[(2,0),(3,0.1),(7,0.3),(8,0.3),(12,0.3),(17,0.3),(18,0.3),(24,0.3),(25,0.1)] -``` - ## avg(x) {#agg_function-avg} 计算平均值。 From 0a6d93d929423a66fd65f9cd0abf4ab067e5a7ba Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Wed, 11 Nov 2020 21:17:41 +0300 Subject: [PATCH 154/205] Update MergeTreeData.h --- src/Storages/MergeTree/MergeTreeData.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/MergeTree/MergeTreeData.h b/src/Storages/MergeTree/MergeTreeData.h index c5cd21ef597..0e3e5aff4f1 100644 --- a/src/Storages/MergeTree/MergeTreeData.h +++ b/src/Storages/MergeTree/MergeTreeData.h @@ -880,7 +880,7 @@ protected: virtual PartitionCommandsResultInfo attachPartition(const ASTPtr & partition, const StorageMetadataPtr & metadata_snapshot, bool part, const Context & context) = 0; virtual void replacePartitionFrom(const StoragePtr & source_table, const ASTPtr & partition, bool replace, const Context & context) = 0; virtual void movePartitionToTable(const StoragePtr & dest_table, const ASTPtr & partition, const Context & context) = 0; - + /// Makes sense only for replicated tables virtual void fetchPartition(const ASTPtr & partition, const StorageMetadataPtr & metadata_snapshot, const String & from, const Context & query_context); From 719230ecd9390ba279a76843b227ad03476acab4 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Wed, 11 Nov 2020 21:29:05 +0300 Subject: [PATCH 155/205] Disable TraceCollector if trace_log is disabled --- src/Interpreters/Context.cpp | 2 ++ 1 file changed, 2 insertions(+) diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index 73604f93d12..d857c98ffa5 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -451,6 +451,8 @@ struct ContextShared void initializeTraceCollector(std::shared_ptr trace_log) { + if (!trace_log) + return; if (hasTraceCollector()) return; From 8c482b978b9bd4142ce6b993092b713dab0ed627 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Wed, 11 Nov 2020 21:30:17 +0300 Subject: [PATCH 156/205] Set thread name for the TraceCollector --- src/Common/TraceCollector.cpp | 3 +++ 1 file changed, 3 insertions(+) diff --git a/src/Common/TraceCollector.cpp b/src/Common/TraceCollector.cpp index d10d5981d57..1548af50d98 100644 --- a/src/Common/TraceCollector.cpp +++ b/src/Common/TraceCollector.cpp @@ -11,6 +11,7 @@ #include #include #include +#include #include @@ -115,6 +116,8 @@ void TraceCollector::stop() void TraceCollector::run() { + setThreadName("TraceCollector"); + ReadBufferFromFileDescriptor in(pipe.fds_rw[0]); while (true) From fe5800a27ec70b3ee438eb07ab6d03bb728fce0d Mon Sep 17 00:00:00 2001 From: "a.palagashvili" Date: Thu, 12 Nov 2020 02:12:51 +0300 Subject: [PATCH 157/205] remove commented code --- src/IO/LZMADeflatingWriteBuffer.cpp | 3 --- src/IO/LZMADeflatingWriteBuffer.h | 1 - src/IO/LZMAInflatingReadBuffer.cpp | 2 -- src/IO/LZMAInflatingReadBuffer.h | 1 - 4 files changed, 7 deletions(-) diff --git a/src/IO/LZMADeflatingWriteBuffer.cpp b/src/IO/LZMADeflatingWriteBuffer.cpp index 378f7c4ea1b..f309aaa870e 100644 --- a/src/IO/LZMADeflatingWriteBuffer.cpp +++ b/src/IO/LZMADeflatingWriteBuffer.cpp @@ -12,9 +12,6 @@ LZMADeflatingWriteBuffer::LZMADeflatingWriteBuffer( std::unique_ptr out_, int compression_level, size_t buf_size, char * existing_memory, size_t alignment) : BufferWithOwnMemory(buf_size, existing_memory, alignment), out(std::move(out_)) { - // FL2_createCStreamMt(number of threads, flag of two dictionaries usage) - // lstr = FL2_createCStreamMt(2, 0); - // size_t res = FL2_initCStream(lstr, compression_level); lstr = LZMA_STREAM_INIT; lstr.allocator = nullptr; diff --git a/src/IO/LZMADeflatingWriteBuffer.h b/src/IO/LZMADeflatingWriteBuffer.h index bd733bfa49b..085913f024d 100644 --- a/src/IO/LZMADeflatingWriteBuffer.h +++ b/src/IO/LZMADeflatingWriteBuffer.h @@ -27,7 +27,6 @@ private: std::unique_ptr out; lzma_stream lstr; - // FL2_CStream * lstr; bool finished = false; }; } diff --git a/src/IO/LZMAInflatingReadBuffer.cpp b/src/IO/LZMAInflatingReadBuffer.cpp index b8aa1e703cb..30359987390 100644 --- a/src/IO/LZMAInflatingReadBuffer.cpp +++ b/src/IO/LZMAInflatingReadBuffer.cpp @@ -55,10 +55,8 @@ bool LZMAInflatingReadBuffer::nextImpl() lstr.next_out = reinterpret_cast(internal_buffer.begin()); lstr.avail_out = internal_buffer.size(); - // std::cout << lstr.avail_in << " " << lstr.avail_out << std::endl; lzma_ret ret = lzma_code(&lstr, action); - // std::cout << ret << std::endl; in->position() = in->buffer().end() - lstr.avail_in; working_buffer.resize(internal_buffer.size() - lstr.avail_out); diff --git a/src/IO/LZMAInflatingReadBuffer.h b/src/IO/LZMAInflatingReadBuffer.h index 95c0c8f7681..6994b2f50ee 100644 --- a/src/IO/LZMAInflatingReadBuffer.h +++ b/src/IO/LZMAInflatingReadBuffer.h @@ -27,7 +27,6 @@ private: std::unique_ptr in; lzma_stream lstr; - // FL2_DStream * lstr; bool eof; }; From 20d0c5ca676aba420e50dab2ceba5bd27747fba5 Mon Sep 17 00:00:00 2001 From: Robert Hodges Date: Wed, 11 Nov 2020 14:48:22 -0800 Subject: [PATCH 158/205] Add more known ClickHouse users Added new entries to adopters.md along with links to talks/slides. --- docs/en/introduction/adopters.md | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/docs/en/introduction/adopters.md b/docs/en/introduction/adopters.md index b367a97771a..1cffead788a 100644 --- a/docs/en/introduction/adopters.md +++ b/docs/en/introduction/adopters.md @@ -11,6 +11,7 @@ toc_title: Adopters | Company | Industry | Usecase | Cluster Size | (Un)Compressed Data Size\* | Reference | |------------------------------------------------------------------------------------------------|---------------------------------|-----------------------|------------------------------------------------------------|------------------------------------------------------------------------------|-------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------| | 2gis | Maps | Monitoring | — | — | [Talk in Russian, July 2019](https://youtu.be/58sPkXfq6nw) | +| Admiral | Martech | Engagement Management | — | — | [Webinar Slides, June 2020](https://altinity.com/presentations/2020/06/16/big-data-in-real-time-how-clickhouse-powers-admirals-visitor-relationships-for-publishers) | | Alibaba Cloud | Cloud | Managed Service | — | — | [Official Website](https://help.aliyun.com/product/144466.html) | | Aloha Browser | Mobile App | Browser backend | — | — | [Slides in Russian, May 2019](https://presentations.clickhouse.tech/meetup22/aloha.pdf) | | Amadeus | Travel | Analytics | — | — | [Press Release, April 2018](https://www.altinity.com/blog/2018/4/5/amadeus-technologies-launches-investment-and-insights-tool-based-on-machine-learning-and-strategy-algorithms) | @@ -29,6 +30,7 @@ toc_title: Adopters | Citadel Securities | Finance | — | — | — | [Contribution, March 2019](https://github.com/ClickHouse/ClickHouse/pull/4774) | | Citymobil | Taxi | Analytics | — | — | [Blog Post in Russian, March 2020](https://habr.com/en/company/citymobil/blog/490660/) | | Cloudflare | CDN | Traffic analysis | 36 servers | — | [Blog post, May 2017](https://blog.cloudflare.com/how-cloudflare-analyzes-1m-dns-queries-per-second/), [Blog post, March 2018](https://blog.cloudflare.com/http-analytics-for-6m-requests-per-second-using-clickhouse/) | +| Comcast | Media | CDN Traffic Analysis | — | — | [ApacheCon 2019 Talk](https://www.youtube.com/watch?v=e9TZ6gFDjNg) | | ContentSquare | Web analytics | Main product | — | — | [Blog post in French, November 2018](http://souslecapot.net/2018/11/21/patrick-chatain-vp-engineering-chez-contentsquare-penser-davantage-amelioration-continue-que-revolution-constante/) | | Corunet | Analytics | Main product | — | — | [Slides in English, April 2019](https://github.com/ClickHouse/clickhouse-presentations/blob/master/meetup21/predictive_models.pdf) | | CraiditX 氪信 | Finance AI | Analysis | — | — | [Slides in English, November 2019](https://github.com/ClickHouse/clickhouse-presentations/blob/master/meetup33/udf.pptx) | @@ -64,7 +66,8 @@ toc_title: Adopters | Marilyn | Advertising | Statistics | — | — | [Talk in Russian, June 2017](https://www.youtube.com/watch?v=iXlIgx2khwc) | | Mello | Marketing | Analytics | 1 server | — | [Article, Oct 2020](https://vc.ru/marketing/166180-razrabotka-tipovogo-otcheta-skvoznoy-analitiki) | | MessageBird | Telecommunications | Statistics | — | — | [Slides in English, November 2018](https://github.com/ClickHouse/clickhouse-presentations/blob/master/meetup20/messagebird.pdf) | -| MindsDB | Machine Learning | Main Product | — | — | [Official Website](https://www.mindsdb.com/blog/machine-learning-models-as-tables-in-ch) | +| MindsDB | Machine Learning | Main Product | — | — | [Official Website](https://www.mindsdb.com/blog/machine-learning-models-as-tables-in-ch) |x +| MUX | Online Video | Video Analytics | — | — | [Talk in English, August 2019](https://altinity.com/presentations/2019/8/13/how-clickhouse-became-the-default-analytics-database-for-mux/) | | MGID | Ad network | Web-analytics | — | — | [Blog post in Russian, April 2020](http://gs-studio.com/news-about-it/32777----clickhouse---c) | | NOC Project | Network Monitoring | Analytics | Main Product | — | [Official Website](https://getnoc.com/features/big-data/) | | Nuna Inc. | Health Data Analytics | — | — | — | [Talk in English, July 2020](https://youtu.be/GMiXCMFDMow?t=170) | From 94c5f389edc0bf5ac87320795941523ebc1be2cb Mon Sep 17 00:00:00 2001 From: Kang Liu <555836+cnkang@users.noreply.github.com> Date: Thu, 12 Nov 2020 09:36:51 +0800 Subject: [PATCH 159/205] Update build-osx.md Correct the filename and the path translated in Chinese for "limit.maxfiles.plist" --- docs/zh/development/build-osx.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/zh/development/build-osx.md b/docs/zh/development/build-osx.md index 24923f75207..c7c386da97e 100644 --- a/docs/zh/development/build-osx.md +++ b/docs/zh/development/build-osx.md @@ -43,7 +43,7 @@ $ cd .. 为此,请创建以下文件: -/图书馆/LaunchDaemons/限制.maxfilesplist: +/资源库/LaunchDaemons/limit.maxfiles.plist: ``` xml From b13189baec7be984fa91d87f3561070332c247b5 Mon Sep 17 00:00:00 2001 From: zhang2014 Date: Thu, 12 Nov 2020 12:45:08 +0800 Subject: [PATCH 160/205] Try fix fasttest submodule clone --- docker/test/fasttest/run.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docker/test/fasttest/run.sh b/docker/test/fasttest/run.sh index 5b2f324e588..ca5596b91f1 100755 --- a/docker/test/fasttest/run.sh +++ b/docker/test/fasttest/run.sh @@ -127,7 +127,7 @@ function clone_submodules ( cd "$FASTTEST_SOURCE" -SUBMODULES_TO_UPDATE=(contrib/boost contrib/zlib-ng contrib/libxml2 contrib/poco contrib/libunwind contrib/ryu contrib/fmtlib contrib/base64 contrib/cctz contrib/libcpuid contrib/double-conversion contrib/libcxx contrib/libcxxabi contrib/libc-headers contrib/lz4 contrib/zstd contrib/fastops contrib/rapidjson contrib/re2 contrib/sparsehash-c11 contrib/croaring contrib/miniselect) +SUBMODULES_TO_UPDATE=(contrib/boost contrib/zlib-ng contrib/libxml2 contrib/poco contrib/libunwind contrib/ryu contrib/fmtlib contrib/base64 contrib/cctz contrib/libcpuid contrib/double-conversion contrib/libcxx contrib/libcxxabi contrib/libc-headers contrib/lz4 contrib/zstd contrib/fastops contrib/rapidjson contrib/re2 contrib/sparsehash-c11 contrib/croaring contrib/miniselect contrib/xz) git submodule sync git submodule update --init --recursive "${SUBMODULES_TO_UPDATE[@]}" From 037e5a51a37099ccae18873eb4bbf61002f2cc89 Mon Sep 17 00:00:00 2001 From: OuO Date: Thu, 12 Nov 2020 14:43:27 +0800 Subject: [PATCH 161/205] update docs/zh/commercial/support.md --- docs/zh/commercial/support.md | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/docs/zh/commercial/support.md b/docs/zh/commercial/support.md index e8462fc962e..f543338d4e6 100644 --- a/docs/zh/commercial/support.md +++ b/docs/zh/commercial/support.md @@ -2,17 +2,17 @@ machine_translated: true machine_translated_rev: 5decc73b5dc60054f19087d3690c4eb99446a6c3 toc_priority: 3 -toc_title: "\u788C\u83BD\u7984Support:" +toc_title: "\u5546\u4e1a\u652f\u6301" --- # ClickHouse商业支持服务提供商 {#clickhouse-commercial-support-service-providers} !!! info "信息" - 如果您已经推出ClickHouse商业支持服务,请随时 [打开拉取请求](https://github.com/ClickHouse/ClickHouse/edit/master/docs/en/commercial/support.md) 将其添加到以下列表。 + 如果您已经推出ClickHouse商业支持服务,请随时 [提交一个 pull-request](https://github.com/ClickHouse/ClickHouse/edit/master/docs/en/commercial/support.md) 将其添加到以下列表。 -## 敏锐性 {#altinity} +## Altinity {#altinity} -隆隆隆隆路虏脢..陇.貌.垄拢卢虏禄and陇.貌路.隆拢脳枚脢虏 隆隆隆隆路虏脢..陇.貌.垄拢卢虏禄.陇 访问 [www.altinity.com](https://www.altinity.com/) 欲了解更多信息. + Altinity 自从 2017 年开始已经为企业提供 ClickHouse 支持服务。Altinity 的客户范围包含百强企业到初创企业。访问 [www.altinity.com](https://www.altinity.com/) 了解更多信息。 ## Mafiree {#mafiree} From 0d10a4990e748258bd56c3fe98f946f2036531d2 Mon Sep 17 00:00:00 2001 From: Alexander Kuzmenkov Date: Thu, 12 Nov 2020 11:27:33 +0300 Subject: [PATCH 162/205] fix --- tests/performance/trim_urls.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/performance/trim_urls.xml b/tests/performance/trim_urls.xml index a257267d070..7a9eeda6867 100644 --- a/tests/performance/trim_urls.xml +++ b/tests/performance/trim_urls.xml @@ -19,7 +19,7 @@ trim(TRAILING '/' FROM trim(BOTH 'htpsw:/' FROM - + From ec6f092683af532b89b008bfb8d8d8db86c9963a Mon Sep 17 00:00:00 2001 From: filimonov <1549571+filimonov@users.noreply.github.com> Date: Thu, 12 Nov 2020 09:40:26 +0100 Subject: [PATCH 163/205] Update AggregateFunctionTimeSeriesGroupSum.h --- src/AggregateFunctions/AggregateFunctionTimeSeriesGroupSum.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/AggregateFunctions/AggregateFunctionTimeSeriesGroupSum.h b/src/AggregateFunctions/AggregateFunctionTimeSeriesGroupSum.h index 7505ac54169..a1399a89e53 100644 --- a/src/AggregateFunctions/AggregateFunctionTimeSeriesGroupSum.h +++ b/src/AggregateFunctions/AggregateFunctionTimeSeriesGroupSum.h @@ -191,7 +191,7 @@ struct AggregateFunctionTimeSeriesGroupSumData writeVarUInt(size, buf); if (size > 0) { - buf.write(reinterpret_cast(result.data()), sizeof(result[0])); + buf.write(reinterpret_cast(result.data()), size * sizeof(result[0])); } } From 1a0add5a22c2e42b3783ab545b402644c60bfd5a Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 12 Nov 2020 11:58:36 +0300 Subject: [PATCH 164/205] Fix compile expressions --- src/Interpreters/ExpressionActions.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Interpreters/ExpressionActions.cpp b/src/Interpreters/ExpressionActions.cpp index eb1dbb35d89..557f917ec65 100644 --- a/src/Interpreters/ExpressionActions.cpp +++ b/src/Interpreters/ExpressionActions.cpp @@ -1049,8 +1049,8 @@ const ActionsDAG::Node & ActionsDAG::addFunction( settings.max_temporary_non_const_columns = all_settings.max_temporary_non_const_columns; #if USE_EMBEDDED_COMPILER - settings.compile_expressions = settings.compile_expressions; - settings.min_count_to_compile_expression = settings.min_count_to_compile_expression; + settings.compile_expressions = all_settings.compile_expressions; + settings.min_count_to_compile_expression = all_settings.min_count_to_compile_expression; if (!compilation_cache) compilation_cache = context.getCompiledExpressionCache(); From 85a48497560500f04fe7dc9c5f65e9c800a6507d Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 12 Nov 2020 12:21:33 +0300 Subject: [PATCH 165/205] Fty to fix ci --- src/IO/LZMADeflatingWriteBuffer.cpp | 3 +++ src/IO/LZMADeflatingWriteBuffer.h | 34 ++++++++++++++++++++++++++++- src/IO/LZMAInflatingReadBuffer.cpp | 2 ++ src/IO/LZMAInflatingReadBuffer.h | 33 ++++++++++++++++++++++++---- 4 files changed, 67 insertions(+), 5 deletions(-) diff --git a/src/IO/LZMADeflatingWriteBuffer.cpp b/src/IO/LZMADeflatingWriteBuffer.cpp index f309aaa870e..e3051f1de65 100644 --- a/src/IO/LZMADeflatingWriteBuffer.cpp +++ b/src/IO/LZMADeflatingWriteBuffer.cpp @@ -1,5 +1,6 @@ #include +#if !defined(ARCADIA_BUILD) namespace DB { @@ -123,3 +124,5 @@ void LZMADeflatingWriteBuffer::finish() } while (lstr.avail_out == 0); } } + +#endif diff --git a/src/IO/LZMADeflatingWriteBuffer.h b/src/IO/LZMADeflatingWriteBuffer.h index 085913f024d..ca3573bbf8c 100644 --- a/src/IO/LZMADeflatingWriteBuffer.h +++ b/src/IO/LZMADeflatingWriteBuffer.h @@ -3,10 +3,16 @@ #include #include -#include +#if !defined(ARCADIA_BUILD) + #include // Y_IGNORE +#endif + namespace DB { + +#if !defined(ARCADIA_BUILD) + /// Performs compression using lzma library and writes compressed data to out_ WriteBuffer. class LZMADeflatingWriteBuffer : public BufferWithOwnMemory { @@ -29,4 +35,30 @@ private: lzma_stream lstr; bool finished = false; }; + +#else + +namespace ErrorCodes +{ + extern const int NOT_IMPLEMENTED; +} + +class LZMADeflatingWriteBuffer : public BufferWithOwnMemory +{ +public: + LZMADeflatingWriteBuffer( + std::unique_ptr out_ [[maybe_unused]], + int compression_level [[maybe_unused]], + size_t buf_size [[maybe_unused]] = DBMS_DEFAULT_BUFFER_SIZE, + char * existing_memory [[maybe_unused]] = nullptr, + size_t alignment [[maybe_unused]] = 0) + { + throw Exception("LZMADeflatingWriteBuffer is not implemented for arcadia build", ErrorCodes::NOT_IMPLEMENTED); + } + +private: + void nextImpl() override {} +}; + +#endif } diff --git a/src/IO/LZMAInflatingReadBuffer.cpp b/src/IO/LZMAInflatingReadBuffer.cpp index 30359987390..e30e8df5f9d 100644 --- a/src/IO/LZMAInflatingReadBuffer.cpp +++ b/src/IO/LZMAInflatingReadBuffer.cpp @@ -1,5 +1,6 @@ #include +#if !defined(ARCADIA_BUILD) namespace DB { namespace ErrorCodes @@ -87,3 +88,4 @@ bool LZMAInflatingReadBuffer::nextImpl() return true; } } +#endif diff --git a/src/IO/LZMAInflatingReadBuffer.h b/src/IO/LZMAInflatingReadBuffer.h index 6994b2f50ee..d591b6eeb8e 100644 --- a/src/IO/LZMAInflatingReadBuffer.h +++ b/src/IO/LZMAInflatingReadBuffer.h @@ -3,14 +3,14 @@ #include #include -#include +#if !defined(ARCADIA_BUILD) + #include // Y_IGNORE +#endif namespace DB { -namespace ErrorCodes -{ -} +#if !defined(ARCADIA_BUILD) class LZMAInflatingReadBuffer : public BufferWithOwnMemory { public: @@ -30,4 +30,29 @@ private: bool eof; }; + +#else + +namespace ErrorCodes +{ + extern const int NOT_IMPLEMENTED; +} + +class LZMAInflatingReadBuffer : public BufferWithOwnMemory +{ +public: + LZMAInflatingReadBuffer( + std::unique_ptr in_ [[maybe_unused]], + size_t buf_size [[maybe_unused]] = DBMS_DEFAULT_BUFFER_SIZE, + char * existing_memory [[maybe_unused]] = nullptr, + size_t alignment [[maybe_unused]] = 0) + { + throw Exception("LZMADeflatingWriteBuffer is not implemented for arcadia build", ErrorCodes::NOT_IMPLEMENTED); + } + +private: + bool nextImpl() override {} +}; + +#endif } From 75e78026deb604309aebd02f64662e9347aed9ff Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 12 Nov 2020 13:03:10 +0300 Subject: [PATCH 166/205] Fty to fix ci --- src/IO/LZMADeflatingWriteBuffer.h | 3 --- src/IO/LZMAInflatingReadBuffer.h | 3 --- 2 files changed, 6 deletions(-) diff --git a/src/IO/LZMADeflatingWriteBuffer.h b/src/IO/LZMADeflatingWriteBuffer.h index ca3573bbf8c..efa4532d372 100644 --- a/src/IO/LZMADeflatingWriteBuffer.h +++ b/src/IO/LZMADeflatingWriteBuffer.h @@ -55,9 +55,6 @@ public: { throw Exception("LZMADeflatingWriteBuffer is not implemented for arcadia build", ErrorCodes::NOT_IMPLEMENTED); } - -private: - void nextImpl() override {} }; #endif diff --git a/src/IO/LZMAInflatingReadBuffer.h b/src/IO/LZMAInflatingReadBuffer.h index d591b6eeb8e..46bd5d83183 100644 --- a/src/IO/LZMAInflatingReadBuffer.h +++ b/src/IO/LZMAInflatingReadBuffer.h @@ -49,9 +49,6 @@ public: { throw Exception("LZMADeflatingWriteBuffer is not implemented for arcadia build", ErrorCodes::NOT_IMPLEMENTED); } - -private: - bool nextImpl() override {} }; #endif From ab96528a9c2e5dbb5b8715ab7f1073c828483b21 Mon Sep 17 00:00:00 2001 From: annvsh Date: Thu, 12 Nov 2020 19:03:24 +0700 Subject: [PATCH 167/205] Fixed --- docs/en/operations/settings/settings.md | 2 +- docs/ru/operations/settings/settings.md | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index 196402b3f62..39055a8c53b 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -321,7 +321,7 @@ Default value: 0. Consider the table: ```sql -CREATE TABLE table_with_enum_column_for_tsv_insert (Id Int32,Value Enum('first' = 1, 'second' = 2)) ENGINE=Memory() FORMAT TabSeparatedRaw; +CREATE TABLE table_with_enum_column_for_tsv_insert (Id Int32,Value Enum('first' = 1, 'second' = 2)) ENGINE=Memory(); ``` When the `input_format_tsv_enum_as_number` setting is enabled: diff --git a/docs/ru/operations/settings/settings.md b/docs/ru/operations/settings/settings.md index aa4f181ea7b..ec87dcca3a1 100644 --- a/docs/ru/operations/settings/settings.md +++ b/docs/ru/operations/settings/settings.md @@ -305,7 +305,7 @@ Disabled by default. Рассмотрим таблицу: ```sql -CREATE TABLE table_with_enum_column_for_tsv_insert (Id Int32,Value Enum('first' = 1, 'second' = 2)) ENGINE=Memory() FORMAT TabSeparatedRaw; +CREATE TABLE table_with_enum_column_for_tsv_insert (Id Int32,Value Enum('first' = 1, 'second' = 2)) ENGINE=Memory(); ``` При включенной настройке `input_format_tsv_enum_as_number`: From b5ec4ab9d426dc91d47d3adcacdda8abbff98e6b Mon Sep 17 00:00:00 2001 From: olgarev <56617294+olgarev@users.noreply.github.com> Date: Thu, 12 Nov 2020 19:14:47 +0300 Subject: [PATCH 168/205] DOCSUP-2807: Documented the date_trunc function (#15429) * Docs updated for date_trunc() and now(). * Minor fix. * Minor fix in en. Translation to Russian. * Minor link fixes. * Apply suggestions from code review Co-authored-by: BayoNet * Updates * Minor fix * Minor updates for now() (en, ru). Co-authored-by: Olga Revyakina Co-authored-by: BayoNet --- .../functions/date-time-functions.md | 128 ++++++++++++++++-- .../functions/date-time-functions.md | 118 +++++++++++++++- 2 files changed, 229 insertions(+), 17 deletions(-) diff --git a/docs/en/sql-reference/functions/date-time-functions.md b/docs/en/sql-reference/functions/date-time-functions.md index a5a347e553a..5f4d31225b8 100644 --- a/docs/en/sql-reference/functions/date-time-functions.md +++ b/docs/en/sql-reference/functions/date-time-functions.md @@ -337,26 +337,124 @@ SELECT toDate('2016-12-27') AS date, toYearWeek(date) AS yearWeek0, toYearWeek(d └────────────┴───────────┴───────────┴───────────┘ ``` -## date_trunc(datepart, time_or_data\[, time_zone\]), dateTrunc(datepart, time_or_data\[, time_zone\]) {#date_trunc} +## date_trunc {#date_trunc} -Truncates a date or date with time based on the specified datepart, such as -- `second` -- `minute` -- `hour` -- `day` -- `week` -- `month` -- `quarter` -- `year` +Truncates date and time data to the specified part of date. -```sql -SELECT date_trunc('hour', now()) +**Syntax** + +``` sql +date_trunc(unit, value[, timezone]) ``` -## now {#now} +Alias: `dateTrunc`. -Accepts zero or one arguments(timezone) and returns the current time at one of the moments of request execution, or current time of specific timezone at one of the moments of request execution if `timezone` argument provided. -This function returns a constant, even if the request took a long time to complete. +**Parameters** + +- `unit` — Part of date. [String](../syntax.md#syntax-string-literal). + Possible values: + + - `second` + - `minute` + - `hour` + - `day` + - `week` + - `month` + - `quarter` + - `year` + +- `value` — Date and time. [DateTime](../../sql-reference/data-types/datetime.md) or [DateTime64](../../sql-reference/data-types/datetime64.md). +- `timezone` — [Timezone name](../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-timezone) for the returned value (optional). If not specified, the function uses the timezone of the `value` parameter. [String](../../sql-reference/data-types/string.md). + +**Returned value** + +- Value, truncated to the specified part of date. + +Type: [Datetime](../../sql-reference/data-types/datetime.md). + +**Example** + +Query without timezone: + +``` sql +SELECT now(), date_trunc('hour', now()); +``` + +Result: + +``` text +┌───────────────now()─┬─date_trunc('hour', now())─┐ +│ 2020-09-28 10:40:45 │ 2020-09-28 10:00:00 │ +└─────────────────────┴───────────────────────────┘ +``` + +Query with the specified timezone: + +```sql +SELECT now(), date_trunc('hour', now(), 'Europe/Moscow'); +``` + +Result: + +```text +┌───────────────now()─┬─date_trunc('hour', now(), 'Europe/Moscow')─┐ +│ 2020-09-28 10:46:26 │ 2020-09-28 13:00:00 │ +└─────────────────────┴────────────────────────────────────────────┘ +``` + +**See also** + +- [toStartOfInterval](#tostartofintervaltime-or-data-interval-x-unit-time-zone) + +# now {#now} + +Returns the current date and time. + +**Syntax** + +``` sql +now([timezone]) +``` + +**Parameters** + +- `timezone` — [Timezone name](../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-timezone) for the returned value (optional). [String](../../sql-reference/data-types/string.md). + +**Returned value** + +- Current date and time. + +Type: [Datetime](../../sql-reference/data-types/datetime.md). + +**Example** + +Query without timezone: + +``` sql +SELECT now(); +``` + +Result: + +``` text +┌───────────────now()─┐ +│ 2020-10-17 07:42:09 │ +└─────────────────────┘ +``` + +Query with the specified timezone: + +``` sql +SELECT now('Europe/Moscow'); +``` + +Result: + +``` text +┌─now('Europe/Moscow')─┐ +│ 2020-10-17 10:42:23 │ +└──────────────────────┘ +``` ## today {#today} diff --git a/docs/ru/sql-reference/functions/date-time-functions.md b/docs/ru/sql-reference/functions/date-time-functions.md index d24de2faae1..deffc935870 100644 --- a/docs/ru/sql-reference/functions/date-time-functions.md +++ b/docs/ru/sql-reference/functions/date-time-functions.md @@ -234,10 +234,124 @@ WITH toDateTime64('2020-01-01 10:20:30.999', 3) AS dt64 SELECT toStartOfSecond(d Переводит дату-с-временем в номер секунды, начиная с некоторого фиксированного момента в прошлом. +## date_trunc {#date_trunc} + +Отсекает от даты и времени части, меньшие чем указанная часть. + +**Синтаксис** + +``` sql +date_trunc(unit, value[, timezone]) +``` + +Синоним: `dateTrunc`. + +**Параметры** + +- `unit` — Название части даты или времени. [String](../syntax.md#syntax-string-literal). + Возможные значения: + + - `second` + - `minute` + - `hour` + - `day` + - `week` + - `month` + - `quarter` + - `year` + +- `value` — Дата и время. [DateTime](../../sql-reference/data-types/datetime.md) или [DateTime64](../../sql-reference/data-types/datetime64.md). +- `timezone` — [Часовой пояс](../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-timezone) для возвращаемого значения (необязательно). Если параметр не задан, используется часовой пояс параметра `value`. [String](../../sql-reference/data-types/string.md) + +**Возвращаемое значение** + +- Дата и время, отсеченные до указанной части. + +Тип: [Datetime](../../sql-reference/data-types/datetime.md). + +**Примеры** + +Запрос без указания часового пояса: + +``` sql +SELECT now(), date_trunc('hour', now()); +``` + +Результат: + +``` text +┌───────────────now()─┬─date_trunc('hour', now())─┐ +│ 2020-09-28 10:40:45 │ 2020-09-28 10:00:00 │ +└─────────────────────┴───────────────────────────┘ +``` + +Запрос с указанием часового пояса: + +```sql +SELECT now(), date_trunc('hour', now(), 'Europe/Moscow'); +``` + +Результат: + +```text +┌───────────────now()─┬─date_trunc('hour', now(), 'Europe/Moscow')─┐ +│ 2020-09-28 10:46:26 │ 2020-09-28 13:00:00 │ +└─────────────────────┴────────────────────────────────────────────┘ +``` + +**См. также** + +- [toStartOfInterval](#tostartofintervaltime-or-data-interval-x-unit-time-zone) + ## now {#now} -Принимает ноль аргументов и возвращает текущее время на один из моментов выполнения запроса. -Функция возвращает константу, даже если запрос выполнялся долго. +Возвращает текущую дату и время. + +**Синтаксис** + +``` sql +now([timezone]) +``` + +**Параметры** + +- `timezone` — [часовой пояс](../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-timezone) для возвращаемого значения (необязательно). [String](../../sql-reference/data-types/string.md) + +**Возвращаемое значение** + +- Текущие дата и время. + +Тип: [Datetime](../../sql-reference/data-types/datetime.md). + +**Пример** + +Запрос без указания часового пояса: + +``` sql +SELECT now(); +``` + +Результат: + +``` text +┌───────────────now()─┐ +│ 2020-10-17 07:42:09 │ +└─────────────────────┘ +``` + +Запрос с указанием часового пояса: + +``` sql +SELECT now('Europe/Moscow'); +``` + +Результат: + +``` text +┌─now('Europe/Moscow')─┐ +│ 2020-10-17 10:42:23 │ +└──────────────────────┘ +``` ## today {#today} From f48232d615df2385b2959743453f8650e6893e09 Mon Sep 17 00:00:00 2001 From: gyuton <40863448+gyuton@users.noreply.github.com> Date: Thu, 12 Nov 2020 19:23:32 +0300 Subject: [PATCH 169/205] DOCSUP-3535: Documented the reinterpretAsUUID function (updated) (#16688) * Updated description * Updated description * Added translation * Fixed links * Fixed links 2 * Fixed mistakes in translation * Improved description and added examples * Minor fixes Co-authored-by: George --- .../functions/type-conversion-functions.md | 54 +++++++++++++++++- .../functions/type-conversion-functions.md | 56 +++++++++++++++++++ 2 files changed, 109 insertions(+), 1 deletion(-) diff --git a/docs/en/sql-reference/functions/type-conversion-functions.md b/docs/en/sql-reference/functions/type-conversion-functions.md index fc066a34b0b..f8458b27a22 100644 --- a/docs/en/sql-reference/functions/type-conversion-functions.md +++ b/docs/en/sql-reference/functions/type-conversion-functions.md @@ -325,7 +325,59 @@ This function accepts a number or date or date with time, and returns a FixedStr ## reinterpretAsUUID {#reinterpretasuuid} -This function accepts FixedString, and returns UUID. Takes 16 bytes string. If the string isn't long enough, the functions work as if the string is padded with the necessary number of null bytes to the end. If the string longer than 16 bytes, the extra bytes at the end are ignored. +This function accepts 16 bytes string, and returns UUID containing bytes representing the corresponding value in network byte order (big-endian). If the string isn't long enough, the functions work as if the string is padded with the necessary number of null bytes to the end. If the string longer than 16 bytes, the extra bytes at the end are ignored. + +**Syntax** + +``` sql +reinterpretAsUUID(fixed_string) +``` + +**Parameters** + +- `fixed_string` — Big-endian byte string. [FixedString](../../sql-reference/data-types/fixedstring.md#fixedstring). + +**Returned value** + +- The UUID type value. [UUID](../../sql-reference/data-types/uuid.md#uuid-data-type). + +**Examples** + +String to UUID. + +Query: + +``` sql +SELECT reinterpretAsUUID(reverse(unhex('000102030405060708090a0b0c0d0e0f'))) +``` + +Result: + +``` text +┌─reinterpretAsUUID(reverse(unhex('000102030405060708090a0b0c0d0e0f')))─┐ +│ 08090a0b-0c0d-0e0f-0001-020304050607 │ +└───────────────────────────────────────────────────────────────────────┘ +``` + +Going back and forth from String to UUID. + +Query: + +``` sql +WITH + generateUUIDv4() AS uuid, + identity(lower(hex(reverse(reinterpretAsString(uuid))))) AS str, + reinterpretAsUUID(reverse(unhex(str))) AS uuid2 +SELECT uuid = uuid2; +``` + +Result: + +``` text +┌─equals(uuid, uuid2)─┐ +│ 1 │ +└─────────────────────┘ +``` ## CAST(x, T) {#type_conversion_function-cast} diff --git a/docs/ru/sql-reference/functions/type-conversion-functions.md b/docs/ru/sql-reference/functions/type-conversion-functions.md index 3733e570f10..4a314bd22d8 100644 --- a/docs/ru/sql-reference/functions/type-conversion-functions.md +++ b/docs/ru/sql-reference/functions/type-conversion-functions.md @@ -319,6 +319,62 @@ SELECT toFixedString('foo\0bar', 8) AS s, toStringCutToZero(s) AS s_cut Функция принимает число или дату или дату-с-временем и возвращает строку, содержащую байты, представляющие соответствующее значение в host order (little endian). При этом, отбрасываются нулевые байты с конца. Например, значение 255 типа UInt32 будет строкой длины 1 байт. +## reinterpretAsUUID {#reinterpretasuuid} + +Функция принимает шестнадцатибайтную строку и интерпретирует ее байты в network order (big-endian). Если строка имеет недостаточную длину, то функция работает так, как будто строка дополнена необходимым количетсвом нулевых байт с конца. Если строка длиннее, чем шестнадцать байт, то игнорируются лишние байты с конца. + +**Синтаксис** + +``` sql +reinterpretAsUUID(fixed_string) +``` + +**Параметры** + +- `fixed_string` — cтрока с big-endian порядком байтов. [FixedString](../../sql-reference/data-types/fixedstring.md#fixedstring). + +**Возвращаемое значение** + +- Значение типа [UUID](../../sql-reference/data-types/uuid.md#uuid-data-type). + +**Примеры** + +Интерпретация строки как UUID. + +Запрос: + +``` sql +SELECT reinterpretAsUUID(reverse(unhex('000102030405060708090a0b0c0d0e0f'))) +``` + +Результат: + +``` text +┌─reinterpretAsUUID(reverse(unhex('000102030405060708090a0b0c0d0e0f')))─┐ +│ 08090a0b-0c0d-0e0f-0001-020304050607 │ +└───────────────────────────────────────────────────────────────────────┘ +``` + +Переход в UUID и обратно. + +Запрос: + +``` sql +WITH + generateUUIDv4() AS uuid, + identity(lower(hex(reverse(reinterpretAsString(uuid))))) AS str, + reinterpretAsUUID(reverse(unhex(str))) AS uuid2 +SELECT uuid = uuid2; +``` + +Результат: + +``` text +┌─equals(uuid, uuid2)─┐ +│ 1 │ +└─────────────────────┘ +``` + ## CAST(x, T) {#type_conversion_function-cast} Преобразует x в тип данных t. From 26f6ce3fb528a73a0abac7541b6af3380a690b8a Mon Sep 17 00:00:00 2001 From: filimonov <1549571+filimonov@users.noreply.github.com> Date: Thu, 12 Nov 2020 17:48:17 +0100 Subject: [PATCH 170/205] Update Install.cpp --- programs/install/Install.cpp | 66 ++++++++++++++++++++---------------- 1 file changed, 37 insertions(+), 29 deletions(-) diff --git a/programs/install/Install.cpp b/programs/install/Install.cpp index 8290118089c..4574b68b2f6 100644 --- a/programs/install/Install.cpp +++ b/programs/install/Install.cpp @@ -329,14 +329,20 @@ int mainEntryClickHouseInstall(int argc, char ** argv) bool has_password_for_default_user = false; - if (!fs::exists(main_config_file)) + if (!fs::exists(config_d)) { fmt::print("Creating config directory {} that is used for tweaks of main server configuration.\n", config_d.string()); fs::create_directory(config_d); + } + if (!fs::exists(users_d)) + { fmt::print("Creating config directory {} that is used for tweaks of users configuration.\n", users_d.string()); fs::create_directory(users_d); + } + if (!fs::exists(main_config_file)) + { std::string_view main_config_content = getResource("config.xml"); if (main_config_content.empty()) { @@ -349,7 +355,30 @@ int mainEntryClickHouseInstall(int argc, char ** argv) out.sync(); out.finalize(); } + } + else + { + fmt::print("Config file {} already exists, will keep it and extract path info from it.\n", main_config_file.string()); + ConfigProcessor processor(main_config_file.string(), /* throw_on_bad_incl = */ false, /* log_to_console = */ false); + ConfigurationPtr configuration(new Poco::Util::XMLConfiguration(processor.processConfig())); + + if (configuration->has("path")) + { + data_path = configuration->getString("path"); + fmt::print("{} has {} as data path.\n", main_config_file.string(), data_path); + } + + if (configuration->has("logger.log")) + { + log_path = fs::path(configuration->getString("logger.log")).remove_filename(); + fmt::print("{} has {} as log path.\n", main_config_file.string(), log_path); + } + } + + + if (!fs::exists(users_config_file)) + { std::string_view users_config_content = getResource("users.xml"); if (users_config_content.empty()) { @@ -365,38 +394,17 @@ int mainEntryClickHouseInstall(int argc, char ** argv) } else { - { - fmt::print("Config file {} already exists, will keep it and extract path info from it.\n", main_config_file.string()); - - ConfigProcessor processor(main_config_file.string(), /* throw_on_bad_incl = */ false, /* log_to_console = */ false); - ConfigurationPtr configuration(new Poco::Util::XMLConfiguration(processor.processConfig())); - - if (configuration->has("path")) - { - data_path = configuration->getString("path"); - fmt::print("{} has {} as data path.\n", main_config_file.string(), data_path); - } - - if (configuration->has("logger.log")) - { - log_path = fs::path(configuration->getString("logger.log")).remove_filename(); - fmt::print("{} has {} as log path.\n", main_config_file.string(), log_path); - } - } + fmt::print("Users config file {} already exists, will keep it and extract path info from it.\n", main_config_file.string()); /// Check if password for default user already specified. + ConfigProcessor processor(users_config_file.string(), /* throw_on_bad_incl = */ false, /* log_to_console = */ false); + ConfigurationPtr configuration(new Poco::Util::XMLConfiguration(processor.processConfig())); - if (fs::exists(users_config_file)) + if (!configuration->getString("users.default.password", "").empty() + || configuration->getString("users.default.password_sha256_hex", "").empty() + || configuration->getString("users.default.password_double_sha1_hex", "").empty()) { - ConfigProcessor processor(users_config_file.string(), /* throw_on_bad_incl = */ false, /* log_to_console = */ false); - ConfigurationPtr configuration(new Poco::Util::XMLConfiguration(processor.processConfig())); - - if (!configuration->getString("users.default.password", "").empty() - || configuration->getString("users.default.password_sha256_hex", "").empty() - || configuration->getString("users.default.password_double_sha1_hex", "").empty()) - { - has_password_for_default_user = true; - } + has_password_for_default_user = true; } } From 385e8f5ee063532b47826a6fd63310cfa5196808 Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Thu, 12 Nov 2020 19:52:58 +0300 Subject: [PATCH 171/205] Update Install.cpp --- programs/install/Install.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/programs/install/Install.cpp b/programs/install/Install.cpp index 4574b68b2f6..6cb37c36b41 100644 --- a/programs/install/Install.cpp +++ b/programs/install/Install.cpp @@ -394,7 +394,7 @@ int mainEntryClickHouseInstall(int argc, char ** argv) } else { - fmt::print("Users config file {} already exists, will keep it and extract path info from it.\n", main_config_file.string()); + fmt::print("Users config file {} already exists, will keep it and extract users info from it.\n", main_config_file.string()); /// Check if password for default user already specified. ConfigProcessor processor(users_config_file.string(), /* throw_on_bad_incl = */ false, /* log_to_console = */ false); From 8eff47420bf07f44ac54bfe1a7e7caf18fda038c Mon Sep 17 00:00:00 2001 From: filimonov <1549571+filimonov@users.noreply.github.com> Date: Thu, 12 Nov 2020 18:18:34 +0100 Subject: [PATCH 172/205] Update Connection.cpp --- src/Client/Connection.cpp | 1 + 1 file changed, 1 insertion(+) diff --git a/src/Client/Connection.cpp b/src/Client/Connection.cpp index 31e88b5d872..a16f35e060f 100644 --- a/src/Client/Connection.cpp +++ b/src/Client/Connection.cpp @@ -73,6 +73,7 @@ void Connection::connect(const ConnectionTimeouts & timeouts) { #if USE_SSL socket = std::make_unique(); + socket->setPeerHostName(host); #else throw Exception{"tcp_secure protocol is disabled because poco library was built without NetSSL support.", ErrorCodes::SUPPORT_IS_DISABLED}; #endif From 8a20133373e1475cac3eb124d605ff91741f3755 Mon Sep 17 00:00:00 2001 From: filimonov <1549571+filimonov@users.noreply.github.com> Date: Thu, 12 Nov 2020 18:28:24 +0100 Subject: [PATCH 173/205] style --- programs/install/Install.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/programs/install/Install.cpp b/programs/install/Install.cpp index 6cb37c36b41..27f4ceec732 100644 --- a/programs/install/Install.cpp +++ b/programs/install/Install.cpp @@ -373,7 +373,7 @@ int mainEntryClickHouseInstall(int argc, char ** argv) { log_path = fs::path(configuration->getString("logger.log")).remove_filename(); fmt::print("{} has {} as log path.\n", main_config_file.string(), log_path); - } + } } From 4cecd89e84547427b8a1121b0e2d223f39088f40 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Thu, 12 Nov 2020 20:36:02 +0300 Subject: [PATCH 174/205] fix dropping of empty parts --- src/Storages/MergeTree/MergeTreeData.cpp | 3 +- src/Storages/MergeTree/MergeTreeData.h | 2 +- .../MergeTree/ReplicatedMergeTreeQueue.cpp | 3 +- src/Storages/StorageMergeTree.cpp | 6 ++- src/Storages/StorageMergeTree.h | 2 +- src/Storages/StorageReplicatedMergeTree.cpp | 44 ++++++++++++++----- src/Storages/StorageReplicatedMergeTree.h | 4 +- tests/integration/test_ttl_replicated/test.py | 6 +++ 8 files changed, 50 insertions(+), 20 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index aa8318d256d..6ff72cfe478 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -1223,7 +1223,8 @@ void MergeTreeData::clearEmptyParts() if (part->rows_count == 0) { ASTPtr literal = std::make_shared(part->name); - dropPartition(literal, /* detach = */ false, /*drop_part = */ true, global_context); + /// If another replica has already started drop, it's ok, no need to throw. + dropPartition(literal, /* detach = */ false, /*drop_part = */ true, global_context, /* throw_if_noop = */ false); } } } diff --git a/src/Storages/MergeTree/MergeTreeData.h b/src/Storages/MergeTree/MergeTreeData.h index 12a5557e2db..34faafb3b1f 100644 --- a/src/Storages/MergeTree/MergeTreeData.h +++ b/src/Storages/MergeTree/MergeTreeData.h @@ -878,7 +878,7 @@ protected: // Partition helpers bool canReplacePartition(const DataPartPtr & src_part) const; - virtual void dropPartition(const ASTPtr & partition, bool detach, bool drop_part, const Context & context) = 0; + virtual void dropPartition(const ASTPtr & partition, bool detach, bool drop_part, const Context & context, bool throw_if_noop = true) = 0; virtual PartitionCommandsResultInfo attachPartition(const ASTPtr & partition, const StorageMetadataPtr & metadata_snapshot, bool part, const Context & context) = 0; virtual void replacePartitionFrom(const StoragePtr & source_table, const ASTPtr & partition, bool replace, const Context & context) = 0; virtual void movePartitionToTable(const StoragePtr & dest_table, const ASTPtr & partition, const Context & context) = 0; diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp index 9263dab638a..7a76c6c8246 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp @@ -54,7 +54,8 @@ void ReplicatedMergeTreeQueue::addVirtualParts(const MergeTreeData::DataParts & bool ReplicatedMergeTreeQueue::isVirtualPart(const MergeTreeData::DataPartPtr & data_part) const { std::lock_guard lock(state_mutex); - return virtual_parts.getContainingPart(data_part->info) != data_part->name; + auto virtual_part_name = virtual_parts.getContainingPart(data_part->info); + return !virtual_part_name.empty() && virtual_part_name != data_part->name; } diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index 5978436d93b..4d4b35ce7c4 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -1089,7 +1089,7 @@ ActionLock StorageMergeTree::stopMergesAndWait() } -void StorageMergeTree::dropPartition(const ASTPtr & partition, bool detach, bool drop_part, const Context & context) +void StorageMergeTree::dropPartition(const ASTPtr & partition, bool detach, bool drop_part, const Context & context, bool throw_if_noop) { { /// Asks to complete merges and does not allow them to start. @@ -1107,8 +1107,10 @@ void StorageMergeTree::dropPartition(const ASTPtr & partition, bool detach, bool if (part) parts_to_remove.push_back(part); - else + else if (throw_if_noop) throw Exception("Part " + part_name + " not found, won't try to drop it.", ErrorCodes::NO_SUCH_DATA_PART); + else + return; } else { diff --git a/src/Storages/StorageMergeTree.h b/src/Storages/StorageMergeTree.h index e3796cb9d10..86d33e25df0 100644 --- a/src/Storages/StorageMergeTree.h +++ b/src/Storages/StorageMergeTree.h @@ -187,7 +187,7 @@ private: void clearOldMutations(bool truncate = false); // Partition helpers - void dropPartition(const ASTPtr & partition, bool detach, bool drop_part, const Context & context) override; + void dropPartition(const ASTPtr & partition, bool detach, bool drop_part, const Context & context, bool throw_if_noop) override; PartitionCommandsResultInfo attachPartition(const ASTPtr & partition, const StorageMetadataPtr & metadata_snapshot, bool part, const Context & context) override; void replacePartitionFrom(const StoragePtr & source_table, const ASTPtr & partition, bool replace, const Context & context) override; diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index b93500000b5..48a8a80c781 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -4230,7 +4230,7 @@ bool StorageReplicatedMergeTree::getFakePartCoveringAllPartsInPartition(const St } -void StorageReplicatedMergeTree::dropPartition(const ASTPtr & partition, bool detach, bool drop_part, const Context & query_context) +void StorageReplicatedMergeTree::dropPartition(const ASTPtr & partition, bool detach, bool drop_part, const Context & query_context, bool throw_if_noop) { assertNotReadonly(); if (!is_leader) @@ -4244,7 +4244,7 @@ void StorageReplicatedMergeTree::dropPartition(const ASTPtr & partition, bool de if (drop_part) { String part_name = partition->as().value.safeGet(); - did_drop = dropPart(zookeeper, part_name, entry, detach); + did_drop = dropPart(zookeeper, part_name, entry, detach, throw_if_noop); } else { @@ -5937,7 +5937,7 @@ bool StorageReplicatedMergeTree::waitForShrinkingQueueSize(size_t queue_size, UI } bool StorageReplicatedMergeTree::dropPart( - zkutil::ZooKeeperPtr & zookeeper, String part_name, LogEntry & entry, bool detach) + zkutil::ZooKeeperPtr & zookeeper, String part_name, LogEntry & entry, bool detach, bool throw_if_noop) { LOG_TRACE(log, "Will try to insert a log entry to DROP_RANGE for part: " + part_name); @@ -5950,32 +5950,52 @@ bool StorageReplicatedMergeTree::dropPart( auto part = getPartIfExists(part_info, {MergeTreeDataPartState::Committed}); if (!part) - throw Exception("Part " + part_name + " not found locally, won't try to drop it.", ErrorCodes::NO_SUCH_DATA_PART); + { + if (throw_if_noop) + throw Exception("Part " + part_name + " not found locally, won't try to drop it.", ErrorCodes::NO_SUCH_DATA_PART); + return false; + } /// There isn't a lot we can do otherwise. Can't cancel merges because it is possible that a replica already /// finished the merge. if (partIsAssignedToBackgroundOperation(part)) - throw Exception("Part " + part_name - + " is currently participating in a background operation (mutation/merge)" - + ", try again later", ErrorCodes::PART_IS_TEMPORARILY_LOCKED); + { + if (throw_if_noop) + throw Exception("Part " + part_name + + " is currently participating in a background operation (mutation/merge)" + + ", try again later", ErrorCodes::PART_IS_TEMPORARILY_LOCKED); + return false; + } if (partIsLastQuorumPart(part->info)) - throw Exception("Part " + part_name + " is last inserted part with quorum in partition. Cannot drop", - ErrorCodes::NOT_IMPLEMENTED); + { + if (throw_if_noop) + throw Exception("Part " + part_name + " is last inserted part with quorum in partition. Cannot drop", + ErrorCodes::NOT_IMPLEMENTED); + return false; + } if (partIsInsertingWithParallelQuorum(part->info)) - throw Exception("Part " + part_name + " is inserting with parallel quorum. Cannot drop", - ErrorCodes::NOT_IMPLEMENTED); + { + if (throw_if_noop) + throw Exception("Part " + part_name + " is inserting with parallel quorum. Cannot drop", + ErrorCodes::NOT_IMPLEMENTED); + return false; + } Coordination::Requests ops; getClearBlocksInPartitionOps(ops, *zookeeper, part_info.partition_id, part_info.min_block, part_info.max_block); size_t clean_block_ops_size = ops.size(); + /// Set fake level to treat this part as virtual in queue. + auto drop_part_info = part->info; + drop_part_info.level = MergeTreePartInfo::MAX_LEVEL; + /// If `part_name` is result of a recent merge and source parts are still available then /// DROP_RANGE with detach will move this part together with source parts to `detached/` dir. entry.type = LogEntry::DROP_RANGE; entry.source_replica = replica_name; - entry.new_part_name = part_name; + entry.new_part_name = drop_part_info.getPartName(); entry.detach = detach; entry.create_time = time(nullptr); diff --git a/src/Storages/StorageReplicatedMergeTree.h b/src/Storages/StorageReplicatedMergeTree.h index 3e6b809cfdf..d3328f9843c 100644 --- a/src/Storages/StorageReplicatedMergeTree.h +++ b/src/Storages/StorageReplicatedMergeTree.h @@ -548,12 +548,12 @@ private: /// Info about how other replicas can access this one. ReplicatedMergeTreeAddress getReplicatedMergeTreeAddress() const; - bool dropPart(zkutil::ZooKeeperPtr & zookeeper, String part_name, LogEntry & entry, bool detach); + bool dropPart(zkutil::ZooKeeperPtr & zookeeper, String part_name, LogEntry & entry, bool detach, bool throw_if_noop); bool dropAllPartsInPartition( zkutil::ZooKeeper & zookeeper, String & partition_id, LogEntry & entry, bool detach); // Partition helpers - void dropPartition(const ASTPtr & partition, bool detach, bool drop_part, const Context & query_context) override; + void dropPartition(const ASTPtr & partition, bool detach, bool drop_part, const Context & query_context, bool throw_if_noop) override; PartitionCommandsResultInfo attachPartition(const ASTPtr & partition, const StorageMetadataPtr & metadata_snapshot, bool part, const Context & query_context) override; void replacePartitionFrom(const StoragePtr & source_table, const ASTPtr & partition, bool replace, const Context & query_context) override; void movePartitionToTable(const StoragePtr & dest_table, const ASTPtr & partition, const Context & query_context) override; diff --git a/tests/integration/test_ttl_replicated/test.py b/tests/integration/test_ttl_replicated/test.py index 2cb384d10a0..e98b3ba5625 100644 --- a/tests/integration/test_ttl_replicated/test.py +++ b/tests/integration/test_ttl_replicated/test.py @@ -323,3 +323,9 @@ def test_ttl_empty_parts(started_cluster): optimize_with_retry(node1, 'test_ttl_empty_parts') assert node1.query("SELECT name FROM system.parts WHERE table = 'test_ttl_empty_parts' AND active ORDER BY name") == "all_0_7_2_8\n" + + node2.query('SYSTEM SYNC REPLICA test_ttl_empty_parts', timeout=20) + + error_msg = ' default.test_ttl_empty_parts (ReplicatedMergeTreeCleanupThread)' + assert not node1.contains_in_log(error_msg) + assert not node2.contains_in_log(error_msg) From 178d8e9b75d154e5a835106dc3c20b8bed3f8f39 Mon Sep 17 00:00:00 2001 From: filimonov <1549571+filimonov@users.noreply.github.com> Date: Thu, 12 Nov 2020 22:07:19 +0100 Subject: [PATCH 175/205] Update Connection.cpp --- src/Client/Connection.cpp | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/Client/Connection.cpp b/src/Client/Connection.cpp index a16f35e060f..2a130f4ee6f 100644 --- a/src/Client/Connection.cpp +++ b/src/Client/Connection.cpp @@ -73,7 +73,8 @@ void Connection::connect(const ConnectionTimeouts & timeouts) { #if USE_SSL socket = std::make_unique(); - socket->setPeerHostName(host); + auto secure_socket = static_cast(socket.get()); + secure_socket->setPeerHostName(host); #else throw Exception{"tcp_secure protocol is disabled because poco library was built without NetSSL support.", ErrorCodes::SUPPORT_IS_DISABLED}; #endif From c0940f2ebd487f7f2001aa258a2912a88d6bd68a Mon Sep 17 00:00:00 2001 From: nikitamikhaylov Date: Fri, 13 Nov 2020 02:27:18 +0300 Subject: [PATCH 176/205] done --- src/Core/Defines.h | 2 +- src/Interpreters/ActionsVisitor.cpp | 2 +- src/Interpreters/DatabaseCatalog.cpp | 2 +- src/Interpreters/executeQuery.cpp | 4 ++-- src/Storages/MergeTree/BackgroundJobsExecutor.cpp | 2 +- src/Storages/MergeTree/BackgroundJobsExecutor.h | 2 +- src/Storages/MergeTree/MergeTreeData.cpp | 2 +- src/Storages/MergeTree/MergeTreeIndexConditionBloomFilter.cpp | 2 +- 8 files changed, 9 insertions(+), 9 deletions(-) diff --git a/src/Core/Defines.h b/src/Core/Defines.h index 4d7d8e08ac3..9b6578092c9 100644 --- a/src/Core/Defines.h +++ b/src/Core/Defines.h @@ -70,7 +70,7 @@ /// Minimum revision supporting OpenTelemetry #define DBMS_MIN_REVISION_WITH_OPENTELEMETRY 54442 -/// Mininum revision supporting interserver secret. +/// Minimum revision supporting interserver secret. #define DBMS_MIN_REVISION_WITH_INTERSERVER_SECRET 54441 /// Version of ClickHouse TCP protocol. Increment it manually when you change the protocol. diff --git a/src/Interpreters/ActionsVisitor.cpp b/src/Interpreters/ActionsVisitor.cpp index 67ef37ba319..5d4da8c5c3e 100644 --- a/src/Interpreters/ActionsVisitor.cpp +++ b/src/Interpreters/ActionsVisitor.cpp @@ -1051,7 +1051,7 @@ SetPtr ActionsMatcher::makeSet(const ASTFunction & node, Data & data, bool no_su * - this function shows the expression IN_data1. * * In case that we have HAVING with IN subquery, we have to force creating set for it. - * Also it doesn't make sence if it is GLOBAL IN or ordinary IN. + * Also it doesn't make sense if it is GLOBAL IN or ordinary IN. */ if (!subquery_for_set.source && data.create_source_for_in) { diff --git a/src/Interpreters/DatabaseCatalog.cpp b/src/Interpreters/DatabaseCatalog.cpp index 9de938ae184..03ec4ccb1eb 100644 --- a/src/Interpreters/DatabaseCatalog.cpp +++ b/src/Interpreters/DatabaseCatalog.cpp @@ -167,7 +167,7 @@ void DatabaseCatalog::shutdownImpl() std::lock_guard lock(databases_mutex); assert(std::find_if(uuid_map.begin(), uuid_map.end(), [](const auto & elem) { - /// Ensure that all UUID mappings are emtpy (i.e. all mappings contain nullptr instead of a pointer to storage) + /// Ensure that all UUID mappings are empty (i.e. all mappings contain nullptr instead of a pointer to storage) const auto & not_empty_mapping = [] (const auto & mapping) { auto & table = mapping.second.second; diff --git a/src/Interpreters/executeQuery.cpp b/src/Interpreters/executeQuery.cpp index cdb3d9b7d7b..c41abd3c7f2 100644 --- a/src/Interpreters/executeQuery.cpp +++ b/src/Interpreters/executeQuery.cpp @@ -259,7 +259,7 @@ static void onExceptionBeforeStart(const String & query_for_logging, Context & c span.finish_time_us = current_time_us; span.duration_ns = 0; - // keep values synchonized to type enum in QueryLogElement::createBlock + /// Keep values synchronized to type enum in QueryLogElement::createBlock. span.attribute_names.push_back("clickhouse.query_status"); span.attribute_values.push_back("ExceptionBeforeStart"); @@ -697,7 +697,7 @@ static std::tuple executeQueryImpl( span.finish_time_us = time_in_microseconds(finish_time); span.duration_ns = elapsed_seconds * 1000000000; - // keep values synchonized to type enum in QueryLogElement::createBlock + /// Keep values synchronized to type enum in QueryLogElement::createBlock. span.attribute_names.push_back("clickhouse.query_status"); span.attribute_values.push_back("QueryFinish"); diff --git a/src/Storages/MergeTree/BackgroundJobsExecutor.cpp b/src/Storages/MergeTree/BackgroundJobsExecutor.cpp index 5aba208a86e..0a23cfb1338 100644 --- a/src/Storages/MergeTree/BackgroundJobsExecutor.cpp +++ b/src/Storages/MergeTree/BackgroundJobsExecutor.cpp @@ -114,7 +114,7 @@ try } }); /// We've scheduled task in the background pool and when it will finish we will be triggered again. But this task can be - /// extremely long and we may have a lot of other small tasks to do, so we schedule ourselfs here. + /// extremely long and we may have a lot of other small tasks to do, so we schedule ourselves here. scheduleTask(true); } catch (...) diff --git a/src/Storages/MergeTree/BackgroundJobsExecutor.h b/src/Storages/MergeTree/BackgroundJobsExecutor.h index 537a6064c86..ab9fc118ec4 100644 --- a/src/Storages/MergeTree/BackgroundJobsExecutor.h +++ b/src/Storages/MergeTree/BackgroundJobsExecutor.h @@ -22,7 +22,7 @@ struct BackgroundTaskSchedulingSettings double task_sleep_seconds_when_no_work_random_part = 1.0; - /// deprected settings, don't affect background execution + /// Deprecated settings, don't affect background execution double thread_sleep_seconds = 10; double task_sleep_seconds_when_no_work_min = 10; }; diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 175bc72b08d..07d881d5fe9 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -2648,7 +2648,7 @@ void MergeTreeData::checkPartCanBeDropped(const ASTPtr & part_ast) String part_name = part_ast->as().value.safeGet(); auto part = getPartIfExists(part_name, {MergeTreeDataPartState::Committed}); if (!part) - throw Exception(ErrorCodes::NO_SUCH_DATA_PART, "No part {} in commited state", part_name); + throw Exception(ErrorCodes::NO_SUCH_DATA_PART, "No part {} in committed state", part_name); auto table_id = getStorageID(); global_context.checkPartitionCanBeDropped(table_id.database_name, table_id.table_name, part->getBytesOnDisk()); diff --git a/src/Storages/MergeTree/MergeTreeIndexConditionBloomFilter.cpp b/src/Storages/MergeTree/MergeTreeIndexConditionBloomFilter.cpp index fa0aa03e820..a9915f01645 100644 --- a/src/Storages/MergeTree/MergeTreeIndexConditionBloomFilter.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexConditionBloomFilter.cpp @@ -345,7 +345,7 @@ static bool indexOfCanUseBloomFilter(const ASTPtr & parent) if (function->arguments->children.size() != 2) return false; - /// We don't allow constant expressions like `indexOf(arr, x) = 1 + 0` but it's neglible. + /// We don't allow constant expressions like `indexOf(arr, x) = 1 + 0` but it's negligible. /// We should return true when the corresponding expression implies that the array contains the element. /// Example: when `indexOf(arr, x)` > 10 is written, it means that arr definitely should contain the element From 7d1c0f5b28e45ebe37aa38d8fa8daa06e20c10e5 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 13 Nov 2020 04:36:34 +0300 Subject: [PATCH 177/205] Fixed bug in git-import tool --- programs/git-import/git-import.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/programs/git-import/git-import.cpp b/programs/git-import/git-import.cpp index 7cdd77b4b7c..b6ea2c7baf4 100644 --- a/programs/git-import/git-import.cpp +++ b/programs/git-import/git-import.cpp @@ -680,7 +680,7 @@ void updateSnapshot(Snapshot & snapshot, const Commit & commit, CommitDiff & fil for (auto & elem : file_changes) { auto & file = elem.second.file_change; - if (file.path != file.old_path) + if (!file.old_path.empty() && file.path != file.old_path) snapshot[file.path] = snapshot[file.old_path]; } From fdf979d5ce37abfea4609e514daf9a2b900be512 Mon Sep 17 00:00:00 2001 From: Amos Bird Date: Fri, 13 Nov 2020 11:43:35 +0800 Subject: [PATCH 178/205] Fix missing conversion of unmutated parts --- .../MergeTree/MergeTreeRangeReader.cpp | 38 ++++++++++--------- 1 file changed, 20 insertions(+), 18 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeRangeReader.cpp b/src/Storages/MergeTree/MergeTreeRangeReader.cpp index 726e405b3e8..ecf0d59b42b 100644 --- a/src/Storages/MergeTree/MergeTreeRangeReader.cpp +++ b/src/Storages/MergeTree/MergeTreeRangeReader.cpp @@ -635,30 +635,32 @@ MergeTreeRangeReader::ReadResult MergeTreeRangeReader::read(size_t max_rows, Mar merge_tree_reader->fillMissingColumns(columns, should_evaluate_missing_defaults, num_rows); } - if (!columns.empty() && should_evaluate_missing_defaults) + if (!columns.empty()) { - auto block = prev_reader->sample_block.cloneWithColumns(read_result.columns); - auto block_before_prewhere = read_result.block_before_prewhere; - for (auto & ctn : block) + if (should_evaluate_missing_defaults) { - if (block_before_prewhere.has(ctn.name)) - block_before_prewhere.erase(ctn.name); - } - - if (block_before_prewhere) - { - if (read_result.need_filter) + auto block = prev_reader->sample_block.cloneWithColumns(read_result.columns); + auto block_before_prewhere = read_result.block_before_prewhere; + for (auto & ctn : block) { - auto old_columns = block_before_prewhere.getColumns(); - filterColumns(old_columns, read_result.getFilterOriginal()->getData()); - block_before_prewhere.setColumns(std::move(old_columns)); + if (block_before_prewhere.has(ctn.name)) + block_before_prewhere.erase(ctn.name); } - for (auto && ctn : block_before_prewhere) - block.insert(std::move(ctn)); - } + if (block_before_prewhere) + { + if (read_result.need_filter) + { + auto old_columns = block_before_prewhere.getColumns(); + filterColumns(old_columns, read_result.getFilterOriginal()->getData()); + block_before_prewhere.setColumns(std::move(old_columns)); + } - merge_tree_reader->evaluateMissingDefaults(block, columns); + for (auto && ctn : block_before_prewhere) + block.insert(std::move(ctn)); + } + merge_tree_reader->evaluateMissingDefaults(block, columns); + } merge_tree_reader->performRequiredConversions(columns); } From e4dd68039ea85d7c0023d1231c2f890a10ddb650 Mon Sep 17 00:00:00 2001 From: Alexander Kuzmenkov Date: Fri, 13 Nov 2020 08:00:43 +0300 Subject: [PATCH 179/205] fix --- tests/performance/trim_urls.xml | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/performance/trim_urls.xml b/tests/performance/trim_urls.xml index 7a9eeda6867..4f8238b5e9a 100644 --- a/tests/performance/trim_urls.xml +++ b/tests/performance/trim_urls.xml @@ -23,6 +23,6 @@ - SELECT ignore({func_fast}URL)) FROM hits_10m_single FORMAT Null - SELECT ignore({func_slow}URL)) FROM hits_100m_single FORMAT Null + SELECT ignore({func_fast}URL)) FROM hits_100m_single FORMAT Null + SELECT ignore({func_slow}URL)) FROM hits_10m_single FORMAT Null From 5169ae577e0d45f0fce0f707bd76f0af72604e96 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Fri, 13 Nov 2020 08:17:43 +0300 Subject: [PATCH 180/205] Update version_date.tsv after release 20.11.3.3 --- utils/list-versions/version_date.tsv | 1 + 1 file changed, 1 insertion(+) diff --git a/utils/list-versions/version_date.tsv b/utils/list-versions/version_date.tsv index 47f6aefe406..6957cc33d87 100644 --- a/utils/list-versions/version_date.tsv +++ b/utils/list-versions/version_date.tsv @@ -1,3 +1,4 @@ +v20.11.3.3-stable 2020-11-13 v20.11.2.1-stable 2020-11-11 v20.10.3.30-stable 2020-10-29 v20.10.2.20-stable 2020-10-23 From 26572e9c6f3aefa9d86cdd57ee628c9548cce568 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Fri, 13 Nov 2020 09:03:13 +0300 Subject: [PATCH 181/205] Update version_date.tsv after release 20.10.4.1 --- utils/list-versions/version_date.tsv | 1 + 1 file changed, 1 insertion(+) diff --git a/utils/list-versions/version_date.tsv b/utils/list-versions/version_date.tsv index 6957cc33d87..f5d22827f6d 100644 --- a/utils/list-versions/version_date.tsv +++ b/utils/list-versions/version_date.tsv @@ -1,5 +1,6 @@ v20.11.3.3-stable 2020-11-13 v20.11.2.1-stable 2020-11-11 +v20.10.4.1-stable 2020-11-13 v20.10.3.30-stable 2020-10-29 v20.10.2.20-stable 2020-10-23 v20.9.4.76-stable 2020-10-29 From 2175fea0acdd08a7187daadd5ed97b846ae9a11a Mon Sep 17 00:00:00 2001 From: Alexander Kuzmenkov Date: Fri, 13 Nov 2020 09:28:36 +0300 Subject: [PATCH 182/205] Some changelogs --- CHANGELOG.md | 192 ++++++++++++++++++++++ utils/simple-backport/format-changelog.py | 5 +- 2 files changed, 196 insertions(+), 1 deletion(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index 4474675e9ee..555c9f28534 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -1,3 +1,12 @@ +## ClickHouse release 20.11 + +### ClickHouse release v20.11.3.3-stable, 2020-11-13 + +#### Bug Fix + +* Fix rare silent crashes when query profiler is on and ClickHouse is installed on OS with glibc version that has (supposedly) broken asynchronous unwind tables for some functions. This fixes [#15301](https://github.com/ClickHouse/ClickHouse/issues/15301). This fixes [#13098](https://github.com/ClickHouse/ClickHouse/issues/13098). [#16846](https://github.com/ClickHouse/ClickHouse/pull/16846) ([alexey-milovidov](https://github.com/alexey-milovidov)). + + ### ClickHouse release v20.11.2.1, 2020-11-11 #### Backward Incompatible Change @@ -119,6 +128,24 @@ ## ClickHouse release 20.10 +### ClickHouse release v20.10.4.1-stable, 2020-11-13 + +#### Bug Fix + +* Fix rare silent crashes when query profiler is on and ClickHouse is installed on OS with glibc version that has (supposedly) broken asynchronous unwind tables for some functions. This fixes [#15301](https://github.com/ClickHouse/ClickHouse/issues/15301). This fixes [#13098](https://github.com/ClickHouse/ClickHouse/issues/13098). [#16846](https://github.com/ClickHouse/ClickHouse/pull/16846) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fix `IN` operator over several columns and tuples with enabled `transform_null_in` setting. Fixes [#15310](https://github.com/ClickHouse/ClickHouse/issues/15310). [#16722](https://github.com/ClickHouse/ClickHouse/pull/16722) ([Anton Popov](https://github.com/CurtizJ)). +* This will fix optimize_read_in_order/optimize_aggregation_in_order with max_threads>0 and expression in ORDER BY. [#16637](https://github.com/ClickHouse/ClickHouse/pull/16637) ([Azat Khuzhin](https://github.com/azat)). +* Now when parsing AVRO from input the LowCardinality is removed from type. Fixes [#16188](https://github.com/ClickHouse/ClickHouse/issues/16188). [#16521](https://github.com/ClickHouse/ClickHouse/pull/16521) ([Mike](https://github.com/myrrc)). +* Fix rapid growth of metadata when using MySQL Master -> MySQL Slave -> ClickHouse MaterializeMySQL Engine, and `slave_parallel_worker` enabled on MySQL Slave, by properly shrinking GTID sets. This fixes [#15951](https://github.com/ClickHouse/ClickHouse/issues/15951). [#16504](https://github.com/ClickHouse/ClickHouse/pull/16504) ([TCeason](https://github.com/TCeason)). +* Fix DROP TABLE for Distributed (racy with INSERT). [#16409](https://github.com/ClickHouse/ClickHouse/pull/16409) ([Azat Khuzhin](https://github.com/azat)). +* Fix processing of very large entries in replication queue. Very large entries may appear in ALTER queries if table structure is extremely large (near 1 MB). This fixes [#16307](https://github.com/ClickHouse/ClickHouse/issues/16307). [#16332](https://github.com/ClickHouse/ClickHouse/pull/16332) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fix bug with MySQL database. When MySQL server used as database engine is down some queries raise Exception, because they try to get tables from disabled server, while it's unnecessary. For example, query `SELECT ... FROM system.parts` should work only with MergeTree tables and don't touch MySQL database at all. [#16032](https://github.com/ClickHouse/ClickHouse/pull/16032) ([Kruglov Pavel](https://github.com/Avogar)). + +#### Improvement + +* Workaround for use S3 with nginx server as proxy. Nginx currenty does not accept urls with empty path like http://domain.com?delete, but vanilla aws-sdk-cpp produces this kind of urls. This commit uses patched aws-sdk-cpp version, which makes urls with "/" as path in this cases, like http://domain.com/?delete. [#16813](https://github.com/ClickHouse/ClickHouse/pull/16813) ([ianton-ru](https://github.com/ianton-ru)). + + ### ClickHouse release v20.10.3.30, 2020-10-28 #### Backward Incompatible Change @@ -331,6 +358,76 @@ ## ClickHouse release 20.9 +### ClickHouse release v20.9.4.76-stable (2020-10-29) + +#### Bug Fix + +* Fix double free in case of exception in function `dictGet`. It could have happened if dictionary was loaded with error. [#16429](https://github.com/ClickHouse/ClickHouse/pull/16429) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fix group by with totals/rollup/cube modifers and min/max functions over group by keys. Fixes [#16393](https://github.com/ClickHouse/ClickHouse/issues/16393). [#16397](https://github.com/ClickHouse/ClickHouse/pull/16397) ([Anton Popov](https://github.com/CurtizJ)). +* Fix async Distributed INSERT w/ prefer_localhost_replica=0 and internal_replication. [#16358](https://github.com/ClickHouse/ClickHouse/pull/16358) ([Azat Khuzhin](https://github.com/azat)). +* Fix a very wrong code in TwoLevelStringHashTable implementation, which might lead to memory leak. I'm suprised how this bug can lurk for so long.... [#16264](https://github.com/ClickHouse/ClickHouse/pull/16264) ([Amos Bird](https://github.com/amosbird)). +* Fix the case when memory can be overallocated regardless to the limit. This closes [#14560](https://github.com/ClickHouse/ClickHouse/issues/14560). [#16206](https://github.com/ClickHouse/ClickHouse/pull/16206) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Not for changelog. [#16031](https://github.com/ClickHouse/ClickHouse/pull/16031) ([tavplubix](https://github.com/tavplubix)). +* Fix `ALTER MODIFY ... ORDER BY` query hang for `ReplicatedVersionedCollapsingMergeTree`. This fixes [#15980](https://github.com/ClickHouse/ClickHouse/issues/15980). [#16011](https://github.com/ClickHouse/ClickHouse/pull/16011) ([alesapin](https://github.com/alesapin)). +* Fix collate name & charset name parser and support `length = 0` for string type. [#16008](https://github.com/ClickHouse/ClickHouse/pull/16008) ([Winter Zhang](https://github.com/zhang2014)). +* Allow to use direct layout for dictionaries with complex keys. [#16007](https://github.com/ClickHouse/ClickHouse/pull/16007) ([Anton Popov](https://github.com/CurtizJ)). +* Prevent replica hang for 5-10 mins when replication error happens after a period of inactivity. [#15987](https://github.com/ClickHouse/ClickHouse/pull/15987) ([filimonov](https://github.com/filimonov)). +* Fix rare segfaults when inserting into or selecting from MaterializedView and concurrently dropping target table (for Atomic database engine). [#15984](https://github.com/ClickHouse/ClickHouse/pull/15984) ([tavplubix](https://github.com/tavplubix)). +* Fix ambiguity in parsing of settings profiles: `CREATE USER ... SETTINGS profile readonly` is now considered as using a profile named `readonly`, not a setting named `profile` with the readonly constraint. This fixes https://github.com/ClickHouse/ClickHouse/issues/15628. [#15982](https://github.com/ClickHouse/ClickHouse/pull/15982) ([Vitaly Baranov](https://github.com/vitlibar)). +* Fix a crash when database creation fails. [#15954](https://github.com/ClickHouse/ClickHouse/pull/15954) ([Winter Zhang](https://github.com/zhang2014)). +* Fixed `DROP TABLE IF EXISTS` failure with `Table ... doesn't exist` error when table is concurrently renamed (for Atomic database engine). Fixed rare deadlock when concurrently executing some DDL queries with multiple tables (like `DROP DATABASE` and `RENAME TABLE`) Fixed `DROP/DETACH DATABASE` failure with `Table ... doesn't exist` when concurrently executing `DROP/DETACH TABLE`. [#15934](https://github.com/ClickHouse/ClickHouse/pull/15934) ([tavplubix](https://github.com/tavplubix)). +* Fix incorrect empty result for query from `Distributed` table if query has `WHERE`, `PREWHERE` and `GLOBAL IN`. Fixes [#15792](https://github.com/ClickHouse/ClickHouse/issues/15792). [#15933](https://github.com/ClickHouse/ClickHouse/pull/15933) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fix possible deadlocks in RBAC. [#15875](https://github.com/ClickHouse/ClickHouse/pull/15875) ([Vitaly Baranov](https://github.com/vitlibar)). +* Fix exception `Block structure mismatch` in `SELECT ... ORDER BY DESC` queries which were executed after `ALTER MODIFY COLUMN` query. Fixes [#15800](https://github.com/ClickHouse/ClickHouse/issues/15800). [#15852](https://github.com/ClickHouse/ClickHouse/pull/15852) ([alesapin](https://github.com/alesapin)). +* Fix `select count()` inaccuracy for MaterializeMySQL. [#15767](https://github.com/ClickHouse/ClickHouse/pull/15767) ([tavplubix](https://github.com/tavplubix)). +* Fix some cases of queries, in which only virtual columns are selected. Previously `Not found column _nothing in block` exception may be thrown. Fixes [#12298](https://github.com/ClickHouse/ClickHouse/issues/12298). [#15756](https://github.com/ClickHouse/ClickHouse/pull/15756) ([Anton Popov](https://github.com/CurtizJ)). +* Fixed too low default value of `max_replicated_logs_to_keep` setting, which might cause replicas to become lost too often. Improve lost replica recovery process by choosing the most up-to-date replica to clone. Also do not remove old parts from lost replica, detach them instead. [#15701](https://github.com/ClickHouse/ClickHouse/pull/15701) ([tavplubix](https://github.com/tavplubix)). +* Fix error `Cannot add simple transform to empty Pipe` which happened while reading from `Buffer` table which has different structure than destination table. It was possible if destination table returned empty result for query. Fixes [#15529](https://github.com/ClickHouse/ClickHouse/issues/15529). [#15662](https://github.com/ClickHouse/ClickHouse/pull/15662) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fixed bug with globs in S3 table function, region from URL was not applied to S3 client configuration. [#15646](https://github.com/ClickHouse/ClickHouse/pull/15646) ([Vladimir Chebotarev](https://github.com/excitoon)). +* Decrement the `ReadonlyReplica` metric when detaching read-only tables. This fixes https://github.com/ClickHouse/ClickHouse/issues/15598. [#15592](https://github.com/ClickHouse/ClickHouse/pull/15592) ([sundyli](https://github.com/sundy-li)). +* Throw an error when a single parameter is passed to ReplicatedMergeTree instead of ignoring it. [#15516](https://github.com/ClickHouse/ClickHouse/pull/15516) ([nvartolomei](https://github.com/nvartolomei)). + +#### Improvement + +* Now it's allowed to execute `ALTER ... ON CLUSTER` queries regardless of the `` setting in cluster config. [#16075](https://github.com/ClickHouse/ClickHouse/pull/16075) ([alesapin](https://github.com/alesapin)). +* Unfold `{database}`, `{table}` and `{uuid}` macros in `zookeeper_path` on replicated table creation. Do not allow `RENAME TABLE` if it may break `zookeeper_path` after server restart. Fixes [#6917](https://github.com/ClickHouse/ClickHouse/issues/6917). [#15348](https://github.com/ClickHouse/ClickHouse/pull/15348) ([tavplubix](https://github.com/tavplubix)). + +#### Other + +* Unfold `{database}`, `{table}` and `{uuid}` macros in `ReplicatedMergeTree` arguments on table creation. [#16160](https://github.com/ClickHouse/ClickHouse/pull/16160) ([tavplubix](https://github.com/tavplubix)). + + +### ClickHouse release v20.9.3.45-stable (2020-10-09) + +#### Bug Fix + +* Fix error `Cannot find column` which may happen at insertion into `MATERIALIZED VIEW` in case if query for `MV` containes `ARRAY JOIN`. [#15717](https://github.com/ClickHouse/ClickHouse/pull/15717) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fix race condition in AMQP-CPP. [#15667](https://github.com/ClickHouse/ClickHouse/pull/15667) ([alesapin](https://github.com/alesapin)). +* Fix the order of destruction for resources in `ReadFromStorage` step of query plan. It might cause crashes in rare cases. Possibly connected with [#15610](https://github.com/ClickHouse/ClickHouse/issues/15610). [#15645](https://github.com/ClickHouse/ClickHouse/pull/15645) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fixed `Element ... is not a constant expression` error when using `JSON*` function result in `VALUES`, `LIMIT` or right side of `IN` operator. [#15589](https://github.com/ClickHouse/ClickHouse/pull/15589) ([tavplubix](https://github.com/tavplubix)). +* Prevent the possibility of error message `Could not calculate available disk space (statvfs), errno: 4, strerror: Interrupted system call`. This fixes [#15541](https://github.com/ClickHouse/ClickHouse/issues/15541). [#15557](https://github.com/ClickHouse/ClickHouse/pull/15557) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Significantly reduce memory usage in AggregatingInOrderTransform/optimize_aggregation_in_order. [#15543](https://github.com/ClickHouse/ClickHouse/pull/15543) ([Azat Khuzhin](https://github.com/azat)). +* Mutation might hang waiting for some non-existent part after `MOVE` or `REPLACE PARTITION` or, in rare cases, after `DETACH` or `DROP PARTITION`. It's fixed. [#15537](https://github.com/ClickHouse/ClickHouse/pull/15537) ([tavplubix](https://github.com/tavplubix)). +* Fix bug when `ILIKE` operator stops being case insensitive if `LIKE` with the same pattern was executed. [#15536](https://github.com/ClickHouse/ClickHouse/pull/15536) ([alesapin](https://github.com/alesapin)). +* Fix `Missing columns` errors when selecting columns which absent in data, but depend on other columns which also absent in data. Fixes [#15530](https://github.com/ClickHouse/ClickHouse/issues/15530). [#15532](https://github.com/ClickHouse/ClickHouse/pull/15532) ([alesapin](https://github.com/alesapin)). +* Fix bug with event subscription in DDLWorker which rarely may lead to query hangs in `ON CLUSTER`. Introduced in [#13450](https://github.com/ClickHouse/ClickHouse/issues/13450). [#15477](https://github.com/ClickHouse/ClickHouse/pull/15477) ([alesapin](https://github.com/alesapin)). +* Report proper error when the second argument of `boundingRatio` aggregate function has a wrong type. [#15407](https://github.com/ClickHouse/ClickHouse/pull/15407) ([detailyang](https://github.com/detailyang)). +* Fix bug where queries like SELECT toStartOfDay(today()) fail complaining about empty time_zone argument. [#15319](https://github.com/ClickHouse/ClickHouse/pull/15319) ([Bharat Nallan](https://github.com/bharatnc)). +* Fix race condition during MergeTree table rename and background cleanup. [#15304](https://github.com/ClickHouse/ClickHouse/pull/15304) ([alesapin](https://github.com/alesapin)). +* Fix rare race condition on server startup when system.logs are enabled. [#15300](https://github.com/ClickHouse/ClickHouse/pull/15300) ([alesapin](https://github.com/alesapin)). +* Fix MSan report in QueryLog. Uninitialized memory can be used for the field `memory_usage`. [#15258](https://github.com/ClickHouse/ClickHouse/pull/15258) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fix instance crash when using joinGet with LowCardinality types. This fixes https://github.com/ClickHouse/ClickHouse/issues/15214. [#15220](https://github.com/ClickHouse/ClickHouse/pull/15220) ([Amos Bird](https://github.com/amosbird)). +* Fix bug in table engine `Buffer` which doesn't allow to insert data of new structure into `Buffer` after `ALTER` query. Fixes [#15117](https://github.com/ClickHouse/ClickHouse/issues/15117). [#15192](https://github.com/ClickHouse/ClickHouse/pull/15192) ([alesapin](https://github.com/alesapin)). +* Adjust decimals field size in mysql column definition packet. [#15152](https://github.com/ClickHouse/ClickHouse/pull/15152) ([maqroll](https://github.com/maqroll)). +* Fixed `Cannot rename ... errno: 22, strerror: Invalid argument` error on DDL query execution in Atomic database when running clickhouse-server in docker on Mac OS. [#15024](https://github.com/ClickHouse/ClickHouse/pull/15024) ([tavplubix](https://github.com/tavplubix)). +* Fix to make predicate push down work when subquery contains finalizeAggregation function. Fixes [#14847](https://github.com/ClickHouse/ClickHouse/issues/14847). [#14937](https://github.com/ClickHouse/ClickHouse/pull/14937) ([filimonov](https://github.com/filimonov)). +* Fix a problem where the server may get stuck on startup while talking to ZooKeeper, if the configuration files have to be fetched from ZK (using the `from_zk` include option). This fixes [#14814](https://github.com/ClickHouse/ClickHouse/issues/14814). [#14843](https://github.com/ClickHouse/ClickHouse/pull/14843) ([Alexander Kuzmenkov](https://github.com/akuzm)). + +#### Improvement + +* Now it's possible to change the type of version column for `VersionedCollapsingMergeTree` with `ALTER` query. [#15442](https://github.com/ClickHouse/ClickHouse/pull/15442) ([alesapin](https://github.com/alesapin)). + + ### ClickHouse release v20.9.2.20, 2020-09-22 #### New Feature @@ -405,6 +502,101 @@ ## ClickHouse release 20.8 +### ClickHouse release v20.8.5.45-lts, 2020-10-29 + +#### Bug Fix + +* Fix double free in case of exception in function `dictGet`. It could have happened if dictionary was loaded with error. [#16429](https://github.com/ClickHouse/ClickHouse/pull/16429) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fix group by with totals/rollup/cube modifers and min/max functions over group by keys. Fixes [#16393](https://github.com/ClickHouse/ClickHouse/issues/16393). [#16397](https://github.com/ClickHouse/ClickHouse/pull/16397) ([Anton Popov](https://github.com/CurtizJ)). +* Fix async Distributed INSERT w/ prefer_localhost_replica=0 and internal_replication. [#16358](https://github.com/ClickHouse/ClickHouse/pull/16358) ([Azat Khuzhin](https://github.com/azat)). +* Fix a possible memory leak during `GROUP BY` with string keys, caused by an error in `TwoLevelStringHashTable` implementation. [#16264](https://github.com/ClickHouse/ClickHouse/pull/16264) ([Amos Bird](https://github.com/amosbird)). +* Fix the case when memory can be overallocated regardless to the limit. This closes [#14560](https://github.com/ClickHouse/ClickHouse/issues/14560). [#16206](https://github.com/ClickHouse/ClickHouse/pull/16206) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fix `ALTER MODIFY ... ORDER BY` query hang for `ReplicatedVersionedCollapsingMergeTree`. This fixes [#15980](https://github.com/ClickHouse/ClickHouse/issues/15980). [#16011](https://github.com/ClickHouse/ClickHouse/pull/16011) ([alesapin](https://github.com/alesapin)). +* Fix collate name & charset name parser and support `length = 0` for string type. [#16008](https://github.com/ClickHouse/ClickHouse/pull/16008) ([Winter Zhang](https://github.com/zhang2014)). +* Allow to use direct layout for dictionaries with complex keys. [#16007](https://github.com/ClickHouse/ClickHouse/pull/16007) ([Anton Popov](https://github.com/CurtizJ)). +* Prevent replica hang for 5-10 mins when replication error happens after a period of inactivity. [#15987](https://github.com/ClickHouse/ClickHouse/pull/15987) ([filimonov](https://github.com/filimonov)). +* Fix rare segfaults when inserting into or selecting from MaterializedView and concurrently dropping target table (for Atomic database engine). [#15984](https://github.com/ClickHouse/ClickHouse/pull/15984) ([tavplubix](https://github.com/tavplubix)). +* Fix ambiguity in parsing of settings profiles: `CREATE USER ... SETTINGS profile readonly` is now considered as using a profile named `readonly`, not a setting named `profile` with the readonly constraint. This fixes [#15628](https://github.com/ClickHouse/ClickHouse/issues/15628). [#15982](https://github.com/ClickHouse/ClickHouse/pull/15982) ([Vitaly Baranov](https://github.com/vitlibar)). +* Fix a crash when database creation fails. [#15954](https://github.com/ClickHouse/ClickHouse/pull/15954) ([Winter Zhang](https://github.com/zhang2014)). +* Fixed `DROP TABLE IF EXISTS` failure with `Table ... doesn't exist` error when table is concurrently renamed (for Atomic database engine). Fixed rare deadlock when concurrently executing some DDL queries with multiple tables (like `DROP DATABASE` and `RENAME TABLE`) Fixed `DROP/DETACH DATABASE` failure with `Table ... doesn't exist` when concurrently executing `DROP/DETACH TABLE`. [#15934](https://github.com/ClickHouse/ClickHouse/pull/15934) ([tavplubix](https://github.com/tavplubix)). +* Fix incorrect empty result for query from `Distributed` table if query has `WHERE`, `PREWHERE` and `GLOBAL IN`. Fixes [#15792](https://github.com/ClickHouse/ClickHouse/issues/15792). [#15933](https://github.com/ClickHouse/ClickHouse/pull/15933) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fix possible deadlocks in RBAC. [#15875](https://github.com/ClickHouse/ClickHouse/pull/15875) ([Vitaly Baranov](https://github.com/vitlibar)). +* Fix exception `Block structure mismatch` in `SELECT ... ORDER BY DESC` queries which were executed after `ALTER MODIFY COLUMN` query. Fixes [#15800](https://github.com/ClickHouse/ClickHouse/issues/15800). [#15852](https://github.com/ClickHouse/ClickHouse/pull/15852) ([alesapin](https://github.com/alesapin)). +* Fix some cases of queries, in which only virtual columns are selected. Previously `Not found column _nothing in block` exception may be thrown. Fixes [#12298](https://github.com/ClickHouse/ClickHouse/issues/12298). [#15756](https://github.com/ClickHouse/ClickHouse/pull/15756) ([Anton Popov](https://github.com/CurtizJ)). +* Fix error `Cannot find column` which may happen at insertion into `MATERIALIZED VIEW` in case if query for `MV` containes `ARRAY JOIN`. [#15717](https://github.com/ClickHouse/ClickHouse/pull/15717) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fixed too low default value of `max_replicated_logs_to_keep` setting, which might cause replicas to become lost too often. Improve lost replica recovery process by choosing the most up-to-date replica to clone. Also do not remove old parts from lost replica, detach them instead. [#15701](https://github.com/ClickHouse/ClickHouse/pull/15701) ([tavplubix](https://github.com/tavplubix)). +* Fix error `Cannot add simple transform to empty Pipe` which happened while reading from `Buffer` table which has different structure than destination table. It was possible if destination table returned empty result for query. Fixes [#15529](https://github.com/ClickHouse/ClickHouse/issues/15529). [#15662](https://github.com/ClickHouse/ClickHouse/pull/15662) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fixed bug with globs in S3 table function, region from URL was not applied to S3 client configuration. [#15646](https://github.com/ClickHouse/ClickHouse/pull/15646) ([Vladimir Chebotarev](https://github.com/excitoon)). +* Decrement the `ReadonlyReplica` metric when detaching read-only tables. This fixes [#15598](https://github.com/ClickHouse/ClickHouse/issues/15598). [#15592](https://github.com/ClickHouse/ClickHouse/pull/15592) ([sundyli](https://github.com/sundy-li)). +* Throw an error when a single parameter is passed to ReplicatedMergeTree instead of ignoring it. [#15516](https://github.com/ClickHouse/ClickHouse/pull/15516) ([nvartolomei](https://github.com/nvartolomei)). + +#### Improvement + +* Now it's allowed to execute `ALTER ... ON CLUSTER` queries regardless of the `` setting in cluster config. [#16075](https://github.com/ClickHouse/ClickHouse/pull/16075) ([alesapin](https://github.com/alesapin)). +* Unfold `{database}`, `{table}` and `{uuid}` macros in `zookeeper_path` on replicated table creation. Do not allow `RENAME TABLE` if it may break `zookeeper_path` after server restart. Fixes [#6917](https://github.com/ClickHouse/ClickHouse/issues/6917). [#15348](https://github.com/ClickHouse/ClickHouse/pull/15348) ([tavplubix](https://github.com/tavplubix)). + +#### Other + +* Unfold `{database}`, `{table}` and `{uuid}` macros in `ReplicatedMergeTree` arguments on table creation. [#16159](https://github.com/ClickHouse/ClickHouse/pull/16159) ([tavplubix](https://github.com/tavplubix)). + + +### ClickHouse release v20.8.4.11-lts, 2020-10-09 + +#### Bug Fix + +* Fix the order of destruction for resources in `ReadFromStorage` step of query plan. It might cause crashes in rare cases. Possibly connected with [#15610](https://github.com/ClickHouse/ClickHouse/issues/15610). [#15645](https://github.com/ClickHouse/ClickHouse/pull/15645) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fixed `Element ... is not a constant expression` error when using `JSON*` function result in `VALUES`, `LIMIT` or right side of `IN` operator. [#15589](https://github.com/ClickHouse/ClickHouse/pull/15589) ([tavplubix](https://github.com/tavplubix)). +* Prevent the possibility of error message `Could not calculate available disk space (statvfs), errno: 4, strerror: Interrupted system call`. This fixes [#15541](https://github.com/ClickHouse/ClickHouse/issues/15541). [#15557](https://github.com/ClickHouse/ClickHouse/pull/15557) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Significantly reduce memory usage in AggregatingInOrderTransform/optimize_aggregation_in_order. [#15543](https://github.com/ClickHouse/ClickHouse/pull/15543) ([Azat Khuzhin](https://github.com/azat)). +* Mutation might hang waiting for some non-existent part after `MOVE` or `REPLACE PARTITION` or, in rare cases, after `DETACH` or `DROP PARTITION`. It's fixed. [#15537](https://github.com/ClickHouse/ClickHouse/pull/15537) ([tavplubix](https://github.com/tavplubix)). +* Fix bug when `ILIKE` operator stops being case insensitive if `LIKE` with the same pattern was executed. [#15536](https://github.com/ClickHouse/ClickHouse/pull/15536) ([alesapin](https://github.com/alesapin)). +* Fix `Missing columns` errors when selecting columns which absent in data, but depend on other columns which also absent in data. Fixes [#15530](https://github.com/ClickHouse/ClickHouse/issues/15530). [#15532](https://github.com/ClickHouse/ClickHouse/pull/15532) ([alesapin](https://github.com/alesapin)). +* Fix bug with event subscription in DDLWorker which rarely may lead to query hangs in `ON CLUSTER`. Introduced in [#13450](https://github.com/ClickHouse/ClickHouse/issues/13450). [#15477](https://github.com/ClickHouse/ClickHouse/pull/15477) ([alesapin](https://github.com/alesapin)). +* Report proper error when the second argument of `boundingRatio` aggregate function has a wrong type. [#15407](https://github.com/ClickHouse/ClickHouse/pull/15407) ([detailyang](https://github.com/detailyang)). +* Fix race condition during MergeTree table rename and background cleanup. [#15304](https://github.com/ClickHouse/ClickHouse/pull/15304) ([alesapin](https://github.com/alesapin)). +* Fix rare race condition on server startup when system.logs are enabled. [#15300](https://github.com/ClickHouse/ClickHouse/pull/15300) ([alesapin](https://github.com/alesapin)). +* Fix MSan report in QueryLog. Uninitialized memory can be used for the field `memory_usage`. [#15258](https://github.com/ClickHouse/ClickHouse/pull/15258) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fix instance crash when using joinGet with LowCardinality types. This fixes https://github.com/ClickHouse/ClickHouse/issues/15214. [#15220](https://github.com/ClickHouse/ClickHouse/pull/15220) ([Amos Bird](https://github.com/amosbird)). +* Fix bug in table engine `Buffer` which doesn't allow to insert data of new structure into `Buffer` after `ALTER` query. Fixes [#15117](https://github.com/ClickHouse/ClickHouse/issues/15117). [#15192](https://github.com/ClickHouse/ClickHouse/pull/15192) ([alesapin](https://github.com/alesapin)). +* Adjust decimals field size in mysql column definition packet. [#15152](https://github.com/ClickHouse/ClickHouse/pull/15152) ([maqroll](https://github.com/maqroll)). +* We already use padded comparison between String and FixedString (https://github.com/ClickHouse/ClickHouse/blob/master/src/Functions/FunctionsComparison.h#L333). This PR applies the same logic to field comparison which corrects the usage of FixedString as primary keys. This fixes https://github.com/ClickHouse/ClickHouse/issues/14908. [#15033](https://github.com/ClickHouse/ClickHouse/pull/15033) ([Amos Bird](https://github.com/amosbird)). +* If function `bar` was called with specifically crafter arguments, buffer overflow was possible. This closes [#13926](https://github.com/ClickHouse/ClickHouse/issues/13926). [#15028](https://github.com/ClickHouse/ClickHouse/pull/15028) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fixed `Cannot rename ... errno: 22, strerror: Invalid argument` error on DDL query execution in Atomic database when running clickhouse-server in docker on Mac OS. [#15024](https://github.com/ClickHouse/ClickHouse/pull/15024) ([tavplubix](https://github.com/tavplubix)). +* Now settings `number_of_free_entries_in_pool_to_execute_mutation` and `number_of_free_entries_in_pool_to_lower_max_size_of_merge` can be equal to `background_pool_size`. [#14975](https://github.com/ClickHouse/ClickHouse/pull/14975) ([alesapin](https://github.com/alesapin)). +* Fix to make predicate push down work when subquery contains finalizeAggregation function. Fixes [#14847](https://github.com/ClickHouse/ClickHouse/issues/14847). [#14937](https://github.com/ClickHouse/ClickHouse/pull/14937) ([filimonov](https://github.com/filimonov)). +* Publish CPU frequencies per logical core in `system.asynchronous_metrics`. This fixes https://github.com/ClickHouse/ClickHouse/issues/14923. [#14924](https://github.com/ClickHouse/ClickHouse/pull/14924) ([Alexander Kuzmenkov](https://github.com/akuzm)). +* Fixed `.metadata.tmp File exists` error when using `MaterializeMySQL` database engine. [#14898](https://github.com/ClickHouse/ClickHouse/pull/14898) ([Winter Zhang](https://github.com/zhang2014)). +* Fix a problem where the server may get stuck on startup while talking to ZooKeeper, if the configuration files have to be fetched from ZK (using the `from_zk` include option). This fixes [#14814](https://github.com/ClickHouse/ClickHouse/issues/14814). [#14843](https://github.com/ClickHouse/ClickHouse/pull/14843) ([Alexander Kuzmenkov](https://github.com/akuzm)). +* Fix wrong monotonicity detection for shrunk `Int -> Int` cast of signed types. It might lead to incorrect query result. This bug is unveiled in [#14513](https://github.com/ClickHouse/ClickHouse/issues/14513). [#14783](https://github.com/ClickHouse/ClickHouse/pull/14783) ([Amos Bird](https://github.com/amosbird)). +* Fixed the incorrect sorting order of `Nullable` column. This fixes [#14344](https://github.com/ClickHouse/ClickHouse/issues/14344). [#14495](https://github.com/ClickHouse/ClickHouse/pull/14495) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). + +#### Improvement + +* Now it's possible to change the type of version column for `VersionedCollapsingMergeTree` with `ALTER` query. [#15442](https://github.com/ClickHouse/ClickHouse/pull/15442) ([alesapin](https://github.com/alesapin)). + + +### ClickHouse release v20.8.3.18-stable, 2020-09-18 + +#### Bug Fix + +* Fix the issue when some invocations of `extractAllGroups` function may trigger "Memory limit exceeded" error. This fixes [#13383](https://github.com/ClickHouse/ClickHouse/issues/13383). [#14889](https://github.com/ClickHouse/ClickHouse/pull/14889) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fix SIGSEGV for an attempt to INSERT into StorageFile(fd). [#14887](https://github.com/ClickHouse/ClickHouse/pull/14887) ([Azat Khuzhin](https://github.com/azat)). +* Fix rare error in `SELECT` queries when the queried column has `DEFAULT` expression which depends on the other column which also has `DEFAULT` and not present in select query and not exists on disk. Partially fixes [#14531](https://github.com/ClickHouse/ClickHouse/issues/14531). [#14845](https://github.com/ClickHouse/ClickHouse/pull/14845) ([alesapin](https://github.com/alesapin)). +* Fixed missed default database name in metadata of materialized view when executing `ALTER ... MODIFY QUERY`. [#14664](https://github.com/ClickHouse/ClickHouse/pull/14664) ([tavplubix](https://github.com/tavplubix)). +* Fix bug when `ALTER UPDATE` mutation with Nullable column in assignment expression and constant value (like `UPDATE x = 42`) leads to incorrect value in column or segfault. Fixes [#13634](https://github.com/ClickHouse/ClickHouse/issues/13634), [#14045](https://github.com/ClickHouse/ClickHouse/issues/14045). [#14646](https://github.com/ClickHouse/ClickHouse/pull/14646) ([alesapin](https://github.com/alesapin)). +* Fix wrong Decimal multiplication result caused wrong decimal scale of result column. [#14603](https://github.com/ClickHouse/ClickHouse/pull/14603) ([Artem Zuikov](https://github.com/4ertus2)). +* Added the checker as neither calling `lc->isNullable()` nor calling `ls->getDictionaryPtr()->isNullable()` would return the correct result. [#14591](https://github.com/ClickHouse/ClickHouse/pull/14591) ([myrrc](https://github.com/myrrc)). +* Cleanup data directory after Zookeeper exceptions during CreateQuery for StorageReplicatedMergeTree Engine. [#14563](https://github.com/ClickHouse/ClickHouse/pull/14563) ([Bharat Nallan](https://github.com/bharatnc)). +* Fix rare segfaults in functions with combinator -Resample, which could appear in result of overflow with very large parameters. [#14562](https://github.com/ClickHouse/ClickHouse/pull/14562) ([Anton Popov](https://github.com/CurtizJ)). + +#### Improvement + +* Speed up server shutdown process if there are ongoing S3 requests. [#14858](https://github.com/ClickHouse/ClickHouse/pull/14858) ([Pavel Kovalenko](https://github.com/Jokser)). +* Allow using multi-volume storage configuration in storage Distributed. [#14839](https://github.com/ClickHouse/ClickHouse/pull/14839) ([Pavel Kovalenko](https://github.com/Jokser)). +* Speed up server shutdown process if there are ongoing S3 requests. [#14496](https://github.com/ClickHouse/ClickHouse/pull/14496) ([Pavel Kovalenko](https://github.com/Jokser)). +* Support custom codecs in compact parts. [#12183](https://github.com/ClickHouse/ClickHouse/pull/12183) ([Anton Popov](https://github.com/CurtizJ)). + + ### ClickHouse release v20.8.2.3-stable, 2020-09-08 #### Backward Incompatible Change diff --git a/utils/simple-backport/format-changelog.py b/utils/simple-backport/format-changelog.py index 91547befed4..861faafdcfd 100755 --- a/utils/simple-backport/format-changelog.py +++ b/utils/simple-backport/format-changelog.py @@ -108,8 +108,11 @@ def print_category(category): user = users[pr["user"]["id"]] user_name = user["name"] if user["name"] else user["login"] - # Substitute issue links + # Substitute issue links. + # 1) issue number w/o markdown link pr["entry"] = re.sub(r'([^[])#([0-9]{4,})', r'\1[#\2](https://github.com/ClickHouse/ClickHouse/issues/\2)', pr["entry"]) + # 2) issue URL w/o markdown link + pr["entry"] = re.sub(r'([^(])https://github.com/ClickHouse/ClickHouse/issues/([0-9]{4,})', r'\1[#\2](https://github.com/ClickHouse/ClickHouse/issues/\2)', pr["entry"]) print(f'* {pr["entry"]} [#{pr["number"]}]({pr["html_url"]}) ([{user_name}]({user["html_url"]})).') From f6fc9137a0e06ed290931653ae12cf8483b0ecf7 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Fri, 13 Nov 2020 09:29:18 +0300 Subject: [PATCH 183/205] Update version_date.tsv after release 20.9.5.5 --- utils/list-versions/version_date.tsv | 1 + 1 file changed, 1 insertion(+) diff --git a/utils/list-versions/version_date.tsv b/utils/list-versions/version_date.tsv index f5d22827f6d..f636dd250f4 100644 --- a/utils/list-versions/version_date.tsv +++ b/utils/list-versions/version_date.tsv @@ -3,6 +3,7 @@ v20.11.2.1-stable 2020-11-11 v20.10.4.1-stable 2020-11-13 v20.10.3.30-stable 2020-10-29 v20.10.2.20-stable 2020-10-23 +v20.9.5.5-stable 2020-11-13 v20.9.4.76-stable 2020-10-29 v20.9.3.45-stable 2020-10-09 v20.9.2.20-stable 2020-09-22 From 2092aed4159b9c5619657b150f7c50d24cf69a10 Mon Sep 17 00:00:00 2001 From: Alexander Kuzmenkov Date: Fri, 13 Nov 2020 09:39:14 +0300 Subject: [PATCH 184/205] more --- CHANGELOG.md | 13 +++++++++++++ 1 file changed, 13 insertions(+) diff --git a/CHANGELOG.md b/CHANGELOG.md index 555c9f28534..fc5b2e19748 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -358,6 +358,19 @@ ## ClickHouse release 20.9 +### ClickHouse release v20.9.5.5-stable, 2020-11-13 + +#### Bug Fix + +* Fix rare silent crashes when query profiler is on and ClickHouse is installed on OS with glibc version that has (supposedly) broken asynchronous unwind tables for some functions. This fixes [#15301](https://github.com/ClickHouse/ClickHouse/issues/15301). This fixes [#13098](https://github.com/ClickHouse/ClickHouse/issues/13098). [#16846](https://github.com/ClickHouse/ClickHouse/pull/16846) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Now when parsing AVRO from input the LowCardinality is removed from type. Fixes [#16188](https://github.com/ClickHouse/ClickHouse/issues/16188). [#16521](https://github.com/ClickHouse/ClickHouse/pull/16521) ([Mike](https://github.com/myrrc)). +* Fix rapid growth of metadata when using MySQL Master -> MySQL Slave -> ClickHouse MaterializeMySQL Engine, and `slave_parallel_worker` enabled on MySQL Slave, by properly shrinking GTID sets. This fixes [#15951](https://github.com/ClickHouse/ClickHouse/issues/15951). [#16504](https://github.com/ClickHouse/ClickHouse/pull/16504) ([TCeason](https://github.com/TCeason)). +* Fix DROP TABLE for Distributed (racy with INSERT). [#16409](https://github.com/ClickHouse/ClickHouse/pull/16409) ([Azat Khuzhin](https://github.com/azat)). +* Fix processing of very large entries in replication queue. Very large entries may appear in ALTER queries if table structure is extremely large (near 1 MB). This fixes [#16307](https://github.com/ClickHouse/ClickHouse/issues/16307). [#16332](https://github.com/ClickHouse/ClickHouse/pull/16332) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fixed the inconsistent behaviour when a part of return data could be dropped because the set for its filtration wasn't created. [#16308](https://github.com/ClickHouse/ClickHouse/pull/16308) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). +* Fix bug with MySQL database. When MySQL server used as database engine is down some queries raise Exception, because they try to get tables from disabled server, while it's unnecessary. For example, query `SELECT ... FROM system.parts` should work only with MergeTree tables and don't touch MySQL database at all. [#16032](https://github.com/ClickHouse/ClickHouse/pull/16032) ([Kruglov Pavel](https://github.com/Avogar)). + + ### ClickHouse release v20.9.4.76-stable (2020-10-29) #### Bug Fix From e590a341e56a967a191dbb6a0b2bfd222a207b43 Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 13 Nov 2020 11:45:51 +0300 Subject: [PATCH 185/205] Add test --- ...alter_low_cardinality_and_select.reference | 6 +++ .../01576_alter_low_cardinality_and_select.sh | 41 +++++++++++++++++++ 2 files changed, 47 insertions(+) create mode 100644 tests/queries/0_stateless/01576_alter_low_cardinality_and_select.reference create mode 100755 tests/queries/0_stateless/01576_alter_low_cardinality_and_select.sh diff --git a/tests/queries/0_stateless/01576_alter_low_cardinality_and_select.reference b/tests/queries/0_stateless/01576_alter_low_cardinality_and_select.reference new file mode 100644 index 00000000000..532cba023b6 --- /dev/null +++ b/tests/queries/0_stateless/01576_alter_low_cardinality_and_select.reference @@ -0,0 +1,6 @@ +733 733 +CREATE TABLE default.alter_table\n(\n `key` UInt64,\n `value` LowCardinality(String)\n)\nENGINE = MergeTree\nORDER BY key\nSETTINGS index_granularity = 8192 +all_1_1_0 +all_2_2_0 +all_3_3_0 +701 701 diff --git a/tests/queries/0_stateless/01576_alter_low_cardinality_and_select.sh b/tests/queries/0_stateless/01576_alter_low_cardinality_and_select.sh new file mode 100755 index 00000000000..1055c23ccad --- /dev/null +++ b/tests/queries/0_stateless/01576_alter_low_cardinality_and_select.sh @@ -0,0 +1,41 @@ +#!/usr/bin/env bash + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +. "$CURDIR"/../shell_config.sh + + +${CLICKHOUSE_CLIENT} --query "DROP TABLE IF EXISTS alter_table" + +${CLICKHOUSE_CLIENT} --query "CREATE TABLE alter_table (key UInt64, value String) ENGINE MergeTree ORDER BY key" + +# we don't need mutations and merges +${CLICKHOUSE_CLIENT} --query "SYSTEM STOP MERGES alter_table" + +${CLICKHOUSE_CLIENT} --query "INSERT INTO alter_table SELECT number, toString(number) FROM numbers(10000)" +${CLICKHOUSE_CLIENT} --query "INSERT INTO alter_table SELECT number, toString(number) FROM numbers(10000, 10000)" +${CLICKHOUSE_CLIENT} --query "INSERT INTO alter_table SELECT number, toString(number) FROM numbers(20000, 10000)" + +${CLICKHOUSE_CLIENT} --query "SELECT * FROM alter_table WHERE value == '733'" + +${CLICKHOUSE_CLIENT} --query "ALTER TABLE alter_table MODIFY COLUMN value LowCardinality(String)" & + +type_query="SELECT type FROM system.columns WHERE name = 'value' and table='alter_table' and database='${CLICKHOUSE_DATABASE}'" +value_type="" + +# waiting until schema will change (but not data) +while [[ "$value_type" != "LowCardinality(String)" ]] +do + sleep 0.1 + value_type=$($CLICKHOUSE_CLIENT --query "$type_query") +done + +# checking type is LowCardinalty +${CLICKHOUSE_CLIENT} --query "SHOW CREATE TABLE alter_table" + +# checking no mutations happened +${CLICKHOUSE_CLIENT} --query "SELECT name FROM system.parts where table='alter_table' and active and database='${CLICKHOUSE_DATABASE}' ORDER BY name" + +# checking that conversions applied "on fly" works +${CLICKHOUSE_CLIENT} --query "SELECT * FROM alter_table PREWHERE key > 700 WHERE value = '701'" + +${CLICKHOUSE_CLIENT} --query "DROP TABLE IF EXISTS alter_table" From 00e8c8a2f6d30b16e468dd2fa35a58723344f645 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 13 Nov 2020 12:28:21 +0300 Subject: [PATCH 186/205] Skip test. --- tests/queries/0_stateless/arcadia_skip_list.txt | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/queries/0_stateless/arcadia_skip_list.txt b/tests/queries/0_stateless/arcadia_skip_list.txt index 711a69e0282..265b1a8295d 100644 --- a/tests/queries/0_stateless/arcadia_skip_list.txt +++ b/tests/queries/0_stateless/arcadia_skip_list.txt @@ -168,3 +168,4 @@ 01555_system_distribution_queue_mask 01557_max_parallel_replicas_no_sample.sql 01525_select_with_offset_fetch_clause +01560_timeseriesgroupsum_segfault From fedefc43ffdcf60926f10dc1473c68c86093862f Mon Sep 17 00:00:00 2001 From: damozhaeva <68770561+damozhaeva@users.noreply.github.com> Date: Fri, 13 Nov 2020 13:12:06 +0300 Subject: [PATCH 187/205] DOCSUP-3417 translate to english (#16797) * English translation * Update utils/db-generator/README.md Co-authored-by: BayoNet * Update utils/db-generator/README.md Co-authored-by: BayoNet * Update utils/db-generator/README.md Co-authored-by: BayoNet * Update utils/db-generator/README.md Co-authored-by: BayoNet * Update utils/db-generator/README.md Co-authored-by: BayoNet * Update utils/db-generator/README.md Co-authored-by: BayoNet * Update utils/db-generator/README.md * Update utils/db-generator/README.md Co-authored-by: Daria Mozhaeva Co-authored-by: BayoNet --- utils/db-generator/README.md | 47 ++++++++++++++++++------------------ 1 file changed, 24 insertions(+), 23 deletions(-) diff --git a/utils/db-generator/README.md b/utils/db-generator/README.md index f49c7911cb6..5596aac66e4 100644 --- a/utils/db-generator/README.md +++ b/utils/db-generator/README.md @@ -1,34 +1,35 @@ -**Анализ запроса в Clickhouse** +# Clickhouse query analysis -В данной работе мы будем рассматривать только select запросы, то есть те запросы, которые из таблицы достают данные. -Встроенный парсер Clickhouse принимает на вход строку, которая является запросом. В select запросе выделяется 14 основных частей: WITH, SELECT, TABLES, PREWHERE, WHERE, GROUP_BY, HAVING, ORDER_BY, LIMIT_BY_OFFSET, LIMIT_BY_LENGTH, LIMIT_BY, LIMIT_OFFSET, LIMIT_LENGTH, SETTINGS. Мы будем рассматривать части SELECT, TABLES, WHERE, GROUP_BY, HAVING, ORDER_BY так как именно в них находятся основные данные, нужные нам для анализа структуры и выявления значений. После анализа запроса парсер выдает нам древовидную структуру, где каждая вершина является определенной операцией выполнения запроса, функцией над значениями, константой, обозначением и тому подобное. Вершины также имеют свои поддеревья, в которых находятся их аргументы или подоперации. Обходя данное дерево мы будем пытаться выявить необходимые нам данные. +Here we will consider only `SELECT` queries, i.e. those queries that get data from the table. +The built-in Clickhouse parser accepts a string as input, which is a query. Among 14 main clauses of `SELECT` statement: `WITH`, `SELECT`, `TABLES`, `PREWHERE`, `WHERE`, `GROUP_BY`, `HAVING`, `ORDER_BY`, `LIMIT_BY_OFFSET`, `LIMIT_BY_LENGTH`, `LIMIT_BY`, `LIMIT_OFFSET`, `LIMIT_LENGTH`, `SETTINGS`, we will analyze the `SELECT`, `TABLES`, `WHERE`, `GROUP_BY`, `HAVING`, `ORDER_BY` clauses because the most of data is there. We need this data to analyze the structure and to identify values. The parser issues a tree structure after parsing a query, where each node is a specific query execution operation, a function over values, a constant, a designation, etc. Nodes also have subtrees where their arguments or suboperations are located. We will try to reveal the data we need by avoiding this tree. -**Анализ схемы** +## Scheme analysis -По запросу необходимо определить возможные таблицы. Имея строку запроса можно понять, какие его части обозначают названия таблиц, таким образом можно определить их количество в нашей базе данных. -В парсере Clickhouse поддеревом запроса, отвечающее за таблицы из которых мы берем данные, является TABLES (Рисунок 1), в нем лежит основная таблица, из которой берутся колонки, а также операции JOIN, которые совершаются в запросе. Обходя все вершины в поддереве мы берем названия таблиц и баз данных в которых они лежат, а также их алиас, то есть укороченные названия, выбранные автором запроса. Эти названия могут понадобиться нам для определения принадлежности колонки в дальнейшем. -Таким образом для запроса мы получаем набор баз данных, а также таблиц и их условных обозначений (алиасов), по которым делается запрос. +It is necessary to determine possible tables by a query. Having a query string, you can understand which parts of it represent the names of the tables, so you can determine their number in our database. +In the Clickhouse parser, `TABLES` (Figure 1) is a query subtree responsible for tables where we get data. It contains the main table where the columns come from, as well as the `JOIN` operations that are performed in the query. Avoiding all nodes in the subtree, we use the names of the tables and databases where they are located, as well as their alias, i.e. the shortened names chosen by the query author. We may need these names to determine the ownership of the column in the future. +Thus, we get a set of databases for the query, as well as tables and their aliases, with the help of them a query is made. -Затем нам необходимо определить множество столбцов, которые присутствуют в запросе и таблицы, к которым они могут относиться. Во время исполнения запроса уже известно множество столбцов в каждой таблице, поэтому при исполнении программа автоматически связывает столбец и таблицу, однако в нашем случае нельзя однозначно трактовать принадлежность столбца к определенной таблице, например в следующем запросе: “SELECT column1, column2, column3 FROM table1 JOIN table2 on table1.column2 = table2.column3 ”. Здесь мы однозначно можем сказать, к какой таблице относятся колонки column2 и column3, однако column1 может принадлежать как первой, так и второй таблице. Для однозначности трактовки таких случаев, мы будем относить данную неопределенные колонки к основной таблице, по которой делается запрос, например в данном случае это будет таблица table1. -Все столбцы в дереве лежат в вершинах типа IDENTIFIER, которые находятся в поддеревьях SELECT, TABLES, WHERE, GROUP_BY, HAVING, ORDER_BY. Рекурсивно обходя поддеревья мы формируем множество всех таблиц, затем мы разделяем колонку на составляющие: таблица (если она явно указана через точку) и само название, затем, так как таблица может являться алиасом, мы заменяем алиас на оригинальное название таблицы. Теперь у нас есть список всех столбцов и таблиц, к которым они относятся, для столбцов без таблиц определяем основную таблицу запроса. +Then we need to define the set of columns that are in the query and the tables they can refer to. The set of columns in each table is already known during the query execution. Therefore, the program automatically links the column and table at runtime. However, in our case, it is impossible to unambiguously interpret the belonging of a column to a specific table, for example, in the following query `SELECT column1, column2, column3 FROM table1 JOIN table2 on table1.column2 = table2.column3`. In this case, we can say which table `column2` and `column3` belong to. However, `column1` can belong to either the first or the second table. We will refer undefined columns to the main table, on which a query is made, for unambiguous interpretation of such cases. For example, in this case, it will be `table1`. +All columns in the tree are in `IDENTIFIER` type nodes, which are in the `SELECT`, `TABLES`, `WHERE`, `GROUP_BY`, `HAVING`, `ORDER_BY` subtrees. We form a set of all tables recursively avoiding the subtrees, then we split the column into constituents such as the table (if it is explicitly specified with a dot) and the name. Then, since the table can be an alias, we replace the alias with the original table name. We now have a list of all the columns and tables they belong to. We define the main query table for non-table columns. -**Анализ столбцов** +## Column analysis -Продолжением является точное определение типов данных для столбцов, у которых в запросе присутствует значение. Примером являются логические условие WHERE, в котором у определенного набора атрибутов проверяется логическое выражение. Если в запросе указано column > 5, то можно сделать вывод, что в данном столбце содержится численное значение, либо если на атрибут применяется выражение LIKE, то атрибут представляет собой строковый тип. -В данной части необходимо научится вычленять из запроса все таки выражения и сопоставлять типы данных для тех столбцов, для которых это возможно сделать. При этом, понятно, что из присутствующих значений не всегда можно сделать однозначное решение о типе конкретного атрибута, например column > 5 может означать множество численных типов таких как UINT8, UINT32, INT32, INT64 и тому подобных. Здесь нужно определиться с трактовкой определенных значений, так как перебор всех возможных может быть достаточно большим, а поэтому занимать продолжительное время. -Для числовых значений было решено использовать INT64(целочисленный тип 64 битности) для целочисленных значений и FLOAT64(число с плавающей точкой 64 битности) для нецелых значений. Также используются типы STRING для строковых значений, DATE для дат, DATETIME для времени. Стоит заметить, что существует еще тип ARRAY, который является оберткой над предыдущими типами и представлять собой массив из значений определенного типа. -Определить значения столбцов мы можем используя логический, арифметические и другие функции над значениями столбцов, которые указаны в запросе. Такие функции лежат в поддеревьях SELECT и WHERE. Параметром функции может быть константа, колонка либо другая функция (Рисунок 2). Таким образом для понимания типа колонки могут помочь следующие параметры: 1) Типы аргументов, которые может принимать функция, например функция TOSTARTOFMINUTE(округляет время до кратного 5 минутам вниз) может принимать только DATETIME, таким образом если аргументом данной функции является колонка, то данная колонка имеет тип DATETIME. 2) типы остальных аргументов в данной функции, например функция EQUALS(равенство), она подразумевает собой равенство типов ее аргументов, таким образом если в данной функции присутствует константа и столбец, то мы можем определить тип столбца как тип константы. +Then we need to exactly define data types for columns that have a value in the query. An example is the boolean `WHERE` clause where we test boolean expressions in its attributes. If the query specifies `column > 5`, then we can conclude that this column contains a numeric value, or if the `LIKE` expression is applied to the attribute, then the attribute has a string type. +In this part, you need to learn how to extract such expressions from a query and match data types for columns, where it is possible. At the same time, it is clear that it is not always possible to make an unambiguous decision about the type of a particular attribute from the available values. For example, `column > 5` can mean many numeric types such as `UINT8`, `UINT32`, `INT32`, `INT64`, etc. It is necessary to determine the interpretation of certain values since searching through all possible values ​​can be quite large and long. +It can take a long time to iterate over all possible values, so we use `INT64` and `FLOAT64` types for numeric values, `STRING` for strings, `DATE` and `DATETIME` for dates, and `ARRAY`. +We can determine column values ​​using boolean, arithmetic and other functions on the column values ​​that are specified in the query. Such functions are in the `SELECT` and `WHERE` subtrees. The function parameter can be a constant, a column or another function (Figure 2). Thus, the following parameters can help to understand the type of the column: +- The types of arguments that a function can take, for example, the `TOSTARTOFMINUTE` function (truncate time up to a multiple of 5 minutes down) can only accept `DATETIME`, so if the argument of this function is a column, then this column has `DATETIME` type. +- The types of the remaining arguments in this function. For example, the `EQUALS` function means equality of its argument types, so if a constant and a column are present in this function, then we can define the type of the column as the type of the constant. -Таким образом, для каждой функции мы определяем возможные типы аргументов, тип возвращаемого значения, а также параметр, являются ли аргументы функции одинакового типа. Рекурсивный обработчик функций будет определять возможные типы столбцов использующихся в данных функциях по значениям аргументов и возвращать возможные типы результата выполнения функции. -Теперь для каждого столбца мы имеем множество возможных типов его значений. Для однозначной трактовки запроса мы выберем один конкретный тип из этого множества. +Thus, we define the possible argument types, the return type, the parameter for each function, and the function arguments of the identical type. The recursive function handler will determine the possible types of columns used in these functions by the values of the arguments, and then return the possible types of the function's result. +Now, for each column, we have many possible types of values. We will choose one specific type from this set to interpret the query unambiguously. -**Определение значений столбцов** +## Column values definition -На этом этапе мы уже имеем определенную структуру таблиц базы данных, нам необходимо заполнить эту таблицу значениям. Нам необходимо понять, какие столбцы зависят друг от друга при исполнении функции (например по двум столбцами делается join, значит они должны иметь одинаковые значения), а также какие значения должны принимать столбцы, чтобы выполнялись различные условия при исполнении. -Для достижения цели ищем все операции сравнения в нашем запросе, если аргументами операции являются два столбца, то мы считаем их связанными, если аргументами являются столбец и значение, то присваиваем данное значение возможным значением данного столбца, а также добавляем данное значение + определенный шум. Для числового типа шумом является случайное число, для даты - случайное количество дней и т.п. При этом для каждой операции сравнения необходим свой обработчик этой операции, который генерирует хотя бы два значения, одно из которых условие операции, а другое нет. Например, для операции column1 > 5, column1 должно присваиваться значение большее 5 и меньшее, либо равное 5, аналогично для операции “column2 LIKE some%string”, столбцу column2 должно присваиваться значение удовлетворяющее выражение, а также не удовлетворяющее. -Теперь для некоторых колонок мы имеем множество связанных с ними колонок и множество значений. Мы знаем, что связность колонок симметрична, но для полноценного определения связности колонок нам необходимо добавить транзитивность, т.к если “column1 = column2” и “column2 = column3”, то “column1 = column3”, но это не вытекает из построения. Соответственно нам необходимо распространить связность по всем колонкам. Затем мы для каждой колонки объединяем множество ее значений со значениями всех связанных с ней. Теперь если у нас остались колонки без значений, мы просто генерируем случайные значения. +At this stage, we already have a certain structure of the database tables, we need to fill this table with values. We should understand which columns depend on each other when executing the function (for example, the join is done according to two columns, which means that they must have the same values). We also need to understand what values ​​the columns must have to fulfill various conditions during execution. +We search for all comparison operations in our query to achieve the goal. If the arguments of the operation are two columns, then we consider them linked. If the arguments are the column and the value, then we assign that value to the possible column value and add the value with some noise. A random number is a noise for a numeric type, it is a random number of days for a date, etc. In this case, a handler for this operation is required for each comparison operation, which generates at least two values, one of them is the operation condition, and the other is not. For example, a value greater than 5 and less than or equal to 5 must be assigned for the operation `column1 > 5`, `column1`, for the operation `column2 LIKE some% string` the same is true. The satisfying and not satisfying expression must be assigned to `column2`. +Now we have many associated columns and many values. We know that the connectivity of columns is symmetric, but we need to add transitivity for a complete definition, because if `column1 = column2` and `column2 = column3`, then `column1 = column3`, but this does not follow from the construction. Accordingly, we need to extend the connectivity across all columns. We combine multiple values for each column with the values associated with it. If we have columns with no values, then we generate random values. -**Генерация записей** +## Generation -Теперь у нас есть полноценной представление схемы базы данных, а также множество значений каждой таблицы. Мы будем генерировать данные посредством декартова произведения множества значений каждого столбца для определенной таблицы. Таким образом мы получаем для каждой таблицы множество, состоящее из множеств значений каждого столбца. -По этим данным мы начинаем генерировать запросы, создающие данную таблицу и заполняет ее данными. По структуре таблицы и типам ее столбцов мы генерируем CREATE QUERY, которая создает данную таблицу. Затем по множеству значений мы генерируем INSERT QUERY, которая заполняет данную таблицу данными. \ No newline at end of file +We have a complete view of the database schema as well as many values ​​for each table now. We will generate data by cartesian product of the value set of each column for a specific table. Thus, we get a set for each table, consisting of sets of values for each column. We start generating queries that create this table and fill it with data. We generate the `CREATE QUERY` that creates this table based on the structure of the table and the types of its columns, and then we generate the `INSERT QUERY` over the set of values, which fills the table with data. From b6befc10f9b00236bea2ce040dfce140fb5df3c9 Mon Sep 17 00:00:00 2001 From: tavplubix Date: Fri, 13 Nov 2020 13:43:37 +0300 Subject: [PATCH 188/205] Update CHANGELOG.md --- CHANGELOG.md | 9 --------- 1 file changed, 9 deletions(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index fc5b2e19748..4c35a5f0255 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -380,7 +380,6 @@ * Fix async Distributed INSERT w/ prefer_localhost_replica=0 and internal_replication. [#16358](https://github.com/ClickHouse/ClickHouse/pull/16358) ([Azat Khuzhin](https://github.com/azat)). * Fix a very wrong code in TwoLevelStringHashTable implementation, which might lead to memory leak. I'm suprised how this bug can lurk for so long.... [#16264](https://github.com/ClickHouse/ClickHouse/pull/16264) ([Amos Bird](https://github.com/amosbird)). * Fix the case when memory can be overallocated regardless to the limit. This closes [#14560](https://github.com/ClickHouse/ClickHouse/issues/14560). [#16206](https://github.com/ClickHouse/ClickHouse/pull/16206) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Not for changelog. [#16031](https://github.com/ClickHouse/ClickHouse/pull/16031) ([tavplubix](https://github.com/tavplubix)). * Fix `ALTER MODIFY ... ORDER BY` query hang for `ReplicatedVersionedCollapsingMergeTree`. This fixes [#15980](https://github.com/ClickHouse/ClickHouse/issues/15980). [#16011](https://github.com/ClickHouse/ClickHouse/pull/16011) ([alesapin](https://github.com/alesapin)). * Fix collate name & charset name parser and support `length = 0` for string type. [#16008](https://github.com/ClickHouse/ClickHouse/pull/16008) ([Winter Zhang](https://github.com/zhang2014)). * Allow to use direct layout for dictionaries with complex keys. [#16007](https://github.com/ClickHouse/ClickHouse/pull/16007) ([Anton Popov](https://github.com/CurtizJ)). @@ -403,10 +402,6 @@ #### Improvement * Now it's allowed to execute `ALTER ... ON CLUSTER` queries regardless of the `` setting in cluster config. [#16075](https://github.com/ClickHouse/ClickHouse/pull/16075) ([alesapin](https://github.com/alesapin)). -* Unfold `{database}`, `{table}` and `{uuid}` macros in `zookeeper_path` on replicated table creation. Do not allow `RENAME TABLE` if it may break `zookeeper_path` after server restart. Fixes [#6917](https://github.com/ClickHouse/ClickHouse/issues/6917). [#15348](https://github.com/ClickHouse/ClickHouse/pull/15348) ([tavplubix](https://github.com/tavplubix)). - -#### Other - * Unfold `{database}`, `{table}` and `{uuid}` macros in `ReplicatedMergeTree` arguments on table creation. [#16160](https://github.com/ClickHouse/ClickHouse/pull/16160) ([tavplubix](https://github.com/tavplubix)). @@ -546,10 +541,6 @@ #### Improvement * Now it's allowed to execute `ALTER ... ON CLUSTER` queries regardless of the `` setting in cluster config. [#16075](https://github.com/ClickHouse/ClickHouse/pull/16075) ([alesapin](https://github.com/alesapin)). -* Unfold `{database}`, `{table}` and `{uuid}` macros in `zookeeper_path` on replicated table creation. Do not allow `RENAME TABLE` if it may break `zookeeper_path` after server restart. Fixes [#6917](https://github.com/ClickHouse/ClickHouse/issues/6917). [#15348](https://github.com/ClickHouse/ClickHouse/pull/15348) ([tavplubix](https://github.com/tavplubix)). - -#### Other - * Unfold `{database}`, `{table}` and `{uuid}` macros in `ReplicatedMergeTree` arguments on table creation. [#16159](https://github.com/ClickHouse/ClickHouse/pull/16159) ([tavplubix](https://github.com/tavplubix)). From 276059394067041079d4b9aa006b7c0e1ced5d45 Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 13 Nov 2020 13:50:15 +0300 Subject: [PATCH 189/205] Better test --- .../01576_alter_low_cardinality_and_select.sh | 9 ++++----- 1 file changed, 4 insertions(+), 5 deletions(-) diff --git a/tests/queries/0_stateless/01576_alter_low_cardinality_and_select.sh b/tests/queries/0_stateless/01576_alter_low_cardinality_and_select.sh index 1055c23ccad..94596da9258 100755 --- a/tests/queries/0_stateless/01576_alter_low_cardinality_and_select.sh +++ b/tests/queries/0_stateless/01576_alter_low_cardinality_and_select.sh @@ -19,14 +19,13 @@ ${CLICKHOUSE_CLIENT} --query "SELECT * FROM alter_table WHERE value == '733'" ${CLICKHOUSE_CLIENT} --query "ALTER TABLE alter_table MODIFY COLUMN value LowCardinality(String)" & -type_query="SELECT type FROM system.columns WHERE name = 'value' and table='alter_table' and database='${CLICKHOUSE_DATABASE}'" -value_type="" - # waiting until schema will change (but not data) -while [[ "$value_type" != "LowCardinality(String)" ]] +show_query="SHOW CREATE TABLE alter_table" +create_query="" +while [[ "$create_query" != *"LowCardinality"* ]] do sleep 0.1 - value_type=$($CLICKHOUSE_CLIENT --query "$type_query") + create_query=$($CLICKHOUSE_CLIENT --query "$show_query") done # checking type is LowCardinalty From 88789b5c146f9284bc77d41dd6a31a8a3ef0e0c5 Mon Sep 17 00:00:00 2001 From: Alexander Kuzmenkov Date: Fri, 13 Nov 2020 13:57:03 +0300 Subject: [PATCH 190/205] 20.3 --- CHANGELOG.md | 68 ++++++++++++++++++++++++++++++++++++++++++++++++++++ 1 file changed, 68 insertions(+) diff --git a/CHANGELOG.md b/CHANGELOG.md index 4c35a5f0255..f196d4421d8 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -1951,6 +1951,74 @@ No changes compared to v20.4.3.16-stable. ## ClickHouse release v20.3 + +### ClickHouse release v20.3.21.2-lts, 2020-11-02 + +#### Bug Fix + +* Fix dictGet in sharding_key (and similar places, i.e. when the function context is stored permanently). [#16205](https://github.com/ClickHouse/ClickHouse/pull/16205) ([Azat Khuzhin](https://github.com/azat)). +* Fix incorrect empty result for query from `Distributed` table if query has `WHERE`, `PREWHERE` and `GLOBAL IN`. Fixes [#15792](https://github.com/ClickHouse/ClickHouse/issues/15792). [#15933](https://github.com/ClickHouse/ClickHouse/pull/15933) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fix missing or excessive headers in `TSV/CSVWithNames` formats. This fixes [#12504](https://github.com/ClickHouse/ClickHouse/issues/12504). [#13343](https://github.com/ClickHouse/ClickHouse/pull/13343) ([Azat Khuzhin](https://github.com/azat)). + + +### ClickHouse release v20.3.20.6-lts, 2020-10-09 + +#### Bug Fix + +* Mutation might hang waiting for some non-existent part after `MOVE` or `REPLACE PARTITION` or, in rare cases, after `DETACH` or `DROP PARTITION`. It's fixed. [#15724](https://github.com/ClickHouse/ClickHouse/pull/15724), [#15537](https://github.com/ClickHouse/ClickHouse/pull/15537) ([tavplubix](https://github.com/tavplubix)). +* Fix hang of queries with a lot of subqueries to same table of `MySQL` engine. Previously, if there were more than 16 subqueries to same `MySQL` table in query, it hang forever. [#15299](https://github.com/ClickHouse/ClickHouse/pull/15299) ([Anton Popov](https://github.com/CurtizJ)). +* Fix 'Unknown identifier' in GROUP BY when query has JOIN over Merge table. [#15242](https://github.com/ClickHouse/ClickHouse/pull/15242) ([Artem Zuikov](https://github.com/4ertus2)). +* Fix to make predicate push down work when subquery contains finalizeAggregation function. Fixes [#14847](https://github.com/ClickHouse/ClickHouse/issues/14847). [#14937](https://github.com/ClickHouse/ClickHouse/pull/14937) ([filimonov](https://github.com/filimonov)). +* Concurrent `ALTER ... REPLACE/MOVE PARTITION ...` queries might cause deadlock. It's fixed. [#13626](https://github.com/ClickHouse/ClickHouse/pull/13626) ([tavplubix](https://github.com/tavplubix)). + + +### ClickHouse release v20.3.19.4-lts, 2020-09-18 + +#### Bug Fix + +* Fix rare error in `SELECT` queries when the queried column has `DEFAULT` expression which depends on the other column which also has `DEFAULT` and not present in select query and not exists on disk. Partially fixes [#14531](https://github.com/ClickHouse/ClickHouse/issues/14531). [#14845](https://github.com/ClickHouse/ClickHouse/pull/14845) ([alesapin](https://github.com/alesapin)). +* Fix bug when `ALTER UPDATE` mutation with Nullable column in assignment expression and constant value (like `UPDATE x = 42`) leads to incorrect value in column or segfault. Fixes [#13634](https://github.com/ClickHouse/ClickHouse/issues/13634), [#14045](https://github.com/ClickHouse/ClickHouse/issues/14045). [#14646](https://github.com/ClickHouse/ClickHouse/pull/14646) ([alesapin](https://github.com/alesapin)). +* Fix wrong Decimal multiplication result caused wrong decimal scale of result column. [#14603](https://github.com/ClickHouse/ClickHouse/pull/14603) ([Artem Zuikov](https://github.com/4ertus2)). + +#### Improvement + +* Support custom codecs in compact parts. [#12183](https://github.com/ClickHouse/ClickHouse/pull/12183) ([Anton Popov](https://github.com/CurtizJ)). + + +### ClickHouse release v20.3.18.10-lts, 2020-09-08 + +#### Bug Fix + +* Stop query execution if exception happened in `PipelineExecutor` itself. This could prevent rare possible query hung. Continuation of [#14334](https://github.com/ClickHouse/ClickHouse/issues/14334). [#14402](https://github.com/ClickHouse/ClickHouse/pull/14402) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fixed the behaviour when sometimes cache-dictionary returned default value instead of present value from source. [#13624](https://github.com/ClickHouse/ClickHouse/pull/13624) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). +* Fix parsing row policies from users.xml when names of databases or tables contain dots. This fixes [#5779](https://github.com/ClickHouse/ClickHouse/issues/5779), [#12527](https://github.com/ClickHouse/ClickHouse/issues/12527). [#13199](https://github.com/ClickHouse/ClickHouse/pull/13199) ([Vitaly Baranov](https://github.com/vitlibar)). +* Fix CAST(Nullable(String), Enum()). [#12745](https://github.com/ClickHouse/ClickHouse/pull/12745) ([Azat Khuzhin](https://github.com/azat)). +* Fixed data race in `text_log`. It does not correspond to any real bug. [#9726](https://github.com/ClickHouse/ClickHouse/pull/9726) ([alexey-milovidov](https://github.com/alexey-milovidov)). + +#### Improvement + +* Fix wrong error for long queries. It was possible to get syntax error other than `Max query size exceeded` for correct query. [#13928](https://github.com/ClickHouse/ClickHouse/pull/13928) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Return NULL/zero when value is not parsed completely in parseDateTimeBestEffortOrNull/Zero functions. This fixes [#7876](https://github.com/ClickHouse/ClickHouse/issues/7876). [#11653](https://github.com/ClickHouse/ClickHouse/pull/11653) ([alexey-milovidov](https://github.com/alexey-milovidov)). + +#### Performance Improvement + +* Slightly optimize very short queries with LowCardinality. [#14129](https://github.com/ClickHouse/ClickHouse/pull/14129) ([Anton Popov](https://github.com/CurtizJ)). + +#### Build/Testing/Packaging Improvement + +* Fix UBSan report (adding zero to nullptr) in HashTable that appeared after migration to clang-10. [#10638](https://github.com/ClickHouse/ClickHouse/pull/10638) ([alexey-milovidov](https://github.com/alexey-milovidov)). + + +### ClickHouse release v20.3.17.173-lts, 2020-08-15 + +#### Bug Fix + +* Fix crash in JOIN with StorageMerge and `set enable_optimize_predicate_expression=1`. [#13679](https://github.com/ClickHouse/ClickHouse/pull/13679) ([Artem Zuikov](https://github.com/4ertus2)). +* Fix invalid return type for comparison of tuples with `NULL` elements. Fixes [#12461](https://github.com/ClickHouse/ClickHouse/issues/12461). [#13420](https://github.com/ClickHouse/ClickHouse/pull/13420) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fix queries with constant columns and `ORDER BY` prefix of primary key. [#13396](https://github.com/ClickHouse/ClickHouse/pull/13396) ([Anton Popov](https://github.com/CurtizJ)). +* Return passed number for numbers with MSB set in roundUpToPowerOfTwoOrZero(). [#13234](https://github.com/ClickHouse/ClickHouse/pull/13234) ([Azat Khuzhin](https://github.com/azat)). + + ### ClickHouse release v20.3.16.165-lts 2020-08-10 #### Bug Fix From 178649ed2015a89e99f2cfb0d04b4052762db2e9 Mon Sep 17 00:00:00 2001 From: Patrick Zippenfenig Date: Fri, 13 Nov 2020 11:58:50 +0100 Subject: [PATCH 191/205] Add client library for Swift --- docs/en/interfaces/third-party/client-libraries.md | 3 +++ 1 file changed, 3 insertions(+) diff --git a/docs/en/interfaces/third-party/client-libraries.md b/docs/en/interfaces/third-party/client-libraries.md index 4e8347c9a6e..c737fad152f 100644 --- a/docs/en/interfaces/third-party/client-libraries.md +++ b/docs/en/interfaces/third-party/client-libraries.md @@ -26,6 +26,9 @@ toc_title: Client Libraries - [go-clickhouse](https://github.com/roistat/go-clickhouse) - [mailrugo-clickhouse](https://github.com/mailru/go-clickhouse) - [golang-clickhouse](https://github.com/leprosus/golang-clickhouse) +- Swift + - [ClickHouseNIO](https://github.com/patrick-zippenfenig/ClickHouseNIO) + - [ClickHouseVapor ORM](https://github.com/patrick-zippenfenig/ClickHouseVapor) - NodeJs - [clickhouse (NodeJs)](https://github.com/TimonKK/clickhouse) - [node-clickhouse](https://github.com/apla/node-clickhouse) From be16b4ef779e74c50712f3a4ee41345a9893d384 Mon Sep 17 00:00:00 2001 From: filimonov <1549571+filimonov@users.noreply.github.com> Date: Fri, 13 Nov 2020 12:04:56 +0100 Subject: [PATCH 192/205] Update Connection.cpp --- src/Client/Connection.cpp | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/src/Client/Connection.cpp b/src/Client/Connection.cpp index 2a130f4ee6f..b810d24b3d3 100644 --- a/src/Client/Connection.cpp +++ b/src/Client/Connection.cpp @@ -73,8 +73,7 @@ void Connection::connect(const ConnectionTimeouts & timeouts) { #if USE_SSL socket = std::make_unique(); - auto secure_socket = static_cast(socket.get()); - secure_socket->setPeerHostName(host); + static_cast(socket.get())->setPeerHostName(host); #else throw Exception{"tcp_secure protocol is disabled because poco library was built without NetSSL support.", ErrorCodes::SUPPORT_IS_DISABLED}; #endif From 902091cc076c0166084d3bbbd8283d3cc1ef32ee Mon Sep 17 00:00:00 2001 From: Ivan <5627721+abyss7@users.noreply.github.com> Date: Fri, 13 Nov 2020 14:28:18 +0300 Subject: [PATCH 193/205] Slightly better miniselect usage (#16896) * Slightly better miniselect usage * Update ColumnDecimal.cpp --- base/common/sort.h | 37 +++++++++++++++ src/AggregateFunctions/QuantileExact.h | 62 +++++++------------------ src/AggregateFunctions/QuantileTiming.h | 32 +++++-------- src/Columns/ColumnArray.cpp | 17 ++----- src/Columns/ColumnDecimal.cpp | 18 ++----- src/Columns/ColumnDecimal.h | 24 +++------- src/Columns/ColumnFixedString.cpp | 53 +++++++-------------- src/Columns/ColumnLowCardinality.cpp | 17 +++---- src/Columns/ColumnString.cpp | 30 ++++-------- src/Columns/ColumnTuple.cpp | 25 ++++------ src/Columns/ColumnVector.cpp | 59 +++++++++-------------- src/Columns/ColumnVectorHelper.h | 1 + 12 files changed, 140 insertions(+), 235 deletions(-) create mode 100644 base/common/sort.h diff --git a/base/common/sort.h b/base/common/sort.h new file mode 100644 index 00000000000..2128014ca5e --- /dev/null +++ b/base/common/sort.h @@ -0,0 +1,37 @@ +#pragma once + +#if !defined(ARCADIA_BUILD) +# include // Y_IGNORE +#else +# include +#endif + +template +void nth_element(RandomIt first, RandomIt nth, RandomIt last) +{ +#if !defined(ARCADIA_BUILD) + ::miniselect::floyd_rivest_select(first, nth, last); +#else + ::std::nth_element(first, nth, last); +#endif +} + +template +void partial_sort(RandomIt first, RandomIt middle, RandomIt last) +{ +#if !defined(ARCADIA_BUILD) + ::miniselect::floyd_rivest_partial_sort(first, middle, last); +#else + ::std::partial_sort(first, middle, last); +#endif +} + +template +void partial_sort(RandomIt first, RandomIt middle, RandomIt last, Compare compare) +{ +#if !defined(ARCADIA_BUILD) + ::miniselect::floyd_rivest_partial_sort(first, middle, last, compare); +#else + ::std::partial_sort(first, middle, last, compare); +#endif +} diff --git a/src/AggregateFunctions/QuantileExact.h b/src/AggregateFunctions/QuantileExact.h index a2acde97f90..bc85d5c1c28 100644 --- a/src/AggregateFunctions/QuantileExact.h +++ b/src/AggregateFunctions/QuantileExact.h @@ -1,19 +1,17 @@ #pragma once -#include -#include #include #include #include #include #include +#include +#include -#if !defined(ARCADIA_BUILD) - #include // Y_IGNORE -#endif namespace DB { + namespace ErrorCodes { extern const int NOT_IMPLEMENTED; @@ -89,12 +87,7 @@ struct QuantileExact : QuantileExactBase> if (!array.empty()) { size_t n = level < 1 ? level * array.size() : (array.size() - 1); - -#if !defined(ARCADIA_BUILD) - miniselect::floyd_rivest_select(array.begin(), array.begin() + n, array.end()); /// NOTE You can think of the radix-select algorithm. -#else - std::nth_element(array.begin(), array.begin() + n, array.end()); /// NOTE You can think of the radix-select algorithm. -#endif + nth_element(array.begin(), array.begin() + n, array.end()); /// NOTE: You can think of the radix-select algorithm. return array[n]; } @@ -113,12 +106,7 @@ struct QuantileExact : QuantileExactBase> auto level = levels[indices[i]]; size_t n = level < 1 ? level * array.size() : (array.size() - 1); - -#if !defined(ARCADIA_BUILD) - miniselect::floyd_rivest_select(array.begin() + prev_n, array.begin() + n, array.end()); -#else - std::nth_element(array.begin() + prev_n, array.begin() + n, array.end()); -#endif + nth_element(array.begin() + prev_n, array.begin() + n, array.end()); result[indices[i]] = array[n]; prev_n = n; } @@ -154,14 +142,10 @@ struct QuantileExactExclusive : public QuantileExact else if (n < 1) return static_cast(array[0]); -#if !defined(ARCADIA_BUILD) - miniselect::floyd_rivest_select(array.begin(), array.begin() + n - 1, array.end()); -#else - std::nth_element(array.begin(), array.begin() + n - 1, array.end()); -#endif - auto nth_element = std::min_element(array.begin() + n, array.end()); + nth_element(array.begin(), array.begin() + n - 1, array.end()); + auto nth_elem = std::min_element(array.begin() + n, array.end()); - return static_cast(array[n - 1]) + (h - n) * static_cast(*nth_element - array[n - 1]); + return static_cast(array[n - 1]) + (h - n) * static_cast(*nth_elem - array[n - 1]); } return std::numeric_limits::quiet_NaN(); @@ -187,14 +171,10 @@ struct QuantileExactExclusive : public QuantileExact result[indices[i]] = static_cast(array[0]); else { -#if !defined(ARCADIA_BUILD) - miniselect::floyd_rivest_select(array.begin() + prev_n, array.begin() + n - 1, array.end()); -#else - std::nth_element(array.begin() + prev_n, array.begin() + n - 1, array.end()); -#endif - auto nth_element = std::min_element(array.begin() + n, array.end()); + nth_element(array.begin() + prev_n, array.begin() + n - 1, array.end()); + auto nth_elem = std::min_element(array.begin() + n, array.end()); - result[indices[i]] = static_cast(array[n - 1]) + (h - n) * static_cast(*nth_element - array[n - 1]); + result[indices[i]] = static_cast(array[n - 1]) + (h - n) * static_cast(*nth_elem - array[n - 1]); prev_n = n - 1; } } @@ -226,14 +206,10 @@ struct QuantileExactInclusive : public QuantileExact return static_cast(array[array.size() - 1]); else if (n < 1) return static_cast(array[0]); -#if !defined(ARCADIA_BUILD) - miniselect::floyd_rivest_select(array.begin(), array.begin() + n - 1, array.end()); -#else - std::nth_element(array.begin(), array.begin() + n - 1, array.end()); -#endif - auto nth_element = std::min_element(array.begin() + n, array.end()); + nth_element(array.begin(), array.begin() + n - 1, array.end()); + auto nth_elem = std::min_element(array.begin() + n, array.end()); - return static_cast(array[n - 1]) + (h - n) * static_cast(*nth_element - array[n - 1]); + return static_cast(array[n - 1]) + (h - n) * static_cast(*nth_elem - array[n - 1]); } return std::numeric_limits::quiet_NaN(); @@ -257,14 +233,10 @@ struct QuantileExactInclusive : public QuantileExact result[indices[i]] = static_cast(array[0]); else { -#if !defined(ARCADIA_BUILD) - miniselect::floyd_rivest_select(array.begin() + prev_n, array.begin() + n - 1, array.end()); -#else - std::nth_element(array.begin() + prev_n, array.begin() + n - 1, array.end()); -#endif - auto nth_element = std::min_element(array.begin() + n, array.end()); + nth_element(array.begin() + prev_n, array.begin() + n - 1, array.end()); + auto nth_elem = std::min_element(array.begin() + n, array.end()); - result[indices[i]] = static_cast(array[n - 1]) + (h - n) * static_cast(*nth_element - array[n - 1]); + result[indices[i]] = static_cast(array[n - 1]) + (h - n) * static_cast(*nth_elem - array[n - 1]); prev_n = n - 1; } } diff --git a/src/AggregateFunctions/QuantileTiming.h b/src/AggregateFunctions/QuantileTiming.h index 1a696088dd4..6070f264ad6 100644 --- a/src/AggregateFunctions/QuantileTiming.h +++ b/src/AggregateFunctions/QuantileTiming.h @@ -1,15 +1,13 @@ #pragma once +#include +#include +#include +#include #include #include -#include -#include -#include -#include +#include -#if !defined(ARCADIA_BUILD) - #include // Y_IGNORE -#endif namespace DB { @@ -140,7 +138,7 @@ namespace detail using Array = PODArray; mutable Array elems; /// mutable because array sorting is not considered a state change. - QuantileTimingMedium() {} + QuantileTimingMedium() = default; QuantileTimingMedium(const UInt16 * begin, const UInt16 * end) : elems(begin, end) {} void insert(UInt64 x) @@ -182,11 +180,7 @@ namespace detail /// Sorting an array will not be considered a violation of constancy. auto & array = elems; -#if !defined(ARCADIA_BUILD) - miniselect::floyd_rivest_select(array.begin(), array.begin() + n, array.end()); -#else - std::nth_element(array.begin(), array.begin() + n, array.end()); -#endif + nth_element(array.begin(), array.begin() + n, array.end()); quantile = array[n]; } @@ -207,11 +201,7 @@ namespace detail ? level * elems.size() : (elems.size() - 1); -#if !defined(ARCADIA_BUILD) - miniselect::floyd_rivest_select(array.begin() + prev_n, array.begin() + n, array.end()); -#else - std::nth_element(array.begin() + prev_n, array.begin() + n, array.end()); -#endif + nth_element(array.begin() + prev_n, array.begin() + n, array.end()); result[level_index] = array[n]; prev_n = n; @@ -282,7 +272,7 @@ namespace detail } public: - Iterator(const QuantileTimingLarge & parent) + explicit Iterator(const QuantileTimingLarge & parent) : begin(parent.count_small), pos(begin), end(&parent.count_big[BIG_SIZE]) { adjust(); @@ -429,8 +419,8 @@ namespace detail template void getMany(const double * levels, const size_t * indices, size_t size, ResultType * result) const { - const auto indices_end = indices + size; - auto index = indices; + const auto * indices_end = indices + size; + const auto * index = indices; UInt64 pos = std::ceil(count * levels[*index]); diff --git a/src/Columns/ColumnArray.cpp b/src/Columns/ColumnArray.cpp index 82d954df334..251c0bd7921 100644 --- a/src/Columns/ColumnArray.cpp +++ b/src/Columns/ColumnArray.cpp @@ -9,6 +9,7 @@ #include #include +#include #include @@ -20,10 +21,6 @@ #include #include -#if !defined(ARCADIA_BUILD) - #include // Y_IGNORE -#endif - namespace DB { @@ -786,11 +783,7 @@ void ColumnArray::getPermutationImpl(size_t limit, Permutation & res, Comparator auto less = [&cmp](size_t lhs, size_t rhs){ return cmp(lhs, rhs) < 0; }; if (limit) -#if !defined(ARCADIA_BUILD) - miniselect::floyd_rivest_partial_sort(res.begin(), res.begin() + limit, res.end(), less); -#else - std::partial_sort(res.begin(), res.begin() + limit, res.end(), less); -#endif + partial_sort(res.begin(), res.begin() + limit, res.end(), less); else std::sort(res.begin(), res.end(), less); } @@ -842,11 +835,7 @@ void ColumnArray::updatePermutationImpl(size_t limit, Permutation & res, EqualRa return; /// Since then we are working inside the interval. -#if !defined(ARCADIA_BUILD) - miniselect::floyd_rivest_partial_sort(res.begin() + first, res.begin() + limit, res.begin() + last, less); -#else - std::partial_sort(res.begin() + first, res.begin() + limit, res.begin() + last, less); -#endif + partial_sort(res.begin() + first, res.begin() + limit, res.begin() + last, less); auto new_first = first; for (auto j = first + 1; j < limit; ++j) { diff --git a/src/Columns/ColumnDecimal.cpp b/src/Columns/ColumnDecimal.cpp index 7c3af5fe095..6a0afe55357 100644 --- a/src/Columns/ColumnDecimal.cpp +++ b/src/Columns/ColumnDecimal.cpp @@ -7,10 +7,8 @@ #include #include +#include #include -#if !defined(ARCADIA_BUILD) - #include // Y_IGNORE -#endif #include @@ -197,21 +195,11 @@ void ColumnDecimal::updatePermutation(bool reverse, size_t limit, int, IColum /// Since then we are working inside the interval. if (reverse) -#if !defined(ARCADIA_BUILD) - miniselect::floyd_rivest_partial_sort(res.begin() + first, res.begin() + limit, res.begin() + last, + partial_sort(res.begin() + first, res.begin() + limit, res.begin() + last, [this](size_t a, size_t b) { return data[a] > data[b]; }); -#else - std::partial_sort(res.begin() + first, res.begin() + limit, res.begin() + last, - [this](size_t a, size_t b) { return data[a] > data[b]; }); -#endif else -#if !defined(ARCADIA_BUILD) - miniselect::floyd_rivest_partial_sort(res.begin() + first, res.begin() + limit, res.begin() + last, + partial_sort(res.begin() + first, res.begin() + limit, res.begin() + last, [this](size_t a, size_t b) { return data[a] < data[b]; }); -#else - std::partial_sort(res.begin() + first, res.begin() + limit, res.begin() + last, - [this](size_t a, size_t b) { return data[a] > data[b]; }); -#endif auto new_first = first; for (auto j = first + 1; j < limit; ++j) { diff --git a/src/Columns/ColumnDecimal.h b/src/Columns/ColumnDecimal.h index abb49531e89..e35af1343b2 100644 --- a/src/Columns/ColumnDecimal.h +++ b/src/Columns/ColumnDecimal.h @@ -1,15 +1,13 @@ #pragma once -#include - -#include +#include #include #include -#include #include -#if !defined(ARCADIA_BUILD) - #include // Y_IGNORE -#endif +#include +#include + +#include namespace DB @@ -256,17 +254,9 @@ protected: sort_end = res.begin() + limit; if (reverse) -#if !defined(ARCADIA_BUILD) - miniselect::floyd_rivest_partial_sort(res.begin(), sort_end, res.end(), [this](size_t a, size_t b) { return data[a] > data[b]; }); -#else - std::partial_sort(res.begin(), sort_end, res.end(), [this](size_t a, size_t b) { return data[a] > data[b]; }); -#endif + partial_sort(res.begin(), sort_end, res.end(), [this](size_t a, size_t b) { return data[a] > data[b]; }); else -#if !defined(ARCADIA_BUILD) - miniselect::floyd_rivest_partial_sort(res.begin(), sort_end, res.end(), [this](size_t a, size_t b) { return data[a] < data[b]; }); -#else - std::partial_sort(res.begin(), sort_end, res.end(), [this](size_t a, size_t b) { return data[a] < data[b]; }); -#endif + partial_sort(res.begin(), sort_end, res.end(), [this](size_t a, size_t b) { return data[a] < data[b]; }); } }; diff --git a/src/Columns/ColumnFixedString.cpp b/src/Columns/ColumnFixedString.cpp index c1724b37fe2..a20e5d3ca0d 100644 --- a/src/Columns/ColumnFixedString.cpp +++ b/src/Columns/ColumnFixedString.cpp @@ -1,25 +1,20 @@ #include + #include - -#include -#include -#include -#include -#include -#include -#include - -#include -#if !defined(ARCADIA_BUILD) - #include // Y_IGNORE -#endif - #include - #include +#include +#include +#include +#include +#include +#include +#include +#include +#include -#ifdef __SSE2__ - #include +#if defined(__SSE2__) +# include #endif @@ -160,17 +155,9 @@ void ColumnFixedString::getPermutation(bool reverse, size_t limit, int /*nan_dir if (limit) { if (reverse) -#if !defined(ARCADIA_BUILD) - miniselect::floyd_rivest_partial_sort(res.begin(), res.begin() + limit, res.end(), less(*this)); -#else - std::partial_sort(res.begin(), res.begin() + limit, res.end(), less(*this)); -#endif + partial_sort(res.begin(), res.begin() + limit, res.end(), less(*this)); else -#if !defined(ARCADIA_BUILD) - miniselect::floyd_rivest_partial_sort(res.begin(), res.begin() + limit, res.end(), less(*this)); -#else - std::partial_sort(res.begin(), res.begin() + limit, res.end(), less(*this)); -#endif + partial_sort(res.begin(), res.begin() + limit, res.end(), less(*this)); } else { @@ -228,17 +215,9 @@ void ColumnFixedString::updatePermutation(bool reverse, size_t limit, int, Permu /// Since then we are working inside the interval. if (reverse) -#if !defined(ARCADIA_BUILD) - miniselect::floyd_rivest_partial_sort(res.begin() + first, res.begin() + limit, res.begin() + last, less(*this)); -#else - std::partial_sort(res.begin() + first, res.begin() + limit, res.begin() + last, less(*this)); -#endif + partial_sort(res.begin() + first, res.begin() + limit, res.begin() + last, less(*this)); else -#if !defined(ARCADIA_BUILD) - miniselect::floyd_rivest_partial_sort(res.begin() + first, res.begin() + limit, res.begin() + last, less(*this)); -#else - std::partial_sort(res.begin() + first, res.begin() + limit, res.begin() + last, less(*this)); -#endif + partial_sort(res.begin() + first, res.begin() + limit, res.begin() + last, less(*this)); auto new_first = first; for (auto j = first + 1; j < limit; ++j) diff --git a/src/Columns/ColumnLowCardinality.cpp b/src/Columns/ColumnLowCardinality.cpp index f61062160b1..8af3b240cb9 100644 --- a/src/Columns/ColumnLowCardinality.cpp +++ b/src/Columns/ColumnLowCardinality.cpp @@ -1,20 +1,19 @@ #include -#include + #include +#include #include #include #include -#include #include - +#include +#include #include -#if !defined(ARCADIA_BUILD) - #include // Y_IGNORE -#endif namespace DB { + namespace ErrorCodes { extern const int ILLEGAL_COLUMN; @@ -397,11 +396,7 @@ void ColumnLowCardinality::updatePermutationImpl(size_t limit, Permutation & res /// Since then we are working inside the interval. -#if !defined(ARCADIA_BUILD) - miniselect::floyd_rivest_partial_sort(res.begin() + first, res.begin() + limit, res.begin() + last, less); -#else - std::partial_sort(res.begin() + first, res.begin() + limit, res.begin() + last, less); -#endif + partial_sort(res.begin() + first, res.begin() + limit, res.begin() + last, less); auto new_first = first; for (auto j = first + 1; j < limit; ++j) diff --git a/src/Columns/ColumnString.cpp b/src/Columns/ColumnString.cpp index 49180919abb..00d6349408f 100644 --- a/src/Columns/ColumnString.cpp +++ b/src/Columns/ColumnString.cpp @@ -1,18 +1,16 @@ -#include -#include -#include -#include -#include -#include #include + +#include #include #include - +#include +#include +#include +#include +#include +#include #include #include -#if !defined(ARCADIA_BUILD) - #include // Y_IGNORE -#endif namespace DB @@ -317,11 +315,7 @@ void ColumnString::getPermutationImpl(size_t limit, Permutation & res, Comparato auto less = [&cmp](size_t lhs, size_t rhs){ return cmp(lhs, rhs) < 0; }; if (limit) -#if !defined(ARCADIA_BUILD) - miniselect::floyd_rivest_partial_sort(res.begin(), res.begin() + limit, res.end(), less); -#else - std::partial_sort(res.begin(), res.begin() + limit, res.end(), less); -#endif + partial_sort(res.begin(), res.begin() + limit, res.end(), less); else std::sort(res.begin(), res.end(), less); } @@ -372,11 +366,7 @@ void ColumnString::updatePermutationImpl(size_t limit, Permutation & res, EqualR return; /// Since then we are working inside the interval. -#if !defined(ARCADIA_BUILD) - miniselect::floyd_rivest_partial_sort(res.begin() + first, res.begin() + limit, res.begin() + last, less); -#else - std::partial_sort(res.begin() + first, res.begin() + limit, res.begin() + last, less); -#endif + partial_sort(res.begin() + first, res.begin() + limit, res.begin() + last, less); size_t new_first = first; for (size_t j = first + 1; j < limit; ++j) diff --git a/src/Columns/ColumnTuple.cpp b/src/Columns/ColumnTuple.cpp index 9130f563735..e3b45ee3d5c 100644 --- a/src/Columns/ColumnTuple.cpp +++ b/src/Columns/ColumnTuple.cpp @@ -1,17 +1,16 @@ #include + #include +#include #include -#include #include +#include +#include +#include +#include +#include #include #include -#include -#include -#include -#include -#if !defined(ARCADIA_BUILD) - #include // Y_IGNORE -#endif namespace DB @@ -354,17 +353,9 @@ void ColumnTuple::getPermutationImpl(size_t limit, Permutation & res, LessOperat limit = 0; if (limit) - { -#if !defined(ARCADIA_BUILD) - miniselect::floyd_rivest_partial_sort(res.begin(), res.begin() + limit, res.end(), less); -#else - std::partial_sort(res.begin(), res.begin() + limit, res.end(), less); -#endif - } + partial_sort(res.begin(), res.begin() + limit, res.end(), less); else - { std::sort(res.begin(), res.end(), less); - } } void ColumnTuple::updatePermutationImpl(bool reverse, size_t limit, int nan_direction_hint, IColumn::Permutation & res, EqualRanges & equal_ranges, const Collator * collator) const diff --git a/src/Columns/ColumnVector.cpp b/src/Columns/ColumnVector.cpp index c02963e4c5a..d768757227b 100644 --- a/src/Columns/ColumnVector.cpp +++ b/src/Columns/ColumnVector.cpp @@ -1,28 +1,27 @@ #include "ColumnVector.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 -#if !defined(ARCADIA_BUILD) - #include // Y_IGNORE -#endif -#ifdef __SSE2__ - #include +#include +#include + +#if defined(__SSE2__) +# include #endif namespace DB @@ -158,17 +157,9 @@ void ColumnVector::getPermutation(bool reverse, size_t limit, int nan_directi res[i] = i; if (reverse) -#if !defined(ARCADIA_BUILD) - miniselect::floyd_rivest_partial_sort(res.begin(), res.begin() + limit, res.end(), greater(*this, nan_direction_hint)); -#else - std::partial_sort(res.begin(), res.begin() + limit, res.end(), greater(*this, nan_direction_hint)); -#endif + partial_sort(res.begin(), res.begin() + limit, res.end(), greater(*this, nan_direction_hint)); else -#if !defined(ARCADIA_BUILD) - miniselect::floyd_rivest_partial_sort(res.begin(), res.begin() + limit, res.end(), less(*this, nan_direction_hint)); -#else - std::partial_sort(res.begin(), res.begin() + limit, res.end(), less(*this, nan_direction_hint)); -#endif + partial_sort(res.begin(), res.begin() + limit, res.end(), less(*this, nan_direction_hint)); } else { @@ -264,17 +255,9 @@ void ColumnVector::updatePermutation(bool reverse, size_t limit, int nan_dire /// Since then, we are working inside the interval. if (reverse) -#if !defined(ARCADIA_BUILD) - miniselect::floyd_rivest_partial_sort(res.begin() + first, res.begin() + limit, res.begin() + last, greater(*this, nan_direction_hint)); -#else - std::partial_sort(res.begin() + first, res.begin() + limit, res.begin() + last, greater(*this, nan_direction_hint)); -#endif + partial_sort(res.begin() + first, res.begin() + limit, res.begin() + last, greater(*this, nan_direction_hint)); else -#if !defined(ARCADIA_BUILD) - miniselect::floyd_rivest_partial_sort(res.begin() + first, res.begin() + limit, res.begin() + last, less(*this, nan_direction_hint)); -#else - std::partial_sort(res.begin() + first, res.begin() + limit, res.begin() + last, less(*this, nan_direction_hint)); -#endif + partial_sort(res.begin() + first, res.begin() + limit, res.begin() + last, less(*this, nan_direction_hint)); size_t new_first = first; for (size_t j = first + 1; j < limit; ++j) diff --git a/src/Columns/ColumnVectorHelper.h b/src/Columns/ColumnVectorHelper.h index d805f44218c..36cbfbf640e 100644 --- a/src/Columns/ColumnVectorHelper.h +++ b/src/Columns/ColumnVectorHelper.h @@ -1,6 +1,7 @@ #pragma once #include +#include namespace DB From 0e4cbf7508e47f7d9a7b29ca55887dc6f6101a5b Mon Sep 17 00:00:00 2001 From: filimonov <1549571+filimonov@users.noreply.github.com> Date: Fri, 13 Nov 2020 12:31:37 +0100 Subject: [PATCH 194/205] Update programs/install/Install.cpp --- programs/install/Install.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/programs/install/Install.cpp b/programs/install/Install.cpp index 27f4ceec732..da22452819a 100644 --- a/programs/install/Install.cpp +++ b/programs/install/Install.cpp @@ -394,7 +394,7 @@ int mainEntryClickHouseInstall(int argc, char ** argv) } else { - fmt::print("Users config file {} already exists, will keep it and extract users info from it.\n", main_config_file.string()); + fmt::print("Users config file {} already exists, will keep it and extract users info from it.\n", users_config_file.string()); /// Check if password for default user already specified. ConfigProcessor processor(users_config_file.string(), /* throw_on_bad_incl = */ false, /* log_to_console = */ false); From c3751e88257c2b6d8d13f0660fb0849ac62db214 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 13 Nov 2020 15:09:27 +0300 Subject: [PATCH 195/205] Added results from Intel --- .../hardware/results/xeon_clx_6230r.json | 56 +++++++++++++++++++ 1 file changed, 56 insertions(+) create mode 100644 website/benchmark/hardware/results/xeon_clx_6230r.json diff --git a/website/benchmark/hardware/results/xeon_clx_6230r.json b/website/benchmark/hardware/results/xeon_clx_6230r.json new file mode 100644 index 00000000000..ee7c55a90f9 --- /dev/null +++ b/website/benchmark/hardware/results/xeon_clx_6230r.json @@ -0,0 +1,56 @@ +[ + { + "system": "2x Intel CLX 6230R", + "system_full": "2x Intel CLX 6230R, 384 GB RAM, P4600 1.6 TB ext4", + "cpu_vendor": "Intel", + "cpu_model": "2x Intel CLX 6230R", + "time": "2020-11-13 00:00:00", + "kind": "server", + "result": + [ +[0.016, 0.003, 0.003], +[0.042, 0.023, 0.022], +[0.036, 0.031, 0.032], +[0.083, 0.037, 0.036], +[0.125, 0.111, 0.109], +[0.206, 0.171, 0.172], +[0.027, 0.028, 0.028], +[0.021, 0.023, 0.023], +[0.232, 0.196, 0.195], +[0.260, 0.213, 0.210], +[0.142, 0.100, 0.103], +[0.141, 0.120, 0.109], +[0.318, 0.255, 0.256], +[0.394, 0.316, 0.311], +[0.345, 0.301, 0.306], +[0.332, 0.333, 0.318], +[0.782, 0.734, 0.719], +[0.494, 0.423, 0.412], +[1.502, 1.379, 1.429], +[0.094, 0.054, 0.057], +[0.860, 0.295, 0.302], +[0.978, 0.343, 0.344], +[1.767, 0.823, 0.827], +[2.214, 0.468, 0.443], +[0.258, 0.116, 0.121], +[0.142, 0.101, 0.104], +[0.254, 0.113, 0.113], +[0.860, 0.354, 0.356], +[0.729, 0.449, 0.447], +[0.846, 0.847, 0.854], +[0.322, 0.247, 0.245], +[0.582, 0.343, 0.348], +[2.110, 2.008, 2.031], +[1.417, 1.158, 1.135], +[1.360, 1.069, 1.084], +[0.377, 0.371, 0.357], +[0.228, 0.160, 0.157], +[0.092, 0.077, 0.076], +[0.053, 0.043, 0.057], +[0.423, 0.343, 0.348], +[0.051, 0.032, 0.034], +[0.023, 0.033, 0.035], +[0.015, 0.010, 0.016] + ] + } +] From b8e33bd35b5b9329025301b591166aec680f7f5d Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 13 Nov 2020 15:42:26 +0300 Subject: [PATCH 196/205] Add comments --- src/Storages/MergeTree/MergeTreeRangeReader.cpp | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/src/Storages/MergeTree/MergeTreeRangeReader.cpp b/src/Storages/MergeTree/MergeTreeRangeReader.cpp index ecf0d59b42b..66b75a15289 100644 --- a/src/Storages/MergeTree/MergeTreeRangeReader.cpp +++ b/src/Storages/MergeTree/MergeTreeRangeReader.cpp @@ -637,6 +637,7 @@ MergeTreeRangeReader::ReadResult MergeTreeRangeReader::read(size_t max_rows, Mar if (!columns.empty()) { + /// If some columns absent in part, than evaulate default values if (should_evaluate_missing_defaults) { auto block = prev_reader->sample_block.cloneWithColumns(read_result.columns); @@ -661,6 +662,7 @@ MergeTreeRangeReader::ReadResult MergeTreeRangeReader::read(size_t max_rows, Mar } merge_tree_reader->evaluateMissingDefaults(block, columns); } + /// If columns not empty, than apply on-fly alter conversions if any required merge_tree_reader->performRequiredConversions(columns); } @@ -679,9 +681,11 @@ MergeTreeRangeReader::ReadResult MergeTreeRangeReader::read(size_t max_rows, Mar merge_tree_reader->fillMissingColumns(read_result.columns, should_evaluate_missing_defaults, read_result.num_rows); + /// If some columns absent in part, than evaulate default values if (should_evaluate_missing_defaults) merge_tree_reader->evaluateMissingDefaults({}, read_result.columns); + /// If result not empty, than apply on-fly alter conversions if any required merge_tree_reader->performRequiredConversions(read_result.columns); } else From b5a8ef3cebcc9a1471ec71c17a41b9ed78789945 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 13 Nov 2020 15:51:50 +0300 Subject: [PATCH 197/205] Added test from #16588 --- .../01576_if_null_external_aggregation.reference | 0 .../0_stateless/01576_if_null_external_aggregation.sql | 7 +++++++ 2 files changed, 7 insertions(+) create mode 100644 tests/queries/0_stateless/01576_if_null_external_aggregation.reference create mode 100644 tests/queries/0_stateless/01576_if_null_external_aggregation.sql diff --git a/tests/queries/0_stateless/01576_if_null_external_aggregation.reference b/tests/queries/0_stateless/01576_if_null_external_aggregation.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/01576_if_null_external_aggregation.sql b/tests/queries/0_stateless/01576_if_null_external_aggregation.sql new file mode 100644 index 00000000000..b9c36a9cecc --- /dev/null +++ b/tests/queries/0_stateless/01576_if_null_external_aggregation.sql @@ -0,0 +1,7 @@ +SET max_bytes_before_external_group_by = 200000000; + +SET max_memory_usage = 1500000000; +SET max_threads = 12; + +SELECT bitAnd(number, pow(2, 20) - 1) as k, argMaxIf(k, number % 2 = 0 ? number : Null, number > 42), uniq(number) AS u FROM numbers(1000000) GROUP BY k format Null; + From a1b133828178b2a836be5341d0088056be5c0e22 Mon Sep 17 00:00:00 2001 From: Anna <42538400+adevyatova@users.noreply.github.com> Date: Fri, 13 Nov 2020 16:08:38 +0300 Subject: [PATCH 198/205] DOCSUP-2869: Document section user_directories in the main config (#16340) * DOCSUP-2869: Document section user_directories in the main config * Update settings.md * Update * Commit for DOCSUP-3418 * Update * Russian translation * Fix typo --- .../settings.md | 41 +++++++++++++++++++ docs/en/sql-reference/statements/system.md | 2 +- docs/en/sql-reference/syntax.md | 2 +- .../settings.md | 41 +++++++++++++++++++ 4 files changed, 84 insertions(+), 2 deletions(-) diff --git a/docs/en/operations/server-configuration-parameters/settings.md b/docs/en/operations/server-configuration-parameters/settings.md index a37ae685368..e111cf3ab75 100644 --- a/docs/en/operations/server-configuration-parameters/settings.md +++ b/docs/en/operations/server-configuration-parameters/settings.md @@ -1081,4 +1081,45 @@ Default value: `/var/lib/clickhouse/access/`. - [Access Control and Account Management](../../operations/access-rights.md#access-control) +## user_directories {#user_directories} + +Section of the configuration file that contains settings: +- Path to configuration file with predefined users. +- Path to folder where users created by SQL commands are stored. + +If this section is specified, the path from [users_config](../../operations/server-configuration-parameters/settings.md#users-config) and [access_control_path](../../operations/server-configuration-parameters/settings.md#access_control_path) won't be used. + +The `user_directories` section can contain any number of items, the order of the items means their precedence (the higher the item the higher the precedence). + +**Example** + +``` xml + + + /etc/clickhouse-server/users.xml + + + /var/lib/clickhouse/access/ + + +``` + +You can also specify settings `memory` — means storing information only in memory, without writing to disk, and `ldap` — means storing information on an LDAP server. + +To add an LDAP server as a remote user directory of users that are not defined locally, define a single `ldap` section with a following parameters: +- `server` — one of LDAP server names defined in `ldap_servers` config section. This parameter is mandatory and cannot be empty. +- `roles` — section with a list of locally defined roles that will be assigned to each user retrieved from the LDAP server. If no roles are specified, user will not be able to perform any actions after authentication. If any of the listed roles is not defined locally at the time of authentication, the authenthication attept will fail as if the provided password was incorrect. + +**Example** + +``` xml + + my_ldap_server + + + + + +``` + [Original article](https://clickhouse.tech/docs/en/operations/server_configuration_parameters/settings/) diff --git a/docs/en/sql-reference/statements/system.md b/docs/en/sql-reference/statements/system.md index 75303fde19e..509b7553536 100644 --- a/docs/en/sql-reference/statements/system.md +++ b/docs/en/sql-reference/statements/system.md @@ -204,7 +204,7 @@ SYSTEM STOP MOVES [[db.]merge_tree_family_table_name] ## Managing ReplicatedMergeTree Tables {#query-language-system-replicated} -ClickHouse can manage background replication related processes in [ReplicatedMergeTree](../../engines/table-engines/mergetree-family/replacingmergetree.md) tables. +ClickHouse can manage background replication related processes in [ReplicatedMergeTree](../../engines/table-engines/mergetree-family/replication/#table_engines-replication) tables. ### STOP FETCHES {#query_language-system-stop-fetches} diff --git a/docs/en/sql-reference/syntax.md b/docs/en/sql-reference/syntax.md index 70994f3d882..296f5c7c5f3 100644 --- a/docs/en/sql-reference/syntax.md +++ b/docs/en/sql-reference/syntax.md @@ -57,7 +57,7 @@ Identifiers are: Identifiers can be quoted or non-quoted. The latter is preferred. -Non-quoted identifiers must match the regex `^[a-zA-Z_][0-9a-zA-Z_]*$` and can not be equal to [keywords](#syntax-keywords). Examples: `x, _1, X_y__Z123_.` +Non-quoted identifiers must match the regex `^[0-9a-zA-Z_]*[a-zA-Z_]$` and can not be equal to [keywords](#syntax-keywords). Examples: `x, _1, X_y__Z123_.` If you want to use identifiers the same as keywords or you want to use other symbols in identifiers, quote it using double quotes or backticks, for example, `"id"`, `` `id` ``. diff --git a/docs/ru/operations/server-configuration-parameters/settings.md b/docs/ru/operations/server-configuration-parameters/settings.md index 2745718381b..9941e4f3ac5 100644 --- a/docs/ru/operations/server-configuration-parameters/settings.md +++ b/docs/ru/operations/server-configuration-parameters/settings.md @@ -1068,4 +1068,45 @@ ClickHouse использует ZooKeeper для хранения метадан - [Управление доступом](../access-rights.md#access-control) +## user_directories {#user_directories} + +Секция конфигурационного файла,которая содержит настройки: +- Путь к конфигурационному файлу с предустановленными пользователями. +- Путь к файлу, в котором содержатся пользователи, созданные при помощи SQL команд. + +Если эта секция определена, путь из [users_config](../../operations/server-configuration-parameters/settings.md#users-config) и [access_control_path](../../operations/server-configuration-parameters/settings.md#access_control_path) не используется. + +Секция `user_directories` может содержать любое количество элементов, порядок расположения элементов обозначает их приоритет (чем выше элемент, тем выше приоритет). + +**Пример** + +``` xml + + + /etc/clickhouse-server/users.xml + + + /var/lib/clickhouse/access/ + + +``` + +Также вы можете указать настройку `memory` — означает хранение информации только в памяти, без записи на диск, и `ldap` — означает хранения информации на [LDAP-сервере](https://en.wikipedia.org/wiki/Lightweight_Directory_Access_Protocol). + +Чтобы добавить LDAP-сервер в качестве удаленного каталога пользователей, которые не определены локально, определите один раздел `ldap` со следующими параметрами: +- `server` — имя одного из LDAP-серверов, определенных в секции `ldap_servers` конфигурациионного файла. Этот параметр явялется необязательным и может быть пустым. +- `roles` — раздел со списком локально определенных ролей, которые будут назначены каждому пользователю, полученному с LDAP-сервера. Если роли не заданы, пользователь не сможет выполнять никаких действий после аутентификации. Если какая-либо из перечисленных ролей не определена локально во время проверки подлинности, попытка проверки подлинности завершится неудачей, как если бы предоставленный пароль был неверным. + +**Пример** + +``` xml + + my_ldap_server + + + + + +``` + [Оригинальная статья](https://clickhouse.tech/docs/ru/operations/server_configuration_parameters/settings/) From b2a8b6135dda680c243fd7f413dd2ab63d03b266 Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 13 Nov 2020 16:24:14 +0300 Subject: [PATCH 199/205] then --- src/Storages/MergeTree/MergeTreeRangeReader.cpp | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeRangeReader.cpp b/src/Storages/MergeTree/MergeTreeRangeReader.cpp index 66b75a15289..e22c6bb3d0c 100644 --- a/src/Storages/MergeTree/MergeTreeRangeReader.cpp +++ b/src/Storages/MergeTree/MergeTreeRangeReader.cpp @@ -637,7 +637,7 @@ MergeTreeRangeReader::ReadResult MergeTreeRangeReader::read(size_t max_rows, Mar if (!columns.empty()) { - /// If some columns absent in part, than evaulate default values + /// If some columns absent in part, then evaulate default values if (should_evaluate_missing_defaults) { auto block = prev_reader->sample_block.cloneWithColumns(read_result.columns); @@ -662,7 +662,7 @@ MergeTreeRangeReader::ReadResult MergeTreeRangeReader::read(size_t max_rows, Mar } merge_tree_reader->evaluateMissingDefaults(block, columns); } - /// If columns not empty, than apply on-fly alter conversions if any required + /// If columns not empty, then apply on-fly alter conversions if any required merge_tree_reader->performRequiredConversions(columns); } @@ -681,11 +681,11 @@ MergeTreeRangeReader::ReadResult MergeTreeRangeReader::read(size_t max_rows, Mar merge_tree_reader->fillMissingColumns(read_result.columns, should_evaluate_missing_defaults, read_result.num_rows); - /// If some columns absent in part, than evaulate default values + /// If some columns absent in part, then evaulate default values if (should_evaluate_missing_defaults) merge_tree_reader->evaluateMissingDefaults({}, read_result.columns); - /// If result not empty, than apply on-fly alter conversions if any required + /// If result not empty, then apply on-fly alter conversions if any required merge_tree_reader->performRequiredConversions(read_result.columns); } else From 5ef2eaeeef32298559ca81deeca416015094ee17 Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 13 Nov 2020 16:25:18 +0300 Subject: [PATCH 200/205] Trying to fix some flaky tests --- .../00804_test_delta_codec_compression.reference | 2 +- .../00804_test_delta_codec_compression.sql | 12 ++++++------ ...40_dictionary_invalidate_query_switchover_long.sh | 10 ++++------ 3 files changed, 11 insertions(+), 13 deletions(-) diff --git a/tests/queries/0_stateless/00804_test_delta_codec_compression.reference b/tests/queries/0_stateless/00804_test_delta_codec_compression.reference index 79302586e92..949d37ed27a 100644 --- a/tests/queries/0_stateless/00804_test_delta_codec_compression.reference +++ b/tests/queries/0_stateless/00804_test_delta_codec_compression.reference @@ -1,4 +1,4 @@ -83 +84 1 46 1 diff --git a/tests/queries/0_stateless/00804_test_delta_codec_compression.sql b/tests/queries/0_stateless/00804_test_delta_codec_compression.sql index 91bc45df63d..6da43298347 100644 --- a/tests/queries/0_stateless/00804_test_delta_codec_compression.sql +++ b/tests/queries/0_stateless/00804_test_delta_codec_compression.sql @@ -23,9 +23,9 @@ OPTIMIZE TABLE default_codec_synthetic FINAL; SELECT floor(big_size / small_size) AS ratio FROM - (SELECT 1 AS key, sum(bytes_on_disk) AS small_size FROM system.parts WHERE database == currentDatabase() and table == 'delta_codec_synthetic') + (SELECT 1 AS key, sum(bytes_on_disk) AS small_size FROM system.parts WHERE database == currentDatabase() and table == 'delta_codec_synthetic' and active) INNER JOIN - (SELECT 1 AS key, sum(bytes_on_disk) as big_size FROM system.parts WHERE database == currentDatabase() and table == 'default_codec_synthetic') + (SELECT 1 AS key, sum(bytes_on_disk) as big_size FROM system.parts WHERE database == currentDatabase() and table == 'default_codec_synthetic' and active) USING(key); SELECT @@ -61,9 +61,9 @@ OPTIMIZE TABLE default_codec_float FINAL; SELECT floor(big_size / small_size) as ratio FROM - (SELECT 1 AS key, sum(bytes_on_disk) AS small_size FROM system.parts WHERE database = currentDatabase() and table = 'delta_codec_float') + (SELECT 1 AS key, sum(bytes_on_disk) AS small_size FROM system.parts WHERE database = currentDatabase() and table = 'delta_codec_float' and active) INNER JOIN - (SELECT 1 AS key, sum(bytes_on_disk) as big_size FROM system.parts WHERE database = currentDatabase() and table = 'default_codec_float') USING(key); + (SELECT 1 AS key, sum(bytes_on_disk) as big_size FROM system.parts WHERE database = currentDatabase() and table = 'default_codec_float' and active) USING(key); SELECT small_hash == big_hash @@ -99,9 +99,9 @@ OPTIMIZE TABLE default_codec_string FINAL; SELECT floor(big_size / small_size) as ratio FROM - (SELECT 1 AS key, sum(bytes_on_disk) AS small_size FROM system.parts WHERE database = currentDatabase() and table = 'delta_codec_string') + (SELECT 1 AS key, sum(bytes_on_disk) AS small_size FROM system.parts WHERE database = currentDatabase() and table = 'delta_codec_string' and active) INNER JOIN - (SELECT 1 AS key, sum(bytes_on_disk) as big_size FROM system.parts WHERE database = currentDatabase() and table = 'default_codec_string') USING(key); + (SELECT 1 AS key, sum(bytes_on_disk) as big_size FROM system.parts WHERE database = currentDatabase() and table = 'default_codec_string' and active) USING(key); SELECT small_hash == big_hash diff --git a/tests/queries/0_stateless/01040_dictionary_invalidate_query_switchover_long.sh b/tests/queries/0_stateless/01040_dictionary_invalidate_query_switchover_long.sh index 6b509ac7925..6879fedf978 100755 --- a/tests/queries/0_stateless/01040_dictionary_invalidate_query_switchover_long.sh +++ b/tests/queries/0_stateless/01040_dictionary_invalidate_query_switchover_long.sh @@ -30,12 +30,9 @@ LAYOUT(FLAT())" $CLICKHOUSE_CLIENT --query "SELECT dictGetUInt8('dictdb.invalidate', 'two', toUInt64(122))" +# No exception happened $CLICKHOUSE_CLIENT --query "SELECT last_exception FROM system.dictionaries WHERE database = 'dictdb' AND name = 'invalidate'" -# Bad solution, but it's quite complicated to detect, that invalidte_query stopped updates. -# In worst case we don't check anything, but fortunately it doesn't lead to false negatives. -sleep 5 - $CLICKHOUSE_CLIENT --query "DROP TABLE dictdb.dict_invalidate" function check_exception_detected() @@ -52,7 +49,7 @@ function check_exception_detected() export -f check_exception_detected; -timeout 10 bash -c check_exception_detected 2> /dev/null +timeout 30 bash -c check_exception_detected 2> /dev/null $CLICKHOUSE_CLIENT --query "SELECT last_exception FROM system.dictionaries WHERE database = 'dictdb' AND name = 'invalidate'" 2>&1 | grep -Eo "Table dictdb.dict_invalidate .* exist." @@ -76,7 +73,8 @@ function check_exception_fixed() } export -f check_exception_fixed; -timeout 10 bash -c check_exception_fixed 2> /dev/null +# it may take a long until dictionary reloads +timeout 60 bash -c check_exception_fixed 2> /dev/null $CLICKHOUSE_CLIENT --query "SELECT last_exception FROM system.dictionaries WHERE database = 'dictdb' AND name = 'invalidate'" 2>&1 $CLICKHOUSE_CLIENT --query "SELECT dictGetUInt8('dictdb.invalidate', 'two', toUInt64(133))" From 14485238b1b82375513f00920346d35293ca5b7e Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 13 Nov 2020 16:28:04 +0300 Subject: [PATCH 201/205] Update 01040_dictionary_invalidate_query_switchover_long.sh --- .../01040_dictionary_invalidate_query_switchover_long.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/01040_dictionary_invalidate_query_switchover_long.sh b/tests/queries/0_stateless/01040_dictionary_invalidate_query_switchover_long.sh index 6879fedf978..f9b4573bfb4 100755 --- a/tests/queries/0_stateless/01040_dictionary_invalidate_query_switchover_long.sh +++ b/tests/queries/0_stateless/01040_dictionary_invalidate_query_switchover_long.sh @@ -73,7 +73,7 @@ function check_exception_fixed() } export -f check_exception_fixed; -# it may take a long until dictionary reloads +# it may take a while until dictionary reloads timeout 60 bash -c check_exception_fixed 2> /dev/null $CLICKHOUSE_CLIENT --query "SELECT last_exception FROM system.dictionaries WHERE database = 'dictdb' AND name = 'invalidate'" 2>&1 From 5ae81f4b59c2df1561e684727c207b469e5e1e2d Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 13 Nov 2020 18:41:00 +0300 Subject: [PATCH 202/205] Remove commented code. --- src/Core/iostream_debug_helpers.cpp | 6 ------ src/Core/iostream_debug_helpers.h | 3 --- 2 files changed, 9 deletions(-) diff --git a/src/Core/iostream_debug_helpers.cpp b/src/Core/iostream_debug_helpers.cpp index f64d1a5ae2d..8dc8a4244ac 100644 --- a/src/Core/iostream_debug_helpers.cpp +++ b/src/Core/iostream_debug_helpers.cpp @@ -106,12 +106,6 @@ std::ostream & operator<<(std::ostream & stream, const Packet & what) return stream; } -//std::ostream & operator<<(std::ostream & stream, const ExpressionAction & what) -//{ -// stream << "ExpressionAction(" << what.toString() << ")"; -// return stream; -//} - std::ostream & operator<<(std::ostream & stream, const ExpressionActions & what) { stream << "ExpressionActions(" << what.dumpActions() << ")"; diff --git a/src/Core/iostream_debug_helpers.h b/src/Core/iostream_debug_helpers.h index 4005508e4b8..7568fa6e445 100644 --- a/src/Core/iostream_debug_helpers.h +++ b/src/Core/iostream_debug_helpers.h @@ -40,9 +40,6 @@ std::ostream & operator<<(std::ostream & stream, const IColumn & what); struct Packet; std::ostream & operator<<(std::ostream & stream, const Packet & what); -//struct ExpressionAction; -//std::ostream & operator<<(std::ostream & stream, const ExpressionAction & what); - class ExpressionActions; std::ostream & operator<<(std::ostream & stream, const ExpressionActions & what); From 7f7f66a1f1c694daab2b1359e45b9eff7d255f46 Mon Sep 17 00:00:00 2001 From: filimonov <1549571+filimonov@users.noreply.github.com> Date: Fri, 13 Nov 2020 17:45:58 +0100 Subject: [PATCH 203/205] add comment & restart CI... --- src/Client/Connection.cpp | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/src/Client/Connection.cpp b/src/Client/Connection.cpp index b810d24b3d3..0a499540c41 100644 --- a/src/Client/Connection.cpp +++ b/src/Client/Connection.cpp @@ -73,6 +73,10 @@ void Connection::connect(const ConnectionTimeouts & timeouts) { #if USE_SSL socket = std::make_unique(); + + /// we resolve the ip when we open SecureStreamSocket, so to make Server Name Indication (SNI) + /// work we need to pass host name separately. It will be send into TLS Hello packet to let + /// the server know which host we want to talk with (single IP can process requests for multiple hosts using SNI). static_cast(socket.get())->setPeerHostName(host); #else throw Exception{"tcp_secure protocol is disabled because poco library was built without NetSSL support.", ErrorCodes::SUPPORT_IS_DISABLED}; From a6f6e5c408c8effd823aecdc895a103dad270fa2 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Fri, 13 Nov 2020 20:40:02 +0300 Subject: [PATCH 204/205] Update version_date.tsv after release 20.8.6.6 --- utils/list-versions/version_date.tsv | 1 + 1 file changed, 1 insertion(+) diff --git a/utils/list-versions/version_date.tsv b/utils/list-versions/version_date.tsv index f636dd250f4..fde3ced83e9 100644 --- a/utils/list-versions/version_date.tsv +++ b/utils/list-versions/version_date.tsv @@ -7,6 +7,7 @@ v20.9.5.5-stable 2020-11-13 v20.9.4.76-stable 2020-10-29 v20.9.3.45-stable 2020-10-09 v20.9.2.20-stable 2020-09-22 +v20.8.6.6-lts 2020-11-13 v20.8.5.45-lts 2020-10-29 v20.8.4.11-lts 2020-10-09 v20.8.3.18-stable 2020-09-18 From 33fcbf813f6d18106682c4a4243fbf9c078580fe Mon Sep 17 00:00:00 2001 From: Alexander Kuzmenkov Date: Fri, 13 Nov 2020 20:43:05 +0300 Subject: [PATCH 205/205] 20.8 --- CHANGELOG.md | 13 +++++++++++++ 1 file changed, 13 insertions(+) diff --git a/CHANGELOG.md b/CHANGELOG.md index f196d4421d8..355c664664d 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -510,6 +510,19 @@ ## ClickHouse release 20.8 +### ClickHouse release v20.8.6.6-lts, 2020-11-13 + +#### Bug Fix + +* Fix rare silent crashes when query profiler is on and ClickHouse is installed on OS with glibc version that has (supposedly) broken asynchronous unwind tables for some functions. This fixes [#15301](https://github.com/ClickHouse/ClickHouse/issues/15301). This fixes [#13098](https://github.com/ClickHouse/ClickHouse/issues/13098). [#16846](https://github.com/ClickHouse/ClickHouse/pull/16846) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Now when parsing AVRO from input the LowCardinality is removed from type. Fixes [#16188](https://github.com/ClickHouse/ClickHouse/issues/16188). [#16521](https://github.com/ClickHouse/ClickHouse/pull/16521) ([Mike](https://github.com/myrrc)). +* Fix rapid growth of metadata when using MySQL Master -> MySQL Slave -> ClickHouse MaterializeMySQL Engine, and `slave_parallel_worker` enabled on MySQL Slave, by properly shrinking GTID sets. This fixes [#15951](https://github.com/ClickHouse/ClickHouse/issues/15951). [#16504](https://github.com/ClickHouse/ClickHouse/pull/16504) ([TCeason](https://github.com/TCeason)). +* Fix DROP TABLE for Distributed (racy with INSERT). [#16409](https://github.com/ClickHouse/ClickHouse/pull/16409) ([Azat Khuzhin](https://github.com/azat)). +* Fix processing of very large entries in replication queue. Very large entries may appear in ALTER queries if table structure is extremely large (near 1 MB). This fixes [#16307](https://github.com/ClickHouse/ClickHouse/issues/16307). [#16332](https://github.com/ClickHouse/ClickHouse/pull/16332) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fixed the inconsistent behaviour when a part of return data could be dropped because the set for its filtration wasn't created. [#16308](https://github.com/ClickHouse/ClickHouse/pull/16308) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). +* Fix bug with MySQL database. When MySQL server used as database engine is down some queries raise Exception, because they try to get tables from disabled server, while it's unnecessary. For example, query `SELECT ... FROM system.parts` should work only with MergeTree tables and don't touch MySQL database at all. [#16032](https://github.com/ClickHouse/ClickHouse/pull/16032) ([Kruglov Pavel](https://github.com/Avogar)). + + ### ClickHouse release v20.8.5.45-lts, 2020-10-29 #### Bug Fix