From a5ed6409adaade65bab8ab2b51b019fb056c1751 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 10 Sep 2020 10:30:03 +0300 Subject: [PATCH 001/321] Add ExpressionDAG [Part 1]. --- src/Interpreters/ActionsVisitor.cpp | 70 +++++++------ src/Interpreters/ActionsVisitor.h | 76 ++++++++------ src/Interpreters/ExpressionActions.cpp | 139 +++++++++++++++++++++++++ src/Interpreters/ExpressionActions.h | 54 ++++++++++ 4 files changed, 273 insertions(+), 66 deletions(-) diff --git a/src/Interpreters/ActionsVisitor.cpp b/src/Interpreters/ActionsVisitor.cpp index 9d6d5f783ff..0742479d34f 100644 --- a/src/Interpreters/ActionsVisitor.cpp +++ b/src/Interpreters/ActionsVisitor.cpp @@ -381,52 +381,56 @@ SetPtr makeExplicitSet( return set; } -ScopeStack::ScopeStack(const ExpressionActionsPtr & actions, const Context & context_) +ScopeStack::ScopeStack(ActionsDAGPtr actions, const Context & context_) : context(context_) { - stack.emplace_back(); - stack.back().actions = actions; - - const Block & sample_block = actions->getSampleBlock(); - for (size_t i = 0, size = sample_block.columns(); i < size; ++i) - stack.back().new_columns.insert(sample_block.getByPosition(i).name); + stack.emplace_back(std::move(actions)); } void ScopeStack::pushLevel(const NamesAndTypesList & input_columns) { - stack.emplace_back(); - Level & prev = stack[stack.size() - 2]; - - ColumnsWithTypeAndName all_columns; - NameSet new_names; + auto & actions = stack.emplace_back(std::make_shared()); + const auto & prev = stack[stack.size() - 2]; for (const auto & input_column : input_columns) - { - all_columns.emplace_back(nullptr, input_column.type, input_column.name); - new_names.insert(input_column.name); - stack.back().new_columns.insert(input_column.name); - } + actions->addInput(input_column.name, input_column.type); - const Block & prev_sample_block = prev.actions->getSampleBlock(); - for (size_t i = 0, size = prev_sample_block.columns(); i < size; ++i) - { - const ColumnWithTypeAndName & col = prev_sample_block.getByPosition(i); - if (!new_names.count(col.name)) - all_columns.push_back(col); - } + const auto & index = actions->getIndex(); - stack.back().actions = std::make_shared(all_columns, context); + for (const auto & [name, node] : prev->getIndex()) + { + if (index.count(name) == 0) + actions->addInput(node->result_name, node->result_type); + } } size_t ScopeStack::getColumnLevel(const std::string & name) { for (int i = static_cast(stack.size()) - 1; i >= 0; --i) - if (stack[i].new_columns.count(name)) + if (stack[i]->getIndex().count(name)) return i; throw Exception("Unknown identifier: " + name, ErrorCodes::UNKNOWN_IDENTIFIER); } +void ScopeStack::addAlias(const std::string & name, std::string alias) +{ + auto level = getColumnLevel(name); + const auto & node = stack[level]->addAlias(name, std::move(alias)); + + for (size_t j = level + 1; j < stack.size(); ++j) + stack[j]->addInput(node.result_name, node.result_type); +} + +void ScopeStack::addArrayJoin(const std::string & source_name, std::string result_name) +{ + auto level = getColumnLevel(source_name); + const auto & node = stack[level]->addAlias(source_name, std::move(result_name)); + + for (size_t j = level + 1; j < stack.size(); ++j) + stack[j]->addInput(node.result_name, node.result_type); +} + void ScopeStack::addAction(const ExpressionAction & action) { size_t level = 0; @@ -460,16 +464,16 @@ void ScopeStack::addActionNoInput(const ExpressionAction & action) stack[level].new_columns.insert(added.begin(), added.end()); } -ExpressionActionsPtr ScopeStack::popLevel() +ActionsDAGPtr ScopeStack::popLevel() { - ExpressionActionsPtr res = stack.back().actions; + auto res = std::move(stack.back()); stack.pop_back(); return res; } -const Block & ScopeStack::getSampleBlock() const +const ActionsDAG::Index & ScopeStack::getIndex() const { - return stack.back().actions->getSampleBlock(); + return stack.back()->getIndex(); } struct CachedColumnName @@ -532,7 +536,7 @@ void ActionsMatcher::visit(const ASTIdentifier & identifier, const ASTPtr & ast, /// Special check for WITH statement alias. Add alias action to be able to use this alias. if (identifier.prefer_alias_to_column_name && !identifier.alias.empty()) - data.addAction(ExpressionAction::addAliases({{identifier.name, identifier.alias}})); + data.addAlias(identifier.name, identifier.alias); } } @@ -562,8 +566,8 @@ void ActionsMatcher::visit(const ASTFunction & node, const ASTPtr & ast, Data & /// 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. String tmp_name = data.getUniqueName("_array_join_" + arg->getColumnName()); - data.addActionNoInput(ExpressionAction::copyColumn(arg->getColumnName(), tmp_name)); - data.addAction(ExpressionAction::arrayJoin(tmp_name, result_name)); + data.addAlias(arg->getColumnName(), tmp_name); + data.addArrayJoin(tmp_name, result_name); } return; diff --git a/src/Interpreters/ActionsVisitor.h b/src/Interpreters/ActionsVisitor.h index d8d85f1c0bf..2fc98d6399d 100644 --- a/src/Interpreters/ActionsVisitor.h +++ b/src/Interpreters/ActionsVisitor.h @@ -16,6 +16,9 @@ struct ExpressionAction; class ExpressionActions; using ExpressionActionsPtr = std::shared_ptr; +class ActionsDAG; +using ActionsDAGPtr = std::shared_ptr; + /// The case of an explicit enumeration of values. SetPtr makeExplicitSet( const ASTFunction * node, const Block & sample_block, bool create_ordered_set, @@ -31,31 +34,28 @@ SetPtr makeExplicitSet( */ struct ScopeStack { - struct Level - { - ExpressionActionsPtr actions; - NameSet new_columns; - }; - - using Levels = std::vector; + using Levels = std::vector; Levels stack; const Context & context; - ScopeStack(const ExpressionActionsPtr & actions, const Context & context_); + ScopeStack(ActionsDAGPtr actions, const Context & context_); void pushLevel(const NamesAndTypesList & input_columns); size_t getColumnLevel(const std::string & name); + void addAlias(const std::string & name, std::string alias); + void addArrayJoin(const std::string & source_name, std::string result_name); + void addAction(const ExpressionAction & action); /// For arrayJoin() to avoid double columns in the input. void addActionNoInput(const ExpressionAction & action); - ExpressionActionsPtr popLevel(); + ActionsDAGPtr popLevel(); - const Block & getSampleBlock() const; + const ActionsDAG::Index & getIndex() const; }; class ASTIdentifier; @@ -91,7 +91,7 @@ public: int next_unique_suffix; Data(const Context & context_, SizeLimits set_size_limit_, size_t subquery_depth_, - const NamesAndTypesList & source_columns_, const ExpressionActionsPtr & actions, + const NamesAndTypesList & source_columns_, ActionsDAGPtr actions, PreparedSets & prepared_sets_, SubqueriesForSets & subqueries_for_sets_, bool no_subqueries_, bool no_makeset_, bool only_consts_, bool no_storage_or_local_) : context(context_), @@ -105,33 +105,43 @@ public: only_consts(only_consts_), no_storage_or_local(no_storage_or_local_), visit_depth(0), - actions_stack(actions, context), + actions_stack(std::move(actions), context), next_unique_suffix(actions_stack.getSampleBlock().columns() + 1) {} - void updateActions(ExpressionActionsPtr & actions) - { - actions = actions_stack.popLevel(); - } - - void addAction(const ExpressionAction & action) - { - actions_stack.addAction(action); - } - void addActionNoInput(const ExpressionAction & action) - { - actions_stack.addActionNoInput(action); - } - - const Block & getSampleBlock() const - { - return actions_stack.getSampleBlock(); - } - +// void updateActions(ExpressionActionsPtr & actions) +// { +// actions = actions_stack.popLevel(); +// } +// +// void addAction(const ExpressionAction & action) +// { +// actions_stack.addAction(action); +// } +// void addActionNoInput(const ExpressionAction & action) +// { +// actions_stack.addActionNoInput(action); +// } +// +// const Block & getSampleBlock() const +// { +// return actions_stack.getSampleBlock(); +// } +// /// Does result of the calculation already exists in the block. - bool hasColumn(const String & columnName) const + bool hasColumn(const String & column_name) const { - return actions_stack.getSampleBlock().has(columnName); + return actions_stack.getIndex().count(column_name) != 0; + } + + void addAlias(const std::string & name, std::string alias) + { + actions_stack.addAlias(name, std::move(alias)); + } + + void addArrayJoin(const std::string & source_name, std::string result_name) + { + actions_stack.addArrayJoin(source_name, std::move(result_name)); } /* diff --git a/src/Interpreters/ExpressionActions.cpp b/src/Interpreters/ExpressionActions.cpp index 33fa6215160..221bf9b1835 100644 --- a/src/Interpreters/ExpressionActions.cpp +++ b/src/Interpreters/ExpressionActions.cpp @@ -1406,4 +1406,143 @@ const ExpressionActionsPtr & ExpressionActionsChain::Step::actions() const return typeid_cast(this)->actions; } +ActionsDAG::Node & ActionsDAG::addNode(Node node) +{ + if (index.count(node.result_name) != 0) + throw Exception("Column '" + node.result_name + "' already exists", ErrorCodes::DUPLICATE_COLUMN); + + auto & res = nodes.emplace_back(std::move(node)); + index[res.result_name] = &res; + return res; +} + +ActionsDAG::Node & ActionsDAG::getNode(const std::string & name) +{ + auto it = index.find(name); + if (it == index.end()) + throw Exception("Unknown identifier: '" + name + "'", ErrorCodes::UNKNOWN_IDENTIFIER); + + return *it->second; +} + +const ActionsDAG::Node & ActionsDAG::addInput(std::string name, DataTypePtr type) +{ + Node node; + node.type = Type::INPUT; + node.result_type = std::move(type); + node.result_name = std::move(name); + + return addNode(std::move(node)); +} + +const ActionsDAG::Node & ActionsDAG::addAlias(const std::string & name, std::string alias) +{ + auto & child = getNode(name); + + Node node; + node.type = Type::ALIAS; + node.result_type = child.result_type; + node.result_name = std::move(alias); + node.column = child.column; + node.children.emplace_back(&child); + + return addNode(std::move(node)); +} + +const ActionsDAG::Node & ActionsDAG::addArrayJoin(const std::string & source_name, std::string result_name) +{ + auto & child = getNode(source_name); + + const DataTypeArray * array_type = typeid_cast(child.result_type.get()); + if (!array_type) + throw Exception("ARRAY JOIN requires array argument", ErrorCodes::TYPE_MISMATCH); + + Node node; + node.type = Type::ARRAY_JOIN; + node.result_type = array_type->getNestedType(); + node.result_name = std::move(result_name); + node.children.emplace_back(&child); + + return addNode(std::move(node)); +} + +const ActionsDAG::Node & ActionsDAG::addFunction(const FunctionOverloadResolverPtr & function, const Names & arguments) +{ + Node node; + node.type = Type::FUNCTION; + + bool all_const = true; + bool all_suitable_for_constant_folding = true; + + ColumnNumbers arguments(argument_names.size()); + for (size_t i = 0; i < argument_names.size(); ++i) + { + arguments[i] = sample_block.getPositionByName(argument_names[i]); + ColumnPtr col = sample_block.safeGetByPosition(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}); + function = function_base->prepare(sample_block, arguments, result_position); + 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())) + { + function->execute(sample_block, arguments, result_position, 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(sample_block, arguments)) + { + res.column = std::move(col); + names_not_for_constant_folding.insert(result_name); + } + } + + node.result_name = function->getName() + "("; + for (size_t i = 0 ; i < arguments.size(); ++i) + { + if (i) + node.result_name += ", "; + node.result_name += arguments[i]; + } + node.result_name += ")"; +} + } diff --git a/src/Interpreters/ExpressionActions.h b/src/Interpreters/ExpressionActions.h index 0607bc1e055..e742a84719d 100644 --- a/src/Interpreters/ExpressionActions.h +++ b/src/Interpreters/ExpressionActions.h @@ -140,6 +140,60 @@ private: class ExpressionActions; using ExpressionActionsPtr = std::shared_ptr; +class ActionsDAG +{ +public: + + enum class Type + { + /// Column which must be in input. + INPUT, + /// Constant column with known value. + COLUMN, + /// Another one name for column. + ALIAS, + FUNCTION, + /// Function arrayJoin. Specially separated because it changes the number of rows. + ARRAY_JOIN, + }; + + struct Node + { + std::vector children; + + Type type; + + std::string result_name; + DataTypePtr result_type; + + /// For COLUMN node and propagated constants. + ColumnPtr column; + }; + + using Index = std::unordered_map; + +private: + std::list nodes; + Index index; + +public: + ActionsDAG() = default; + ActionsDAG(const ActionsDAG &) = delete; + ActionsDAG & operator=(const ActionsDAG &) = delete; + + const std::list & getNodes() const; + const Index & getIndex() const { return index; } + + const Node & addInput(std::string name, DataTypePtr type); + const Node & addAlias(const std::string & name, std::string alias); + const Node & addArrayJoin(const std::string & source_name, std::string result_name); + const Node & addFunction(const FunctionOverloadResolverPtr & function, const Names & arguments); + +private: + Node & addNode(Node node); + Node & getNode(const std::string & name); +}; + /** Contains a sequence of actions on the block. */ class ExpressionActions From fcf00a304c53531fa5c2d089b9d8f7f859c429a6 Mon Sep 17 00:00:00 2001 From: hustnn Date: Thu, 10 Sep 2020 17:01:01 +0800 Subject: [PATCH 002/321] Fix potential memory leak caused by zookeeper exist calling --- src/Common/ZooKeeper/ZooKeeper.cpp | 7 ++++--- src/Common/ZooKeeper/ZooKeeperImpl.cpp | 21 +++++++++++++++++++-- src/Common/ZooKeeper/ZooKeeperImpl.h | 1 + src/Storages/MergeTree/LeaderElection.h | 3 ++- 4 files changed, 26 insertions(+), 6 deletions(-) diff --git a/src/Common/ZooKeeper/ZooKeeper.cpp b/src/Common/ZooKeeper/ZooKeeper.cpp index e50aa8f1700..635b1bb5243 100644 --- a/src/Common/ZooKeeper/ZooKeeper.cpp +++ b/src/Common/ZooKeeper/ZooKeeper.cpp @@ -651,7 +651,7 @@ bool ZooKeeper::waitForDisappear(const std::string & path, const WaitCondition & { WaitForDisappearStatePtr state = std::make_shared(); - auto callback = [state](const Coordination::ExistsResponse & response) + auto callback = [state](const Coordination::GetResponse & response) { state->code = int32_t(response.error); if (state->code) @@ -671,8 +671,9 @@ bool ZooKeeper::waitForDisappear(const std::string & path, const WaitCondition & while (!condition || !condition()) { - /// NOTE: if the node doesn't exist, the watch will leak. - impl->exists(path, callback, watch); + /// Use getData insteand of exists to avoid watch leak. + impl->get(path, callback, watch); + if (!condition) state->event.wait(); else if (!state->event.tryWait(1000)) diff --git a/src/Common/ZooKeeper/ZooKeeperImpl.cpp b/src/Common/ZooKeeper/ZooKeeperImpl.cpp index a3e82612ac8..7bc29869518 100644 --- a/src/Common/ZooKeeper/ZooKeeperImpl.cpp +++ b/src/Common/ZooKeeper/ZooKeeperImpl.cpp @@ -1092,8 +1092,6 @@ void ZooKeeper::sendThread() { info.request->has_watch = true; CurrentMetrics::add(CurrentMetrics::ZooKeeperWatch); - std::lock_guard lock(watches_mutex); - watches[info.request->getPath()].emplace_back(std::move(info.watch)); } if (expired) @@ -1278,6 +1276,22 @@ void ZooKeeper::receiveEvent() response->removeRootPath(root_path); } + /// Find from operations + if (request_info.watch) + { + bool add_watch = false; + if (request_info.isExists) + add_watch = (response->error == Error::ZOK || response->error == Error::ZNONODE); + else + add_watch = response->error == Error::ZOK; + + if (add_watch) + { + std::lock_guard lock(watches_mutex); + watches[request_info.request->getPath()].emplace_back(std::move(request_info.watch)); + } + } + int32_t actual_length = in->count() - count_before_event; if (length != actual_length) throw Exception("Response length doesn't match. Expected: " + toString(length) + ", actual: " + toString(actual_length), Error::ZMARSHALLINGERROR); @@ -1545,6 +1559,7 @@ void ZooKeeper::exists( request_info.request = std::make_shared(std::move(request)); request_info.callback = [callback](const Response & response) { callback(dynamic_cast(response)); }; request_info.watch = watch; + request_info.isExists = true; pushRequest(std::move(request_info)); ProfileEvents::increment(ProfileEvents::ZooKeeperExists); @@ -1563,6 +1578,7 @@ void ZooKeeper::get( request_info.request = std::make_shared(std::move(request)); request_info.callback = [callback](const Response & response) { callback(dynamic_cast(response)); }; request_info.watch = watch; + request_info.isExists = false; pushRequest(std::move(request_info)); ProfileEvents::increment(ProfileEvents::ZooKeeperGet); @@ -1601,6 +1617,7 @@ void ZooKeeper::list( request_info.request = std::make_shared(std::move(request)); request_info.callback = [callback](const Response & response) { callback(dynamic_cast(response)); }; request_info.watch = watch; + request_info.isExists = false; pushRequest(std::move(request_info)); ProfileEvents::increment(ProfileEvents::ZooKeeperList); diff --git a/src/Common/ZooKeeper/ZooKeeperImpl.h b/src/Common/ZooKeeper/ZooKeeperImpl.h index 840cbdbde3f..8dd0413d83d 100644 --- a/src/Common/ZooKeeper/ZooKeeperImpl.h +++ b/src/Common/ZooKeeper/ZooKeeperImpl.h @@ -197,6 +197,7 @@ private: ResponseCallback callback; WatchCallback watch; clock::time_point time; + bool isExists; }; using RequestsQueue = ConcurrentBoundedQueue; diff --git a/src/Storages/MergeTree/LeaderElection.h b/src/Storages/MergeTree/LeaderElection.h index 89715085eda..a5f7ebce84f 100644 --- a/src/Storages/MergeTree/LeaderElection.h +++ b/src/Storages/MergeTree/LeaderElection.h @@ -124,7 +124,8 @@ private: /// Watch for the node in front of us. --my_node_it; - if (!zookeeper.existsWatch(path + "/" + *my_node_it, nullptr, task->getWatchCallback())) + std::string get_path_value; + if (!zookeeper.tryGetWatch(path + "/" + *my_node_it, get_path_value, nullptr, task->getWatchCallback())) task->schedule(); success = true; From c1469aff938d10b512290437a8c9b1ab1c428019 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 10 Sep 2020 19:01:41 +0300 Subject: [PATCH 003/321] Add ExpressionDAG [Part 2]. --- src/Interpreters/ActionsVisitor.cpp | 97 ++++++----- src/Interpreters/ActionsVisitor.h | 55 ++++--- src/Interpreters/ExpressionActions.cpp | 208 ++++++++++++++++++++---- src/Interpreters/ExpressionActions.h | 24 ++- src/Interpreters/ExpressionAnalyzer.cpp | 25 ++- src/Interpreters/ExpressionAnalyzer.h | 9 +- 6 files changed, 292 insertions(+), 126 deletions(-) diff --git a/src/Interpreters/ActionsVisitor.cpp b/src/Interpreters/ActionsVisitor.cpp index 0742479d34f..75b29f61ae7 100644 --- a/src/Interpreters/ActionsVisitor.cpp +++ b/src/Interpreters/ActionsVisitor.cpp @@ -339,7 +339,7 @@ static Block createBlockForSet( } SetPtr makeExplicitSet( - const ASTFunction * node, const Block & sample_block, bool create_ordered_set, + const ASTFunction * node, const ActionsDAG::Index & index, bool create_ordered_set, const Context & context, const SizeLimits & size_limits, PreparedSets & prepared_sets) { const IAST & args = *node->arguments; @@ -350,7 +350,10 @@ SetPtr makeExplicitSet( const ASTPtr & left_arg = args.children.at(0); const ASTPtr & right_arg = args.children.at(1); - const DataTypePtr & left_arg_type = sample_block.getByName(left_arg->getColumnName()).type; + 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; DataTypes set_element_types = {left_arg_type}; const auto * left_tuple_type = typeid_cast(left_arg_type.get()); @@ -413,6 +416,15 @@ size_t ScopeStack::getColumnLevel(const std::string & name) throw Exception("Unknown identifier: " + name, ErrorCodes::UNKNOWN_IDENTIFIER); } +void ScopeStack::addColumn(ColumnWithTypeAndName column) +{ + auto level = getColumnLevel(column.name); + const auto & node = stack[level]->addColumn(std::move(column)); + + for (size_t j = level + 1; j < stack.size(); ++j) + stack[j]->addInput(node.result_name, node.result_type); +} + void ScopeStack::addAlias(const std::string & name, std::string alias) { auto level = getColumnLevel(name); @@ -431,37 +443,20 @@ void ScopeStack::addArrayJoin(const std::string & source_name, std::string resul stack[j]->addInput(node.result_name, node.result_type); } -void ScopeStack::addAction(const ExpressionAction & action) +void ScopeStack::addFunction( + const FunctionOverloadResolverPtr & function, + const Names & argument_names, + std::string result_name, + bool compile_expressions) { size_t level = 0; - Names required = action.getNeededColumns(); - for (const auto & elem : required) - level = std::max(level, getColumnLevel(elem)); + for (const auto & argument : argument_names) + level = std::max(level, getColumnLevel(argument)); - Names added; - stack[level].actions->add(action, added); + const auto & node = stack[level]->addFunction(function, argument_names, std::move(result_name), compile_expressions); - stack[level].new_columns.insert(added.begin(), added.end()); - - for (const auto & elem : added) - { - const ColumnWithTypeAndName & col = stack[level].actions->getSampleBlock().getByName(elem); - for (size_t j = level + 1; j < stack.size(); ++j) - stack[j].actions->addInput(col); - } -} - -void ScopeStack::addActionNoInput(const ExpressionAction & action) -{ - size_t level = 0; - Names required = action.getNeededColumns(); - for (const auto & elem : required) - level = std::max(level, getColumnLevel(elem)); - - Names added; - stack[level].actions->add(action, added); - - stack[level].new_columns.insert(added.begin(), added.end()); + for (size_t j = level + 1; j < stack.size(); ++j) + stack[j]->addInput(node.result_name, node.result_type); } ActionsDAGPtr ScopeStack::popLevel() @@ -471,6 +466,11 @@ ActionsDAGPtr ScopeStack::popLevel() return res; } +std::string ScopeStack::dumpNames() const +{ + return stack.back()->dumpNames(); +} + const ActionsDAG::Index & ScopeStack::getIndex() const { return stack.back()->getIndex(); @@ -592,10 +592,10 @@ void ActionsMatcher::visit(const ASTFunction & node, const ASTPtr & ast, Data & auto argument_name = node.arguments->children.at(0)->getColumnName(); - data.addAction(ExpressionAction::applyFunction( + data.addFunction( FunctionFactory::instance().get(node.name + "IgnoreSet", data.context), { argument_name, argument_name }, - column_name.get(ast))); + column_name.get(ast)); } return; } @@ -667,7 +667,7 @@ void ActionsMatcher::visit(const ASTFunction & node, const ASTPtr & ast, Data & column.column = ColumnConst::create(std::move(column_set), 1); else column.column = std::move(column_set); - data.addAction(ExpressionAction::addColumn(column)); + data.addColumn(column); } argument_types.push_back(column.type); @@ -683,7 +683,7 @@ void ActionsMatcher::visit(const ASTFunction & node, const ASTPtr & ast, Data & ColumnConst::create(std::move(column_string), 1), std::make_shared(), data.getUniqueName("__" + node.name)); - data.addAction(ExpressionAction::addColumn(column)); + data.addColumn(column); argument_types.push_back(column.type); argument_names.push_back(column.name); } @@ -703,9 +703,11 @@ void ActionsMatcher::visit(const ASTFunction & node, const ASTPtr & ast, Data & child_column_name = as_literal->unique_column_name; } - if (data.hasColumn(child_column_name)) + const auto & index = data.actions_stack.getIndex(); + auto it = index.find(child_column_name); + if (it != index.end()) { - argument_types.push_back(data.getSampleBlock().getByName(child_column_name).type); + argument_types.push_back(it->second->result_type); argument_names.push_back(child_column_name); } else @@ -713,7 +715,7 @@ void ActionsMatcher::visit(const ASTFunction & node, const ASTPtr & ast, Data & if (data.only_consts) arguments_present = false; else - throw Exception("Unknown identifier: " + child_column_name + " there are columns: " + data.getSampleBlock().dumpNames(), + throw Exception("Unknown identifier: " + child_column_name + " there are columns: " + data.actions_stack.dumpNames(), ErrorCodes::UNKNOWN_IDENTIFIER); } } @@ -750,7 +752,8 @@ void ActionsMatcher::visit(const ASTFunction & node, const ASTPtr & ast, Data & data.actions_stack.pushLevel(lambda_arguments); visit(lambda->arguments->children.at(1), data); - ExpressionActionsPtr lambda_actions = data.actions_stack.popLevel(); + 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)); @@ -769,7 +772,7 @@ void ActionsMatcher::visit(const ASTFunction & node, const ASTPtr & ast, Data & auto function_capture = std::make_unique( lambda_actions, captured, lambda_arguments, result_type, result_name); auto function_capture_adapter = std::make_shared(std::move(function_capture)); - data.addAction(ExpressionAction::applyFunction(function_capture_adapter, captured, lambda_name)); + data.addFunction(function_capture_adapter, captured, lambda_name); argument_types[i] = std::make_shared(lambda_type->getArgumentTypes(), result_type); argument_names[i] = lambda_name; @@ -791,7 +794,7 @@ void ActionsMatcher::visit(const ASTFunction & node, const ASTPtr & ast, Data & if (arguments_present) { - data.addAction(ExpressionAction::applyFunction(function_builder, argument_names, column_name.get(ast))); + data.addFunction(function_builder, argument_names, column_name.get(ast)); } } @@ -806,8 +809,12 @@ void ActionsMatcher::visit(const ASTLiteral & literal, const ASTPtr & /* ast */, if (literal.unique_column_name.empty()) { const auto default_name = literal.getColumnName(); - const auto & block = data.getSampleBlock(); - const auto * existing_column = block.findByName(default_name); + const auto & index = data.actions_stack.getIndex(); + const ActionsDAG::Node * existing_column = nullptr; + + auto it = index.find(default_name); + if (it != index.end()) + existing_column = it->second; /* * To approximate CSE, bind all identical literals to a single temporary @@ -843,7 +850,7 @@ void ActionsMatcher::visit(const ASTLiteral & literal, const ASTPtr & /* ast */, column.column = type->createColumnConst(1, value); column.type = type; - data.addAction(ExpressionAction::addColumn(column)); + data.addColumn(std::move(column)); } SetPtr ActionsMatcher::makeSet(const ASTFunction & node, Data & data, bool no_subqueries) @@ -855,7 +862,6 @@ SetPtr ActionsMatcher::makeSet(const ASTFunction & node, Data & data, bool no_su const IAST & args = *node.arguments; const ASTPtr & left_in_operand = args.children.at(0); const ASTPtr & right_in_operand = args.children.at(1); - const Block & sample_block = data.getSampleBlock(); /// If the subquery or table name for SELECT. const auto * identifier = right_in_operand->as(); @@ -916,9 +922,10 @@ SetPtr ActionsMatcher::makeSet(const ASTFunction & node, Data & data, bool no_su } else { - if (sample_block.has(left_in_operand->getColumnName())) + const auto & index = data.actions_stack.getIndex(); + if (index.count(left_in_operand->getColumnName()) != 0) /// An explicit enumeration of values in parentheses. - return makeExplicitSet(&node, sample_block, false, data.context, data.set_size_limit, data.prepared_sets); + return makeExplicitSet(&node, index, false, data.context, data.set_size_limit, data.prepared_sets); else return {}; } diff --git a/src/Interpreters/ActionsVisitor.h b/src/Interpreters/ActionsVisitor.h index 2fc98d6399d..109d547ed55 100644 --- a/src/Interpreters/ActionsVisitor.h +++ b/src/Interpreters/ActionsVisitor.h @@ -21,7 +21,7 @@ using ActionsDAGPtr = std::shared_ptr; /// The case of an explicit enumeration of values. SetPtr makeExplicitSet( - const ASTFunction * node, const Block & sample_block, bool create_ordered_set, + const ASTFunction * node, const ActionsDAG::Index & index, bool create_ordered_set, const Context & context, const SizeLimits & limits, PreparedSets & prepared_sets); @@ -46,16 +46,19 @@ struct ScopeStack size_t getColumnLevel(const std::string & name); + void addColumn(ColumnWithTypeAndName column); void addAlias(const std::string & name, std::string alias); void addArrayJoin(const std::string & source_name, std::string result_name); - - void addAction(const ExpressionAction & action); - /// For arrayJoin() to avoid double columns in the input. - void addActionNoInput(const ExpressionAction & action); + void addFunction( + const FunctionOverloadResolverPtr & function, + const Names & argument_names, + std::string result_name, + bool compile_expressions); ActionsDAGPtr popLevel(); const ActionsDAG::Index & getIndex() const; + std::string dumpNames() const; }; class ASTIdentifier; @@ -106,34 +109,20 @@ public: no_storage_or_local(no_storage_or_local_), visit_depth(0), actions_stack(std::move(actions), context), - next_unique_suffix(actions_stack.getSampleBlock().columns() + 1) + next_unique_suffix(actions_stack.getIndex().size() + 1) {} -// void updateActions(ExpressionActionsPtr & actions) -// { -// actions = actions_stack.popLevel(); -// } -// -// void addAction(const ExpressionAction & action) -// { -// actions_stack.addAction(action); -// } -// void addActionNoInput(const ExpressionAction & action) -// { -// actions_stack.addActionNoInput(action); -// } -// -// const Block & getSampleBlock() const -// { -// return actions_stack.getSampleBlock(); -// } -// /// Does result of the calculation already exists in the block. bool hasColumn(const String & column_name) const { return actions_stack.getIndex().count(column_name) != 0; } + void addColumn(ColumnWithTypeAndName column) + { + actions_stack.addColumn(std::move(column)); + } + void addAlias(const std::string & name, std::string alias) { actions_stack.addAlias(name, std::move(alias)); @@ -144,18 +133,30 @@ public: 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); + } + + ActionsDAGPtr getActions() + { + return actions_stack.popLevel(); + } + /* * Generate a column name that is not present in the sample block, using * the given prefix and an optional numeric suffix. */ String getUniqueName(const String & prefix) { - const auto & block = getSampleBlock(); auto result = prefix; // First, try the name without any suffix, because it is currently // used both as a display name and a column id. - while (block.has(result)) + while (hasColumn(result)) { result = prefix + "_" + toString(next_unique_suffix); ++next_unique_suffix; diff --git a/src/Interpreters/ExpressionActions.cpp b/src/Interpreters/ExpressionActions.cpp index 221bf9b1835..44b46e579f0 100644 --- a/src/Interpreters/ExpressionActions.cpp +++ b/src/Interpreters/ExpressionActions.cpp @@ -13,8 +13,10 @@ #include #include #include +#include #include #include +#include #if !defined(ARCADIA_BUILD) # include "config_core.h" @@ -1406,6 +1408,12 @@ const ExpressionActionsPtr & ExpressionActionsChain::Step::actions() const return typeid_cast(this)->actions; } +ActionsDAG::ActionsDAG(const NamesAndTypesList & inputs) +{ + for (const auto & input : inputs) + addInput(input.name, input.type); +} + ActionsDAG::Node & ActionsDAG::addNode(Node node) { if (index.count(node.result_name) != 0) @@ -1435,6 +1443,17 @@ const ActionsDAG::Node & ActionsDAG::addInput(std::string name, DataTypePtr type return addNode(std::move(node)); } +const ActionsDAG::Node & ActionsDAG::addColumn(ColumnWithTypeAndName column) +{ + Node node; + node.type = Type::COLUMN; + node.result_type = std::move(column.type); + node.result_name = std::move(column.name); + node.column = std::move(column.column); + + return addNode(std::move(node)); +} + const ActionsDAG::Node & ActionsDAG::addAlias(const std::string & name, std::string alias) { auto & child = getNode(name); @@ -1444,6 +1463,7 @@ const ActionsDAG::Node & ActionsDAG::addAlias(const std::string & name, std::str node.result_type = child.result_type; node.result_name = std::move(alias); node.column = child.column; + node.allow_constant_folding = child.allow_constant_folding; node.children.emplace_back(&child); return addNode(std::move(node)); @@ -1466,49 +1486,61 @@ const ActionsDAG::Node & ActionsDAG::addArrayJoin(const std::string & source_nam return addNode(std::move(node)); } -const ActionsDAG::Node & ActionsDAG::addFunction(const FunctionOverloadResolverPtr & function, const Names & arguments) +const ActionsDAG::Node & ActionsDAG::addFunction( + const FunctionOverloadResolverPtr & function, + const Names & argument_names, + std::string result_name, + bool compile_expressions) { + size_t num_arguments = argument_names.size(); + Node node; node.type = Type::FUNCTION; + node.function_builder = function; + node.children.reserve(num_arguments); bool all_const = true; - bool all_suitable_for_constant_folding = true; + ColumnsWithTypeAndName arguments(num_arguments); + ColumnNumbers argument_numbers(num_arguments); - ColumnNumbers arguments(argument_names.size()); - for (size_t i = 0; i < argument_names.size(); ++i) + for (size_t i = 0; i < num_arguments; ++i) { - arguments[i] = sample_block.getPositionByName(argument_names[i]); - ColumnPtr col = sample_block.safeGetByPosition(arguments[i]).column; - if (!col || !isColumnConst(*col)) + auto & child = getNode(argument_names[i]); + node.children.emplace_back(&child); + node.allow_constant_folding = node.allow_constant_folding && child.allow_constant_folding; + + ColumnWithTypeAndName argument; + argument.column = child.column; + argument.type = child.result_type; + + if (!argument.column || !isColumnConst(*argument.column)) all_const = false; - if (names_not_for_constant_folding.count(argument_names[i])) - all_suitable_for_constant_folding = false; + arguments[i] = std::move(argument); + argument_numbers[i] = i; } - size_t result_position = sample_block.columns(); - sample_block.insert({nullptr, result_type, result_name}); - function = function_base->prepare(sample_block, arguments, result_position); - function->createLowCardinalityResultCache(settings.max_threads); + node.function_base = function->build(arguments); + node.result_type = node.function_base->getReturnType(); - bool compile_expressions = false; + Block sample_block(std::move(arguments)); + sample_block.insert({nullptr, node.result_type, node.result_name}); + node.function = node.function_base->prepare(sample_block, argument_numbers, num_arguments); + + bool do_compile_expressions = false; #if USE_EMBEDDED_COMPILER - compile_expressions = settings.compile_expressions; + 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 - if (all_const && function_base->isSuitableForConstantFolding() && (!compile_expressions || function_base->isDeterministic())) + if (all_const && node.function_base->isSuitableForConstantFolding() && (!do_compile_expressions || node.function_base->isDeterministic())) { - function->execute(sample_block, arguments, result_position, sample_block.rows(), true); + node.function->execute(sample_block, argument_numbers, num_arguments, 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 + ColumnWithTypeAndName & col = sample_block.safeGetByPosition(num_arguments); + if (isColumnConst(*col.column)) { /// 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. @@ -1517,32 +1549,136 @@ const ActionsDAG::Node & ActionsDAG::addFunction(const FunctionOverloadResolverP if (col.column->empty()) col.column = col.column->cloneResized(1); - if (!all_suitable_for_constant_folding) - names_not_for_constant_folding.insert(result_name); + node.column = std::move(col.column); } } /// 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 (!node.column && node.function_base->isSuitableForConstantFolding()) { - if (auto col = function_base->getResultIfAlwaysReturnsConstantAndHasArguments(sample_block, arguments)) + if (auto col = node.function_base->getResultIfAlwaysReturnsConstantAndHasArguments(sample_block, argument_numbers)) { - res.column = std::move(col); - names_not_for_constant_folding.insert(result_name); + node.column = std::move(col); + node.allow_constant_folding = false; } } - node.result_name = function->getName() + "("; - for (size_t i = 0 ; i < arguments.size(); ++i) + if (result_name.empty()) { - if (i) - node.result_name += ", "; - node.result_name += arguments[i]; + 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 += ")"; } - node.result_name += ")"; + + node.result_name = std::move(result_name); + + return addNode(std::move(node)); +} + +NamesAndTypesList ActionsDAG::getNamesAndTypesList() const +{ + NamesAndTypesList result; + for (const auto & node : nodes) + result.emplace_back(node.result_name, node.result_type); + + return result; +} + +std::string ActionsDAG::dumpNames() const +{ + WriteBufferFromOwnString out; + for (auto it = nodes.begin(); it != nodes.end(); ++it) + { + if (it != nodes.begin()) + out << ", "; + out << it->result_name; + } + return out.str(); +} + +ExpressionActionsPtr ActionsDAG::buildExpressions(const Context & context) +{ + struct Data + { + Node * node = nullptr; + size_t num_created_children = 0; + std::vector parents; + }; + + std::vector data(nodes.size()); + std::unordered_map reverse_index; + + for (auto & node : nodes) + { + size_t id = reverse_index.size(); + data[id].node = &node; + reverse_index[&node] = id; + } + + std::stack ready_nodes; + std::stack ready_array_joins; + + for (auto & node : nodes) + { + for (const auto & child : node.children) + data[reverse_index[child]].parents.emplace_back(&node); + + if (node.children.empty()) + ready_nodes.emplace(&node); + } + + auto expressions = std::make_shared(NamesAndTypesList(), context); + + while (!ready_nodes.empty() || !ready_array_joins.empty()) + { + auto & stack = ready_nodes.empty() ? ready_array_joins : ready_nodes; + Node * node = stack.top(); + stack.pop(); + + Names argument_names; + for (const auto & child : node->children) + argument_names.emplace_back(child->result_name); + + switch (node->type) + { + 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)); + break; + case Type::ARRAY_JOIN: + expressions->add(ExpressionAction::arrayJoin(argument_names.at(0), node->result_name)); + break; + case Type::FUNCTION: + expressions->add(ExpressionAction::applyFunction(node->function_builder, argument_names, node->result_name)); + break; + } + + for (const auto & parent : data[reverse_index[node]].parents) + { + auto & cur = data[reverse_index[parent]]; + ++cur.num_created_children; + + if (parent->children.size() == cur.num_created_children) + { + auto & push_stack = parent->type == Type::ARRAY_JOIN ? ready_array_joins : ready_nodes; + push_stack.push(parent); + } + } + } + + return expressions; } } diff --git a/src/Interpreters/ExpressionActions.h b/src/Interpreters/ExpressionActions.h index e742a84719d..4af80fc504d 100644 --- a/src/Interpreters/ExpressionActions.h +++ b/src/Interpreters/ExpressionActions.h @@ -152,9 +152,9 @@ public: COLUMN, /// Another one name for column. ALIAS, - FUNCTION, /// Function arrayJoin. Specially separated because it changes the number of rows. ARRAY_JOIN, + FUNCTION, }; struct Node @@ -166,8 +166,17 @@ public: std::string result_name; DataTypePtr result_type; + 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; + /// For COLUMN node and propagated constants. ColumnPtr column; + /// Some functions like `ignore()` always return constant but can't be replaced by constant it. + /// We calculate such constants in order to avoid unnecessary materialization, but prohibit it's folding. + bool allow_constant_folding = true; }; using Index = std::unordered_map; @@ -180,14 +189,25 @@ public: ActionsDAG() = default; ActionsDAG(const ActionsDAG &) = delete; ActionsDAG & operator=(const ActionsDAG &) = delete; + ActionsDAG(const NamesAndTypesList & inputs); const std::list & getNodes() const; const Index & getIndex() const { return index; } + NamesAndTypesList getNamesAndTypesList() const; + std::string dumpNames() const; + const Node & addInput(std::string name, DataTypePtr type); + const Node & addColumn(ColumnWithTypeAndName column); const Node & addAlias(const std::string & name, std::string alias); const Node & addArrayJoin(const std::string & source_name, std::string result_name); - const Node & addFunction(const FunctionOverloadResolverPtr & function, const Names & arguments); + const Node & addFunction( + const FunctionOverloadResolverPtr & function, + const Names & argument_names, + std::string result_name, + bool compile_expressions); + + ExpressionActionsPtr buildExpressions(const Context & context); private: Node & addNode(Node node); diff --git a/src/Interpreters/ExpressionAnalyzer.cpp b/src/Interpreters/ExpressionAnalyzer.cpp index d9fc44d9b8f..0b7be0b1ea4 100644 --- a/src/Interpreters/ExpressionAnalyzer.cpp +++ b/src/Interpreters/ExpressionAnalyzer.cpp @@ -153,7 +153,7 @@ void ExpressionAnalyzer::analyzeAggregation() auto * select_query = query->as(); - ExpressionActionsPtr temp_actions = std::make_shared(sourceColumns(), context); + auto temp_actions = std::make_shared(sourceColumns()); if (select_query) { @@ -362,12 +362,11 @@ void SelectQueryExpressionAnalyzer::makeSetsForIndex(const ASTPtr & node) } else { - ExpressionActionsPtr temp_actions = std::make_shared(columns_after_join, context); + auto temp_actions = std::make_shared(columns_after_join); getRootActions(left_in_operand, true, temp_actions); - Block sample_block_with_calculated_columns = temp_actions->getSampleBlock(); - if (sample_block_with_calculated_columns.has(left_in_operand->getColumnName())) - makeExplicitSet(func, sample_block_with_calculated_columns, true, context, + if (temp_actions->getIndex().count(left_in_operand->getColumnName()) != 0) + makeExplicitSet(func, temp_actions->getIndex(), true, context, settings.size_limits_for_set, prepared_sets); } } @@ -375,25 +374,25 @@ void SelectQueryExpressionAnalyzer::makeSetsForIndex(const ASTPtr & node) } -void ExpressionAnalyzer::getRootActions(const ASTPtr & ast, bool no_subqueries, ExpressionActionsPtr & actions, bool only_consts) +void ExpressionAnalyzer::getRootActions(const ASTPtr & ast, bool no_subqueries, ActionsDAGPtr & actions, bool only_consts) { LogAST log; ActionsVisitor::Data visitor_data(context, settings.size_limits_for_set, subquery_depth, - sourceColumns(), actions, prepared_sets, subqueries_for_sets, + sourceColumns(), std::move(actions), prepared_sets, subqueries_for_sets, no_subqueries, false, only_consts, !isRemoteStorage()); ActionsVisitor(visitor_data, log.stream()).visit(ast); - visitor_data.updateActions(actions); + actions = visitor_data.getActions(); } -void ExpressionAnalyzer::getRootActionsNoMakeSet(const ASTPtr & ast, bool no_subqueries, ExpressionActionsPtr & actions, bool only_consts) +void ExpressionAnalyzer::getRootActionsNoMakeSet(const ASTPtr & ast, bool no_subqueries, ActionsDAGPtr & actions, bool only_consts) { LogAST log; ActionsVisitor::Data visitor_data(context, settings.size_limits_for_set, subquery_depth, - sourceColumns(), actions, prepared_sets, subqueries_for_sets, + sourceColumns(), std::move(actions), prepared_sets, subqueries_for_sets, no_subqueries, true, only_consts, !isRemoteStorage()); ActionsVisitor(visitor_data, log.stream()).visit(ast); - visitor_data.updateActions(actions); + visitor_data.getActions(); } @@ -443,14 +442,14 @@ const ASTSelectQuery * SelectQueryExpressionAnalyzer::getAggregatingQuery() cons } /// "Big" ARRAY JOIN. -ArrayJoinActionPtr ExpressionAnalyzer::addMultipleArrayJoinAction(ExpressionActionsPtr & actions, bool array_join_is_left) const +ArrayJoinActionPtr ExpressionAnalyzer::addMultipleArrayJoinAction(ActionsDAGPtr & actions, bool array_join_is_left) const { NameSet result_columns; for (const auto & result_source : syntax->array_join_result_to_source) { /// Assign new names to columns, if needed. if (result_source.first != result_source.second) - actions->add(ExpressionAction::copyColumn(result_source.second, result_source.first)); + actions->addAlias(result_source.second, result_source.first); /// Make ARRAY JOIN (replace arrays with their insides) for the columns in these new names. result_columns.insert(result_source.first); diff --git a/src/Interpreters/ExpressionAnalyzer.h b/src/Interpreters/ExpressionAnalyzer.h index 1cc9d75b19f..458ba9b6770 100644 --- a/src/Interpreters/ExpressionAnalyzer.h +++ b/src/Interpreters/ExpressionAnalyzer.h @@ -37,6 +37,9 @@ using StorageMetadataPtr = std::shared_ptr; class ArrayJoinAction; using ArrayJoinActionPtr = std::shared_ptr; +class ActionsDAG; +using ActionsDAGPtr = std::shared_ptr; + /// Create columns in block or return false if not possible bool sanitizeBlock(Block & block, bool throw_if_cannot_create_column = false); @@ -137,15 +140,15 @@ protected: /// Find global subqueries in the GLOBAL IN/JOIN sections. Fills in external_tables. void initGlobalSubqueriesAndExternalTables(bool do_global); - ArrayJoinActionPtr addMultipleArrayJoinAction(ExpressionActionsPtr & actions, bool is_left) const; + ArrayJoinActionPtr addMultipleArrayJoinAction(ActionsDAGPtr & actions, bool is_left) const; - void getRootActions(const ASTPtr & ast, bool no_subqueries, ExpressionActionsPtr & actions, bool only_consts = false); + void getRootActions(const ASTPtr & ast, bool no_subqueries, ActionsDAGPtr & actions, bool only_consts = false); /** Similar to getRootActions but do not make sets when analyzing IN functions. It's used in * analyzeAggregation which happens earlier than analyzing PREWHERE and WHERE. If we did, the * prepared sets would not be applicable for MergeTree index optimization. */ - void getRootActionsNoMakeSet(const ASTPtr & ast, bool no_subqueries, ExpressionActionsPtr & actions, bool only_consts = false); + void getRootActionsNoMakeSet(const ASTPtr & ast, bool no_subqueries, ActionsDAGPtr & actions, bool only_consts = false); /** Add aggregation keys to aggregation_keys, aggregate functions to aggregate_descriptions, * Create a set of columns aggregated_columns resulting after the aggregation, if any, From 1cae264eae3a18a1c8ed06b0a3fd4ed200e59caf Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 10 Sep 2020 21:36:51 +0300 Subject: [PATCH 004/321] Add ExpressionDAG [Part 3]. --- src/Interpreters/ArrayJoinAction.cpp | 8 +++--- src/Interpreters/ArrayJoinAction.h | 2 +- src/Interpreters/ExpressionActions.cpp | 16 ++++++++++++ src/Interpreters/ExpressionActions.h | 2 ++ src/Interpreters/ExpressionAnalyzer.cpp | 33 +++++++++++++------------ src/Interpreters/ExpressionAnalyzer.h | 2 +- 6 files changed, 42 insertions(+), 21 deletions(-) diff --git a/src/Interpreters/ArrayJoinAction.cpp b/src/Interpreters/ArrayJoinAction.cpp index 176dc8258ce..62064bf3292 100644 --- a/src/Interpreters/ArrayJoinAction.cpp +++ b/src/Interpreters/ArrayJoinAction.cpp @@ -35,11 +35,13 @@ ArrayJoinAction::ArrayJoinAction(const NameSet & array_joined_columns_, bool arr } -void ArrayJoinAction::prepare(Block & sample_block) +void ArrayJoinAction::prepare(ColumnsWithTypeAndName & sample) const { - for (const auto & name : columns) + for (auto & current : sample) { - ColumnWithTypeAndName & current = sample_block.getByName(name); + if (columns.count(current.name) == 0) + continue; + const DataTypeArray * array_type = typeid_cast(&*current.type); if (!array_type) throw Exception("ARRAY JOIN requires array argument", ErrorCodes::TYPE_MISMATCH); diff --git a/src/Interpreters/ArrayJoinAction.h b/src/Interpreters/ArrayJoinAction.h index 9467e579e62..81f720f00ef 100644 --- a/src/Interpreters/ArrayJoinAction.h +++ b/src/Interpreters/ArrayJoinAction.h @@ -28,7 +28,7 @@ public: FunctionOverloadResolverPtr function_builder; ArrayJoinAction(const NameSet & array_joined_columns_, bool array_join_is_left, const Context & context); - void prepare(Block & sample_block); + void prepare(ColumnsWithTypeAndName & sample) const; void execute(Block & block); }; diff --git a/src/Interpreters/ExpressionActions.cpp b/src/Interpreters/ExpressionActions.cpp index 44b46e579f0..3772ec4fcf7 100644 --- a/src/Interpreters/ExpressionActions.cpp +++ b/src/Interpreters/ExpressionActions.cpp @@ -1414,6 +1414,12 @@ ActionsDAG::ActionsDAG(const NamesAndTypesList & inputs) addInput(input.name, input.type); } +ActionsDAG::ActionsDAG(const ColumnsWithTypeAndName & inputs) +{ + for (const auto & input : inputs) + addInput(input.name, input.type); +} + ActionsDAG::Node & ActionsDAG::addNode(Node node) { if (index.count(node.result_name) != 0) @@ -1582,6 +1588,16 @@ const ActionsDAG::Node & ActionsDAG::addFunction( return addNode(std::move(node)); } +ColumnsWithTypeAndName ActionsDAG::getResultColumns() const +{ + ColumnsWithTypeAndName result; + result.reserve(nodes.size()); + for (const auto & node : nodes) + result.emplace_back(node.column, node.result_type, node.result_name); + + return result; +} + NamesAndTypesList ActionsDAG::getNamesAndTypesList() const { NamesAndTypesList result; diff --git a/src/Interpreters/ExpressionActions.h b/src/Interpreters/ExpressionActions.h index 4af80fc504d..79107d3baa9 100644 --- a/src/Interpreters/ExpressionActions.h +++ b/src/Interpreters/ExpressionActions.h @@ -190,10 +190,12 @@ public: ActionsDAG(const ActionsDAG &) = delete; ActionsDAG & operator=(const ActionsDAG &) = delete; ActionsDAG(const NamesAndTypesList & inputs); + ActionsDAG(const ColumnsWithTypeAndName & inputs); const std::list & getNodes() const; const Index & getIndex() const { return index; } + ColumnsWithTypeAndName getResultColumns() const; NamesAndTypesList getNamesAndTypesList() const; std::string dumpNames() const; diff --git a/src/Interpreters/ExpressionAnalyzer.cpp b/src/Interpreters/ExpressionAnalyzer.cpp index 0b7be0b1ea4..053d353bdfb 100644 --- a/src/Interpreters/ExpressionAnalyzer.cpp +++ b/src/Interpreters/ExpressionAnalyzer.cpp @@ -165,14 +165,14 @@ void ExpressionAnalyzer::analyzeAggregation() getRootActionsNoMakeSet(array_join_expression_list, true, temp_actions, false); if (auto array_join = addMultipleArrayJoinAction(temp_actions, is_array_join_left)) { - auto sample_block = temp_actions->getSampleBlock(); - array_join->prepare(sample_block); - temp_actions = std::make_shared(sample_block.getColumnsWithTypeAndName(), context); + auto sample_columns = temp_actions->getResultColumns(); + array_join->prepare(sample_columns); + temp_actions = std::make_shared(sample_columns); } - for (auto & column : temp_actions->getSampleBlock().getNamesAndTypesList()) + for (auto & column : temp_actions->getResultColumns()) if (syntax->array_join_result_to_source.count(column.name)) - array_join_columns.emplace_back(column); + array_join_columns.emplace_back(column.name, column.type); } columns_after_array_join = sourceColumns(); @@ -182,9 +182,9 @@ void ExpressionAnalyzer::analyzeAggregation() if (join) { getRootActionsNoMakeSet(analyzedJoin().leftKeysList(), true, temp_actions, false); - auto sample_columns = temp_actions->getSampleBlock().getColumnsWithTypeAndName(); + auto sample_columns = temp_actions->getResultColumns(); analyzedJoin().addJoinedColumnsAndCorrectNullability(sample_columns); - temp_actions = std::make_shared(sample_columns, context); + temp_actions = std::make_shared(sample_columns); } columns_after_join = columns_after_array_join; @@ -212,15 +212,16 @@ void ExpressionAnalyzer::analyzeAggregation() getRootActionsNoMakeSet(group_asts[i], true, temp_actions, false); const auto & column_name = group_asts[i]->getColumnName(); - const auto & block = temp_actions->getSampleBlock(); + const auto & index = temp_actions->getIndex(); - if (!block.has(column_name)) + auto it = index.find(column_name); + if (it == index.end()) throw Exception("Unknown identifier (in GROUP BY): " + column_name, ErrorCodes::UNKNOWN_IDENTIFIER); - const auto & col = block.getByName(column_name); + const auto & node = it->second; /// Constant expressions have non-null column pointer at this stage. - if (col.column && isColumnConst(*col.column)) + if (node->column && isColumnConst(*node->column)) { /// But don't remove last key column if no aggregate functions, otherwise aggregation will not work. if (!aggregate_descriptions.empty() || size > 1) @@ -235,7 +236,7 @@ void ExpressionAnalyzer::analyzeAggregation() } } - NameAndTypePair key{column_name, col.type}; + NameAndTypePair key{column_name, node->result_type}; /// Aggregation keys are uniqued. if (!unique_keys.count(key.name)) @@ -256,14 +257,14 @@ void ExpressionAnalyzer::analyzeAggregation() } } else - aggregated_columns = temp_actions->getSampleBlock().getNamesAndTypesList(); + aggregated_columns = temp_actions->getNamesAndTypesList(); for (const auto & desc : aggregate_descriptions) aggregated_columns.emplace_back(desc.column_name, desc.function->getReturnType()); } else { - aggregated_columns = temp_actions->getSampleBlock().getNamesAndTypesList(); + aggregated_columns = temp_actions->getNamesAndTypesList(); } } @@ -396,7 +397,7 @@ void ExpressionAnalyzer::getRootActionsNoMakeSet(const ASTPtr & ast, bool no_sub } -bool ExpressionAnalyzer::makeAggregateDescriptions(ExpressionActionsPtr & actions) +bool ExpressionAnalyzer::makeAggregateDescriptions(ActionsDAGPtr & actions) { for (const ASTFunction * node : aggregates()) { @@ -411,7 +412,7 @@ bool ExpressionAnalyzer::makeAggregateDescriptions(ExpressionActionsPtr & action { getRootActionsNoMakeSet(arguments[i], true, actions); const std::string & name = arguments[i]->getColumnName(); - types[i] = actions->getSampleBlock().getByName(name).type; + types[i] = actions->getIndex().find(name)->second->result_type; aggregate.argument_names[i] = name; } diff --git a/src/Interpreters/ExpressionAnalyzer.h b/src/Interpreters/ExpressionAnalyzer.h index 458ba9b6770..bf4a4f564a4 100644 --- a/src/Interpreters/ExpressionAnalyzer.h +++ b/src/Interpreters/ExpressionAnalyzer.h @@ -156,7 +156,7 @@ protected: * Set has_aggregation = true if there is GROUP BY or at least one aggregate function. */ void analyzeAggregation(); - bool makeAggregateDescriptions(ExpressionActionsPtr & actions); + bool makeAggregateDescriptions(ActionsDAGPtr & actions); const ASTSelectQuery * getSelectQuery() const; From b461f97e4a1ce54be2669a519c8fb9b2a75786a9 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 11 Sep 2020 15:24:41 +0300 Subject: [PATCH 005/321] Add ExpressionDAG [Part 4]. --- src/Interpreters/ExpressionActions.cpp | 92 +++++++++++++++++------ src/Interpreters/ExpressionActions.h | 26 ++++--- src/Interpreters/ExpressionAnalyzer.cpp | 62 ++++++++------- src/Interpreters/ExpressionAnalyzer.h | 2 +- src/Interpreters/MutationsInterpreter.cpp | 10 +-- 5 files changed, 127 insertions(+), 65 deletions(-) diff --git a/src/Interpreters/ExpressionActions.cpp b/src/Interpreters/ExpressionActions.cpp index 3772ec4fcf7..16c01b8747a 100644 --- a/src/Interpreters/ExpressionActions.cpp +++ b/src/Interpreters/ExpressionActions.cpp @@ -1251,7 +1251,7 @@ void ExpressionActionsChain::addStep() throw Exception("Cannot add action to empty ExpressionActionsChain", ErrorCodes::LOGICAL_ERROR); ColumnsWithTypeAndName columns = steps.back()->getResultColumns(); - steps.push_back(std::make_unique(std::make_shared(columns, context))); + steps.push_back(std::make_unique(std::make_shared(columns))); } void ExpressionActionsChain::finalize() @@ -1398,12 +1398,17 @@ void ExpressionActionsChain::JoinStep::finalize(const Names & required_output_) std::swap(result_columns, new_result_columns); } -ExpressionActionsPtr & ExpressionActionsChain::Step::actions() +ActionsDAGPtr & ExpressionActionsChain::Step::actions() { - return typeid_cast(this)->actions; + return typeid_cast(this)->actions_dag; } -const ExpressionActionsPtr & ExpressionActionsChain::Step::actions() const +const ActionsDAGPtr & ExpressionActionsChain::Step::actions() const +{ + return typeid_cast(this)->actions_dag; +} + +ExpressionActionsPtr ExpressionActionsChain::Step::getExpression() const { return typeid_cast(this)->actions; } @@ -1420,13 +1425,18 @@ ActionsDAG::ActionsDAG(const ColumnsWithTypeAndName & inputs) addInput(input.name, input.type); } -ActionsDAG::Node & ActionsDAG::addNode(Node node) +ActionsDAG::Node & ActionsDAG::addNode(Node node, bool can_replace) { - if (index.count(node.result_name) != 0) + auto it = index.find(node.result_name); + if (it != index.end() && !can_replace) throw Exception("Column '" + node.result_name + "' already exists", ErrorCodes::DUPLICATE_COLUMN); auto & res = nodes.emplace_back(std::move(node)); index[res.result_name] = &res; + + if (it != index.end()) + it->second->renaming_parent = &res; + return res; } @@ -1460,7 +1470,7 @@ const ActionsDAG::Node & ActionsDAG::addColumn(ColumnWithTypeAndName column) return addNode(std::move(node)); } -const ActionsDAG::Node & ActionsDAG::addAlias(const std::string & name, std::string alias) +const ActionsDAG::Node & ActionsDAG::addAlias(const std::string & name, std::string alias, bool can_replace) { auto & child = getNode(name); @@ -1472,7 +1482,7 @@ const ActionsDAG::Node & ActionsDAG::addAlias(const std::string & name, std::str node.allow_constant_folding = child.allow_constant_folding; node.children.emplace_back(&child); - return addNode(std::move(node)); + return addNode(std::move(node), can_replace); } const ActionsDAG::Node & ActionsDAG::addArrayJoin(const std::string & source_name, std::string result_name) @@ -1591,9 +1601,10 @@ const ActionsDAG::Node & ActionsDAG::addFunction( ColumnsWithTypeAndName ActionsDAG::getResultColumns() const { ColumnsWithTypeAndName result; - result.reserve(nodes.size()); + result.reserve(index.size()); for (const auto & node : nodes) - result.emplace_back(node.column, node.result_type, node.result_name); + if (!node.renaming_parent) + result.emplace_back(node.column, node.result_type, node.result_name); return result; } @@ -1602,11 +1613,23 @@ NamesAndTypesList ActionsDAG::getNamesAndTypesList() const { NamesAndTypesList result; for (const auto & node : nodes) - result.emplace_back(node.result_name, node.result_type); + if (!node.renaming_parent) + result.emplace_back(node.result_name, node.result_type); return result; } +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); + + return names; +} + std::string ActionsDAG::dumpNames() const { WriteBufferFromOwnString out; @@ -1625,7 +1648,9 @@ ExpressionActionsPtr ActionsDAG::buildExpressions(const Context & context) { Node * node = nullptr; size_t num_created_children = 0; + size_t num_expected_children = 0; std::vector parents; + Node * renamed_child = nullptr; }; std::vector data(nodes.size()); @@ -1643,13 +1668,38 @@ ExpressionActionsPtr ActionsDAG::buildExpressions(const Context & context) for (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.children.empty()) + 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) + { + if (node.children.empty() && data[reverse_index[&node]].renamed_child == nullptr) 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); while (!ready_nodes.empty() || !ready_array_joins.empty()) @@ -1662,6 +1712,8 @@ ExpressionActionsPtr ActionsDAG::buildExpressions(const Context & context) for (const auto & child : node->children) argument_names.emplace_back(child->result_name); + auto & cur = data[reverse_index[node]]; + switch (node->type) { case Type::INPUT: @@ -1671,7 +1723,7 @@ ExpressionActionsPtr ActionsDAG::buildExpressions(const Context & context) 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)); + expressions->add(ExpressionAction::copyColumn(argument_names.at(0), node->result_name, cur.renamed_child != nullptr)); break; case Type::ARRAY_JOIN: expressions->add(ExpressionAction::arrayJoin(argument_names.at(0), node->result_name)); @@ -1681,17 +1733,11 @@ ExpressionActionsPtr ActionsDAG::buildExpressions(const Context & context) break; } - for (const auto & parent : data[reverse_index[node]].parents) - { - auto & cur = data[reverse_index[parent]]; - ++cur.num_created_children; + for (const auto & parent : cur.parents) + update_parent(parent); - if (parent->children.size() == cur.num_created_children) - { - auto & push_stack = parent->type == Type::ARRAY_JOIN ? ready_array_joins : ready_nodes; - push_stack.push(parent); - } - } + if (node->renaming_parent) + update_parent(node->renaming_parent); } return expressions; diff --git a/src/Interpreters/ExpressionActions.h b/src/Interpreters/ExpressionActions.h index 79107d3baa9..e6e5c038ac3 100644 --- a/src/Interpreters/ExpressionActions.h +++ b/src/Interpreters/ExpressionActions.h @@ -160,6 +160,8 @@ 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; @@ -192,16 +194,16 @@ public: ActionsDAG(const NamesAndTypesList & inputs); ActionsDAG(const ColumnsWithTypeAndName & inputs); - const std::list & getNodes() const; const Index & getIndex() const { return index; } ColumnsWithTypeAndName getResultColumns() const; NamesAndTypesList getNamesAndTypesList() const; + Names getNames() const; std::string dumpNames() const; const Node & addInput(std::string name, DataTypePtr type); const Node & addColumn(ColumnWithTypeAndName column); - const Node & addAlias(const std::string & name, std::string alias); + const Node & addAlias(const std::string & name, std::string alias, bool can_replace); const Node & addArrayJoin(const std::string & source_name, std::string result_name); const Node & addFunction( const FunctionOverloadResolverPtr & function, @@ -212,10 +214,12 @@ public: ExpressionActionsPtr buildExpressions(const Context & context); private: - Node & addNode(Node node); + Node & addNode(Node node, bool can_replace = false); Node & getNode(const std::string & name); }; +using ActionsDAGPtr = std::shared_ptr; + /** Contains a sequence of actions on the block. */ class ExpressionActions @@ -363,17 +367,19 @@ struct ExpressionActionsChain virtual std::string dump() const = 0; /// Only for ExpressionActionsStep - ExpressionActionsPtr & actions(); - const ExpressionActionsPtr & actions() const; + ActionsDAGPtr & actions(); + const ActionsDAGPtr & actions() const; + ExpressionActionsPtr getExpression() const; }; struct ExpressionActionsStep : public Step { + ActionsDAGPtr actions_dag; ExpressionActionsPtr actions; - explicit ExpressionActionsStep(ExpressionActionsPtr actions_, Names required_output_ = Names()) + explicit ExpressionActionsStep(ActionsDAGPtr actions_, Names required_output_ = Names()) : Step(std::move(required_output_)) - , actions(std::move(actions_)) + , actions_dag(std::move(actions_)) { } @@ -458,7 +464,9 @@ struct ExpressionActionsChain throw Exception("Empty ExpressionActionsChain", ErrorCodes::LOGICAL_ERROR); } - return steps.back()->actions(); + auto * step = typeid_cast(&steps.back()); + step->actions = step->actions_dag->buildExpressions(context); + return step->actions; } Step & getLastStep() @@ -472,7 +480,7 @@ struct ExpressionActionsChain Step & lastStep(const NamesAndTypesList & columns) { if (steps.empty()) - steps.emplace_back(std::make_unique(std::make_shared(columns, context))); + steps.emplace_back(std::make_unique(std::make_shared(columns))); return *steps.back(); } diff --git a/src/Interpreters/ExpressionAnalyzer.cpp b/src/Interpreters/ExpressionAnalyzer.cpp index 053d353bdfb..779c9ee7bf7 100644 --- a/src/Interpreters/ExpressionAnalyzer.cpp +++ b/src/Interpreters/ExpressionAnalyzer.cpp @@ -472,8 +472,8 @@ ArrayJoinActionPtr SelectQueryExpressionAnalyzer::appendArrayJoin(ExpressionActi getRootActions(array_join_expression_list, only_types, step.actions()); - before_array_join = chain.getLastActions(); auto array_join = addMultipleArrayJoinAction(step.actions(), is_array_join_left); + before_array_join = chain.getLastActions(); chain.steps.push_back(std::make_unique( array_join, step.getResultColumns())); @@ -615,13 +615,14 @@ JoinPtr SelectQueryExpressionAnalyzer::makeTableJoin(const ASTTablesInSelectQuer return subquery_for_join.join; } -bool SelectQueryExpressionAnalyzer::appendPrewhere( +ExpressionActionsPtr SelectQueryExpressionAnalyzer::appendPrewhere( ExpressionActionsChain & chain, bool only_types, const Names & additional_required_columns) { const auto * select_query = getSelectQuery(); + ExpressionActionsPtr prewhere_actions; if (!select_query->prewhere()) - return false; + return prewhere_actions; auto & step = chain.lastStep(sourceColumns()); getRootActions(select_query->prewhere(), only_types, step.actions()); @@ -629,15 +630,16 @@ bool SelectQueryExpressionAnalyzer::appendPrewhere( step.required_output.push_back(prewhere_column_name); step.can_remove_required_output.push_back(true); - auto filter_type = step.actions()->getSampleBlock().getByName(prewhere_column_name).type; + auto filter_type = step.actions()->getIndex().find(prewhere_column_name)->second->result_type; if (!filter_type->canBeUsedInBooleanContext()) throw Exception("Invalid type for filter in PREWHERE: " + filter_type->getName(), ErrorCodes::ILLEGAL_TYPE_OF_COLUMN_FOR_FILTER); { /// Remove unused source_columns from prewhere actions. - auto tmp_actions = std::make_shared(sourceColumns(), context); - getRootActions(select_query->prewhere(), only_types, tmp_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}); auto required_columns = tmp_actions->getRequiredColumns(); NameSet required_source_columns(required_columns.begin(), required_columns.end()); @@ -653,7 +655,7 @@ bool SelectQueryExpressionAnalyzer::appendPrewhere( } } - auto names = step.actions()->getSampleBlock().getNames(); + auto names = step.actions()->getNames(); NameSet name_set(names.begin(), names.end()); for (const auto & column : sourceColumns()) @@ -661,7 +663,8 @@ bool SelectQueryExpressionAnalyzer::appendPrewhere( name_set.erase(column.name); Names required_output(name_set.begin(), name_set.end()); - step.actions()->finalize(required_output); + prewhere_actions = chain.getLastActions(); + prewhere_actions->finalize(required_output); } { @@ -672,8 +675,8 @@ bool 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 = step.actions()->getSampleBlock().getColumnsWithTypeAndName(); - auto required_columns = step.actions()->getRequiredColumns(); + ColumnsWithTypeAndName columns = prewhere_actions->getSampleBlock().getColumnsWithTypeAndName(); + auto required_columns = prewhere_actions->getRequiredColumns(); NameSet prewhere_input_names(required_columns.begin(), required_columns.end()); NameSet unused_source_columns; @@ -687,11 +690,13 @@ bool SelectQueryExpressionAnalyzer::appendPrewhere( } chain.steps.emplace_back(std::make_unique( - std::make_shared(std::move(columns), context))); + std::make_shared(std::move(columns)))); chain.steps.back()->additional_input = std::move(unused_source_columns); + chain.getLastActions(); + chain.addStep(); } - return true; + return prewhere_actions; } void SelectQueryExpressionAnalyzer::appendPreliminaryFilter(ExpressionActionsChain & chain, ExpressionActionsPtr actions, String column_name) @@ -699,7 +704,8 @@ void SelectQueryExpressionAnalyzer::appendPreliminaryFilter(ExpressionActionsCha ExpressionActionsChain::Step & step = chain.lastStep(sourceColumns()); // FIXME: assert(filter_info); - step.actions() = std::move(actions); + auto * expression_step = typeid_cast(&step); + expression_step->actions = std::move(actions); step.required_output.push_back(std::move(column_name)); step.can_remove_required_output = {true}; @@ -721,7 +727,7 @@ bool SelectQueryExpressionAnalyzer::appendWhere(ExpressionActionsChain & chain, getRootActions(select_query->where(), only_types, step.actions()); - auto filter_type = step.actions()->getSampleBlock().getByName(where_column_name).type; + auto filter_type = step.actions()->getIndex().find(where_column_name)->second->result_type; if (!filter_type->canBeUsedInBooleanContext()) throw Exception("Invalid type for filter in WHERE: " + filter_type->getName(), ErrorCodes::ILLEGAL_TYPE_OF_COLUMN_FOR_FILTER); @@ -750,8 +756,9 @@ bool SelectQueryExpressionAnalyzer::appendGroupBy(ExpressionActionsChain & chain { for (auto & child : asts) { - group_by_elements_actions.emplace_back(std::make_shared(columns_after_join, context)); - getRootActions(child, only_types, group_by_elements_actions.back()); + 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)); } } @@ -838,8 +845,9 @@ bool SelectQueryExpressionAnalyzer::appendOrderBy(ExpressionActionsChain & chain { for (auto & child : select_query->orderBy()->children) { - order_by_elements_actions.emplace_back(std::make_shared(columns_after_join, context)); - getRootActions(child, only_types, order_by_elements_actions.back()); + 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)); } } return true; @@ -919,7 +927,7 @@ void SelectQueryExpressionAnalyzer::appendProjectResult(ExpressionActionsChain & } } - step.actions()->add(ExpressionAction::project(result_columns)); + chain.getLastActions()->add(ExpressionAction::project(result_columns)); } @@ -933,7 +941,7 @@ void ExpressionAnalyzer::appendExpression(ExpressionActionsChain & chain, const ExpressionActionsPtr ExpressionAnalyzer::getActions(bool add_aliases, bool project_result) { - ExpressionActionsPtr actions = std::make_shared(aggregated_columns, context); + auto actions_dag = std::make_shared(aggregated_columns); NamesWithAliases result_columns; Names result_names; @@ -954,9 +962,11 @@ ExpressionActionsPtr ExpressionAnalyzer::getActions(bool add_aliases, bool proje alias = name; result_columns.emplace_back(name, alias); result_names.push_back(alias); - getRootActions(ast, false, actions); + getRootActions(ast, false, actions_dag); } + auto actions = actions_dag->buildExpressions(context); + if (add_aliases) { if (project_result) @@ -980,10 +990,10 @@ ExpressionActionsPtr ExpressionAnalyzer::getActions(bool add_aliases, bool proje ExpressionActionsPtr ExpressionAnalyzer::getConstActions() { - ExpressionActionsPtr actions = std::make_shared(NamesAndTypesList(), context); + auto actions = std::make_shared(NamesAndTypesList()); getRootActions(query, true, actions, true); - return actions; + return actions->buildExpressions(context); } ExpressionActionsPtr SelectQueryExpressionAnalyzer::simpleSelectActions() @@ -1064,10 +1074,9 @@ ExpressionAnalysisResult::ExpressionAnalysisResult( query_analyzer.appendPreliminaryFilter(chain, filter_info->actions, filter_info->column_name); } - if (query_analyzer.appendPrewhere(chain, !first_stage, additional_required_columns_after_prewhere)) + if (auto actions = query_analyzer.appendPrewhere(chain, !first_stage, additional_required_columns_after_prewhere)) { - prewhere_info = std::make_shared( - chain.steps.front()->actions(), query.prewhere()->getColumnName()); + prewhere_info = std::make_shared(actions, query.prewhere()->getColumnName()); if (allowEarlyConstantFolding(*prewhere_info->prewhere_actions, settings)) { @@ -1081,7 +1090,6 @@ ExpressionAnalysisResult::ExpressionAnalysisResult( prewhere_constant_filter_description = ConstantFilterDescription(*column_elem.column); } } - chain.addStep(); } array_join = query_analyzer.appendArrayJoin(chain, before_array_join, only_types || !first_stage); diff --git a/src/Interpreters/ExpressionAnalyzer.h b/src/Interpreters/ExpressionAnalyzer.h index bf4a4f564a4..7728cd9e6ea 100644 --- a/src/Interpreters/ExpressionAnalyzer.h +++ b/src/Interpreters/ExpressionAnalyzer.h @@ -319,7 +319,7 @@ private: void appendPreliminaryFilter(ExpressionActionsChain & chain, ExpressionActionsPtr 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). - bool appendPrewhere(ExpressionActionsChain & chain, bool only_types, const Names & additional_required_columns); + ExpressionActionsPtr 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); diff --git a/src/Interpreters/MutationsInterpreter.cpp b/src/Interpreters/MutationsInterpreter.cpp index 089e3d1c23f..2639c94a9ca 100644 --- a/src/Interpreters/MutationsInterpreter.cpp +++ b/src/Interpreters/MutationsInterpreter.cpp @@ -612,8 +612,8 @@ ASTPtr MutationsInterpreter::prepareInterpreterSelectQuery(std::vector & for (const auto & kv : stage.column_to_updated) { - actions_chain.getLastActions()->add(ExpressionAction::copyColumn( - kv.second->getColumnName(), kv.first, /* can_replace = */ true)); + actions_chain.getLastStep().actions()->addAlias( + kv.second->getColumnName(), kv.first, /* can_replace = */ true); } } @@ -624,7 +624,7 @@ ASTPtr MutationsInterpreter::prepareInterpreterSelectQuery(std::vector & actions_chain.finalize(); /// Propagate information about columns needed as input. - for (const auto & column : actions_chain.steps.front()->actions()->getRequiredColumnsWithTypes()) + for (const auto & column : actions_chain.steps.front()->getRequiredColumns()) prepared_stages[i - 1].output_columns.insert(column.name); } @@ -670,7 +670,7 @@ void MutationsInterpreter::addStreamsForLaterStages(const std::vector & p /// Execute DELETEs. pipeline.addSimpleTransform([&](const Block & header) { - return std::make_shared(header, step->actions(), stage.filter_column_names[i], false); + return std::make_shared(header, step->getExpression(), stage.filter_column_names[i], false); }); } else @@ -678,7 +678,7 @@ void MutationsInterpreter::addStreamsForLaterStages(const std::vector & p /// Execute UPDATE or final projection. pipeline.addSimpleTransform([&](const Block & header) { - return std::make_shared(header, step->actions()); + return std::make_shared(header, step->getExpression()); }); } } From 4aed515e194f5df3b1104f8d7a2efc56a0055f9b Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Sun, 13 Sep 2020 16:51:31 +0300 Subject: [PATCH 006/321] Add ExpressionDAG [Part 5]. --- src/Interpreters/ActionsVisitor.cpp | 79 ++++++++++++++--------- src/Interpreters/ActionsVisitor.h | 12 +++- src/Interpreters/ExpressionActions.cpp | 75 +++++++++++++++++---- src/Interpreters/ExpressionActions.h | 9 ++- src/Interpreters/ExpressionAnalyzer.cpp | 40 ++++++++---- src/Interpreters/ExpressionAnalyzer.h | 2 +- src/Interpreters/MutationsInterpreter.cpp | 1 + 7 files changed, 152 insertions(+), 66 deletions(-) diff --git a/src/Interpreters/ActionsVisitor.cpp b/src/Interpreters/ActionsVisitor.cpp index 75b29f61ae7..4b761225291 100644 --- a/src/Interpreters/ActionsVisitor.cpp +++ b/src/Interpreters/ActionsVisitor.cpp @@ -53,6 +53,7 @@ namespace ErrorCodes extern const int TYPE_MISMATCH; extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; extern const int INCORRECT_ELEMENT_OF_SET; + extern const int BAD_ARGUMENTS; } static NamesAndTypesList::iterator findColumn(const String & name, NamesAndTypesList & cols) @@ -387,60 +388,81 @@ SetPtr makeExplicitSet( ScopeStack::ScopeStack(ActionsDAGPtr actions, const Context & context_) : context(context_) { - stack.emplace_back(std::move(actions)); + auto & level = stack.emplace_back(); + level.actions = std::move(actions); + + for (const auto & [name, node] : level.actions->getIndex()) + if (node->type == ActionsDAG::Type::INPUT) + level.inputs.emplace(name); } void ScopeStack::pushLevel(const NamesAndTypesList & input_columns) { - auto & actions = stack.emplace_back(std::make_shared()); + auto & level = stack.emplace_back(); + level.actions = std::make_shared(); const auto & prev = stack[stack.size() - 2]; for (const auto & input_column : input_columns) - actions->addInput(input_column.name, input_column.type); + { + level.actions->addInput(input_column.name, input_column.type); + level.inputs.emplace(input_column.name); + } - const auto & index = actions->getIndex(); + const auto & index = level.actions->getIndex(); - for (const auto & [name, node] : prev->getIndex()) + for (const auto & [name, node] : prev.actions->getIndex()) { if (index.count(name) == 0) - actions->addInput(node->result_name, node->result_type); + level.actions->addInput({node->column, node->result_type, node->result_name}); } } size_t ScopeStack::getColumnLevel(const std::string & name) { for (int i = static_cast(stack.size()) - 1; i >= 0; --i) - if (stack[i]->getIndex().count(name)) + { + if (stack[i].inputs.count(name)) return i; + const auto & index = stack[i].actions->getIndex(); + auto it = index.find(name); + + if (it != index.end() && it->second->type != ActionsDAG::Type::INPUT) + return i; + } + throw Exception("Unknown identifier: " + name, ErrorCodes::UNKNOWN_IDENTIFIER); } void ScopeStack::addColumn(ColumnWithTypeAndName column) { - auto level = getColumnLevel(column.name); - const auto & node = stack[level]->addColumn(std::move(column)); + const auto & node = stack[0].actions->addColumn(std::move(column)); - for (size_t j = level + 1; j < stack.size(); ++j) - stack[j]->addInput(node.result_name, node.result_type); + for (size_t j = 1; j < stack.size(); ++j) + stack[j].actions->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]->addAlias(name, std::move(alias)); + auto level = getColumnLevel(name); + const auto & node = stack[level].actions->addAlias(name, std::move(alias)); for (size_t j = level + 1; j < stack.size(); ++j) - stack[j]->addInput(node.result_name, node.result_type); + stack[j].actions->addInput({node.column, node.result_type, node.result_name}); } -void ScopeStack::addArrayJoin(const std::string & source_name, std::string result_name) +void ScopeStack::addArrayJoin(const std::string & source_name, std::string result_name, std::string unique_column_name) { - auto level = getColumnLevel(source_name); - const auto & node = stack[level]->addAlias(source_name, std::move(result_name)); + getColumnLevel(source_name); - for (size_t j = level + 1; j < stack.size(); ++j) - stack[j]->addInput(node.result_name, node.result_type); + if (stack.front().actions->getIndex().count(source_name) == 0) + 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)); + + for (size_t j = 1; j < stack.size(); ++j) + stack[j].actions->addInput({node.column, node.result_type, node.result_name}); } void ScopeStack::addFunction( @@ -453,27 +475,27 @@ void ScopeStack::addFunction( for (const auto & argument : argument_names) level = std::max(level, getColumnLevel(argument)); - const auto & node = stack[level]->addFunction(function, argument_names, std::move(result_name), compile_expressions); + const auto & node = stack[level].actions->addFunction(function, argument_names, std::move(result_name), compile_expressions); for (size_t j = level + 1; j < stack.size(); ++j) - stack[j]->addInput(node.result_name, node.result_type); + stack[j].actions->addInput({node.column, node.result_type, node.result_name}); } ActionsDAGPtr ScopeStack::popLevel() { auto res = std::move(stack.back()); stack.pop_back(); - return res; + return res.actions; } std::string ScopeStack::dumpNames() const { - return stack.back()->dumpNames(); + return stack.back().actions->dumpNames(); } const ActionsDAG::Index & ScopeStack::getIndex() const { - return stack.back()->getIndex(); + return stack.back().actions->getIndex(); } struct CachedColumnName @@ -560,14 +582,7 @@ void ActionsMatcher::visit(const ASTFunction & node, const ASTPtr & ast, Data & if (!data.only_consts) { String result_name = column_name.get(ast); - /// 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. - String tmp_name = data.getUniqueName("_array_join_" + arg->getColumnName()); - data.addAlias(arg->getColumnName(), tmp_name); - data.addArrayJoin(tmp_name, result_name); + data.addArrayJoin(arg->getColumnName(), result_name); } return; diff --git a/src/Interpreters/ActionsVisitor.h b/src/Interpreters/ActionsVisitor.h index 109d547ed55..da8ee25a2f4 100644 --- a/src/Interpreters/ActionsVisitor.h +++ b/src/Interpreters/ActionsVisitor.h @@ -34,7 +34,13 @@ SetPtr makeExplicitSet( */ struct ScopeStack { - using Levels = std::vector; + struct Level + { + ActionsDAGPtr actions; + NameSet inputs; + }; + + using Levels = std::vector; Levels stack; @@ -48,7 +54,7 @@ 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); + void addArrayJoin(const std::string & source_name, std::string result_name, std::string unique_column_name); void addFunction( const FunctionOverloadResolverPtr & function, const Names & argument_names, @@ -130,7 +136,7 @@ public: void addArrayJoin(const std::string & source_name, std::string result_name) { - actions_stack.addArrayJoin(source_name, std::move(result_name)); + actions_stack.addArrayJoin(source_name, std::move(result_name), getUniqueName("_array_join_" + source_name)); } void addFunction(const FunctionOverloadResolverPtr & function, diff --git a/src/Interpreters/ExpressionActions.cpp b/src/Interpreters/ExpressionActions.cpp index 16c01b8747a..9319f6db854 100644 --- a/src/Interpreters/ExpressionActions.cpp +++ b/src/Interpreters/ExpressionActions.cpp @@ -16,7 +16,7 @@ #include #include #include -#include +#include #if !defined(ARCADIA_BUILD) # include "config_core.h" @@ -188,7 +188,8 @@ void ExpressionAction::prepare(Block & sample_block, const Settings & settings, size_t result_position = sample_block.columns(); sample_block.insert({nullptr, result_type, result_name}); - function = function_base->prepare(sample_block, arguments, result_position); + if (!function) + function = function_base->prepare(sample_block, arguments, result_position); function->createLowCardinalityResultCache(settings.max_threads); bool compile_expressions = false; @@ -200,7 +201,10 @@ void ExpressionAction::prepare(Block & sample_block, const Settings & settings, /// so we don't want to unfold non deterministic functions if (all_const && function_base->isSuitableForConstantFolding() && (!compile_expressions || function_base->isDeterministic())) { - function->execute(sample_block, arguments, result_position, sample_block.rows(), true); + if (added_column) + sample_block.getByPosition(result_position).column = added_column; + else + function->execute(sample_block, arguments, result_position, 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); @@ -588,8 +592,11 @@ void ExpressionActions::addImpl(ExpressionAction action, Names & new_names) arguments[i] = sample_block.getByName(action.argument_names[i]); } - action.function_base = action.function_builder->build(arguments); - action.result_type = action.function_base->getReturnType(); + 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) @@ -1250,6 +1257,12 @@ void ExpressionActionsChain::addStep() 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(); steps.push_back(std::make_unique(std::make_shared(columns))); } @@ -1422,7 +1435,7 @@ ActionsDAG::ActionsDAG(const NamesAndTypesList & inputs) ActionsDAG::ActionsDAG(const ColumnsWithTypeAndName & inputs) { for (const auto & input : inputs) - addInput(input.name, input.type); + addInput(input); } ActionsDAG::Node & ActionsDAG::addNode(Node node, bool can_replace) @@ -1432,11 +1445,11 @@ ActionsDAG::Node & ActionsDAG::addNode(Node node, bool can_replace) throw Exception("Column '" + node.result_name + "' already exists", ErrorCodes::DUPLICATE_COLUMN); auto & res = nodes.emplace_back(std::move(node)); - index[res.result_name] = &res; if (it != index.end()) it->second->renaming_parent = &res; + index[res.result_name] = &res; return res; } @@ -1459,8 +1472,22 @@ const ActionsDAG::Node & ActionsDAG::addInput(std::string name, DataTypePtr type return addNode(std::move(node)); } +const ActionsDAG::Node & ActionsDAG::addInput(ColumnWithTypeAndName column) +{ + Node node; + node.type = Type::INPUT; + node.result_type = std::move(column.type); + node.result_name = std::move(column.name); + node.column = std::move(column.column); + + return addNode(std::move(node)); +} + const ActionsDAG::Node & ActionsDAG::addColumn(ColumnWithTypeAndName column) { + if (!column.column) + throw Exception("Cannot add column " + column.name + " because it is nullptr", ErrorCodes::LOGICAL_ERROR); + Node node; node.type = Type::COLUMN; node.result_type = std::move(column.type); @@ -1485,7 +1512,8 @@ 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) +const ActionsDAG::Node & ActionsDAG::addArrayJoin( + const std::string & source_name, std::string result_name, std::string unique_column_name) { auto & child = getNode(source_name); @@ -1497,6 +1525,7 @@ const ActionsDAG::Node & ActionsDAG::addArrayJoin(const std::string & source_nam 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)); @@ -1506,7 +1535,7 @@ const ActionsDAG::Node & ActionsDAG::addFunction( const FunctionOverloadResolverPtr & function, const Names & argument_names, std::string result_name, - bool compile_expressions) + bool compile_expressions [[maybe_unused]]) { size_t num_arguments = argument_names.size(); @@ -1663,8 +1692,8 @@ ExpressionActionsPtr ActionsDAG::buildExpressions(const Context & context) reverse_index[&node] = id; } - std::stack ready_nodes; - std::stack ready_array_joins; + std::queue ready_nodes; + std::queue ready_array_joins; for (auto & node : nodes) { @@ -1705,7 +1734,7 @@ ExpressionActionsPtr ActionsDAG::buildExpressions(const Context & context) while (!ready_nodes.empty() || !ready_array_joins.empty()) { auto & stack = ready_nodes.empty() ? ready_array_joins : ready_nodes; - Node * node = stack.top(); + Node * node = stack.front(); stack.pop(); Names argument_names; @@ -1726,11 +1755,29 @@ ExpressionActionsPtr ActionsDAG::buildExpressions(const Context & context) expressions->add(ExpressionAction::copyColumn(argument_names.at(0), node->result_name, cur.renamed_child != nullptr)); break; case Type::ARRAY_JOIN: - expressions->add(ExpressionAction::arrayJoin(argument_names.at(0), node->result_name)); + /// 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: - expressions->add(ExpressionAction::applyFunction(node->function_builder, argument_names, node->result_name)); + { + 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; + } } for (const auto & parent : cur.parents) diff --git a/src/Interpreters/ExpressionActions.h b/src/Interpreters/ExpressionActions.h index e6e5c038ac3..b35f8972c97 100644 --- a/src/Interpreters/ExpressionActions.h +++ b/src/Interpreters/ExpressionActions.h @@ -168,6 +168,8 @@ public: 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; @@ -202,9 +204,10 @@ public: std::string dumpNames() 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); - const Node & addArrayJoin(const std::string & source_name, std::string result_name); + 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 & addFunction( const FunctionOverloadResolverPtr & function, const Names & argument_names, @@ -464,7 +467,7 @@ struct ExpressionActionsChain throw Exception("Empty ExpressionActionsChain", ErrorCodes::LOGICAL_ERROR); } - auto * step = typeid_cast(&steps.back()); + auto * step = typeid_cast(steps.back().get()); step->actions = step->actions_dag->buildExpressions(context); return step->actions; } diff --git a/src/Interpreters/ExpressionAnalyzer.cpp b/src/Interpreters/ExpressionAnalyzer.cpp index 779c9ee7bf7..a4337f7b12c 100644 --- a/src/Interpreters/ExpressionAnalyzer.cpp +++ b/src/Interpreters/ExpressionAnalyzer.cpp @@ -158,24 +158,37 @@ void ExpressionAnalyzer::analyzeAggregation() if (select_query) { NamesAndTypesList array_join_columns; + columns_after_array_join = sourceColumns(); bool is_array_join_left; if (ASTPtr array_join_expression_list = select_query->arrayJoinExpressionList(is_array_join_left)) { getRootActionsNoMakeSet(array_join_expression_list, true, temp_actions, false); - if (auto array_join = addMultipleArrayJoinAction(temp_actions, is_array_join_left)) - { - auto sample_columns = temp_actions->getResultColumns(); - array_join->prepare(sample_columns); - temp_actions = std::make_shared(sample_columns); - } + + auto array_join = addMultipleArrayJoinAction(temp_actions, is_array_join_left); + auto sample_columns = temp_actions->getResultColumns(); + array_join->prepare(sample_columns); + temp_actions = std::make_shared(sample_columns); + + NamesAndTypesList new_columns_after_array_join; + NameSet added_columns; for (auto & column : temp_actions->getResultColumns()) + { if (syntax->array_join_result_to_source.count(column.name)) - array_join_columns.emplace_back(column.name, column.type); + { + new_columns_after_array_join.emplace_back(column.name, column.type); + added_columns.emplace(column.name); + } + } + + for (auto & column : columns_after_array_join) + if (added_columns.count(column.name) == 0) + new_columns_after_array_join.emplace_back(column.name, column.type); + + columns_after_array_join.swap(new_columns_after_array_join); } - columns_after_array_join = sourceColumns(); columns_after_array_join.insert(columns_after_array_join.end(), array_join_columns.begin(), array_join_columns.end()); const ASTTablesInSelectQueryElement * join = select_query->join(); @@ -393,7 +406,7 @@ void ExpressionAnalyzer::getRootActionsNoMakeSet(const ASTPtr & ast, bool no_sub sourceColumns(), std::move(actions), prepared_sets, subqueries_for_sets, no_subqueries, true, only_consts, !isRemoteStorage()); ActionsVisitor(visitor_data, log.stream()).visit(ast); - visitor_data.getActions(); + actions = visitor_data.getActions(); } @@ -881,7 +894,7 @@ bool SelectQueryExpressionAnalyzer::appendLimitBy(ExpressionActionsChain & chain return true; } -void SelectQueryExpressionAnalyzer::appendProjectResult(ExpressionActionsChain & chain) const +ExpressionActionsPtr SelectQueryExpressionAnalyzer::appendProjectResult(ExpressionActionsChain & chain) const { const auto * select_query = getSelectQuery(); @@ -927,7 +940,9 @@ void SelectQueryExpressionAnalyzer::appendProjectResult(ExpressionActionsChain & } } - chain.getLastActions()->add(ExpressionAction::project(result_columns)); + auto actions = chain.getLastActions(); + actions->add(ExpressionAction::project(result_columns)); + return actions; } @@ -1175,8 +1190,7 @@ ExpressionAnalysisResult::ExpressionAnalysisResult( chain.addStep(); } - query_analyzer.appendProjectResult(chain); - final_projection = chain.getLastActions(); + final_projection = query_analyzer.appendProjectResult(chain); finalize_chain(chain); } diff --git a/src/Interpreters/ExpressionAnalyzer.h b/src/Interpreters/ExpressionAnalyzer.h index 7728cd9e6ea..825bf858713 100644 --- a/src/Interpreters/ExpressionAnalyzer.h +++ b/src/Interpreters/ExpressionAnalyzer.h @@ -269,7 +269,7 @@ public: /// 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. - void appendProjectResult(ExpressionActionsChain & chain) const; + ExpressionActionsPtr appendProjectResult(ExpressionActionsChain & chain) const; private: StorageMetadataPtr metadata_snapshot; diff --git a/src/Interpreters/MutationsInterpreter.cpp b/src/Interpreters/MutationsInterpreter.cpp index 2639c94a9ca..06738ce6ff6 100644 --- a/src/Interpreters/MutationsInterpreter.cpp +++ b/src/Interpreters/MutationsInterpreter.cpp @@ -620,6 +620,7 @@ ASTPtr MutationsInterpreter::prepareInterpreterSelectQuery(std::vector & /// Remove all intermediate columns. actions_chain.addStep(); actions_chain.getLastStep().required_output.assign(stage.output_columns.begin(), stage.output_columns.end()); + actions_chain.getLastActions(); actions_chain.finalize(); From cbf828270162a5b90272e8328aaaac2c43b9fdbe Mon Sep 17 00:00:00 2001 From: Vxider Date: Sun, 13 Sep 2020 23:11:18 +0800 Subject: [PATCH 007/321] add disable_set_and_join_persistency --- src/Core/Settings.h | 1 + src/Storages/StorageJoin.cpp | 7 ++++- src/Storages/StorageJoin.h | 1 + src/Storages/StorageSet.cpp | 52 +++++++++++++++++++++++++++--------- src/Storages/StorageSet.h | 3 +++ 5 files changed, 51 insertions(+), 13 deletions(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index b39c223a5e9..cc4a512a08b 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -171,6 +171,7 @@ class IColumn; M(Bool, enable_http_compression, 0, "Compress the result if the client over HTTP said that it understands data compressed by gzip or deflate.", 0) \ M(Int64, http_zlib_compression_level, 3, "Compression level - used if the client on HTTP said that it understands data compressed by gzip or deflate.", 0) \ \ + M(Bool, disable_set_and_join_persistency, false, "Disable persistency for StorageSet and StorageJoin to reduce IO overhead", 0) \ M(Bool, http_native_compression_disable_checksumming_on_decompress, 0, "If you uncompress the POST data from the client compressed by the native format, do not check the checksum.", 0) \ \ M(String, count_distinct_implementation, "uniqExact", "What aggregate function to use for implementation of count(DISTINCT ...)", 0) \ diff --git a/src/Storages/StorageJoin.cpp b/src/Storages/StorageJoin.cpp index 4e3fe01e858..a3bd70596e9 100644 --- a/src/Storages/StorageJoin.cpp +++ b/src/Storages/StorageJoin.cpp @@ -44,8 +44,9 @@ StorageJoin::StorageJoin( const ColumnsDescription & columns_, const ConstraintsDescription & constraints_, bool overwrite_, + bool disable_set_and_join_persistency_, const Context & context_) - : StorageSetOrJoinBase{relative_path_, table_id_, columns_, constraints_, context_} + : StorageSetOrJoinBase{relative_path_, table_id_, columns_, constraints_, disable_set_and_join_persistency_, context_} , key_names(key_names_) , use_nulls(use_nulls_) , limits(limits_) @@ -118,6 +119,7 @@ void registerStorageJoin(StorageFactory & factory) auto join_overflow_mode = settings.join_overflow_mode; auto join_any_take_last_row = settings.join_any_take_last_row; auto old_any_join = settings.any_join_distinct_right_table_keys; + auto disable_set_and_join_persistency = settings.disable_set_and_join_persistency; if (args.storage_def && args.storage_def->settings) { @@ -135,6 +137,8 @@ void registerStorageJoin(StorageFactory & factory) join_any_take_last_row = setting.value; else if (setting.name == "any_join_distinct_right_table_keys") old_any_join = setting.value; + else if (setting.name == "disable_set_and_join_persistency") + disable_set_and_join_persistency = setting.value; else throw Exception( "Unknown setting " + setting.name + " for storage " + args.engine_name, @@ -217,6 +221,7 @@ void registerStorageJoin(StorageFactory & factory) args.columns, args.constraints, join_any_take_last_row, + disable_set_and_join_persistency, args.context); }; diff --git a/src/Storages/StorageJoin.h b/src/Storages/StorageJoin.h index 386282708c9..9f06304d10d 100644 --- a/src/Storages/StorageJoin.h +++ b/src/Storages/StorageJoin.h @@ -72,6 +72,7 @@ protected: const ColumnsDescription & columns_, const ConstraintsDescription & constraints_, bool overwrite, + bool disable_set_and_join_persistency_, const Context & context_); }; diff --git a/src/Storages/StorageSet.cpp b/src/Storages/StorageSet.cpp index f2946afbbfd..0b54d04910c 100644 --- a/src/Storages/StorageSet.cpp +++ b/src/Storages/StorageSet.cpp @@ -12,6 +12,8 @@ #include #include #include +#include +#include namespace DB @@ -35,7 +37,7 @@ public: SetOrJoinBlockOutputStream( StorageSetOrJoinBase & table_, const StorageMetadataPtr & metadata_snapshot_, const String & backup_path_, const String & backup_tmp_path_, - const String & backup_file_name_); + const String & backup_file_name_, bool disable_set_and_join_persistency_); Block getHeader() const override { return metadata_snapshot->getSampleBlock(); } void write(const Block & block) override; @@ -50,6 +52,7 @@ private: WriteBufferFromFile backup_buf; CompressedWriteBuffer compressed_backup_buf; NativeBlockOutputStream backup_stream; + bool disable_set_and_join_persistency; }; @@ -58,7 +61,8 @@ SetOrJoinBlockOutputStream::SetOrJoinBlockOutputStream( const StorageMetadataPtr & metadata_snapshot_, const String & backup_path_, const String & backup_tmp_path_, - const String & backup_file_name_) + const String & backup_file_name_, + bool disable_set_and_join_persistency_) : table(table_) , metadata_snapshot(metadata_snapshot_) , backup_path(backup_path_) @@ -67,6 +71,7 @@ SetOrJoinBlockOutputStream::SetOrJoinBlockOutputStream( , backup_buf(backup_tmp_path + backup_file_name) , compressed_backup_buf(backup_buf) , backup_stream(compressed_backup_buf, 0, metadata_snapshot->getSampleBlock()) + , disable_set_and_join_persistency(disable_set_and_join_persistency_) { } @@ -76,24 +81,28 @@ void SetOrJoinBlockOutputStream::write(const Block & block) Block sorted_block = block.sortColumns(); table.insertBlock(sorted_block); - backup_stream.write(sorted_block); + if(!disable_set_and_join_persistency) + backup_stream.write(sorted_block); } void SetOrJoinBlockOutputStream::writeSuffix() { table.finishInsert(); - backup_stream.flush(); - compressed_backup_buf.next(); - backup_buf.next(); + if(!disable_set_and_join_persistency) + { + backup_stream.flush(); + compressed_backup_buf.next(); + backup_buf.next(); - Poco::File(backup_tmp_path + backup_file_name).renameTo(backup_path + backup_file_name); + Poco::File(backup_tmp_path + backup_file_name).renameTo(backup_path + backup_file_name); + } } BlockOutputStreamPtr StorageSetOrJoinBase::write(const ASTPtr & /*query*/, const StorageMetadataPtr & metadata_snapshot, const Context & /*context*/) { UInt64 id = ++increment; - return std::make_shared(*this, metadata_snapshot, path, path + "tmp/", toString(id) + ".bin"); + return std::make_shared(*this, metadata_snapshot, path, path + "tmp/", toString(id) + ".bin", disable_set_and_join_persistency); } @@ -102,8 +111,10 @@ StorageSetOrJoinBase::StorageSetOrJoinBase( const StorageID & table_id_, const ColumnsDescription & columns_, const ConstraintsDescription & constraints_, + bool disable_set_and_join_persistency_, const Context & context_) - : IStorage(table_id_) + : IStorage(table_id_), + disable_set_and_join_persistency(disable_set_and_join_persistency_) { StorageInMemoryMetadata storage_metadata; storage_metadata.setColumns(columns_); @@ -124,8 +135,9 @@ StorageSet::StorageSet( const StorageID & table_id_, const ColumnsDescription & columns_, const ConstraintsDescription & constraints_, + bool disable_set_and_join_persistency_, const Context & context_) - : StorageSetOrJoinBase{relative_path_, table_id_, columns_, constraints_, context_}, + : StorageSetOrJoinBase{relative_path_, table_id_, columns_, constraints_, disable_set_and_join_persistency_, context_}, set(std::make_shared(SizeLimits(), false, true)) { @@ -229,8 +241,24 @@ void registerStorageSet(StorageFactory & factory) "Engine " + args.engine_name + " doesn't support any arguments (" + toString(args.engine_args.size()) + " given)", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); - return StorageSet::create(args.relative_data_path, args.table_id, args.columns, args.constraints, args.context); - }); + const auto & settings = args.context.getSettingsRef(); + auto disable_set_and_join_persistency = settings.disable_set_and_join_persistency; + + if (args.storage_def && args.storage_def->settings) + { + for (const auto & setting : args.storage_def->settings->changes) + { + if (setting.name == "disable_set_and_join_persistency") + disable_set_and_join_persistency = setting.value; + else + throw Exception( + "Unknown setting " + setting.name + " for storage " + args.engine_name, + ErrorCodes::BAD_ARGUMENTS); + } + } + + return StorageSet::create(args.relative_data_path, args.table_id, args.columns, args.constraints, disable_set_and_join_persistency, args.context); + }, StorageFactory::StorageFeatures{ .supports_settings = true, }); } diff --git a/src/Storages/StorageSet.h b/src/Storages/StorageSet.h index de7c65bbc3e..834f8f025c7 100644 --- a/src/Storages/StorageSet.h +++ b/src/Storages/StorageSet.h @@ -31,10 +31,12 @@ protected: const StorageID & table_id_, const ColumnsDescription & columns_, const ConstraintsDescription & constraints_, + bool disable_set_and_join_persistency_, const Context & context_); String base_path; String path; + bool disable_set_and_join_persistency; std::atomic increment = 0; /// For the backup file names. @@ -82,6 +84,7 @@ protected: const StorageID & table_id_, const ColumnsDescription & columns_, const ConstraintsDescription & constraints_, + bool disable_set_and_join_persistency_, const Context & context_); }; From b5498e6ed28f31f403f3603c727b10a2ba402609 Mon Sep 17 00:00:00 2001 From: Vxider Date: Mon, 14 Sep 2020 09:39:32 +0800 Subject: [PATCH 008/321] update code style --- src/Storages/StorageSet.cpp | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/src/Storages/StorageSet.cpp b/src/Storages/StorageSet.cpp index 0b54d04910c..9bda779e4c1 100644 --- a/src/Storages/StorageSet.cpp +++ b/src/Storages/StorageSet.cpp @@ -21,6 +21,7 @@ namespace DB namespace ErrorCodes { + extern const int BAD_ARGUMENTS; extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; } @@ -81,14 +82,14 @@ void SetOrJoinBlockOutputStream::write(const Block & block) Block sorted_block = block.sortColumns(); table.insertBlock(sorted_block); - if(!disable_set_and_join_persistency) + if (!disable_set_and_join_persistency) backup_stream.write(sorted_block); } void SetOrJoinBlockOutputStream::writeSuffix() { table.finishInsert(); - if(!disable_set_and_join_persistency) + if (!disable_set_and_join_persistency) { backup_stream.flush(); compressed_backup_buf.next(); From 407bccdefa9130e2ab85662c0c118293f97c20c5 Mon Sep 17 00:00:00 2001 From: Vxider Date: Mon, 14 Sep 2020 15:46:27 +0800 Subject: [PATCH 009/321] Support default arguments for Replicated engine --- src/Core/Settings.h | 2 ++ src/Storages/MergeTree/registerStorageMergeTree.cpp | 13 ++++++++++++- 2 files changed, 14 insertions(+), 1 deletion(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index b39c223a5e9..d0b81f5440e 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -156,6 +156,8 @@ class IColumn; M(UInt64, max_concurrent_queries_for_user, 0, "The maximum number of concurrent requests per user.", 0) \ \ M(Bool, insert_deduplicate, true, "For INSERT queries in the replicated table, specifies that deduplication of insertings blocks should be performed", 0) \ + M(String, default_replica_path, "", "Default replica path for ReplicatedMergeTree. Allow to omit arguments for Replicated table engine if default_replica_path and default_replica_name are specified.", 0) \ + M(String, default_replica_name, "", "Default replica name for ReplicatedMergeTree. Allow to omit arguments for Replicated table engine if default_replica_path and default_replica_name are specified.", 0) \ \ M(UInt64, insert_quorum, 0, "For INSERT queries in the replicated table, wait writing for the specified number of replicas and linearize the addition of the data. 0 - disabled.", 0) \ M(Milliseconds, insert_quorum_timeout, 600000, "", 0) \ diff --git a/src/Storages/MergeTree/registerStorageMergeTree.cpp b/src/Storages/MergeTree/registerStorageMergeTree.cpp index d3af3942428..a0ac364b76f 100644 --- a/src/Storages/MergeTree/registerStorageMergeTree.cpp +++ b/src/Storages/MergeTree/registerStorageMergeTree.cpp @@ -273,8 +273,13 @@ static StoragePtr create(const StorageFactory::Arguments & args) String name_part = args.engine_name.substr(0, args.engine_name.size() - strlen("MergeTree")); bool replicated = startsWith(name_part, "Replicated"); + bool has_replicated_default_args = false; if (replicated) + { name_part = name_part.substr(strlen("Replicated")); + has_replicated_default_args = args.context.getSettingsRef().default_replica_path.value != "" + && args.context.getSettingsRef().default_replica_name.value != ""; + } MergeTreeData::MergingParams merging_params; merging_params.mode = MergeTreeData::MergingParams::Ordinary; @@ -318,7 +323,7 @@ static StoragePtr create(const StorageFactory::Arguments & args) if (replicated) { - if (is_extended_storage_def) + if (is_extended_storage_def || has_replicated_default_args) { add_optional_param("path in ZooKeeper"); add_optional_param("replica name"); @@ -421,6 +426,12 @@ static StoragePtr create(const StorageFactory::Arguments & args) "No replica name in config" + getMergeTreeVerboseHelp(is_extended_storage_def), ErrorCodes::NO_REPLICA_NAME_GIVEN); ++arg_num; } + else if (has_replicated_default_args && !has_arguments) + { + zookeeper_path + = args.context.getSettingsRef().default_replica_path.value + "/" + args.table_id.database_name + "." + args.table_id.table_name; + replica_name = args.context.getSettingsRef().default_replica_name; + } else if (is_extended_storage_def && !has_arguments) { /// Try use default values if arguments are not specified. From 1c82d2eb23bcdbe89703f292fa98ed64ed9b1585 Mon Sep 17 00:00:00 2001 From: Vxider Date: Tue, 15 Sep 2020 17:16:10 +0800 Subject: [PATCH 010/321] use table-level setting --- src/Core/Settings.h | 1 - src/Storages/JoinSettings.cpp | 41 ++++++++++++++++++++++++++++++++ src/Storages/JoinSettings.h | 30 ++++++++++++++++++++++++ src/Storages/SetSettings.cpp | 43 ++++++++++++++++++++++++++++++++++ src/Storages/SetSettings.h | 30 ++++++++++++++++++++++++ src/Storages/StorageJoin.cpp | 20 +++++++++------- src/Storages/StorageJoin.h | 5 ++-- src/Storages/StorageSet.cpp | 44 ++++++++++++++--------------------- src/Storages/StorageSet.h | 11 +++++---- 9 files changed, 183 insertions(+), 42 deletions(-) create mode 100644 src/Storages/JoinSettings.cpp create mode 100644 src/Storages/JoinSettings.h create mode 100644 src/Storages/SetSettings.cpp create mode 100644 src/Storages/SetSettings.h diff --git a/src/Core/Settings.h b/src/Core/Settings.h index cc4a512a08b..b39c223a5e9 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -171,7 +171,6 @@ class IColumn; M(Bool, enable_http_compression, 0, "Compress the result if the client over HTTP said that it understands data compressed by gzip or deflate.", 0) \ M(Int64, http_zlib_compression_level, 3, "Compression level - used if the client on HTTP said that it understands data compressed by gzip or deflate.", 0) \ \ - M(Bool, disable_set_and_join_persistency, false, "Disable persistency for StorageSet and StorageJoin to reduce IO overhead", 0) \ M(Bool, http_native_compression_disable_checksumming_on_decompress, 0, "If you uncompress the POST data from the client compressed by the native format, do not check the checksum.", 0) \ \ M(String, count_distinct_implementation, "uniqExact", "What aggregate function to use for implementation of count(DISTINCT ...)", 0) \ diff --git a/src/Storages/JoinSettings.cpp b/src/Storages/JoinSettings.cpp new file mode 100644 index 00000000000..0d060cdcf41 --- /dev/null +++ b/src/Storages/JoinSettings.cpp @@ -0,0 +1,41 @@ +#include +#include +#include +#include +#include + + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int BAD_ARGUMENTS; + extern const int UNKNOWN_SETTING; +} IMPLEMENT_SETTINGS_TRAITS(joinSettingsTraits, LIST_OF_JOIN_SETTINGS) + +void JoinSettings::loadFromQuery(ASTStorage & storage_def) +{ + if (storage_def.settings) + { + try + { + applyChanges(storage_def.settings->changes); + } + catch (Exception & e) + { + if (e.code() == ErrorCodes::UNKNOWN_SETTING) + throw Exception(e.message() + " for storage " + storage_def.engine->name, ErrorCodes::BAD_ARGUMENTS); + else + e.rethrow(); + } + } + else + { + auto settings_ast = std::make_shared(); + settings_ast->is_standalone = false; + storage_def.set(storage_def.settings, settings_ast); + } +} + +} diff --git a/src/Storages/JoinSettings.h b/src/Storages/JoinSettings.h new file mode 100644 index 00000000000..1758250c737 --- /dev/null +++ b/src/Storages/JoinSettings.h @@ -0,0 +1,30 @@ +#pragma once + +#include +#include + + +namespace DB +{ +class ASTStorage; + + +#define JOIN_RELATED_SETTINGS(M) \ + M(Bool, disable_persistency, false, "Disable persistency for StorageJoin to reduce IO overhead", 0) + +#define LIST_OF_JOIN_SETTINGS(M) \ + JOIN_RELATED_SETTINGS(M) \ + FORMAT_FACTORY_SETTINGS(M) + +DECLARE_SETTINGS_TRAITS(joinSettingsTraits, LIST_OF_JOIN_SETTINGS) + + +/** Settings for the Join engine. + * Could be loaded from a CREATE TABLE query (SETTINGS clause). + */ +struct JoinSettings : public BaseSettings +{ + void loadFromQuery(ASTStorage & storage_def); +}; + +} diff --git a/src/Storages/SetSettings.cpp b/src/Storages/SetSettings.cpp new file mode 100644 index 00000000000..f7ff1c446f2 --- /dev/null +++ b/src/Storages/SetSettings.cpp @@ -0,0 +1,43 @@ +#include +#include +#include +#include +#include + + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int BAD_ARGUMENTS; + extern const int UNKNOWN_SETTING; +} + +IMPLEMENT_SETTINGS_TRAITS(setSettingsTraits, LIST_OF_SET_SETTINGS) + +void SetSettings::loadFromQuery(ASTStorage & storage_def) +{ + if (storage_def.settings) + { + try + { + applyChanges(storage_def.settings->changes); + } + catch (Exception & e) + { + if (e.code() == ErrorCodes::UNKNOWN_SETTING) + throw Exception(e.message() + " for storage " + storage_def.engine->name, ErrorCodes::BAD_ARGUMENTS); + else + e.rethrow(); + } + } + else + { + auto settings_ast = std::make_shared(); + settings_ast->is_standalone = false; + storage_def.set(storage_def.settings, settings_ast); + } +} + +} diff --git a/src/Storages/SetSettings.h b/src/Storages/SetSettings.h new file mode 100644 index 00000000000..1bd3e3895ef --- /dev/null +++ b/src/Storages/SetSettings.h @@ -0,0 +1,30 @@ +#pragma once + +#include +#include + + +namespace DB +{ +class ASTStorage; + + +#define SET_RELATED_SETTINGS(M) \ + M(Bool, disable_persistency, false, "Disable persistency for StorageSet to reduce IO overhead", 0) + +#define LIST_OF_SET_SETTINGS(M) \ + SET_RELATED_SETTINGS(M) \ + FORMAT_FACTORY_SETTINGS(M) + +DECLARE_SETTINGS_TRAITS(setSettingsTraits, LIST_OF_SET_SETTINGS) + + +/** Settings for the Set engine. + * Could be loaded from a CREATE TABLE query (SETTINGS clause). + */ +struct SetSettings : public BaseSettings +{ + void loadFromQuery(ASTStorage & storage_def); +}; + +} diff --git a/src/Storages/StorageJoin.cpp b/src/Storages/StorageJoin.cpp index a3bd70596e9..6ca6411a323 100644 --- a/src/Storages/StorageJoin.cpp +++ b/src/Storages/StorageJoin.cpp @@ -44,9 +44,9 @@ StorageJoin::StorageJoin( const ColumnsDescription & columns_, const ConstraintsDescription & constraints_, bool overwrite_, - bool disable_set_and_join_persistency_, - const Context & context_) - : StorageSetOrJoinBase{relative_path_, table_id_, columns_, constraints_, disable_set_and_join_persistency_, context_} + const Context & context_, + bool disable_persistency_) + : StorageSetOrJoinBase{relative_path_, table_id_, columns_, constraints_, context_, disable_persistency_} , key_names(key_names_) , use_nulls(use_nulls_) , limits(limits_) @@ -119,7 +119,7 @@ void registerStorageJoin(StorageFactory & factory) auto join_overflow_mode = settings.join_overflow_mode; auto join_any_take_last_row = settings.join_any_take_last_row; auto old_any_join = settings.any_join_distinct_right_table_keys; - auto disable_set_and_join_persistency = settings.disable_set_and_join_persistency; + bool disable_persistency = false; if (args.storage_def && args.storage_def->settings) { @@ -137,8 +137,12 @@ void registerStorageJoin(StorageFactory & factory) join_any_take_last_row = setting.value; else if (setting.name == "any_join_distinct_right_table_keys") old_any_join = setting.value; - else if (setting.name == "disable_set_and_join_persistency") - disable_set_and_join_persistency = setting.value; + else if (setting.name == "disable_persistency") + { + auto join_settings = std::make_unique(); + join_settings->loadFromQuery(*args.storage_def); + disable_persistency = join_settings->disable_persistency; + } else throw Exception( "Unknown setting " + setting.name + " for storage " + args.engine_name, @@ -221,8 +225,8 @@ void registerStorageJoin(StorageFactory & factory) args.columns, args.constraints, join_any_take_last_row, - disable_set_and_join_persistency, - args.context); + args.context, + disable_persistency); }; factory.registerStorage("Join", creator_fn, StorageFactory::StorageFeatures{ .supports_settings = true, }); diff --git a/src/Storages/StorageJoin.h b/src/Storages/StorageJoin.h index 9f06304d10d..99cba0b4579 100644 --- a/src/Storages/StorageJoin.h +++ b/src/Storages/StorageJoin.h @@ -3,6 +3,7 @@ #include #include +#include #include @@ -72,8 +73,8 @@ protected: const ColumnsDescription & columns_, const ConstraintsDescription & constraints_, bool overwrite, - bool disable_set_and_join_persistency_, - const Context & context_); + const Context & context_, + bool disable_persistency_); }; } diff --git a/src/Storages/StorageSet.cpp b/src/Storages/StorageSet.cpp index 9bda779e4c1..2f17fe28242 100644 --- a/src/Storages/StorageSet.cpp +++ b/src/Storages/StorageSet.cpp @@ -38,7 +38,7 @@ public: SetOrJoinBlockOutputStream( StorageSetOrJoinBase & table_, const StorageMetadataPtr & metadata_snapshot_, const String & backup_path_, const String & backup_tmp_path_, - const String & backup_file_name_, bool disable_set_and_join_persistency_); + const String & backup_file_name_, bool disable_persistency_); Block getHeader() const override { return metadata_snapshot->getSampleBlock(); } void write(const Block & block) override; @@ -53,7 +53,7 @@ private: WriteBufferFromFile backup_buf; CompressedWriteBuffer compressed_backup_buf; NativeBlockOutputStream backup_stream; - bool disable_set_and_join_persistency; + bool disable_persistency; }; @@ -63,7 +63,7 @@ SetOrJoinBlockOutputStream::SetOrJoinBlockOutputStream( const String & backup_path_, const String & backup_tmp_path_, const String & backup_file_name_, - bool disable_set_and_join_persistency_) + bool disable_persistency_) : table(table_) , metadata_snapshot(metadata_snapshot_) , backup_path(backup_path_) @@ -72,7 +72,7 @@ SetOrJoinBlockOutputStream::SetOrJoinBlockOutputStream( , backup_buf(backup_tmp_path + backup_file_name) , compressed_backup_buf(backup_buf) , backup_stream(compressed_backup_buf, 0, metadata_snapshot->getSampleBlock()) - , disable_set_and_join_persistency(disable_set_and_join_persistency_) + , disable_persistency(disable_persistency_) { } @@ -82,14 +82,14 @@ void SetOrJoinBlockOutputStream::write(const Block & block) Block sorted_block = block.sortColumns(); table.insertBlock(sorted_block); - if (!disable_set_and_join_persistency) + if (!disable_persistency) backup_stream.write(sorted_block); } void SetOrJoinBlockOutputStream::writeSuffix() { table.finishInsert(); - if (!disable_set_and_join_persistency) + if (!disable_persistency) { backup_stream.flush(); compressed_backup_buf.next(); @@ -103,7 +103,7 @@ void SetOrJoinBlockOutputStream::writeSuffix() BlockOutputStreamPtr StorageSetOrJoinBase::write(const ASTPtr & /*query*/, const StorageMetadataPtr & metadata_snapshot, const Context & /*context*/) { UInt64 id = ++increment; - return std::make_shared(*this, metadata_snapshot, path, path + "tmp/", toString(id) + ".bin", disable_set_and_join_persistency); + return std::make_shared(*this, metadata_snapshot, path, path + "tmp/", toString(id) + ".bin", disable_persistency); } @@ -112,10 +112,10 @@ StorageSetOrJoinBase::StorageSetOrJoinBase( const StorageID & table_id_, const ColumnsDescription & columns_, const ConstraintsDescription & constraints_, - bool disable_set_and_join_persistency_, - const Context & context_) + const Context & context_, + bool disable_persistency_) : IStorage(table_id_), - disable_set_and_join_persistency(disable_set_and_join_persistency_) + disable_persistency(disable_persistency_) { StorageInMemoryMetadata storage_metadata; storage_metadata.setColumns(columns_); @@ -136,9 +136,9 @@ StorageSet::StorageSet( const StorageID & table_id_, const ColumnsDescription & columns_, const ConstraintsDescription & constraints_, - bool disable_set_and_join_persistency_, - const Context & context_) - : StorageSetOrJoinBase{relative_path_, table_id_, columns_, constraints_, disable_set_and_join_persistency_, context_}, + const Context & context_, + bool disable_persistency_) + : StorageSetOrJoinBase{relative_path_, table_id_, columns_, constraints_, context_, disable_persistency_}, set(std::make_shared(SizeLimits(), false, true)) { @@ -242,23 +242,15 @@ void registerStorageSet(StorageFactory & factory) "Engine " + args.engine_name + " doesn't support any arguments (" + toString(args.engine_args.size()) + " given)", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); - const auto & settings = args.context.getSettingsRef(); - auto disable_set_and_join_persistency = settings.disable_set_and_join_persistency; + bool has_settings = args.storage_def->settings; - if (args.storage_def && args.storage_def->settings) + auto set_settings = std::make_unique(); + if (has_settings) { - for (const auto & setting : args.storage_def->settings->changes) - { - if (setting.name == "disable_set_and_join_persistency") - disable_set_and_join_persistency = setting.value; - else - throw Exception( - "Unknown setting " + setting.name + " for storage " + args.engine_name, - ErrorCodes::BAD_ARGUMENTS); - } + set_settings->loadFromQuery(*args.storage_def); } - return StorageSet::create(args.relative_data_path, args.table_id, args.columns, args.constraints, disable_set_and_join_persistency, args.context); + return StorageSet::create(args.relative_data_path, args.table_id, args.columns, args.constraints, args.context, set_settings->disable_persistency); }, StorageFactory::StorageFeatures{ .supports_settings = true, }); } diff --git a/src/Storages/StorageSet.h b/src/Storages/StorageSet.h index 834f8f025c7..681315d7d11 100644 --- a/src/Storages/StorageSet.h +++ b/src/Storages/StorageSet.h @@ -3,6 +3,7 @@ #include #include +#include namespace DB @@ -31,12 +32,12 @@ protected: const StorageID & table_id_, const ColumnsDescription & columns_, const ConstraintsDescription & constraints_, - bool disable_set_and_join_persistency_, - const Context & context_); + const Context & context_, + bool disable_persistency_); String base_path; String path; - bool disable_set_and_join_persistency; + bool disable_persistency; std::atomic increment = 0; /// For the backup file names. @@ -84,8 +85,8 @@ protected: const StorageID & table_id_, const ColumnsDescription & columns_, const ConstraintsDescription & constraints_, - bool disable_set_and_join_persistency_, - const Context & context_); + const Context & context_, + bool disable_persistency_); }; } From 9b98ec2cc0e73e177a44fc46b34c390d72c59a65 Mon Sep 17 00:00:00 2001 From: Vxider Date: Tue, 15 Sep 2020 17:16:48 +0800 Subject: [PATCH 011/321] add functional test --- .../0_stateless/01493_storage_set_persistency.reference | 0 .../queries/0_stateless/01493_storage_set_persistency.sql | 8 ++++++++ .../0_stateless/01494_storage_join_persistency.reference | 0 .../0_stateless/01494_storage_join_persistency.sql | 8 ++++++++ 4 files changed, 16 insertions(+) create mode 100644 tests/queries/0_stateless/01493_storage_set_persistency.reference create mode 100644 tests/queries/0_stateless/01493_storage_set_persistency.sql create mode 100644 tests/queries/0_stateless/01494_storage_join_persistency.reference create mode 100644 tests/queries/0_stateless/01494_storage_join_persistency.sql diff --git a/tests/queries/0_stateless/01493_storage_set_persistency.reference b/tests/queries/0_stateless/01493_storage_set_persistency.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/01493_storage_set_persistency.sql b/tests/queries/0_stateless/01493_storage_set_persistency.sql new file mode 100644 index 00000000000..4fc372f1ab6 --- /dev/null +++ b/tests/queries/0_stateless/01493_storage_set_persistency.sql @@ -0,0 +1,8 @@ +DROP TABLE IF EXISTS set; + +CREATE TABLE set (x String) ENGINE = Set() SETTINGS disable_persistency=1; + +DETACH TABLE set; +ATTACH TABLE set (x String) ENGINE = Set() SETTINGS disable_persistency=1; + +DROP TABLE set; \ No newline at end of file diff --git a/tests/queries/0_stateless/01494_storage_join_persistency.reference b/tests/queries/0_stateless/01494_storage_join_persistency.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/01494_storage_join_persistency.sql b/tests/queries/0_stateless/01494_storage_join_persistency.sql new file mode 100644 index 00000000000..75ca64298ff --- /dev/null +++ b/tests/queries/0_stateless/01494_storage_join_persistency.sql @@ -0,0 +1,8 @@ +DROP TABLE IF EXISTS join; + +CREATE TABLE join (k UInt64, s String) ENGINE = Join(ANY, LEFT, k) SETTINGS disable_persistency=1; + +DETACH TABLE join; +ATTACH TABLE join (k UInt64, s String) ENGINE = Join(ANY, LEFT, k) SETTINGS disable_persistency=1; + +DROP TABLE join; From 023daddbeae6fe44245e7a4da85af8ec5b39bd8e Mon Sep 17 00:00:00 2001 From: hexiaoting Date: Tue, 15 Sep 2020 18:13:39 +0800 Subject: [PATCH 012/321] Fix bug for LineAsString Format --- src/Formats/FormatFactory.cpp | 2 + .../Impl/LineAsStringRowInputFormat.cpp | 42 ++++++++++++++----- .../01460_line_as_string_format.reference | 2 + .../01460_line_as_string_format.sh | 12 ++++++ 4 files changed, 48 insertions(+), 10 deletions(-) diff --git a/src/Formats/FormatFactory.cpp b/src/Formats/FormatFactory.cpp index 522149d3cfd..db1b7f9bb41 100644 --- a/src/Formats/FormatFactory.cpp +++ b/src/Formats/FormatFactory.cpp @@ -331,6 +331,7 @@ void registerFileSegmentationEngineCSV(FormatFactory & factory); void registerFileSegmentationEngineJSONEachRow(FormatFactory & factory); void registerFileSegmentationEngineRegexp(FormatFactory & factory); void registerFileSegmentationEngineJSONAsString(FormatFactory & factory); +void registerFileSegmentationEngineLineAsString(FormatFactory & factory); /// Formats for both input/output. @@ -400,6 +401,7 @@ FormatFactory::FormatFactory() registerFileSegmentationEngineJSONEachRow(*this); registerFileSegmentationEngineRegexp(*this); registerFileSegmentationEngineJSONAsString(*this); + registerFileSegmentationEngineLineAsString(*this); registerInputFormatNative(*this); registerOutputFormatNative(*this); diff --git a/src/Processors/Formats/Impl/LineAsStringRowInputFormat.cpp b/src/Processors/Formats/Impl/LineAsStringRowInputFormat.cpp index befc635386f..8f5eee4bb1b 100644 --- a/src/Processors/Formats/Impl/LineAsStringRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/LineAsStringRowInputFormat.cpp @@ -36,7 +36,7 @@ void LineAsStringRowInputFormat::readLineObject(IColumn & column) while (newline) { - pos = find_first_symbols<'\n', '\\'>(buf.position(), buf.buffer().end()); + pos = find_first_symbols<'\n'>(buf.position(), buf.buffer().end()); buf.position() = pos; if (buf.position() == buf.buffer().end()) { @@ -47,12 +47,6 @@ void LineAsStringRowInputFormat::readLineObject(IColumn & column) { newline = false; } - else if (*buf.position() == '\\') - { - ++buf.position(); - if (!buf.eof()) - ++buf.position(); - } } buf.makeContinuousMemoryFromCheckpointToPos(); @@ -64,10 +58,12 @@ void LineAsStringRowInputFormat::readLineObject(IColumn & column) bool LineAsStringRowInputFormat::readRow(MutableColumns & columns, RowReadExtension &) { - if (!buf.eof()) - readLineObject(*columns[0]); + if (buf.eof()) + return false; - return !buf.eof(); + readLineObject(*columns[0]); + + return true; } void registerInputFormatProcessorLineAsString(FormatFactory & factory) @@ -82,4 +78,30 @@ void registerInputFormatProcessorLineAsString(FormatFactory & factory) }); } +static bool fileSegmentationEngineLineAsStringpImpl(ReadBuffer & in, DB::Memory<> & memory, size_t min_chunk_size) +{ + char * pos = in.position(); + bool need_more_data = true; + + while (loadAtPosition(in, memory, pos) && need_more_data) + { + pos = find_first_symbols<'\n'>(pos, in.buffer().end()); + if (pos == in.buffer().end()) + continue; + + if (memory.size() + static_cast(pos - in.position()) >= min_chunk_size) + need_more_data = false; + + ++pos; + } + + saveUpToPosition(in, memory, pos); + return loadAtPosition(in, memory, pos); +} + +void registerFileSegmentationEngineLineAsString(FormatFactory & factory) +{ + factory.registerFileSegmentationEngine("LineAsString", &fileSegmentationEngineLineAsStringpImpl); +} + } diff --git a/tests/queries/0_stateless/01460_line_as_string_format.reference b/tests/queries/0_stateless/01460_line_as_string_format.reference index c795fba4309..c72d3e1f15c 100644 --- a/tests/queries/0_stateless/01460_line_as_string_format.reference +++ b/tests/queries/0_stateless/01460_line_as_string_format.reference @@ -6,3 +6,5 @@ Finally implement this new feature. 42 ClickHouse 42 ClickHouse is a `fast` #open-source# (OLAP) database "management" :system: +1000000 +1000 diff --git a/tests/queries/0_stateless/01460_line_as_string_format.sh b/tests/queries/0_stateless/01460_line_as_string_format.sh index 60a960d2bf8..4f94111df08 100755 --- a/tests/queries/0_stateless/01460_line_as_string_format.sh +++ b/tests/queries/0_stateless/01460_line_as_string_format.sh @@ -29,3 +29,15 @@ echo 'ClickHouse is a `fast` #open-source# (OLAP) 'database' "management" :syste $CLICKHOUSE_CLIENT --query="SELECT * FROM line_as_string2 order by c"; $CLICKHOUSE_CLIENT --query="DROP TABLE line_as_string2" + +$CLICKHOUSE_CLIENT --query="select repeat('aaa',50) from numbers(1000000)" > "${CLICKHOUSE_TMP}"/data1 +$CLICKHOUSE_CLIENT --query="CREATE TABLE line_as_string3(field String) ENGINE = Memory"; +$CLICKHOUSE_CLIENT --query="INSERT INTO line_as_string3 FORMAT LineAsString" < "${CLICKHOUSE_TMP}"/data1 +$CLICKHOUSE_CLIENT --query="SELECT count(*) FROM line_as_string3"; +$CLICKHOUSE_CLIENT --query="DROP TABLE line_as_string3" + +$CLICKHOUSE_CLIENT --query="select randomString(50000) FROM numbers(1000)" > "${CLICKHOUSE_TMP}"/data2 +$CLICKHOUSE_CLIENT --query="CREATE TABLE line_as_string4(field String) ENGINE = Memory"; +$CLICKHOUSE_CLIENT --query="INSERT INTO line_as_string4 FORMAT LineAsString" < "${CLICKHOUSE_TMP}"/data2 +$CLICKHOUSE_CLIENT --query="SELECT count(*) FROM line_as_string4"; +$CLICKHOUSE_CLIENT --query="DROP TABLE line_as_string4" From e136eba3f8333a9e4c17131678ebbc0a28f87350 Mon Sep 17 00:00:00 2001 From: Vxider Date: Tue, 15 Sep 2020 18:58:16 +0800 Subject: [PATCH 013/321] fix style check --- src/Storages/StorageSet.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/src/Storages/StorageSet.cpp b/src/Storages/StorageSet.cpp index 2f17fe28242..3f011408b0d 100644 --- a/src/Storages/StorageSet.cpp +++ b/src/Storages/StorageSet.cpp @@ -21,7 +21,6 @@ namespace DB namespace ErrorCodes { - extern const int BAD_ARGUMENTS; extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; } From 209b41aaab159715da1373ec398ecebbd5b7c534 Mon Sep 17 00:00:00 2001 From: Niu Zhaojie Date: Tue, 15 Sep 2020 21:42:37 +0800 Subject: [PATCH 014/321] Use opt num to identify request type --- src/Common/ZooKeeper/ZooKeeperImpl.cpp | 12 +++++++----- src/Common/ZooKeeper/ZooKeeperImpl.h | 1 - 2 files changed, 7 insertions(+), 6 deletions(-) diff --git a/src/Common/ZooKeeper/ZooKeeperImpl.cpp b/src/Common/ZooKeeper/ZooKeeperImpl.cpp index 7bc29869518..39a77fea0cf 100644 --- a/src/Common/ZooKeeper/ZooKeeperImpl.cpp +++ b/src/Common/ZooKeeper/ZooKeeperImpl.cpp @@ -1276,11 +1276,16 @@ void ZooKeeper::receiveEvent() response->removeRootPath(root_path); } - /// Find from operations + /// Instead of setting the watch in sendEvent, set it in receiveEvent becuase need to check the response. + /// The watch shouldn't be set if the node does not exist and it will never exist like sequential ephemeral nodes. + /// By using getData() instead of exists(), a watch won't be set if the node doesn't exist. if (request_info.watch) { bool add_watch = false; - if (request_info.isExists) + /// 3 indicates the ZooKeeperExistsRequest. + // For exists, we set the watch on both node exist and nonexist case. + // For other case like getData, we only set the watch when node exists. + if (request_info.request->getOpNum() == 3) add_watch = (response->error == Error::ZOK || response->error == Error::ZNONODE); else add_watch = response->error == Error::ZOK; @@ -1559,7 +1564,6 @@ void ZooKeeper::exists( request_info.request = std::make_shared(std::move(request)); request_info.callback = [callback](const Response & response) { callback(dynamic_cast(response)); }; request_info.watch = watch; - request_info.isExists = true; pushRequest(std::move(request_info)); ProfileEvents::increment(ProfileEvents::ZooKeeperExists); @@ -1578,7 +1582,6 @@ void ZooKeeper::get( request_info.request = std::make_shared(std::move(request)); request_info.callback = [callback](const Response & response) { callback(dynamic_cast(response)); }; request_info.watch = watch; - request_info.isExists = false; pushRequest(std::move(request_info)); ProfileEvents::increment(ProfileEvents::ZooKeeperGet); @@ -1617,7 +1620,6 @@ void ZooKeeper::list( request_info.request = std::make_shared(std::move(request)); request_info.callback = [callback](const Response & response) { callback(dynamic_cast(response)); }; request_info.watch = watch; - request_info.isExists = false; pushRequest(std::move(request_info)); ProfileEvents::increment(ProfileEvents::ZooKeeperList); diff --git a/src/Common/ZooKeeper/ZooKeeperImpl.h b/src/Common/ZooKeeper/ZooKeeperImpl.h index 8dd0413d83d..840cbdbde3f 100644 --- a/src/Common/ZooKeeper/ZooKeeperImpl.h +++ b/src/Common/ZooKeeper/ZooKeeperImpl.h @@ -197,7 +197,6 @@ private: ResponseCallback callback; WatchCallback watch; clock::time_point time; - bool isExists; }; using RequestsQueue = ConcurrentBoundedQueue; From 5da83fb14d39fa1f06c9fbe6ae4126cff837fd58 Mon Sep 17 00:00:00 2001 From: Ariel Robaldo <31019067+arielrobaldo@users.noreply.github.com> Date: Wed, 16 Sep 2020 20:21:01 -0300 Subject: [PATCH 015/321] Update index.md incomplete yet, only performed first 40 lines, hope to continue soon --- docs/es/index.md | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/docs/es/index.md b/docs/es/index.md index 9a2918ff7c8..45c41a2c7f9 100644 --- a/docs/es/index.md +++ b/docs/es/index.md @@ -1,15 +1,15 @@ --- -machine_translated: true -machine_translated_rev: 72537a2d527c63c07aa5d2361a8829f3895cf2bd +machine_translated: false +machine_translated_rev: toc_priority: 0 -toc_title: "Descripci\xF3n" +toc_title: "Descripción" --- # ¿Qué es ClickHouse? {#what-is-clickhouse} -ClickHouse es un sistema de gestión de bases de datos orientado a columnas (DBMS) para el procesamiento analítico en línea de consultas (OLAP). +ClickHouse es un sistema de gestión de bases de datos (DBMS), orientado a columnas, para el procesamiento analítico de consultas en línea (OLAP). -En un “normal” DBMS orientado a filas, los datos se almacenan en este orden: +En un DBMS “normal”, orientado a filas, los datos se almacenan en este orden: | Fila | Argumento | JavaEnable | Titular | GoodEvent | EventTime | |------|-------------|------------|---------------------------|-----------|---------------------| @@ -36,7 +36,7 @@ Estos ejemplos solo muestran el orden en el que se organizan los datos. Los valo Ejemplos de un DBMS orientado a columnas: Vertica, Paraccel (Actian Matrix y Amazon Redshift), Sybase IQ, Exasol, Infobright, InfiniDB, MonetDB (VectorWise y Actian Vector), LucidDB, SAP HANA, Google Dremel, Google PowerDrill, Druid y kdb+. -Different orders for storing data are better suited to different scenarios. The data access scenario refers to what queries are made, how often, and in what proportion; how much data is read for each type of query – rows, columns, and bytes; the relationship between reading and updating data; the working size of the data and how locally it is used; whether transactions are used, and how isolated they are; requirements for data replication and logical integrity; requirements for latency and throughput for each type of query, and so on. +Los diferentes modos de ordenar los datos al guardarlos se adecúan mejor a diferentes escenarios. El escenario de acceso a los datos se refiere a qué consultas se hacen, con qué frecuencia y en qué proporción; cuántos datos se leen para cada tipo de consulta - filas, columnas y bytes; la relación entre lectura y actualización de datos; el tamaño de trabajo de los datos y qué tan localmente son usados; si se usan transacciones y qué tan aisladas están;requerimientos de replicación de los datos y de integridad lógica, requerimientos de latencia y caudal (throughput) para cada tipo de consulta, y cosas por el estilo. Cuanto mayor sea la carga en el sistema, más importante es personalizar el sistema configurado para que coincida con los requisitos del escenario de uso, y más fino será esta personalización. No existe un sistema que sea igualmente adecuado para escenarios significativamente diferentes. Si un sistema es adaptable a un amplio conjunto de escenarios, bajo una carga alta, el sistema manejará todos los escenarios igualmente mal, o funcionará bien para solo uno o algunos de los escenarios posibles. From 9ceb4934c2826f94e620c826e7bb8cbd29cf1e57 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 17 Sep 2020 13:32:16 +0300 Subject: [PATCH 016/321] Use clang-tidy 11 --- cmake/analysis.cmake | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/cmake/analysis.cmake b/cmake/analysis.cmake index daaa730ac4b..a90fba7c70b 100644 --- a/cmake/analysis.cmake +++ b/cmake/analysis.cmake @@ -6,7 +6,7 @@ if (ENABLE_CLANG_TIDY) message(FATAL_ERROR "clang-tidy requires CMake version at least 3.6.") endif() - find_program (CLANG_TIDY_PATH NAMES "clang-tidy" "clang-tidy-10" "clang-tidy-9" "clang-tidy-8") + find_program (CLANG_TIDY_PATH NAMES "clang-tidy" "clang-tidy-11" "clang-tidy-10" "clang-tidy-9" "clang-tidy-8") if (CLANG_TIDY_PATH) message(STATUS "Using clang-tidy: ${CLANG_TIDY_PATH}. The checks will be run during build process. See the .clang-tidy file at the root directory to configure the checks.") set (USE_CLANG_TIDY 1) From 1322a1ce7a8883950ee631017982baefced80474 Mon Sep 17 00:00:00 2001 From: Vxider Date: Thu, 17 Sep 2020 21:53:39 +0800 Subject: [PATCH 017/321] add blank line to end of file --- tests/queries/0_stateless/01493_storage_set_persistency.sql | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/01493_storage_set_persistency.sql b/tests/queries/0_stateless/01493_storage_set_persistency.sql index 4fc372f1ab6..d70a372fbc7 100644 --- a/tests/queries/0_stateless/01493_storage_set_persistency.sql +++ b/tests/queries/0_stateless/01493_storage_set_persistency.sql @@ -5,4 +5,4 @@ CREATE TABLE set (x String) ENGINE = Set() SETTINGS disable_persistency=1; DETACH TABLE set; ATTACH TABLE set (x String) ENGINE = Set() SETTINGS disable_persistency=1; -DROP TABLE set; \ No newline at end of file +DROP TABLE set; From 8cf4f0fdb370d36e1953f12383c3d4822519706e Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 17 Sep 2020 17:44:46 +0300 Subject: [PATCH 018/321] Fix clang-tidy --- base/common/coverage.cpp | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/base/common/coverage.cpp b/base/common/coverage.cpp index 9f3c5ca653a..043f97f9593 100644 --- a/base/common/coverage.cpp +++ b/base/common/coverage.cpp @@ -3,12 +3,11 @@ #if WITH_COVERAGE # include - # include # if defined(__clang__) -extern "C" void __llvm_profile_dump(); +extern "C" void __llvm_profile_dump(); // NOLINT # elif defined(__GNUC__) || defined(__GNUG__) extern "C" void __gcov_exit(); # endif @@ -23,7 +22,7 @@ void dumpCoverageReportIfPossible() std::lock_guard lock(mutex); # if defined(__clang__) - __llvm_profile_dump(); + __llvm_profile_dump(); // NOLINT # elif defined(__GNUC__) || defined(__GNUG__) __gcov_exit(); # endif From 7152fec2418a59e9b397cf4e74d2c5d18d8aba24 Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Thu, 17 Sep 2020 21:24:14 +0300 Subject: [PATCH 019/321] Update JoinSettings.cpp --- src/Storages/JoinSettings.cpp | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/src/Storages/JoinSettings.cpp b/src/Storages/JoinSettings.cpp index 0d060cdcf41..15637d67dea 100644 --- a/src/Storages/JoinSettings.cpp +++ b/src/Storages/JoinSettings.cpp @@ -12,7 +12,9 @@ namespace ErrorCodes { extern const int BAD_ARGUMENTS; extern const int UNKNOWN_SETTING; -} IMPLEMENT_SETTINGS_TRAITS(joinSettingsTraits, LIST_OF_JOIN_SETTINGS) +} + +IMPLEMENT_SETTINGS_TRAITS(joinSettingsTraits, LIST_OF_JOIN_SETTINGS) void JoinSettings::loadFromQuery(ASTStorage & storage_def) { From e96a3ac5f3072de60eab3b8b86f5c2cea5245826 Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 18 Sep 2020 14:37:58 +0300 Subject: [PATCH 020/321] DataType in enumerate streams --- src/Compression/CompressionCodecMultiple.cpp | 14 ++++++++++++++ src/Compression/CompressionCodecMultiple.h | 2 ++ src/Compression/ICompressionCodec.cpp | 12 ++++++++++++ src/Compression/ICompressionCodec.h | 2 ++ src/DataTypes/DataTypeArray.cpp | 2 +- src/DataTypes/DataTypeLowCardinality.cpp | 2 +- src/DataTypes/DataTypeNullable.cpp | 2 +- src/DataTypes/IDataType.h | 7 ++++--- .../MergeTree/IMergedBlockOutputStream.cpp | 4 ++-- .../MergeTree/MergeTreeDataMergerMutator.cpp | 8 ++++---- src/Storages/MergeTree/MergeTreeDataPartWide.cpp | 10 +++++----- .../MergeTree/MergeTreeDataPartWriterWide.cpp | 10 +++++----- src/Storages/MergeTree/MergeTreeReaderWide.cpp | 2 +- src/Storages/MergeTree/checkDataPart.cpp | 2 +- src/Storages/StorageLog.cpp | 10 +++++----- src/Storages/StorageTinyLog.cpp | 2 +- 16 files changed, 61 insertions(+), 30 deletions(-) diff --git a/src/Compression/CompressionCodecMultiple.cpp b/src/Compression/CompressionCodecMultiple.cpp index a0336d66a05..c9300288c0f 100644 --- a/src/Compression/CompressionCodecMultiple.cpp +++ b/src/Compression/CompressionCodecMultiple.cpp @@ -30,6 +30,20 @@ CompressionCodecMultiple::CompressionCodecMultiple(Codecs codecs_) setCodecDescription("", arguments); } + +CompressionCodecPtr CompressionCodecMultiple::filterNonGeneralCompressionCodecs(const CompressionCodecMultiple * codec) +{ + Codecs filtered; + for (const auto & subcodec : codec->codecs) + if (!subcodec->isGenericCompression()) + filtered.push_back(subcodec); + + if (filtered.empty()) + return nullptr; + + return std::make_shared(filtered); +} + uint8_t CompressionCodecMultiple::getMethodByte() const { return static_cast(CompressionMethodByte::Multiple); diff --git a/src/Compression/CompressionCodecMultiple.h b/src/Compression/CompressionCodecMultiple.h index 1eb61842048..c6f9f3b9d5b 100644 --- a/src/Compression/CompressionCodecMultiple.h +++ b/src/Compression/CompressionCodecMultiple.h @@ -17,6 +17,8 @@ public: static std::vector getCodecsBytesFromData(const char * source); + static CompressionCodecPtr filterNonGeneralCompressionCodecs(const CompressionCodecMultiple * codec); + void updateHash(SipHash & hash) const override; protected: diff --git a/src/Compression/ICompressionCodec.cpp b/src/Compression/ICompressionCodec.cpp index 1b2c90e5163..9a501f5099d 100644 --- a/src/Compression/ICompressionCodec.cpp +++ b/src/Compression/ICompressionCodec.cpp @@ -7,6 +7,7 @@ #include #include #include +#include namespace DB @@ -128,4 +129,15 @@ uint8_t ICompressionCodec::readMethod(const char * source) return static_cast(source[0]); } + +CompressionCodecPtr tryGetGeneralCompressionCodecs(const CompressionCodecPtr & codec) +{ + if (codec->getMethodByte() == static_cast(CompressionMethodByte::Multiple)) + return CompressionCodecMultiple::filterNonGeneralCompressionCodecs(dynamic_cast(codec.get())); + else if (!codec->isGenericCompression()) + return nullptr; + else + return codec; +} + } diff --git a/src/Compression/ICompressionCodec.h b/src/Compression/ICompressionCodec.h index fa143af8b9c..f3371d6d4b9 100644 --- a/src/Compression/ICompressionCodec.h +++ b/src/Compression/ICompressionCodec.h @@ -95,4 +95,6 @@ private: ASTPtr full_codec_desc; }; +CompressionCodecPtr tryGetGeneralCompressionCodecs(const CompressionCodecPtr & codec); + } diff --git a/src/DataTypes/DataTypeArray.cpp b/src/DataTypes/DataTypeArray.cpp index 49666cca428..ed570beeda6 100644 --- a/src/DataTypes/DataTypeArray.cpp +++ b/src/DataTypes/DataTypeArray.cpp @@ -151,7 +151,7 @@ namespace void DataTypeArray::enumerateStreams(const StreamCallback & callback, SubstreamPath & path) const { path.push_back(Substream::ArraySizes); - callback(path); + callback(path, *this); path.back() = Substream::ArrayElements; nested->enumerateStreams(callback, path); path.pop_back(); diff --git a/src/DataTypes/DataTypeLowCardinality.cpp b/src/DataTypes/DataTypeLowCardinality.cpp index b4f28a8853f..06bd7c61028 100644 --- a/src/DataTypes/DataTypeLowCardinality.cpp +++ b/src/DataTypes/DataTypeLowCardinality.cpp @@ -54,7 +54,7 @@ void DataTypeLowCardinality::enumerateStreams(const StreamCallback & callback, S path.push_back(Substream::DictionaryKeys); dictionary_type->enumerateStreams(callback, path); path.back() = Substream::DictionaryIndexes; - callback(path); + callback(path, *this); path.pop_back(); } diff --git a/src/DataTypes/DataTypeNullable.cpp b/src/DataTypes/DataTypeNullable.cpp index 9c738da9f6a..ed501939901 100644 --- a/src/DataTypes/DataTypeNullable.cpp +++ b/src/DataTypes/DataTypeNullable.cpp @@ -44,7 +44,7 @@ bool DataTypeNullable::onlyNull() const void DataTypeNullable::enumerateStreams(const StreamCallback & callback, SubstreamPath & path) const { path.push_back(Substream::NullMap); - callback(path); + callback(path, *this); path.back() = Substream::NullableElements; nested_data_type->enumerateStreams(callback, path); path.pop_back(); diff --git a/src/DataTypes/IDataType.h b/src/DataTypes/IDataType.h index 6adcc0fda90..66485753060 100644 --- a/src/DataTypes/IDataType.h +++ b/src/DataTypes/IDataType.h @@ -99,15 +99,17 @@ public: /// Index of tuple element, starting at 1. String tuple_element_name; + bool is_specialized_codecs_allowed = true; + Substream(Type type_) : type(type_) {} }; using SubstreamPath = std::vector; - using StreamCallback = std::function; + using StreamCallback = std::function; virtual void enumerateStreams(const StreamCallback & callback, SubstreamPath & path) const { - callback(path); + callback(path, *this); } void enumerateStreams(const StreamCallback & callback, SubstreamPath && path) const { enumerateStreams(callback, path); } void enumerateStreams(const StreamCallback & callback) const { enumerateStreams(callback, {}); } @@ -685,4 +687,3 @@ template <> inline constexpr bool IsDataTypeDateOrDateTime = t template <> inline constexpr bool IsDataTypeDateOrDateTime = true; } - diff --git a/src/Storages/MergeTree/IMergedBlockOutputStream.cpp b/src/Storages/MergeTree/IMergedBlockOutputStream.cpp index 4e98caf066a..e5cc6d6ecbf 100644 --- a/src/Storages/MergeTree/IMergedBlockOutputStream.cpp +++ b/src/Storages/MergeTree/IMergedBlockOutputStream.cpp @@ -51,7 +51,7 @@ NameSet IMergedBlockOutputStream::removeEmptyColumnsFromPart( for (const NameAndTypePair & column : columns) { column.type->enumerateStreams( - [&](const IDataType::SubstreamPath & substream_path) + [&](const IDataType::SubstreamPath & substream_path, const IDataType & /* substream_path */) { ++stream_counts[IDataType::getFileNameForStream(column.name, substream_path)]; }, @@ -62,7 +62,7 @@ NameSet IMergedBlockOutputStream::removeEmptyColumnsFromPart( const String mrk_extension = data_part->getMarksFileExtension(); for (const auto & column_name : empty_columns) { - IDataType::StreamCallback callback = [&](const IDataType::SubstreamPath & substream_path) + IDataType::StreamCallback callback = [&](const IDataType::SubstreamPath & substream_path, const IDataType & /* substream_path */) { String stream_name = IDataType::getFileNameForStream(column_name, substream_path); /// Delete files if they are no longer shared with another column. diff --git a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp index 89631b713ed..5afd660fb4f 100644 --- a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp +++ b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp @@ -1449,7 +1449,7 @@ NameToNameVector MergeTreeDataMergerMutator::collectFilesForRenames( for (const NameAndTypePair & column : source_part->getColumns()) { column.type->enumerateStreams( - [&](const IDataType::SubstreamPath & substream_path) + [&](const IDataType::SubstreamPath & substream_path, const IDataType & /* substream_type */) { ++stream_counts[IDataType::getFileNameForStream(column.name, substream_path)]; }, @@ -1467,7 +1467,7 @@ NameToNameVector MergeTreeDataMergerMutator::collectFilesForRenames( } else if (command.type == MutationCommand::Type::DROP_COLUMN) { - IDataType::StreamCallback callback = [&](const IDataType::SubstreamPath & substream_path) + IDataType::StreamCallback callback = [&](const IDataType::SubstreamPath & substream_path, const IDataType & /* substream_type */) { String stream_name = IDataType::getFileNameForStream(command.column_name, substream_path); /// Delete files if they are no longer shared with another column. @@ -1488,7 +1488,7 @@ NameToNameVector MergeTreeDataMergerMutator::collectFilesForRenames( String escaped_name_from = escapeForFileName(command.column_name); String escaped_name_to = escapeForFileName(command.rename_to); - IDataType::StreamCallback callback = [&](const IDataType::SubstreamPath & substream_path) + IDataType::StreamCallback callback = [&](const IDataType::SubstreamPath & substream_path, const IDataType & /* substream_type */) { String stream_from = IDataType::getFileNameForStream(command.column_name, substream_path); @@ -1521,7 +1521,7 @@ NameSet MergeTreeDataMergerMutator::collectFilesToSkip( /// Skip updated files for (const auto & entry : updated_header) { - IDataType::StreamCallback callback = [&](const IDataType::SubstreamPath & substream_path) + IDataType::StreamCallback callback = [&](const IDataType::SubstreamPath & substream_path, const IDataType & /* substream_type */) { String stream_name = IDataType::getFileNameForStream(entry.name, substream_path); files_to_skip.insert(stream_name + ".bin"); diff --git a/src/Storages/MergeTree/MergeTreeDataPartWide.cpp b/src/Storages/MergeTree/MergeTreeDataPartWide.cpp index c53362c847d..39898a7cc57 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWide.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartWide.cpp @@ -77,7 +77,7 @@ ColumnSize MergeTreeDataPartWide::getColumnSizeImpl( if (checksums.empty()) return size; - type.enumerateStreams([&](const IDataType::SubstreamPath & substream_path) + type.enumerateStreams([&](const IDataType::SubstreamPath & substream_path, const IDataType & /* substream_type */) { String file_name = IDataType::getFileNameForStream(column_name, substream_path); @@ -155,7 +155,7 @@ void MergeTreeDataPartWide::checkConsistency(bool require_part_metadata) const for (const NameAndTypePair & name_type : columns) { IDataType::SubstreamPath stream_path; - name_type.type->enumerateStreams([&](const IDataType::SubstreamPath & substream_path) + name_type.type->enumerateStreams([&](const IDataType::SubstreamPath & substream_path, const IDataType & /* substream_type */) { String file_name = IDataType::getFileNameForStream(name_type.name, substream_path); String mrk_file_name = file_name + index_granularity_info.marks_file_extension; @@ -177,7 +177,7 @@ void MergeTreeDataPartWide::checkConsistency(bool require_part_metadata) const std::optional marks_size; for (const NameAndTypePair & name_type : columns) { - name_type.type->enumerateStreams([&](const IDataType::SubstreamPath & substream_path) + name_type.type->enumerateStreams([&](const IDataType::SubstreamPath & substream_path, const IDataType & /* substream_type */) { auto file_path = path + IDataType::getFileNameForStream(name_type.name, substream_path) + index_granularity_info.marks_file_extension; @@ -205,7 +205,7 @@ bool MergeTreeDataPartWide::hasColumnFiles(const String & column_name, const IDa { bool res = true; - type.enumerateStreams([&](const IDataType::SubstreamPath & substream_path) + type.enumerateStreams([&](const IDataType::SubstreamPath & substream_path, const IDataType & /* substream_type */) { String file_name = IDataType::getFileNameForStream(column_name, substream_path); @@ -222,7 +222,7 @@ bool MergeTreeDataPartWide::hasColumnFiles(const String & column_name, const IDa String MergeTreeDataPartWide::getFileNameForColumn(const NameAndTypePair & column) const { String filename; - column.type->enumerateStreams([&](const IDataType::SubstreamPath & substream_path) + column.type->enumerateStreams([&](const IDataType::SubstreamPath & substream_path, const IDataType & /* substream_type */) { if (filename.empty()) filename = IDataType::getFileNameForStream(column.name, substream_path); diff --git a/src/Storages/MergeTree/MergeTreeDataPartWriterWide.cpp b/src/Storages/MergeTree/MergeTreeDataPartWriterWide.cpp index a558c57b5f0..6515774d337 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWriterWide.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartWriterWide.cpp @@ -37,7 +37,7 @@ void MergeTreeDataPartWriterWide::addStreams( const CompressionCodecPtr & effective_codec, size_t estimated_size) { - IDataType::StreamCallback callback = [&] (const IDataType::SubstreamPath & substream_path) + IDataType::StreamCallback callback = [&] (const IDataType::SubstreamPath & substream_path, const IDataType & /* substream_type */) { String stream_name = IDataType::getFileNameForStream(name, substream_path); /// Shared offsets for Nested type. @@ -130,7 +130,7 @@ void MergeTreeDataPartWriterWide::writeSingleMark( size_t number_of_rows, DB::IDataType::SubstreamPath & path) { - type.enumerateStreams([&] (const IDataType::SubstreamPath & substream_path) + type.enumerateStreams([&] (const IDataType::SubstreamPath & substream_path, const IDataType & /* substream_type */) { bool is_offsets = !substream_path.empty() && substream_path.back().type == IDataType::Substream::ArraySizes; @@ -170,7 +170,7 @@ size_t MergeTreeDataPartWriterWide::writeSingleGranule( type.serializeBinaryBulkWithMultipleStreams(column, from_row, number_of_rows, serialize_settings, serialization_state); /// So that instead of the marks pointing to the end of the compressed block, there were marks pointing to the beginning of the next one. - type.enumerateStreams([&] (const IDataType::SubstreamPath & substream_path) + type.enumerateStreams([&] (const IDataType::SubstreamPath & substream_path, const IDataType & /* substream_type */) { bool is_offsets = !substream_path.empty() && substream_path.back().type == IDataType::Substream::ArraySizes; @@ -251,7 +251,7 @@ void MergeTreeDataPartWriterWide::writeColumn( current_column_mark++; } - type.enumerateStreams([&] (const IDataType::SubstreamPath & substream_path) + type.enumerateStreams([&] (const IDataType::SubstreamPath & substream_path, const IDataType & /* substream_type */) { bool is_offsets = !substream_path.empty() && substream_path.back().type == IDataType::Substream::ArraySizes; if (is_offsets) @@ -312,7 +312,7 @@ void MergeTreeDataPartWriterWide::writeFinalMark( { writeSingleMark(column_name, *column_type, offset_columns, 0, path); /// Memoize information about offsets - column_type->enumerateStreams([&] (const IDataType::SubstreamPath & substream_path) + column_type->enumerateStreams([&] (const IDataType::SubstreamPath & substream_path, const IDataType & /* substream_type */) { bool is_offsets = !substream_path.empty() && substream_path.back().type == IDataType::Substream::ArraySizes; if (is_offsets) diff --git a/src/Storages/MergeTree/MergeTreeReaderWide.cpp b/src/Storages/MergeTree/MergeTreeReaderWide.cpp index 352d1f93589..1dacdacbae0 100644 --- a/src/Storages/MergeTree/MergeTreeReaderWide.cpp +++ b/src/Storages/MergeTree/MergeTreeReaderWide.cpp @@ -162,7 +162,7 @@ size_t MergeTreeReaderWide::readRows(size_t from_mark, bool continue_reading, si void MergeTreeReaderWide::addStreams(const String & name, const IDataType & type, const ReadBufferFromFileBase::ProfileCallback & profile_callback, clockid_t clock_type) { - IDataType::StreamCallback callback = [&] (const IDataType::SubstreamPath & substream_path) + IDataType::StreamCallback callback = [&] (const IDataType::SubstreamPath & substream_path, const IDataType & /* substream_type */) { String stream_name = IDataType::getFileNameForStream(name, substream_path); diff --git a/src/Storages/MergeTree/checkDataPart.cpp b/src/Storages/MergeTree/checkDataPart.cpp index 63b061b9702..2838c8eb881 100644 --- a/src/Storages/MergeTree/checkDataPart.cpp +++ b/src/Storages/MergeTree/checkDataPart.cpp @@ -120,7 +120,7 @@ IMergeTreeDataPart::Checksums checkDataPart( { for (const auto & column : columns_list) { - column.type->enumerateStreams([&](const IDataType::SubstreamPath & substream_path) + column.type->enumerateStreams([&](const IDataType::SubstreamPath & substream_path, const IDataType & /* substream_type */) { String file_name = IDataType::getFileNameForStream(column.name, substream_path) + ".bin"; checksums_data.files[file_name] = checksum_compressed_file(disk, path + file_name); diff --git a/src/Storages/StorageLog.cpp b/src/Storages/StorageLog.cpp index 45ab3293723..8258db4352e 100644 --- a/src/Storages/StorageLog.cpp +++ b/src/Storages/StorageLog.cpp @@ -360,7 +360,7 @@ void LogBlockOutputStream::writeData(const String & name, const IDataType & type { IDataType::SerializeBinaryBulkSettings settings; - type.enumerateStreams([&] (const IDataType::SubstreamPath & path) + type.enumerateStreams([&] (const IDataType::SubstreamPath & path, const IDataType & /* substream_type */) { String stream_name = IDataType::getFileNameForStream(name, path); if (written_streams.count(stream_name)) @@ -380,7 +380,7 @@ void LogBlockOutputStream::writeData(const String & name, const IDataType & type if (serialize_states.count(name) == 0) type.serializeBinaryBulkStatePrefix(settings, serialize_states[name]); - type.enumerateStreams([&] (const IDataType::SubstreamPath & path) + type.enumerateStreams([&] (const IDataType::SubstreamPath & path, const IDataType & /* substream_type */) { String stream_name = IDataType::getFileNameForStream(name, path); if (written_streams.count(stream_name)) @@ -398,7 +398,7 @@ void LogBlockOutputStream::writeData(const String & name, const IDataType & type type.serializeBinaryBulkWithMultipleStreams(column, 0, 0, settings, serialize_states[name]); - type.enumerateStreams([&] (const IDataType::SubstreamPath & path) + type.enumerateStreams([&] (const IDataType::SubstreamPath & path, const IDataType & /* substream_type */) { String stream_name = IDataType::getFileNameForStream(name, path); if (!written_streams.emplace(stream_name).second) @@ -485,7 +485,7 @@ void StorageLog::addFiles(const String & column_name, const IDataType & type) throw Exception("Duplicate column with name " + column_name + " in constructor of StorageLog.", ErrorCodes::DUPLICATE_COLUMN); - IDataType::StreamCallback stream_callback = [&] (const IDataType::SubstreamPath & substream_path) + IDataType::StreamCallback stream_callback = [&] (const IDataType::SubstreamPath & substream_path, const IDataType & /* substream_type */) { String stream_name = IDataType::getFileNameForStream(column_name, substream_path); @@ -592,7 +592,7 @@ const StorageLog::Marks & StorageLog::getMarksWithRealRowCount(const StorageMeta * (Example: for Array data type, first stream is array sizes; and number of array sizes is the number of arrays). */ IDataType::SubstreamPath substream_root_path; - column_type->enumerateStreams([&](const IDataType::SubstreamPath & substream_path) + column_type->enumerateStreams([&](const IDataType::SubstreamPath & substream_path, const IDataType & /* substream_type */) { if (filename.empty()) filename = IDataType::getFileNameForStream(column_name, substream_path); diff --git a/src/Storages/StorageTinyLog.cpp b/src/Storages/StorageTinyLog.cpp index e7fe7e2d5f9..75ad43eef3f 100644 --- a/src/Storages/StorageTinyLog.cpp +++ b/src/Storages/StorageTinyLog.cpp @@ -389,7 +389,7 @@ void StorageTinyLog::addFiles(const String & column_name, const IDataType & type throw Exception("Duplicate column with name " + column_name + " in constructor of StorageTinyLog.", ErrorCodes::DUPLICATE_COLUMN); - IDataType::StreamCallback stream_callback = [&] (const IDataType::SubstreamPath & substream_path) + IDataType::StreamCallback stream_callback = [&] (const IDataType::SubstreamPath & substream_path, const IDataType & /* substream_type */) { String stream_name = IDataType::getFileNameForStream(column_name, substream_path); if (!files.count(stream_name)) From 0c535062fce8027ecb98272904a6cb899c51e763 Mon Sep 17 00:00:00 2001 From: Vxider Date: Fri, 18 Sep 2020 20:48:55 +0800 Subject: [PATCH 021/321] rename disable_persistency to persistency --- src/Storages/JoinSettings.h | 2 +- src/Storages/SetSettings.h | 2 +- src/Storages/StorageJoin.cpp | 12 ++++---- src/Storages/StorageJoin.h | 2 +- src/Storages/StorageSet.cpp | 24 +++++++-------- src/Storages/StorageSet.h | 6 ++-- .../01493_storage_set_persistency.reference | 5 ++++ .../01493_storage_set_persistency.sql | 29 +++++++++++++++++-- .../01494_storage_join_persistency.reference | 5 ++++ .../01494_storage_join_persistency.sql | 26 +++++++++++++++-- 10 files changed, 84 insertions(+), 29 deletions(-) diff --git a/src/Storages/JoinSettings.h b/src/Storages/JoinSettings.h index 1758250c737..396185d9f77 100644 --- a/src/Storages/JoinSettings.h +++ b/src/Storages/JoinSettings.h @@ -10,7 +10,7 @@ class ASTStorage; #define JOIN_RELATED_SETTINGS(M) \ - M(Bool, disable_persistency, false, "Disable persistency for StorageJoin to reduce IO overhead", 0) + M(Bool, persistency, true, "Disable setting to avoid the overhead of writing to disk for StorageJoin", 0) #define LIST_OF_JOIN_SETTINGS(M) \ JOIN_RELATED_SETTINGS(M) \ diff --git a/src/Storages/SetSettings.h b/src/Storages/SetSettings.h index 1bd3e3895ef..2e55e6e8ab5 100644 --- a/src/Storages/SetSettings.h +++ b/src/Storages/SetSettings.h @@ -10,7 +10,7 @@ class ASTStorage; #define SET_RELATED_SETTINGS(M) \ - M(Bool, disable_persistency, false, "Disable persistency for StorageSet to reduce IO overhead", 0) + M(Bool, persistency, true, "Disable setting to avoid the overhead of writing to disk for StorageSet", 0) #define LIST_OF_SET_SETTINGS(M) \ SET_RELATED_SETTINGS(M) \ diff --git a/src/Storages/StorageJoin.cpp b/src/Storages/StorageJoin.cpp index 6ca6411a323..14637856439 100644 --- a/src/Storages/StorageJoin.cpp +++ b/src/Storages/StorageJoin.cpp @@ -45,8 +45,8 @@ StorageJoin::StorageJoin( const ConstraintsDescription & constraints_, bool overwrite_, const Context & context_, - bool disable_persistency_) - : StorageSetOrJoinBase{relative_path_, table_id_, columns_, constraints_, context_, disable_persistency_} + bool persistency_) + : StorageSetOrJoinBase{relative_path_, table_id_, columns_, constraints_, context_, persistency_} , key_names(key_names_) , use_nulls(use_nulls_) , limits(limits_) @@ -119,7 +119,7 @@ void registerStorageJoin(StorageFactory & factory) auto join_overflow_mode = settings.join_overflow_mode; auto join_any_take_last_row = settings.join_any_take_last_row; auto old_any_join = settings.any_join_distinct_right_table_keys; - bool disable_persistency = false; + bool persistency_ = true; if (args.storage_def && args.storage_def->settings) { @@ -137,11 +137,11 @@ void registerStorageJoin(StorageFactory & factory) join_any_take_last_row = setting.value; else if (setting.name == "any_join_distinct_right_table_keys") old_any_join = setting.value; - else if (setting.name == "disable_persistency") + else if (setting.name == "persistency") { auto join_settings = std::make_unique(); join_settings->loadFromQuery(*args.storage_def); - disable_persistency = join_settings->disable_persistency; + persistency_ = join_settings->persistency; } else throw Exception( @@ -226,7 +226,7 @@ void registerStorageJoin(StorageFactory & factory) args.constraints, join_any_take_last_row, args.context, - disable_persistency); + persistency_); }; factory.registerStorage("Join", creator_fn, StorageFactory::StorageFeatures{ .supports_settings = true, }); diff --git a/src/Storages/StorageJoin.h b/src/Storages/StorageJoin.h index 99cba0b4579..2751031ffbe 100644 --- a/src/Storages/StorageJoin.h +++ b/src/Storages/StorageJoin.h @@ -74,7 +74,7 @@ protected: const ConstraintsDescription & constraints_, bool overwrite, const Context & context_, - bool disable_persistency_); + bool persistency_); }; } diff --git a/src/Storages/StorageSet.cpp b/src/Storages/StorageSet.cpp index 3f011408b0d..5edcc68ba77 100644 --- a/src/Storages/StorageSet.cpp +++ b/src/Storages/StorageSet.cpp @@ -37,7 +37,7 @@ public: SetOrJoinBlockOutputStream( StorageSetOrJoinBase & table_, const StorageMetadataPtr & metadata_snapshot_, const String & backup_path_, const String & backup_tmp_path_, - const String & backup_file_name_, bool disable_persistency_); + const String & backup_file_name_, bool persistency_); Block getHeader() const override { return metadata_snapshot->getSampleBlock(); } void write(const Block & block) override; @@ -52,7 +52,7 @@ private: WriteBufferFromFile backup_buf; CompressedWriteBuffer compressed_backup_buf; NativeBlockOutputStream backup_stream; - bool disable_persistency; + bool persistency; }; @@ -62,7 +62,7 @@ SetOrJoinBlockOutputStream::SetOrJoinBlockOutputStream( const String & backup_path_, const String & backup_tmp_path_, const String & backup_file_name_, - bool disable_persistency_) + bool persistency_) : table(table_) , metadata_snapshot(metadata_snapshot_) , backup_path(backup_path_) @@ -71,7 +71,7 @@ SetOrJoinBlockOutputStream::SetOrJoinBlockOutputStream( , backup_buf(backup_tmp_path + backup_file_name) , compressed_backup_buf(backup_buf) , backup_stream(compressed_backup_buf, 0, metadata_snapshot->getSampleBlock()) - , disable_persistency(disable_persistency_) + , persistency(persistency_) { } @@ -81,14 +81,14 @@ void SetOrJoinBlockOutputStream::write(const Block & block) Block sorted_block = block.sortColumns(); table.insertBlock(sorted_block); - if (!disable_persistency) + if (persistency) backup_stream.write(sorted_block); } void SetOrJoinBlockOutputStream::writeSuffix() { table.finishInsert(); - if (!disable_persistency) + if (persistency) { backup_stream.flush(); compressed_backup_buf.next(); @@ -102,7 +102,7 @@ void SetOrJoinBlockOutputStream::writeSuffix() BlockOutputStreamPtr StorageSetOrJoinBase::write(const ASTPtr & /*query*/, const StorageMetadataPtr & metadata_snapshot, const Context & /*context*/) { UInt64 id = ++increment; - return std::make_shared(*this, metadata_snapshot, path, path + "tmp/", toString(id) + ".bin", disable_persistency); + return std::make_shared(*this, metadata_snapshot, path, path + "tmp/", toString(id) + ".bin", persistency); } @@ -112,9 +112,9 @@ StorageSetOrJoinBase::StorageSetOrJoinBase( const ColumnsDescription & columns_, const ConstraintsDescription & constraints_, const Context & context_, - bool disable_persistency_) + bool persistency_) : IStorage(table_id_), - disable_persistency(disable_persistency_) + persistency(persistency_) { StorageInMemoryMetadata storage_metadata; storage_metadata.setColumns(columns_); @@ -136,8 +136,8 @@ StorageSet::StorageSet( const ColumnsDescription & columns_, const ConstraintsDescription & constraints_, const Context & context_, - bool disable_persistency_) - : StorageSetOrJoinBase{relative_path_, table_id_, columns_, constraints_, context_, disable_persistency_}, + bool persistency_) + : StorageSetOrJoinBase{relative_path_, table_id_, columns_, constraints_, context_, persistency_}, set(std::make_shared(SizeLimits(), false, true)) { @@ -249,7 +249,7 @@ void registerStorageSet(StorageFactory & factory) set_settings->loadFromQuery(*args.storage_def); } - return StorageSet::create(args.relative_data_path, args.table_id, args.columns, args.constraints, args.context, set_settings->disable_persistency); + return StorageSet::create(args.relative_data_path, args.table_id, args.columns, args.constraints, args.context, set_settings->persistency); }, StorageFactory::StorageFeatures{ .supports_settings = true, }); } diff --git a/src/Storages/StorageSet.h b/src/Storages/StorageSet.h index 681315d7d11..87ad1521824 100644 --- a/src/Storages/StorageSet.h +++ b/src/Storages/StorageSet.h @@ -33,11 +33,11 @@ protected: const ColumnsDescription & columns_, const ConstraintsDescription & constraints_, const Context & context_, - bool disable_persistency_); + bool persistency_); String base_path; String path; - bool disable_persistency; + bool persistency; std::atomic increment = 0; /// For the backup file names. @@ -86,7 +86,7 @@ protected: const ColumnsDescription & columns_, const ConstraintsDescription & constraints_, const Context & context_, - bool disable_persistency_); + bool persistency_); }; } diff --git a/tests/queries/0_stateless/01493_storage_set_persistency.reference b/tests/queries/0_stateless/01493_storage_set_persistency.reference index e69de29bb2d..3ac4944d0d6 100644 --- a/tests/queries/0_stateless/01493_storage_set_persistency.reference +++ b/tests/queries/0_stateless/01493_storage_set_persistency.reference @@ -0,0 +1,5 @@ +----- Default Settings ----- +1 +----- Settings persistency=1 ----- +1 +----- Settings persistency=0 ----- diff --git a/tests/queries/0_stateless/01493_storage_set_persistency.sql b/tests/queries/0_stateless/01493_storage_set_persistency.sql index d70a372fbc7..54f19a870d2 100644 --- a/tests/queries/0_stateless/01493_storage_set_persistency.sql +++ b/tests/queries/0_stateless/01493_storage_set_persistency.sql @@ -1,8 +1,33 @@ DROP TABLE IF EXISTS set; +DROP TABLE IF EXISTS number; -CREATE TABLE set (x String) ENGINE = Set() SETTINGS disable_persistency=1; +CREATE TABLE number (number UInt64) ENGINE = Memory(); +INSERT INTO number values (1); +SELECT '----- Default Settings -----'; +CREATE TABLE set (val UInt64) ENGINE = Set(); +INSERT INTO set VALUES (1); DETACH TABLE set; -ATTACH TABLE set (x String) ENGINE = Set() SETTINGS disable_persistency=1; +ATTACH TABLE set (val UInt64) ENGINE = Set(); +SELECT number FROM number WHERE number IN set LIMIT 1; DROP TABLE set; + +SELECT '----- Settings persistency=1 -----'; +CREATE TABLE set (val UInt64) ENGINE = Set() SETTINGS persistency=1; +INSERT INTO set VALUES (1); +DETACH TABLE set; +ATTACH TABLE set (val UInt64) ENGINE = Set() SETTINGS persistency=1; +SELECT number FROM number WHERE number IN set LIMIT 1; + +DROP TABLE set; + +SELECT '----- Settings persistency=0 -----'; +CREATE TABLE set (val UInt64) ENGINE = Set() SETTINGS persistency=0; +INSERT INTO set VALUES (1); +DETACH TABLE set; +ATTACH TABLE set (val UInt64) ENGINE = Set() SETTINGS persistency=0; +SELECT number FROM number WHERE number IN set LIMIT 1; + +DROP TABLE set; +DROP TABLE number; \ No newline at end of file diff --git a/tests/queries/0_stateless/01494_storage_join_persistency.reference b/tests/queries/0_stateless/01494_storage_join_persistency.reference index e69de29bb2d..506f916ac8b 100644 --- a/tests/queries/0_stateless/01494_storage_join_persistency.reference +++ b/tests/queries/0_stateless/01494_storage_join_persistency.reference @@ -0,0 +1,5 @@ +----- Default Settings ----- +1 21 +----- Settings persistency=1 ----- +1 21 +----- Settings persistency=0 ----- diff --git a/tests/queries/0_stateless/01494_storage_join_persistency.sql b/tests/queries/0_stateless/01494_storage_join_persistency.sql index 75ca64298ff..6e1fee3430e 100644 --- a/tests/queries/0_stateless/01494_storage_join_persistency.sql +++ b/tests/queries/0_stateless/01494_storage_join_persistency.sql @@ -1,8 +1,28 @@ DROP TABLE IF EXISTS join; -CREATE TABLE join (k UInt64, s String) ENGINE = Join(ANY, LEFT, k) SETTINGS disable_persistency=1; - +SELECT '----- Default Settings -----'; +CREATE TABLE join (k UInt64, s String) ENGINE = Join(ANY, LEFT, k); +INSERT INTO join VALUES (1,21); DETACH TABLE join; -ATTACH TABLE join (k UInt64, s String) ENGINE = Join(ANY, LEFT, k) SETTINGS disable_persistency=1; +ATTACH TABLE join (k UInt64, s String) ENGINE = Join(ANY, LEFT, k); +SELECT * from join; + +DROP TABLE join; + +SELECT '----- Settings persistency=1 -----'; +CREATE TABLE join (k UInt64, s String) ENGINE = Join(ANY, LEFT, k) SETTINGS persistency=1; +INSERT INTO join VALUES (1,21); +DETACH TABLE join; +ATTACH TABLE join (k UInt64, s String) ENGINE = Join(ANY, LEFT, k) SETTINGS persistency=1; +SELECT * from join; + +DROP TABLE join; + +SELECT '----- Settings persistency=0 -----'; +CREATE TABLE join (k UInt64, s String) ENGINE = Join(ANY, LEFT, k) SETTINGS persistency=0; +INSERT INTO join VALUES (1,21); +DETACH TABLE join; +ATTACH TABLE join (k UInt64, s String) ENGINE = Join(ANY, LEFT, k) SETTINGS persistency=0; +SELECT * from join; DROP TABLE join; From c9249f77c797f34da2c90226078a758b0a0a7046 Mon Sep 17 00:00:00 2001 From: Vxider Date: Fri, 18 Sep 2020 20:58:27 +0800 Subject: [PATCH 022/321] rename persistency to persistent --- src/Storages/JoinSettings.h | 2 +- src/Storages/SetSettings.h | 2 +- src/Storages/StorageJoin.cpp | 12 +++++----- src/Storages/StorageJoin.h | 2 +- src/Storages/StorageSet.cpp | 24 +++++++++---------- src/Storages/StorageSet.h | 6 ++--- .../01493_storage_set_persistency.reference | 4 ++-- .../01493_storage_set_persistency.sql | 12 +++++----- .../01494_storage_join_persistency.reference | 4 ++-- .../01494_storage_join_persistency.sql | 12 +++++----- 10 files changed, 40 insertions(+), 40 deletions(-) diff --git a/src/Storages/JoinSettings.h b/src/Storages/JoinSettings.h index 396185d9f77..13be557a439 100644 --- a/src/Storages/JoinSettings.h +++ b/src/Storages/JoinSettings.h @@ -10,7 +10,7 @@ class ASTStorage; #define JOIN_RELATED_SETTINGS(M) \ - M(Bool, persistency, true, "Disable setting to avoid the overhead of writing to disk for StorageJoin", 0) + M(Bool, persistent, true, "Disable setting to avoid the overhead of writing to disk for StorageJoin", 0) #define LIST_OF_JOIN_SETTINGS(M) \ JOIN_RELATED_SETTINGS(M) \ diff --git a/src/Storages/SetSettings.h b/src/Storages/SetSettings.h index 2e55e6e8ab5..c5926e409c2 100644 --- a/src/Storages/SetSettings.h +++ b/src/Storages/SetSettings.h @@ -10,7 +10,7 @@ class ASTStorage; #define SET_RELATED_SETTINGS(M) \ - M(Bool, persistency, true, "Disable setting to avoid the overhead of writing to disk for StorageSet", 0) + M(Bool, persistent, true, "Disable setting to avoid the overhead of writing to disk for StorageSet", 0) #define LIST_OF_SET_SETTINGS(M) \ SET_RELATED_SETTINGS(M) \ diff --git a/src/Storages/StorageJoin.cpp b/src/Storages/StorageJoin.cpp index 14637856439..1f8175099e4 100644 --- a/src/Storages/StorageJoin.cpp +++ b/src/Storages/StorageJoin.cpp @@ -45,8 +45,8 @@ StorageJoin::StorageJoin( const ConstraintsDescription & constraints_, bool overwrite_, const Context & context_, - bool persistency_) - : StorageSetOrJoinBase{relative_path_, table_id_, columns_, constraints_, context_, persistency_} + bool persistent_) + : StorageSetOrJoinBase{relative_path_, table_id_, columns_, constraints_, context_, persistent_} , key_names(key_names_) , use_nulls(use_nulls_) , limits(limits_) @@ -119,7 +119,7 @@ void registerStorageJoin(StorageFactory & factory) auto join_overflow_mode = settings.join_overflow_mode; auto join_any_take_last_row = settings.join_any_take_last_row; auto old_any_join = settings.any_join_distinct_right_table_keys; - bool persistency_ = true; + bool persistent_ = true; if (args.storage_def && args.storage_def->settings) { @@ -137,11 +137,11 @@ void registerStorageJoin(StorageFactory & factory) join_any_take_last_row = setting.value; else if (setting.name == "any_join_distinct_right_table_keys") old_any_join = setting.value; - else if (setting.name == "persistency") + else if (setting.name == "persistent") { auto join_settings = std::make_unique(); join_settings->loadFromQuery(*args.storage_def); - persistency_ = join_settings->persistency; + persistent_ = join_settings->persistent; } else throw Exception( @@ -226,7 +226,7 @@ void registerStorageJoin(StorageFactory & factory) args.constraints, join_any_take_last_row, args.context, - persistency_); + persistent_); }; factory.registerStorage("Join", creator_fn, StorageFactory::StorageFeatures{ .supports_settings = true, }); diff --git a/src/Storages/StorageJoin.h b/src/Storages/StorageJoin.h index 2751031ffbe..95037c4d33a 100644 --- a/src/Storages/StorageJoin.h +++ b/src/Storages/StorageJoin.h @@ -74,7 +74,7 @@ protected: const ConstraintsDescription & constraints_, bool overwrite, const Context & context_, - bool persistency_); + bool persistent_); }; } diff --git a/src/Storages/StorageSet.cpp b/src/Storages/StorageSet.cpp index 5edcc68ba77..d6d8b9e1449 100644 --- a/src/Storages/StorageSet.cpp +++ b/src/Storages/StorageSet.cpp @@ -37,7 +37,7 @@ public: SetOrJoinBlockOutputStream( StorageSetOrJoinBase & table_, const StorageMetadataPtr & metadata_snapshot_, const String & backup_path_, const String & backup_tmp_path_, - const String & backup_file_name_, bool persistency_); + const String & backup_file_name_, bool persistent_); Block getHeader() const override { return metadata_snapshot->getSampleBlock(); } void write(const Block & block) override; @@ -52,7 +52,7 @@ private: WriteBufferFromFile backup_buf; CompressedWriteBuffer compressed_backup_buf; NativeBlockOutputStream backup_stream; - bool persistency; + bool persistent; }; @@ -62,7 +62,7 @@ SetOrJoinBlockOutputStream::SetOrJoinBlockOutputStream( const String & backup_path_, const String & backup_tmp_path_, const String & backup_file_name_, - bool persistency_) + bool persistent_) : table(table_) , metadata_snapshot(metadata_snapshot_) , backup_path(backup_path_) @@ -71,7 +71,7 @@ SetOrJoinBlockOutputStream::SetOrJoinBlockOutputStream( , backup_buf(backup_tmp_path + backup_file_name) , compressed_backup_buf(backup_buf) , backup_stream(compressed_backup_buf, 0, metadata_snapshot->getSampleBlock()) - , persistency(persistency_) + , persistent(persistent_) { } @@ -81,14 +81,14 @@ void SetOrJoinBlockOutputStream::write(const Block & block) Block sorted_block = block.sortColumns(); table.insertBlock(sorted_block); - if (persistency) + if (persistent) backup_stream.write(sorted_block); } void SetOrJoinBlockOutputStream::writeSuffix() { table.finishInsert(); - if (persistency) + if (persistent) { backup_stream.flush(); compressed_backup_buf.next(); @@ -102,7 +102,7 @@ void SetOrJoinBlockOutputStream::writeSuffix() BlockOutputStreamPtr StorageSetOrJoinBase::write(const ASTPtr & /*query*/, const StorageMetadataPtr & metadata_snapshot, const Context & /*context*/) { UInt64 id = ++increment; - return std::make_shared(*this, metadata_snapshot, path, path + "tmp/", toString(id) + ".bin", persistency); + return std::make_shared(*this, metadata_snapshot, path, path + "tmp/", toString(id) + ".bin", persistent); } @@ -112,9 +112,9 @@ StorageSetOrJoinBase::StorageSetOrJoinBase( const ColumnsDescription & columns_, const ConstraintsDescription & constraints_, const Context & context_, - bool persistency_) + bool persistent_) : IStorage(table_id_), - persistency(persistency_) + persistent(persistent_) { StorageInMemoryMetadata storage_metadata; storage_metadata.setColumns(columns_); @@ -136,8 +136,8 @@ StorageSet::StorageSet( const ColumnsDescription & columns_, const ConstraintsDescription & constraints_, const Context & context_, - bool persistency_) - : StorageSetOrJoinBase{relative_path_, table_id_, columns_, constraints_, context_, persistency_}, + bool persistent_) + : StorageSetOrJoinBase{relative_path_, table_id_, columns_, constraints_, context_, persistent_}, set(std::make_shared(SizeLimits(), false, true)) { @@ -249,7 +249,7 @@ void registerStorageSet(StorageFactory & factory) set_settings->loadFromQuery(*args.storage_def); } - return StorageSet::create(args.relative_data_path, args.table_id, args.columns, args.constraints, args.context, set_settings->persistency); + return StorageSet::create(args.relative_data_path, args.table_id, args.columns, args.constraints, args.context, set_settings->persistent); }, StorageFactory::StorageFeatures{ .supports_settings = true, }); } diff --git a/src/Storages/StorageSet.h b/src/Storages/StorageSet.h index 87ad1521824..40d7925de13 100644 --- a/src/Storages/StorageSet.h +++ b/src/Storages/StorageSet.h @@ -33,11 +33,11 @@ protected: const ColumnsDescription & columns_, const ConstraintsDescription & constraints_, const Context & context_, - bool persistency_); + bool persistent_); String base_path; String path; - bool persistency; + bool persistent; std::atomic increment = 0; /// For the backup file names. @@ -86,7 +86,7 @@ protected: const ColumnsDescription & columns_, const ConstraintsDescription & constraints_, const Context & context_, - bool persistency_); + bool persistent_); }; } diff --git a/tests/queries/0_stateless/01493_storage_set_persistency.reference b/tests/queries/0_stateless/01493_storage_set_persistency.reference index 3ac4944d0d6..c721770a22a 100644 --- a/tests/queries/0_stateless/01493_storage_set_persistency.reference +++ b/tests/queries/0_stateless/01493_storage_set_persistency.reference @@ -1,5 +1,5 @@ ----- Default Settings ----- 1 ------ Settings persistency=1 ----- +----- Settings persistent=1 ----- 1 ------ Settings persistency=0 ----- +----- Settings persistent=0 ----- diff --git a/tests/queries/0_stateless/01493_storage_set_persistency.sql b/tests/queries/0_stateless/01493_storage_set_persistency.sql index 54f19a870d2..2ab5b409733 100644 --- a/tests/queries/0_stateless/01493_storage_set_persistency.sql +++ b/tests/queries/0_stateless/01493_storage_set_persistency.sql @@ -13,20 +13,20 @@ SELECT number FROM number WHERE number IN set LIMIT 1; DROP TABLE set; -SELECT '----- Settings persistency=1 -----'; -CREATE TABLE set (val UInt64) ENGINE = Set() SETTINGS persistency=1; +SELECT '----- Settings persistent=1 -----'; +CREATE TABLE set (val UInt64) ENGINE = Set() SETTINGS persistent=1; INSERT INTO set VALUES (1); DETACH TABLE set; -ATTACH TABLE set (val UInt64) ENGINE = Set() SETTINGS persistency=1; +ATTACH TABLE set (val UInt64) ENGINE = Set() SETTINGS persistent=1; SELECT number FROM number WHERE number IN set LIMIT 1; DROP TABLE set; -SELECT '----- Settings persistency=0 -----'; -CREATE TABLE set (val UInt64) ENGINE = Set() SETTINGS persistency=0; +SELECT '----- Settings persistent=0 -----'; +CREATE TABLE set (val UInt64) ENGINE = Set() SETTINGS persistent=0; INSERT INTO set VALUES (1); DETACH TABLE set; -ATTACH TABLE set (val UInt64) ENGINE = Set() SETTINGS persistency=0; +ATTACH TABLE set (val UInt64) ENGINE = Set() SETTINGS persistent=0; SELECT number FROM number WHERE number IN set LIMIT 1; DROP TABLE set; diff --git a/tests/queries/0_stateless/01494_storage_join_persistency.reference b/tests/queries/0_stateless/01494_storage_join_persistency.reference index 506f916ac8b..31ba11136ea 100644 --- a/tests/queries/0_stateless/01494_storage_join_persistency.reference +++ b/tests/queries/0_stateless/01494_storage_join_persistency.reference @@ -1,5 +1,5 @@ ----- Default Settings ----- 1 21 ------ Settings persistency=1 ----- +----- Settings persistent=1 ----- 1 21 ------ Settings persistency=0 ----- +----- Settings persistent=0 ----- diff --git a/tests/queries/0_stateless/01494_storage_join_persistency.sql b/tests/queries/0_stateless/01494_storage_join_persistency.sql index 6e1fee3430e..4ca24db943a 100644 --- a/tests/queries/0_stateless/01494_storage_join_persistency.sql +++ b/tests/queries/0_stateless/01494_storage_join_persistency.sql @@ -9,20 +9,20 @@ SELECT * from join; DROP TABLE join; -SELECT '----- Settings persistency=1 -----'; -CREATE TABLE join (k UInt64, s String) ENGINE = Join(ANY, LEFT, k) SETTINGS persistency=1; +SELECT '----- Settings persistent=1 -----'; +CREATE TABLE join (k UInt64, s String) ENGINE = Join(ANY, LEFT, k) SETTINGS persistent=1; INSERT INTO join VALUES (1,21); DETACH TABLE join; -ATTACH TABLE join (k UInt64, s String) ENGINE = Join(ANY, LEFT, k) SETTINGS persistency=1; +ATTACH TABLE join (k UInt64, s String) ENGINE = Join(ANY, LEFT, k) SETTINGS persistent=1; SELECT * from join; DROP TABLE join; -SELECT '----- Settings persistency=0 -----'; -CREATE TABLE join (k UInt64, s String) ENGINE = Join(ANY, LEFT, k) SETTINGS persistency=0; +SELECT '----- Settings persistent=0 -----'; +CREATE TABLE join (k UInt64, s String) ENGINE = Join(ANY, LEFT, k) SETTINGS persistent=0; INSERT INTO join VALUES (1,21); DETACH TABLE join; -ATTACH TABLE join (k UInt64, s String) ENGINE = Join(ANY, LEFT, k) SETTINGS persistency=0; +ATTACH TABLE join (k UInt64, s String) ENGINE = Join(ANY, LEFT, k) SETTINGS persistent=0; SELECT * from join; DROP TABLE join; From 91826c46b57d278f59ef738197ee73d23c18bfb3 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 19 Sep 2020 02:04:36 +0300 Subject: [PATCH 023/321] Fix terribly wrong code --- base/common/phdr_cache.cpp | 4 +- programs/server/Server.cpp | 1 - src/CMakeLists.txt | 1 + src/Common/SymbolIndex.cpp | 2 +- src/Common/tests/int_hashes_perf.cpp | 7 +- src/DataTypes/convertMySQLDataType.cpp | 126 ++++---- src/Disks/S3/registerDiskS3.cpp | 1 - src/Functions/appendTrailingCharIfAbsent.cpp | 2 +- src/IO/S3/PocoHTTPClient.cpp | 19 +- src/IO/S3Common.cpp | 2 - ...est_aux_funcs_for_adaptive_granularity.cpp | 2 +- ...for_adaptive_granularity_compact_parts.cpp | 2 +- utils/db-generator/query_db_generator.cpp | 284 +++++++++--------- 13 files changed, 225 insertions(+), 228 deletions(-) diff --git a/base/common/phdr_cache.cpp b/base/common/phdr_cache.cpp index f362fb64285..4f6a066adab 100644 --- a/base/common/phdr_cache.cpp +++ b/base/common/phdr_cache.cpp @@ -14,7 +14,7 @@ # pragma clang diagnostic ignored "-Wunused-macros" #endif -#define __msan_unpoison(X, Y) +#define __msan_unpoison(X, Y) // NOLINT #if defined(__has_feature) # if __has_feature(memory_sanitizer) # undef __msan_unpoison @@ -84,7 +84,7 @@ extern "C" #ifdef ADDRESS_SANITIZER void __lsan_ignore_object(const void *); #else -void __lsan_ignore_object(const void *) {} +void __lsan_ignore_object(const void *) {} // NOLINT #endif } diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index aa947b22593..97212e3ddcf 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -43,7 +43,6 @@ #include #include #include -#include #include #include #include diff --git a/src/CMakeLists.txt b/src/CMakeLists.txt index b6e8c395b26..8f84939c5a4 100644 --- a/src/CMakeLists.txt +++ b/src/CMakeLists.txt @@ -67,6 +67,7 @@ set(dbms_sources) add_headers_and_sources(clickhouse_common_io Common) add_headers_and_sources(clickhouse_common_io Common/HashTable) add_headers_and_sources(clickhouse_common_io IO) +add_headers_and_sources(clickhouse_common_io IO/S3) list (REMOVE_ITEM clickhouse_common_io_sources Common/malloc.cpp Common/new_delete.cpp) if(USE_RDKAFKA) diff --git a/src/Common/SymbolIndex.cpp b/src/Common/SymbolIndex.cpp index ffa7f0462c9..a738512bb30 100644 --- a/src/Common/SymbolIndex.cpp +++ b/src/Common/SymbolIndex.cpp @@ -59,7 +59,7 @@ Otherwise you will get only exported symbols from program headers. # pragma clang diagnostic ignored "-Wunused-macros" #endif -#define __msan_unpoison_string(X) +#define __msan_unpoison_string(X) // NOLINT #if defined(__has_feature) # if __has_feature(memory_sanitizer) # undef __msan_unpoison_string diff --git a/src/Common/tests/int_hashes_perf.cpp b/src/Common/tests/int_hashes_perf.cpp index 7e8495cef27..569e9273bc4 100644 --- a/src/Common/tests/int_hashes_perf.cpp +++ b/src/Common/tests/int_hashes_perf.cpp @@ -187,11 +187,6 @@ static inline size_t tabulation(UInt64 x) return res; } -static inline size_t _intHash64(UInt64 x) -{ - return static_cast(intHash64(x)); -} - const size_t BUF_SIZE = 1024; @@ -284,7 +279,7 @@ int main(int argc, char ** argv) if (!method || method == 1) test (n, data.data(), "0: identity"); if (!method || method == 2) test (n, data.data(), "1: intHash32"); - if (!method || method == 3) test<_intHash64>(n, data.data(), "2: intHash64"); + if (!method || method == 3) test (n, data.data(), "2: intHash64"); if (!method || method == 4) test (n, data.data(), "3: two rounds"); if (!method || method == 5) test (n, data.data(), "4: two rounds and two variables"); if (!method || method == 6) test (n, data.data(), "5: two rounds with less ops"); diff --git a/src/DataTypes/convertMySQLDataType.cpp b/src/DataTypes/convertMySQLDataType.cpp index a509cf8b091..ef6fb7de874 100644 --- a/src/DataTypes/convertMySQLDataType.cpp +++ b/src/DataTypes/convertMySQLDataType.cpp @@ -40,74 +40,76 @@ DataTypePtr convertMySQLDataType(MultiEnum type_support, { // we expect mysql_data_type to be either "basic_type" or "type_with_params(param1, param2, ...)" auto data_type = std::string_view(mysql_data_type); - const auto param_start_pos = data_type.find("("); + const auto param_start_pos = data_type.find('('); const auto type_name = data_type.substr(0, param_start_pos); - DataTypePtr res = [&]() -> DataTypePtr { - if (type_name == "tinyint") - { - if (is_unsigned) - return std::make_shared(); - else - return std::make_shared(); - } - if (type_name == "smallint") - { - if (is_unsigned) - return std::make_shared(); - else - return std::make_shared(); - } - if (type_name == "int" || type_name == "mediumint") - { - if (is_unsigned) - return std::make_shared(); - else - return std::make_shared(); - } - if (type_name == "bigint") - { - if (is_unsigned) - return std::make_shared(); - else - return std::make_shared(); - } - if (type_name == "float") - return std::make_shared(); - if (type_name == "double") - return std::make_shared(); - if (type_name == "date") - return std::make_shared(); - if (type_name == "binary") - return std::make_shared(length); - if (type_name == "datetime" || type_name == "timestamp") - { - if (!type_support.isSet(MySQLDataTypesSupport::DATETIME64)) - return std::make_shared(); + DataTypePtr res; - if (type_name == "timestamp" && scale == 0) - { - return std::make_shared(); - } - else if (type_name == "datetime" || type_name == "timestamp") - { - return std::make_shared(scale); - } - } - - if (type_support.isSet(MySQLDataTypesSupport::DECIMAL) && (type_name == "numeric" || type_name == "decimal")) + if (type_name == "tinyint") + { + if (is_unsigned) + res = std::make_shared(); + else + res = std::make_shared(); + } + else if (type_name == "smallint") + { + if (is_unsigned) + res = std::make_shared(); + else + res = std::make_shared(); + } + else if (type_name == "int" || type_name == "mediumint") + { + if (is_unsigned) + res = std::make_shared(); + else + res = std::make_shared(); + } + else if (type_name == "bigint") + { + if (is_unsigned) + res = std::make_shared(); + else + res = std::make_shared(); + } + else if (type_name == "float") + res = std::make_shared(); + else if (type_name == "double") + res = std::make_shared(); + else if (type_name == "date") + res = std::make_shared(); + else if (type_name == "binary") + res = std::make_shared(length); + else if (type_name == "datetime" || type_name == "timestamp") + { + if (!type_support.isSet(MySQLDataTypesSupport::DATETIME64)) { - if (precision <= DecimalUtils::maxPrecision()) - return std::make_shared>(precision, scale); - else if (precision <= DecimalUtils::maxPrecision()) - return std::make_shared>(precision, scale); - else if (precision <= DecimalUtils::maxPrecision()) - return std::make_shared>(precision, scale); + res = std::make_shared(); } - + else if (type_name == "timestamp" && scale == 0) + { + res = std::make_shared(); + } + else if (type_name == "datetime" || type_name == "timestamp") + { + res = std::make_shared(scale); + } + } + else if (type_support.isSet(MySQLDataTypesSupport::DECIMAL) && (type_name == "numeric" || type_name == "decimal")) + { + if (precision <= DecimalUtils::maxPrecision()) + res = std::make_shared>(precision, scale); + else if (precision <= DecimalUtils::maxPrecision()) + res = std::make_shared>(precision, scale); + else if (precision <= DecimalUtils::maxPrecision()) + res = std::make_shared>(precision, scale); + } + else + { /// Also String is fallback for all unknown types. - return std::make_shared(); - }(); + res = std::make_shared(); + } if (is_nullable) res = std::make_shared(res); diff --git a/src/Disks/S3/registerDiskS3.cpp b/src/Disks/S3/registerDiskS3.cpp index fbd19ce1cd9..1c7a5e24282 100644 --- a/src/Disks/S3/registerDiskS3.cpp +++ b/src/Disks/S3/registerDiskS3.cpp @@ -4,7 +4,6 @@ #include #include "DiskS3.h" #include "Disks/DiskCacheWrapper.h" -#include "Disks/DiskCacheWrapper.cpp" #include "Disks/DiskFactory.h" #include "ProxyConfiguration.h" #include "ProxyListConfiguration.h" diff --git a/src/Functions/appendTrailingCharIfAbsent.cpp b/src/Functions/appendTrailingCharIfAbsent.cpp index 67a3cbabe6d..eb625374707 100644 --- a/src/Functions/appendTrailingCharIfAbsent.cpp +++ b/src/Functions/appendTrailingCharIfAbsent.cpp @@ -92,7 +92,7 @@ private: src_offset = src_offsets[i]; dst_offset += src_length; - if (src_length > 1 && dst_data[dst_offset - 2] != trailing_char_str.front()) + if (src_length > 1 && dst_data[dst_offset - 2] != UInt8(trailing_char_str.front())) { dst_data[dst_offset - 1] = trailing_char_str.front(); dst_data[dst_offset] = 0; diff --git a/src/IO/S3/PocoHTTPClient.cpp b/src/IO/S3/PocoHTTPClient.cpp index e068f3581bd..6d601b1e9ed 100644 --- a/src/IO/S3/PocoHTTPClient.cpp +++ b/src/IO/S3/PocoHTTPClient.cpp @@ -3,7 +3,6 @@ #include #include #include -#include #include #include #include @@ -101,7 +100,7 @@ void PocoHTTPClient::MakeRequestInternal( EnumSize, }; - auto selectMetric = [&request](S3MetricType type) + auto select_metric = [&request](S3MetricType type) { const ProfileEvents::Event events_map[][2] = { {ProfileEvents::S3ReadMicroseconds, ProfileEvents::S3WriteMicroseconds}, @@ -128,12 +127,12 @@ void PocoHTTPClient::MakeRequestInternal( throw Exception("Unsupported request method", ErrorCodes::NOT_IMPLEMENTED); }; - ProfileEvents::increment(selectMetric(S3MetricType::Count)); + ProfileEvents::increment(select_metric(S3MetricType::Count)); - const int MAX_REDIRECT_ATTEMPTS = 10; + static constexpr int max_redirect_attempts = 10; try { - for (int attempt = 0; attempt < MAX_REDIRECT_ATTEMPTS; ++attempt) + for (int attempt = 0; attempt < max_redirect_attempts; ++attempt) { Poco::URI poco_uri(uri); @@ -202,7 +201,7 @@ void PocoHTTPClient::MakeRequestInternal( auto & response_body_stream = session->receiveResponse(poco_response); watch.stop(); - ProfileEvents::increment(selectMetric(S3MetricType::Microseconds), watch.elapsedMicroseconds()); + ProfileEvents::increment(select_metric(S3MetricType::Microseconds), watch.elapsedMicroseconds()); int status_code = static_cast(poco_response.getStatus()); LOG_DEBUG(log, "Response status: {}, {}", status_code, poco_response.getReason()); @@ -214,7 +213,7 @@ void PocoHTTPClient::MakeRequestInternal( uri = location; LOG_DEBUG(log, "Redirecting request to new location: {}", location); - ProfileEvents::increment(selectMetric(S3MetricType::Redirects)); + ProfileEvents::increment(select_metric(S3MetricType::Redirects)); continue; } @@ -240,11 +239,11 @@ void PocoHTTPClient::MakeRequestInternal( if (status_code == 429 || status_code == 503) { // API throttling - ProfileEvents::increment(selectMetric(S3MetricType::Throttling)); + ProfileEvents::increment(select_metric(S3MetricType::Throttling)); } else { - ProfileEvents::increment(selectMetric(S3MetricType::Errors)); + ProfileEvents::increment(select_metric(S3MetricType::Errors)); } } else @@ -261,7 +260,7 @@ void PocoHTTPClient::MakeRequestInternal( response->SetClientErrorType(Aws::Client::CoreErrors::NETWORK_CONNECTION); response->SetClientErrorMessage(getCurrentExceptionMessage(false)); - ProfileEvents::increment(selectMetric(S3MetricType::Errors)); + ProfileEvents::increment(select_metric(S3MetricType::Errors)); } } } diff --git a/src/IO/S3Common.cpp b/src/IO/S3Common.cpp index ba9a61ab922..db7aaf1549b 100644 --- a/src/IO/S3Common.cpp +++ b/src/IO/S3Common.cpp @@ -12,9 +12,7 @@ # include # include # include -# include # include -# include # include # include # include diff --git a/src/Storages/tests/gtest_aux_funcs_for_adaptive_granularity.cpp b/src/Storages/tests/gtest_aux_funcs_for_adaptive_granularity.cpp index d9ddb8e9722..4e7bff0ef41 100644 --- a/src/Storages/tests/gtest_aux_funcs_for_adaptive_granularity.cpp +++ b/src/Storages/tests/gtest_aux_funcs_for_adaptive_granularity.cpp @@ -3,7 +3,7 @@ #include // I know that inclusion of .cpp is not good at all -#include +#include // NOLINT using namespace DB; static Block getBlockWithSize(size_t required_size_in_bytes, size_t size_of_row_in_bytes) diff --git a/src/Storages/tests/gtest_aux_funcs_for_adaptive_granularity_compact_parts.cpp b/src/Storages/tests/gtest_aux_funcs_for_adaptive_granularity_compact_parts.cpp index f87293dcd5d..09b24c7dad6 100644 --- a/src/Storages/tests/gtest_aux_funcs_for_adaptive_granularity_compact_parts.cpp +++ b/src/Storages/tests/gtest_aux_funcs_for_adaptive_granularity_compact_parts.cpp @@ -3,7 +3,7 @@ #include // I know that inclusion of .cpp is not good at all -#include +#include // NOLINT using namespace DB; diff --git a/utils/db-generator/query_db_generator.cpp b/utils/db-generator/query_db_generator.cpp index 88f46325c72..16aa8aa7769 100644 --- a/utils/db-generator/query_db_generator.cpp +++ b/utils/db-generator/query_db_generator.cpp @@ -6,7 +6,6 @@ #include #include #include -#include #include #include #include @@ -16,6 +15,7 @@ #include #include + using ColumnType = uint32_t; using TableAndColumn = std::pair; pcg64 rng; @@ -92,7 +92,7 @@ TableAndColumn get_table_a_column(const std::string & c) } -enum type : ColumnType +enum Type : ColumnType { i = 1, // int @@ -114,9 +114,11 @@ enum type : ColumnType }; -std::map type_definition = { - {type::i, "Int64"}, {type::f, "Float64"}, {type::s, "String"}, {type::d, "Date"}, {type::dt, "DateTime"}, {type::b, "UInt8"} +std::map type_definition = +{ + {Type::i, "Int64"}, {Type::f, "Float64"}, {Type::s, "String"}, {Type::d, "Date"}, {Type::dt, "DateTime"}, {Type::b, "UInt8"} }; + ColumnType time_type(std::string value) { if (value.length() == 12) @@ -124,14 +126,14 @@ ColumnType time_type(std::string value) for (size_t i : {5, 8}) { if (value[i] != '-') - return type::s; + return Type::s; } for (size_t i : {1, 2, 3, 4, 6, 7, 9, 10}) { if (!isdigit(value[i])) - return type::s; + return Type::s; } - return type::d; + return Type::d; } if (value.length() == 21) @@ -139,18 +141,18 @@ ColumnType time_type(std::string value) for (size_t i : {5, 8}) { if (value[i] != '-') - return type::s; + return Type::s; } for (size_t i : {14, 17}) { if (value[i] != '-') - return type::s; + return Type::s; } if (value[11] != '-') - return type::s; - return type::dt; + return Type::s; + return Type::dt; } - return type::s; + return Type::s; } // Casting inner clickhouse parser type to our type ColumnType type_cast(int t) @@ -164,21 +166,21 @@ ColumnType type_cast(int t) case 19: case 20: case 21: - return type::i; + return Type::i; case 3: - return type::f; + return Type::f; case 16: - return type::s; + return Type::s; case 17: - return type::a | type::all; + return Type::a | Type::all; case 18: - return type::t | type::all; + return Type::t | Type::all; } - return type::all; + return Type::all; } @@ -197,28 +199,28 @@ public: , is_array(is_a) {} std::string value{}; - ColumnType type = type::all; + ColumnType type = Type::all; bool is_array = false; }; std::map func_to_return_type = { - {"divide", FuncRet(type::f, "")}, {"e", FuncRet(type::f, "e()")}, {"pi", FuncRet(type::f, "pi()")}, {"exp", FuncRet(type::f, "")}, - {"log", FuncRet(type::f,"")}, {"exp2", FuncRet(type::f, "")}, {"log2", FuncRet(type::f, "")}, {"exp10", FuncRet(type::f, "")}, - {"log10", FuncRet(type::f, "")}, {"sqrt", FuncRet(type::f, "")}, {"cbrt", FuncRet(type::f, "")}, {"erf", FuncRet(type::f, "")}, - {"erfc", FuncRet(type::f, "")}, {"lgamma", FuncRet(type::f, "")}, {"tgamma", FuncRet(type::f, "")}, {"sin", FuncRet(type::f, "")}, - {"cos", FuncRet(type::f, "")}, {"tan", FuncRet(type::f, "")}, {"asin", FuncRet(type::f, "")}, {"acos", FuncRet(type::f, "")}, - {"atan", FuncRet(type::f, "")}, {"pow", FuncRet(type::f, "")}, {"splitbystring", FuncRet(type::s | type::a,"")}, - {"splitbychar", FuncRet(type::s | type::a, "")}, {"alphatokens", FuncRet(type::s | type::a, "")}, {"toyear", FuncRet(type::i, "")}, - {"tomonth", FuncRet(type::i, "")}, {"todayofmonth", FuncRet(type::i, "")}, {"tohour", FuncRet(type::dt, "")}, {"tominute", FuncRet(type::dt, "")}, - {"toseconds", FuncRet(type::dt, "")}, {"tounixtimestamp", FuncRet(type::i, "")}, {"tostartofyear", FuncRet(type::dt | type::d, "")}, - {"tostartofquater",FuncRet(type::dt | type::d, "")}, {"tostartofmonth", FuncRet(type::dt | type::d, "")}, {"tomonday", FuncRet(type::dt | type::d, "")}, - {"tostartoffiveminutes", FuncRet(type::dt, "")}, {"tostartoftenminutes", FuncRet(type::dt, "")}, {"tostartoffifteenminutes", FuncRet(type::dt, "")}, - {"tostartofinterval", FuncRet(type::dt, "")}, {"totime", FuncRet(type::dt, "")}, {"torelativemonthnum", FuncRet(type::i, "")}, - {"torelativeweeknum", FuncRet(type::i, "")}, {"torelativedaynum", FuncRet(type::i, "")}, {"torelativehournum", FuncRet(type::i, "")}, - {"torelativeminutenum", FuncRet(type::i, "")}, {"torelativesecondsnum", FuncRet(type::i, "")}, {"datediff", FuncRet(type::d | type::dt, "")}, - {"formatdatetime", FuncRet(type::s, "")}, {"now", FuncRet(type::dt | type::d, "now()")}, {"today", FuncRet(type::d | type::dt, "today()")}, - {"yesterday", FuncRet(type::d | type::dt, "yesterday()")} + {"divide", FuncRet(Type::f, "")}, {"e", FuncRet(Type::f, "e()")}, {"pi", FuncRet(Type::f, "pi()")}, {"exp", FuncRet(Type::f, "")}, + {"log", FuncRet(Type::f,"")}, {"exp2", FuncRet(Type::f, "")}, {"log2", FuncRet(Type::f, "")}, {"exp10", FuncRet(Type::f, "")}, + {"log10", FuncRet(Type::f, "")}, {"sqrt", FuncRet(Type::f, "")}, {"cbrt", FuncRet(Type::f, "")}, {"erf", FuncRet(Type::f, "")}, + {"erfc", FuncRet(Type::f, "")}, {"lgamma", FuncRet(Type::f, "")}, {"tgamma", FuncRet(Type::f, "")}, {"sin", FuncRet(Type::f, "")}, + {"cos", FuncRet(Type::f, "")}, {"tan", FuncRet(Type::f, "")}, {"asin", FuncRet(Type::f, "")}, {"acos", FuncRet(Type::f, "")}, + {"atan", FuncRet(Type::f, "")}, {"pow", FuncRet(Type::f, "")}, {"splitbystring", FuncRet(Type::s | Type::a,"")}, + {"splitbychar", FuncRet(Type::s | Type::a, "")}, {"alphatokens", FuncRet(Type::s | Type::a, "")}, {"toyear", FuncRet(Type::i, "")}, + {"tomonth", FuncRet(Type::i, "")}, {"todayofmonth", FuncRet(Type::i, "")}, {"tohour", FuncRet(Type::dt, "")}, {"tominute", FuncRet(Type::dt, "")}, + {"toseconds", FuncRet(Type::dt, "")}, {"tounixtimestamp", FuncRet(Type::i, "")}, {"tostartofyear", FuncRet(Type::dt | Type::d, "")}, + {"tostartofquater",FuncRet(Type::dt | Type::d, "")}, {"tostartofmonth", FuncRet(Type::dt | Type::d, "")}, {"tomonday", FuncRet(Type::dt | Type::d, "")}, + {"tostartoffiveminutes", FuncRet(Type::dt, "")}, {"tostartoftenminutes", FuncRet(Type::dt, "")}, {"tostartoffifteenminutes", FuncRet(Type::dt, "")}, + {"tostartofinterval", FuncRet(Type::dt, "")}, {"totime", FuncRet(Type::dt, "")}, {"torelativemonthnum", FuncRet(Type::i, "")}, + {"torelativeweeknum", FuncRet(Type::i, "")}, {"torelativedaynum", FuncRet(Type::i, "")}, {"torelativehournum", FuncRet(Type::i, "")}, + {"torelativeminutenum", FuncRet(Type::i, "")}, {"torelativesecondsnum", FuncRet(Type::i, "")}, {"datediff", FuncRet(Type::d | Type::dt, "")}, + {"formatdatetime", FuncRet(Type::s, "")}, {"now", FuncRet(Type::dt | Type::d, "now()")}, {"today", FuncRet(Type::d | Type::dt, "today()")}, + {"yesterday", FuncRet(Type::d | Type::dt, "yesterday()")} }; std::set func_args_same_types = { @@ -226,23 +228,23 @@ std::set func_args_same_types = { }; std::map func_to_param_type = { - {"tostartofminute", type::dt}, {"plus", type::i | type::f | type::d | type::dt}, {"multiply", type::i | type::f}, - {"minus", type::i | type::f | type::d | type::dt}, {"negate", type::i | type::f}, {"divide", type::i | type::f}, - {"abs", type::i | type::f}, {"gcd", type::i | type::f}, {"lcm", type::i | type::f}, {"bitnot", type::i}, {"bitshiftleft", type::i}, - {"bitshiftright", type::i}, {"bittest", type::i}, {"exp", type::i | type::f}, {"log", type::i | type::f}, - {"exp2", type::i | type::f}, {"log2", type::i | type::f}, {"exp10", type::i | type::f}, {"log10", type::i | type::f}, - {"sqrt", type::i | type::f}, {"cbrt", type::i | type::f}, {"erf", type::i | type::f}, {"erfc", type::i | type::f}, - {"lgamma", type::i | type::f}, {"tgamma", type::i | type::f}, {"sin", type::i | type::f}, {"cos", type::i | type::f}, - {"tan", type::i | type::f}, {"asin", type::i | type::f}, {"acos", type::i | type::f}, {"atan", type::i | type::f}, - {"pow", type::i | type::f}, {"arrayjoin", type::all | type::a}, {"substring", type::s}, {"splitbystring", type::s}, {"splitbychar", type::s}, - {"alphatokens", type::s}, {"toyear", type::d | type::dt}, {"tomonth", type::d | type::dt}, {"todayofmonth", type::d | type::dt}, {"tohour", type::dt}, - {"tominute", type::dt}, {"tosecond", type::dt}, {"touixtimestamp", type::dt}, {"tostartofyear", type::d | type::dt}, - {"tostartofquarter", type::d | type::dt}, {"tostartofmonth", type::d | type::dt}, {"tomonday", type::d | type::dt}, - {"tostartoffiveminute", type::dt}, {"tostartoftenminutes", type::dt}, {"tostartoffifteenminutes", type::d | type::dt}, - {"tostartofinterval", type::d | type::dt}, {"totime", type::d | type::dt}, {"torelativehonthnum", type::d | type::dt}, - {"torelativeweeknum", type::d | type::dt}, {"torelativedaynum", type::d | type::dt}, {"torelativehournum", type::d | type::dt}, - {"torelativeminutenum", type::d | type::dt}, {"torelativesecondnum", type::d | type::dt}, {"datediff", type::d | type::dt}, - {"formatdatetime", type::dt} + {"tostartofminute", Type::dt}, {"plus", Type::i | Type::f | Type::d | Type::dt}, {"multiply", Type::i | Type::f}, + {"minus", Type::i | Type::f | Type::d | Type::dt}, {"negate", Type::i | Type::f}, {"divide", Type::i | Type::f}, + {"abs", Type::i | Type::f}, {"gcd", Type::i | Type::f}, {"lcm", Type::i | Type::f}, {"bitnot", Type::i}, {"bitshiftleft", Type::i}, + {"bitshiftright", Type::i}, {"bittest", Type::i}, {"exp", Type::i | Type::f}, {"log", Type::i | Type::f}, + {"exp2", Type::i | Type::f}, {"log2", Type::i | Type::f}, {"exp10", Type::i | Type::f}, {"log10", Type::i | Type::f}, + {"sqrt", Type::i | Type::f}, {"cbrt", Type::i | Type::f}, {"erf", Type::i | Type::f}, {"erfc", Type::i | Type::f}, + {"lgamma", Type::i | Type::f}, {"tgamma", Type::i | Type::f}, {"sin", Type::i | Type::f}, {"cos", Type::i | Type::f}, + {"tan", Type::i | Type::f}, {"asin", Type::i | Type::f}, {"acos", Type::i | Type::f}, {"atan", Type::i | Type::f}, + {"pow", Type::i | Type::f}, {"arrayjoin", Type::all | Type::a}, {"substring", Type::s}, {"splitbystring", Type::s}, {"splitbychar", Type::s}, + {"alphatokens", Type::s}, {"toyear", Type::d | Type::dt}, {"tomonth", Type::d | Type::dt}, {"todayofmonth", Type::d | Type::dt}, {"tohour", Type::dt}, + {"tominute", Type::dt}, {"tosecond", Type::dt}, {"touixtimestamp", Type::dt}, {"tostartofyear", Type::d | Type::dt}, + {"tostartofquarter", Type::d | Type::dt}, {"tostartofmonth", Type::d | Type::dt}, {"tomonday", Type::d | Type::dt}, + {"tostartoffiveminute", Type::dt}, {"tostartoftenminutes", Type::dt}, {"tostartoffifteenminutes", Type::d | Type::dt}, + {"tostartofinterval", Type::d | Type::dt}, {"totime", Type::d | Type::dt}, {"torelativehonthnum", Type::d | Type::dt}, + {"torelativeweeknum", Type::d | Type::dt}, {"torelativedaynum", Type::d | Type::dt}, {"torelativehournum", Type::d | Type::dt}, + {"torelativeminutenum", Type::d | Type::dt}, {"torelativesecondnum", Type::d | Type::dt}, {"datediff", Type::d | Type::dt}, + {"formatdatetime", Type::dt} }; @@ -252,7 +254,7 @@ public: TableAndColumn name; std::set equals; std::set values; - ColumnType type = type::all; + ColumnType type = Type::all; bool is_array = false; Column() = default; @@ -260,7 +262,7 @@ public: explicit Column(const std::string & column_name) { name = std::make_pair("", column_name); - type = type::all; + type = Type::all; } void merge(Column other) @@ -275,15 +277,15 @@ public: void printType() const { - if (type & type::i) + if (type & Type::i) std::cout << "I"; - if (type & type::f) + if (type & Type::f) std::cout << "F"; - if (type & type::s) + if (type & Type::s) std::cout << "S"; - if (type & type::d) + if (type & Type::d) std::cout << "D"; - if (type & type::dt) + if (type & Type::dt) std::cout << "DT"; if (is_array) std::cout << "ARR"; @@ -307,22 +309,22 @@ public: std::string generateOneValue() const { - if (type & type::i) + if (type & Type::i) return randomInteger(); - if (type & type::f) + if (type & Type::f) return randomFloat(); - if (type & type::d) + if (type & Type::d) return randomDate(); - if (type & type::dt) + if (type & Type::dt) return randomDatetime(); - if (type & type::s) + if (type & Type::s) return "'" + randomString(rng() % 40) + "'"; - if (type & type::b) + if (type & Type::b) return "0"; return ""; @@ -332,7 +334,7 @@ public: { if (values.size() > 2 && amount == 0) return false; - while (values.size() < 1 or amount > 0) + while (values.empty() or amount > 0) { amount -= 1; if (is_array) @@ -357,18 +359,18 @@ public: void unifyType() { - if (type & type::i) - type = type::i; - else if (type & type::f) - type = type::f; - else if (type & type::d) - type = type::d; - else if (type & type::dt) - type = type::dt; - else if (type & type::s) - type = type::s; - else if (type & type::b) - type = type::b; + if (type & Type::i) + type = Type::i; + else if (type & Type::f) + type = Type::f; + else if (type & Type::d) + type = Type::d; + else if (type & Type::dt) + type = Type::dt; + else if (type & Type::s) + type = Type::s; + else if (type & Type::b) + type = Type::b; else throw std::runtime_error("Error in determination column type " + name.first + '.' + name.second); } @@ -381,13 +383,15 @@ decartMul( std::set & mul) { std::set> result; - for (auto v : prev) - for (auto m : mul) + for (const auto & v : prev) + { + for (const auto & m : mul) { std::vector tmp = v; tmp.push_back(m); result.insert(tmp); } + } return result; } @@ -438,7 +442,7 @@ public: { name = other.name; columns.insert(other.columns.begin(), other.columns.end()); - for (auto desc : other.column_description) + for (const auto & desc : other.column_description) column_description[desc.first].merge(desc.second); } @@ -576,7 +580,7 @@ public: void merge(TableList other) { - for (auto table : other.tables) + for (const auto & table : other.tables) tables[table.first].merge(table.second); nested.insert(other.nested.begin(), other.nested.end()); if (main_table.empty()) @@ -617,14 +621,14 @@ FuncRet arrayJoinFunc(DB::ASTPtr ch, std::map & columns) for (const auto & indent : indents) { auto c = Column(indent); - c.type = type::all; + c.type = Type::all; c.is_array = true; if (columns.count(indent)) columns[indent].merge(c); else columns[indent] = c; } - FuncRet r(type::all, ""); + FuncRet r(Type::all, ""); return r; } return FuncRet(); @@ -637,7 +641,7 @@ FuncRet inFunc(DB::ASTPtr ch, std::map & columns) { std::set indents{}; std::set values{}; - ColumnType type_value = type::all; + ColumnType type_value = Type::all; for (auto & arg : x->arguments->children) { @@ -656,20 +660,20 @@ FuncRet inFunc(DB::ASTPtr ch, std::map & columns) for (auto & val : arr_values) { type = type_cast(val.getType()); - if (type == type::s || type == type::d || type == type::dt) + if (type == Type::s || type == Type::d || type == Type::dt) type = time_type(applyVisitor(DB::FieldVisitorToString(), val)); type_value &= type; values.insert(applyVisitor(DB::FieldVisitorToString(), val)); } }; - if (type & type::a) + if (type & Type::a) { auto arr_values = literal->value.get(); routine(arr_values); } - if (type & type::a) + if (type & Type::a) { auto arr_values = literal->value.get(); routine(arr_values); @@ -685,7 +689,7 @@ FuncRet inFunc(DB::ASTPtr ch, std::map & columns) else f = handlers[""]; FuncRet ret = f(arg, columns); - if (ret.value != "") + if (!ret.value.empty()) { values.insert(ret.value); } @@ -703,7 +707,7 @@ FuncRet inFunc(DB::ASTPtr ch, std::map & columns) else columns[indent] = c; } - FuncRet r(type::b | type::i, ""); + FuncRet r(Type::b | Type::i, ""); return r; } return FuncRet(); @@ -716,7 +720,7 @@ FuncRet arrayFunc(DB::ASTPtr ch, std::map & columns) { std::set indents = {}; std::string value = "["; - ColumnType type_value = type::i | type::f | type::d | type::dt | type::s; + ColumnType type_value = Type::i | Type::f | Type::d | Type::dt | Type::s; bool no_indent = true; for (const auto & arg : x->arguments->children) { @@ -730,7 +734,7 @@ FuncRet arrayFunc(DB::ASTPtr ch, std::map & columns) if (literal) { ColumnType type = type_cast(literal->value.getType()); - if (type == type::s || type == type::d || type == type::dt) + if (type == Type::s || type == Type::d || type == Type::dt) type = time_type(value); type_value &= type; @@ -764,7 +768,7 @@ FuncRet arithmeticFunc(DB::ASTPtr ch, std::map & columns) { std::set indents = {}; std::set values = {}; - ColumnType type_value = type::i | type::f | type::d | type::dt; + ColumnType type_value = Type::i | Type::f | Type::d | Type::dt; ColumnType args_types = 0; bool no_indent = true; for (auto & arg : x->arguments->children) @@ -793,11 +797,11 @@ FuncRet arithmeticFunc(DB::ASTPtr ch, std::map & columns) } args_types |= type; } - if (args_types & (type::d | type::dt)) - type_value -= type::f; - if (args_types & type::f) - type_value -= type::d | type::dt; - for (auto indent : indents) + if (args_types & (Type::d | Type::dt)) + type_value -= Type::f; + if (args_types & Type::f) + type_value -= Type::d | Type::dt; + for (const auto & indent : indents) { auto c = Column(indent); c.type = type_value; @@ -807,14 +811,14 @@ FuncRet arithmeticFunc(DB::ASTPtr ch, std::map & columns) columns[indent] = c; } ColumnType ret_type = 0; - if (args_types & type::dt) - ret_type = type::dt; - else if (args_types & type::d) - ret_type = type::d | type::dt; - else if (args_types & type::f) - ret_type = type::f; + if (args_types & Type::dt) + ret_type = Type::dt; + else if (args_types & Type::d) + ret_type = Type::d | Type::dt; + else if (args_types & Type::f) + ret_type = Type::f; else - ret_type = type::d | type::f | type::dt | type::i; + ret_type = Type::d | Type::f | Type::dt | Type::i; FuncRet r(ret_type, ""); if (no_indent) { @@ -833,7 +837,7 @@ FuncRet likeFunc(DB::ASTPtr ch, std::map & columns) { std::set indents = {}; std::set values = {}; - ColumnType type_value = type::s; + ColumnType type_value = Type::s; for (auto & arg : x->arguments->children) { auto ident = std::dynamic_pointer_cast(arg); @@ -866,7 +870,7 @@ FuncRet likeFunc(DB::ASTPtr ch, std::map & columns) else columns[indent] = c; } - FuncRet r(type::b, ""); + FuncRet r(Type::b, ""); return r; } return FuncRet(); @@ -874,22 +878,22 @@ FuncRet likeFunc(DB::ASTPtr ch, std::map & columns) FuncRet simpleFunc(DB::ASTPtr ch, std::map & columns) { - auto X = std::dynamic_pointer_cast(ch); - if (X) + auto x = std::dynamic_pointer_cast(ch); + if (x) { std::set indents = {}; std::set values = {}; - ColumnType type_value = type::all; + ColumnType type_value = Type::all; bool is_array = false; bool no_indent = true; - if (func_to_param_type.count(boost::algorithm::to_lower_copy(X->name))) + if (func_to_param_type.count(boost::algorithm::to_lower_copy(x->name))) { - type_value &= func_to_param_type[boost::algorithm::to_lower_copy(X->name)]; - is_array = func_to_param_type[boost::algorithm::to_lower_copy(X->name)] & type::a; + type_value &= func_to_param_type[boost::algorithm::to_lower_copy(x->name)]; + is_array = func_to_param_type[boost::algorithm::to_lower_copy(x->name)] & Type::a; } - for (auto arg : X->arguments->children) + for (const auto & arg : x->arguments->children) { - ColumnType type = type::all; + ColumnType type = Type::all; std::string value; auto ident = std::dynamic_pointer_cast(arg); if (ident) @@ -902,7 +906,7 @@ FuncRet simpleFunc(DB::ASTPtr ch, std::map & columns) { value = applyVisitor(DB::FieldVisitorToString(), literal->value); type = type_cast(literal->value.getType()); - is_array |= type & type::a; + is_array |= type & Type::a; } auto subfunc = std::dynamic_pointer_cast(arg); if (subfunc) @@ -922,31 +926,31 @@ FuncRet simpleFunc(DB::ASTPtr ch, std::map & columns) } if (!value.empty()) { - if (type == type::i) + if (type == Type::i) { values.insert(value); values.insert(value + " + " + randomInteger(1, 10)); values.insert(value + " - " + randomInteger(1, 10)); } - if (type == type::f) + if (type == Type::f) { values.insert(value); values.insert(value + " + " + randomFloat(1, 10)); values.insert(value + " - " + randomFloat(1, 10)); } - if (type & type::s || type & type::d || type & type::dt) + if (type & Type::s || type & Type::d || type & Type::dt) { - if (type == type::s) + if (type == Type::s) type = time_type(value); - if (type == type::s) + if (type == Type::s) values.insert(value); - if (type & type::d) + if (type & Type::d) { values.insert(value); values.insert("toDate(" + value + ") + " + randomInteger(1, 10)); values.insert("toDate(" + value + ") - " + randomInteger(1, 10)); } - else if (type & type::dt) + else if (type & Type::dt) { values.insert(value); values.insert( @@ -956,7 +960,7 @@ FuncRet simpleFunc(DB::ASTPtr ch, std::map & columns) } } } - if (func_args_same_types.count(boost::algorithm::to_lower_copy(X->name))) + if (func_args_same_types.count(boost::algorithm::to_lower_copy(x->name))) type_value &= type; } for (const auto & indent : indents) @@ -965,7 +969,7 @@ FuncRet simpleFunc(DB::ASTPtr ch, std::map & columns) c.type = type_value; c.is_array = is_array; if (func_args_same_types.count( - boost::algorithm::to_lower_copy(X->name))) + boost::algorithm::to_lower_copy(x->name))) c.values = values; for (const auto & ind : indents) if (ind != indent) @@ -976,31 +980,31 @@ FuncRet simpleFunc(DB::ASTPtr ch, std::map & columns) else columns[indent] = c; } - if (func_to_return_type.count(boost::algorithm::to_lower_copy(X->name))) + if (func_to_return_type.count(boost::algorithm::to_lower_copy(x->name))) { if (no_indent) { std::ostringstream ss; formatAST(*ch, ss); - auto r = func_to_return_type[boost::algorithm::to_lower_copy(X->name)]; + auto r = func_to_return_type[boost::algorithm::to_lower_copy(x->name)]; r.value = ss.str(); return r; } - return func_to_return_type[boost::algorithm::to_lower_copy(X->name)]; + return func_to_return_type[boost::algorithm::to_lower_copy(x->name)]; } else if (func_to_param_type.count( - boost::algorithm::to_lower_copy(X->name))) + boost::algorithm::to_lower_copy(x->name))) { if (no_indent) { std::ostringstream ss; formatAST(*ch, ss); return FuncRet( - func_to_param_type[boost::algorithm::to_lower_copy(X->name)], + func_to_param_type[boost::algorithm::to_lower_copy(x->name)], ss.str()); } return FuncRet( - func_to_param_type[boost::algorithm::to_lower_copy(X->name)], + func_to_param_type[boost::algorithm::to_lower_copy(x->name)], ""); } } @@ -1070,7 +1074,7 @@ connectedEqualityFind( std::set & visited) { std::set result; - for (auto & column : now.equals) + for (const auto & column : now.equals) if (!visited.count(column)) { visited.insert(column); @@ -1117,18 +1121,18 @@ unificateColumns( result[column.second.name.first + "." + column.second.name.second], result, visited); - for (auto c : equal) + for (const auto & c : equal) result[c.first + "." + c.second].equals = equal; } for (auto & column : result) - for (auto e : column.second.equals) + for (const auto & e : column.second.equals) column.second.merge(result[e.first + "." + e.second]); for (auto & column : result) { column.second.unifyType(); if (column.second.generateValues()) - for (auto e : column.second.equals) + for (const auto & e : column.second.equals) result[e.first + "." + e.second].merge(column.second); } @@ -1174,18 +1178,18 @@ void parseSelectQuery(DB::ASTPtr ast, TableList & all_tables) for (auto & child : x->children) { auto ch = std::dynamic_pointer_cast(child); - auto TEast = std::dynamic_pointer_cast(ch->table_expression); - if (TEast && TEast->database_and_table_name) + auto table_expression_ast = std::dynamic_pointer_cast(ch->table_expression); + if (table_expression_ast && table_expression_ast->database_and_table_name) { - auto table_name = *(getIndent(TEast->database_and_table_name).begin()); + auto table_name = *(getIndent(table_expression_ast->database_and_table_name).begin()); all_tables.addTable(table_name); auto alias = getAlias(ch); if (!alias.empty()) all_tables.aliases[alias] = table_name; } - if (TEast && TEast->subquery) + if (table_expression_ast && table_expression_ast->subquery) { - for (auto select : getSelect(TEast->subquery)) + for (const auto & select : getSelect(table_expression_ast->subquery)) { TableList local; parseSelectQuery(select, local); From 76147cc0c1aedfc4bc810af3287edd8adc9cbb76 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 19 Sep 2020 15:52:17 +0300 Subject: [PATCH 024/321] Generate ya.make --- src/Storages/ya.make | 2 ++ 1 file changed, 2 insertions(+) diff --git a/src/Storages/ya.make b/src/Storages/ya.make index bbc5acdfa7a..107433b5e73 100644 --- a/src/Storages/ya.make +++ b/src/Storages/ya.make @@ -20,6 +20,7 @@ SRCS( getStructureOfRemoteTable.cpp IndicesDescription.cpp IStorage.cpp + JoinSettings.cpp KeyDescription.cpp LiveView/StorageLiveView.cpp LiveView/TemporaryLiveViewCleaner.cpp @@ -108,6 +109,7 @@ SRCS( ReadInOrderOptimizer.cpp registerStorages.cpp SelectQueryDescription.cpp + SetSettings.cpp StorageBuffer.cpp StorageDictionary.cpp StorageDistributed.cpp From de540c6955f00acfa5d265020cdc116eeb996ec6 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Sat, 19 Sep 2020 15:59:37 +0300 Subject: [PATCH 025/321] minor fixes --- src/Core/Settings.h | 2 -- .../MergeTree/registerStorageMergeTree.cpp | 20 +++++-------------- 2 files changed, 5 insertions(+), 17 deletions(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index d0b81f5440e..b39c223a5e9 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -156,8 +156,6 @@ class IColumn; M(UInt64, max_concurrent_queries_for_user, 0, "The maximum number of concurrent requests per user.", 0) \ \ M(Bool, insert_deduplicate, true, "For INSERT queries in the replicated table, specifies that deduplication of insertings blocks should be performed", 0) \ - M(String, default_replica_path, "", "Default replica path for ReplicatedMergeTree. Allow to omit arguments for Replicated table engine if default_replica_path and default_replica_name are specified.", 0) \ - M(String, default_replica_name, "", "Default replica name for ReplicatedMergeTree. Allow to omit arguments for Replicated table engine if default_replica_path and default_replica_name are specified.", 0) \ \ M(UInt64, insert_quorum, 0, "For INSERT queries in the replicated table, wait writing for the specified number of replicas and linearize the addition of the data. 0 - disabled.", 0) \ M(Milliseconds, insert_quorum_timeout, 600000, "", 0) \ diff --git a/src/Storages/MergeTree/registerStorageMergeTree.cpp b/src/Storages/MergeTree/registerStorageMergeTree.cpp index a0ac364b76f..4deee54f092 100644 --- a/src/Storages/MergeTree/registerStorageMergeTree.cpp +++ b/src/Storages/MergeTree/registerStorageMergeTree.cpp @@ -273,13 +273,8 @@ static StoragePtr create(const StorageFactory::Arguments & args) String name_part = args.engine_name.substr(0, args.engine_name.size() - strlen("MergeTree")); bool replicated = startsWith(name_part, "Replicated"); - bool has_replicated_default_args = false; if (replicated) - { name_part = name_part.substr(strlen("Replicated")); - has_replicated_default_args = args.context.getSettingsRef().default_replica_path.value != "" - && args.context.getSettingsRef().default_replica_name.value != ""; - } MergeTreeData::MergingParams merging_params; merging_params.mode = MergeTreeData::MergingParams::Ordinary; @@ -323,7 +318,7 @@ static StoragePtr create(const StorageFactory::Arguments & args) if (replicated) { - if (is_extended_storage_def || has_replicated_default_args) + if (is_extended_storage_def) { add_optional_param("path in ZooKeeper"); add_optional_param("replica name"); @@ -426,18 +421,13 @@ static StoragePtr create(const StorageFactory::Arguments & args) "No replica name in config" + getMergeTreeVerboseHelp(is_extended_storage_def), ErrorCodes::NO_REPLICA_NAME_GIVEN); ++arg_num; } - else if (has_replicated_default_args && !has_arguments) - { - zookeeper_path - = args.context.getSettingsRef().default_replica_path.value + "/" + args.table_id.database_name + "." + args.table_id.table_name; - replica_name = args.context.getSettingsRef().default_replica_name; - } else if (is_extended_storage_def && !has_arguments) { /// Try use default values if arguments are not specified. - /// It works for ON CLUSTER queries when database engine is Atomic and there are {shard} and {replica} in config. - zookeeper_path = "/clickhouse/tables/{uuid}/{shard}"; - replica_name = "{replica}"; /// TODO maybe use hostname if {replica} is not defined? + /// Note: {uuid} macro works for ON CLUSTER queries when database engine is Atomic. + zookeeper_path = args.context.getConfigRef().getString("default_replica_path", "/clickhouse/tables/{uuid}/{shard}"); + /// TODO maybe use hostname if {replica} is not defined? + replica_name = args.context.getConfigRef().getString("default_replica_name", "{replica}"); } else throw Exception("Expected two string literal arguments: zookeper_path and replica_name", ErrorCodes::BAD_ARGUMENTS); From cd7354117c592711dfb46cbaea1e6244c3193e0d Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Sat, 19 Sep 2020 19:38:36 +0300 Subject: [PATCH 026/321] Update StorageJoin.cpp --- src/Storages/StorageJoin.cpp | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/src/Storages/StorageJoin.cpp b/src/Storages/StorageJoin.cpp index 1f8175099e4..33c67229cfa 100644 --- a/src/Storages/StorageJoin.cpp +++ b/src/Storages/StorageJoin.cpp @@ -119,7 +119,7 @@ void registerStorageJoin(StorageFactory & factory) auto join_overflow_mode = settings.join_overflow_mode; auto join_any_take_last_row = settings.join_any_take_last_row; auto old_any_join = settings.any_join_distinct_right_table_keys; - bool persistent_ = true; + bool persistent = true; if (args.storage_def && args.storage_def->settings) { @@ -141,7 +141,7 @@ void registerStorageJoin(StorageFactory & factory) { auto join_settings = std::make_unique(); join_settings->loadFromQuery(*args.storage_def); - persistent_ = join_settings->persistent; + persistent = join_settings->persistent; } else throw Exception( @@ -226,7 +226,7 @@ void registerStorageJoin(StorageFactory & factory) args.constraints, join_any_take_last_row, args.context, - persistent_); + persistent); }; factory.registerStorage("Join", creator_fn, StorageFactory::StorageFeatures{ .supports_settings = true, }); From a0ffdb1f456413ab3fb66f37fb962d8c0635b9ce Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Sat, 19 Sep 2020 19:39:24 +0300 Subject: [PATCH 027/321] Update 01493_storage_set_persistency.sql --- .../queries/0_stateless/01493_storage_set_persistency.sql | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/tests/queries/0_stateless/01493_storage_set_persistency.sql b/tests/queries/0_stateless/01493_storage_set_persistency.sql index 2ab5b409733..9298a24bf73 100644 --- a/tests/queries/0_stateless/01493_storage_set_persistency.sql +++ b/tests/queries/0_stateless/01493_storage_set_persistency.sql @@ -8,7 +8,7 @@ SELECT '----- Default Settings -----'; CREATE TABLE set (val UInt64) ENGINE = Set(); INSERT INTO set VALUES (1); DETACH TABLE set; -ATTACH TABLE set (val UInt64) ENGINE = Set(); +ATTACH TABLE set; SELECT number FROM number WHERE number IN set LIMIT 1; DROP TABLE set; @@ -17,7 +17,7 @@ SELECT '----- Settings persistent=1 -----'; CREATE TABLE set (val UInt64) ENGINE = Set() SETTINGS persistent=1; INSERT INTO set VALUES (1); DETACH TABLE set; -ATTACH TABLE set (val UInt64) ENGINE = Set() SETTINGS persistent=1; +ATTACH TABLE set; SELECT number FROM number WHERE number IN set LIMIT 1; DROP TABLE set; @@ -26,8 +26,8 @@ SELECT '----- Settings persistent=0 -----'; CREATE TABLE set (val UInt64) ENGINE = Set() SETTINGS persistent=0; INSERT INTO set VALUES (1); DETACH TABLE set; -ATTACH TABLE set (val UInt64) ENGINE = Set() SETTINGS persistent=0; +ATTACH TABLE set; SELECT number FROM number WHERE number IN set LIMIT 1; DROP TABLE set; -DROP TABLE number; \ No newline at end of file +DROP TABLE number; From 55544939b3f98f0dbe1d543500d0d1afee489a8c Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Sat, 19 Sep 2020 19:39:47 +0300 Subject: [PATCH 028/321] Update 01494_storage_join_persistency.sql --- .../queries/0_stateless/01494_storage_join_persistency.sql | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/tests/queries/0_stateless/01494_storage_join_persistency.sql b/tests/queries/0_stateless/01494_storage_join_persistency.sql index 4ca24db943a..c362a78dccd 100644 --- a/tests/queries/0_stateless/01494_storage_join_persistency.sql +++ b/tests/queries/0_stateless/01494_storage_join_persistency.sql @@ -4,7 +4,7 @@ SELECT '----- Default Settings -----'; CREATE TABLE join (k UInt64, s String) ENGINE = Join(ANY, LEFT, k); INSERT INTO join VALUES (1,21); DETACH TABLE join; -ATTACH TABLE join (k UInt64, s String) ENGINE = Join(ANY, LEFT, k); +ATTACH TABLE join; SELECT * from join; DROP TABLE join; @@ -13,7 +13,7 @@ SELECT '----- Settings persistent=1 -----'; CREATE TABLE join (k UInt64, s String) ENGINE = Join(ANY, LEFT, k) SETTINGS persistent=1; INSERT INTO join VALUES (1,21); DETACH TABLE join; -ATTACH TABLE join (k UInt64, s String) ENGINE = Join(ANY, LEFT, k) SETTINGS persistent=1; +ATTACH TABLE join; SELECT * from join; DROP TABLE join; @@ -22,7 +22,7 @@ SELECT '----- Settings persistent=0 -----'; CREATE TABLE join (k UInt64, s String) ENGINE = Join(ANY, LEFT, k) SETTINGS persistent=0; INSERT INTO join VALUES (1,21); DETACH TABLE join; -ATTACH TABLE join (k UInt64, s String) ENGINE = Join(ANY, LEFT, k) SETTINGS persistent=0; +ATTACH TABLE join; SELECT * from join; DROP TABLE join; From 3d535de9a56b69f9977d1bc4f9e1a2727065707d Mon Sep 17 00:00:00 2001 From: hustnn Date: Wed, 16 Sep 2020 16:32:37 +0800 Subject: [PATCH 029/321] Remove root path before adding to watches --- src/Common/ZooKeeper/ZooKeeperImpl.cpp | 17 ++++++++++++++++- 1 file changed, 16 insertions(+), 1 deletion(-) diff --git a/src/Common/ZooKeeper/ZooKeeperImpl.cpp b/src/Common/ZooKeeper/ZooKeeperImpl.cpp index 39a77fea0cf..c9c3aae67e5 100644 --- a/src/Common/ZooKeeper/ZooKeeperImpl.cpp +++ b/src/Common/ZooKeeper/ZooKeeperImpl.cpp @@ -422,6 +422,18 @@ void ZooKeeperRequest::write(WriteBuffer & out) const } +static void removeRootPath(String & path, const String & root_path) +{ + if (root_path.empty()) + return; + + if (path.size() <= root_path.size()) + throw Exception("Received path is not longer than root_path", Error::ZDATAINCONSISTENCY); + + path = path.substr(root_path.size()); +} + + struct ZooKeeperResponse : virtual Response { virtual ~ZooKeeperResponse() = default; @@ -1292,8 +1304,11 @@ void ZooKeeper::receiveEvent() if (add_watch) { + /// The key of wathces should exclude the root_path + String req_path = request_info.request->getPath(); + removeRootPath(req_path, root_path); std::lock_guard lock(watches_mutex); - watches[request_info.request->getPath()].emplace_back(std::move(request_info.watch)); + watches[req_path].emplace_back(std::move(request_info.watch)); } } From 09f6a454ae10bf0fc3940be05595b96dec9d7041 Mon Sep 17 00:00:00 2001 From: alesapin Date: Mon, 21 Sep 2020 10:17:58 +0300 Subject: [PATCH 030/321] Some intermediate stage --- src/DataTypes/IDataType.h | 3 +-- src/Storages/MergeTree/MergeTreeDataPartWriterWide.cpp | 10 +++++++--- 2 files changed, 8 insertions(+), 5 deletions(-) diff --git a/src/DataTypes/IDataType.h b/src/DataTypes/IDataType.h index 66485753060..88bffac9b22 100644 --- a/src/DataTypes/IDataType.h +++ b/src/DataTypes/IDataType.h @@ -99,8 +99,6 @@ public: /// Index of tuple element, starting at 1. String tuple_element_name; - bool is_specialized_codecs_allowed = true; - Substream(Type type_) : type(type_) {} }; @@ -444,6 +442,7 @@ public: static String getFileNameForStream(const String & column_name, const SubstreamPath & path); + static bool isNonGenericCompressionAllowedForStream(const SubstreamPath & path); private: friend class DataTypeFactory; /// Customize this DataType diff --git a/src/Storages/MergeTree/MergeTreeDataPartWriterWide.cpp b/src/Storages/MergeTree/MergeTreeDataPartWriterWide.cpp index 6515774d337..a7b8a45c3bc 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWriterWide.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartWriterWide.cpp @@ -1,5 +1,6 @@ #include #include +#include namespace DB { @@ -28,22 +29,25 @@ MergeTreeDataPartWriterWide::MergeTreeDataPartWriterWide( { const auto & columns = metadata_snapshot->getColumns(); for (const auto & it : columns_list) - addStreams(it.name, *it.type, columns.getCodecOrDefault(it.name, default_codec), settings.estimated_size); + addStreams(it.name, *it.type, columns.getCodecOrDefaultDesc(it.name, default_codec), default_codec, settings.estimated_size); } void MergeTreeDataPartWriterWide::addStreams( const String & name, const IDataType & type, - const CompressionCodecPtr & effective_codec, + const ASTPtr & effective_codec_desc, + const CompressionCodecPtr & default_codec, size_t estimated_size) { - IDataType::StreamCallback callback = [&] (const IDataType::SubstreamPath & substream_path, const IDataType & /* substream_type */) + IDataType::StreamCallback callback = [&] (const IDataType::SubstreamPath & substream_path, const IDataType & substream_type) { String stream_name = IDataType::getFileNameForStream(name, substream_path); /// Shared offsets for Nested type. if (column_streams.count(stream_name)) return; + auto compression_codec = CompressionCodecFactory::instance().get(effective_codec_desc, &substream_type, default_codec); + column_streams[stream_name] = std::make_unique( stream_name, data_part->volume->getDisk(), From 7fc5a2dae1e4a622d98c41613b2b3bb89168f297 Mon Sep 17 00:00:00 2001 From: alesapin Date: Mon, 21 Sep 2020 10:18:23 +0300 Subject: [PATCH 031/321] Some intermediate stage 2 --- src/Compression/CompressionFactory.h | 4 ++-- src/DataTypes/IDataType.cpp | 12 ++++++++++++ src/Storages/ColumnsDescription.cpp | 10 ++++++++++ src/Storages/ColumnsDescription.h | 1 + src/Storages/MergeTree/MergeTreeDataPartWriterWide.h | 3 ++- 5 files changed, 27 insertions(+), 3 deletions(-) diff --git a/src/Compression/CompressionFactory.h b/src/Compression/CompressionFactory.h index f30050f81ba..8dae34389b0 100644 --- a/src/Compression/CompressionFactory.h +++ b/src/Compression/CompressionFactory.h @@ -38,7 +38,7 @@ public: CompressionCodecPtr getDefaultCodec() const; /// Validate codecs AST specified by user and parses codecs description (substitute default parameters) - ASTPtr validateCodecAndGetPreprocessedAST(const ASTPtr & ast, DataTypePtr column_type, bool sanity_check) const; + ASTPtr validateCodecAndGetPreprocessedAST(const ASTPtr & ast, const IDataType * column_type, bool sanity_check) const; /// Validate codecs AST specified by user void validateCodec(const String & family_name, std::optional level, bool sanity_check) const; @@ -48,7 +48,7 @@ public: /// be able to work without information about type. Also AST can contain /// codec, which can be alias to current default codec, which can be changed /// in runtime. - CompressionCodecPtr get(const ASTPtr & ast, DataTypePtr column_type, CompressionCodecPtr current_default = nullptr) const; + CompressionCodecPtr get(const ASTPtr & ast, const IDataType * column_type, CompressionCodecPtr current_default = nullptr) const; /// Get codec by method byte (no params available) CompressionCodecPtr get(const uint8_t byte_code) const; diff --git a/src/DataTypes/IDataType.cpp b/src/DataTypes/IDataType.cpp index 561166cbc78..8b6fa7f35f8 100644 --- a/src/DataTypes/IDataType.cpp +++ b/src/DataTypes/IDataType.cpp @@ -130,6 +130,18 @@ String IDataType::getFileNameForStream(const String & column_name, const IDataTy } +bool IDataType::isNonGenericCompressionAllowedForStream(const SubstreamPath & path) +{ + for (const Substream & elem : path) + { + if (elem.type == Substream::NullMap + || elem.type == Substream::ArraySizes + || elem.type == Substream::DictionaryIndexes) + return false; + } + return true; +} + void IDataType::insertDefaultInto(IColumn & column) const { column.insertDefault(); diff --git a/src/Storages/ColumnsDescription.cpp b/src/Storages/ColumnsDescription.cpp index 6e4bc4dc80c..84d1ffb4602 100644 --- a/src/Storages/ColumnsDescription.cpp +++ b/src/Storages/ColumnsDescription.cpp @@ -426,6 +426,16 @@ CompressionCodecPtr ColumnsDescription::getCodecOrDefault(const String & column_ return getCodecOrDefault(column_name, CompressionCodecFactory::instance().getDefaultCodec()); } +ASTPtr ColumnsDescription::getCodecDescOrDefault(const String & column_name, CompressionCodecPtr default_codec) +{ + const auto it = columns.get<1>().find(column_name); + + if (it == columns.get<1>().end() || !it->codec) + return default_codec->getFullCodecDesc(); + + return it->codec; +} + ColumnsDescription::ColumnTTLs ColumnsDescription::getColumnTTLs() const { ColumnTTLs ret; diff --git a/src/Storages/ColumnsDescription.h b/src/Storages/ColumnsDescription.h index 6e2d3299091..6e22a2fdffc 100644 --- a/src/Storages/ColumnsDescription.h +++ b/src/Storages/ColumnsDescription.h @@ -115,6 +115,7 @@ public: bool hasCompressionCodec(const String & column_name) const; CompressionCodecPtr getCodecOrDefault(const String & column_name, CompressionCodecPtr default_codec) const; CompressionCodecPtr getCodecOrDefault(const String & column_name) const; + ASTPtr getCodecDescOrDefault(const String & column_name, CompressionCodecPtr default_codec); String toString() const; static ColumnsDescription parse(const String & str); diff --git a/src/Storages/MergeTree/MergeTreeDataPartWriterWide.h b/src/Storages/MergeTree/MergeTreeDataPartWriterWide.h index 02ab2a7ca56..678f8acd3e6 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWriterWide.h +++ b/src/Storages/MergeTree/MergeTreeDataPartWriterWide.h @@ -66,7 +66,8 @@ private: void addStreams( const String & name, const IDataType & type, - const CompressionCodecPtr & effective_codec, + const ASTPtr & effective_codec_desc, + const CompressionCodecPtr & default_codec, size_t estimated_size); SerializationStates serialization_states; From a3e999784b66f8c22e08cb45656ebdf236df2d46 Mon Sep 17 00:00:00 2001 From: alesapin Date: Mon, 21 Sep 2020 14:24:10 +0300 Subject: [PATCH 032/321] Buildable code --- src/Compression/CompressionCodecDelta.cpp | 4 ++-- src/Compression/CompressionCodecDoubleDelta.cpp | 4 ++-- src/Compression/CompressionCodecGorilla.cpp | 4 ++-- src/Compression/CompressionCodecT64.cpp | 4 ++-- src/Compression/CompressionFactory.cpp | 13 +++++-------- src/Compression/CompressionFactory.h | 12 ++++++++++-- src/Compression/ICompressionCodec.h | 1 - src/Storages/ColumnsDescription.cpp | 2 +- src/Storages/ColumnsDescription.h | 2 +- .../MergeTree/MergeTreeDataPartWriterWide.cpp | 5 ++--- .../MergeTree/MergeTreeDataPartWriterWide.h | 1 - 11 files changed, 27 insertions(+), 25 deletions(-) diff --git a/src/Compression/CompressionCodecDelta.cpp b/src/Compression/CompressionCodecDelta.cpp index a10d2589576..f3d6953bd2a 100644 --- a/src/Compression/CompressionCodecDelta.cpp +++ b/src/Compression/CompressionCodecDelta.cpp @@ -136,7 +136,7 @@ void CompressionCodecDelta::doDecompressData(const char * source, UInt32 source_ namespace { -UInt8 getDeltaBytesSize(DataTypePtr column_type) +UInt8 getDeltaBytesSize(const IDataType * column_type) { if (!column_type->isValueUnambiguouslyRepresentedInFixedSizeContiguousMemoryRegion()) throw Exception(ErrorCodes::BAD_ARGUMENTS, "Codec Delta is not applicable for {} because the data type is not of fixed size", @@ -155,7 +155,7 @@ UInt8 getDeltaBytesSize(DataTypePtr column_type) void registerCodecDelta(CompressionCodecFactory & factory) { UInt8 method_code = UInt8(CompressionMethodByte::Delta); - factory.registerCompressionCodecWithType("Delta", method_code, [&](const ASTPtr & arguments, DataTypePtr column_type) -> CompressionCodecPtr + factory.registerCompressionCodecWithType("Delta", method_code, [&](const ASTPtr & arguments, const IDataType * column_type) -> CompressionCodecPtr { UInt8 delta_bytes_size = 0; diff --git a/src/Compression/CompressionCodecDoubleDelta.cpp b/src/Compression/CompressionCodecDoubleDelta.cpp index 96fd29fe356..6895a80264d 100644 --- a/src/Compression/CompressionCodecDoubleDelta.cpp +++ b/src/Compression/CompressionCodecDoubleDelta.cpp @@ -307,7 +307,7 @@ void decompressDataForType(const char * source, UInt32 source_size, char * dest) } } -UInt8 getDataBytesSize(DataTypePtr column_type) +UInt8 getDataBytesSize(const IDataType * column_type) { if (!column_type->isValueUnambiguouslyRepresentedInFixedSizeContiguousMemoryRegion()) throw Exception(ErrorCodes::BAD_ARGUMENTS, "Codec DoubleDelta is not applicable for {} because the data type is not of fixed size", @@ -413,7 +413,7 @@ void registerCodecDoubleDelta(CompressionCodecFactory & factory) { UInt8 method_code = UInt8(CompressionMethodByte::DoubleDelta); factory.registerCompressionCodecWithType("DoubleDelta", method_code, - [&](const ASTPtr & arguments, DataTypePtr column_type) -> CompressionCodecPtr + [&](const ASTPtr & arguments, const IDataType * column_type) -> CompressionCodecPtr { if (arguments) throw Exception("Codec DoubleDelta does not accept any arguments", ErrorCodes::BAD_ARGUMENTS); diff --git a/src/Compression/CompressionCodecGorilla.cpp b/src/Compression/CompressionCodecGorilla.cpp index d739623a94b..582a9f41874 100644 --- a/src/Compression/CompressionCodecGorilla.cpp +++ b/src/Compression/CompressionCodecGorilla.cpp @@ -222,7 +222,7 @@ void decompressDataForType(const char * source, UInt32 source_size, char * dest) } } -UInt8 getDataBytesSize(DataTypePtr column_type) +UInt8 getDataBytesSize(const IDataType * column_type) { if (!column_type->isValueUnambiguouslyRepresentedInFixedSizeContiguousMemoryRegion()) throw Exception(ErrorCodes::BAD_ARGUMENTS, "Codec Gorilla is not applicable for {} because the data type is not of fixed size", @@ -329,7 +329,7 @@ void registerCodecGorilla(CompressionCodecFactory & factory) { UInt8 method_code = UInt8(CompressionMethodByte::Gorilla); factory.registerCompressionCodecWithType("Gorilla", method_code, - [&](const ASTPtr & arguments, DataTypePtr column_type) -> CompressionCodecPtr + [&](const ASTPtr & arguments, const IDataType * column_type) -> CompressionCodecPtr { if (arguments) throw Exception("Codec Gorilla does not accept any arguments", ErrorCodes::BAD_ARGUMENTS); diff --git a/src/Compression/CompressionCodecT64.cpp b/src/Compression/CompressionCodecT64.cpp index f081652f613..095610af615 100644 --- a/src/Compression/CompressionCodecT64.cpp +++ b/src/Compression/CompressionCodecT64.cpp @@ -136,7 +136,7 @@ TypeIndex baseType(TypeIndex type_idx) return TypeIndex::Nothing; } -TypeIndex typeIdx(const DataTypePtr & data_type) +TypeIndex typeIdx(const IDataType * data_type) { if (!data_type) return TypeIndex::Nothing; @@ -656,7 +656,7 @@ void CompressionCodecT64::updateHash(SipHash & hash) const void registerCodecT64(CompressionCodecFactory & factory) { - auto reg_func = [&](const ASTPtr & arguments, DataTypePtr type) -> CompressionCodecPtr + auto reg_func = [&](const ASTPtr & arguments, const IDataType * type) -> CompressionCodecPtr { Variant variant = Variant::Byte; diff --git a/src/Compression/CompressionFactory.cpp b/src/Compression/CompressionFactory.cpp index cebeee7c5a8..0f511aec96d 100644 --- a/src/Compression/CompressionFactory.cpp +++ b/src/Compression/CompressionFactory.cpp @@ -57,7 +57,7 @@ void CompressionCodecFactory::validateCodec(const String & family_name, std::opt } } -ASTPtr CompressionCodecFactory::validateCodecAndGetPreprocessedAST(const ASTPtr & ast, DataTypePtr column_type, bool sanity_check) const +ASTPtr CompressionCodecFactory::validateCodecAndGetPreprocessedAST(const ASTPtr & ast, const IDataType * column_type, bool sanity_check) const { if (const auto * func = ast->as()) { @@ -140,16 +140,13 @@ ASTPtr CompressionCodecFactory::validateCodecAndGetPreprocessedAST(const ASTPtr " (Note: you can enable setting 'allow_suspicious_codecs' to skip this check).", ErrorCodes::BAD_ARGUMENTS); } - std::shared_ptr result = std::make_shared(); - result->name = "CODEC"; - result->arguments = codecs_descriptions; - return result; + return ast; } throw Exception("Unknown codec family: " + queryToString(ast), ErrorCodes::UNKNOWN_CODEC); } -CompressionCodecPtr CompressionCodecFactory::get(const ASTPtr & ast, DataTypePtr column_type, CompressionCodecPtr current_default) const +CompressionCodecPtr CompressionCodecFactory::get(const ASTPtr & ast, const IDataType * column_type, CompressionCodecPtr current_default) const { if (current_default == nullptr) current_default = default_codec; @@ -203,7 +200,7 @@ CompressionCodecPtr CompressionCodecFactory::get(const uint8_t byte_code) const } -CompressionCodecPtr CompressionCodecFactory::getImpl(const String & family_name, const ASTPtr & arguments, DataTypePtr column_type) const +CompressionCodecPtr CompressionCodecFactory::getImpl(const String & family_name, const ASTPtr & arguments, const IDataType * column_type) const { if (family_name == "Multiple") throw Exception("Codec Multiple cannot be specified directly", ErrorCodes::UNKNOWN_CODEC); @@ -235,7 +232,7 @@ void CompressionCodecFactory::registerCompressionCodecWithType( void CompressionCodecFactory::registerCompressionCodec(const String & family_name, std::optional byte_code, Creator creator) { - registerCompressionCodecWithType(family_name, byte_code, [family_name, creator](const ASTPtr & ast, DataTypePtr /* data_type */) + registerCompressionCodecWithType(family_name, byte_code, [family_name, creator](const ASTPtr & ast, const IDataType * /* data_type */) { return creator(ast); }); diff --git a/src/Compression/CompressionFactory.h b/src/Compression/CompressionFactory.h index 8dae34389b0..784fb26ebbc 100644 --- a/src/Compression/CompressionFactory.h +++ b/src/Compression/CompressionFactory.h @@ -26,7 +26,7 @@ class CompressionCodecFactory final : private boost::noncopyable { protected: using Creator = std::function; - using CreatorWithType = std::function; + using CreatorWithType = std::function; using SimpleCreator = std::function; using CompressionCodecsDictionary = std::unordered_map; using CompressionCodecsCodeDictionary = std::unordered_map; @@ -39,6 +39,10 @@ public: /// Validate codecs AST specified by user and parses codecs description (substitute default parameters) ASTPtr validateCodecAndGetPreprocessedAST(const ASTPtr & ast, const IDataType * column_type, bool sanity_check) const; + ASTPtr validateCodecAndGetPreprocessedAST(const ASTPtr & ast, const DataTypePtr & column_type, bool sanity_check) const + { + return validateCodecAndGetPreprocessedAST(ast, column_type.get(), sanity_check); + } /// Validate codecs AST specified by user void validateCodec(const String & family_name, std::optional level, bool sanity_check) const; @@ -49,6 +53,10 @@ public: /// codec, which can be alias to current default codec, which can be changed /// in runtime. CompressionCodecPtr get(const ASTPtr & ast, const IDataType * column_type, CompressionCodecPtr current_default = nullptr) const; + CompressionCodecPtr get(const ASTPtr & ast, const DataTypePtr & column_type, CompressionCodecPtr current_default = nullptr) const + { + return get(ast, column_type.get(), current_default); + } /// Get codec by method byte (no params available) CompressionCodecPtr get(const uint8_t byte_code) const; @@ -65,7 +73,7 @@ public: void registerSimpleCompressionCodec(const String & family_name, std::optional byte_code, SimpleCreator creator); protected: - CompressionCodecPtr getImpl(const String & family_name, const ASTPtr & arguments, DataTypePtr column_type) const; + CompressionCodecPtr getImpl(const String & family_name, const ASTPtr & arguments, const IDataType * column_type) const; private: CompressionCodecsDictionary family_name_with_codec; diff --git a/src/Compression/ICompressionCodec.h b/src/Compression/ICompressionCodec.h index f3371d6d4b9..ff2197dbe2b 100644 --- a/src/Compression/ICompressionCodec.h +++ b/src/Compression/ICompressionCodec.h @@ -17,7 +17,6 @@ using CompressionCodecPtr = std::shared_ptr; using Codecs = std::vector; class IDataType; -using DataTypePtr = std::shared_ptr; /** * Represents interface for compression codecs like LZ4, ZSTD, etc. diff --git a/src/Storages/ColumnsDescription.cpp b/src/Storages/ColumnsDescription.cpp index 84d1ffb4602..48cde6b6aa9 100644 --- a/src/Storages/ColumnsDescription.cpp +++ b/src/Storages/ColumnsDescription.cpp @@ -426,7 +426,7 @@ CompressionCodecPtr ColumnsDescription::getCodecOrDefault(const String & column_ return getCodecOrDefault(column_name, CompressionCodecFactory::instance().getDefaultCodec()); } -ASTPtr ColumnsDescription::getCodecDescOrDefault(const String & column_name, CompressionCodecPtr default_codec) +ASTPtr ColumnsDescription::getCodecDescOrDefault(const String & column_name, CompressionCodecPtr default_codec) const { const auto it = columns.get<1>().find(column_name); diff --git a/src/Storages/ColumnsDescription.h b/src/Storages/ColumnsDescription.h index 6e22a2fdffc..8d1009b7263 100644 --- a/src/Storages/ColumnsDescription.h +++ b/src/Storages/ColumnsDescription.h @@ -115,7 +115,7 @@ public: bool hasCompressionCodec(const String & column_name) const; CompressionCodecPtr getCodecOrDefault(const String & column_name, CompressionCodecPtr default_codec) const; CompressionCodecPtr getCodecOrDefault(const String & column_name) const; - ASTPtr getCodecDescOrDefault(const String & column_name, CompressionCodecPtr default_codec); + ASTPtr getCodecDescOrDefault(const String & column_name, CompressionCodecPtr default_codec) const; String toString() const; static ColumnsDescription parse(const String & str); diff --git a/src/Storages/MergeTree/MergeTreeDataPartWriterWide.cpp b/src/Storages/MergeTree/MergeTreeDataPartWriterWide.cpp index a7b8a45c3bc..bce00715c08 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWriterWide.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartWriterWide.cpp @@ -29,14 +29,13 @@ MergeTreeDataPartWriterWide::MergeTreeDataPartWriterWide( { const auto & columns = metadata_snapshot->getColumns(); for (const auto & it : columns_list) - addStreams(it.name, *it.type, columns.getCodecOrDefaultDesc(it.name, default_codec), default_codec, settings.estimated_size); + addStreams(it.name, *it.type, columns.getCodecDescOrDefault(it.name, default_codec), settings.estimated_size); } void MergeTreeDataPartWriterWide::addStreams( const String & name, const IDataType & type, const ASTPtr & effective_codec_desc, - const CompressionCodecPtr & default_codec, size_t estimated_size) { IDataType::StreamCallback callback = [&] (const IDataType::SubstreamPath & substream_path, const IDataType & substream_type) @@ -53,7 +52,7 @@ void MergeTreeDataPartWriterWide::addStreams( data_part->volume->getDisk(), part_path + stream_name, DATA_FILE_EXTENSION, part_path + stream_name, marks_file_extension, - effective_codec, + compression_codec, settings.max_compress_block_size, estimated_size, settings.aio_threshold); diff --git a/src/Storages/MergeTree/MergeTreeDataPartWriterWide.h b/src/Storages/MergeTree/MergeTreeDataPartWriterWide.h index 678f8acd3e6..6dd3104c5b4 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWriterWide.h +++ b/src/Storages/MergeTree/MergeTreeDataPartWriterWide.h @@ -67,7 +67,6 @@ private: const String & name, const IDataType & type, const ASTPtr & effective_codec_desc, - const CompressionCodecPtr & default_codec, size_t estimated_size); SerializationStates serialization_states; From 4debccb7fef2b51fe862d02647ea85fb23f7006e Mon Sep 17 00:00:00 2001 From: alesapin Date: Mon, 21 Sep 2020 17:22:13 +0300 Subject: [PATCH 033/321] Almost working version --- src/Compression/CompressionCodecMultiple.cpp | 14 ------- src/Compression/CompressionCodecMultiple.h | 2 - src/Compression/CompressionFactory.cpp | 39 ++++++++++++++++--- src/Compression/CompressionFactory.h | 7 ++-- src/Compression/ICompressionCodec.cpp | 11 ------ src/Compression/ICompressionCodec.h | 2 - src/DataTypes/IDataType.cpp | 2 +- src/DataTypes/IDataType.h | 2 +- .../MergeTree/MergeTreeDataPartWriterWide.cpp | 17 +++++++- 9 files changed, 56 insertions(+), 40 deletions(-) diff --git a/src/Compression/CompressionCodecMultiple.cpp b/src/Compression/CompressionCodecMultiple.cpp index c9300288c0f..a0336d66a05 100644 --- a/src/Compression/CompressionCodecMultiple.cpp +++ b/src/Compression/CompressionCodecMultiple.cpp @@ -30,20 +30,6 @@ CompressionCodecMultiple::CompressionCodecMultiple(Codecs codecs_) setCodecDescription("", arguments); } - -CompressionCodecPtr CompressionCodecMultiple::filterNonGeneralCompressionCodecs(const CompressionCodecMultiple * codec) -{ - Codecs filtered; - for (const auto & subcodec : codec->codecs) - if (!subcodec->isGenericCompression()) - filtered.push_back(subcodec); - - if (filtered.empty()) - return nullptr; - - return std::make_shared(filtered); -} - uint8_t CompressionCodecMultiple::getMethodByte() const { return static_cast(CompressionMethodByte::Multiple); diff --git a/src/Compression/CompressionCodecMultiple.h b/src/Compression/CompressionCodecMultiple.h index c6f9f3b9d5b..1eb61842048 100644 --- a/src/Compression/CompressionCodecMultiple.h +++ b/src/Compression/CompressionCodecMultiple.h @@ -17,8 +17,6 @@ public: static std::vector getCodecsBytesFromData(const char * source); - static CompressionCodecPtr filterNonGeneralCompressionCodecs(const CompressionCodecMultiple * codec); - void updateHash(SipHash & hash) const override; protected: diff --git a/src/Compression/CompressionFactory.cpp b/src/Compression/CompressionFactory.cpp index 0f511aec96d..faa13b74a5c 100644 --- a/src/Compression/CompressionFactory.cpp +++ b/src/Compression/CompressionFactory.cpp @@ -99,7 +99,25 @@ ASTPtr CompressionCodecFactory::validateCodecAndGetPreprocessedAST(const ASTPtr } else { - result_codec = getImpl(codec_family_name, codec_arguments, column_type); + if (column_type) + { + IDataType::StreamCallback callback = [&](const IDataType::SubstreamPath & substream_path, const IDataType & substream_type) + { + if (IDataType::isSpecialCompressionAllowed(substream_path)) + result_codec = getImpl(codec_family_name, codec_arguments, &substream_type); + }; + + IDataType::SubstreamPath stream_path; + column_type->enumerateStreams(callback, stream_path); + + if (!result_codec) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot find any substream with data type for type {}. It's a bug", column_type->getName()); + } + else + { + result_codec = getImpl(codec_family_name, codec_arguments, nullptr); + } + codecs_descriptions->children.emplace_back(result_codec->getCodecDesc()); } @@ -140,13 +158,16 @@ ASTPtr CompressionCodecFactory::validateCodecAndGetPreprocessedAST(const ASTPtr " (Note: you can enable setting 'allow_suspicious_codecs' to skip this check).", ErrorCodes::BAD_ARGUMENTS); } - return ast; + std::shared_ptr result = std::make_shared(); + result->name = "CODEC"; + result->arguments = codecs_descriptions; + return result; } throw Exception("Unknown codec family: " + queryToString(ast), ErrorCodes::UNKNOWN_CODEC); } -CompressionCodecPtr CompressionCodecFactory::get(const ASTPtr & ast, const IDataType * column_type, CompressionCodecPtr current_default) const +CompressionCodecPtr CompressionCodecFactory::get(const ASTPtr & ast, const IDataType * column_type, CompressionCodecPtr current_default, bool only_generic) const { if (current_default == nullptr) current_default = default_codec; @@ -172,10 +193,16 @@ CompressionCodecPtr CompressionCodecFactory::get(const ASTPtr & ast, const IData else throw Exception("Unexpected AST element for compression codec", ErrorCodes::UNEXPECTED_AST_STRUCTURE); + CompressionCodecPtr codec; if (codec_family_name == DEFAULT_CODEC_NAME) - codecs.emplace_back(current_default); + codec = current_default; else - codecs.emplace_back(getImpl(codec_family_name, codec_arguments, column_type)); + codec = getImpl(codec_family_name, codec_arguments, column_type); + + if (only_generic && !codec->isGenericCompression()) + continue; + + codecs.emplace_back(codec); } CompressionCodecPtr res; @@ -184,6 +211,8 @@ CompressionCodecPtr CompressionCodecFactory::get(const ASTPtr & ast, const IData return codecs.back(); else if (codecs.size() > 1) return std::make_shared(codecs); + else + return nullptr; } throw Exception("Unexpected AST structure for compression codec: " + queryToString(ast), ErrorCodes::UNEXPECTED_AST_STRUCTURE); diff --git a/src/Compression/CompressionFactory.h b/src/Compression/CompressionFactory.h index 784fb26ebbc..ec32e712e7b 100644 --- a/src/Compression/CompressionFactory.h +++ b/src/Compression/CompressionFactory.h @@ -52,10 +52,10 @@ public: /// be able to work without information about type. Also AST can contain /// codec, which can be alias to current default codec, which can be changed /// in runtime. - CompressionCodecPtr get(const ASTPtr & ast, const IDataType * column_type, CompressionCodecPtr current_default = nullptr) const; - CompressionCodecPtr get(const ASTPtr & ast, const DataTypePtr & column_type, CompressionCodecPtr current_default = nullptr) const + CompressionCodecPtr get(const ASTPtr & ast, const IDataType * column_type, CompressionCodecPtr current_default = nullptr, bool only_generic = false) const; + CompressionCodecPtr get(const ASTPtr & ast, const DataTypePtr & column_type, CompressionCodecPtr current_default = nullptr, bool only_generic = false) const { - return get(ast, column_type.get(), current_default); + return get(ast, column_type.get(), current_default, only_generic); } /// Get codec by method byte (no params available) @@ -67,6 +67,7 @@ public: /// Register codec with parameters and column type void registerCompressionCodecWithType(const String & family_name, std::optional byte_code, CreatorWithType creator); /// Register codec with parameters + void registerCompressionCodec(const String & family_name, std::optional byte_code, Creator creator); /// Register codec without parameters diff --git a/src/Compression/ICompressionCodec.cpp b/src/Compression/ICompressionCodec.cpp index 9a501f5099d..baf6e9b2b86 100644 --- a/src/Compression/ICompressionCodec.cpp +++ b/src/Compression/ICompressionCodec.cpp @@ -129,15 +129,4 @@ uint8_t ICompressionCodec::readMethod(const char * source) return static_cast(source[0]); } - -CompressionCodecPtr tryGetGeneralCompressionCodecs(const CompressionCodecPtr & codec) -{ - if (codec->getMethodByte() == static_cast(CompressionMethodByte::Multiple)) - return CompressionCodecMultiple::filterNonGeneralCompressionCodecs(dynamic_cast(codec.get())); - else if (!codec->isGenericCompression()) - return nullptr; - else - return codec; -} - } diff --git a/src/Compression/ICompressionCodec.h b/src/Compression/ICompressionCodec.h index ff2197dbe2b..2018218f2e7 100644 --- a/src/Compression/ICompressionCodec.h +++ b/src/Compression/ICompressionCodec.h @@ -94,6 +94,4 @@ private: ASTPtr full_codec_desc; }; -CompressionCodecPtr tryGetGeneralCompressionCodecs(const CompressionCodecPtr & codec); - } diff --git a/src/DataTypes/IDataType.cpp b/src/DataTypes/IDataType.cpp index 8b6fa7f35f8..d1c9f1bde77 100644 --- a/src/DataTypes/IDataType.cpp +++ b/src/DataTypes/IDataType.cpp @@ -130,7 +130,7 @@ String IDataType::getFileNameForStream(const String & column_name, const IDataTy } -bool IDataType::isNonGenericCompressionAllowedForStream(const SubstreamPath & path) +bool IDataType::isSpecialCompressionAllowed(const SubstreamPath & path) { for (const Substream & elem : path) { diff --git a/src/DataTypes/IDataType.h b/src/DataTypes/IDataType.h index 88bffac9b22..63332a2048c 100644 --- a/src/DataTypes/IDataType.h +++ b/src/DataTypes/IDataType.h @@ -442,7 +442,7 @@ public: static String getFileNameForStream(const String & column_name, const SubstreamPath & path); - static bool isNonGenericCompressionAllowedForStream(const SubstreamPath & path); + static bool isSpecialCompressionAllowed(const SubstreamPath & path); private: friend class DataTypeFactory; /// Customize this DataType diff --git a/src/Storages/MergeTree/MergeTreeDataPartWriterWide.cpp b/src/Storages/MergeTree/MergeTreeDataPartWriterWide.cpp index bce00715c08..ffc824b18d6 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWriterWide.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartWriterWide.cpp @@ -45,7 +45,22 @@ void MergeTreeDataPartWriterWide::addStreams( if (column_streams.count(stream_name)) return; - auto compression_codec = CompressionCodecFactory::instance().get(effective_codec_desc, &substream_type, default_codec); + CompressionCodecPtr compression_codec; + if (IDataType::isSpecialCompressionAllowed(substream_path)) + { + compression_codec = CompressionCodecFactory::instance().get(effective_codec_desc, &substream_type, default_codec); + } + else + { + compression_codec = CompressionCodecFactory::instance().get(effective_codec_desc, nullptr, default_codec, false); + } + + if (compression_codec == nullptr) + throw Exception( + ErrorCodes::LOGICAL_ERROR, + "No generic compression provided for column {} with type {}", + backQuote(name), + type.getName()); column_streams[stream_name] = std::make_unique( stream_name, From 8517dda807b78e8ac46c4afdd0783e22399644b3 Mon Sep 17 00:00:00 2001 From: alesapin Date: Mon, 21 Sep 2020 17:47:10 +0300 Subject: [PATCH 034/321] First working test --- src/Compression/CompressionCodecT64.cpp | 2 +- .../MergeTree/MergeTreeDataPartWriterWide.cpp | 2 +- ...1504_compression_multiple_streams.reference | 1 + .../01504_compression_multiple_streams.sql | 18 ++++++++++++++++++ 4 files changed, 21 insertions(+), 2 deletions(-) create mode 100644 tests/queries/0_stateless/01504_compression_multiple_streams.reference create mode 100644 tests/queries/0_stateless/01504_compression_multiple_streams.sql diff --git a/src/Compression/CompressionCodecT64.cpp b/src/Compression/CompressionCodecT64.cpp index 095610af615..f0fb8351410 100644 --- a/src/Compression/CompressionCodecT64.cpp +++ b/src/Compression/CompressionCodecT64.cpp @@ -683,7 +683,7 @@ void registerCodecT64(CompressionCodecFactory & factory) auto type_idx = typeIdx(type); if (type && type_idx == TypeIndex::Nothing) - throw Exception("T64 codec is not supported for specified type", ErrorCodes::ILLEGAL_SYNTAX_FOR_CODEC_TYPE); + throw Exception("T64 codec is not supported for specified type " + type->getName(), ErrorCodes::ILLEGAL_SYNTAX_FOR_CODEC_TYPE); return std::make_shared(type_idx, variant); }; diff --git a/src/Storages/MergeTree/MergeTreeDataPartWriterWide.cpp b/src/Storages/MergeTree/MergeTreeDataPartWriterWide.cpp index ffc824b18d6..5da21aae1da 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWriterWide.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartWriterWide.cpp @@ -52,7 +52,7 @@ void MergeTreeDataPartWriterWide::addStreams( } else { - compression_codec = CompressionCodecFactory::instance().get(effective_codec_desc, nullptr, default_codec, false); + compression_codec = CompressionCodecFactory::instance().get(effective_codec_desc, nullptr, default_codec, true); } if (compression_codec == nullptr) diff --git a/tests/queries/0_stateless/01504_compression_multiple_streams.reference b/tests/queries/0_stateless/01504_compression_multiple_streams.reference new file mode 100644 index 00000000000..51018368774 --- /dev/null +++ b/tests/queries/0_stateless/01504_compression_multiple_streams.reference @@ -0,0 +1 @@ +1 1 [[1]] (1,[1]) diff --git a/tests/queries/0_stateless/01504_compression_multiple_streams.sql b/tests/queries/0_stateless/01504_compression_multiple_streams.sql new file mode 100644 index 00000000000..af586413d8d --- /dev/null +++ b/tests/queries/0_stateless/01504_compression_multiple_streams.sql @@ -0,0 +1,18 @@ +DROP TABLE IF EXISTS columns_with_multiple_streams; + +CREATE TABLE columns_with_multiple_streams ( + field0 Nullable(Int64) CODEC(Delta(2), LZ4), + field1 Nullable(Int64) CODEC(Delta, LZ4), + field2 Array(Array(Int64)) CODEC(Delta, LZ4), + filed3 Tuple(UInt32, Array(UInt64)) CODEC(T64, Default) +) +ENGINE = MergeTree +ORDER BY tuple() +SETTINGS min_rows_for_wide_part = 0, min_bytes_for_wide_part = 0; + +INSERT INTO columns_with_multiple_streams VALUES(1, 1, [[1]], tuple(1, [1])); + +SELECT * FROM columns_with_multiple_streams; + +DROP TABLE IF EXISTS columns_with_multiple_streams; + From e37099f65e7f88bef52d82169fdea92e3a4ea31c Mon Sep 17 00:00:00 2001 From: alesapin Date: Mon, 21 Sep 2020 18:45:51 +0300 Subject: [PATCH 035/321] Better test --- ...504_compression_multiple_streams.reference | 8 +++++++ .../01504_compression_multiple_streams.sql | 23 ++++++++++++++++++- 2 files changed, 30 insertions(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/01504_compression_multiple_streams.reference b/tests/queries/0_stateless/01504_compression_multiple_streams.reference index 51018368774..a21f562672e 100644 --- a/tests/queries/0_stateless/01504_compression_multiple_streams.reference +++ b/tests/queries/0_stateless/01504_compression_multiple_streams.reference @@ -1 +1,9 @@ 1 1 [[1]] (1,[1]) +1 1 [[1]] (1,[1]) +CREATE TABLE default.columns_with_multiple_streams\n(\n `field0` Nullable(Int64) CODEC(Delta(2), LZ4),\n `field1` Nullable(UInt8) CODEC(Delta(8), LZ4),\n `field2` Array(Array(Int64)) CODEC(Delta(8), LZ4),\n `field3` Tuple(UInt32, Array(UInt64)) CODEC(T64, Default)\n)\nENGINE = MergeTree\nORDER BY tuple()\nSETTINGS min_rows_for_wide_part = 0, min_bytes_for_wide_part = 0, index_granularity = 8192 +1 1 [[1]] (1,[1]) +2 2 [[2]] (2,[2]) +CREATE TABLE default.columns_with_multiple_streams\n(\n `field0` Nullable(Int64) CODEC(Delta(2), LZ4),\n `field1` Nullable(UInt8) CODEC(Delta(8), LZ4),\n `field2` Array(Array(Int64)) CODEC(Delta(8), LZ4),\n `field3` Tuple(UInt32, Array(UInt64)) CODEC(Delta(8), Default)\n)\nENGINE = MergeTree\nORDER BY tuple()\nSETTINGS min_rows_for_wide_part = 0, min_bytes_for_wide_part = 0, index_granularity = 8192 +1 1 [[1]] (1,[1]) +2 2 [[2]] (2,[2]) +3 3 [[3]] (3,[3]) diff --git a/tests/queries/0_stateless/01504_compression_multiple_streams.sql b/tests/queries/0_stateless/01504_compression_multiple_streams.sql index af586413d8d..72c669db426 100644 --- a/tests/queries/0_stateless/01504_compression_multiple_streams.sql +++ b/tests/queries/0_stateless/01504_compression_multiple_streams.sql @@ -4,7 +4,7 @@ CREATE TABLE columns_with_multiple_streams ( field0 Nullable(Int64) CODEC(Delta(2), LZ4), field1 Nullable(Int64) CODEC(Delta, LZ4), field2 Array(Array(Int64)) CODEC(Delta, LZ4), - filed3 Tuple(UInt32, Array(UInt64)) CODEC(T64, Default) + field3 Tuple(UInt32, Array(UInt64)) CODEC(T64, Default) ) ENGINE = MergeTree ORDER BY tuple() @@ -14,5 +14,26 @@ INSERT INTO columns_with_multiple_streams VALUES(1, 1, [[1]], tuple(1, [1])); SELECT * FROM columns_with_multiple_streams; +DETACH TABLE columns_with_multiple_streams; +ATTACH TABLE columns_with_multiple_streams; + +SELECT * FROM columns_with_multiple_streams; + +ALTER TABLE columns_with_multiple_streams MODIFY COLUMN field1 Nullable(UInt8); + +INSERT INTO columns_with_multiple_streams VALUES(2, 2, [[2]], tuple(2, [2])); + +SHOW CREATE TABLE columns_with_multiple_streams; + +SELECT * FROM columns_with_multiple_streams ORDER BY field0; + +ALTER TABLE columns_with_multiple_streams MODIFY COLUMN field3 CODEC(Delta, Default); + +SHOW CREATE TABLE columns_with_multiple_streams; + +INSERT INTO columns_with_multiple_streams VALUES(3, 3, [[3]], tuple(3, [3])); + +SELECT * FROM columns_with_multiple_streams ORDER BY field0; + DROP TABLE IF EXISTS columns_with_multiple_streams; From bcea1a420796a269739896df4750ff2a2722bb75 Mon Sep 17 00:00:00 2001 From: alesapin Date: Mon, 21 Sep 2020 18:55:21 +0300 Subject: [PATCH 036/321] Remove redundant change --- src/Compression/CompressionFactory.h | 1 - 1 file changed, 1 deletion(-) diff --git a/src/Compression/CompressionFactory.h b/src/Compression/CompressionFactory.h index ec32e712e7b..b13c1e314a5 100644 --- a/src/Compression/CompressionFactory.h +++ b/src/Compression/CompressionFactory.h @@ -67,7 +67,6 @@ public: /// Register codec with parameters and column type void registerCompressionCodecWithType(const String & family_name, std::optional byte_code, CreatorWithType creator); /// Register codec with parameters - void registerCompressionCodec(const String & family_name, std::optional byte_code, Creator creator); /// Register codec without parameters From fbd6f10ff7619bb67d79de4db8b94c608fd4a155 Mon Sep 17 00:00:00 2001 From: alesapin Date: Mon, 21 Sep 2020 19:07:14 +0300 Subject: [PATCH 037/321] Use default codec instead of throwing exception --- src/Storages/MergeTree/MergeTreeDataPartWriterWide.cpp | 6 +----- 1 file changed, 1 insertion(+), 5 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeDataPartWriterWide.cpp b/src/Storages/MergeTree/MergeTreeDataPartWriterWide.cpp index 5da21aae1da..d070a6bf2a5 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWriterWide.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartWriterWide.cpp @@ -56,11 +56,7 @@ void MergeTreeDataPartWriterWide::addStreams( } if (compression_codec == nullptr) - throw Exception( - ErrorCodes::LOGICAL_ERROR, - "No generic compression provided for column {} with type {}", - backQuote(name), - type.getName()); + compression_codec = CompressionCodecFactory::instance().getDefaultCodec(); column_streams[stream_name] = std::make_unique( stream_name, From e87b96109543def08cfb38cc51ec099ad080dc0a Mon Sep 17 00:00:00 2001 From: alesapin Date: Mon, 21 Sep 2020 20:35:09 +0300 Subject: [PATCH 038/321] Trying to add same logic for compact parts --- .../MergeTreeDataPartWriterCompact.cpp | 68 +++++++++++++++---- .../MergeTreeDataPartWriterCompact.h | 13 ++-- ...504_compression_multiple_streams.reference | 6 ++ .../01504_compression_multiple_streams.sql | 43 ++++++++++++ 4 files changed, 111 insertions(+), 19 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.cpp b/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.cpp index 235c76e744b..a8650d9f49b 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.cpp @@ -3,6 +3,7 @@ namespace DB { + MergeTreeDataPartWriterCompact::MergeTreeDataPartWriterCompact( const MergeTreeData::DataPartPtr & data_part_, const NamesAndTypesList & columns_list_, @@ -30,14 +31,41 @@ MergeTreeDataPartWriterCompact::MergeTreeDataPartWriterCompact( { const auto & storage_columns = metadata_snapshot->getColumns(); for (const auto & column : columns_list) - { - auto codec = storage_columns.getCodecOrDefault(column.name, default_codec); - auto & stream = streams_by_codec[codec->getHash()]; - if (!stream) - stream = std::make_shared(plain_hashing, codec); + addStreams(column.name, *column.type, storage_columns.getCodecDescOrDefault(column.name, default_codec)); +} - compressed_streams.push_back(stream); - } +void MergeTreeDataPartWriterCompact::addStreams(const String & name, const IDataType & type, const ASTPtr & effective_codec_desc) +{ + IDataType::StreamCallback callback = [&] (const IDataType::SubstreamPath & substream_path, const IDataType & substream_type) + { + String stream_name = IDataType::getFileNameForStream(name, substream_path); + + /// Shared offsets for Nested type. + if (compressed_streams.count(stream_name)) + return; + + CompressionCodecPtr compression_codec; + if (IDataType::isSpecialCompressionAllowed(substream_path)) + { + compression_codec = CompressionCodecFactory::instance().get(effective_codec_desc, &substream_type, default_codec); + } + else + { + compression_codec = CompressionCodecFactory::instance().get(effective_codec_desc, nullptr, default_codec, true); + } + + if (compression_codec == nullptr) + compression_codec = CompressionCodecFactory::instance().getDefaultCodec(); + UInt64 codec_id = compression_codec->getHash(); + auto & stream = streams_by_codec[codec_id]; + if (!stream) + stream = std::make_shared(codec_id, plain_hashing, compression_codec); + + compressed_streams.emplace(stream_name, stream); + }; + + IDataType::SubstreamPath stream_path; + type.enumerateStreams(callback, stream_path); } void MergeTreeDataPartWriterCompact::write( @@ -110,18 +138,27 @@ void MergeTreeDataPartWriterCompact::writeBlock(const Block & block) auto name_and_type = columns_list.begin(); for (size_t i = 0; i < columns_list.size(); ++i, ++name_and_type) { - auto & stream = compressed_streams[i]; + std::unordered_map used_streams; + auto stream_getter = [&, this](const IDataType::SubstreamPath & substream_path) -> WriteBuffer * + { + String stream_name = IDataType::getFileNameForStream(name_and_type->name, substream_path); + auto & result_stream = compressed_streams[stream_name]; - /// Offset should be 0, because compressed block is written for every granule. - assert(stream->hashing_buf.offset() == 0); + /// Offset should be 0, because compressed block is written for every granule. + if (used_streams.try_emplace(result_stream->codec_id, result_stream).second) + assert(result_stream->hashing_buf.offset() == 0); + + return &result_stream->hashing_buf; + }; writeIntBinary(plain_hashing.count(), marks); writeIntBinary(UInt64(0), marks); - writeColumnSingleGranule(block.getByName(name_and_type->name), stream, current_row, rows_to_write); + writeColumnSingleGranule(block.getByName(name_and_type->name), stream_getter, current_row, rows_to_write); /// Write one compressed block per column in granule for more optimal reading. - stream->hashing_buf.next(); + for (auto & [_, stream] : used_streams) + stream->hashing_buf.next(); } ++from_mark; @@ -145,13 +182,14 @@ void MergeTreeDataPartWriterCompact::writeBlock(const Block & block) void MergeTreeDataPartWriterCompact::writeColumnSingleGranule( const ColumnWithTypeAndName & column, - const CompressedStreamPtr & stream, - size_t from_row, size_t number_of_rows) + IDataType::OutputStreamGetter stream_getter, + size_t from_row, + size_t number_of_rows) { IDataType::SerializeBinaryBulkStatePtr state; IDataType::SerializeBinaryBulkSettings serialize_settings; - serialize_settings.getter = [&stream](IDataType::SubstreamPath) -> WriteBuffer * { return &stream->hashing_buf; }; + serialize_settings.getter = stream_getter; serialize_settings.position_independent_encoding = true; serialize_settings.low_cardinality_max_dictionary_size = 0; diff --git a/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.h b/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.h index 4beb0dba340..79dab1dacf0 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.h +++ b/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.h @@ -30,6 +30,8 @@ private: void addToChecksums(MergeTreeDataPartChecksums & checksums); + void addStreams(const String & name, const IDataType & type, const ASTPtr & effective_codec_desc); + Block header; /** Simplified SquashingTransform. The original one isn't suitable in this case @@ -54,11 +56,14 @@ private: struct CompressedStream { + UInt64 codec_id; CompressedWriteBuffer compressed_buf; HashingWriteBuffer hashing_buf; - CompressedStream(WriteBuffer & buf, const CompressionCodecPtr & codec) - : compressed_buf(buf, codec), hashing_buf(compressed_buf) {} + CompressedStream(UInt64 codec_id_, WriteBuffer & buf, const CompressionCodecPtr & codec) + : codec_id(codec_id_) + , compressed_buf(buf, codec) + , hashing_buf(compressed_buf) {} }; using CompressedStreamPtr = std::shared_ptr; @@ -67,7 +72,7 @@ private: std::unordered_map streams_by_codec; /// For better performance save pointer to stream by every column. - std::vector compressed_streams; + std::unordered_map compressed_streams; /// marks -> marks_file std::unique_ptr marks_file; @@ -76,7 +81,7 @@ private: /// Write single granule of one column (rows between 2 marks) static void writeColumnSingleGranule( const ColumnWithTypeAndName & column, - const CompressedStreamPtr & stream, + IDataType::OutputStreamGetter stream_getter, size_t from_row, size_t number_of_rows); }; diff --git a/tests/queries/0_stateless/01504_compression_multiple_streams.reference b/tests/queries/0_stateless/01504_compression_multiple_streams.reference index a21f562672e..aa222be0653 100644 --- a/tests/queries/0_stateless/01504_compression_multiple_streams.reference +++ b/tests/queries/0_stateless/01504_compression_multiple_streams.reference @@ -7,3 +7,9 @@ CREATE TABLE default.columns_with_multiple_streams\n(\n `field0` Nullable(Int 1 1 [[1]] (1,[1]) 2 2 [[2]] (2,[2]) 3 3 [[3]] (3,[3]) +1 1 [[1]] (1,[1]) +1 1 [[1]] (1,[1]) +CREATE TABLE default.columns_with_multiple_streams_compact\n(\n `field0` Nullable(Int64) CODEC(Delta(2), LZ4),\n `field1` Nullable(UInt8) CODEC(Delta(8), LZ4),\n `field2` Array(Array(Int64)) CODEC(Delta(8), LZ4),\n `field3` Tuple(UInt32, Array(UInt64)) CODEC(T64, Default)\n)\nENGINE = MergeTree\nORDER BY tuple()\nSETTINGS min_rows_for_wide_part = 100000, min_bytes_for_wide_part = 100000, index_granularity = 8192 +1 1 [[1]] (1,[1]) +2 2 [[2]] (2,[2]) +CREATE TABLE default.columns_with_multiple_streams_compact\n(\n `field0` Nullable(Int64) CODEC(Delta(2), LZ4),\n `field1` Nullable(UInt8) CODEC(Delta(8), LZ4),\n `field2` Array(Array(Int64)) CODEC(Delta(8), LZ4),\n `field3` Tuple(UInt32, Array(UInt64)) CODEC(Delta(8), Default)\n)\nENGINE = MergeTree\nORDER BY tuple()\nSETTINGS min_rows_for_wide_part = 100000, min_bytes_for_wide_part = 100000, index_granularity = 8192 diff --git a/tests/queries/0_stateless/01504_compression_multiple_streams.sql b/tests/queries/0_stateless/01504_compression_multiple_streams.sql index 72c669db426..3b02b4b030c 100644 --- a/tests/queries/0_stateless/01504_compression_multiple_streams.sql +++ b/tests/queries/0_stateless/01504_compression_multiple_streams.sql @@ -1,5 +1,7 @@ DROP TABLE IF EXISTS columns_with_multiple_streams; +SET mutations_sync = 2; + CREATE TABLE columns_with_multiple_streams ( field0 Nullable(Int64) CODEC(Delta(2), LZ4), field1 Nullable(Int64) CODEC(Delta, LZ4), @@ -33,7 +35,48 @@ SHOW CREATE TABLE columns_with_multiple_streams; INSERT INTO columns_with_multiple_streams VALUES(3, 3, [[3]], tuple(3, [3])); +OPTIMIZE TABLE columns_with_multiple_streams FINAL; + SELECT * FROM columns_with_multiple_streams ORDER BY field0; DROP TABLE IF EXISTS columns_with_multiple_streams; +CREATE TABLE columns_with_multiple_streams_compact ( + field0 Nullable(Int64) CODEC(Delta(2), LZ4), + field1 Nullable(Int64) CODEC(Delta, LZ4), + field2 Array(Array(Int64)) CODEC(Delta, LZ4), + field3 Tuple(UInt32, Array(UInt64)) CODEC(Delta, Default) +) +ENGINE = MergeTree +ORDER BY tuple() +SETTINGS min_rows_for_wide_part = 100000, min_bytes_for_wide_part = 100000; + +INSERT INTO columns_with_multiple_streams_compact VALUES(1, 1, [[1]], tuple(1, [1])); + +SELECT * FROM columns_with_multiple_streams_compact; + +DETACH TABLE columns_with_multiple_streams_compact; +ATTACH TABLE columns_with_multiple_streams_compact; + +SELECT * FROM columns_with_multiple_streams_compact; + +ALTER TABLE columns_with_multiple_streams_compact MODIFY COLUMN field1 Nullable(UInt8); + +INSERT INTO columns_with_multiple_streams_compact VALUES(2, 2, [[2]], tuple(2, [2])); + +SHOW CREATE TABLE columns_with_multiple_streams_compact; + +SELECT * FROM columns_with_multiple_streams_compact ORDER BY field0; + +ALTER TABLE columns_with_multiple_streams_compact MODIFY COLUMN field3 CODEC(Delta, Default); + +SELECT * FROM columns_with_multiple_streams_compact ORDER BY field0; + +SHOW CREATE TABLE columns_with_multiple_streams_compact; + +INSERT INTO columns_with_multiple_streams_compact VALUES(3, 3, [[3]], tuple(3, [3])); + +SELECT * FROM columns_with_multiple_streams_compact ORDER BY field0; + +DROP TABLE IF EXISTS columns_with_multiple_streams_compact; + From 3fd85430f2431af8b8ef11a1c80966a0820be6e5 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Mon, 21 Sep 2020 01:07:56 +0300 Subject: [PATCH 039/321] Set sanitizers options in the current shell too /etc/environment will not be read in current shell, so export variables from that file manually, to make clickhouse-local/clickhouse-client use them too. --- docker/test/base/Dockerfile | 7 ++++++- docker/test/stress/run.sh | 3 +++ 2 files changed, 9 insertions(+), 1 deletion(-) diff --git a/docker/test/base/Dockerfile b/docker/test/base/Dockerfile index aa3f1d738c2..506e32c18b3 100644 --- a/docker/test/base/Dockerfile +++ b/docker/test/base/Dockerfile @@ -48,10 +48,15 @@ RUN apt-get update \ tzdata \ --yes --no-install-recommends -# Sanitizer options +# Sanitizer options for services (clickhouse-server) RUN echo "TSAN_OPTIONS='verbosity=1000 halt_on_error=1 history_size=7'" >> /etc/environment; \ echo "UBSAN_OPTIONS='print_stacktrace=1'" >> /etc/environment; \ echo "MSAN_OPTIONS='abort_on_error=1'" >> /etc/environment; \ ln -s /usr/lib/llvm-${LLVM_VERSION}/bin/llvm-symbolizer /usr/bin/llvm-symbolizer; +# Sanitizer options for current shell (not current, but the one that will be spawned on "docker run") +# (but w/o verbosity for TSAN, otherwise test.reference will not match) +ENV TSAN_OPTIONS='halt_on_error=1 history_size=7' +ENV UBSAN_OPTIONS='print_stacktrace=1' +ENV MSAN_OPTIONS='abort_on_error=1' CMD sleep 1 diff --git a/docker/test/stress/run.sh b/docker/test/stress/run.sh index 8295e90b3ef..ad088ab493c 100755 --- a/docker/test/stress/run.sh +++ b/docker/test/stress/run.sh @@ -43,7 +43,10 @@ ln -s /usr/share/clickhouse-test/config/log_queries.xml /etc/clickhouse-server/u ln -s /usr/share/clickhouse-test/config/part_log.xml /etc/clickhouse-server/config.d/ ln -s /usr/share/clickhouse-test/config/text_log.xml /etc/clickhouse-server/config.d/ +# for clickhouse-server (via service) echo "ASAN_OPTIONS='malloc_context_size=10 verbosity=1 allocator_release_to_os_interval_ms=10000'" >> /etc/environment +# for clickhouse-client +export ASAN_OPTIONS='malloc_context_size=10 verbosity=1 allocator_release_to_os_interval_ms=10000' start From 35f1db0043d0deb82d65fb541df0bf6a9e9869ec Mon Sep 17 00:00:00 2001 From: hustnn Date: Thu, 10 Sep 2020 17:01:01 +0800 Subject: [PATCH 040/321] Fix potential memory leak caused by zookeeper exist calling --- src/Common/ZooKeeper/ZooKeeper.cpp | 7 ++++--- src/Common/ZooKeeper/ZooKeeperImpl.cpp | 21 +++++++++++++++++++-- src/Common/ZooKeeper/ZooKeeperImpl.h | 1 + src/Storages/MergeTree/LeaderElection.h | 3 ++- 4 files changed, 26 insertions(+), 6 deletions(-) diff --git a/src/Common/ZooKeeper/ZooKeeper.cpp b/src/Common/ZooKeeper/ZooKeeper.cpp index 41d715b23f1..bee875d1c74 100644 --- a/src/Common/ZooKeeper/ZooKeeper.cpp +++ b/src/Common/ZooKeeper/ZooKeeper.cpp @@ -663,7 +663,7 @@ bool ZooKeeper::waitForDisappear(const std::string & path, const WaitCondition & { WaitForDisappearStatePtr state = std::make_shared(); - auto callback = [state](const Coordination::ExistsResponse & response) + auto callback = [state](const Coordination::GetResponse & response) { state->code = int32_t(response.error); if (state->code) @@ -683,8 +683,9 @@ bool ZooKeeper::waitForDisappear(const std::string & path, const WaitCondition & while (!condition || !condition()) { - /// NOTE: if the node doesn't exist, the watch will leak. - impl->exists(path, callback, watch); + /// Use getData insteand of exists to avoid watch leak. + impl->get(path, callback, watch); + if (!condition) state->event.wait(); else if (!state->event.tryWait(1000)) diff --git a/src/Common/ZooKeeper/ZooKeeperImpl.cpp b/src/Common/ZooKeeper/ZooKeeperImpl.cpp index 2ee0c5aba17..a970867c095 100644 --- a/src/Common/ZooKeeper/ZooKeeperImpl.cpp +++ b/src/Common/ZooKeeper/ZooKeeperImpl.cpp @@ -1092,8 +1092,6 @@ void ZooKeeper::sendThread() { info.request->has_watch = true; CurrentMetrics::add(CurrentMetrics::ZooKeeperWatch); - std::lock_guard lock(watches_mutex); - watches[info.request->getPath()].emplace_back(std::move(info.watch)); } if (expired) @@ -1278,6 +1276,22 @@ void ZooKeeper::receiveEvent() response->removeRootPath(root_path); } + /// Find from operations + if (request_info.watch) + { + bool add_watch = false; + if (request_info.isExists) + add_watch = (response->error == Error::ZOK || response->error == Error::ZNONODE); + else + add_watch = response->error == Error::ZOK; + + if (add_watch) + { + std::lock_guard lock(watches_mutex); + watches[request_info.request->getPath()].emplace_back(std::move(request_info.watch)); + } + } + int32_t actual_length = in->count() - count_before_event; if (length != actual_length) throw Exception("Response length doesn't match. Expected: " + toString(length) + ", actual: " + toString(actual_length), Error::ZMARSHALLINGERROR); @@ -1545,6 +1559,7 @@ void ZooKeeper::exists( request_info.request = std::make_shared(std::move(request)); request_info.callback = [callback](const Response & response) { callback(dynamic_cast(response)); }; request_info.watch = watch; + request_info.isExists = true; pushRequest(std::move(request_info)); ProfileEvents::increment(ProfileEvents::ZooKeeperExists); @@ -1563,6 +1578,7 @@ void ZooKeeper::get( request_info.request = std::make_shared(std::move(request)); request_info.callback = [callback](const Response & response) { callback(dynamic_cast(response)); }; request_info.watch = watch; + request_info.isExists = false; pushRequest(std::move(request_info)); ProfileEvents::increment(ProfileEvents::ZooKeeperGet); @@ -1601,6 +1617,7 @@ void ZooKeeper::list( request_info.request = std::make_shared(std::move(request)); request_info.callback = [callback](const Response & response) { callback(dynamic_cast(response)); }; request_info.watch = watch; + request_info.isExists = false; pushRequest(std::move(request_info)); ProfileEvents::increment(ProfileEvents::ZooKeeperList); diff --git a/src/Common/ZooKeeper/ZooKeeperImpl.h b/src/Common/ZooKeeper/ZooKeeperImpl.h index 085b0e9856a..fb68e98355d 100644 --- a/src/Common/ZooKeeper/ZooKeeperImpl.h +++ b/src/Common/ZooKeeper/ZooKeeperImpl.h @@ -197,6 +197,7 @@ private: ResponseCallback callback; WatchCallback watch; clock::time_point time; + bool isExists; }; using RequestsQueue = ConcurrentBoundedQueue; diff --git a/src/Storages/MergeTree/LeaderElection.h b/src/Storages/MergeTree/LeaderElection.h index 89715085eda..a5f7ebce84f 100644 --- a/src/Storages/MergeTree/LeaderElection.h +++ b/src/Storages/MergeTree/LeaderElection.h @@ -124,7 +124,8 @@ private: /// Watch for the node in front of us. --my_node_it; - if (!zookeeper.existsWatch(path + "/" + *my_node_it, nullptr, task->getWatchCallback())) + std::string get_path_value; + if (!zookeeper.tryGetWatch(path + "/" + *my_node_it, get_path_value, nullptr, task->getWatchCallback())) task->schedule(); success = true; From 76e0147bf4a887be8b837a1b25d6aaa343abdca4 Mon Sep 17 00:00:00 2001 From: Niu Zhaojie Date: Tue, 15 Sep 2020 21:42:37 +0800 Subject: [PATCH 041/321] Use opt num to identify request type --- src/Common/ZooKeeper/ZooKeeperImpl.cpp | 12 +++++++----- src/Common/ZooKeeper/ZooKeeperImpl.h | 1 - 2 files changed, 7 insertions(+), 6 deletions(-) diff --git a/src/Common/ZooKeeper/ZooKeeperImpl.cpp b/src/Common/ZooKeeper/ZooKeeperImpl.cpp index a970867c095..8af1dde549c 100644 --- a/src/Common/ZooKeeper/ZooKeeperImpl.cpp +++ b/src/Common/ZooKeeper/ZooKeeperImpl.cpp @@ -1276,11 +1276,16 @@ void ZooKeeper::receiveEvent() response->removeRootPath(root_path); } - /// Find from operations + /// Instead of setting the watch in sendEvent, set it in receiveEvent becuase need to check the response. + /// The watch shouldn't be set if the node does not exist and it will never exist like sequential ephemeral nodes. + /// By using getData() instead of exists(), a watch won't be set if the node doesn't exist. if (request_info.watch) { bool add_watch = false; - if (request_info.isExists) + /// 3 indicates the ZooKeeperExistsRequest. + // For exists, we set the watch on both node exist and nonexist case. + // For other case like getData, we only set the watch when node exists. + if (request_info.request->getOpNum() == 3) add_watch = (response->error == Error::ZOK || response->error == Error::ZNONODE); else add_watch = response->error == Error::ZOK; @@ -1559,7 +1564,6 @@ void ZooKeeper::exists( request_info.request = std::make_shared(std::move(request)); request_info.callback = [callback](const Response & response) { callback(dynamic_cast(response)); }; request_info.watch = watch; - request_info.isExists = true; pushRequest(std::move(request_info)); ProfileEvents::increment(ProfileEvents::ZooKeeperExists); @@ -1578,7 +1582,6 @@ void ZooKeeper::get( request_info.request = std::make_shared(std::move(request)); request_info.callback = [callback](const Response & response) { callback(dynamic_cast(response)); }; request_info.watch = watch; - request_info.isExists = false; pushRequest(std::move(request_info)); ProfileEvents::increment(ProfileEvents::ZooKeeperGet); @@ -1617,7 +1620,6 @@ void ZooKeeper::list( request_info.request = std::make_shared(std::move(request)); request_info.callback = [callback](const Response & response) { callback(dynamic_cast(response)); }; request_info.watch = watch; - request_info.isExists = false; pushRequest(std::move(request_info)); ProfileEvents::increment(ProfileEvents::ZooKeeperList); diff --git a/src/Common/ZooKeeper/ZooKeeperImpl.h b/src/Common/ZooKeeper/ZooKeeperImpl.h index fb68e98355d..085b0e9856a 100644 --- a/src/Common/ZooKeeper/ZooKeeperImpl.h +++ b/src/Common/ZooKeeper/ZooKeeperImpl.h @@ -197,7 +197,6 @@ private: ResponseCallback callback; WatchCallback watch; clock::time_point time; - bool isExists; }; using RequestsQueue = ConcurrentBoundedQueue; From 3a59a7d5b690c4ac04dbcf853e5184f2ca9b7d00 Mon Sep 17 00:00:00 2001 From: hustnn Date: Wed, 16 Sep 2020 16:32:37 +0800 Subject: [PATCH 042/321] Remove root path before adding to watches --- src/Common/ZooKeeper/ZooKeeperImpl.cpp | 17 ++++++++++++++++- 1 file changed, 16 insertions(+), 1 deletion(-) diff --git a/src/Common/ZooKeeper/ZooKeeperImpl.cpp b/src/Common/ZooKeeper/ZooKeeperImpl.cpp index 8af1dde549c..abb8158781b 100644 --- a/src/Common/ZooKeeper/ZooKeeperImpl.cpp +++ b/src/Common/ZooKeeper/ZooKeeperImpl.cpp @@ -422,6 +422,18 @@ void ZooKeeperRequest::write(WriteBuffer & out) const } +static void removeRootPath(String & path, const String & root_path) +{ + if (root_path.empty()) + return; + + if (path.size() <= root_path.size()) + throw Exception("Received path is not longer than root_path", Error::ZDATAINCONSISTENCY); + + path = path.substr(root_path.size()); +} + + struct ZooKeeperResponse : virtual Response { virtual ~ZooKeeperResponse() override = default; @@ -1292,8 +1304,11 @@ void ZooKeeper::receiveEvent() if (add_watch) { + /// The key of wathces should exclude the root_path + String req_path = request_info.request->getPath(); + removeRootPath(req_path, root_path); std::lock_guard lock(watches_mutex); - watches[request_info.request->getPath()].emplace_back(std::move(request_info.watch)); + watches[req_path].emplace_back(std::move(request_info.watch)); } } From 96d06c6ae65ed5ec4edbb3b77fc05e2211601910 Mon Sep 17 00:00:00 2001 From: alesapin Date: Tue, 22 Sep 2020 15:16:15 +0300 Subject: [PATCH 043/321] Less changes --- .../MergeTreeDataPartWriterCompact.cpp | 23 +++++++++++-------- .../MergeTreeDataPartWriterCompact.h | 6 ++--- ...504_compression_multiple_streams.reference | 7 +++++- .../01504_compression_multiple_streams.sql | 2 ++ 4 files changed, 24 insertions(+), 14 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.cpp b/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.cpp index a8650d9f49b..0dae5342ce9 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.cpp @@ -59,7 +59,7 @@ void MergeTreeDataPartWriterCompact::addStreams(const String & name, const IData UInt64 codec_id = compression_codec->getHash(); auto & stream = streams_by_codec[codec_id]; if (!stream) - stream = std::make_shared(codec_id, plain_hashing, compression_codec); + stream = std::make_shared(plain_hashing, compression_codec); compressed_streams.emplace(stream_name, stream); }; @@ -138,27 +138,32 @@ void MergeTreeDataPartWriterCompact::writeBlock(const Block & block) auto name_and_type = columns_list.begin(); for (size_t i = 0; i < columns_list.size(); ++i, ++name_and_type) { - std::unordered_map used_streams; + CompressedStreamPtr prev_stream; auto stream_getter = [&, this](const IDataType::SubstreamPath & substream_path) -> WriteBuffer * { String stream_name = IDataType::getFileNameForStream(name_and_type->name, substream_path); - auto & result_stream = compressed_streams[stream_name]; - /// Offset should be 0, because compressed block is written for every granule. - if (used_streams.try_emplace(result_stream->codec_id, result_stream).second) + auto & result_stream = compressed_streams[stream_name]; + /// Write one compressed block per column in granule for more optimal reading. + if (prev_stream && prev_stream != result_stream) + { + /// Offset should be 0, because compressed block is written for every granule. assert(result_stream->hashing_buf.offset() == 0); + prev_stream->hashing_buf.next(); + } + + + prev_stream = result_stream; return &result_stream->hashing_buf; }; + writeIntBinary(plain_hashing.count(), marks); writeIntBinary(UInt64(0), marks); writeColumnSingleGranule(block.getByName(name_and_type->name), stream_getter, current_row, rows_to_write); - - /// Write one compressed block per column in granule for more optimal reading. - for (auto & [_, stream] : used_streams) - stream->hashing_buf.next(); + prev_stream->hashing_buf.next(); } ++from_mark; diff --git a/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.h b/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.h index 79dab1dacf0..4f7dcdb01bd 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.h +++ b/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.h @@ -56,13 +56,11 @@ private: struct CompressedStream { - UInt64 codec_id; CompressedWriteBuffer compressed_buf; HashingWriteBuffer hashing_buf; - CompressedStream(UInt64 codec_id_, WriteBuffer & buf, const CompressionCodecPtr & codec) - : codec_id(codec_id_) - , compressed_buf(buf, codec) + CompressedStream(WriteBuffer & buf, const CompressionCodecPtr & codec) + : compressed_buf(buf, codec) , hashing_buf(compressed_buf) {} }; diff --git a/tests/queries/0_stateless/01504_compression_multiple_streams.reference b/tests/queries/0_stateless/01504_compression_multiple_streams.reference index aa222be0653..da3ff3bfd9e 100644 --- a/tests/queries/0_stateless/01504_compression_multiple_streams.reference +++ b/tests/queries/0_stateless/01504_compression_multiple_streams.reference @@ -9,7 +9,12 @@ CREATE TABLE default.columns_with_multiple_streams\n(\n `field0` Nullable(Int 3 3 [[3]] (3,[3]) 1 1 [[1]] (1,[1]) 1 1 [[1]] (1,[1]) -CREATE TABLE default.columns_with_multiple_streams_compact\n(\n `field0` Nullable(Int64) CODEC(Delta(2), LZ4),\n `field1` Nullable(UInt8) CODEC(Delta(8), LZ4),\n `field2` Array(Array(Int64)) CODEC(Delta(8), LZ4),\n `field3` Tuple(UInt32, Array(UInt64)) CODEC(T64, Default)\n)\nENGINE = MergeTree\nORDER BY tuple()\nSETTINGS min_rows_for_wide_part = 100000, min_bytes_for_wide_part = 100000, index_granularity = 8192 +CREATE TABLE default.columns_with_multiple_streams_compact\n(\n `field0` Nullable(Int64) CODEC(Delta(2), LZ4),\n `field1` Nullable(UInt8) CODEC(Delta(8), LZ4),\n `field2` Array(Array(Int64)) CODEC(Delta(8), LZ4),\n `field3` Tuple(UInt32, Array(UInt64)) CODEC(Delta(8), Default)\n)\nENGINE = MergeTree\nORDER BY tuple()\nSETTINGS min_rows_for_wide_part = 100000, min_bytes_for_wide_part = 100000, index_granularity = 8192 +1 1 [[1]] (1,[1]) +2 2 [[2]] (2,[2]) 1 1 [[1]] (1,[1]) 2 2 [[2]] (2,[2]) CREATE TABLE default.columns_with_multiple_streams_compact\n(\n `field0` Nullable(Int64) CODEC(Delta(2), LZ4),\n `field1` Nullable(UInt8) CODEC(Delta(8), LZ4),\n `field2` Array(Array(Int64)) CODEC(Delta(8), LZ4),\n `field3` Tuple(UInt32, Array(UInt64)) CODEC(Delta(8), Default)\n)\nENGINE = MergeTree\nORDER BY tuple()\nSETTINGS min_rows_for_wide_part = 100000, min_bytes_for_wide_part = 100000, index_granularity = 8192 +1 1 [[1]] (1,[1]) +2 2 [[2]] (2,[2]) +3 3 [[3]] (3,[3]) diff --git a/tests/queries/0_stateless/01504_compression_multiple_streams.sql b/tests/queries/0_stateless/01504_compression_multiple_streams.sql index 3b02b4b030c..16eab7c14ec 100644 --- a/tests/queries/0_stateless/01504_compression_multiple_streams.sql +++ b/tests/queries/0_stateless/01504_compression_multiple_streams.sql @@ -41,6 +41,8 @@ SELECT * FROM columns_with_multiple_streams ORDER BY field0; DROP TABLE IF EXISTS columns_with_multiple_streams; +DROP TABLE IF EXISTS columns_with_multiple_streams_compact; + CREATE TABLE columns_with_multiple_streams_compact ( field0 Nullable(Int64) CODEC(Delta(2), LZ4), field1 Nullable(Int64) CODEC(Delta, LZ4), From fd394f699cf5c474c18ef4dea9343602692f5193 Mon Sep 17 00:00:00 2001 From: alesapin Date: Tue, 22 Sep 2020 15:49:55 +0300 Subject: [PATCH 044/321] Better comments --- src/Compression/CompressionFactory.cpp | 35 ++++++++++++++++--- src/Compression/CompressionFactory.h | 10 +++++- src/DataTypes/IDataType.h | 6 +++- .../MergeTreeDataPartWriterCompact.cpp | 16 ++++----- .../MergeTreeDataPartWriterCompact.h | 6 ++-- .../MergeTree/MergeTreeDataPartWriterWide.cpp | 10 ++---- ...504_compression_multiple_streams.reference | 6 ++-- 7 files changed, 59 insertions(+), 30 deletions(-) diff --git a/src/Compression/CompressionFactory.cpp b/src/Compression/CompressionFactory.cpp index faa13b74a5c..91b4aa4b8de 100644 --- a/src/Compression/CompressionFactory.cpp +++ b/src/Compression/CompressionFactory.cpp @@ -6,6 +6,7 @@ #include #include #include +#include #include #include @@ -67,6 +68,7 @@ ASTPtr CompressionCodecFactory::validateCodecAndGetPreprocessedAST(const ASTPtr bool has_none = false; std::optional generic_compression_codec_pos; + bool can_substitute_codec_arguments = true; for (size_t i = 0; i < func->arguments->children.size(); ++i) { const auto & inner_codec_ast = func->arguments->children[i]; @@ -101,10 +103,19 @@ ASTPtr CompressionCodecFactory::validateCodecAndGetPreprocessedAST(const ASTPtr { if (column_type) { + CompressionCodecPtr prev_codec; IDataType::StreamCallback callback = [&](const IDataType::SubstreamPath & substream_path, const IDataType & substream_type) { if (IDataType::isSpecialCompressionAllowed(substream_path)) + { result_codec = getImpl(codec_family_name, codec_arguments, &substream_type); + + /// Case for column Tuple, which compressed with codec which depends on data type, like Delta. + /// We cannot substitute parameters for such codecs. + if (prev_codec && prev_codec->getHash() != result_codec->getHash()) + can_substitute_codec_arguments = false; + prev_codec = result_codec; + } }; IDataType::SubstreamPath stream_path; @@ -158,10 +169,24 @@ ASTPtr CompressionCodecFactory::validateCodecAndGetPreprocessedAST(const ASTPtr " (Note: you can enable setting 'allow_suspicious_codecs' to skip this check).", ErrorCodes::BAD_ARGUMENTS); } - std::shared_ptr result = std::make_shared(); - result->name = "CODEC"; - result->arguments = codecs_descriptions; - return result; + /// For columns with nested types like Tuple(UInt32, UInt64) we + /// obviously cannot substitute parameters for codecs which depend on + /// data type, because for the first column Delta(4) is suitable and + /// Delta(8) for the second. So we should leave codec description as is + /// and deduce them in get method for each subtype separately. For all + /// other types it's better to substitute parameters, for better + /// readability and backward compatibility. + if (can_substitute_codec_arguments) + { + std::shared_ptr result = std::make_shared(); + result->name = "CODEC"; + result->arguments = codecs_descriptions; + return result; + } + else + { + return ast; + } } throw Exception("Unknown codec family: " + queryToString(ast), ErrorCodes::UNKNOWN_CODEC); @@ -212,7 +237,7 @@ CompressionCodecPtr CompressionCodecFactory::get(const ASTPtr & ast, const IData else if (codecs.size() > 1) return std::make_shared(codecs); else - return nullptr; + return std::make_shared(); } throw Exception("Unexpected AST structure for compression codec: " + queryToString(ast), ErrorCodes::UNEXPECTED_AST_STRUCTURE); diff --git a/src/Compression/CompressionFactory.h b/src/Compression/CompressionFactory.h index b13c1e314a5..8658f1a3f41 100644 --- a/src/Compression/CompressionFactory.h +++ b/src/Compression/CompressionFactory.h @@ -39,6 +39,8 @@ public: /// Validate codecs AST specified by user and parses codecs description (substitute default parameters) ASTPtr validateCodecAndGetPreprocessedAST(const ASTPtr & ast, const IDataType * column_type, bool sanity_check) const; + + /// Just wrapper for previous method. ASTPtr validateCodecAndGetPreprocessedAST(const ASTPtr & ast, const DataTypePtr & column_type, bool sanity_check) const { return validateCodecAndGetPreprocessedAST(ast, column_type.get(), sanity_check); @@ -51,8 +53,14 @@ public: /// information about type to improve inner settings, but every codec should /// be able to work without information about type. Also AST can contain /// codec, which can be alias to current default codec, which can be changed - /// in runtime. + /// in runtime. If only_generic is true than method will filter all + /// isGenericCompression() == false codecs from result. If nothing found + /// will return nullptr. It's useful for auxiliary parts of complex columns + /// like Nullable, Array and so on. If all codecs are non generic and + /// only_generic = true, than codec NONE will be returned. CompressionCodecPtr get(const ASTPtr & ast, const IDataType * column_type, CompressionCodecPtr current_default = nullptr, bool only_generic = false) const; + + /// Just wrapper for previous method. CompressionCodecPtr get(const ASTPtr & ast, const DataTypePtr & column_type, CompressionCodecPtr current_default = nullptr, bool only_generic = false) const { return get(ast, column_type.get(), current_default, only_generic); diff --git a/src/DataTypes/IDataType.h b/src/DataTypes/IDataType.h index 63332a2048c..5e25d47534e 100644 --- a/src/DataTypes/IDataType.h +++ b/src/DataTypes/IDataType.h @@ -104,7 +104,8 @@ public: using SubstreamPath = std::vector; - using StreamCallback = std::function; + using StreamCallback = std::function; + virtual void enumerateStreams(const StreamCallback & callback, SubstreamPath & path) const { callback(path, *this); @@ -442,6 +443,9 @@ public: static String getFileNameForStream(const String & column_name, const SubstreamPath & path); + /// Substream path supports special compression methods like codec Delta. + /// For all other substreams (like ArraySizes, NullMasks, etc.) we use only + /// generic compression codecs like LZ4. static bool isSpecialCompressionAllowed(const SubstreamPath & path); private: friend class DataTypeFactory; diff --git a/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.cpp b/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.cpp index 0dae5342ce9..8586c647073 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.cpp @@ -45,17 +45,13 @@ void MergeTreeDataPartWriterCompact::addStreams(const String & name, const IData return; CompressionCodecPtr compression_codec; - if (IDataType::isSpecialCompressionAllowed(substream_path)) - { - compression_codec = CompressionCodecFactory::instance().get(effective_codec_desc, &substream_type, default_codec); - } - else - { - compression_codec = CompressionCodecFactory::instance().get(effective_codec_desc, nullptr, default_codec, true); - } - if (compression_codec == nullptr) - compression_codec = CompressionCodecFactory::instance().getDefaultCodec(); + /// If we can use special codec than just get it + if (IDataType::isSpecialCompressionAllowed(substream_path)) + compression_codec = CompressionCodecFactory::instance().get(effective_codec_desc, &substream_type, default_codec); + else /// otherwise return only generic codecs and don't use info about data_type + compression_codec = CompressionCodecFactory::instance().get(effective_codec_desc, nullptr, default_codec, true); + UInt64 codec_id = compression_codec->getHash(); auto & stream = streams_by_codec[codec_id]; if (!stream) diff --git a/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.h b/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.h index 4f7dcdb01bd..a5a1a859e7a 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.h +++ b/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.h @@ -54,6 +54,7 @@ private: std::unique_ptr plain_file; HashingWriteBuffer plain_hashing; + /// Compressed stream which allows to write with codec. struct CompressedStream { CompressedWriteBuffer compressed_buf; @@ -66,10 +67,11 @@ private: using CompressedStreamPtr = std::shared_ptr; - /// Create compressed stream for every different codec. + /// Create compressed stream for every different codec. All streams write to + /// a single file on disk. std::unordered_map streams_by_codec; - /// For better performance save pointer to stream by every column. + /// Stream for each column's substreams path (look at addStreams). std::unordered_map compressed_streams; /// marks -> marks_file diff --git a/src/Storages/MergeTree/MergeTreeDataPartWriterWide.cpp b/src/Storages/MergeTree/MergeTreeDataPartWriterWide.cpp index d070a6bf2a5..03a4152a447 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWriterWide.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartWriterWide.cpp @@ -46,17 +46,11 @@ void MergeTreeDataPartWriterWide::addStreams( return; CompressionCodecPtr compression_codec; + /// If we can use special codec than just get it if (IDataType::isSpecialCompressionAllowed(substream_path)) - { compression_codec = CompressionCodecFactory::instance().get(effective_codec_desc, &substream_type, default_codec); - } - else - { + else /// otherwise return only generic codecs and don't use info about data_type compression_codec = CompressionCodecFactory::instance().get(effective_codec_desc, nullptr, default_codec, true); - } - - if (compression_codec == nullptr) - compression_codec = CompressionCodecFactory::instance().getDefaultCodec(); column_streams[stream_name] = std::make_unique( stream_name, diff --git a/tests/queries/0_stateless/01504_compression_multiple_streams.reference b/tests/queries/0_stateless/01504_compression_multiple_streams.reference index da3ff3bfd9e..21e40b5a5c6 100644 --- a/tests/queries/0_stateless/01504_compression_multiple_streams.reference +++ b/tests/queries/0_stateless/01504_compression_multiple_streams.reference @@ -3,18 +3,18 @@ CREATE TABLE default.columns_with_multiple_streams\n(\n `field0` Nullable(Int64) CODEC(Delta(2), LZ4),\n `field1` Nullable(UInt8) CODEC(Delta(8), LZ4),\n `field2` Array(Array(Int64)) CODEC(Delta(8), LZ4),\n `field3` Tuple(UInt32, Array(UInt64)) CODEC(T64, Default)\n)\nENGINE = MergeTree\nORDER BY tuple()\nSETTINGS min_rows_for_wide_part = 0, min_bytes_for_wide_part = 0, index_granularity = 8192 1 1 [[1]] (1,[1]) 2 2 [[2]] (2,[2]) -CREATE TABLE default.columns_with_multiple_streams\n(\n `field0` Nullable(Int64) CODEC(Delta(2), LZ4),\n `field1` Nullable(UInt8) CODEC(Delta(8), LZ4),\n `field2` Array(Array(Int64)) CODEC(Delta(8), LZ4),\n `field3` Tuple(UInt32, Array(UInt64)) CODEC(Delta(8), Default)\n)\nENGINE = MergeTree\nORDER BY tuple()\nSETTINGS min_rows_for_wide_part = 0, min_bytes_for_wide_part = 0, index_granularity = 8192 +CREATE TABLE default.columns_with_multiple_streams\n(\n `field0` Nullable(Int64) CODEC(Delta(2), LZ4),\n `field1` Nullable(UInt8) CODEC(Delta(8), LZ4),\n `field2` Array(Array(Int64)) CODEC(Delta(8), LZ4),\n `field3` Tuple(UInt32, Array(UInt64)) CODEC(Delta, Default)\n)\nENGINE = MergeTree\nORDER BY tuple()\nSETTINGS min_rows_for_wide_part = 0, min_bytes_for_wide_part = 0, index_granularity = 8192 1 1 [[1]] (1,[1]) 2 2 [[2]] (2,[2]) 3 3 [[3]] (3,[3]) 1 1 [[1]] (1,[1]) 1 1 [[1]] (1,[1]) -CREATE TABLE default.columns_with_multiple_streams_compact\n(\n `field0` Nullable(Int64) CODEC(Delta(2), LZ4),\n `field1` Nullable(UInt8) CODEC(Delta(8), LZ4),\n `field2` Array(Array(Int64)) CODEC(Delta(8), LZ4),\n `field3` Tuple(UInt32, Array(UInt64)) CODEC(Delta(8), Default)\n)\nENGINE = MergeTree\nORDER BY tuple()\nSETTINGS min_rows_for_wide_part = 100000, min_bytes_for_wide_part = 100000, index_granularity = 8192 +CREATE TABLE default.columns_with_multiple_streams_compact\n(\n `field0` Nullable(Int64) CODEC(Delta(2), LZ4),\n `field1` Nullable(UInt8) CODEC(Delta(8), LZ4),\n `field2` Array(Array(Int64)) CODEC(Delta(8), LZ4),\n `field3` Tuple(UInt32, Array(UInt64)) CODEC(Delta, Default)\n)\nENGINE = MergeTree\nORDER BY tuple()\nSETTINGS min_rows_for_wide_part = 100000, min_bytes_for_wide_part = 100000, index_granularity = 8192 1 1 [[1]] (1,[1]) 2 2 [[2]] (2,[2]) 1 1 [[1]] (1,[1]) 2 2 [[2]] (2,[2]) -CREATE TABLE default.columns_with_multiple_streams_compact\n(\n `field0` Nullable(Int64) CODEC(Delta(2), LZ4),\n `field1` Nullable(UInt8) CODEC(Delta(8), LZ4),\n `field2` Array(Array(Int64)) CODEC(Delta(8), LZ4),\n `field3` Tuple(UInt32, Array(UInt64)) CODEC(Delta(8), Default)\n)\nENGINE = MergeTree\nORDER BY tuple()\nSETTINGS min_rows_for_wide_part = 100000, min_bytes_for_wide_part = 100000, index_granularity = 8192 +CREATE TABLE default.columns_with_multiple_streams_compact\n(\n `field0` Nullable(Int64) CODEC(Delta(2), LZ4),\n `field1` Nullable(UInt8) CODEC(Delta(8), LZ4),\n `field2` Array(Array(Int64)) CODEC(Delta(8), LZ4),\n `field3` Tuple(UInt32, Array(UInt64)) CODEC(Delta, Default)\n)\nENGINE = MergeTree\nORDER BY tuple()\nSETTINGS min_rows_for_wide_part = 100000, min_bytes_for_wide_part = 100000, index_granularity = 8192 1 1 [[1]] (1,[1]) 2 2 [[2]] (2,[2]) 3 3 [[3]] (3,[3]) From 9b6b5ae08d286de5b7bedd742d463a0061a1c2ca Mon Sep 17 00:00:00 2001 From: alesapin Date: Tue, 22 Sep 2020 16:16:42 +0300 Subject: [PATCH 045/321] More tests --- .../__init__.py | 0 .../test_compression_nested_columns/test.py | 68 +++++++++++++++++++ ...504_compression_multiple_streams.reference | 3 + .../01504_compression_multiple_streams.sql | 32 +++++++++ 4 files changed, 103 insertions(+) create mode 100644 tests/integration/test_compression_nested_columns/__init__.py create mode 100644 tests/integration/test_compression_nested_columns/test.py diff --git a/tests/integration/test_compression_nested_columns/__init__.py b/tests/integration/test_compression_nested_columns/__init__.py new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/integration/test_compression_nested_columns/test.py b/tests/integration/test_compression_nested_columns/test.py new file mode 100644 index 00000000000..f73adadd770 --- /dev/null +++ b/tests/integration/test_compression_nested_columns/test.py @@ -0,0 +1,68 @@ +import random +import string + +import pytest +from helpers.cluster import ClickHouseCluster + +cluster = ClickHouseCluster(__file__) + +node1 = cluster.add_instance('node1', with_zookeeper=True) +node2 = cluster.add_instance('node2', with_zookeeper=True) + + +@pytest.fixture(scope="module") +def start_cluster(): + try: + cluster.start() + + yield cluster + finally: + cluster.shutdown() + +def get_compression_codec_byte(node, table_name, part_name, filename): + cmd = "tail -c +17 /var/lib/clickhouse/data/default/{}/{}/{}.bin | od -x -N 1 | head -n 1 | awk '{{print $2}}'".format( + table_name, part_name, filename) + return node.exec_in_container(["bash", "-c", cmd]).strip() + +CODECS_MAPPING = { + 'NONE' : '0002', + 'LZ4': '0082', + 'LZ4HC': '0082', # not an error, same byte + 'ZSTD': '0090', + 'Multiple': '0091', + 'Delta': '0092', + 'T64': '0093', +} + +def test_nested_compression_codec(start_cluster): + for i, node in enumerate([node1, node2]): + node.query(""" + CREATE TABLE compression_table ( + key UInt64, + column_ok Nullable(UInt64) CODEC(Delta, LZ4), + column_array Array(Array(UInt64)) CODEC(T64, LZ4), + column_bad LowCardinality(Int64) CODEC(Delta) + ) ENGINE = ReplicatedMergeTree('/t', '{}') ORDER BY tuple() PARTITION BY key + SETTINGS min_rows_for_wide_part = 0, min_bytes_for_wide_part = 0; + """.format(i), settings={"allow_suspicious_codecs" : "1", "allow_suspicious_low_cardinality_types" : "1"}) + + node1.query("INSERT INTO compression_table VALUES (1, 1, [[77]], 32)") + + node2.query("SYSTEM SYNC REPLICA compression_table", timeout=5) + + node1.query("DETACH TABLE compression_table") + node2.query("DETACH TABLE compression_table") + + node1.query("ATTACH TABLE compression_table") + node2.query("ATTACH TABLE compression_table") + + for node in [node1, node2]: + assert get_compression_codec_byte(node, "compression_table", "1_0_0_0", "column_ok") == CODECS_MAPPING['Multiple'] + assert get_compression_codec_byte(node, "compression_table", "1_0_0_0", "column_ok.null") == CODECS_MAPPING['LZ4'] + + assert get_compression_codec_byte(node1, "compression_table", "1_0_0_0", "column_array") == CODECS_MAPPING['Multiple'] + assert get_compression_codec_byte(node2, "compression_table", "1_0_0_0", "column_array.size0") == CODECS_MAPPING['LZ4'] + assert get_compression_codec_byte(node2, "compression_table", "1_0_0_0", "column_array.size1") == CODECS_MAPPING['LZ4'] + + assert get_compression_codec_byte(node2, "compression_table", "1_0_0_0", "column_bad.dict") == CODECS_MAPPING['Delta'] + assert get_compression_codec_byte(node1, "compression_table", "1_0_0_0", "column_bad") == CODECS_MAPPING['NONE'] diff --git a/tests/queries/0_stateless/01504_compression_multiple_streams.reference b/tests/queries/0_stateless/01504_compression_multiple_streams.reference index 21e40b5a5c6..4d3aba66526 100644 --- a/tests/queries/0_stateless/01504_compression_multiple_streams.reference +++ b/tests/queries/0_stateless/01504_compression_multiple_streams.reference @@ -18,3 +18,6 @@ CREATE TABLE default.columns_with_multiple_streams_compact\n(\n `field0` Null 1 1 [[1]] (1,[1]) 2 2 [[2]] (2,[2]) 3 3 [[3]] (3,[3]) +1 +2 +3 diff --git a/tests/queries/0_stateless/01504_compression_multiple_streams.sql b/tests/queries/0_stateless/01504_compression_multiple_streams.sql index 16eab7c14ec..7cdf1b52651 100644 --- a/tests/queries/0_stateless/01504_compression_multiple_streams.sql +++ b/tests/queries/0_stateless/01504_compression_multiple_streams.sql @@ -82,3 +82,35 @@ SELECT * FROM columns_with_multiple_streams_compact ORDER BY field0; DROP TABLE IF EXISTS columns_with_multiple_streams_compact; +DROP TABLE IF EXISTS columns_with_multiple_streams_bad_case; + +-- validation still works, non-sense codecs checked +CREATE TABLE columns_with_multiple_streams_bad_case ( + field0 Nullable(String) CODEC(Delta, LZ4) +) +ENGINE = MergeTree +ORDER BY tuple(); --{serverError 36} + +CREATE TABLE columns_with_multiple_streams_bad_case ( + field0 Tuple(Array(UInt64), String) CODEC(T64, LZ4) +) +ENGINE = MergeTree +ORDER BY tuple(); --{serverError 431} + +SET allow_suspicious_codecs = 1; + +CREATE TABLE columns_with_multiple_streams_bad_case ( + field0 Nullable(UInt64) CODEC(Delta) +) +ENGINE = MergeTree +ORDER BY tuple(); + +INSERT INTO columns_with_multiple_streams_bad_case VALUES(1), (2); + +INSERT INTO columns_with_multiple_streams_bad_case VALUES(3); + +OPTIMIZE TABLE columns_with_multiple_streams_bad_case FINAL; + +SELECT * FROM columns_with_multiple_streams_bad_case ORDER BY field0; + +DROP TABLE IF EXISTS columns_with_multiple_streams_bad_case; From 7015f272e79fedf3353c7f7fd14faa1fb4dd5924 Mon Sep 17 00:00:00 2001 From: alesapin Date: Tue, 22 Sep 2020 16:48:38 +0300 Subject: [PATCH 046/321] Better comment --- src/Storages/MergeTree/MergeTreeDataPartWriterCompact.cpp | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.cpp b/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.cpp index 8586c647073..7afeacd20aa 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.cpp @@ -134,6 +134,10 @@ void MergeTreeDataPartWriterCompact::writeBlock(const Block & block) auto name_and_type = columns_list.begin(); for (size_t i = 0; i < columns_list.size(); ++i, ++name_and_type) { + /// Tricky part, because we share compressed streams between different columns substreams. + /// Compressed streams write data to the single file, but with different compression codecs. + /// So we flush each stream (using next()) before using new one, because otherwise we will override + /// data in result file. CompressedStreamPtr prev_stream; auto stream_getter = [&, this](const IDataType::SubstreamPath & substream_path) -> WriteBuffer * { From d56a455ea5bbd947969ccd525e7b940e33730a60 Mon Sep 17 00:00:00 2001 From: alesapin Date: Tue, 22 Sep 2020 17:47:07 +0300 Subject: [PATCH 047/321] Remove outdated test --- .../queries/0_stateless/01380_coded_delta_exception_code.sql | 4 ---- 1 file changed, 4 deletions(-) diff --git a/tests/queries/0_stateless/01380_coded_delta_exception_code.sql b/tests/queries/0_stateless/01380_coded_delta_exception_code.sql index fc679e30e3f..587fac958cd 100644 --- a/tests/queries/0_stateless/01380_coded_delta_exception_code.sql +++ b/tests/queries/0_stateless/01380_coded_delta_exception_code.sql @@ -1,7 +1,3 @@ -CREATE TABLE delta_codec_synthetic (`id` UInt64 NULL CODEC(Delta, ZSTD(22))) ENGINE = MergeTree() ORDER BY tuple(); -- { serverError 36 } -CREATE TABLE delta_codec_synthetic (`id` UInt64 NULL CODEC(DoubleDelta, ZSTD(22))) ENGINE = MergeTree() ORDER BY tuple(); -- { serverError 36 } -CREATE TABLE delta_codec_synthetic (`id` UInt64 NULL CODEC(Gorilla, ZSTD(22))) ENGINE = MergeTree() ORDER BY tuple(); -- { serverError 36 } - CREATE TABLE delta_codec_synthetic (`id` Decimal(38, 10) CODEC(Delta, ZSTD(22))) ENGINE = MergeTree() ORDER BY tuple(); -- { serverError 36 } CREATE TABLE delta_codec_synthetic (`id` Decimal(38, 10) CODEC(DoubleDelta, ZSTD(22))) ENGINE = MergeTree() ORDER BY tuple(); -- { serverError 36 } CREATE TABLE delta_codec_synthetic (`id` Decimal(38, 10) CODEC(Gorilla, ZSTD(22))) ENGINE = MergeTree() ORDER BY tuple(); -- { serverError 36 } From 50eba27d1aa7784d51d9e5d49fa79c9934a906f3 Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 23 Sep 2020 10:24:25 +0300 Subject: [PATCH 048/321] Update CompressionFactory.h --- src/Compression/CompressionFactory.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Compression/CompressionFactory.h b/src/Compression/CompressionFactory.h index 8658f1a3f41..dc06e853898 100644 --- a/src/Compression/CompressionFactory.h +++ b/src/Compression/CompressionFactory.h @@ -55,7 +55,7 @@ public: /// codec, which can be alias to current default codec, which can be changed /// in runtime. If only_generic is true than method will filter all /// isGenericCompression() == false codecs from result. If nothing found - /// will return nullptr. It's useful for auxiliary parts of complex columns + /// will return codec NONE. It's useful for auxiliary parts of complex columns /// like Nullable, Array and so on. If all codecs are non generic and /// only_generic = true, than codec NONE will be returned. CompressionCodecPtr get(const ASTPtr & ast, const IDataType * column_type, CompressionCodecPtr current_default = nullptr, bool only_generic = false) const; From bb4b739f4322575109373e2c74faa1fae5ed454f Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 23 Sep 2020 13:11:48 +0300 Subject: [PATCH 049/321] Fix PVS-Studio warning --- src/Storages/MergeTree/MergeTreeDataPartWriterCompact.cpp | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.cpp b/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.cpp index 7afeacd20aa..2f413c76e54 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.cpp @@ -152,7 +152,6 @@ void MergeTreeDataPartWriterCompact::writeBlock(const Block & block) prev_stream->hashing_buf.next(); } - prev_stream = result_stream; return &result_stream->hashing_buf; @@ -163,6 +162,10 @@ void MergeTreeDataPartWriterCompact::writeBlock(const Block & block) writeIntBinary(UInt64(0), marks); writeColumnSingleGranule(block.getByName(name_and_type->name), stream_getter, current_row, rows_to_write); + + if (prev_stream == nullptr) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Not found any streams for column {} serialization. It's a bug", name_and_type->name); + prev_stream->hashing_buf.next(); } From 87b64baea500921c977eea5eb61244fa376c0fec Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 23 Sep 2020 16:21:06 +0300 Subject: [PATCH 050/321] Fix again --- src/Storages/MergeTree/MergeTreeDataPartWriterCompact.cpp | 6 ++---- 1 file changed, 2 insertions(+), 4 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.cpp b/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.cpp index 2f413c76e54..c81894ee36d 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.cpp @@ -163,10 +163,8 @@ void MergeTreeDataPartWriterCompact::writeBlock(const Block & block) writeColumnSingleGranule(block.getByName(name_and_type->name), stream_getter, current_row, rows_to_write); - if (prev_stream == nullptr) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Not found any streams for column {} serialization. It's a bug", name_and_type->name); - - prev_stream->hashing_buf.next(); + /// Each type always have at least one substream + prev_stream->hashing_buf.next(); //-V522 } ++from_mark; From dee34a834fd5749dfa8f49a53fab733c6850554c Mon Sep 17 00:00:00 2001 From: Alexander Kuzmenkov Date: Thu, 24 Sep 2020 14:46:03 +0300 Subject: [PATCH 051/321] debug perf test run time --- docker/test/performance-comparison/compare.sh | 52 ++++++++++++++++--- docker/test/performance-comparison/perf.py | 34 ++++++++++++ 2 files changed, 79 insertions(+), 7 deletions(-) diff --git a/docker/test/performance-comparison/compare.sh b/docker/test/performance-comparison/compare.sh index ddcc303da0d..bed6702ea4a 100755 --- a/docker/test/performance-comparison/compare.sh +++ b/docker/test/performance-comparison/compare.sh @@ -658,13 +658,15 @@ create view test_runs as group by test ; -create table test_times_report engine File(TSV, 'report/test-times.tsv') as - select wall_clock_time_per_test.test, real, - toDecimal64(total_client_time, 3), +create view test_times_view as + select + wall_clock_time_per_test.test test, + real, + total_client_time, queries, - toDecimal64(query_max, 3), - toDecimal64(real / queries, 3) avg_real_per_query, - toDecimal64(query_min, 3), + query_max, + real / queries avg_real_per_query, + query_min, runs from test_time -- wall clock times are also measured for skipped tests, so don't @@ -673,7 +675,43 @@ create table test_times_report engine File(TSV, 'report/test-times.tsv') as on wall_clock_time_per_test.test = test_time.test full join test_runs on test_runs.test = test_time.test - order by avg_real_per_query desc; + ; + +-- WITH TOTALS doesn't work with INSERT SELECT, so we have to jump through these +-- hoops: https://github.com/ClickHouse/ClickHouse/issues/15227 +create view test_times_view_total as + select + 'Total' test, + sum(real), + sum(total_client_time), + sum(queries), + max(query_max), + sum(real) / sum(queries) avg_real_per_query, + min(query_min), + -- Totaling the number of runs doesn't make sense, but use the max so + -- that the reporting script doesn't complain about queries being too + -- long. + max(runs) + from test_times_view + ; + +create table test_times_report engine File(TSV, 'report/test-times.tsv') as + select + test, + toDecimal64(real, 3), + toDecimal64(total_client_time, 3), + queries, + toDecimal64(query_max, 3), + toDecimal64(avg_real_per_query, 3), + toDecimal64(query_min, 3), + runs + from ( + select * from test_times_view + union all + select * from test_times_view_total + ) + order by test = 'Total' desc, avg_real_per_query desc + ; -- report for all queries page, only main metric create table all_tests_report engine File(TSV, 'report/all-queries.tsv') as diff --git a/docker/test/performance-comparison/perf.py b/docker/test/performance-comparison/perf.py index 2e047e6fb84..1bb1f6777dd 100755 --- a/docker/test/performance-comparison/perf.py +++ b/docker/test/performance-comparison/perf.py @@ -18,9 +18,22 @@ import xml.etree.ElementTree as et from threading import Thread from scipy import stats + +total_start_seconds = time.perf_counter() +stage_start_seconds = total_start_seconds + +def reportStageEnd(stage): + global stage_start_seconds, total_start_seconds + + current = time.perf_counter() + print(f'stage\t{stage}\t{current - stage_start_seconds:.3f}\t{current - total_start_seconds:.3f}') + stage_start_seconds = current + + def tsv_escape(s): return s.replace('\\', '\\\\').replace('\t', '\\t').replace('\n', '\\n').replace('\r','') + parser = argparse.ArgumentParser(description='Run performance test.') # Explicitly decode files as UTF-8 because sometimes we have Russian characters in queries, and LANG=C is set. parser.add_argument('file', metavar='FILE', type=argparse.FileType('r', encoding='utf-8'), nargs=1, help='test description file') @@ -33,11 +46,15 @@ parser.add_argument('--print-queries', action='store_true', help='Print test que parser.add_argument('--print-settings', action='store_true', help='Print test settings and exit.') args = parser.parse_args() +reportStageEnd('start') + test_name = os.path.splitext(os.path.basename(args.file[0].name))[0] tree = et.parse(args.file[0]) root = tree.getroot() +reportStageEnd('parse') + # Process query parameters subst_elems = root.findall('substitutions/substitution') available_parameters = {} # { 'table': ['hits_10m', 'hits_100m'], ... } @@ -109,6 +126,8 @@ if not args.long: if 'max_ignored_relative_change' in root.attrib: print(f'report-threshold\t{root.attrib["max_ignored_relative_change"]}') +reportStageEnd('before-connect') + # Open connections servers = [{'host': host, 'port': port} for (host, port) in zip(args.host, args.port)] all_connections = [clickhouse_driver.Client(**server) for server in servers] @@ -116,6 +135,8 @@ all_connections = [clickhouse_driver.Client(**server) for server in servers] for s in servers: print('server\t{}\t{}'.format(s['host'], s['port'])) +reportStageEnd('connect') + # Run drop queries, ignoring errors. Do this before all other activity, because # clickhouse_driver disconnects on error (this is not configurable), and the new # connection loses the changes in settings. @@ -129,6 +150,8 @@ for conn_index, c in enumerate(all_connections): except: pass +reportStageEnd('drop-1') + # Apply settings. # If there are errors, report them and continue -- maybe a new test uses a setting # that is not in master, but the queries can still run. If we have multiple @@ -146,6 +169,8 @@ for conn_index, c in enumerate(all_connections): except: print(traceback.format_exc(), file=sys.stderr) +reportStageEnd('settings') + # Check tables that should exist. If they don't exist, just skip this test. tables = [e.text for e in root.findall('preconditions/table_exists')] for t in tables: @@ -158,6 +183,8 @@ for t in tables: print(f'skipped\t{tsv_escape(skipped_message)}') sys.exit(0) +reportStageEnd('preconditions') + # Run create and fill queries. We will run them simultaneously for both servers, # to save time. # The weird search is to keep the relative order of elements, which matters, and @@ -188,6 +215,8 @@ for t in threads: for t in threads: t.join() +reportStageEnd('create') + # Run the queries in randomized order, but preserve their indexes as specified # in the test XML. To avoid using too much time, limit the number of queries # we run per test. @@ -342,9 +371,14 @@ for query_index in queries_to_run: break # And don't bother with short queries + +reportStageEnd('run') + # Run drop queries drop_queries = substitute_parameters(drop_query_templates) for conn_index, c in enumerate(all_connections): for q in drop_queries: c.execute(q) print(f'drop\t{conn_index}\t{c.last_query.elapsed}\t{tsv_escape(q)}') + +reportStageEnd('drop-2') From 646e8431052eec62465fe354c4826b924d2204ff Mon Sep 17 00:00:00 2001 From: alesapin Date: Thu, 24 Sep 2020 19:26:04 +0300 Subject: [PATCH 052/321] 100% lld-11 for clang-11 builds --- docker/packager/binary/Dockerfile | 6 +----- docker/packager/deb/Dockerfile | 13 ++----------- tests/ci/ci_config.json | 6 +++--- 3 files changed, 6 insertions(+), 19 deletions(-) diff --git a/docker/packager/binary/Dockerfile b/docker/packager/binary/Dockerfile index b563da04875..b96d53a6ea0 100644 --- a/docker/packager/binary/Dockerfile +++ b/docker/packager/binary/Dockerfile @@ -1,7 +1,7 @@ # docker build -t yandex/clickhouse-binary-builder . FROM ubuntu:20.04 -ENV DEBIAN_FRONTEND=noninteractive LLVM_VERSION=10 +ENV DEBIAN_FRONTEND=noninteractive LLVM_VERSION=11 RUN apt-get update \ && apt-get install ca-certificates lsb-release wget gnupg apt-transport-https \ @@ -32,10 +32,6 @@ RUN apt-get update \ curl \ gcc-9 \ g++-9 \ - llvm-${LLVM_VERSION} \ - clang-${LLVM_VERSION} \ - lld-${LLVM_VERSION} \ - clang-tidy-${LLVM_VERSION} \ clang-11 \ clang-tidy-11 \ lld-11 \ diff --git a/docker/packager/deb/Dockerfile b/docker/packager/deb/Dockerfile index a3c87f13fe4..df648ab97df 100644 --- a/docker/packager/deb/Dockerfile +++ b/docker/packager/deb/Dockerfile @@ -1,7 +1,7 @@ # docker build -t yandex/clickhouse-deb-builder . FROM ubuntu:20.04 -ENV DEBIAN_FRONTEND=noninteractive LLVM_VERSION=10 +ENV DEBIAN_FRONTEND=noninteractive LLVM_VERSION=11 RUN apt-get update \ && apt-get install ca-certificates lsb-release wget gnupg apt-transport-https \ @@ -49,15 +49,6 @@ RUN apt-get update \ lld-11 \ llvm-11 \ llvm-11-dev \ - clang-${LLVM_VERSION} \ - clang-tidy-${LLVM_VERSION} \ - lld-${LLVM_VERSION} \ - llvm-${LLVM_VERSION} \ - llvm-${LLVM_VERSION}-dev \ - llvm-9-dev \ - lld-9 \ - clang-9 \ - clang-tidy-9 \ ninja-build \ perl \ pkg-config \ @@ -84,7 +75,7 @@ RUN apt-get update \ RUN rm /etc/apt/sources.list.d/proposed-repositories.list && apt-get update # This symlink required by gcc to find lld compiler -RUN ln -s /usr/bin/lld-${LLVM_VERSION} /usr/bin/ld.lld +RUN ln -s /usr/bin/lld-11 /usr/bin/ld.lld COPY build.sh / diff --git a/tests/ci/ci_config.json b/tests/ci/ci_config.json index 220d8d801ec..e36932b9955 100644 --- a/tests/ci/ci_config.json +++ b/tests/ci/ci_config.json @@ -12,7 +12,7 @@ "with_coverage": false }, { - "compiler": "gcc-10", + "compiler": "clang-11", "build-type": "", "sanitizer": "", "package-type": "performance", @@ -72,7 +72,7 @@ "with_coverage": false }, { - "compiler": "clang-10", + "compiler": "clang-11", "build-type": "", "sanitizer": "", "package-type": "deb", @@ -102,7 +102,7 @@ "with_coverage": false }, { - "compiler": "clang-10", + "compiler": "clang-11", "build-type": "", "sanitizer": "", "package-type": "binary", From ab01fb36da20562bf41c298d31961b3917a59432 Mon Sep 17 00:00:00 2001 From: tavplubix Date: Thu, 24 Sep 2020 20:05:13 +0300 Subject: [PATCH 053/321] Update ProcessList.cpp --- src/Interpreters/ProcessList.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Interpreters/ProcessList.cpp b/src/Interpreters/ProcessList.cpp index d86b5678f6d..cb9a75f26ba 100644 --- a/src/Interpreters/ProcessList.cpp +++ b/src/Interpreters/ProcessList.cpp @@ -431,7 +431,7 @@ QueryStatusInfo QueryStatus::getInfo(bool get_thread_list, bool get_profile_even } if (get_settings && query_context) - res.query_settings = std::make_shared(query_context->getSettingsRef()); + res.query_settings = std::make_shared(query_context->getSettings()); return res; } From 5ff5ca1fa6306f602aa8ba09e6884854313cc4e3 Mon Sep 17 00:00:00 2001 From: tavplubix Date: Thu, 24 Sep 2020 20:08:15 +0300 Subject: [PATCH 054/321] Update Context.cpp --- src/Interpreters/Context.cpp | 1 + 1 file changed, 1 insertion(+) diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index 704b21f3a4a..a3e0e306961 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -968,6 +968,7 @@ StoragePtr Context::getViewSource() Settings Context::getSettings() const { + auto lock = getLock(); return settings; } From 87726a7af243a7319902db91ad9e526fc41c23f4 Mon Sep 17 00:00:00 2001 From: feng lv Date: Fri, 25 Sep 2020 15:20:42 +0800 Subject: [PATCH 055/321] function allow an argument with timezone --- src/Functions/now.cpp | 39 ++++++++++++++++++++++++++++++++++----- 1 file changed, 34 insertions(+), 5 deletions(-) diff --git a/src/Functions/now.cpp b/src/Functions/now.cpp index e71c3a77f11..f78db91cc58 100644 --- a/src/Functions/now.cpp +++ b/src/Functions/now.cpp @@ -5,11 +5,19 @@ #include #include +#include + #include namespace DB { +namespace ErrorCodes +{ + extern const int ILLEGAL_TYPE_OF_ARGUMENT; + extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; +} + namespace { @@ -35,7 +43,7 @@ private: class FunctionBaseNow : public IFunctionBaseImpl { public: - explicit FunctionBaseNow(time_t time_) : time_value(time_), return_type(std::make_shared()) {} + explicit FunctionBaseNow(time_t time_, DataTypePtr return_type_) : time_value(time_), return_type(return_type_) {} String getName() const override { return "now"; } @@ -72,14 +80,35 @@ public: bool isDeterministic() const override { return false; } + bool isVariadic() const override { return true; } + size_t getNumberOfArguments() const override { return 0; } static FunctionOverloadResolverImplPtr create(const Context &) { return std::make_unique(); } - DataTypePtr getReturnType(const DataTypes &) const override { return std::make_shared(); } - - FunctionBaseImplPtr build(const ColumnsWithTypeAndName &, const DataTypePtr &) const override + DataTypePtr getReturnType(const ColumnsWithTypeAndName & arguments) const override { - return std::make_unique(time(nullptr)); + if (arguments.size() > 1) + { + throw Exception("Arguments size of function " + getName() + " should be 0 or 1", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); + } + if (arguments.size() == 1 && !isStringOrFixedString(arguments[0].type)) + { + throw Exception( + "Arguments of function " + getName() + " should be String or FixedString", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + } + if (arguments.size() == 1) + { + return std::make_shared(extractTimeZoneNameFromFunctionArguments(arguments, 0, 0)); + } + return std::make_shared(); + } + + FunctionBaseImplPtr build(const ColumnsWithTypeAndName & arguments, const DataTypePtr &) const override + { + if (arguments.size() == 1) + return std::make_unique( + time(nullptr), std::make_shared(extractTimeZoneNameFromFunctionArguments(arguments, 0, 0))); + return std::make_unique(time(nullptr), std::make_shared()); } }; From 92f7847b1d1546577bd4eb3901b34a406d83631b Mon Sep 17 00:00:00 2001 From: feng lv Date: Fri, 25 Sep 2020 16:00:51 +0800 Subject: [PATCH 056/321] add test --- tests/queries/0_stateless/01497_now_support_timezone.reference | 1 + tests/queries/0_stateless/01497_now_support_timezone.sql | 1 + 2 files changed, 2 insertions(+) create mode 100644 tests/queries/0_stateless/01497_now_support_timezone.reference create mode 100644 tests/queries/0_stateless/01497_now_support_timezone.sql diff --git a/tests/queries/0_stateless/01497_now_support_timezone.reference b/tests/queries/0_stateless/01497_now_support_timezone.reference new file mode 100644 index 00000000000..45a4fb75db8 --- /dev/null +++ b/tests/queries/0_stateless/01497_now_support_timezone.reference @@ -0,0 +1 @@ +8 diff --git a/tests/queries/0_stateless/01497_now_support_timezone.sql b/tests/queries/0_stateless/01497_now_support_timezone.sql new file mode 100644 index 00000000000..cc5ae77e94d --- /dev/null +++ b/tests/queries/0_stateless/01497_now_support_timezone.sql @@ -0,0 +1 @@ +SELECT toHour(now('Asia/Shanghai')) - toHour(now('UTC')); From 81d08b59e51b0420e51b0ffd7750c8bfa34d4c93 Mon Sep 17 00:00:00 2001 From: Amos Bird Date: Fri, 25 Sep 2020 16:38:09 +0800 Subject: [PATCH 057/321] Replace useless multiset with unordered_set --- src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp | 2 +- src/Storages/VirtualColumnUtils.h | 6 +++--- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp index 2b8b886daaf..8b5b337bcec 100644 --- a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp +++ b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp @@ -223,7 +223,7 @@ Pipe MergeTreeDataSelectExecutor::readFromParts( if (part_column_queried) VirtualColumnUtils::filterBlockWithQuery(query_info.query, virtual_columns_block, context); - std::multiset part_values = VirtualColumnUtils::extractSingleValueFromBlock(virtual_columns_block, "_part"); + auto part_values = VirtualColumnUtils::extractSingleValueFromBlock(virtual_columns_block, "_part"); metadata_snapshot->check(real_column_names, data.getVirtuals(), data.getStorageID()); diff --git a/src/Storages/VirtualColumnUtils.h b/src/Storages/VirtualColumnUtils.h index 89b69eb79e3..445a996ab87 100644 --- a/src/Storages/VirtualColumnUtils.h +++ b/src/Storages/VirtualColumnUtils.h @@ -1,6 +1,6 @@ #pragma once -#include +#include #include #include @@ -30,9 +30,9 @@ void filterBlockWithQuery(const ASTPtr & query, Block & block, const Context & c /// Extract from the input stream a set of `name` column values template -std::multiset extractSingleValueFromBlock(const Block & block, const String & name) +auto extractSingleValueFromBlock(const Block & block, const String & name) { - std::multiset res; + std::unordered_set res; const ColumnWithTypeAndName & data = block.getByName(name); size_t rows = block.rows(); for (size_t i = 0; i < rows; ++i) From 3ff16325d80b6f284f26ffffa46a93d812557cbc Mon Sep 17 00:00:00 2001 From: Alexander Kuzmenkov Date: Fri, 25 Sep 2020 12:52:09 +0300 Subject: [PATCH 058/321] fix the profiler --- docker/test/performance-comparison/compare.sh | 50 +++++++++--------- docker/test/performance-comparison/perf.py | 51 ++++++++++--------- 2 files changed, 53 insertions(+), 48 deletions(-) diff --git a/docker/test/performance-comparison/compare.sh b/docker/test/performance-comparison/compare.sh index bed6702ea4a..6775e662070 100755 --- a/docker/test/performance-comparison/compare.sh +++ b/docker/test/performance-comparison/compare.sh @@ -181,6 +181,9 @@ function run_tests # Randomize test order. test_files=$(for f in $test_files; do echo "$f"; done | sort -R) + # Limit profiling time to 10 minutes, not to run for too long. + profile_seconds_left=600 + # Run the tests. test_name="" for test in $test_files @@ -194,15 +197,23 @@ function run_tests test_name=$(basename "$test" ".xml") echo test "$test_name" + # Don't profile if we're past the time limit. + profile_seconds=$((profile_seconds_left > 0 ? 10 : 0)) + TIMEFORMAT=$(printf "$test_name\t%%3R\t%%3U\t%%3S\n") # The grep is to filter out set -x output and keep only time output. # The '2>&1 >/dev/null' redirects stderr to stdout, and discards stdout. { \ time "$script_dir/perf.py" --host localhost localhost --port 9001 9002 \ --runs "$CHPC_RUNS" --max-queries "$CHPC_MAX_QUERIES" \ + --profile-seconds "$profile_seconds" \ -- "$test" > "$test_name-raw.tsv" 2> "$test_name-err.log" ; \ } 2>&1 >/dev/null | tee >(grep -v ^+ >> "wall-clock-times.tsv") \ || echo "Test $test_name failed with error code $?" >> "$test_name-err.log" + + profile_seconds_left=$(awk -F' ' \ + 'BEGIN { s = '$profile_seconds_left'; } /^profile-total/ { s -= $1 } END { print s }' \ + "$test_name-raw.tsv") done unset TIMEFORMAT @@ -294,6 +305,7 @@ for test_file in *-raw.tsv do test_name=$(basename "$test_file" "-raw.tsv") sed -n "s/^query\t/$test_name\t/p" < "$test_file" >> "analyze/query-runs.tsv" + sed -n "s/^profile\t/$test_name\t/p" < "$test_file" >> "analyze/query-profiles.tsv" sed -n "s/^client-time\t/$test_name\t/p" < "$test_file" >> "analyze/client-times.tsv" sed -n "s/^report-threshold\t/$test_name\t/p" < "$test_file" >> "analyze/report-thresholds.tsv" sed -n "s/^skipped\t/$test_name\t/p" < "$test_file" >> "analyze/skipped-tests.tsv" @@ -732,13 +744,12 @@ create table all_tests_report engine File(TSV, 'report/all-queries.tsv') as test, query_index, query_display_name from queries order by test, query_index; --- queries for which we will build flamegraphs (see below) -create table queries_for_flamegraph engine File(TSVWithNamesAndTypes, - 'report/queries-for-flamegraph.tsv') as - select test, query_index from queries where unstable_show or changed_show - ; - +-- Report of queries that have inconsistent 'short' markings: +-- 1) have short duration, but are not marked as 'short' +-- 2) the reverse -- marked 'short' but take too long. +-- The threshold for 2) is significantly larger than the threshold for 1), to +-- avoid jitter. create view shortness as select (test, query_index) in @@ -756,11 +767,6 @@ create view shortness and times.query_index = query_display_names.query_index ; --- Report of queries that have inconsistent 'short' markings: --- 1) have short duration, but are not marked as 'short' --- 2) the reverse -- marked 'short' but take too long. --- The threshold for 2) is significantly larger than the threshold for 1), to --- avoid jitter. create table inconsistent_short_marking_report engine File(TSV, 'report/unexpected-query-duration.tsv') as select @@ -797,18 +803,15 @@ create table all_query_metrics_tsv engine File(TSV, 'report/all-query-metrics.ts " 2> >(tee -a report/errors.log 1>&2) -# Prepare source data for metrics and flamegraphs for unstable queries. +# Prepare source data for metrics and flamegraphs for queries that were profiled +# by perf.py. for version in {right,left} do rm -rf data clickhouse-local --query " -create view queries_for_flamegraph as - select * from file('report/queries-for-flamegraph.tsv', TSVWithNamesAndTypes, - 'test text, query_index int'); - -create view query_runs as +create view query_profiles as with 0 as left, 1 as right - select * from file('analyze/query-runs.tsv', TSV, + select * from file('analyze/query-profiles.tsv', TSV, 'test text, query_index int, query_id text, version UInt8, time float') where version = $version ; @@ -820,15 +823,12 @@ create view query_display_names as select * from create table unstable_query_runs engine File(TSVWithNamesAndTypes, 'unstable-query-runs.$version.rep') as - select query_runs.test test, query_runs.query_index query_index, + select query_profiles.test test, query_profiles.query_index query_index, query_display_name, query_id - from query_runs - join queries_for_flamegraph on - query_runs.test = queries_for_flamegraph.test - and query_runs.query_index = queries_for_flamegraph.query_index + from query_profiles left join query_display_names on - query_runs.test = query_display_names.test - and query_runs.query_index = query_display_names.query_index + query_profiles.test = query_display_names.test + and query_profiles.query_index = query_display_names.query_index ; create view query_log as select * diff --git a/docker/test/performance-comparison/perf.py b/docker/test/performance-comparison/perf.py index 3aaab838f83..853b3f76c7c 100755 --- a/docker/test/performance-comparison/perf.py +++ b/docker/test/performance-comparison/perf.py @@ -42,6 +42,7 @@ parser.add_argument('--port', nargs='*', default=[9000], help="Space-separated l parser.add_argument('--runs', type=int, default=1, help='Number of query runs per server.') parser.add_argument('--max-queries', type=int, default=None, help='Test no more than this number of queries, chosen at random.') parser.add_argument('--queries-to-run', nargs='*', type=int, default=None, help='Space-separated list of indexes of queries to test.') +parser.add_argument('--profile-seconds', type=int, default=0, help='For how many seconds to profile a query for which the performance has changed.') parser.add_argument('--long', action='store_true', help='Do not skip the tests tagged as long.') parser.add_argument('--print-queries', action='store_true', help='Print test queries and exit.') parser.add_argument('--print-settings', action='store_true', help='Print test settings and exit.') @@ -235,6 +236,7 @@ if args.queries_to_run: queries_to_run = args.queries_to_run # Run test queries. +profile_total_seconds = 0 for query_index in queries_to_run: q = test_queries[query_index] query_prefix = f'{test_name}.query{query_index}' @@ -354,35 +356,38 @@ for query_index in queries_to_run: client_seconds = time.perf_counter() - start_seconds print(f'client-time\t{query_index}\t{client_seconds}\t{server_seconds}') - #print(all_server_times) - #print(stats.ttest_ind(all_server_times[0], all_server_times[1], equal_var = False).pvalue) - # Run additional profiling queries to collect profile data, but only if test times appeared to be different. # We have to do it after normal runs because otherwise it will affect test statistics too much - if len(all_server_times) == 2 and stats.ttest_ind(all_server_times[0], all_server_times[1], equal_var = False).pvalue < 0.1: - run = 0 - while True: - run_id = f'{query_prefix}.profile{run}' + if len(all_server_times) != 2: + continue - for conn_index, c in enumerate(this_query_connections): - try: - res = c.execute(q, query_id = run_id, settings = {'query_profiler_real_time_period_ns': 10000000}) - print(f'profile\t{query_index}\t{run_id}\t{conn_index}\t{c.last_query.elapsed}') - except Exception as e: - # Add query id to the exception to make debugging easier. - e.args = (run_id, *e.args) - e.message = run_id + ': ' + e.message - raise + pvalue = stats.ttest_ind(all_server_times[0], all_server_times[1], equal_var = False).pvalue + print(f'pvalue\t{pvalue}') + if pvalue > 0.05: + continue - elapsed = c.last_query.elapsed - profile_seconds += elapsed + # Perform profile runs for fixed amount of time. Don't limit the number + # of runs, because we also have short queries. + profile_start_seconds = time.perf_counter() + run = 0 + while time.perf_counter() - profile_start_seconds < args.profile_seconds: + run_id = f'{query_prefix}.profile{run}' - run += 1 - # Don't spend too much time for profile runs - if run > args.runs or profile_seconds > 10: - break - # And don't bother with short queries + for conn_index, c in enumerate(this_query_connections): + try: + res = c.execute(q, query_id = run_id, settings = {'query_profiler_real_time_period_ns': 10000000}) + print(f'profile\t{query_index}\t{run_id}\t{conn_index}\t{c.last_query.elapsed}') + except Exception as e: + # Add query id to the exception to make debugging easier. + e.args = (run_id, *e.args) + e.message = run_id + ': ' + e.message + raise + run += 1 + + profile_total_seconds += time.perf_counter() - profile_start_seconds + +print(f'profile-total\t{profile_total_seconds}') reportStageEnd('run') From 085f63a9bfa6a6294451612b39b5d51c15f8f0aa Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Fri, 25 Sep 2020 14:12:48 +0300 Subject: [PATCH 059/321] fix hang of queries with a lot of subqueries to same mysql table --- src/Formats/MySQLBlockInputStream.cpp | 59 +++++++++++++++++--- src/Formats/MySQLBlockInputStream.h | 41 ++++++++++++-- src/Storages/StorageMySQL.cpp | 2 +- tests/integration/test_storage_mysql/test.py | 19 +++++++ 4 files changed, 107 insertions(+), 14 deletions(-) diff --git a/src/Formats/MySQLBlockInputStream.cpp b/src/Formats/MySQLBlockInputStream.cpp index be1e254b22f..73def337240 100644 --- a/src/Formats/MySQLBlockInputStream.cpp +++ b/src/Formats/MySQLBlockInputStream.cpp @@ -23,13 +23,27 @@ namespace ErrorCodes extern const int NUMBER_OF_COLUMNS_DOESNT_MATCH; } +MySQLBlockInputStream::Connection::Connection( + const mysqlxx::PoolWithFailover::Entry & entry_, + const std::string & query_str) + : entry(entry_) + , query{entry->query(query_str)} + , result{query.use()} +{ +} MySQLBlockInputStream::MySQLBlockInputStream( - const mysqlxx::PoolWithFailover::Entry & entry_, const std::string & query_str, const Block & sample_block, const UInt64 max_block_size_, const bool auto_close_) - : entry{entry_}, query{this->entry->query(query_str)}, result{query.use()}, max_block_size{max_block_size_}, auto_close{auto_close_} + const mysqlxx::PoolWithFailover::Entry & entry, + const std::string & query_str, + const Block & sample_block, + const UInt64 max_block_size_, + const bool auto_close_) + : connection{std::make_unique(entry, query_str)} + , max_block_size{max_block_size_} + , auto_close{auto_close_} { - if (sample_block.columns() != result.getNumFields()) - throw Exception{"mysqlxx::UseQueryResult contains " + toString(result.getNumFields()) + " columns while " + if (sample_block.columns() != connection->result.getNumFields()) + throw Exception{"mysqlxx::UseQueryResult contains " + toString(connection->result.getNumFields()) + " columns while " + toString(sample_block.columns()) + " expected", ErrorCodes::NUMBER_OF_COLUMNS_DOESNT_MATCH}; @@ -106,11 +120,11 @@ namespace Block MySQLBlockInputStream::readImpl() { - auto row = result.fetch(); + auto row = connection->result.fetch(); if (!row) { if (auto_close) - entry.disconnect(); + connection->entry.disconnect(); return {}; } @@ -145,11 +159,42 @@ Block MySQLBlockInputStream::readImpl() if (num_rows == max_block_size) break; - row = result.fetch(); + row = connection->result.fetch(); } return description.sample_block.cloneWithColumns(std::move(columns)); } +MySQLBlockInputStream::MySQLBlockInputStream( + const Block & sample_block_, + UInt64 max_block_size_, + bool auto_close_) + : max_block_size(max_block_size_) + , auto_close(auto_close_) +{ + description.init(sample_block_); +} + +MySQLLazyBlockInputStream::MySQLLazyBlockInputStream( + mysqlxx::Pool & pool_, + const std::string & query_str_, + const Block & sample_block_, + const UInt64 max_block_size_, + const bool auto_close_) + : MySQLBlockInputStream(sample_block_, max_block_size_, auto_close_) + , pool(pool_) + , query_str(query_str_) +{ +} + +void MySQLLazyBlockInputStream::readPrefix() +{ + connection = std::make_unique(pool.get(), query_str); + if (description.sample_block.columns() != connection->result.getNumFields()) + throw Exception{"mysqlxx::UseQueryResult contains " + toString(connection->result.getNumFields()) + " columns while " + + toString(description.sample_block.columns()) + " expected", + ErrorCodes::NUMBER_OF_COLUMNS_DOESNT_MATCH}; +} + } #endif diff --git a/src/Formats/MySQLBlockInputStream.h b/src/Formats/MySQLBlockInputStream.h index 238994acbd8..2eaeb5b8d59 100644 --- a/src/Formats/MySQLBlockInputStream.h +++ b/src/Formats/MySQLBlockInputStream.h @@ -10,12 +10,13 @@ namespace DB { + /// Allows processing results of a MySQL query as a sequence of Blocks, simplifies chaining -class MySQLBlockInputStream final : public IBlockInputStream +class MySQLBlockInputStream : public IBlockInputStream { public: MySQLBlockInputStream( - const mysqlxx::PoolWithFailover::Entry & entry_, + const mysqlxx::PoolWithFailover::Entry & entry, const std::string & query_str, const Block & sample_block, const UInt64 max_block_size_, @@ -25,15 +26,43 @@ public: Block getHeader() const override { return description.sample_block.cloneEmpty(); } -private: +protected: + MySQLBlockInputStream(const Block & sample_block_, UInt64 max_block_size_, bool auto_close_); Block readImpl() override; - mysqlxx::PoolWithFailover::Entry entry; - mysqlxx::Query query; - mysqlxx::UseQueryResult result; + struct Connection + { + Connection(const mysqlxx::PoolWithFailover::Entry & entry_, const std::string & query_str); + + mysqlxx::PoolWithFailover::Entry entry; + mysqlxx::Query query; + mysqlxx::UseQueryResult result; + }; + + std::unique_ptr connection; + const UInt64 max_block_size; const bool auto_close; ExternalResultDescription description; }; +/// Like MySQLBlockInputStream, but allocates connection only when reading is starting. +/// It allows to create a lot of stream objects without occupation of all connection pool. +class MySQLLazyBlockInputStream final : public MySQLBlockInputStream +{ +public: + MySQLLazyBlockInputStream( + mysqlxx::Pool & pool_, + const std::string & query_str_, + const Block & sample_block_, + const UInt64 max_block_size_, + const bool auto_close_ = false); + +private: + void readPrefix() override; + + mysqlxx::Pool & pool; + std::string query_str; +}; + } diff --git a/src/Storages/StorageMySQL.cpp b/src/Storages/StorageMySQL.cpp index b2e622663c0..afbca0d9430 100644 --- a/src/Storages/StorageMySQL.cpp +++ b/src/Storages/StorageMySQL.cpp @@ -96,7 +96,7 @@ Pipe StorageMySQL::read( /// TODO: rewrite MySQLBlockInputStream return Pipe(std::make_shared( - std::make_shared(pool.get(), query, sample_block, max_block_size_))); + std::make_shared(pool, query, sample_block, max_block_size_, /* auto_close = */ true))); } diff --git a/tests/integration/test_storage_mysql/test.py b/tests/integration/test_storage_mysql/test.py index 0b73866eaee..83ef1e6c86a 100644 --- a/tests/integration/test_storage_mysql/test.py +++ b/tests/integration/test_storage_mysql/test.py @@ -33,6 +33,25 @@ def started_cluster(): cluster.shutdown() +def test_many_connections(started_cluster): + table_name = 'test_many_connections' + conn = get_mysql_conn() + create_mysql_table(conn, table_name) + + node1.query(''' +CREATE TABLE {}(id UInt32, name String, age UInt32, money UInt32) ENGINE = MySQL('mysql1:3306', 'clickhouse', '{}', 'root', 'clickhouse'); +'''.format(table_name, table_name)) + + node1.query("INSERT INTO {} (id, name) SELECT number, concat('name_', toString(number)) from numbers(10) ".format(table_name)) + + query = "SELECT count() FROM (" + for i in range (24): + query += "SELECT id FROM {t} UNION ALL " + query += "SELECT id FROM {t})" + + assert node1.query(query.format(t=table_name)) == '250\n' + conn.close() + def test_insert_select(started_cluster): table_name = 'test_insert_select' conn = get_mysql_conn() From 97671a17719eb9c101095e01cac7baa282908126 Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 25 Sep 2020 14:27:00 +0300 Subject: [PATCH 060/321] Ugly fix for default database race --- programs/server/Server.cpp | 13 +++++++++---- src/Interpreters/Context.cpp | 12 ++++++++++++ src/Interpreters/Context.h | 3 +++ src/Interpreters/SystemLog.h | 2 +- 4 files changed, 25 insertions(+), 5 deletions(-) diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index b703d0ac6a7..6341653ee2f 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -671,6 +671,10 @@ int Server::main(const std::vector & /*args*/) total_memory_tracker.setDescription("(total)"); total_memory_tracker.setMetric(CurrentMetrics::MemoryTracking); + /// Set current database name before loading tables and databases because + /// system logs may copy global context. + global_context->setCurrentDatabaseNameInGlobalContext(default_database); + LOG_INFO(log, "Loading metadata from {}", path); try @@ -678,11 +682,14 @@ int Server::main(const std::vector & /*args*/) loadMetadataSystem(*global_context); /// After attaching system databases we can initialize system log. global_context->initializeSystemLogs(); + auto & database_catalog = DatabaseCatalog::instance(); /// After the system database is created, attach virtual system tables (in addition to query_log and part_log) - attachSystemTablesServer(*DatabaseCatalog::instance().getSystemDatabase(), has_zookeeper); + attachSystemTablesServer(*database_catalog.getSystemDatabase(), has_zookeeper); /// Then, load remaining databases loadMetadata(*global_context, default_database); - DatabaseCatalog::instance().loadDatabases(); + database_catalog.loadDatabases(); + /// After loading validate that default database exists + database_catalog.assertDatabaseExists(default_database); } catch (...) { @@ -745,8 +752,6 @@ int Server::main(const std::vector & /*args*/) LOG_INFO(log, "Query Profiler and TraceCollector are disabled because they require PHDR cache to be created" " (otherwise the function 'dl_iterate_phdr' is not lock free and not async-signal safe)."); - global_context->setCurrentDatabase(default_database); - if (has_zookeeper && config().has("distributed_ddl")) { /// DDL worker should be started after all tables were loaded diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index 704b21f3a4a..be35c8a9184 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -1088,6 +1088,18 @@ String Context::getInitialQueryId() const } +void Context::setCurrentDatabaseNameInGlobalContext(const String & name) +{ + if (global_context != this) + throw Exception("Cannot set current database for non global context, this method should be used during server initialization", ErrorCodes::LOGICAL_ERROR); + auto lock = getLock(); + + if (!current_database.empty()) + throw Exception("Default database name cannot be changed in global context without server restart", ErrorCodes::LOGICAL_ERROR); + + current_database = name; +} + void Context::setCurrentDatabase(const String & name) { DatabaseCatalog::instance().assertDatabaseExists(name); diff --git a/src/Interpreters/Context.h b/src/Interpreters/Context.h index 3d66ef239e7..bd5e17fe2e4 100644 --- a/src/Interpreters/Context.h +++ b/src/Interpreters/Context.h @@ -359,6 +359,9 @@ public: String getInitialQueryId() const; void setCurrentDatabase(const String & name); + /// Set current_database for global context. We don't validate that database + /// exists because it should be set before databases loading. + void setCurrentDatabaseNameInGlobalContext(const String & name); void setCurrentQueryId(const String & query_id); void killCurrentQuery(); diff --git a/src/Interpreters/SystemLog.h b/src/Interpreters/SystemLog.h index 03b1b735cbc..2a0ce9cef53 100644 --- a/src/Interpreters/SystemLog.h +++ b/src/Interpreters/SystemLog.h @@ -438,7 +438,7 @@ void SystemLog::flushImpl(const std::vector & to_flush, ASTPtr query_ptr(insert.release()); // we need query context to do inserts to target table with MV containing subqueries or joins - auto insert_context = Context(context); + Context insert_context(context); insert_context.makeQueryContext(); InterpreterInsertQuery interpreter(query_ptr, insert_context); From d4c8ba84102c948f1da37c3f00abd1ca6fa2a15f Mon Sep 17 00:00:00 2001 From: tavplubix Date: Fri, 25 Sep 2020 14:58:04 +0300 Subject: [PATCH 061/321] Update ASTColumnsTransformers.h --- src/Parsers/ASTColumnsTransformers.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Parsers/ASTColumnsTransformers.h b/src/Parsers/ASTColumnsTransformers.h index ddf0d70dc35..4b7a933647e 100644 --- a/src/Parsers/ASTColumnsTransformers.h +++ b/src/Parsers/ASTColumnsTransformers.h @@ -53,7 +53,7 @@ public: ASTPtr clone() const override { auto replacement = std::make_shared(*this); - replacement->name = name; + replacement->children.clear(); replacement->expr = expr->clone(); replacement->children.push_back(replacement->expr); return replacement; From f5c38b34ae5f439a46c661413a429efc30aca05f Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 25 Sep 2020 15:07:38 +0300 Subject: [PATCH 062/321] Fix race condition during write ahead log cleanup and rename --- src/Storages/MergeTree/MergeTreeData.h | 2 ++ src/Storages/MergeTree/ReplicatedMergeTreeCleanupThread.cpp | 4 +++- src/Storages/StorageMergeTree.cpp | 4 +++- 3 files changed, 8 insertions(+), 2 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeData.h b/src/Storages/MergeTree/MergeTreeData.h index 0fc5ec43048..1125eb32b66 100644 --- a/src/Storages/MergeTree/MergeTreeData.h +++ b/src/Storages/MergeTree/MergeTreeData.h @@ -717,6 +717,8 @@ protected: bool require_part_metadata; + /// Relative path data, changes during rename for ordinary databases use + /// under lockForShare if rename is possible. String relative_data_path; diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeCleanupThread.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeCleanupThread.cpp index a5216e6fda3..11f23a5c110 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeCleanupThread.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeCleanupThread.cpp @@ -56,10 +56,12 @@ void ReplicatedMergeTreeCleanupThread::run() void ReplicatedMergeTreeCleanupThread::iterate() { storage.clearOldPartsAndRemoveFromZK(); - storage.clearOldWriteAheadLogs(); { auto lock = storage.lockForShare(RWLockImpl::NO_QUERY, storage.getSettings()->lock_acquire_timeout_for_background_operations); + /// Both use relative_data_path which changes during rename, so we + /// do it under share lock + storage.clearOldWriteAheadLogs(); storage.clearOldTemporaryDirectories(); } diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index 347474753dc..55fb42b550e 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -919,11 +919,13 @@ BackgroundProcessingPoolTaskResult StorageMergeTree::mergeMutateTask() { { auto share_lock = lockForShare(RWLockImpl::NO_QUERY, getSettings()->lock_acquire_timeout_for_background_operations); + /// All use relative_data_path which changes during rename + /// so execute under share lock. clearOldPartsFromFilesystem(); clearOldTemporaryDirectories(); + clearOldWriteAheadLogs(); } clearOldMutations(); - clearOldWriteAheadLogs(); } ///TODO: read deduplicate option from table config From 86322fa3e1b86a2e96452805c18e440c29e6dab0 Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 25 Sep 2020 15:36:02 +0300 Subject: [PATCH 063/321] Add test --- ...condition_rename_clear_zookeeper.reference | 1 + ...8_race_condition_rename_clear_zookeeper.sh | 27 +++++++++++++++++++ 2 files changed, 28 insertions(+) create mode 100644 tests/queries/0_stateless/01508_race_condition_rename_clear_zookeeper.reference create mode 100755 tests/queries/0_stateless/01508_race_condition_rename_clear_zookeeper.sh diff --git a/tests/queries/0_stateless/01508_race_condition_rename_clear_zookeeper.reference b/tests/queries/0_stateless/01508_race_condition_rename_clear_zookeeper.reference new file mode 100644 index 00000000000..13de30f45d1 --- /dev/null +++ b/tests/queries/0_stateless/01508_race_condition_rename_clear_zookeeper.reference @@ -0,0 +1 @@ +3000 diff --git a/tests/queries/0_stateless/01508_race_condition_rename_clear_zookeeper.sh b/tests/queries/0_stateless/01508_race_condition_rename_clear_zookeeper.sh new file mode 100755 index 00000000000..2af1cb214a4 --- /dev/null +++ b/tests/queries/0_stateless/01508_race_condition_rename_clear_zookeeper.sh @@ -0,0 +1,27 @@ +#!/usr/bin/env bash + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +. "$CURDIR"/../shell_config.sh + +$CLICKHOUSE_CLIENT --query "DROP TABLE IF EXISTS table_for_renames0" +$CLICKHOUSE_CLIENT --query "DROP TABLE IF EXISTS table_for_renames50" + + +$CLICKHOUSE_CLIENT --query "CREATE TABLE table_for_renames0 (value UInt64, data String) ENGINE ReplicatedMergeTree('/clickhouse/tables/test_01508/concurrent_rename', '1') ORDER BY tuple() SETTINGS cleanup_delay_period = 1, cleanup_delay_period_random_add = 0, min_rows_for_compact_part = 100000, min_rows_for_compact_part = 10000000, write_ahead_log_max_bytes = 1" + + +$CLICKHOUSE_CLIENT --query "INSERT INTO table_for_renames0 SELECT number, toString(number) FROM numbers(1000)" + +$CLICKHOUSE_CLIENT --query "INSERT INTO table_for_renames0 SELECT number, toString(number) FROM numbers(1000, 1000)" + +$CLICKHOUSE_CLIENT --query "INSERT INTO table_for_renames0 SELECT number, toString(number) FROM numbers(2000, 1000)" + +for i in $(seq 1 50); do + prev_i=$((i - 1)) + $CLICKHOUSE_CLIENT --query "RENAME TABLE table_for_renames$prev_i TO table_for_renames$i" +done + +$CLICKHOUSE_CLIENT --query "SELECT COUNT() from table_for_renames50" + +$CLICKHOUSE_CLIENT --query "DROP TABLE IF EXISTS table_for_renames0" +$CLICKHOUSE_CLIENT --query "DROP TABLE IF EXISTS table_for_renames50" From 54fa67838e95364c5169ccb67b8e7dfc0b90e337 Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 25 Sep 2020 16:32:00 +0300 Subject: [PATCH 064/321] Add flaky test --- tests/integration/test_flaky_check/__init__.py | 0 tests/integration/test_flaky_check/test.py | 5 +++++ 2 files changed, 5 insertions(+) create mode 100644 tests/integration/test_flaky_check/__init__.py create mode 100644 tests/integration/test_flaky_check/test.py diff --git a/tests/integration/test_flaky_check/__init__.py b/tests/integration/test_flaky_check/__init__.py new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/integration/test_flaky_check/test.py b/tests/integration/test_flaky_check/test.py new file mode 100644 index 00000000000..4da85151c46 --- /dev/null +++ b/tests/integration/test_flaky_check/test.py @@ -0,0 +1,5 @@ +import random + + +def test_flaky(): + assert random.randint(0, 2) >= 1 From ada193e7b716f1ebd04abefe8fbbdcb225e1f766 Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 25 Sep 2020 16:33:33 +0300 Subject: [PATCH 065/321] Add diff --- tests/integration/test_ttl_move/test.py | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/integration/test_ttl_move/test.py b/tests/integration/test_ttl_move/test.py index 377ee0e5d75..a4f8606011d 100644 --- a/tests/integration/test_ttl_move/test.py +++ b/tests/integration/test_ttl_move/test.py @@ -8,6 +8,7 @@ import pytest from helpers.client import QueryRuntimeException from helpers.cluster import ClickHouseCluster +# test cluster = ClickHouseCluster(__file__) node1 = cluster.add_instance('node1', From 0e164a9ec73af16b3ca660370dd2ad8b76808aae Mon Sep 17 00:00:00 2001 From: Evgeniia Sudarikova Date: Fri, 25 Sep 2020 16:34:35 +0300 Subject: [PATCH 066/321] add EN description --- .../functions/type-conversion-functions.md | 41 +++++++++++++++++++ 1 file changed, 41 insertions(+) diff --git a/docs/en/sql-reference/functions/type-conversion-functions.md b/docs/en/sql-reference/functions/type-conversion-functions.md index e5c321041c2..4859c8a7aad 100644 --- a/docs/en/sql-reference/functions/type-conversion-functions.md +++ b/docs/en/sql-reference/functions/type-conversion-functions.md @@ -729,4 +729,45 @@ SELECT fromUnixTimestamp64Milli(i64, 'UTC') └──────────────────────────────────────┘ ``` +## formatRow {#formatrow} + +Converts arbitrary expressions into a string via given format. + +**Syntax** + +``` sql +formatRow(format, x, y, ...) +``` + +**Parameters** + +- `format` — text format. For example [CSV](../../interfaces/formats.md#csv), [TSV](../../interfaces/formats.md#tabseparated). +- `x`,`y`, ... — expressions. + +**Returned value** + +- expressions converted to the `format`. + +**Example** + +Query: + +``` sql +SELECT formatRow('CSV', number, 'good') +FROM numbers(3) +``` + +Result: + +``` text +┌─formatRow('CSV', number, 'good')─┐ +│ 0,"good" + │ +│ 1,"good" + │ +│ 2,"good" + │ +└──────────────────────────────────┘ +``` + [Original article](https://clickhouse.tech/docs/en/query_language/functions/type_conversion_functions/) From d52e1634a2ead29e3477e3e0e85d0fa3b3f5ab57 Mon Sep 17 00:00:00 2001 From: Anna Date: Fri, 25 Sep 2020 17:32:39 +0300 Subject: [PATCH 067/321] Add normalizedQuery normalizedQueryHash desc --- .../sql-reference/functions/url-functions.md | 86 +++++++++++++++++++ 1 file changed, 86 insertions(+) diff --git a/docs/en/sql-reference/functions/url-functions.md b/docs/en/sql-reference/functions/url-functions.md index 89154ac3d02..a76912d9048 100644 --- a/docs/en/sql-reference/functions/url-functions.md +++ b/docs/en/sql-reference/functions/url-functions.md @@ -244,4 +244,90 @@ Removes the query string and fragment identifier. The question mark and number s Removes the ‘name’ URL parameter, if present. This function works under the assumption that the parameter name is encoded in the URL exactly the same way as in the passed argument. +## normalizeQuery {#normalized-query} + +Replaces literals, sequences of literals and complex aliases with placeholders. + +**Syntax** (without SELECT) + +``` sql + +``` + +Alias: ``. (Optional) + +More text (Optional). + +**Parameters** (Optional) + +- `x` — Description. [Type name](relative/path/to/type/dscr.md#type). +- `y` — Description. [Type name](relative/path/to/type/dscr.md#type). + +**Returned value(s)** + +- Returned values list. + +Type: [Type](relative/path/to/type/dscr.md#type). + +**Example** + +SELECT count(*) FROM table WHERE date = '2020-01-02' AND id IN (1, 2, 3) LIMIT 10, 10 +should be replaced to +SELECT count(*) FROM table WHERE date = ? AND id IN (?) LIMIT ? + +Query: + +``` sql +``` + +Result: + +``` text +``` + +## normalizedQueryHash {#normalized-query-hash} + +Returns identical 64bit hash values for similar queries. It helps to analyze query log. +calculate a hash of query structure without the values of literals. + +**Syntax** (without SELECT) + +``` sql + +``` + +Alias: ``. (Optional) + +More text (Optional). + +**Parameters** (Optional) + +- `x` — Description. [Type name](relative/path/to/type/dscr.md#type). +- `y` — Description. [Type name](relative/path/to/type/dscr.md#type). + +**Returned value(s)** + +- Returned values list. + +Type: [Type](relative/path/to/type/dscr.md#type). + +**Example** + +The example must show usage and/or a use cases. The following text contains recommended parts of an example. + +Input table (Optional): + +``` text +``` + +Query: + +``` sql +``` + +Result: + +``` text +``` + [Original article](https://clickhouse.tech/docs/en/query_language/functions/url_functions/) From 3eee9e31ddc750b1c46cafb602d0b12941628a68 Mon Sep 17 00:00:00 2001 From: Denis Zhuravlev Date: Fri, 25 Sep 2020 12:48:51 -0300 Subject: [PATCH 068/321] more tests for #11767 (from #11407 #15294) --- .../01323_add_scalars_in_time.reference | 3 ++ .../0_stateless/01323_add_scalars_in_time.sql | 52 +++++++++++++++++++ 2 files changed, 55 insertions(+) diff --git a/tests/queries/0_stateless/01323_add_scalars_in_time.reference b/tests/queries/0_stateless/01323_add_scalars_in_time.reference index 49e20d0fea8..408efa7f823 100644 --- a/tests/queries/0_stateless/01323_add_scalars_in_time.reference +++ b/tests/queries/0_stateless/01323_add_scalars_in_time.reference @@ -1,2 +1,5 @@ [0,2,3] id2 [1,2,3] id1 +test [1,2,3,4] +2 fre +3 jhg diff --git a/tests/queries/0_stateless/01323_add_scalars_in_time.sql b/tests/queries/0_stateless/01323_add_scalars_in_time.sql index b49c2ef416e..d09534d2b43 100644 --- a/tests/queries/0_stateless/01323_add_scalars_in_time.sql +++ b/tests/queries/0_stateless/01323_add_scalars_in_time.sql @@ -17,3 +17,55 @@ WHERE id LIKE 'id%' GROUP BY id; DROP TABLE tags; + + +-- https://github.com/ClickHouse/ClickHouse/issues/15294 + +drop table if exists TestTable; + +create table TestTable (column String, start DateTime, end DateTime) engine MergeTree order by start; + +insert into TestTable (column, start, end) values('test', toDateTime('2020-07-20 09:00:00'), toDateTime('2020-07-20 20:00:00')),('test1', toDateTime('2020-07-20 09:00:00'), toDateTime('2020-07-20 20:00:00')),('test2', toDateTime('2020-07-20 09:00:00'), toDateTime('2020-07-20 20:00:00')); + +SELECT column, +(SELECT d from (select [1, 2, 3, 4] as d)) as d +FROM TestTable +where column == 'test' +GROUP BY column; + +drop table TestTable; + +-- https://github.com/ClickHouse/ClickHouse/issues/11407 + +drop table if exists aaa; +drop table if exists bbb; + +CREATE TABLE aaa ( + id UInt16, + data String +) +ENGINE = MergeTree() +PARTITION BY tuple() +ORDER BY id; + +INSERT INTO aaa VALUES (1, 'sef'),(2, 'fre'),(3, 'jhg'); + +CREATE TABLE bbb ( + id UInt16, + data String +) +ENGINE = MergeTree() +PARTITION BY tuple() +ORDER BY id; + +INSERT INTO bbb VALUES (2, 'fre'),(3, 'jhg'); + +with (select groupArray(id) from bbb) as ids +select * + from aaa + where has(ids, id) +order by id; + + +drop table aaa; +drop table bbb; From af0bb5bd263d0b12600f0ea2d86b0cf6a6d76bf1 Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 25 Sep 2020 19:15:34 +0300 Subject: [PATCH 069/321] One more time --- tests/integration/test_s3_with_proxy/test.py | 14 ++++++++++---- tests/integration/test_ttl_move/test.py | 1 - 2 files changed, 10 insertions(+), 5 deletions(-) diff --git a/tests/integration/test_s3_with_proxy/test.py b/tests/integration/test_s3_with_proxy/test.py index 9df209826f9..3d118266455 100644 --- a/tests/integration/test_s3_with_proxy/test.py +++ b/tests/integration/test_s3_with_proxy/test.py @@ -1,5 +1,6 @@ import logging import os +import time import pytest from helpers.cluster import ClickHouseCluster @@ -37,10 +38,15 @@ def cluster(): def check_proxy_logs(cluster, proxy_instance, http_methods={"POST", "PUT", "GET", "DELETE"}): - logs = cluster.get_container_logs(proxy_instance) - # Check that all possible interactions with Minio are present - for http_method in http_methods: - assert logs.find(http_method + " http://minio1") >= 0 + for i in range(10): + logs = cluster.get_container_logs(proxy_instance) + # Check that all possible interactions with Minio are present + for http_method in http_methods: + if logs.find(http_method + " http://minio1") >= 0: + return + time.sleep(1) + else: + assert False, "http method not found in logs" @pytest.mark.parametrize( diff --git a/tests/integration/test_ttl_move/test.py b/tests/integration/test_ttl_move/test.py index a4f8606011d..377ee0e5d75 100644 --- a/tests/integration/test_ttl_move/test.py +++ b/tests/integration/test_ttl_move/test.py @@ -8,7 +8,6 @@ import pytest from helpers.client import QueryRuntimeException from helpers.cluster import ClickHouseCluster -# test cluster = ClickHouseCluster(__file__) node1 = cluster.add_instance('node1', From 8f9ec303154fc3e04222a4c999ec6f8e8a1bcf01 Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 25 Sep 2020 19:23:57 +0300 Subject: [PATCH 070/321] Delete flaky test --- tests/integration/test_flaky_check/__init__.py | 0 tests/integration/test_flaky_check/test.py | 5 ----- 2 files changed, 5 deletions(-) delete mode 100644 tests/integration/test_flaky_check/__init__.py delete mode 100644 tests/integration/test_flaky_check/test.py diff --git a/tests/integration/test_flaky_check/__init__.py b/tests/integration/test_flaky_check/__init__.py deleted file mode 100644 index e69de29bb2d..00000000000 diff --git a/tests/integration/test_flaky_check/test.py b/tests/integration/test_flaky_check/test.py deleted file mode 100644 index 4da85151c46..00000000000 --- a/tests/integration/test_flaky_check/test.py +++ /dev/null @@ -1,5 +0,0 @@ -import random - - -def test_flaky(): - assert random.randint(0, 2) >= 1 From a351980717dcf31ca977d1e062ab0a1d1734b52b Mon Sep 17 00:00:00 2001 From: bharatnc Date: Sat, 5 Sep 2020 17:01:51 -0700 Subject: [PATCH 071/321] TextLog - add event_time_microseconds field --- base/loggers/ExtendedLogChannel.cpp | 1 + base/loggers/ExtendedLogChannel.h | 1 + base/loggers/OwnSplitChannel.cpp | 1 + src/Interpreters/TextLog.cpp | 3 +++ src/Interpreters/TextLog.h | 1 + 5 files changed, 7 insertions(+) diff --git a/base/loggers/ExtendedLogChannel.cpp b/base/loggers/ExtendedLogChannel.cpp index 421113b425f..4ab6ad5c96a 100644 --- a/base/loggers/ExtendedLogChannel.cpp +++ b/base/loggers/ExtendedLogChannel.cpp @@ -23,6 +23,7 @@ ExtendedLogMessage ExtendedLogMessage::getFrom(const Poco::Message & base) msg_ext.time_seconds = static_cast(tv.tv_sec); msg_ext.time_microseconds = static_cast(tv.tv_usec); + msg_ext.time_in_microseconds = static_cast((tv.tv_sec) * 1000000U + (tv.tv_usec)); if (current_thread) { diff --git a/base/loggers/ExtendedLogChannel.h b/base/loggers/ExtendedLogChannel.h index 3e9b61e8ae4..771a27159f0 100644 --- a/base/loggers/ExtendedLogChannel.h +++ b/base/loggers/ExtendedLogChannel.h @@ -23,6 +23,7 @@ public: uint32_t time_seconds = 0; uint32_t time_microseconds = 0; + uint64_t time_in_microseconds = 0; uint64_t thread_id = 0; std::string query_id; diff --git a/base/loggers/OwnSplitChannel.cpp b/base/loggers/OwnSplitChannel.cpp index 3ae928864fb..fc167ec3bf6 100644 --- a/base/loggers/OwnSplitChannel.cpp +++ b/base/loggers/OwnSplitChannel.cpp @@ -76,6 +76,7 @@ void OwnSplitChannel::logSplit(const Poco::Message & msg) TextLogElement elem; elem.event_time = msg_ext.time_seconds; + elem.event_time_microseconds = msg_ext.time_in_microseconds; elem.microseconds = msg_ext.time_microseconds; elem.thread_name = getThreadName(); diff --git a/src/Interpreters/TextLog.cpp b/src/Interpreters/TextLog.cpp index 243bf6d299a..d1cc6a052e8 100644 --- a/src/Interpreters/TextLog.cpp +++ b/src/Interpreters/TextLog.cpp @@ -2,6 +2,7 @@ #include #include #include +#include #include #include #include @@ -29,6 +30,7 @@ Block TextLogElement::createBlock() { {std::make_shared(), "event_date"}, {std::make_shared(), "event_time"}, + {std::make_shared(6), "event_time_microseconds"}, {std::make_shared(), "microseconds"}, {std::make_shared(std::make_shared()), "thread_name"}, @@ -52,6 +54,7 @@ void TextLogElement::appendToBlock(MutableColumns & columns) const columns[i++]->insert(DateLUT::instance().toDayNum(event_time)); columns[i++]->insert(event_time); + columns[i++]->insert(event_time_microseconds); columns[i++]->insert(microseconds); columns[i++]->insertData(thread_name.data(), thread_name.size()); diff --git a/src/Interpreters/TextLog.h b/src/Interpreters/TextLog.h index d5d1610dfb5..814b3c73044 100644 --- a/src/Interpreters/TextLog.h +++ b/src/Interpreters/TextLog.h @@ -9,6 +9,7 @@ using Poco::Message; struct TextLogElement { time_t event_time{}; + UInt64 event_time_microseconds{}; UInt32 microseconds; String thread_name; From 16ba71e2ec0b33abf865558ac112834014041f6d Mon Sep 17 00:00:00 2001 From: bharatnc Date: Wed, 9 Sep 2020 23:09:48 -0700 Subject: [PATCH 072/321] Rename test_test_log_level to include related tests --- .../{test_text_log_level => test_text_log_table}/__init__.py | 0 .../configs/config.d/text_log.xml | 0 .../{test_text_log_level => test_text_log_table}/test.py | 0 3 files changed, 0 insertions(+), 0 deletions(-) rename tests/integration/{test_text_log_level => test_text_log_table}/__init__.py (100%) rename tests/integration/{test_text_log_level => test_text_log_table}/configs/config.d/text_log.xml (100%) rename tests/integration/{test_text_log_level => test_text_log_table}/test.py (100%) diff --git a/tests/integration/test_text_log_level/__init__.py b/tests/integration/test_text_log_table/__init__.py similarity index 100% rename from tests/integration/test_text_log_level/__init__.py rename to tests/integration/test_text_log_table/__init__.py diff --git a/tests/integration/test_text_log_level/configs/config.d/text_log.xml b/tests/integration/test_text_log_table/configs/config.d/text_log.xml similarity index 100% rename from tests/integration/test_text_log_level/configs/config.d/text_log.xml rename to tests/integration/test_text_log_table/configs/config.d/text_log.xml diff --git a/tests/integration/test_text_log_level/test.py b/tests/integration/test_text_log_table/test.py similarity index 100% rename from tests/integration/test_text_log_level/test.py rename to tests/integration/test_text_log_table/test.py From 69e0d59c64a9b6c4c1b542733954ed3f34aa3651 Mon Sep 17 00:00:00 2001 From: bharatnc Date: Wed, 9 Sep 2020 23:11:58 -0700 Subject: [PATCH 073/321] TextLog - add tests for field event_time_microsec --- tests/integration/test_text_log_table/test.py | 26 ++++++++++++++++++- 1 file changed, 25 insertions(+), 1 deletion(-) diff --git a/tests/integration/test_text_log_table/test.py b/tests/integration/test_text_log_table/test.py index 44679481266..60c2e35893d 100644 --- a/tests/integration/test_text_log_table/test.py +++ b/tests/integration/test_text_log_table/test.py @@ -22,7 +22,7 @@ def start_cluster(): def test_basic(start_cluster): with pytest.raises(QueryRuntimeException): - # generates log with "Error" level + # generate log with "Error" level node.query('SELECT * FROM no_such_table') node.query('SYSTEM FLUSH LOGS') @@ -31,3 +31,27 @@ def test_basic(start_cluster): assert int(node.query("SELECT count() FROM system.text_log WHERE level = 'Debug'")) == 0 assert int(node.query("SELECT count() FROM system.text_log WHERE level = 'Information'")) >= 1 assert int(node.query("SELECT count() FROM system.text_log WHERE level = 'Error'")) >= 1 + +# compare the event_time and event_time_microseconds and test +# that they are exactly equal upto their seconds parts. +def test_field_event_time_microseconds(start_cluster): + with pytest.raises(QueryRuntimeException): + # generate log with "Error" level + node.query('SELECT * FROM no_such_table') + node.query('SYSTEM FLUSH LOGS') + equals_query = '''WITH ( + ( + SELECT event_time_microseconds + FROM system.text_log + ORDER BY event_time DESC + LIMIT 1 + ) AS time_with_microseconds, + ( + SELECT event_time + FROM system.text_log + ORDER BY event_time DESC + LIMIT 1 + ) AS time) + SELECT if(dateDiff('second', toDateTime(time_with_microseconds), toDateTime(time)) = 0, 'ok', 'fail') + ''' + assert 'ok\n' in node.query(equals_query) From b0d4fe511429d5487a8848a29bc2483b5d60276f Mon Sep 17 00:00:00 2001 From: bharatnc Date: Wed, 9 Sep 2020 23:36:32 -0700 Subject: [PATCH 074/321] TraceLog - add field event_time_microseconds --- src/Common/Stopwatch.h | 6 ++++++ src/Common/TraceCollector.cpp | 3 ++- src/Interpreters/TraceLog.cpp | 3 +++ src/Interpreters/TraceLog.h | 1 + 4 files changed, 12 insertions(+), 1 deletion(-) diff --git a/src/Common/Stopwatch.h b/src/Common/Stopwatch.h index 772caa75373..6a0d1f37eb4 100644 --- a/src/Common/Stopwatch.h +++ b/src/Common/Stopwatch.h @@ -13,6 +13,12 @@ inline UInt64 clock_gettime_ns(clockid_t clock_type = CLOCK_MONOTONIC) return UInt64(ts.tv_sec * 1000000000LL + ts.tv_nsec); } +inline UInt64 clock_gettime_microseconds(clockid_t clock_type = CLOCK_MONOTONIC) +{ + struct timespec ts; + clock_gettime(clock_type, &ts); + return UInt64((ts.tv_sec * 1000000LL) + (ts.tv_nsec / 1000)); +} /** Differs from Poco::Stopwatch only by using 'clock_gettime' instead of 'gettimeofday', * returns nanoseconds instead of microseconds, and also by other minor differencies. diff --git a/src/Common/TraceCollector.cpp b/src/Common/TraceCollector.cpp index c69c9d1dc1a..35e90a6e583 100644 --- a/src/Common/TraceCollector.cpp +++ b/src/Common/TraceCollector.cpp @@ -142,7 +142,8 @@ void TraceCollector::run() if (trace_log) { UInt64 time = clock_gettime_ns(CLOCK_REALTIME); - TraceLogElement element{time_t(time / 1000000000), time, trace_type, thread_id, query_id, trace, size}; + UInt64 time_in_microseconds = clock_gettime_microseconds(CLOCK_REALTIME); + TraceLogElement element{time_t(time / 1000000000), time_in_microseconds, time, trace_type, thread_id, query_id, trace, size}; trace_log->add(element); } } diff --git a/src/Interpreters/TraceLog.cpp b/src/Interpreters/TraceLog.cpp index f7e82032f49..40bcc0db445 100644 --- a/src/Interpreters/TraceLog.cpp +++ b/src/Interpreters/TraceLog.cpp @@ -4,6 +4,7 @@ #include #include #include +#include #include @@ -26,6 +27,7 @@ Block TraceLogElement::createBlock() { {std::make_shared(), "event_date"}, {std::make_shared(), "event_time"}, + {std::make_shared(6), "event_time_microseconds"}, {std::make_shared(), "timestamp_ns"}, {std::make_shared(), "revision"}, {std::make_shared(trace_values), "trace_type"}, @@ -42,6 +44,7 @@ void TraceLogElement::appendToBlock(MutableColumns & columns) const columns[i++]->insert(DateLUT::instance().toDayNum(event_time)); columns[i++]->insert(event_time); + columns[i++]->insert(event_time_microseconds); columns[i++]->insert(timestamp_ns); columns[i++]->insert(ClickHouseRevision::getVersionRevision()); columns[i++]->insert(static_cast(trace_type)); diff --git a/src/Interpreters/TraceLog.h b/src/Interpreters/TraceLog.h index 06f7f27299d..d694a6201f7 100644 --- a/src/Interpreters/TraceLog.h +++ b/src/Interpreters/TraceLog.h @@ -18,6 +18,7 @@ struct TraceLogElement static const TraceDataType::Values trace_values; time_t event_time{}; + UInt64 event_time_microseconds{}; UInt64 timestamp_ns{}; TraceType trace_type{}; UInt64 thread_id{}; From 380c2bee7a7436bbabb07fc721a217f9d09e9956 Mon Sep 17 00:00:00 2001 From: bharatnc Date: Wed, 9 Sep 2020 23:52:47 -0700 Subject: [PATCH 075/321] TraceLog - add tests for event_time_microseconds --- .../01473_event_time_microseconds.reference | 2 ++ .../01473_event_time_microseconds.sql | 17 +++++++++++++++++ 2 files changed, 19 insertions(+) diff --git a/tests/queries/0_stateless/01473_event_time_microseconds.reference b/tests/queries/0_stateless/01473_event_time_microseconds.reference index 8aa31f9ab6a..f6d90d4dd91 100644 --- a/tests/queries/0_stateless/01473_event_time_microseconds.reference +++ b/tests/queries/0_stateless/01473_event_time_microseconds.reference @@ -1,2 +1,4 @@ 01473_metric_log_table_event_start_time_microseconds_test ok +01473_trace_log_table_event_start_time_microseconds_test +ok diff --git a/tests/queries/0_stateless/01473_event_time_microseconds.sql b/tests/queries/0_stateless/01473_event_time_microseconds.sql index 138df77ffec..474250730d6 100644 --- a/tests/queries/0_stateless/01473_event_time_microseconds.sql +++ b/tests/queries/0_stateless/01473_event_time_microseconds.sql @@ -22,3 +22,20 @@ WITH ( LIMIT 1 ) AS time) SELECT if(dateDiff('second', toDateTime(time_with_microseconds), toDateTime(time)) = 0, 'ok', 'fail') + +select '01473_trace_log_table_event_start_time_microseconds_test'; +system flush logs; +WITH ( + ( + SELECT event_time_microseconds + FROM system.trace_log + ORDER BY event_time DESC + LIMIT 1 + ) AS time_with_microseconds, + ( + SELECT event_time + FROM system.trace_log + ORDER BY event_time DESC + LIMIT 1 + ) AS time) +SELECT if(dateDiff('second', toDateTime(time_with_microseconds), toDateTime(time)) = 0, 'ok', 'fail'); -- success From f252523442f9cac95e14c08dcbaecc85b271f892 Mon Sep 17 00:00:00 2001 From: bharatnc Date: Fri, 11 Sep 2020 09:51:24 -0700 Subject: [PATCH 076/321] Query,QueryThread Logs - add event_time_microseconds field --- src/Interpreters/QueryLog.cpp | 2 ++ src/Interpreters/QueryLog.h | 1 + src/Interpreters/QueryThreadLog.cpp | 2 ++ src/Interpreters/QueryThreadLog.h | 1 + src/Interpreters/ThreadStatusExt.cpp | 1 + src/Interpreters/executeQuery.cpp | 4 ++++ 6 files changed, 11 insertions(+) diff --git a/src/Interpreters/QueryLog.cpp b/src/Interpreters/QueryLog.cpp index 75e0fae615a..c2273a1db2c 100644 --- a/src/Interpreters/QueryLog.cpp +++ b/src/Interpreters/QueryLog.cpp @@ -39,6 +39,7 @@ Block QueryLogElement::createBlock() {std::move(query_status_datatype), "type"}, {std::make_shared(), "event_date"}, {std::make_shared(), "event_time"}, + {std::make_shared(6), "event_time_microseconds"}, {std::make_shared(), "query_start_time"}, {std::make_shared(6), "query_start_time_microseconds"}, {std::make_shared(), "query_duration_ms"}, @@ -97,6 +98,7 @@ void QueryLogElement::appendToBlock(MutableColumns & columns) const columns[i++]->insert(type); columns[i++]->insert(DateLUT::instance().toDayNum(event_time)); columns[i++]->insert(event_time); + columns[i++]->insert(event_time_microseconds); columns[i++]->insert(query_start_time); columns[i++]->insert(query_start_time_microseconds); columns[i++]->insert(query_duration_ms); diff --git a/src/Interpreters/QueryLog.h b/src/Interpreters/QueryLog.h index d1297feb3fb..9d42b787160 100644 --- a/src/Interpreters/QueryLog.h +++ b/src/Interpreters/QueryLog.h @@ -30,6 +30,7 @@ struct QueryLogElement /// Depending on the type of query and type of stage, not all the fields may be filled. time_t event_time{}; + UInt64 event_time_microseconds{}; time_t query_start_time{}; UInt64 query_start_time_microseconds{}; UInt64 query_duration_ms{}; diff --git a/src/Interpreters/QueryThreadLog.cpp b/src/Interpreters/QueryThreadLog.cpp index e5a8cf7c5cf..2ecb03d622a 100644 --- a/src/Interpreters/QueryThreadLog.cpp +++ b/src/Interpreters/QueryThreadLog.cpp @@ -23,6 +23,7 @@ Block QueryThreadLogElement::createBlock() return { {std::make_shared(), "event_date"}, {std::make_shared(), "event_time"}, + {std::make_shared(6), "event_time_microseconds"}, {std::make_shared(), "query_start_time"}, {std::make_shared(6), "query_start_time_microseconds"}, {std::make_shared(), "query_duration_ms"}, @@ -73,6 +74,7 @@ void QueryThreadLogElement::appendToBlock(MutableColumns & columns) const columns[i++]->insert(DateLUT::instance().toDayNum(event_time)); columns[i++]->insert(event_time); + columns[i++]->insert(event_time_microseconds); columns[i++]->insert(query_start_time); columns[i++]->insert(query_start_time_microseconds); columns[i++]->insert(query_duration_ms); diff --git a/src/Interpreters/QueryThreadLog.h b/src/Interpreters/QueryThreadLog.h index 66a480bfa0d..715902b29ad 100644 --- a/src/Interpreters/QueryThreadLog.h +++ b/src/Interpreters/QueryThreadLog.h @@ -16,6 +16,7 @@ namespace DB struct QueryThreadLogElement { time_t event_time{}; + UInt64 event_time_microseconds{}; /// When query was attached to current thread time_t query_start_time{}; /// same as above but adds microsecond precision diff --git a/src/Interpreters/ThreadStatusExt.cpp b/src/Interpreters/ThreadStatusExt.cpp index 0f610d9f6d2..ea33db3809b 100644 --- a/src/Interpreters/ThreadStatusExt.cpp +++ b/src/Interpreters/ThreadStatusExt.cpp @@ -323,6 +323,7 @@ void ThreadStatus::logToQueryThreadLog(QueryThreadLog & thread_log) QueryThreadLogElement elem; elem.event_time = time(nullptr); + elem.event_time_microseconds = getCurrentTimeMicroseconds(); elem.query_start_time = query_start_time; elem.query_start_time_microseconds = query_start_time_microseconds; elem.query_duration_ms = (getCurrentTimeNanoseconds() - query_start_time_nanoseconds) / 1000000U; diff --git a/src/Interpreters/executeQuery.cpp b/src/Interpreters/executeQuery.cpp index 59896065a87..68c0c2698c0 100644 --- a/src/Interpreters/executeQuery.cpp +++ b/src/Interpreters/executeQuery.cpp @@ -210,6 +210,7 @@ static void onExceptionBeforeStart(const String & query_for_logging, Context & c // event_time_microseconds from the same timespec. So it can be assumed that both of these // times are equal upto the precision of a second. elem.event_time = current_time; + elem.event_time_microseconds = current_time_microseconds; elem.query_start_time = current_time; elem.query_start_time_microseconds = current_time_microseconds; @@ -484,6 +485,7 @@ static std::tuple executeQueryImpl( elem.type = QueryLogElementType::QUERY_START; elem.event_time = current_time; + elem.event_time_microseconds = current_time_microseconds; elem.query_start_time = current_time; elem.query_start_time_microseconds = current_time_microseconds; @@ -555,6 +557,7 @@ static std::tuple executeQueryImpl( elem.type = QueryLogElementType::QUERY_FINISH; elem.event_time = time(nullptr); + elem.event_time_microseconds = getCurrentTimeMicroseconds(); status_info_to_query_log(elem, info, ast); @@ -616,6 +619,7 @@ static std::tuple executeQueryImpl( elem.type = QueryLogElementType::EXCEPTION_WHILE_PROCESSING; elem.event_time = time(nullptr); + elem.event_time_microseconds = getCurrentTimeMicroseconds(); elem.query_duration_ms = 1000 * (elem.event_time - elem.query_start_time); elem.exception_code = getCurrentExceptionCode(); elem.exception = getCurrentExceptionMessage(false); From e619bd26e610bed85052a1a495773a64ffb261b0 Mon Sep 17 00:00:00 2001 From: bharatnc Date: Fri, 11 Sep 2020 10:10:44 -0700 Subject: [PATCH 077/321] Query,QueryThread Logs - add tests --- .../01473_event_time_microseconds.reference | 4 ++ .../01473_event_time_microseconds.sql | 46 ++++++++++++++++--- 2 files changed, 44 insertions(+), 6 deletions(-) diff --git a/tests/queries/0_stateless/01473_event_time_microseconds.reference b/tests/queries/0_stateless/01473_event_time_microseconds.reference index f6d90d4dd91..6d95c8c1b81 100644 --- a/tests/queries/0_stateless/01473_event_time_microseconds.reference +++ b/tests/queries/0_stateless/01473_event_time_microseconds.reference @@ -2,3 +2,7 @@ ok 01473_trace_log_table_event_start_time_microseconds_test ok +01473_query_log_table_event_start_time_microseconds_test +ok +01473_query_thread_log_table_event_start_time_microseconds_test +ok diff --git a/tests/queries/0_stateless/01473_event_time_microseconds.sql b/tests/queries/0_stateless/01473_event_time_microseconds.sql index 474250730d6..537664b1ef0 100644 --- a/tests/queries/0_stateless/01473_event_time_microseconds.sql +++ b/tests/queries/0_stateless/01473_event_time_microseconds.sql @@ -3,10 +3,10 @@ -- an integration test as those metrics take 60s by default to be updated. -- Refer: tests/integration/test_asynchronous_metric_log_table. -set log_queries = 1; +SET log_queries = 1; -select '01473_metric_log_table_event_start_time_microseconds_test'; -system flush logs; +SELECT '01473_metric_log_table_event_start_time_microseconds_test'; +SYSTEM FLUSH LOGS; -- query assumes that the event_time field is accurate. WITH ( ( @@ -21,10 +21,10 @@ WITH ( ORDER BY event_time DESC LIMIT 1 ) AS time) -SELECT if(dateDiff('second', toDateTime(time_with_microseconds), toDateTime(time)) = 0, 'ok', 'fail') +SELECT if(dateDiff('second', toDateTime(time_with_microseconds), toDateTime(time)) = 0, 'ok', 'fail'); -select '01473_trace_log_table_event_start_time_microseconds_test'; -system flush logs; +SELECT '01473_trace_log_table_event_start_time_microseconds_test'; +SYSTEM FLUSH LOGS; WITH ( ( SELECT event_time_microseconds @@ -39,3 +39,37 @@ WITH ( LIMIT 1 ) AS time) SELECT if(dateDiff('second', toDateTime(time_with_microseconds), toDateTime(time)) = 0, 'ok', 'fail'); -- success + +SELECT '01473_query_log_table_event_start_time_microseconds_test'; +SYSTEM FLUSH LOGS; +WITH ( + ( + SELECT event_time_microseconds + FROM system.query_log + ORDER BY event_time DESC + LIMIT 1 + ) AS time_with_microseconds, + ( + SELECT event_time + FROM system.query_log + ORDER BY event_time DESC + LIMIT 1 + ) AS time) +SELECT if(dateDiff('second', toDateTime(time_with_microseconds), toDateTime(time)) = 0, 'ok', 'fail'); -- success + +SELECT '01473_query_thread_log_table_event_start_time_microseconds_test'; +SYSTEM FLUSH LOGS; +WITH ( + ( + SELECT event_time_microseconds + FROM system.query_thread_log + ORDER BY event_time DESC + LIMIT 1 + ) AS time_with_microseconds, + ( + SELECT event_time + FROM system.query_thread_log + ORDER BY event_time DESC + LIMIT 1 + ) AS time) +SELECT if(dateDiff('second', toDateTime(time_with_microseconds), toDateTime(time)) = 0, 'ok', 'fail'); -- success From 5359e3bf705c56d300e0527789872d97037aab7e Mon Sep 17 00:00:00 2001 From: bharatnc Date: Fri, 11 Sep 2020 10:34:27 -0700 Subject: [PATCH 078/321] Field event_time_microseconds - add documentation --- docs/en/operations/system-tables/query_log.md | 90 ++++++++++--------- .../system-tables/query_thread_log.md | 80 +++++++++-------- docs/en/operations/system-tables/text_log.md | 25 ++++++ docs/en/operations/system-tables/trace_log.md | 19 ++-- 4 files changed, 125 insertions(+), 89 deletions(-) diff --git a/docs/en/operations/system-tables/query_log.md b/docs/en/operations/system-tables/query_log.md index 72927b5a7e9..8b663475fa8 100644 --- a/docs/en/operations/system-tables/query_log.md +++ b/docs/en/operations/system-tables/query_log.md @@ -33,6 +33,7 @@ Columns: - `'ExceptionWhileProcessing' = 4` — Exception during the query execution. - `event_date` ([Date](../../sql-reference/data-types/date.md)) — Query starting date. - `event_time` ([DateTime](../../sql-reference/data-types/datetime.md)) — Query starting time. +- `event_time_microseconds` ([DateTime](../../sql-reference/data-types/datetime.md)) — Query starting time with microseconds precision. - `query_start_time` ([DateTime](../../sql-reference/data-types/datetime.md)) — Start time of query execution. - `query_start_time_microseconds` ([DateTime64](../../sql-reference/data-types/datetime64.md)) — Start time of query execution with microsecond precision. - `query_duration_ms` ([UInt64](../../sql-reference/data-types/int-uint.md#uint-ranges)) — Duration of query execution in milliseconds. @@ -84,54 +85,57 @@ Columns: **Example** ``` sql -SELECT * FROM system.query_log LIMIT 1 FORMAT Vertical; +SELECT * FROM system.query_log LIMIT 1 \G ``` ``` text Row 1: ────── -type: QueryStart -event_date: 2020-05-13 -event_time: 2020-05-13 14:02:28 -query_start_time: 2020-05-13 14:02:28 -query_duration_ms: 0 -read_rows: 0 -read_bytes: 0 -written_rows: 0 -written_bytes: 0 -result_rows: 0 -result_bytes: 0 -memory_usage: 0 -query: SELECT 1 -exception_code: 0 -exception: -stack_trace: -is_initial_query: 1 -user: default -query_id: 5e834082-6f6d-4e34-b47b-cd1934f4002a -address: ::ffff:127.0.0.1 -port: 57720 -initial_user: default -initial_query_id: 5e834082-6f6d-4e34-b47b-cd1934f4002a -initial_address: ::ffff:127.0.0.1 -initial_port: 57720 -interface: 1 -os_user: bayonet -client_hostname: clickhouse.ru-central1.internal -client_name: ClickHouse client -client_revision: 54434 -client_version_major: 20 -client_version_minor: 4 -client_version_patch: 1 -http_method: 0 -http_user_agent: -quota_key: -revision: 54434 -thread_ids: [] -ProfileEvents.Names: [] -ProfileEvents.Values: [] -Settings.Names: ['use_uncompressed_cache','load_balancing','log_queries','max_memory_usage'] -Settings.Values: ['0','random','1','10000000000'] +type: QueryStart +event_date: 2020-09-11 +event_time: 2020-09-11 10:08:17 +event_time_microseconds: 2020-09-11 10:08:17.063321 +query_start_time: 2020-09-11 10:08:17 +query_start_time_microseconds: 2020-09-11 10:08:17.063321 +query_duration_ms: 0 +read_rows: 0 +read_bytes: 0 +written_rows: 0 +written_bytes: 0 +result_rows: 0 +result_bytes: 0 +memory_usage: 0 +current_database: default +query: INSERT INTO test1 VALUES +exception_code: 0 +exception: +stack_trace: +is_initial_query: 1 +user: default +query_id: 50a320fd-85a8-49b8-8761-98a86bcbacef +address: ::ffff:127.0.0.1 +port: 33452 +initial_user: default +initial_query_id: 50a320fd-85a8-49b8-8761-98a86bcbacef +initial_address: ::ffff:127.0.0.1 +initial_port: 33452 +interface: 1 +os_user: bharatnc +client_hostname: tower +client_name: ClickHouse +client_revision: 54437 +client_version_major: 20 +client_version_minor: 7 +client_version_patch: 2 +http_method: 0 +http_user_agent: +quota_key: +revision: 54440 +thread_ids: [] +ProfileEvents.Names: [] +ProfileEvents.Values: [] +Settings.Names: ['use_uncompressed_cache','load_balancing','log_queries','max_memory_usage','allow_introspection_functions'] +Settings.Values: ['0','random','1','10000000000','1'] ``` **See Also** diff --git a/docs/en/operations/system-tables/query_thread_log.md b/docs/en/operations/system-tables/query_thread_log.md index 3dcd05c4cc3..84408edd117 100644 --- a/docs/en/operations/system-tables/query_thread_log.md +++ b/docs/en/operations/system-tables/query_thread_log.md @@ -15,6 +15,7 @@ Columns: - `event_date` ([Date](../../sql-reference/data-types/date.md)) — The date when the thread has finished execution of the query. - `event_time` ([DateTime](../../sql-reference/data-types/datetime.md)) — The date and time when the thread has finished execution of the query. +- `event_time_microsecinds` ([DateTime](../../sql-reference/data-types/datetime.md)) — The date and time when the thread has finished execution of the query with microseconds precision. - `query_start_time` ([DateTime](../../sql-reference/data-types/datetime.md)) — Start time of query execution. - `query_start_time_microseconds` ([DateTime64](../../sql-reference/data-types/datetime64.md)) — Start time of query execution with microsecond precision. - `query_duration_ms` ([UInt64](../../sql-reference/data-types/int-uint.md#uint-ranges)) — Duration of query execution. @@ -63,50 +64,51 @@ Columns: **Example** ``` sql - SELECT * FROM system.query_thread_log LIMIT 1 FORMAT Vertical + SELECT * FROM system.query_thread_log LIMIT 1 \G ``` ``` text Row 1: ────── -event_date: 2020-05-13 -event_time: 2020-05-13 14:02:28 -query_start_time: 2020-05-13 14:02:28 -query_duration_ms: 0 -read_rows: 1 -read_bytes: 1 -written_rows: 0 -written_bytes: 0 -memory_usage: 0 -peak_memory_usage: 0 -thread_name: QueryPipelineEx -thread_id: 28952 -master_thread_id: 28924 -query: SELECT 1 -is_initial_query: 1 -user: default -query_id: 5e834082-6f6d-4e34-b47b-cd1934f4002a -address: ::ffff:127.0.0.1 -port: 57720 -initial_user: default -initial_query_id: 5e834082-6f6d-4e34-b47b-cd1934f4002a -initial_address: ::ffff:127.0.0.1 -initial_port: 57720 -interface: 1 -os_user: bayonet -client_hostname: clickhouse.ru-central1.internal -client_name: ClickHouse client -client_revision: 54434 -client_version_major: 20 -client_version_minor: 4 -client_version_patch: 1 -http_method: 0 -http_user_agent: -quota_key: -revision: 54434 -ProfileEvents.Names: ['ContextLock','RealTimeMicroseconds','UserTimeMicroseconds','OSCPUWaitMicroseconds','OSCPUVirtualTimeMicroseconds'] -ProfileEvents.Values: [1,97,81,5,81] -... +event_date: 2020-09-11 +event_time: 2020-09-11 10:08:17 +event_time_microseconds: 2020-09-11 10:08:17.134042 +query_start_time: 2020-09-11 10:08:17 +query_start_time_microseconds: 2020-09-11 10:08:17.063150 +query_duration_ms: 70 +read_rows: 0 +read_bytes: 0 +written_rows: 1 +written_bytes: 12 +memory_usage: 4300844 +peak_memory_usage: 4300844 +thread_name: TCPHandler +thread_id: 638133 +master_thread_id: 638133 +query: INSERT INTO test1 VALUES +is_initial_query: 1 +user: default +query_id: 50a320fd-85a8-49b8-8761-98a86bcbacef +address: ::ffff:127.0.0.1 +port: 33452 +initial_user: default +initial_query_id: 50a320fd-85a8-49b8-8761-98a86bcbacef +initial_address: ::ffff:127.0.0.1 +initial_port: 33452 +interface: 1 +os_user: bharatnc +client_hostname: tower +client_name: ClickHouse +client_revision: 54437 +client_version_major: 20 +client_version_minor: 7 +client_version_patch: 2 +http_method: 0 +http_user_agent: +quota_key: +revision: 54440 +ProfileEvents.Names: ['Query','InsertQuery','FileOpen','WriteBufferFromFileDescriptorWrite','WriteBufferFromFileDescriptorWriteBytes','ReadCompressedBytes','CompressedReadBufferBlocks','CompressedReadBufferBytes','IOBufferAllocs','IOBufferAllocBytes','FunctionExecute','CreatedWriteBufferOrdinary','DiskWriteElapsedMicroseconds','NetworkReceiveElapsedMicroseconds','NetworkSendElapsedMicroseconds','InsertedRows','InsertedBytes','SelectedRows','SelectedBytes','MergeTreeDataWriterRows','MergeTreeDataWriterUncompressedBytes','MergeTreeDataWriterCompressedBytes','MergeTreeDataWriterBlocks','MergeTreeDataWriterBlocksAlreadySorted','ContextLock','RWLockAcquiredReadLocks','RealTimeMicroseconds','UserTimeMicroseconds','SoftPageFaults','OSCPUVirtualTimeMicroseconds','OSWriteBytes','OSReadChars','OSWriteChars'] +ProfileEvents.Values: [1,1,11,11,591,148,3,71,29,6533808,1,11,72,18,47,1,12,1,12,1,12,189,1,1,10,2,70853,2748,49,2747,45056,422,1520] ``` **See Also** diff --git a/docs/en/operations/system-tables/text_log.md b/docs/en/operations/system-tables/text_log.md index bd92519b96b..3c3281ff8c6 100644 --- a/docs/en/operations/system-tables/text_log.md +++ b/docs/en/operations/system-tables/text_log.md @@ -6,6 +6,7 @@ Columns: - `event_date` (Date) — Date of the entry. - `event_time` (DateTime) — Time of the entry. +- `event_time_microseconds` (DateTime) — Time of the entry with microseconds precision. - `microseconds` (UInt32) — Microseconds of the entry. - `thread_name` (String) — Name of the thread from which the logging was done. - `thread_id` (UInt64) — OS thread ID. @@ -25,4 +26,28 @@ Columns: - `source_file` (LowCardinality(String)) — Source file from which the logging was done. - `source_line` (UInt64) — Source line from which the logging was done. +**Example** + +``` sql +SELECT * FROM system.text_log LIMIT 1 \G +``` + +``` text +Row 1: +────── +event_date: 2020-09-10 +event_time: 2020-09-10 11:23:07 +event_time_microseconds: 2020-09-10 11:23:07.871397 +microseconds: 871397 +thread_name: clickhouse-serv +thread_id: 564917 +level: Information +query_id: +logger_name: DNSCacheUpdater +message: Update period 15 seconds +revision: 54440 +source_file: /ClickHouse/src/Interpreters/DNSCacheUpdater.cpp; void DB::DNSCacheUpdater::start() +source_line: 45 +``` + [Original article](https://clickhouse.tech/docs/en/operations/system_tables/text_log) \ No newline at end of file diff --git a/docs/en/operations/system-tables/trace_log.md b/docs/en/operations/system-tables/trace_log.md index b911fdd2263..0fb967822d2 100644 --- a/docs/en/operations/system-tables/trace_log.md +++ b/docs/en/operations/system-tables/trace_log.md @@ -12,6 +12,8 @@ Columns: - `event_time` ([DateTime](../../sql-reference/data-types/datetime.md)) — Timestamp of the sampling moment. +- `event_time_microseconds` ([DateTime](../../sql-reference/data-types/datetime.md)) — Timestamp of the sampling moment with microseconds precision. + - `timestamp_ns` ([UInt64](../../sql-reference/data-types/int-uint.md)) — Timestamp of the sampling moment in nanoseconds. - `revision` ([UInt32](../../sql-reference/data-types/int-uint.md)) — ClickHouse server build revision. @@ -38,13 +40,16 @@ SELECT * FROM system.trace_log LIMIT 1 \G ``` text Row 1: ────── -event_date: 2019-11-15 -event_time: 2019-11-15 15:09:38 -revision: 54428 -timer_type: Real -thread_number: 48 -query_id: acc4d61f-5bd1-4a3e-bc91-2180be37c915 -trace: [94222141367858,94222152240175,94222152325351,94222152329944,94222152330796,94222151449980,94222144088167,94222151682763,94222144088167,94222151682763,94222144088167,94222144058283,94222144059248,94222091840750,94222091842302,94222091831228,94222189631488,140509950166747,140509942945935] +event_date: 2020-09-10 +event_time: 2020-09-10 11:23:09 +event_time_microseconds: 2020-09-10 11:23:09.872924 +timestamp_ns: 1599762189872924510 +revision: 54440 +trace_type: Memory +thread_id: 564963 +query_id: +trace: [371912858,371912789,371798468,371799717,371801313,371790250,624462773,566365041,566440261,566445834,566460071,566459914,566459842,566459580,566459469,566459389,566459341,566455774,371993941,371988245,372158848,372187428,372187309,372187093,372185478,140222123165193,140222122205443] +size: 5244400 ``` [Original article](https://clickhouse.tech/docs/en/operations/system_tables/trace_log) \ No newline at end of file From 1cdcaaa817221898b46d52921ee1aba81f1cd0e4 Mon Sep 17 00:00:00 2001 From: bharatnc Date: Fri, 11 Sep 2020 20:43:47 -0700 Subject: [PATCH 079/321] TraceLog - replace unit test with integration test --- .../test_trace_log_table/__init__.py | 0 .../configs/trace_log.xml | 8 +++ .../integration/test_trace_log_table/test.py | 65 +++++++++++++++++++ .../01473_event_time_microseconds.reference | 2 - .../01473_event_time_microseconds.sql | 17 ----- 5 files changed, 73 insertions(+), 19 deletions(-) create mode 100644 tests/integration/test_trace_log_table/__init__.py create mode 100644 tests/integration/test_trace_log_table/configs/trace_log.xml create mode 100644 tests/integration/test_trace_log_table/test.py diff --git a/tests/integration/test_trace_log_table/__init__.py b/tests/integration/test_trace_log_table/__init__.py new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/integration/test_trace_log_table/configs/trace_log.xml b/tests/integration/test_trace_log_table/configs/trace_log.xml new file mode 100644 index 00000000000..48a93dc0cb9 --- /dev/null +++ b/tests/integration/test_trace_log_table/configs/trace_log.xml @@ -0,0 +1,8 @@ + + + system + trace_log
+ toYYYYMM(event_date) + 1000 +
+
diff --git a/tests/integration/test_trace_log_table/test.py b/tests/integration/test_trace_log_table/test.py new file mode 100644 index 00000000000..9f68e4b4443 --- /dev/null +++ b/tests/integration/test_trace_log_table/test.py @@ -0,0 +1,65 @@ +# pylint: disable=unused-argument +# pylint: disable=redefined-outer-name +import time + +import pytest +from helpers.cluster import ClickHouseCluster + +cluster = ClickHouseCluster(__file__) + +node = cluster.add_instance('node', with_zookeeper=True, main_configs=["configs/trace_log.xml"]) + + +@pytest.fixture(scope='module') +def start_cluster(): + try: + cluster.start() + + yield cluster + finally: + cluster.shutdown() + + +# Tests that the event_time_microseconds field in the system.trace_log table gets populated. +# To make the tests work better, the default flush_interval_milliseconds is being overridden +# to 1000 ms. Also the query_profiler_real_time_period_ns and the query_profiler_cpu_time_period_ns +# are set to suitable values so that traces are properly populated. Test compares the event_time and +# event_time_microseconds fields and asserts that they are exactly equal upto their seconds parts. Also +# one additional test to ensure that the count(event_time_microseconds) is > 0; +def test_field_event_time_microseconds(start_cluster): + node.query('SET query_profiler_real_time_period_ns = 0;') + node.query('SET query_profiler_cpu_time_period_ns = 1000000;') + node.query('SET log_queries = 1;') + node.query("CREATE DATABASE replica;") + query_create = '''CREATE TABLE replica.test + ( + id Int64, + event_time DateTime + ) + Engine=MergeTree() + PARTITION BY toYYYYMMDD(event_time) + ORDER BY id;''' + node.query(query_create) + node.query('''INSERT INTO replica.test VALUES (1, now())''') + node.query("SYSTEM FLUSH LOGS;") + # TODO: is sleep necessary ? + time.sleep(1) + # query assumes that the event_time field is already accurate + equals_query = '''WITH ( + ( + SELECT event_time_microseconds + FROM system.trace_log + ORDER BY event_time DESC + LIMIT 1 + ) AS time_with_microseconds, + ( + SELECT event_time + FROM system.trace_log + ORDER BY event_time DESC + LIMIT 1 + ) AS t) + SELECT if(dateDiff('second', toDateTime(time_with_microseconds), toDateTime(t)) = 0, 'ok', 'fail') + ''' + assert 'ok\n' in node.query(equals_query) + assert 'ok\n' in node.query( + '''SELECT if((SELECT COUNT(event_time_microseconds) FROM system.trace_log) > 0, 'ok', 'fail')''') diff --git a/tests/queries/0_stateless/01473_event_time_microseconds.reference b/tests/queries/0_stateless/01473_event_time_microseconds.reference index 6d95c8c1b81..7da9f21de2f 100644 --- a/tests/queries/0_stateless/01473_event_time_microseconds.reference +++ b/tests/queries/0_stateless/01473_event_time_microseconds.reference @@ -1,7 +1,5 @@ 01473_metric_log_table_event_start_time_microseconds_test ok -01473_trace_log_table_event_start_time_microseconds_test -ok 01473_query_log_table_event_start_time_microseconds_test ok 01473_query_thread_log_table_event_start_time_microseconds_test diff --git a/tests/queries/0_stateless/01473_event_time_microseconds.sql b/tests/queries/0_stateless/01473_event_time_microseconds.sql index 537664b1ef0..48d620be197 100644 --- a/tests/queries/0_stateless/01473_event_time_microseconds.sql +++ b/tests/queries/0_stateless/01473_event_time_microseconds.sql @@ -23,23 +23,6 @@ WITH ( ) AS time) SELECT if(dateDiff('second', toDateTime(time_with_microseconds), toDateTime(time)) = 0, 'ok', 'fail'); -SELECT '01473_trace_log_table_event_start_time_microseconds_test'; -SYSTEM FLUSH LOGS; -WITH ( - ( - SELECT event_time_microseconds - FROM system.trace_log - ORDER BY event_time DESC - LIMIT 1 - ) AS time_with_microseconds, - ( - SELECT event_time - FROM system.trace_log - ORDER BY event_time DESC - LIMIT 1 - ) AS time) -SELECT if(dateDiff('second', toDateTime(time_with_microseconds), toDateTime(time)) = 0, 'ok', 'fail'); -- success - SELECT '01473_query_log_table_event_start_time_microseconds_test'; SYSTEM FLUSH LOGS; WITH ( From 3f3b68d676b6f1e08fddf29f4010dc93b52bfaef Mon Sep 17 00:00:00 2001 From: bharatnc Date: Sun, 13 Sep 2020 01:04:56 -0700 Subject: [PATCH 080/321] TraceLog - mark to skip integration test --- tests/integration/test_trace_log_table/test.py | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/integration/test_trace_log_table/test.py b/tests/integration/test_trace_log_table/test.py index 9f68e4b4443..c1c8934b9f2 100644 --- a/tests/integration/test_trace_log_table/test.py +++ b/tests/integration/test_trace_log_table/test.py @@ -26,6 +26,7 @@ def start_cluster(): # are set to suitable values so that traces are properly populated. Test compares the event_time and # event_time_microseconds fields and asserts that they are exactly equal upto their seconds parts. Also # one additional test to ensure that the count(event_time_microseconds) is > 0; +@pytest.mark.skip(reason="TODO: system.trace_log not populated in time on CI but works fine on dev") def test_field_event_time_microseconds(start_cluster): node.query('SET query_profiler_real_time_period_ns = 0;') node.query('SET query_profiler_cpu_time_period_ns = 1000000;') From 324a06145cd55e3538077dd1e236e3648f580cc8 Mon Sep 17 00:00:00 2001 From: bharatnc Date: Mon, 14 Sep 2020 18:40:06 -0700 Subject: [PATCH 081/321] EventTime - construct time in s and us from same timespec --- src/Interpreters/ThreadStatusExt.cpp | 9 +++++++-- src/Interpreters/executeQuery.cpp | 20 +++++++++++++++----- 2 files changed, 22 insertions(+), 7 deletions(-) diff --git a/src/Interpreters/ThreadStatusExt.cpp b/src/Interpreters/ThreadStatusExt.cpp index ea33db3809b..d820094db1d 100644 --- a/src/Interpreters/ThreadStatusExt.cpp +++ b/src/Interpreters/ThreadStatusExt.cpp @@ -322,8 +322,13 @@ void ThreadStatus::logToQueryThreadLog(QueryThreadLog & thread_log) { QueryThreadLogElement elem; - elem.event_time = time(nullptr); - elem.event_time_microseconds = getCurrentTimeMicroseconds(); + // event_time and event_time_microseconds are being constructed from the same timespec + // to ensure that both the times are equal upto the precision of a second. + struct timespec ts; + clock_gettime(CLOCK_MONOTONIC, &ts); + + elem.event_time = ts.tv_sec; + elem.event_time_microseconds = UInt64((ts.tv_sec * 1000000LL) + (ts.tv_nsec / 1000)); elem.query_start_time = query_start_time; elem.query_start_time_microseconds = query_start_time_microseconds; elem.query_duration_ms = (getCurrentTimeNanoseconds() - query_start_time_nanoseconds) / 1000000U; diff --git a/src/Interpreters/executeQuery.cpp b/src/Interpreters/executeQuery.cpp index 68c0c2698c0..3f2c88b3678 100644 --- a/src/Interpreters/executeQuery.cpp +++ b/src/Interpreters/executeQuery.cpp @@ -319,8 +319,9 @@ static std::tuple executeQueryImpl( auto query_for_logging = prepareQueryForLogging(query, context); logQuery(query_for_logging, context, internal); - if (!internal) + if (!internal) { onExceptionBeforeStart(query_for_logging, context, current_time, current_time_microseconds, ast); + } throw; } @@ -556,9 +557,13 @@ static std::tuple executeQueryImpl( elem.type = QueryLogElementType::QUERY_FINISH; - elem.event_time = time(nullptr); - elem.event_time_microseconds = getCurrentTimeMicroseconds(); + // event_time and event-time_microseconds are being constructed from the same timespec + // to ensure that both the times are equal upto the precision of a second. + struct timespec tspec; + clock_gettime(CLOCK_MONOTONIC, &tspec); + elem.event_time = tspec.tv_sec; + elem.event_time_microseconds = UInt64((tspec.tv_sec * 1000000LL) + (tspec.tv_nsec / 1000)); status_info_to_query_log(elem, info, ast); auto progress_callback = context.getProgressCallback(); @@ -618,8 +623,13 @@ static std::tuple executeQueryImpl( elem.type = QueryLogElementType::EXCEPTION_WHILE_PROCESSING; - elem.event_time = time(nullptr); - elem.event_time_microseconds = getCurrentTimeMicroseconds(); + // event_time and event_time_microseconds are being constructed from the timespec + // to ensure that both the times will be equal upto the precision of a second. + struct timespec tspec; + clock_gettime(CLOCK_MONOTONIC, &tspec); + + elem.event_time = tspec.tv_sec; + elem.event_time_microseconds = UInt64((tspec.tv_sec * 1000000LL) + (tspec.tv_nsec / 1000)); elem.query_duration_ms = 1000 * (elem.event_time - elem.query_start_time); elem.exception_code = getCurrentExceptionCode(); elem.exception = getCurrentExceptionMessage(false); From 0df3fa00dbab7d7c8718f5ebad16a14a263f9e0c Mon Sep 17 00:00:00 2001 From: bharatnc Date: Mon, 14 Sep 2020 18:57:31 -0700 Subject: [PATCH 082/321] TraceLog - revert back to functional test --- .../test_trace_log_table/__init__.py | 0 .../configs/trace_log.xml | 8 --- .../integration/test_trace_log_table/test.py | 66 ------------------- .../01473_event_time_microseconds.reference | 2 + .../01473_event_time_microseconds.sql | 22 +++++++ 5 files changed, 24 insertions(+), 74 deletions(-) delete mode 100644 tests/integration/test_trace_log_table/__init__.py delete mode 100644 tests/integration/test_trace_log_table/configs/trace_log.xml delete mode 100644 tests/integration/test_trace_log_table/test.py diff --git a/tests/integration/test_trace_log_table/__init__.py b/tests/integration/test_trace_log_table/__init__.py deleted file mode 100644 index e69de29bb2d..00000000000 diff --git a/tests/integration/test_trace_log_table/configs/trace_log.xml b/tests/integration/test_trace_log_table/configs/trace_log.xml deleted file mode 100644 index 48a93dc0cb9..00000000000 --- a/tests/integration/test_trace_log_table/configs/trace_log.xml +++ /dev/null @@ -1,8 +0,0 @@ - - - system - trace_log
- toYYYYMM(event_date) - 1000 -
-
diff --git a/tests/integration/test_trace_log_table/test.py b/tests/integration/test_trace_log_table/test.py deleted file mode 100644 index c1c8934b9f2..00000000000 --- a/tests/integration/test_trace_log_table/test.py +++ /dev/null @@ -1,66 +0,0 @@ -# pylint: disable=unused-argument -# pylint: disable=redefined-outer-name -import time - -import pytest -from helpers.cluster import ClickHouseCluster - -cluster = ClickHouseCluster(__file__) - -node = cluster.add_instance('node', with_zookeeper=True, main_configs=["configs/trace_log.xml"]) - - -@pytest.fixture(scope='module') -def start_cluster(): - try: - cluster.start() - - yield cluster - finally: - cluster.shutdown() - - -# Tests that the event_time_microseconds field in the system.trace_log table gets populated. -# To make the tests work better, the default flush_interval_milliseconds is being overridden -# to 1000 ms. Also the query_profiler_real_time_period_ns and the query_profiler_cpu_time_period_ns -# are set to suitable values so that traces are properly populated. Test compares the event_time and -# event_time_microseconds fields and asserts that they are exactly equal upto their seconds parts. Also -# one additional test to ensure that the count(event_time_microseconds) is > 0; -@pytest.mark.skip(reason="TODO: system.trace_log not populated in time on CI but works fine on dev") -def test_field_event_time_microseconds(start_cluster): - node.query('SET query_profiler_real_time_period_ns = 0;') - node.query('SET query_profiler_cpu_time_period_ns = 1000000;') - node.query('SET log_queries = 1;') - node.query("CREATE DATABASE replica;") - query_create = '''CREATE TABLE replica.test - ( - id Int64, - event_time DateTime - ) - Engine=MergeTree() - PARTITION BY toYYYYMMDD(event_time) - ORDER BY id;''' - node.query(query_create) - node.query('''INSERT INTO replica.test VALUES (1, now())''') - node.query("SYSTEM FLUSH LOGS;") - # TODO: is sleep necessary ? - time.sleep(1) - # query assumes that the event_time field is already accurate - equals_query = '''WITH ( - ( - SELECT event_time_microseconds - FROM system.trace_log - ORDER BY event_time DESC - LIMIT 1 - ) AS time_with_microseconds, - ( - SELECT event_time - FROM system.trace_log - ORDER BY event_time DESC - LIMIT 1 - ) AS t) - SELECT if(dateDiff('second', toDateTime(time_with_microseconds), toDateTime(t)) = 0, 'ok', 'fail') - ''' - assert 'ok\n' in node.query(equals_query) - assert 'ok\n' in node.query( - '''SELECT if((SELECT COUNT(event_time_microseconds) FROM system.trace_log) > 0, 'ok', 'fail')''') diff --git a/tests/queries/0_stateless/01473_event_time_microseconds.reference b/tests/queries/0_stateless/01473_event_time_microseconds.reference index 7da9f21de2f..6d95c8c1b81 100644 --- a/tests/queries/0_stateless/01473_event_time_microseconds.reference +++ b/tests/queries/0_stateless/01473_event_time_microseconds.reference @@ -1,5 +1,7 @@ 01473_metric_log_table_event_start_time_microseconds_test ok +01473_trace_log_table_event_start_time_microseconds_test +ok 01473_query_log_table_event_start_time_microseconds_test ok 01473_query_thread_log_table_event_start_time_microseconds_test diff --git a/tests/queries/0_stateless/01473_event_time_microseconds.sql b/tests/queries/0_stateless/01473_event_time_microseconds.sql index 48d620be197..8a545af9efa 100644 --- a/tests/queries/0_stateless/01473_event_time_microseconds.sql +++ b/tests/queries/0_stateless/01473_event_time_microseconds.sql @@ -23,6 +23,28 @@ WITH ( ) AS time) SELECT if(dateDiff('second', toDateTime(time_with_microseconds), toDateTime(time)) = 0, 'ok', 'fail'); +SELECT '01473_trace_log_table_event_start_time_microseconds_test'; +SET log_queries = 1; +SET query_profiler_real_time_period_ns = 0; +SET query_profiler_cpu_time_period_ns = 1000000; +-- a long enough query to trigger the query profiler and to record trace log +SELECT sleep(2) FORMAT Null; +SYSTEM FLUSH LOGS; +WITH ( + ( + SELECT event_time_microseconds + FROM system.trace_log + ORDER BY event_time DESC + LIMIT 1 + ) AS time_with_microseconds, + ( + SELECT event_time + FROM system.trace_log + ORDER BY event_time DESC + LIMIT 1 + ) AS t) +SELECT if(dateDiff('second', toDateTime(time_with_microseconds), toDateTime(t)) = 0, 'ok', 'fail'); -- success + SELECT '01473_query_log_table_event_start_time_microseconds_test'; SYSTEM FLUSH LOGS; WITH ( From 24ea6b2380debd0108f62064fe989ef01068f389 Mon Sep 17 00:00:00 2001 From: bharatnc Date: Mon, 14 Sep 2020 19:12:00 -0700 Subject: [PATCH 083/321] TraceCollector - time s&us using same timespec --- src/Common/Stopwatch.h | 7 ------- src/Common/TraceCollector.cpp | 9 +++++++-- 2 files changed, 7 insertions(+), 9 deletions(-) diff --git a/src/Common/Stopwatch.h b/src/Common/Stopwatch.h index 6a0d1f37eb4..a7f5e76d5be 100644 --- a/src/Common/Stopwatch.h +++ b/src/Common/Stopwatch.h @@ -13,13 +13,6 @@ inline UInt64 clock_gettime_ns(clockid_t clock_type = CLOCK_MONOTONIC) return UInt64(ts.tv_sec * 1000000000LL + ts.tv_nsec); } -inline UInt64 clock_gettime_microseconds(clockid_t clock_type = CLOCK_MONOTONIC) -{ - struct timespec ts; - clock_gettime(clock_type, &ts); - return UInt64((ts.tv_sec * 1000000LL) + (ts.tv_nsec / 1000)); -} - /** Differs from Poco::Stopwatch only by using 'clock_gettime' instead of 'gettimeofday', * returns nanoseconds instead of microseconds, and also by other minor differencies. */ diff --git a/src/Common/TraceCollector.cpp b/src/Common/TraceCollector.cpp index 35e90a6e583..104b747d431 100644 --- a/src/Common/TraceCollector.cpp +++ b/src/Common/TraceCollector.cpp @@ -141,8 +141,13 @@ void TraceCollector::run() if (trace_log) { - UInt64 time = clock_gettime_ns(CLOCK_REALTIME); - UInt64 time_in_microseconds = clock_gettime_microseconds(CLOCK_REALTIME); + // time and time_in_microseconds are both being constructed from the same timespec so that the + // times will be equal upto the precision of a second. + struct timespec ts; + clock_gettime(CLOCK_REALTIME, &ts); + + UInt64 time = UInt64(ts.tv_sec * 1000000000LL + ts.tv_nsec); + UInt64 time_in_microseconds = UInt64((ts.tv_sec * 1000000LL) + (ts.tv_nsec / 1000)); TraceLogElement element{time_t(time / 1000000000), time_in_microseconds, time, trace_type, thread_id, query_id, trace, size}; trace_log->add(element); } From b27579e3f785cac8455f308aeed193d0345fa926 Mon Sep 17 00:00:00 2001 From: bharatnc Date: Mon, 14 Sep 2020 23:49:02 -0700 Subject: [PATCH 084/321] Use std::chrono instead of clock_gettime --- src/Interpreters/ThreadStatusExt.cpp | 24 ++++++++++++++++++------ src/Interpreters/executeQuery.cpp | 28 +++++++++++++--------------- 2 files changed, 31 insertions(+), 21 deletions(-) diff --git a/src/Interpreters/ThreadStatusExt.cpp b/src/Interpreters/ThreadStatusExt.cpp index d820094db1d..6a6deb2b4d3 100644 --- a/src/Interpreters/ThreadStatusExt.cpp +++ b/src/Interpreters/ThreadStatusExt.cpp @@ -318,17 +318,29 @@ void ThreadStatus::detachQuery(bool exit_if_already_detached, bool thread_exits) #endif } +inline UInt64 time_in_microseconds(std::chrono::time_point timepoint) +{ + return std::chrono::duration_cast(timepoint.time_since_epoch()).count(); +} + + +inline UInt64 time_in_seconds(std::chrono::time_point timepoint) +{ + return std::chrono::duration_cast(timepoint.time_since_epoch()).count(); +} + void ThreadStatus::logToQueryThreadLog(QueryThreadLog & thread_log) { QueryThreadLogElement elem; - // event_time and event_time_microseconds are being constructed from the same timespec - // to ensure that both the times are equal upto the precision of a second. - struct timespec ts; - clock_gettime(CLOCK_MONOTONIC, &ts); + // construct current_time and current_time_microseconds using the same time point + // so that the two times will always be equal up to a precision of a second. + const auto now = std::chrono::system_clock::now(); + auto current_time = time_in_seconds(now); + auto current_time_microseconds = time_in_microseconds(now); - elem.event_time = ts.tv_sec; - elem.event_time_microseconds = UInt64((ts.tv_sec * 1000000LL) + (ts.tv_nsec / 1000)); + elem.event_time = current_time; + elem.event_time_microseconds = current_time_microseconds; elem.query_start_time = query_start_time; elem.query_start_time_microseconds = query_start_time_microseconds; elem.query_duration_ms = (getCurrentTimeNanoseconds() - query_start_time_nanoseconds) / 1000000U; diff --git a/src/Interpreters/executeQuery.cpp b/src/Interpreters/executeQuery.cpp index 3f2c88b3678..c82748eadc0 100644 --- a/src/Interpreters/executeQuery.cpp +++ b/src/Interpreters/executeQuery.cpp @@ -206,8 +206,8 @@ static void onExceptionBeforeStart(const String & query_for_logging, Context & c elem.type = QueryLogElementType::EXCEPTION_BEFORE_START; - // all callers to onExceptionBeforeStart upstream construct the timespec for event_time and - // event_time_microseconds from the same timespec. So it can be assumed that both of these + // all callers to onExceptionBeforeStart method construct the timespec for event_time and + // event_time_microseconds from the same time point. So, it can be assumed that both of these // times are equal upto the precision of a second. elem.event_time = current_time; elem.event_time_microseconds = current_time_microseconds; @@ -319,7 +319,8 @@ static std::tuple executeQueryImpl( auto query_for_logging = prepareQueryForLogging(query, context); logQuery(query_for_logging, context, internal); - if (!internal) { + if (!internal) + { onExceptionBeforeStart(query_for_logging, context, current_time, current_time_microseconds, ast); } @@ -557,13 +558,11 @@ static std::tuple executeQueryImpl( elem.type = QueryLogElementType::QUERY_FINISH; - // event_time and event-time_microseconds are being constructed from the same timespec - // to ensure that both the times are equal upto the precision of a second. - struct timespec tspec; - clock_gettime(CLOCK_MONOTONIC, &tspec); - - elem.event_time = tspec.tv_sec; - elem.event_time_microseconds = UInt64((tspec.tv_sec * 1000000LL) + (tspec.tv_nsec / 1000)); + // construct event_time and event_time_microseconds using the same time point + // so that the two times will always be equal up to a precision of a second. + const auto time_now = std::chrono::system_clock::now(); + elem.event_time = time_in_seconds(time_now); + elem.event_time_microseconds = time_in_microseconds(time_now); status_info_to_query_log(elem, info, ast); auto progress_callback = context.getProgressCallback(); @@ -623,13 +622,12 @@ static std::tuple executeQueryImpl( elem.type = QueryLogElementType::EXCEPTION_WHILE_PROCESSING; - // event_time and event_time_microseconds are being constructed from the timespec + // event_time and event_time_microseconds are being constructed from the same time point // to ensure that both the times will be equal upto the precision of a second. - struct timespec tspec; - clock_gettime(CLOCK_MONOTONIC, &tspec); + const auto time_now = std::chrono::system_clock::now(); - elem.event_time = tspec.tv_sec; - elem.event_time_microseconds = UInt64((tspec.tv_sec * 1000000LL) + (tspec.tv_nsec / 1000)); + elem.event_time = time_in_seconds(time_now); + elem.event_time_microseconds = time_in_microseconds(time_now); elem.query_duration_ms = 1000 * (elem.event_time - elem.query_start_time); elem.exception_code = getCurrentExceptionCode(); elem.exception = getCurrentExceptionMessage(false); From 4f8266c7a1f8aa6a93220ed7537542ed92e7784d Mon Sep 17 00:00:00 2001 From: bharatnc Date: Tue, 15 Sep 2020 17:12:06 -0700 Subject: [PATCH 085/321] Make trace_log test to pass --- src/Interpreters/ThreadStatusExt.cpp | 11 ----------- .../0_stateless/01473_event_time_microseconds.sql | 6 ++++-- 2 files changed, 4 insertions(+), 13 deletions(-) diff --git a/src/Interpreters/ThreadStatusExt.cpp b/src/Interpreters/ThreadStatusExt.cpp index 6a6deb2b4d3..61245782ba9 100644 --- a/src/Interpreters/ThreadStatusExt.cpp +++ b/src/Interpreters/ThreadStatusExt.cpp @@ -318,17 +318,6 @@ void ThreadStatus::detachQuery(bool exit_if_already_detached, bool thread_exits) #endif } -inline UInt64 time_in_microseconds(std::chrono::time_point timepoint) -{ - return std::chrono::duration_cast(timepoint.time_since_epoch()).count(); -} - - -inline UInt64 time_in_seconds(std::chrono::time_point timepoint) -{ - return std::chrono::duration_cast(timepoint.time_since_epoch()).count(); -} - void ThreadStatus::logToQueryThreadLog(QueryThreadLog & thread_log) { QueryThreadLogElement elem; diff --git a/tests/queries/0_stateless/01473_event_time_microseconds.sql b/tests/queries/0_stateless/01473_event_time_microseconds.sql index 8a545af9efa..e4bc3b29655 100644 --- a/tests/queries/0_stateless/01473_event_time_microseconds.sql +++ b/tests/queries/0_stateless/01473_event_time_microseconds.sql @@ -25,11 +25,13 @@ SELECT if(dateDiff('second', toDateTime(time_with_microseconds), toDateTime(time SELECT '01473_trace_log_table_event_start_time_microseconds_test'; SET log_queries = 1; -SET query_profiler_real_time_period_ns = 0; -SET query_profiler_cpu_time_period_ns = 1000000; +SET query_profiler_real_time_period_ns = 10000000; +SET query_profiler_cpu_time_period_ns = 10000000; -- a long enough query to trigger the query profiler and to record trace log +SELECT count() FROM numbers(1000000000) FORMAT Null; SELECT sleep(2) FORMAT Null; SYSTEM FLUSH LOGS; +SELECT sleep(2) FORMAT Null; WITH ( ( SELECT event_time_microseconds From 5c8c636cf1692c286999902e5b6f98fe05801865 Mon Sep 17 00:00:00 2001 From: bharatnc Date: Fri, 25 Sep 2020 11:03:16 -0700 Subject: [PATCH 086/321] Fix - allow empty tz in exp like toStartOf*(today()) --- src/Functions/FunctionDateOrDateTimeToSomething.h | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/src/Functions/FunctionDateOrDateTimeToSomething.h b/src/Functions/FunctionDateOrDateTimeToSomething.h index f50cab25d2a..2fde07416a3 100644 --- a/src/Functions/FunctionDateOrDateTimeToSomething.h +++ b/src/Functions/FunctionDateOrDateTimeToSomething.h @@ -71,7 +71,9 @@ public: if constexpr (std::is_same_v) { std::string time_zone = extractTimeZoneNameFromFunctionArguments(arguments, 1, 0); - if (time_zone.empty()) + /// only validate the time_zone part if the number of arguments is 2. This is mainly + /// to accommodate functions like toStartOfDay(today()), toStartOfDay(yesterday()) etc. + if (arguments.size() == 2 && time_zone.empty()) throw Exception( "Function " + getName() + " supports a 2nd argument (optional) that must be non-empty and be a valid time zone", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); From e036c2714e607661a28c423f0b420008ce5fb0b8 Mon Sep 17 00:00:00 2001 From: bharatnc Date: Fri, 25 Sep 2020 11:18:23 -0700 Subject: [PATCH 087/321] Add unit test case --- .../01472_toStartOfInterval_disallow_empty_tz_field.sql | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/tests/queries/0_stateless/01472_toStartOfInterval_disallow_empty_tz_field.sql b/tests/queries/0_stateless/01472_toStartOfInterval_disallow_empty_tz_field.sql index f7d6fd2f357..c2bdac2b279 100644 --- a/tests/queries/0_stateless/01472_toStartOfInterval_disallow_empty_tz_field.sql +++ b/tests/queries/0_stateless/01472_toStartOfInterval_disallow_empty_tz_field.sql @@ -21,3 +21,7 @@ SELECT toStartOfHour(toDateTime('2017-12-31 01:59:00', 'UTC'), 'UTC'); -- succes SELECT toStartOfMinute(toDateTime('2017-12-31 00:00:00', 'UTC'), ''); -- {serverError 43} SELECT toStartOfMinute(toDateTime('2017-12-31 00:01:30', 'UTC'), 'UTC'); -- success + +-- special case - allow empty time_zone when using functions like today(), yesterday() etc. +SELECT toStartOfDay(today()) FORMAT Null; -- success +SELECT toStartOfDay(yesterday()) FORMAT Null; -- success From 35ab3657c3cd2e313279733f538f43fccbb903ac Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 25 Sep 2020 21:39:55 +0300 Subject: [PATCH 088/321] Add flaky tests config --- tests/ci/ci_config.json | 12 ++++++++++++ 1 file changed, 12 insertions(+) diff --git a/tests/ci/ci_config.json b/tests/ci/ci_config.json index 220d8d801ec..c01025ea865 100644 --- a/tests/ci/ci_config.json +++ b/tests/ci/ci_config.json @@ -441,6 +441,18 @@ "with_coverage": false } }, + "Integration tests flaky check (asan)": { + "required_build_properties": { + "compiler": "clang-11", + "package_type": "deb", + "build_type": "relwithdebuginfo", + "sanitizer": "address", + "bundled": "bundled", + "splitted": "unsplitted", + "clang-tidy": "disable", + "with_coverage": false + } + }, "Compatibility check": { "required_build_properties": { "compiler": "gcc-10", From 48699ae0b93db1aabf9aebef6c40d7acab689ef8 Mon Sep 17 00:00:00 2001 From: bharatnc Date: Fri, 25 Sep 2020 12:25:29 -0700 Subject: [PATCH 089/321] Fix - failing tests --- .../00921_datetime64_compatibility.reference | 17 ++++++++--------- 1 file changed, 8 insertions(+), 9 deletions(-) diff --git a/tests/queries/0_stateless/00921_datetime64_compatibility.reference b/tests/queries/0_stateless/00921_datetime64_compatibility.reference index 079469c8c2d..4042c756ba4 100644 --- a/tests/queries/0_stateless/00921_datetime64_compatibility.reference +++ b/tests/queries/0_stateless/00921_datetime64_compatibility.reference @@ -88,38 +88,37 @@ SELECT toStartOfWeek(N) "Date","2019-09-15" ------------------------------------------ SELECT toStartOfDay(N) - -Code: 43: Function toStartOfDay supports a 2nd argument (optional) that must be non-empty and be a valid time zone. +"DateTime","2019-09-16 00:00:00" "DateTime('Europe/Minsk')","2019-09-16 00:00:00" "DateTime('Europe/Minsk')","2019-09-16 00:00:00" ------------------------------------------ SELECT toStartOfHour(N) -Code: 43: Function toStartOfHour supports a 2nd argument (optional) that must be non-empty and be a valid time zone. +Code: 43: Illegal type Date of argument for function toStartOfHour. "DateTime('Europe/Minsk')","2019-09-16 19:00:00" "DateTime('Europe/Minsk')","2019-09-16 19:00:00" ------------------------------------------ SELECT toStartOfMinute(N) -Code: 43: Function toStartOfMinute supports a 2nd argument (optional) that must be non-empty and be a valid time zone. +Code: 43: Illegal type Date of argument for function toStartOfMinute. "DateTime('Europe/Minsk')","2019-09-16 19:20:00" "DateTime('Europe/Minsk')","2019-09-16 19:20:00" ------------------------------------------ SELECT toStartOfFiveMinute(N) -Code: 43: Function toStartOfFiveMinute supports a 2nd argument (optional) that must be non-empty and be a valid time zone. +Code: 43: Illegal type Date of argument for function toStartOfFiveMinute. "DateTime('Europe/Minsk')","2019-09-16 19:20:00" "DateTime('Europe/Minsk')","2019-09-16 19:20:00" ------------------------------------------ SELECT toStartOfTenMinutes(N) -Code: 43: Function toStartOfTenMinutes supports a 2nd argument (optional) that must be non-empty and be a valid time zone. +Code: 43: Illegal type Date of argument for function toStartOfTenMinutes. "DateTime('Europe/Minsk')","2019-09-16 19:20:00" "DateTime('Europe/Minsk')","2019-09-16 19:20:00" ------------------------------------------ SELECT toStartOfFifteenMinutes(N) -Code: 43: Function toStartOfFifteenMinutes supports a 2nd argument (optional) that must be non-empty and be a valid time zone. +Code: 43: Illegal type Date of argument for function toStartOfFifteenMinutes. "DateTime('Europe/Minsk')","2019-09-16 19:15:00" "DateTime('Europe/Minsk')","2019-09-16 19:15:00" ------------------------------------------ @@ -167,7 +166,7 @@ Code: 43: Illegal type Date of argument for function date_trunc. ------------------------------------------ SELECT toTime(N) -Code: 43: Function toTime supports a 2nd argument (optional) that must be non-empty and be a valid time zone. +Code: 43: Illegal type Date of argument for function toTime. "DateTime('Europe/Minsk')","1970-01-02 19:20:11" "DateTime('Europe/Minsk')","1970-01-02 19:20:11" ------------------------------------------ @@ -233,7 +232,7 @@ SELECT toYearWeek(N) ------------------------------------------ SELECT timeSlot(N) -Code: 43: Function timeSlot supports a 2nd argument (optional) that must be non-empty and be a valid time zone. +Code: 43: Illegal type Date of argument for function timeSlot. "DateTime('Europe/Minsk')","2019-09-16 19:00:00" "DateTime('Europe/Minsk')","2019-09-16 19:00:00" ------------------------------------------ From 62c2c2000cd0c33d8a8b39d7c34a0104848af9b7 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 26 Sep 2020 03:00:54 +0300 Subject: [PATCH 090/321] Query obfuscator: development --- src/CMakeLists.txt | 2 +- src/Common/BitHelpers.h | 43 +- src/Common/StringUtils/StringUtils.h | 23 +- src/Parsers/obfuscateQueries.cpp | 934 ++++++++++++++++++ src/Parsers/obfuscateQueries.h | 29 + src/Parsers/tests/gtest_obfuscate_queries.cpp | 92 ++ 6 files changed, 1104 insertions(+), 19 deletions(-) create mode 100644 src/Parsers/obfuscateQueries.cpp create mode 100644 src/Parsers/obfuscateQueries.h create mode 100644 src/Parsers/tests/gtest_obfuscate_queries.cpp diff --git a/src/CMakeLists.txt b/src/CMakeLists.txt index 0016c51b7f8..40815228eac 100644 --- a/src/CMakeLists.txt +++ b/src/CMakeLists.txt @@ -415,6 +415,6 @@ if (ENABLE_TESTS AND USE_GTEST) -Wno-gnu-zero-variadic-macro-arguments ) - target_link_libraries(unit_tests_dbms PRIVATE ${GTEST_BOTH_LIBRARIES} clickhouse_functions clickhouse_aggregate_functions clickhouse_parsers dbms clickhouse_common_zookeeper string_utils) + target_link_libraries(unit_tests_dbms PRIVATE ${GTEST_BOTH_LIBRARIES} clickhouse_functions clickhouse_aggregate_functions clickhouse_table_functions clickhouse_parsers dbms clickhouse_common_zookeeper string_utils) add_check(unit_tests_dbms) endif () diff --git a/src/Common/BitHelpers.h b/src/Common/BitHelpers.h index 699e379b8d3..eac5fdac80e 100644 --- a/src/Common/BitHelpers.h +++ b/src/Common/BitHelpers.h @@ -1,22 +1,12 @@ #pragma once #include +#include #include #include #include -/** Returns log2 of number, rounded down. - * Compiles to single 'bsr' instruction on x86. - * For zero argument, result is unspecified. - */ -inline unsigned int bitScanReverse(unsigned int x) -{ - assert(x != 0); - return sizeof(unsigned int) * 8 - 1 - __builtin_clz(x); -} - - /** For zero argument, result is zero. * For arguments with most significand bit set, result is n. * For other arguments, returns value, rounded up to power of two. @@ -41,10 +31,9 @@ inline size_t roundUpToPowerOfTwoOrZero(size_t n) template -inline size_t getLeadingZeroBits(T x) +inline size_t getLeadingZeroBitsUnsafe(T x) { - if (!x) - return sizeof(x) * 8; + assert(x != 0); if constexpr (sizeof(T) <= sizeof(unsigned int)) { @@ -60,10 +49,32 @@ inline size_t getLeadingZeroBits(T x) } } + +template +inline size_t getLeadingZeroBits(T x) +{ + if (!x) + return sizeof(x) * 8; + + return getLeadingZeroBitsUnsafe(x); +} + +/** Returns log2 of number, rounded down. + * Compiles to single 'bsr' instruction on x86. + * For zero argument, result is unspecified. + */ +template +inline uint32_t bitScanReverse(T x) +{ + return sizeof(T) * 8 - 1 - getLeadingZeroBitsUnsafe(x); +} + // Unsafe since __builtin_ctz()-family explicitly state that result is undefined on x == 0 template inline size_t getTrailingZeroBitsUnsafe(T x) { + assert(x != 0); + if constexpr (sizeof(T) <= sizeof(unsigned int)) { return __builtin_ctz(x); @@ -88,8 +99,8 @@ inline size_t getTrailingZeroBits(T x) } /** Returns a mask that has '1' for `bits` LSB set: - * maskLowBits(3) => 00000111 - */ + * maskLowBits(3) => 00000111 + */ template inline T maskLowBits(unsigned char bits) { diff --git a/src/Common/StringUtils/StringUtils.h b/src/Common/StringUtils/StringUtils.h index a1e8fb79435..904e3035dd8 100644 --- a/src/Common/StringUtils/StringUtils.h +++ b/src/Common/StringUtils/StringUtils.h @@ -67,10 +67,19 @@ inline bool isASCII(char c) return static_cast(c) < 0x80; } +inline bool isLowerAlphaASCII(char c) +{ + return (c >= 'a' && c <= 'z'); +} + +inline bool isUpperAlphaASCII(char c) +{ + return (c >= 'A' && c <= 'Z'); +} + inline bool isAlphaASCII(char c) { - return (c >= 'a' && c <= 'z') - || (c >= 'A' && c <= 'Z'); + return isLowerAlphaASCII(c) || isUpperAlphaASCII(c); } inline bool isNumericASCII(char c) @@ -122,6 +131,16 @@ inline bool isPrintableASCII(char c) return uc >= 32 && uc <= 126; /// 127 is ASCII DEL. } +inline bool isPunctuationASCII(char c) +{ + uint8_t uc = c; + return (uc >= 33 && uc <= 47) + || (uc >= 58 && uc <= 64) + || (uc >= 91 && uc <= 96) + || (uc >= 123 && uc <= 125); +} + + inline bool isValidIdentifier(const std::string_view & str) { return !str.empty() && isValidIdentifierBegin(str[0]) && std::all_of(str.begin() + 1, str.end(), isWordCharASCII); diff --git a/src/Parsers/obfuscateQueries.cpp b/src/Parsers/obfuscateQueries.cpp new file mode 100644 index 00000000000..66221005d77 --- /dev/null +++ b/src/Parsers/obfuscateQueries.cpp @@ -0,0 +1,934 @@ +#include + +#include +#include +#include +#include +#include +#include +#include +#include +#include + + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int TOO_MANY_TEMPORARY_COLUMNS; +} + + +namespace +{ + +const std::unordered_set keywords +{ + "CREATE", "DATABASE", "IF", "NOT", "EXISTS", "TEMPORARY", "TABLE", "ON", "CLUSTER", "DEFAULT", + "MATERIALIZED", "ALIAS", "ENGINE", "AS", "VIEW", "POPULATE", "SETTINGS", "ATTACH", "DETACH", "DROP", + "RENAME", "TO", "ALTER", "ADD", "MODIFY", "CLEAR", "COLUMN", "AFTER", "COPY", "PROJECT", + "PRIMARY", "KEY", "CHECK", "PARTITION", "PART", "FREEZE", "FETCH", "FROM", "SHOW", "INTO", + "OUTFILE", "FORMAT", "TABLES", "DATABASES", "LIKE", "PROCESSLIST", "CASE", "WHEN", "THEN", "ELSE", + "END", "DESCRIBE", "DESC", "USE", "SET", "OPTIMIZE", "FINAL", "DEDUPLICATE", "INSERT", "VALUES", + "SELECT", "DISTINCT", "SAMPLE", "ARRAY", "JOIN", "GLOBAL", "LOCAL", "ANY", "ALL", "INNER", + "LEFT", "RIGHT", "FULL", "OUTER", "CROSS", "USING", "PREWHERE", "WHERE", "GROUP", "BY", + "WITH", "TOTALS", "HAVING", "ORDER", "COLLATE", "LIMIT", "UNION", "AND", "OR", "ASC", + "IN", "KILL", "QUERY", "SYNC", "ASYNC", "TEST", "BETWEEN", "TRUNCATE", "USER", "ROLE", + "PROFILE", "QUOTA", "POLICY", "ROW", "GRANT", "REVOKE", "OPTION", "ADMIN", "EXCEPT", "REPLACE", + "IDENTIFIED", "HOST", "NAME", "READONLY", "WRITABLE", "PERMISSIVE", "FOR", "RESTRICTIVE", "FOR", "RANDOMIZED", + "INTERVAL", "LIMITS", "ONLY", "TRACKING", "IP", "REGEXP", "ILIKE", "DICTIONARY" +}; + +const std::unordered_set keep_words +{ + "id", "name", "value", "num", + "Id", "Name", "Value", "Num", + "ID", "NAME", "VALUE", "NUM", +}; + +/// The list of nouns collected from here: http://www.desiquintans.com/nounlist, Public domain. +std::initializer_list nouns +{ +"aardvark", "abacus", "abbey", "abbreviation", "abdomen", "ability", "abnormality", "abolishment", "abortion", +"abrogation", "absence", "abundance", "abuse", "academics", "academy", "accelerant", "accelerator", "accent", "acceptance", "access", +"accessory", "accident", "accommodation", "accompanist", "accomplishment", "accord", "accordance", "accordion", "account", "accountability", +"accountant", "accounting", "accuracy", "accusation", "acetate", "achievement", "achiever", "acid", "acknowledgment", "acorn", "acoustics", +"acquaintance", "acquisition", "acre", "acrylic", "act", "action", "activation", "activist", "activity", "actor", "actress", "acupuncture", +"ad", "adaptation", "adapter", "addiction", "addition", "address", "adjective", "adjustment", "admin", "administration", "administrator", +"admire", "admission", "adobe", "adoption", "adrenalin", "adrenaline", "adult", "adulthood", "advance", "advancement", "advantage", "advent", +"adverb", "advertisement", "advertising", "advice", "adviser", "advocacy", "advocate", "affair", "affect", "affidavit", "affiliate", +"affinity", "afoul", "afterlife", "aftermath", "afternoon", "aftershave", "aftershock", "afterthought", "age", "agency", "agenda", "agent", +"aggradation", "aggression", "aglet", "agony", "agreement", "agriculture", "aid", "aide", "aim", "air", "airbag", "airbus", "aircraft", +"airfare", "airfield", "airforce", "airline", "airmail", "airman", "airplane", "airport", "airship", "airspace", "alarm", "alb", "albatross", +"album", "alcohol", "alcove", "alder", "ale", "alert", "alfalfa", "algebra", "algorithm", "alias", "alibi", "alien", "allegation", "allergist", +"alley", "alliance", "alligator", "allocation", "allowance", "alloy", "alluvium", "almanac", "almighty", "almond", "alpaca", "alpenglow", +"alpenhorn", "alpha", "alphabet", "altar", "alteration", "alternative", "altitude", "alto", "aluminium", "aluminum", "amazement", "amazon", +"ambassador", "amber", "ambience", "ambiguity", "ambition", "ambulance", "amendment", "amenity", "ammunition", "amnesty", "amount", "amusement", +"anagram", "analgesia", "analog", "analogue", "analogy", "analysis", "analyst", "analytics", "anarchist", "anarchy", "anatomy", "ancestor", +"anchovy", "android", "anesthesiologist", "anesthesiology", "angel", "anger", "angina", "angiosperm", "angle", "angora", "angstrom", +"anguish", "animal", "anime", "anise", "ankle", "anklet", "anniversary", "announcement", "annual", "anorak", "answer", "ant", "anteater", +"antecedent", "antechamber", "antelope", "antennae", "anterior", "anthropology", "antibody", "anticipation", "anticodon", "antigen", +"antique", "antiquity", "antler", "antling", "anxiety", "anybody", "anyone", "anything", "anywhere", "apartment", "ape", "aperitif", +"apology", "app", "apparatus", "apparel", "appeal", "appearance", "appellation", "appendix", "appetiser", "appetite", "appetizer", "applause", +"apple", "applewood", "appliance", "application", "appointment", "appreciation", "apprehension", "approach", "appropriation", "approval", +"apricot", "apron", "apse", "aquarium", "aquifer", "arcade", "arch", "archaeologist", "archaeology", "archeology", "archer", +"architect", "architecture", "archives", "area", "arena", "argument", "arithmetic", "ark", "arm", "armadillo", "armament", +"armchair", "armoire", "armor", "armour", "armpit", "armrest", "army", "arrangement", "array", "arrest", "arrival", "arrogance", "arrow", +"art", "artery", "arthur", "artichoke", "article", "artifact", "artificer", "artist", "ascend", "ascent", "ascot", "ash", "ashram", "ashtray", +"aside", "asparagus", "aspect", "asphalt", "aspic", "ass", "assassination", "assault", "assembly", "assertion", "assessment", "asset", +"assignment", "assist", "assistance", "assistant", "associate", "association", "assumption", "assurance", "asterisk", "astrakhan", "astrolabe", +"astrologer", "astrology", "astronomy", "asymmetry", "atelier", "atheist", "athlete", "athletics", "atmosphere", "atom", "atrium", "attachment", +"attack", "attacker", "attainment", "attempt", "attendance", "attendant", "attention", "attenuation", "attic", "attitude", "attorney", +"attraction", "attribute", "auction", "audience", "audit", "auditorium", "aunt", "authentication", "authenticity", "author", "authorisation", +"authority", "authorization", "auto", "autoimmunity", "automation", "automaton", "autumn", "availability", "avalanche", "avenue", "average", +"avocado", "award", "awareness", "awe", "axis", "azimuth", "babe", "baboon", "babushka", "baby", "bachelor", "back", "backbone", +"backburn", "backdrop", "background", "backpack", "backup", "backyard", "bacon", "bacterium", "badge", "badger", "bafflement", "bag", +"bagel", "baggage", "baggie", "baggy", "bagpipe", "bail", "bait", "bake", "baker", "bakery", "bakeware", "balaclava", "balalaika", "balance", +"balcony", "ball", "ballet", "balloon", "balloonist", "ballot", "ballpark", "bamboo", "ban", "banana", "band", "bandana", "bandanna", +"bandolier", "bandwidth", "bangle", "banjo", "bank", "bankbook", "banker", "banking", "bankruptcy", "banner", "banquette", "banyan", +"baobab", "bar", "barbecue", "barbeque", "barber", "barbiturate", "bargain", "barge", "baritone", "barium", "bark", "barley", "barn", +"barometer", "barracks", "barrage", "barrel", "barrier", "barstool", "bartender", "base", "baseball", "baseboard", "baseline", "basement", +"basics", "basil", "basin", "basis", "basket", "basketball", "bass", "bassinet", "bassoon", "bat", "bath", "bather", "bathhouse", "bathrobe", +"bathroom", "bathtub", "battalion", "batter", "battery", "batting", "battle", "battleship", "bay", "bayou", "beach", "bead", "beak", +"beam", "bean", "beancurd", "beanie", "beanstalk", "bear", "beard", "beast", "beastie", "beat", "beating", "beauty", "beaver", "beck", +"bed", "bedrock", "bedroom", "bee", "beech", "beef", "beer", "beet", "beetle", "beggar", "beginner", "beginning", "begonia", "behalf", +"behavior", "behaviour", "beheading", "behest", "behold", "being", "belfry", "belief", "believer", "bell", "belligerency", "bellows", +"belly", "belt", "bench", "bend", "beneficiary", "benefit", "beret", "berry", "bestseller", "bet", "beverage", "beyond", +"bias", "bibliography", "bicycle", "bid", "bidder", "bidding", "bidet", "bifocals", "bijou", "bike", "bikini", "bill", "billboard", "billing", +"billion", "bin", "binoculars", "biology", "biopsy", "biosphere", "biplane", "birch", "bird", "birdbath", "birdcage", +"birdhouse", "birth", "birthday", "biscuit", "bit", "bite", "bitten", "bitter", "black", "blackberry", "blackbird", "blackboard", "blackfish", +"blackness", "bladder", "blade", "blame", "blank", "blanket", "blast", "blazer", "blend", "blessing", "blight", "blind", "blinker", "blister", +"blizzard", "block", "blocker", "blog", "blogger", "blood", "bloodflow", "bloom", "bloomer", "blossom", "blouse", "blow", "blowgun", +"blowhole", "blue", "blueberry", "blush", "boar", "board", "boat", "boatload", "boatyard", "bob", "bobcat", "body", "bog", "bolero", +"bolt", "bomb", "bomber", "bombing", "bond", "bonding", "bondsman", "bone", "bonfire", "bongo", "bonnet", "bonsai", "bonus", "boogeyman", +"book", "bookcase", "bookend", "booking", "booklet", "bookmark", "boolean", "boom", "boon", "boost", "booster", "boot", "bootee", "bootie", +"booty", "border", "bore", "borrower", "borrowing", "bosom", "boss", "botany", "bother", "bottle", "bottling", "bottom", "bottom-line", +"boudoir", "bough", "boulder", "boulevard", "boundary", "bouquet", "bourgeoisie", "bout", "boutique", "bow", "bower", "bowl", "bowler", +"bowling", "bowtie", "box", "boxer", "boxspring", "boy", "boycott", "boyfriend", "boyhood", "boysenberry", "bra", "brace", "bracelet", +"bracket", "brain", "brake", "bran", "branch", "brand", "brandy", "brass", "brassiere", "bratwurst", "bread", "breadcrumb", "breadfruit", +"break", "breakdown", "breakfast", "breakpoint", "breakthrough", "breast", "breastplate", "breath", "breeze", "brewer", "bribery", "brick", +"bricklaying", "bride", "bridge", "brief", "briefing", "briefly", "briefs", "brilliant", "brink", "brisket", "broad", "broadcast", "broccoli", +"brochure", "brocolli", "broiler", "broker", "bronchitis", "bronco", "bronze", "brooch", "brood", "brook", "broom", "brother", "brother-in-law", +"brow", "brown", "brownie", "browser", "browsing", "brunch", "brush", "brushfire", "brushing", "bubble", "buck", "bucket", "buckle", +"buckwheat", "bud", "buddy", "budget", "buffalo", "buffer", "buffet", "bug", "buggy", "bugle", "builder", "building", "bulb", "bulk", +"bull", "bulldozer", "bullet", "bump", "bumper", "bun", "bunch", "bungalow", "bunghole", "bunkhouse", "burden", "bureau", +"burglar", "burial", "burlesque", "burn", "burning", "burrito", "burro", "burrow", "burst", "bus", "bush", "business", "businessman", +"bust", "bustle", "butane", "butcher", "butler", "butter", "butterfly", "button", "buy", "buyer", "buying", "buzz", "buzzard", "c-clamp", +"cabana", "cabbage", "cabin", "cabinet", "cable", "caboose", "cacao", "cactus", "caddy", "cadet", "cafe", "caffeine", "caftan", "cage", +"cake", "calcification", "calculation", "calculator", "calculus", "calendar", "calf", "caliber", "calibre", "calico", "call", "calm", +"calorie", "camel", "cameo", "camera", "camp", "campaign", "campaigning", "campanile", "camper", "campus", "can", "canal", "cancer", +"candelabra", "candidacy", "candidate", "candle", "candy", "cane", "cannibal", "cannon", "canoe", "canon", "canopy", "cantaloupe", "canteen", +"canvas", "cap", "capability", "capacity", "cape", "caper", "capital", "capitalism", "capitulation", "capon", "cappelletti", "cappuccino", +"captain", "caption", "captor", "car", "carabao", "caramel", "caravan", "carbohydrate", "carbon", "carboxyl", "card", "cardboard", "cardigan", +"care", "career", "cargo", "caribou", "carload", "carnation", "carnival", "carol", "carotene", "carp", "carpenter", "carpet", "carpeting", +"carport", "carriage", "carrier", "carrot", "carry", "cart", "cartel", "carter", "cartilage", "cartload", "cartoon", "cartridge", "carving", +"cascade", "case", "casement", "cash", "cashew", "cashier", "casino", "casket", "cassava", "casserole", "cassock", "cast", "castanet", +"castle", "casualty", "cat", "catacomb", "catalogue", "catalysis", "catalyst", "catamaran", "catastrophe", "catch", "catcher", "category", +"caterpillar", "cathedral", "cation", "catsup", "cattle", "cauliflower", "causal", "cause", "causeway", "caution", "cave", "caviar", +"cayenne", "ceiling", "celebration", "celebrity", "celeriac", "celery", "cell", "cellar", "cello", "celsius", "cement", "cemetery", "cenotaph", +"census", "cent", "center", "centimeter", "centre", "centurion", "century", "cephalopod", "ceramic", "ceramics", "cereal", "ceremony", +"certainty", "certificate", "certification", "cesspool", "chafe", "chain", "chainstay", "chair", "chairlift", "chairman", "chairperson", +"chaise", "chalet", "chalice", "chalk", "challenge", "chamber", "champagne", "champion", "championship", "chance", "chandelier", "change", +"channel", "chaos", "chap", "chapel", "chaplain", "chapter", "character", "characteristic", "characterization", "chard", "charge", "charger", +"charity", "charlatan", "charm", "charset", "chart", "charter", "chasm", "chassis", "chastity", "chasuble", "chateau", "chatter", "chauffeur", +"chauvinist", "check", "checkbook", "checking", "checkout", "checkroom", "cheddar", "cheek", "cheer", "cheese", "cheesecake", "cheetah", +"chef", "chem", "chemical", "chemistry", "chemotaxis", "cheque", "cherry", "chess", "chest", "chestnut", "chick", "chicken", "chicory", +"chief", "chiffonier", "child", "childbirth", "childhood", "chili", "chill", "chime", "chimpanzee", "chin", "chinchilla", "chino", "chip", +"chipmunk", "chivalry", "chive", "chives", "chocolate", "choice", "choir", "choker", "cholesterol", "choosing", "chop", +"chops", "chopstick", "chopsticks", "chord", "chorus", "chow", "chowder", "chrome", "chromolithograph", "chronicle", "chronograph", "chronometer", +"chrysalis", "chub", "chuck", "chug", "church", "churn", "chutney", "cicada", "cigarette", "cilantro", "cinder", "cinema", "cinnamon", +"circadian", "circle", "circuit", "circulation", "circumference", "circumstance", "cirrhosis", "cirrus", "citizen", "citizenship", "citron", +"citrus", "city", "civilian", "civilisation", "civilization", "claim", "clam", "clamp", "clan", "clank", "clapboard", "clarification", +"clarinet", "clarity", "clasp", "class", "classic", "classification", "classmate", "classroom", "clause", "clave", "clavicle", "clavier", +"claw", "clay", "cleaner", "clearance", "clearing", "cleat", "cleavage", "clef", "cleft", "clergyman", "cleric", "clerk", "click", "client", +"cliff", "climate", "climb", "clinic", "clip", "clipboard", "clipper", "cloak", "cloakroom", "clock", "clockwork", "clogs", "cloister", +"clone", "close", "closet", "closing", "closure", "cloth", "clothes", "clothing", "cloud", "cloudburst", "clove", "clover", "cloves", +"club", "clue", "cluster", "clutch", "coach", "coal", "coalition", "coast", "coaster", "coat", "cob", "cobbler", "cobweb", +"cock", "cockpit", "cockroach", "cocktail", "cocoa", "coconut", "cod", "code", "codepage", "codling", "codon", "codpiece", "coevolution", +"cofactor", "coffee", "coffin", "cohesion", "cohort", "coil", "coin", "coincidence", "coinsurance", "coke", "cold", "coleslaw", "coliseum", +"collaboration", "collagen", "collapse", "collar", "collard", "collateral", "colleague", "collection", "collectivisation", "collectivization", +"collector", "college", "collision", "colloquy", "colon", "colonial", "colonialism", "colonisation", "colonization", "colony", "color", +"colorlessness", "colt", "column", "columnist", "comb", "combat", "combination", "combine", "comeback", "comedy", "comestible", "comfort", +"comfortable", "comic", "comics", "comma", "command", "commander", "commandment", "comment", "commerce", "commercial", "commission", +"commitment", "committee", "commodity", "common", "commonsense", "commotion", "communicant", "communication", "communion", "communist", +"community", "commuter", "company", "comparison", "compass", "compassion", "compassionate", "compensation", "competence", "competition", +"competitor", "complaint", "complement", "completion", "complex", "complexity", "compliance", "complication", "complicity", "compliment", +"component", "comportment", "composer", "composite", "composition", "compost", "comprehension", "compress", "compromise", "comptroller", +"compulsion", "computer", "comradeship", "con", "concentrate", "concentration", "concept", "conception", "concern", "concert", "conclusion", +"concrete", "condition", "conditioner", "condominium", "condor", "conduct", "conductor", "cone", "confectionery", "conference", "confidence", +"confidentiality", "configuration", "confirmation", "conflict", "conformation", "confusion", "conga", "congo", "congregation", "congress", +"congressman", "congressperson", "conifer", "connection", "connotation", "conscience", "consciousness", "consensus", "consent", "consequence", +"conservation", "conservative", "consideration", "consignment", "consist", "consistency", "console", "consonant", "conspiracy", "conspirator", +"constant", "constellation", "constitution", "constraint", "construction", "consul", "consulate", "consulting", "consumer", "consumption", +"contact", "contact lens", "contagion", "container", "content", "contention", "contest", "context", "continent", "contingency", "continuity", +"contour", "contract", "contractor", "contrail", "contrary", "contrast", "contribution", "contributor", "control", "controller", "controversy", +"convection", "convenience", "convention", "conversation", "conversion", "convert", "convertible", "conviction", "cook", "cookbook", +"cookie", "cooking", "coonskin", "cooperation", "coordination", "coordinator", "cop", "cope", "copper", "copy", "copying", +"copyright", "copywriter", "coral", "cord", "corduroy", "core", "cork", "cormorant", "corn", "corner", "cornerstone", "cornet", "cornflakes", +"cornmeal", "corporal", "corporation", "corporatism", "corps", "corral", "correspondence", "correspondent", "corridor", "corruption", +"corsage", "cosset", "cost", "costume", "cot", "cottage", "cotton", "couch", "cougar", "cough", "council", "councilman", "councilor", +"councilperson", "counsel", "counseling", "counselling", "counsellor", "counselor", "count", "counter", "counterpart", +"counterterrorism", "countess", "country", "countryside", "county", "couple", "coupon", "courage", "course", "court", "courthouse", "courtroom", +"cousin", "covariate", "cover", "coverage", "coverall", "cow", "cowbell", "cowboy", "coyote", "crab", "crack", "cracker", "crackers", +"cradle", "craft", "craftsman", "cranberry", "crane", "cranky", "crap", "crash", "crate", "cravat", "craw", "crawdad", "crayfish", "crayon", +"crazy", "cream", "creation", "creationism", "creationist", "creative", "creativity", "creator", "creature", "creche", "credential", +"credenza", "credibility", "credit", "creditor", "creek", "creme brulee", "crepe", "crest", "crew", "crewman", "crewmate", "crewmember", +"crewmen", "cria", "crib", "cribbage", "cricket", "cricketer", "crime", "criminal", "crinoline", "crisis", "crisp", "criteria", "criterion", +"critic", "criticism", "crocodile", "crocus", "croissant", "crook", "crop", "cross", "crotch", +"croup", "crow", "crowd", "crown", "crucifixion", "crude", "cruelty", "cruise", "crumb", "crunch", "crusader", "crush", "crust", "cry", +"crystal", "crystallography", "cub", "cube", "cuckoo", "cucumber", "cue", "cuisine", "cultivar", "cultivator", "culture", +"culvert", "cummerbund", "cup", "cupboard", "cupcake", "cupola", "curd", "cure", "curio", "curiosity", "curl", "curler", "currant", "currency", +"current", "curriculum", "curry", "curse", "cursor", "curtailment", "curtain", "curve", "cushion", "custard", "custody", "custom", "customer", +"cut", "cuticle", "cutlet", "cutover", "cutting", "cyclamen", "cycle", "cyclone", "cyclooxygenase", "cygnet", "cylinder", "cymbal", "cynic", +"cyst", "cytokine", "cytoplasm", "dad", "daddy", "daffodil", "dagger", "dahlia", "daikon", "daily", "dairy", "daisy", "dam", "damage", +"dame", "damn", "dance", "dancer", "dancing", "dandelion", "danger", "dare", "dark", "darkness", "darn", "dart", "dash", "dashboard", +"data", "database", "date", "daughter", "dawn", "day", "daybed", "daylight", "dead", "deadline", "deal", "dealer", "dealing", "dearest", +"death", "deathwatch", "debate", "debris", "debt", "debtor", "decade", "decadence", "decency", "decimal", "decision", "decision-making", +"deck", "declaration", "declination", "decline", "decoder", "decongestant", "decoration", "decrease", "decryption", "dedication", "deduce", +"deduction", "deed", "deep", "deer", "default", "defeat", "defendant", "defender", "defense", "deficit", "definition", "deformation", +"degradation", "degree", "delay", "deliberation", "delight", "delivery", "demand", "democracy", "democrat", "demon", "demur", "den", +"denim", "denominator", "density", "dentist", "deodorant", "department", "departure", "dependency", "dependent", "deployment", "deposit", +"deposition", "depot", "depression", "depressive", "depth", "deputy", "derby", "derivation", "derivative", "derrick", "descendant", "descent", +"description", "desert", "design", "designation", "designer", "desire", "desk", "desktop", "dessert", "destination", "destiny", "destroyer", +"destruction", "detail", "detainee", "detainment", "detection", "detective", "detector", "detention", "determination", "detour", "devastation", +"developer", "developing", "development", "developmental", "deviance", "deviation", "device", "devil", "dew", "dhow", "diabetes", "diadem", +"diagnosis", "diagram", "dial", "dialect", "dialogue", "diam", "diamond", "diaper", "diaphragm", "diarist", "diary", "dibble", "dick", +"dickey", "dictaphone", "dictator", "diction", "dictionary", "die", "diesel", "diet", "difference", "differential", "difficulty", "diffuse", +"dig", "digestion", "digestive", "digger", "digging", "digit", "dignity", "dilapidation", "dill", "dilution", "dime", "dimension", "dimple", +"diner", "dinghy", "dining", "dinner", "dinosaur", "dioxide", "dip", "diploma", "diplomacy", "dipstick", "direction", "directive", "director", +"directory", "dirndl", "dirt", "disability", "disadvantage", "disagreement", "disappointment", "disarmament", "disaster", "discharge", +"discipline", "disclaimer", "disclosure", "disco", "disconnection", "discount", "discourse", "discovery", "discrepancy", "discretion", +"discrimination", "discussion", "disdain", "disease", "disembodiment", "disengagement", "disguise", "disgust", "dish", "dishwasher", +"disk", "disparity", "dispatch", "displacement", "display", "disposal", "disposer", "disposition", "dispute", "disregard", "disruption", +"dissemination", "dissonance", "distance", "distinction", "distortion", "distribution", "distributor", "district", "divalent", "divan", +"diver", "diversity", "divide", "dividend", "divider", "divine", "diving", "division", "divorce", "doc", "dock", "doctor", "doctorate", +"doctrine", "document", "documentary", "documentation", "doe", "dog", "doggie", "dogsled", "dogwood", "doing", "doll", "dollar", "dollop", +"dolman", "dolor", "dolphin", "domain", "dome", "domination", "donation", "donkey", "donor", "donut", "door", "doorbell", "doorknob", +"doorpost", "doorway", "dory", "dose", "dot", "double", "doubling", "doubt", "doubter", "dough", "doughnut", "down", "downfall", "downforce", +"downgrade", "download", "downstairs", "downtown", "downturn", "dozen", "draft", "drag", "dragon", "dragonfly", "dragonfruit", "dragster", +"drain", "drainage", "drake", "drama", "dramaturge", "drapes", "draw", "drawbridge", "drawer", "drawing", "dream", "dreamer", "dredger", +"dress", "dresser", "dressing", "drill", "drink", "drinking", "drive", "driver", "driveway", "driving", "drizzle", "dromedary", "drop", +"drudgery", "drug", "drum", "drummer", "drunk", "dryer", "duck", "duckling", "dud", "dude", "due", "duel", "dueling", "duffel", "dugout", +"dulcimer", "dumbwaiter", "dump", "dump truck", "dune", "dune buggy", "dungarees", "dungeon", "duplexer", "duration", "durian", "dusk", +"dust", "dust storm", "duster", "duty", "dwarf", "dwell", "dwelling", "dynamics", "dynamite", "dynamo", "dynasty", "dysfunction", "e-book", +"eagle", "eaglet", "ear", "eardrum", "earmuffs", "earnings", "earplug", "earring", "earrings", "earth", "earthquake", +"earthworm", "ease", "easel", "east", "eating", "eaves", "eavesdropper", "ecclesia", "echidna", "eclipse", "ecliptic", "ecology", "economics", +"economy", "ecosystem", "ectoderm", "ectodermal", "ecumenist", "eddy", "edge", "edger", "edible", "editing", "edition", "editor", "editorial", +"education", "eel", "effacement", "effect", "effective", "effectiveness", "effector", "efficacy", "efficiency", "effort", "egg", "egghead", +"eggnog", "eggplant", "ego", "eicosanoid", "ejector", "elbow", "elderberry", "election", "electricity", "electrocardiogram", "electronics", +"element", "elephant", "elevation", "elevator", "eleventh", "elf", "elicit", "eligibility", "elimination", "elite", "elixir", "elk", +"ellipse", "elm", "elongation", "elver", "email", "emanate", "embarrassment", "embassy", "embellishment", "embossing", "embryo", "emerald", +"emergence", "emergency", "emergent", "emery", "emission", "emitter", "emotion", "emphasis", "empire", "employ", "employee", "employer", +"employment", "empowerment", "emu", "enactment", "encirclement", "enclave", "enclosure", "encounter", "encouragement", "encyclopedia", +"end", "endive", "endoderm", "endorsement", "endothelium", "endpoint", "enemy", "energy", "enforcement", "engagement", "engine", "engineer", +"engineering", "enigma", "enjoyment", "enquiry", "enrollment", "enterprise", "entertainment", "enthusiasm", "entirety", "entity", "entrance", +"entree", "entrepreneur", "entry", "envelope", "environment", "envy", "enzyme", "epauliere", "epee", "ephemera", "ephemeris", "ephyra", +"epic", "episode", "epithelium", "epoch", "eponym", "epoxy", "equal", "equality", "equation", "equinox", "equipment", "equity", "equivalent", +"era", "eraser", "erection", "erosion", "error", "escalator", "escape", "escort", "espadrille", "espalier", "essay", "essence", "essential", +"establishment", "estate", "estimate", "estrogen", "estuary", "eternity", "ethernet", "ethics", "ethnicity", "ethyl", "euphonium", "eurocentrism", +"evaluation", "evaluator", "evaporation", "eve", "evening", "event", "everybody", "everyone", "everything", "eviction", +"evidence", "evil", "evocation", "evolution", "exaggeration", "exam", "examination", "examiner", "example", +"exasperation", "excellence", "exception", "excerpt", "excess", "exchange", "excitement", "exclamation", "excursion", "excuse", "execution", +"executive", "executor", "exercise", "exhaust", "exhaustion", "exhibit", "exhibition", "exile", "existence", "exit", "exocrine", "expansion", +"expansionism", "expectancy", "expectation", "expedition", "expense", "experience", "experiment", "experimentation", "expert", "expertise", +"explanation", "exploration", "explorer", "explosion", "export", "expose", "exposition", "exposure", "expression", "extension", "extent", +"exterior", "external", "extinction", "extreme", "extremist", "eye", "eyeball", "eyebrow", "eyebrows", "eyeglasses", "eyelash", "eyelashes", +"eyelid", "eyelids", "eyeliner", "eyestrain", "eyrie", "fabric", "face", "facelift", "facet", "facility", "facsimile", "fact", "factor", +"factory", "faculty", "fahrenheit", "fail", "failure", "fairness", "fairy", "faith", "faithful", "fall", "fallacy", "fame", +"familiar", "familiarity", "family", "fan", "fang", "fanlight", "fanny", "fantasy", "farm", "farmer", "farming", "farmland", +"farrow", "fascia", "fashion", "fat", "fate", "father", "fatigue", "fatigues", "faucet", "fault", "fav", "fava", "favor", +"favorite", "fawn", "fax", "fear", "feast", "feather", "feature", "fedelini", "federation", "fedora", "fee", "feed", "feedback", "feeding", +"feel", "feeling", "fellow", "felony", "female", "fen", "fence", "fencing", "fender", "feng", "fennel", "ferret", "ferry", "ferryboat", +"fertilizer", "festival", "fetus", "few", "fiber", "fiberglass", "fibre", "fibroblast", "fibrosis", "ficlet", "fiction", "fiddle", "field", +"fiery", "fiesta", "fifth", "fig", "fight", "fighter", "figure", "figurine", "file", "filing", "fill", "fillet", "filly", "film", "filter", +"filth", "final", "finance", "financing", "finding", "fine", "finer", "finger", "fingerling", "fingernail", "finish", "finisher", "fir", +"fire", "fireman", "fireplace", "firewall", "firm", "first", "fish", "fishbone", "fisherman", "fishery", "fishing", "fishmonger", "fishnet", +"fisting", "fit", "fitness", "fix", "fixture", "flag", "flair", "flame", "flan", "flanker", "flare", "flash", "flat", "flatboat", "flavor", +"flax", "fleck", "fledgling", "fleece", "flesh", "flexibility", "flick", "flicker", "flight", "flint", "flintlock", "flock", +"flood", "floodplain", "floor", "floozie", "flour", "flow", "flower", "flu", "flugelhorn", "fluke", "flume", "flung", "flute", "fly", +"flytrap", "foal", "foam", "fob", "focus", "fog", "fold", "folder", "folk", "folklore", "follower", "following", "fondue", "font", "food", +"foodstuffs", "fool", "foot", "footage", "football", "footnote", "footprint", "footrest", "footstep", "footstool", "footwear", "forage", +"forager", "foray", "force", "ford", "forearm", "forebear", "forecast", "forehead", "foreigner", "forelimb", "forest", "forestry", "forever", +"forgery", "fork", "form", "formal", "formamide", "format", "formation", "former", "formicarium", "formula", "fort", "forte", "fortnight", +"fortress", "fortune", "forum", "foundation", "founder", "founding", "fountain", "fourths", "fowl", "fox", "foxglove", "fraction", "fragrance", +"frame", "framework", "fratricide", "fraud", "fraudster", "freak", "freckle", "freedom", "freelance", "freezer", "freezing", "freight", +"freighter", "frenzy", "freon", "frequency", "fresco", "friction", "fridge", "friend", "friendship", "fries", "frigate", "fright", "fringe", +"fritter", "frock", "frog", "front", "frontier", "frost", "frosting", "frown", "fruit", "frustration", "fry", "fuck", "fuel", "fugato", +"fulfillment", "full", "fun", "function", "functionality", "fund", "funding", "fundraising", "funeral", "fur", "furnace", "furniture", +"furry", "fusarium", "futon", "future", "gadget", "gaffe", "gaffer", "gain", "gaiters", "gale", "gallery", "galley", +"gallon", "galoshes", "gambling", "game", "gamebird", "gaming", "gander", "gang", "gap", "garage", "garb", "garbage", "garden", +"garlic", "garment", "garter", "gas", "gasket", "gasoline", "gasp", "gastronomy", "gastropod", "gate", "gateway", "gather", "gathering", +"gator", "gauge", "gauntlet", "gavel", "gazebo", "gazelle", "gear", "gearshift", "geek", "gel", "gelatin", "gelding", "gem", "gemsbok", +"gender", "gene", "general", "generation", "generator", "generosity", "genetics", "genie", "genius", "genocide", "genre", "gentleman", +"geography", "geology", "geometry", "geranium", "gerbil", "gesture", "geyser", "gherkin", "ghost", "giant", "gift", "gig", "gigantism", +"giggle", "ginger", "gingerbread", "ginseng", "giraffe", "girdle", "girl", "girlfriend", "git", "glacier", "gladiolus", "glance", "gland", +"glass", "glasses", "glee", "glen", "glider", "gliding", "glimpse", "globe", "glockenspiel", "gloom", "glory", "glove", "glow", "glucose", +"glue", "glut", "glutamate", "gnat", "gnu", "goal", "goat", "gobbler", "god", "goddess", "godfather", "godmother", "godparent", +"goggles", "going", "gold", "goldfish", "golf", "gondola", "gong", "good", "goodbye", "goodie", "goodness", "goodnight", +"goodwill", "goose", "gopher", "gorilla", "gosling", "gossip", "governance", "government", "governor", "gown", "grace", "grade", +"gradient", "graduate", "graduation", "graffiti", "graft", "grain", "gram", "grammar", "gran", "grand", "grandchild", "granddaughter", +"grandfather", "grandma", "grandmom", "grandmother", "grandpa", "grandparent", "grandson", "granny", "granola", "grant", "grape", "grapefruit", +"graph", "graphic", "grasp", "grass", "grasshopper", "grassland", "gratitude", "gravel", "gravitas", "gravity", "gravy", "gray", "grease", +"greatness", "greed", "green", "greenhouse", "greens", "grenade", "grey", "grid", "grief", +"grill", "grin", "grip", "gripper", "grit", "grocery", "ground", "group", "grouper", "grouse", "grove", "growth", "grub", "guacamole", +"guarantee", "guard", "guava", "guerrilla", "guess", "guest", "guestbook", "guidance", "guide", "guideline", "guilder", "guilt", "guilty", +"guinea", "guitar", "guitarist", "gum", "gumshoe", "gun", "gunpowder", "gutter", "guy", "gym", "gymnast", "gymnastics", "gynaecology", +"gyro", "habit", "habitat", "hacienda", "hacksaw", "hackwork", "hail", "hair", "haircut", "hake", "half", "half-sister", +"halibut", "hall", "halloween", "hallway", "halt", "ham", "hamburger", "hammer", "hammock", "hamster", "hand", "handball", +"handful", "handgun", "handicap", "handle", "handlebar", "handmaiden", "handover", "handrail", "handsaw", "hanger", "happening", "happiness", +"harald", "harbor", "harbour", "hardboard", "hardcover", "hardening", "hardhat", "hardship", "hardware", "hare", "harm", +"harmonica", "harmonise", "harmonize", "harmony", "harp", "harpooner", "harpsichord", "harvest", "harvester", "hash", "hashtag", "hassock", +"haste", "hat", "hatbox", "hatchet", "hatchling", "hate", "hatred", "haunt", "haven", "haversack", "havoc", "hawk", "hay", "haze", "hazel", +"hazelnut", "head", "headache", "headlight", "headline", "headphones", "headquarters", "headrest", "health", "hearing", +"hearsay", "heart", "heartache", "heartbeat", "hearth", "hearthside", "heartwood", "heat", "heater", "heating", "heaven", +"heavy", "hectare", "hedge", "hedgehog", "heel", "heifer", "height", "heir", "heirloom", "helicopter", "helium", "hell", "hellcat", "hello", +"helmet", "helo", "help", "hemisphere", "hemp", "hen", "hepatitis", "herb", "herbs", "heritage", "hermit", "hero", "heroine", "heron", +"herring", "hesitation", "heterosexual", "hexagon", "heyday", "hiccups", "hide", "hierarchy", "high", "highland", "highlight", +"highway", "hike", "hiking", "hill", "hint", "hip", "hippodrome", "hippopotamus", "hire", "hiring", "historian", "history", "hit", "hive", +"hobbit", "hobby", "hockey", "hoe", "hog", "hold", "holder", "hole", "holiday", "home", "homeland", "homeownership", "hometown", "homework", +"homicide", "homogenate", "homonym", "homosexual", "homosexuality", "honesty", "honey", "honeybee", "honeydew", "honor", "honoree", "hood", +"hoof", "hook", "hop", "hope", "hops", "horde", "horizon", "hormone", "horn", "hornet", "horror", "horse", "horseradish", "horst", "hose", +"hosiery", "hospice", "hospital", "hospitalisation", "hospitality", "hospitalization", "host", "hostel", "hostess", "hotdog", "hotel", +"hound", "hour", "hourglass", "house", "houseboat", "household", "housewife", "housework", "housing", "hovel", "hovercraft", "howard", +"howitzer", "hub", "hubcap", "hubris", "hug", "hugger", "hull", "human", "humanity", "humidity", "hummus", "humor", "humour", "hunchback", +"hundred", "hunger", "hunt", "hunter", "hunting", "hurdle", "hurdler", "hurricane", "hurry", "hurt", "husband", "hut", "hutch", "hyacinth", +"hybridisation", "hybridization", "hydrant", "hydraulics", "hydrocarb", "hydrocarbon", "hydrofoil", "hydrogen", "hydrolyse", "hydrolysis", +"hydrolyze", "hydroxyl", "hyena", "hygienic", "hype", "hyphenation", "hypochondria", "hypothermia", "hypothesis", "ice", "ice-cream", +"iceberg", "icebreaker", "icecream", "icicle", "icing", "icon", "icy", "id", "idea", "ideal", "identification", "identity", "ideology", +"idiom", "idiot", "igloo", "ignorance", "ignorant", "ikebana", "illegal", "illiteracy", "illness", "illusion", "illustration", "image", +"imagination", "imbalance", "imitation", "immigrant", "immigration", "immortal", "impact", "impairment", "impala", "impediment", "implement", +"implementation", "implication", "import", "importance", "impostor", "impress", "impression", "imprisonment", "impropriety", "improvement", +"impudence", "impulse", "inability", "inauguration", "inbox", "incandescence", "incarnation", "incense", "incentive", +"inch", "incidence", "incident", "incision", "inclusion", "income", "incompetence", "inconvenience", "increase", "incubation", "independence", +"independent", "index", "indication", "indicator", "indigence", "individual", "industrialisation", "industrialization", "industry", "inequality", +"inevitable", "infancy", "infant", "infarction", "infection", "infiltration", "infinite", "infix", "inflammation", "inflation", "influence", +"influx", "info", "information", "infrastructure", "infusion", "inglenook", "ingrate", "ingredient", "inhabitant", "inheritance", "inhibition", +"inhibitor", "initial", "initialise", "initialize", "initiative", "injunction", "injury", "injustice", "ink", "inlay", "inn", "innervation", +"innocence", "innocent", "innovation", "input", "inquiry", "inscription", "insect", "insectarium", "insert", "inside", "insight", "insolence", +"insomnia", "inspection", "inspector", "inspiration", "installation", "instance", "instant", "instinct", "institute", "institution", +"instruction", "instructor", "instrument", "instrumentalist", "instrumentation", "insulation", "insurance", "insurgence", "insurrection", +"integer", "integral", "integration", "integrity", "intellect", "intelligence", "intensity", "intent", "intention", "intentionality", +"interaction", "interchange", "interconnection", "intercourse", "interest", "interface", "interferometer", "interior", "interject", "interloper", +"internet", "interpretation", "interpreter", "interval", "intervenor", "intervention", "interview", "interviewer", "intestine", "introduction", +"intuition", "invader", "invasion", "invention", "inventor", "inventory", "inverse", "inversion", "investigation", "investigator", "investment", +"investor", "invitation", "invite", "invoice", "involvement", "iridescence", "iris", "iron", "ironclad", "irony", "irrigation", "ischemia", +"island", "isogloss", "isolation", "issue", "item", "itinerary", "ivory", "jack", "jackal", "jacket", "jackfruit", "jade", "jaguar", +"jail", "jailhouse", "jalapeño", "jam", "jar", "jasmine", "jaw", "jazz", "jealousy", "jeans", "jeep", "jelly", "jellybeans", "jellyfish", +"jerk", "jet", "jewel", "jeweller", "jewellery", "jewelry", "jicama", "jiffy", "job", "jockey", "jodhpurs", "joey", "jogging", "joint", +"joke", "jot", "journal", "journalism", "journalist", "journey", "joy", "judge", "judgment", "judo", "jug", "juggernaut", "juice", "julienne", +"jumbo", "jump", "jumper", "jumpsuit", "jungle", "junior", "junk", "junker", "junket", "jury", "justice", "justification", "jute", "kale", +"kamikaze", "kangaroo", "karate", "kayak", "kazoo", "kebab", "keep", "keeper", "kendo", "kennel", "ketch", "ketchup", "kettle", "kettledrum", +"key", "keyboard", "keyboarding", "keystone", "kick", "kid", "kidney", "kielbasa", "kill", "killer", "killing", "kilogram", +"kilometer", "kilt", "kimono", "kinase", "kind", "kindness", "king", "kingdom", "kingfish", "kiosk", "kiss", "kit", "kitchen", "kite", +"kitsch", "kitten", "kitty", "kiwi", "knee", "kneejerk", "knickers", "knife", "knight", "knitting", "knock", "knot", "know-how", +"knowledge", "knuckle", "koala", "kohlrabi", "kumquat", "lab", "label", "labor", "laboratory", "laborer", "labour", "labourer", "lace", +"lack", "lacquerware", "lad", "ladder", "ladle", "lady", "ladybug", "lag", "lake", "lamb", "lambkin", "lament", "lamp", "lanai", "land", +"landform", "landing", "landmine", "landscape", "lane", "language", "lantern", "lap", "laparoscope", "lapdog", "laptop", "larch", "lard", +"larder", "lark", "larva", "laryngitis", "lasagna", "lashes", "last", "latency", "latex", "lathe", "latitude", "latte", "latter", "laugh", +"laughter", "laundry", "lava", "law", "lawmaker", "lawn", "lawsuit", "lawyer", "lay", "layer", "layout", "lead", "leader", "leadership", +"leading", "leaf", "league", "leaker", "leap", "learning", "leash", "leather", "leave", "leaver", "lecture", "leek", "leeway", "left", +"leg", "legacy", "legal", "legend", "legging", "legislation", "legislator", "legislature", "legitimacy", "legume", "leisure", "lemon", +"lemonade", "lemur", "lender", "lending", "length", "lens", "lentil", "leopard", "leprosy", "leptocephalus", "lesbian", "lesson", "letter", +"lettuce", "level", "lever", "leverage", "leveret", "liability", "liar", "liberty", "libido", "library", "licence", "license", "licensing", +"licorice", "lid", "lie", "lieu", "lieutenant", "life", "lifestyle", "lifetime", "lift", "ligand", "light", "lighting", "lightning", +"lightscreen", "ligula", "likelihood", "likeness", "lilac", "lily", "limb", "lime", "limestone", "limit", "limitation", "limo", "line", +"linen", "liner", "linguist", "linguistics", "lining", "link", "linkage", "linseed", "lion", "lip", "lipid", "lipoprotein", "lipstick", +"liquid", "liquidity", "liquor", "list", "listening", "listing", "literate", "literature", "litigation", "litmus", "litter", "littleneck", +"liver", "livestock", "living", "lizard", "llama", "load", "loading", "loaf", "loafer", "loan", "lobby", "lobotomy", "lobster", "local", +"locality", "location", "lock", "locker", "locket", "locomotive", "locust", "lode", "loft", "log", "loggia", "logic", "login", "logistics", +"logo", "loincloth", "lollipop", "loneliness", "longboat", "longitude", "look", "lookout", "loop", "loophole", "loquat", "lord", "loss", +"lot", "lotion", "lottery", "lounge", "louse", "lout", "love", "lover", "lox", "loyalty", "luck", "luggage", "lumber", "lumberman", "lunch", +"luncheonette", "lunchmeat", "lunchroom", "lung", "lunge", "lust", "lute", "luxury", "lychee", "lycra", "lye", "lymphocyte", "lynx", +"lyocell", "lyre", "lyrics", "lysine", "mRNA", "macadamia", "macaroni", "macaroon", "macaw", "machine", "machinery", "macrame", "macro", +"macrofauna", "madam", "maelstrom", "maestro", "magazine", "maggot", "magic", "magnet", "magnitude", "maid", "maiden", "mail", "mailbox", +"mailer", "mailing", "mailman", "main", "mainland", "mainstream", "maintainer", "maintenance", "maize", "major", "majority", +"makeover", "maker", "makeup", "making", "male", "malice", "mall", "mallard", "mallet", "malnutrition", "mama", "mambo", "mammoth", "man", +"manacle", "management", "manager", "manatee", "mandarin", "mandate", "mandolin", "mangle", "mango", "mangrove", "manhunt", "maniac", +"manicure", "manifestation", "manipulation", "mankind", "manner", "manor", "mansard", "manservant", "mansion", "mantel", "mantle", "mantua", +"manufacturer", "manufacturing", "many", "map", "maple", "mapping", "maracas", "marathon", "marble", "march", "mare", "margarine", "margin", +"mariachi", "marimba", "marines", "marionberry", "mark", "marker", "market", "marketer", "marketing", "marketplace", "marksman", "markup", +"marmalade", "marriage", "marsh", "marshland", "marshmallow", "marten", "marxism", "mascara", "mask", "masonry", "mass", "massage", "mast", +"master", "masterpiece", "mastication", "mastoid", "mat", "match", "matchmaker", "mate", "material", "maternity", "math", "mathematics", +"matrix", "matter", "mattock", "mattress", "max", "maximum", "maybe", "mayonnaise", "mayor", "meadow", "meal", "mean", "meander", "meaning", +"means", "meantime", "measles", "measure", "measurement", "meat", "meatball", "meatloaf", "mecca", "mechanic", "mechanism", "med", "medal", +"media", "median", "medication", "medicine", "medium", "meet", "meeting", "melatonin", "melody", "melon", "member", "membership", "membrane", +"meme", "memo", "memorial", "memory", "men", "menopause", "menorah", "mention", "mentor", "menu", "merchandise", "merchant", "mercury", +"meridian", "meringue", "merit", "mesenchyme", "mess", "message", "messenger", "messy", "metabolite", "metal", "metallurgist", "metaphor", +"meteor", "meteorology", "meter", "methane", "method", "methodology", "metric", "metro", "metronome", "mezzanine", "microlending", "micronutrient", +"microphone", "microwave", "midden", "middle", "middleman", "midline", "midnight", "midwife", "might", "migrant", "migration", +"mile", "mileage", "milepost", "milestone", "military", "milk", "milkshake", "mill", "millennium", "millet", "millimeter", "million", +"millisecond", "millstone", "mime", "mimosa", "min", "mincemeat", "mind", "mine", "mineral", "mineshaft", "mini", "minibus", +"minimalism", "minimum", "mining", "minion", "minister", "mink", "minnow", "minor", "minority", "mint", "minute", "miracle", +"mirror", "miscarriage", "miscommunication", "misfit", "misnomer", "misogyny", "misplacement", "misreading", "misrepresentation", "miss", +"missile", "mission", "missionary", "mist", "mistake", "mister", "misunderstand", "miter", "mitten", "mix", "mixer", "mixture", "moai", +"moat", "mob", "mobile", "mobility", "mobster", "moccasins", "mocha", "mochi", "mode", "model", "modeling", "modem", "modernist", "modernity", +"modification", "molar", "molasses", "molding", "mole", "molecule", "mom", "moment", "monastery", "monasticism", "money", "monger", "monitor", +"monitoring", "monk", "monkey", "monocle", "monopoly", "monotheism", "monsoon", "monster", "month", "monument", "mood", "moody", "moon", +"moonlight", "moonscape", "moonshine", "moose", "mop", "morale", "morbid", "morbidity", "morning", "moron", "morphology", "morsel", "mortal", +"mortality", "mortgage", "mortise", "mosque", "mosquito", "most", "motel", "moth", "mother", "motion", "motivation", +"motive", "motor", "motorboat", "motorcar", "motorcycle", "mound", "mountain", "mouse", "mouser", "mousse", "moustache", "mouth", "mouton", +"movement", "mover", "movie", "mower", "mozzarella", "mud", "muffin", "mug", "mukluk", "mule", "multimedia", "murder", "muscat", "muscatel", +"muscle", "musculature", "museum", "mushroom", "music", "musician", "muskrat", "mussel", "mustache", "mustard", +"mutation", "mutt", "mutton", "mycoplasma", "mystery", "myth", "mythology", "nail", "name", "naming", "nanoparticle", "napkin", "narrative", +"nasal", "nation", "nationality", "native", "naturalisation", "nature", "navigation", "necessity", "neck", "necklace", "necktie", "nectar", +"nectarine", "need", "needle", "neglect", "negligee", "negotiation", "neighbor", "neighborhood", "neighbour", "neighbourhood", "neologism", +"neon", "neonate", "nephew", "nerve", "nest", "nestling", "nestmate", "net", "netball", "netbook", "netsuke", "network", "networking", +"neurobiologist", "neuron", "neuropathologist", "neuropsychiatry", "news", "newsletter", "newspaper", "newsprint", "newsstand", "nexus", +"nibble", "nicety", "niche", "nick", "nickel", "nickname", "niece", "night", "nightclub", "nightgown", "nightingale", "nightlife", "nightlight", +"nightmare", "ninja", "nit", "nitrogen", "nobody", "nod", "node", "noir", "noise", "nonbeliever", "nonconformist", "nondisclosure", "nonsense", +"noodle", "noodles", "noon", "norm", "normal", "normalisation", "normalization", "north", "nose", "notation", "note", "notebook", "notepad", +"nothing", "notice", "notion", "notoriety", "nougat", "noun", "nourishment", "novel", "nucleotidase", "nucleotide", "nudge", "nuke", +"number", "numeracy", "numeric", "numismatist", "nun", "nurse", "nursery", "nursing", "nurture", "nut", "nutmeg", "nutrient", "nutrition", +"nylon", "nymph", "oak", "oar", "oasis", "oat", "oatmeal", "oats", "obedience", "obesity", "obi", "object", "objection", "objective", +"obligation", "oboe", "observation", "observatory", "obsession", "obsidian", "obstacle", "occasion", "occupation", "occurrence", "ocean", +"ocelot", "octagon", "octave", "octavo", "octet", "octopus", "odometer", "odyssey", "oeuvre", "offence", "offense", "offer", +"offering", "office", "officer", "official", "offset", "oil", "okra", "oldie", "oleo", "olive", "omega", "omelet", "omission", "omnivore", +"oncology", "onion", "online", "onset", "opening", "opera", "operating", "operation", "operator", "ophthalmologist", "opinion", "opium", +"opossum", "opponent", "opportunist", "opportunity", "opposite", "opposition", "optimal", "optimisation", "optimist", "optimization", +"option", "orange", "orangutan", "orator", "orchard", "orchestra", "orchid", "order", "ordinary", "ordination", "ore", "oregano", "organ", +"organisation", "organising", "organization", "organizing", "orient", "orientation", "origin", "original", "originality", "ornament", +"osmosis", "osprey", "ostrich", "other", "otter", "ottoman", "ounce", "outback", "outcome", "outfielder", "outfit", "outhouse", "outlaw", +"outlay", "outlet", "outline", "outlook", "output", "outrage", "outrigger", "outrun", "outset", "outside", "oval", "ovary", "oven", "overcharge", +"overclocking", "overcoat", "overexertion", "overflight", "overhead", "overheard", "overload", "overnighter", "overshoot", "oversight", +"overview", "overweight", "owl", "owner", "ownership", "ox", "oxford", "oxygen", "oyster", "ozone", "pace", "pacemaker", "pack", "package", +"packaging", "packet", "pad", "paddle", "paddock", "pagan", "page", "pagoda", "pail", "pain", "paint", "painter", "painting", "paintwork", +"pair", "pajamas", "palace", "palate", "palm", "pamphlet", "pan", "pancake", "pancreas", "panda", "panel", "panic", "pannier", "panpipe", +"pansy", "panther", "panties", "pantologist", "pantology", "pantry", "pants", "pantsuit", "panty", "pantyhose", "papa", "papaya", "paper", +"paperback", "paperwork", "parable", "parachute", "parade", "paradise", "paragraph", "parallelogram", "paramecium", "paramedic", "parameter", +"paranoia", "parcel", "parchment", "pard", "pardon", "parent", "parenthesis", "parenting", "park", "parka", "parking", "parliament", +"parole", "parrot", "parser", "parsley", "parsnip", "part", "participant", "participation", "particle", "particular", "partner", "partnership", +"partridge", "party", "pass", "passage", "passbook", "passenger", "passing", "passion", "passive", "passport", "password", "past", "pasta", +"paste", "pastor", "pastoralist", "pastry", "pasture", "pat", "patch", "pate", "patent", "patentee", "path", "pathogenesis", "pathology", +"pathway", "patience", "patient", "patina", "patio", "patriarch", "patrimony", "patriot", "patrol", "patroller", "patrolling", "patron", +"pattern", "patty", "pattypan", "pause", "pavement", "pavilion", "paw", "pawnshop", "pay", "payee", "payment", "payoff", "pea", "peace", +"peach", "peacoat", "peacock", "peak", "peanut", "pear", "pearl", "peasant", "pecan", "pecker", "pedal", "peek", "peen", "peer", "peer-to-peer", +"pegboard", "pelican", "pelt", "pen", "penalty", "pence", "pencil", "pendant", "pendulum", "penguin", "penicillin", "peninsula", "penis", +"pennant", "penny", "pension", "pentagon", "peony", "people", "pepper", "pepperoni", "percent", "percentage", "perception", "perch", +"perennial", "perfection", "performance", "perfume", "period", "periodical", "peripheral", "permafrost", "permission", "permit", "perp", +"perpendicular", "persimmon", "person", "personal", "personality", "personnel", "perspective", "pest", "pet", "petal", "petition", "petitioner", +"petticoat", "pew", "pharmacist", "pharmacopoeia", "phase", "pheasant", "phenomenon", "phenotype", "pheromone", "philanthropy", "philosopher", +"philosophy", "phone", "phosphate", "photo", "photodiode", "photograph", "photographer", "photography", "photoreceptor", "phrase", "phrasing", +"physical", "physics", "physiology", "pianist", "piano", "piccolo", "pick", "pickax", "pickaxe", "picket", "pickle", "pickup", "picnic", +"picture", "picturesque", "pie", "piece", "pier", "piety", "pig", "pigeon", "piglet", "pigpen", "pigsty", "pike", "pilaf", "pile", "pilgrim", +"pilgrimage", "pill", "pillar", "pillbox", "pillow", "pilot", "pimp", "pimple", "pin", "pinafore", "pine", "pineapple", +"pinecone", "ping", "pink", "pinkie", "pinot", "pinstripe", "pint", "pinto", "pinworm", "pioneer", "pipe", "pipeline", "piracy", "pirate", +"piss", "pistol", "pit", "pita", "pitch", "pitcher", "pitching", "pith", "pizza", "place", "placebo", "placement", "placode", "plagiarism", +"plain", "plaintiff", "plan", "plane", "planet", "planning", "plant", "plantation", "planter", "planula", "plaster", "plasterboard", +"plastic", "plate", "platelet", "platform", "platinum", "platter", "platypus", "play", "player", "playground", "playroom", "playwright", +"plea", "pleasure", "pleat", "pledge", "plenty", "plier", "pliers", "plight", "plot", "plough", "plover", "plow", "plowman", "plug", +"plugin", "plum", "plumber", "plume", "plunger", "plywood", "pneumonia", "pocket", "pocketbook", "pod", "podcast", "poem", +"poet", "poetry", "poignance", "point", "poison", "poisoning", "poker", "polarisation", "polarization", "pole", "polenta", "police", +"policeman", "policy", "polish", "politician", "politics", "poll", "polliwog", "pollutant", "pollution", "polo", "polyester", "polyp", +"pomegranate", "pomelo", "pompom", "poncho", "pond", "pony", "pool", "poor", "pop", "popcorn", "poppy", "popsicle", "popularity", "population", +"populist", "porcelain", "porch", "porcupine", "pork", "porpoise", "port", "porter", "portfolio", "porthole", "portion", "portrait", +"position", "possession", "possibility", "possible", "post", "postage", "postbox", "poster", "posterior", "postfix", "pot", "potato", +"potential", "pottery", "potty", "pouch", "poultry", "pound", "pounding", "poverty", "powder", "power", "practice", "practitioner", "prairie", +"praise", "pray", "prayer", "precedence", "precedent", "precipitation", "precision", "predecessor", "preface", "preference", "prefix", +"pregnancy", "prejudice", "prelude", "premeditation", "premier", "premise", "premium", "preoccupation", "preparation", "prescription", +"presence", "present", "presentation", "preservation", "preserves", "presidency", "president", "press", "pressroom", "pressure", "pressurisation", +"pressurization", "prestige", "presume", "pretzel", "prevalence", "prevention", "prey", "price", "pricing", "pride", "priest", "priesthood", +"primary", "primate", "prince", "princess", "principal", "principle", "print", "printer", "printing", "prior", "priority", "prison", +"prisoner", "privacy", "private", "privilege", "prize", "prizefight", "probability", "probation", "probe", "problem", "procedure", "proceedings", +"process", "processing", "processor", "proctor", "procurement", "produce", "producer", "product", "production", "productivity", "profession", +"professional", "professor", "profile", "profit", "progenitor", "program", "programme", "programming", "progress", "progression", "prohibition", +"project", "proliferation", "promenade", "promise", "promotion", "prompt", "pronoun", "pronunciation", "proof", "propaganda", +"propane", "property", "prophet", "proponent", "proportion", "proposal", "proposition", "proprietor", "prose", "prosecution", "prosecutor", +"prospect", "prosperity", "prostacyclin", "prostanoid", "prostrate", "protection", "protein", "protest", "protocol", "providence", "provider", +"province", "provision", "prow", "proximal", "proximity", "prune", "pruner", "pseudocode", "pseudoscience", "psychiatrist", "psychoanalyst", +"psychologist", "psychology", "ptarmigan", "pub", "public", "publication", "publicity", "publisher", "publishing", "pudding", "puddle", +"puffin", "pug", "puggle", "pulley", "pulse", "puma", "pump", "pumpernickel", "pumpkin", "pumpkinseed", "pun", "punch", "punctuation", +"punishment", "pup", "pupa", "pupil", "puppet", "puppy", "purchase", "puritan", "purity", "purple", "purpose", "purr", "purse", "pursuit", +"push", "pusher", "put", "puzzle", "pyramid", "pyridine", "quadrant", "quail", "qualification", "quality", "quantity", "quart", "quarter", +"quartet", "quartz", "queen", "query", "quest", "question", "questioner", "questionnaire", "quiche", "quicksand", "quiet", "quill", "quilt", +"quince", "quinoa", "quit", "quiver", "quota", "quotation", "quote", "rabbi", "rabbit", "raccoon", "race", "racer", "racing", "racism", +"racist", "rack", "radar", "radiator", "radio", "radiosonde", "radish", "raffle", "raft", "rag", "rage", "raid", "rail", "railing", "railroad", +"railway", "raiment", "rain", "rainbow", "raincoat", "rainmaker", "rainstorm", "rainy", "raise", "raisin", "rake", "rally", "ram", "rambler", +"ramen", "ramie", "ranch", "rancher", "randomisation", "randomization", "range", "ranger", "rank", "rap", "rape", "raspberry", "rat", +"rate", "ratepayer", "rating", "ratio", "rationale", "rations", "raven", "ravioli", "rawhide", "ray", "rayon", "razor", "reach", "reactant", +"reaction", "read", "reader", "readiness", "reading", "real", "reality", "realization", "realm", "reamer", "rear", "reason", "reasoning", +"rebel", "rebellion", "reboot", "recall", "recapitulation", "receipt", "receiver", "reception", "receptor", "recess", "recession", "recipe", +"recipient", "reciprocity", "reclamation", "recliner", "recognition", "recollection", "recommendation", "reconsideration", "record", +"recorder", "recording", "recovery", "recreation", "recruit", "rectangle", "red", "redesign", "redhead", "redirect", "rediscovery", "reduction", +"reef", "refectory", "reference", "referendum", "reflection", "reform", "refreshments", "refrigerator", "refuge", "refund", "refusal", +"refuse", "regard", "regime", "region", "regionalism", "register", "registration", "registry", "regret", "regulation", "regulator", "rehospitalisation", +"rehospitalization", "reindeer", "reinscription", "reject", "relation", "relationship", "relative", "relaxation", "relay", "release", +"reliability", "relief", "religion", "relish", "reluctance", "remains", "remark", "reminder", "remnant", "remote", "removal", "renaissance", +"rent", "reorganisation", "reorganization", "repair", "reparation", "repayment", "repeat", "replacement", "replica", "replication", "reply", +"report", "reporter", "reporting", "repository", "representation", "representative", "reprocessing", "republic", "republican", "reputation", +"request", "requirement", "resale", "rescue", "research", "researcher", "resemblance", "reservation", "reserve", "reservoir", "reset", +"residence", "resident", "residue", "resist", "resistance", "resolution", "resolve", "resort", "resource", "respect", "respite", "response", +"responsibility", "rest", "restaurant", "restoration", "restriction", "restroom", "restructuring", "result", "resume", "retailer", "retention", +"rethinking", "retina", "retirement", "retouching", "retreat", "retrospect", "retrospective", "retrospectivity", "return", "reunion", +"revascularisation", "revascularization", "reveal", "revelation", "revenant", "revenge", "revenue", "reversal", "reverse", "review", +"revitalisation", "revitalization", "revival", "revolution", "revolver", "reward", "rhetoric", "rheumatism", "rhinoceros", "rhubarb", +"rhyme", "rhythm", "rib", "ribbon", "rice", "riddle", "ride", "rider", "ridge", "riding", "rifle", "right", "rim", "ring", "ringworm", +"riot", "rip", "ripple", "rise", "riser", "risk", "rite", "ritual", "river", "riverbed", "rivulet", "road", "roadway", "roar", "roast", +"robe", "robin", "robot", "robotics", "rock", "rocker", "rocket", "rod", "role", "roll", "roller", "romaine", "romance", +"roof", "room", "roommate", "rooster", "root", "rope", "rose", "rosemary", "roster", "rostrum", "rotation", "round", "roundabout", "route", +"router", "routine", "row", "rowboat", "rowing", "rubber", "rubbish", "rubric", "ruby", "ruckus", "rudiment", "ruffle", "rug", "rugby", +"ruin", "rule", "ruler", "ruling", "rum", "rumor", "run", "runaway", "runner", "running", "runway", "rush", "rust", "rutabaga", "rye", +"sabre", "sac", "sack", "saddle", "sadness", "safari", "safe", "safeguard", "safety", "saffron", "sage", "sail", "sailboat", "sailing", +"sailor", "saint", "sake", "salad", "salami", "salary", "sale", "salesman", "salmon", "salon", "saloon", "salsa", "salt", "salute", "samovar", +"sampan", "sample", "samurai", "sanction", "sanctity", "sanctuary", "sand", "sandal", "sandbar", "sandpaper", "sandwich", "sanity", "sardine", +"sari", "sarong", "sash", "satellite", "satin", "satire", "satisfaction", "sauce", "saucer", "sauerkraut", "sausage", "savage", "savannah", +"saving", "savings", "savior", "saviour", "savory", "saw", "saxophone", "scaffold", "scale", "scallion", "scallops", "scalp", "scam", +"scanner", "scarecrow", "scarf", "scarification", "scenario", "scene", "scenery", "scent", "schedule", "scheduling", "schema", "scheme", +"schizophrenic", "schnitzel", "scholar", "scholarship", "school", "schoolhouse", "schooner", "science", "scientist", "scimitar", "scissors", +"scooter", "scope", "score", "scorn", "scorpion", "scotch", "scout", "scow", "scrambled", "scrap", "scraper", "scratch", "screamer", +"screen", "screening", "screenwriting", "screw", "screwdriver", "scrim", "scrip", "script", "scripture", "scrutiny", "sculpting", +"sculptural", "sculpture", "sea", "seabass", "seafood", "seagull", "seal", "seaplane", "search", "seashore", "seaside", "season", "seat", +"seaweed", "second", "secrecy", "secret", "secretariat", "secretary", "secretion", "section", "sectional", "sector", "security", "sediment", +"seed", "seeder", "seeker", "seep", "segment", "seizure", "selection", "self", "seller", +"selling", "semantics", "semester", "semicircle", "semicolon", "semiconductor", "seminar", "senate", "senator", "sender", "senior", "sense", +"sensibility", "sensitive", "sensitivity", "sensor", "sentence", "sentencing", "sentiment", "sepal", "separation", "septicaemia", "sequel", +"sequence", "serial", "series", "sermon", "serum", "serval", "servant", "server", "service", "servitude", "sesame", "session", "set", +"setback", "setting", "settlement", "settler", "severity", "sewer", "sex", "sexuality", "shack", "shackle", "shade", "shadow", "shadowbox", +"shakedown", "shaker", "shallot", "shallows", "shame", "shampoo", "shanty", "shape", "share", "shareholder", "shark", "shaw", "shawl", +"shear", "shearling", "sheath", "shed", "sheep", "sheet", "shelf", "shell", "shelter", "sherbet", "sherry", "shield", "shift", "shin", +"shine", "shingle", "ship", "shipper", "shipping", "shipyard", "shirt", "shirtdress", "shit", "shoat", "shock", "shoe", "shoe-horn", +"shoehorn", "shoelace", "shoemaker", "shoes", "shoestring", "shofar", "shoot", "shootdown", "shop", "shopper", "shopping", "shore", "shoreline", +"short", "shortage", "shorts", "shortwave", "shot", "shoulder", "shout", "shovel", "show", "shower", "shred", "shrimp", +"shrine", "shutdown", "sibling", "sick", "sickness", "side", "sideboard", "sideburns", "sidecar", "sidestream", "sidewalk", "siding", +"siege", "sigh", "sight", "sightseeing", "sign", "signal", "signature", "signet", "significance", "signify", "signup", "silence", "silica", +"silicon", "silk", "silkworm", "sill", "silly", "silo", "silver", "similarity", "simple", "simplicity", "simplification", "simvastatin", +"sin", "singer", "singing", "singular", "sink", "sinuosity", "sip", "sir", "sister", "sitar", "site", "situation", "size", +"skate", "skating", "skean", "skeleton", "ski", "skiing", "skill", "skin", "skirt", "skull", "skullcap", "skullduggery", "skunk", "sky", +"skylight", "skyline", "skyscraper", "skywalk", "slang", "slapstick", "slash", "slate", "slave", "slavery", "slaw", "sled", "sledge", +"sleep", "sleepiness", "sleeping", "sleet", "sleuth", "slice", "slide", "slider", "slime", "slip", "slipper", "slippers", "slope", "slot", +"sloth", "slump", "smell", "smelting", "smile", "smith", "smock", "smog", "smoke", "smoking", "smolt", "smuggling", "snack", "snail", +"snake", "snakebite", "snap", "snarl", "sneaker", "sneakers", "sneeze", "sniffle", "snob", "snorer", "snow", "snowboarding", "snowflake", +"snowman", "snowmobiling", "snowplow", "snowstorm", "snowsuit", "snuck", "snug", "snuggle", "soap", "soccer", "socialism", "socialist", +"society", "sociology", "sock", "socks", "soda", "sofa", "softball", "softdrink", "softening", "software", "soil", "soldier", "sole", +"solicitation", "solicitor", "solidarity", "solidity", "soliloquy", "solitaire", "solution", "solvency", "sombrero", "somebody", "someone", +"someplace", "somersault", "something", "somewhere", "son", "sonar", "sonata", "song", "songbird", "sonnet", "soot", "sophomore", "soprano", +"sorbet", "sorghum", "sorrel", "sorrow", "sort", "soul", "soulmate", "sound", "soundness", "soup", "source", "sourwood", "sousaphone", +"south", "southeast", "souvenir", "sovereignty", "sow", "soy", "soybean", "space", "spacing", "spade", "spaghetti", "span", "spandex", +"spank", "sparerib", "spark", "sparrow", "spasm", "spat", "spatula", "spawn", "speaker", "speakerphone", "speaking", "spear", "spec", +"special", "specialist", "specialty", "species", "specification", "spectacle", "spectacles", "spectrograph", "spectrum", "speculation", +"speech", "speed", "speedboat", "spell", "spelling", "spelt", "spending", "sphere", "sphynx", "spice", "spider", "spiderling", "spike", +"spill", "spinach", "spine", "spiral", "spirit", "spiritual", "spirituality", "spit", "spite", "spleen", "splendor", "split", "spokesman", +"spokeswoman", "sponge", "sponsor", "sponsorship", "spool", "spoon", "spork", "sport", "sportsman", "spot", "spotlight", "spouse", "sprag", +"sprat", "spray", "spread", "spreadsheet", "spree", "spring", "sprinkles", "sprinter", "sprout", "spruce", "spud", "spume", "spur", "spy", +"spyglass", "square", "squash", "squatter", "squeegee", "squid", "squirrel", "stab", "stability", "stable", "stack", "stacking", "stadium", +"staff", "stag", "stage", "stain", "stair", "staircase", "stake", "stalk", "stall", "stallion", "stamen", "stamina", "stamp", "stance", +"stand", "standard", "standardisation", "standardization", "standing", "standoff", "standpoint", "star", "starboard", "start", "starter", +"state", "statement", "statin", "station", "statistic", "statistics", "statue", "status", "statute", "stay", "steak", +"stealth", "steam", "steamroller", "steel", "steeple", "stem", "stench", "stencil", "step", "step-daughter", +"stepdaughter", "stepmother", +"stepson", "stereo", "stew", "steward", "stick", "sticker", "stiletto", "still", "stimulation", "stimulus", "sting", +"stinger", "stitch", "stitcher", "stock", "stockings", "stole", "stomach", "stone", "stonework", "stool", +"stop", "stopsign", "stopwatch", "storage", "store", "storey", "storm", "story", "storyboard", "stot", "stove", "strait", +"strand", "stranger", "strap", "strategy", "straw", "strawberry", "strawman", "stream", "street", "streetcar", "strength", "stress", +"stretch", "strife", "strike", "string", "strip", "stripe", "strobe", "stroke", "structure", "strudel", "struggle", "stucco", "stud", +"student", "studio", "study", "stuff", "stumbling", "stump", "stupidity", "sturgeon", "sty", "style", "styling", "stylus", "sub", "subcomponent", +"subconscious", "subcontractor", "subexpression", "subgroup", "subject", "submarine", "submitter", "subprime", "subroutine", "subscription", +"subsection", "subset", "subsidence", "subsidiary", "subsidy", "substance", "substitution", "subtitle", "suburb", "subway", "success", +"succotash", "suck", "sucker", "suede", "suet", "suffocation", "sugar", "suggestion", "suicide", "suit", "suitcase", "suite", "sulfur", +"sultan", "sum", "summary", "summer", "summit", "sun", "sunbeam", "sunbonnet", "sundae", "sunday", "sundial", "sunflower", "sunglasses", +"sunlamp", "sunlight", "sunrise", "sunroom", "sunset", "sunshine", "superiority", "supermarket", "supernatural", "supervision", "supervisor", +"supper", "supplement", "supplier", "supply", "support", "supporter", "suppression", "supreme", "surface", "surfboard", "surge", "surgeon", +"surgery", "surname", "surplus", "surprise", "surround", "surroundings", "surrounds", "survey", "survival", "survivor", "sushi", "suspect", +"suspenders", "suspension", "sustainment", "sustenance", "swallow", "swamp", "swan", "swanling", "swath", "sweat", "sweater", "sweatshirt", +"sweatshop", "sweatsuit", "sweets", "swell", "swim", "swimming", "swimsuit", "swine", "swing", "switch", "switchboard", "switching", +"swivel", "sword", "swordfight", "swordfish", "sycamore", "symbol", "symmetry", "sympathy", "symptom", "syndicate", "syndrome", "synergy", +"synod", "synonym", "synthesis", "syrup", "system", "tab", "tabby", "tabernacle", "table", "tablecloth", "tablet", "tabletop", +"tachometer", "tackle", "taco", "tactics", "tactile", "tadpole", "tag", "tail", "tailbud", "tailor", "tailspin", "takeover", +"tale", "talent", "talk", "talking", "tamale", "tambour", "tambourine", "tan", "tandem", "tangerine", "tank", "tank-top", +"tanker", "tankful", "tap", "tape", "tapioca", "target", "taro", "tarragon", "tart", "task", "tassel", "taste", "tatami", "tattler", +"tattoo", "tavern", "tax", "taxi", "taxicab", "taxpayer", "tea", "teacher", "teaching", "team", "teammate", "teapot", "tear", "tech", +"technician", "technique", "technologist", "technology", "tectonics", "teen", "teenager", "teepee", "telephone", "telescreen", "teletype", +"television", "tell", "teller", "temp", "temper", "temperature", "temple", "tempo", "temporariness", "temporary", "temptation", "temptress", +"tenant", "tendency", "tender", "tenement", "tenet", "tennis", "tenor", "tension", "tensor", "tent", "tentacle", "tenth", "tepee", "teriyaki", +"term", "terminal", "termination", "terminology", "termite", "terrace", "terracotta", "terrapin", "terrarium", "territory", "terror", +"terrorism", "terrorist", "test", "testament", "testimonial", "testimony", "testing", "text", "textbook", "textual", "texture", "thanks", +"thaw", "theater", "theft", "theism", "theme", "theology", "theory", "therapist", "therapy", "thermals", "thermometer", "thermostat", +"thesis", "thickness", "thief", "thigh", "thing", "thinking", "thirst", "thistle", "thong", "thongs", "thorn", "thought", "thousand", +"thread", "threat", "threshold", "thrift", "thrill", "throat", "throne", "thrush", "thrust", "thug", "thumb", "thump", "thunder", "thunderbolt", +"thunderhead", "thunderstorm", "thyme", "tiara", "tic", "tick", "ticket", "tide", "tie", "tiger", "tights", "tile", "till", "tilt", "timbale", +"timber", "time", "timeline", "timeout", "timer", "timetable", "timing", "timpani", "tin", "tinderbox", "tinkle", "tintype", "tip", "tire", +"tissue", "titanium", "title", "toad", "toast", "toaster", "tobacco", "today", "toe", "toenail", "toffee", "tofu", "tog", "toga", "toilet", +"tolerance", "tolerant", "toll", "tomatillo", "tomato", "tomb", "tomography", "tomorrow", "ton", "tonality", "tone", "tongue", +"tonic", "tonight", "tool", "toot", "tooth", "toothbrush", "toothpaste", "toothpick", "top", "topic", "topsail", "toque", +"toreador", "tornado", "torso", "torte", "tortellini", "tortilla", "tortoise", "tosser", "total", "tote", "touch", "tour", +"tourism", "tourist", "tournament", "towel", "tower", "town", "townhouse", "township", "toy", "trace", "trachoma", "track", +"tracking", "tracksuit", "tract", "tractor", "trade", "trader", "trading", "tradition", "traditionalism", "traffic", "trafficker", "tragedy", +"trail", "trailer", "trailpatrol", "train", "trainer", "training", "trait", "tram", "tramp", "trance", "transaction", "transcript", "transfer", +"transformation", "transit", "transition", "translation", "transmission", "transom", "transparency", "transplantation", "transport", +"transportation", "trap", "trapdoor", "trapezium", "trapezoid", "trash", "travel", "traveler", "tray", "treasure", "treasury", "treat", +"treatment", "treaty", "tree", "trek", "trellis", "tremor", "trench", "trend", "triad", "trial", "triangle", "tribe", "tributary", "trick", +"trigger", "trigonometry", "trillion", "trim", "trinket", "trip", "tripod", "tritone", "triumph", "trolley", "trombone", "troop", "trooper", +"trophy", "trouble", "trousers", "trout", "trove", "trowel", "truck", "trumpet", "trunk", "trust", "trustee", "truth", "try", "tsunami", +"tub", "tuba", "tube", "tuber", "tug", "tugboat", "tuition", "tulip", "tumbler", "tummy", "tuna", "tune", "tunic", "tunnel", +"turban", "turf", "turkey", "turmeric", "turn", "turning", "turnip", "turnover", "turnstile", "turret", "turtle", "tusk", "tussle", "tutu", +"tuxedo", "tweet", "tweezers", "twig", "twilight", "twine", "twins", "twist", "twister", "twitter", "type", "typeface", "typewriter", +"typhoon", "ukulele", "ultimatum", "umbrella", "unblinking", "uncertainty", "uncle", "underclothes", "underestimate", "underground", +"underneath", "underpants", "underpass", "undershirt", "understanding", "understatement", "undertaker", "underwear", "underweight", "underwire", +"underwriting", "unemployment", "unibody", "uniform", "uniformity", "union", "unique", "unit", "unity", "universe", "university", "update", +"upgrade", "uplift", "upper", "upstairs", "upward", "urge", "urgency", "urn", "usage", "use", "user", "usher", "usual", "utensil", "utilisation", +"utility", "utilization", "vacation", "vaccine", "vacuum", "vagrant", "valance", "valentine", "validate", "validity", "valley", "valuable", +"value", "vampire", "van", "vanadyl", "vane", "vanilla", "vanity", "variability", "variable", "variant", "variation", "variety", "vascular", +"vase", "vault", "vaulting", "veal", "vector", "vegetable", "vegetarian", "vegetarianism", "vegetation", "vehicle", "veil", "vein", "veldt", +"vellum", "velocity", "velodrome", "velvet", "vendor", "veneer", "vengeance", "venison", "venom", "venti", "venture", "venue", "veranda", +"verb", "verdict", "verification", "vermicelli", "vernacular", "verse", "version", "vertigo", "verve", "vessel", "vest", "vestment", +"vet", "veteran", "veterinarian", "veto", "viability", "vibe", "vibraphone", "vibration", "vibrissae", "vice", "vicinity", "victim", +"victory", "video", "view", "viewer", "vignette", "villa", "village", "vine", "vinegar", "vineyard", "vintage", "vintner", "vinyl", "viola", +"violation", "violence", "violet", "violin", "virginal", "virtue", "virus", "visa", "viscose", "vise", "vision", "visit", "visitor", +"visor", "vista", "visual", "vitality", "vitamin", "vitro", "vivo", "vixen", "vodka", "vogue", "voice", "void", "vol", "volatility", +"volcano", "volleyball", "volume", "volunteer", "volunteering", "vomit", "vote", "voter", "voting", "voyage", "vulture", "wad", "wafer", +"waffle", "wage", "wagon", "waist", "waistband", "wait", "waiter", "waiting", "waitress", "waiver", "wake", "walk", "walker", "walking", +"walkway", "wall", "wallaby", "wallet", "walnut", "walrus", "wampum", "wannabe", "want", "war", "warden", "wardrobe", "warfare", "warlock", +"warlord", "warming", "warmth", "warning", "warrant", "warren", "warrior", "wasabi", "wash", "washbasin", "washcloth", "washer", +"washtub", "wasp", "waste", "wastebasket", "wasting", "watch", "watcher", "watchmaker", "water", "waterbed", "watercress", "waterfall", +"waterfront", "watermelon", "waterskiing", "waterspout", "waterwheel", "wave", "waveform", "wax", "way", "weakness", "wealth", "weapon", +"wear", "weasel", "weather", "web", "webinar", "webmail", "webpage", "website", "wedding", "wedge", "weed", "weeder", "weedkiller", "week", +"weekend", "weekender", "weight", "weird", "welcome", "welfare", "well", "west", "western", "wetland", "wetsuit", +"whack", "whale", "wharf", "wheat", "wheel", "whelp", "whey", "whip", "whirlpool", "whirlwind", "whisker", "whiskey", "whisper", "whistle", +"white", "whole", "wholesale", "wholesaler", "whorl", "wick", "widget", "widow", "width", "wife", "wifi", "wild", "wildebeest", "wilderness", +"wildlife", "will", "willingness", "willow", "win", "wind", "windage", "window", "windscreen", "windshield", "wine", "winery", +"wing", "wingman", "wingtip", "wink", "winner", "winter", "wire", "wiretap", "wiring", "wisdom", "wiseguy", "wish", "wisteria", "wit", +"witch", "withdrawal", "witness", "wok", "wolf", "woman", "wombat", "wonder", "wont", "wood", "woodchuck", "woodland", +"woodshed", "woodwind", "wool", "woolens", "word", "wording", "work", "workbench", "worker", "workforce", "workhorse", "working", "workout", +"workplace", "workshop", "world", "worm", "worry", "worship", "worshiper", "worth", "wound", "wrap", "wraparound", "wrapper", "wrapping", +"wreck", "wrecker", "wren", "wrench", "wrestler", "wriggler", "wrinkle", "wrist", "writer", "writing", "wrong", "xylophone", "yacht", +"yahoo", "yak", "yam", "yang", "yard", "yarmulke", "yarn", "yawl", "year", "yeast", "yellow", "yellowjacket", "yesterday", "yew", "yin", +"yoga", "yogurt", "yoke", "yolk", "young", "youngster", "yourself", "youth", "yoyo", "yurt", "zampone", "zebra", "zebrafish", "zen", +"zephyr", "zero", "ziggurat", "zinc", "zipper", "zither", "zombie", "zone", "zoo", "zoologist", "zoology", "zucchini" +}; + + +std::string_view obfuscateWord(std::string_view src, WordMap & obfuscate_map, WordSet & used_nouns, SipHash hash_func) +{ + /// Prevent using too many nouns + if (obfuscate_map.size() * 2 > nouns.size()) + throw Exception("Too many unique identifiers in queries", ErrorCodes::TOO_MANY_TEMPORARY_COLUMNS); + + std::string_view & mapped = obfuscate_map[src]; + if (!mapped.empty()) + return mapped; + + hash_func.update(src.data(), src.size()); + std::string_view noun = nouns.begin()[hash_func.get64() % nouns.size()]; + + /// Prevent collisions + while (!used_nouns.insert(noun).second) + { + hash_func.update('\0'); + noun = nouns.begin()[hash_func.get64() % nouns.size()]; + } + + mapped = noun; + return mapped; +} + + +void obfuscateIdentifier(std::string_view src, WriteBuffer & result, WordMap & obfuscate_map, WordSet & used_nouns, SipHash hash_func) +{ + /// Find words in form 'snake_case', 'CamelCase' or 'ALL_CAPS'. + + const char * src_pos = src.data(); + const char * src_end = src_pos + src.size(); + + const char * word_begin = src_pos; + bool word_has_alphanumerics = false; + + auto append_word = [&] + { + std::string_view word(word_begin, src_pos - word_begin); + + if (keep_words.count(word)) + { + result.write(word.data(), word.size()); + } + else + { + std::string_view obfuscated_word = obfuscateWord(word, obfuscate_map, used_nouns, hash_func); + + /// Match the style of source word. + bool first_caps = !word.empty() && isUpperAlphaASCII(word[0]); + bool all_caps = first_caps && word.size() >= 2 && isUpperAlphaASCII(word[1]); + + for (size_t i = 0, size = obfuscated_word.size(); i < size; ++i) + { + if (all_caps || (i == 0 && first_caps)) + result.write(toUpperIfAlphaASCII(obfuscated_word[i])); + else + result.write(obfuscated_word[i]); + } + } + + word_begin = src_pos; + word_has_alphanumerics = false; + }; + + while (src_pos < src_end) + { + if (isAlphaNumericASCII(src_pos[0])) + word_has_alphanumerics = true; + + if (word_has_alphanumerics && src_pos[0] == '_') + { + append_word(); + result.write('_'); + ++word_begin; + } + else if (word_has_alphanumerics && isUpperAlphaASCII(src_pos[0]) && isLowerAlphaASCII(src_pos[-1])) /// xX + { + append_word(); + } + + ++src_pos; + } + + if (word_begin < src_pos) + append_word(); +} + + +void obfuscateLiteral(std::string_view src, WriteBuffer & result, SipHash hash_func) +{ + const char * src_pos = src.data(); + const char * src_end = src_pos + src.size(); + + while (src_pos < src_end) + { + /// Date + if (src_pos + strlen("0000-00-00") <= src_end + && isNumericASCII(src_pos[0]) + && isNumericASCII(src_pos[1]) + && isNumericASCII(src_pos[2]) + && isNumericASCII(src_pos[3]) + && src_pos[4] == '-' + && isNumericASCII(src_pos[5]) + && isNumericASCII(src_pos[6]) + && src_pos[7] == '-' + && isNumericASCII(src_pos[8]) + && isNumericASCII(src_pos[9])) + { + DayNum date; + ReadBufferFromMemory in(src_pos, strlen("0000-00-00")); + readDateText(date, in); + + SipHash hash_func_date = hash_func; + + if (date != 0) + { + date += hash_func_date.get64() % 256; + } + + writeDateText(date, result); + src_pos += strlen("0000-00-00"); + + /// DateTime + if (src_pos + strlen(" 00:00:00") <= src_end + && isNumericASCII(src_pos[1]) + && isNumericASCII(src_pos[2]) + && src_pos[3] == ':' + && isNumericASCII(src_pos[4]) + && isNumericASCII(src_pos[5]) + && src_pos[6] == ':' + && isNumericASCII(src_pos[7]) + && isNumericASCII(src_pos[8])) + { + result.write(src_pos[0]); + + hash_func_date.update(src_pos + 1, strlen("00:00:00")); + + uint64_t hash_value = hash_func_date.get64(); + uint32_t new_hour = hash_value % 24; + hash_value /= 24; + uint32_t new_minute = hash_value % 60; + hash_value /= 60; + uint32_t new_second = hash_value % 60; + + result.write('0' + (new_hour / 10)); + result.write('0' + (new_hour % 10)); + result.write(':'); + result.write('0' + (new_minute / 10)); + result.write('0' + (new_minute % 10)); + result.write(':'); + result.write('0' + (new_second / 10)); + result.write('0' + (new_second % 10)); + + src_pos += strlen(" 00:00:00"); + } + } + else if (isNumericASCII(src_pos[0])) + { + /// Number + if (src_pos[0] == '0' || src_pos[0] == '1') + { + /// Keep zero and one as is. + result.write(src_pos[0]); + ++src_pos; + } + else + { + ReadBufferFromMemory in(src_pos, src_end - src_pos); + uint64_t num; + readIntText(num, in); + SipHash hash_func_num = hash_func; + hash_func_num.update(src_pos, in.count()); + src_pos += in.count(); + + /// Obfuscate number but keep it within same power of two range. + + uint64_t obfuscated = hash_func_num.get64(); + uint64_t log2 = bitScanReverse(num); + + obfuscated = (1ULL << log2) + obfuscated % (1ULL << log2); + writeIntText(obfuscated, result); + } + } + else if (src_pos + 1 < src_end + && (src_pos[0] == 'e' || src_pos[0] == 'E') + && (isNumericASCII(src_pos[1]) || (src_pos[1] == '-' && src_pos + 2 < src_end && isNumericASCII(src_pos[2])))) + { + /// Something like an exponent of floating point number. Keep it as is. + /// But if it looks like a large number, overflow it into 16 bit. + + result.write(src_pos[0]); + ++src_pos; + + ReadBufferFromMemory in(src_pos, src_end - src_pos); + int16_t num; + readIntText(num, in); + writeIntText(num, result); + src_pos += in.count(); + } + else if (isAlphaASCII(src_pos[0])) + { + /// Alphabetial characters + + const char * alpha_end = src_pos + 1; + while (alpha_end < src_end && isAlphaASCII(*alpha_end)) + ++alpha_end; + + hash_func.update(src_pos, alpha_end - src_pos); + pcg64 rng(hash_func.get64()); + + while (src_pos < alpha_end) + { + auto random = rng(); + if (isLowerAlphaASCII(*src_pos)) + result.write('a' + random % 26); + else + result.write('A' + random % 26); + + ++src_pos; + } + } + else if (isASCII(src_pos[0])) + { + /// Punctuation, whitespace and control characters - keep as is. + + result.write(src_pos[0]); + ++src_pos; + } + else if (src_pos[0] <= '\xBF') + { + /// Continuation of UTF-8 sequence. + hash_func.update(src_pos[0]); + uint64_t hash = hash_func.get64(); + + char c = 0x80 + hash % (0xC0 - 0x80); + result.write(c); + + ++src_pos; + } + else if (src_pos[0]) + { + /// Start of UTF-8 sequence. + hash_func.update(src_pos[0]); + uint64_t hash = hash_func.get64(); + + if (src_pos[0] < '\xE0') + { + char c = 0xC0 + hash % 32; + result.write(c); + } + else if (src_pos[0] < '\xF0') + { + char c = 0xE0 + hash % 16; + result.write(c); + } + else + { + char c = 0xF0 + hash % 8; + result.write(c); + } + + ++src_pos; + } + } +} + +} + + +void obfuscateQueries( + std::string_view src, + WriteBuffer & result, + WordMap & obfuscate_map, + WordSet & used_nouns, + SipHash hash_func, + KnownIdentifierFunc known_identifier_func) +{ + Lexer lexer(src.data(), src.data() + src.size()); + while (true) + { + Token token = lexer.nextToken(); + std::string_view whole_token(token.begin, token.size()); + + if (token.isEnd()) + break; + + if (token.type == TokenType::BareWord) + { + if (keywords.count(whole_token) + || known_identifier_func(whole_token)) + { + /// Keep keywords as is. + result.write(token.begin, token.size()); + } + else + { + /// Obfuscate identifiers + obfuscateIdentifier(whole_token, result, obfuscate_map, used_nouns, hash_func); + } + } + else if (token.type == TokenType::QuotedIdentifier) + { + assert(token.size() >= 2); + + /// Write quotes and the obfuscated content inside. + result.write(*token.begin); + obfuscateIdentifier({token.begin + 1, token.size() - 2}, result, obfuscate_map, used_nouns, hash_func); + result.write(token.end[-1]); + } + else if (token.type == TokenType::Number) + { + obfuscateLiteral(whole_token, result, hash_func); + } + else if (token.type == TokenType::StringLiteral) + { + assert(token.size() >= 2); + + result.write(*token.begin); + obfuscateLiteral({token.begin + 1, token.size() - 2}, result, hash_func); + result.write(token.end[-1]); + } + else if (token.type == TokenType::Comment) + { + /// Skip comments - they may contain confidential info. + } + else + { + /// Everyting else is kept as is. + result.write(token.begin, token.size()); + } + } +} + +} + diff --git a/src/Parsers/obfuscateQueries.h b/src/Parsers/obfuscateQueries.h new file mode 100644 index 00000000000..214237e19cf --- /dev/null +++ b/src/Parsers/obfuscateQueries.h @@ -0,0 +1,29 @@ +#pragma once + +#include +#include +#include +#include +#include + +#include + + +namespace DB +{ + +class WriteBuffer; + +using WordMap = std::unordered_map; +using WordSet = std::unordered_set; +using KnownIdentifierFunc = std::function; + +void obfuscateQueries( + std::string_view src, + WriteBuffer & result, + WordMap & obfuscate_map, + WordSet & used_nouns, + SipHash hash_func, + KnownIdentifierFunc known_identifier_func); + +} diff --git a/src/Parsers/tests/gtest_obfuscate_queries.cpp b/src/Parsers/tests/gtest_obfuscate_queries.cpp new file mode 100644 index 00000000000..ada15109d17 --- /dev/null +++ b/src/Parsers/tests/gtest_obfuscate_queries.cpp @@ -0,0 +1,92 @@ +#include +#include +#include + +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + +#include +#include + + +using namespace DB; + + +TEST(ObfuscateQueries, Test1) +{ + WordMap obfuscated_words_map; + WordSet used_nouns; + SipHash hash_func; + + std::string salt = "Hello, world"; + hash_func.update(salt); + + SharedContextHolder shared_context; + const ContextHolder & context_holder = getContext(); + + registerFunctions(); + registerAggregateFunctions(); + registerTableFunctions(); + registerStorages(); + + std::unordered_set additional_names; + + auto all_known_storage_names = StorageFactory::instance().getAllRegisteredNames(); + auto all_known_data_type_names = DataTypeFactory::instance().getAllRegisteredNames(); + + additional_names.insert(all_known_storage_names.begin(), all_known_storage_names.end()); + additional_names.insert(all_known_data_type_names.begin(), all_known_data_type_names.end()); + + KnownIdentifierFunc is_known_identifier = [&](std::string_view name) + { + std::string what(name); + + return FunctionFactory::instance().tryGet(what, context_holder.context) != nullptr + || AggregateFunctionFactory::instance().isAggregateFunctionName(what) + || TableFunctionFactory::instance().isTableFunctionName(what) + || additional_names.count(what); + }; + + WriteBufferFromOwnString out; + + obfuscateQueries( + R"( +SELECT + VisitID, + Goals.ID, Goals.EventTime, + WatchIDs, + EAction.ProductName, EAction.ProductPrice, EAction.ProductCurrency, EAction.ProductQuantity, EAction.EventTime, EAction.Type +FROM merge.visits_v2 +WHERE + StartDate >= '2020-09-17' AND StartDate <= '2020-09-25' + AND CounterID = 24226447 + AND intHash32(UserID) = 416638616 AND intHash64(UserID) = 13269091395366875299 + AND VisitID IN (5653048135597886819, 5556254872710352304, 5516214175671455313, 5476714937521999313, 5464051549483503043) + AND Sign = 1 +)", + out, obfuscated_words_map, used_nouns, hash_func, is_known_identifier); + + EXPECT_EQ(out.str(), R"( +SELECT + CorduroyID, + Steel.ID, Steel.AcornSidestream, + WealthBRANCH, + GOVERNMENT.SedimentName, GOVERNMENT.SedimentExhaustion, GOVERNMENT.SedimentFencing, GOVERNMENT.SedimentOpossum, GOVERNMENT.AcornSidestream, GOVERNMENT.Lute +FROM merge.luncheonette_pants +WHERE + GovernanceCreche >= '2021-04-16' AND GovernanceCreche <= '2021-04-24' + AND StarboardID = 26446940 + AND intHash32(MessyID) = 474525514 AND intHash64(MessyID) = 13916317227779800149 + AND CorduroyID IN (5223158832904664474, 5605365157729463108, 7543250143731591192, 8715842063486405567, 7837015536326316923) + AND Tea = 1 +)"); +} + From ed4ff51ed8328c087575d4c3c4bb703356052290 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 26 Sep 2020 03:38:59 +0300 Subject: [PATCH 091/321] Integrate into clickhouse-format; add test and comments --- programs/format/Format.cpp | 98 ++++++++++++++++--- src/Parsers/obfuscateQueries.cpp | 2 +- src/Parsers/obfuscateQueries.h | 21 ++++ .../01508_query_obfuscator.reference | 1 + .../0_stateless/01508_query_obfuscator.sh | 6 ++ 5 files changed, 113 insertions(+), 15 deletions(-) create mode 100644 tests/queries/0_stateless/01508_query_obfuscator.reference create mode 100755 tests/queries/0_stateless/01508_query_obfuscator.sh diff --git a/programs/format/Format.cpp b/programs/format/Format.cpp index daf2d671568..01f952bf95e 100644 --- a/programs/format/Format.cpp +++ b/programs/format/Format.cpp @@ -1,13 +1,29 @@ #include +#include +#include #include #include #include +#include #include #include #include +#include #include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + + #pragma GCC diagnostic ignored "-Wunused-function" #pragma GCC diagnostic ignored "-Wmissing-declarations" @@ -22,6 +38,8 @@ int mainEntryClickHouseFormat(int argc, char ** argv) ("oneline", "format in single line") ("quiet,q", "just check syntax, no output on success") ("multiquery,n", "allow multiple queries in the same file") + ("obfuscate", "obfuscate instead of formatting") + ("seed", po::value(), "seed (arbitrary string) that determines the result of obfuscation") ; boost::program_options::variables_map options; @@ -40,10 +58,17 @@ int mainEntryClickHouseFormat(int argc, char ** argv) bool oneline = options.count("oneline"); bool quiet = options.count("quiet"); bool multiple = options.count("multiquery"); + bool obfuscate = options.count("obfuscate"); - if (quiet && (hilite || oneline)) + if (quiet && (hilite || oneline || obfuscate)) { - std::cerr << "Options 'hilite' or 'oneline' have no sense in 'quiet' mode." << std::endl; + std::cerr << "Options 'hilite' or 'oneline' or 'obfuscate' have no sense in 'quiet' mode." << std::endl; + return 2; + } + + if (obfuscate && (hilite || oneline || quiet)) + { + std::cerr << "Options 'hilite' or 'oneline' or 'quiet' have no sense in 'obfuscate' mode." << std::endl; return 2; } @@ -51,21 +76,66 @@ int mainEntryClickHouseFormat(int argc, char ** argv) ReadBufferFromFileDescriptor in(STDIN_FILENO); readStringUntilEOF(query, in); - const char * pos = query.data(); - const char * end = pos + query.size(); - - ParserQuery parser(end); - do + if (obfuscate) { - ASTPtr res = parseQueryAndMovePosition(parser, pos, end, "query", multiple, 0, DBMS_DEFAULT_MAX_PARSER_DEPTH); - if (!quiet) + WordMap obfuscated_words_map; + WordSet used_nouns; + SipHash hash_func; + + if (options.count("seed")) { - formatAST(*res, std::cout, hilite, oneline); - if (multiple) - std::cout << "\n;\n"; - std::cout << std::endl; + std::string seed; + hash_func.update(options["seed"].as()); } - } while (multiple && pos != end); + + SharedContextHolder shared_context = Context::createShared(); + Context context = Context::createGlobal(shared_context.get()); + context.makeGlobalContext(); + + registerFunctions(); + registerAggregateFunctions(); + registerTableFunctions(); + registerStorages(); + + std::unordered_set additional_names; + + auto all_known_storage_names = StorageFactory::instance().getAllRegisteredNames(); + auto all_known_data_type_names = DataTypeFactory::instance().getAllRegisteredNames(); + + additional_names.insert(all_known_storage_names.begin(), all_known_storage_names.end()); + additional_names.insert(all_known_data_type_names.begin(), all_known_data_type_names.end()); + + KnownIdentifierFunc is_known_identifier = [&](std::string_view name) + { + std::string what(name); + + return FunctionFactory::instance().tryGet(what, context) != nullptr + || AggregateFunctionFactory::instance().isAggregateFunctionName(what) + || TableFunctionFactory::instance().isTableFunctionName(what) + || additional_names.count(what); + }; + + WriteBufferFromFileDescriptor out(STDOUT_FILENO); + obfuscateQueries(query, out, obfuscated_words_map, used_nouns, hash_func, is_known_identifier); + } + else + { + const char * pos = query.data(); + const char * end = pos + query.size(); + + ParserQuery parser(end); + do + { + ASTPtr res = parseQueryAndMovePosition(parser, pos, end, "query", multiple, 0, DBMS_DEFAULT_MAX_PARSER_DEPTH); + if (!quiet) + { + formatAST(*res, std::cout, hilite, oneline); + if (multiple) + std::cout << "\n;\n"; + std::cout << std::endl; + } + } while (multiple && pos != end); + } } catch (...) { diff --git a/src/Parsers/obfuscateQueries.cpp b/src/Parsers/obfuscateQueries.cpp index 66221005d77..9dcf3d6ada3 100644 --- a/src/Parsers/obfuscateQueries.cpp +++ b/src/Parsers/obfuscateQueries.cpp @@ -836,7 +836,7 @@ void obfuscateLiteral(std::string_view src, WriteBuffer & result, SipHash hash_f ++src_pos; } - else if (src_pos[0]) + else { /// Start of UTF-8 sequence. hash_func.update(src_pos[0]); diff --git a/src/Parsers/obfuscateQueries.h b/src/Parsers/obfuscateQueries.h index 214237e19cf..2e65ec427f6 100644 --- a/src/Parsers/obfuscateQueries.h +++ b/src/Parsers/obfuscateQueries.h @@ -18,6 +18,27 @@ using WordMap = std::unordered_map; using WordSet = std::unordered_set; using KnownIdentifierFunc = std::function; +/** Takes one or multiple queries and obfuscate them by replacing identifiers to pseudorandom words + * and replacing literals to random values, while preserving the structure of the queries and the general sense. + * + * Its intended use case is when the user wants to share their queries for testing and debugging + * but is afraid to disclose the details about their column names, domain area and values of constants. + * + * It can obfuscate multiple queries in consistent fashion - identical names will be transformed to identical results. + * + * The function is not guaranteed to always give correct result or to be secure. It's implemented in "best effort" fashion. + * + * @param src - a string with source queries. + * @param result - where the obfuscated queries will be written. + * @param obfuscate_map - information about substituted identifiers + * (pass empty map at the beginning or reuse it from previous invocation to get consistent result) + * @param used_nouns - information about words used for substitution + * (pass empty set at the beginning or reuse it from previous invocation to get consistent result) + * @param hash_func - hash function that will be used as a pseudorandom source, + * it's recommended to preseed the function before passing here. + * @param known_identifier_func - a function that returns true if identifier is known name + * (of function, aggregate function, etc. that should be kept as is). If it returns false, identifier will be obfuscated. + */ void obfuscateQueries( std::string_view src, WriteBuffer & result, diff --git a/tests/queries/0_stateless/01508_query_obfuscator.reference b/tests/queries/0_stateless/01508_query_obfuscator.reference new file mode 100644 index 00000000000..e2f61cf345d --- /dev/null +++ b/tests/queries/0_stateless/01508_query_obfuscator.reference @@ -0,0 +1 @@ +SELECT 116, 'Qqfu://2020-02-10isqkc1203 sp 2000-05-27T18:38:01', 13e100, Meantime_id_brisketHeavy, exclamation(Grit), avgIf(remote('128.0.0.1')) diff --git a/tests/queries/0_stateless/01508_query_obfuscator.sh b/tests/queries/0_stateless/01508_query_obfuscator.sh new file mode 100755 index 00000000000..797271edb30 --- /dev/null +++ b/tests/queries/0_stateless/01508_query_obfuscator.sh @@ -0,0 +1,6 @@ +#!/usr/bin/env bash + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +. "$CURDIR"/../shell_config.sh + +$CLICKHOUSE_FORMAT --seed Hello --obfuscate <<< "SELECT 123, 'Test://2020-01-01hello1234 at 2000-01-01T01:02:03', 12e100, Gibberish_id_testCool, hello(World), avgIf(remote('127.0.0.1'))" From 673f72cf8c147fa0252052e87ae0b2346d2e0e6f Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 26 Sep 2020 03:48:09 +0300 Subject: [PATCH 092/321] Remove some profanity --- src/Parsers/obfuscateQueries.cpp | 17 ++++++++--------- 1 file changed, 8 insertions(+), 9 deletions(-) diff --git a/src/Parsers/obfuscateQueries.cpp b/src/Parsers/obfuscateQueries.cpp index 9dcf3d6ada3..e8ea68ba0c6 100644 --- a/src/Parsers/obfuscateQueries.cpp +++ b/src/Parsers/obfuscateQueries.cpp @@ -76,7 +76,7 @@ std::initializer_list nouns "architect", "architecture", "archives", "area", "arena", "argument", "arithmetic", "ark", "arm", "armadillo", "armament", "armchair", "armoire", "armor", "armour", "armpit", "armrest", "army", "arrangement", "array", "arrest", "arrival", "arrogance", "arrow", "art", "artery", "arthur", "artichoke", "article", "artifact", "artificer", "artist", "ascend", "ascent", "ascot", "ash", "ashram", "ashtray", -"aside", "asparagus", "aspect", "asphalt", "aspic", "ass", "assassination", "assault", "assembly", "assertion", "assessment", "asset", +"aside", "asparagus", "aspect", "asphalt", "aspic", "assassination", "assault", "assembly", "assertion", "assessment", "asset", "assignment", "assist", "assistance", "assistant", "associate", "association", "assumption", "assurance", "asterisk", "astrakhan", "astrolabe", "astrologer", "astrology", "astronomy", "asymmetry", "atelier", "atheist", "athlete", "athletics", "atmosphere", "atom", "atrium", "attachment", "attack", "attacker", "attainment", "attempt", "attendance", "attendant", "attention", "attenuation", "attic", "attitude", "attorney", @@ -182,7 +182,7 @@ std::initializer_list nouns "current", "curriculum", "curry", "curse", "cursor", "curtailment", "curtain", "curve", "cushion", "custard", "custody", "custom", "customer", "cut", "cuticle", "cutlet", "cutover", "cutting", "cyclamen", "cycle", "cyclone", "cyclooxygenase", "cygnet", "cylinder", "cymbal", "cynic", "cyst", "cytokine", "cytoplasm", "dad", "daddy", "daffodil", "dagger", "dahlia", "daikon", "daily", "dairy", "daisy", "dam", "damage", -"dame", "damn", "dance", "dancer", "dancing", "dandelion", "danger", "dare", "dark", "darkness", "darn", "dart", "dash", "dashboard", +"dame", "dance", "dancer", "dancing", "dandelion", "danger", "dare", "dark", "darkness", "darn", "dart", "dash", "dashboard", "data", "database", "date", "daughter", "dawn", "day", "daybed", "daylight", "dead", "deadline", "deal", "dealer", "dealing", "dearest", "death", "deathwatch", "debate", "debris", "debt", "debtor", "decade", "decadence", "decency", "decimal", "decision", "decision-making", "deck", "declaration", "declination", "decline", "decoder", "decongestant", "decoration", "decrease", "decryption", "dedication", "deduce", @@ -193,8 +193,7 @@ std::initializer_list nouns "description", "desert", "design", "designation", "designer", "desire", "desk", "desktop", "dessert", "destination", "destiny", "destroyer", "destruction", "detail", "detainee", "detainment", "detection", "detective", "detector", "detention", "determination", "detour", "devastation", "developer", "developing", "development", "developmental", "deviance", "deviation", "device", "devil", "dew", "dhow", "diabetes", "diadem", -"diagnosis", "diagram", "dial", "dialect", "dialogue", "diam", "diamond", "diaper", "diaphragm", "diarist", "diary", "dibble", "dick", -"dickey", "dictaphone", "dictator", "diction", "dictionary", "die", "diesel", "diet", "difference", "differential", "difficulty", "diffuse", +"diagnosis", "diagram", "dial", "dialect", "dialogue", "diam", "diamond", "diaper", "diaphragm", "diarist", "diary", "dibble", "dickey", "dictaphone", "dictator", "diction", "dictionary", "die", "diesel", "diet", "difference", "differential", "difficulty", "diffuse", "dig", "digestion", "digestive", "digger", "digging", "digit", "dignity", "dilapidation", "dill", "dilution", "dime", "dimension", "dimple", "diner", "dinghy", "dining", "dinner", "dinosaur", "dioxide", "dip", "diploma", "diplomacy", "dipstick", "direction", "directive", "director", "directory", "dirndl", "dirt", "disability", "disadvantage", "disagreement", "disappointment", "disarmament", "disaster", "discharge", @@ -254,7 +253,7 @@ std::initializer_list nouns "fortress", "fortune", "forum", "foundation", "founder", "founding", "fountain", "fourths", "fowl", "fox", "foxglove", "fraction", "fragrance", "frame", "framework", "fratricide", "fraud", "fraudster", "freak", "freckle", "freedom", "freelance", "freezer", "freezing", "freight", "freighter", "frenzy", "freon", "frequency", "fresco", "friction", "fridge", "friend", "friendship", "fries", "frigate", "fright", "fringe", -"fritter", "frock", "frog", "front", "frontier", "frost", "frosting", "frown", "fruit", "frustration", "fry", "fuck", "fuel", "fugato", +"fritter", "frock", "frog", "front", "frontier", "frost", "frosting", "frown", "fruit", "frustration", "fry", "fuel", "fugato", "fulfillment", "full", "fun", "function", "functionality", "fund", "funding", "fundraising", "funeral", "fur", "furnace", "furniture", "furry", "fusarium", "futon", "future", "gadget", "gaffe", "gaffer", "gain", "gaiters", "gale", "gallery", "galley", "gallon", "galoshes", "gambling", "game", "gamebird", "gaming", "gander", "gang", "gap", "garage", "garb", "garbage", "garden", @@ -413,7 +412,7 @@ std::initializer_list nouns "picture", "picturesque", "pie", "piece", "pier", "piety", "pig", "pigeon", "piglet", "pigpen", "pigsty", "pike", "pilaf", "pile", "pilgrim", "pilgrimage", "pill", "pillar", "pillbox", "pillow", "pilot", "pimp", "pimple", "pin", "pinafore", "pine", "pineapple", "pinecone", "ping", "pink", "pinkie", "pinot", "pinstripe", "pint", "pinto", "pinworm", "pioneer", "pipe", "pipeline", "piracy", "pirate", -"piss", "pistol", "pit", "pita", "pitch", "pitcher", "pitching", "pith", "pizza", "place", "placebo", "placement", "placode", "plagiarism", +"pistol", "pit", "pita", "pitch", "pitcher", "pitching", "pith", "pizza", "place", "placebo", "placement", "placode", "plagiarism", "plain", "plaintiff", "plan", "plane", "planet", "planning", "plant", "plantation", "planter", "planula", "plaster", "plasterboard", "plastic", "plate", "platelet", "platform", "platinum", "platter", "platypus", "play", "player", "playground", "playroom", "playwright", "plea", "pleasure", "pleat", "pledge", "plenty", "plier", "pliers", "plight", "plot", "plough", "plover", "plow", "plowman", "plug", @@ -466,7 +465,7 @@ std::initializer_list nouns "riot", "rip", "ripple", "rise", "riser", "risk", "rite", "ritual", "river", "riverbed", "rivulet", "road", "roadway", "roar", "roast", "robe", "robin", "robot", "robotics", "rock", "rocker", "rocket", "rod", "role", "roll", "roller", "romaine", "romance", "roof", "room", "roommate", "rooster", "root", "rope", "rose", "rosemary", "roster", "rostrum", "rotation", "round", "roundabout", "route", -"router", "routine", "row", "rowboat", "rowing", "rubber", "rubbish", "rubric", "ruby", "ruckus", "rudiment", "ruffle", "rug", "rugby", +"router", "routine", "row", "rowboat", "rowing", "rubber", "rubric", "ruby", "ruckus", "rudiment", "ruffle", "rug", "rugby", "ruin", "rule", "ruler", "ruling", "rum", "rumor", "run", "runaway", "runner", "running", "runway", "rush", "rust", "rutabaga", "rye", "sabre", "sac", "sack", "saddle", "sadness", "safari", "safe", "safeguard", "safety", "saffron", "sage", "sail", "sailboat", "sailing", "sailor", "saint", "sake", "salad", "salami", "salary", "sale", "salesman", "salmon", "salon", "saloon", "salsa", "salt", "salute", "samovar", @@ -486,7 +485,7 @@ std::initializer_list nouns "setback", "setting", "settlement", "settler", "severity", "sewer", "sex", "sexuality", "shack", "shackle", "shade", "shadow", "shadowbox", "shakedown", "shaker", "shallot", "shallows", "shame", "shampoo", "shanty", "shape", "share", "shareholder", "shark", "shaw", "shawl", "shear", "shearling", "sheath", "shed", "sheep", "sheet", "shelf", "shell", "shelter", "sherbet", "sherry", "shield", "shift", "shin", -"shine", "shingle", "ship", "shipper", "shipping", "shipyard", "shirt", "shirtdress", "shit", "shoat", "shock", "shoe", "shoe-horn", +"shine", "shingle", "ship", "shipper", "shipping", "shipyard", "shirt", "shirtdress", "shoat", "shock", "shoe", "shoe-horn", "shoehorn", "shoelace", "shoemaker", "shoes", "shoestring", "shofar", "shoot", "shootdown", "shop", "shopper", "shopping", "shore", "shoreline", "short", "shortage", "shorts", "shortwave", "shot", "shoulder", "shout", "shovel", "show", "shower", "shred", "shrimp", "shrine", "shutdown", "sibling", "sick", "sickness", "side", "sideboard", "sideburns", "sidecar", "sidestream", "sidewalk", "siding", @@ -494,7 +493,7 @@ std::initializer_list nouns "silicon", "silk", "silkworm", "sill", "silly", "silo", "silver", "similarity", "simple", "simplicity", "simplification", "simvastatin", "sin", "singer", "singing", "singular", "sink", "sinuosity", "sip", "sir", "sister", "sitar", "site", "situation", "size", "skate", "skating", "skean", "skeleton", "ski", "skiing", "skill", "skin", "skirt", "skull", "skullcap", "skullduggery", "skunk", "sky", -"skylight", "skyline", "skyscraper", "skywalk", "slang", "slapstick", "slash", "slate", "slave", "slavery", "slaw", "sled", "sledge", +"skylight", "skyline", "skyscraper", "skywalk", "slang", "slapstick", "slash", "slate", "slavery", "slaw", "sled", "sledge", "sleep", "sleepiness", "sleeping", "sleet", "sleuth", "slice", "slide", "slider", "slime", "slip", "slipper", "slippers", "slope", "slot", "sloth", "slump", "smell", "smelting", "smile", "smith", "smock", "smog", "smoke", "smoking", "smolt", "smuggling", "snack", "snail", "snake", "snakebite", "snap", "snarl", "sneaker", "sneakers", "sneeze", "sniffle", "snob", "snorer", "snow", "snowboarding", "snowflake", From 22861b0c3dcd864584954b79436000a38306d130 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 26 Sep 2020 03:49:30 +0300 Subject: [PATCH 093/321] Remove some profanity --- src/Parsers/obfuscateQueries.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Parsers/obfuscateQueries.cpp b/src/Parsers/obfuscateQueries.cpp index e8ea68ba0c6..03f0cef2605 100644 --- a/src/Parsers/obfuscateQueries.cpp +++ b/src/Parsers/obfuscateQueries.cpp @@ -171,7 +171,7 @@ std::initializer_list nouns "councilperson", "counsel", "counseling", "counselling", "counsellor", "counselor", "count", "counter", "counterpart", "counterterrorism", "countess", "country", "countryside", "county", "couple", "coupon", "courage", "course", "court", "courthouse", "courtroom", "cousin", "covariate", "cover", "coverage", "coverall", "cow", "cowbell", "cowboy", "coyote", "crab", "crack", "cracker", "crackers", -"cradle", "craft", "craftsman", "cranberry", "crane", "cranky", "crap", "crash", "crate", "cravat", "craw", "crawdad", "crayfish", "crayon", +"cradle", "craft", "craftsman", "cranberry", "crane", "cranky", "crash", "crate", "cravat", "craw", "crawdad", "crayfish", "crayon", "crazy", "cream", "creation", "creationism", "creationist", "creative", "creativity", "creator", "creature", "creche", "credential", "credenza", "credibility", "credit", "creditor", "creek", "creme brulee", "crepe", "crest", "crew", "crewman", "crewmate", "crewmember", "crewmen", "cria", "crib", "cribbage", "cricket", "cricketer", "crime", "criminal", "crinoline", "crisis", "crisp", "criteria", "criterion", From d78fc63deb0d8a37e7a10e045b5807e9dd154ca7 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 26 Sep 2020 03:50:00 +0300 Subject: [PATCH 094/321] Remove duplicate word --- src/Parsers/obfuscateQueries.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Parsers/obfuscateQueries.cpp b/src/Parsers/obfuscateQueries.cpp index 03f0cef2605..17e751f5149 100644 --- a/src/Parsers/obfuscateQueries.cpp +++ b/src/Parsers/obfuscateQueries.cpp @@ -450,7 +450,7 @@ std::initializer_list nouns "recipient", "reciprocity", "reclamation", "recliner", "recognition", "recollection", "recommendation", "reconsideration", "record", "recorder", "recording", "recovery", "recreation", "recruit", "rectangle", "red", "redesign", "redhead", "redirect", "rediscovery", "reduction", "reef", "refectory", "reference", "referendum", "reflection", "reform", "refreshments", "refrigerator", "refuge", "refund", "refusal", -"refuse", "regard", "regime", "region", "regionalism", "register", "registration", "registry", "regret", "regulation", "regulator", "rehospitalisation", +"refuse", "regard", "regime", "region", "regionalism", "register", "registration", "registry", "regret", "regulation", "regulator", "rehospitalization", "reindeer", "reinscription", "reject", "relation", "relationship", "relative", "relaxation", "relay", "release", "reliability", "relief", "religion", "relish", "reluctance", "remains", "remark", "reminder", "remnant", "remote", "removal", "renaissance", "rent", "reorganisation", "reorganization", "repair", "reparation", "repayment", "repeat", "replacement", "replica", "replication", "reply", From fc10803deffd4585bb516753552a8c1eb387f246 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 26 Sep 2020 03:50:56 +0300 Subject: [PATCH 095/321] Fix comment --- src/Parsers/obfuscateQueries.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Parsers/obfuscateQueries.h b/src/Parsers/obfuscateQueries.h index 2e65ec427f6..0a192649a92 100644 --- a/src/Parsers/obfuscateQueries.h +++ b/src/Parsers/obfuscateQueries.h @@ -18,7 +18,7 @@ using WordMap = std::unordered_map; using WordSet = std::unordered_set; using KnownIdentifierFunc = std::function; -/** Takes one or multiple queries and obfuscate them by replacing identifiers to pseudorandom words +/** Takes one or multiple queries and obfuscates them by replacing identifiers to pseudorandom words * and replacing literals to random values, while preserving the structure of the queries and the general sense. * * Its intended use case is when the user wants to share their queries for testing and debugging From bda2c2f58fcc590bed044200f4a5e0c563691d54 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 26 Sep 2020 03:53:26 +0300 Subject: [PATCH 096/321] Remove sensitive words --- src/Parsers/obfuscateQueries.cpp | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/src/Parsers/obfuscateQueries.cpp b/src/Parsers/obfuscateQueries.cpp index 17e751f5149..0d62fe435d8 100644 --- a/src/Parsers/obfuscateQueries.cpp +++ b/src/Parsers/obfuscateQueries.cpp @@ -283,10 +283,10 @@ std::initializer_list nouns "hearsay", "heart", "heartache", "heartbeat", "hearth", "hearthside", "heartwood", "heat", "heater", "heating", "heaven", "heavy", "hectare", "hedge", "hedgehog", "heel", "heifer", "height", "heir", "heirloom", "helicopter", "helium", "hell", "hellcat", "hello", "helmet", "helo", "help", "hemisphere", "hemp", "hen", "hepatitis", "herb", "herbs", "heritage", "hermit", "hero", "heroine", "heron", -"herring", "hesitation", "heterosexual", "hexagon", "heyday", "hiccups", "hide", "hierarchy", "high", "highland", "highlight", +"herring", "hesitation", "hexagon", "heyday", "hiccups", "hide", "hierarchy", "high", "highland", "highlight", "highway", "hike", "hiking", "hill", "hint", "hip", "hippodrome", "hippopotamus", "hire", "hiring", "historian", "history", "hit", "hive", "hobbit", "hobby", "hockey", "hoe", "hog", "hold", "holder", "hole", "holiday", "home", "homeland", "homeownership", "hometown", "homework", -"homicide", "homogenate", "homonym", "homosexual", "homosexuality", "honesty", "honey", "honeybee", "honeydew", "honor", "honoree", "hood", +"homicide", "homogenate", "homonym", "honesty", "honey", "honeybee", "honeydew", "honor", "honoree", "hood", "hoof", "hook", "hop", "hope", "hops", "horde", "horizon", "hormone", "horn", "hornet", "horror", "horse", "horseradish", "horst", "hose", "hosiery", "hospice", "hospital", "hospitalisation", "hospitality", "hospitalization", "host", "hostel", "hostess", "hotdog", "hotel", "hound", "hour", "hourglass", "house", "houseboat", "household", "housewife", "housework", "housing", "hovel", "hovercraft", "howard", @@ -328,7 +328,7 @@ std::initializer_list nouns "laughter", "laundry", "lava", "law", "lawmaker", "lawn", "lawsuit", "lawyer", "lay", "layer", "layout", "lead", "leader", "leadership", "leading", "leaf", "league", "leaker", "leap", "learning", "leash", "leather", "leave", "leaver", "lecture", "leek", "leeway", "left", "leg", "legacy", "legal", "legend", "legging", "legislation", "legislator", "legislature", "legitimacy", "legume", "leisure", "lemon", -"lemonade", "lemur", "lender", "lending", "length", "lens", "lentil", "leopard", "leprosy", "leptocephalus", "lesbian", "lesson", "letter", +"lemonade", "lemur", "lender", "lending", "length", "lens", "lentil", "leopard", "leprosy", "leptocephalus", "lesson", "letter", "lettuce", "level", "lever", "leverage", "leveret", "liability", "liar", "liberty", "libido", "library", "licence", "license", "licensing", "licorice", "lid", "lie", "lieu", "lieutenant", "life", "lifestyle", "lifetime", "lift", "ligand", "light", "lighting", "lightning", "lightscreen", "ligula", "likelihood", "likeness", "lilac", "lily", "limb", "lime", "limestone", "limit", "limitation", "limo", "line", From f97d40584d0011644e0967a0ba8bd5c9cc595e93 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 26 Sep 2020 04:10:26 +0300 Subject: [PATCH 097/321] Fix "Arcadia" --- src/Parsers/ya.make | 1 + 1 file changed, 1 insertion(+) diff --git a/src/Parsers/ya.make b/src/Parsers/ya.make index 0a0c301b722..4ec97b8b55b 100644 --- a/src/Parsers/ya.make +++ b/src/Parsers/ya.make @@ -85,6 +85,7 @@ SRCS( MySQL/ASTDeclareReference.cpp MySQL/ASTDeclareSubPartition.cpp MySQL/ASTDeclareTableOptions.cpp + obfuscateQueries.cpp parseDatabaseAndTableName.cpp parseIdentifierOrStringLiteral.cpp parseIntervalKind.cpp From 4445cb649525488ba5dec9180a36f58bef547d8c Mon Sep 17 00:00:00 2001 From: Denis Zhuravlev Date: Fri, 25 Sep 2020 22:20:36 -0300 Subject: [PATCH 098/321] tests for partition pruning --- .../01508_partition_pruning.queries | 118 ++++++++++++ .../01508_partition_pruning.reference | 177 ++++++++++++++++++ .../0_stateless/01508_partition_pruning.sh | 37 ++++ 3 files changed, 332 insertions(+) create mode 100644 tests/queries/0_stateless/01508_partition_pruning.queries create mode 100644 tests/queries/0_stateless/01508_partition_pruning.reference create mode 100755 tests/queries/0_stateless/01508_partition_pruning.sh diff --git a/tests/queries/0_stateless/01508_partition_pruning.queries b/tests/queries/0_stateless/01508_partition_pruning.queries new file mode 100644 index 00000000000..05f46e8e496 --- /dev/null +++ b/tests/queries/0_stateless/01508_partition_pruning.queries @@ -0,0 +1,118 @@ +DROP TABLE IF EXISTS tMM +DROP TABLE IF EXISTS tDD +DROP TABLE IF EXISTS sDD +DROP TABLE IF EXISTS xMM +CREATE TABLE tMM(d DateTime,a Int64) ENGINE = MergeTree PARTITION BY toYYYYMM(d) ORDER BY tuple() +SYSTEM STOP MERGES tMM; +INSERT INTO tMM SELECT toDateTime('2020-08-16 00:00:00') + number*60, number FROM numbers(5000) +INSERT INTO tMM SELECT toDateTime('2020-08-16 00:00:00') + number*60, number FROM numbers(5000) +INSERT INTO tMM SELECT toDateTime('2020-09-01 00:00:00') + number*60, number FROM numbers(5000) +INSERT INTO tMM SELECT toDateTime('2020-09-01 00:00:00') + number*60, number FROM numbers(5000) +INSERT INTO tMM SELECT toDateTime('2020-10-01 00:00:00') + number*60, number FROM numbers(5000) +INSERT INTO tMM SELECT toDateTime('2020-10-15 00:00:00') + number*60, number FROM numbers(5000) + +CREATE TABLE tDD(d DateTime,a Int) ENGINE = MergeTree PARTITION BY toYYYYMMDD(d) ORDER BY tuple() +insert into tDD select toDateTime(toDate('2020-09-23')), number from numbers(10000) UNION ALL select toDateTime(toDateTime('2020-09-23 11:00:00')), number from numbers(10000) UNION ALL select toDateTime(toDate('2020-09-24')), number from numbers(10000) UNION ALL select toDateTime(toDate('2020-09-25')), number from numbers(10000) UNION ALL select toDateTime(toDate('2020-08-15')), number from numbers(10000); + +CREATE TABLE sDD(d UInt64,a Int) ENGINE = MergeTree PARTITION BY toYYYYMM(toDate(intDiv(d,1000))) ORDER BY tuple() +insert into sDD select (1597536000+number*60)*1000, number from numbers(5000); +insert into sDD select (1597536000+number*60)*1000, number from numbers(5000); +insert into sDD select (1598918400+number*60)*1000, number from numbers(5000); +insert into sDD select (1598918400+number*60)*1000, number from numbers(5000); +insert into sDD select (1601510400+number*60)*1000, number from numbers(5000); +insert into sDD select (1602720000+number*60)*1000, number from numbers(5000); + +CREATE TABLE xMM(d DateTime,a Int64, f Int64) ENGINE = MergeTree PARTITION BY (toYYYYMM(d), a) ORDER BY tuple() +SYSTEM STOP MERGES xMM; +INSERT INTO xMM SELECT toDateTime('2020-08-16 00:00:00') + number*60, 1, number FROM numbers(5000) +INSERT INTO xMM SELECT toDateTime('2020-08-16 00:00:00') + number*60, 2, number FROM numbers(5000) +INSERT INTO xMM SELECT toDateTime('2020-09-01 00:00:00') + number*60, 3, number FROM numbers(5000) +INSERT INTO xMM SELECT toDateTime('2020-09-01 00:00:00') + number*60, 2, number FROM numbers(5000) +INSERT INTO xMM SELECT toDateTime('2020-10-01 00:00:00') + number*60, 1, number FROM numbers(5000) +INSERT INTO xMM SELECT toDateTime('2020-10-15 00:00:00') + number*60, 1, number FROM numbers(5000) + + + +select uniqExact(_part), count() from tMM where toDate(d)=toDate('2020-09-15') +select uniqExact(_part), count() from tMM where toDate(d)=toDate('2020-09-01') +select uniqExact(_part), count() from tMM where toDate(d)=toDate('2020-10-15') +select uniqExact(_part), count() from tMM where toDate(d)='2020-09-15' +select uniqExact(_part), count() from tMM where toYYYYMM(d)=202009 +select uniqExact(_part), count() from tMM where toYYYYMMDD(d)=20200816 +select uniqExact(_part), count() from tMM where toYYYYMMDD(d)=20201015 +select uniqExact(_part), count() from tMM where toDate(d)='2020-10-15' +select uniqExact(_part), count() from tMM where d >= '2020-09-01 00:00:00' and d<'2020-10-15 00:00:00' +select uniqExact(_part), count() from tMM where d >= '2020-01-16 00:00:00' and d < toDateTime('2021-08-17 00:00:00') +select uniqExact(_part), count() from tMM where d >= '2020-09-16 00:00:00' and d < toDateTime('2020-10-01 00:00:00') +select uniqExact(_part), count() from tMM where d >= '2020-09-12 00:00:00' and d < '2020-10-16 00:00:00' +select uniqExact(_part), count() from tMM where toStartOfDay(d) >= '2020-09-12 00:00:00' +select uniqExact(_part), count() from tMM where toStartOfDay(d) = '2020-09-01 00:00:00' +select uniqExact(_part), count() from tMM where toStartOfDay(d) = '2020-10-01 00:00:00' +select uniqExact(_part), count() from tMM where toStartOfDay(d) >= '2020-09-15 00:00:00' and d < '2020-10-16 00:00:00' +select uniqExact(_part), count() from tMM where toYYYYMM(d) between 202009 and 202010 +select uniqExact(_part), count() from tMM where toYYYYMM(d) between 202009 and 202009 +select uniqExact(_part), count() from tMM where toYYYYMM(d) between 202009 and 202010 and toStartOfDay(d) = '2020-10-01 00:00:00' +select uniqExact(_part), count() from tMM where toYYYYMM(d) >= 202009 and toStartOfDay(d) < '2020-10-02 00:00:00' +select uniqExact(_part), count() from tMM where toYYYYMM(d) > 202009 and toStartOfDay(d) < '2020-10-02 00:00:00' +select uniqExact(_part), count() from tMM where toYYYYMM(d)+1 > 202009 and toStartOfDay(d) < '2020-10-02 00:00:00' +select uniqExact(_part), count() from tMM where toYYYYMM(d)+1 > 202010 and toStartOfDay(d) < '2020-10-02 00:00:00' +select uniqExact(_part), count() from tMM where toYYYYMM(d)+1 > 202010 +select uniqExact(_part), count() from tMM where toYYYYMM(d-1)+1 = 202010 +select uniqExact(_part), count() from tMM where toStartOfMonth(d) >= '2020-09-15' +select uniqExact(_part), count() from tMM where toStartOfMonth(d) >= '2020-09-01' +select uniqExact(_part), count() from tMM where toStartOfMonth(d) >= '2020-09-01' and toStartOfMonth(d) < '2020-10-01' + +SYSTEM START MERGES tMM +OPTIMIZE TABLE tMM FINAL + +select uniqExact(_part), count() from tMM where toYYYYMM(d-1)+1 = 202010 +select uniqExact(_part), count() from tMM where toYYYYMM(d)+1 > 202010 +select uniqExact(_part), count() from tMM where toYYYYMM(d) between 202009 and 202010 + +OPTIMIZE TABLE tDD FINAL + +select uniqExact(_part), count() from tDD where toDate(d)=toDate('2020-09-24') +select uniqExact(_part), count() FROM tDD WHERE toDate(d) = toDate('2020-09-24') +select uniqExact(_part), count() FROM tDD WHERE toDate(d) = '2020-09-24' +select uniqExact(_part), count() FROM tDD WHERE toDate(d) >= '2020-09-23' and toDate(d) <= '2020-09-26' +select uniqExact(_part), count() FROM tDD WHERE toYYYYMMDD(d) >= 20200923 and toDate(d) <= '2020-09-26' + + +select uniqExact(_part), count() from sDD where toYYYYMM(toDateTime(intDiv(d,1000))-1)+1 = 202010 +select uniqExact(_part), count() from sDD where toYYYYMM(toDateTime(intDiv(d,1000))-1) = 202010 +select uniqExact(_part), count() from sDD where toYYYYMM(toDateTime(intDiv(d,1000))-1) = 202110 +select uniqExact(_part), count() from sDD where toYYYYMM(toDateTime(intDiv(d,1000)))+1 > 202009 and toStartOfDay(toDateTime(intDiv(d,1000))) < '2020-10-02 00:00:00' +select uniqExact(_part), count() from sDD where toYYYYMM(toDateTime(intDiv(d,1000)))+1 > 202009 and toDateTime(intDiv(d,1000)) < '2020-10-01 00:00:00' +select uniqExact(_part), count() from sDD where d >= 1598918400000 +select uniqExact(_part), count() from sDD where d >= 1598918400000 and toYYYYMM(toDateTime(intDiv(d,1000))-1) < 202010 + + + +select uniqExact(_part), count() from xMM where toStartOfDay(d) >= '2020-10-01 00:00:00' +select uniqExact(_part), count() from xMM where d >= '2020-09-01 00:00:00' and d <= '2020-10-01 00:00:00' +select uniqExact(_part), count() from xMM where d >= '2020-09-01 00:00:00' and d < '2020-10-01 00:00:00' +select uniqExact(_part), count() from xMM where d >= '2020-09-01 00:00:00' and d <= '2020-10-01 00:00:00' and a=1 +select uniqExact(_part), count() from xMM where d >= '2020-09-01 00:00:00' and d <= '2020-10-01 00:00:00' and a<>3 +select uniqExact(_part), count() from xMM where d >= '2020-09-01 00:00:00' and d < '2020-10-01 00:00:00' and a<>3 +select uniqExact(_part), count() from xMM where d >= '2020-09-01 00:00:00' and d < '2020-11-01 00:00:00' and a = 1 +select uniqExact(_part), count() from xMM where a = 1 +select uniqExact(_part), count() from xMM where a = 66 +select uniqExact(_part), count() from xMM where a <> 66 +select uniqExact(_part), count() from xMM where a = 2 + +SYSTEM START MERGES xMM; +optimize table xMM final; + +select uniqExact(_part), count() from xMM where a = 1 +select uniqExact(_part), count() from xMM where toStartOfDay(d) >= '2020-10-01 00:00:00' +select uniqExact(_part), count() from xMM where a <> 66 +select uniqExact(_part), count() from xMM where d >= '2020-09-01 00:00:00' and d <= '2020-10-01 00:00:00' and a<>3 +select uniqExact(_part), count() from xMM where d >= '2020-09-01 00:00:00' and d < '2020-10-01 00:00:00' and a<>3 + +DROP TABLE tMM +DROP TABLE tDD +DROP TABLE sDD +DROP TABLE xMM + + + diff --git a/tests/queries/0_stateless/01508_partition_pruning.reference b/tests/queries/0_stateless/01508_partition_pruning.reference new file mode 100644 index 00000000000..d967c760e91 --- /dev/null +++ b/tests/queries/0_stateless/01508_partition_pruning.reference @@ -0,0 +1,177 @@ +select uniqExact(_part), count() from tMM where toDate(d)=toDate('2020-09-15') +Selected 0 parts by partition key, 0 parts by primary key, 0 marks by primary key, 0 marks to read from 0 ranges + +select uniqExact(_part), count() from tMM where toDate(d)=toDate('2020-09-01') +Selected 2 parts by partition key, 2 parts by primary key, 2 marks by primary key, 2 marks to read from 2 ranges + +select uniqExact(_part), count() from tMM where toDate(d)=toDate('2020-10-15') +Selected 1 parts by partition key, 1 parts by primary key, 1 marks by primary key, 1 marks to read from 1 ranges + +select uniqExact(_part), count() from tMM where toDate(d)='2020-09-15' +Selected 0 parts by partition key, 0 parts by primary key, 0 marks by primary key, 0 marks to read from 0 ranges + +select uniqExact(_part), count() from tMM where toYYYYMM(d)=202009 +Selected 2 parts by partition key, 2 parts by primary key, 2 marks by primary key, 2 marks to read from 2 ranges + +select uniqExact(_part), count() from tMM where toYYYYMMDD(d)=20200816 +Selected 2 parts by partition key, 2 parts by primary key, 2 marks by primary key, 2 marks to read from 2 ranges + +select uniqExact(_part), count() from tMM where toYYYYMMDD(d)=20201015 +Selected 1 parts by partition key, 1 parts by primary key, 1 marks by primary key, 1 marks to read from 1 ranges + +select uniqExact(_part), count() from tMM where toDate(d)='2020-10-15' +Selected 1 parts by partition key, 1 parts by primary key, 1 marks by primary key, 1 marks to read from 1 ranges + +select uniqExact(_part), count() from tMM where d >= '2020-09-01 00:00:00' and d<'2020-10-15 00:00:00' +Selected 3 parts by partition key, 3 parts by primary key, 3 marks by primary key, 3 marks to read from 3 ranges + +select uniqExact(_part), count() from tMM where d >= '2020-01-16 00:00:00' and d < toDateTime('2021-08-17 00:00:00') +Selected 6 parts by partition key, 6 parts by primary key, 6 marks by primary key, 6 marks to read from 6 ranges + +select uniqExact(_part), count() from tMM where d >= '2020-09-16 00:00:00' and d < toDateTime('2020-10-01 00:00:00') +Selected 0 parts by partition key, 0 parts by primary key, 0 marks by primary key, 0 marks to read from 0 ranges + +select uniqExact(_part), count() from tMM where d >= '2020-09-12 00:00:00' and d < '2020-10-16 00:00:00' +Selected 2 parts by partition key, 2 parts by primary key, 2 marks by primary key, 2 marks to read from 2 ranges + +select uniqExact(_part), count() from tMM where toStartOfDay(d) >= '2020-09-12 00:00:00' +Selected 2 parts by partition key, 2 parts by primary key, 2 marks by primary key, 2 marks to read from 2 ranges + +select uniqExact(_part), count() from tMM where toStartOfDay(d) = '2020-09-01 00:00:00' +Selected 2 parts by partition key, 2 parts by primary key, 2 marks by primary key, 2 marks to read from 2 ranges + +select uniqExact(_part), count() from tMM where toStartOfDay(d) = '2020-10-01 00:00:00' +Selected 1 parts by partition key, 1 parts by primary key, 1 marks by primary key, 1 marks to read from 1 ranges + +select uniqExact(_part), count() from tMM where toStartOfDay(d) >= '2020-09-15 00:00:00' and d < '2020-10-16 00:00:00' +Selected 2 parts by partition key, 2 parts by primary key, 2 marks by primary key, 2 marks to read from 2 ranges + +select uniqExact(_part), count() from tMM where toYYYYMM(d) between 202009 and 202010 +Selected 4 parts by partition key, 4 parts by primary key, 4 marks by primary key, 4 marks to read from 4 ranges + +select uniqExact(_part), count() from tMM where toYYYYMM(d) between 202009 and 202009 +Selected 2 parts by partition key, 2 parts by primary key, 2 marks by primary key, 2 marks to read from 2 ranges + +select uniqExact(_part), count() from tMM where toYYYYMM(d) between 202009 and 202010 and toStartOfDay(d) = '2020-10-01 00:00:00' +Selected 1 parts by partition key, 1 parts by primary key, 1 marks by primary key, 1 marks to read from 1 ranges + +select uniqExact(_part), count() from tMM where toYYYYMM(d) >= 202009 and toStartOfDay(d) < '2020-10-02 00:00:00' +Selected 3 parts by partition key, 3 parts by primary key, 3 marks by primary key, 3 marks to read from 3 ranges + +select uniqExact(_part), count() from tMM where toYYYYMM(d) > 202009 and toStartOfDay(d) < '2020-10-02 00:00:00' +Selected 1 parts by partition key, 1 parts by primary key, 1 marks by primary key, 1 marks to read from 1 ranges + +select uniqExact(_part), count() from tMM where toYYYYMM(d)+1 > 202009 and toStartOfDay(d) < '2020-10-02 00:00:00' +Selected 3 parts by partition key, 3 parts by primary key, 3 marks by primary key, 3 marks to read from 3 ranges + +select uniqExact(_part), count() from tMM where toYYYYMM(d)+1 > 202010 and toStartOfDay(d) < '2020-10-02 00:00:00' +Selected 1 parts by partition key, 1 parts by primary key, 1 marks by primary key, 1 marks to read from 1 ranges + +select uniqExact(_part), count() from tMM where toYYYYMM(d)+1 > 202010 +Selected 2 parts by partition key, 2 parts by primary key, 2 marks by primary key, 2 marks to read from 2 ranges + +select uniqExact(_part), count() from tMM where toYYYYMM(d-1)+1 = 202010 +Selected 3 parts by partition key, 3 parts by primary key, 3 marks by primary key, 3 marks to read from 3 ranges + +select uniqExact(_part), count() from tMM where toStartOfMonth(d) >= '2020-09-15' +Selected 2 parts by partition key, 2 parts by primary key, 2 marks by primary key, 2 marks to read from 2 ranges + +select uniqExact(_part), count() from tMM where toStartOfMonth(d) >= '2020-09-01' +Selected 4 parts by partition key, 4 parts by primary key, 4 marks by primary key, 4 marks to read from 4 ranges + +select uniqExact(_part), count() from tMM where toStartOfMonth(d) >= '2020-09-01' and toStartOfMonth(d) < '2020-10-01' +Selected 2 parts by partition key, 2 parts by primary key, 2 marks by primary key, 2 marks to read from 2 ranges + +select uniqExact(_part), count() from tMM where toYYYYMM(d-1)+1 = 202010 +Selected 2 parts by partition key, 2 parts by primary key, 2 marks by primary key, 2 marks to read from 2 ranges + +select uniqExact(_part), count() from tMM where toYYYYMM(d)+1 > 202010 +Selected 1 parts by partition key, 1 parts by primary key, 1 marks by primary key, 1 marks to read from 1 ranges + +select uniqExact(_part), count() from tMM where toYYYYMM(d) between 202009 and 202010 +Selected 2 parts by partition key, 2 parts by primary key, 2 marks by primary key, 2 marks to read from 2 ranges + +select uniqExact(_part), count() from tDD where toDate(d)=toDate('2020-09-24') +Selected 1 parts by partition key, 1 parts by primary key, 1 marks by primary key, 1 marks to read from 1 ranges + +select uniqExact(_part), count() FROM tDD WHERE toDate(d) = toDate('2020-09-24') +Selected 1 parts by partition key, 1 parts by primary key, 1 marks by primary key, 1 marks to read from 1 ranges + +select uniqExact(_part), count() FROM tDD WHERE toDate(d) = '2020-09-24' +Selected 1 parts by partition key, 1 parts by primary key, 1 marks by primary key, 1 marks to read from 1 ranges + +select uniqExact(_part), count() FROM tDD WHERE toDate(d) >= '2020-09-23' and toDate(d) <= '2020-09-26' +Selected 3 parts by partition key, 3 parts by primary key, 4 marks by primary key, 4 marks to read from 3 ranges + +select uniqExact(_part), count() FROM tDD WHERE toYYYYMMDD(d) >= 20200923 and toDate(d) <= '2020-09-26' +Selected 3 parts by partition key, 3 parts by primary key, 4 marks by primary key, 4 marks to read from 3 ranges + +select uniqExact(_part), count() from sDD where toYYYYMM(toDateTime(intDiv(d,1000))-1)+1 = 202010 +Selected 3 parts by partition key, 3 parts by primary key, 3 marks by primary key, 3 marks to read from 3 ranges + +select uniqExact(_part), count() from sDD where toYYYYMM(toDateTime(intDiv(d,1000))-1) = 202010 +Selected 2 parts by partition key, 2 parts by primary key, 2 marks by primary key, 2 marks to read from 2 ranges + +select uniqExact(_part), count() from sDD where toYYYYMM(toDateTime(intDiv(d,1000))-1) = 202110 +Selected 0 parts by partition key, 0 parts by primary key, 0 marks by primary key, 0 marks to read from 0 ranges + +select uniqExact(_part), count() from sDD where toYYYYMM(toDateTime(intDiv(d,1000)))+1 > 202009 and toStartOfDay(toDateTime(intDiv(d,1000))) < '2020-10-02 00:00:00' +Selected 3 parts by partition key, 3 parts by primary key, 3 marks by primary key, 3 marks to read from 3 ranges + +select uniqExact(_part), count() from sDD where toYYYYMM(toDateTime(intDiv(d,1000)))+1 > 202009 and toDateTime(intDiv(d,1000)) < '2020-10-01 00:00:00' +Selected 2 parts by partition key, 2 parts by primary key, 2 marks by primary key, 2 marks to read from 2 ranges + +select uniqExact(_part), count() from sDD where d >= 1598918400000 +Selected 4 parts by partition key, 4 parts by primary key, 4 marks by primary key, 4 marks to read from 4 ranges + +select uniqExact(_part), count() from sDD where d >= 1598918400000 and toYYYYMM(toDateTime(intDiv(d,1000))-1) < 202010 +Selected 3 parts by partition key, 3 parts by primary key, 3 marks by primary key, 3 marks to read from 3 ranges + +select uniqExact(_part), count() from xMM where toStartOfDay(d) >= '2020-10-01 00:00:00' +Selected 2 parts by partition key, 2 parts by primary key, 2 marks by primary key, 2 marks to read from 2 ranges + +select uniqExact(_part), count() from xMM where d >= '2020-09-01 00:00:00' and d <= '2020-10-01 00:00:00' +Selected 3 parts by partition key, 3 parts by primary key, 3 marks by primary key, 3 marks to read from 3 ranges + +select uniqExact(_part), count() from xMM where d >= '2020-09-01 00:00:00' and d < '2020-10-01 00:00:00' +Selected 2 parts by partition key, 2 parts by primary key, 2 marks by primary key, 2 marks to read from 2 ranges + +select uniqExact(_part), count() from xMM where d >= '2020-09-01 00:00:00' and d <= '2020-10-01 00:00:00' and a=1 +Selected 1 parts by partition key, 1 parts by primary key, 1 marks by primary key, 1 marks to read from 1 ranges + +select uniqExact(_part), count() from xMM where d >= '2020-09-01 00:00:00' and d <= '2020-10-01 00:00:00' and a<>3 +Selected 2 parts by partition key, 2 parts by primary key, 2 marks by primary key, 2 marks to read from 2 ranges + +select uniqExact(_part), count() from xMM where d >= '2020-09-01 00:00:00' and d < '2020-10-01 00:00:00' and a<>3 +Selected 1 parts by partition key, 1 parts by primary key, 1 marks by primary key, 1 marks to read from 1 ranges + +select uniqExact(_part), count() from xMM where d >= '2020-09-01 00:00:00' and d < '2020-11-01 00:00:00' and a = 1 +Selected 2 parts by partition key, 2 parts by primary key, 2 marks by primary key, 2 marks to read from 2 ranges + +select uniqExact(_part), count() from xMM where a = 1 +Selected 3 parts by partition key, 3 parts by primary key, 3 marks by primary key, 3 marks to read from 3 ranges + +select uniqExact(_part), count() from xMM where a = 66 +Selected 0 parts by partition key, 0 parts by primary key, 0 marks by primary key, 0 marks to read from 0 ranges + +select uniqExact(_part), count() from xMM where a <> 66 +Selected 6 parts by partition key, 6 parts by primary key, 6 marks by primary key, 6 marks to read from 6 ranges + +select uniqExact(_part), count() from xMM where a = 2 +Selected 2 parts by partition key, 2 parts by primary key, 2 marks by primary key, 2 marks to read from 2 ranges + +select uniqExact(_part), count() from xMM where a = 1 +Selected 2 parts by partition key, 2 parts by primary key, 2 marks by primary key, 2 marks to read from 2 ranges + +select uniqExact(_part), count() from xMM where toStartOfDay(d) >= '2020-10-01 00:00:00' +Selected 1 parts by partition key, 1 parts by primary key, 1 marks by primary key, 1 marks to read from 1 ranges + +select uniqExact(_part), count() from xMM where a <> 66 +Selected 5 parts by partition key, 5 parts by primary key, 5 marks by primary key, 5 marks to read from 5 ranges + +select uniqExact(_part), count() from xMM where d >= '2020-09-01 00:00:00' and d <= '2020-10-01 00:00:00' and a<>3 +Selected 2 parts by partition key, 2 parts by primary key, 2 marks by primary key, 2 marks to read from 2 ranges + +select uniqExact(_part), count() from xMM where d >= '2020-09-01 00:00:00' and d < '2020-10-01 00:00:00' and a<>3 +Selected 1 parts by partition key, 1 parts by primary key, 1 marks by primary key, 1 marks to read from 1 ranges + diff --git a/tests/queries/0_stateless/01508_partition_pruning.sh b/tests/queries/0_stateless/01508_partition_pruning.sh new file mode 100755 index 00000000000..3d5f58ca73d --- /dev/null +++ b/tests/queries/0_stateless/01508_partition_pruning.sh @@ -0,0 +1,37 @@ +#!/usr/bin/env bash + +#-------------------------------------------- +# Description of test result: +# Test the correctness of the optimization +# by asserting read marks in the log. +# Relation of read marks and optimization: +# read marks = +# the number of monotonic marks filtered through predicates +# + no monotonic marks count +#-------------------------------------------- + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +. "$CURDIR"/../shell_config.sh + +#export CLICKHOUSE_CLIENT="clickhouse-client --send_logs_level=none" +#export CLICKHOUSE_CLIENT_SERVER_LOGS_LEVEL=none +#export CURDIR=. + + +queries="${CURDIR}/01508_partition_pruning.queries" +while IFS= read -r sql +do + [ -z "$sql" ] && continue + if [[ "$sql" == select* ]] ; + then + CLICKHOUSE_CLIENT=$(echo ${CLICKHOUSE_CLIENT} | sed 's/'"--send_logs_level=${CLICKHOUSE_CLIENT_SERVER_LOGS_LEVEL}"'/--send_logs_level=debug/g') + echo "$sql" + ${CLICKHOUSE_CLIENT} --query "$sql" 2>&1 | grep -oh "Selected .* parts by partition key, *. parts by primary key, .* marks by primary key, .* marks to read from .* ranges.*$" + CLICKHOUSE_CLIENT=$(echo ${CLICKHOUSE_CLIENT} | sed 's/--send_logs_level=debug/'"--send_logs_level=${CLICKHOUSE_CLIENT_SERVER_LOGS_LEVEL}"'/g') + echo "" + else + ${CLICKHOUSE_CLIENT} --query "$sql" + fi +done < "$queries" + + From 6f1a144f199a347ea7edaab584ff964029712dc1 Mon Sep 17 00:00:00 2001 From: Denis Zhuravlev Date: Fri, 25 Sep 2020 22:23:29 -0300 Subject: [PATCH 099/321] tests for partition pruning --- tests/queries/0_stateless/01508_partition_pruning.sh | 8 ++------ 1 file changed, 2 insertions(+), 6 deletions(-) diff --git a/tests/queries/0_stateless/01508_partition_pruning.sh b/tests/queries/0_stateless/01508_partition_pruning.sh index 3d5f58ca73d..e06b907f19f 100755 --- a/tests/queries/0_stateless/01508_partition_pruning.sh +++ b/tests/queries/0_stateless/01508_partition_pruning.sh @@ -2,12 +2,8 @@ #-------------------------------------------- # Description of test result: -# Test the correctness of the optimization -# by asserting read marks in the log. -# Relation of read marks and optimization: -# read marks = -# the number of monotonic marks filtered through predicates -# + no monotonic marks count +# Test the correctness of the partition +# pruning #-------------------------------------------- CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) From 3a2c809173c526c93ea27c7796e1d8cbbdbe8586 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 26 Sep 2020 04:59:25 +0300 Subject: [PATCH 100/321] Add missing dependencies --- programs/format/CMakeLists.txt | 3 +++ 1 file changed, 3 insertions(+) diff --git a/programs/format/CMakeLists.txt b/programs/format/CMakeLists.txt index ab06708cd3a..49f17ef163f 100644 --- a/programs/format/CMakeLists.txt +++ b/programs/format/CMakeLists.txt @@ -5,6 +5,9 @@ set (CLICKHOUSE_FORMAT_LINK boost::program_options clickhouse_common_io clickhouse_parsers + clickhouse_functions + clickhouse_aggregate_functions + clickhouse_table_functions dbms ) From c51502bce47b7d6fbcc0ab696d82d115a43b584c Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 26 Sep 2020 05:06:14 +0300 Subject: [PATCH 101/321] Remove words with dashes --- src/Parsers/obfuscateQueries.cpp | 24 ++++++++++++------------ 1 file changed, 12 insertions(+), 12 deletions(-) diff --git a/src/Parsers/obfuscateQueries.cpp b/src/Parsers/obfuscateQueries.cpp index 0d62fe435d8..fa7c5b4d3e4 100644 --- a/src/Parsers/obfuscateQueries.cpp +++ b/src/Parsers/obfuscateQueries.cpp @@ -103,18 +103,18 @@ std::initializer_list nouns "blowhole", "blue", "blueberry", "blush", "boar", "board", "boat", "boatload", "boatyard", "bob", "bobcat", "body", "bog", "bolero", "bolt", "bomb", "bomber", "bombing", "bond", "bonding", "bondsman", "bone", "bonfire", "bongo", "bonnet", "bonsai", "bonus", "boogeyman", "book", "bookcase", "bookend", "booking", "booklet", "bookmark", "boolean", "boom", "boon", "boost", "booster", "boot", "bootee", "bootie", -"booty", "border", "bore", "borrower", "borrowing", "bosom", "boss", "botany", "bother", "bottle", "bottling", "bottom", "bottom-line", +"booty", "border", "bore", "borrower", "borrowing", "bosom", "boss", "botany", "bother", "bottle", "bottling", "bottom", "boudoir", "bough", "boulder", "boulevard", "boundary", "bouquet", "bourgeoisie", "bout", "boutique", "bow", "bower", "bowl", "bowler", "bowling", "bowtie", "box", "boxer", "boxspring", "boy", "boycott", "boyfriend", "boyhood", "boysenberry", "bra", "brace", "bracelet", "bracket", "brain", "brake", "bran", "branch", "brand", "brandy", "brass", "brassiere", "bratwurst", "bread", "breadcrumb", "breadfruit", "break", "breakdown", "breakfast", "breakpoint", "breakthrough", "breast", "breastplate", "breath", "breeze", "brewer", "bribery", "brick", "bricklaying", "bride", "bridge", "brief", "briefing", "briefly", "briefs", "brilliant", "brink", "brisket", "broad", "broadcast", "broccoli", -"brochure", "brocolli", "broiler", "broker", "bronchitis", "bronco", "bronze", "brooch", "brood", "brook", "broom", "brother", "brother-in-law", +"brochure", "brocolli", "broiler", "broker", "bronchitis", "bronco", "bronze", "brooch", "brood", "brook", "broom", "brother", "brow", "brown", "brownie", "browser", "browsing", "brunch", "brush", "brushfire", "brushing", "bubble", "buck", "bucket", "buckle", "buckwheat", "bud", "buddy", "budget", "buffalo", "buffer", "buffet", "bug", "buggy", "bugle", "builder", "building", "bulb", "bulk", "bull", "bulldozer", "bullet", "bump", "bumper", "bun", "bunch", "bungalow", "bunghole", "bunkhouse", "burden", "bureau", "burglar", "burial", "burlesque", "burn", "burning", "burrito", "burro", "burrow", "burst", "bus", "bush", "business", "businessman", -"bust", "bustle", "butane", "butcher", "butler", "butter", "butterfly", "button", "buy", "buyer", "buying", "buzz", "buzzard", "c-clamp", +"bust", "bustle", "butane", "butcher", "butler", "butter", "butterfly", "button", "buy", "buyer", "buying", "buzz", "buzzard", "cabana", "cabbage", "cabin", "cabinet", "cable", "caboose", "cacao", "cactus", "caddy", "cadet", "cafe", "caffeine", "caftan", "cage", "cake", "calcification", "calculation", "calculator", "calculus", "calendar", "calf", "caliber", "calibre", "calico", "call", "calm", "calorie", "camel", "cameo", "camera", "camp", "campaign", "campaigning", "campanile", "camper", "campus", "can", "canal", "cancer", @@ -184,7 +184,7 @@ std::initializer_list nouns "cyst", "cytokine", "cytoplasm", "dad", "daddy", "daffodil", "dagger", "dahlia", "daikon", "daily", "dairy", "daisy", "dam", "damage", "dame", "dance", "dancer", "dancing", "dandelion", "danger", "dare", "dark", "darkness", "darn", "dart", "dash", "dashboard", "data", "database", "date", "daughter", "dawn", "day", "daybed", "daylight", "dead", "deadline", "deal", "dealer", "dealing", "dearest", -"death", "deathwatch", "debate", "debris", "debt", "debtor", "decade", "decadence", "decency", "decimal", "decision", "decision-making", +"death", "deathwatch", "debate", "debris", "debt", "debtor", "decade", "decadence", "decency", "decimal", "decision", "deck", "declaration", "declination", "decline", "decoder", "decongestant", "decoration", "decrease", "decryption", "dedication", "deduce", "deduction", "deed", "deep", "deer", "default", "defeat", "defendant", "defender", "defense", "deficit", "definition", "deformation", "degradation", "degree", "delay", "deliberation", "delight", "delivery", "demand", "democracy", "democrat", "demon", "demur", "den", @@ -210,7 +210,7 @@ std::initializer_list nouns "dress", "dresser", "dressing", "drill", "drink", "drinking", "drive", "driver", "driveway", "driving", "drizzle", "dromedary", "drop", "drudgery", "drug", "drum", "drummer", "drunk", "dryer", "duck", "duckling", "dud", "dude", "due", "duel", "dueling", "duffel", "dugout", "dulcimer", "dumbwaiter", "dump", "dump truck", "dune", "dune buggy", "dungarees", "dungeon", "duplexer", "duration", "durian", "dusk", -"dust", "dust storm", "duster", "duty", "dwarf", "dwell", "dwelling", "dynamics", "dynamite", "dynamo", "dynasty", "dysfunction", "e-book", +"dust", "dust storm", "duster", "duty", "dwarf", "dwell", "dwelling", "dynamics", "dynamite", "dynamo", "dynasty", "dysfunction", "eagle", "eaglet", "ear", "eardrum", "earmuffs", "earnings", "earplug", "earring", "earrings", "earth", "earthquake", "earthworm", "ease", "easel", "east", "eating", "eaves", "eavesdropper", "ecclesia", "echidna", "eclipse", "ecliptic", "ecology", "economics", "economy", "ecosystem", "ectoderm", "ectodermal", "ecumenist", "eddy", "edge", "edger", "edible", "editing", "edition", "editor", "editorial", @@ -273,7 +273,7 @@ std::initializer_list nouns "grill", "grin", "grip", "gripper", "grit", "grocery", "ground", "group", "grouper", "grouse", "grove", "growth", "grub", "guacamole", "guarantee", "guard", "guava", "guerrilla", "guess", "guest", "guestbook", "guidance", "guide", "guideline", "guilder", "guilt", "guilty", "guinea", "guitar", "guitarist", "gum", "gumshoe", "gun", "gunpowder", "gutter", "guy", "gym", "gymnast", "gymnastics", "gynaecology", -"gyro", "habit", "habitat", "hacienda", "hacksaw", "hackwork", "hail", "hair", "haircut", "hake", "half", "half-sister", +"gyro", "habit", "habitat", "hacienda", "hacksaw", "hackwork", "hail", "hair", "haircut", "hake", "half", "halibut", "hall", "halloween", "hallway", "halt", "ham", "hamburger", "hammer", "hammock", "hamster", "hand", "handball", "handful", "handgun", "handicap", "handle", "handlebar", "handmaiden", "handover", "handrail", "handsaw", "hanger", "happening", "happiness", "harald", "harbor", "harbour", "hardboard", "hardcover", "hardening", "hardhat", "hardship", "hardware", "hare", "harm", @@ -293,7 +293,7 @@ std::initializer_list nouns "howitzer", "hub", "hubcap", "hubris", "hug", "hugger", "hull", "human", "humanity", "humidity", "hummus", "humor", "humour", "hunchback", "hundred", "hunger", "hunt", "hunter", "hunting", "hurdle", "hurdler", "hurricane", "hurry", "hurt", "husband", "hut", "hutch", "hyacinth", "hybridisation", "hybridization", "hydrant", "hydraulics", "hydrocarb", "hydrocarbon", "hydrofoil", "hydrogen", "hydrolyse", "hydrolysis", -"hydrolyze", "hydroxyl", "hyena", "hygienic", "hype", "hyphenation", "hypochondria", "hypothermia", "hypothesis", "ice", "ice-cream", +"hydrolyze", "hydroxyl", "hyena", "hygienic", "hype", "hyphenation", "hypochondria", "hypothermia", "hypothesis", "ice", "iceberg", "icebreaker", "icecream", "icicle", "icing", "icon", "icy", "id", "idea", "ideal", "identification", "identity", "ideology", "idiom", "idiot", "igloo", "ignorance", "ignorant", "ikebana", "illegal", "illiteracy", "illness", "illusion", "illustration", "image", "imagination", "imbalance", "imitation", "immigrant", "immigration", "immortal", "impact", "impairment", "impala", "impediment", "implement", @@ -320,7 +320,7 @@ std::initializer_list nouns "kamikaze", "kangaroo", "karate", "kayak", "kazoo", "kebab", "keep", "keeper", "kendo", "kennel", "ketch", "ketchup", "kettle", "kettledrum", "key", "keyboard", "keyboarding", "keystone", "kick", "kid", "kidney", "kielbasa", "kill", "killer", "killing", "kilogram", "kilometer", "kilt", "kimono", "kinase", "kind", "kindness", "king", "kingdom", "kingfish", "kiosk", "kiss", "kit", "kitchen", "kite", -"kitsch", "kitten", "kitty", "kiwi", "knee", "kneejerk", "knickers", "knife", "knight", "knitting", "knock", "knot", "know-how", +"kitsch", "kitten", "kitty", "kiwi", "knee", "kneejerk", "knickers", "knife", "knight", "knitting", "knock", "knot", "knowledge", "knuckle", "koala", "kohlrabi", "kumquat", "lab", "label", "labor", "laboratory", "laborer", "labour", "labourer", "lace", "lack", "lacquerware", "lad", "ladder", "ladle", "lady", "ladybug", "lag", "lake", "lamb", "lambkin", "lament", "lamp", "lanai", "land", "landform", "landing", "landmine", "landscape", "lane", "language", "lantern", "lap", "laparoscope", "lapdog", "laptop", "larch", "lard", @@ -401,7 +401,7 @@ std::initializer_list nouns "paste", "pastor", "pastoralist", "pastry", "pasture", "pat", "patch", "pate", "patent", "patentee", "path", "pathogenesis", "pathology", "pathway", "patience", "patient", "patina", "patio", "patriarch", "patrimony", "patriot", "patrol", "patroller", "patrolling", "patron", "pattern", "patty", "pattypan", "pause", "pavement", "pavilion", "paw", "pawnshop", "pay", "payee", "payment", "payoff", "pea", "peace", -"peach", "peacoat", "peacock", "peak", "peanut", "pear", "pearl", "peasant", "pecan", "pecker", "pedal", "peek", "peen", "peer", "peer-to-peer", +"peach", "peacoat", "peacock", "peak", "peanut", "pear", "pearl", "peasant", "pecan", "pecker", "pedal", "peek", "peen", "peer", "pegboard", "pelican", "pelt", "pen", "penalty", "pence", "pencil", "pendant", "pendulum", "penguin", "penicillin", "peninsula", "penis", "pennant", "penny", "pension", "pentagon", "peony", "people", "pepper", "pepperoni", "percent", "percentage", "perception", "perch", "perennial", "perfection", "performance", "perfume", "period", "periodical", "peripheral", "permafrost", "permission", "permit", "perp", @@ -485,7 +485,7 @@ std::initializer_list nouns "setback", "setting", "settlement", "settler", "severity", "sewer", "sex", "sexuality", "shack", "shackle", "shade", "shadow", "shadowbox", "shakedown", "shaker", "shallot", "shallows", "shame", "shampoo", "shanty", "shape", "share", "shareholder", "shark", "shaw", "shawl", "shear", "shearling", "sheath", "shed", "sheep", "sheet", "shelf", "shell", "shelter", "sherbet", "sherry", "shield", "shift", "shin", -"shine", "shingle", "ship", "shipper", "shipping", "shipyard", "shirt", "shirtdress", "shoat", "shock", "shoe", "shoe-horn", +"shine", "shingle", "ship", "shipper", "shipping", "shipyard", "shirt", "shirtdress", "shoat", "shock", "shoe", "shoehorn", "shoelace", "shoemaker", "shoes", "shoestring", "shofar", "shoot", "shootdown", "shop", "shopper", "shopping", "shore", "shoreline", "short", "shortage", "shorts", "shortwave", "shot", "shoulder", "shout", "shovel", "show", "shower", "shred", "shrimp", "shrine", "shutdown", "sibling", "sick", "sickness", "side", "sideboard", "sideburns", "sidecar", "sidestream", "sidewalk", "siding", @@ -513,7 +513,7 @@ std::initializer_list nouns "staff", "stag", "stage", "stain", "stair", "staircase", "stake", "stalk", "stall", "stallion", "stamen", "stamina", "stamp", "stance", "stand", "standard", "standardisation", "standardization", "standing", "standoff", "standpoint", "star", "starboard", "start", "starter", "state", "statement", "statin", "station", "statistic", "statistics", "statue", "status", "statute", "stay", "steak", -"stealth", "steam", "steamroller", "steel", "steeple", "stem", "stench", "stencil", "step", "step-daughter", +"stealth", "steam", "steamroller", "steel", "steeple", "stem", "stench", "stencil", "step", "stepdaughter", "stepmother", "stepson", "stereo", "stew", "steward", "stick", "sticker", "stiletto", "still", "stimulation", "stimulus", "sting", "stinger", "stitch", "stitcher", "stock", "stockings", "stole", "stomach", "stone", "stonework", "stool", @@ -533,7 +533,7 @@ std::initializer_list nouns "swivel", "sword", "swordfight", "swordfish", "sycamore", "symbol", "symmetry", "sympathy", "symptom", "syndicate", "syndrome", "synergy", "synod", "synonym", "synthesis", "syrup", "system", "tab", "tabby", "tabernacle", "table", "tablecloth", "tablet", "tabletop", "tachometer", "tackle", "taco", "tactics", "tactile", "tadpole", "tag", "tail", "tailbud", "tailor", "tailspin", "takeover", -"tale", "talent", "talk", "talking", "tamale", "tambour", "tambourine", "tan", "tandem", "tangerine", "tank", "tank-top", +"tale", "talent", "talk", "talking", "tamale", "tambour", "tambourine", "tan", "tandem", "tangerine", "tank", "tanker", "tankful", "tap", "tape", "tapioca", "target", "taro", "tarragon", "tart", "task", "tassel", "taste", "tatami", "tattler", "tattoo", "tavern", "tax", "taxi", "taxicab", "taxpayer", "tea", "teacher", "teaching", "team", "teammate", "teapot", "tear", "tech", "technician", "technique", "technologist", "technology", "tectonics", "teen", "teenager", "teepee", "telephone", "telescreen", "teletype", From f3349c8d138e882793e01f8708b7e0fadb7ad937 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 26 Sep 2020 05:13:20 +0300 Subject: [PATCH 102/321] Keywords are case-insensitive --- src/Parsers/obfuscateQueries.cpp | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/src/Parsers/obfuscateQueries.cpp b/src/Parsers/obfuscateQueries.cpp index fa7c5b4d3e4..3594bdef3eb 100644 --- a/src/Parsers/obfuscateQueries.cpp +++ b/src/Parsers/obfuscateQueries.cpp @@ -2,6 +2,7 @@ #include #include +#include #include #include #include @@ -884,7 +885,10 @@ void obfuscateQueries( if (token.type == TokenType::BareWord) { - if (keywords.count(whole_token) + std::string whole_token_uppercase(whole_token); + Poco::toUpperInPlace(whole_token_uppercase); + + if (keywords.count(whole_token_uppercase) || known_identifier_func(whole_token)) { /// Keep keywords as is. From 11ab250a712c686b6fee7ea14df60d2c57634bf3 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 26 Sep 2020 05:18:18 +0300 Subject: [PATCH 103/321] More test cases --- tests/queries/0_stateless/01508_query_obfuscator.reference | 3 ++- tests/queries/0_stateless/01508_query_obfuscator.sh | 2 ++ 2 files changed, 4 insertions(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/01508_query_obfuscator.reference b/tests/queries/0_stateless/01508_query_obfuscator.reference index e2f61cf345d..7066528c870 100644 --- a/tests/queries/0_stateless/01508_query_obfuscator.reference +++ b/tests/queries/0_stateless/01508_query_obfuscator.reference @@ -1 +1,2 @@ -SELECT 116, 'Qqfu://2020-02-10isqkc1203 sp 2000-05-27T18:38:01', 13e100, Meantime_id_brisketHeavy, exclamation(Grit), avgIf(remote('128.0.0.1')) +SELECT 116, 'Qqfu://2020-02-10isqkc1203 sp 2000-05-27T18:38:01', 13e100, Residue_id_breakfastDevice, park(Innervation), avgIf(remote('128.0.0.1')) +SELECT shell_dust_tintype between crumb and shoat, case when peach >= 116 then bombing else null end diff --git a/tests/queries/0_stateless/01508_query_obfuscator.sh b/tests/queries/0_stateless/01508_query_obfuscator.sh index 797271edb30..a5dd30b67ba 100755 --- a/tests/queries/0_stateless/01508_query_obfuscator.sh +++ b/tests/queries/0_stateless/01508_query_obfuscator.sh @@ -4,3 +4,5 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) . "$CURDIR"/../shell_config.sh $CLICKHOUSE_FORMAT --seed Hello --obfuscate <<< "SELECT 123, 'Test://2020-01-01hello1234 at 2000-01-01T01:02:03', 12e100, Gibberish_id_testCool, hello(World), avgIf(remote('127.0.0.1'))" +$CLICKHOUSE_FORMAT --seed Hello --obfuscate <<< "SELECT cost_first_screen between a and b, case when x >= 123 then y else null end" + From 5e146eda0403a781f78191f11b2a4f835d0b1307 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 26 Sep 2020 06:02:05 +0300 Subject: [PATCH 104/321] Fix error --- src/Common/BitHelpers.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Common/BitHelpers.h b/src/Common/BitHelpers.h index eac5fdac80e..e79daeba14e 100644 --- a/src/Common/BitHelpers.h +++ b/src/Common/BitHelpers.h @@ -66,7 +66,7 @@ inline size_t getLeadingZeroBits(T x) template inline uint32_t bitScanReverse(T x) { - return sizeof(T) * 8 - 1 - getLeadingZeroBitsUnsafe(x); + return (std::max(sizeof(T), sizeof(unsigned int))) * 8 - 1 - getLeadingZeroBitsUnsafe(x); } // Unsafe since __builtin_ctz()-family explicitly state that result is undefined on x == 0 From 6fade0f3638af03377e93258486479a3c68bf025 Mon Sep 17 00:00:00 2001 From: Denis Zhuravlev Date: Sat, 26 Sep 2020 00:47:54 -0300 Subject: [PATCH 105/321] stabilize merges --- tests/queries/0_stateless/01508_partition_pruning.queries | 3 +++ 1 file changed, 3 insertions(+) diff --git a/tests/queries/0_stateless/01508_partition_pruning.queries b/tests/queries/0_stateless/01508_partition_pruning.queries index 05f46e8e496..6c4698c482d 100644 --- a/tests/queries/0_stateless/01508_partition_pruning.queries +++ b/tests/queries/0_stateless/01508_partition_pruning.queries @@ -12,9 +12,11 @@ INSERT INTO tMM SELECT toDateTime('2020-10-01 00:00:00') + number*60, number FRO INSERT INTO tMM SELECT toDateTime('2020-10-15 00:00:00') + number*60, number FROM numbers(5000) CREATE TABLE tDD(d DateTime,a Int) ENGINE = MergeTree PARTITION BY toYYYYMMDD(d) ORDER BY tuple() +SYSTEM STOP MERGES tDD; insert into tDD select toDateTime(toDate('2020-09-23')), number from numbers(10000) UNION ALL select toDateTime(toDateTime('2020-09-23 11:00:00')), number from numbers(10000) UNION ALL select toDateTime(toDate('2020-09-24')), number from numbers(10000) UNION ALL select toDateTime(toDate('2020-09-25')), number from numbers(10000) UNION ALL select toDateTime(toDate('2020-08-15')), number from numbers(10000); CREATE TABLE sDD(d UInt64,a Int) ENGINE = MergeTree PARTITION BY toYYYYMM(toDate(intDiv(d,1000))) ORDER BY tuple() +SYSTEM STOP MERGES sDD; insert into sDD select (1597536000+number*60)*1000, number from numbers(5000); insert into sDD select (1597536000+number*60)*1000, number from numbers(5000); insert into sDD select (1598918400+number*60)*1000, number from numbers(5000); @@ -69,6 +71,7 @@ select uniqExact(_part), count() from tMM where toYYYYMM(d-1)+1 = 202010 select uniqExact(_part), count() from tMM where toYYYYMM(d)+1 > 202010 select uniqExact(_part), count() from tMM where toYYYYMM(d) between 202009 and 202010 +SYSTEM START MERGES tDD OPTIMIZE TABLE tDD FINAL select uniqExact(_part), count() from tDD where toDate(d)=toDate('2020-09-24') From 57b480e65d5713198e0bd81e8cf84564d4e7d1b0 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 26 Sep 2020 07:03:55 +0300 Subject: [PATCH 106/321] Fix PVS warning --- programs/client/Suggest.cpp | 2 +- src/Parsers/obfuscateQueries.cpp | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/programs/client/Suggest.cpp b/programs/client/Suggest.cpp index ac18a131c3a..e85e7a21261 100644 --- a/programs/client/Suggest.cpp +++ b/programs/client/Suggest.cpp @@ -80,7 +80,7 @@ Suggest::Suggest() "WITH", "TOTALS", "HAVING", "ORDER", "COLLATE", "LIMIT", "UNION", "AND", "OR", "ASC", "IN", "KILL", "QUERY", "SYNC", "ASYNC", "TEST", "BETWEEN", "TRUNCATE", "USER", "ROLE", "PROFILE", "QUOTA", "POLICY", "ROW", "GRANT", "REVOKE", "OPTION", "ADMIN", "EXCEPT", "REPLACE", - "IDENTIFIED", "HOST", "NAME", "READONLY", "WRITABLE", "PERMISSIVE", "FOR", "RESTRICTIVE", "FOR", "RANDOMIZED", + "IDENTIFIED", "HOST", "NAME", "READONLY", "WRITABLE", "PERMISSIVE", "FOR", "RESTRICTIVE", "RANDOMIZED", "INTERVAL", "LIMITS", "ONLY", "TRACKING", "IP", "REGEXP", "ILIKE"}; } diff --git a/src/Parsers/obfuscateQueries.cpp b/src/Parsers/obfuscateQueries.cpp index 3594bdef3eb..32382b70bd7 100644 --- a/src/Parsers/obfuscateQueries.cpp +++ b/src/Parsers/obfuscateQueries.cpp @@ -37,7 +37,7 @@ const std::unordered_set keywords "WITH", "TOTALS", "HAVING", "ORDER", "COLLATE", "LIMIT", "UNION", "AND", "OR", "ASC", "IN", "KILL", "QUERY", "SYNC", "ASYNC", "TEST", "BETWEEN", "TRUNCATE", "USER", "ROLE", "PROFILE", "QUOTA", "POLICY", "ROW", "GRANT", "REVOKE", "OPTION", "ADMIN", "EXCEPT", "REPLACE", - "IDENTIFIED", "HOST", "NAME", "READONLY", "WRITABLE", "PERMISSIVE", "FOR", "RESTRICTIVE", "FOR", "RANDOMIZED", + "IDENTIFIED", "HOST", "NAME", "READONLY", "WRITABLE", "PERMISSIVE", "FOR", "RESTRICTIVE", "RANDOMIZED", "INTERVAL", "LIMITS", "ONLY", "TRACKING", "IP", "REGEXP", "ILIKE", "DICTIONARY" }; From 6d5cd95d3f073442acb93afaab9d12514e75414b Mon Sep 17 00:00:00 2001 From: feng lv Date: Sat, 26 Sep 2020 14:48:59 +0800 Subject: [PATCH 107/321] fix --- src/Functions/now.cpp | 9 +++++++++ 1 file changed, 9 insertions(+) diff --git a/src/Functions/now.cpp b/src/Functions/now.cpp index f78db91cc58..3ec3ee736b3 100644 --- a/src/Functions/now.cpp +++ b/src/Functions/now.cpp @@ -105,6 +105,15 @@ public: FunctionBaseImplPtr build(const ColumnsWithTypeAndName & arguments, const DataTypePtr &) const override { + if (arguments.size() > 1) + { + throw Exception("Arguments size of function " + getName() + " should be 0 or 1", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); + } + if (arguments.size() == 1 && !isStringOrFixedString(arguments[0].type)) + { + throw Exception( + "Arguments of function " + getName() + " should be String or FixedString", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + } if (arguments.size() == 1) return std::make_unique( time(nullptr), std::make_shared(extractTimeZoneNameFromFunctionArguments(arguments, 0, 0))); From da8a93806921085fdd6fda50fdf2db84f53e74e4 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 26 Sep 2020 09:50:34 +0300 Subject: [PATCH 108/321] Replace unit test to functional test --- src/CMakeLists.txt | 2 +- src/Parsers/tests/gtest_obfuscate_queries.cpp | 92 ------------------- .../01508_query_obfuscator.reference | 14 +++ .../0_stateless/01508_query_obfuscator.sh | 14 +++ 4 files changed, 29 insertions(+), 93 deletions(-) delete mode 100644 src/Parsers/tests/gtest_obfuscate_queries.cpp diff --git a/src/CMakeLists.txt b/src/CMakeLists.txt index 40815228eac..0016c51b7f8 100644 --- a/src/CMakeLists.txt +++ b/src/CMakeLists.txt @@ -415,6 +415,6 @@ if (ENABLE_TESTS AND USE_GTEST) -Wno-gnu-zero-variadic-macro-arguments ) - target_link_libraries(unit_tests_dbms PRIVATE ${GTEST_BOTH_LIBRARIES} clickhouse_functions clickhouse_aggregate_functions clickhouse_table_functions clickhouse_parsers dbms clickhouse_common_zookeeper string_utils) + target_link_libraries(unit_tests_dbms PRIVATE ${GTEST_BOTH_LIBRARIES} clickhouse_functions clickhouse_aggregate_functions clickhouse_parsers dbms clickhouse_common_zookeeper string_utils) add_check(unit_tests_dbms) endif () diff --git a/src/Parsers/tests/gtest_obfuscate_queries.cpp b/src/Parsers/tests/gtest_obfuscate_queries.cpp deleted file mode 100644 index ada15109d17..00000000000 --- a/src/Parsers/tests/gtest_obfuscate_queries.cpp +++ /dev/null @@ -1,92 +0,0 @@ -#include -#include -#include - -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include - -#include -#include - - -using namespace DB; - - -TEST(ObfuscateQueries, Test1) -{ - WordMap obfuscated_words_map; - WordSet used_nouns; - SipHash hash_func; - - std::string salt = "Hello, world"; - hash_func.update(salt); - - SharedContextHolder shared_context; - const ContextHolder & context_holder = getContext(); - - registerFunctions(); - registerAggregateFunctions(); - registerTableFunctions(); - registerStorages(); - - std::unordered_set additional_names; - - auto all_known_storage_names = StorageFactory::instance().getAllRegisteredNames(); - auto all_known_data_type_names = DataTypeFactory::instance().getAllRegisteredNames(); - - additional_names.insert(all_known_storage_names.begin(), all_known_storage_names.end()); - additional_names.insert(all_known_data_type_names.begin(), all_known_data_type_names.end()); - - KnownIdentifierFunc is_known_identifier = [&](std::string_view name) - { - std::string what(name); - - return FunctionFactory::instance().tryGet(what, context_holder.context) != nullptr - || AggregateFunctionFactory::instance().isAggregateFunctionName(what) - || TableFunctionFactory::instance().isTableFunctionName(what) - || additional_names.count(what); - }; - - WriteBufferFromOwnString out; - - obfuscateQueries( - R"( -SELECT - VisitID, - Goals.ID, Goals.EventTime, - WatchIDs, - EAction.ProductName, EAction.ProductPrice, EAction.ProductCurrency, EAction.ProductQuantity, EAction.EventTime, EAction.Type -FROM merge.visits_v2 -WHERE - StartDate >= '2020-09-17' AND StartDate <= '2020-09-25' - AND CounterID = 24226447 - AND intHash32(UserID) = 416638616 AND intHash64(UserID) = 13269091395366875299 - AND VisitID IN (5653048135597886819, 5556254872710352304, 5516214175671455313, 5476714937521999313, 5464051549483503043) - AND Sign = 1 -)", - out, obfuscated_words_map, used_nouns, hash_func, is_known_identifier); - - EXPECT_EQ(out.str(), R"( -SELECT - CorduroyID, - Steel.ID, Steel.AcornSidestream, - WealthBRANCH, - GOVERNMENT.SedimentName, GOVERNMENT.SedimentExhaustion, GOVERNMENT.SedimentFencing, GOVERNMENT.SedimentOpossum, GOVERNMENT.AcornSidestream, GOVERNMENT.Lute -FROM merge.luncheonette_pants -WHERE - GovernanceCreche >= '2021-04-16' AND GovernanceCreche <= '2021-04-24' - AND StarboardID = 26446940 - AND intHash32(MessyID) = 474525514 AND intHash64(MessyID) = 13916317227779800149 - AND CorduroyID IN (5223158832904664474, 5605365157729463108, 7543250143731591192, 8715842063486405567, 7837015536326316923) - AND Tea = 1 -)"); -} - diff --git a/tests/queries/0_stateless/01508_query_obfuscator.reference b/tests/queries/0_stateless/01508_query_obfuscator.reference index 7066528c870..0064ac73a09 100644 --- a/tests/queries/0_stateless/01508_query_obfuscator.reference +++ b/tests/queries/0_stateless/01508_query_obfuscator.reference @@ -1,2 +1,16 @@ SELECT 116, 'Qqfu://2020-02-10isqkc1203 sp 2000-05-27T18:38:01', 13e100, Residue_id_breakfastDevice, park(Innervation), avgIf(remote('128.0.0.1')) SELECT shell_dust_tintype between crumb and shoat, case when peach >= 116 then bombing else null end + +SELECT + ChimeID, + Testimonial.ID, Testimonial.SipCauseway, + TankfulTRUMPET, + HUMIDITY.TermiteName, HUMIDITY.TermiteSculptural, HUMIDITY.TermiteGuilt, HUMIDITY.TermiteIntensity, HUMIDITY.SipCauseway, HUMIDITY.Coat +FROM merge.tinkle_efficiency +WHERE + FaithSeller >= '2020-10-13' AND FaithSeller <= '2020-10-21' + AND MandolinID = 30750384 + AND intHash32(GafferID) = 448362928 AND intHash64(GafferID) = 12572659331310383983 + AND ChimeID IN (8195672321757027078, 7079643623150622129, 5057006826979676478, 7886875230160484653, 7494974311229040743) + AND Stot = 1 + diff --git a/tests/queries/0_stateless/01508_query_obfuscator.sh b/tests/queries/0_stateless/01508_query_obfuscator.sh index a5dd30b67ba..d60e42489fa 100755 --- a/tests/queries/0_stateless/01508_query_obfuscator.sh +++ b/tests/queries/0_stateless/01508_query_obfuscator.sh @@ -6,3 +6,17 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) $CLICKHOUSE_FORMAT --seed Hello --obfuscate <<< "SELECT 123, 'Test://2020-01-01hello1234 at 2000-01-01T01:02:03', 12e100, Gibberish_id_testCool, hello(World), avgIf(remote('127.0.0.1'))" $CLICKHOUSE_FORMAT --seed Hello --obfuscate <<< "SELECT cost_first_screen between a and b, case when x >= 123 then y else null end" +$CLICKHOUSE_FORMAT --seed Hello --obfuscate <<< " +SELECT + VisitID, + Goals.ID, Goals.EventTime, + WatchIDs, + EAction.ProductName, EAction.ProductPrice, EAction.ProductCurrency, EAction.ProductQuantity, EAction.EventTime, EAction.Type +FROM merge.visits_v2 +WHERE + StartDate >= '2020-09-17' AND StartDate <= '2020-09-25' + AND CounterID = 24226447 + AND intHash32(UserID) = 416638616 AND intHash64(UserID) = 13269091395366875299 + AND VisitID IN (5653048135597886819, 5556254872710352304, 5516214175671455313, 5476714937521999313, 5464051549483503043) + AND Sign = 1 +" From 5f8d7fab1787b3e0f1f9808b2e565e84620f1b36 Mon Sep 17 00:00:00 2001 From: Anna Date: Sat, 26 Sep 2020 10:07:16 +0300 Subject: [PATCH 109/321] Upd desc and move to string-functions.md --- .../functions/string-functions.md | 71 ++++++++++++++++++ .../sql-reference/functions/url-functions.md | 73 ++++++++----------- 2 files changed, 102 insertions(+), 42 deletions(-) diff --git a/docs/en/sql-reference/functions/string-functions.md b/docs/en/sql-reference/functions/string-functions.md index 8ed3b073fa8..045b3cd1520 100644 --- a/docs/en/sql-reference/functions/string-functions.md +++ b/docs/en/sql-reference/functions/string-functions.md @@ -487,4 +487,75 @@ Returns the CRC64 checksum of a string, using CRC-64-ECMA polynomial. The result type is UInt64. +## normalizeQuery {#normalized-query} + +Replaces literals, sequences of literals and complex aliases with placeholders. + +**Syntax** +``` sql +normalizeQuery(x) +``` + +**Parameters** + +- `x` — Sequence of characters. [String](../../sql-reference/data-types/string.md). + +**Returned value(s)** + +- Sequence of characters with placeholders. + +Type: [String](../../sql-reference/data-types/string.md). + +**Example** + +Query: + +``` sql +SELECT normalizeQuery('[1, 2, 3, x]') AS query; +``` + +Result: + +``` text +┌─query────┐ +│ [?.., x] │ +└──────────┘ +``` + +## normalizedQueryHash {#normalized-query-hash} + +Returns identical 64bit hash values without the values of literals for similar queries. It helps to analyze query log. + +**Syntax** + +``` sql +normalizedQueryHash(x) +``` + +**Parameters** + +- `x` — Sequence of characters. [String](../../sql-reference/data-types/string.md). + +**Returned value** + +- Hash value. + +Type: [UInt64](../../sql-reference/data-types/int-uint.md#uint-ranges). + +**Example** + +Query: + +``` sql +SELECT normalizedQueryHash('SELECT 1 AS `xyz`') != normalizedQueryHash('SELECT 1 AS `abc`') AS res; +``` + +Result: + +``` text +┌─res─┐ +│ 1 │ +└─────┘ +``` + [Original article](https://clickhouse.tech/docs/en/query_language/functions/string_functions/) diff --git a/docs/en/sql-reference/functions/url-functions.md b/docs/en/sql-reference/functions/url-functions.md index a76912d9048..fe072531aa4 100644 --- a/docs/en/sql-reference/functions/url-functions.md +++ b/docs/en/sql-reference/functions/url-functions.md @@ -248,86 +248,75 @@ Removes the ‘name’ URL parameter, if present. This function works under the Replaces literals, sequences of literals and complex aliases with placeholders. -**Syntax** (without SELECT) - +**Syntax** ``` sql - +normalizeQuery(x) ``` - -Alias: ``. (Optional) - -More text (Optional). - -**Parameters** (Optional) - -- `x` — Description. [Type name](relative/path/to/type/dscr.md#type). -- `y` — Description. [Type name](relative/path/to/type/dscr.md#type). - -**Returned value(s)** - -- Returned values list. - -Type: [Type](relative/path/to/type/dscr.md#type). - -**Example** - +!!! note "Note" SELECT count(*) FROM table WHERE date = '2020-01-02' AND id IN (1, 2, 3) LIMIT 10, 10 should be replaced to SELECT count(*) FROM table WHERE date = ? AND id IN (?) LIMIT ? +**Parameters** + +- `x` — Sequence of characters. [String](../../sql-reference/data-types/string.md). + +**Returned value(s)** + +- Sequence of characters with placeholders. + +Type: [String](../../sql-reference/data-types/string.md). + +**Example** + Query: ``` sql +SELECT normalizeQuery('[1, 2, 3, x]') AS query; ``` Result: ``` text +┌─query────┐ +│ [?.., x] │ +└──────────┘ ``` ## normalizedQueryHash {#normalized-query-hash} -Returns identical 64bit hash values for similar queries. It helps to analyze query log. -calculate a hash of query structure without the values of literals. +Returns identical 64bit hash values without the values of literals for similar queries. It helps to analyze query log. -**Syntax** (without SELECT) +**Syntax** ``` sql - +normalizedQueryHash(x) ``` -Alias: ``. (Optional) +**Parameters** -More text (Optional). +- `x` — Sequence of characters. [String](../../sql-reference/data-types/string.md). -**Parameters** (Optional) +**Returned value** -- `x` — Description. [Type name](relative/path/to/type/dscr.md#type). -- `y` — Description. [Type name](relative/path/to/type/dscr.md#type). +- Hash value. -**Returned value(s)** - -- Returned values list. - -Type: [Type](relative/path/to/type/dscr.md#type). +Type: [UInt64](../../sql-reference/data-types/int-uint.md#uint-ranges). **Example** -The example must show usage and/or a use cases. The following text contains recommended parts of an example. - -Input table (Optional): - -``` text -``` - Query: ``` sql +SELECT normalizedQueryHash('SELECT 1 AS `xyz`') != normalizedQueryHash('SELECT 1 AS `abc`') AS res; ``` Result: ``` text +┌─res─┐ +│ 1 │ +└─────┘ ``` [Original article](https://clickhouse.tech/docs/en/query_language/functions/url_functions/) From d1bb6b655553d429dd47dbc2079f8de26a4a238c Mon Sep 17 00:00:00 2001 From: alesapin Date: Sat, 26 Sep 2020 11:47:26 +0300 Subject: [PATCH 110/321] Update test.py --- tests/integration/test_s3_with_proxy/test.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/integration/test_s3_with_proxy/test.py b/tests/integration/test_s3_with_proxy/test.py index 3d118266455..70a50ae0e15 100644 --- a/tests/integration/test_s3_with_proxy/test.py +++ b/tests/integration/test_s3_with_proxy/test.py @@ -40,7 +40,7 @@ def cluster(): def check_proxy_logs(cluster, proxy_instance, http_methods={"POST", "PUT", "GET", "DELETE"}): for i in range(10): logs = cluster.get_container_logs(proxy_instance) - # Check that all possible interactions with Minio are present + # Check with retry that all possible interactions with Minio are present for http_method in http_methods: if logs.find(http_method + " http://minio1") >= 0: return From 44846da2b18c5cadbe20368d723f49f8326a9528 Mon Sep 17 00:00:00 2001 From: Denis Zhuravlev Date: Sat, 26 Sep 2020 12:21:48 -0300 Subject: [PATCH 111/321] added test for queries results --- .../01508_partition_pruning.queries | 176 +++++++++--------- .../01508_partition_pruning.reference | 3 + ..._partition_pruning_check_results.reference | 60 ++++++ .../01508_partition_pruning_check_results.sql | 121 ++++++++++++ 4 files changed, 272 insertions(+), 88 deletions(-) create mode 100644 tests/queries/0_stateless/01508_partition_pruning_check_results.reference create mode 100644 tests/queries/0_stateless/01508_partition_pruning_check_results.sql diff --git a/tests/queries/0_stateless/01508_partition_pruning.queries b/tests/queries/0_stateless/01508_partition_pruning.queries index 6c4698c482d..4fbe97a0f90 100644 --- a/tests/queries/0_stateless/01508_partition_pruning.queries +++ b/tests/queries/0_stateless/01508_partition_pruning.queries @@ -1,21 +1,21 @@ -DROP TABLE IF EXISTS tMM -DROP TABLE IF EXISTS tDD -DROP TABLE IF EXISTS sDD -DROP TABLE IF EXISTS xMM -CREATE TABLE tMM(d DateTime,a Int64) ENGINE = MergeTree PARTITION BY toYYYYMM(d) ORDER BY tuple() +DROP TABLE IF EXISTS tMM; +DROP TABLE IF EXISTS tDD; +DROP TABLE IF EXISTS sDD; +DROP TABLE IF EXISTS xMM; +CREATE TABLE tMM(d DateTime,a Int64) ENGINE = MergeTree PARTITION BY toYYYYMM(d) ORDER BY tuple() SETTINGS index_granularity = 8192; SYSTEM STOP MERGES tMM; -INSERT INTO tMM SELECT toDateTime('2020-08-16 00:00:00') + number*60, number FROM numbers(5000) -INSERT INTO tMM SELECT toDateTime('2020-08-16 00:00:00') + number*60, number FROM numbers(5000) -INSERT INTO tMM SELECT toDateTime('2020-09-01 00:00:00') + number*60, number FROM numbers(5000) -INSERT INTO tMM SELECT toDateTime('2020-09-01 00:00:00') + number*60, number FROM numbers(5000) -INSERT INTO tMM SELECT toDateTime('2020-10-01 00:00:00') + number*60, number FROM numbers(5000) -INSERT INTO tMM SELECT toDateTime('2020-10-15 00:00:00') + number*60, number FROM numbers(5000) +INSERT INTO tMM SELECT toDateTime('2020-08-16 00:00:00') + number*60, number FROM numbers(5000); +INSERT INTO tMM SELECT toDateTime('2020-08-16 00:00:00') + number*60, number FROM numbers(5000); +INSERT INTO tMM SELECT toDateTime('2020-09-01 00:00:00') + number*60, number FROM numbers(5000); +INSERT INTO tMM SELECT toDateTime('2020-09-01 00:00:00') + number*60, number FROM numbers(5000); +INSERT INTO tMM SELECT toDateTime('2020-10-01 00:00:00') + number*60, number FROM numbers(5000); +INSERT INTO tMM SELECT toDateTime('2020-10-15 00:00:00') + number*60, number FROM numbers(5000); -CREATE TABLE tDD(d DateTime,a Int) ENGINE = MergeTree PARTITION BY toYYYYMMDD(d) ORDER BY tuple() +CREATE TABLE tDD(d DateTime,a Int) ENGINE = MergeTree PARTITION BY toYYYYMMDD(d) ORDER BY tuple() SETTINGS index_granularity = 8192; SYSTEM STOP MERGES tDD; insert into tDD select toDateTime(toDate('2020-09-23')), number from numbers(10000) UNION ALL select toDateTime(toDateTime('2020-09-23 11:00:00')), number from numbers(10000) UNION ALL select toDateTime(toDate('2020-09-24')), number from numbers(10000) UNION ALL select toDateTime(toDate('2020-09-25')), number from numbers(10000) UNION ALL select toDateTime(toDate('2020-08-15')), number from numbers(10000); -CREATE TABLE sDD(d UInt64,a Int) ENGINE = MergeTree PARTITION BY toYYYYMM(toDate(intDiv(d,1000))) ORDER BY tuple() +CREATE TABLE sDD(d UInt64,a Int) ENGINE = MergeTree PARTITION BY toYYYYMM(toDate(intDiv(d,1000))) ORDER BY tuple() SETTINGS index_granularity = 8192; SYSTEM STOP MERGES sDD; insert into sDD select (1597536000+number*60)*1000, number from numbers(5000); insert into sDD select (1597536000+number*60)*1000, number from numbers(5000); @@ -24,98 +24,98 @@ insert into sDD select (1598918400+number*60)*1000, number from numbers(5000); insert into sDD select (1601510400+number*60)*1000, number from numbers(5000); insert into sDD select (1602720000+number*60)*1000, number from numbers(5000); -CREATE TABLE xMM(d DateTime,a Int64, f Int64) ENGINE = MergeTree PARTITION BY (toYYYYMM(d), a) ORDER BY tuple() +CREATE TABLE xMM(d DateTime,a Int64, f Int64) ENGINE = MergeTree PARTITION BY (toYYYYMM(d), a) ORDER BY tuple() SETTINGS index_granularity = 8192; SYSTEM STOP MERGES xMM; -INSERT INTO xMM SELECT toDateTime('2020-08-16 00:00:00') + number*60, 1, number FROM numbers(5000) -INSERT INTO xMM SELECT toDateTime('2020-08-16 00:00:00') + number*60, 2, number FROM numbers(5000) -INSERT INTO xMM SELECT toDateTime('2020-09-01 00:00:00') + number*60, 3, number FROM numbers(5000) -INSERT INTO xMM SELECT toDateTime('2020-09-01 00:00:00') + number*60, 2, number FROM numbers(5000) -INSERT INTO xMM SELECT toDateTime('2020-10-01 00:00:00') + number*60, 1, number FROM numbers(5000) -INSERT INTO xMM SELECT toDateTime('2020-10-15 00:00:00') + number*60, 1, number FROM numbers(5000) +INSERT INTO xMM SELECT toDateTime('2020-08-16 00:00:00') + number*60, 1, number FROM numbers(5000); +INSERT INTO xMM SELECT toDateTime('2020-08-16 00:00:00') + number*60, 2, number FROM numbers(5000); +INSERT INTO xMM SELECT toDateTime('2020-09-01 00:00:00') + number*60, 3, number FROM numbers(5000); +INSERT INTO xMM SELECT toDateTime('2020-09-01 00:00:00') + number*60, 2, number FROM numbers(5000); +INSERT INTO xMM SELECT toDateTime('2020-10-01 00:00:00') + number*60, 1, number FROM numbers(5000); +INSERT INTO xMM SELECT toDateTime('2020-10-15 00:00:00') + number*60, 1, number FROM numbers(5000); -select uniqExact(_part), count() from tMM where toDate(d)=toDate('2020-09-15') -select uniqExact(_part), count() from tMM where toDate(d)=toDate('2020-09-01') -select uniqExact(_part), count() from tMM where toDate(d)=toDate('2020-10-15') -select uniqExact(_part), count() from tMM where toDate(d)='2020-09-15' -select uniqExact(_part), count() from tMM where toYYYYMM(d)=202009 -select uniqExact(_part), count() from tMM where toYYYYMMDD(d)=20200816 -select uniqExact(_part), count() from tMM where toYYYYMMDD(d)=20201015 -select uniqExact(_part), count() from tMM where toDate(d)='2020-10-15' -select uniqExact(_part), count() from tMM where d >= '2020-09-01 00:00:00' and d<'2020-10-15 00:00:00' -select uniqExact(_part), count() from tMM where d >= '2020-01-16 00:00:00' and d < toDateTime('2021-08-17 00:00:00') -select uniqExact(_part), count() from tMM where d >= '2020-09-16 00:00:00' and d < toDateTime('2020-10-01 00:00:00') -select uniqExact(_part), count() from tMM where d >= '2020-09-12 00:00:00' and d < '2020-10-16 00:00:00' -select uniqExact(_part), count() from tMM where toStartOfDay(d) >= '2020-09-12 00:00:00' -select uniqExact(_part), count() from tMM where toStartOfDay(d) = '2020-09-01 00:00:00' -select uniqExact(_part), count() from tMM where toStartOfDay(d) = '2020-10-01 00:00:00' -select uniqExact(_part), count() from tMM where toStartOfDay(d) >= '2020-09-15 00:00:00' and d < '2020-10-16 00:00:00' -select uniqExact(_part), count() from tMM where toYYYYMM(d) between 202009 and 202010 -select uniqExact(_part), count() from tMM where toYYYYMM(d) between 202009 and 202009 -select uniqExact(_part), count() from tMM where toYYYYMM(d) between 202009 and 202010 and toStartOfDay(d) = '2020-10-01 00:00:00' -select uniqExact(_part), count() from tMM where toYYYYMM(d) >= 202009 and toStartOfDay(d) < '2020-10-02 00:00:00' -select uniqExact(_part), count() from tMM where toYYYYMM(d) > 202009 and toStartOfDay(d) < '2020-10-02 00:00:00' -select uniqExact(_part), count() from tMM where toYYYYMM(d)+1 > 202009 and toStartOfDay(d) < '2020-10-02 00:00:00' -select uniqExact(_part), count() from tMM where toYYYYMM(d)+1 > 202010 and toStartOfDay(d) < '2020-10-02 00:00:00' -select uniqExact(_part), count() from tMM where toYYYYMM(d)+1 > 202010 -select uniqExact(_part), count() from tMM where toYYYYMM(d-1)+1 = 202010 -select uniqExact(_part), count() from tMM where toStartOfMonth(d) >= '2020-09-15' -select uniqExact(_part), count() from tMM where toStartOfMonth(d) >= '2020-09-01' -select uniqExact(_part), count() from tMM where toStartOfMonth(d) >= '2020-09-01' and toStartOfMonth(d) < '2020-10-01' +select uniqExact(_part), count() from tMM where toDate(d)=toDate('2020-09-15'); +select uniqExact(_part), count() from tMM where toDate(d)=toDate('2020-09-01'); +select uniqExact(_part), count() from tMM where toDate(d)=toDate('2020-10-15'); +select uniqExact(_part), count() from tMM where toDate(d)='2020-09-15'; +select uniqExact(_part), count() from tMM where toYYYYMM(d)=202009; +select uniqExact(_part), count() from tMM where toYYYYMMDD(d)=20200816; +select uniqExact(_part), count() from tMM where toYYYYMMDD(d)=20201015; +select uniqExact(_part), count() from tMM where toDate(d)='2020-10-15'; +select uniqExact(_part), count() from tMM where d >= '2020-09-01 00:00:00' and d<'2020-10-15 00:00:00'; +select uniqExact(_part), count() from tMM where d >= '2020-01-16 00:00:00' and d < toDateTime('2021-08-17 00:00:00'); +select uniqExact(_part), count() from tMM where d >= '2020-09-16 00:00:00' and d < toDateTime('2020-10-01 00:00:00'); +select uniqExact(_part), count() from tMM where d >= '2020-09-12 00:00:00' and d < '2020-10-16 00:00:00'; +select uniqExact(_part), count() from tMM where toStartOfDay(d) >= '2020-09-12 00:00:00'; +select uniqExact(_part), count() from tMM where toStartOfDay(d) = '2020-09-01 00:00:00'; +select uniqExact(_part), count() from tMM where toStartOfDay(d) = '2020-10-01 00:00:00'; +select uniqExact(_part), count() from tMM where toStartOfDay(d) >= '2020-09-15 00:00:00' and d < '2020-10-16 00:00:00'; +select uniqExact(_part), count() from tMM where toYYYYMM(d) between 202009 and 202010; +select uniqExact(_part), count() from tMM where toYYYYMM(d) between 202009 and 202009; +select uniqExact(_part), count() from tMM where toYYYYMM(d) between 202009 and 202010 and toStartOfDay(d) = '2020-10-01 00:00:00'; +select uniqExact(_part), count() from tMM where toYYYYMM(d) >= 202009 and toStartOfDay(d) < '2020-10-02 00:00:00'; +select uniqExact(_part), count() from tMM where toYYYYMM(d) > 202009 and toStartOfDay(d) < '2020-10-02 00:00:00'; +select uniqExact(_part), count() from tMM where toYYYYMM(d)+1 > 202009 and toStartOfDay(d) < '2020-10-02 00:00:00'; +select uniqExact(_part), count() from tMM where toYYYYMM(d)+1 > 202010 and toStartOfDay(d) < '2020-10-02 00:00:00'; +select uniqExact(_part), count() from tMM where toYYYYMM(d)+1 > 202010; +select uniqExact(_part), count() from tMM where toYYYYMM(d-1)+1 = 202010; +select uniqExact(_part), count() from tMM where toStartOfMonth(d) >= '2020-09-15'; +select uniqExact(_part), count() from tMM where toStartOfMonth(d) >= '2020-09-01'; +select uniqExact(_part), count() from tMM where toStartOfMonth(d) >= '2020-09-01' and toStartOfMonth(d) < '2020-10-01'; -SYSTEM START MERGES tMM -OPTIMIZE TABLE tMM FINAL +SYSTEM START MERGES tMM; +OPTIMIZE TABLE tMM FINAL; -select uniqExact(_part), count() from tMM where toYYYYMM(d-1)+1 = 202010 -select uniqExact(_part), count() from tMM where toYYYYMM(d)+1 > 202010 -select uniqExact(_part), count() from tMM where toYYYYMM(d) between 202009 and 202010 +select uniqExact(_part), count() from tMM where toYYYYMM(d-1)+1 = 202010; +select uniqExact(_part), count() from tMM where toYYYYMM(d)+1 > 202010; +select uniqExact(_part), count() from tMM where toYYYYMM(d) between 202009 and 202010; -SYSTEM START MERGES tDD -OPTIMIZE TABLE tDD FINAL +SYSTEM START MERGES tDD; +OPTIMIZE TABLE tDD FINAL; -select uniqExact(_part), count() from tDD where toDate(d)=toDate('2020-09-24') -select uniqExact(_part), count() FROM tDD WHERE toDate(d) = toDate('2020-09-24') -select uniqExact(_part), count() FROM tDD WHERE toDate(d) = '2020-09-24' -select uniqExact(_part), count() FROM tDD WHERE toDate(d) >= '2020-09-23' and toDate(d) <= '2020-09-26' -select uniqExact(_part), count() FROM tDD WHERE toYYYYMMDD(d) >= 20200923 and toDate(d) <= '2020-09-26' +select uniqExact(_part), count() from tDD where toDate(d)=toDate('2020-09-24'); +select uniqExact(_part), count() FROM tDD WHERE toDate(d) = toDate('2020-09-24'); +select uniqExact(_part), count() FROM tDD WHERE toDate(d) = '2020-09-24'; +select uniqExact(_part), count() FROM tDD WHERE toDate(d) >= '2020-09-23' and toDate(d) <= '2020-09-26'; +select uniqExact(_part), count() FROM tDD WHERE toYYYYMMDD(d) >= 20200923 and toDate(d) <= '2020-09-26'; -select uniqExact(_part), count() from sDD where toYYYYMM(toDateTime(intDiv(d,1000))-1)+1 = 202010 -select uniqExact(_part), count() from sDD where toYYYYMM(toDateTime(intDiv(d,1000))-1) = 202010 -select uniqExact(_part), count() from sDD where toYYYYMM(toDateTime(intDiv(d,1000))-1) = 202110 -select uniqExact(_part), count() from sDD where toYYYYMM(toDateTime(intDiv(d,1000)))+1 > 202009 and toStartOfDay(toDateTime(intDiv(d,1000))) < '2020-10-02 00:00:00' -select uniqExact(_part), count() from sDD where toYYYYMM(toDateTime(intDiv(d,1000)))+1 > 202009 and toDateTime(intDiv(d,1000)) < '2020-10-01 00:00:00' -select uniqExact(_part), count() from sDD where d >= 1598918400000 -select uniqExact(_part), count() from sDD where d >= 1598918400000 and toYYYYMM(toDateTime(intDiv(d,1000))-1) < 202010 +select uniqExact(_part), count() from sDD; +select uniqExact(_part), count() from sDD where toYYYYMM(toDateTime(intDiv(d,1000))-1)+1 = 202010; +select uniqExact(_part), count() from sDD where toYYYYMM(toDateTime(intDiv(d,1000))-1) = 202010; +select uniqExact(_part), count() from sDD where toYYYYMM(toDateTime(intDiv(d,1000))-1) = 202110; +select uniqExact(_part), count() from sDD where toYYYYMM(toDateTime(intDiv(d,1000)))+1 > 202009 and toStartOfDay(toDateTime(intDiv(d,1000))) < '2020-10-02 00:00:00'; +select uniqExact(_part), count() from sDD where toYYYYMM(toDateTime(intDiv(d,1000)))+1 > 202009 and toDateTime(intDiv(d,1000)) < '2020-10-01 00:00:00'; +select uniqExact(_part), count() from sDD where d >= 1598918400000; +select uniqExact(_part), count() from sDD where d >= 1598918400000 and toYYYYMM(toDateTime(intDiv(d,1000))-1) < 202010; -select uniqExact(_part), count() from xMM where toStartOfDay(d) >= '2020-10-01 00:00:00' -select uniqExact(_part), count() from xMM where d >= '2020-09-01 00:00:00' and d <= '2020-10-01 00:00:00' -select uniqExact(_part), count() from xMM where d >= '2020-09-01 00:00:00' and d < '2020-10-01 00:00:00' -select uniqExact(_part), count() from xMM where d >= '2020-09-01 00:00:00' and d <= '2020-10-01 00:00:00' and a=1 -select uniqExact(_part), count() from xMM where d >= '2020-09-01 00:00:00' and d <= '2020-10-01 00:00:00' and a<>3 -select uniqExact(_part), count() from xMM where d >= '2020-09-01 00:00:00' and d < '2020-10-01 00:00:00' and a<>3 -select uniqExact(_part), count() from xMM where d >= '2020-09-01 00:00:00' and d < '2020-11-01 00:00:00' and a = 1 -select uniqExact(_part), count() from xMM where a = 1 -select uniqExact(_part), count() from xMM where a = 66 -select uniqExact(_part), count() from xMM where a <> 66 -select uniqExact(_part), count() from xMM where a = 2 +select uniqExact(_part), count() from xMM where toStartOfDay(d) >= '2020-10-01 00:00:00'; +select uniqExact(_part), count() from xMM where d >= '2020-09-01 00:00:00' and d <= '2020-10-01 00:00:00'; +select uniqExact(_part), count() from xMM where d >= '2020-09-01 00:00:00' and d < '2020-10-01 00:00:00'; +select uniqExact(_part), count() from xMM where d >= '2020-09-01 00:00:00' and d <= '2020-10-01 00:00:00' and a=1; +select uniqExact(_part), count() from xMM where d >= '2020-09-01 00:00:00' and d <= '2020-10-01 00:00:00' and a<>3; +select uniqExact(_part), count() from xMM where d >= '2020-09-01 00:00:00' and d < '2020-10-01 00:00:00' and a<>3; +select uniqExact(_part), count() from xMM where d >= '2020-09-01 00:00:00' and d < '2020-11-01 00:00:00' and a = 1; +select uniqExact(_part), count() from xMM where a = 1; +select uniqExact(_part), count() from xMM where a = 66; +select uniqExact(_part), count() from xMM where a <> 66; +select uniqExact(_part), count() from xMM where a = 2; SYSTEM START MERGES xMM; optimize table xMM final; -select uniqExact(_part), count() from xMM where a = 1 -select uniqExact(_part), count() from xMM where toStartOfDay(d) >= '2020-10-01 00:00:00' -select uniqExact(_part), count() from xMM where a <> 66 -select uniqExact(_part), count() from xMM where d >= '2020-09-01 00:00:00' and d <= '2020-10-01 00:00:00' and a<>3 -select uniqExact(_part), count() from xMM where d >= '2020-09-01 00:00:00' and d < '2020-10-01 00:00:00' and a<>3 - -DROP TABLE tMM -DROP TABLE tDD -DROP TABLE sDD -DROP TABLE xMM +select uniqExact(_part), count() from xMM where a = 1; +select uniqExact(_part), count() from xMM where toStartOfDay(d) >= '2020-10-01 00:00:00'; +select uniqExact(_part), count() from xMM where a <> 66; +select uniqExact(_part), count() from xMM where d >= '2020-09-01 00:00:00' and d <= '2020-10-01 00:00:00' and a<>3; +select uniqExact(_part), count() from xMM where d >= '2020-09-01 00:00:00' and d < '2020-10-01 00:00:00' and a<>3; +DROP TABLE tMM; +DROP TABLE tDD; +DROP TABLE sDD; +DROP TABLE xMM; diff --git a/tests/queries/0_stateless/01508_partition_pruning.reference b/tests/queries/0_stateless/01508_partition_pruning.reference index d967c760e91..58ba0c38106 100644 --- a/tests/queries/0_stateless/01508_partition_pruning.reference +++ b/tests/queries/0_stateless/01508_partition_pruning.reference @@ -106,6 +106,9 @@ Selected 3 parts by partition key, 3 parts by primary key, 4 marks by primary ke select uniqExact(_part), count() FROM tDD WHERE toYYYYMMDD(d) >= 20200923 and toDate(d) <= '2020-09-26' Selected 3 parts by partition key, 3 parts by primary key, 4 marks by primary key, 4 marks to read from 3 ranges +select uniqExact(_part), count() from sDD; +Selected 6 parts by partition key, 6 parts by primary key, 6 marks by primary key, 6 marks to read from 6 ranges + select uniqExact(_part), count() from sDD where toYYYYMM(toDateTime(intDiv(d,1000))-1)+1 = 202010 Selected 3 parts by partition key, 3 parts by primary key, 3 marks by primary key, 3 marks to read from 3 ranges diff --git a/tests/queries/0_stateless/01508_partition_pruning_check_results.reference b/tests/queries/0_stateless/01508_partition_pruning_check_results.reference new file mode 100644 index 00000000000..52182ee1b6f --- /dev/null +++ b/tests/queries/0_stateless/01508_partition_pruning_check_results.reference @@ -0,0 +1,60 @@ +0 0 +2 2880 +1 1440 +0 0 +2 10000 +2 2880 +1 1440 +1 1440 +3 15000 +6 30000 +0 0 +2 6440 +2 10000 +2 2880 +1 1440 +2 6440 +4 20000 +2 10000 +1 1440 +3 11440 +1 1440 +3 11440 +1 1440 +2 10000 +3 9999 +2 10000 +4 20000 +2 10000 +2 9999 +1 10000 +2 20000 +1 10000 +1 10000 +1 10000 +3 40000 +3 40000 +6 30000 +3 9999 +2 9999 +0 0 +3 11440 +2 10000 +4 20000 +3 10001 +2 10000 +3 10001 +2 10000 +1 1 +2 5001 +1 5000 +2 10000 +3 15000 +0 0 +6 30000 +2 10000 +2 15000 +1 10000 +5 30000 +2 5001 +1 5000 diff --git a/tests/queries/0_stateless/01508_partition_pruning_check_results.sql b/tests/queries/0_stateless/01508_partition_pruning_check_results.sql new file mode 100644 index 00000000000..4fbe97a0f90 --- /dev/null +++ b/tests/queries/0_stateless/01508_partition_pruning_check_results.sql @@ -0,0 +1,121 @@ +DROP TABLE IF EXISTS tMM; +DROP TABLE IF EXISTS tDD; +DROP TABLE IF EXISTS sDD; +DROP TABLE IF EXISTS xMM; +CREATE TABLE tMM(d DateTime,a Int64) ENGINE = MergeTree PARTITION BY toYYYYMM(d) ORDER BY tuple() SETTINGS index_granularity = 8192; +SYSTEM STOP MERGES tMM; +INSERT INTO tMM SELECT toDateTime('2020-08-16 00:00:00') + number*60, number FROM numbers(5000); +INSERT INTO tMM SELECT toDateTime('2020-08-16 00:00:00') + number*60, number FROM numbers(5000); +INSERT INTO tMM SELECT toDateTime('2020-09-01 00:00:00') + number*60, number FROM numbers(5000); +INSERT INTO tMM SELECT toDateTime('2020-09-01 00:00:00') + number*60, number FROM numbers(5000); +INSERT INTO tMM SELECT toDateTime('2020-10-01 00:00:00') + number*60, number FROM numbers(5000); +INSERT INTO tMM SELECT toDateTime('2020-10-15 00:00:00') + number*60, number FROM numbers(5000); + +CREATE TABLE tDD(d DateTime,a Int) ENGINE = MergeTree PARTITION BY toYYYYMMDD(d) ORDER BY tuple() SETTINGS index_granularity = 8192; +SYSTEM STOP MERGES tDD; +insert into tDD select toDateTime(toDate('2020-09-23')), number from numbers(10000) UNION ALL select toDateTime(toDateTime('2020-09-23 11:00:00')), number from numbers(10000) UNION ALL select toDateTime(toDate('2020-09-24')), number from numbers(10000) UNION ALL select toDateTime(toDate('2020-09-25')), number from numbers(10000) UNION ALL select toDateTime(toDate('2020-08-15')), number from numbers(10000); + +CREATE TABLE sDD(d UInt64,a Int) ENGINE = MergeTree PARTITION BY toYYYYMM(toDate(intDiv(d,1000))) ORDER BY tuple() SETTINGS index_granularity = 8192; +SYSTEM STOP MERGES sDD; +insert into sDD select (1597536000+number*60)*1000, number from numbers(5000); +insert into sDD select (1597536000+number*60)*1000, number from numbers(5000); +insert into sDD select (1598918400+number*60)*1000, number from numbers(5000); +insert into sDD select (1598918400+number*60)*1000, number from numbers(5000); +insert into sDD select (1601510400+number*60)*1000, number from numbers(5000); +insert into sDD select (1602720000+number*60)*1000, number from numbers(5000); + +CREATE TABLE xMM(d DateTime,a Int64, f Int64) ENGINE = MergeTree PARTITION BY (toYYYYMM(d), a) ORDER BY tuple() SETTINGS index_granularity = 8192; +SYSTEM STOP MERGES xMM; +INSERT INTO xMM SELECT toDateTime('2020-08-16 00:00:00') + number*60, 1, number FROM numbers(5000); +INSERT INTO xMM SELECT toDateTime('2020-08-16 00:00:00') + number*60, 2, number FROM numbers(5000); +INSERT INTO xMM SELECT toDateTime('2020-09-01 00:00:00') + number*60, 3, number FROM numbers(5000); +INSERT INTO xMM SELECT toDateTime('2020-09-01 00:00:00') + number*60, 2, number FROM numbers(5000); +INSERT INTO xMM SELECT toDateTime('2020-10-01 00:00:00') + number*60, 1, number FROM numbers(5000); +INSERT INTO xMM SELECT toDateTime('2020-10-15 00:00:00') + number*60, 1, number FROM numbers(5000); + + + +select uniqExact(_part), count() from tMM where toDate(d)=toDate('2020-09-15'); +select uniqExact(_part), count() from tMM where toDate(d)=toDate('2020-09-01'); +select uniqExact(_part), count() from tMM where toDate(d)=toDate('2020-10-15'); +select uniqExact(_part), count() from tMM where toDate(d)='2020-09-15'; +select uniqExact(_part), count() from tMM where toYYYYMM(d)=202009; +select uniqExact(_part), count() from tMM where toYYYYMMDD(d)=20200816; +select uniqExact(_part), count() from tMM where toYYYYMMDD(d)=20201015; +select uniqExact(_part), count() from tMM where toDate(d)='2020-10-15'; +select uniqExact(_part), count() from tMM where d >= '2020-09-01 00:00:00' and d<'2020-10-15 00:00:00'; +select uniqExact(_part), count() from tMM where d >= '2020-01-16 00:00:00' and d < toDateTime('2021-08-17 00:00:00'); +select uniqExact(_part), count() from tMM where d >= '2020-09-16 00:00:00' and d < toDateTime('2020-10-01 00:00:00'); +select uniqExact(_part), count() from tMM where d >= '2020-09-12 00:00:00' and d < '2020-10-16 00:00:00'; +select uniqExact(_part), count() from tMM where toStartOfDay(d) >= '2020-09-12 00:00:00'; +select uniqExact(_part), count() from tMM where toStartOfDay(d) = '2020-09-01 00:00:00'; +select uniqExact(_part), count() from tMM where toStartOfDay(d) = '2020-10-01 00:00:00'; +select uniqExact(_part), count() from tMM where toStartOfDay(d) >= '2020-09-15 00:00:00' and d < '2020-10-16 00:00:00'; +select uniqExact(_part), count() from tMM where toYYYYMM(d) between 202009 and 202010; +select uniqExact(_part), count() from tMM where toYYYYMM(d) between 202009 and 202009; +select uniqExact(_part), count() from tMM where toYYYYMM(d) between 202009 and 202010 and toStartOfDay(d) = '2020-10-01 00:00:00'; +select uniqExact(_part), count() from tMM where toYYYYMM(d) >= 202009 and toStartOfDay(d) < '2020-10-02 00:00:00'; +select uniqExact(_part), count() from tMM where toYYYYMM(d) > 202009 and toStartOfDay(d) < '2020-10-02 00:00:00'; +select uniqExact(_part), count() from tMM where toYYYYMM(d)+1 > 202009 and toStartOfDay(d) < '2020-10-02 00:00:00'; +select uniqExact(_part), count() from tMM where toYYYYMM(d)+1 > 202010 and toStartOfDay(d) < '2020-10-02 00:00:00'; +select uniqExact(_part), count() from tMM where toYYYYMM(d)+1 > 202010; +select uniqExact(_part), count() from tMM where toYYYYMM(d-1)+1 = 202010; +select uniqExact(_part), count() from tMM where toStartOfMonth(d) >= '2020-09-15'; +select uniqExact(_part), count() from tMM where toStartOfMonth(d) >= '2020-09-01'; +select uniqExact(_part), count() from tMM where toStartOfMonth(d) >= '2020-09-01' and toStartOfMonth(d) < '2020-10-01'; + +SYSTEM START MERGES tMM; +OPTIMIZE TABLE tMM FINAL; + +select uniqExact(_part), count() from tMM where toYYYYMM(d-1)+1 = 202010; +select uniqExact(_part), count() from tMM where toYYYYMM(d)+1 > 202010; +select uniqExact(_part), count() from tMM where toYYYYMM(d) between 202009 and 202010; + +SYSTEM START MERGES tDD; +OPTIMIZE TABLE tDD FINAL; + +select uniqExact(_part), count() from tDD where toDate(d)=toDate('2020-09-24'); +select uniqExact(_part), count() FROM tDD WHERE toDate(d) = toDate('2020-09-24'); +select uniqExact(_part), count() FROM tDD WHERE toDate(d) = '2020-09-24'; +select uniqExact(_part), count() FROM tDD WHERE toDate(d) >= '2020-09-23' and toDate(d) <= '2020-09-26'; +select uniqExact(_part), count() FROM tDD WHERE toYYYYMMDD(d) >= 20200923 and toDate(d) <= '2020-09-26'; + + +select uniqExact(_part), count() from sDD; +select uniqExact(_part), count() from sDD where toYYYYMM(toDateTime(intDiv(d,1000))-1)+1 = 202010; +select uniqExact(_part), count() from sDD where toYYYYMM(toDateTime(intDiv(d,1000))-1) = 202010; +select uniqExact(_part), count() from sDD where toYYYYMM(toDateTime(intDiv(d,1000))-1) = 202110; +select uniqExact(_part), count() from sDD where toYYYYMM(toDateTime(intDiv(d,1000)))+1 > 202009 and toStartOfDay(toDateTime(intDiv(d,1000))) < '2020-10-02 00:00:00'; +select uniqExact(_part), count() from sDD where toYYYYMM(toDateTime(intDiv(d,1000)))+1 > 202009 and toDateTime(intDiv(d,1000)) < '2020-10-01 00:00:00'; +select uniqExact(_part), count() from sDD where d >= 1598918400000; +select uniqExact(_part), count() from sDD where d >= 1598918400000 and toYYYYMM(toDateTime(intDiv(d,1000))-1) < 202010; + + + +select uniqExact(_part), count() from xMM where toStartOfDay(d) >= '2020-10-01 00:00:00'; +select uniqExact(_part), count() from xMM where d >= '2020-09-01 00:00:00' and d <= '2020-10-01 00:00:00'; +select uniqExact(_part), count() from xMM where d >= '2020-09-01 00:00:00' and d < '2020-10-01 00:00:00'; +select uniqExact(_part), count() from xMM where d >= '2020-09-01 00:00:00' and d <= '2020-10-01 00:00:00' and a=1; +select uniqExact(_part), count() from xMM where d >= '2020-09-01 00:00:00' and d <= '2020-10-01 00:00:00' and a<>3; +select uniqExact(_part), count() from xMM where d >= '2020-09-01 00:00:00' and d < '2020-10-01 00:00:00' and a<>3; +select uniqExact(_part), count() from xMM where d >= '2020-09-01 00:00:00' and d < '2020-11-01 00:00:00' and a = 1; +select uniqExact(_part), count() from xMM where a = 1; +select uniqExact(_part), count() from xMM where a = 66; +select uniqExact(_part), count() from xMM where a <> 66; +select uniqExact(_part), count() from xMM where a = 2; + +SYSTEM START MERGES xMM; +optimize table xMM final; + +select uniqExact(_part), count() from xMM where a = 1; +select uniqExact(_part), count() from xMM where toStartOfDay(d) >= '2020-10-01 00:00:00'; +select uniqExact(_part), count() from xMM where a <> 66; +select uniqExact(_part), count() from xMM where d >= '2020-09-01 00:00:00' and d <= '2020-10-01 00:00:00' and a<>3; +select uniqExact(_part), count() from xMM where d >= '2020-09-01 00:00:00' and d < '2020-10-01 00:00:00' and a<>3; + +DROP TABLE tMM; +DROP TABLE tDD; +DROP TABLE sDD; +DROP TABLE xMM; + + From 06f3991b745df55df3fc2196b322dd52cdb42612 Mon Sep 17 00:00:00 2001 From: Denis Zhuravlev Date: Sat, 26 Sep 2020 12:25:47 -0300 Subject: [PATCH 112/321] added test for queries results --- .../01508_partition_pruning.reference | 118 +++++++++--------- 1 file changed, 59 insertions(+), 59 deletions(-) diff --git a/tests/queries/0_stateless/01508_partition_pruning.reference b/tests/queries/0_stateless/01508_partition_pruning.reference index 58ba0c38106..bf68bd78cb9 100644 --- a/tests/queries/0_stateless/01508_partition_pruning.reference +++ b/tests/queries/0_stateless/01508_partition_pruning.reference @@ -1,180 +1,180 @@ -select uniqExact(_part), count() from tMM where toDate(d)=toDate('2020-09-15') +select uniqExact(_part), count() from tMM where toDate(d)=toDate('2020-09-15'); Selected 0 parts by partition key, 0 parts by primary key, 0 marks by primary key, 0 marks to read from 0 ranges -select uniqExact(_part), count() from tMM where toDate(d)=toDate('2020-09-01') +select uniqExact(_part), count() from tMM where toDate(d)=toDate('2020-09-01'); Selected 2 parts by partition key, 2 parts by primary key, 2 marks by primary key, 2 marks to read from 2 ranges -select uniqExact(_part), count() from tMM where toDate(d)=toDate('2020-10-15') +select uniqExact(_part), count() from tMM where toDate(d)=toDate('2020-10-15'); Selected 1 parts by partition key, 1 parts by primary key, 1 marks by primary key, 1 marks to read from 1 ranges -select uniqExact(_part), count() from tMM where toDate(d)='2020-09-15' +select uniqExact(_part), count() from tMM where toDate(d)='2020-09-15'; Selected 0 parts by partition key, 0 parts by primary key, 0 marks by primary key, 0 marks to read from 0 ranges -select uniqExact(_part), count() from tMM where toYYYYMM(d)=202009 +select uniqExact(_part), count() from tMM where toYYYYMM(d)=202009; Selected 2 parts by partition key, 2 parts by primary key, 2 marks by primary key, 2 marks to read from 2 ranges -select uniqExact(_part), count() from tMM where toYYYYMMDD(d)=20200816 +select uniqExact(_part), count() from tMM where toYYYYMMDD(d)=20200816; Selected 2 parts by partition key, 2 parts by primary key, 2 marks by primary key, 2 marks to read from 2 ranges -select uniqExact(_part), count() from tMM where toYYYYMMDD(d)=20201015 +select uniqExact(_part), count() from tMM where toYYYYMMDD(d)=20201015; Selected 1 parts by partition key, 1 parts by primary key, 1 marks by primary key, 1 marks to read from 1 ranges -select uniqExact(_part), count() from tMM where toDate(d)='2020-10-15' +select uniqExact(_part), count() from tMM where toDate(d)='2020-10-15'; Selected 1 parts by partition key, 1 parts by primary key, 1 marks by primary key, 1 marks to read from 1 ranges -select uniqExact(_part), count() from tMM where d >= '2020-09-01 00:00:00' and d<'2020-10-15 00:00:00' +select uniqExact(_part), count() from tMM where d >= '2020-09-01 00:00:00' and d<'2020-10-15 00:00:00'; Selected 3 parts by partition key, 3 parts by primary key, 3 marks by primary key, 3 marks to read from 3 ranges -select uniqExact(_part), count() from tMM where d >= '2020-01-16 00:00:00' and d < toDateTime('2021-08-17 00:00:00') +select uniqExact(_part), count() from tMM where d >= '2020-01-16 00:00:00' and d < toDateTime('2021-08-17 00:00:00'); Selected 6 parts by partition key, 6 parts by primary key, 6 marks by primary key, 6 marks to read from 6 ranges -select uniqExact(_part), count() from tMM where d >= '2020-09-16 00:00:00' and d < toDateTime('2020-10-01 00:00:00') +select uniqExact(_part), count() from tMM where d >= '2020-09-16 00:00:00' and d < toDateTime('2020-10-01 00:00:00'); Selected 0 parts by partition key, 0 parts by primary key, 0 marks by primary key, 0 marks to read from 0 ranges -select uniqExact(_part), count() from tMM where d >= '2020-09-12 00:00:00' and d < '2020-10-16 00:00:00' +select uniqExact(_part), count() from tMM where d >= '2020-09-12 00:00:00' and d < '2020-10-16 00:00:00'; Selected 2 parts by partition key, 2 parts by primary key, 2 marks by primary key, 2 marks to read from 2 ranges -select uniqExact(_part), count() from tMM where toStartOfDay(d) >= '2020-09-12 00:00:00' +select uniqExact(_part), count() from tMM where toStartOfDay(d) >= '2020-09-12 00:00:00'; Selected 2 parts by partition key, 2 parts by primary key, 2 marks by primary key, 2 marks to read from 2 ranges -select uniqExact(_part), count() from tMM where toStartOfDay(d) = '2020-09-01 00:00:00' +select uniqExact(_part), count() from tMM where toStartOfDay(d) = '2020-09-01 00:00:00'; Selected 2 parts by partition key, 2 parts by primary key, 2 marks by primary key, 2 marks to read from 2 ranges -select uniqExact(_part), count() from tMM where toStartOfDay(d) = '2020-10-01 00:00:00' +select uniqExact(_part), count() from tMM where toStartOfDay(d) = '2020-10-01 00:00:00'; Selected 1 parts by partition key, 1 parts by primary key, 1 marks by primary key, 1 marks to read from 1 ranges -select uniqExact(_part), count() from tMM where toStartOfDay(d) >= '2020-09-15 00:00:00' and d < '2020-10-16 00:00:00' +select uniqExact(_part), count() from tMM where toStartOfDay(d) >= '2020-09-15 00:00:00' and d < '2020-10-16 00:00:00'; Selected 2 parts by partition key, 2 parts by primary key, 2 marks by primary key, 2 marks to read from 2 ranges -select uniqExact(_part), count() from tMM where toYYYYMM(d) between 202009 and 202010 +select uniqExact(_part), count() from tMM where toYYYYMM(d) between 202009 and 202010; Selected 4 parts by partition key, 4 parts by primary key, 4 marks by primary key, 4 marks to read from 4 ranges -select uniqExact(_part), count() from tMM where toYYYYMM(d) between 202009 and 202009 +select uniqExact(_part), count() from tMM where toYYYYMM(d) between 202009 and 202009; Selected 2 parts by partition key, 2 parts by primary key, 2 marks by primary key, 2 marks to read from 2 ranges -select uniqExact(_part), count() from tMM where toYYYYMM(d) between 202009 and 202010 and toStartOfDay(d) = '2020-10-01 00:00:00' +select uniqExact(_part), count() from tMM where toYYYYMM(d) between 202009 and 202010 and toStartOfDay(d) = '2020-10-01 00:00:00'; Selected 1 parts by partition key, 1 parts by primary key, 1 marks by primary key, 1 marks to read from 1 ranges -select uniqExact(_part), count() from tMM where toYYYYMM(d) >= 202009 and toStartOfDay(d) < '2020-10-02 00:00:00' +select uniqExact(_part), count() from tMM where toYYYYMM(d) >= 202009 and toStartOfDay(d) < '2020-10-02 00:00:00'; Selected 3 parts by partition key, 3 parts by primary key, 3 marks by primary key, 3 marks to read from 3 ranges -select uniqExact(_part), count() from tMM where toYYYYMM(d) > 202009 and toStartOfDay(d) < '2020-10-02 00:00:00' +select uniqExact(_part), count() from tMM where toYYYYMM(d) > 202009 and toStartOfDay(d) < '2020-10-02 00:00:00'; Selected 1 parts by partition key, 1 parts by primary key, 1 marks by primary key, 1 marks to read from 1 ranges -select uniqExact(_part), count() from tMM where toYYYYMM(d)+1 > 202009 and toStartOfDay(d) < '2020-10-02 00:00:00' +select uniqExact(_part), count() from tMM where toYYYYMM(d)+1 > 202009 and toStartOfDay(d) < '2020-10-02 00:00:00'; Selected 3 parts by partition key, 3 parts by primary key, 3 marks by primary key, 3 marks to read from 3 ranges -select uniqExact(_part), count() from tMM where toYYYYMM(d)+1 > 202010 and toStartOfDay(d) < '2020-10-02 00:00:00' +select uniqExact(_part), count() from tMM where toYYYYMM(d)+1 > 202010 and toStartOfDay(d) < '2020-10-02 00:00:00'; Selected 1 parts by partition key, 1 parts by primary key, 1 marks by primary key, 1 marks to read from 1 ranges -select uniqExact(_part), count() from tMM where toYYYYMM(d)+1 > 202010 +select uniqExact(_part), count() from tMM where toYYYYMM(d)+1 > 202010; Selected 2 parts by partition key, 2 parts by primary key, 2 marks by primary key, 2 marks to read from 2 ranges -select uniqExact(_part), count() from tMM where toYYYYMM(d-1)+1 = 202010 +select uniqExact(_part), count() from tMM where toYYYYMM(d-1)+1 = 202010; Selected 3 parts by partition key, 3 parts by primary key, 3 marks by primary key, 3 marks to read from 3 ranges -select uniqExact(_part), count() from tMM where toStartOfMonth(d) >= '2020-09-15' +select uniqExact(_part), count() from tMM where toStartOfMonth(d) >= '2020-09-15'; Selected 2 parts by partition key, 2 parts by primary key, 2 marks by primary key, 2 marks to read from 2 ranges -select uniqExact(_part), count() from tMM where toStartOfMonth(d) >= '2020-09-01' +select uniqExact(_part), count() from tMM where toStartOfMonth(d) >= '2020-09-01'; Selected 4 parts by partition key, 4 parts by primary key, 4 marks by primary key, 4 marks to read from 4 ranges -select uniqExact(_part), count() from tMM where toStartOfMonth(d) >= '2020-09-01' and toStartOfMonth(d) < '2020-10-01' +select uniqExact(_part), count() from tMM where toStartOfMonth(d) >= '2020-09-01' and toStartOfMonth(d) < '2020-10-01'; Selected 2 parts by partition key, 2 parts by primary key, 2 marks by primary key, 2 marks to read from 2 ranges -select uniqExact(_part), count() from tMM where toYYYYMM(d-1)+1 = 202010 +select uniqExact(_part), count() from tMM where toYYYYMM(d-1)+1 = 202010; Selected 2 parts by partition key, 2 parts by primary key, 2 marks by primary key, 2 marks to read from 2 ranges -select uniqExact(_part), count() from tMM where toYYYYMM(d)+1 > 202010 +select uniqExact(_part), count() from tMM where toYYYYMM(d)+1 > 202010; Selected 1 parts by partition key, 1 parts by primary key, 1 marks by primary key, 1 marks to read from 1 ranges -select uniqExact(_part), count() from tMM where toYYYYMM(d) between 202009 and 202010 +select uniqExact(_part), count() from tMM where toYYYYMM(d) between 202009 and 202010; Selected 2 parts by partition key, 2 parts by primary key, 2 marks by primary key, 2 marks to read from 2 ranges -select uniqExact(_part), count() from tDD where toDate(d)=toDate('2020-09-24') +select uniqExact(_part), count() from tDD where toDate(d)=toDate('2020-09-24'); Selected 1 parts by partition key, 1 parts by primary key, 1 marks by primary key, 1 marks to read from 1 ranges -select uniqExact(_part), count() FROM tDD WHERE toDate(d) = toDate('2020-09-24') +select uniqExact(_part), count() FROM tDD WHERE toDate(d) = toDate('2020-09-24'); Selected 1 parts by partition key, 1 parts by primary key, 1 marks by primary key, 1 marks to read from 1 ranges -select uniqExact(_part), count() FROM tDD WHERE toDate(d) = '2020-09-24' +select uniqExact(_part), count() FROM tDD WHERE toDate(d) = '2020-09-24'; Selected 1 parts by partition key, 1 parts by primary key, 1 marks by primary key, 1 marks to read from 1 ranges -select uniqExact(_part), count() FROM tDD WHERE toDate(d) >= '2020-09-23' and toDate(d) <= '2020-09-26' +select uniqExact(_part), count() FROM tDD WHERE toDate(d) >= '2020-09-23' and toDate(d) <= '2020-09-26'; Selected 3 parts by partition key, 3 parts by primary key, 4 marks by primary key, 4 marks to read from 3 ranges -select uniqExact(_part), count() FROM tDD WHERE toYYYYMMDD(d) >= 20200923 and toDate(d) <= '2020-09-26' +select uniqExact(_part), count() FROM tDD WHERE toYYYYMMDD(d) >= 20200923 and toDate(d) <= '2020-09-26'; Selected 3 parts by partition key, 3 parts by primary key, 4 marks by primary key, 4 marks to read from 3 ranges select uniqExact(_part), count() from sDD; Selected 6 parts by partition key, 6 parts by primary key, 6 marks by primary key, 6 marks to read from 6 ranges -select uniqExact(_part), count() from sDD where toYYYYMM(toDateTime(intDiv(d,1000))-1)+1 = 202010 +select uniqExact(_part), count() from sDD where toYYYYMM(toDateTime(intDiv(d,1000))-1)+1 = 202010; Selected 3 parts by partition key, 3 parts by primary key, 3 marks by primary key, 3 marks to read from 3 ranges -select uniqExact(_part), count() from sDD where toYYYYMM(toDateTime(intDiv(d,1000))-1) = 202010 +select uniqExact(_part), count() from sDD where toYYYYMM(toDateTime(intDiv(d,1000))-1) = 202010; Selected 2 parts by partition key, 2 parts by primary key, 2 marks by primary key, 2 marks to read from 2 ranges -select uniqExact(_part), count() from sDD where toYYYYMM(toDateTime(intDiv(d,1000))-1) = 202110 +select uniqExact(_part), count() from sDD where toYYYYMM(toDateTime(intDiv(d,1000))-1) = 202110; Selected 0 parts by partition key, 0 parts by primary key, 0 marks by primary key, 0 marks to read from 0 ranges -select uniqExact(_part), count() from sDD where toYYYYMM(toDateTime(intDiv(d,1000)))+1 > 202009 and toStartOfDay(toDateTime(intDiv(d,1000))) < '2020-10-02 00:00:00' +select uniqExact(_part), count() from sDD where toYYYYMM(toDateTime(intDiv(d,1000)))+1 > 202009 and toStartOfDay(toDateTime(intDiv(d,1000))) < '2020-10-02 00:00:00'; Selected 3 parts by partition key, 3 parts by primary key, 3 marks by primary key, 3 marks to read from 3 ranges -select uniqExact(_part), count() from sDD where toYYYYMM(toDateTime(intDiv(d,1000)))+1 > 202009 and toDateTime(intDiv(d,1000)) < '2020-10-01 00:00:00' +select uniqExact(_part), count() from sDD where toYYYYMM(toDateTime(intDiv(d,1000)))+1 > 202009 and toDateTime(intDiv(d,1000)) < '2020-10-01 00:00:00'; Selected 2 parts by partition key, 2 parts by primary key, 2 marks by primary key, 2 marks to read from 2 ranges -select uniqExact(_part), count() from sDD where d >= 1598918400000 +select uniqExact(_part), count() from sDD where d >= 1598918400000; Selected 4 parts by partition key, 4 parts by primary key, 4 marks by primary key, 4 marks to read from 4 ranges -select uniqExact(_part), count() from sDD where d >= 1598918400000 and toYYYYMM(toDateTime(intDiv(d,1000))-1) < 202010 +select uniqExact(_part), count() from sDD where d >= 1598918400000 and toYYYYMM(toDateTime(intDiv(d,1000))-1) < 202010; Selected 3 parts by partition key, 3 parts by primary key, 3 marks by primary key, 3 marks to read from 3 ranges -select uniqExact(_part), count() from xMM where toStartOfDay(d) >= '2020-10-01 00:00:00' +select uniqExact(_part), count() from xMM where toStartOfDay(d) >= '2020-10-01 00:00:00'; Selected 2 parts by partition key, 2 parts by primary key, 2 marks by primary key, 2 marks to read from 2 ranges -select uniqExact(_part), count() from xMM where d >= '2020-09-01 00:00:00' and d <= '2020-10-01 00:00:00' +select uniqExact(_part), count() from xMM where d >= '2020-09-01 00:00:00' and d <= '2020-10-01 00:00:00'; Selected 3 parts by partition key, 3 parts by primary key, 3 marks by primary key, 3 marks to read from 3 ranges -select uniqExact(_part), count() from xMM where d >= '2020-09-01 00:00:00' and d < '2020-10-01 00:00:00' +select uniqExact(_part), count() from xMM where d >= '2020-09-01 00:00:00' and d < '2020-10-01 00:00:00'; Selected 2 parts by partition key, 2 parts by primary key, 2 marks by primary key, 2 marks to read from 2 ranges -select uniqExact(_part), count() from xMM where d >= '2020-09-01 00:00:00' and d <= '2020-10-01 00:00:00' and a=1 +select uniqExact(_part), count() from xMM where d >= '2020-09-01 00:00:00' and d <= '2020-10-01 00:00:00' and a=1; Selected 1 parts by partition key, 1 parts by primary key, 1 marks by primary key, 1 marks to read from 1 ranges -select uniqExact(_part), count() from xMM where d >= '2020-09-01 00:00:00' and d <= '2020-10-01 00:00:00' and a<>3 +select uniqExact(_part), count() from xMM where d >= '2020-09-01 00:00:00' and d <= '2020-10-01 00:00:00' and a<>3; Selected 2 parts by partition key, 2 parts by primary key, 2 marks by primary key, 2 marks to read from 2 ranges -select uniqExact(_part), count() from xMM where d >= '2020-09-01 00:00:00' and d < '2020-10-01 00:00:00' and a<>3 +select uniqExact(_part), count() from xMM where d >= '2020-09-01 00:00:00' and d < '2020-10-01 00:00:00' and a<>3; Selected 1 parts by partition key, 1 parts by primary key, 1 marks by primary key, 1 marks to read from 1 ranges -select uniqExact(_part), count() from xMM where d >= '2020-09-01 00:00:00' and d < '2020-11-01 00:00:00' and a = 1 +select uniqExact(_part), count() from xMM where d >= '2020-09-01 00:00:00' and d < '2020-11-01 00:00:00' and a = 1; Selected 2 parts by partition key, 2 parts by primary key, 2 marks by primary key, 2 marks to read from 2 ranges -select uniqExact(_part), count() from xMM where a = 1 +select uniqExact(_part), count() from xMM where a = 1; Selected 3 parts by partition key, 3 parts by primary key, 3 marks by primary key, 3 marks to read from 3 ranges -select uniqExact(_part), count() from xMM where a = 66 +select uniqExact(_part), count() from xMM where a = 66; Selected 0 parts by partition key, 0 parts by primary key, 0 marks by primary key, 0 marks to read from 0 ranges -select uniqExact(_part), count() from xMM where a <> 66 +select uniqExact(_part), count() from xMM where a <> 66; Selected 6 parts by partition key, 6 parts by primary key, 6 marks by primary key, 6 marks to read from 6 ranges -select uniqExact(_part), count() from xMM where a = 2 +select uniqExact(_part), count() from xMM where a = 2; Selected 2 parts by partition key, 2 parts by primary key, 2 marks by primary key, 2 marks to read from 2 ranges -select uniqExact(_part), count() from xMM where a = 1 +select uniqExact(_part), count() from xMM where a = 1; Selected 2 parts by partition key, 2 parts by primary key, 2 marks by primary key, 2 marks to read from 2 ranges -select uniqExact(_part), count() from xMM where toStartOfDay(d) >= '2020-10-01 00:00:00' +select uniqExact(_part), count() from xMM where toStartOfDay(d) >= '2020-10-01 00:00:00'; Selected 1 parts by partition key, 1 parts by primary key, 1 marks by primary key, 1 marks to read from 1 ranges -select uniqExact(_part), count() from xMM where a <> 66 +select uniqExact(_part), count() from xMM where a <> 66; Selected 5 parts by partition key, 5 parts by primary key, 5 marks by primary key, 5 marks to read from 5 ranges -select uniqExact(_part), count() from xMM where d >= '2020-09-01 00:00:00' and d <= '2020-10-01 00:00:00' and a<>3 +select uniqExact(_part), count() from xMM where d >= '2020-09-01 00:00:00' and d <= '2020-10-01 00:00:00' and a<>3; Selected 2 parts by partition key, 2 parts by primary key, 2 marks by primary key, 2 marks to read from 2 ranges -select uniqExact(_part), count() from xMM where d >= '2020-09-01 00:00:00' and d < '2020-10-01 00:00:00' and a<>3 +select uniqExact(_part), count() from xMM where d >= '2020-09-01 00:00:00' and d < '2020-10-01 00:00:00' and a<>3; Selected 1 parts by partition key, 1 parts by primary key, 1 marks by primary key, 1 marks to read from 1 ranges From 34addcf61f653e50e493ff8d94fab015db91fa59 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Sat, 26 Sep 2020 22:18:28 +0300 Subject: [PATCH 113/321] add review suggestions --- programs/local/LocalServer.cpp | 2 +- programs/server/Server.cpp | 4 +- src/Common/Macros.cpp | 54 +++++++++++--- src/Common/Macros.h | 11 ++- src/Databases/DatabaseAtomic.cpp | 2 + src/Interpreters/InterpreterCreateQuery.cpp | 2 +- src/Storages/IStorage.h | 2 + .../MergeTree/registerStorageMergeTree.cpp | 74 ++++++++++++++++--- src/Storages/StorageFactory.cpp | 11 ++- src/Storages/StorageReplicatedMergeTree.cpp | 13 +++- src/Storages/StorageReplicatedMergeTree.h | 8 +- tests/config/config.d/macros.xml | 2 + .../integration/test_distributed_ddl/test.py | 8 +- ..._zookeeper_path_macros_unfolding.reference | 4 + .../01148_zookeeper_path_macros_unfolding.sql | 19 +++++ 15 files changed, 182 insertions(+), 34 deletions(-) create mode 100644 tests/queries/0_stateless/01148_zookeeper_path_macros_unfolding.reference create mode 100644 tests/queries/0_stateless/01148_zookeeper_path_macros_unfolding.sql diff --git a/programs/local/LocalServer.cpp b/programs/local/LocalServer.cpp index 5cdf5766a44..41da477152c 100644 --- a/programs/local/LocalServer.cpp +++ b/programs/local/LocalServer.cpp @@ -210,7 +210,7 @@ try /// Maybe useless if (config().has("macros")) - context->setMacros(std::make_unique(config(), "macros")); + context->setMacros(std::make_unique(config(), "macros", log)); /// Skip networking diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index b703d0ac6a7..84945eeed4d 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -534,7 +534,7 @@ int Server::main(const std::vector & /*args*/) } if (config().has("macros")) - global_context->setMacros(std::make_unique(config(), "macros")); + global_context->setMacros(std::make_unique(config(), "macros", log)); /// Initialize main config reloader. std::string include_from_path = config().getString("include_from", "/etc/metrika.xml"); @@ -559,7 +559,7 @@ int Server::main(const std::vector & /*args*/) //setTextLog(global_context->getTextLog()); //buildLoggers(*config, logger()); global_context->setClustersConfig(config); - global_context->setMacros(std::make_unique(*config, "macros")); + global_context->setMacros(std::make_unique(*config, "macros", log)); global_context->setExternalAuthenticatorsConfig(*config); /// Setup protection to avoid accidental DROP for big tables (that are greater than 50 GB by default) diff --git a/src/Common/Macros.cpp b/src/Common/Macros.cpp index e3735c44359..b98daa0ec78 100644 --- a/src/Common/Macros.cpp +++ b/src/Common/Macros.cpp @@ -2,6 +2,7 @@ #include #include #include +#include namespace DB @@ -12,19 +13,31 @@ namespace ErrorCodes extern const int SYNTAX_ERROR; } -Macros::Macros(const Poco::Util::AbstractConfiguration & config, const String & root_key) +Macros::Macros(const Poco::Util::AbstractConfiguration & config, const String & root_key, Poco::Logger * log) { Poco::Util::AbstractConfiguration::Keys keys; config.keys(root_key, keys); for (const String & key : keys) { macros[key] = config.getString(root_key + "." + key); + if (key == "database" || key == "table" || key == "uuid") + { + LOG_WARNING(log, + "Config file contains '{}' macro. This macro has special meaning " + "and it's explicit definition is not recommended. Implicit unfolding for " + "'database', 'table' and 'uuid' macros will be disabled.", + key); + enable_special_macros = false; + } } } String Macros::expand(const String & s, MacroExpansionInfo & info) const { + /// Do not allow recursion if we expand only special macros, because it will be infinite recursion + assert(info.level == 0 || !info.expand_special_macros_only); + if (s.find('{') == String::npos) return s; @@ -34,6 +47,10 @@ String Macros::expand(const String & s, if (info.level >= 10) throw Exception("Too deep recursion while expanding macros: '" + s + "'", ErrorCodes::SYNTAX_ERROR); + /// If config file contains explicit special macro, then we do not expand it in this mode. + if (!enable_special_macros && info.expand_special_macros_only) + return s; + String res; size_t pos = 0; while (true) @@ -59,15 +76,21 @@ String Macros::expand(const String & s, auto it = macros.find(macro_name); /// Prefer explicit macros over implicit. - if (it != macros.end()) + if (it != macros.end() && !info.expand_special_macros_only) res += it->second; - else if (macro_name == "database" && !info.database_name.empty()) - res += info.database_name; - else if (macro_name == "table" && !info.table_name.empty()) - res += info.table_name; + else if (macro_name == "database" && !info.table_id.database_name.empty()) + { + res += info.table_id.database_name; + info.expanded_database = true; + } + else if (macro_name == "table" && !info.table_id.table_name.empty()) + { + res += info.table_id.table_name; + info.expanded_table = true; + } else if (macro_name == "uuid") { - if (info.uuid == UUIDHelpers::Nil) + if (info.table_id.uuid == UUIDHelpers::Nil) throw Exception("Macro 'uuid' and empty arguments of ReplicatedMergeTree " "are supported only for ON CLUSTER queries with Atomic database engine", ErrorCodes::SYNTAX_ERROR); @@ -76,12 +99,16 @@ String Macros::expand(const String & s, /// It becomes impossible to check if {uuid} is contained inside some unknown macro. if (info.level) throw Exception("Macro 'uuid' should not be inside another macro", ErrorCodes::SYNTAX_ERROR); - res += toString(info.uuid); + res += toString(info.table_id.uuid); info.expanded_uuid = true; } - else if (info.ignore_unknown) + else if (info.ignore_unknown || info.expand_special_macros_only) { + if (info.expand_special_macros_only) + res += '{'; res += macro_name; + if (info.expand_special_macros_only) + res += '}'; info.has_unknown = true; } else @@ -93,6 +120,9 @@ String Macros::expand(const String & s, } ++info.level; + if (info.expand_special_macros_only) + return res; + return expand(res, info); } @@ -113,9 +143,9 @@ String Macros::expand(const String & s) const String Macros::expand(const String & s, const StorageID & table_id, bool allow_uuid) const { MacroExpansionInfo info; - info.database_name = table_id.database_name; - info.table_name = table_id.table_name; - info.uuid = allow_uuid ? table_id.uuid : UUIDHelpers::Nil; + info.table_id = table_id; + if (!allow_uuid) + info.table_id.uuid = UUIDHelpers::Nil; return expand(s, info); } diff --git a/src/Common/Macros.h b/src/Common/Macros.h index 6e4f25d55ef..c04405a83de 100644 --- a/src/Common/Macros.h +++ b/src/Common/Macros.h @@ -13,6 +13,7 @@ namespace Poco { class AbstractConfiguration; } + class Logger; } @@ -25,18 +26,19 @@ class Macros { public: Macros() = default; - Macros(const Poco::Util::AbstractConfiguration & config, const String & key); + Macros(const Poco::Util::AbstractConfiguration & config, const String & key, Poco::Logger * log); struct MacroExpansionInfo { /// Settings - String database_name; - String table_name; - UUID uuid = UUIDHelpers::Nil; + StorageID table_id = StorageID::createEmpty(); bool ignore_unknown = false; + bool expand_special_macros_only = false; /// Information about macro expansion size_t level = 0; + bool expanded_database = false; + bool expanded_table = false; bool expanded_uuid = false; bool has_unknown = false; }; @@ -64,6 +66,7 @@ public: private: MacroMap macros; + bool enable_special_macros = true; }; diff --git a/src/Databases/DatabaseAtomic.cpp b/src/Databases/DatabaseAtomic.cpp index ed17a8eccb1..6ad547898e2 100644 --- a/src/Databases/DatabaseAtomic.cpp +++ b/src/Databases/DatabaseAtomic.cpp @@ -207,11 +207,13 @@ void DatabaseAtomic::renameTable(const Context & context, const String & table_n throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Cannot move dictionary to other database"); StoragePtr table = getTableUnlocked(table_name, db_lock); + table->checkTableCanBeRenamed(); assert_can_move_mat_view(table); StoragePtr other_table; if (exchange) { other_table = other_db.getTableUnlocked(to_table_name, other_db_lock); + other_table->checkTableCanBeRenamed(); assert_can_move_mat_view(other_table); } diff --git a/src/Interpreters/InterpreterCreateQuery.cpp b/src/Interpreters/InterpreterCreateQuery.cpp index 6f318b3658a..1b9c2ca3431 100644 --- a/src/Interpreters/InterpreterCreateQuery.cpp +++ b/src/Interpreters/InterpreterCreateQuery.cpp @@ -888,7 +888,7 @@ void InterpreterCreateQuery::prepareOnClusterQuery(ASTCreateQuery & create, cons { String zk_path = create.storage->engine->arguments->children[0]->as()->value.get(); Macros::MacroExpansionInfo info; - info.uuid = create.uuid; + info.table_id.uuid = create.uuid; info.ignore_unknown = true; context.getMacros()->expand(zk_path, info); if (!info.expanded_uuid) diff --git a/src/Storages/IStorage.h b/src/Storages/IStorage.h index 40500e78de1..4a2e70aa84b 100644 --- a/src/Storages/IStorage.h +++ b/src/Storages/IStorage.h @@ -337,6 +337,8 @@ public: throw Exception("Truncate is not supported by storage " + getName(), ErrorCodes::NOT_IMPLEMENTED); } + virtual void checkTableCanBeRenamed() const {} + /** Rename the table. * Renaming a name in a file with metadata, the name in the list of tables in the RAM, is done separately. * In this function, you need to rename the directory with the data, if any. diff --git a/src/Storages/MergeTree/registerStorageMergeTree.cpp b/src/Storages/MergeTree/registerStorageMergeTree.cpp index a9f7576f896..2b0d1a60657 100644 --- a/src/Storages/MergeTree/registerStorageMergeTree.cpp +++ b/src/Storages/MergeTree/registerStorageMergeTree.cpp @@ -412,26 +412,31 @@ static StoragePtr create(const StorageFactory::Arguments & args) /// For Replicated. String zookeeper_path; String replica_name; + bool allow_renaming = true; if (replicated) { bool has_arguments = arg_num + 2 <= arg_cnt; bool has_valid_arguments = has_arguments && engine_args[arg_num]->as() && engine_args[arg_num + 1]->as(); + ASTLiteral * ast_zk_path; + ASTLiteral * ast_replica_name; + if (has_valid_arguments) { - const auto * ast = engine_args[arg_num]->as(); - if (ast && ast->value.getType() == Field::Types::String) - zookeeper_path = safeGet(ast->value); + /// Get path and name from engine arguments + ast_zk_path = engine_args[arg_num]->as(); + if (ast_zk_path && ast_zk_path->value.getType() == Field::Types::String) + zookeeper_path = safeGet(ast_zk_path->value); else throw Exception( "Path in ZooKeeper must be a string literal" + getMergeTreeVerboseHelp(is_extended_storage_def), ErrorCodes::BAD_ARGUMENTS); ++arg_num; - ast = engine_args[arg_num]->as(); - if (ast && ast->value.getType() == Field::Types::String) - replica_name = safeGet(ast->value); + ast_replica_name = engine_args[arg_num]->as(); + if (ast_replica_name && ast_replica_name->value.getType() == Field::Types::String) + replica_name = safeGet(ast_replica_name->value); else throw Exception( "Replica name must be a string literal" + getMergeTreeVerboseHelp(is_extended_storage_def), ErrorCodes::BAD_ARGUMENTS); @@ -448,6 +453,20 @@ static StoragePtr create(const StorageFactory::Arguments & args) zookeeper_path = args.context.getConfigRef().getString("default_replica_path", "/clickhouse/tables/{uuid}/{shard}"); /// TODO maybe use hostname if {replica} is not defined? replica_name = args.context.getConfigRef().getString("default_replica_name", "{replica}"); + + /// Modify query, so default values will be written to metadata + assert(arg_num == 0); + ASTs old_args; + std::swap(engine_args, old_args); + auto path_arg = std::make_shared(zookeeper_path); + auto name_arg = std::make_shared(replica_name); + ast_zk_path = path_arg.get(); + ast_replica_name = name_arg.get(); + engine_args.emplace_back(std::move(path_arg)); + engine_args.emplace_back(std::move(name_arg)); + std::move(std::begin(old_args), std::end(old_args), std::back_inserter(engine_args)); + arg_num = 2; + arg_cnt += 2; } else throw Exception("Expected two string literal arguments: zookeper_path and replica_name", ErrorCodes::BAD_ARGUMENTS); @@ -455,8 +474,44 @@ static StoragePtr create(const StorageFactory::Arguments & args) /// Allow implicit {uuid} macros only for zookeeper_path in ON CLUSTER queries bool is_on_cluster = args.local_context.getClientInfo().query_kind == ClientInfo::QueryKind::SECONDARY_QUERY; bool allow_uuid_macro = is_on_cluster || args.query.attach; - zookeeper_path = args.context.getMacros()->expand(zookeeper_path, args.table_id, allow_uuid_macro); - replica_name = args.context.getMacros()->expand(replica_name, args.table_id, false); + + /// Unfold {database} and {table} macro on table creation, so table can be renamed. + /// We also unfold {uuid} macro, so path will not be broken after moving table from Atomic to Ordinary database. + if (!args.attach) + { + Macros::MacroExpansionInfo info; + /// NOTE: it's not recursive + info.expand_special_macros_only = true; + info.table_id = args.table_id; + if (!allow_uuid_macro) + info.table_id.uuid = UUIDHelpers::Nil; + zookeeper_path = args.context.getMacros()->expand(zookeeper_path, info); + + info.level = 0; + info.table_id.uuid = UUIDHelpers::Nil; + replica_name = args.context.getMacros()->expand(replica_name, info); + } + + ast_zk_path->value = zookeeper_path; + ast_replica_name->value = replica_name; + + /// Expand other macros (such as {shard} and {replica}). We do not expand them on previous step + /// to make possible copying metadata files between replicas. + Macros::MacroExpansionInfo info; + info.table_id = args.table_id; + if (!allow_uuid_macro) + info.table_id.uuid = UUIDHelpers::Nil; + zookeeper_path = args.context.getMacros()->expand(zookeeper_path, info); + + info.level = 0; + info.table_id.uuid = UUIDHelpers::Nil; + replica_name = args.context.getMacros()->expand(replica_name, info); + + /// We do not allow renaming table with these macros in metadata, because zookeeper_path will be broken after RENAME TABLE. + /// NOTE: it may happen if table was created by older version of ClickHouse (< 20.10) and macros was not unfolded on table creation + /// or if one of these macros is recursively expanded from some other macro. + if (info.expanded_database || info.expanded_table) + allow_renaming = false; } /// This merging param maybe used as part of sorting key @@ -707,7 +762,8 @@ static StoragePtr create(const StorageFactory::Arguments & args) date_column_name, merging_params, std::move(storage_settings), - args.has_force_restore_data_flag); + args.has_force_restore_data_flag, + allow_renaming); else return StorageMergeTree::create( args.table_id, diff --git a/src/Storages/StorageFactory.cpp b/src/Storages/StorageFactory.cpp index 0a8ceb4b8e5..eda9f36010f 100644 --- a/src/Storages/StorageFactory.cpp +++ b/src/Storages/StorageFactory.cpp @@ -180,7 +180,16 @@ StoragePtr StorageFactory::get( .has_force_restore_data_flag = has_force_restore_data_flag }; - return storages.at(name).creator_fn(arguments); + auto res = storages.at(name).creator_fn(arguments); + if (!empty_engine_args.empty()) + { + /// Storage creator modified empty arguments list, so we should modify the query + assert(storage_def && storage_def->engine && !storage_def->engine->arguments); + storage_def->engine->arguments = std::make_shared(); + storage_def->engine->children.push_back(storage_def->engine->arguments); + storage_def->engine->arguments->children = empty_engine_args; + } + return res; } StorageFactory & StorageFactory::instance() diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 65c0c5ac313..9613bd5111d 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -178,7 +178,8 @@ StorageReplicatedMergeTree::StorageReplicatedMergeTree( const String & date_column_name, const MergingParams & merging_params_, std::unique_ptr settings_, - bool has_force_restore_data_flag) + bool has_force_restore_data_flag, + bool allow_renaming_) : MergeTreeData(table_id_, relative_data_path_, metadata_, @@ -200,6 +201,7 @@ StorageReplicatedMergeTree::StorageReplicatedMergeTree( , cleanup_thread(*this) , part_check_thread(*this) , restarting_thread(*this) + , allow_renaming(allow_renaming_) { queue_updating_task = global_context.getSchedulePool().createTask( getStorageID().getFullTableName() + " (StorageReplicatedMergeTree::queueUpdatingTask)", [this]{ queueUpdatingTask(); }); @@ -4187,8 +4189,17 @@ void StorageReplicatedMergeTree::checkTableCanBeDropped() const global_context.checkTableCanBeDropped(table_id.database_name, table_id.table_name, getTotalActiveSizeInBytes()); } +void StorageReplicatedMergeTree::checkTableCanBeRenamed() const +{ + if (!allow_renaming) + throw Exception("Cannot rename Replicated table, because zookeeper_path contains implicit 'database' or 'table' macro. " + "We cannot rename path in ZooKeeper, so path may become inconsistent with table name. If you really want to rename table, " + "you should edit metadata file first and restart server or reattach the table.", ErrorCodes::NOT_IMPLEMENTED); +} + void StorageReplicatedMergeTree::rename(const String & new_path_to_table_data, const StorageID & new_table_id) { + checkTableCanBeRenamed(); MergeTreeData::rename(new_path_to_table_data, new_table_id); /// Update table name in zookeeper diff --git a/src/Storages/StorageReplicatedMergeTree.h b/src/Storages/StorageReplicatedMergeTree.h index 2bc9265331d..d851082d5c2 100644 --- a/src/Storages/StorageReplicatedMergeTree.h +++ b/src/Storages/StorageReplicatedMergeTree.h @@ -128,6 +128,8 @@ public: void truncate(const ASTPtr &, const StorageMetadataPtr &, const Context &, TableExclusiveLockHolder &) override; + void checkTableCanBeRenamed() const override; + void rename(const String & new_path_to_table_data, const StorageID & new_table_id) override; bool supportsIndexForIn() const override { return true; } @@ -304,6 +306,9 @@ private: /// True if replica was created for existing table with fixed granularity bool other_replicas_fixed_granularity = false; + /// Do not allow RENAME TABLE if zookeeper_path contains {database} or {table} macro + const bool allow_renaming; + template void foreachCommittedParts(const Func & func) const; @@ -571,7 +576,8 @@ protected: const String & date_column_name, const MergingParams & merging_params_, std::unique_ptr settings_, - bool has_force_restore_data_flag); + bool has_force_restore_data_flag, + bool allow_renaming_); }; diff --git a/tests/config/config.d/macros.xml b/tests/config/config.d/macros.xml index 97c3065471f..4902b12bc81 100644 --- a/tests/config/config.d/macros.xml +++ b/tests/config/config.d/macros.xml @@ -3,5 +3,7 @@ Hello, world! s1 r1 + /clickhouse/tables/{database}/{shard}/ + table_{table} diff --git a/tests/integration/test_distributed_ddl/test.py b/tests/integration/test_distributed_ddl/test.py index b788dafe167..9f01fa7ed5b 100755 --- a/tests/integration/test_distributed_ddl/test.py +++ b/tests/integration/test_distributed_ddl/test.py @@ -330,10 +330,12 @@ def test_replicated_without_arguments(test_cluster): assert "are supported only for ON CLUSTER queries with Atomic database engine" in \ instance.query_and_get_error("CREATE TABLE test_atomic.rmt (n UInt64, s String) ENGINE=ReplicatedMergeTree ORDER BY n") test_cluster.ddl_check_query(instance, - "CREATE TABLE test_atomic.rmt ON CLUSTER cluster (n UInt64, s String) ENGINE=ReplicatedMergeTree ORDER BY n") + "CREATE TABLE test_atomic.rmt ON CLUSTER cluster (n UInt64, s String) ENGINE=ReplicatedMergeTree() ORDER BY n") test_cluster.ddl_check_query(instance, "DROP TABLE test_atomic.rmt ON CLUSTER cluster") test_cluster.ddl_check_query(instance, - "CREATE TABLE test_atomic.rmt ON CLUSTER cluster (n UInt64, s String) ENGINE=ReplicatedMergeTree ORDER BY n") + "CREATE TABLE test_atomic.rmt UUID '12345678-0000-4000-8000-000000000001' ON CLUSTER cluster (n UInt64, s String) ENGINE=ReplicatedMergeTree ORDER BY n") + assert instance.query("SHOW CREATE test_atomic.rmt FORMAT TSVRaw") == \ + "CREATE TABLE test_atomic.rmt\n(\n `n` UInt64,\n `s` String\n)\nENGINE = ReplicatedMergeTree('/clickhouse/tables/12345678-0000-4000-8000-000000000001/{shard}', '{replica}')\nORDER BY n\nSETTINGS index_granularity = 8192\n" test_cluster.ddl_check_query(instance, "RENAME TABLE test_atomic.rmt TO test_atomic.rmt_renamed ON CLUSTER cluster") test_cluster.ddl_check_query(instance, "CREATE TABLE test_atomic.rmt ON CLUSTER cluster (n UInt64, s String) ENGINE=ReplicatedMergeTree('/clickhouse/tables/{uuid}/{shard}', '{replica}') ORDER BY n") @@ -349,6 +351,8 @@ def test_replicated_without_arguments(test_cluster): assert "are supported only for ON CLUSTER queries with Atomic database engine" in \ instance.query_and_get_error("CREATE TABLE test_ordinary.rmt ON CLUSTER cluster (n UInt64, s String) ENGINE=ReplicatedMergeTree('/{shard}/{uuid}/', '{replica}') ORDER BY n") test_cluster.ddl_check_query(instance, "CREATE TABLE test_ordinary.rmt ON CLUSTER cluster (n UInt64, s String) ENGINE=ReplicatedMergeTree('/{shard}/{table}/', '{replica}') ORDER BY n") + assert instance.query("SHOW CREATE test_ordinary.rmt FORMAT TSVRaw") == \ + "CREATE TABLE test_ordinary.rmt\n(\n `n` UInt64,\n `s` String\n)\nENGINE = ReplicatedMergeTree('/{shard}/rmt/', '{replica}')\nORDER BY n\nSETTINGS index_granularity = 8192\n" test_cluster.ddl_check_query(instance, "DROP DATABASE test_ordinary ON CLUSTER cluster") test_cluster.pm_random_drops.push_rules(rules) diff --git a/tests/queries/0_stateless/01148_zookeeper_path_macros_unfolding.reference b/tests/queries/0_stateless/01148_zookeeper_path_macros_unfolding.reference new file mode 100644 index 00000000000..d217855586b --- /dev/null +++ b/tests/queries/0_stateless/01148_zookeeper_path_macros_unfolding.reference @@ -0,0 +1,4 @@ +CREATE TABLE default.rmt\n(\n `n` UInt64,\n `s` String\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/test_01148/{shard}/default/rmt\', \'{replica}\')\nORDER BY n\nSETTINGS index_granularity = 8192 +CREATE TABLE default.rmt1\n(\n `n` UInt64,\n `s` String\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/test_01148/{shard}/default/rmt\', \'{replica}\')\nORDER BY n\nSETTINGS index_granularity = 8192 +CREATE TABLE default.rmt\n(\n `n` UInt64,\n `s` String\n)\nENGINE = ReplicatedMergeTree(\'{default_path_test}test_01148\', \'{default_name_test}\')\nORDER BY n\nSETTINGS index_granularity = 8192 +CREATE TABLE default.rmt\n(\n `n` UInt64,\n `s` String\n)\nENGINE = ReplicatedMergeTree(\'{default_path_test}test_01148\', \'{default_name_test}\')\nORDER BY n\nSETTINGS index_granularity = 8192 diff --git a/tests/queries/0_stateless/01148_zookeeper_path_macros_unfolding.sql b/tests/queries/0_stateless/01148_zookeeper_path_macros_unfolding.sql new file mode 100644 index 00000000000..e13b25d1706 --- /dev/null +++ b/tests/queries/0_stateless/01148_zookeeper_path_macros_unfolding.sql @@ -0,0 +1,19 @@ +DROP TABLE IF EXISTS rmt; + +CREATE TABLE rmt (n UInt64, s String) ENGINE = ReplicatedMergeTree('/clickhouse/test_01148/{shard}/{database}/{table}', '{replica}') ORDER BY n; +SHOW CREATE TABLE rmt; +RENAME TABLE rmt TO rmt1; +DETACH TABLE rmt1; +ATTACH TABLE rmt1; +SHOW CREATE TABLE rmt1; + +CREATE TABLE rmt (n UInt64, s String) ENGINE = ReplicatedMergeTree('{default_path_test}{uuid}', '{default_name_test}') ORDER BY n; -- { serverError 62 } +CREATE TABLE rmt (n UInt64, s String) ENGINE = ReplicatedMergeTree('{default_path_test}test_01148', '{default_name_test}') ORDER BY n; +SHOW CREATE TABLE rmt; +RENAME TABLE rmt TO rmt2; -- { serverError 48 } +DETACH TABLE rmt; +ATTACH TABLE rmt; +SHOW CREATE TABLE rmt; + +DROP TABLE rmt; +DROP TABLE rmt1; From 837a829da592426732133d3cf999546a31629e3e Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Sun, 27 Sep 2020 00:19:10 +0300 Subject: [PATCH 114/321] Update 01323_add_scalars_in_time.sql --- tests/queries/0_stateless/01323_add_scalars_in_time.sql | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/01323_add_scalars_in_time.sql b/tests/queries/0_stateless/01323_add_scalars_in_time.sql index d09534d2b43..2d7cf270017 100644 --- a/tests/queries/0_stateless/01323_add_scalars_in_time.sql +++ b/tests/queries/0_stateless/01323_add_scalars_in_time.sql @@ -58,7 +58,7 @@ ENGINE = MergeTree() PARTITION BY tuple() ORDER BY id; -INSERT INTO bbb VALUES (2, 'fre'),(3, 'jhg'); +INSERT INTO bbb VALUES (2, 'fre'), (3, 'jhg'); with (select groupArray(id) from bbb) as ids select * From cb901ac3045d818cb1a4609bc29f19ba481077d0 Mon Sep 17 00:00:00 2001 From: Denis Zhuravlev Date: Sat, 26 Sep 2020 19:07:13 -0300 Subject: [PATCH 115/321] fix test (UTC timezone specified) --- .../01508_partition_pruning.queries | 19 +-- .../01508_partition_pruning.reference | 76 ++++++++++- .../0_stateless/01508_partition_pruning.sh | 3 +- ..._partition_pruning_check_results.reference | 60 --------- .../01508_partition_pruning_check_results.sql | 121 ------------------ 5 files changed, 83 insertions(+), 196 deletions(-) delete mode 100644 tests/queries/0_stateless/01508_partition_pruning_check_results.reference delete mode 100644 tests/queries/0_stateless/01508_partition_pruning_check_results.sql diff --git a/tests/queries/0_stateless/01508_partition_pruning.queries b/tests/queries/0_stateless/01508_partition_pruning.queries index 4fbe97a0f90..a30b541c3bc 100644 --- a/tests/queries/0_stateless/01508_partition_pruning.queries +++ b/tests/queries/0_stateless/01508_partition_pruning.queries @@ -34,7 +34,7 @@ INSERT INTO xMM SELECT toDateTime('2020-10-01 00:00:00') + number*60, 1, number INSERT INTO xMM SELECT toDateTime('2020-10-15 00:00:00') + number*60, 1, number FROM numbers(5000); - +SELECT '--------- tMM ----------------------------'; select uniqExact(_part), count() from tMM where toDate(d)=toDate('2020-09-15'); select uniqExact(_part), count() from tMM where toDate(d)=toDate('2020-09-01'); select uniqExact(_part), count() from tMM where toDate(d)=toDate('2020-10-15'); @@ -71,6 +71,8 @@ select uniqExact(_part), count() from tMM where toYYYYMM(d-1)+1 = 202010; select uniqExact(_part), count() from tMM where toYYYYMM(d)+1 > 202010; select uniqExact(_part), count() from tMM where toYYYYMM(d) between 202009 and 202010; + +SELECT '--------- tDD ----------------------------'; SYSTEM START MERGES tDD; OPTIMIZE TABLE tDD FINAL; @@ -81,17 +83,18 @@ select uniqExact(_part), count() FROM tDD WHERE toDate(d) >= '2020-09-23' and to select uniqExact(_part), count() FROM tDD WHERE toYYYYMMDD(d) >= 20200923 and toDate(d) <= '2020-09-26'; +SELECT '--------- sDD ----------------------------'; select uniqExact(_part), count() from sDD; -select uniqExact(_part), count() from sDD where toYYYYMM(toDateTime(intDiv(d,1000))-1)+1 = 202010; -select uniqExact(_part), count() from sDD where toYYYYMM(toDateTime(intDiv(d,1000))-1) = 202010; -select uniqExact(_part), count() from sDD where toYYYYMM(toDateTime(intDiv(d,1000))-1) = 202110; -select uniqExact(_part), count() from sDD where toYYYYMM(toDateTime(intDiv(d,1000)))+1 > 202009 and toStartOfDay(toDateTime(intDiv(d,1000))) < '2020-10-02 00:00:00'; -select uniqExact(_part), count() from sDD where toYYYYMM(toDateTime(intDiv(d,1000)))+1 > 202009 and toDateTime(intDiv(d,1000)) < '2020-10-01 00:00:00'; +select uniqExact(_part), count() from sDD where toYYYYMM(toDateTime(intDiv(d,1000),'UTC')-1)+1 = 202010; +select uniqExact(_part), count() from sDD where toYYYYMM(toDateTime(intDiv(d,1000),'UTC')-1) = 202010; +select uniqExact(_part), count() from sDD where toYYYYMM(toDateTime(intDiv(d,1000),'UTC')-1) = 202110; +select uniqExact(_part), count() from sDD where toYYYYMM(toDateTime(intDiv(d,1000),'UTC'))+1 > 202009 and toStartOfDay(toDateTime(intDiv(d,1000))) < '2020-10-02 00:00:00'; +select uniqExact(_part), count() from sDD where toYYYYMM(toDateTime(intDiv(d,1000),'UTC'))+1 > 202009 and toDateTime(intDiv(d,1000)) < '2020-10-01 00:00:00'; select uniqExact(_part), count() from sDD where d >= 1598918400000; -select uniqExact(_part), count() from sDD where d >= 1598918400000 and toYYYYMM(toDateTime(intDiv(d,1000))-1) < 202010; - +select uniqExact(_part), count() from sDD where d >= 1598918400000 and toYYYYMM(toDateTime(intDiv(d,1000),'UTC')-1) < 202010; +SELECT '--------- xMM ----------------------------'; select uniqExact(_part), count() from xMM where toStartOfDay(d) >= '2020-10-01 00:00:00'; select uniqExact(_part), count() from xMM where d >= '2020-09-01 00:00:00' and d <= '2020-10-01 00:00:00'; select uniqExact(_part), count() from xMM where d >= '2020-09-01 00:00:00' and d < '2020-10-01 00:00:00'; diff --git a/tests/queries/0_stateless/01508_partition_pruning.reference b/tests/queries/0_stateless/01508_partition_pruning.reference index bf68bd78cb9..97a27f6fa8b 100644 --- a/tests/queries/0_stateless/01508_partition_pruning.reference +++ b/tests/queries/0_stateless/01508_partition_pruning.reference @@ -1,180 +1,244 @@ +--------- tMM ---------------------------- select uniqExact(_part), count() from tMM where toDate(d)=toDate('2020-09-15'); +0 0 Selected 0 parts by partition key, 0 parts by primary key, 0 marks by primary key, 0 marks to read from 0 ranges select uniqExact(_part), count() from tMM where toDate(d)=toDate('2020-09-01'); +2 2880 Selected 2 parts by partition key, 2 parts by primary key, 2 marks by primary key, 2 marks to read from 2 ranges select uniqExact(_part), count() from tMM where toDate(d)=toDate('2020-10-15'); +1 1440 Selected 1 parts by partition key, 1 parts by primary key, 1 marks by primary key, 1 marks to read from 1 ranges select uniqExact(_part), count() from tMM where toDate(d)='2020-09-15'; +0 0 Selected 0 parts by partition key, 0 parts by primary key, 0 marks by primary key, 0 marks to read from 0 ranges select uniqExact(_part), count() from tMM where toYYYYMM(d)=202009; +2 10000 Selected 2 parts by partition key, 2 parts by primary key, 2 marks by primary key, 2 marks to read from 2 ranges select uniqExact(_part), count() from tMM where toYYYYMMDD(d)=20200816; +2 2880 Selected 2 parts by partition key, 2 parts by primary key, 2 marks by primary key, 2 marks to read from 2 ranges select uniqExact(_part), count() from tMM where toYYYYMMDD(d)=20201015; +1 1440 Selected 1 parts by partition key, 1 parts by primary key, 1 marks by primary key, 1 marks to read from 1 ranges select uniqExact(_part), count() from tMM where toDate(d)='2020-10-15'; +1 1440 Selected 1 parts by partition key, 1 parts by primary key, 1 marks by primary key, 1 marks to read from 1 ranges select uniqExact(_part), count() from tMM where d >= '2020-09-01 00:00:00' and d<'2020-10-15 00:00:00'; +3 15000 Selected 3 parts by partition key, 3 parts by primary key, 3 marks by primary key, 3 marks to read from 3 ranges select uniqExact(_part), count() from tMM where d >= '2020-01-16 00:00:00' and d < toDateTime('2021-08-17 00:00:00'); +6 30000 Selected 6 parts by partition key, 6 parts by primary key, 6 marks by primary key, 6 marks to read from 6 ranges select uniqExact(_part), count() from tMM where d >= '2020-09-16 00:00:00' and d < toDateTime('2020-10-01 00:00:00'); +0 0 Selected 0 parts by partition key, 0 parts by primary key, 0 marks by primary key, 0 marks to read from 0 ranges select uniqExact(_part), count() from tMM where d >= '2020-09-12 00:00:00' and d < '2020-10-16 00:00:00'; +2 6440 Selected 2 parts by partition key, 2 parts by primary key, 2 marks by primary key, 2 marks to read from 2 ranges select uniqExact(_part), count() from tMM where toStartOfDay(d) >= '2020-09-12 00:00:00'; +2 10000 Selected 2 parts by partition key, 2 parts by primary key, 2 marks by primary key, 2 marks to read from 2 ranges select uniqExact(_part), count() from tMM where toStartOfDay(d) = '2020-09-01 00:00:00'; +2 2880 Selected 2 parts by partition key, 2 parts by primary key, 2 marks by primary key, 2 marks to read from 2 ranges select uniqExact(_part), count() from tMM where toStartOfDay(d) = '2020-10-01 00:00:00'; +1 1440 Selected 1 parts by partition key, 1 parts by primary key, 1 marks by primary key, 1 marks to read from 1 ranges select uniqExact(_part), count() from tMM where toStartOfDay(d) >= '2020-09-15 00:00:00' and d < '2020-10-16 00:00:00'; +2 6440 Selected 2 parts by partition key, 2 parts by primary key, 2 marks by primary key, 2 marks to read from 2 ranges select uniqExact(_part), count() from tMM where toYYYYMM(d) between 202009 and 202010; +4 20000 Selected 4 parts by partition key, 4 parts by primary key, 4 marks by primary key, 4 marks to read from 4 ranges select uniqExact(_part), count() from tMM where toYYYYMM(d) between 202009 and 202009; +2 10000 Selected 2 parts by partition key, 2 parts by primary key, 2 marks by primary key, 2 marks to read from 2 ranges select uniqExact(_part), count() from tMM where toYYYYMM(d) between 202009 and 202010 and toStartOfDay(d) = '2020-10-01 00:00:00'; +1 1440 Selected 1 parts by partition key, 1 parts by primary key, 1 marks by primary key, 1 marks to read from 1 ranges select uniqExact(_part), count() from tMM where toYYYYMM(d) >= 202009 and toStartOfDay(d) < '2020-10-02 00:00:00'; +3 11440 Selected 3 parts by partition key, 3 parts by primary key, 3 marks by primary key, 3 marks to read from 3 ranges select uniqExact(_part), count() from tMM where toYYYYMM(d) > 202009 and toStartOfDay(d) < '2020-10-02 00:00:00'; +1 1440 Selected 1 parts by partition key, 1 parts by primary key, 1 marks by primary key, 1 marks to read from 1 ranges select uniqExact(_part), count() from tMM where toYYYYMM(d)+1 > 202009 and toStartOfDay(d) < '2020-10-02 00:00:00'; +3 11440 Selected 3 parts by partition key, 3 parts by primary key, 3 marks by primary key, 3 marks to read from 3 ranges select uniqExact(_part), count() from tMM where toYYYYMM(d)+1 > 202010 and toStartOfDay(d) < '2020-10-02 00:00:00'; +1 1440 Selected 1 parts by partition key, 1 parts by primary key, 1 marks by primary key, 1 marks to read from 1 ranges select uniqExact(_part), count() from tMM where toYYYYMM(d)+1 > 202010; +2 10000 Selected 2 parts by partition key, 2 parts by primary key, 2 marks by primary key, 2 marks to read from 2 ranges select uniqExact(_part), count() from tMM where toYYYYMM(d-1)+1 = 202010; +3 9999 Selected 3 parts by partition key, 3 parts by primary key, 3 marks by primary key, 3 marks to read from 3 ranges select uniqExact(_part), count() from tMM where toStartOfMonth(d) >= '2020-09-15'; +2 10000 Selected 2 parts by partition key, 2 parts by primary key, 2 marks by primary key, 2 marks to read from 2 ranges select uniqExact(_part), count() from tMM where toStartOfMonth(d) >= '2020-09-01'; +4 20000 Selected 4 parts by partition key, 4 parts by primary key, 4 marks by primary key, 4 marks to read from 4 ranges select uniqExact(_part), count() from tMM where toStartOfMonth(d) >= '2020-09-01' and toStartOfMonth(d) < '2020-10-01'; +2 10000 Selected 2 parts by partition key, 2 parts by primary key, 2 marks by primary key, 2 marks to read from 2 ranges select uniqExact(_part), count() from tMM where toYYYYMM(d-1)+1 = 202010; +2 9999 Selected 2 parts by partition key, 2 parts by primary key, 2 marks by primary key, 2 marks to read from 2 ranges select uniqExact(_part), count() from tMM where toYYYYMM(d)+1 > 202010; +1 10000 Selected 1 parts by partition key, 1 parts by primary key, 1 marks by primary key, 1 marks to read from 1 ranges select uniqExact(_part), count() from tMM where toYYYYMM(d) between 202009 and 202010; +2 20000 Selected 2 parts by partition key, 2 parts by primary key, 2 marks by primary key, 2 marks to read from 2 ranges +--------- tDD ---------------------------- select uniqExact(_part), count() from tDD where toDate(d)=toDate('2020-09-24'); +1 10000 Selected 1 parts by partition key, 1 parts by primary key, 1 marks by primary key, 1 marks to read from 1 ranges select uniqExact(_part), count() FROM tDD WHERE toDate(d) = toDate('2020-09-24'); +1 10000 Selected 1 parts by partition key, 1 parts by primary key, 1 marks by primary key, 1 marks to read from 1 ranges select uniqExact(_part), count() FROM tDD WHERE toDate(d) = '2020-09-24'; +1 10000 Selected 1 parts by partition key, 1 parts by primary key, 1 marks by primary key, 1 marks to read from 1 ranges select uniqExact(_part), count() FROM tDD WHERE toDate(d) >= '2020-09-23' and toDate(d) <= '2020-09-26'; +3 40000 Selected 3 parts by partition key, 3 parts by primary key, 4 marks by primary key, 4 marks to read from 3 ranges select uniqExact(_part), count() FROM tDD WHERE toYYYYMMDD(d) >= 20200923 and toDate(d) <= '2020-09-26'; +3 40000 Selected 3 parts by partition key, 3 parts by primary key, 4 marks by primary key, 4 marks to read from 3 ranges +--------- sDD ---------------------------- select uniqExact(_part), count() from sDD; +6 30000 Selected 6 parts by partition key, 6 parts by primary key, 6 marks by primary key, 6 marks to read from 6 ranges -select uniqExact(_part), count() from sDD where toYYYYMM(toDateTime(intDiv(d,1000))-1)+1 = 202010; +select uniqExact(_part), count() from sDD where toYYYYMM(toDateTime(intDiv(d,1000),'UTC')-1)+1 = 202010; +3 9999 Selected 3 parts by partition key, 3 parts by primary key, 3 marks by primary key, 3 marks to read from 3 ranges -select uniqExact(_part), count() from sDD where toYYYYMM(toDateTime(intDiv(d,1000))-1) = 202010; +select uniqExact(_part), count() from sDD where toYYYYMM(toDateTime(intDiv(d,1000),'UTC')-1) = 202010; +2 9999 Selected 2 parts by partition key, 2 parts by primary key, 2 marks by primary key, 2 marks to read from 2 ranges -select uniqExact(_part), count() from sDD where toYYYYMM(toDateTime(intDiv(d,1000))-1) = 202110; +select uniqExact(_part), count() from sDD where toYYYYMM(toDateTime(intDiv(d,1000),'UTC')-1) = 202110; +0 0 Selected 0 parts by partition key, 0 parts by primary key, 0 marks by primary key, 0 marks to read from 0 ranges -select uniqExact(_part), count() from sDD where toYYYYMM(toDateTime(intDiv(d,1000)))+1 > 202009 and toStartOfDay(toDateTime(intDiv(d,1000))) < '2020-10-02 00:00:00'; +select uniqExact(_part), count() from sDD where toYYYYMM(toDateTime(intDiv(d,1000),'UTC'))+1 > 202009 and toStartOfDay(toDateTime(intDiv(d,1000))) < '2020-10-02 00:00:00'; +3 11440 Selected 3 parts by partition key, 3 parts by primary key, 3 marks by primary key, 3 marks to read from 3 ranges -select uniqExact(_part), count() from sDD where toYYYYMM(toDateTime(intDiv(d,1000)))+1 > 202009 and toDateTime(intDiv(d,1000)) < '2020-10-01 00:00:00'; +select uniqExact(_part), count() from sDD where toYYYYMM(toDateTime(intDiv(d,1000),'UTC'))+1 > 202009 and toDateTime(intDiv(d,1000)) < '2020-10-01 00:00:00'; +2 10000 Selected 2 parts by partition key, 2 parts by primary key, 2 marks by primary key, 2 marks to read from 2 ranges select uniqExact(_part), count() from sDD where d >= 1598918400000; +4 20000 Selected 4 parts by partition key, 4 parts by primary key, 4 marks by primary key, 4 marks to read from 4 ranges -select uniqExact(_part), count() from sDD where d >= 1598918400000 and toYYYYMM(toDateTime(intDiv(d,1000))-1) < 202010; +select uniqExact(_part), count() from sDD where d >= 1598918400000 and toYYYYMM(toDateTime(intDiv(d,1000),'UTC')-1) < 202010; +3 10001 Selected 3 parts by partition key, 3 parts by primary key, 3 marks by primary key, 3 marks to read from 3 ranges +--------- xMM ---------------------------- select uniqExact(_part), count() from xMM where toStartOfDay(d) >= '2020-10-01 00:00:00'; +2 10000 Selected 2 parts by partition key, 2 parts by primary key, 2 marks by primary key, 2 marks to read from 2 ranges select uniqExact(_part), count() from xMM where d >= '2020-09-01 00:00:00' and d <= '2020-10-01 00:00:00'; +3 10001 Selected 3 parts by partition key, 3 parts by primary key, 3 marks by primary key, 3 marks to read from 3 ranges select uniqExact(_part), count() from xMM where d >= '2020-09-01 00:00:00' and d < '2020-10-01 00:00:00'; +2 10000 Selected 2 parts by partition key, 2 parts by primary key, 2 marks by primary key, 2 marks to read from 2 ranges select uniqExact(_part), count() from xMM where d >= '2020-09-01 00:00:00' and d <= '2020-10-01 00:00:00' and a=1; +1 1 Selected 1 parts by partition key, 1 parts by primary key, 1 marks by primary key, 1 marks to read from 1 ranges select uniqExact(_part), count() from xMM where d >= '2020-09-01 00:00:00' and d <= '2020-10-01 00:00:00' and a<>3; +2 5001 Selected 2 parts by partition key, 2 parts by primary key, 2 marks by primary key, 2 marks to read from 2 ranges select uniqExact(_part), count() from xMM where d >= '2020-09-01 00:00:00' and d < '2020-10-01 00:00:00' and a<>3; +1 5000 Selected 1 parts by partition key, 1 parts by primary key, 1 marks by primary key, 1 marks to read from 1 ranges select uniqExact(_part), count() from xMM where d >= '2020-09-01 00:00:00' and d < '2020-11-01 00:00:00' and a = 1; +2 10000 Selected 2 parts by partition key, 2 parts by primary key, 2 marks by primary key, 2 marks to read from 2 ranges select uniqExact(_part), count() from xMM where a = 1; +3 15000 Selected 3 parts by partition key, 3 parts by primary key, 3 marks by primary key, 3 marks to read from 3 ranges select uniqExact(_part), count() from xMM where a = 66; +0 0 Selected 0 parts by partition key, 0 parts by primary key, 0 marks by primary key, 0 marks to read from 0 ranges select uniqExact(_part), count() from xMM where a <> 66; +6 30000 Selected 6 parts by partition key, 6 parts by primary key, 6 marks by primary key, 6 marks to read from 6 ranges select uniqExact(_part), count() from xMM where a = 2; +2 10000 Selected 2 parts by partition key, 2 parts by primary key, 2 marks by primary key, 2 marks to read from 2 ranges select uniqExact(_part), count() from xMM where a = 1; +2 15000 Selected 2 parts by partition key, 2 parts by primary key, 2 marks by primary key, 2 marks to read from 2 ranges select uniqExact(_part), count() from xMM where toStartOfDay(d) >= '2020-10-01 00:00:00'; +1 10000 Selected 1 parts by partition key, 1 parts by primary key, 1 marks by primary key, 1 marks to read from 1 ranges select uniqExact(_part), count() from xMM where a <> 66; +5 30000 Selected 5 parts by partition key, 5 parts by primary key, 5 marks by primary key, 5 marks to read from 5 ranges select uniqExact(_part), count() from xMM where d >= '2020-09-01 00:00:00' and d <= '2020-10-01 00:00:00' and a<>3; +2 5001 Selected 2 parts by partition key, 2 parts by primary key, 2 marks by primary key, 2 marks to read from 2 ranges select uniqExact(_part), count() from xMM where d >= '2020-09-01 00:00:00' and d < '2020-10-01 00:00:00' and a<>3; +1 5000 Selected 1 parts by partition key, 1 parts by primary key, 1 marks by primary key, 1 marks to read from 1 ranges diff --git a/tests/queries/0_stateless/01508_partition_pruning.sh b/tests/queries/0_stateless/01508_partition_pruning.sh index e06b907f19f..e155431380f 100755 --- a/tests/queries/0_stateless/01508_partition_pruning.sh +++ b/tests/queries/0_stateless/01508_partition_pruning.sh @@ -20,8 +20,9 @@ do [ -z "$sql" ] && continue if [[ "$sql" == select* ]] ; then - CLICKHOUSE_CLIENT=$(echo ${CLICKHOUSE_CLIENT} | sed 's/'"--send_logs_level=${CLICKHOUSE_CLIENT_SERVER_LOGS_LEVEL}"'/--send_logs_level=debug/g') echo "$sql" + ${CLICKHOUSE_CLIENT} --query "$sql" + CLICKHOUSE_CLIENT=$(echo ${CLICKHOUSE_CLIENT} | sed 's/'"--send_logs_level=${CLICKHOUSE_CLIENT_SERVER_LOGS_LEVEL}"'/--send_logs_level=debug/g') ${CLICKHOUSE_CLIENT} --query "$sql" 2>&1 | grep -oh "Selected .* parts by partition key, *. parts by primary key, .* marks by primary key, .* marks to read from .* ranges.*$" CLICKHOUSE_CLIENT=$(echo ${CLICKHOUSE_CLIENT} | sed 's/--send_logs_level=debug/'"--send_logs_level=${CLICKHOUSE_CLIENT_SERVER_LOGS_LEVEL}"'/g') echo "" diff --git a/tests/queries/0_stateless/01508_partition_pruning_check_results.reference b/tests/queries/0_stateless/01508_partition_pruning_check_results.reference deleted file mode 100644 index 52182ee1b6f..00000000000 --- a/tests/queries/0_stateless/01508_partition_pruning_check_results.reference +++ /dev/null @@ -1,60 +0,0 @@ -0 0 -2 2880 -1 1440 -0 0 -2 10000 -2 2880 -1 1440 -1 1440 -3 15000 -6 30000 -0 0 -2 6440 -2 10000 -2 2880 -1 1440 -2 6440 -4 20000 -2 10000 -1 1440 -3 11440 -1 1440 -3 11440 -1 1440 -2 10000 -3 9999 -2 10000 -4 20000 -2 10000 -2 9999 -1 10000 -2 20000 -1 10000 -1 10000 -1 10000 -3 40000 -3 40000 -6 30000 -3 9999 -2 9999 -0 0 -3 11440 -2 10000 -4 20000 -3 10001 -2 10000 -3 10001 -2 10000 -1 1 -2 5001 -1 5000 -2 10000 -3 15000 -0 0 -6 30000 -2 10000 -2 15000 -1 10000 -5 30000 -2 5001 -1 5000 diff --git a/tests/queries/0_stateless/01508_partition_pruning_check_results.sql b/tests/queries/0_stateless/01508_partition_pruning_check_results.sql deleted file mode 100644 index 4fbe97a0f90..00000000000 --- a/tests/queries/0_stateless/01508_partition_pruning_check_results.sql +++ /dev/null @@ -1,121 +0,0 @@ -DROP TABLE IF EXISTS tMM; -DROP TABLE IF EXISTS tDD; -DROP TABLE IF EXISTS sDD; -DROP TABLE IF EXISTS xMM; -CREATE TABLE tMM(d DateTime,a Int64) ENGINE = MergeTree PARTITION BY toYYYYMM(d) ORDER BY tuple() SETTINGS index_granularity = 8192; -SYSTEM STOP MERGES tMM; -INSERT INTO tMM SELECT toDateTime('2020-08-16 00:00:00') + number*60, number FROM numbers(5000); -INSERT INTO tMM SELECT toDateTime('2020-08-16 00:00:00') + number*60, number FROM numbers(5000); -INSERT INTO tMM SELECT toDateTime('2020-09-01 00:00:00') + number*60, number FROM numbers(5000); -INSERT INTO tMM SELECT toDateTime('2020-09-01 00:00:00') + number*60, number FROM numbers(5000); -INSERT INTO tMM SELECT toDateTime('2020-10-01 00:00:00') + number*60, number FROM numbers(5000); -INSERT INTO tMM SELECT toDateTime('2020-10-15 00:00:00') + number*60, number FROM numbers(5000); - -CREATE TABLE tDD(d DateTime,a Int) ENGINE = MergeTree PARTITION BY toYYYYMMDD(d) ORDER BY tuple() SETTINGS index_granularity = 8192; -SYSTEM STOP MERGES tDD; -insert into tDD select toDateTime(toDate('2020-09-23')), number from numbers(10000) UNION ALL select toDateTime(toDateTime('2020-09-23 11:00:00')), number from numbers(10000) UNION ALL select toDateTime(toDate('2020-09-24')), number from numbers(10000) UNION ALL select toDateTime(toDate('2020-09-25')), number from numbers(10000) UNION ALL select toDateTime(toDate('2020-08-15')), number from numbers(10000); - -CREATE TABLE sDD(d UInt64,a Int) ENGINE = MergeTree PARTITION BY toYYYYMM(toDate(intDiv(d,1000))) ORDER BY tuple() SETTINGS index_granularity = 8192; -SYSTEM STOP MERGES sDD; -insert into sDD select (1597536000+number*60)*1000, number from numbers(5000); -insert into sDD select (1597536000+number*60)*1000, number from numbers(5000); -insert into sDD select (1598918400+number*60)*1000, number from numbers(5000); -insert into sDD select (1598918400+number*60)*1000, number from numbers(5000); -insert into sDD select (1601510400+number*60)*1000, number from numbers(5000); -insert into sDD select (1602720000+number*60)*1000, number from numbers(5000); - -CREATE TABLE xMM(d DateTime,a Int64, f Int64) ENGINE = MergeTree PARTITION BY (toYYYYMM(d), a) ORDER BY tuple() SETTINGS index_granularity = 8192; -SYSTEM STOP MERGES xMM; -INSERT INTO xMM SELECT toDateTime('2020-08-16 00:00:00') + number*60, 1, number FROM numbers(5000); -INSERT INTO xMM SELECT toDateTime('2020-08-16 00:00:00') + number*60, 2, number FROM numbers(5000); -INSERT INTO xMM SELECT toDateTime('2020-09-01 00:00:00') + number*60, 3, number FROM numbers(5000); -INSERT INTO xMM SELECT toDateTime('2020-09-01 00:00:00') + number*60, 2, number FROM numbers(5000); -INSERT INTO xMM SELECT toDateTime('2020-10-01 00:00:00') + number*60, 1, number FROM numbers(5000); -INSERT INTO xMM SELECT toDateTime('2020-10-15 00:00:00') + number*60, 1, number FROM numbers(5000); - - - -select uniqExact(_part), count() from tMM where toDate(d)=toDate('2020-09-15'); -select uniqExact(_part), count() from tMM where toDate(d)=toDate('2020-09-01'); -select uniqExact(_part), count() from tMM where toDate(d)=toDate('2020-10-15'); -select uniqExact(_part), count() from tMM where toDate(d)='2020-09-15'; -select uniqExact(_part), count() from tMM where toYYYYMM(d)=202009; -select uniqExact(_part), count() from tMM where toYYYYMMDD(d)=20200816; -select uniqExact(_part), count() from tMM where toYYYYMMDD(d)=20201015; -select uniqExact(_part), count() from tMM where toDate(d)='2020-10-15'; -select uniqExact(_part), count() from tMM where d >= '2020-09-01 00:00:00' and d<'2020-10-15 00:00:00'; -select uniqExact(_part), count() from tMM where d >= '2020-01-16 00:00:00' and d < toDateTime('2021-08-17 00:00:00'); -select uniqExact(_part), count() from tMM where d >= '2020-09-16 00:00:00' and d < toDateTime('2020-10-01 00:00:00'); -select uniqExact(_part), count() from tMM where d >= '2020-09-12 00:00:00' and d < '2020-10-16 00:00:00'; -select uniqExact(_part), count() from tMM where toStartOfDay(d) >= '2020-09-12 00:00:00'; -select uniqExact(_part), count() from tMM where toStartOfDay(d) = '2020-09-01 00:00:00'; -select uniqExact(_part), count() from tMM where toStartOfDay(d) = '2020-10-01 00:00:00'; -select uniqExact(_part), count() from tMM where toStartOfDay(d) >= '2020-09-15 00:00:00' and d < '2020-10-16 00:00:00'; -select uniqExact(_part), count() from tMM where toYYYYMM(d) between 202009 and 202010; -select uniqExact(_part), count() from tMM where toYYYYMM(d) between 202009 and 202009; -select uniqExact(_part), count() from tMM where toYYYYMM(d) between 202009 and 202010 and toStartOfDay(d) = '2020-10-01 00:00:00'; -select uniqExact(_part), count() from tMM where toYYYYMM(d) >= 202009 and toStartOfDay(d) < '2020-10-02 00:00:00'; -select uniqExact(_part), count() from tMM where toYYYYMM(d) > 202009 and toStartOfDay(d) < '2020-10-02 00:00:00'; -select uniqExact(_part), count() from tMM where toYYYYMM(d)+1 > 202009 and toStartOfDay(d) < '2020-10-02 00:00:00'; -select uniqExact(_part), count() from tMM where toYYYYMM(d)+1 > 202010 and toStartOfDay(d) < '2020-10-02 00:00:00'; -select uniqExact(_part), count() from tMM where toYYYYMM(d)+1 > 202010; -select uniqExact(_part), count() from tMM where toYYYYMM(d-1)+1 = 202010; -select uniqExact(_part), count() from tMM where toStartOfMonth(d) >= '2020-09-15'; -select uniqExact(_part), count() from tMM where toStartOfMonth(d) >= '2020-09-01'; -select uniqExact(_part), count() from tMM where toStartOfMonth(d) >= '2020-09-01' and toStartOfMonth(d) < '2020-10-01'; - -SYSTEM START MERGES tMM; -OPTIMIZE TABLE tMM FINAL; - -select uniqExact(_part), count() from tMM where toYYYYMM(d-1)+1 = 202010; -select uniqExact(_part), count() from tMM where toYYYYMM(d)+1 > 202010; -select uniqExact(_part), count() from tMM where toYYYYMM(d) between 202009 and 202010; - -SYSTEM START MERGES tDD; -OPTIMIZE TABLE tDD FINAL; - -select uniqExact(_part), count() from tDD where toDate(d)=toDate('2020-09-24'); -select uniqExact(_part), count() FROM tDD WHERE toDate(d) = toDate('2020-09-24'); -select uniqExact(_part), count() FROM tDD WHERE toDate(d) = '2020-09-24'; -select uniqExact(_part), count() FROM tDD WHERE toDate(d) >= '2020-09-23' and toDate(d) <= '2020-09-26'; -select uniqExact(_part), count() FROM tDD WHERE toYYYYMMDD(d) >= 20200923 and toDate(d) <= '2020-09-26'; - - -select uniqExact(_part), count() from sDD; -select uniqExact(_part), count() from sDD where toYYYYMM(toDateTime(intDiv(d,1000))-1)+1 = 202010; -select uniqExact(_part), count() from sDD where toYYYYMM(toDateTime(intDiv(d,1000))-1) = 202010; -select uniqExact(_part), count() from sDD where toYYYYMM(toDateTime(intDiv(d,1000))-1) = 202110; -select uniqExact(_part), count() from sDD where toYYYYMM(toDateTime(intDiv(d,1000)))+1 > 202009 and toStartOfDay(toDateTime(intDiv(d,1000))) < '2020-10-02 00:00:00'; -select uniqExact(_part), count() from sDD where toYYYYMM(toDateTime(intDiv(d,1000)))+1 > 202009 and toDateTime(intDiv(d,1000)) < '2020-10-01 00:00:00'; -select uniqExact(_part), count() from sDD where d >= 1598918400000; -select uniqExact(_part), count() from sDD where d >= 1598918400000 and toYYYYMM(toDateTime(intDiv(d,1000))-1) < 202010; - - - -select uniqExact(_part), count() from xMM where toStartOfDay(d) >= '2020-10-01 00:00:00'; -select uniqExact(_part), count() from xMM where d >= '2020-09-01 00:00:00' and d <= '2020-10-01 00:00:00'; -select uniqExact(_part), count() from xMM where d >= '2020-09-01 00:00:00' and d < '2020-10-01 00:00:00'; -select uniqExact(_part), count() from xMM where d >= '2020-09-01 00:00:00' and d <= '2020-10-01 00:00:00' and a=1; -select uniqExact(_part), count() from xMM where d >= '2020-09-01 00:00:00' and d <= '2020-10-01 00:00:00' and a<>3; -select uniqExact(_part), count() from xMM where d >= '2020-09-01 00:00:00' and d < '2020-10-01 00:00:00' and a<>3; -select uniqExact(_part), count() from xMM where d >= '2020-09-01 00:00:00' and d < '2020-11-01 00:00:00' and a = 1; -select uniqExact(_part), count() from xMM where a = 1; -select uniqExact(_part), count() from xMM where a = 66; -select uniqExact(_part), count() from xMM where a <> 66; -select uniqExact(_part), count() from xMM where a = 2; - -SYSTEM START MERGES xMM; -optimize table xMM final; - -select uniqExact(_part), count() from xMM where a = 1; -select uniqExact(_part), count() from xMM where toStartOfDay(d) >= '2020-10-01 00:00:00'; -select uniqExact(_part), count() from xMM where a <> 66; -select uniqExact(_part), count() from xMM where d >= '2020-09-01 00:00:00' and d <= '2020-10-01 00:00:00' and a<>3; -select uniqExact(_part), count() from xMM where d >= '2020-09-01 00:00:00' and d < '2020-10-01 00:00:00' and a<>3; - -DROP TABLE tMM; -DROP TABLE tDD; -DROP TABLE sDD; -DROP TABLE xMM; - - From a8618c96f798c36ae3ea07688dc35ff604d87b9c Mon Sep 17 00:00:00 2001 From: Denis Zhuravlev Date: Sat, 26 Sep 2020 21:02:18 -0300 Subject: [PATCH 116/321] fix test (UTC timezone specified) more UTC --- tests/queries/0_stateless/01508_partition_pruning.queries | 4 ++-- tests/queries/0_stateless/01508_partition_pruning.reference | 4 ++-- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/tests/queries/0_stateless/01508_partition_pruning.queries b/tests/queries/0_stateless/01508_partition_pruning.queries index a30b541c3bc..3773e907c53 100644 --- a/tests/queries/0_stateless/01508_partition_pruning.queries +++ b/tests/queries/0_stateless/01508_partition_pruning.queries @@ -88,8 +88,8 @@ select uniqExact(_part), count() from sDD; select uniqExact(_part), count() from sDD where toYYYYMM(toDateTime(intDiv(d,1000),'UTC')-1)+1 = 202010; select uniqExact(_part), count() from sDD where toYYYYMM(toDateTime(intDiv(d,1000),'UTC')-1) = 202010; select uniqExact(_part), count() from sDD where toYYYYMM(toDateTime(intDiv(d,1000),'UTC')-1) = 202110; -select uniqExact(_part), count() from sDD where toYYYYMM(toDateTime(intDiv(d,1000),'UTC'))+1 > 202009 and toStartOfDay(toDateTime(intDiv(d,1000))) < '2020-10-02 00:00:00'; -select uniqExact(_part), count() from sDD where toYYYYMM(toDateTime(intDiv(d,1000),'UTC'))+1 > 202009 and toDateTime(intDiv(d,1000)) < '2020-10-01 00:00:00'; +select uniqExact(_part), count() from sDD where toYYYYMM(toDateTime(intDiv(d,1000),'UTC'))+1 > 202009 and toStartOfDay(toDateTime(intDiv(d,1000),'UTC')) < toDateTime('2020-10-02 00:00:00','UTC'); +select uniqExact(_part), count() from sDD where toYYYYMM(toDateTime(intDiv(d,1000),'UTC'))+1 > 202009 and toDateTime(intDiv(d,1000),'UTC') < toDateTime('2020-10-01 00:00:00','UTC'); select uniqExact(_part), count() from sDD where d >= 1598918400000; select uniqExact(_part), count() from sDD where d >= 1598918400000 and toYYYYMM(toDateTime(intDiv(d,1000),'UTC')-1) < 202010; diff --git a/tests/queries/0_stateless/01508_partition_pruning.reference b/tests/queries/0_stateless/01508_partition_pruning.reference index 97a27f6fa8b..0cc40d23b41 100644 --- a/tests/queries/0_stateless/01508_partition_pruning.reference +++ b/tests/queries/0_stateless/01508_partition_pruning.reference @@ -161,11 +161,11 @@ select uniqExact(_part), count() from sDD where toYYYYMM(toDateTime(intDiv(d,100 0 0 Selected 0 parts by partition key, 0 parts by primary key, 0 marks by primary key, 0 marks to read from 0 ranges -select uniqExact(_part), count() from sDD where toYYYYMM(toDateTime(intDiv(d,1000),'UTC'))+1 > 202009 and toStartOfDay(toDateTime(intDiv(d,1000))) < '2020-10-02 00:00:00'; +select uniqExact(_part), count() from sDD where toYYYYMM(toDateTime(intDiv(d,1000),'UTC'))+1 > 202009 and toStartOfDay(toDateTime(intDiv(d,1000),'UTC')) < toDateTime('2020-10-02 00:00:00','UTC'); 3 11440 Selected 3 parts by partition key, 3 parts by primary key, 3 marks by primary key, 3 marks to read from 3 ranges -select uniqExact(_part), count() from sDD where toYYYYMM(toDateTime(intDiv(d,1000),'UTC'))+1 > 202009 and toDateTime(intDiv(d,1000)) < '2020-10-01 00:00:00'; +select uniqExact(_part), count() from sDD where toYYYYMM(toDateTime(intDiv(d,1000),'UTC'))+1 > 202009 and toDateTime(intDiv(d,1000),'UTC') < toDateTime('2020-10-01 00:00:00','UTC'); 2 10000 Selected 2 parts by partition key, 2 parts by primary key, 2 marks by primary key, 2 marks to read from 2 ranges From 6ad48ade830da8a03e54698d2a9d3ac7aa6ecad2 Mon Sep 17 00:00:00 2001 From: hexiaoting Date: Sun, 27 Sep 2020 16:42:04 +0800 Subject: [PATCH 117/321] fix bug for building query_db_generator.cpp --- utils/db-generator/query_db_generator.cpp | 1 + 1 file changed, 1 insertion(+) diff --git a/utils/db-generator/query_db_generator.cpp b/utils/db-generator/query_db_generator.cpp index 16aa8aa7769..88418e9b6c3 100644 --- a/utils/db-generator/query_db_generator.cpp +++ b/utils/db-generator/query_db_generator.cpp @@ -1,3 +1,4 @@ +#include #include #include #include From 3bc8d9daa9799d816d1089c4972ff2d1c9dcdb16 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Sun, 27 Sep 2020 14:10:45 +0300 Subject: [PATCH 118/321] fix sync --- src/Common/Macros.cpp | 11 ++++++----- src/Common/Macros.h | 2 +- 2 files changed, 7 insertions(+), 6 deletions(-) diff --git a/src/Common/Macros.cpp b/src/Common/Macros.cpp index b98daa0ec78..b8e25499c0b 100644 --- a/src/Common/Macros.cpp +++ b/src/Common/Macros.cpp @@ -22,11 +22,12 @@ Macros::Macros(const Poco::Util::AbstractConfiguration & config, const String & macros[key] = config.getString(root_key + "." + key); if (key == "database" || key == "table" || key == "uuid") { - LOG_WARNING(log, - "Config file contains '{}' macro. This macro has special meaning " - "and it's explicit definition is not recommended. Implicit unfolding for " - "'database', 'table' and 'uuid' macros will be disabled.", - key); + if (log) + LOG_WARNING(log, + "Config file contains '{}' macro. This macro has special meaning " + "and it's explicit definition is not recommended. Implicit unfolding for " + "'database', 'table' and 'uuid' macros will be disabled.", + key); enable_special_macros = false; } } diff --git a/src/Common/Macros.h b/src/Common/Macros.h index c04405a83de..3082452e297 100644 --- a/src/Common/Macros.h +++ b/src/Common/Macros.h @@ -26,7 +26,7 @@ class Macros { public: Macros() = default; - Macros(const Poco::Util::AbstractConfiguration & config, const String & key, Poco::Logger * log); + Macros(const Poco::Util::AbstractConfiguration & config, const String & key, Poco::Logger * log = nullptr); struct MacroExpansionInfo { From 0dd244126d1213fd5cd913319f4a779b2b34d4e0 Mon Sep 17 00:00:00 2001 From: Denny Crane Date: Sun, 27 Sep 2020 10:03:13 -0300 Subject: [PATCH 119/321] Update 01508_partition_pruning.sh --- tests/queries/0_stateless/01508_partition_pruning.sh | 7 +++++-- 1 file changed, 5 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/01508_partition_pruning.sh b/tests/queries/0_stateless/01508_partition_pruning.sh index e155431380f..c886946c7d9 100755 --- a/tests/queries/0_stateless/01508_partition_pruning.sh +++ b/tests/queries/0_stateless/01508_partition_pruning.sh @@ -1,10 +1,13 @@ #!/usr/bin/env bash -#-------------------------------------------- +#------------------------------------------------------------------------------------------- # Description of test result: # Test the correctness of the partition # pruning -#-------------------------------------------- +# +# Script executes queries from a file 01508_partition_pruning.queries (1 line = 1 query) +# Queries are started with 'select' (but NOT with 'SELECT') are executed with log_level=debug +#------------------------------------------------------------------------------------------- CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) . "$CURDIR"/../shell_config.sh From 74865398628f12a113cfb93152044c224fee7c45 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Sun, 27 Sep 2020 16:25:28 +0300 Subject: [PATCH 120/321] try fix test --- .../__init__.py | 0 .../configs/config.d/text_log.xml | 0 tests/integration/test_text_log_level/test.py | 33 ++++++++ tests/integration/test_text_log_table/test.py | 57 -------------- .../01473_event_time_microseconds.reference | 2 + .../01473_event_time_microseconds.sql | 77 +++++++------------ tests/queries/skip_list.json | 5 ++ 7 files changed, 66 insertions(+), 108 deletions(-) rename tests/integration/{test_text_log_table => test_text_log_level}/__init__.py (100%) rename tests/integration/{test_text_log_table => test_text_log_level}/configs/config.d/text_log.xml (100%) create mode 100644 tests/integration/test_text_log_level/test.py delete mode 100644 tests/integration/test_text_log_table/test.py diff --git a/tests/integration/test_text_log_table/__init__.py b/tests/integration/test_text_log_level/__init__.py similarity index 100% rename from tests/integration/test_text_log_table/__init__.py rename to tests/integration/test_text_log_level/__init__.py diff --git a/tests/integration/test_text_log_table/configs/config.d/text_log.xml b/tests/integration/test_text_log_level/configs/config.d/text_log.xml similarity index 100% rename from tests/integration/test_text_log_table/configs/config.d/text_log.xml rename to tests/integration/test_text_log_level/configs/config.d/text_log.xml diff --git a/tests/integration/test_text_log_level/test.py b/tests/integration/test_text_log_level/test.py new file mode 100644 index 00000000000..44679481266 --- /dev/null +++ b/tests/integration/test_text_log_level/test.py @@ -0,0 +1,33 @@ +# pylint: disable=unused-argument +# pylint: disable=redefined-outer-name + +import pytest +from helpers.client import QueryRuntimeException +from helpers.cluster import ClickHouseCluster + +cluster = ClickHouseCluster(__file__) + +node = cluster.add_instance('node', main_configs=["configs/config.d/text_log.xml"]) + + +@pytest.fixture(scope='module') +def start_cluster(): + try: + cluster.start() + + yield cluster + finally: + cluster.shutdown() + + +def test_basic(start_cluster): + with pytest.raises(QueryRuntimeException): + # generates log with "Error" level + node.query('SELECT * FROM no_such_table') + + node.query('SYSTEM FLUSH LOGS') + + assert int(node.query("SELECT count() FROM system.text_log WHERE level = 'Trace'")) == 0 + assert int(node.query("SELECT count() FROM system.text_log WHERE level = 'Debug'")) == 0 + assert int(node.query("SELECT count() FROM system.text_log WHERE level = 'Information'")) >= 1 + assert int(node.query("SELECT count() FROM system.text_log WHERE level = 'Error'")) >= 1 diff --git a/tests/integration/test_text_log_table/test.py b/tests/integration/test_text_log_table/test.py deleted file mode 100644 index 60c2e35893d..00000000000 --- a/tests/integration/test_text_log_table/test.py +++ /dev/null @@ -1,57 +0,0 @@ -# pylint: disable=unused-argument -# pylint: disable=redefined-outer-name - -import pytest -from helpers.client import QueryRuntimeException -from helpers.cluster import ClickHouseCluster - -cluster = ClickHouseCluster(__file__) - -node = cluster.add_instance('node', main_configs=["configs/config.d/text_log.xml"]) - - -@pytest.fixture(scope='module') -def start_cluster(): - try: - cluster.start() - - yield cluster - finally: - cluster.shutdown() - - -def test_basic(start_cluster): - with pytest.raises(QueryRuntimeException): - # generate log with "Error" level - node.query('SELECT * FROM no_such_table') - - node.query('SYSTEM FLUSH LOGS') - - assert int(node.query("SELECT count() FROM system.text_log WHERE level = 'Trace'")) == 0 - assert int(node.query("SELECT count() FROM system.text_log WHERE level = 'Debug'")) == 0 - assert int(node.query("SELECT count() FROM system.text_log WHERE level = 'Information'")) >= 1 - assert int(node.query("SELECT count() FROM system.text_log WHERE level = 'Error'")) >= 1 - -# compare the event_time and event_time_microseconds and test -# that they are exactly equal upto their seconds parts. -def test_field_event_time_microseconds(start_cluster): - with pytest.raises(QueryRuntimeException): - # generate log with "Error" level - node.query('SELECT * FROM no_such_table') - node.query('SYSTEM FLUSH LOGS') - equals_query = '''WITH ( - ( - SELECT event_time_microseconds - FROM system.text_log - ORDER BY event_time DESC - LIMIT 1 - ) AS time_with_microseconds, - ( - SELECT event_time - FROM system.text_log - ORDER BY event_time DESC - LIMIT 1 - ) AS time) - SELECT if(dateDiff('second', toDateTime(time_with_microseconds), toDateTime(time)) = 0, 'ok', 'fail') - ''' - assert 'ok\n' in node.query(equals_query) diff --git a/tests/queries/0_stateless/01473_event_time_microseconds.reference b/tests/queries/0_stateless/01473_event_time_microseconds.reference index 6d95c8c1b81..b5533c11268 100644 --- a/tests/queries/0_stateless/01473_event_time_microseconds.reference +++ b/tests/queries/0_stateless/01473_event_time_microseconds.reference @@ -6,3 +6,5 @@ ok ok 01473_query_thread_log_table_event_start_time_microseconds_test ok +01473_text_log_table_event_start_time_microseconds_test +ok diff --git a/tests/queries/0_stateless/01473_event_time_microseconds.sql b/tests/queries/0_stateless/01473_event_time_microseconds.sql index e4bc3b29655..2e536bba7ac 100644 --- a/tests/queries/0_stateless/01473_event_time_microseconds.sql +++ b/tests/queries/0_stateless/01473_event_time_microseconds.sql @@ -4,79 +4,54 @@ -- Refer: tests/integration/test_asynchronous_metric_log_table. SET log_queries = 1; +SET query_profiler_real_time_period_ns = 100000000; +-- a long enough query to trigger the query profiler and to record trace log +SELECT sleep(2) FORMAT Null; +SET query_profiler_real_time_period_ns = 1000000000; +SYSTEM FLUSH LOGS; SELECT '01473_metric_log_table_event_start_time_microseconds_test'; -SYSTEM FLUSH LOGS; -- query assumes that the event_time field is accurate. WITH ( - ( - SELECT event_time_microseconds + SELECT event_time_microseconds, event_time FROM system.metric_log ORDER BY event_time DESC LIMIT 1 - ) AS time_with_microseconds, - ( - SELECT event_time - FROM system.metric_log - ORDER BY event_time DESC - LIMIT 1 - ) AS time) -SELECT if(dateDiff('second', toDateTime(time_with_microseconds), toDateTime(time)) = 0, 'ok', 'fail'); + ) AS time +SELECT if(dateDiff('second', toDateTime(time.1), toDateTime(time.2)) = 0, 'ok', toString(time)); SELECT '01473_trace_log_table_event_start_time_microseconds_test'; -SET log_queries = 1; -SET query_profiler_real_time_period_ns = 10000000; -SET query_profiler_cpu_time_period_ns = 10000000; --- a long enough query to trigger the query profiler and to record trace log -SELECT count() FROM numbers(1000000000) FORMAT Null; -SELECT sleep(2) FORMAT Null; -SYSTEM FLUSH LOGS; -SELECT sleep(2) FORMAT Null; WITH ( - ( - SELECT event_time_microseconds + SELECT event_time_microseconds, event_time FROM system.trace_log ORDER BY event_time DESC LIMIT 1 - ) AS time_with_microseconds, - ( - SELECT event_time - FROM system.trace_log - ORDER BY event_time DESC - LIMIT 1 - ) AS t) -SELECT if(dateDiff('second', toDateTime(time_with_microseconds), toDateTime(t)) = 0, 'ok', 'fail'); -- success + ) AS time +SELECT if(dateDiff('second', toDateTime(time.1), toDateTime(time.2)) = 0, 'ok', toString(time)); SELECT '01473_query_log_table_event_start_time_microseconds_test'; -SYSTEM FLUSH LOGS; WITH ( - ( - SELECT event_time_microseconds + SELECT event_time_microseconds, event_time FROM system.query_log ORDER BY event_time DESC LIMIT 1 - ) AS time_with_microseconds, - ( - SELECT event_time - FROM system.query_log - ORDER BY event_time DESC - LIMIT 1 - ) AS time) -SELECT if(dateDiff('second', toDateTime(time_with_microseconds), toDateTime(time)) = 0, 'ok', 'fail'); -- success + ) AS time +SELECT if(dateDiff('second', toDateTime(time.1), toDateTime(time.2)) = 0, 'ok', toString(time)); SELECT '01473_query_thread_log_table_event_start_time_microseconds_test'; -SYSTEM FLUSH LOGS; WITH ( - ( - SELECT event_time_microseconds + SELECT event_time_microseconds, event_time FROM system.query_thread_log ORDER BY event_time DESC LIMIT 1 - ) AS time_with_microseconds, - ( - SELECT event_time - FROM system.query_thread_log - ORDER BY event_time DESC - LIMIT 1 - ) AS time) -SELECT if(dateDiff('second', toDateTime(time_with_microseconds), toDateTime(time)) = 0, 'ok', 'fail'); -- success + ) AS time +SELECT if(dateDiff('second', toDateTime(time.1), toDateTime(time.2)) = 0, 'ok', toString(time)); + +SELECT '01473_text_log_table_event_start_time_microseconds_test'; +WITH ( + SELECT event_time_microseconds, event_time + FROM system.query_thread_log + ORDER BY event_time DESC + LIMIT 1 + ) AS time +SELECT if(dateDiff('second', toDateTime(time.1), toDateTime(time.2)) = 0, 'ok', toString(time)); diff --git a/tests/queries/skip_list.json b/tests/queries/skip_list.json index 26e5bbf78cf..ac8bd77fcac 100644 --- a/tests/queries/skip_list.json +++ b/tests/queries/skip_list.json @@ -17,6 +17,7 @@ "00151_replace_partition_with_different_granularity", "00157_cache_dictionary", "01193_metadata_loading", + "01473_event_time_microseconds", "01474_executable_dictionary" /// informational stderr from sanitizer at start ], "address-sanitizer": [ @@ -25,6 +26,7 @@ "memory_profiler", "odbc_roundtrip", "01103_check_cpu_instructions_at_startup", + "01473_event_time_microseconds", "01193_metadata_loading" ], "ub-sanitizer": [ @@ -33,6 +35,7 @@ "memory_profiler", "01103_check_cpu_instructions_at_startup", "00900_orc_load", + "01473_event_time_microseconds", "01193_metadata_loading" ], "memory-sanitizer": [ @@ -43,6 +46,7 @@ "01086_odbc_roundtrip", /// can't pass because odbc libraries are not instrumented "00877_memory_limit_for_new_delete", /// memory limits don't work correctly under msan because it replaces malloc/free "01114_mysql_database_engine_segfault", /// it fails in _nss_files_parse_servent while using NSS from GLibc to authenticate (need to get rid of it) + "01473_event_time_microseconds", "01193_metadata_loading" ], "debug-build": [ @@ -57,6 +61,7 @@ "01037_polygon_dicts_", "hyperscan", "01193_metadata_loading", + "01473_event_time_microseconds", "01396_inactive_replica_cleanup_nodes" ], "unbundled-build": [ From a1bc246d53ad4b617b512d4ea8735207c3d19701 Mon Sep 17 00:00:00 2001 From: Evgeniia Sudarikova Date: Sun, 27 Sep 2020 21:08:17 +0300 Subject: [PATCH 121/321] edited after review --- .../sql-reference/functions/type-conversion-functions.md | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/docs/en/sql-reference/functions/type-conversion-functions.md b/docs/en/sql-reference/functions/type-conversion-functions.md index 4859c8a7aad..bbf8b7adc8e 100644 --- a/docs/en/sql-reference/functions/type-conversion-functions.md +++ b/docs/en/sql-reference/functions/type-conversion-functions.md @@ -739,14 +739,14 @@ Converts arbitrary expressions into a string via given format. formatRow(format, x, y, ...) ``` -**Parameters** +**Parameters** -- `format` — text format. For example [CSV](../../interfaces/formats.md#csv), [TSV](../../interfaces/formats.md#tabseparated). -- `x`,`y`, ... — expressions. +- `format` — Text format. For example, [CSV](../../interfaces/formats.md#csv), [TSV](../../interfaces/formats.md#tabseparated). +- `x`,`y`, ... — Expressions. **Returned value** -- expressions converted to the `format`. +- A formatted string terminated with the new line symbol. **Example** From cc7d6ed77590220944516f098b3addbf12229cd5 Mon Sep 17 00:00:00 2001 From: Evgeniia Sudarikova Date: Sun, 27 Sep 2020 21:24:34 +0300 Subject: [PATCH 122/321] add RU description --- .../functions/type-conversion-functions.md | 40 +++++++++++++++++++ 1 file changed, 40 insertions(+) diff --git a/docs/ru/sql-reference/functions/type-conversion-functions.md b/docs/ru/sql-reference/functions/type-conversion-functions.md index 9ceb105095d..f4c811c0656 100644 --- a/docs/ru/sql-reference/functions/type-conversion-functions.md +++ b/docs/ru/sql-reference/functions/type-conversion-functions.md @@ -717,4 +717,44 @@ SELECT toLowCardinality('1') └───────────────────────┘ ``` +## formatRow {#formatrow} + +Преобразует произвольные выражения в строку заданного формата. + +**Синтаксис** + +``` sql +formatRow(format, x, y, ...) +``` + +**Параметры** + +- `format` — Текстовый формат. Например, [CSV](../../interfaces/formats.md#csv), [TSV](../../interfaces/formats.md#tabseparated). +- `x`,`y`, ... — Выражения. + +**Возвращаемое значение** + +- Отформатированная строка, которая заканчивается символом новой строки. + +**Пример** + +Запрос: + +``` sql +SELECT formatRow('CSV', number, 'good') +FROM numbers(3) +``` + +Ответ: + +``` text +┌─formatRow('CSV', number, 'good')─┐ +│ 0,"good" + │ +│ 1,"good" + │ +│ 2,"good" + │ +└──────────────────────────────────┘ +``` [Оригинальная статья](https://clickhouse.tech/docs/ru/query_language/functions/type_conversion_functions/) From b800a7fdf140aae8fd0798d3ec1404b8a6129a87 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 28 Sep 2020 01:09:32 +0300 Subject: [PATCH 123/321] Add support for "Raw" column format for Regexp format --- .../Formats/Impl/RegexpRowInputFormat.cpp | 8 ++++++++ .../01508_format_regexp_raw.reference | 1 + .../0_stateless/01508_format_regexp_raw.sh | 17 +++++++++++++++++ 3 files changed, 26 insertions(+) create mode 100644 tests/queries/0_stateless/01508_format_regexp_raw.reference create mode 100755 tests/queries/0_stateless/01508_format_regexp_raw.sh diff --git a/src/Processors/Formats/Impl/RegexpRowInputFormat.cpp b/src/Processors/Formats/Impl/RegexpRowInputFormat.cpp index 7763de1642d..d193a502f2e 100644 --- a/src/Processors/Formats/Impl/RegexpRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/RegexpRowInputFormat.cpp @@ -51,6 +51,8 @@ RegexpRowInputFormat::ColumnFormat RegexpRowInputFormat::stringToFormat(const St return ColumnFormat::Csv; if (format == "JSON") return ColumnFormat::Json; + if (format == "Raw") + return ColumnFormat::Raw; throw Exception("Unsupported column format \"" + format + "\".", ErrorCodes::BAD_ARGUMENTS); } @@ -88,6 +90,12 @@ bool RegexpRowInputFormat::readField(size_t index, MutableColumns & columns) else type->deserializeAsTextJSON(*columns[index], field_buf, format_settings); break; + case ColumnFormat::Raw: + if (parse_as_nullable) + read = DataTypeNullable::deserializeWholeText(*columns[index], field_buf, format_settings, type); + else + type->deserializeAsWholeText(*columns[index], field_buf, format_settings); + break; default: break; } diff --git a/tests/queries/0_stateless/01508_format_regexp_raw.reference b/tests/queries/0_stateless/01508_format_regexp_raw.reference new file mode 100644 index 00000000000..f0e7d928857 --- /dev/null +++ b/tests/queries/0_stateless/01508_format_regexp_raw.reference @@ -0,0 +1 @@ +abc\\ Hello, world! diff --git a/tests/queries/0_stateless/01508_format_regexp_raw.sh b/tests/queries/0_stateless/01508_format_regexp_raw.sh new file mode 100755 index 00000000000..699fca1be61 --- /dev/null +++ b/tests/queries/0_stateless/01508_format_regexp_raw.sh @@ -0,0 +1,17 @@ +#!/usr/bin/env bash + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +. "$CURDIR"/../shell_config.sh + +${CLICKHOUSE_CLIENT} -n --query " +DROP TABLE IF EXISTS t; +CREATE TABLE t (a String, b String) ENGINE = Memory; +" + +${CLICKHOUSE_CLIENT} --format_regexp_escaping_rule 'Raw' --format_regexp '^(.+?) separator (.+?)$' --query ' +INSERT INTO t FORMAT Regexp abc\ separator Hello, world!' + +${CLICKHOUSE_CLIENT} -n --query " +SELECT * FROM t; +DROP TABLE t; +" From a7ddd8489a6c040cdcd9f280e6a500ab8e0c2dd1 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 28 Sep 2020 03:11:19 +0300 Subject: [PATCH 124/321] Added format RawBLOB --- src/Formats/FormatFactory.cpp | 6 ++ .../Impl/JSONAsStringRowInputFormat.cpp | 17 ++++-- .../Formats/Impl/RawBLOBRowInputFormat.cpp | 55 +++++++++++++++++++ .../Formats/Impl/RawBLOBRowInputFormat.h | 24 ++++++++ .../Formats/Impl/RawBLOBRowOutputFormat.cpp | 38 +++++++++++++ .../Formats/Impl/RawBLOBRowOutputFormat.h | 41 ++++++++++++++ .../01509_format_raw_blob.reference | 2 + .../0_stateless/01509_format_raw_blob.sh | 18 ++++++ 8 files changed, 196 insertions(+), 5 deletions(-) create mode 100644 src/Processors/Formats/Impl/RawBLOBRowInputFormat.cpp create mode 100644 src/Processors/Formats/Impl/RawBLOBRowInputFormat.h create mode 100644 src/Processors/Formats/Impl/RawBLOBRowOutputFormat.cpp create mode 100644 src/Processors/Formats/Impl/RawBLOBRowOutputFormat.h create mode 100644 tests/queries/0_stateless/01509_format_raw_blob.reference create mode 100755 tests/queries/0_stateless/01509_format_raw_blob.sh diff --git a/src/Formats/FormatFactory.cpp b/src/Formats/FormatFactory.cpp index 522149d3cfd..e9f37ff9752 100644 --- a/src/Formats/FormatFactory.cpp +++ b/src/Formats/FormatFactory.cpp @@ -367,6 +367,8 @@ void registerInputFormatProcessorArrow(FormatFactory & factory); void registerOutputFormatProcessorArrow(FormatFactory & factory); void registerInputFormatProcessorAvro(FormatFactory & factory); void registerOutputFormatProcessorAvro(FormatFactory & factory); +void registerInputFormatProcessorRawBLOB(FormatFactory & factory); +void registerOutputFormatProcessorRawBLOB(FormatFactory & factory); /// Output only (presentational) formats. @@ -426,6 +428,9 @@ FormatFactory::FormatFactory() registerOutputFormatProcessorTemplate(*this); registerInputFormatProcessorMsgPack(*this); registerOutputFormatProcessorMsgPack(*this); + registerInputFormatProcessorRawBLOB(*this); + registerOutputFormatProcessorRawBLOB(*this); + #if !defined(ARCADIA_BUILD) registerInputFormatProcessorORC(*this); registerOutputFormatProcessorORC(*this); @@ -456,6 +461,7 @@ FormatFactory::FormatFactory() registerInputFormatProcessorRegexp(*this); registerInputFormatProcessorJSONAsString(*this); registerInputFormatProcessorLineAsString(*this); + #if !defined(ARCADIA_BUILD) registerInputFormatProcessorCapnProto(*this); #endif diff --git a/src/Processors/Formats/Impl/JSONAsStringRowInputFormat.cpp b/src/Processors/Formats/Impl/JSONAsStringRowInputFormat.cpp index 42fa764f011..bc57803152f 100644 --- a/src/Processors/Formats/Impl/JSONAsStringRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/JSONAsStringRowInputFormat.cpp @@ -1,5 +1,7 @@ #include #include +#include +#include #include #include @@ -8,17 +10,22 @@ namespace DB namespace ErrorCodes { - extern const int LOGICAL_ERROR; + extern const int BAD_ARGUMENTS; extern const int INCORRECT_DATA; } JSONAsStringRowInputFormat::JSONAsStringRowInputFormat(const Block & header_, ReadBuffer & in_, Params params_) : IRowInputFormat(header_, in_, std::move(params_)), buf(in) { - if (header_.columns() > 1 || header_.getDataTypes()[0]->getTypeId() != TypeIndex::String) - { - throw Exception("This input format is only suitable for tables with a single column of type String.", ErrorCodes::LOGICAL_ERROR); - } + if (header_.columns() > 1) + throw Exception(ErrorCodes::BAD_ARGUMENTS, + "This input format is only suitable for tables with a single column of type String but the number of columns is {}", + header_.columns()); + + if (!isString(removeNullable(removeLowCardinality(header_.getByPosition(0).type)))) + throw Exception(ErrorCodes::BAD_ARGUMENTS, + "This input format is only suitable for tables with a single column of type String but the column type is {}", + header_.getByPosition(0).type->getName()); } void JSONAsStringRowInputFormat::resetParser() diff --git a/src/Processors/Formats/Impl/RawBLOBRowInputFormat.cpp b/src/Processors/Formats/Impl/RawBLOBRowInputFormat.cpp new file mode 100644 index 00000000000..bd9eaada05e --- /dev/null +++ b/src/Processors/Formats/Impl/RawBLOBRowInputFormat.cpp @@ -0,0 +1,55 @@ +#include +#include +#include +#include +#include + + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int BAD_ARGUMENTS; +} + +RawBLOBRowInputFormat::RawBLOBRowInputFormat(const Block & header_, ReadBuffer & in_, Params params_) + : IRowInputFormat(header_, in_, std::move(params_)) +{ + if (header_.columns() > 1) + throw Exception(ErrorCodes::BAD_ARGUMENTS, + "This input format is only suitable for tables with a single column of type String but the number of columns is {}", + header_.columns()); + + if (!isString(removeNullable(removeLowCardinality(header_.getByPosition(0).type)))) + throw Exception(ErrorCodes::BAD_ARGUMENTS, + "This input format is only suitable for tables with a single column of type String but the column type is {}", + header_.getByPosition(0).type->getName()); +} + +bool RawBLOBRowInputFormat::readRow(MutableColumns & columns, RowReadExtension &) +{ + if (in.eof()) + return false; + + /// One excessive copy. + String blob; + readStringUntilEOF(blob, in); + columns.at(0)->insertData(blob.data(), blob.size()); + return false; +} + +void registerInputFormatProcessorRawBLOB(FormatFactory & factory) +{ + factory.registerInputFormatProcessor("RawBLOB", []( + ReadBuffer & buf, + const Block & sample, + const RowInputFormatParams & params, + const FormatSettings &) + { + return std::make_shared(sample, buf, params); + }); +} + +} + diff --git a/src/Processors/Formats/Impl/RawBLOBRowInputFormat.h b/src/Processors/Formats/Impl/RawBLOBRowInputFormat.h new file mode 100644 index 00000000000..fd2c849687a --- /dev/null +++ b/src/Processors/Formats/Impl/RawBLOBRowInputFormat.h @@ -0,0 +1,24 @@ +#pragma once + +#include + + +namespace DB +{ + +class ReadBuffer; + +/// This format slurps all input data into single value. +/// This format can only parse a table with single field of type String or similar. + +class RawBLOBRowInputFormat : public IRowInputFormat +{ +public: + RawBLOBRowInputFormat(const Block & header_, ReadBuffer & in_, Params params_); + + bool readRow(MutableColumns & columns, RowReadExtension &) override; + String getName() const override { return "RawBLOBRowInputFormat"; } +}; + +} + diff --git a/src/Processors/Formats/Impl/RawBLOBRowOutputFormat.cpp b/src/Processors/Formats/Impl/RawBLOBRowOutputFormat.cpp new file mode 100644 index 00000000000..786edce5edc --- /dev/null +++ b/src/Processors/Formats/Impl/RawBLOBRowOutputFormat.cpp @@ -0,0 +1,38 @@ +#include +#include +#include + +namespace DB +{ + + +RawBLOBRowOutputFormat::RawBLOBRowOutputFormat( + WriteBuffer & out_, + const Block & header_, + FormatFactory::WriteCallback callback) + : IRowOutputFormat(header_, out_, callback) +{ +} + + +void RawBLOBRowOutputFormat::writeField(const IColumn & column, const IDataType &, size_t row_num) +{ + StringRef value = column.getDataAt(row_num); + out.write(value.data, value.size); +} + + +void registerOutputFormatProcessorRawBLOB(FormatFactory & factory) +{ + factory.registerOutputFormatProcessor("RawBLOB", []( + WriteBuffer & buf, + const Block & sample, + FormatFactory::WriteCallback callback, + const FormatSettings &) + { + return std::make_shared(buf, sample, callback); + }); +} + +} + diff --git a/src/Processors/Formats/Impl/RawBLOBRowOutputFormat.h b/src/Processors/Formats/Impl/RawBLOBRowOutputFormat.h new file mode 100644 index 00000000000..8062092f5cd --- /dev/null +++ b/src/Processors/Formats/Impl/RawBLOBRowOutputFormat.h @@ -0,0 +1,41 @@ +#pragma once + +#include +#include + + +namespace DB +{ + +class WriteBuffer; + + +/** This format only allows to output one column of type String or similar. + * It is output as raw bytes without any delimiters or escaping. + * + * The difference between RawBLOB and TSVRaw: + * - only single column dataset is supported; + * - data is output in binary; + * - no newline at the end of each value. + * + * The difference between RawBLOB and RowBinary: + * - only single column dataset is supported; + * - strings are output without their lengths. + * + * If you are output more than one value, the output format is ambiguous and you may not be able to read data back. + */ +class RawBLOBRowOutputFormat : public IRowOutputFormat +{ +public: + RawBLOBRowOutputFormat( + WriteBuffer & out_, + const Block & header_, + FormatFactory::WriteCallback callback); + + String getName() const override { return "RawBLOBRowOutputFormat"; } + + void writeField(const IColumn & column, const IDataType &, size_t row_num) override; +}; + +} + diff --git a/tests/queries/0_stateless/01509_format_raw_blob.reference b/tests/queries/0_stateless/01509_format_raw_blob.reference new file mode 100644 index 00000000000..dfa8f538e67 --- /dev/null +++ b/tests/queries/0_stateless/01509_format_raw_blob.reference @@ -0,0 +1,2 @@ +96b229180107fd2d23fd0a2ef9326701 - +96b229180107fd2d23fd0a2ef9326701 - diff --git a/tests/queries/0_stateless/01509_format_raw_blob.sh b/tests/queries/0_stateless/01509_format_raw_blob.sh new file mode 100755 index 00000000000..68d3844d727 --- /dev/null +++ b/tests/queries/0_stateless/01509_format_raw_blob.sh @@ -0,0 +1,18 @@ +#!/usr/bin/env bash + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +. "$CURDIR"/../shell_config.sh + +${CLICKHOUSE_CLIENT} -n --query " +DROP TABLE IF EXISTS t; +CREATE TABLE t (a LowCardinality(Nullable(String))) ENGINE = Memory; +" + +${CLICKHOUSE_CLIENT} --query "INSERT INTO t FORMAT RawBLOB" < ${BASH_SOURCE[0]} + +cat ${BASH_SOURCE[0]} | md5sum +${CLICKHOUSE_CLIENT} -n --query "SELECT * FROM t FORMAT RawBLOB" | md5sum + +${CLICKHOUSE_CLIENT} --query " +DROP TABLE t; +" From 79f510d63a54f9efc793373b2014bc9b3095c439 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 28 Sep 2020 03:22:50 +0300 Subject: [PATCH 125/321] Fix error in LowCardinality; add test --- src/DataTypes/DataTypeLowCardinality.cpp | 2 +- ..._format_regexp_raw_low_cardinality.reference | 1 + .../01510_format_regexp_raw_low_cardinality.sh | 17 +++++++++++++++++ 3 files changed, 19 insertions(+), 1 deletion(-) create mode 100644 tests/queries/0_stateless/01510_format_regexp_raw_low_cardinality.reference create mode 100755 tests/queries/0_stateless/01510_format_regexp_raw_low_cardinality.sh diff --git a/src/DataTypes/DataTypeLowCardinality.cpp b/src/DataTypes/DataTypeLowCardinality.cpp index b4f28a8853f..fe7ce7bf94a 100644 --- a/src/DataTypes/DataTypeLowCardinality.cpp +++ b/src/DataTypes/DataTypeLowCardinality.cpp @@ -774,7 +774,7 @@ void DataTypeLowCardinality::deserializeTextQuoted(IColumn & column, ReadBuffer void DataTypeLowCardinality::deserializeWholeText(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const { - deserializeImpl(column, &IDataType::deserializeAsTextEscaped, istr, settings); + deserializeImpl(column, &IDataType::deserializeAsWholeText, istr, settings); } void DataTypeLowCardinality::serializeTextCSV(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const diff --git a/tests/queries/0_stateless/01510_format_regexp_raw_low_cardinality.reference b/tests/queries/0_stateless/01510_format_regexp_raw_low_cardinality.reference new file mode 100644 index 00000000000..f0e7d928857 --- /dev/null +++ b/tests/queries/0_stateless/01510_format_regexp_raw_low_cardinality.reference @@ -0,0 +1 @@ +abc\\ Hello, world! diff --git a/tests/queries/0_stateless/01510_format_regexp_raw_low_cardinality.sh b/tests/queries/0_stateless/01510_format_regexp_raw_low_cardinality.sh new file mode 100755 index 00000000000..0f65280e1ce --- /dev/null +++ b/tests/queries/0_stateless/01510_format_regexp_raw_low_cardinality.sh @@ -0,0 +1,17 @@ +#!/usr/bin/env bash + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +. "$CURDIR"/../shell_config.sh + +${CLICKHOUSE_CLIENT} -n --query " +DROP TABLE IF EXISTS t; +CREATE TABLE t (a String, b LowCardinality(Nullable(String))) ENGINE = Memory; +" + +${CLICKHOUSE_CLIENT} --format_regexp_escaping_rule 'Raw' --format_regexp '^(.+?) separator (.+?)$' --query ' +INSERT INTO t FORMAT Regexp abc\ separator Hello, world!' + +${CLICKHOUSE_CLIENT} -n --query " +SELECT * FROM t; +DROP TABLE t; +" From 57c6af479c392be61f80629cef38edb7ce042e73 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 28 Sep 2020 03:26:15 +0300 Subject: [PATCH 126/321] Fix "Arcadia" --- src/Processors/ya.make | 2 ++ 1 file changed, 2 insertions(+) diff --git a/src/Processors/ya.make b/src/Processors/ya.make index c69d274a70e..b2f8b9ba7c2 100644 --- a/src/Processors/ya.make +++ b/src/Processors/ya.make @@ -47,6 +47,8 @@ SRCS( Formats/Impl/PrettySpaceBlockOutputFormat.cpp Formats/Impl/ProtobufRowInputFormat.cpp Formats/Impl/ProtobufRowOutputFormat.cpp + Formats/Impl/RawBLOBRowInputFormat.cpp + Formats/Impl/RawBLOBRowOutputFormat.cpp Formats/Impl/RegexpRowInputFormat.cpp Formats/Impl/TabSeparatedRowInputFormat.cpp Formats/Impl/TabSeparatedRowOutputFormat.cpp From 07b931a41473481f7a4978109b48ec9610c25877 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 28 Sep 2020 04:21:40 +0300 Subject: [PATCH 127/321] Prepare for LLVM-11 --- CMakeLists.txt | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/CMakeLists.txt b/CMakeLists.txt index be1b6ac04f5..f4e230fbd93 100644 --- a/CMakeLists.txt +++ b/CMakeLists.txt @@ -173,7 +173,7 @@ endif () # Make sure the final executable has symbols exported set (CMAKE_EXE_LINKER_FLAGS "${CMAKE_EXE_LINKER_FLAGS} -rdynamic") -find_program (OBJCOPY_PATH NAMES "llvm-objcopy" "llvm-objcopy-10" "llvm-objcopy-9" "llvm-objcopy-8" "objcopy") +find_program (OBJCOPY_PATH NAMES "llvm-objcopy" "llvm-objcopy-11" "llvm-objcopy-10" "llvm-objcopy-9" "llvm-objcopy-8" "objcopy") if (OBJCOPY_PATH) message(STATUS "Using objcopy: ${OBJCOPY_PATH}.") @@ -313,7 +313,7 @@ if (COMPILER_CLANG) endif () # Always prefer llvm tools when using clang. For instance, we cannot use GNU ar when llvm LTO is enabled - find_program (LLVM_AR_PATH NAMES "llvm-ar" "llvm-ar-10" "llvm-ar-9" "llvm-ar-8") + find_program (LLVM_AR_PATH NAMES "llvm-ar" "llvm-ar-11" "llvm-ar-10" "llvm-ar-9" "llvm-ar-8") if (LLVM_AR_PATH) message(STATUS "Using llvm-ar: ${LLVM_AR_PATH}.") @@ -322,7 +322,7 @@ if (COMPILER_CLANG) message(WARNING "Cannot find llvm-ar. System ar will be used instead. It does not work with ThinLTO.") endif () - find_program (LLVM_RANLIB_PATH NAMES "llvm-ranlib" "llvm-ranlib-10" "llvm-ranlib-9" "llvm-ranlib-8") + find_program (LLVM_RANLIB_PATH NAMES "llvm-ranlib" "llvm-ranlib-11" "llvm-ranlib-10" "llvm-ranlib-9" "llvm-ranlib-8") if (LLVM_RANLIB_PATH) message(STATUS "Using llvm-ranlib: ${LLVM_RANLIB_PATH}.") From 64a431ae8b026bde8ccda498356482457eb5b6a1 Mon Sep 17 00:00:00 2001 From: Olga Revyakina Date: Mon, 28 Sep 2020 05:59:01 +0300 Subject: [PATCH 128/321] Custom settings docs in english. --- .../settings.md | 14 ++++++++ docs/en/operations/settings/index.md | 26 ++++++++++++++ .../functions/other-functions.md | 36 +++++++++++++++++++ 3 files changed, 76 insertions(+) diff --git a/docs/en/operations/server-configuration-parameters/settings.md b/docs/en/operations/server-configuration-parameters/settings.md index ee0373c70b4..fc150ee551d 100644 --- a/docs/en/operations/server-configuration-parameters/settings.md +++ b/docs/en/operations/server-configuration-parameters/settings.md @@ -66,6 +66,20 @@ If no conditions met for a data part, ClickHouse uses the `lz4` compression.
``` +## custom_settings_prefixes {#custom_settings_prefixes} + +List of prefixes for [custom settings](../../operations/settings/index.md#custom_settings). The prefixes must be separated with commas. + +**Example** + +```xml +custom_ +``` + +**See Also** + +- [Custom settings](../../operations/settings/index.md#custom_settings) + ## default\_database {#default-database} The default database. diff --git a/docs/en/operations/settings/index.md b/docs/en/operations/settings/index.md index 59394d77b11..cd483694521 100644 --- a/docs/en/operations/settings/index.md +++ b/docs/en/operations/settings/index.md @@ -28,4 +28,30 @@ Ways to configure settings, in order of priority: Settings that can only be made in the server config file are not covered in this section. +## Custom Settings {#custom_settings} + +In addition to the common [settings](../../operations/settings/settings.md), users can define custom settings. + +A custom setting name must begin with one of predefined prefixes. The list of these prefixes must be declared in the [custom_settings_prefixes](../../operations/server-configuration-parameters/settings.md#custom_settings_prefixes) parameter in the server configuration file. + +```xml +custom_ +``` + +To define a custom setting use `SET` command: + +```sql +SET custom_a = 123; +``` + +To get the current value of a custom setting use `getSetting()` function: + +```sql +SELECT getSetting('custom_a'); +``` + +**See Also** + +- [Server Configuration Settings](../../operations/server-configuration-parameters/settings.md) + [Original article](https://clickhouse.tech/docs/en/operations/settings/) diff --git a/docs/en/sql-reference/functions/other-functions.md b/docs/en/sql-reference/functions/other-functions.md index 1c059e9f97b..3d06bbfff9e 100644 --- a/docs/en/sql-reference/functions/other-functions.md +++ b/docs/en/sql-reference/functions/other-functions.md @@ -1491,4 +1491,40 @@ Result: ``` +## getSetting {#getSetting} + +Returns the current value of a [custom setting](../../operations/settings/index.md#custom_settings). + +**Syntax** + +```sql +getSetting('custom_setting'); +``` + +**Parameter** + +- `custom_setting` — The setting name. [String](../../sql-reference/data-types/string.md). + +**Returned value** + +- The setting current value. + +**Example** + +```sql +SET custom_a = 123; +SELECT getSetting('custom_a'); +``` + +**Result** + +``` +123 +``` + +**See Also** + +- [Custom Settings](../../operations/settings/index.md#custom_settings) + + [Original article](https://clickhouse.tech/docs/en/query_language/functions/other_functions/) From e428c5158d6fe0525b64efcb867db95c5c6f4cee Mon Sep 17 00:00:00 2001 From: Arthur Wong Date: Mon, 28 Sep 2020 16:12:14 +0800 Subject: [PATCH 129/321] Update apply-catboost-model.md --- docs/zh/guides/apply-catboost-model.md | 38 ++++++++++++++------------ 1 file changed, 20 insertions(+), 18 deletions(-) diff --git a/docs/zh/guides/apply-catboost-model.md b/docs/zh/guides/apply-catboost-model.md index 3657a947ad2..8368fde0d26 100644 --- a/docs/zh/guides/apply-catboost-model.md +++ b/docs/zh/guides/apply-catboost-model.md @@ -7,9 +7,11 @@ toc_title: "\u5E94\u7528CatBoost\u6A21\u578B" # 在ClickHouse中应用Catboost模型 {#applying-catboost-model-in-clickhouse} -[CatBoost](https://catboost.ai) 是一个自由和开源的梯度提升库开发 [Yandex](https://yandex.com/company/) 用于机器学习。 +[CatBoost](https://catboost.ai) 是一个用于机器学习的免费开源梯度提升开发库 [Yandex](https://yandex.com/company/) 。 + + +通过这篇指导,您将学会如何将预先从SQL推理出的运行模型作为训练好的模型应用到ClickHouse中去。 -通过此指令,您将学习如何通过从SQL运行模型推理在ClickHouse中应用预先训练好的模型。 在ClickHouse中应用CatBoost模型: @@ -18,18 +20,18 @@ toc_title: "\u5E94\u7528CatBoost\u6A21\u578B" 3. [将CatBoost集成到ClickHouse中](#integrate-catboost-into-clickhouse) (可选步骤)。 4. [从SQL运行模型推理](#run-model-inference). -有关训练CatBoost模型的详细信息,请参阅 [培训和应用模型](https://catboost.ai/docs/features/training.html#training). +有关训练CatBoost模型的详细信息,请参阅 [训练和使用模型](https://catboost.ai/docs/features/training.html#training). ## 先决条件 {#prerequisites} -如果你没有 [Docker](https://docs.docker.com/install/) 然而,安装它。 +请先安装好 [Docker](https://docs.docker.com/install/)。 !!! note "注" [Docker](https://www.docker.com) 是一个软件平台,允许您创建容器,将CatBoost和ClickHouse安装与系统的其余部分隔离。 在应用CatBoost模型之前: -**1.** 拉 [码头窗口映像](https://hub.docker.com/r/yandex/tutorial-catboost-clickhouse) 从注册表: +**1.** 从容器仓库拉取docker映像 (https://hub.docker.com/r/yandex/tutorial-catboost-clickhouse) : ``` bash $ docker pull yandex/tutorial-catboost-clickhouse @@ -126,15 +128,15 @@ FROM amazon_train CatBoost集成到ClickHouse步骤: -**1.** 构建评估库。 +**1.** 构建测试库文件。 -评估CatBoost模型的最快方法是编译 `libcatboostmodel.` 图书馆. 有关如何构建库的详细信息,请参阅 [CatBoost文件](https://catboost.ai/docs/concepts/c-plus-plus-api_dynamic-c-pluplus-wrapper.html). +测试CatBoost模型的最快方法是编译 `libcatboostmodel.` 库文件. 有关如何构建库文件的详细信息,请参阅 [CatBoost文件](https://catboost.ai/docs/concepts/c-plus-plus-api_dynamic-c-pluplus-wrapper.html). -**2.** 例如,在任何地方和任何名称创建一个新目录, `data` 并将创建的库放入其中。 Docker映像已经包含了库 `data/libcatboostmodel.so`. +**2.** 任意创建一个新目录, 如 `data` 并将创建的库文件放入其中。 Docker映像已经包含了库 `data/libcatboostmodel.so`. -**3.** 例如,在任何地方和任何名称为config model创建一个新目录, `models`. +**3.** 任意创建一个新目录来放配置模型, 如 `models`. -**4.** 创建具有任意名称的模型配置文件,例如, `models/amazon_model.xml`. +**4.** 任意创建一个模型配置文件,如 `models/amazon_model.xml`. **5.** 描述模型配置: @@ -153,7 +155,7 @@ CatBoost集成到ClickHouse步骤: ``` -**6.** 将CatBoost的路径和模型配置添加到ClickHouse配置: +**6.** 将CatBoost库文件的路径和模型配置添加到ClickHouse配置: ``` xml @@ -161,11 +163,11 @@ CatBoost集成到ClickHouse步骤: /home/catboost/models/*_model.xml ``` -## 4. 从SQL运行模型推理 {#run-model-inference} +## 4. 运行从SQL推理的模型 {#run-model-inference} -对于测试模型,运行ClickHouse客户端 `$ clickhouse client`. +测试模型是否正常,运行ClickHouse客户端 `$ clickhouse client`. -让我们确保模型正常工作: +让我们确保模型能正常工作: ``` sql :) SELECT @@ -182,10 +184,10 @@ CatBoost集成到ClickHouse步骤: ACTION AS target FROM amazon_train LIMIT 10 -``` +``` !!! note "注" - 功能 [模型值](../sql-reference/functions/other-functions.md#function-modelevaluate) 返回带有多类模型的每类原始预测的元组。 + 函数 [modelEvaluate](../sql-reference/functions/other-functions.md#function-modelevaluate) 返回带有多类模型的每类原始预测的元组。 让我们预测一下: @@ -208,7 +210,7 @@ LIMIT 10 ``` !!! note "注" - 更多信息 [exp()](../sql-reference/functions/math-functions.md) 功能。 + 查看函数说明 [exp()](../sql-reference/functions/math-functions.md) 。 让我们计算样本的LogLoss: @@ -234,6 +236,6 @@ FROM ``` !!! note "注" - 更多信息 [avg()](../sql-reference/aggregate-functions/reference.md#agg_function-avg) 和 [日志()](../sql-reference/functions/math-functions.md) 功能。 + 查看函数说明 [avg()](../sql-reference/aggregate-functions/reference.md#agg_function-avg) 和 [log()](../sql-reference/functions/math-functions.md) 。 [原始文章](https://clickhouse.tech/docs/en/guides/apply_catboost_model/) From c18cdec77c5f6cb43956bf17b5e573b1630bfa5d Mon Sep 17 00:00:00 2001 From: Anton Ivashkin Date: Thu, 24 Sep 2020 18:57:54 +0300 Subject: [PATCH 130/321] Use S3 multi object delete Some strange with s3_with_proxy test --- src/Disks/S3/DiskS3.cpp | 75 ++++++++++++++++---- src/Disks/S3/DiskS3.h | 6 ++ tests/integration/test_s3_with_https/test.py | 2 +- tests/integration/test_s3_with_proxy/test.py | 6 +- 4 files changed, 73 insertions(+), 16 deletions(-) diff --git a/src/Disks/S3/DiskS3.cpp b/src/Disks/S3/DiskS3.cpp index 6abb72efeb0..fc4ff60d69d 100644 --- a/src/Disks/S3/DiskS3.cpp +++ b/src/Disks/S3/DiskS3.cpp @@ -18,7 +18,7 @@ #include #include -#include +#include #include #include @@ -36,6 +36,30 @@ namespace ErrorCodes extern const int NOT_IMPLEMENTED; } +class DiskS3::AwsS3KeyKeeper : public std::list> +{ +public: + void addKey(const String & key); + +private: + /// limit for one DeleteObject request + /// see https://docs.aws.amazon.com/AmazonS3/latest/API/API_DeleteObjects.html + const static size_t chunk_limit = 1000; +}; + +void DiskS3::AwsS3KeyKeeper::addKey(const String & key) +{ + if (empty() || back().size() >= chunk_limit) + { /// add one more chunk + push_back(value_type()); + back().reserve(chunk_limit); + } + + Aws::S3::Model::ObjectIdentifier obj; + obj.SetKey(key); + back().push_back(obj); +} + namespace { String getRandomName() @@ -634,7 +658,7 @@ std::unique_ptr DiskS3::writeFile(const String & path, } } -void DiskS3::remove(const String & path) +void DiskS3::removeMeta(const String & path, AwsS3KeyKeeper & keys) { LOG_DEBUG(&Poco::Logger::get("DiskS3"), "Remove file by path: {}", backQuote(metadata_path + path)); @@ -647,14 +671,9 @@ void DiskS3::remove(const String & path) if (metadata.ref_count == 0) { file.remove(); + for (const auto & [s3_object_path, _] : metadata.s3_objects) - { - /// TODO: Make operation idempotent. Do not throw exception if key is already deleted. - Aws::S3::Model::DeleteObjectRequest request; - request.SetBucket(bucket); - request.SetKey(s3_root_path + s3_object_path); - throwIfError(client->DeleteObject(request)); - } + keys.addKey(s3_root_path + s3_object_path); } else /// In other case decrement number of references, save metadata and delete file. { @@ -665,25 +684,57 @@ void DiskS3::remove(const String & path) } else file.remove(); + } -void DiskS3::removeRecursive(const String & path) +void DiskS3::removeMetaRecursive(const String & path, AwsS3KeyKeeper & keys) { checkStackSize(); /// This is needed to prevent stack overflow in case of cyclic symlinks. Poco::File file(metadata_path + path); if (file.isFile()) { - remove(path); + removeMeta(path, keys); } else { for (auto it{iterateDirectory(path)}; it->isValid(); it->next()) - removeRecursive(it->path()); + removeMetaRecursive(it->path(), keys); file.remove(); } } +void DiskS3::removeAws(const AwsS3KeyKeeper & keys) +{ + if (!keys.empty()) + { + for (const auto & chunk : keys) + { + Aws::S3::Model::Delete delkeys; + delkeys.SetObjects(chunk); + + /// TODO: Make operation idempotent. Do not throw exception if key is already deleted. + Aws::S3::Model::DeleteObjectsRequest request; + request.SetBucket(bucket); + request.SetDelete(delkeys); + throwIfError(client->DeleteObjects(request)); + } + } +} + +void DiskS3::remove(const String & path) +{ + AwsS3KeyKeeper keys; + removeMeta(path, keys); + removeAws(keys); +} + +void DiskS3::removeRecursive(const String & path) +{ + AwsS3KeyKeeper keys; + removeMetaRecursive(path, keys); + removeAws(keys); +} bool DiskS3::tryReserve(UInt64 bytes) { diff --git a/src/Disks/S3/DiskS3.h b/src/Disks/S3/DiskS3.h index 2d9c7f79865..fe8c47931b5 100644 --- a/src/Disks/S3/DiskS3.h +++ b/src/Disks/S3/DiskS3.h @@ -21,6 +21,8 @@ class DiskS3 : public IDisk public: friend class DiskS3Reservation; + class AwsS3KeyKeeper; + DiskS3( String name_, std::shared_ptr client_, @@ -111,6 +113,10 @@ public: private: bool tryReserve(UInt64 bytes); + void removeMeta(const String & path, AwsS3KeyKeeper & keys); + void removeMetaRecursive(const String & path, AwsS3KeyKeeper & keys); + void removeAws(const AwsS3KeyKeeper & keys); + private: const String name; std::shared_ptr client; diff --git a/tests/integration/test_s3_with_https/test.py b/tests/integration/test_s3_with_https/test.py index 17f24bba794..58e3b3c2a3b 100644 --- a/tests/integration/test_s3_with_https/test.py +++ b/tests/integration/test_s3_with_https/test.py @@ -10,7 +10,7 @@ logging.getLogger().addHandler(logging.StreamHandler()) def check_proxy_logs(cluster, proxy_instance): logs = cluster.get_container_logs(proxy_instance) # Check that all possible interactions with Minio are present - for http_method in ["PUT", "GET", "DELETE"]: + for http_method in ["PUT", "GET", "POST"]: assert logs.find(http_method + " https://minio1") >= 0 diff --git a/tests/integration/test_s3_with_proxy/test.py b/tests/integration/test_s3_with_proxy/test.py index 9df209826f9..d1a6b2fc641 100644 --- a/tests/integration/test_s3_with_proxy/test.py +++ b/tests/integration/test_s3_with_proxy/test.py @@ -36,7 +36,7 @@ def cluster(): cluster.shutdown() -def check_proxy_logs(cluster, proxy_instance, http_methods={"POST", "PUT", "GET", "DELETE"}): +def check_proxy_logs(cluster, proxy_instance, http_methods={"POST", "PUT", "GET"}): logs = cluster.get_container_logs(proxy_instance) # Check that all possible interactions with Minio are present for http_method in http_methods: @@ -66,5 +66,5 @@ def test_s3_with_proxy_list(cluster, policy): node.query("DROP TABLE IF EXISTS s3_test NO DELAY") - for proxy in ["proxy1", "proxy2"]: - check_proxy_logs(cluster, proxy, ["PUT", "GET", "DELETE"]) + check_proxy_logs(cluster, "proxy1", ["PUT", "POST"]) + check_proxy_logs(cluster, "proxy2", ["PUT", "POST", "GET"]) From b4f1eb07ff19c5326f16f912450af26f6351e165 Mon Sep 17 00:00:00 2001 From: Artem Zuikov Date: Mon, 28 Sep 2020 12:07:45 +0300 Subject: [PATCH 131/321] slightly better bigint plus/minus (#15320) --- base/common/wide_integer_impl.h | 116 ++++++++++++++++++-------------- 1 file changed, 66 insertions(+), 50 deletions(-) diff --git a/base/common/wide_integer_impl.h b/base/common/wide_integer_impl.h index 5673ac46c4a..b8a564c5a77 100644 --- a/base/common/wide_integer_impl.h +++ b/base/common/wide_integer_impl.h @@ -358,7 +358,12 @@ private: } else { - if (number * sizeof(base_type) < sizeof(T)) + if constexpr (sizeof(T) <= sizeof(base_type)) + { + if (!number) + return x; + } + else if (number * sizeof(base_type) < sizeof(T)) return x >> (number * base_bits); // & std::numeric_limits::max() return 0; } @@ -366,26 +371,32 @@ private: template constexpr static integer - op_minus(const integer & lhs, T rhs) + minus(const integer & lhs, T rhs) { - integer res; + constexpr const unsigned rhs_items = (sizeof(T) > sizeof(base_type)) ? (sizeof(T) / sizeof(base_type)) : 1; + constexpr const unsigned op_items = (item_count < rhs_items) ? item_count : rhs_items; - bool is_underflow = false; - for (unsigned i = 0; i < item_count; ++i) + integer res(lhs); + bool underflows[item_count] = {}; + + for (unsigned i = 0; i < op_items; ++i) { - base_type lhs_item = lhs.items[little(i)]; base_type rhs_item = get_item(rhs, i); + base_type & res_item = res.items[little(i)]; - if (is_underflow) + underflows[i] = res_item < rhs_item; + res_item -= rhs_item; + } + + for (unsigned i = 1; i < item_count; ++i) + { + if (underflows[i-1]) { - is_underflow = (lhs_item == 0); - --lhs_item; + base_type & res_item = res.items[little(i)]; + if (res_item == 0) + underflows[i] = true; + --res_item; } - - if (lhs_item < rhs_item) - is_underflow = true; - - res.items[little(i)] = lhs_item - rhs_item; } return res; @@ -393,39 +404,44 @@ private: template constexpr static integer - op_plus(const integer & lhs, T rhs) + plus(const integer & lhs, T rhs) { - integer res; + constexpr const unsigned rhs_items = (sizeof(T) > sizeof(base_type)) ? (sizeof(T) / sizeof(base_type)) : 1; + constexpr const unsigned op_items = (item_count < rhs_items) ? item_count : rhs_items; - bool is_overflow = false; - for (unsigned i = 0; i < item_count; ++i) + integer res(lhs); + bool overflows[item_count] = {}; + + for (unsigned i = 0; i < op_items; ++i) { - base_type lhs_item = lhs.items[little(i)]; base_type rhs_item = get_item(rhs, i); - - if (is_overflow) - { - ++lhs_item; - is_overflow = (lhs_item == 0); - } - base_type & res_item = res.items[little(i)]; - res_item = lhs_item + rhs_item; - if (res_item < rhs_item) - is_overflow = true; + res_item += rhs_item; + overflows[i] = res_item < rhs_item; + } + + for (unsigned i = 1; i < item_count; ++i) + { + if (overflows[i-1]) + { + base_type & res_item = res.items[little(i)]; + ++res_item; + if (res_item == 0) + overflows[i] = true; + } } return res; } template - constexpr static auto op_multiply(const integer & lhs, const T & rhs) + constexpr static auto multiply(const integer & lhs, const T & rhs) { integer res{}; #if 1 - integer lhs2 = op_plus(lhs, shift_left(lhs, 1)); - integer lhs3 = op_plus(lhs2, shift_left(lhs, 2)); + integer lhs2 = plus(lhs, shift_left(lhs, 1)); + integer lhs3 = plus(lhs2, shift_left(lhs, 2)); #endif for (unsigned i = 0; i < item_count; ++i) { @@ -437,7 +453,7 @@ private: #if 1 /// optimization if ((rhs_item & 0x7) == 0x7) { - res = op_plus(res, shift_left(lhs3, pos)); + res = plus(res, shift_left(lhs3, pos)); rhs_item >>= 3; pos += 3; continue; @@ -445,14 +461,14 @@ private: if ((rhs_item & 0x3) == 0x3) { - res = op_plus(res, shift_left(lhs2, pos)); + res = plus(res, shift_left(lhs2, pos)); rhs_item >>= 2; pos += 2; continue; } #endif if (rhs_item & 1) - res = op_plus(res, shift_left(lhs, pos)); + res = plus(res, shift_left(lhs, pos)); rhs_item >>= 1; ++pos; @@ -475,7 +491,7 @@ public: constexpr static integer operator_unary_minus(const integer & lhs) noexcept(std::is_same_v) { - return op_plus(operator_unary_tilda(lhs), 1); + return plus(operator_unary_tilda(lhs), 1); } template @@ -484,9 +500,9 @@ public: if constexpr (should_keep_size()) { if (is_negative(rhs)) - return op_minus(lhs, -rhs); + return minus(lhs, -rhs); else - return op_plus(lhs, rhs); + return plus(lhs, rhs); } else { @@ -502,9 +518,9 @@ public: if constexpr (should_keep_size()) { if (is_negative(rhs)) - return op_plus(lhs, -rhs); + return plus(lhs, -rhs); else - return op_minus(lhs, rhs); + return minus(lhs, rhs); } else { @@ -523,12 +539,12 @@ public: if constexpr (std::is_signed_v) { - res = op_multiply((is_negative(lhs) ? make_positive(lhs) : lhs), + res = multiply((is_negative(lhs) ? make_positive(lhs) : lhs), (is_negative(rhs) ? make_positive(rhs) : rhs)); } else { - res = op_multiply(lhs, (is_negative(rhs) ? make_positive(rhs) : rhs)); + res = multiply(lhs, (is_negative(rhs) ? make_positive(rhs) : rhs)); } if (std::is_same_v && is_negative(lhs) != is_negative(rhs)) @@ -775,20 +791,20 @@ public: { if (*c >= '0' && *c <= '9') { - res = op_multiply(res, 16U); - res = op_plus(res, *c - '0'); + res = multiply(res, 16U); + res = plus(res, *c - '0'); ++c; } else if (*c >= 'a' && *c <= 'f') { - res = op_multiply(res, 16U); - res = op_plus(res, *c - 'a' + 10U); + res = multiply(res, 16U); + res = plus(res, *c - 'a' + 10U); ++c; } else if (*c >= 'A' && *c <= 'F') { // tolower must be used, but it is not constexpr - res = op_multiply(res, 16U); - res = op_plus(res, *c - 'A' + 10U); + res = multiply(res, 16U); + res = plus(res, *c - 'A' + 10U); ++c; } else @@ -802,8 +818,8 @@ public: if (*c < '0' || *c > '9') throwError("invalid char from"); - res = op_multiply(res, 10U); - res = op_plus(res, *c - '0'); + res = multiply(res, 10U); + res = plus(res, *c - '0'); ++c; } } From 1adc0a820558cb13ce09622dc906bb983b46c758 Mon Sep 17 00:00:00 2001 From: alesapin Date: Mon, 28 Sep 2020 12:58:42 +0300 Subject: [PATCH 132/321] Fix race condition in client suggestions --- base/common/LineReader.cpp | 4 ++-- base/common/LineReader.h | 3 ++- base/common/ReadlineLineReader.cpp | 3 ++- base/common/ReplxxLineReader.cpp | 5 +++-- 4 files changed, 9 insertions(+), 6 deletions(-) diff --git a/base/common/LineReader.cpp b/base/common/LineReader.cpp index dd2e09b0393..b2bc929a1df 100644 --- a/base/common/LineReader.cpp +++ b/base/common/LineReader.cpp @@ -38,10 +38,10 @@ bool hasInputData() } -LineReader::Suggest::WordsRange LineReader::Suggest::getCompletions(const String & prefix, size_t prefix_length) const +std::optional LineReader::Suggest::getCompletions(const String & prefix, size_t prefix_length) const { if (!ready) - return std::make_pair(words.end(), words.end()); + return std::nullopt; std::string_view last_word; diff --git a/base/common/LineReader.h b/base/common/LineReader.h index 77dc70d8808..159a93ce40d 100644 --- a/base/common/LineReader.h +++ b/base/common/LineReader.h @@ -4,6 +4,7 @@ #include #include +#include class LineReader { @@ -18,7 +19,7 @@ public: std::atomic ready{false}; /// Get iterators for the matched range of words if any. - WordsRange getCompletions(const String & prefix, size_t prefix_length) const; + std::optional getCompletions(const String & prefix, size_t prefix_length) const; }; using Patterns = std::vector; diff --git a/base/common/ReadlineLineReader.cpp b/base/common/ReadlineLineReader.cpp index 095df319acc..4e67ed5e4a3 100644 --- a/base/common/ReadlineLineReader.cpp +++ b/base/common/ReadlineLineReader.cpp @@ -30,7 +30,8 @@ static LineReader::Suggest::Words::const_iterator end; static void findRange(const char * prefix, size_t prefix_length) { std::string prefix_str(prefix); - std::tie(pos, end) = suggest->getCompletions(prefix_str, prefix_length); + if (auto completions = suggest->getCompletions(prefix_str, prefix_length)) + std::tie(pos, end) = *completions; } /// Iterates through matched range. diff --git a/base/common/ReplxxLineReader.cpp b/base/common/ReplxxLineReader.cpp index cc8b963958c..c4f9c65a116 100644 --- a/base/common/ReplxxLineReader.cpp +++ b/base/common/ReplxxLineReader.cpp @@ -70,8 +70,9 @@ ReplxxLineReader::ReplxxLineReader( auto callback = [&suggest] (const String & context, size_t context_size) { - auto range = suggest.getCompletions(context, context_size); - return Replxx::completions_t(range.first, range.second); + if (auto range = suggest.getCompletions(context, context_size)) + return Replxx::completions_t(range->first, range->second); + return Replxx::completions_t(); }; rx.set_completion_callback(callback); From d3047d0783e44e981a9922bdd2a9c02061378d7b Mon Sep 17 00:00:00 2001 From: zhang2014 Date: Mon, 28 Sep 2020 18:02:30 +0800 Subject: [PATCH 133/321] ISSUES-15365 fix attach mysql database failure --- src/Core/MultiEnum.h | 2 +- src/Databases/DatabaseFactory.cpp | 10 ++- .../MySQL/ConnectionMySQLSettings.cpp | 65 +++++++++++++++++++ src/Databases/MySQL/ConnectionMySQLSettings.h | 33 ++++++++++ .../MySQL/DatabaseConnectionMySQL.cpp | 6 +- src/Databases/MySQL/DatabaseConnectionMySQL.h | 8 +-- 6 files changed, 114 insertions(+), 10 deletions(-) create mode 100644 src/Databases/MySQL/ConnectionMySQLSettings.cpp create mode 100644 src/Databases/MySQL/ConnectionMySQLSettings.h diff --git a/src/Core/MultiEnum.h b/src/Core/MultiEnum.h index ddfc5b13e86..a7e3393c9c3 100644 --- a/src/Core/MultiEnum.h +++ b/src/Core/MultiEnum.h @@ -86,7 +86,7 @@ struct MultiEnum return right.operator==(left); } - template + template >::type> friend bool operator!=(L left, MultiEnum right) { return !(right.operator==(left)); diff --git a/src/Databases/DatabaseFactory.cpp b/src/Databases/DatabaseFactory.cpp index 1bf4db74bf0..2bb4c595c05 100644 --- a/src/Databases/DatabaseFactory.cpp +++ b/src/Databases/DatabaseFactory.cpp @@ -19,6 +19,7 @@ #if USE_MYSQL # include +# include # include # include # include @@ -83,7 +84,7 @@ DatabasePtr DatabaseFactory::getImpl(const ASTCreateQuery & create, const String throw Exception("Database engine " + engine_name + " cannot have arguments", ErrorCodes::BAD_ARGUMENTS); if (engine_define->engine->parameters || engine_define->partition_by || engine_define->primary_key || engine_define->order_by || - engine_define->sample_by || (engine_name != "MaterializeMySQL" && engine_define->settings)) + engine_define->sample_by || (!endsWith(engine_name, "MySQL") && engine_define->settings)) throw Exception("Database engine " + engine_name + " cannot have parameters, primary_key, order_by, sample_by, settings", ErrorCodes::UNKNOWN_ELEMENT_IN_AST); @@ -133,8 +134,13 @@ DatabasePtr DatabaseFactory::getImpl(const ASTCreateQuery & create, const String , std::move(materialize_mode_settings)); } + auto mysql_database_settings = std::make_unique(); + + mysql_database_settings->loadFromQueryContext(context); + mysql_database_settings->loadFromQuery(*engine_define); /// higher priority + return std::make_shared( - context, database_name, metadata_path, engine_define, mysql_database_name, std::move(mysql_pool)); + context, database_name, metadata_path, engine_define, mysql_database_name, std::move(mysql_database_settings), std::move(mysql_pool)); } catch (...) { diff --git a/src/Databases/MySQL/ConnectionMySQLSettings.cpp b/src/Databases/MySQL/ConnectionMySQLSettings.cpp new file mode 100644 index 00000000000..fa92e793225 --- /dev/null +++ b/src/Databases/MySQL/ConnectionMySQLSettings.cpp @@ -0,0 +1,65 @@ +#include + +#include +#include +#include +#include + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int UNKNOWN_SETTING; + extern const int BAD_ARGUMENTS; +} + +IMPLEMENT_SETTINGS_TRAITS(ConnectionMySQLSettingsTraits, LIST_OF_CONNECTION_MYSQL_SETTINGS) + +void ConnectionMySQLSettings::loadFromQuery(ASTStorage & storage_def) +{ + if (storage_def.settings) + { + try + { + applyChanges(storage_def.settings->changes); + } + catch (Exception & e) + { + if (e.code() == ErrorCodes::UNKNOWN_SETTING) + throw Exception(e.message() + " for database " + storage_def.engine->name, ErrorCodes::BAD_ARGUMENTS); + else + e.rethrow(); + } + } + else + { + auto settings_ast = std::make_shared(); + settings_ast->is_standalone = false; + storage_def.set(storage_def.settings, settings_ast); + } + + SettingsChanges & changes = storage_def.settings->changes; +#define ADD_IF_ABSENT(NAME) \ + if (std::find_if(changes.begin(), changes.end(), \ + [](const SettingChange & c) { return c.name == #NAME; }) \ + == changes.end()) \ + changes.push_back(SettingChange{#NAME, static_cast(NAME)}); + + APPLY_FOR_IMMUTABLE_CONNECTION_MYSQL_SETTINGS(ADD_IF_ABSENT) +#undef ADD_IF_ABSENT +} + +void ConnectionMySQLSettings::loadFromQueryContext(const Context & context) +{ + if (!context.hasQueryContext()) + return; + + const Settings & settings = context.getQueryContext().getSettingsRef(); + + if (settings.mysql_datatypes_support_level.value != mysql_datatypes_support_level.value) + set("mysql_datatypes_support_level", settings.mysql_datatypes_support_level.toString()); +} + + +} diff --git a/src/Databases/MySQL/ConnectionMySQLSettings.h b/src/Databases/MySQL/ConnectionMySQLSettings.h new file mode 100644 index 00000000000..90279f846a4 --- /dev/null +++ b/src/Databases/MySQL/ConnectionMySQLSettings.h @@ -0,0 +1,33 @@ +#pragma once + +#include +#include +#include + +namespace DB +{ + +class Context; +class ASTStorage; + +#define LIST_OF_CONNECTION_MYSQL_SETTINGS(M) \ + M(MySQLDataTypesSupport, mysql_datatypes_support_level, 0, "Which MySQL types should be converted to corresponding ClickHouse types (rather than being represented as String). Can be empty or any combination of 'decimal' or 'datetime64'. When empty MySQL's DECIMAL and DATETIME/TIMESTAMP with non-zero precison are seen as String on ClickHouse's side.", 0) \ + +/// Settings that should not change after the creation of a database. +#define APPLY_FOR_IMMUTABLE_CONNECTION_MYSQL_SETTINGS(M) \ + M(mysql_datatypes_support_level) + +DECLARE_SETTINGS_TRAITS(ConnectionMySQLSettingsTraits, LIST_OF_CONNECTION_MYSQL_SETTINGS) + + +/** Settings for the MySQL database engine. + * Could be loaded from a CREATE DATABASE query (SETTINGS clause) and Query settings. + */ +struct ConnectionMySQLSettings : public BaseSettings +{ + void loadFromQuery(ASTStorage & storage_def); + + void loadFromQueryContext(const Context & context); +}; + +} diff --git a/src/Databases/MySQL/DatabaseConnectionMySQL.cpp b/src/Databases/MySQL/DatabaseConnectionMySQL.cpp index 9c94014bf23..8eb874fba8f 100644 --- a/src/Databases/MySQL/DatabaseConnectionMySQL.cpp +++ b/src/Databases/MySQL/DatabaseConnectionMySQL.cpp @@ -45,13 +45,13 @@ static constexpr const std::chrono::seconds cleaner_sleep_time{30}; static const std::chrono::seconds lock_acquire_timeout{10}; DatabaseConnectionMySQL::DatabaseConnectionMySQL(const Context & context, const String & database_name_, const String & metadata_path_, - const ASTStorage * database_engine_define_, const String & database_name_in_mysql_, mysqlxx::Pool && pool) + const ASTStorage * database_engine_define_, const String & database_name_in_mysql_, std::unique_ptr settings_, mysqlxx::Pool && pool) : IDatabase(database_name_) , global_context(context.getGlobalContext()) , metadata_path(metadata_path_) , database_engine_define(database_engine_define_->clone()) , database_name_in_mysql(database_name_in_mysql_) - , mysql_datatypes_support_level(context.getQueryContext().getSettingsRef().mysql_datatypes_support_level) + , database_settings(std::move(settings_)) , mysql_pool(std::move(pool)) { empty(); /// test database is works fine. @@ -273,7 +273,7 @@ std::map DatabaseConnectionMySQL::fetchTablesColumnsL database_name_in_mysql, tables_name, settings.external_table_functions_use_nulls, - mysql_datatypes_support_level); + database_settings->mysql_datatypes_support_level); } void DatabaseConnectionMySQL::shutdown() diff --git a/src/Databases/MySQL/DatabaseConnectionMySQL.h b/src/Databases/MySQL/DatabaseConnectionMySQL.h index e9f72adc013..7bf5e8c1d88 100644 --- a/src/Databases/MySQL/DatabaseConnectionMySQL.h +++ b/src/Databases/MySQL/DatabaseConnectionMySQL.h @@ -8,6 +8,7 @@ #include #include #include +#include #include #include @@ -36,7 +37,8 @@ public: DatabaseConnectionMySQL( const Context & context, const String & database_name, const String & metadata_path, - const ASTStorage * database_engine_define, const String & database_name_in_mysql, mysqlxx::Pool && pool); + const ASTStorage * database_engine_define, const String & database_name_in_mysql, std::unique_ptr settings_, + mysqlxx::Pool && pool); String getEngineName() const override { return "MySQL"; } @@ -76,9 +78,7 @@ private: String metadata_path; ASTPtr database_engine_define; String database_name_in_mysql; - // Cache setting for later from query context upon creation, - // so column types depend on the settings set at query-level. - MultiEnum mysql_datatypes_support_level; + std::unique_ptr database_settings; std::atomic quit{false}; std::condition_variable cond; From 7e90e00f1aca925ddb5febe77bb9f31035e1cd0d Mon Sep 17 00:00:00 2001 From: zhang2014 Date: Mon, 28 Sep 2020 18:27:02 +0800 Subject: [PATCH 134/321] ISSUES-15365 add integration test --- .../test_mysql_database_engine/test.py | 27 +++++++++++++++++++ 1 file changed, 27 insertions(+) diff --git a/tests/integration/test_mysql_database_engine/test.py b/tests/integration/test_mysql_database_engine/test.py index 399b9263123..6290477f849 100644 --- a/tests/integration/test_mysql_database_engine/test.py +++ b/tests/integration/test_mysql_database_engine/test.py @@ -160,6 +160,33 @@ def test_bad_arguments_for_mysql_database_engine(started_cluster): mysql_node.query("DROP DATABASE test_bad_arguments") +def test_data_types_support_level_for_mysql_database_engine(started_cluster): + with contextlib.closing(MySQLNodeInstance('root', 'clickhouse', '127.0.0.1', port=3308)) as mysql_node: + mysql_node.query("CREATE DATABASE IF NOT EXISTS test DEFAULT CHARACTER SET 'utf8'") + clickhouse_node.query("CREATE DATABASE test_database ENGINE = MySQL('mysql1:3306', test_database, 'root', 'clickhouse')", + settings={"mysql_datatypes_support_level": "decimal,datetime64"}) + + assert "SETTINGS mysql_datatypes_support_level = \'decimal,datetime64\'" in clickhouse_node.query("SHOW CREATE DATABASE test_database FORMAT TSV") + clickhouse_node.query("DETACH DATABASE test_database") + + # without context settings + clickhouse_node.query("ATTACH DATABASE test_database") + assert "SETTINGS mysql_datatypes_support_level = \'decimal,datetime64\'" in clickhouse_node.query("SHOW CREATE DATABASE test_database FORMAT TSV") + + clickhouse_node.query( + "CREATE DATABASE test_database_1 ENGINE = MySQL('mysql1:3306', test_database, 'root', 'clickhouse') SETTINGS mysql_datatypes_support_level = 'decimal,datetime64'", + settings={"mysql_datatypes_support_level": "decimal"}) + + assert "SETTINGS mysql_datatypes_support_level = \'decimal,datetime64\'" in clickhouse_node.query("SHOW CREATE DATABASE test_database_1 FORMAT TSV") + clickhouse_node.query("DETACH DATABASE test_database_1") + + # without context settings + clickhouse_node.query("ATTACH DATABASE test_database_1") + assert "SETTINGS mysql_datatypes_support_level = \'decimal,datetime64\'" in clickhouse_node.query("SHOW CREATE DATABASE test_database_1 FORMAT TSV") + + mysql_node.query("DROP DATABASE test") + + decimal_values = [0.123, 0.4, 5.67, 8.91011, 123456789.123, -0.123, -0.4, -5.67, -8.91011, -123456789.123] timestamp_values = ['2015-05-18 07:40:01.123', '2019-09-16 19:20:11.123'] timestamp_values_no_subsecond = ['2015-05-18 07:40:01', '2019-09-16 19:20:11'] From e40d23cfe5ed802f66906b058ebb4dd70835d894 Mon Sep 17 00:00:00 2001 From: zhang2014 Date: Mon, 28 Sep 2020 19:11:40 +0800 Subject: [PATCH 135/321] ISSUES-15365 add ya make --- src/Databases/ya.make | 1 + 1 file changed, 1 insertion(+) diff --git a/src/Databases/ya.make b/src/Databases/ya.make index 726127bfe52..75adf1e9c5c 100644 --- a/src/Databases/ya.make +++ b/src/Databases/ya.make @@ -23,6 +23,7 @@ SRCS( MySQL/MaterializeMetadata.cpp MySQL/MaterializeMySQLSettings.cpp MySQL/MaterializeMySQLSyncThread.cpp + MySQL/ConnectionMySQLSettings.cpp ) From a526a767dfccc793bea0c732e7f8b3e5c398e2cc Mon Sep 17 00:00:00 2001 From: alesapin Date: Mon, 28 Sep 2020 14:17:07 +0300 Subject: [PATCH 136/321] Fix some flaky tests --- tests/integration/test_mysql_protocol/test.py | 4 +++- .../0_stateless/00652_replicated_mutations_zookeeper.sh | 4 +++- 2 files changed, 6 insertions(+), 2 deletions(-) diff --git a/tests/integration/test_mysql_protocol/test.py b/tests/integration/test_mysql_protocol/test.py index b5fd312007a..3e737dc2644 100644 --- a/tests/integration/test_mysql_protocol/test.py +++ b/tests/integration/test_mysql_protocol/test.py @@ -238,7 +238,9 @@ def test_mysql_federated(mysql_server, server_address): node.query('''INSERT INTO mysql_federated.test VALUES (0), (1), (5)''', settings={"password": "123"}) def check_retryable_error_in_stderr(stderr): - return "Can't connect to local MySQL server through socket" in stderr or "MySQL server has gone away" in stderr + return ("Can't connect to local MySQL server through socket" in stderr + or "MySQL server has gone away" in stderr + or "Server shutdown in progress" in stderr) code, (stdout, stderr) = mysql_server.exec_run(''' mysql diff --git a/tests/queries/0_stateless/00652_replicated_mutations_zookeeper.sh b/tests/queries/0_stateless/00652_replicated_mutations_zookeeper.sh index af2bf2dca2b..380f6ee6ff8 100755 --- a/tests/queries/0_stateless/00652_replicated_mutations_zookeeper.sh +++ b/tests/queries/0_stateless/00652_replicated_mutations_zookeeper.sh @@ -12,7 +12,7 @@ ${CLICKHOUSE_CLIENT} --query="CREATE TABLE mutations_r1(d Date, x UInt32, s Stri ${CLICKHOUSE_CLIENT} --query="CREATE TABLE mutations_r2(d Date, x UInt32, s String, m MATERIALIZED x + 2) ENGINE ReplicatedMergeTree('/clickhouse/tables/test_00652/mutations', 'r2', d, intDiv(x, 10), 8192)" # Test a mutation on empty table -${CLICKHOUSE_CLIENT} --query="ALTER TABLE mutations_r1 DELETE WHERE x = 1" +${CLICKHOUSE_CLIENT} --query="ALTER TABLE mutations_r1 DELETE WHERE x = 1 SETTINGS mutations_sync = 2" # Insert some data ${CLICKHOUSE_CLIENT} --query="INSERT INTO mutations_r1(d, x, s) VALUES \ @@ -34,6 +34,8 @@ ${CLICKHOUSE_CLIENT} --query="ALTER TABLE mutations_r1 DELETE WHERE m = 3 SETTIN ${CLICKHOUSE_CLIENT} --query="INSERT INTO mutations_r1(d, x, s) VALUES \ ('2000-01-01', 5, 'e'), ('2000-02-01', 5, 'e')" +${CLICKHOUSE_CLIENT} --query "SYSTEM SYNC REPLICA mutations_r2" + # Check that the table contains only the data that should not be deleted. ${CLICKHOUSE_CLIENT} --query="SELECT d, x, s, m FROM mutations_r2 ORDER BY d, x" # Check the contents of the system.mutations table. From 1e7a0f0e5f272bc300fa0d8c46696f858adb396c Mon Sep 17 00:00:00 2001 From: Alexander Kuzmenkov Date: Mon, 28 Sep 2020 14:44:49 +0300 Subject: [PATCH 137/321] typo --- docker/test/performance-comparison/compare.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docker/test/performance-comparison/compare.sh b/docker/test/performance-comparison/compare.sh index 6775e662070..0a45ad34e8d 100755 --- a/docker/test/performance-comparison/compare.sh +++ b/docker/test/performance-comparison/compare.sh @@ -212,7 +212,7 @@ function run_tests || echo "Test $test_name failed with error code $?" >> "$test_name-err.log" profile_seconds_left=$(awk -F' ' \ - 'BEGIN { s = '$profile_seconds_left'; } /^profile-total/ { s -= $1 } END { print s }' \ + 'BEGIN { s = '$profile_seconds_left'; } /^profile-total/ { s -= $2 } END { print s }' \ "$test_name-raw.tsv") done From 6e70f2ce7afab4202cd925f098f7b4f523b19b55 Mon Sep 17 00:00:00 2001 From: detailyang Date: Mon, 28 Sep 2020 19:56:42 +0800 Subject: [PATCH 138/321] bugfix:aggregate:fix boundingratio args check --- src/AggregateFunctions/AggregateFunctionBoundingRatio.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/AggregateFunctions/AggregateFunctionBoundingRatio.h b/src/AggregateFunctions/AggregateFunctionBoundingRatio.h index 9ceb7976f4a..7c254668f8d 100644 --- a/src/AggregateFunctions/AggregateFunctionBoundingRatio.h +++ b/src/AggregateFunctions/AggregateFunctionBoundingRatio.h @@ -115,7 +115,7 @@ public: : IAggregateFunctionDataHelper(arguments, {}) { const auto x_arg = arguments.at(0).get(); - const auto y_arg = arguments.at(0).get(); + const auto y_arg = arguments.at(1).get(); if (!x_arg->isValueRepresentedByNumber() || !y_arg->isValueRepresentedByNumber()) throw Exception("Illegal types of arguments of aggregate function " + getName() + ", must have number representation.", From c48d212320bab7fb48257165d6e24f81e005f491 Mon Sep 17 00:00:00 2001 From: Alexander Kuzmenkov Date: Mon, 28 Sep 2020 15:09:08 +0300 Subject: [PATCH 139/321] performance comparison --- docker/test/performance-comparison/perf.py | 18 +++++++++--------- 1 file changed, 9 insertions(+), 9 deletions(-) diff --git a/docker/test/performance-comparison/perf.py b/docker/test/performance-comparison/perf.py index 23686091e45..d27f3ec3d76 100755 --- a/docker/test/performance-comparison/perf.py +++ b/docker/test/performance-comparison/perf.py @@ -78,11 +78,16 @@ for e in root.findall('query'): assert(len(test_queries) == len(is_short)) +# If we're given a list of queries to run, check that it makes sense. +for i in args.queries_to_run or []: + if i < 0 or i >= len(test_queries): + print(f'There is no query no. {i} in this test, only [{0}-{len(test_queries) - 1}] are present') + exit(1) -# If we're only asked to print the queries, do that and exit +# If we're only asked to print the queries, do that and exit. if args.print_queries: - for q in test_queries: - print(q) + for i in args.queries_to_run or range(0, len(test_queries)): + print(test_queries[i]) exit(0) # Print short queries @@ -196,12 +201,7 @@ if args.max_queries: queries_to_run = random.sample(range(0, len(test_queries)), min(len(test_queries), args.max_queries)) if args.queries_to_run: - # Run the specified queries, with some sanity check. - for i in args.queries_to_run: - if i < 0 or i >= len(test_queries): - print(f'There is no query no. "{i}" in this test, only [{0}-{len(test_queries) - 1}] are present') - exit(1) - + # Run the specified queries. queries_to_run = args.queries_to_run # Run test queries. From e19f51680ab4e19036fd993f3664fa9523b44eea Mon Sep 17 00:00:00 2001 From: myrrc Date: Mon, 28 Sep 2020 15:47:10 +0300 Subject: [PATCH 140/321] fixing links using abs path --- docs/tools/build.py | 2 +- docs/tools/cmake_in_clickhouse_generator.py | 10 ++++++---- 2 files changed, 7 insertions(+), 5 deletions(-) diff --git a/docs/tools/build.py b/docs/tools/build.py index c91cc8d5f3c..bcbf3ac27cd 100755 --- a/docs/tools/build.py +++ b/docs/tools/build.py @@ -185,7 +185,7 @@ def build(args): test.test_templates(args.website_dir) if not args.skip_docs: - generate_cmake_flags_files(os.path.join(os.path.dirname(__file__), '..', '..')) + generate_cmake_flags_files() build_docs(args) from github import build_releases diff --git a/docs/tools/cmake_in_clickhouse_generator.py b/docs/tools/cmake_in_clickhouse_generator.py index b15df76151e..1595a836c9d 100644 --- a/docs/tools/cmake_in_clickhouse_generator.py +++ b/docs/tools/cmake_in_clickhouse_generator.py @@ -98,7 +98,9 @@ def process_folder(root_path:str, name: str) -> None: if f == "CMakeLists.txt" or ".cmake" in f: process_file(root, f) -def generate_cmake_flags_files(root_path: str) -> None: +def generate_cmake_flags_files() -> None: + root_path: str = os.path.join(os.path.dirname(__file__), '..', '..') + output_file_name: str = os.path.join(root_path, "docs/en/development/cmake-in-clickhouse.md") header_file_name: str = os.path.join(root_path, "docs/_includes/cmake_in_clickhouse_header.md") footer_file_name: str = os.path.join(root_path, "docs/_includes/cmake_in_clickhouse_footer.md") @@ -132,14 +134,14 @@ def generate_cmake_flags_files(root_path: str) -> None: f.write(entities[k][1] + "\n") ignored_keys.append(k) - f.write("\n### External libraries system/bundled mode\n" + table_header) + f.write("\n\n### External libraries system/bundled mode\n" + table_header) for k in sorted_keys: if k.startswith("USE_INTERNAL_"): f.write(entities[k][1] + "\n") ignored_keys.append(k) - f.write("\n### Other flags\n" + table_header) + f.write("\n\n### Other flags\n" + table_header) for k in sorted(set(sorted_keys).difference(set(ignored_keys))): f.write(entities[k][1] + "\n") @@ -149,4 +151,4 @@ def generate_cmake_flags_files(root_path: str) -> None: if __name__ == '__main__': - generate_cmake_flags_files("../../") + generate_cmake_flags_files() From e5a6d068202093b56fce9405405e7c8ba56e67f6 Mon Sep 17 00:00:00 2001 From: zhang2014 Date: Mon, 28 Sep 2020 21:15:02 +0800 Subject: [PATCH 141/321] ISSUES-15365 try fix integration test failure --- src/Databases/MySQL/DatabaseConnectionMySQL.cpp | 13 ++++++++++++- 1 file changed, 12 insertions(+), 1 deletion(-) diff --git a/src/Databases/MySQL/DatabaseConnectionMySQL.cpp b/src/Databases/MySQL/DatabaseConnectionMySQL.cpp index 8eb874fba8f..03d218d132f 100644 --- a/src/Databases/MySQL/DatabaseConnectionMySQL.cpp +++ b/src/Databases/MySQL/DatabaseConnectionMySQL.cpp @@ -133,9 +133,20 @@ static ASTPtr getCreateQueryFromStorage(const StoragePtr & storage, const ASTPtr columns_expression_list->children.emplace_back(column_declaration); } + ASTStorage * ast_storage = table_storage_define->as(); + ASTs storage_children = ast_storage->children; + auto storage_engine_arguments = ast_storage->engine->arguments; + + /// Add table_name to engine arguments auto mysql_table_name = std::make_shared(table_id.table_name); - auto storage_engine_arguments = table_storage_define->as()->engine->arguments; storage_engine_arguments->children.insert(storage_engine_arguments->children.begin() + 2, mysql_table_name); + + /// Unset settings + storage_children.erase( + std::remove_if(storage_children.begin(), storage_children.end(), + [&](const ASTPtr & element) { return element.get() == ast_storage->settings; }), + storage_children.end()); + ast_storage->settings = nullptr; } return create_table_query; From 6c0c13cf18d75a9866d8aad4fdcc8f738b63216f Mon Sep 17 00:00:00 2001 From: myrrc Date: Mon, 28 Sep 2020 16:28:36 +0300 Subject: [PATCH 142/321] slightly rewrote paths handling --- docs/tools/cmake_in_clickhouse_generator.py | 16 +++++++++------- 1 file changed, 9 insertions(+), 7 deletions(-) diff --git a/docs/tools/cmake_in_clickhouse_generator.py b/docs/tools/cmake_in_clickhouse_generator.py index 1595a836c9d..3437c8ad956 100644 --- a/docs/tools/cmake_in_clickhouse_generator.py +++ b/docs/tools/cmake_in_clickhouse_generator.py @@ -66,8 +66,8 @@ def build_entity(path: str, entity: Entity, line_comment: Tuple[int, str]) -> No entities[name] = path, formatted_entity -def process_file(root_path: str, input_name: str) -> None: - with open(os.path.join(root_path, input_name), 'r') as cmake_file: +def process_file(root_path: str, file_path: str, file_name: str) -> None: + with open(os.path.join(file_path, file_name), 'r') as cmake_file: contents: str = cmake_file.read() def get_line_and_comment(target: str) -> Tuple[int, str]: @@ -88,15 +88,17 @@ def process_file(root_path: str, input_name: str) -> None: matches: Optional[List[Entity]] = re.findall(cmake_option_regex, contents, re.MULTILINE) + file_rel_path_with_name: str = os.path.join(file_path[len(root_path):], file_name)[1:] + if matches: for entity in matches: - build_entity(os.path.join(root_path[6:], input_name), entity, get_line_and_comment(entity[0])) + build_entity(file_rel_path_with_name, entity, get_line_and_comment(entity[0])) -def process_folder(root_path:str, name: str) -> None: +def process_folder(root_path: str, name: str) -> None: for root, _, files in os.walk(os.path.join(root_path, name)): for f in files: if f == "CMakeLists.txt" or ".cmake" in f: - process_file(root, f) + process_file(root_path, root, f) def generate_cmake_flags_files() -> None: root_path: str = os.path.join(os.path.dirname(__file__), '..', '..') @@ -105,8 +107,8 @@ def generate_cmake_flags_files() -> None: header_file_name: str = os.path.join(root_path, "docs/_includes/cmake_in_clickhouse_header.md") footer_file_name: str = os.path.join(root_path, "docs/_includes/cmake_in_clickhouse_footer.md") - process_file(root_path, "CMakeLists.txt") - process_file(root_path, "programs/CMakeLists.txt") + process_file(root_path, root_path, "CMakeLists.txt") + process_file(root_path, os.path.join(root_path, "programs"), "CMakeLists.txt") process_folder(root_path, "base") process_folder(root_path, "cmake") From b14fb0e8d560ef2209ebf7058940ea75376adaff Mon Sep 17 00:00:00 2001 From: zhang2014 Date: Mon, 28 Sep 2020 23:00:01 +0800 Subject: [PATCH 143/321] ISSUES-15365 try fix integration test --- tests/integration/test_mysql_database_engine/test.py | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/tests/integration/test_mysql_database_engine/test.py b/tests/integration/test_mysql_database_engine/test.py index 6290477f849..26dd2ee42ea 100644 --- a/tests/integration/test_mysql_database_engine/test.py +++ b/tests/integration/test_mysql_database_engine/test.py @@ -124,6 +124,9 @@ def test_clickhouse_dml_for_mysql_database(started_cluster): clickhouse_node.query("INSERT INTO `test_database`.`test_table`(`i``d`) select number from numbers(10000)") assert clickhouse_node.query("SELECT count() FROM `test_database`.`test_table`").rstrip() == '10000' + clickhouse_node.query("DROP DATABASE test_database") + assert 'test_database' not in clickhouse_node.query('SHOW DATABASES') + mysql_node.query("DROP DATABASE test_database") @@ -184,6 +187,8 @@ def test_data_types_support_level_for_mysql_database_engine(started_cluster): clickhouse_node.query("ATTACH DATABASE test_database_1") assert "SETTINGS mysql_datatypes_support_level = \'decimal,datetime64\'" in clickhouse_node.query("SHOW CREATE DATABASE test_database_1 FORMAT TSV") + clickhouse_node.query("DROP DATABASE test_database") + assert 'test_database' not in clickhouse_node.query('SHOW DATABASES') mysql_node.query("DROP DATABASE test") From 5b142ac37feb6af34971960e275428f49c27a4d8 Mon Sep 17 00:00:00 2001 From: filimonov <1549571+filimonov@users.noreply.github.com> Date: Mon, 28 Sep 2020 17:01:28 +0200 Subject: [PATCH 144/321] Update settings.md --- .../server-configuration-parameters/settings.md | 12 ++++++++++++ 1 file changed, 12 insertions(+) diff --git a/docs/en/operations/server-configuration-parameters/settings.md b/docs/en/operations/server-configuration-parameters/settings.md index ee0373c70b4..739a8f7d5ab 100644 --- a/docs/en/operations/server-configuration-parameters/settings.md +++ b/docs/en/operations/server-configuration-parameters/settings.md @@ -66,6 +66,18 @@ If no conditions met for a data part, ClickHouse uses the `lz4` compression.
``` +## core_dump + +Configures soft limit for core dump file size, one gigabyte by default. +```xml + + 1073741824 + +``` + +(Hard limit is configured via system tools) + + ## default\_database {#default-database} The default database. From 90147226954b915417420d63a01642b2f4101b13 Mon Sep 17 00:00:00 2001 From: dmi-feo Date: Mon, 28 Sep 2020 19:25:36 +0300 Subject: [PATCH 145/321] fix insert_quorum_timeout doc https://github.com/ClickHouse/ClickHouse/blob/06d7b6cbef68cfca16460271976ee5f2aa0f35d6/src/Core/Settings.h#L160 --- docs/en/operations/settings/settings.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index e97a418f1ed..dd55ea454c2 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -1144,9 +1144,9 @@ See also: ## insert\_quorum\_timeout {#settings-insert_quorum_timeout} -Write to quorum timeout in seconds. If the timeout has passed and no write has taken place yet, ClickHouse will generate an exception and the client must repeat the query to write the same block to the same or any other replica. +Write to quorum timeout in milliseconds. If the timeout has passed and no write has taken place yet, ClickHouse will generate an exception and the client must repeat the query to write the same block to the same or any other replica. -Default value: 60 seconds. +Default value: 600000 milliseconds. See also: From 347080ff9aace39db72b4e3950d54dbd89ed0419 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Mon, 28 Sep 2020 20:37:11 +0300 Subject: [PATCH 146/321] Remove column from header descriotion for EXPLAIN PLAN and EXPLAIN PIPELINE --- src/Core/ColumnWithTypeAndName.cpp | 7 ++++++- src/Core/ColumnWithTypeAndName.h | 1 + src/Processors/QueryPlan/IQueryPlanStep.cpp | 2 +- src/Processors/QueryPlan/QueryPlan.cpp | 2 +- 4 files changed, 9 insertions(+), 3 deletions(-) diff --git a/src/Core/ColumnWithTypeAndName.cpp b/src/Core/ColumnWithTypeAndName.cpp index 29da8e86439..012645cb065 100644 --- a/src/Core/ColumnWithTypeAndName.cpp +++ b/src/Core/ColumnWithTypeAndName.cpp @@ -27,7 +27,7 @@ bool ColumnWithTypeAndName::operator==(const ColumnWithTypeAndName & other) cons } -void ColumnWithTypeAndName::dumpStructure(WriteBuffer & out) const +void ColumnWithTypeAndName::dumpNameAndType(WriteBuffer & out) const { out << name; @@ -35,6 +35,11 @@ void ColumnWithTypeAndName::dumpStructure(WriteBuffer & out) const out << ' ' << type->getName(); else out << " nullptr"; +} + +void ColumnWithTypeAndName::dumpStructure(WriteBuffer & out) const +{ + dumpNameAndType(out); if (column) out << ' ' << column->dumpStructure(); diff --git a/src/Core/ColumnWithTypeAndName.h b/src/Core/ColumnWithTypeAndName.h index 27b09710258..2ecc2df9b5a 100644 --- a/src/Core/ColumnWithTypeAndName.h +++ b/src/Core/ColumnWithTypeAndName.h @@ -33,6 +33,7 @@ struct ColumnWithTypeAndName ColumnWithTypeAndName cloneEmpty() const; bool operator==(const ColumnWithTypeAndName & other) const; + void dumpNameAndType(WriteBuffer & out) const; void dumpStructure(WriteBuffer & out) const; String dumpStructure() const; }; diff --git a/src/Processors/QueryPlan/IQueryPlanStep.cpp b/src/Processors/QueryPlan/IQueryPlanStep.cpp index 0be40019c58..71c4caaa795 100644 --- a/src/Processors/QueryPlan/IQueryPlanStep.cpp +++ b/src/Processors/QueryPlan/IQueryPlanStep.cpp @@ -45,7 +45,7 @@ static void doDescribeHeader(const Block & header, size_t count, IQueryPlanStep: settings.out << prefix; first = false; - elem.dumpStructure(settings.out); + elem.dumpNameAndType(settings.out); settings.out << '\n'; } } diff --git a/src/Processors/QueryPlan/QueryPlan.cpp b/src/Processors/QueryPlan/QueryPlan.cpp index 74781f4b5d9..1ff844480a9 100644 --- a/src/Processors/QueryPlan/QueryPlan.cpp +++ b/src/Processors/QueryPlan/QueryPlan.cpp @@ -225,7 +225,7 @@ static void explainStep( settings.out << "\n" << prefix << " "; first = false; - elem.dumpStructure(settings.out); + elem.dumpNameAndType(settings.out); } } From d9e5593808f771c8c3968082a769d2e5c8469f8e Mon Sep 17 00:00:00 2001 From: Denny Crane Date: Mon, 28 Sep 2020 15:26:38 -0300 Subject: [PATCH 147/321] Update settings.md fix description of max_server_memory_usage --- docs/en/operations/server-configuration-parameters/settings.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/operations/server-configuration-parameters/settings.md b/docs/en/operations/server-configuration-parameters/settings.md index ee0373c70b4..2510231b4ba 100644 --- a/docs/en/operations/server-configuration-parameters/settings.md +++ b/docs/en/operations/server-configuration-parameters/settings.md @@ -405,7 +405,7 @@ Limits total RAM usage by the ClickHouse server. Possible values: - Positive integer. -- 0 — Unlimited. +- 0 (auto). Default value: `0`. From 6671b26f2057a7ea59e45a276a51e4c913341ea2 Mon Sep 17 00:00:00 2001 From: Alexander Kuzmenkov Date: Mon, 28 Sep 2020 21:43:47 +0300 Subject: [PATCH 148/321] fixes --- docker/test/performance-comparison/compare.sh | 3 ++- docker/test/performance-comparison/perf.py | 19 +++++++++++++------ 2 files changed, 15 insertions(+), 7 deletions(-) diff --git a/docker/test/performance-comparison/compare.sh b/docker/test/performance-comparison/compare.sh index 0a45ad34e8d..7b81b3c562a 100755 --- a/docker/test/performance-comparison/compare.sh +++ b/docker/test/performance-comparison/compare.sh @@ -198,7 +198,8 @@ function run_tests echo test "$test_name" # Don't profile if we're past the time limit. - profile_seconds=$((profile_seconds_left > 0 ? 10 : 0)) + # Use awk because bash doesn't support floating point arithmetics. + profile_seconds=$(awk "BEGIN { print $profile_seconds_left > 0 ? 10 : 0 }") TIMEFORMAT=$(printf "$test_name\t%%3R\t%%3U\t%%3S\n") # The grep is to filter out set -x output and keep only time output. diff --git a/docker/test/performance-comparison/perf.py b/docker/test/performance-comparison/perf.py index 89cf62e4f56..99f395f79b2 100755 --- a/docker/test/performance-comparison/perf.py +++ b/docker/test/performance-comparison/perf.py @@ -130,17 +130,19 @@ if not args.long: sys.exit(0) # Print report threshold for the test if it is set. +ignored_change = 0.05 if 'max_ignored_relative_change' in root.attrib: - print(f'report-threshold\t{root.attrib["max_ignored_relative_change"]}') + ignored_change = float(root.attrib["max_ignored_relative_change"]) + print(f'report-threshold\t{ignored_change}') reportStageEnd('before-connect') # Open connections -servers = [{'host': host, 'port': port} for (host, port) in zip(args.host, args.port)] +servers = [{'host': host or args.host[0], 'port': port or args.port[0]} for (host, port) in itertools.zip_longest(args.host, args.port)] all_connections = [clickhouse_driver.Client(**server) for server in servers] -for s in servers: - print('server\t{}\t{}'.format(s['host'], s['port'])) +for i, s in enumerate(servers): + print(f'server\t{i}\t{s["host"]}\t{s["port"]}') reportStageEnd('connect') @@ -361,9 +363,14 @@ for query_index in queries_to_run: if len(all_server_times) != 2: continue + if len(all_server_times[0]) < 3: + # Don't fail if for some reason there are not enough measurements. + continue + pvalue = stats.ttest_ind(all_server_times[0], all_server_times[1], equal_var = False).pvalue - print(f'pvalue\t{pvalue}') - if pvalue > 0.05: + diff = statistics.median(all_server_times[1]) - statistics.median(all_server_times[0]) + print(f'diff\t{diff}\t{pvalue}') + if abs(diff) < ignored_change or pvalue > 0.05: continue # Perform profile runs for fixed amount of time. Don't limit the number From 81afc197d50544fb4ac816c6722894070364b494 Mon Sep 17 00:00:00 2001 From: Alexander Kuzmenkov Date: Mon, 28 Sep 2020 21:53:28 +0300 Subject: [PATCH 149/321] restart From ac859470a3d9c4504bf3e6c45a42b9e05b90a099 Mon Sep 17 00:00:00 2001 From: Anna Date: Mon, 28 Sep 2020 23:58:08 +0300 Subject: [PATCH 150/321] Translate to Russian --- .../functions/string-functions.md | 2 +- .../sql-reference/functions/url-functions.md | 75 ------------------- .../functions/string-functions.md | 71 ++++++++++++++++++ 3 files changed, 72 insertions(+), 76 deletions(-) diff --git a/docs/en/sql-reference/functions/string-functions.md b/docs/en/sql-reference/functions/string-functions.md index 045b3cd1520..a31237ecfb5 100644 --- a/docs/en/sql-reference/functions/string-functions.md +++ b/docs/en/sql-reference/functions/string-functions.md @@ -500,7 +500,7 @@ normalizeQuery(x) - `x` — Sequence of characters. [String](../../sql-reference/data-types/string.md). -**Returned value(s)** +**Returned value** - Sequence of characters with placeholders. diff --git a/docs/en/sql-reference/functions/url-functions.md b/docs/en/sql-reference/functions/url-functions.md index fe072531aa4..89154ac3d02 100644 --- a/docs/en/sql-reference/functions/url-functions.md +++ b/docs/en/sql-reference/functions/url-functions.md @@ -244,79 +244,4 @@ Removes the query string and fragment identifier. The question mark and number s Removes the ‘name’ URL parameter, if present. This function works under the assumption that the parameter name is encoded in the URL exactly the same way as in the passed argument. -## normalizeQuery {#normalized-query} - -Replaces literals, sequences of literals and complex aliases with placeholders. - -**Syntax** -``` sql -normalizeQuery(x) -``` -!!! note "Note" -SELECT count(*) FROM table WHERE date = '2020-01-02' AND id IN (1, 2, 3) LIMIT 10, 10 -should be replaced to -SELECT count(*) FROM table WHERE date = ? AND id IN (?) LIMIT ? - -**Parameters** - -- `x` — Sequence of characters. [String](../../sql-reference/data-types/string.md). - -**Returned value(s)** - -- Sequence of characters with placeholders. - -Type: [String](../../sql-reference/data-types/string.md). - -**Example** - -Query: - -``` sql -SELECT normalizeQuery('[1, 2, 3, x]') AS query; -``` - -Result: - -``` text -┌─query────┐ -│ [?.., x] │ -└──────────┘ -``` - -## normalizedQueryHash {#normalized-query-hash} - -Returns identical 64bit hash values without the values of literals for similar queries. It helps to analyze query log. - -**Syntax** - -``` sql -normalizedQueryHash(x) -``` - -**Parameters** - -- `x` — Sequence of characters. [String](../../sql-reference/data-types/string.md). - -**Returned value** - -- Hash value. - -Type: [UInt64](../../sql-reference/data-types/int-uint.md#uint-ranges). - -**Example** - -Query: - -``` sql -SELECT normalizedQueryHash('SELECT 1 AS `xyz`') != normalizedQueryHash('SELECT 1 AS `abc`') AS res; -``` - -Result: - -``` text -┌─res─┐ -│ 1 │ -└─────┘ -``` - [Original article](https://clickhouse.tech/docs/en/query_language/functions/url_functions/) diff --git a/docs/ru/sql-reference/functions/string-functions.md b/docs/ru/sql-reference/functions/string-functions.md index a9e254f215b..134c096de84 100644 --- a/docs/ru/sql-reference/functions/string-functions.md +++ b/docs/ru/sql-reference/functions/string-functions.md @@ -479,4 +479,75 @@ SELECT trimBoth(' Hello, world! ') Тип результата — UInt64. +## normalizeQuery {#normalized-query} + +Заменяет литералы, последовательности литералов и сложные псевдонимы заполнителями. + +**Синтаксис** +``` sql +normalizeQuery(x) +``` + +**Параметры** + +- `x` — Последовательность символов. [String](../../sql-reference/data-types/string.md). + +**Возвращаемое значение** + +- Последовательность символов с заполнителями. + +Тип: [String](../../sql-reference/data-types/string.md). + +**Пример** + +Запрос: + +``` sql +SELECT normalizeQuery('[1, 2, 3, x]') AS query; +``` + +Результат: + +``` text +┌─query────┐ +│ [?.., x] │ +└──────────┘ +``` + +## normalizedQueryHash {#normalized-query-hash} + +Возвращает идентичные 64-битные хэш - суммы без значений литералов для аналогичных запросов. Это помогает анализировать журнал запросов. + +**Синтаксис** + +``` sql +normalizedQueryHash(x) +``` + +**Параметры** + +- `x` — Последовательность символов. [String](../../sql-reference/data-types/string.md). + +**Возвращаемое значение** + +- Хэш-сумма. + +Тип: [UInt64](../../sql-reference/data-types/int-uint.md#uint-ranges). + +**Пример** + +Запрос: + +``` sql +SELECT normalizedQueryHash('SELECT 1 AS `xyz`') != normalizedQueryHash('SELECT 1 AS `abc`') AS res; +``` + +Результат: + +``` text +┌─res─┐ +│ 1 │ +└─────┘ +``` + [Оригинальная статья](https://clickhouse.tech/docs/ru/query_language/functions/string_functions/) From e4800e7b35aa17021d5aed4a6fbfbfe75e441301 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 29 Sep 2020 00:10:36 +0300 Subject: [PATCH 151/321] Fix comment --- src/Processors/Formats/Impl/RawBLOBRowOutputFormat.h | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/src/Processors/Formats/Impl/RawBLOBRowOutputFormat.h b/src/Processors/Formats/Impl/RawBLOBRowOutputFormat.h index 8062092f5cd..b6c13597d4b 100644 --- a/src/Processors/Formats/Impl/RawBLOBRowOutputFormat.h +++ b/src/Processors/Formats/Impl/RawBLOBRowOutputFormat.h @@ -10,16 +10,16 @@ namespace DB class WriteBuffer; -/** This format only allows to output one column of type String or similar. - * It is output as raw bytes without any delimiters or escaping. +/** This format only allows to output columns of type String + * or types that have contiguous representation in memory. + * They are output as raw bytes without any delimiters or escaping. * * The difference between RawBLOB and TSVRaw: - * - only single column dataset is supported; - * - data is output in binary; + * - data is output in binary, no escaping; + * - no delimiters between values; * - no newline at the end of each value. * * The difference between RawBLOB and RowBinary: - * - only single column dataset is supported; * - strings are output without their lengths. * * If you are output more than one value, the output format is ambiguous and you may not be able to read data back. From e65465ac1675219edd8ac1bc7f7f4d3e1f3d11b9 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 29 Sep 2020 00:11:08 +0300 Subject: [PATCH 152/321] Fix install script --- 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 bd60fbb63ba..ae0c22c8fcc 100644 --- a/programs/install/Install.cpp +++ b/programs/install/Install.cpp @@ -552,7 +552,7 @@ int mainEntryClickHouseInstall(int argc, char ** argv) #if defined(__linux__) fmt::print("Setting capabilities for clickhouse binary. This is optional.\n"); - std::string command = fmt::format("command setcap && setcap 'cap_net_admin,cap_ipc_lock,cap_sys_nice+ep' {}", main_bin_path.string()); + std::string command = fmt::format("command -v setcap && setcap 'cap_net_admin,cap_ipc_lock,cap_sys_nice+ep' {}", main_bin_path.string()); fmt::print(" {}\n", command); executeScript(command); #endif From dee57d7976bda06f2b89d3327e8f4bf26abaa018 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 29 Sep 2020 00:19:13 +0300 Subject: [PATCH 153/321] Format Regexp: change default escaping rule to Raw --- src/Core/Settings.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 9449cd571a1..19738d826c6 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -462,7 +462,7 @@ class IColumn; M(String, format_custom_result_after_delimiter, "", "Suffix after result set (for CustomSeparated format)", 0) \ \ M(String, format_regexp, "", "Regular expression (for Regexp format)", 0) \ - M(String, format_regexp_escaping_rule, "Escaped", "Field escaping rule (for Regexp format)", 0) \ + M(String, format_regexp_escaping_rule, "Raw", "Field escaping rule (for Regexp format)", 0) \ M(Bool, format_regexp_skip_unmatched, false, "Skip lines unmatched by regular expression (for Regexp format", 0) \ \ M(Bool, output_format_enable_streaming, false, "Enable streaming in output formats that support it.", 0) \ From 9e94c077e7561238414586fd7d070031fa646f49 Mon Sep 17 00:00:00 2001 From: zhang2014 Date: Tue, 29 Sep 2020 10:39:30 +0800 Subject: [PATCH 154/321] IESSUES-15365 try fix integration test --- src/Databases/ya.make | 2 +- tests/integration/test_mysql_database_engine/test.py | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/src/Databases/ya.make b/src/Databases/ya.make index 75adf1e9c5c..b4173057e03 100644 --- a/src/Databases/ya.make +++ b/src/Databases/ya.make @@ -17,13 +17,13 @@ SRCS( DatabaseOrdinary.cpp DatabasesCommon.cpp DatabaseWithDictionaries.cpp + MySQL/ConnectionMySQLSettings.cpp MySQL/DatabaseConnectionMySQL.cpp MySQL/DatabaseMaterializeMySQL.cpp MySQL/FetchTablesColumnsList.cpp MySQL/MaterializeMetadata.cpp MySQL/MaterializeMySQLSettings.cpp MySQL/MaterializeMySQLSyncThread.cpp - MySQL/ConnectionMySQLSettings.cpp ) diff --git a/tests/integration/test_mysql_database_engine/test.py b/tests/integration/test_mysql_database_engine/test.py index 26dd2ee42ea..0e79d9ebcbc 100644 --- a/tests/integration/test_mysql_database_engine/test.py +++ b/tests/integration/test_mysql_database_engine/test.py @@ -166,7 +166,7 @@ def test_bad_arguments_for_mysql_database_engine(started_cluster): def test_data_types_support_level_for_mysql_database_engine(started_cluster): with contextlib.closing(MySQLNodeInstance('root', 'clickhouse', '127.0.0.1', port=3308)) as mysql_node: mysql_node.query("CREATE DATABASE IF NOT EXISTS test DEFAULT CHARACTER SET 'utf8'") - clickhouse_node.query("CREATE DATABASE test_database ENGINE = MySQL('mysql1:3306', test_database, 'root', 'clickhouse')", + clickhouse_node.query("CREATE DATABASE test_database ENGINE = MySQL('mysql1:3306', test, 'root', 'clickhouse')", settings={"mysql_datatypes_support_level": "decimal,datetime64"}) assert "SETTINGS mysql_datatypes_support_level = \'decimal,datetime64\'" in clickhouse_node.query("SHOW CREATE DATABASE test_database FORMAT TSV") @@ -177,7 +177,7 @@ def test_data_types_support_level_for_mysql_database_engine(started_cluster): assert "SETTINGS mysql_datatypes_support_level = \'decimal,datetime64\'" in clickhouse_node.query("SHOW CREATE DATABASE test_database FORMAT TSV") clickhouse_node.query( - "CREATE DATABASE test_database_1 ENGINE = MySQL('mysql1:3306', test_database, 'root', 'clickhouse') SETTINGS mysql_datatypes_support_level = 'decimal,datetime64'", + "CREATE DATABASE test_database_1 ENGINE = MySQL('mysql1:3306', test, 'root', 'clickhouse') SETTINGS mysql_datatypes_support_level = 'decimal,datetime64'", settings={"mysql_datatypes_support_level": "decimal"}) assert "SETTINGS mysql_datatypes_support_level = \'decimal,datetime64\'" in clickhouse_node.query("SHOW CREATE DATABASE test_database_1 FORMAT TSV") From 18985645d94f819e5d931b705115c42196f6c845 Mon Sep 17 00:00:00 2001 From: zhang2014 Date: Tue, 29 Sep 2020 12:17:49 +0800 Subject: [PATCH 155/321] ISSUES-15365 try fix integration test failure --- tests/integration/test_mysql_database_engine/test.py | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/tests/integration/test_mysql_database_engine/test.py b/tests/integration/test_mysql_database_engine/test.py index 0e79d9ebcbc..b67e9a1cb91 100644 --- a/tests/integration/test_mysql_database_engine/test.py +++ b/tests/integration/test_mysql_database_engine/test.py @@ -169,23 +169,23 @@ def test_data_types_support_level_for_mysql_database_engine(started_cluster): clickhouse_node.query("CREATE DATABASE test_database ENGINE = MySQL('mysql1:3306', test, 'root', 'clickhouse')", settings={"mysql_datatypes_support_level": "decimal,datetime64"}) - assert "SETTINGS mysql_datatypes_support_level = \'decimal,datetime64\'" in clickhouse_node.query("SHOW CREATE DATABASE test_database FORMAT TSV") + assert "SETTINGS mysql_datatypes_support_level = \\'decimal,datetime64\\'" in clickhouse_node.query("SHOW CREATE DATABASE test_database FORMAT TSV") clickhouse_node.query("DETACH DATABASE test_database") # without context settings clickhouse_node.query("ATTACH DATABASE test_database") - assert "SETTINGS mysql_datatypes_support_level = \'decimal,datetime64\'" in clickhouse_node.query("SHOW CREATE DATABASE test_database FORMAT TSV") + assert "SETTINGS mysql_datatypes_support_level = \\'decimal,datetime64\\'" in clickhouse_node.query("SHOW CREATE DATABASE test_database FORMAT TSV") clickhouse_node.query( "CREATE DATABASE test_database_1 ENGINE = MySQL('mysql1:3306', test, 'root', 'clickhouse') SETTINGS mysql_datatypes_support_level = 'decimal,datetime64'", settings={"mysql_datatypes_support_level": "decimal"}) - assert "SETTINGS mysql_datatypes_support_level = \'decimal,datetime64\'" in clickhouse_node.query("SHOW CREATE DATABASE test_database_1 FORMAT TSV") + assert "SETTINGS mysql_datatypes_support_level = \\'decimal,datetime64\\'" in clickhouse_node.query("SHOW CREATE DATABASE test_database_1 FORMAT TSV") clickhouse_node.query("DETACH DATABASE test_database_1") # without context settings clickhouse_node.query("ATTACH DATABASE test_database_1") - assert "SETTINGS mysql_datatypes_support_level = \'decimal,datetime64\'" in clickhouse_node.query("SHOW CREATE DATABASE test_database_1 FORMAT TSV") + assert "SETTINGS mysql_datatypes_support_level = \\'decimal,datetime64\\'" in clickhouse_node.query("SHOW CREATE DATABASE test_database_1 FORMAT TSV") clickhouse_node.query("DROP DATABASE test_database") assert 'test_database' not in clickhouse_node.query('SHOW DATABASES') From a3bc152f73bf0679e1b63f91079ed149a7001ab0 Mon Sep 17 00:00:00 2001 From: zhang2014 Date: Tue, 29 Sep 2020 14:37:19 +0800 Subject: [PATCH 156/321] ISSUES-15365 try fix integration test failure --- tests/integration/test_mysql_database_engine/test.py | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/integration/test_mysql_database_engine/test.py b/tests/integration/test_mysql_database_engine/test.py index b67e9a1cb91..a8824b383ab 100644 --- a/tests/integration/test_mysql_database_engine/test.py +++ b/tests/integration/test_mysql_database_engine/test.py @@ -188,6 +188,7 @@ def test_data_types_support_level_for_mysql_database_engine(started_cluster): assert "SETTINGS mysql_datatypes_support_level = \\'decimal,datetime64\\'" in clickhouse_node.query("SHOW CREATE DATABASE test_database_1 FORMAT TSV") clickhouse_node.query("DROP DATABASE test_database") + clickhouse_node.query("DROP DATABASE test_database_1") assert 'test_database' not in clickhouse_node.query('SHOW DATABASES') mysql_node.query("DROP DATABASE test") From eafcec7ebf984afe3ae2e0cf05863201f97f5c05 Mon Sep 17 00:00:00 2001 From: alesapin Date: Tue, 29 Sep 2020 10:15:05 +0300 Subject: [PATCH 157/321] Move release build to clang-11 --- docker/packager/binary/Dockerfile | 2 +- docker/packager/deb/Dockerfile | 9 +++++++-- tests/ci/ci_config.json | 20 ++++++++++---------- 3 files changed, 18 insertions(+), 13 deletions(-) diff --git a/docker/packager/binary/Dockerfile b/docker/packager/binary/Dockerfile index b96d53a6ea0..bc17998fb59 100644 --- a/docker/packager/binary/Dockerfile +++ b/docker/packager/binary/Dockerfile @@ -11,7 +11,7 @@ RUN apt-get update \ && echo "${LLVM_PUBKEY_HASH} /tmp/llvm-snapshot.gpg.key" | sha384sum -c \ && apt-key add /tmp/llvm-snapshot.gpg.key \ && export CODENAME="$(lsb_release --codename --short | tr 'A-Z' 'a-z')" \ - && echo "deb [trusted=yes] http://apt.llvm.org/${CODENAME}/ llvm-toolchain-${CODENAME}-11 main" >> \ + && echo "deb [trusted=yes] http://apt.llvm.org/${CODENAME}/ llvm-toolchain-${CODENAME}-${LLVM_VERSION} main" >> \ /etc/apt/sources.list # initial packages diff --git a/docker/packager/deb/Dockerfile b/docker/packager/deb/Dockerfile index df648ab97df..5bd8a74cb99 100644 --- a/docker/packager/deb/Dockerfile +++ b/docker/packager/deb/Dockerfile @@ -11,7 +11,7 @@ RUN apt-get update \ && echo "${LLVM_PUBKEY_HASH} /tmp/llvm-snapshot.gpg.key" | sha384sum -c \ && apt-key add /tmp/llvm-snapshot.gpg.key \ && export CODENAME="$(lsb_release --codename --short | tr 'A-Z' 'a-z')" \ - && echo "deb [trusted=yes] http://apt.llvm.org/${CODENAME}/ llvm-toolchain-${CODENAME}-11 main" >> \ + && echo "deb [trusted=yes] http://apt.llvm.org/${CODENAME}/ llvm-toolchain-${CODENAME}-${LLVM_VERSION} main" >> \ /etc/apt/sources.list # initial packages @@ -49,6 +49,11 @@ RUN apt-get update \ lld-11 \ llvm-11 \ llvm-11-dev \ + clang-10 \ + clang-tidy-10 \ + lld-10 \ + llvm-10 \ + llvm-10-dev \ ninja-build \ perl \ pkg-config \ @@ -75,7 +80,7 @@ RUN apt-get update \ RUN rm /etc/apt/sources.list.d/proposed-repositories.list && apt-get update # This symlink required by gcc to find lld compiler -RUN ln -s /usr/bin/lld-11 /usr/bin/ld.lld +RUN ln -s /usr/bin/lld-${LLVM_VERSION} /usr/bin/ld.lld COPY build.sh / diff --git a/tests/ci/ci_config.json b/tests/ci/ci_config.json index 76d535b843f..66e91dfdc07 100644 --- a/tests/ci/ci_config.json +++ b/tests/ci/ci_config.json @@ -1,7 +1,7 @@ { "build_config": [ { - "compiler": "gcc-10", + "compiler": "clang-11", "build-type": "", "sanitizer": "", "package-type": "deb", @@ -124,7 +124,7 @@ "with_coverage": true }, { - "compiler": "clang-11", + "compiler": "clang-10", "build-type": "", "sanitizer": "", "package-type": "binary", @@ -227,7 +227,7 @@ }, "Functional stateful tests (release)": { "required_build_properties": { - "compiler": "gcc-10", + "compiler": "clang-11", "package_type": "deb", "build_type": "relwithdebuginfo", "sanitizer": "none", @@ -239,7 +239,7 @@ }, "Functional stateful tests (release, DatabaseOrdinary)": { "required_build_properties": { - "compiler": "gcc-10", + "compiler": "clang-11", "package_type": "deb", "build_type": "relwithdebuginfo", "sanitizer": "none", @@ -311,7 +311,7 @@ }, "Functional stateless tests (release)": { "required_build_properties": { - "compiler": "gcc-10", + "compiler": "clang-11", "package_type": "deb", "build_type": "relwithdebuginfo", "sanitizer": "none", @@ -335,7 +335,7 @@ }, "Functional stateless tests (release, polymorphic parts enabled)": { "required_build_properties": { - "compiler": "gcc-10", + "compiler": "clang-11", "package_type": "deb", "build_type": "relwithdebuginfo", "sanitizer": "none", @@ -347,7 +347,7 @@ }, "Functional stateless tests (release, DatabaseOrdinary)": { "required_build_properties": { - "compiler": "gcc-10", + "compiler": "clang-11", "package_type": "deb", "build_type": "relwithdebuginfo", "sanitizer": "none", @@ -431,7 +431,7 @@ }, "Integration tests (release)": { "required_build_properties": { - "compiler": "gcc-10", + "compiler": "clang-11", "package_type": "deb", "build_type": "relwithdebuginfo", "sanitizer": "none", @@ -455,7 +455,7 @@ }, "Compatibility check": { "required_build_properties": { - "compiler": "gcc-10", + "compiler": "clang-11", "package_type": "deb", "build_type": "relwithdebuginfo", "sanitizer": "none", @@ -479,7 +479,7 @@ }, "Testflows check": { "required_build_properties": { - "compiler": "gcc-10", + "compiler": "clang-11", "package_type": "deb", "build_type": "relwithdebuginfo", "sanitizer": "none", From 9f89fe095a8312ba0ff11ae064ef98c1ed157aee Mon Sep 17 00:00:00 2001 From: alesapin Date: Tue, 29 Sep 2020 10:17:02 +0300 Subject: [PATCH 158/321] Missed changes --- docker/packager/binary/Dockerfile | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/docker/packager/binary/Dockerfile b/docker/packager/binary/Dockerfile index bc17998fb59..168ea895eec 100644 --- a/docker/packager/binary/Dockerfile +++ b/docker/packager/binary/Dockerfile @@ -32,6 +32,11 @@ RUN apt-get update \ curl \ gcc-9 \ g++-9 \ + clang-10 \ + clang-tidy-10 \ + lld-10 \ + llvm-10 \ + llvm-10-dev \ clang-11 \ clang-tidy-11 \ lld-11 \ From e77c11a2c96f861271af43937e01b147259ccecb Mon Sep 17 00:00:00 2001 From: alesapin Date: Tue, 29 Sep 2020 10:19:03 +0300 Subject: [PATCH 159/321] Fix config --- tests/ci/ci_config.json | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/ci/ci_config.json b/tests/ci/ci_config.json index 66e91dfdc07..7f428c1ea59 100644 --- a/tests/ci/ci_config.json +++ b/tests/ci/ci_config.json @@ -467,7 +467,7 @@ }, "Split build smoke test": { "required_build_properties": { - "compiler": "clang-11", + "compiler": "clang-10", "package_type": "binary", "build_type": "relwithdebuginfo", "sanitizer": "none", From f9f83611143016a65ae67b61df5f0a06a15ca721 Mon Sep 17 00:00:00 2001 From: alesapin Date: Tue, 29 Sep 2020 10:21:51 +0300 Subject: [PATCH 160/321] Better ci config --- tests/ci/ci_config.json | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/tests/ci/ci_config.json b/tests/ci/ci_config.json index 7f428c1ea59..f2eab2c65c4 100644 --- a/tests/ci/ci_config.json +++ b/tests/ci/ci_config.json @@ -72,7 +72,7 @@ "with_coverage": false }, { - "compiler": "clang-11", + "compiler": "clang-10", "build-type": "", "sanitizer": "", "package-type": "deb", @@ -102,7 +102,7 @@ "with_coverage": false }, { - "compiler": "clang-11", + "compiler": "clang-10", "build-type": "", "sanitizer": "", "package-type": "binary", @@ -124,7 +124,7 @@ "with_coverage": true }, { - "compiler": "clang-10", + "compiler": "clang-11", "build-type": "", "sanitizer": "", "package-type": "binary", @@ -467,7 +467,7 @@ }, "Split build smoke test": { "required_build_properties": { - "compiler": "clang-10", + "compiler": "clang-11", "package_type": "binary", "build_type": "relwithdebuginfo", "sanitizer": "none", From 789434994a3760bb0cbddad50bb00c8080751af9 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Tue, 22 Sep 2020 12:14:37 +0300 Subject: [PATCH 161/321] Reset job on failure too in ThreadPool::worker() job should be reseted before decrementing scheduled_jobs to ensure that the Job destroyed before wait() returns. --- src/Common/ThreadPool.cpp | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/src/Common/ThreadPool.cpp b/src/Common/ThreadPool.cpp index dda16c7725d..21116e9d432 100644 --- a/src/Common/ThreadPool.cpp +++ b/src/Common/ThreadPool.cpp @@ -234,10 +234,16 @@ void ThreadPoolImpl::worker(typename std::list::iterator thread_ std::is_same_v ? CurrentMetrics::GlobalThreadActive : CurrentMetrics::LocalThreadActive); job(); + /// job should be reseted before decrementing scheduled_jobs to + /// ensure that the Job destroyed before wait() returns. job = {}; } catch (...) { + /// job should be reseted before decrementing scheduled_jobs to + /// ensure that the Job destroyed before wait() returns. + job = {}; + { std::unique_lock lock(mutex); if (!first_exception) From eeea4106c2f44bd0885cbe58cb37c70f76499c1a Mon Sep 17 00:00:00 2001 From: alesapin Date: Tue, 29 Sep 2020 10:59:23 +0300 Subject: [PATCH 162/321] Suppress race in OpenSSL --- tests/tsan_suppressions.txt | 2 ++ 1 file changed, 2 insertions(+) diff --git a/tests/tsan_suppressions.txt b/tests/tsan_suppressions.txt index 668710a33d7..c07a31f2127 100644 --- a/tests/tsan_suppressions.txt +++ b/tests/tsan_suppressions.txt @@ -1,2 +1,4 @@ # looks like a bug in clang-11 thread sanitizer, detects normal data race with random FD in this method race:DB::LazyPipeFDs::close +# races in openSSL https://github.com/openssl/openssl/issues/11974 +race:evp_cipher_cache_constants From 1fcc36ef800b21e8ae43e984583b01c62a5c42c6 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Tue, 22 Sep 2020 12:15:57 +0300 Subject: [PATCH 163/321] Use ThreadPool to ensure that everything has been finished in PipelineExecutor This will fix possible UAF, since ThreadFromGlobalPool cannot wait until the job will be reseted, for this you need to call ThreadPool::wait() (that waits until scheduled_jobs will be zero), and this guarantee that the job was reseted, otherwise some variables that job is referencing may be already destroyed, while job hasn't been destoyed yet (example that I found PipelineExecutor -> ThreadGroupStatusPtr -> MemoryTracker -> ~MemoryTracker -> log, while log had been destroyed already, here is TSAN report -- https://gist.github.com/azat/d480dc3af5a0a44de4b038e20807c4b9, copied from https://clickhouse-test-reports.s3.yandex.net/15035/79133a426fdf042e383ea0cdccc4dc8273baa3a7/functional_stateless_tests_(thread)/test_run.txt.out.log) --- src/Processors/Executors/PipelineExecutor.cpp | 22 +++++++++---------- 1 file changed, 11 insertions(+), 11 deletions(-) diff --git a/src/Processors/Executors/PipelineExecutor.cpp b/src/Processors/Executors/PipelineExecutor.cpp index 858f53cb047..9a9f5d53354 100644 --- a/src/Processors/Executors/PipelineExecutor.cpp +++ b/src/Processors/Executors/PipelineExecutor.cpp @@ -694,9 +694,7 @@ void PipelineExecutor::executeImpl(size_t num_threads) { initializeExecution(num_threads); - using ThreadsData = std::vector; - ThreadsData threads; - threads.reserve(num_threads); + ThreadPool threads(num_threads); bool finished_flag = false; @@ -704,10 +702,7 @@ void PipelineExecutor::executeImpl(size_t num_threads) if (!finished_flag) { finish(); - - for (auto & thread : threads) - if (thread.joinable()) - thread.join(); + threads.wait(); } ); @@ -717,7 +712,7 @@ void PipelineExecutor::executeImpl(size_t num_threads) for (size_t i = 0; i < num_threads; ++i) { - threads.emplace_back([this, thread_group, thread_num = i, num_threads] + threads.scheduleOrThrowOnError([this, thread_group, thread_num = i, num_threads] { /// ThreadStatus thread_status; @@ -744,9 +739,14 @@ void PipelineExecutor::executeImpl(size_t num_threads) }); } - for (auto & thread : threads) - if (thread.joinable()) - thread.join(); + /// Because ThreadPool::wait() waits until scheduled_jobs will be zero, + /// and this guarantee that the job was reseted, otherwise + /// some variables that job is referencing may be already destroyed, + /// while job hasn't been destoyed yet (example that pops up -- + /// PipelineExecutor -> ThreadGroupStatusPtr -> MemoryTracker -> + /// ~MemoryTracker -> log, while log had been destroyed already) + /// (see 01505_pipeline_executor_UAF) + threads.wait(); } else executeSingleThread(0, num_threads); From d553e56db73ab672891ca15af6ada801ab79bbd2 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Tue, 22 Sep 2020 08:18:52 +0300 Subject: [PATCH 164/321] Add a test for UAF in ThreadPool (triggered under ASAN) --- .../01505_pipeline_executor_UAF.reference | 0 .../0_stateless/01505_pipeline_executor_UAF.sh | 12 ++++++++++++ 2 files changed, 12 insertions(+) create mode 100644 tests/queries/0_stateless/01505_pipeline_executor_UAF.reference create mode 100755 tests/queries/0_stateless/01505_pipeline_executor_UAF.sh diff --git a/tests/queries/0_stateless/01505_pipeline_executor_UAF.reference b/tests/queries/0_stateless/01505_pipeline_executor_UAF.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/01505_pipeline_executor_UAF.sh b/tests/queries/0_stateless/01505_pipeline_executor_UAF.sh new file mode 100755 index 00000000000..283e6662a43 --- /dev/null +++ b/tests/queries/0_stateless/01505_pipeline_executor_UAF.sh @@ -0,0 +1,12 @@ +#!/usr/bin/env bash + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +. "$CURDIR"/../shell_config.sh + +# Regression for UAF in ThreadPool. +# (Triggered under TSAN) +for i in {1..10}; do + ${CLICKHOUSE_LOCAL} -q 'select * from numbers_mt(100000000) settings max_threads=100 FORMAT Null' + # Binding to specific CPU is not required, but this makes the test more reliable. + taskset --cpu-list 0 ${CLICKHOUSE_LOCAL} -q 'select * from numbers_mt(100000000) settings max_threads=100 FORMAT Null' +done From 7270de2b0a12785181951e311cecfd46ebd99a15 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 29 Sep 2020 11:12:43 +0300 Subject: [PATCH 165/321] Added test. --- tests/queries/0_stateless/01508_explain_header.reference | 7 +++++++ tests/queries/0_stateless/01508_explain_header.sql | 1 + 2 files changed, 8 insertions(+) create mode 100644 tests/queries/0_stateless/01508_explain_header.reference create mode 100644 tests/queries/0_stateless/01508_explain_header.sql diff --git a/tests/queries/0_stateless/01508_explain_header.reference b/tests/queries/0_stateless/01508_explain_header.reference new file mode 100644 index 00000000000..50216432e14 --- /dev/null +++ b/tests/queries/0_stateless/01508_explain_header.reference @@ -0,0 +1,7 @@ +Expression (Projection) +Header: x UInt8 + Expression (Before ORDER BY and SELECT) + Header: _dummy UInt8 + 1 UInt8 + ReadFromStorage (Read from SystemOne) + Header: dummy UInt8 diff --git a/tests/queries/0_stateless/01508_explain_header.sql b/tests/queries/0_stateless/01508_explain_header.sql new file mode 100644 index 00000000000..fb274c84d8a --- /dev/null +++ b/tests/queries/0_stateless/01508_explain_header.sql @@ -0,0 +1 @@ +explain header = 1 select 1 as x; From 6b31bcb0740419687367afb442fd4a80c4c8da8b Mon Sep 17 00:00:00 2001 From: alesapin Date: Tue, 29 Sep 2020 11:13:47 +0300 Subject: [PATCH 166/321] Add release task config --- tests/ci/ci_config.json | 12 ++++++++++++ 1 file changed, 12 insertions(+) diff --git a/tests/ci/ci_config.json b/tests/ci/ci_config.json index c12c6dad999..7626ac5a904 100644 --- a/tests/ci/ci_config.json +++ b/tests/ci/ci_config.json @@ -572,6 +572,18 @@ "clang-tidy": "disable", "with_coverage": false } + }, + "Release": { + "required_build_properties": { + "compiler": "gcc-10", + "package_type": "deb", + "build_type": "relwithdebuginfo", + "sanitizer": "none", + "bundled": "bundled", + "splitted": "unsplitted", + "clang-tidy": "disable", + "with_coverage": false + } } } } From 2be3854e616ead81906ec8ef34a9144756078c45 Mon Sep 17 00:00:00 2001 From: alesapin Date: Tue, 29 Sep 2020 11:15:33 +0300 Subject: [PATCH 167/321] Move release to clang-11 --- tests/ci/ci_config.json | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/ci/ci_config.json b/tests/ci/ci_config.json index cfe99d2b31e..6810b54bd32 100644 --- a/tests/ci/ci_config.json +++ b/tests/ci/ci_config.json @@ -575,7 +575,7 @@ }, "Release": { "required_build_properties": { - "compiler": "gcc-10", + "compiler": "clang-11", "package_type": "deb", "build_type": "relwithdebuginfo", "sanitizer": "none", From 0adcb0a9a360e87e6fee59aa11f16edaa22b795f Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Mon, 21 Sep 2020 01:07:56 +0300 Subject: [PATCH 168/321] Set sanitizers options in the current shell too /etc/environment will not be read in current shell, so export variables from that file manually, to make clickhouse-local/clickhouse-client use them too. --- docker/test/base/Dockerfile | 7 ++++++- docker/test/stress/run.sh | 3 +++ 2 files changed, 9 insertions(+), 1 deletion(-) diff --git a/docker/test/base/Dockerfile b/docker/test/base/Dockerfile index aa3f1d738c2..506e32c18b3 100644 --- a/docker/test/base/Dockerfile +++ b/docker/test/base/Dockerfile @@ -48,10 +48,15 @@ RUN apt-get update \ tzdata \ --yes --no-install-recommends -# Sanitizer options +# Sanitizer options for services (clickhouse-server) RUN echo "TSAN_OPTIONS='verbosity=1000 halt_on_error=1 history_size=7'" >> /etc/environment; \ echo "UBSAN_OPTIONS='print_stacktrace=1'" >> /etc/environment; \ echo "MSAN_OPTIONS='abort_on_error=1'" >> /etc/environment; \ ln -s /usr/lib/llvm-${LLVM_VERSION}/bin/llvm-symbolizer /usr/bin/llvm-symbolizer; +# Sanitizer options for current shell (not current, but the one that will be spawned on "docker run") +# (but w/o verbosity for TSAN, otherwise test.reference will not match) +ENV TSAN_OPTIONS='halt_on_error=1 history_size=7' +ENV UBSAN_OPTIONS='print_stacktrace=1' +ENV MSAN_OPTIONS='abort_on_error=1' CMD sleep 1 diff --git a/docker/test/stress/run.sh b/docker/test/stress/run.sh index 28c66a72d39..14f0301d43e 100755 --- a/docker/test/stress/run.sh +++ b/docker/test/stress/run.sh @@ -42,7 +42,10 @@ function start() # install test configs /usr/share/clickhouse-test/config/install.sh +# for clickhouse-server (via service) echo "ASAN_OPTIONS='malloc_context_size=10 verbosity=1 allocator_release_to_os_interval_ms=10000'" >> /etc/environment +# for clickhouse-client +export ASAN_OPTIONS='malloc_context_size=10 verbosity=1 allocator_release_to_os_interval_ms=10000' start From 5df5fb4cd0b13a3234e469a25883dff7df2ed74b Mon Sep 17 00:00:00 2001 From: alesapin Date: Mon, 28 Sep 2020 12:58:42 +0300 Subject: [PATCH 169/321] Fix race condition in client suggestions --- base/common/LineReader.cpp | 4 ++-- base/common/LineReader.h | 3 ++- base/common/ReadlineLineReader.cpp | 3 ++- base/common/ReplxxLineReader.cpp | 5 +++-- 4 files changed, 9 insertions(+), 6 deletions(-) diff --git a/base/common/LineReader.cpp b/base/common/LineReader.cpp index dd2e09b0393..b2bc929a1df 100644 --- a/base/common/LineReader.cpp +++ b/base/common/LineReader.cpp @@ -38,10 +38,10 @@ bool hasInputData() } -LineReader::Suggest::WordsRange LineReader::Suggest::getCompletions(const String & prefix, size_t prefix_length) const +std::optional LineReader::Suggest::getCompletions(const String & prefix, size_t prefix_length) const { if (!ready) - return std::make_pair(words.end(), words.end()); + return std::nullopt; std::string_view last_word; diff --git a/base/common/LineReader.h b/base/common/LineReader.h index 77dc70d8808..159a93ce40d 100644 --- a/base/common/LineReader.h +++ b/base/common/LineReader.h @@ -4,6 +4,7 @@ #include #include +#include class LineReader { @@ -18,7 +19,7 @@ public: std::atomic ready{false}; /// Get iterators for the matched range of words if any. - WordsRange getCompletions(const String & prefix, size_t prefix_length) const; + std::optional getCompletions(const String & prefix, size_t prefix_length) const; }; using Patterns = std::vector; diff --git a/base/common/ReadlineLineReader.cpp b/base/common/ReadlineLineReader.cpp index 095df319acc..4e67ed5e4a3 100644 --- a/base/common/ReadlineLineReader.cpp +++ b/base/common/ReadlineLineReader.cpp @@ -30,7 +30,8 @@ static LineReader::Suggest::Words::const_iterator end; static void findRange(const char * prefix, size_t prefix_length) { std::string prefix_str(prefix); - std::tie(pos, end) = suggest->getCompletions(prefix_str, prefix_length); + if (auto completions = suggest->getCompletions(prefix_str, prefix_length)) + std::tie(pos, end) = *completions; } /// Iterates through matched range. diff --git a/base/common/ReplxxLineReader.cpp b/base/common/ReplxxLineReader.cpp index cc8b963958c..c4f9c65a116 100644 --- a/base/common/ReplxxLineReader.cpp +++ b/base/common/ReplxxLineReader.cpp @@ -70,8 +70,9 @@ ReplxxLineReader::ReplxxLineReader( auto callback = [&suggest] (const String & context, size_t context_size) { - auto range = suggest.getCompletions(context, context_size); - return Replxx::completions_t(range.first, range.second); + if (auto range = suggest.getCompletions(context, context_size)) + return Replxx::completions_t(range->first, range->second); + return Replxx::completions_t(); }; rx.set_completion_callback(callback); From 142c482210bfedcd7b6cdb2873d84335e2889a05 Mon Sep 17 00:00:00 2001 From: bharatnc Date: Sat, 5 Sep 2020 17:01:51 -0700 Subject: [PATCH 170/321] TextLog - add event_time_microseconds field --- base/loggers/ExtendedLogChannel.cpp | 1 + base/loggers/ExtendedLogChannel.h | 1 + base/loggers/OwnSplitChannel.cpp | 1 + src/Interpreters/TextLog.cpp | 3 +++ src/Interpreters/TextLog.h | 1 + 5 files changed, 7 insertions(+) diff --git a/base/loggers/ExtendedLogChannel.cpp b/base/loggers/ExtendedLogChannel.cpp index 421113b425f..4ab6ad5c96a 100644 --- a/base/loggers/ExtendedLogChannel.cpp +++ b/base/loggers/ExtendedLogChannel.cpp @@ -23,6 +23,7 @@ ExtendedLogMessage ExtendedLogMessage::getFrom(const Poco::Message & base) msg_ext.time_seconds = static_cast(tv.tv_sec); msg_ext.time_microseconds = static_cast(tv.tv_usec); + msg_ext.time_in_microseconds = static_cast((tv.tv_sec) * 1000000U + (tv.tv_usec)); if (current_thread) { diff --git a/base/loggers/ExtendedLogChannel.h b/base/loggers/ExtendedLogChannel.h index 3e9b61e8ae4..771a27159f0 100644 --- a/base/loggers/ExtendedLogChannel.h +++ b/base/loggers/ExtendedLogChannel.h @@ -23,6 +23,7 @@ public: uint32_t time_seconds = 0; uint32_t time_microseconds = 0; + uint64_t time_in_microseconds = 0; uint64_t thread_id = 0; std::string query_id; diff --git a/base/loggers/OwnSplitChannel.cpp b/base/loggers/OwnSplitChannel.cpp index 3ae928864fb..fc167ec3bf6 100644 --- a/base/loggers/OwnSplitChannel.cpp +++ b/base/loggers/OwnSplitChannel.cpp @@ -76,6 +76,7 @@ void OwnSplitChannel::logSplit(const Poco::Message & msg) TextLogElement elem; elem.event_time = msg_ext.time_seconds; + elem.event_time_microseconds = msg_ext.time_in_microseconds; elem.microseconds = msg_ext.time_microseconds; elem.thread_name = getThreadName(); diff --git a/src/Interpreters/TextLog.cpp b/src/Interpreters/TextLog.cpp index 243bf6d299a..d1cc6a052e8 100644 --- a/src/Interpreters/TextLog.cpp +++ b/src/Interpreters/TextLog.cpp @@ -2,6 +2,7 @@ #include #include #include +#include #include #include #include @@ -29,6 +30,7 @@ Block TextLogElement::createBlock() { {std::make_shared(), "event_date"}, {std::make_shared(), "event_time"}, + {std::make_shared(6), "event_time_microseconds"}, {std::make_shared(), "microseconds"}, {std::make_shared(std::make_shared()), "thread_name"}, @@ -52,6 +54,7 @@ void TextLogElement::appendToBlock(MutableColumns & columns) const columns[i++]->insert(DateLUT::instance().toDayNum(event_time)); columns[i++]->insert(event_time); + columns[i++]->insert(event_time_microseconds); columns[i++]->insert(microseconds); columns[i++]->insertData(thread_name.data(), thread_name.size()); diff --git a/src/Interpreters/TextLog.h b/src/Interpreters/TextLog.h index d5d1610dfb5..814b3c73044 100644 --- a/src/Interpreters/TextLog.h +++ b/src/Interpreters/TextLog.h @@ -9,6 +9,7 @@ using Poco::Message; struct TextLogElement { time_t event_time{}; + UInt64 event_time_microseconds{}; UInt32 microseconds; String thread_name; From e51dfc1314a823c11cd48866caf19a70e3dbb03f Mon Sep 17 00:00:00 2001 From: bharatnc Date: Wed, 9 Sep 2020 23:09:48 -0700 Subject: [PATCH 171/321] Rename test_test_log_level to include related tests --- .../{test_text_log_level => test_text_log_table}/__init__.py | 0 .../configs/config.d/text_log.xml | 0 .../{test_text_log_level => test_text_log_table}/test.py | 0 3 files changed, 0 insertions(+), 0 deletions(-) rename tests/integration/{test_text_log_level => test_text_log_table}/__init__.py (100%) rename tests/integration/{test_text_log_level => test_text_log_table}/configs/config.d/text_log.xml (100%) rename tests/integration/{test_text_log_level => test_text_log_table}/test.py (100%) diff --git a/tests/integration/test_text_log_level/__init__.py b/tests/integration/test_text_log_table/__init__.py similarity index 100% rename from tests/integration/test_text_log_level/__init__.py rename to tests/integration/test_text_log_table/__init__.py diff --git a/tests/integration/test_text_log_level/configs/config.d/text_log.xml b/tests/integration/test_text_log_table/configs/config.d/text_log.xml similarity index 100% rename from tests/integration/test_text_log_level/configs/config.d/text_log.xml rename to tests/integration/test_text_log_table/configs/config.d/text_log.xml diff --git a/tests/integration/test_text_log_level/test.py b/tests/integration/test_text_log_table/test.py similarity index 100% rename from tests/integration/test_text_log_level/test.py rename to tests/integration/test_text_log_table/test.py From b4e1116b55ad283f11014b25b95a466d3ca9c67b Mon Sep 17 00:00:00 2001 From: bharatnc Date: Wed, 9 Sep 2020 23:11:58 -0700 Subject: [PATCH 172/321] TextLog - add tests for field event_time_microsec --- tests/integration/test_text_log_table/test.py | 26 ++++++++++++++++++- 1 file changed, 25 insertions(+), 1 deletion(-) diff --git a/tests/integration/test_text_log_table/test.py b/tests/integration/test_text_log_table/test.py index 44679481266..60c2e35893d 100644 --- a/tests/integration/test_text_log_table/test.py +++ b/tests/integration/test_text_log_table/test.py @@ -22,7 +22,7 @@ def start_cluster(): def test_basic(start_cluster): with pytest.raises(QueryRuntimeException): - # generates log with "Error" level + # generate log with "Error" level node.query('SELECT * FROM no_such_table') node.query('SYSTEM FLUSH LOGS') @@ -31,3 +31,27 @@ def test_basic(start_cluster): assert int(node.query("SELECT count() FROM system.text_log WHERE level = 'Debug'")) == 0 assert int(node.query("SELECT count() FROM system.text_log WHERE level = 'Information'")) >= 1 assert int(node.query("SELECT count() FROM system.text_log WHERE level = 'Error'")) >= 1 + +# compare the event_time and event_time_microseconds and test +# that they are exactly equal upto their seconds parts. +def test_field_event_time_microseconds(start_cluster): + with pytest.raises(QueryRuntimeException): + # generate log with "Error" level + node.query('SELECT * FROM no_such_table') + node.query('SYSTEM FLUSH LOGS') + equals_query = '''WITH ( + ( + SELECT event_time_microseconds + FROM system.text_log + ORDER BY event_time DESC + LIMIT 1 + ) AS time_with_microseconds, + ( + SELECT event_time + FROM system.text_log + ORDER BY event_time DESC + LIMIT 1 + ) AS time) + SELECT if(dateDiff('second', toDateTime(time_with_microseconds), toDateTime(time)) = 0, 'ok', 'fail') + ''' + assert 'ok\n' in node.query(equals_query) From b3c5aaff9e4c2efa5d015717fa3c39a5c00c46fa Mon Sep 17 00:00:00 2001 From: bharatnc Date: Wed, 9 Sep 2020 23:36:32 -0700 Subject: [PATCH 173/321] TraceLog - add field event_time_microseconds --- src/Common/Stopwatch.h | 6 ++++++ src/Common/TraceCollector.cpp | 3 ++- src/Interpreters/TraceLog.cpp | 3 +++ src/Interpreters/TraceLog.h | 1 + 4 files changed, 12 insertions(+), 1 deletion(-) diff --git a/src/Common/Stopwatch.h b/src/Common/Stopwatch.h index 772caa75373..6a0d1f37eb4 100644 --- a/src/Common/Stopwatch.h +++ b/src/Common/Stopwatch.h @@ -13,6 +13,12 @@ inline UInt64 clock_gettime_ns(clockid_t clock_type = CLOCK_MONOTONIC) return UInt64(ts.tv_sec * 1000000000LL + ts.tv_nsec); } +inline UInt64 clock_gettime_microseconds(clockid_t clock_type = CLOCK_MONOTONIC) +{ + struct timespec ts; + clock_gettime(clock_type, &ts); + return UInt64((ts.tv_sec * 1000000LL) + (ts.tv_nsec / 1000)); +} /** Differs from Poco::Stopwatch only by using 'clock_gettime' instead of 'gettimeofday', * returns nanoseconds instead of microseconds, and also by other minor differencies. diff --git a/src/Common/TraceCollector.cpp b/src/Common/TraceCollector.cpp index c69c9d1dc1a..35e90a6e583 100644 --- a/src/Common/TraceCollector.cpp +++ b/src/Common/TraceCollector.cpp @@ -142,7 +142,8 @@ void TraceCollector::run() if (trace_log) { UInt64 time = clock_gettime_ns(CLOCK_REALTIME); - TraceLogElement element{time_t(time / 1000000000), time, trace_type, thread_id, query_id, trace, size}; + UInt64 time_in_microseconds = clock_gettime_microseconds(CLOCK_REALTIME); + TraceLogElement element{time_t(time / 1000000000), time_in_microseconds, time, trace_type, thread_id, query_id, trace, size}; trace_log->add(element); } } diff --git a/src/Interpreters/TraceLog.cpp b/src/Interpreters/TraceLog.cpp index f7e82032f49..40bcc0db445 100644 --- a/src/Interpreters/TraceLog.cpp +++ b/src/Interpreters/TraceLog.cpp @@ -4,6 +4,7 @@ #include #include #include +#include #include @@ -26,6 +27,7 @@ Block TraceLogElement::createBlock() { {std::make_shared(), "event_date"}, {std::make_shared(), "event_time"}, + {std::make_shared(6), "event_time_microseconds"}, {std::make_shared(), "timestamp_ns"}, {std::make_shared(), "revision"}, {std::make_shared(trace_values), "trace_type"}, @@ -42,6 +44,7 @@ void TraceLogElement::appendToBlock(MutableColumns & columns) const columns[i++]->insert(DateLUT::instance().toDayNum(event_time)); columns[i++]->insert(event_time); + columns[i++]->insert(event_time_microseconds); columns[i++]->insert(timestamp_ns); columns[i++]->insert(ClickHouseRevision::getVersionRevision()); columns[i++]->insert(static_cast(trace_type)); diff --git a/src/Interpreters/TraceLog.h b/src/Interpreters/TraceLog.h index 06f7f27299d..d694a6201f7 100644 --- a/src/Interpreters/TraceLog.h +++ b/src/Interpreters/TraceLog.h @@ -18,6 +18,7 @@ struct TraceLogElement static const TraceDataType::Values trace_values; time_t event_time{}; + UInt64 event_time_microseconds{}; UInt64 timestamp_ns{}; TraceType trace_type{}; UInt64 thread_id{}; From c6b5ef11748f602cf32887bf997daadf3bb28b72 Mon Sep 17 00:00:00 2001 From: bharatnc Date: Wed, 9 Sep 2020 23:52:47 -0700 Subject: [PATCH 174/321] TraceLog - add tests for event_time_microseconds --- .../01473_event_time_microseconds.reference | 2 ++ .../01473_event_time_microseconds.sql | 17 +++++++++++++++++ 2 files changed, 19 insertions(+) diff --git a/tests/queries/0_stateless/01473_event_time_microseconds.reference b/tests/queries/0_stateless/01473_event_time_microseconds.reference index 8aa31f9ab6a..f6d90d4dd91 100644 --- a/tests/queries/0_stateless/01473_event_time_microseconds.reference +++ b/tests/queries/0_stateless/01473_event_time_microseconds.reference @@ -1,2 +1,4 @@ 01473_metric_log_table_event_start_time_microseconds_test ok +01473_trace_log_table_event_start_time_microseconds_test +ok diff --git a/tests/queries/0_stateless/01473_event_time_microseconds.sql b/tests/queries/0_stateless/01473_event_time_microseconds.sql index 138df77ffec..474250730d6 100644 --- a/tests/queries/0_stateless/01473_event_time_microseconds.sql +++ b/tests/queries/0_stateless/01473_event_time_microseconds.sql @@ -22,3 +22,20 @@ WITH ( LIMIT 1 ) AS time) SELECT if(dateDiff('second', toDateTime(time_with_microseconds), toDateTime(time)) = 0, 'ok', 'fail') + +select '01473_trace_log_table_event_start_time_microseconds_test'; +system flush logs; +WITH ( + ( + SELECT event_time_microseconds + FROM system.trace_log + ORDER BY event_time DESC + LIMIT 1 + ) AS time_with_microseconds, + ( + SELECT event_time + FROM system.trace_log + ORDER BY event_time DESC + LIMIT 1 + ) AS time) +SELECT if(dateDiff('second', toDateTime(time_with_microseconds), toDateTime(time)) = 0, 'ok', 'fail'); -- success From 1830df1d6b03848344161a689d56518839885710 Mon Sep 17 00:00:00 2001 From: bharatnc Date: Fri, 11 Sep 2020 09:51:24 -0700 Subject: [PATCH 175/321] Query,QueryThread Logs - add event_time_microseconds field --- src/Interpreters/QueryLog.cpp | 2 ++ src/Interpreters/QueryLog.h | 1 + src/Interpreters/QueryThreadLog.cpp | 2 ++ src/Interpreters/QueryThreadLog.h | 1 + src/Interpreters/ThreadStatusExt.cpp | 1 + src/Interpreters/executeQuery.cpp | 4 ++++ 6 files changed, 11 insertions(+) diff --git a/src/Interpreters/QueryLog.cpp b/src/Interpreters/QueryLog.cpp index 75e0fae615a..c2273a1db2c 100644 --- a/src/Interpreters/QueryLog.cpp +++ b/src/Interpreters/QueryLog.cpp @@ -39,6 +39,7 @@ Block QueryLogElement::createBlock() {std::move(query_status_datatype), "type"}, {std::make_shared(), "event_date"}, {std::make_shared(), "event_time"}, + {std::make_shared(6), "event_time_microseconds"}, {std::make_shared(), "query_start_time"}, {std::make_shared(6), "query_start_time_microseconds"}, {std::make_shared(), "query_duration_ms"}, @@ -97,6 +98,7 @@ void QueryLogElement::appendToBlock(MutableColumns & columns) const columns[i++]->insert(type); columns[i++]->insert(DateLUT::instance().toDayNum(event_time)); columns[i++]->insert(event_time); + columns[i++]->insert(event_time_microseconds); columns[i++]->insert(query_start_time); columns[i++]->insert(query_start_time_microseconds); columns[i++]->insert(query_duration_ms); diff --git a/src/Interpreters/QueryLog.h b/src/Interpreters/QueryLog.h index d1297feb3fb..9d42b787160 100644 --- a/src/Interpreters/QueryLog.h +++ b/src/Interpreters/QueryLog.h @@ -30,6 +30,7 @@ struct QueryLogElement /// Depending on the type of query and type of stage, not all the fields may be filled. time_t event_time{}; + UInt64 event_time_microseconds{}; time_t query_start_time{}; UInt64 query_start_time_microseconds{}; UInt64 query_duration_ms{}; diff --git a/src/Interpreters/QueryThreadLog.cpp b/src/Interpreters/QueryThreadLog.cpp index e5a8cf7c5cf..2ecb03d622a 100644 --- a/src/Interpreters/QueryThreadLog.cpp +++ b/src/Interpreters/QueryThreadLog.cpp @@ -23,6 +23,7 @@ Block QueryThreadLogElement::createBlock() return { {std::make_shared(), "event_date"}, {std::make_shared(), "event_time"}, + {std::make_shared(6), "event_time_microseconds"}, {std::make_shared(), "query_start_time"}, {std::make_shared(6), "query_start_time_microseconds"}, {std::make_shared(), "query_duration_ms"}, @@ -73,6 +74,7 @@ void QueryThreadLogElement::appendToBlock(MutableColumns & columns) const columns[i++]->insert(DateLUT::instance().toDayNum(event_time)); columns[i++]->insert(event_time); + columns[i++]->insert(event_time_microseconds); columns[i++]->insert(query_start_time); columns[i++]->insert(query_start_time_microseconds); columns[i++]->insert(query_duration_ms); diff --git a/src/Interpreters/QueryThreadLog.h b/src/Interpreters/QueryThreadLog.h index 66a480bfa0d..715902b29ad 100644 --- a/src/Interpreters/QueryThreadLog.h +++ b/src/Interpreters/QueryThreadLog.h @@ -16,6 +16,7 @@ namespace DB struct QueryThreadLogElement { time_t event_time{}; + UInt64 event_time_microseconds{}; /// When query was attached to current thread time_t query_start_time{}; /// same as above but adds microsecond precision diff --git a/src/Interpreters/ThreadStatusExt.cpp b/src/Interpreters/ThreadStatusExt.cpp index 0f610d9f6d2..ea33db3809b 100644 --- a/src/Interpreters/ThreadStatusExt.cpp +++ b/src/Interpreters/ThreadStatusExt.cpp @@ -323,6 +323,7 @@ void ThreadStatus::logToQueryThreadLog(QueryThreadLog & thread_log) QueryThreadLogElement elem; elem.event_time = time(nullptr); + elem.event_time_microseconds = getCurrentTimeMicroseconds(); elem.query_start_time = query_start_time; elem.query_start_time_microseconds = query_start_time_microseconds; elem.query_duration_ms = (getCurrentTimeNanoseconds() - query_start_time_nanoseconds) / 1000000U; diff --git a/src/Interpreters/executeQuery.cpp b/src/Interpreters/executeQuery.cpp index 59896065a87..68c0c2698c0 100644 --- a/src/Interpreters/executeQuery.cpp +++ b/src/Interpreters/executeQuery.cpp @@ -210,6 +210,7 @@ static void onExceptionBeforeStart(const String & query_for_logging, Context & c // event_time_microseconds from the same timespec. So it can be assumed that both of these // times are equal upto the precision of a second. elem.event_time = current_time; + elem.event_time_microseconds = current_time_microseconds; elem.query_start_time = current_time; elem.query_start_time_microseconds = current_time_microseconds; @@ -484,6 +485,7 @@ static std::tuple executeQueryImpl( elem.type = QueryLogElementType::QUERY_START; elem.event_time = current_time; + elem.event_time_microseconds = current_time_microseconds; elem.query_start_time = current_time; elem.query_start_time_microseconds = current_time_microseconds; @@ -555,6 +557,7 @@ static std::tuple executeQueryImpl( elem.type = QueryLogElementType::QUERY_FINISH; elem.event_time = time(nullptr); + elem.event_time_microseconds = getCurrentTimeMicroseconds(); status_info_to_query_log(elem, info, ast); @@ -616,6 +619,7 @@ static std::tuple executeQueryImpl( elem.type = QueryLogElementType::EXCEPTION_WHILE_PROCESSING; elem.event_time = time(nullptr); + elem.event_time_microseconds = getCurrentTimeMicroseconds(); elem.query_duration_ms = 1000 * (elem.event_time - elem.query_start_time); elem.exception_code = getCurrentExceptionCode(); elem.exception = getCurrentExceptionMessage(false); From e8ff5315d1ffa9987b497c5f567c07a75268278d Mon Sep 17 00:00:00 2001 From: bharatnc Date: Fri, 11 Sep 2020 10:10:44 -0700 Subject: [PATCH 176/321] Query,QueryThread Logs - add tests --- .../01473_event_time_microseconds.reference | 4 ++ .../01473_event_time_microseconds.sql | 46 ++++++++++++++++--- 2 files changed, 44 insertions(+), 6 deletions(-) diff --git a/tests/queries/0_stateless/01473_event_time_microseconds.reference b/tests/queries/0_stateless/01473_event_time_microseconds.reference index f6d90d4dd91..6d95c8c1b81 100644 --- a/tests/queries/0_stateless/01473_event_time_microseconds.reference +++ b/tests/queries/0_stateless/01473_event_time_microseconds.reference @@ -2,3 +2,7 @@ ok 01473_trace_log_table_event_start_time_microseconds_test ok +01473_query_log_table_event_start_time_microseconds_test +ok +01473_query_thread_log_table_event_start_time_microseconds_test +ok diff --git a/tests/queries/0_stateless/01473_event_time_microseconds.sql b/tests/queries/0_stateless/01473_event_time_microseconds.sql index 474250730d6..537664b1ef0 100644 --- a/tests/queries/0_stateless/01473_event_time_microseconds.sql +++ b/tests/queries/0_stateless/01473_event_time_microseconds.sql @@ -3,10 +3,10 @@ -- an integration test as those metrics take 60s by default to be updated. -- Refer: tests/integration/test_asynchronous_metric_log_table. -set log_queries = 1; +SET log_queries = 1; -select '01473_metric_log_table_event_start_time_microseconds_test'; -system flush logs; +SELECT '01473_metric_log_table_event_start_time_microseconds_test'; +SYSTEM FLUSH LOGS; -- query assumes that the event_time field is accurate. WITH ( ( @@ -21,10 +21,10 @@ WITH ( ORDER BY event_time DESC LIMIT 1 ) AS time) -SELECT if(dateDiff('second', toDateTime(time_with_microseconds), toDateTime(time)) = 0, 'ok', 'fail') +SELECT if(dateDiff('second', toDateTime(time_with_microseconds), toDateTime(time)) = 0, 'ok', 'fail'); -select '01473_trace_log_table_event_start_time_microseconds_test'; -system flush logs; +SELECT '01473_trace_log_table_event_start_time_microseconds_test'; +SYSTEM FLUSH LOGS; WITH ( ( SELECT event_time_microseconds @@ -39,3 +39,37 @@ WITH ( LIMIT 1 ) AS time) SELECT if(dateDiff('second', toDateTime(time_with_microseconds), toDateTime(time)) = 0, 'ok', 'fail'); -- success + +SELECT '01473_query_log_table_event_start_time_microseconds_test'; +SYSTEM FLUSH LOGS; +WITH ( + ( + SELECT event_time_microseconds + FROM system.query_log + ORDER BY event_time DESC + LIMIT 1 + ) AS time_with_microseconds, + ( + SELECT event_time + FROM system.query_log + ORDER BY event_time DESC + LIMIT 1 + ) AS time) +SELECT if(dateDiff('second', toDateTime(time_with_microseconds), toDateTime(time)) = 0, 'ok', 'fail'); -- success + +SELECT '01473_query_thread_log_table_event_start_time_microseconds_test'; +SYSTEM FLUSH LOGS; +WITH ( + ( + SELECT event_time_microseconds + FROM system.query_thread_log + ORDER BY event_time DESC + LIMIT 1 + ) AS time_with_microseconds, + ( + SELECT event_time + FROM system.query_thread_log + ORDER BY event_time DESC + LIMIT 1 + ) AS time) +SELECT if(dateDiff('second', toDateTime(time_with_microseconds), toDateTime(time)) = 0, 'ok', 'fail'); -- success From ac493605259193164a80e3555851ce5e93727085 Mon Sep 17 00:00:00 2001 From: bharatnc Date: Fri, 11 Sep 2020 10:34:27 -0700 Subject: [PATCH 177/321] Field event_time_microseconds - add documentation --- docs/en/operations/system-tables/query_log.md | 90 ++++++++++--------- .../system-tables/query_thread_log.md | 80 +++++++++-------- docs/en/operations/system-tables/text_log.md | 25 ++++++ docs/en/operations/system-tables/trace_log.md | 19 ++-- 4 files changed, 125 insertions(+), 89 deletions(-) diff --git a/docs/en/operations/system-tables/query_log.md b/docs/en/operations/system-tables/query_log.md index 72927b5a7e9..8b663475fa8 100644 --- a/docs/en/operations/system-tables/query_log.md +++ b/docs/en/operations/system-tables/query_log.md @@ -33,6 +33,7 @@ Columns: - `'ExceptionWhileProcessing' = 4` — Exception during the query execution. - `event_date` ([Date](../../sql-reference/data-types/date.md)) — Query starting date. - `event_time` ([DateTime](../../sql-reference/data-types/datetime.md)) — Query starting time. +- `event_time_microseconds` ([DateTime](../../sql-reference/data-types/datetime.md)) — Query starting time with microseconds precision. - `query_start_time` ([DateTime](../../sql-reference/data-types/datetime.md)) — Start time of query execution. - `query_start_time_microseconds` ([DateTime64](../../sql-reference/data-types/datetime64.md)) — Start time of query execution with microsecond precision. - `query_duration_ms` ([UInt64](../../sql-reference/data-types/int-uint.md#uint-ranges)) — Duration of query execution in milliseconds. @@ -84,54 +85,57 @@ Columns: **Example** ``` sql -SELECT * FROM system.query_log LIMIT 1 FORMAT Vertical; +SELECT * FROM system.query_log LIMIT 1 \G ``` ``` text Row 1: ────── -type: QueryStart -event_date: 2020-05-13 -event_time: 2020-05-13 14:02:28 -query_start_time: 2020-05-13 14:02:28 -query_duration_ms: 0 -read_rows: 0 -read_bytes: 0 -written_rows: 0 -written_bytes: 0 -result_rows: 0 -result_bytes: 0 -memory_usage: 0 -query: SELECT 1 -exception_code: 0 -exception: -stack_trace: -is_initial_query: 1 -user: default -query_id: 5e834082-6f6d-4e34-b47b-cd1934f4002a -address: ::ffff:127.0.0.1 -port: 57720 -initial_user: default -initial_query_id: 5e834082-6f6d-4e34-b47b-cd1934f4002a -initial_address: ::ffff:127.0.0.1 -initial_port: 57720 -interface: 1 -os_user: bayonet -client_hostname: clickhouse.ru-central1.internal -client_name: ClickHouse client -client_revision: 54434 -client_version_major: 20 -client_version_minor: 4 -client_version_patch: 1 -http_method: 0 -http_user_agent: -quota_key: -revision: 54434 -thread_ids: [] -ProfileEvents.Names: [] -ProfileEvents.Values: [] -Settings.Names: ['use_uncompressed_cache','load_balancing','log_queries','max_memory_usage'] -Settings.Values: ['0','random','1','10000000000'] +type: QueryStart +event_date: 2020-09-11 +event_time: 2020-09-11 10:08:17 +event_time_microseconds: 2020-09-11 10:08:17.063321 +query_start_time: 2020-09-11 10:08:17 +query_start_time_microseconds: 2020-09-11 10:08:17.063321 +query_duration_ms: 0 +read_rows: 0 +read_bytes: 0 +written_rows: 0 +written_bytes: 0 +result_rows: 0 +result_bytes: 0 +memory_usage: 0 +current_database: default +query: INSERT INTO test1 VALUES +exception_code: 0 +exception: +stack_trace: +is_initial_query: 1 +user: default +query_id: 50a320fd-85a8-49b8-8761-98a86bcbacef +address: ::ffff:127.0.0.1 +port: 33452 +initial_user: default +initial_query_id: 50a320fd-85a8-49b8-8761-98a86bcbacef +initial_address: ::ffff:127.0.0.1 +initial_port: 33452 +interface: 1 +os_user: bharatnc +client_hostname: tower +client_name: ClickHouse +client_revision: 54437 +client_version_major: 20 +client_version_minor: 7 +client_version_patch: 2 +http_method: 0 +http_user_agent: +quota_key: +revision: 54440 +thread_ids: [] +ProfileEvents.Names: [] +ProfileEvents.Values: [] +Settings.Names: ['use_uncompressed_cache','load_balancing','log_queries','max_memory_usage','allow_introspection_functions'] +Settings.Values: ['0','random','1','10000000000','1'] ``` **See Also** diff --git a/docs/en/operations/system-tables/query_thread_log.md b/docs/en/operations/system-tables/query_thread_log.md index 3dcd05c4cc3..84408edd117 100644 --- a/docs/en/operations/system-tables/query_thread_log.md +++ b/docs/en/operations/system-tables/query_thread_log.md @@ -15,6 +15,7 @@ Columns: - `event_date` ([Date](../../sql-reference/data-types/date.md)) — The date when the thread has finished execution of the query. - `event_time` ([DateTime](../../sql-reference/data-types/datetime.md)) — The date and time when the thread has finished execution of the query. +- `event_time_microsecinds` ([DateTime](../../sql-reference/data-types/datetime.md)) — The date and time when the thread has finished execution of the query with microseconds precision. - `query_start_time` ([DateTime](../../sql-reference/data-types/datetime.md)) — Start time of query execution. - `query_start_time_microseconds` ([DateTime64](../../sql-reference/data-types/datetime64.md)) — Start time of query execution with microsecond precision. - `query_duration_ms` ([UInt64](../../sql-reference/data-types/int-uint.md#uint-ranges)) — Duration of query execution. @@ -63,50 +64,51 @@ Columns: **Example** ``` sql - SELECT * FROM system.query_thread_log LIMIT 1 FORMAT Vertical + SELECT * FROM system.query_thread_log LIMIT 1 \G ``` ``` text Row 1: ────── -event_date: 2020-05-13 -event_time: 2020-05-13 14:02:28 -query_start_time: 2020-05-13 14:02:28 -query_duration_ms: 0 -read_rows: 1 -read_bytes: 1 -written_rows: 0 -written_bytes: 0 -memory_usage: 0 -peak_memory_usage: 0 -thread_name: QueryPipelineEx -thread_id: 28952 -master_thread_id: 28924 -query: SELECT 1 -is_initial_query: 1 -user: default -query_id: 5e834082-6f6d-4e34-b47b-cd1934f4002a -address: ::ffff:127.0.0.1 -port: 57720 -initial_user: default -initial_query_id: 5e834082-6f6d-4e34-b47b-cd1934f4002a -initial_address: ::ffff:127.0.0.1 -initial_port: 57720 -interface: 1 -os_user: bayonet -client_hostname: clickhouse.ru-central1.internal -client_name: ClickHouse client -client_revision: 54434 -client_version_major: 20 -client_version_minor: 4 -client_version_patch: 1 -http_method: 0 -http_user_agent: -quota_key: -revision: 54434 -ProfileEvents.Names: ['ContextLock','RealTimeMicroseconds','UserTimeMicroseconds','OSCPUWaitMicroseconds','OSCPUVirtualTimeMicroseconds'] -ProfileEvents.Values: [1,97,81,5,81] -... +event_date: 2020-09-11 +event_time: 2020-09-11 10:08:17 +event_time_microseconds: 2020-09-11 10:08:17.134042 +query_start_time: 2020-09-11 10:08:17 +query_start_time_microseconds: 2020-09-11 10:08:17.063150 +query_duration_ms: 70 +read_rows: 0 +read_bytes: 0 +written_rows: 1 +written_bytes: 12 +memory_usage: 4300844 +peak_memory_usage: 4300844 +thread_name: TCPHandler +thread_id: 638133 +master_thread_id: 638133 +query: INSERT INTO test1 VALUES +is_initial_query: 1 +user: default +query_id: 50a320fd-85a8-49b8-8761-98a86bcbacef +address: ::ffff:127.0.0.1 +port: 33452 +initial_user: default +initial_query_id: 50a320fd-85a8-49b8-8761-98a86bcbacef +initial_address: ::ffff:127.0.0.1 +initial_port: 33452 +interface: 1 +os_user: bharatnc +client_hostname: tower +client_name: ClickHouse +client_revision: 54437 +client_version_major: 20 +client_version_minor: 7 +client_version_patch: 2 +http_method: 0 +http_user_agent: +quota_key: +revision: 54440 +ProfileEvents.Names: ['Query','InsertQuery','FileOpen','WriteBufferFromFileDescriptorWrite','WriteBufferFromFileDescriptorWriteBytes','ReadCompressedBytes','CompressedReadBufferBlocks','CompressedReadBufferBytes','IOBufferAllocs','IOBufferAllocBytes','FunctionExecute','CreatedWriteBufferOrdinary','DiskWriteElapsedMicroseconds','NetworkReceiveElapsedMicroseconds','NetworkSendElapsedMicroseconds','InsertedRows','InsertedBytes','SelectedRows','SelectedBytes','MergeTreeDataWriterRows','MergeTreeDataWriterUncompressedBytes','MergeTreeDataWriterCompressedBytes','MergeTreeDataWriterBlocks','MergeTreeDataWriterBlocksAlreadySorted','ContextLock','RWLockAcquiredReadLocks','RealTimeMicroseconds','UserTimeMicroseconds','SoftPageFaults','OSCPUVirtualTimeMicroseconds','OSWriteBytes','OSReadChars','OSWriteChars'] +ProfileEvents.Values: [1,1,11,11,591,148,3,71,29,6533808,1,11,72,18,47,1,12,1,12,1,12,189,1,1,10,2,70853,2748,49,2747,45056,422,1520] ``` **See Also** diff --git a/docs/en/operations/system-tables/text_log.md b/docs/en/operations/system-tables/text_log.md index bd92519b96b..3c3281ff8c6 100644 --- a/docs/en/operations/system-tables/text_log.md +++ b/docs/en/operations/system-tables/text_log.md @@ -6,6 +6,7 @@ Columns: - `event_date` (Date) — Date of the entry. - `event_time` (DateTime) — Time of the entry. +- `event_time_microseconds` (DateTime) — Time of the entry with microseconds precision. - `microseconds` (UInt32) — Microseconds of the entry. - `thread_name` (String) — Name of the thread from which the logging was done. - `thread_id` (UInt64) — OS thread ID. @@ -25,4 +26,28 @@ Columns: - `source_file` (LowCardinality(String)) — Source file from which the logging was done. - `source_line` (UInt64) — Source line from which the logging was done. +**Example** + +``` sql +SELECT * FROM system.text_log LIMIT 1 \G +``` + +``` text +Row 1: +────── +event_date: 2020-09-10 +event_time: 2020-09-10 11:23:07 +event_time_microseconds: 2020-09-10 11:23:07.871397 +microseconds: 871397 +thread_name: clickhouse-serv +thread_id: 564917 +level: Information +query_id: +logger_name: DNSCacheUpdater +message: Update period 15 seconds +revision: 54440 +source_file: /ClickHouse/src/Interpreters/DNSCacheUpdater.cpp; void DB::DNSCacheUpdater::start() +source_line: 45 +``` + [Original article](https://clickhouse.tech/docs/en/operations/system_tables/text_log) \ No newline at end of file diff --git a/docs/en/operations/system-tables/trace_log.md b/docs/en/operations/system-tables/trace_log.md index b911fdd2263..0fb967822d2 100644 --- a/docs/en/operations/system-tables/trace_log.md +++ b/docs/en/operations/system-tables/trace_log.md @@ -12,6 +12,8 @@ Columns: - `event_time` ([DateTime](../../sql-reference/data-types/datetime.md)) — Timestamp of the sampling moment. +- `event_time_microseconds` ([DateTime](../../sql-reference/data-types/datetime.md)) — Timestamp of the sampling moment with microseconds precision. + - `timestamp_ns` ([UInt64](../../sql-reference/data-types/int-uint.md)) — Timestamp of the sampling moment in nanoseconds. - `revision` ([UInt32](../../sql-reference/data-types/int-uint.md)) — ClickHouse server build revision. @@ -38,13 +40,16 @@ SELECT * FROM system.trace_log LIMIT 1 \G ``` text Row 1: ────── -event_date: 2019-11-15 -event_time: 2019-11-15 15:09:38 -revision: 54428 -timer_type: Real -thread_number: 48 -query_id: acc4d61f-5bd1-4a3e-bc91-2180be37c915 -trace: [94222141367858,94222152240175,94222152325351,94222152329944,94222152330796,94222151449980,94222144088167,94222151682763,94222144088167,94222151682763,94222144088167,94222144058283,94222144059248,94222091840750,94222091842302,94222091831228,94222189631488,140509950166747,140509942945935] +event_date: 2020-09-10 +event_time: 2020-09-10 11:23:09 +event_time_microseconds: 2020-09-10 11:23:09.872924 +timestamp_ns: 1599762189872924510 +revision: 54440 +trace_type: Memory +thread_id: 564963 +query_id: +trace: [371912858,371912789,371798468,371799717,371801313,371790250,624462773,566365041,566440261,566445834,566460071,566459914,566459842,566459580,566459469,566459389,566459341,566455774,371993941,371988245,372158848,372187428,372187309,372187093,372185478,140222123165193,140222122205443] +size: 5244400 ``` [Original article](https://clickhouse.tech/docs/en/operations/system_tables/trace_log) \ No newline at end of file From 051d1a5f0784117cc1967e2a9c8cdb279b286e8b Mon Sep 17 00:00:00 2001 From: bharatnc Date: Fri, 11 Sep 2020 20:43:47 -0700 Subject: [PATCH 178/321] TraceLog - replace unit test with integration test --- .../test_trace_log_table/__init__.py | 0 .../configs/trace_log.xml | 8 +++ .../integration/test_trace_log_table/test.py | 65 +++++++++++++++++++ .../01473_event_time_microseconds.reference | 2 - .../01473_event_time_microseconds.sql | 17 ----- 5 files changed, 73 insertions(+), 19 deletions(-) create mode 100644 tests/integration/test_trace_log_table/__init__.py create mode 100644 tests/integration/test_trace_log_table/configs/trace_log.xml create mode 100644 tests/integration/test_trace_log_table/test.py diff --git a/tests/integration/test_trace_log_table/__init__.py b/tests/integration/test_trace_log_table/__init__.py new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/integration/test_trace_log_table/configs/trace_log.xml b/tests/integration/test_trace_log_table/configs/trace_log.xml new file mode 100644 index 00000000000..48a93dc0cb9 --- /dev/null +++ b/tests/integration/test_trace_log_table/configs/trace_log.xml @@ -0,0 +1,8 @@ + + + system + trace_log
+ toYYYYMM(event_date) + 1000 +
+
diff --git a/tests/integration/test_trace_log_table/test.py b/tests/integration/test_trace_log_table/test.py new file mode 100644 index 00000000000..9f68e4b4443 --- /dev/null +++ b/tests/integration/test_trace_log_table/test.py @@ -0,0 +1,65 @@ +# pylint: disable=unused-argument +# pylint: disable=redefined-outer-name +import time + +import pytest +from helpers.cluster import ClickHouseCluster + +cluster = ClickHouseCluster(__file__) + +node = cluster.add_instance('node', with_zookeeper=True, main_configs=["configs/trace_log.xml"]) + + +@pytest.fixture(scope='module') +def start_cluster(): + try: + cluster.start() + + yield cluster + finally: + cluster.shutdown() + + +# Tests that the event_time_microseconds field in the system.trace_log table gets populated. +# To make the tests work better, the default flush_interval_milliseconds is being overridden +# to 1000 ms. Also the query_profiler_real_time_period_ns and the query_profiler_cpu_time_period_ns +# are set to suitable values so that traces are properly populated. Test compares the event_time and +# event_time_microseconds fields and asserts that they are exactly equal upto their seconds parts. Also +# one additional test to ensure that the count(event_time_microseconds) is > 0; +def test_field_event_time_microseconds(start_cluster): + node.query('SET query_profiler_real_time_period_ns = 0;') + node.query('SET query_profiler_cpu_time_period_ns = 1000000;') + node.query('SET log_queries = 1;') + node.query("CREATE DATABASE replica;") + query_create = '''CREATE TABLE replica.test + ( + id Int64, + event_time DateTime + ) + Engine=MergeTree() + PARTITION BY toYYYYMMDD(event_time) + ORDER BY id;''' + node.query(query_create) + node.query('''INSERT INTO replica.test VALUES (1, now())''') + node.query("SYSTEM FLUSH LOGS;") + # TODO: is sleep necessary ? + time.sleep(1) + # query assumes that the event_time field is already accurate + equals_query = '''WITH ( + ( + SELECT event_time_microseconds + FROM system.trace_log + ORDER BY event_time DESC + LIMIT 1 + ) AS time_with_microseconds, + ( + SELECT event_time + FROM system.trace_log + ORDER BY event_time DESC + LIMIT 1 + ) AS t) + SELECT if(dateDiff('second', toDateTime(time_with_microseconds), toDateTime(t)) = 0, 'ok', 'fail') + ''' + assert 'ok\n' in node.query(equals_query) + assert 'ok\n' in node.query( + '''SELECT if((SELECT COUNT(event_time_microseconds) FROM system.trace_log) > 0, 'ok', 'fail')''') diff --git a/tests/queries/0_stateless/01473_event_time_microseconds.reference b/tests/queries/0_stateless/01473_event_time_microseconds.reference index 6d95c8c1b81..7da9f21de2f 100644 --- a/tests/queries/0_stateless/01473_event_time_microseconds.reference +++ b/tests/queries/0_stateless/01473_event_time_microseconds.reference @@ -1,7 +1,5 @@ 01473_metric_log_table_event_start_time_microseconds_test ok -01473_trace_log_table_event_start_time_microseconds_test -ok 01473_query_log_table_event_start_time_microseconds_test ok 01473_query_thread_log_table_event_start_time_microseconds_test diff --git a/tests/queries/0_stateless/01473_event_time_microseconds.sql b/tests/queries/0_stateless/01473_event_time_microseconds.sql index 537664b1ef0..48d620be197 100644 --- a/tests/queries/0_stateless/01473_event_time_microseconds.sql +++ b/tests/queries/0_stateless/01473_event_time_microseconds.sql @@ -23,23 +23,6 @@ WITH ( ) AS time) SELECT if(dateDiff('second', toDateTime(time_with_microseconds), toDateTime(time)) = 0, 'ok', 'fail'); -SELECT '01473_trace_log_table_event_start_time_microseconds_test'; -SYSTEM FLUSH LOGS; -WITH ( - ( - SELECT event_time_microseconds - FROM system.trace_log - ORDER BY event_time DESC - LIMIT 1 - ) AS time_with_microseconds, - ( - SELECT event_time - FROM system.trace_log - ORDER BY event_time DESC - LIMIT 1 - ) AS time) -SELECT if(dateDiff('second', toDateTime(time_with_microseconds), toDateTime(time)) = 0, 'ok', 'fail'); -- success - SELECT '01473_query_log_table_event_start_time_microseconds_test'; SYSTEM FLUSH LOGS; WITH ( From 726b6ff6b6c6f25735692c2eb4362bbe91b1895d Mon Sep 17 00:00:00 2001 From: bharatnc Date: Sun, 13 Sep 2020 01:04:56 -0700 Subject: [PATCH 179/321] TraceLog - mark to skip integration test --- tests/integration/test_trace_log_table/test.py | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/integration/test_trace_log_table/test.py b/tests/integration/test_trace_log_table/test.py index 9f68e4b4443..c1c8934b9f2 100644 --- a/tests/integration/test_trace_log_table/test.py +++ b/tests/integration/test_trace_log_table/test.py @@ -26,6 +26,7 @@ def start_cluster(): # are set to suitable values so that traces are properly populated. Test compares the event_time and # event_time_microseconds fields and asserts that they are exactly equal upto their seconds parts. Also # one additional test to ensure that the count(event_time_microseconds) is > 0; +@pytest.mark.skip(reason="TODO: system.trace_log not populated in time on CI but works fine on dev") def test_field_event_time_microseconds(start_cluster): node.query('SET query_profiler_real_time_period_ns = 0;') node.query('SET query_profiler_cpu_time_period_ns = 1000000;') From 2972f6ae397de6ede1c3830abecf51bbe7ddbc1a Mon Sep 17 00:00:00 2001 From: bharatnc Date: Mon, 14 Sep 2020 18:40:06 -0700 Subject: [PATCH 180/321] EventTime - construct time in s and us from same timespec --- src/Interpreters/ThreadStatusExt.cpp | 9 +++++++-- src/Interpreters/executeQuery.cpp | 20 +++++++++++++++----- 2 files changed, 22 insertions(+), 7 deletions(-) diff --git a/src/Interpreters/ThreadStatusExt.cpp b/src/Interpreters/ThreadStatusExt.cpp index ea33db3809b..d820094db1d 100644 --- a/src/Interpreters/ThreadStatusExt.cpp +++ b/src/Interpreters/ThreadStatusExt.cpp @@ -322,8 +322,13 @@ void ThreadStatus::logToQueryThreadLog(QueryThreadLog & thread_log) { QueryThreadLogElement elem; - elem.event_time = time(nullptr); - elem.event_time_microseconds = getCurrentTimeMicroseconds(); + // event_time and event_time_microseconds are being constructed from the same timespec + // to ensure that both the times are equal upto the precision of a second. + struct timespec ts; + clock_gettime(CLOCK_MONOTONIC, &ts); + + elem.event_time = ts.tv_sec; + elem.event_time_microseconds = UInt64((ts.tv_sec * 1000000LL) + (ts.tv_nsec / 1000)); elem.query_start_time = query_start_time; elem.query_start_time_microseconds = query_start_time_microseconds; elem.query_duration_ms = (getCurrentTimeNanoseconds() - query_start_time_nanoseconds) / 1000000U; diff --git a/src/Interpreters/executeQuery.cpp b/src/Interpreters/executeQuery.cpp index 68c0c2698c0..3f2c88b3678 100644 --- a/src/Interpreters/executeQuery.cpp +++ b/src/Interpreters/executeQuery.cpp @@ -319,8 +319,9 @@ static std::tuple executeQueryImpl( auto query_for_logging = prepareQueryForLogging(query, context); logQuery(query_for_logging, context, internal); - if (!internal) + if (!internal) { onExceptionBeforeStart(query_for_logging, context, current_time, current_time_microseconds, ast); + } throw; } @@ -556,9 +557,13 @@ static std::tuple executeQueryImpl( elem.type = QueryLogElementType::QUERY_FINISH; - elem.event_time = time(nullptr); - elem.event_time_microseconds = getCurrentTimeMicroseconds(); + // event_time and event-time_microseconds are being constructed from the same timespec + // to ensure that both the times are equal upto the precision of a second. + struct timespec tspec; + clock_gettime(CLOCK_MONOTONIC, &tspec); + elem.event_time = tspec.tv_sec; + elem.event_time_microseconds = UInt64((tspec.tv_sec * 1000000LL) + (tspec.tv_nsec / 1000)); status_info_to_query_log(elem, info, ast); auto progress_callback = context.getProgressCallback(); @@ -618,8 +623,13 @@ static std::tuple executeQueryImpl( elem.type = QueryLogElementType::EXCEPTION_WHILE_PROCESSING; - elem.event_time = time(nullptr); - elem.event_time_microseconds = getCurrentTimeMicroseconds(); + // event_time and event_time_microseconds are being constructed from the timespec + // to ensure that both the times will be equal upto the precision of a second. + struct timespec tspec; + clock_gettime(CLOCK_MONOTONIC, &tspec); + + elem.event_time = tspec.tv_sec; + elem.event_time_microseconds = UInt64((tspec.tv_sec * 1000000LL) + (tspec.tv_nsec / 1000)); elem.query_duration_ms = 1000 * (elem.event_time - elem.query_start_time); elem.exception_code = getCurrentExceptionCode(); elem.exception = getCurrentExceptionMessage(false); From ac59add38cd52b41a707ccb37f919c20dfdf47b1 Mon Sep 17 00:00:00 2001 From: bharatnc Date: Mon, 14 Sep 2020 18:57:31 -0700 Subject: [PATCH 181/321] TraceLog - revert back to functional test --- .../test_trace_log_table/__init__.py | 0 .../configs/trace_log.xml | 8 --- .../integration/test_trace_log_table/test.py | 66 ------------------- .../01473_event_time_microseconds.reference | 2 + .../01473_event_time_microseconds.sql | 22 +++++++ 5 files changed, 24 insertions(+), 74 deletions(-) delete mode 100644 tests/integration/test_trace_log_table/__init__.py delete mode 100644 tests/integration/test_trace_log_table/configs/trace_log.xml delete mode 100644 tests/integration/test_trace_log_table/test.py diff --git a/tests/integration/test_trace_log_table/__init__.py b/tests/integration/test_trace_log_table/__init__.py deleted file mode 100644 index e69de29bb2d..00000000000 diff --git a/tests/integration/test_trace_log_table/configs/trace_log.xml b/tests/integration/test_trace_log_table/configs/trace_log.xml deleted file mode 100644 index 48a93dc0cb9..00000000000 --- a/tests/integration/test_trace_log_table/configs/trace_log.xml +++ /dev/null @@ -1,8 +0,0 @@ - - - system - trace_log
- toYYYYMM(event_date) - 1000 -
-
diff --git a/tests/integration/test_trace_log_table/test.py b/tests/integration/test_trace_log_table/test.py deleted file mode 100644 index c1c8934b9f2..00000000000 --- a/tests/integration/test_trace_log_table/test.py +++ /dev/null @@ -1,66 +0,0 @@ -# pylint: disable=unused-argument -# pylint: disable=redefined-outer-name -import time - -import pytest -from helpers.cluster import ClickHouseCluster - -cluster = ClickHouseCluster(__file__) - -node = cluster.add_instance('node', with_zookeeper=True, main_configs=["configs/trace_log.xml"]) - - -@pytest.fixture(scope='module') -def start_cluster(): - try: - cluster.start() - - yield cluster - finally: - cluster.shutdown() - - -# Tests that the event_time_microseconds field in the system.trace_log table gets populated. -# To make the tests work better, the default flush_interval_milliseconds is being overridden -# to 1000 ms. Also the query_profiler_real_time_period_ns and the query_profiler_cpu_time_period_ns -# are set to suitable values so that traces are properly populated. Test compares the event_time and -# event_time_microseconds fields and asserts that they are exactly equal upto their seconds parts. Also -# one additional test to ensure that the count(event_time_microseconds) is > 0; -@pytest.mark.skip(reason="TODO: system.trace_log not populated in time on CI but works fine on dev") -def test_field_event_time_microseconds(start_cluster): - node.query('SET query_profiler_real_time_period_ns = 0;') - node.query('SET query_profiler_cpu_time_period_ns = 1000000;') - node.query('SET log_queries = 1;') - node.query("CREATE DATABASE replica;") - query_create = '''CREATE TABLE replica.test - ( - id Int64, - event_time DateTime - ) - Engine=MergeTree() - PARTITION BY toYYYYMMDD(event_time) - ORDER BY id;''' - node.query(query_create) - node.query('''INSERT INTO replica.test VALUES (1, now())''') - node.query("SYSTEM FLUSH LOGS;") - # TODO: is sleep necessary ? - time.sleep(1) - # query assumes that the event_time field is already accurate - equals_query = '''WITH ( - ( - SELECT event_time_microseconds - FROM system.trace_log - ORDER BY event_time DESC - LIMIT 1 - ) AS time_with_microseconds, - ( - SELECT event_time - FROM system.trace_log - ORDER BY event_time DESC - LIMIT 1 - ) AS t) - SELECT if(dateDiff('second', toDateTime(time_with_microseconds), toDateTime(t)) = 0, 'ok', 'fail') - ''' - assert 'ok\n' in node.query(equals_query) - assert 'ok\n' in node.query( - '''SELECT if((SELECT COUNT(event_time_microseconds) FROM system.trace_log) > 0, 'ok', 'fail')''') diff --git a/tests/queries/0_stateless/01473_event_time_microseconds.reference b/tests/queries/0_stateless/01473_event_time_microseconds.reference index 7da9f21de2f..6d95c8c1b81 100644 --- a/tests/queries/0_stateless/01473_event_time_microseconds.reference +++ b/tests/queries/0_stateless/01473_event_time_microseconds.reference @@ -1,5 +1,7 @@ 01473_metric_log_table_event_start_time_microseconds_test ok +01473_trace_log_table_event_start_time_microseconds_test +ok 01473_query_log_table_event_start_time_microseconds_test ok 01473_query_thread_log_table_event_start_time_microseconds_test diff --git a/tests/queries/0_stateless/01473_event_time_microseconds.sql b/tests/queries/0_stateless/01473_event_time_microseconds.sql index 48d620be197..8a545af9efa 100644 --- a/tests/queries/0_stateless/01473_event_time_microseconds.sql +++ b/tests/queries/0_stateless/01473_event_time_microseconds.sql @@ -23,6 +23,28 @@ WITH ( ) AS time) SELECT if(dateDiff('second', toDateTime(time_with_microseconds), toDateTime(time)) = 0, 'ok', 'fail'); +SELECT '01473_trace_log_table_event_start_time_microseconds_test'; +SET log_queries = 1; +SET query_profiler_real_time_period_ns = 0; +SET query_profiler_cpu_time_period_ns = 1000000; +-- a long enough query to trigger the query profiler and to record trace log +SELECT sleep(2) FORMAT Null; +SYSTEM FLUSH LOGS; +WITH ( + ( + SELECT event_time_microseconds + FROM system.trace_log + ORDER BY event_time DESC + LIMIT 1 + ) AS time_with_microseconds, + ( + SELECT event_time + FROM system.trace_log + ORDER BY event_time DESC + LIMIT 1 + ) AS t) +SELECT if(dateDiff('second', toDateTime(time_with_microseconds), toDateTime(t)) = 0, 'ok', 'fail'); -- success + SELECT '01473_query_log_table_event_start_time_microseconds_test'; SYSTEM FLUSH LOGS; WITH ( From cdeedda1d2efa8ddeb3a41d745480cf3b8800ca7 Mon Sep 17 00:00:00 2001 From: bharatnc Date: Mon, 14 Sep 2020 19:12:00 -0700 Subject: [PATCH 182/321] TraceCollector - time s&us using same timespec --- src/Common/Stopwatch.h | 7 ------- src/Common/TraceCollector.cpp | 9 +++++++-- 2 files changed, 7 insertions(+), 9 deletions(-) diff --git a/src/Common/Stopwatch.h b/src/Common/Stopwatch.h index 6a0d1f37eb4..a7f5e76d5be 100644 --- a/src/Common/Stopwatch.h +++ b/src/Common/Stopwatch.h @@ -13,13 +13,6 @@ inline UInt64 clock_gettime_ns(clockid_t clock_type = CLOCK_MONOTONIC) return UInt64(ts.tv_sec * 1000000000LL + ts.tv_nsec); } -inline UInt64 clock_gettime_microseconds(clockid_t clock_type = CLOCK_MONOTONIC) -{ - struct timespec ts; - clock_gettime(clock_type, &ts); - return UInt64((ts.tv_sec * 1000000LL) + (ts.tv_nsec / 1000)); -} - /** Differs from Poco::Stopwatch only by using 'clock_gettime' instead of 'gettimeofday', * returns nanoseconds instead of microseconds, and also by other minor differencies. */ diff --git a/src/Common/TraceCollector.cpp b/src/Common/TraceCollector.cpp index 35e90a6e583..104b747d431 100644 --- a/src/Common/TraceCollector.cpp +++ b/src/Common/TraceCollector.cpp @@ -141,8 +141,13 @@ void TraceCollector::run() if (trace_log) { - UInt64 time = clock_gettime_ns(CLOCK_REALTIME); - UInt64 time_in_microseconds = clock_gettime_microseconds(CLOCK_REALTIME); + // time and time_in_microseconds are both being constructed from the same timespec so that the + // times will be equal upto the precision of a second. + struct timespec ts; + clock_gettime(CLOCK_REALTIME, &ts); + + UInt64 time = UInt64(ts.tv_sec * 1000000000LL + ts.tv_nsec); + UInt64 time_in_microseconds = UInt64((ts.tv_sec * 1000000LL) + (ts.tv_nsec / 1000)); TraceLogElement element{time_t(time / 1000000000), time_in_microseconds, time, trace_type, thread_id, query_id, trace, size}; trace_log->add(element); } From eb24d19cd4efaf79a477438006344a9d0961bdc4 Mon Sep 17 00:00:00 2001 From: bharatnc Date: Mon, 14 Sep 2020 23:49:02 -0700 Subject: [PATCH 183/321] Use std::chrono instead of clock_gettime --- src/Interpreters/ThreadStatusExt.cpp | 24 ++++++++++++++++++------ src/Interpreters/executeQuery.cpp | 28 +++++++++++++--------------- 2 files changed, 31 insertions(+), 21 deletions(-) diff --git a/src/Interpreters/ThreadStatusExt.cpp b/src/Interpreters/ThreadStatusExt.cpp index d820094db1d..6a6deb2b4d3 100644 --- a/src/Interpreters/ThreadStatusExt.cpp +++ b/src/Interpreters/ThreadStatusExt.cpp @@ -318,17 +318,29 @@ void ThreadStatus::detachQuery(bool exit_if_already_detached, bool thread_exits) #endif } +inline UInt64 time_in_microseconds(std::chrono::time_point timepoint) +{ + return std::chrono::duration_cast(timepoint.time_since_epoch()).count(); +} + + +inline UInt64 time_in_seconds(std::chrono::time_point timepoint) +{ + return std::chrono::duration_cast(timepoint.time_since_epoch()).count(); +} + void ThreadStatus::logToQueryThreadLog(QueryThreadLog & thread_log) { QueryThreadLogElement elem; - // event_time and event_time_microseconds are being constructed from the same timespec - // to ensure that both the times are equal upto the precision of a second. - struct timespec ts; - clock_gettime(CLOCK_MONOTONIC, &ts); + // construct current_time and current_time_microseconds using the same time point + // so that the two times will always be equal up to a precision of a second. + const auto now = std::chrono::system_clock::now(); + auto current_time = time_in_seconds(now); + auto current_time_microseconds = time_in_microseconds(now); - elem.event_time = ts.tv_sec; - elem.event_time_microseconds = UInt64((ts.tv_sec * 1000000LL) + (ts.tv_nsec / 1000)); + elem.event_time = current_time; + elem.event_time_microseconds = current_time_microseconds; elem.query_start_time = query_start_time; elem.query_start_time_microseconds = query_start_time_microseconds; elem.query_duration_ms = (getCurrentTimeNanoseconds() - query_start_time_nanoseconds) / 1000000U; diff --git a/src/Interpreters/executeQuery.cpp b/src/Interpreters/executeQuery.cpp index 3f2c88b3678..c82748eadc0 100644 --- a/src/Interpreters/executeQuery.cpp +++ b/src/Interpreters/executeQuery.cpp @@ -206,8 +206,8 @@ static void onExceptionBeforeStart(const String & query_for_logging, Context & c elem.type = QueryLogElementType::EXCEPTION_BEFORE_START; - // all callers to onExceptionBeforeStart upstream construct the timespec for event_time and - // event_time_microseconds from the same timespec. So it can be assumed that both of these + // all callers to onExceptionBeforeStart method construct the timespec for event_time and + // event_time_microseconds from the same time point. So, it can be assumed that both of these // times are equal upto the precision of a second. elem.event_time = current_time; elem.event_time_microseconds = current_time_microseconds; @@ -319,7 +319,8 @@ static std::tuple executeQueryImpl( auto query_for_logging = prepareQueryForLogging(query, context); logQuery(query_for_logging, context, internal); - if (!internal) { + if (!internal) + { onExceptionBeforeStart(query_for_logging, context, current_time, current_time_microseconds, ast); } @@ -557,13 +558,11 @@ static std::tuple executeQueryImpl( elem.type = QueryLogElementType::QUERY_FINISH; - // event_time and event-time_microseconds are being constructed from the same timespec - // to ensure that both the times are equal upto the precision of a second. - struct timespec tspec; - clock_gettime(CLOCK_MONOTONIC, &tspec); - - elem.event_time = tspec.tv_sec; - elem.event_time_microseconds = UInt64((tspec.tv_sec * 1000000LL) + (tspec.tv_nsec / 1000)); + // construct event_time and event_time_microseconds using the same time point + // so that the two times will always be equal up to a precision of a second. + const auto time_now = std::chrono::system_clock::now(); + elem.event_time = time_in_seconds(time_now); + elem.event_time_microseconds = time_in_microseconds(time_now); status_info_to_query_log(elem, info, ast); auto progress_callback = context.getProgressCallback(); @@ -623,13 +622,12 @@ static std::tuple executeQueryImpl( elem.type = QueryLogElementType::EXCEPTION_WHILE_PROCESSING; - // event_time and event_time_microseconds are being constructed from the timespec + // event_time and event_time_microseconds are being constructed from the same time point // to ensure that both the times will be equal upto the precision of a second. - struct timespec tspec; - clock_gettime(CLOCK_MONOTONIC, &tspec); + const auto time_now = std::chrono::system_clock::now(); - elem.event_time = tspec.tv_sec; - elem.event_time_microseconds = UInt64((tspec.tv_sec * 1000000LL) + (tspec.tv_nsec / 1000)); + elem.event_time = time_in_seconds(time_now); + elem.event_time_microseconds = time_in_microseconds(time_now); elem.query_duration_ms = 1000 * (elem.event_time - elem.query_start_time); elem.exception_code = getCurrentExceptionCode(); elem.exception = getCurrentExceptionMessage(false); From 83fda9fe58b96a3c605d11a82678b648f9ec879c Mon Sep 17 00:00:00 2001 From: bharatnc Date: Tue, 15 Sep 2020 17:12:06 -0700 Subject: [PATCH 184/321] Make trace_log test to pass --- src/Interpreters/ThreadStatusExt.cpp | 11 ----------- .../0_stateless/01473_event_time_microseconds.sql | 6 ++++-- 2 files changed, 4 insertions(+), 13 deletions(-) diff --git a/src/Interpreters/ThreadStatusExt.cpp b/src/Interpreters/ThreadStatusExt.cpp index 6a6deb2b4d3..61245782ba9 100644 --- a/src/Interpreters/ThreadStatusExt.cpp +++ b/src/Interpreters/ThreadStatusExt.cpp @@ -318,17 +318,6 @@ void ThreadStatus::detachQuery(bool exit_if_already_detached, bool thread_exits) #endif } -inline UInt64 time_in_microseconds(std::chrono::time_point timepoint) -{ - return std::chrono::duration_cast(timepoint.time_since_epoch()).count(); -} - - -inline UInt64 time_in_seconds(std::chrono::time_point timepoint) -{ - return std::chrono::duration_cast(timepoint.time_since_epoch()).count(); -} - void ThreadStatus::logToQueryThreadLog(QueryThreadLog & thread_log) { QueryThreadLogElement elem; diff --git a/tests/queries/0_stateless/01473_event_time_microseconds.sql b/tests/queries/0_stateless/01473_event_time_microseconds.sql index 8a545af9efa..e4bc3b29655 100644 --- a/tests/queries/0_stateless/01473_event_time_microseconds.sql +++ b/tests/queries/0_stateless/01473_event_time_microseconds.sql @@ -25,11 +25,13 @@ SELECT if(dateDiff('second', toDateTime(time_with_microseconds), toDateTime(time SELECT '01473_trace_log_table_event_start_time_microseconds_test'; SET log_queries = 1; -SET query_profiler_real_time_period_ns = 0; -SET query_profiler_cpu_time_period_ns = 1000000; +SET query_profiler_real_time_period_ns = 10000000; +SET query_profiler_cpu_time_period_ns = 10000000; -- a long enough query to trigger the query profiler and to record trace log +SELECT count() FROM numbers(1000000000) FORMAT Null; SELECT sleep(2) FORMAT Null; SYSTEM FLUSH LOGS; +SELECT sleep(2) FORMAT Null; WITH ( ( SELECT event_time_microseconds From be54d8768acb61f39072009969af73098cd99f32 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Sun, 27 Sep 2020 16:25:28 +0300 Subject: [PATCH 185/321] try fix test --- .../__init__.py | 0 .../configs/config.d/text_log.xml | 0 tests/integration/test_text_log_level/test.py | 33 ++++++++ tests/integration/test_text_log_table/test.py | 57 -------------- .../01473_event_time_microseconds.reference | 2 + .../01473_event_time_microseconds.sql | 77 +++++++------------ tests/queries/skip_list.json | 5 ++ 7 files changed, 66 insertions(+), 108 deletions(-) rename tests/integration/{test_text_log_table => test_text_log_level}/__init__.py (100%) rename tests/integration/{test_text_log_table => test_text_log_level}/configs/config.d/text_log.xml (100%) create mode 100644 tests/integration/test_text_log_level/test.py delete mode 100644 tests/integration/test_text_log_table/test.py diff --git a/tests/integration/test_text_log_table/__init__.py b/tests/integration/test_text_log_level/__init__.py similarity index 100% rename from tests/integration/test_text_log_table/__init__.py rename to tests/integration/test_text_log_level/__init__.py diff --git a/tests/integration/test_text_log_table/configs/config.d/text_log.xml b/tests/integration/test_text_log_level/configs/config.d/text_log.xml similarity index 100% rename from tests/integration/test_text_log_table/configs/config.d/text_log.xml rename to tests/integration/test_text_log_level/configs/config.d/text_log.xml diff --git a/tests/integration/test_text_log_level/test.py b/tests/integration/test_text_log_level/test.py new file mode 100644 index 00000000000..44679481266 --- /dev/null +++ b/tests/integration/test_text_log_level/test.py @@ -0,0 +1,33 @@ +# pylint: disable=unused-argument +# pylint: disable=redefined-outer-name + +import pytest +from helpers.client import QueryRuntimeException +from helpers.cluster import ClickHouseCluster + +cluster = ClickHouseCluster(__file__) + +node = cluster.add_instance('node', main_configs=["configs/config.d/text_log.xml"]) + + +@pytest.fixture(scope='module') +def start_cluster(): + try: + cluster.start() + + yield cluster + finally: + cluster.shutdown() + + +def test_basic(start_cluster): + with pytest.raises(QueryRuntimeException): + # generates log with "Error" level + node.query('SELECT * FROM no_such_table') + + node.query('SYSTEM FLUSH LOGS') + + assert int(node.query("SELECT count() FROM system.text_log WHERE level = 'Trace'")) == 0 + assert int(node.query("SELECT count() FROM system.text_log WHERE level = 'Debug'")) == 0 + assert int(node.query("SELECT count() FROM system.text_log WHERE level = 'Information'")) >= 1 + assert int(node.query("SELECT count() FROM system.text_log WHERE level = 'Error'")) >= 1 diff --git a/tests/integration/test_text_log_table/test.py b/tests/integration/test_text_log_table/test.py deleted file mode 100644 index 60c2e35893d..00000000000 --- a/tests/integration/test_text_log_table/test.py +++ /dev/null @@ -1,57 +0,0 @@ -# pylint: disable=unused-argument -# pylint: disable=redefined-outer-name - -import pytest -from helpers.client import QueryRuntimeException -from helpers.cluster import ClickHouseCluster - -cluster = ClickHouseCluster(__file__) - -node = cluster.add_instance('node', main_configs=["configs/config.d/text_log.xml"]) - - -@pytest.fixture(scope='module') -def start_cluster(): - try: - cluster.start() - - yield cluster - finally: - cluster.shutdown() - - -def test_basic(start_cluster): - with pytest.raises(QueryRuntimeException): - # generate log with "Error" level - node.query('SELECT * FROM no_such_table') - - node.query('SYSTEM FLUSH LOGS') - - assert int(node.query("SELECT count() FROM system.text_log WHERE level = 'Trace'")) == 0 - assert int(node.query("SELECT count() FROM system.text_log WHERE level = 'Debug'")) == 0 - assert int(node.query("SELECT count() FROM system.text_log WHERE level = 'Information'")) >= 1 - assert int(node.query("SELECT count() FROM system.text_log WHERE level = 'Error'")) >= 1 - -# compare the event_time and event_time_microseconds and test -# that they are exactly equal upto their seconds parts. -def test_field_event_time_microseconds(start_cluster): - with pytest.raises(QueryRuntimeException): - # generate log with "Error" level - node.query('SELECT * FROM no_such_table') - node.query('SYSTEM FLUSH LOGS') - equals_query = '''WITH ( - ( - SELECT event_time_microseconds - FROM system.text_log - ORDER BY event_time DESC - LIMIT 1 - ) AS time_with_microseconds, - ( - SELECT event_time - FROM system.text_log - ORDER BY event_time DESC - LIMIT 1 - ) AS time) - SELECT if(dateDiff('second', toDateTime(time_with_microseconds), toDateTime(time)) = 0, 'ok', 'fail') - ''' - assert 'ok\n' in node.query(equals_query) diff --git a/tests/queries/0_stateless/01473_event_time_microseconds.reference b/tests/queries/0_stateless/01473_event_time_microseconds.reference index 6d95c8c1b81..b5533c11268 100644 --- a/tests/queries/0_stateless/01473_event_time_microseconds.reference +++ b/tests/queries/0_stateless/01473_event_time_microseconds.reference @@ -6,3 +6,5 @@ ok ok 01473_query_thread_log_table_event_start_time_microseconds_test ok +01473_text_log_table_event_start_time_microseconds_test +ok diff --git a/tests/queries/0_stateless/01473_event_time_microseconds.sql b/tests/queries/0_stateless/01473_event_time_microseconds.sql index e4bc3b29655..2e536bba7ac 100644 --- a/tests/queries/0_stateless/01473_event_time_microseconds.sql +++ b/tests/queries/0_stateless/01473_event_time_microseconds.sql @@ -4,79 +4,54 @@ -- Refer: tests/integration/test_asynchronous_metric_log_table. SET log_queries = 1; +SET query_profiler_real_time_period_ns = 100000000; +-- a long enough query to trigger the query profiler and to record trace log +SELECT sleep(2) FORMAT Null; +SET query_profiler_real_time_period_ns = 1000000000; +SYSTEM FLUSH LOGS; SELECT '01473_metric_log_table_event_start_time_microseconds_test'; -SYSTEM FLUSH LOGS; -- query assumes that the event_time field is accurate. WITH ( - ( - SELECT event_time_microseconds + SELECT event_time_microseconds, event_time FROM system.metric_log ORDER BY event_time DESC LIMIT 1 - ) AS time_with_microseconds, - ( - SELECT event_time - FROM system.metric_log - ORDER BY event_time DESC - LIMIT 1 - ) AS time) -SELECT if(dateDiff('second', toDateTime(time_with_microseconds), toDateTime(time)) = 0, 'ok', 'fail'); + ) AS time +SELECT if(dateDiff('second', toDateTime(time.1), toDateTime(time.2)) = 0, 'ok', toString(time)); SELECT '01473_trace_log_table_event_start_time_microseconds_test'; -SET log_queries = 1; -SET query_profiler_real_time_period_ns = 10000000; -SET query_profiler_cpu_time_period_ns = 10000000; --- a long enough query to trigger the query profiler and to record trace log -SELECT count() FROM numbers(1000000000) FORMAT Null; -SELECT sleep(2) FORMAT Null; -SYSTEM FLUSH LOGS; -SELECT sleep(2) FORMAT Null; WITH ( - ( - SELECT event_time_microseconds + SELECT event_time_microseconds, event_time FROM system.trace_log ORDER BY event_time DESC LIMIT 1 - ) AS time_with_microseconds, - ( - SELECT event_time - FROM system.trace_log - ORDER BY event_time DESC - LIMIT 1 - ) AS t) -SELECT if(dateDiff('second', toDateTime(time_with_microseconds), toDateTime(t)) = 0, 'ok', 'fail'); -- success + ) AS time +SELECT if(dateDiff('second', toDateTime(time.1), toDateTime(time.2)) = 0, 'ok', toString(time)); SELECT '01473_query_log_table_event_start_time_microseconds_test'; -SYSTEM FLUSH LOGS; WITH ( - ( - SELECT event_time_microseconds + SELECT event_time_microseconds, event_time FROM system.query_log ORDER BY event_time DESC LIMIT 1 - ) AS time_with_microseconds, - ( - SELECT event_time - FROM system.query_log - ORDER BY event_time DESC - LIMIT 1 - ) AS time) -SELECT if(dateDiff('second', toDateTime(time_with_microseconds), toDateTime(time)) = 0, 'ok', 'fail'); -- success + ) AS time +SELECT if(dateDiff('second', toDateTime(time.1), toDateTime(time.2)) = 0, 'ok', toString(time)); SELECT '01473_query_thread_log_table_event_start_time_microseconds_test'; -SYSTEM FLUSH LOGS; WITH ( - ( - SELECT event_time_microseconds + SELECT event_time_microseconds, event_time FROM system.query_thread_log ORDER BY event_time DESC LIMIT 1 - ) AS time_with_microseconds, - ( - SELECT event_time - FROM system.query_thread_log - ORDER BY event_time DESC - LIMIT 1 - ) AS time) -SELECT if(dateDiff('second', toDateTime(time_with_microseconds), toDateTime(time)) = 0, 'ok', 'fail'); -- success + ) AS time +SELECT if(dateDiff('second', toDateTime(time.1), toDateTime(time.2)) = 0, 'ok', toString(time)); + +SELECT '01473_text_log_table_event_start_time_microseconds_test'; +WITH ( + SELECT event_time_microseconds, event_time + FROM system.query_thread_log + ORDER BY event_time DESC + LIMIT 1 + ) AS time +SELECT if(dateDiff('second', toDateTime(time.1), toDateTime(time.2)) = 0, 'ok', toString(time)); diff --git a/tests/queries/skip_list.json b/tests/queries/skip_list.json index 26e5bbf78cf..ac8bd77fcac 100644 --- a/tests/queries/skip_list.json +++ b/tests/queries/skip_list.json @@ -17,6 +17,7 @@ "00151_replace_partition_with_different_granularity", "00157_cache_dictionary", "01193_metadata_loading", + "01473_event_time_microseconds", "01474_executable_dictionary" /// informational stderr from sanitizer at start ], "address-sanitizer": [ @@ -25,6 +26,7 @@ "memory_profiler", "odbc_roundtrip", "01103_check_cpu_instructions_at_startup", + "01473_event_time_microseconds", "01193_metadata_loading" ], "ub-sanitizer": [ @@ -33,6 +35,7 @@ "memory_profiler", "01103_check_cpu_instructions_at_startup", "00900_orc_load", + "01473_event_time_microseconds", "01193_metadata_loading" ], "memory-sanitizer": [ @@ -43,6 +46,7 @@ "01086_odbc_roundtrip", /// can't pass because odbc libraries are not instrumented "00877_memory_limit_for_new_delete", /// memory limits don't work correctly under msan because it replaces malloc/free "01114_mysql_database_engine_segfault", /// it fails in _nss_files_parse_servent while using NSS from GLibc to authenticate (need to get rid of it) + "01473_event_time_microseconds", "01193_metadata_loading" ], "debug-build": [ @@ -57,6 +61,7 @@ "01037_polygon_dicts_", "hyperscan", "01193_metadata_loading", + "01473_event_time_microseconds", "01396_inactive_replica_cleanup_nodes" ], "unbundled-build": [ From 52a13a71806ac6c8d3910ad535e77c153fa31c77 Mon Sep 17 00:00:00 2001 From: Vxider Date: Mon, 14 Sep 2020 15:46:27 +0800 Subject: [PATCH 186/321] Support default arguments for Replicated engine --- src/Core/Settings.h | 2 ++ src/Storages/MergeTree/registerStorageMergeTree.cpp | 13 ++++++++++++- 2 files changed, 14 insertions(+), 1 deletion(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 9449cd571a1..24c7e517239 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -155,6 +155,8 @@ class IColumn; M(UInt64, max_concurrent_queries_for_user, 0, "The maximum number of concurrent requests per user.", 0) \ \ M(Bool, insert_deduplicate, true, "For INSERT queries in the replicated table, specifies that deduplication of insertings blocks should be performed", 0) \ + M(String, default_replica_path, "", "Default replica path for ReplicatedMergeTree. Allow to omit arguments for Replicated table engine if default_replica_path and default_replica_name are specified.", 0) \ + M(String, default_replica_name, "", "Default replica name for ReplicatedMergeTree. Allow to omit arguments for Replicated table engine if default_replica_path and default_replica_name are specified.", 0) \ \ M(UInt64, insert_quorum, 0, "For INSERT queries in the replicated table, wait writing for the specified number of replicas and linearize the addition of the data. 0 - disabled.", 0) \ M(Milliseconds, insert_quorum_timeout, 600000, "", 0) \ diff --git a/src/Storages/MergeTree/registerStorageMergeTree.cpp b/src/Storages/MergeTree/registerStorageMergeTree.cpp index 1188fd2edd3..dc75d0f0e8e 100644 --- a/src/Storages/MergeTree/registerStorageMergeTree.cpp +++ b/src/Storages/MergeTree/registerStorageMergeTree.cpp @@ -293,8 +293,13 @@ static StoragePtr create(const StorageFactory::Arguments & args) String name_part = args.engine_name.substr(0, args.engine_name.size() - strlen("MergeTree")); bool replicated = startsWith(name_part, "Replicated"); + bool has_replicated_default_args = false; if (replicated) + { name_part = name_part.substr(strlen("Replicated")); + has_replicated_default_args = args.context.getSettingsRef().default_replica_path.value != "" + && args.context.getSettingsRef().default_replica_name.value != ""; + } MergeTreeData::MergingParams merging_params; merging_params.mode = MergeTreeData::MergingParams::Ordinary; @@ -338,7 +343,7 @@ static StoragePtr create(const StorageFactory::Arguments & args) if (replicated) { - if (is_extended_storage_def) + if (is_extended_storage_def || has_replicated_default_args) { add_optional_param("path in ZooKeeper"); add_optional_param("replica name"); @@ -441,6 +446,12 @@ static StoragePtr create(const StorageFactory::Arguments & args) "No replica name in config" + getMergeTreeVerboseHelp(is_extended_storage_def), ErrorCodes::NO_REPLICA_NAME_GIVEN); ++arg_num; } + else if (has_replicated_default_args && !has_arguments) + { + zookeeper_path + = args.context.getSettingsRef().default_replica_path.value + "/" + args.table_id.database_name + "." + args.table_id.table_name; + replica_name = args.context.getSettingsRef().default_replica_name; + } else if (is_extended_storage_def && !has_arguments) { /// Try use default values if arguments are not specified. From a7c595ab7ac6859460fc02e1169c59a6222ee801 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Sat, 19 Sep 2020 15:59:37 +0300 Subject: [PATCH 187/321] minor fixes --- src/Core/Settings.h | 2 -- .../MergeTree/registerStorageMergeTree.cpp | 20 +++++-------------- 2 files changed, 5 insertions(+), 17 deletions(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 24c7e517239..9449cd571a1 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -155,8 +155,6 @@ class IColumn; M(UInt64, max_concurrent_queries_for_user, 0, "The maximum number of concurrent requests per user.", 0) \ \ M(Bool, insert_deduplicate, true, "For INSERT queries in the replicated table, specifies that deduplication of insertings blocks should be performed", 0) \ - M(String, default_replica_path, "", "Default replica path for ReplicatedMergeTree. Allow to omit arguments for Replicated table engine if default_replica_path and default_replica_name are specified.", 0) \ - M(String, default_replica_name, "", "Default replica name for ReplicatedMergeTree. Allow to omit arguments for Replicated table engine if default_replica_path and default_replica_name are specified.", 0) \ \ M(UInt64, insert_quorum, 0, "For INSERT queries in the replicated table, wait writing for the specified number of replicas and linearize the addition of the data. 0 - disabled.", 0) \ M(Milliseconds, insert_quorum_timeout, 600000, "", 0) \ diff --git a/src/Storages/MergeTree/registerStorageMergeTree.cpp b/src/Storages/MergeTree/registerStorageMergeTree.cpp index dc75d0f0e8e..a9f7576f896 100644 --- a/src/Storages/MergeTree/registerStorageMergeTree.cpp +++ b/src/Storages/MergeTree/registerStorageMergeTree.cpp @@ -293,13 +293,8 @@ static StoragePtr create(const StorageFactory::Arguments & args) String name_part = args.engine_name.substr(0, args.engine_name.size() - strlen("MergeTree")); bool replicated = startsWith(name_part, "Replicated"); - bool has_replicated_default_args = false; if (replicated) - { name_part = name_part.substr(strlen("Replicated")); - has_replicated_default_args = args.context.getSettingsRef().default_replica_path.value != "" - && args.context.getSettingsRef().default_replica_name.value != ""; - } MergeTreeData::MergingParams merging_params; merging_params.mode = MergeTreeData::MergingParams::Ordinary; @@ -343,7 +338,7 @@ static StoragePtr create(const StorageFactory::Arguments & args) if (replicated) { - if (is_extended_storage_def || has_replicated_default_args) + if (is_extended_storage_def) { add_optional_param("path in ZooKeeper"); add_optional_param("replica name"); @@ -446,18 +441,13 @@ static StoragePtr create(const StorageFactory::Arguments & args) "No replica name in config" + getMergeTreeVerboseHelp(is_extended_storage_def), ErrorCodes::NO_REPLICA_NAME_GIVEN); ++arg_num; } - else if (has_replicated_default_args && !has_arguments) - { - zookeeper_path - = args.context.getSettingsRef().default_replica_path.value + "/" + args.table_id.database_name + "." + args.table_id.table_name; - replica_name = args.context.getSettingsRef().default_replica_name; - } else if (is_extended_storage_def && !has_arguments) { /// Try use default values if arguments are not specified. - /// It works for ON CLUSTER queries when database engine is Atomic and there are {shard} and {replica} in config. - zookeeper_path = "/clickhouse/tables/{uuid}/{shard}"; - replica_name = "{replica}"; /// TODO maybe use hostname if {replica} is not defined? + /// Note: {uuid} macro works for ON CLUSTER queries when database engine is Atomic. + zookeeper_path = args.context.getConfigRef().getString("default_replica_path", "/clickhouse/tables/{uuid}/{shard}"); + /// TODO maybe use hostname if {replica} is not defined? + replica_name = args.context.getConfigRef().getString("default_replica_name", "{replica}"); } else throw Exception("Expected two string literal arguments: zookeper_path and replica_name", ErrorCodes::BAD_ARGUMENTS); From f67fd49a73d8fd65d23e3a8daab1dbfef0bb14f6 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Sat, 26 Sep 2020 22:18:28 +0300 Subject: [PATCH 188/321] add review suggestions --- programs/local/LocalServer.cpp | 2 +- programs/server/Server.cpp | 4 +- src/Common/Macros.cpp | 54 +++++++++++--- src/Common/Macros.h | 11 ++- src/Databases/DatabaseAtomic.cpp | 2 + src/Interpreters/InterpreterCreateQuery.cpp | 2 +- src/Storages/IStorage.h | 2 + .../MergeTree/registerStorageMergeTree.cpp | 74 ++++++++++++++++--- src/Storages/StorageFactory.cpp | 11 ++- src/Storages/StorageReplicatedMergeTree.cpp | 13 +++- src/Storages/StorageReplicatedMergeTree.h | 8 +- tests/config/config.d/macros.xml | 2 + .../integration/test_distributed_ddl/test.py | 8 +- ..._zookeeper_path_macros_unfolding.reference | 4 + .../01148_zookeeper_path_macros_unfolding.sql | 19 +++++ 15 files changed, 182 insertions(+), 34 deletions(-) create mode 100644 tests/queries/0_stateless/01148_zookeeper_path_macros_unfolding.reference create mode 100644 tests/queries/0_stateless/01148_zookeeper_path_macros_unfolding.sql diff --git a/programs/local/LocalServer.cpp b/programs/local/LocalServer.cpp index 5cdf5766a44..41da477152c 100644 --- a/programs/local/LocalServer.cpp +++ b/programs/local/LocalServer.cpp @@ -210,7 +210,7 @@ try /// Maybe useless if (config().has("macros")) - context->setMacros(std::make_unique(config(), "macros")); + context->setMacros(std::make_unique(config(), "macros", log)); /// Skip networking diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index 6341653ee2f..0019ba7e76c 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -534,7 +534,7 @@ int Server::main(const std::vector & /*args*/) } if (config().has("macros")) - global_context->setMacros(std::make_unique(config(), "macros")); + global_context->setMacros(std::make_unique(config(), "macros", log)); /// Initialize main config reloader. std::string include_from_path = config().getString("include_from", "/etc/metrika.xml"); @@ -559,7 +559,7 @@ int Server::main(const std::vector & /*args*/) //setTextLog(global_context->getTextLog()); //buildLoggers(*config, logger()); global_context->setClustersConfig(config); - global_context->setMacros(std::make_unique(*config, "macros")); + global_context->setMacros(std::make_unique(*config, "macros", log)); global_context->setExternalAuthenticatorsConfig(*config); /// Setup protection to avoid accidental DROP for big tables (that are greater than 50 GB by default) diff --git a/src/Common/Macros.cpp b/src/Common/Macros.cpp index e3735c44359..b98daa0ec78 100644 --- a/src/Common/Macros.cpp +++ b/src/Common/Macros.cpp @@ -2,6 +2,7 @@ #include #include #include +#include namespace DB @@ -12,19 +13,31 @@ namespace ErrorCodes extern const int SYNTAX_ERROR; } -Macros::Macros(const Poco::Util::AbstractConfiguration & config, const String & root_key) +Macros::Macros(const Poco::Util::AbstractConfiguration & config, const String & root_key, Poco::Logger * log) { Poco::Util::AbstractConfiguration::Keys keys; config.keys(root_key, keys); for (const String & key : keys) { macros[key] = config.getString(root_key + "." + key); + if (key == "database" || key == "table" || key == "uuid") + { + LOG_WARNING(log, + "Config file contains '{}' macro. This macro has special meaning " + "and it's explicit definition is not recommended. Implicit unfolding for " + "'database', 'table' and 'uuid' macros will be disabled.", + key); + enable_special_macros = false; + } } } String Macros::expand(const String & s, MacroExpansionInfo & info) const { + /// Do not allow recursion if we expand only special macros, because it will be infinite recursion + assert(info.level == 0 || !info.expand_special_macros_only); + if (s.find('{') == String::npos) return s; @@ -34,6 +47,10 @@ String Macros::expand(const String & s, if (info.level >= 10) throw Exception("Too deep recursion while expanding macros: '" + s + "'", ErrorCodes::SYNTAX_ERROR); + /// If config file contains explicit special macro, then we do not expand it in this mode. + if (!enable_special_macros && info.expand_special_macros_only) + return s; + String res; size_t pos = 0; while (true) @@ -59,15 +76,21 @@ String Macros::expand(const String & s, auto it = macros.find(macro_name); /// Prefer explicit macros over implicit. - if (it != macros.end()) + if (it != macros.end() && !info.expand_special_macros_only) res += it->second; - else if (macro_name == "database" && !info.database_name.empty()) - res += info.database_name; - else if (macro_name == "table" && !info.table_name.empty()) - res += info.table_name; + else if (macro_name == "database" && !info.table_id.database_name.empty()) + { + res += info.table_id.database_name; + info.expanded_database = true; + } + else if (macro_name == "table" && !info.table_id.table_name.empty()) + { + res += info.table_id.table_name; + info.expanded_table = true; + } else if (macro_name == "uuid") { - if (info.uuid == UUIDHelpers::Nil) + if (info.table_id.uuid == UUIDHelpers::Nil) throw Exception("Macro 'uuid' and empty arguments of ReplicatedMergeTree " "are supported only for ON CLUSTER queries with Atomic database engine", ErrorCodes::SYNTAX_ERROR); @@ -76,12 +99,16 @@ String Macros::expand(const String & s, /// It becomes impossible to check if {uuid} is contained inside some unknown macro. if (info.level) throw Exception("Macro 'uuid' should not be inside another macro", ErrorCodes::SYNTAX_ERROR); - res += toString(info.uuid); + res += toString(info.table_id.uuid); info.expanded_uuid = true; } - else if (info.ignore_unknown) + else if (info.ignore_unknown || info.expand_special_macros_only) { + if (info.expand_special_macros_only) + res += '{'; res += macro_name; + if (info.expand_special_macros_only) + res += '}'; info.has_unknown = true; } else @@ -93,6 +120,9 @@ String Macros::expand(const String & s, } ++info.level; + if (info.expand_special_macros_only) + return res; + return expand(res, info); } @@ -113,9 +143,9 @@ String Macros::expand(const String & s) const String Macros::expand(const String & s, const StorageID & table_id, bool allow_uuid) const { MacroExpansionInfo info; - info.database_name = table_id.database_name; - info.table_name = table_id.table_name; - info.uuid = allow_uuid ? table_id.uuid : UUIDHelpers::Nil; + info.table_id = table_id; + if (!allow_uuid) + info.table_id.uuid = UUIDHelpers::Nil; return expand(s, info); } diff --git a/src/Common/Macros.h b/src/Common/Macros.h index 6e4f25d55ef..c04405a83de 100644 --- a/src/Common/Macros.h +++ b/src/Common/Macros.h @@ -13,6 +13,7 @@ namespace Poco { class AbstractConfiguration; } + class Logger; } @@ -25,18 +26,19 @@ class Macros { public: Macros() = default; - Macros(const Poco::Util::AbstractConfiguration & config, const String & key); + Macros(const Poco::Util::AbstractConfiguration & config, const String & key, Poco::Logger * log); struct MacroExpansionInfo { /// Settings - String database_name; - String table_name; - UUID uuid = UUIDHelpers::Nil; + StorageID table_id = StorageID::createEmpty(); bool ignore_unknown = false; + bool expand_special_macros_only = false; /// Information about macro expansion size_t level = 0; + bool expanded_database = false; + bool expanded_table = false; bool expanded_uuid = false; bool has_unknown = false; }; @@ -64,6 +66,7 @@ public: private: MacroMap macros; + bool enable_special_macros = true; }; diff --git a/src/Databases/DatabaseAtomic.cpp b/src/Databases/DatabaseAtomic.cpp index ed17a8eccb1..6ad547898e2 100644 --- a/src/Databases/DatabaseAtomic.cpp +++ b/src/Databases/DatabaseAtomic.cpp @@ -207,11 +207,13 @@ void DatabaseAtomic::renameTable(const Context & context, const String & table_n throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Cannot move dictionary to other database"); StoragePtr table = getTableUnlocked(table_name, db_lock); + table->checkTableCanBeRenamed(); assert_can_move_mat_view(table); StoragePtr other_table; if (exchange) { other_table = other_db.getTableUnlocked(to_table_name, other_db_lock); + other_table->checkTableCanBeRenamed(); assert_can_move_mat_view(other_table); } diff --git a/src/Interpreters/InterpreterCreateQuery.cpp b/src/Interpreters/InterpreterCreateQuery.cpp index 6f318b3658a..1b9c2ca3431 100644 --- a/src/Interpreters/InterpreterCreateQuery.cpp +++ b/src/Interpreters/InterpreterCreateQuery.cpp @@ -888,7 +888,7 @@ void InterpreterCreateQuery::prepareOnClusterQuery(ASTCreateQuery & create, cons { String zk_path = create.storage->engine->arguments->children[0]->as()->value.get(); Macros::MacroExpansionInfo info; - info.uuid = create.uuid; + info.table_id.uuid = create.uuid; info.ignore_unknown = true; context.getMacros()->expand(zk_path, info); if (!info.expanded_uuid) diff --git a/src/Storages/IStorage.h b/src/Storages/IStorage.h index 40500e78de1..4a2e70aa84b 100644 --- a/src/Storages/IStorage.h +++ b/src/Storages/IStorage.h @@ -337,6 +337,8 @@ public: throw Exception("Truncate is not supported by storage " + getName(), ErrorCodes::NOT_IMPLEMENTED); } + virtual void checkTableCanBeRenamed() const {} + /** Rename the table. * Renaming a name in a file with metadata, the name in the list of tables in the RAM, is done separately. * In this function, you need to rename the directory with the data, if any. diff --git a/src/Storages/MergeTree/registerStorageMergeTree.cpp b/src/Storages/MergeTree/registerStorageMergeTree.cpp index a9f7576f896..2b0d1a60657 100644 --- a/src/Storages/MergeTree/registerStorageMergeTree.cpp +++ b/src/Storages/MergeTree/registerStorageMergeTree.cpp @@ -412,26 +412,31 @@ static StoragePtr create(const StorageFactory::Arguments & args) /// For Replicated. String zookeeper_path; String replica_name; + bool allow_renaming = true; if (replicated) { bool has_arguments = arg_num + 2 <= arg_cnt; bool has_valid_arguments = has_arguments && engine_args[arg_num]->as() && engine_args[arg_num + 1]->as(); + ASTLiteral * ast_zk_path; + ASTLiteral * ast_replica_name; + if (has_valid_arguments) { - const auto * ast = engine_args[arg_num]->as(); - if (ast && ast->value.getType() == Field::Types::String) - zookeeper_path = safeGet(ast->value); + /// Get path and name from engine arguments + ast_zk_path = engine_args[arg_num]->as(); + if (ast_zk_path && ast_zk_path->value.getType() == Field::Types::String) + zookeeper_path = safeGet(ast_zk_path->value); else throw Exception( "Path in ZooKeeper must be a string literal" + getMergeTreeVerboseHelp(is_extended_storage_def), ErrorCodes::BAD_ARGUMENTS); ++arg_num; - ast = engine_args[arg_num]->as(); - if (ast && ast->value.getType() == Field::Types::String) - replica_name = safeGet(ast->value); + ast_replica_name = engine_args[arg_num]->as(); + if (ast_replica_name && ast_replica_name->value.getType() == Field::Types::String) + replica_name = safeGet(ast_replica_name->value); else throw Exception( "Replica name must be a string literal" + getMergeTreeVerboseHelp(is_extended_storage_def), ErrorCodes::BAD_ARGUMENTS); @@ -448,6 +453,20 @@ static StoragePtr create(const StorageFactory::Arguments & args) zookeeper_path = args.context.getConfigRef().getString("default_replica_path", "/clickhouse/tables/{uuid}/{shard}"); /// TODO maybe use hostname if {replica} is not defined? replica_name = args.context.getConfigRef().getString("default_replica_name", "{replica}"); + + /// Modify query, so default values will be written to metadata + assert(arg_num == 0); + ASTs old_args; + std::swap(engine_args, old_args); + auto path_arg = std::make_shared(zookeeper_path); + auto name_arg = std::make_shared(replica_name); + ast_zk_path = path_arg.get(); + ast_replica_name = name_arg.get(); + engine_args.emplace_back(std::move(path_arg)); + engine_args.emplace_back(std::move(name_arg)); + std::move(std::begin(old_args), std::end(old_args), std::back_inserter(engine_args)); + arg_num = 2; + arg_cnt += 2; } else throw Exception("Expected two string literal arguments: zookeper_path and replica_name", ErrorCodes::BAD_ARGUMENTS); @@ -455,8 +474,44 @@ static StoragePtr create(const StorageFactory::Arguments & args) /// Allow implicit {uuid} macros only for zookeeper_path in ON CLUSTER queries bool is_on_cluster = args.local_context.getClientInfo().query_kind == ClientInfo::QueryKind::SECONDARY_QUERY; bool allow_uuid_macro = is_on_cluster || args.query.attach; - zookeeper_path = args.context.getMacros()->expand(zookeeper_path, args.table_id, allow_uuid_macro); - replica_name = args.context.getMacros()->expand(replica_name, args.table_id, false); + + /// Unfold {database} and {table} macro on table creation, so table can be renamed. + /// We also unfold {uuid} macro, so path will not be broken after moving table from Atomic to Ordinary database. + if (!args.attach) + { + Macros::MacroExpansionInfo info; + /// NOTE: it's not recursive + info.expand_special_macros_only = true; + info.table_id = args.table_id; + if (!allow_uuid_macro) + info.table_id.uuid = UUIDHelpers::Nil; + zookeeper_path = args.context.getMacros()->expand(zookeeper_path, info); + + info.level = 0; + info.table_id.uuid = UUIDHelpers::Nil; + replica_name = args.context.getMacros()->expand(replica_name, info); + } + + ast_zk_path->value = zookeeper_path; + ast_replica_name->value = replica_name; + + /// Expand other macros (such as {shard} and {replica}). We do not expand them on previous step + /// to make possible copying metadata files between replicas. + Macros::MacroExpansionInfo info; + info.table_id = args.table_id; + if (!allow_uuid_macro) + info.table_id.uuid = UUIDHelpers::Nil; + zookeeper_path = args.context.getMacros()->expand(zookeeper_path, info); + + info.level = 0; + info.table_id.uuid = UUIDHelpers::Nil; + replica_name = args.context.getMacros()->expand(replica_name, info); + + /// We do not allow renaming table with these macros in metadata, because zookeeper_path will be broken after RENAME TABLE. + /// NOTE: it may happen if table was created by older version of ClickHouse (< 20.10) and macros was not unfolded on table creation + /// or if one of these macros is recursively expanded from some other macro. + if (info.expanded_database || info.expanded_table) + allow_renaming = false; } /// This merging param maybe used as part of sorting key @@ -707,7 +762,8 @@ static StoragePtr create(const StorageFactory::Arguments & args) date_column_name, merging_params, std::move(storage_settings), - args.has_force_restore_data_flag); + args.has_force_restore_data_flag, + allow_renaming); else return StorageMergeTree::create( args.table_id, diff --git a/src/Storages/StorageFactory.cpp b/src/Storages/StorageFactory.cpp index 0a8ceb4b8e5..eda9f36010f 100644 --- a/src/Storages/StorageFactory.cpp +++ b/src/Storages/StorageFactory.cpp @@ -180,7 +180,16 @@ StoragePtr StorageFactory::get( .has_force_restore_data_flag = has_force_restore_data_flag }; - return storages.at(name).creator_fn(arguments); + auto res = storages.at(name).creator_fn(arguments); + if (!empty_engine_args.empty()) + { + /// Storage creator modified empty arguments list, so we should modify the query + assert(storage_def && storage_def->engine && !storage_def->engine->arguments); + storage_def->engine->arguments = std::make_shared(); + storage_def->engine->children.push_back(storage_def->engine->arguments); + storage_def->engine->arguments->children = empty_engine_args; + } + return res; } StorageFactory & StorageFactory::instance() diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 65c0c5ac313..9613bd5111d 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -178,7 +178,8 @@ StorageReplicatedMergeTree::StorageReplicatedMergeTree( const String & date_column_name, const MergingParams & merging_params_, std::unique_ptr settings_, - bool has_force_restore_data_flag) + bool has_force_restore_data_flag, + bool allow_renaming_) : MergeTreeData(table_id_, relative_data_path_, metadata_, @@ -200,6 +201,7 @@ StorageReplicatedMergeTree::StorageReplicatedMergeTree( , cleanup_thread(*this) , part_check_thread(*this) , restarting_thread(*this) + , allow_renaming(allow_renaming_) { queue_updating_task = global_context.getSchedulePool().createTask( getStorageID().getFullTableName() + " (StorageReplicatedMergeTree::queueUpdatingTask)", [this]{ queueUpdatingTask(); }); @@ -4187,8 +4189,17 @@ void StorageReplicatedMergeTree::checkTableCanBeDropped() const global_context.checkTableCanBeDropped(table_id.database_name, table_id.table_name, getTotalActiveSizeInBytes()); } +void StorageReplicatedMergeTree::checkTableCanBeRenamed() const +{ + if (!allow_renaming) + throw Exception("Cannot rename Replicated table, because zookeeper_path contains implicit 'database' or 'table' macro. " + "We cannot rename path in ZooKeeper, so path may become inconsistent with table name. If you really want to rename table, " + "you should edit metadata file first and restart server or reattach the table.", ErrorCodes::NOT_IMPLEMENTED); +} + void StorageReplicatedMergeTree::rename(const String & new_path_to_table_data, const StorageID & new_table_id) { + checkTableCanBeRenamed(); MergeTreeData::rename(new_path_to_table_data, new_table_id); /// Update table name in zookeeper diff --git a/src/Storages/StorageReplicatedMergeTree.h b/src/Storages/StorageReplicatedMergeTree.h index 2bc9265331d..d851082d5c2 100644 --- a/src/Storages/StorageReplicatedMergeTree.h +++ b/src/Storages/StorageReplicatedMergeTree.h @@ -128,6 +128,8 @@ public: void truncate(const ASTPtr &, const StorageMetadataPtr &, const Context &, TableExclusiveLockHolder &) override; + void checkTableCanBeRenamed() const override; + void rename(const String & new_path_to_table_data, const StorageID & new_table_id) override; bool supportsIndexForIn() const override { return true; } @@ -304,6 +306,9 @@ private: /// True if replica was created for existing table with fixed granularity bool other_replicas_fixed_granularity = false; + /// Do not allow RENAME TABLE if zookeeper_path contains {database} or {table} macro + const bool allow_renaming; + template void foreachCommittedParts(const Func & func) const; @@ -571,7 +576,8 @@ protected: const String & date_column_name, const MergingParams & merging_params_, std::unique_ptr settings_, - bool has_force_restore_data_flag); + bool has_force_restore_data_flag, + bool allow_renaming_); }; diff --git a/tests/config/config.d/macros.xml b/tests/config/config.d/macros.xml index 97c3065471f..4902b12bc81 100644 --- a/tests/config/config.d/macros.xml +++ b/tests/config/config.d/macros.xml @@ -3,5 +3,7 @@ Hello, world! s1 r1 + /clickhouse/tables/{database}/{shard}/ + table_{table} diff --git a/tests/integration/test_distributed_ddl/test.py b/tests/integration/test_distributed_ddl/test.py index b788dafe167..9f01fa7ed5b 100755 --- a/tests/integration/test_distributed_ddl/test.py +++ b/tests/integration/test_distributed_ddl/test.py @@ -330,10 +330,12 @@ def test_replicated_without_arguments(test_cluster): assert "are supported only for ON CLUSTER queries with Atomic database engine" in \ instance.query_and_get_error("CREATE TABLE test_atomic.rmt (n UInt64, s String) ENGINE=ReplicatedMergeTree ORDER BY n") test_cluster.ddl_check_query(instance, - "CREATE TABLE test_atomic.rmt ON CLUSTER cluster (n UInt64, s String) ENGINE=ReplicatedMergeTree ORDER BY n") + "CREATE TABLE test_atomic.rmt ON CLUSTER cluster (n UInt64, s String) ENGINE=ReplicatedMergeTree() ORDER BY n") test_cluster.ddl_check_query(instance, "DROP TABLE test_atomic.rmt ON CLUSTER cluster") test_cluster.ddl_check_query(instance, - "CREATE TABLE test_atomic.rmt ON CLUSTER cluster (n UInt64, s String) ENGINE=ReplicatedMergeTree ORDER BY n") + "CREATE TABLE test_atomic.rmt UUID '12345678-0000-4000-8000-000000000001' ON CLUSTER cluster (n UInt64, s String) ENGINE=ReplicatedMergeTree ORDER BY n") + assert instance.query("SHOW CREATE test_atomic.rmt FORMAT TSVRaw") == \ + "CREATE TABLE test_atomic.rmt\n(\n `n` UInt64,\n `s` String\n)\nENGINE = ReplicatedMergeTree('/clickhouse/tables/12345678-0000-4000-8000-000000000001/{shard}', '{replica}')\nORDER BY n\nSETTINGS index_granularity = 8192\n" test_cluster.ddl_check_query(instance, "RENAME TABLE test_atomic.rmt TO test_atomic.rmt_renamed ON CLUSTER cluster") test_cluster.ddl_check_query(instance, "CREATE TABLE test_atomic.rmt ON CLUSTER cluster (n UInt64, s String) ENGINE=ReplicatedMergeTree('/clickhouse/tables/{uuid}/{shard}', '{replica}') ORDER BY n") @@ -349,6 +351,8 @@ def test_replicated_without_arguments(test_cluster): assert "are supported only for ON CLUSTER queries with Atomic database engine" in \ instance.query_and_get_error("CREATE TABLE test_ordinary.rmt ON CLUSTER cluster (n UInt64, s String) ENGINE=ReplicatedMergeTree('/{shard}/{uuid}/', '{replica}') ORDER BY n") test_cluster.ddl_check_query(instance, "CREATE TABLE test_ordinary.rmt ON CLUSTER cluster (n UInt64, s String) ENGINE=ReplicatedMergeTree('/{shard}/{table}/', '{replica}') ORDER BY n") + assert instance.query("SHOW CREATE test_ordinary.rmt FORMAT TSVRaw") == \ + "CREATE TABLE test_ordinary.rmt\n(\n `n` UInt64,\n `s` String\n)\nENGINE = ReplicatedMergeTree('/{shard}/rmt/', '{replica}')\nORDER BY n\nSETTINGS index_granularity = 8192\n" test_cluster.ddl_check_query(instance, "DROP DATABASE test_ordinary ON CLUSTER cluster") test_cluster.pm_random_drops.push_rules(rules) diff --git a/tests/queries/0_stateless/01148_zookeeper_path_macros_unfolding.reference b/tests/queries/0_stateless/01148_zookeeper_path_macros_unfolding.reference new file mode 100644 index 00000000000..d217855586b --- /dev/null +++ b/tests/queries/0_stateless/01148_zookeeper_path_macros_unfolding.reference @@ -0,0 +1,4 @@ +CREATE TABLE default.rmt\n(\n `n` UInt64,\n `s` String\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/test_01148/{shard}/default/rmt\', \'{replica}\')\nORDER BY n\nSETTINGS index_granularity = 8192 +CREATE TABLE default.rmt1\n(\n `n` UInt64,\n `s` String\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/test_01148/{shard}/default/rmt\', \'{replica}\')\nORDER BY n\nSETTINGS index_granularity = 8192 +CREATE TABLE default.rmt\n(\n `n` UInt64,\n `s` String\n)\nENGINE = ReplicatedMergeTree(\'{default_path_test}test_01148\', \'{default_name_test}\')\nORDER BY n\nSETTINGS index_granularity = 8192 +CREATE TABLE default.rmt\n(\n `n` UInt64,\n `s` String\n)\nENGINE = ReplicatedMergeTree(\'{default_path_test}test_01148\', \'{default_name_test}\')\nORDER BY n\nSETTINGS index_granularity = 8192 diff --git a/tests/queries/0_stateless/01148_zookeeper_path_macros_unfolding.sql b/tests/queries/0_stateless/01148_zookeeper_path_macros_unfolding.sql new file mode 100644 index 00000000000..e13b25d1706 --- /dev/null +++ b/tests/queries/0_stateless/01148_zookeeper_path_macros_unfolding.sql @@ -0,0 +1,19 @@ +DROP TABLE IF EXISTS rmt; + +CREATE TABLE rmt (n UInt64, s String) ENGINE = ReplicatedMergeTree('/clickhouse/test_01148/{shard}/{database}/{table}', '{replica}') ORDER BY n; +SHOW CREATE TABLE rmt; +RENAME TABLE rmt TO rmt1; +DETACH TABLE rmt1; +ATTACH TABLE rmt1; +SHOW CREATE TABLE rmt1; + +CREATE TABLE rmt (n UInt64, s String) ENGINE = ReplicatedMergeTree('{default_path_test}{uuid}', '{default_name_test}') ORDER BY n; -- { serverError 62 } +CREATE TABLE rmt (n UInt64, s String) ENGINE = ReplicatedMergeTree('{default_path_test}test_01148', '{default_name_test}') ORDER BY n; +SHOW CREATE TABLE rmt; +RENAME TABLE rmt TO rmt2; -- { serverError 48 } +DETACH TABLE rmt; +ATTACH TABLE rmt; +SHOW CREATE TABLE rmt; + +DROP TABLE rmt; +DROP TABLE rmt1; From 5674949c183db9220c19bd86e8a58537d15949b9 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Sun, 27 Sep 2020 14:10:45 +0300 Subject: [PATCH 189/321] fix sync --- src/Common/Macros.cpp | 11 ++++++----- src/Common/Macros.h | 2 +- 2 files changed, 7 insertions(+), 6 deletions(-) diff --git a/src/Common/Macros.cpp b/src/Common/Macros.cpp index b98daa0ec78..b8e25499c0b 100644 --- a/src/Common/Macros.cpp +++ b/src/Common/Macros.cpp @@ -22,11 +22,12 @@ Macros::Macros(const Poco::Util::AbstractConfiguration & config, const String & macros[key] = config.getString(root_key + "." + key); if (key == "database" || key == "table" || key == "uuid") { - LOG_WARNING(log, - "Config file contains '{}' macro. This macro has special meaning " - "and it's explicit definition is not recommended. Implicit unfolding for " - "'database', 'table' and 'uuid' macros will be disabled.", - key); + if (log) + LOG_WARNING(log, + "Config file contains '{}' macro. This macro has special meaning " + "and it's explicit definition is not recommended. Implicit unfolding for " + "'database', 'table' and 'uuid' macros will be disabled.", + key); enable_special_macros = false; } } diff --git a/src/Common/Macros.h b/src/Common/Macros.h index c04405a83de..3082452e297 100644 --- a/src/Common/Macros.h +++ b/src/Common/Macros.h @@ -26,7 +26,7 @@ class Macros { public: Macros() = default; - Macros(const Poco::Util::AbstractConfiguration & config, const String & key, Poco::Logger * log); + Macros(const Poco::Util::AbstractConfiguration & config, const String & key, Poco::Logger * log = nullptr); struct MacroExpansionInfo { From 2b326462314130eab92a6887501e35b552c216fd Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 28 Sep 2020 01:09:32 +0300 Subject: [PATCH 190/321] Add support for "Raw" column format for Regexp format --- .../Formats/Impl/RegexpRowInputFormat.cpp | 8 ++++++++ .../01508_format_regexp_raw.reference | 1 + .../0_stateless/01508_format_regexp_raw.sh | 17 +++++++++++++++++ 3 files changed, 26 insertions(+) create mode 100644 tests/queries/0_stateless/01508_format_regexp_raw.reference create mode 100755 tests/queries/0_stateless/01508_format_regexp_raw.sh diff --git a/src/Processors/Formats/Impl/RegexpRowInputFormat.cpp b/src/Processors/Formats/Impl/RegexpRowInputFormat.cpp index 7763de1642d..d193a502f2e 100644 --- a/src/Processors/Formats/Impl/RegexpRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/RegexpRowInputFormat.cpp @@ -51,6 +51,8 @@ RegexpRowInputFormat::ColumnFormat RegexpRowInputFormat::stringToFormat(const St return ColumnFormat::Csv; if (format == "JSON") return ColumnFormat::Json; + if (format == "Raw") + return ColumnFormat::Raw; throw Exception("Unsupported column format \"" + format + "\".", ErrorCodes::BAD_ARGUMENTS); } @@ -88,6 +90,12 @@ bool RegexpRowInputFormat::readField(size_t index, MutableColumns & columns) else type->deserializeAsTextJSON(*columns[index], field_buf, format_settings); break; + case ColumnFormat::Raw: + if (parse_as_nullable) + read = DataTypeNullable::deserializeWholeText(*columns[index], field_buf, format_settings, type); + else + type->deserializeAsWholeText(*columns[index], field_buf, format_settings); + break; default: break; } diff --git a/tests/queries/0_stateless/01508_format_regexp_raw.reference b/tests/queries/0_stateless/01508_format_regexp_raw.reference new file mode 100644 index 00000000000..f0e7d928857 --- /dev/null +++ b/tests/queries/0_stateless/01508_format_regexp_raw.reference @@ -0,0 +1 @@ +abc\\ Hello, world! diff --git a/tests/queries/0_stateless/01508_format_regexp_raw.sh b/tests/queries/0_stateless/01508_format_regexp_raw.sh new file mode 100755 index 00000000000..699fca1be61 --- /dev/null +++ b/tests/queries/0_stateless/01508_format_regexp_raw.sh @@ -0,0 +1,17 @@ +#!/usr/bin/env bash + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +. "$CURDIR"/../shell_config.sh + +${CLICKHOUSE_CLIENT} -n --query " +DROP TABLE IF EXISTS t; +CREATE TABLE t (a String, b String) ENGINE = Memory; +" + +${CLICKHOUSE_CLIENT} --format_regexp_escaping_rule 'Raw' --format_regexp '^(.+?) separator (.+?)$' --query ' +INSERT INTO t FORMAT Regexp abc\ separator Hello, world!' + +${CLICKHOUSE_CLIENT} -n --query " +SELECT * FROM t; +DROP TABLE t; +" From 6e9b6b31ae2c4ff6b68207e2ade3526daf36d679 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 28 Sep 2020 03:22:50 +0300 Subject: [PATCH 191/321] Fix error in LowCardinality; add test --- src/DataTypes/DataTypeLowCardinality.cpp | 2 +- ..._format_regexp_raw_low_cardinality.reference | 1 + .../01510_format_regexp_raw_low_cardinality.sh | 17 +++++++++++++++++ 3 files changed, 19 insertions(+), 1 deletion(-) create mode 100644 tests/queries/0_stateless/01510_format_regexp_raw_low_cardinality.reference create mode 100755 tests/queries/0_stateless/01510_format_regexp_raw_low_cardinality.sh diff --git a/src/DataTypes/DataTypeLowCardinality.cpp b/src/DataTypes/DataTypeLowCardinality.cpp index b4f28a8853f..fe7ce7bf94a 100644 --- a/src/DataTypes/DataTypeLowCardinality.cpp +++ b/src/DataTypes/DataTypeLowCardinality.cpp @@ -774,7 +774,7 @@ void DataTypeLowCardinality::deserializeTextQuoted(IColumn & column, ReadBuffer void DataTypeLowCardinality::deserializeWholeText(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const { - deserializeImpl(column, &IDataType::deserializeAsTextEscaped, istr, settings); + deserializeImpl(column, &IDataType::deserializeAsWholeText, istr, settings); } void DataTypeLowCardinality::serializeTextCSV(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const diff --git a/tests/queries/0_stateless/01510_format_regexp_raw_low_cardinality.reference b/tests/queries/0_stateless/01510_format_regexp_raw_low_cardinality.reference new file mode 100644 index 00000000000..f0e7d928857 --- /dev/null +++ b/tests/queries/0_stateless/01510_format_regexp_raw_low_cardinality.reference @@ -0,0 +1 @@ +abc\\ Hello, world! diff --git a/tests/queries/0_stateless/01510_format_regexp_raw_low_cardinality.sh b/tests/queries/0_stateless/01510_format_regexp_raw_low_cardinality.sh new file mode 100755 index 00000000000..0f65280e1ce --- /dev/null +++ b/tests/queries/0_stateless/01510_format_regexp_raw_low_cardinality.sh @@ -0,0 +1,17 @@ +#!/usr/bin/env bash + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +. "$CURDIR"/../shell_config.sh + +${CLICKHOUSE_CLIENT} -n --query " +DROP TABLE IF EXISTS t; +CREATE TABLE t (a String, b LowCardinality(Nullable(String))) ENGINE = Memory; +" + +${CLICKHOUSE_CLIENT} --format_regexp_escaping_rule 'Raw' --format_regexp '^(.+?) separator (.+?)$' --query ' +INSERT INTO t FORMAT Regexp abc\ separator Hello, world!' + +${CLICKHOUSE_CLIENT} -n --query " +SELECT * FROM t; +DROP TABLE t; +" From 7d31d1efb899434cf7c83ca1310b11f4eda3c29d Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 29 Sep 2020 00:11:08 +0300 Subject: [PATCH 192/321] Fix install script --- 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 bd60fbb63ba..ae0c22c8fcc 100644 --- a/programs/install/Install.cpp +++ b/programs/install/Install.cpp @@ -552,7 +552,7 @@ int mainEntryClickHouseInstall(int argc, char ** argv) #if defined(__linux__) fmt::print("Setting capabilities for clickhouse binary. This is optional.\n"); - std::string command = fmt::format("command setcap && setcap 'cap_net_admin,cap_ipc_lock,cap_sys_nice+ep' {}", main_bin_path.string()); + std::string command = fmt::format("command -v setcap && setcap 'cap_net_admin,cap_ipc_lock,cap_sys_nice+ep' {}", main_bin_path.string()); fmt::print(" {}\n", command); executeScript(command); #endif From 92a1194671fad522b8f82775a039a29aa5c916a2 Mon Sep 17 00:00:00 2001 From: alesapin Date: Mon, 28 Sep 2020 14:17:07 +0300 Subject: [PATCH 193/321] Fix some flaky tests --- tests/integration/test_mysql_protocol/test.py | 4 +++- .../0_stateless/00652_replicated_mutations_zookeeper.sh | 4 +++- 2 files changed, 6 insertions(+), 2 deletions(-) diff --git a/tests/integration/test_mysql_protocol/test.py b/tests/integration/test_mysql_protocol/test.py index b5fd312007a..3e737dc2644 100644 --- a/tests/integration/test_mysql_protocol/test.py +++ b/tests/integration/test_mysql_protocol/test.py @@ -238,7 +238,9 @@ def test_mysql_federated(mysql_server, server_address): node.query('''INSERT INTO mysql_federated.test VALUES (0), (1), (5)''', settings={"password": "123"}) def check_retryable_error_in_stderr(stderr): - return "Can't connect to local MySQL server through socket" in stderr or "MySQL server has gone away" in stderr + return ("Can't connect to local MySQL server through socket" in stderr + or "MySQL server has gone away" in stderr + or "Server shutdown in progress" in stderr) code, (stdout, stderr) = mysql_server.exec_run(''' mysql diff --git a/tests/queries/0_stateless/00652_replicated_mutations_zookeeper.sh b/tests/queries/0_stateless/00652_replicated_mutations_zookeeper.sh index af2bf2dca2b..380f6ee6ff8 100755 --- a/tests/queries/0_stateless/00652_replicated_mutations_zookeeper.sh +++ b/tests/queries/0_stateless/00652_replicated_mutations_zookeeper.sh @@ -12,7 +12,7 @@ ${CLICKHOUSE_CLIENT} --query="CREATE TABLE mutations_r1(d Date, x UInt32, s Stri ${CLICKHOUSE_CLIENT} --query="CREATE TABLE mutations_r2(d Date, x UInt32, s String, m MATERIALIZED x + 2) ENGINE ReplicatedMergeTree('/clickhouse/tables/test_00652/mutations', 'r2', d, intDiv(x, 10), 8192)" # Test a mutation on empty table -${CLICKHOUSE_CLIENT} --query="ALTER TABLE mutations_r1 DELETE WHERE x = 1" +${CLICKHOUSE_CLIENT} --query="ALTER TABLE mutations_r1 DELETE WHERE x = 1 SETTINGS mutations_sync = 2" # Insert some data ${CLICKHOUSE_CLIENT} --query="INSERT INTO mutations_r1(d, x, s) VALUES \ @@ -34,6 +34,8 @@ ${CLICKHOUSE_CLIENT} --query="ALTER TABLE mutations_r1 DELETE WHERE m = 3 SETTIN ${CLICKHOUSE_CLIENT} --query="INSERT INTO mutations_r1(d, x, s) VALUES \ ('2000-01-01', 5, 'e'), ('2000-02-01', 5, 'e')" +${CLICKHOUSE_CLIENT} --query "SYSTEM SYNC REPLICA mutations_r2" + # Check that the table contains only the data that should not be deleted. ${CLICKHOUSE_CLIENT} --query="SELECT d, x, s, m FROM mutations_r2 ORDER BY d, x" # Check the contents of the system.mutations table. From 4615c6d1311ca3eb39fa7dc14686626aaf828a72 Mon Sep 17 00:00:00 2001 From: alesapin Date: Tue, 29 Sep 2020 11:13:47 +0300 Subject: [PATCH 194/321] Add release task config --- tests/ci/ci_config.json | 12 ++++++++++++ 1 file changed, 12 insertions(+) diff --git a/tests/ci/ci_config.json b/tests/ci/ci_config.json index c12c6dad999..7626ac5a904 100644 --- a/tests/ci/ci_config.json +++ b/tests/ci/ci_config.json @@ -572,6 +572,18 @@ "clang-tidy": "disable", "with_coverage": false } + }, + "Release": { + "required_build_properties": { + "compiler": "gcc-10", + "package_type": "deb", + "build_type": "relwithdebuginfo", + "sanitizer": "none", + "bundled": "bundled", + "splitted": "unsplitted", + "clang-tidy": "disable", + "with_coverage": false + } } } } From 6da04c7d8b09b6965513535b0810b7cfd64ca441 Mon Sep 17 00:00:00 2001 From: Konstantin Malanchev Date: Tue, 29 Sep 2020 10:51:52 +0200 Subject: [PATCH 195/321] Fix a typo in Russian docs --- docs/ru/sql-reference/statements/select/into-outfile.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/sql-reference/statements/select/into-outfile.md b/docs/ru/sql-reference/statements/select/into-outfile.md index 8f0126068d1..0f5cf01e9d1 100644 --- a/docs/ru/sql-reference/statements/select/into-outfile.md +++ b/docs/ru/sql-reference/statements/select/into-outfile.md @@ -1,6 +1,6 @@ # Секция INTO OUTFILE {#into-outfile-clause} -Чтобы перенаправить вывод `SELECT` запроса в указанный файл на стороне клиента, добавьте к нему секцию `INTO OUTFILE filename` (где filenam — строковый литерал). +Чтобы перенаправить вывод `SELECT` запроса в указанный файл на стороне клиента, добавьте к нему секцию `INTO OUTFILE filename` (где filename — строковый литерал). ## Детали реализации {#implementation-details} From a7d3a024c73711cf7387eca187639faf5d3f5202 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Tue, 29 Sep 2020 11:56:37 +0300 Subject: [PATCH 196/321] Revert "Revert "Test and doc for PR12771 krb5 + cyrus-sasl + kerberized kafka"" This reverts commit c298c633a793e52543f38df78ef0e8098be6f0d6. --- .gitmodules | 1 + cmake/find/rdkafka.cmake | 4 +- contrib/cyrus-sasl | 2 +- docker/images.json | 4 + docker/test/integration/base/Dockerfile | 3 +- .../test/integration/kerberos_kdc/Dockerfile | 15 ++ .../docker_compose_kerberized_kafka.yml | 59 +++++++ .../integration/runner/dockerd-entrypoint.sh | 1 + .../table-engines/integrations/kafka.md | 16 ++ tests/integration/helpers/cluster.py | 40 ++++- tests/integration/runner | 2 + .../test_storage_kerberized_kafka/__init__.py | 0 .../clickhouse_path/EMPTY_DIR | 0 .../configs/kafka.xml | 26 ++++ .../configs/log_conf.xml | 11 ++ .../kerberos_image_config.sh | 132 ++++++++++++++++ .../secrets/broker_jaas.conf | 14 ++ .../secrets/krb.conf | 22 +++ .../secrets/zookeeper_jaas.conf | 14 ++ .../test_storage_kerberized_kafka/test.py | 146 ++++++++++++++++++ 20 files changed, 505 insertions(+), 7 deletions(-) create mode 100644 docker/test/integration/kerberos_kdc/Dockerfile create mode 100644 docker/test/integration/runner/compose/docker_compose_kerberized_kafka.yml create mode 100644 tests/integration/test_storage_kerberized_kafka/__init__.py create mode 100644 tests/integration/test_storage_kerberized_kafka/clickhouse_path/EMPTY_DIR create mode 100644 tests/integration/test_storage_kerberized_kafka/configs/kafka.xml create mode 100644 tests/integration/test_storage_kerberized_kafka/configs/log_conf.xml create mode 100644 tests/integration/test_storage_kerberized_kafka/kerberos_image_config.sh create mode 100644 tests/integration/test_storage_kerberized_kafka/secrets/broker_jaas.conf create mode 100644 tests/integration/test_storage_kerberized_kafka/secrets/krb.conf create mode 100644 tests/integration/test_storage_kerberized_kafka/secrets/zookeeper_jaas.conf create mode 100644 tests/integration/test_storage_kerberized_kafka/test.py diff --git a/.gitmodules b/.gitmodules index eb21c4bfd00..865a876b276 100644 --- a/.gitmodules +++ b/.gitmodules @@ -186,3 +186,4 @@ [submodule "contrib/cyrus-sasl"] path = contrib/cyrus-sasl url = https://github.com/cyrusimap/cyrus-sasl + branch = cyrus-sasl-2.1 diff --git a/cmake/find/rdkafka.cmake b/cmake/find/rdkafka.cmake index d9f815dbcdd..ac11322f408 100644 --- a/cmake/find/rdkafka.cmake +++ b/cmake/find/rdkafka.cmake @@ -14,10 +14,10 @@ if (NOT ENABLE_RDKAFKA) return() endif() -if (NOT ARCH_ARM AND USE_LIBGSASL) +if (NOT ARCH_ARM) option (USE_INTERNAL_RDKAFKA_LIBRARY "Set to FALSE to use system librdkafka instead of the bundled" ${NOT_UNBUNDLED}) elseif(USE_INTERNAL_RDKAFKA_LIBRARY) - message (${RECONFIGURE_MESSAGE_LEVEL} "Can't use internal librdkafka with ARCH_ARM=${ARCH_ARM} AND USE_LIBGSASL=${USE_LIBGSASL}") + message (${RECONFIGURE_MESSAGE_LEVEL} "Can't use internal librdkafka with ARCH_ARM=${ARCH_ARM}") endif () if (NOT EXISTS "${ClickHouse_SOURCE_DIR}/contrib/cppkafka/CMakeLists.txt") diff --git a/contrib/cyrus-sasl b/contrib/cyrus-sasl index 6054630889f..9995bf9d8e1 160000 --- a/contrib/cyrus-sasl +++ b/contrib/cyrus-sasl @@ -1 +1 @@ -Subproject commit 6054630889fd1cd8d0659573d69badcee1e23a00 +Subproject commit 9995bf9d8e14f58934d9313ac64f13780d6dd3c9 diff --git a/docker/images.json b/docker/images.json index 8c2cb35b004..e9e91864e1e 100644 --- a/docker/images.json +++ b/docker/images.json @@ -133,6 +133,10 @@ "name": "yandex/clickhouse-postgresql-java-client", "dependent": [] }, + "docker/test/integration/kerberos_kdc": { + "name": "yandex/clickhouse-kerberos-kdc", + "dependent": [] + }, "docker/test/base": { "name": "yandex/clickhouse-test-base", "dependent": [ diff --git a/docker/test/integration/base/Dockerfile b/docker/test/integration/base/Dockerfile index 35decd907c0..3e4e88965e0 100644 --- a/docker/test/integration/base/Dockerfile +++ b/docker/test/integration/base/Dockerfile @@ -16,7 +16,8 @@ RUN apt-get update \ odbc-postgresql \ sqlite3 \ curl \ - tar + tar \ + krb5-user RUN rm -rf \ /var/lib/apt/lists/* \ /var/cache/debconf \ diff --git a/docker/test/integration/kerberos_kdc/Dockerfile b/docker/test/integration/kerberos_kdc/Dockerfile new file mode 100644 index 00000000000..ea231b1191d --- /dev/null +++ b/docker/test/integration/kerberos_kdc/Dockerfile @@ -0,0 +1,15 @@ +# docker build -t yandex/clickhouse-kerberos-kdc . + +FROM centos:6.6 +# old OS to make is faster and smaller + +RUN yum install -y krb5-server krb5-libs krb5-auth-dialog krb5-workstation + +EXPOSE 88 749 + +RUN touch /config.sh +# should be overwritten e.g. via docker_compose volumes +# volumes: /some_path/my_kerberos_config.sh:/config.sh:ro + + +ENTRYPOINT ["/bin/bash", "/config.sh"] diff --git a/docker/test/integration/runner/compose/docker_compose_kerberized_kafka.yml b/docker/test/integration/runner/compose/docker_compose_kerberized_kafka.yml new file mode 100644 index 00000000000..3ce0000b148 --- /dev/null +++ b/docker/test/integration/runner/compose/docker_compose_kerberized_kafka.yml @@ -0,0 +1,59 @@ +version: '2.3' + +services: + kafka_kerberized_zookeeper: + image: confluentinc/cp-zookeeper:5.2.0 + # restart: always + hostname: kafka_kerberized_zookeeper + environment: + ZOOKEEPER_SERVER_ID: 1 + ZOOKEEPER_CLIENT_PORT: 2181 + ZOOKEEPER_SERVERS: "kafka_kerberized_zookeeper:2888:3888" + KAFKA_OPTS: "-Djava.security.auth.login.config=/etc/kafka/secrets/zookeeper_jaas.conf -Djava.security.krb5.conf=/etc/kafka/secrets/krb.conf -Dzookeeper.authProvider.1=org.apache.zookeeper.server.auth.SASLAuthenticationProvider -Dsun.security.krb5.debug=true" + volumes: + - ${KERBERIZED_KAFKA_DIR}/secrets:/etc/kafka/secrets + - /dev/urandom:/dev/random + depends_on: + - kafka_kerberos + security_opt: + - label:disable + + kerberized_kafka1: + image: confluentinc/cp-kafka:5.2.0 + # restart: always + hostname: kerberized_kafka1 + ports: + - "9092:9092" + - "9093:9093" + environment: + KAFKA_LISTENERS: OUTSIDE://:19092,UNSECURED_OUTSIDE://:19093,UNSECURED_INSIDE://:9093 + KAFKA_ADVERTISED_LISTENERS: OUTSIDE://kerberized_kafka1:19092,UNSECURED_OUTSIDE://kerberized_kafka1:19093,UNSECURED_INSIDE://localhost:9093 + # KAFKA_LISTENERS: INSIDE://kerberized_kafka1:9092,OUTSIDE://kerberized_kafka1:19092 + # KAFKA_ADVERTISED_LISTENERS: INSIDE://localhost:9092,OUTSIDE://kerberized_kafka1:19092 + KAFKA_SASL_MECHANISM_INTER_BROKER_PROTOCOL: GSSAPI + KAFKA_SASL_ENABLED_MECHANISMS: GSSAPI + KAFKA_SASL_KERBEROS_SERVICE_NAME: kafka + KAFKA_LISTENER_SECURITY_PROTOCOL_MAP: OUTSIDE:SASL_PLAINTEXT,UNSECURED_OUTSIDE:PLAINTEXT,UNSECURED_INSIDE:PLAINTEXT, + KAFKA_INTER_BROKER_LISTENER_NAME: OUTSIDE + KAFKA_BROKER_ID: 1 + KAFKA_ZOOKEEPER_CONNECT: "kafka_kerberized_zookeeper:2181" + KAFKA_LOG4J_LOGGERS: "kafka.controller=INFO,kafka.producer.async.DefaultEventHandler=INFO,state.change.logger=INFO" + KAFKA_OFFSETS_TOPIC_REPLICATION_FACTOR: 1 + KAFKA_OPTS: "-Djava.security.auth.login.config=/etc/kafka/secrets/broker_jaas.conf -Djava.security.krb5.conf=/etc/kafka/secrets/krb.conf -Dsun.security.krb5.debug=true" + volumes: + - ${KERBERIZED_KAFKA_DIR}/secrets:/etc/kafka/secrets + - /dev/urandom:/dev/random + depends_on: + - kafka_kerberized_zookeeper + - kafka_kerberos + security_opt: + - label:disable + + kafka_kerberos: + image: yandex/clickhouse-kerberos-kdc:${DOCKER_KERBEROS_KDC_TAG} + hostname: kafka_kerberos + volumes: + - ${KERBERIZED_KAFKA_DIR}/secrets:/tmp/keytab + - ${KERBERIZED_KAFKA_DIR}/../../kerberos_image_config.sh:/config.sh + - /dev/urandom:/dev/random + ports: [88, 749] diff --git a/docker/test/integration/runner/dockerd-entrypoint.sh b/docker/test/integration/runner/dockerd-entrypoint.sh index c38260279ed..cbdb7317b1e 100755 --- a/docker/test/integration/runner/dockerd-entrypoint.sh +++ b/docker/test/integration/runner/dockerd-entrypoint.sh @@ -27,6 +27,7 @@ export DOCKER_MYSQL_JAVA_CLIENT_TAG=${DOCKER_MYSQL_JAVA_CLIENT_TAG:=latest} export DOCKER_MYSQL_JS_CLIENT_TAG=${DOCKER_MYSQL_JS_CLIENT_TAG:=latest} export DOCKER_MYSQL_PHP_CLIENT_TAG=${DOCKER_MYSQL_PHP_CLIENT_TAG:=latest} export DOCKER_POSTGRESQL_JAVA_CLIENT_TAG=${DOCKER_POSTGRESQL_JAVA_CLIENT_TAG:=latest} +export DOCKER_KERBEROS_KDC_TAG=${DOCKER_KERBEROS_KDC_TAG:=latest} cd /ClickHouse/tests/integration exec "$@" diff --git a/docs/en/engines/table-engines/integrations/kafka.md b/docs/en/engines/table-engines/integrations/kafka.md index fe9aa2ca25e..d0a4bc928a7 100644 --- a/docs/en/engines/table-engines/integrations/kafka.md +++ b/docs/en/engines/table-engines/integrations/kafka.md @@ -165,6 +165,22 @@ Similar to GraphiteMergeTree, the Kafka engine supports extended configuration u For a list of possible configuration options, see the [librdkafka configuration reference](https://github.com/edenhill/librdkafka/blob/master/CONFIGURATION.md). Use the underscore (`_`) instead of a dot in the ClickHouse configuration. For example, `check.crcs=true` will be `true`. +### Kerberos support {#kafka-kerberos-support} + +To deal with Kerberos-aware Kafka, add `security_protocol` child element with `sasl_plaintext` value. It is enough if Kerberos ticket-granting ticket is obtained and cached by OS facilities. +ClickHouse is able to maintain Kerberos credentials using a keytab file. Consider `sasl_kerberos_service_name`, `sasl_kerberos_keytab`, `sasl_kerberos_principal` and `sasl.kerberos.kinit.cmd` child elements. + +Example: + +``` xml + + + SASL_PLAINTEXT + /home/kafkauser/kafkauser.keytab + kafkauser/kafkahost@EXAMPLE.COM + +``` + ## Virtual Columns {#virtual-columns} - `_topic` — Kafka topic. diff --git a/tests/integration/helpers/cluster.py b/tests/integration/helpers/cluster.py index 0b7fa9264bd..d99d8a17844 100644 --- a/tests/integration/helpers/cluster.py +++ b/tests/integration/helpers/cluster.py @@ -45,7 +45,6 @@ def _create_env_file(path, variables, fname=DEFAULT_ENV_NAME): f.write("=".join([var, value]) + "\n") return full_path - def subprocess_check_call(args): # Uncomment for debugging # print('run:', ' ' . join(args)) @@ -125,6 +124,7 @@ class ClickHouseCluster: self.base_zookeeper_cmd = None self.base_mysql_cmd = [] self.base_kafka_cmd = [] + self.base_kerberized_kafka_cmd = [] self.base_rabbitmq_cmd = [] self.base_cassandra_cmd = [] self.pre_zookeeper_commands = [] @@ -133,6 +133,7 @@ class ClickHouseCluster: self.with_mysql = False self.with_postgres = False self.with_kafka = False + self.with_kerberized_kafka = False self.with_rabbitmq = False self.with_odbc_drivers = False self.with_hdfs = False @@ -169,7 +170,7 @@ class ClickHouseCluster: def add_instance(self, name, base_config_dir=None, main_configs=None, user_configs=None, dictionaries=None, macros=None, - with_zookeeper=False, with_mysql=False, with_kafka=False, with_rabbitmq=False, + with_zookeeper=False, with_mysql=False, with_kafka=False, with_kerberized_kafka=False, with_rabbitmq=False, clickhouse_path_dir=None, with_odbc_drivers=False, with_postgres=False, with_hdfs=False, with_mongo=False, with_redis=False, with_minio=False, with_cassandra=False, @@ -207,6 +208,7 @@ class ClickHouseCluster: zookeeper_config_path=self.zookeeper_config_path, with_mysql=with_mysql, with_kafka=with_kafka, + with_kerberized_kafka=with_kerberized_kafka, with_rabbitmq=with_rabbitmq, with_mongo=with_mongo, with_redis=with_redis, @@ -290,6 +292,13 @@ class ClickHouseCluster: p.join(docker_compose_yml_dir, 'docker_compose_kafka.yml')] cmds.append(self.base_kafka_cmd) + if with_kerberized_kafka and not self.with_kerberized_kafka: + self.with_kerberized_kafka = True + self.base_cmd.extend(['--file', p.join(docker_compose_yml_dir, 'docker_compose_kerberized_kafka.yml')]) + self.base_kerberized_kafka_cmd = ['docker-compose', '--project-directory', self.base_dir, '--project-name', + self.project_name, '--file', p.join(docker_compose_yml_dir, 'docker_compose_kerberized_kafka.yml')] + cmds.append(self.base_kerberized_kafka_cmd) + if with_rabbitmq and not self.with_rabbitmq: self.with_rabbitmq = True self.base_cmd.extend(['--file', p.join(docker_compose_yml_dir, 'docker_compose_rabbitmq.yml')]) @@ -608,6 +617,11 @@ class ClickHouseCluster: self.kafka_docker_id = self.get_instance_docker_id('kafka1') self.wait_schema_registry_to_start(120) + if self.with_kerberized_kafka and self.base_kerberized_kafka_cmd: + env = os.environ.copy() + env['KERBERIZED_KAFKA_DIR'] = instance.path + '/' + subprocess.check_call(self.base_kerberized_kafka_cmd + common_opts + ['--renew-anon-volumes'], env=env) + self.kerberized_kafka_docker_id = self.get_instance_docker_id('kerberized_kafka1') if self.with_rabbitmq and self.base_rabbitmq_cmd: subprocess_check_call(self.base_rabbitmq_cmd + common_opts + ['--renew-anon-volumes']) self.rabbitmq_docker_id = self.get_instance_docker_id('rabbitmq1') @@ -788,9 +802,12 @@ services: - {instance_config_dir}:/etc/clickhouse-server/ - {db_dir}:/var/lib/clickhouse/ - {logs_dir}:/var/log/clickhouse-server/ + - /etc/passwd:/etc/passwd:ro {binary_volume} {odbc_bridge_volume} {odbc_ini_path} + {keytab_path} + {krb5_conf} entrypoint: {entrypoint_cmd} tmpfs: {tmpfs} cap_add: @@ -820,7 +837,7 @@ class ClickHouseInstance: def __init__( self, cluster, base_path, name, base_config_dir, custom_main_configs, custom_user_configs, custom_dictionaries, - macros, with_zookeeper, zookeeper_config_path, with_mysql, with_kafka, with_rabbitmq, with_mongo, + macros, with_zookeeper, zookeeper_config_path, with_mysql, with_kafka, with_kerberized_kafka, with_rabbitmq, with_mongo, with_redis, with_minio, with_cassandra, server_bin_path, odbc_bridge_bin_path, clickhouse_path_dir, with_odbc_drivers, hostname=None, env_variables=None, @@ -839,6 +856,7 @@ class ClickHouseInstance: self.custom_user_config_paths = [p.abspath(p.join(base_path, c)) for c in custom_user_configs] self.custom_dictionaries_paths = [p.abspath(p.join(base_path, c)) for c in custom_dictionaries] self.clickhouse_path_dir = p.abspath(p.join(base_path, clickhouse_path_dir)) if clickhouse_path_dir else None + self.kerberos_secrets_dir = p.abspath(p.join(base_path, 'secrets')) self.macros = macros if macros is not None else {} self.with_zookeeper = with_zookeeper self.zookeeper_config_path = zookeeper_config_path @@ -848,6 +866,7 @@ class ClickHouseInstance: self.with_mysql = with_mysql self.with_kafka = with_kafka + self.with_kerberized_kafka = with_kerberized_kafka self.with_rabbitmq = with_rabbitmq self.with_mongo = with_mongo self.with_redis = with_redis @@ -863,6 +882,13 @@ class ClickHouseInstance: else: self.odbc_ini_path = "" + if with_kerberized_kafka: + self.keytab_path = '- ' + os.path.dirname(self.docker_compose_path) + "/secrets:/tmp/keytab" + self.krb5_conf = '- ' + os.path.dirname(self.docker_compose_path) + "/secrets/krb.conf:/etc/krb5.conf:ro" + else: + self.keytab_path = "" + self.krb5_conf = "" + self.docker_client = None self.ip_address = None self.client = None @@ -1192,6 +1218,9 @@ class ClickHouseInstance: if self.with_zookeeper: shutil.copy(self.zookeeper_config_path, conf_d_dir) + if self.with_kerberized_kafka: + shutil.copytree(self.kerberos_secrets_dir, p.abspath(p.join(self.path, 'secrets'))) + # Copy config.d configs print "Copy custom test config files {} to {}".format(self.custom_main_config_paths, self.config_d_dir) for path in self.custom_main_config_paths: @@ -1227,6 +1256,9 @@ class ClickHouseInstance: depends_on.append("kafka1") depends_on.append("schema-registry") + if self.with_kerberized_kafka: + depends_on.append("kerberized_kafka1") + if self.with_rabbitmq: depends_on.append("rabbitmq1") @@ -1290,6 +1322,8 @@ class ClickHouseInstance: user=os.getuid(), env_file=env_file, odbc_ini_path=odbc_ini_path, + keytab_path=self.keytab_path, + krb5_conf=self.krb5_conf, entrypoint_cmd=entrypoint_cmd, networks=networks, app_net=app_net, diff --git a/tests/integration/runner b/tests/integration/runner index f097a42e52a..f38ab0aa042 100755 --- a/tests/integration/runner +++ b/tests/integration/runner @@ -156,6 +156,8 @@ if __name__ == "__main__": env_tags += "-e {}={} ".format("DOCKER_POSTGRESQL_JAVA_CLIENT_TAG", tag) elif image == "yandex/clickhouse-integration-test": env_tags += "-e {}={}".format("DOCKER_BASE_TAG", tag) + elif image == "yandex/clickhouse-kerberos-kdc": + env_tags += "-e {}={}".format("DOCKER_KERBEROS_KDC_TAG", tag) else: logging.info("Unknown image {}".format(image)) diff --git a/tests/integration/test_storage_kerberized_kafka/__init__.py b/tests/integration/test_storage_kerberized_kafka/__init__.py new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/integration/test_storage_kerberized_kafka/clickhouse_path/EMPTY_DIR b/tests/integration/test_storage_kerberized_kafka/clickhouse_path/EMPTY_DIR new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/integration/test_storage_kerberized_kafka/configs/kafka.xml b/tests/integration/test_storage_kerberized_kafka/configs/kafka.xml new file mode 100644 index 00000000000..0302bd78e3f --- /dev/null +++ b/tests/integration/test_storage_kerberized_kafka/configs/kafka.xml @@ -0,0 +1,26 @@ + + + earliest + + SASL_PLAINTEXT + GSSAPI + kafka + /tmp/keytab/clickhouse.keytab + kafkauser/instance@TEST.CLICKHOUSE.TECH + security + false + + + + + 300 + + 6000 + + diff --git a/tests/integration/test_storage_kerberized_kafka/configs/log_conf.xml b/tests/integration/test_storage_kerberized_kafka/configs/log_conf.xml new file mode 100644 index 00000000000..95466269afe --- /dev/null +++ b/tests/integration/test_storage_kerberized_kafka/configs/log_conf.xml @@ -0,0 +1,11 @@ + + + trace + /var/log/clickhouse-server/log.log + /var/log/clickhouse-server/log.err.log + 1000M + 10 + /var/log/clickhouse-server/stderr.log + /var/log/clickhouse-server/stdout.log + + \ No newline at end of file diff --git a/tests/integration/test_storage_kerberized_kafka/kerberos_image_config.sh b/tests/integration/test_storage_kerberized_kafka/kerberos_image_config.sh new file mode 100644 index 00000000000..dda10d47d94 --- /dev/null +++ b/tests/integration/test_storage_kerberized_kafka/kerberos_image_config.sh @@ -0,0 +1,132 @@ +#!/bin/bash + + +set -x # trace + +: "${REALM:=TEST.CLICKHOUSE.TECH}" +: "${DOMAIN_REALM:=test.clickhouse.tech}" +: "${KERB_MASTER_KEY:=masterkey}" +: "${KERB_ADMIN_USER:=admin}" +: "${KERB_ADMIN_PASS:=admin}" + +create_config() { + : "${KDC_ADDRESS:=$(hostname -f)}" + + cat>/etc/krb5.conf</var/kerberos/krb5kdc/kdc.conf< /var/kerberos/krb5kdc/kadm5.acl +} + +create_keytabs() { + + kadmin.local -q "addprinc -randkey zookeeper/kafka_kerberized_zookeeper@${REALM}" + kadmin.local -q "ktadd -norandkey -k /tmp/keytab/kafka_kerberized_zookeeper.keytab zookeeper/kafka_kerberized_zookeeper@${REALM}" + + kadmin.local -q "addprinc -randkey kafka/kerberized_kafka1@${REALM}" + kadmin.local -q "ktadd -norandkey -k /tmp/keytab/kerberized_kafka.keytab kafka/kerberized_kafka1@${REALM}" + + kadmin.local -q "addprinc -randkey zkclient@${REALM}" + kadmin.local -q "ktadd -norandkey -k /tmp/keytab/zkclient.keytab zkclient@${REALM}" + + + kadmin.local -q "addprinc -randkey kafkauser/instance@${REALM}" + kadmin.local -q "ktadd -norandkey -k /tmp/keytab/clickhouse.keytab kafkauser/instance@${REALM}" + + chmod g+r /tmp/keytab/clickhouse.keytab + +} + +main() { + + if [ ! -f /kerberos_initialized ]; then + create_config + create_db + create_admin_user + start_kdc + + touch /kerberos_initialized + fi + + if [ ! -f /var/kerberos/krb5kdc/principal ]; then + while true; do sleep 1000; done + else + start_kdc + create_keytabs + tail -F /var/log/kerberos/krb5kdc.log + fi + +} + +[[ "$0" == "${BASH_SOURCE[0]}" ]] && main "$@" diff --git a/tests/integration/test_storage_kerberized_kafka/secrets/broker_jaas.conf b/tests/integration/test_storage_kerberized_kafka/secrets/broker_jaas.conf new file mode 100644 index 00000000000..8a55ec2faa0 --- /dev/null +++ b/tests/integration/test_storage_kerberized_kafka/secrets/broker_jaas.conf @@ -0,0 +1,14 @@ +KafkaServer { + com.sun.security.auth.module.Krb5LoginModule required + useKeyTab=true + storeKey=true + keyTab="/etc/kafka/secrets/kerberized_kafka.keytab" + principal="kafka/kerberized_kafka1@TEST.CLICKHOUSE.TECH"; +}; +Client { + com.sun.security.auth.module.Krb5LoginModule required + useKeyTab=true + storeKey=true + keyTab="/etc/kafka/secrets/zkclient.keytab" + principal="zkclient@TEST.CLICKHOUSE.TECH"; +}; diff --git a/tests/integration/test_storage_kerberized_kafka/secrets/krb.conf b/tests/integration/test_storage_kerberized_kafka/secrets/krb.conf new file mode 100644 index 00000000000..1efdf510f22 --- /dev/null +++ b/tests/integration/test_storage_kerberized_kafka/secrets/krb.conf @@ -0,0 +1,22 @@ +[logging] + default = FILE:/var/log/kerberos/krb5libs.log + kdc = FILE:/var/log/kerberos/krb5kdc.log + admin_server = FILE:/var/log/kerberos/kadmind.log + +[libdefaults] + default_realm = TEST.CLICKHOUSE.TECH + dns_lookup_realm = false + dns_lookup_kdc = false + ticket_lifetime = 15s + renew_lifetime = 15s + forwardable = true + +[realms] + TEST.CLICKHOUSE.TECH = { + kdc = kafka_kerberos + admin_server = kafka_kerberos + } + +[domain_realm] + .TEST.CLICKHOUSE.TECH = TEST.CLICKHOUSE.TECH + TEST.CLICKHOUSE.TECH = TEST.CLICKHOUSE.TECH diff --git a/tests/integration/test_storage_kerberized_kafka/secrets/zookeeper_jaas.conf b/tests/integration/test_storage_kerberized_kafka/secrets/zookeeper_jaas.conf new file mode 100644 index 00000000000..1b1f8103f42 --- /dev/null +++ b/tests/integration/test_storage_kerberized_kafka/secrets/zookeeper_jaas.conf @@ -0,0 +1,14 @@ +Server { + com.sun.security.auth.module.Krb5LoginModule required + useKeyTab=true + storeKey=true + keyTab="/etc/kafka/secrets/kafka_kerberized_zookeeper.keytab" + principal="zookeeper/kafka_kerberized_zookeeper@TEST.CLICKHOUSE.TECH"; +}; +Client { + com.sun.security.auth.module.Krb5LoginModule required + useKeyTab=true + storeKey=true + keyTab="/etc/kafka/secrets/zkclient.keytab" + principal="zkclient@TEST.CLICKHOUSE.TECH"; +}; diff --git a/tests/integration/test_storage_kerberized_kafka/test.py b/tests/integration/test_storage_kerberized_kafka/test.py new file mode 100644 index 00000000000..ec23d340977 --- /dev/null +++ b/tests/integration/test_storage_kerberized_kafka/test.py @@ -0,0 +1,146 @@ +import os.path as p +import random +import threading +import time +import pytest + +from helpers.cluster import ClickHouseCluster +from helpers.test_tools import TSV +from helpers.client import QueryRuntimeException +from helpers.network import PartitionManager + +import json +import subprocess +import kafka.errors +from kafka import KafkaAdminClient, KafkaProducer, KafkaConsumer, BrokerConnection +from kafka.admin import NewTopic +from kafka.protocol.admin import DescribeGroupsResponse_v1, DescribeGroupsRequest_v1 +from kafka.protocol.group import MemberAssignment +import socket + +cluster = ClickHouseCluster(__file__) +instance = cluster.add_instance('instance', + main_configs=['configs/kafka.xml', 'configs/log_conf.xml' ], + with_kerberized_kafka=True, + clickhouse_path_dir="clickhouse_path" + ) +kafka_id = '' # instance.cluster.kafka_docker_id + +# Helpers + +def check_kafka_is_available(): + + # plaintext + p = subprocess.Popen(('docker', + 'exec', + '-i', + kafka_id, + '/usr/bin/kafka-broker-api-versions', + '--bootstrap-server', + 'localhost:9093'), + stdout=subprocess.PIPE) + p.communicate() + return p.returncode == 0 + + +def wait_kafka_is_available(max_retries=50): + retries = 0 + while True: + if check_kafka_is_available(): + break + else: + retries += 1 + if retries > max_retries: + raise "Kafka is not available" + print("Waiting for Kafka to start up") + time.sleep(1) + + +def kafka_produce(topic, messages, timestamp=None): + producer = KafkaProducer(bootstrap_servers="localhost:9093") + for message in messages: + producer.send(topic=topic, value=message, timestamp_ms=timestamp) + producer.flush() + print ("Produced {} messages for topic {}".format(len(messages), topic)) + + + +# Fixtures + +@pytest.fixture(scope="module") +def kafka_cluster(): + try: + global kafka_id + cluster.start() + kafka_id = instance.cluster.kerberized_kafka_docker_id + print("kafka_id is {}".format(kafka_id)) + yield cluster + + finally: + cluster.shutdown() + + +@pytest.fixture(autouse=True) +def kafka_setup_teardown(): + instance.query('DROP DATABASE IF EXISTS test; CREATE DATABASE test;') + wait_kafka_is_available() + print("kafka is available - running test") + yield # run test + +# Tests + +@pytest.mark.timeout(180) # wait to build containers +def test_kafka_json_as_string(kafka_cluster): + kafka_produce('kafka_json_as_string', ['{"t": 123, "e": {"x": "woof"} }', '', '{"t": 124, "e": {"x": "test"} }', '{"F1":"V1","F2":{"F21":"V21","F22":{},"F23":"V23","F24":"2019-12-24T16:28:04"},"F3":"V3"}']) + + instance.query(''' + CREATE TABLE test.kafka (field String) + ENGINE = Kafka + SETTINGS kafka_broker_list = 'kerberized_kafka1:19092', + kafka_topic_list = 'kafka_json_as_string', + kafka_group_name = 'kafka_json_as_string', + kafka_format = 'JSONAsString', + kafka_flush_interval_ms=1000; + ''') + + result = instance.query('SELECT * FROM test.kafka;') + expected = '''\ +{"t": 123, "e": {"x": "woof"} } +{"t": 124, "e": {"x": "test"} } +{"F1":"V1","F2":{"F21":"V21","F22":{},"F23":"V23","F24":"2019-12-24T16:28:04"},"F3":"V3"} +''' + assert TSV(result) == TSV(expected) + assert instance.contains_in_log("Parsing of message (topic: kafka_json_as_string, partition: 0, offset: 1) return no rows") + +def test_kafka_json_as_string_no_kdc(kafka_cluster): + kafka_produce('kafka_json_as_string_no_kdc', ['{"t": 123, "e": {"x": "woof"} }', '', '{"t": 124, "e": {"x": "test"} }', '{"F1":"V1","F2":{"F21":"V21","F22":{},"F23":"V23","F24":"2019-12-24T16:28:04"},"F3":"V3"}']) + + kafka_cluster.pause_container('kafka_kerberos') + time.sleep(45) # wait for ticket expiration + + instance.query(''' + CREATE TABLE test.kafka_no_kdc (field String) + ENGINE = Kafka + SETTINGS kafka_broker_list = 'kerberized_kafka1:19092', + kafka_topic_list = 'kafka_json_as_string_no_kdc', + kafka_group_name = 'kafka_json_as_string_no_kdc', + kafka_format = 'JSONAsString', + kafka_flush_interval_ms=1000; + ''') + + result = instance.query('SELECT * FROM test.kafka_no_kdc;') + expected = '' + + kafka_cluster.unpause_container('kafka_kerberos') + + + assert TSV(result) == TSV(expected) + assert instance.contains_in_log("StorageKafka (kafka_no_kdc): Nothing to commit") + assert instance.contains_in_log("Ticket expired") + assert instance.contains_in_log("Kerberos ticket refresh failed") + + +if __name__ == '__main__': + cluster.start() + raw_input("Cluster created, press any key to destroy...") + cluster.shutdown() From 6ebfd1aaf6a51d2ad9c97fd02d494f05e4512128 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Tue, 29 Sep 2020 11:58:04 +0300 Subject: [PATCH 197/321] fix --- tests/integration/runner | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/integration/runner b/tests/integration/runner index f38ab0aa042..78d93af2929 100755 --- a/tests/integration/runner +++ b/tests/integration/runner @@ -155,7 +155,7 @@ if __name__ == "__main__": elif image == "yandex/clickhouse-postgresql-java-client": env_tags += "-e {}={} ".format("DOCKER_POSTGRESQL_JAVA_CLIENT_TAG", tag) elif image == "yandex/clickhouse-integration-test": - env_tags += "-e {}={}".format("DOCKER_BASE_TAG", tag) + env_tags += "-e {}={} ".format("DOCKER_BASE_TAG", tag) elif image == "yandex/clickhouse-kerberos-kdc": env_tags += "-e {}={}".format("DOCKER_KERBEROS_KDC_TAG", tag) else: From 7bc12ca0c313150d52ebb3284d2419dd4e4f7251 Mon Sep 17 00:00:00 2001 From: alesapin Date: Tue, 29 Sep 2020 11:59:40 +0300 Subject: [PATCH 198/321] Fix bugs --- docs/tools/cmake_in_clickhouse_generator.py | 12 ++++++++---- 1 file changed, 8 insertions(+), 4 deletions(-) diff --git a/docs/tools/cmake_in_clickhouse_generator.py b/docs/tools/cmake_in_clickhouse_generator.py index 3437c8ad956..1414ffc4b9e 100644 --- a/docs/tools/cmake_in_clickhouse_generator.py +++ b/docs/tools/cmake_in_clickhouse_generator.py @@ -55,7 +55,7 @@ def build_entity(path: str, entity: Entity, line_comment: Tuple[int, str]) -> No anchor=make_anchor(name), name=name, path=path, - line=line if line > 0 else 1) + line=line) formatted_description: str = "".join(description.split("\n")) @@ -75,7 +75,7 @@ def process_file(root_path: str, file_path: str, file_name: str) -> None: comment: str = "" for n, line in enumerate(contents_list): - if line.find(target) == -1: + if 'option' not in line.lower() or target not in line: continue for maybe_comment_line in contents_list[n - 1::-1]: @@ -84,11 +84,15 @@ def process_file(root_path: str, file_path: str, file_name: str) -> None: comment = re.sub("\s*#\s*", "", maybe_comment_line) + " " + comment - return n, comment + # line numbering starts with 1 + return n + 1, comment matches: Optional[List[Entity]] = re.findall(cmake_option_regex, contents, re.MULTILINE) - file_rel_path_with_name: str = os.path.join(file_path[len(root_path):], file_name)[1:] + + file_rel_path_with_name: str = os.path.join(file_path[len(root_path):], file_name) + if file_rel_path_with_name.startswith('/'): + file_rel_path_with_name = file_rel_path_with_name[1:] if matches: for entity in matches: From a7e7fd084f9ddb4a300c46c18f720bbfa975d58f Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 29 Sep 2020 13:48:11 +0300 Subject: [PATCH 199/321] Destroy resurces captured by lambda after ThreadFromGlobalPool::join(). --- src/Common/ThreadPool.h | 7 +++++- src/Processors/Executors/PipelineExecutor.cpp | 22 +++++++++---------- 2 files changed, 17 insertions(+), 12 deletions(-) diff --git a/src/Common/ThreadPool.h b/src/Common/ThreadPool.h index 55796905b73..ecb03dd9659 100644 --- a/src/Common/ThreadPool.h +++ b/src/Common/ThreadPool.h @@ -165,10 +165,15 @@ public: { try { + /// Move capture in order to destroy it before `state->set()`. + /// It will guarantee that after ThreadFromGlobalPool::join all resources are destroyed. + auto function = std::move(func); + auto arguments = std::move(args); + /// Thread status holds raw pointer on query context, thus it always must be destroyed /// before sending signal that permits to join this thread. DB::ThreadStatus thread_status; - std::apply(func, args); + std::apply(function, arguments); } catch (...) { diff --git a/src/Processors/Executors/PipelineExecutor.cpp b/src/Processors/Executors/PipelineExecutor.cpp index 9a9f5d53354..858f53cb047 100644 --- a/src/Processors/Executors/PipelineExecutor.cpp +++ b/src/Processors/Executors/PipelineExecutor.cpp @@ -694,7 +694,9 @@ void PipelineExecutor::executeImpl(size_t num_threads) { initializeExecution(num_threads); - ThreadPool threads(num_threads); + using ThreadsData = std::vector; + ThreadsData threads; + threads.reserve(num_threads); bool finished_flag = false; @@ -702,7 +704,10 @@ void PipelineExecutor::executeImpl(size_t num_threads) if (!finished_flag) { finish(); - threads.wait(); + + for (auto & thread : threads) + if (thread.joinable()) + thread.join(); } ); @@ -712,7 +717,7 @@ void PipelineExecutor::executeImpl(size_t num_threads) for (size_t i = 0; i < num_threads; ++i) { - threads.scheduleOrThrowOnError([this, thread_group, thread_num = i, num_threads] + threads.emplace_back([this, thread_group, thread_num = i, num_threads] { /// ThreadStatus thread_status; @@ -739,14 +744,9 @@ void PipelineExecutor::executeImpl(size_t num_threads) }); } - /// Because ThreadPool::wait() waits until scheduled_jobs will be zero, - /// and this guarantee that the job was reseted, otherwise - /// some variables that job is referencing may be already destroyed, - /// while job hasn't been destoyed yet (example that pops up -- - /// PipelineExecutor -> ThreadGroupStatusPtr -> MemoryTracker -> - /// ~MemoryTracker -> log, while log had been destroyed already) - /// (see 01505_pipeline_executor_UAF) - threads.wait(); + for (auto & thread : threads) + if (thread.joinable()) + thread.join(); } else executeSingleThread(0, num_threads); From ac4f0cdf229f08bef7c791af83044e1c96e3e8c6 Mon Sep 17 00:00:00 2001 From: Alexander Kuzmenkov Date: Tue, 29 Sep 2020 14:07:31 +0300 Subject: [PATCH 200/321] fix --- docker/test/performance-comparison/compare.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docker/test/performance-comparison/compare.sh b/docker/test/performance-comparison/compare.sh index 7b81b3c562a..74f1e81e8f3 100755 --- a/docker/test/performance-comparison/compare.sh +++ b/docker/test/performance-comparison/compare.sh @@ -199,7 +199,7 @@ function run_tests # Don't profile if we're past the time limit. # Use awk because bash doesn't support floating point arithmetics. - profile_seconds=$(awk "BEGIN { print $profile_seconds_left > 0 ? 10 : 0 }") + profile_seconds=$(awk "BEGIN { print ($profile_seconds_left > 0 ? 10 : 0) }") TIMEFORMAT=$(printf "$test_name\t%%3R\t%%3U\t%%3S\n") # The grep is to filter out set -x output and keep only time output. From 65a05a970d79529c6fab3a2fb2db3966b1c67e3d Mon Sep 17 00:00:00 2001 From: alesapin Date: Tue, 29 Sep 2020 15:07:01 +0300 Subject: [PATCH 201/321] Better message on error --- src/Storages/MergeTree/MergeTreeData.cpp | 10 +++ ..._versioned_collapsing_merge_tree.reference | 13 ++++ ...ersion_versioned_collapsing_merge_tree.sql | 41 ++++++++++++ ..._collapsing_merge_tree_zookeeper.reference | 16 +++++ ...sioned_collapsing_merge_tree_zookeeper.sql | 64 +++++++++++++++++++ 5 files changed, 144 insertions(+) create mode 100644 tests/queries/0_stateless/01511_alter_version_versioned_collapsing_merge_tree.reference create mode 100644 tests/queries/0_stateless/01511_alter_version_versioned_collapsing_merge_tree.sql create mode 100644 tests/queries/0_stateless/01511_alter_version_versioned_collapsing_merge_tree_zookeeper.reference create mode 100644 tests/queries/0_stateless/01511_alter_version_versioned_collapsing_merge_tree_zookeeper.sql diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 97f481eee8a..83fc75d662e 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -1458,6 +1458,16 @@ void MergeTreeData::checkAlterIsPossible(const AlterCommands & commands, const S } else if (command.isRequireMutationStage(getInMemoryMetadata())) { + /// Type change for version column is allowed despite it's a part of sorting key + if (command.type == AlterCommand::MODIFY_COLUMN && command.column_name == merging_params.version_column) + { + if (!command.data_type->canBeUsedAsVersion()) + throw Exception("Cannot alter version column " + backQuoteIfNeed(command.column_name) + + " to type " + command.data_type->getName() + + " because version column must be of an integer type or of type Date or DateTime" + , ErrorCodes::ALTER_OF_COLUMN_IS_FORBIDDEN); + continue; + } /// This alter will override data on disk. Let's check that it doesn't /// modify immutable column. if (columns_alter_type_forbidden.count(command.column_name)) diff --git a/tests/queries/0_stateless/01511_alter_version_versioned_collapsing_merge_tree.reference b/tests/queries/0_stateless/01511_alter_version_versioned_collapsing_merge_tree.reference new file mode 100644 index 00000000000..3ff3fea93e5 --- /dev/null +++ b/tests/queries/0_stateless/01511_alter_version_versioned_collapsing_merge_tree.reference @@ -0,0 +1,13 @@ +1 1 1 -1 +2 2 2 -1 +CREATE TABLE default.table_with_version\n(\n `key` UInt64,\n `value` String,\n `version` UInt8,\n `sign` Int8\n)\nENGINE = VersionedCollapsingMergeTree(sign, version)\nORDER BY key\nSETTINGS index_granularity = 8192 +1 1 1 -1 +2 2 2 -1 +CREATE TABLE default.table_with_version\n(\n `key` UInt64,\n `value` String,\n `version` UInt32,\n `sign` Int8\n)\nENGINE = VersionedCollapsingMergeTree(sign, version)\nORDER BY key\nSETTINGS index_granularity = 8192 +1 1 2 1 +2 2 2 -1 +1 1 2 1 +2 2 2 -1 +3 3 65555 1 +1 1 2 1 +2 2 2 -1 diff --git a/tests/queries/0_stateless/01511_alter_version_versioned_collapsing_merge_tree.sql b/tests/queries/0_stateless/01511_alter_version_versioned_collapsing_merge_tree.sql new file mode 100644 index 00000000000..62cb4e77640 --- /dev/null +++ b/tests/queries/0_stateless/01511_alter_version_versioned_collapsing_merge_tree.sql @@ -0,0 +1,41 @@ +DROP TABLE IF EXISTS table_with_version; + +CREATE TABLE table_with_version +( + key UInt64, + value String, + version UInt8, + sign Int8 +) +ENGINE VersionedCollapsingMergeTree(sign, version) +ORDER BY key; + +INSERT INTO table_with_version VALUES (1, '1', 1, -1); +INSERT INTO table_with_version VALUES (2, '2', 2, -1); + +SELECT * FROM table_with_version ORDER BY key; + +SHOW CREATE TABLE table_with_version; + +ALTER TABLE table_with_version MODIFY COLUMN version UInt32; + +SELECT * FROM table_with_version ORDER BY key; + +SHOW CREATE TABLE table_with_version; + +INSERT INTO TABLE table_with_version VALUES(1, '1', 1, 1); +INSERT INTO TABLE table_with_version VALUES(1, '1', 2, 1); + +SELECT * FROM table_with_version FINAL ORDER BY key; + +INSERT INTO TABLE table_with_version VALUES(3, '3', 65555, 1); + +SELECT * FROM table_with_version FINAL ORDER BY key; + +INSERT INTO TABLE table_with_version VALUES(3, '3', 65555, -1); + +SELECT * FROM table_with_version FINAL ORDER BY key; + +ALTER TABLE table_with_version MODIFY COLUMN version String; --{serverError 524} + +DROP TABLE IF EXISTS table_with_version; diff --git a/tests/queries/0_stateless/01511_alter_version_versioned_collapsing_merge_tree_zookeeper.reference b/tests/queries/0_stateless/01511_alter_version_versioned_collapsing_merge_tree_zookeeper.reference new file mode 100644 index 00000000000..c6cd81a4aca --- /dev/null +++ b/tests/queries/0_stateless/01511_alter_version_versioned_collapsing_merge_tree_zookeeper.reference @@ -0,0 +1,16 @@ +1 1 1 -1 +2 2 2 -1 +CREATE TABLE default.table_with_version_replicated_1\n(\n `key` UInt64,\n `value` String,\n `version` UInt8,\n `sign` Int8\n)\nENGINE = ReplicatedVersionedCollapsingMergeTree(\'/clickhouse/test_01511/t\', \'1\', sign, version)\nORDER BY key\nSETTINGS index_granularity = 8192 +1 1 1 -1 +2 2 2 -1 +CREATE TABLE default.table_with_version_replicated_1\n(\n `key` UInt64,\n `value` String,\n `version` UInt32,\n `sign` Int8\n)\nENGINE = ReplicatedVersionedCollapsingMergeTree(\'/clickhouse/test_01511/t\', \'1\', sign, version)\nORDER BY key\nSETTINGS index_granularity = 8192 +1 1 2 1 +2 2 2 -1 +1 1 2 1 +2 2 2 -1 +3 3 65555 1 +1 1 2 1 +2 2 2 -1 +CREATE TABLE default.table_with_version_replicated_2\n(\n `key` UInt64,\n `value` String,\n `version` UInt32,\n `sign` Int8\n)\nENGINE = ReplicatedVersionedCollapsingMergeTree(\'/clickhouse/test_01511/t\', \'2\', sign, version)\nORDER BY key\nSETTINGS index_granularity = 8192 +1 1 2 1 +2 2 2 -1 diff --git a/tests/queries/0_stateless/01511_alter_version_versioned_collapsing_merge_tree_zookeeper.sql b/tests/queries/0_stateless/01511_alter_version_versioned_collapsing_merge_tree_zookeeper.sql new file mode 100644 index 00000000000..1307f055e5c --- /dev/null +++ b/tests/queries/0_stateless/01511_alter_version_versioned_collapsing_merge_tree_zookeeper.sql @@ -0,0 +1,64 @@ +DROP TABLE IF EXISTS table_with_version_replicated_1; +DROP TABLE IF EXISTS table_with_version_replicated_2; + +CREATE TABLE table_with_version_replicated_1 +( + key UInt64, + value String, + version UInt8, + sign Int8 +) +ENGINE ReplicatedVersionedCollapsingMergeTree('/clickhouse/test_01511/t', '1', sign, version) +ORDER BY key; + +CREATE TABLE table_with_version_replicated_2 +( + key UInt64, + value String, + version UInt8, + sign Int8 +) +ENGINE ReplicatedVersionedCollapsingMergeTree('/clickhouse/test_01511/t', '2', sign, version) +ORDER BY key; + +INSERT INTO table_with_version_replicated_1 VALUES (1, '1', 1, -1); +INSERT INTO table_with_version_replicated_1 VALUES (2, '2', 2, -1); + +SELECT * FROM table_with_version_replicated_1 ORDER BY key; + +SHOW CREATE TABLE table_with_version_replicated_1; + +ALTER TABLE table_with_version_replicated_1 MODIFY COLUMN version UInt32 SETTINGS replication_alter_partitions_sync=2; + +SELECT * FROM table_with_version_replicated_1 ORDER BY key; + +SHOW CREATE TABLE table_with_version_replicated_1; + +INSERT INTO TABLE table_with_version_replicated_1 VALUES(1, '1', 1, 1); +INSERT INTO TABLE table_with_version_replicated_1 VALUES(1, '1', 2, 1); + +SELECT * FROM table_with_version_replicated_1 FINAL ORDER BY key; + +INSERT INTO TABLE table_with_version_replicated_1 VALUES(3, '3', 65555, 1); + +SELECT * FROM table_with_version_replicated_1 FINAL ORDER BY key; + +INSERT INTO TABLE table_with_version_replicated_1 VALUES(3, '3', 65555, -1); + +SYSTEM SYNC REPLICA table_with_version_replicated_2; + +DETACH TABLE table_with_version_replicated_1; +DETACH TABLE table_with_version_replicated_2; +ATTACH TABLE table_with_version_replicated_2; +ATTACH TABLE table_with_version_replicated_1; + +SELECT * FROM table_with_version_replicated_1 FINAL ORDER BY key; + +SYSTEM SYNC REPLICA table_with_version_replicated_2; + +SHOW CREATE TABLE table_with_version_replicated_2; + +SELECT * FROM table_with_version_replicated_2 FINAL ORDER BY key; + +DROP TABLE IF EXISTS table_with_version_replicated_1; +DROP TABLE IF EXISTS table_with_version_replicated_2; From 078f7075ba8f054d001b1dac4d1195271fe7a56e Mon Sep 17 00:00:00 2001 From: alesapin Date: Tue, 29 Sep 2020 15:15:45 +0300 Subject: [PATCH 202/321] Fix cmake --- cmake/find/llvm.cmake | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/cmake/find/llvm.cmake b/cmake/find/llvm.cmake index 70d0e356c39..e0ba1d9b039 100644 --- a/cmake/find/llvm.cmake +++ b/cmake/find/llvm.cmake @@ -26,7 +26,7 @@ endif () if (NOT USE_INTERNAL_LLVM_LIBRARY) set (LLVM_PATHS "/usr/local/lib/llvm") - foreach(llvm_v 9 8) + foreach(llvm_v 10 9 8) if (NOT LLVM_FOUND) find_package (LLVM ${llvm_v} CONFIG PATHS ${LLVM_PATHS}) endif () From 4f000388a75f41924e0a96a481fdbef3164d1293 Mon Sep 17 00:00:00 2001 From: feng lv Date: Tue, 29 Sep 2020 20:30:36 +0800 Subject: [PATCH 203/321] add setting output_format_pretty_row_numbers --- src/Core/Settings.h | 1 + src/Formats/FormatFactory.cpp | 1 + src/Formats/FormatSettings.h | 2 ++ .../Formats/Impl/PrettyBlockOutputFormat.cpp | 4 +++ .../Formats/Impl/PrettyBlockOutputFormat.h | 2 ++ .../Impl/PrettyCompactBlockOutputFormat.cpp | 29 +++++++++++++++++++ 6 files changed, 39 insertions(+) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 9449cd571a1..db7f29f199c 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -468,6 +468,7 @@ class IColumn; M(Bool, output_format_enable_streaming, false, "Enable streaming in output formats that support it.", 0) \ M(Bool, output_format_write_statistics, true, "Write statistics about read rows, bytes, time elapsed in suitable output formats.", 0) \ M(Bool, allow_non_metadata_alters, true, "Allow to execute alters which affects not only tables metadata, but also data on disk", 0) \ + M(Bool, output_format_pretty_row_numbers, false, "Add row numbers before each row for pretty output format", 0) \ #define LIST_OF_SETTINGS(M) \ COMMON_SETTINGS(M) \ diff --git a/src/Formats/FormatFactory.cpp b/src/Formats/FormatFactory.cpp index 522149d3cfd..2dc39e228c9 100644 --- a/src/Formats/FormatFactory.cpp +++ b/src/Formats/FormatFactory.cpp @@ -107,6 +107,7 @@ static FormatSettings getOutputFormatSetting(const Settings & settings, const Co format_settings.pretty.charset = settings.output_format_pretty_grid_charset.toString() == "ASCII" ? FormatSettings::Pretty::Charset::ASCII : FormatSettings::Pretty::Charset::UTF8; + format_settings.pretty.output_format_pretty_row_numbers = settings.output_format_pretty_row_numbers; format_settings.template_settings.resultset_format = settings.format_template_resultset; format_settings.template_settings.row_format = settings.format_template_row; format_settings.template_settings.row_between_delimiter = settings.format_template_rows_between_delimiter; diff --git a/src/Formats/FormatSettings.h b/src/Formats/FormatSettings.h index cd5cab8cf5a..a97bd9bf6c6 100644 --- a/src/Formats/FormatSettings.h +++ b/src/Formats/FormatSettings.h @@ -45,6 +45,8 @@ struct FormatSettings UInt64 max_value_width = 10000; bool color = true; + bool output_format_pretty_row_numbers = false; + enum class Charset { UTF8, diff --git a/src/Processors/Formats/Impl/PrettyBlockOutputFormat.cpp b/src/Processors/Formats/Impl/PrettyBlockOutputFormat.cpp index 9a9c150fad4..ec4b3286448 100644 --- a/src/Processors/Formats/Impl/PrettyBlockOutputFormat.cpp +++ b/src/Processors/Formats/Impl/PrettyBlockOutputFormat.cpp @@ -33,6 +33,10 @@ void PrettyBlockOutputFormat::calculateWidths( WidthsPerColumn & widths, Widths & max_padded_widths, Widths & name_widths) { size_t num_rows = std::min(chunk.getNumRows(), format_settings.pretty.max_rows); + + auto max_row_number = std::to_string(num_rows); + row_number_width = max_row_number.size() + 2; + size_t num_columns = chunk.getNumColumns(); const auto & columns = chunk.getColumns(); diff --git a/src/Processors/Formats/Impl/PrettyBlockOutputFormat.h b/src/Processors/Formats/Impl/PrettyBlockOutputFormat.h index 59628cf322c..de79fe5ee2a 100644 --- a/src/Processors/Formats/Impl/PrettyBlockOutputFormat.h +++ b/src/Processors/Formats/Impl/PrettyBlockOutputFormat.h @@ -33,6 +33,8 @@ protected: size_t terminal_width = 0; bool suffix_written = false; + size_t row_number_width = 7; // "10000. " + const FormatSettings format_settings; using Widths = PODArray; diff --git a/src/Processors/Formats/Impl/PrettyCompactBlockOutputFormat.cpp b/src/Processors/Formats/Impl/PrettyCompactBlockOutputFormat.cpp index 254bb9381f2..13000a7b9d3 100644 --- a/src/Processors/Formats/Impl/PrettyCompactBlockOutputFormat.cpp +++ b/src/Processors/Formats/Impl/PrettyCompactBlockOutputFormat.cpp @@ -69,6 +69,15 @@ void PrettyCompactBlockOutputFormat::writeHeader( const Widths & max_widths, const Widths & name_widths) { + if (format_settings.pretty.output_format_pretty_row_numbers) + { + /// Write left blank + for (size_t i = 0; i < row_number_width; ++i) + { + writeCString(" ", out); + } + } + const GridSymbols & grid_symbols = format_settings.pretty.charset == FormatSettings::Pretty::Charset::UTF8 ? utf8_grid_symbols : ascii_grid_symbols; @@ -117,6 +126,15 @@ void PrettyCompactBlockOutputFormat::writeHeader( void PrettyCompactBlockOutputFormat::writeBottom(const Widths & max_widths) { + if (format_settings.pretty.output_format_pretty_row_numbers) + { + /// Write left blank + for (size_t i = 0; i < row_number_width; ++i) + { + writeCString(" ", out); + } + } + const GridSymbols & grid_symbols = format_settings.pretty.charset == FormatSettings::Pretty::Charset::UTF8 ? utf8_grid_symbols : ascii_grid_symbols; @@ -144,6 +162,17 @@ void PrettyCompactBlockOutputFormat::writeRow( const WidthsPerColumn & widths, const Widths & max_widths) { + if (format_settings.pretty.output_format_pretty_row_numbers) + { + // Write row number; + auto row_num_string = std::to_string(row_num + 1) + ". "; + for (size_t i = 0; i < row_number_width - row_num_string.size(); ++i) + { + writeCString(" ", out); + } + writeString(row_num_string, out); + } + const GridSymbols & grid_symbols = format_settings.pretty.charset == FormatSettings::Pretty::Charset::UTF8 ? utf8_grid_symbols : ascii_grid_symbols; From 5d19ed04f6d638791f96dad01c8c0bb414e96bc9 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Tue, 29 Sep 2020 16:42:58 +0300 Subject: [PATCH 204/321] allow synchronous drop/detach table for Atomic --- src/Databases/DatabaseAtomic.cpp | 36 +++++++++++++++---- src/Databases/DatabaseAtomic.h | 4 ++- src/Interpreters/DatabaseCatalog.cpp | 16 +++++++++ src/Interpreters/DatabaseCatalog.h | 5 +++ src/Interpreters/InterpreterDropQuery.cpp | 13 +++++++ tests/integration/test_merge_tree_s3/test.py | 3 -- tests/integration/test_rename_column/test.py | 1 - tests/integration/test_ttl_move/test.py | 1 - tests/integration/test_ttl_replicated/test.py | 1 - 9 files changed, 66 insertions(+), 14 deletions(-) diff --git a/src/Databases/DatabaseAtomic.cpp b/src/Databases/DatabaseAtomic.cpp index 6ad547898e2..fa4511cd104 100644 --- a/src/Databases/DatabaseAtomic.cpp +++ b/src/Databases/DatabaseAtomic.cpp @@ -83,7 +83,7 @@ void DatabaseAtomic::attachTable(const String & name, const StoragePtr & table, assert(relative_table_path != data_path && !relative_table_path.empty()); DetachedTables not_in_use; std::unique_lock lock(mutex); - not_in_use = cleenupDetachedTables(); + not_in_use = cleanupDetachedTables(); auto table_id = table->getStorageID(); assertDetachedTableNotInUse(table_id.uuid); DatabaseWithDictionaries::attachTableUnlocked(name, table, lock); @@ -97,7 +97,7 @@ StoragePtr DatabaseAtomic::detachTable(const String & name) auto table = DatabaseWithDictionaries::detachTableUnlocked(name, lock); table_name_to_path.erase(name); detached_tables.emplace(table->getStorageID().uuid, table); - not_in_use = cleenupDetachedTables(); + not_in_use = cleanupDetachedTables(); return table; } @@ -263,7 +263,7 @@ void DatabaseAtomic::commitCreateTable(const ASTCreateQuery & query, const Stora if (query.database != database_name) throw Exception(ErrorCodes::UNKNOWN_DATABASE, "Database was renamed to `{}`, cannot create table in `{}`", database_name, query.database); - not_in_use = cleenupDetachedTables(); + not_in_use = cleanupDetachedTables(); assertDetachedTableNotInUse(query.uuid); renameNoReplace(table_metadata_tmp_path, table_metadata_path); attachTableUnlocked(query.table, table, lock); /// Should never throw @@ -306,7 +306,7 @@ void DatabaseAtomic::assertDetachedTableNotInUse(const UUID & uuid) ", because it was detached but still used by some query. Retry later.", ErrorCodes::TABLE_ALREADY_EXISTS); } -DatabaseAtomic::DetachedTables DatabaseAtomic::cleenupDetachedTables() +DatabaseAtomic::DetachedTables DatabaseAtomic::cleanupDetachedTables() { DetachedTables not_in_use; auto it = detached_tables.begin(); @@ -324,14 +324,14 @@ DatabaseAtomic::DetachedTables DatabaseAtomic::cleenupDetachedTables() return not_in_use; } -void DatabaseAtomic::assertCanBeDetached(bool cleenup) +void DatabaseAtomic::assertCanBeDetached(bool cleanup) { - if (cleenup) + if (cleanup) { DetachedTables not_in_use; { std::lock_guard lock(mutex); - not_in_use = cleenupDetachedTables(); + not_in_use = cleanupDetachedTables(); } } std::lock_guard lock(mutex); @@ -500,6 +500,28 @@ void DatabaseAtomic::renameDictionaryInMemoryUnlocked(const StorageID & old_name const auto & dict = dynamic_cast(*result.object); dict.updateDictionaryName(new_name); } +void DatabaseAtomic::waitDetachedTableNotInUse(const UUID & uuid) +{ + { + std::lock_guard lock{mutex}; + if (detached_tables.count(uuid) == 0) + return; + } + + /// Table is in use while its shared_ptr counter is greater than 1. + /// We cannot trigger condvar on shared_ptr destruction, so it's busy wait. + while (true) + { + DetachedTables not_in_use; + { + std::lock_guard lock{mutex}; + not_in_use = cleanupDetachedTables(); + if (detached_tables.count(uuid) == 0) + return; + } + std::this_thread::sleep_for(std::chrono::milliseconds(500)); + } +} } diff --git a/src/Databases/DatabaseAtomic.h b/src/Databases/DatabaseAtomic.h index 02c922f8b91..8fa0d1837e7 100644 --- a/src/Databases/DatabaseAtomic.h +++ b/src/Databases/DatabaseAtomic.h @@ -58,6 +58,8 @@ public: void tryCreateSymlink(const String & table_name, const String & actual_data_path); void tryRemoveSymlink(const String & table_name); + void waitDetachedTableNotInUse(const UUID & uuid); + private: void commitAlterTable(const StorageID & table_id, const String & table_metadata_tmp_path, const String & table_metadata_path) override; void commitCreateTable(const ASTCreateQuery & query, const StoragePtr & table, @@ -65,7 +67,7 @@ private: void assertDetachedTableNotInUse(const UUID & uuid); typedef std::unordered_map DetachedTables; - [[nodiscard]] DetachedTables cleenupDetachedTables(); + [[nodiscard]] DetachedTables cleanupDetachedTables(); void tryCreateMetadataSymlink(); diff --git a/src/Interpreters/DatabaseCatalog.cpp b/src/Interpreters/DatabaseCatalog.cpp index 049341918b9..e58213fe366 100644 --- a/src/Interpreters/DatabaseCatalog.cpp +++ b/src/Interpreters/DatabaseCatalog.cpp @@ -701,6 +701,7 @@ void DatabaseCatalog::enqueueDroppedTableCleanup(StorageID table_id, StoragePtr tables_marked_dropped.push_front({table_id, table, dropped_metadata_path, 0}); else tables_marked_dropped.push_back({table_id, table, dropped_metadata_path, drop_time}); + tables_marked_dropped_ids.insert(table_id.uuid); /// If list of dropped tables was empty, start a drop task if (drop_task && tables_marked_dropped.size() == 1) (*drop_task)->schedule(); @@ -742,6 +743,9 @@ void DatabaseCatalog::dropTableDataTask() try { dropTableFinally(table); + std::lock_guard lock(tables_marked_dropped_mutex); + auto removed = tables_marked_dropped_ids.erase(table.table_id.uuid); + assert(removed); } catch (...) { @@ -755,6 +759,8 @@ void DatabaseCatalog::dropTableDataTask() need_reschedule = true; } } + + wait_table_finally_dropped.notify_all(); } /// Do not schedule a task if there is no tables to drop @@ -814,6 +820,16 @@ String DatabaseCatalog::resolveDictionaryName(const String & name) const return toString(db_and_table.second->getStorageID().uuid); } +void DatabaseCatalog::waitTableFinallyDropped(const UUID & uuid) +{ + if (uuid == UUIDHelpers::Nil) + return; + std::unique_lock lock{tables_marked_dropped_mutex}; + wait_table_finally_dropped.wait(lock, [&](){ + return tables_marked_dropped_ids.count(uuid) == 0; + }); +} + DDLGuard::DDLGuard(Map & map_, std::shared_mutex & db_mutex_, std::unique_lock guards_lock_, const String & elem) : map(map_), db_mutex(db_mutex_), guards_lock(std::move(guards_lock_)) diff --git a/src/Interpreters/DatabaseCatalog.h b/src/Interpreters/DatabaseCatalog.h index 8ef3ecfe656..7bc6923bde4 100644 --- a/src/Interpreters/DatabaseCatalog.h +++ b/src/Interpreters/DatabaseCatalog.h @@ -9,6 +9,7 @@ #include #include #include +#include #include #include #include @@ -179,6 +180,8 @@ public: /// Try convert qualified dictionary name to persistent UUID String resolveDictionaryName(const String & name) const; + void waitTableFinallyDropped(const UUID & uuid); + private: // The global instance of database catalog. unique_ptr is to allow // deferred initialization. Thought I'd use std::optional, but I can't @@ -249,11 +252,13 @@ private: mutable std::mutex ddl_guards_mutex; TablesMarkedAsDropped tables_marked_dropped; + std::unordered_set tables_marked_dropped_ids; mutable std::mutex tables_marked_dropped_mutex; std::unique_ptr drop_task; static constexpr time_t default_drop_delay_sec = 8 * 60; time_t drop_delay_sec = default_drop_delay_sec; + std::condition_variable wait_table_finally_dropped; }; } diff --git a/src/Interpreters/InterpreterDropQuery.cpp b/src/Interpreters/InterpreterDropQuery.cpp index dec6a275872..b6d61b6c969 100644 --- a/src/Interpreters/InterpreterDropQuery.cpp +++ b/src/Interpreters/InterpreterDropQuery.cpp @@ -124,6 +124,19 @@ BlockIO InterpreterDropQuery::executeToTable( } } + table.reset(); + ddl_guard = {}; + if (query.no_delay) + { + if (query.kind == ASTDropQuery::Kind::Drop) + DatabaseCatalog::instance().waitTableFinallyDropped(table_id.uuid); + else if (query.kind == ASTDropQuery::Kind::Detach) + { + if (auto * atomic = typeid_cast(database.get())) + atomic->waitDetachedTableNotInUse(table_id.uuid); + } + } + return {}; } diff --git a/tests/integration/test_merge_tree_s3/test.py b/tests/integration/test_merge_tree_s3/test.py index 7acb8c5fe00..45b3c3c65f0 100644 --- a/tests/integration/test_merge_tree_s3/test.py +++ b/tests/integration/test_merge_tree_s3/test.py @@ -75,7 +75,6 @@ def drop_table(cluster): minio = cluster.minio_client node.query("DROP TABLE IF EXISTS s3_test NO DELAY") - time.sleep(1) try: assert len(list(minio.list_objects(cluster.minio_bucket, 'data/'))) == 0 finally: @@ -317,7 +316,6 @@ def test_move_replace_partition_to_another_table(cluster): minio.list_objects(cluster.minio_bucket, 'data/'))) == FILES_OVERHEAD * 2 + FILES_OVERHEAD_PER_PART_WIDE * 4 node.query("DROP TABLE s3_clone NO DELAY") - time.sleep(1) assert node.query("SELECT sum(id) FROM s3_test FORMAT Values") == "(0)" assert node.query("SELECT count(*) FROM s3_test FORMAT Values") == "(16384)" # Data should remain in S3 @@ -330,7 +328,6 @@ def test_move_replace_partition_to_another_table(cluster): list(minio.list_objects(cluster.minio_bucket, 'data/'))) == FILES_OVERHEAD + FILES_OVERHEAD_PER_PART_WIDE * 4 node.query("DROP TABLE s3_test NO DELAY") - time.sleep(1) # Backup data should remain in S3. assert len(list(minio.list_objects(cluster.minio_bucket, 'data/'))) == FILES_OVERHEAD_PER_PART_WIDE * 4 diff --git a/tests/integration/test_rename_column/test.py b/tests/integration/test_rename_column/test.py index 51921c3385c..b7ab8a75ba5 100644 --- a/tests/integration/test_rename_column/test.py +++ b/tests/integration/test_rename_column/test.py @@ -37,7 +37,6 @@ def started_cluster(): def drop_table(nodes, table_name): for node in nodes: node.query("DROP TABLE IF EXISTS {} NO DELAY".format(table_name)) - time.sleep(1) def create_table(nodes, table_name, with_storage_policy=False, with_time_column=False, diff --git a/tests/integration/test_ttl_move/test.py b/tests/integration/test_ttl_move/test.py index 377ee0e5d75..59705f7b3dd 100644 --- a/tests/integration/test_ttl_move/test.py +++ b/tests/integration/test_ttl_move/test.py @@ -403,7 +403,6 @@ def test_moves_to_disk_eventually_work(started_cluster, name, engine): node1.query("DROP TABLE {} NO DELAY".format(name_temp)) - time.sleep(2) used_disks = get_used_disks_for_table(node1, name) assert set(used_disks) == {"jbod2"} diff --git a/tests/integration/test_ttl_replicated/test.py b/tests/integration/test_ttl_replicated/test.py index 878db2da11f..13fb779a6e6 100644 --- a/tests/integration/test_ttl_replicated/test.py +++ b/tests/integration/test_ttl_replicated/test.py @@ -27,7 +27,6 @@ def started_cluster(): def drop_table(nodes, table_name): for node in nodes: node.query("DROP TABLE IF EXISTS {} NO DELAY".format(table_name)) - time.sleep(1) # Column TTL works only with wide parts, because it's very expensive to apply it for compact parts def test_ttl_columns(started_cluster): From 8692c104958dd12f6d41980ad4fd84986a976edc Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Tue, 29 Sep 2020 17:12:40 +0300 Subject: [PATCH 205/321] add alias --- src/Parsers/ParserDropQuery.cpp | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/Parsers/ParserDropQuery.cpp b/src/Parsers/ParserDropQuery.cpp index 28ab3bdac32..31a6250a006 100644 --- a/src/Parsers/ParserDropQuery.cpp +++ b/src/Parsers/ParserDropQuery.cpp @@ -22,6 +22,7 @@ bool parseDropQuery(IParser::Pos & pos, ASTPtr & node, Expected & expected, bool ParserKeyword s_if_exists("IF EXISTS"); ParserIdentifier name_p; ParserKeyword s_no_delay("NO DELAY"); + ParserKeyword s_sync("SYNC"); ASTPtr database; ASTPtr table; @@ -79,7 +80,7 @@ bool parseDropQuery(IParser::Pos & pos, ASTPtr & node, Expected & expected, bool return false; } - if (s_no_delay.ignore(pos, expected)) + if (s_no_delay.ignore(pos, expected) || s_sync.ignore(pos, expected)) no_delay = true; } From cf86e19f3860d9d0bdc0a2d3958c3c1150be133d Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Tue, 29 Sep 2020 17:15:05 +0300 Subject: [PATCH 206/321] fix build --- src/Interpreters/DatabaseCatalog.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Interpreters/DatabaseCatalog.cpp b/src/Interpreters/DatabaseCatalog.cpp index e58213fe366..df3929671a3 100644 --- a/src/Interpreters/DatabaseCatalog.cpp +++ b/src/Interpreters/DatabaseCatalog.cpp @@ -744,7 +744,7 @@ void DatabaseCatalog::dropTableDataTask() { dropTableFinally(table); std::lock_guard lock(tables_marked_dropped_mutex); - auto removed = tables_marked_dropped_ids.erase(table.table_id.uuid); + [[maybe_unused]] auto removed = tables_marked_dropped_ids.erase(table.table_id.uuid); assert(removed); } catch (...) From 4ab39d2bee0e6f03507ba3fa1c62ba835d7a105b Mon Sep 17 00:00:00 2001 From: alesapin Date: Tue, 29 Sep 2020 18:19:30 +0300 Subject: [PATCH 207/321] Fix style --- src/Interpreters/DatabaseCatalog.cpp | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/Interpreters/DatabaseCatalog.cpp b/src/Interpreters/DatabaseCatalog.cpp index df3929671a3..203e2292c08 100644 --- a/src/Interpreters/DatabaseCatalog.cpp +++ b/src/Interpreters/DatabaseCatalog.cpp @@ -825,7 +825,8 @@ void DatabaseCatalog::waitTableFinallyDropped(const UUID & uuid) if (uuid == UUIDHelpers::Nil) return; std::unique_lock lock{tables_marked_dropped_mutex}; - wait_table_finally_dropped.wait(lock, [&](){ + wait_table_finally_dropped.wait(lock, [&]() + { return tables_marked_dropped_ids.count(uuid) == 0; }); } From 86cfc6f914bf623068bb1babf135b6e34268b1af Mon Sep 17 00:00:00 2001 From: Artem Zuikov Date: Tue, 29 Sep 2020 20:36:14 +0300 Subject: [PATCH 208/321] change error code (#15452) --- src/Interpreters/IdentifierSemantic.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Interpreters/IdentifierSemantic.cpp b/src/Interpreters/IdentifierSemantic.cpp index f661ec2ae71..256a3784c77 100644 --- a/src/Interpreters/IdentifierSemantic.cpp +++ b/src/Interpreters/IdentifierSemantic.cpp @@ -8,7 +8,7 @@ namespace DB namespace ErrorCodes { - extern const int LOGICAL_ERROR; + extern const int SYNTAX_ERROR; extern const int AMBIGUOUS_COLUMN_NAME; } @@ -147,7 +147,7 @@ std::optional IdentifierSemantic::chooseTableColumnMatch(const ASTIdenti StorageID IdentifierSemantic::extractDatabaseAndTable(const ASTIdentifier & identifier) { if (identifier.name_parts.size() > 2) - throw Exception("Logical error: more than two components in table expression", ErrorCodes::LOGICAL_ERROR); + throw Exception("Syntax error: more than two components in table expression", ErrorCodes::SYNTAX_ERROR); if (identifier.name_parts.size() == 2) return { identifier.name_parts[0], identifier.name_parts[1], identifier.uuid }; From 4fd1db73a13ffdff205a495ac75db90bfd7fe1f5 Mon Sep 17 00:00:00 2001 From: Artem Zuikov Date: Tue, 29 Sep 2020 20:52:34 +0300 Subject: [PATCH 209/321] Faster 256-bit multiplication (#15418) --- base/common/wide_integer_impl.h | 116 +++++++++++++----- tests/performance/bigint_arithm.xml | 15 +++ .../01485_256_bit_multiply.reference | 1 + .../0_stateless/01485_256_bit_multiply.sql | 4 + 4 files changed, 102 insertions(+), 34 deletions(-) create mode 100644 tests/performance/bigint_arithm.xml create mode 100644 tests/queries/0_stateless/01485_256_bit_multiply.reference create mode 100644 tests/queries/0_stateless/01485_256_bit_multiply.sql diff --git a/base/common/wide_integer_impl.h b/base/common/wide_integer_impl.h index b8a564c5a77..ea4da63da20 100644 --- a/base/common/wide_integer_impl.h +++ b/base/common/wide_integer_impl.h @@ -436,46 +436,94 @@ private: } template - constexpr static auto multiply(const integer & lhs, const T & rhs) + constexpr static integer + multiply(const integer & lhs, const T & rhs) { - integer res{}; -#if 1 - integer lhs2 = plus(lhs, shift_left(lhs, 1)); - integer lhs3 = plus(lhs2, shift_left(lhs, 2)); -#endif - for (unsigned i = 0; i < item_count; ++i) + if constexpr (Bits == 256 && sizeof(base_type) == 8) { - base_type rhs_item = get_item(rhs, i); - unsigned pos = i * base_bits; + /// @sa https://github.com/abseil/abseil-cpp/blob/master/absl/numeric/int128.h + using HalfType = unsigned __int128; - while (rhs_item) + HalfType a01 = (HalfType(lhs.items[little(1)]) << 64) + lhs.items[little(0)]; + HalfType a23 = (HalfType(lhs.items[little(3)]) << 64) + lhs.items[little(2)]; + HalfType a0 = lhs.items[little(0)]; + HalfType a1 = lhs.items[little(1)]; + + HalfType b01 = rhs; + uint64_t b0 = b01; + uint64_t b1 = 0; + HalfType b23 = 0; + if constexpr (sizeof(T) > 8) + b1 = b01 >> 64; + if constexpr (sizeof(T) > 16) + b23 = (HalfType(rhs.items[little(3)]) << 64) + rhs.items[little(2)]; + + HalfType r23 = a23 * b01 + a01 * b23 + a1 * b1; + HalfType r01 = a0 * b0; + HalfType r12 = (r01 >> 64) + (r23 << 64); + HalfType r12_x = a1 * b0; + + integer res; + res.items[little(0)] = r01; + res.items[little(3)] = r23 >> 64; + + if constexpr (sizeof(T) > 8) { -#if 1 /// optimization - if ((rhs_item & 0x7) == 0x7) - { - res = plus(res, shift_left(lhs3, pos)); - rhs_item >>= 3; - pos += 3; - continue; - } - - if ((rhs_item & 0x3) == 0x3) - { - res = plus(res, shift_left(lhs2, pos)); - rhs_item >>= 2; - pos += 2; - continue; - } -#endif - if (rhs_item & 1) - res = plus(res, shift_left(lhs, pos)); - - rhs_item >>= 1; - ++pos; + HalfType r12_y = a0 * b1; + r12_x += r12_y; + if (r12_x < r12_y) + ++res.items[little(3)]; } - } - return res; + r12 += r12_x; + if (r12 < r12_x) + ++res.items[little(3)]; + + res.items[little(1)] = r12; + res.items[little(2)] = r12 >> 64; + return res; + } + else + { + integer res{}; +#if 1 + integer lhs2 = plus(lhs, shift_left(lhs, 1)); + integer lhs3 = plus(lhs2, shift_left(lhs, 2)); +#endif + for (unsigned i = 0; i < item_count; ++i) + { + base_type rhs_item = get_item(rhs, i); + unsigned pos = i * base_bits; + + while (rhs_item) + { +#if 1 /// optimization + if ((rhs_item & 0x7) == 0x7) + { + res = plus(res, shift_left(lhs3, pos)); + rhs_item >>= 3; + pos += 3; + continue; + } + + if ((rhs_item & 0x3) == 0x3) + { + res = plus(res, shift_left(lhs2, pos)); + rhs_item >>= 2; + pos += 2; + continue; + } +#endif + if (rhs_item & 1) + res = plus(res, shift_left(lhs, pos)); + + rhs_item >>= 1; + ++pos; + } + } + + return res; + } } public: diff --git a/tests/performance/bigint_arithm.xml b/tests/performance/bigint_arithm.xml new file mode 100644 index 00000000000..4d81f2e2c2d --- /dev/null +++ b/tests/performance/bigint_arithm.xml @@ -0,0 +1,15 @@ + + + 10G + + + SELECT toInt128(number) + number FROM numbers_mt(1000000000) FORMAT Null + SELECT toInt128(number) - number FROM numbers_mt(1000000000) FORMAT Null + SELECT toInt128(number) * number FROM numbers_mt(1000000000) FORMAT Null + SELECT toInt128(number) / number FROM numbers_mt(1000000000) FORMAT Null + + SELECT toInt256(number) + number FROM numbers_mt(100000000) FORMAT Null + SELECT toInt256(number) - number FROM numbers_mt(100000000) FORMAT Null + SELECT toInt256(number) * number FROM numbers_mt(100000000) FORMAT Null + SELECT intDiv(toInt256(number + 1), number + 1) FROM numbers_mt(100000000) FORMAT Null + diff --git a/tests/queries/0_stateless/01485_256_bit_multiply.reference b/tests/queries/0_stateless/01485_256_bit_multiply.reference new file mode 100644 index 00000000000..573541ac970 --- /dev/null +++ b/tests/queries/0_stateless/01485_256_bit_multiply.reference @@ -0,0 +1 @@ +0 diff --git a/tests/queries/0_stateless/01485_256_bit_multiply.sql b/tests/queries/0_stateless/01485_256_bit_multiply.sql new file mode 100644 index 00000000000..f3f6cc05058 --- /dev/null +++ b/tests/queries/0_stateless/01485_256_bit_multiply.sql @@ -0,0 +1,4 @@ +select count() from +( + select toInt128(number) * number x, toInt256(number) * number y from numbers_mt(100000000) where x != y +); From e943a22d289da0be3c38c6805331df784c8c6cef Mon Sep 17 00:00:00 2001 From: Alexander Kuzmenkov Date: Tue, 29 Sep 2020 21:01:49 +0300 Subject: [PATCH 210/321] allow fast test to run locally --- docker/test/fasttest/run.sh | 165 +++++++++++++++++++++++++----------- 1 file changed, 114 insertions(+), 51 deletions(-) diff --git a/docker/test/fasttest/run.sh b/docker/test/fasttest/run.sh index a277ddf9d36..b93aa0ce5cd 100755 --- a/docker/test/fasttest/run.sh +++ b/docker/test/fasttest/run.sh @@ -16,28 +16,55 @@ stage=${stage:-} read -ra FASTTEST_CMAKE_FLAGS <<< "${FASTTEST_CMAKE_FLAGS:-}" -function kill_clickhouse +FASTTEST_WORKSPACE=$(readlink -f "${FASTTEST_WORKSPACE:-.}") +FASTTEST_SOURCE=$(readlink -f "${FASTTEST_SOURCE:-$FASTTEST_WORKSPACE/ch}") +FASTTEST_BUILD=$(readlink -f "${FASTTEST_BUILD:-${BUILD:-$FASTTEST_WORKSPACE/build}}") +FASTTEST_DATA=$(readlink -f "${FASTTEST_DATA:-$FASTTEST_WORKSPACE/db-fasttest}") +FASTTEST_OUTPUT=$(readlink -f "${FASTTEST_OUTPUT:-$FASTTEST_WORKSPACE}") + +server_pid=none + +function stop_server { + if ! kill -- "$server_pid" + then + echo "The server we started ($server_pid) is not present, won't do anything" + return 0 + fi + for _ in {1..60} do - if ! pkill -f clickhouse-server ; then break ; fi + if ! kill -- "$server_pid" ; then break ; fi sleep 1 done - if pgrep -f clickhouse-server + if kill -0 -- "$server_pid" then pstree -apgT jobs - echo "Failed to kill the ClickHouse server $(pgrep -f clickhouse-server)" + echo "Failed to kill the ClickHouse server pid '$server_pid'" return 1 fi } -function wait_for_server_start +function start_server { + set -m # Spawn server in its own process groups + clickhouse-server --config-file="$FASTTEST_DATA/config.xml" -- --path "$FASTTEST_DATA" --user_files_path "$FASTTEST_DATA/user_files" &>> "$FASTTEST_OUTPUT/server.log" & + server_pid=$! + set +m + + if [ "$server_pid" == "0" ] + then + echo "Failed to start ClickHouse server" + # Avoid zero PID because `kill` treats it as our process group PID. + server_pid="none" + return 1 + fi + for _ in {1..60} do - if clickhouse-client --query "select 1" || ! pgrep -f clickhouse-server + if clickhouse-client --query "select 1" || ! kill -0 -- "$server_pid" then break fi @@ -47,20 +74,26 @@ function wait_for_server_start if ! clickhouse-client --query "select 1" then echo "Failed to wait until ClickHouse server starts." + server_pid="none" return 1 fi - echo "ClickHouse server pid '$(pgrep -f clickhouse-server)' started and responded" + if ! kill -0 -- "$server_pid" + then + echo "Wrong clickhouse server started: PID '$server_pid' we started is not running, but '$(pgrep -f clickhouse-server)' is running" + server_pid="none" + return 1 + fi + + echo "ClickHouse server pid '$server_pid' started and responded" } function clone_root { -git clone https://github.com/ClickHouse/ClickHouse.git | ts '%Y-%m-%d %H:%M:%S' | tee /test_output/clone_log.txt -cd ClickHouse -CLICKHOUSE_DIR=$(pwd) -export CLICKHOUSE_DIR - +git clone https://github.com/ClickHouse/ClickHouse.git -- "$FASTTEST_SOURCE" | ts '%Y-%m-%d %H:%M:%S' | tee "$FASTTEST_OUTPUT/clone_log.txt" +( +cd "$FASTTEST_SOURCE" if [ "$PULL_REQUEST_NUMBER" != "0" ]; then if git fetch origin "+refs/pull/$PULL_REQUEST_NUMBER/merge"; then git checkout FETCH_HEAD @@ -71,22 +104,37 @@ if [ "$PULL_REQUEST_NUMBER" != "0" ]; then echo 'Checked out to commit' fi else - if [ "$COMMIT_SHA" != "" ]; then + if [ -v COMMIT_SHA ]; then git checkout "$COMMIT_SHA" fi fi +) } -function run +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) -git submodule update --init --recursive "${SUBMODULES_TO_UPDATE[@]}" | ts '%Y-%m-%d %H:%M:%S' | tee /test_output/submodule_log.txt +git submodule sync +git submodule update --init --recursive "${SUBMODULES_TO_UPDATE[@]}" +git submodule foreach git reset --hard +git submodule foreach git checkout @ -f +git submodule foreach git clean -xfd +) +} + +function build +{ CMAKE_LIBS_CONFIG=(-DENABLE_LIBRARIES=0 -DENABLE_TESTS=0 -DENABLE_UTILS=0 -DENABLE_EMBEDDED_COMPILER=0 -DENABLE_THINLTO=0 -DUSE_UNWIND=1) -export CCACHE_DIR=/ccache -export CCACHE_BASEDIR=/ClickHouse +# TODO remove this? we don't use ccache anyway. An option would be to download it +# from S3 simultaneously with cloning. +export CCACHE_DIR="$FASTTEST_WORKSPACE/ccache" +export CCACHE_BASEDIR="$FASTTEST_SOURCE" export CCACHE_NOHASHDIR=true export CCACHE_COMPILERCHECK=content export CCACHE_MAXSIZE=15G @@ -94,34 +142,40 @@ export CCACHE_MAXSIZE=15G ccache --show-stats ||: ccache --zero-stats ||: -mkdir build -cd build -cmake .. -DCMAKE_INSTALL_PREFIX=/usr -DCMAKE_CXX_COMPILER=clang++-10 -DCMAKE_C_COMPILER=clang-10 "${CMAKE_LIBS_CONFIG[@]}" "${FASTTEST_CMAKE_FLAGS[@]}" | ts '%Y-%m-%d %H:%M:%S' | tee /test_output/cmake_log.txt -time ninja clickhouse-bundle | ts '%Y-%m-%d %H:%M:%S' | tee /test_output/build_log.txt -ninja install | ts '%Y-%m-%d %H:%M:%S' | tee /test_output/install_log.txt - +mkdir "$FASTTEST_BUILD" ||: +( +cd "$FASTTEST_BUILD" +cmake "$FASTTEST_SOURCE" -DCMAKE_CXX_COMPILER=clang++-10 -DCMAKE_C_COMPILER=clang-10 "${CMAKE_LIBS_CONFIG[@]}" "${FASTTEST_CMAKE_FLAGS[@]}" | ts '%Y-%m-%d %H:%M:%S' | tee "$FASTTEST_OUTPUT/cmake_log.txt" +time ninja clickhouse-bundle | ts '%Y-%m-%d %H:%M:%S' | tee "$FASTTEST_OUTPUT/build_log.txt" +) ccache --show-stats ||: +} -mkdir -p /etc/clickhouse-server -mkdir -p /etc/clickhouse-client -mkdir -p /etc/clickhouse-server/config.d -mkdir -p /etc/clickhouse-server/users.d -ln -s /test_output /var/log/clickhouse-server -cp "$CLICKHOUSE_DIR/programs/server/config.xml" /etc/clickhouse-server/ -cp "$CLICKHOUSE_DIR/programs/server/users.xml" /etc/clickhouse-server/ +function configure +{ +export PATH="$FASTTEST_BUILD/programs:$FASTTEST_SOURCE/tests:$PATH" -# install tests config -$CLICKHOUSE_DIR/tests/config/install.sh +clickhouse-client --version +clickhouse-test --help + +mkdir -p "$FASTTEST_DATA"{,/client-config} +cp -a "$FASTTEST_SOURCE/programs/server/"{config,users}.xml "$FASTTEST_DATA" +cp -a "$FASTTEST_SOURCE/programs/server/"{config,users}.xml "$FASTTEST_DATA" +"$FASTTEST_SOURCE/tests/config/install.sh" "$FASTTEST_DATA" "$FASTTEST_DATA/client-config" # doesn't support SSL -rm -f /etc/clickhouse-server/config.d/secure_ports.xml +rm -f "$FASTTEST_DATA/config.d/secure_ports.xml" +} + +function run_tests +{ +clickhouse-server --version +clickhouse-test --help # Kill the server in case we are running locally and not in docker -kill_clickhouse +stop_server ||: -clickhouse-server --config /etc/clickhouse-server/config.xml --daemon - -wait_for_server_start +start_server TESTS_TO_SKIP=( parquet @@ -191,11 +245,10 @@ TESTS_TO_SKIP=( 01460_DistributedFilesToInsert ) -time clickhouse-test -j 8 --no-long --testname --shard --zookeeper --skip "${TESTS_TO_SKIP[@]}" 2>&1 | ts '%Y-%m-%d %H:%M:%S' | tee /test_output/test_log.txt - +time clickhouse-test -j 8 --no-long --testname --shard --zookeeper --skip "${TESTS_TO_SKIP[@]}" 2>&1 | ts '%Y-%m-%d %H:%M:%S' | tee "$FASTTEST_OUTPUT/test_log.txt" # substr is to remove semicolon after test name -readarray -t FAILED_TESTS < <(awk '/FAIL|TIMEOUT|ERROR/ { print substr($3, 1, length($3)-1) }' /test_output/test_log.txt | tee /test_output/failed-parallel-tests.txt) +readarray -t FAILED_TESTS < <(awk '/FAIL|TIMEOUT|ERROR/ { print substr($3, 1, length($3)-1) }' "$FASTTEST_OUTPUT/test_log.txt" | tee "$FASTTEST_OUTPUT/failed-parallel-tests.txt") # We will rerun sequentially any tests that have failed during parallel run. # They might have failed because there was some interference from other tests @@ -206,19 +259,16 @@ readarray -t FAILED_TESTS < <(awk '/FAIL|TIMEOUT|ERROR/ { print substr($3, 1, le # explicit instead of guessing. if [[ -n "${FAILED_TESTS[*]}" ]] then - kill_clickhouse + stop_server ||: # Clean the data so that there is no interference from the previous test run. - rm -rf /var/lib/clickhouse ||: - mkdir /var/lib/clickhouse + rm -rf "$FASTTEST_DATA"/{meta,}data ||: - clickhouse-server --config /etc/clickhouse-server/config.xml --daemon - - wait_for_server_start + start_server echo "Going to run again: ${FAILED_TESTS[*]}" - clickhouse-test --no-long --testname --shard --zookeeper "${FAILED_TESTS[@]}" 2>&1 | ts '%Y-%m-%d %H:%M:%S' | tee -a /test_output/test_log.txt + clickhouse-test --no-long --testname --shard --zookeeper "${FAILED_TESTS[@]}" 2>&1 | ts '%Y-%m-%d %H:%M:%S' | tee -a "$FASTTEST_OUTPUT/test_log.txt" else echo "No failed tests" fi @@ -228,20 +278,33 @@ case "$stage" in "") ls -la ;& - "clone_root") clone_root # Pass control to the script from cloned sources, unless asked otherwise. if ! [ -v FASTTEST_LOCAL_SCRIPT ] then - stage=run "$CLICKHOUSE_DIR/docker/test/fasttest/run.sh" + # 'run' is deprecated, used for compatibility with old scripts. + # Replace with 'clone_submodules' after Nov 1, 2020. + stage=run "$FASTTEST_SOURCE/docker/test/fasttest/run.sh" exit $? fi ;& - "run") - run + # A deprecated stage that is called by old script and equivalent to everything + # after cloning root, starting with cloning submodules. + ;& +"clone_submodules") + clone_submodules | ts '%Y-%m-%d %H:%M:%S' | tee "$FASTTEST_OUTPUT/submodule_log.txt" + ;& +"build") + build + ;& +"configure") + configure + ;& +"run_tests") + run_tests ;& esac From 6062c55569685b991cfaf246f7129ce63b1fbfbf Mon Sep 17 00:00:00 2001 From: Alexander Kuzmenkov Date: Tue, 29 Sep 2020 21:55:06 +0300 Subject: [PATCH 211/321] fasttest fixup --- docker/test/fasttest/run.sh | 27 ++++++++++++-------- tests/config/install.sh | 50 ++++++++++++++++++------------------- 2 files changed, 41 insertions(+), 36 deletions(-) diff --git a/docker/test/fasttest/run.sh b/docker/test/fasttest/run.sh index b93aa0ce5cd..6a3f16d52f6 100755 --- a/docker/test/fasttest/run.sh +++ b/docker/test/fasttest/run.sh @@ -26,15 +26,9 @@ server_pid=none function stop_server { - if ! kill -- "$server_pid" - then - echo "The server we started ($server_pid) is not present, won't do anything" - return 0 - fi - for _ in {1..60} do - if ! kill -- "$server_pid" ; then break ; fi + if ! pkill -f "clickhouse-server" && ! kill -- "$server_pid" ; then break ; fi sleep 1 done @@ -45,6 +39,8 @@ function stop_server echo "Failed to kill the ClickHouse server pid '$server_pid'" return 1 fi + + server_pid=none } function start_server @@ -126,10 +122,9 @@ git submodule foreach git clean -xfd ) } - -function build +function run_cmake { -CMAKE_LIBS_CONFIG=(-DENABLE_LIBRARIES=0 -DENABLE_TESTS=0 -DENABLE_UTILS=0 -DENABLE_EMBEDDED_COMPILER=0 -DENABLE_THINLTO=0 -DUSE_UNWIND=1) +CMAKE_LIBS_CONFIG=("-DENABLE_LIBRARIES=0" "-DENABLE_TESTS=0" "-DENABLE_UTILS=0" "-DENABLE_EMBEDDED_COMPILER=0" "-DENABLE_THINLTO=0" "-DUSE_UNWIND=1") # TODO remove this? we don't use ccache anyway. An option would be to download it # from S3 simultaneously with cloning. @@ -143,13 +138,20 @@ ccache --show-stats ||: ccache --zero-stats ||: mkdir "$FASTTEST_BUILD" ||: + ( cd "$FASTTEST_BUILD" cmake "$FASTTEST_SOURCE" -DCMAKE_CXX_COMPILER=clang++-10 -DCMAKE_C_COMPILER=clang-10 "${CMAKE_LIBS_CONFIG[@]}" "${FASTTEST_CMAKE_FLAGS[@]}" | ts '%Y-%m-%d %H:%M:%S' | tee "$FASTTEST_OUTPUT/cmake_log.txt" -time ninja clickhouse-bundle | ts '%Y-%m-%d %H:%M:%S' | tee "$FASTTEST_OUTPUT/build_log.txt" ) +} +function build +{ +( +cd "$FASTTEST_BUILD" +time ninja clickhouse-bundle | ts '%Y-%m-%d %H:%M:%S' | tee "$FASTTEST_OUTPUT/build_log.txt" ccache --show-stats ||: +) } function configure @@ -297,6 +299,9 @@ case "$stage" in "clone_submodules") clone_submodules | ts '%Y-%m-%d %H:%M:%S' | tee "$FASTTEST_OUTPUT/submodule_log.txt" ;& +"run_cmake") + run_cmake + ;& "build") build ;& diff --git a/tests/config/install.sh b/tests/config/install.sh index 0f33854ef95..ef9604904e7 100755 --- a/tests/config/install.sh +++ b/tests/config/install.sh @@ -15,40 +15,40 @@ mkdir -p $DEST_SERVER_PATH/config.d/ mkdir -p $DEST_SERVER_PATH/users.d/ mkdir -p $DEST_CLIENT_PATH -ln -s $SRC_PATH/config.d/zookeeper.xml $DEST_SERVER_PATH/config.d/ -ln -s $SRC_PATH/config.d/listen.xml $DEST_SERVER_PATH/config.d/ -ln -s $SRC_PATH/config.d/part_log.xml $DEST_SERVER_PATH/config.d/ -ln -s $SRC_PATH/config.d/text_log.xml $DEST_SERVER_PATH/config.d/ -ln -s $SRC_PATH/config.d/metric_log.xml $DEST_SERVER_PATH/config.d/ -ln -s $SRC_PATH/config.d/custom_settings_prefixes.xml $DEST_SERVER_PATH/config.d/ -ln -s $SRC_PATH/config.d/macros.xml $DEST_SERVER_PATH/config.d/ -ln -s $SRC_PATH/config.d/disks.xml $DEST_SERVER_PATH/config.d/ -ln -s $SRC_PATH/config.d/secure_ports.xml $DEST_SERVER_PATH/config.d/ -ln -s $SRC_PATH/config.d/clusters.xml $DEST_SERVER_PATH/config.d/ -ln -s $SRC_PATH/config.d/graphite.xml $DEST_SERVER_PATH/config.d/ -ln -s $SRC_PATH/config.d/database_atomic.xml $DEST_SERVER_PATH/config.d/ -ln -s $SRC_PATH/users.d/log_queries.xml $DEST_SERVER_PATH/users.d/ -ln -s $SRC_PATH/users.d/readonly.xml $DEST_SERVER_PATH/users.d/ -ln -s $SRC_PATH/users.d/access_management.xml $DEST_SERVER_PATH/users.d/ +ln -sf $SRC_PATH/config.d/zookeeper.xml $DEST_SERVER_PATH/config.d/ +ln -sf $SRC_PATH/config.d/listen.xml $DEST_SERVER_PATH/config.d/ +ln -sf $SRC_PATH/config.d/part_log.xml $DEST_SERVER_PATH/config.d/ +ln -sf $SRC_PATH/config.d/text_log.xml $DEST_SERVER_PATH/config.d/ +ln -sf $SRC_PATH/config.d/metric_log.xml $DEST_SERVER_PATH/config.d/ +ln -sf $SRC_PATH/config.d/custom_settings_prefixes.xml $DEST_SERVER_PATH/config.d/ +ln -sf $SRC_PATH/config.d/macros.xml $DEST_SERVER_PATH/config.d/ +ln -sf $SRC_PATH/config.d/disks.xml $DEST_SERVER_PATH/config.d/ +ln -sf $SRC_PATH/config.d/secure_ports.xml $DEST_SERVER_PATH/config.d/ +ln -sf $SRC_PATH/config.d/clusters.xml $DEST_SERVER_PATH/config.d/ +ln -sf $SRC_PATH/config.d/graphite.xml $DEST_SERVER_PATH/config.d/ +ln -sf $SRC_PATH/config.d/database_atomic.xml $DEST_SERVER_PATH/config.d/ +ln -sf $SRC_PATH/users.d/log_queries.xml $DEST_SERVER_PATH/users.d/ +ln -sf $SRC_PATH/users.d/readonly.xml $DEST_SERVER_PATH/users.d/ +ln -sf $SRC_PATH/users.d/access_management.xml $DEST_SERVER_PATH/users.d/ -ln -s $SRC_PATH/ints_dictionary.xml $DEST_SERVER_PATH/ -ln -s $SRC_PATH/strings_dictionary.xml $DEST_SERVER_PATH/ -ln -s $SRC_PATH/decimals_dictionary.xml $DEST_SERVER_PATH/ -ln -s $SRC_PATH/executable_dictionary.xml $DEST_SERVER_PATH/ +ln -sf $SRC_PATH/ints_dictionary.xml $DEST_SERVER_PATH/ +ln -sf $SRC_PATH/strings_dictionary.xml $DEST_SERVER_PATH/ +ln -sf $SRC_PATH/decimals_dictionary.xml $DEST_SERVER_PATH/ +ln -sf $SRC_PATH/executable_dictionary.xml $DEST_SERVER_PATH/ -ln -s $SRC_PATH/server.key $DEST_SERVER_PATH/ -ln -s $SRC_PATH/server.crt $DEST_SERVER_PATH/ -ln -s $SRC_PATH/dhparam.pem $DEST_SERVER_PATH/ +ln -sf $SRC_PATH/server.key $DEST_SERVER_PATH/ +ln -sf $SRC_PATH/server.crt $DEST_SERVER_PATH/ +ln -sf $SRC_PATH/dhparam.pem $DEST_SERVER_PATH/ # Retain any pre-existing config and allow ClickHouse to load it if required -ln -s --backup=simple --suffix=_original.xml \ +ln -sf --backup=simple --suffix=_original.xml \ $SRC_PATH/config.d/query_masking_rules.xml $DEST_SERVER_PATH/config.d/ if [[ -n "$USE_POLYMORPHIC_PARTS" ]] && [[ "$USE_POLYMORPHIC_PARTS" -eq 1 ]]; then - ln -s $SRC_PATH/config.d/polymorphic_parts.xml $DEST_SERVER_PATH/config.d/ + ln -sf $SRC_PATH/config.d/polymorphic_parts.xml $DEST_SERVER_PATH/config.d/ fi if [[ -n "$USE_DATABASE_ORDINARY" ]] && [[ "$USE_DATABASE_ORDINARY" -eq 1 ]]; then - ln -s $SRC_PATH/users.d/database_ordinary.xml $DEST_SERVER_PATH/users.d/ + ln -sf $SRC_PATH/users.d/database_ordinary.xml $DEST_SERVER_PATH/users.d/ fi ln -sf $SRC_PATH/client_config.xml $DEST_CLIENT_PATH/config.xml From f8047102fe7df51021ee8cb316813bc63434515f Mon Sep 17 00:00:00 2001 From: Alexander Kuzmenkov Date: Tue, 29 Sep 2020 22:06:45 +0300 Subject: [PATCH 212/321] fixup --- docker/test/performance-comparison/perf.py | 13 +++++++------ 1 file changed, 7 insertions(+), 6 deletions(-) diff --git a/docker/test/performance-comparison/perf.py b/docker/test/performance-comparison/perf.py index 99f395f79b2..fb14f050cfa 100755 --- a/docker/test/performance-comparison/perf.py +++ b/docker/test/performance-comparison/perf.py @@ -130,10 +130,10 @@ if not args.long: sys.exit(0) # Print report threshold for the test if it is set. -ignored_change = 0.05 +ignored_relative_change = 0.05 if 'max_ignored_relative_change' in root.attrib: - ignored_change = float(root.attrib["max_ignored_relative_change"]) - print(f'report-threshold\t{ignored_change}') + ignored_relative_change = float(root.attrib["max_ignored_relative_change"]) + print(f'report-threshold\t{ignored_relative_change}') reportStageEnd('before-connect') @@ -368,9 +368,10 @@ for query_index in queries_to_run: continue pvalue = stats.ttest_ind(all_server_times[0], all_server_times[1], equal_var = False).pvalue - diff = statistics.median(all_server_times[1]) - statistics.median(all_server_times[0]) - print(f'diff\t{diff}\t{pvalue}') - if abs(diff) < ignored_change or pvalue > 0.05: + median = [statistics.median(t) for t in all_server_times] + relative_diff = (median[1] - median[0]) / max(median) + print(f'diff\t{relative_diff}\t{pvalue}') + if abs(relative_diff) < ignored_relative_change or pvalue > 0.05: continue # Perform profile runs for fixed amount of time. Don't limit the number From a2c70c4611664da7129eb78bdd1dc4fdcf092853 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 29 Sep 2020 22:11:40 +0300 Subject: [PATCH 213/321] Destroy resurces captured by lambda after ThreadFromGlobalPool::join(). --- src/Common/ThreadPool.h | 29 ++++++++++++----------------- 1 file changed, 12 insertions(+), 17 deletions(-) diff --git a/src/Common/ThreadPool.h b/src/Common/ThreadPool.h index ecb03dd9659..4f08427b4b9 100644 --- a/src/Common/ThreadPool.h +++ b/src/Common/ThreadPool.h @@ -163,24 +163,19 @@ public: func = std::forward(func), args = std::make_tuple(std::forward(args)...)] { - try - { - /// Move capture in order to destroy it before `state->set()`. - /// It will guarantee that after ThreadFromGlobalPool::join all resources are destroyed. - auto function = std::move(func); - auto arguments = std::move(args); + SCOPE_EXIT({ + /// Destroy function before exit. + /// It will guarantee that after ThreadFromGlobalPool::join all captured params are destroyed. + func = {}; + state->set(); + }); - /// Thread status holds raw pointer on query context, thus it always must be destroyed - /// before sending signal that permits to join this thread. - DB::ThreadStatus thread_status; - std::apply(function, arguments); - } - catch (...) - { - state->set(); - throw; - } - state->set(); + auto arguments = std::move(args); + + /// Thread status holds raw pointer on query context, thus it always must be destroyed + /// before sending signal that permits to join this thread. + DB::ThreadStatus thread_status; + std::apply(func, arguments); }); } From 3cc53ed65bd4bd6ea4b9d61eb8c5a8e83b88dd97 Mon Sep 17 00:00:00 2001 From: Alexander Kuzmenkov Date: Tue, 29 Sep 2020 22:20:35 +0300 Subject: [PATCH 214/321] boop the CI From b204bcd4b464d6ef2ed2a78112c38bad43b28aca Mon Sep 17 00:00:00 2001 From: Alexander Kuzmenkov Date: Tue, 29 Sep 2020 23:09:52 +0300 Subject: [PATCH 215/321] fasttest fixup 2 --- docker/test/fasttest/run.sh | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/docker/test/fasttest/run.sh b/docker/test/fasttest/run.sh index 6a3f16d52f6..6898000185d 100755 --- a/docker/test/fasttest/run.sh +++ b/docker/test/fasttest/run.sh @@ -306,7 +306,9 @@ case "$stage" in build ;& "configure") - configure + # The `install_log.txt` is also needed for compatibility with old CI task -- + # if there is no log, it will decide that build failed. + configure | ts '%Y-%m-%d %H:%M:%S' | tee "$FASTTEST_OUTPUT/install_log.txt" ;& "run_tests") run_tests From 727289cb09b4e0e57008b7a8accf57cd7f8ca397 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 29 Sep 2020 23:43:02 +0300 Subject: [PATCH 216/321] Destroy resurces captured by lambda after ThreadFromGlobalPool::join(). --- src/Common/ThreadPool.h | 9 +++++---- 1 file changed, 5 insertions(+), 4 deletions(-) diff --git a/src/Common/ThreadPool.h b/src/Common/ThreadPool.h index 4f08427b4b9..77e0876d1cf 100644 --- a/src/Common/ThreadPool.h +++ b/src/Common/ThreadPool.h @@ -11,6 +11,7 @@ #include #include +#include /** Very simple thread pool similar to boost::threadpool. @@ -161,21 +162,21 @@ public: GlobalThreadPool::instance().scheduleOrThrow([ state = state, func = std::forward(func), - args = std::make_tuple(std::forward(args)...)] + args = std::make_tuple(std::forward(args)...)]() mutable { SCOPE_EXIT({ - /// Destroy function before exit. + /// Destroy function before exit. /// It will guarantee that after ThreadFromGlobalPool::join all captured params are destroyed. - func = {}; state->set(); }); + auto function = std::move(func); auto arguments = std::move(args); /// Thread status holds raw pointer on query context, thus it always must be destroyed /// before sending signal that permits to join this thread. DB::ThreadStatus thread_status; - std::apply(func, arguments); + std::apply(function, arguments); }); } From 218a947e05eeb021e3ff07320a8bd4f67d688661 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 30 Sep 2020 01:20:53 +0300 Subject: [PATCH 217/321] Suppress clang-tidy diagnostics --- contrib/poco | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/poco b/contrib/poco index 297fc905e16..757d947235b 160000 --- a/contrib/poco +++ b/contrib/poco @@ -1 +1 @@ -Subproject commit 297fc905e166392156f83b96aaa5f44e8a6a35c4 +Subproject commit 757d947235b307675cff964f29b19d388140a9eb From b658f348a78fa8588584803010822ad8f96e9c04 Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Wed, 30 Sep 2020 01:23:42 +0300 Subject: [PATCH 218/321] Update MergeTreeDataPartWriterWide.cpp --- src/Storages/MergeTree/MergeTreeDataPartWriterWide.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeDataPartWriterWide.cpp b/src/Storages/MergeTree/MergeTreeDataPartWriterWide.cpp index 03a4152a447..c15c39e7b7f 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWriterWide.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartWriterWide.cpp @@ -46,10 +46,10 @@ void MergeTreeDataPartWriterWide::addStreams( return; CompressionCodecPtr compression_codec; - /// If we can use special codec than just get it + /// If we can use special codec then just get it if (IDataType::isSpecialCompressionAllowed(substream_path)) compression_codec = CompressionCodecFactory::instance().get(effective_codec_desc, &substream_type, default_codec); - else /// otherwise return only generic codecs and don't use info about data_type + else /// otherwise return only generic codecs and don't use info about the data_type compression_codec = CompressionCodecFactory::instance().get(effective_codec_desc, nullptr, default_codec, true); column_streams[stream_name] = std::make_unique( From e156e6115e064ddfa613f6277211f607bd432500 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Wed, 30 Sep 2020 01:30:34 +0300 Subject: [PATCH 219/321] add setting --- src/Core/Settings.h | 1 + src/Databases/DatabaseAtomic.cpp | 2 +- src/Databases/DatabaseAtomic.h | 2 +- src/Interpreters/InterpreterDropQuery.cpp | 3 +++ 4 files changed, 6 insertions(+), 2 deletions(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 9449cd571a1..8c61c73e201 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -352,6 +352,7 @@ class IColumn; \ M(DefaultDatabaseEngine, default_database_engine, DefaultDatabaseEngine::Atomic, "Default database engine.", 0) \ M(Bool, show_table_uuid_in_table_create_query_if_not_nil, false, "For tables in databases with Engine=Atomic show UUID of the table in its CREATE query.", 0) \ + M(Bool, database_atomic_wait_for_drop_and_detach_synchronously, false, "When executing DROP or DETACH TABLE in Atomic database, wait for table data to be finally dropped or detached.", 0) \ M(Bool, enable_scalar_subquery_optimization, true, "If it is set to true, prevent scalar subqueries from (de)serializing large scalar values and possibly avoid running the same subquery more than once.", 0) \ M(Bool, optimize_trivial_count_query, true, "Process trivial 'SELECT count() FROM table' query from metadata.", 0) \ M(UInt64, mutations_sync, 0, "Wait for synchronous execution of ALTER TABLE UPDATE/DELETE queries (mutations). 0 - execute asynchronously. 1 - wait current server. 2 - wait all replicas if they exist.", 0) \ diff --git a/src/Databases/DatabaseAtomic.cpp b/src/Databases/DatabaseAtomic.cpp index fa4511cd104..46d56e97436 100644 --- a/src/Databases/DatabaseAtomic.cpp +++ b/src/Databases/DatabaseAtomic.cpp @@ -519,7 +519,7 @@ void DatabaseAtomic::waitDetachedTableNotInUse(const UUID & uuid) if (detached_tables.count(uuid) == 0) return; } - std::this_thread::sleep_for(std::chrono::milliseconds(500)); + std::this_thread::sleep_for(std::chrono::milliseconds(100)); } } diff --git a/src/Databases/DatabaseAtomic.h b/src/Databases/DatabaseAtomic.h index 8fa0d1837e7..2d90ed96f1d 100644 --- a/src/Databases/DatabaseAtomic.h +++ b/src/Databases/DatabaseAtomic.h @@ -51,7 +51,7 @@ public: void loadStoredObjects(Context & context, bool has_force_restore_data_flag, bool force_attach) override; /// Atomic database cannot be detached if there is detached table which still in use - void assertCanBeDetached(bool cleenup); + void assertCanBeDetached(bool cleanup); UUID tryGetTableUUID(const String & table_name) const override; diff --git a/src/Interpreters/InterpreterDropQuery.cpp b/src/Interpreters/InterpreterDropQuery.cpp index b6d61b6c969..c70431e5238 100644 --- a/src/Interpreters/InterpreterDropQuery.cpp +++ b/src/Interpreters/InterpreterDropQuery.cpp @@ -41,6 +41,9 @@ BlockIO InterpreterDropQuery::execute() if (!drop.cluster.empty()) return executeDDLQueryOnCluster(query_ptr, context, getRequiredAccessForDDLOnCluster()); + if (context.getSettingsRef().database_atomic_wait_for_drop_and_detach_synchronously) + drop.no_delay = true; + if (!drop.table.empty()) { if (!drop.is_dictionary) From 78dbf10c7bbd822c404d2262b68e5c4c546d61fa Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 30 Sep 2020 03:00:45 +0300 Subject: [PATCH 220/321] Fix exception safety in ShellCommand --- src/Common/ShellCommand.cpp | 12 ++++-------- src/Common/ShellCommand.h | 2 +- src/IO/ReadBufferFromFile.cpp | 3 ++- src/IO/ReadBufferFromFile.h | 5 ++++- src/IO/WriteBufferFromFile.cpp | 3 ++- src/IO/WriteBufferFromFile.h | 2 +- 6 files changed, 14 insertions(+), 13 deletions(-) diff --git a/src/Common/ShellCommand.cpp b/src/Common/ShellCommand.cpp index bbb8801f190..992419adb6d 100644 --- a/src/Common/ShellCommand.cpp +++ b/src/Common/ShellCommand.cpp @@ -35,12 +35,14 @@ namespace ErrorCodes extern const int CANNOT_CREATE_CHILD_PROCESS; } -ShellCommand::ShellCommand(pid_t pid_, int in_fd_, int out_fd_, int err_fd_, bool terminate_in_destructor_) +ShellCommand::ShellCommand(pid_t pid_, int & in_fd_, int & out_fd_, int & err_fd_, bool terminate_in_destructor_) : pid(pid_) , terminate_in_destructor(terminate_in_destructor_) , in(in_fd_) , out(out_fd_) - , err(err_fd_) {} + , err(err_fd_) +{ +} Poco::Logger * ShellCommand::getLogger() { @@ -144,12 +146,6 @@ std::unique_ptr ShellCommand::executeImpl( pid, pipe_stdin.fds_rw[1], pipe_stdout.fds_rw[0], pipe_stderr.fds_rw[0], terminate_in_destructor)); LOG_TRACE(getLogger(), "Started shell command '{}' with pid {}", filename, pid); - - /// Now the ownership of the file descriptors is passed to the result. - pipe_stdin.fds_rw[1] = -1; - pipe_stdout.fds_rw[0] = -1; - pipe_stderr.fds_rw[0] = -1; - return res; } diff --git a/src/Common/ShellCommand.h b/src/Common/ShellCommand.h index e609a3fc85b..1c681227efd 100644 --- a/src/Common/ShellCommand.h +++ b/src/Common/ShellCommand.h @@ -30,7 +30,7 @@ private: bool wait_called = false; bool terminate_in_destructor; - ShellCommand(pid_t pid_, int in_fd_, int out_fd_, int err_fd_, bool terminate_in_destructor_); + ShellCommand(pid_t pid_, int & in_fd_, int & out_fd_, int & err_fd_, bool terminate_in_destructor_); static Poco::Logger * getLogger(); diff --git a/src/IO/ReadBufferFromFile.cpp b/src/IO/ReadBufferFromFile.cpp index 226615c757e..d0f94441622 100644 --- a/src/IO/ReadBufferFromFile.cpp +++ b/src/IO/ReadBufferFromFile.cpp @@ -54,7 +54,7 @@ ReadBufferFromFile::ReadBufferFromFile( ReadBufferFromFile::ReadBufferFromFile( - int fd_, + int & fd_, const std::string & original_file_name, size_t buf_size, char * existing_memory, @@ -63,6 +63,7 @@ ReadBufferFromFile::ReadBufferFromFile( ReadBufferFromFileDescriptor(fd_, buf_size, existing_memory, alignment), file_name(original_file_name.empty() ? "(fd = " + toString(fd_) + ")" : original_file_name) { + fd_ = -1; } diff --git a/src/IO/ReadBufferFromFile.h b/src/IO/ReadBufferFromFile.h index a621553daae..cebda605b21 100644 --- a/src/IO/ReadBufferFromFile.h +++ b/src/IO/ReadBufferFromFile.h @@ -29,7 +29,10 @@ public: char * existing_memory = nullptr, size_t alignment = 0); /// Use pre-opened file descriptor. - ReadBufferFromFile(int fd, const std::string & original_file_name = {}, size_t buf_size = DBMS_DEFAULT_BUFFER_SIZE, + ReadBufferFromFile( + int & fd, /// Will be set to -1 if constructor didn't throw and ownership of file descriptor is passed to the object. + const std::string & original_file_name = {}, + size_t buf_size = DBMS_DEFAULT_BUFFER_SIZE, char * existing_memory = nullptr, size_t alignment = 0); ~ReadBufferFromFile() override; diff --git a/src/IO/WriteBufferFromFile.cpp b/src/IO/WriteBufferFromFile.cpp index 4ade2e2c971..aeed4862fba 100644 --- a/src/IO/WriteBufferFromFile.cpp +++ b/src/IO/WriteBufferFromFile.cpp @@ -59,7 +59,7 @@ WriteBufferFromFile::WriteBufferFromFile( /// Use pre-opened file descriptor. WriteBufferFromFile::WriteBufferFromFile( - int fd_, + int & fd_, const std::string & original_file_name, size_t buf_size, char * existing_memory, @@ -68,6 +68,7 @@ WriteBufferFromFile::WriteBufferFromFile( WriteBufferFromFileDescriptor(fd_, buf_size, existing_memory, alignment), file_name(original_file_name.empty() ? "(fd = " + toString(fd_) + ")" : original_file_name) { + fd_ = -1; } diff --git a/src/IO/WriteBufferFromFile.h b/src/IO/WriteBufferFromFile.h index 12092b742e5..77530c323d2 100644 --- a/src/IO/WriteBufferFromFile.h +++ b/src/IO/WriteBufferFromFile.h @@ -39,7 +39,7 @@ public: /// Use pre-opened file descriptor. WriteBufferFromFile( - int fd, + int & fd, /// Will be set to -1 if constructor didn't throw and ownership of file descriptor is passed to the object. const std::string & original_file_name = {}, size_t buf_size = DBMS_DEFAULT_BUFFER_SIZE, char * existing_memory = nullptr, From c7fe3233732245cb094727fd8ec3c72c6c04734b Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Wed, 30 Sep 2020 03:45:57 +0300 Subject: [PATCH 221/321] Update 01497_now_support_timezone.sql --- tests/queries/0_stateless/01497_now_support_timezone.sql | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/01497_now_support_timezone.sql b/tests/queries/0_stateless/01497_now_support_timezone.sql index cc5ae77e94d..b1e9bad58c5 100644 --- a/tests/queries/0_stateless/01497_now_support_timezone.sql +++ b/tests/queries/0_stateless/01497_now_support_timezone.sql @@ -1 +1 @@ -SELECT toHour(now('Asia/Shanghai')) - toHour(now('UTC')); +SELECT (toHour(now('Asia/Shanghai')) - toHour(now('UTC'))) IN (8, -16); From 3fd272f0f0109ff3b7751b712e5f6e4d223dfb1b Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Wed, 30 Sep 2020 03:46:06 +0300 Subject: [PATCH 222/321] Update 01497_now_support_timezone.reference --- tests/queries/0_stateless/01497_now_support_timezone.reference | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/01497_now_support_timezone.reference b/tests/queries/0_stateless/01497_now_support_timezone.reference index 45a4fb75db8..d00491fd7e5 100644 --- a/tests/queries/0_stateless/01497_now_support_timezone.reference +++ b/tests/queries/0_stateless/01497_now_support_timezone.reference @@ -1 +1 @@ -8 +1 From c857b5dd5dce6705b4706e06ba74f87ae04836a2 Mon Sep 17 00:00:00 2001 From: feng lv Date: Wed, 30 Sep 2020 09:50:23 +0800 Subject: [PATCH 223/321] fix --- src/Processors/Formats/Impl/PrettyBlockOutputFormat.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Processors/Formats/Impl/PrettyBlockOutputFormat.cpp b/src/Processors/Formats/Impl/PrettyBlockOutputFormat.cpp index ec4b3286448..43fff9a66ff 100644 --- a/src/Processors/Formats/Impl/PrettyBlockOutputFormat.cpp +++ b/src/Processors/Formats/Impl/PrettyBlockOutputFormat.cpp @@ -34,8 +34,8 @@ void PrettyBlockOutputFormat::calculateWidths( { size_t num_rows = std::min(chunk.getNumRows(), format_settings.pretty.max_rows); - auto max_row_number = std::to_string(num_rows); - row_number_width = max_row_number.size() + 2; + /// len(num_rows) + len(". ") + row_number_width = std::floor(std::log10(num_rows)) + 3; size_t num_columns = chunk.getNumColumns(); const auto & columns = chunk.getColumns(); From 61d5f89235698f9b57c1ab8f5d8e0461d45f880d Mon Sep 17 00:00:00 2001 From: feng lv Date: Wed, 30 Sep 2020 10:57:04 +0800 Subject: [PATCH 224/321] update document --- docs/en/sql-reference/functions/date-time-functions.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/functions/date-time-functions.md b/docs/en/sql-reference/functions/date-time-functions.md index c5986cd9ebc..910937b3fa9 100644 --- a/docs/en/sql-reference/functions/date-time-functions.md +++ b/docs/en/sql-reference/functions/date-time-functions.md @@ -357,7 +357,7 @@ SELECT date_trunc('hour', now()) ## now {#now} -Accepts zero arguments and returns the current time at one of the moments of request execution. +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. ## today {#today} From 4c6e0108a2b65eea030bd567f0cccb2c11406107 Mon Sep 17 00:00:00 2001 From: "dependabot-preview[bot]" <27856297+dependabot-preview[bot]@users.noreply.github.com> Date: Wed, 30 Sep 2020 06:08:53 +0000 Subject: [PATCH 225/321] Bump mkdocs-macros-plugin from 0.4.13 to 0.4.17 in /docs/tools Bumps [mkdocs-macros-plugin](https://github.com/fralau/mkdocs_macros_plugin) from 0.4.13 to 0.4.17. - [Release notes](https://github.com/fralau/mkdocs_macros_plugin/releases) - [Commits](https://github.com/fralau/mkdocs_macros_plugin/commits) Signed-off-by: dependabot-preview[bot] --- docs/tools/requirements.txt | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/tools/requirements.txt b/docs/tools/requirements.txt index e31e43b99cd..68a72a6a785 100644 --- a/docs/tools/requirements.txt +++ b/docs/tools/requirements.txt @@ -18,7 +18,7 @@ Markdown==3.2.1 MarkupSafe==1.1.1 mkdocs==1.1.2 mkdocs-htmlproofer-plugin==0.0.3 -mkdocs-macros-plugin==0.4.13 +mkdocs-macros-plugin==0.4.17 nltk==3.5 nose==1.3.7 protobuf==3.13.0 From bc36625b9dcf695022795f6ea9b1d111be4f3e76 Mon Sep 17 00:00:00 2001 From: Ivan Blinkov Date: Wed, 30 Sep 2020 10:51:32 +0300 Subject: [PATCH 226/321] Update README.md --- README.md | 1 - 1 file changed, 1 deletion(-) diff --git a/README.md b/README.md index 6b909dd710c..5753118eb4d 100644 --- a/README.md +++ b/README.md @@ -17,5 +17,4 @@ ClickHouse is an open-source column-oriented database management system that all ## Upcoming Events -* [ClickHouse for Edge Analytics](https://ones2020.sched.com/event/bWPs) on September 29, 2020. * [ClickHouse online meetup (in Russian)](https://clck.ru/R2zB9) on October 1, 2020. From 0e412686bc54d21dad8a218478c8f6555650ceb4 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Wed, 30 Sep 2020 11:25:22 +0300 Subject: [PATCH 227/321] Better comments. --- src/Common/ThreadPool.h | 10 ++++------ 1 file changed, 4 insertions(+), 6 deletions(-) diff --git a/src/Common/ThreadPool.h b/src/Common/ThreadPool.h index 77e0876d1cf..eb385547fa8 100644 --- a/src/Common/ThreadPool.h +++ b/src/Common/ThreadPool.h @@ -162,14 +162,12 @@ public: GlobalThreadPool::instance().scheduleOrThrow([ state = state, func = std::forward(func), - args = std::make_tuple(std::forward(args)...)]() mutable + args = std::make_tuple(std::forward(args)...)]() mutable /// mutable is needed to destroy capture { - SCOPE_EXIT({ - /// Destroy function before exit. - /// It will guarantee that after ThreadFromGlobalPool::join all captured params are destroyed. - state->set(); - }); + SCOPE_EXIT(state->set()); + /// This moves are needed to destroy function and arguments before exit. + /// It will guarantee that after ThreadFromGlobalPool::join all captured params are destroyed. auto function = std::move(func); auto arguments = std::move(args); From 6a4beb359b3eed12391a7d198e66647eee6ec10c Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Wed, 30 Sep 2020 12:51:07 +0300 Subject: [PATCH 228/321] Add experimental codecov.yml --- .github/codecov.yml | 17 +++++++++++++++++ 1 file changed, 17 insertions(+) create mode 100644 .github/codecov.yml diff --git a/.github/codecov.yml b/.github/codecov.yml new file mode 100644 index 00000000000..9b75efc791d --- /dev/null +++ b/.github/codecov.yml @@ -0,0 +1,17 @@ +codecov: + max_report_age: off + strict_yaml_branch: "master" + +ignore: + - "contrib" + - "docs" + - "benchmark" + - "tests" + - "docker" + - "debian" + - "cmake" + +comment: false + +github_checks: + annotations: false \ No newline at end of file From 698768add00c19321c7fc7491a156cfc5273bf3f Mon Sep 17 00:00:00 2001 From: Vladimir Chebotarev Date: Mon, 28 Sep 2020 10:37:24 +0300 Subject: [PATCH 229/321] Fixed `format` argument in S3 storage when compression method is selected. --- src/Storages/StorageS3.cpp | 9 +++++++-- 1 file changed, 7 insertions(+), 2 deletions(-) diff --git a/src/Storages/StorageS3.cpp b/src/Storages/StorageS3.cpp index 6d17a17409f..b4dfa6dcf66 100644 --- a/src/Storages/StorageS3.cpp +++ b/src/Storages/StorageS3.cpp @@ -349,8 +349,6 @@ void registerStorageS3Impl(const String & name, StorageFactory & factory) Poco::URI uri (url); S3::URI s3_uri (uri); - String format_name = engine_args[engine_args.size() - 1]->as().value.safeGet(); - String access_key_id; String secret_access_key; if (engine_args.size() >= 4) @@ -362,10 +360,17 @@ void registerStorageS3Impl(const String & name, StorageFactory & factory) UInt64 min_upload_part_size = args.local_context.getSettingsRef().s3_min_upload_part_size; String compression_method; + String format_name; if (engine_args.size() == 3 || engine_args.size() == 5) + { compression_method = engine_args.back()->as().value.safeGet(); + format_name = engine_args[engine_args.size() - 2]->as().value.safeGet(); + } else + { compression_method = "auto"; + format_name = engine_args.back()->as().value.safeGet(); + } return StorageS3::create(s3_uri, access_key_id, secret_access_key, args.table_id, format_name, min_upload_part_size, args.columns, args.constraints, args.context); }, From b7a6f15f1943d77388090c3561cacc372d56bb5b Mon Sep 17 00:00:00 2001 From: Vladimir Chebotarev Date: Tue, 29 Sep 2020 02:30:41 +0300 Subject: [PATCH 230/321] Added test for GZIP in S3 storage. --- tests/integration/test_storage_s3/test.py | 82 +++++++++++++++++++++++ 1 file changed, 82 insertions(+) diff --git a/tests/integration/test_storage_s3/test.py b/tests/integration/test_storage_s3/test.py index e39296525d0..cc2d187558c 100644 --- a/tests/integration/test_storage_s3/test.py +++ b/tests/integration/test_storage_s3/test.py @@ -1,7 +1,9 @@ +import gzip import json import logging import os import random +import StringIO import threading import helpers.client @@ -57,6 +59,11 @@ def prepare_s3_bucket(cluster): minio_client.make_bucket(cluster.minio_restricted_bucket) +def put_s3_file_content(cluster, bucket, filename, data): + buf = StringIO.StringIO(data) + cluster.minio_client.put_object(bucket, filename, buf, len(data)) + + # Returns content of given S3 file as string. def get_s3_file_content(cluster, bucket, filename): # type: (ClickHouseCluster, str) -> str @@ -344,3 +351,78 @@ def test_infinite_redirect(cluster): exception_raised = True finally: assert exception_raised + + +def test_storage_s3_get_gzip(cluster): + bucket = cluster.minio_bucket + instance = cluster.instances["dummy"] + filename = "test_get_gzip.bin" + name = "test_get_gzip" + data = [ + "Sophia Intrieri,55", + "Jack Taylor,71", + "Christopher Silva,66", + "Clifton Purser,35", + "Richard Aceuedo,43", + "Lisa Hensley,31", + "Alice Wehrley,1", + "Mary Farmer,47", + "Samara Ramirez,19", + "Shirley Lloyd,51", + "Santos Cowger,0", + "Richard Mundt,88", + "Jerry Gonzalez,15", + "Angela James,10", + "Norman Ortega,33", + "" + ] + buf = StringIO.StringIO() + compressed = gzip.GzipFile(fileobj=buf, mode="wb") + compressed.write("\n".join(data)) + compressed.close() + put_s3_file_content(cluster, bucket, filename, buf.getvalue()) + + try: + run_query(instance, "CREATE TABLE {} (name String, id UInt32) ENGINE = S3('http://{}:{}/{}/{}', 'CSV', 'gzip')".format( + name, cluster.minio_host, cluster.minio_port, bucket, filename)) + + run_query(instance, "SELECT sum(id) FROM {}".format(name)).splitlines() == ["565"] + + finally: + run_query(instance, "DROP TABLE {}".format(name)) + + +def test_storage_s3_put_gzip(cluster): + bucket = cluster.minio_bucket + instance = cluster.instances["dummy"] + filename = "test_put_gzip.bin" + name = "test_put_gzip" + data = [ + "'Joseph Tomlinson',5", + "'Earnest Essary',44", + "'Matha Pannell',24", + "'Michael Shavers',46", + "'Elias Groce',38", + "'Pamela Bramlet',50", + "'Lewis Harrell',49", + "'Tamara Fyall',58", + "'George Dixon',38", + "'Alice Walls',49", + "'Paula Mais',24", + "'Myrtle Pelt',93", + "'Sylvia Naffziger',18", + "'Amanda Cave',83", + "'Yolanda Joseph',89" + ] + try: + run_query(instance, "CREATE TABLE {} (name String, id UInt32) ENGINE = S3('http://{}:{}/{}/{}', 'CSV', 'gzip')".format( + name, cluster.minio_host, cluster.minio_port, bucket, filename)) + + run_query(instance, "INSERT INTO {} VALUES ({})".format(name, "),(".join(data))) + + buf = StringIO.StringIO(get_s3_file_content(cluster, bucket, filename)) + f = gzip.GzipFile(fileobj=buf, mode="rb") + uncompressed_content = f.read() + assert sum([ int(i.split(',')[1]) for i in uncompressed_content.splitlines() ]) == 708 + finally: + run_query(instance, "DROP TABLE {}".format(name)) From 782823de4d7b30cb2529a6fcd8f6eb8ffc3e7675 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Wed, 30 Sep 2020 14:45:47 +0300 Subject: [PATCH 231/321] Fix build. --- src/Interpreters/ActionsVisitor.cpp | 20 ++++++++++++-------- src/Interpreters/ActionsVisitor.h | 4 ++-- src/Interpreters/ExpressionAnalyzer.cpp | 2 +- 3 files changed, 15 insertions(+), 11 deletions(-) diff --git a/src/Interpreters/ActionsVisitor.cpp b/src/Interpreters/ActionsVisitor.cpp index 741a380e441..4af17658b6d 100644 --- a/src/Interpreters/ActionsVisitor.cpp +++ b/src/Interpreters/ActionsVisitor.cpp @@ -329,7 +329,7 @@ Block createBlockForSet( } SetPtr makeExplicitSet( - const ASTFunction * node, const ActionsDAG::Index & index, bool create_ordered_set, + const ASTFunction * node, const ActionsDAG & actions, bool create_ordered_set, const Context & context, const SizeLimits & size_limits, PreparedSets & prepared_sets) { const IAST & args = *node->arguments; @@ -340,6 +340,7 @@ SetPtr makeExplicitSet( const ASTPtr & left_arg = args.children.at(0); const ASTPtr & right_arg = args.children.at(1); + const auto & index = actions.getIndex(); auto it = index.find(left_arg->getColumnName()); if (it == index.end()) throw Exception("Unknown identifier: '" + left_arg->getColumnName() + "'", ErrorCodes::UNKNOWN_IDENTIFIER); @@ -408,8 +409,10 @@ void ScopeStack::pushLevel(const NamesAndTypesList & input_columns) size_t ScopeStack::getColumnLevel(const std::string & name) { - for (int i = static_cast(stack.size()) - 1; i >= 0; --i) + for (size_t i = stack.size(); i > 0;) { + --i; + if (stack[i].inputs.count(name)) return i; @@ -482,9 +485,9 @@ std::string ScopeStack::dumpNames() const return stack.back().actions->dumpNames(); } -const ActionsDAG::Index & ScopeStack::getIndex() const +const ActionsDAG & ScopeStack::getLastActions() const { - return stack.back().actions->getIndex(); + return *stack.back().actions; } struct CachedColumnName @@ -707,7 +710,7 @@ void ActionsMatcher::visit(const ASTFunction & node, const ASTPtr & ast, Data & child_column_name = as_literal->unique_column_name; } - const auto & index = data.actions_stack.getIndex(); + const auto & index = data.actions_stack.getLastActions().getIndex(); auto it = index.find(child_column_name); if (it != index.end()) { @@ -813,7 +816,7 @@ void ActionsMatcher::visit(const ASTLiteral & literal, const ASTPtr & /* ast */, if (literal.unique_column_name.empty()) { const auto default_name = literal.getColumnName(); - const auto & index = data.actions_stack.getIndex(); + const auto & index = data.actions_stack.getLastActions().getIndex(); const ActionsDAG::Node * existing_column = nullptr; auto it = index.find(default_name); @@ -927,10 +930,11 @@ SetPtr ActionsMatcher::makeSet(const ASTFunction & node, Data & data, bool no_su } else { - const auto & index = data.actions_stack.getIndex(); + const auto & last_actions = data.actions_stack.getLastActions(); + const auto & index = last_actions.getIndex(); if (index.count(left_in_operand->getColumnName()) != 0) /// An explicit enumeration of values in parentheses. - return makeExplicitSet(&node, index, false, data.context, data.set_size_limit, data.prepared_sets); + return makeExplicitSet(&node, last_actions, false, data.context, data.set_size_limit, data.prepared_sets); else return {}; } diff --git a/src/Interpreters/ActionsVisitor.h b/src/Interpreters/ActionsVisitor.h index b6fc7039a48..4d91f9de01b 100644 --- a/src/Interpreters/ActionsVisitor.h +++ b/src/Interpreters/ActionsVisitor.h @@ -21,7 +21,7 @@ using ActionsDAGPtr = std::shared_ptr; /// The case of an explicit enumeration of values. SetPtr makeExplicitSet( - const ASTFunction * node, const ActionsDAG::Index & index, bool create_ordered_set, + const ASTFunction * node, const ActionsDAG & actions, bool create_ordered_set, const Context & context, const SizeLimits & limits, PreparedSets & prepared_sets); /** Create a block for set from expression. @@ -89,7 +89,7 @@ struct ScopeStack ActionsDAGPtr popLevel(); - const ActionsDAG::Index & getIndex() const; + const ActionsDAG & getLastActions() const; std::string dumpNames() const; }; diff --git a/src/Interpreters/ExpressionAnalyzer.cpp b/src/Interpreters/ExpressionAnalyzer.cpp index 2778117c4eb..2f0dee58141 100644 --- a/src/Interpreters/ExpressionAnalyzer.cpp +++ b/src/Interpreters/ExpressionAnalyzer.cpp @@ -380,7 +380,7 @@ void SelectQueryExpressionAnalyzer::makeSetsForIndex(const ASTPtr & node) getRootActions(left_in_operand, true, temp_actions); if (temp_actions->getIndex().count(left_in_operand->getColumnName()) != 0) - makeExplicitSet(func, temp_actions->getIndex(), true, context, + makeExplicitSet(func, *temp_actions, true, context, settings.size_limits_for_set, prepared_sets); } } From 1617442242a30d76bd73a5457dbc923474c8c38a Mon Sep 17 00:00:00 2001 From: Alexander Kuzmenkov Date: Wed, 30 Sep 2020 14:55:24 +0300 Subject: [PATCH 232/321] fixup --- docker/test/performance-comparison/perf.py | 7 +++++-- 1 file changed, 5 insertions(+), 2 deletions(-) diff --git a/docker/test/performance-comparison/perf.py b/docker/test/performance-comparison/perf.py index fb14f050cfa..337f13690b6 100755 --- a/docker/test/performance-comparison/perf.py +++ b/docker/test/performance-comparison/perf.py @@ -369,8 +369,11 @@ for query_index in queries_to_run: pvalue = stats.ttest_ind(all_server_times[0], all_server_times[1], equal_var = False).pvalue median = [statistics.median(t) for t in all_server_times] - relative_diff = (median[1] - median[0]) / max(median) - print(f'diff\t{relative_diff}\t{pvalue}') + # Keep this consistent with the value used in report. Should eventually move + # to (median[1] - median[0]) / min(median), which is compatible with "times" + # difference we use in report (max(median) / min(median)). + relative_diff = (median[1] - median[0]) / median[0] + print(f'diff\t{query_index}\t{median[0]}\t{median[1]}\t{relative_diff}\t{pvalue}') if abs(relative_diff) < ignored_relative_change or pvalue > 0.05: continue From 9f63e882c7a71c0908401eb4f349c73b779ba138 Mon Sep 17 00:00:00 2001 From: Alexander Kuzmenkov Date: Wed, 30 Sep 2020 15:03:52 +0300 Subject: [PATCH 233/321] fixup --- docker/test/performance-comparison/report.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docker/test/performance-comparison/report.py b/docker/test/performance-comparison/report.py index 5e4e0a161e1..69015c2ce1a 100755 --- a/docker/test/performance-comparison/report.py +++ b/docker/test/performance-comparison/report.py @@ -487,7 +487,7 @@ if args.report == 'main': for r in rows: anchor = f'{currentTableAnchor()}.{r[0]}' total_runs = (int(r[7]) + 1) * 2 # one prewarm run, two servers - if float(r[5]) > allowed_average_run_time * total_runs: + if r[0] != 'Total' and float(r[5]) > allowed_average_run_time * total_runs: # FIXME should be 15s max -- investigate parallel_insert slow_average_tests += 1 attrs[5] = f'style="background: {color_bad}"' @@ -495,7 +495,7 @@ if args.report == 'main': else: attrs[5] = '' - if float(r[4]) > allowed_single_run_time * total_runs: + if r[0] != 'Total' and float(r[4]) > allowed_single_run_time * total_runs: slow_average_tests += 1 attrs[4] = f'style="background: {color_bad}"' errors_explained.append([f'Some query of the test \'{r[0]}\' is too slow to run. See the all queries report']) From a38c7ff2a895983ed7ad1bd516cdb5ac35ed2b00 Mon Sep 17 00:00:00 2001 From: Vladimir Chebotarev Date: Wed, 30 Sep 2020 15:04:21 +0300 Subject: [PATCH 234/321] Somehow uncompressed PUT works and gzipped PUT doesn't, in S3 storage. --- src/Storages/StorageS3.cpp | 15 ++++++++-- src/Storages/StorageS3.h | 2 +- tests/integration/test_storage_s3/test.py | 36 +++++++++++++++++++++++ 3 files changed, 50 insertions(+), 3 deletions(-) diff --git a/src/Storages/StorageS3.cpp b/src/Storages/StorageS3.cpp index b4dfa6dcf66..469350d6eaa 100644 --- a/src/Storages/StorageS3.cpp +++ b/src/Storages/StorageS3.cpp @@ -197,7 +197,7 @@ StorageS3::StorageS3( const ColumnsDescription & columns_, const ConstraintsDescription & constraints_, Context & context_, - const String & compression_method_ = "") + const String & compression_method_) : IStorage(table_id_) , uri(uri_) , context_global(context_) @@ -372,7 +372,18 @@ void registerStorageS3Impl(const String & name, StorageFactory & factory) format_name = engine_args.back()->as().value.safeGet(); } - return StorageS3::create(s3_uri, access_key_id, secret_access_key, args.table_id, format_name, min_upload_part_size, args.columns, args.constraints, args.context); + return StorageS3::create( + s3_uri, + access_key_id, + secret_access_key, + args.table_id, + format_name, + min_upload_part_size, + args.columns, + args.constraints, + args.context, + compression_method + ); }, { .source_access_type = AccessType::S3, diff --git a/src/Storages/StorageS3.h b/src/Storages/StorageS3.h index a172d951642..5a702aa8785 100644 --- a/src/Storages/StorageS3.h +++ b/src/Storages/StorageS3.h @@ -34,7 +34,7 @@ public: const ColumnsDescription & columns_, const ConstraintsDescription & constraints_, Context & context_, - const String & compression_method_); + const String & compression_method_ = ""); String getName() const override { diff --git a/tests/integration/test_storage_s3/test.py b/tests/integration/test_storage_s3/test.py index cc2d187558c..d878953b6d8 100644 --- a/tests/integration/test_storage_s3/test.py +++ b/tests/integration/test_storage_s3/test.py @@ -392,6 +392,41 @@ def test_storage_s3_get_gzip(cluster): run_query(instance, "DROP TABLE {}".format(name)) +def test_storage_s3_put_uncompressed(cluster): + bucket = cluster.minio_bucket + instance = cluster.instances["dummy"] + filename = "test_put_uncompressed.bin" + name = "test_put_uncompressed" + data = [ + "'Gloria Thompson',99", + "'Matthew Tang',98", + "'Patsy Anderson',23", + "'Nancy Badillo',93", + "'Roy Hunt',5", + "'Adam Kirk',51", + "'Joshua Douds',28", + "'Jolene Ryan',0", + "'Roxanne Padilla',50", + "'Howard Roberts',41", + "'Ricardo Broughton',13", + "'Roland Speer',83", + "'Cathy Cohan',58", + "'Kathie Dawson',100", + "'Gregg Mcquistion',11", + ] + try: + run_query(instance, "CREATE TABLE {} (name String, id UInt32) ENGINE = S3('http://{}:{}/{}/{}', 'CSV')".format( + name, cluster.minio_host, cluster.minio_port, bucket, filename)) + + run_query(instance, "INSERT INTO {} VALUES ({})".format(name, "),(".join(data))) + run_query(instance, "SELECT sum(id) FROM {}".format(name)).splitlines() == ["753"] + + uncompressed_content = get_s3_file_content(cluster, bucket, filename) + assert sum([ int(i.split(',')[1]) for i in uncompressed_content.splitlines() ]) == 753 + finally: + run_query(instance, "DROP TABLE {}".format(name)) + + def test_storage_s3_put_gzip(cluster): bucket = cluster.minio_bucket instance = cluster.instances["dummy"] @@ -419,6 +454,7 @@ def test_storage_s3_put_gzip(cluster): name, cluster.minio_host, cluster.minio_port, bucket, filename)) run_query(instance, "INSERT INTO {} VALUES ({})".format(name, "),(".join(data))) + run_query(instance, "SELECT sum(id) FROM {}".format(name)).splitlines() == ["708"] buf = StringIO.StringIO(get_s3_file_content(cluster, bucket, filename)) f = gzip.GzipFile(fileobj=buf, mode="rb") From a1aca0471102f7537bcc2857dbb9377fe2e1a5dd Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Wed, 30 Sep 2020 15:32:37 +0300 Subject: [PATCH 235/321] Fix build. --- src/Interpreters/ActionsVisitor.cpp | 26 ++++++++++++++++++++++++++ src/Interpreters/ActionsVisitor.h | 22 ++-------------------- 2 files changed, 28 insertions(+), 20 deletions(-) diff --git a/src/Interpreters/ActionsVisitor.cpp b/src/Interpreters/ActionsVisitor.cpp index 4af17658b6d..d41f1c0499f 100644 --- a/src/Interpreters/ActionsVisitor.cpp +++ b/src/Interpreters/ActionsVisitor.cpp @@ -375,6 +375,32 @@ SetPtr makeExplicitSet( return set; } +ActionsMatcher::Data::Data( + const Context & context_, SizeLimits set_size_limit_, size_t subquery_depth_, + const NamesAndTypesList & source_columns_, ActionsDAGPtr actions, + PreparedSets & prepared_sets_, SubqueriesForSets & subqueries_for_sets_, + bool no_subqueries_, bool no_makeset_, bool only_consts_, bool no_storage_or_local_) + : context(context_) + , set_size_limit(set_size_limit_) + , subquery_depth(subquery_depth_) + , source_columns(source_columns_) + , prepared_sets(prepared_sets_) + , subqueries_for_sets(subqueries_for_sets_) + , no_subqueries(no_subqueries_) + , no_makeset(no_makeset_) + , only_consts(only_consts_) + , no_storage_or_local(no_storage_or_local_) + , visit_depth(0) + , actions_stack(std::move(actions), context) + , next_unique_suffix(actions_stack.getLastActions().getIndex().size() + 1) +{ +} + +bool ActionsMatcher::Data::hasColumn(const String & column_name) const +{ + return actions_stack.getLastActions().getIndex().count(column_name) != 0; +} + ScopeStack::ScopeStack(ActionsDAGPtr actions, const Context & context_) : context(context_) { diff --git a/src/Interpreters/ActionsVisitor.h b/src/Interpreters/ActionsVisitor.h index 4d91f9de01b..ac53669c20f 100644 --- a/src/Interpreters/ActionsVisitor.h +++ b/src/Interpreters/ActionsVisitor.h @@ -128,28 +128,10 @@ public: Data(const Context & context_, SizeLimits set_size_limit_, size_t subquery_depth_, const NamesAndTypesList & source_columns_, ActionsDAGPtr actions, PreparedSets & prepared_sets_, SubqueriesForSets & subqueries_for_sets_, - bool no_subqueries_, bool no_makeset_, bool only_consts_, bool no_storage_or_local_) - : context(context_), - set_size_limit(set_size_limit_), - subquery_depth(subquery_depth_), - source_columns(source_columns_), - prepared_sets(prepared_sets_), - subqueries_for_sets(subqueries_for_sets_), - no_subqueries(no_subqueries_), - no_makeset(no_makeset_), - only_consts(only_consts_), - no_storage_or_local(no_storage_or_local_), - visit_depth(0), - actions_stack(std::move(actions), context), - next_unique_suffix(actions_stack.getIndex().size() + 1) - {} + bool no_subqueries_, bool no_makeset_, bool only_consts_, bool no_storage_or_local_); /// Does result of the calculation already exists in the block. - bool hasColumn(const String & column_name) const - { - return actions_stack.getIndex().count(column_name) != 0; - } - + bool hasColumn(const String & column_name) const; void addColumn(ColumnWithTypeAndName column) { actions_stack.addColumn(std::move(column)); From b4600243560274ca4c8a7f149cde298ac0c3a340 Mon Sep 17 00:00:00 2001 From: Vladimir Chebotarev Date: Wed, 30 Sep 2020 16:09:55 +0300 Subject: [PATCH 236/321] Fixed bug in GZIP compression in S3 storage. --- src/IO/ZlibDeflatingWriteBuffer.cpp | 1 + tests/integration/test_storage_s3/test.py | 3 +++ 2 files changed, 4 insertions(+) diff --git a/src/IO/ZlibDeflatingWriteBuffer.cpp b/src/IO/ZlibDeflatingWriteBuffer.cpp index 8efe96877e4..68b82487bc2 100644 --- a/src/IO/ZlibDeflatingWriteBuffer.cpp +++ b/src/IO/ZlibDeflatingWriteBuffer.cpp @@ -116,6 +116,7 @@ void ZlibDeflatingWriteBuffer::finish() if (rc == Z_STREAM_END) { + out->finalize(); finished = true; return; } diff --git a/tests/integration/test_storage_s3/test.py b/tests/integration/test_storage_s3/test.py index d878953b6d8..f752e72c677 100644 --- a/tests/integration/test_storage_s3/test.py +++ b/tests/integration/test_storage_s3/test.py @@ -5,6 +5,7 @@ import os import random import StringIO import threading +import time import helpers.client import pytest @@ -419,6 +420,7 @@ def test_storage_s3_put_uncompressed(cluster): name, cluster.minio_host, cluster.minio_port, bucket, filename)) run_query(instance, "INSERT INTO {} VALUES ({})".format(name, "),(".join(data))) + run_query(instance, "SELECT sum(id) FROM {}".format(name)).splitlines() == ["753"] uncompressed_content = get_s3_file_content(cluster, bucket, filename) @@ -454,6 +456,7 @@ def test_storage_s3_put_gzip(cluster): name, cluster.minio_host, cluster.minio_port, bucket, filename)) run_query(instance, "INSERT INTO {} VALUES ({})".format(name, "),(".join(data))) + run_query(instance, "SELECT sum(id) FROM {}".format(name)).splitlines() == ["708"] buf = StringIO.StringIO(get_s3_file_content(cluster, bucket, filename)) From 0b1fe69ad779531cb0f799e577a630056a7b1f90 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Wed, 30 Sep 2020 16:31:06 +0300 Subject: [PATCH 237/321] Fix build. --- src/Interpreters/ActionsVisitor.h | 3 +++ 1 file changed, 3 insertions(+) diff --git a/src/Interpreters/ActionsVisitor.h b/src/Interpreters/ActionsVisitor.h index ac53669c20f..f4da9932163 100644 --- a/src/Interpreters/ActionsVisitor.h +++ b/src/Interpreters/ActionsVisitor.h @@ -19,6 +19,9 @@ using ExpressionActionsPtr = std::shared_ptr; class ActionsDAG; using ActionsDAGPtr = std::shared_ptr; +class IFunctionOverloadResolver; +using FunctionOverloadResolverPtr = std::shared_ptr; + /// The case of an explicit enumeration of values. SetPtr makeExplicitSet( const ASTFunction * node, const ActionsDAG & actions, bool create_ordered_set, From 0516af03ce11f2d9c11152ae4da183678dceaefa Mon Sep 17 00:00:00 2001 From: Alexander Kuzmenkov Date: Wed, 30 Sep 2020 00:05:05 +0300 Subject: [PATCH 238/321] fasttest fixup 3 --- docker/test/fasttest/run.sh | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/docker/test/fasttest/run.sh b/docker/test/fasttest/run.sh index 6898000185d..a9cbe87ecf9 100755 --- a/docker/test/fasttest/run.sh +++ b/docker/test/fasttest/run.sh @@ -156,7 +156,8 @@ ccache --show-stats ||: function configure { -export PATH="$FASTTEST_BUILD/programs:$FASTTEST_SOURCE/tests:$PATH" +PATH="$FASTTEST_BUILD/programs:$FASTTEST_SOURCE/tests:$PATH" +export PATH clickhouse-client --version clickhouse-test --help @@ -171,6 +172,7 @@ rm -f "$FASTTEST_DATA/config.d/secure_ports.xml" function run_tests { +echo "$PATH" clickhouse-server --version clickhouse-test --help From 9194f1d7e57ed3fa7d0000f8c75394b9a408822e Mon Sep 17 00:00:00 2001 From: Alexander Kuzmenkov Date: Wed, 30 Sep 2020 17:21:06 +0300 Subject: [PATCH 239/321] fasttest fixup 3 --- docker/test/fasttest/run.sh | 6 ++---- 1 file changed, 2 insertions(+), 4 deletions(-) diff --git a/docker/test/fasttest/run.sh b/docker/test/fasttest/run.sh index a9cbe87ecf9..4a47fcfe4dc 100755 --- a/docker/test/fasttest/run.sh +++ b/docker/test/fasttest/run.sh @@ -156,9 +156,6 @@ ccache --show-stats ||: function configure { -PATH="$FASTTEST_BUILD/programs:$FASTTEST_SOURCE/tests:$PATH" -export PATH - clickhouse-client --version clickhouse-test --help @@ -172,7 +169,6 @@ rm -f "$FASTTEST_DATA/config.d/secure_ports.xml" function run_tests { -echo "$PATH" clickhouse-server --version clickhouse-test --help @@ -306,6 +302,8 @@ case "$stage" in ;& "build") build + PATH="$FASTTEST_BUILD/programs:$FASTTEST_SOURCE/tests:$PATH" + export PATH ;& "configure") # The `install_log.txt` is also needed for compatibility with old CI task -- From cfe9a0c4a12a1a8750b9636d8c9dafdacb74c482 Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 30 Sep 2020 17:58:27 +0300 Subject: [PATCH 240/321] Fix event subscription in DDLWorker --- src/Interpreters/DDLWorker.cpp | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/Interpreters/DDLWorker.cpp b/src/Interpreters/DDLWorker.cpp index 5b346eec54a..3591c7768ee 100644 --- a/src/Interpreters/DDLWorker.cpp +++ b/src/Interpreters/DDLWorker.cpp @@ -910,7 +910,7 @@ bool DDLWorker::tryExecuteQueryOnLeaderReplica( String executed_by; zkutil::EventPtr event = std::make_shared(); - if (zookeeper->tryGet(is_executed_path, executed_by)) + if (zookeeper->tryGet(is_executed_path, executed_by, nullptr, event)) { LOG_DEBUG(log, "Task {} has already been executed by replica ({}) of the same shard.", task.entry_name, executed_by); return true; @@ -961,6 +961,7 @@ bool DDLWorker::tryExecuteQueryOnLeaderReplica( if (event->tryWait(std::uniform_int_distribution(0, 1000)(rng))) { + LOG_DEBUG(log, "Task {} has already been executed by replica ({}) of the same shard.", task.entry_name, zookeeper->get(is_executed_path)); executed_by_leader = true; break; } From af051ec2d90ab034831175f17aa9b2bedbb17694 Mon Sep 17 00:00:00 2001 From: Alexander Kuzmenkov Date: Wed, 30 Sep 2020 18:12:53 +0300 Subject: [PATCH 241/321] fasttest compat with old script --- docker/test/fasttest/run.sh | 12 +++++++++++- 1 file changed, 11 insertions(+), 1 deletion(-) diff --git a/docker/test/fasttest/run.sh b/docker/test/fasttest/run.sh index 4a47fcfe4dc..c4a9a0ea5f3 100755 --- a/docker/test/fasttest/run.sh +++ b/docker/test/fasttest/run.sh @@ -284,8 +284,14 @@ case "$stage" in # Pass control to the script from cloned sources, unless asked otherwise. if ! [ -v FASTTEST_LOCAL_SCRIPT ] then - # 'run' is deprecated, used for compatibility with old scripts. + # 'run' stage is deprecated, used for compatibility with old scripts. # Replace with 'clone_submodules' after Nov 1, 2020. + # cd and CLICKHOUSE_DIR are also a setup for old scripts, remove as well. + # In modern script we undo it by changing back into workspace dir right + # away, see below. Remove that as well. + cd "$FASTTEST_SOURCE" + CLICKHOUSE_DIR=$(pwd) + export CLICKHOUSE_DIR stage=run "$FASTTEST_SOURCE/docker/test/fasttest/run.sh" exit $? fi @@ -295,6 +301,10 @@ case "$stage" in # after cloning root, starting with cloning submodules. ;& "clone_submodules") + # Recover after being called from the old script that changes into source directory. + # See the compatibility hacks in `clone_root` stage above. Remove at the same time, + # after Nov 1, 2020. + cd "$FASTTEST_WORKSPACE" clone_submodules | ts '%Y-%m-%d %H:%M:%S' | tee "$FASTTEST_OUTPUT/submodule_log.txt" ;& "run_cmake") From 0497ce4f9c8218c6d17dd4a4dc97ca81d267c74d Mon Sep 17 00:00:00 2001 From: Vladimir Chebotarev Date: Wed, 30 Sep 2020 16:24:36 +0300 Subject: [PATCH 242/321] Possibly better fix. --- src/IO/WriteBufferFromS3.cpp | 26 ++++++++++++++++---------- src/IO/WriteBufferFromS3.h | 4 ++++ src/IO/ZlibDeflatingWriteBuffer.cpp | 1 - 3 files changed, 20 insertions(+), 11 deletions(-) diff --git a/src/IO/WriteBufferFromS3.cpp b/src/IO/WriteBufferFromS3.cpp index bac14acb9cd..1ca8d8988d9 100644 --- a/src/IO/WriteBufferFromS3.cpp +++ b/src/IO/WriteBufferFromS3.cpp @@ -56,7 +56,6 @@ WriteBufferFromS3::WriteBufferFromS3( initiate(); } - void WriteBufferFromS3::nextImpl() { if (!offset()) @@ -79,23 +78,31 @@ void WriteBufferFromS3::nextImpl() } } - void WriteBufferFromS3::finalize() { - next(); - - if (is_multipart) - writePart(temporary_buffer->str()); - - complete(); + finalizeImpl(); } +void WriteBufferFromS3::finalizeImpl() +{ + if (!finalized) + { + next(); + + if (is_multipart) + writePart(temporary_buffer->str()); + + complete(); + + finalized = true; + } +} WriteBufferFromS3::~WriteBufferFromS3() { try { - next(); + finalizeImpl(); } catch (...) { @@ -103,7 +110,6 @@ WriteBufferFromS3::~WriteBufferFromS3() } } - void WriteBufferFromS3::initiate() { Aws::S3::Model::CreateMultipartUploadRequest req; diff --git a/src/IO/WriteBufferFromS3.h b/src/IO/WriteBufferFromS3.h index 93a6947609e..1a1e859d913 100644 --- a/src/IO/WriteBufferFromS3.h +++ b/src/IO/WriteBufferFromS3.h @@ -57,9 +57,13 @@ public: ~WriteBufferFromS3() override; private: + bool finalized = false; + void initiate(); void writePart(const String & data); void complete(); + + void finalizeImpl(); }; } diff --git a/src/IO/ZlibDeflatingWriteBuffer.cpp b/src/IO/ZlibDeflatingWriteBuffer.cpp index 68b82487bc2..8efe96877e4 100644 --- a/src/IO/ZlibDeflatingWriteBuffer.cpp +++ b/src/IO/ZlibDeflatingWriteBuffer.cpp @@ -116,7 +116,6 @@ void ZlibDeflatingWriteBuffer::finish() if (rc == Z_STREAM_END) { - out->finalize(); finished = true; return; } From 6e239012279a22461014d508c5c8dffeaac2e3e9 Mon Sep 17 00:00:00 2001 From: Amos Bird Date: Wed, 30 Sep 2020 23:23:01 +0800 Subject: [PATCH 243/321] enable FastMemcpy properly --- CMakeLists.txt | 6 ++++++ base/glibc-compatibility/CMakeLists.txt | 2 -- contrib/FastMemcpy/CMakeLists.txt | 2 -- 3 files changed, 6 insertions(+), 4 deletions(-) diff --git a/CMakeLists.txt b/CMakeLists.txt index f4e230fbd93..0969d49d9a5 100644 --- a/CMakeLists.txt +++ b/CMakeLists.txt @@ -369,6 +369,12 @@ endif () include(cmake/dbms_glob_sources.cmake) +if (${ENABLE_LIBRARIES} OR ${GLIBC_COMPATIBILITY}) + option (ENABLE_FASTMEMCPY "Enable FastMemcpy library (only internal)" ON) +else () + option (ENABLE_FASTMEMCPY "Enable FastMemcpy library (only internal)" OFF) +endif () + if (OS_LINUX OR OS_ANDROID) include(cmake/linux/default_libs.cmake) elseif (OS_DARWIN) diff --git a/base/glibc-compatibility/CMakeLists.txt b/base/glibc-compatibility/CMakeLists.txt index 2bd4e20d3bc..1a76ef3df32 100644 --- a/base/glibc-compatibility/CMakeLists.txt +++ b/base/glibc-compatibility/CMakeLists.txt @@ -1,6 +1,4 @@ if (GLIBC_COMPATIBILITY) - set (ENABLE_FASTMEMCPY ON) - enable_language(ASM) include(CheckIncludeFile) diff --git a/contrib/FastMemcpy/CMakeLists.txt b/contrib/FastMemcpy/CMakeLists.txt index 8efe6d45dff..0f3072b4e05 100644 --- a/contrib/FastMemcpy/CMakeLists.txt +++ b/contrib/FastMemcpy/CMakeLists.txt @@ -1,5 +1,3 @@ -option (ENABLE_FASTMEMCPY "Enable FastMemcpy library (only internal)" ${ENABLE_LIBRARIES}) - if (NOT OS_LINUX OR ARCH_AARCH64) set (ENABLE_FASTMEMCPY OFF) endif () From 34e4932c8f6abc30963780f378e341071aadef54 Mon Sep 17 00:00:00 2001 From: tavplubix Date: Wed, 30 Sep 2020 19:16:35 +0300 Subject: [PATCH 244/321] Update test.py --- tests/integration/test_ttl_move/test.py | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/integration/test_ttl_move/test.py b/tests/integration/test_ttl_move/test.py index 59705f7b3dd..377ee0e5d75 100644 --- a/tests/integration/test_ttl_move/test.py +++ b/tests/integration/test_ttl_move/test.py @@ -403,6 +403,7 @@ def test_moves_to_disk_eventually_work(started_cluster, name, engine): node1.query("DROP TABLE {} NO DELAY".format(name_temp)) + time.sleep(2) used_disks = get_used_disks_for_table(node1, name) assert set(used_disks) == {"jbod2"} From 499e0766e547e31ff93c5d16851084a4b15c0ede Mon Sep 17 00:00:00 2001 From: Alexander Kuzmenkov Date: Wed, 30 Sep 2020 20:06:14 +0300 Subject: [PATCH 245/321] Check docker scripts with shellcheck --- docker/builder/build.sh | 3 +- docker/packager/binary/build.sh | 4 +-- docker/packager/deb/build.sh | 8 +++--- docker/packager/unbundled/build.sh | 6 ++-- docker/test/fuzzer/run-fuzzer.sh | 5 +++- .../integration/runner/dockerd-entrypoint.sh | 2 +- .../compare-releases.sh | 2 +- .../test/performance-comparison/download.sh | 2 +- docker/test/stateful/run.sh | 8 +++--- docker/test/stateful_with_coverage/run.sh | 28 +++++++++---------- docker/test/stateless/run.sh | 4 +-- docker/test/stateless_unbundled/run.sh | 4 +-- docker/test/stateless_with_coverage/run.sh | 18 ++++++------ docker/test/stress/run.sh | 6 ++-- docker/test/test_runner.sh | 12 ++++---- .../testflows/runner/dockerd-entrypoint.sh | 2 +- utils/check-style/check-style | 3 ++ 17 files changed, 63 insertions(+), 54 deletions(-) diff --git a/docker/builder/build.sh b/docker/builder/build.sh index 983c25fdec4..d814bcdf2b4 100755 --- a/docker/builder/build.sh +++ b/docker/builder/build.sh @@ -1,9 +1,10 @@ #!/usr/bin/env bash +set -e #ccache -s # uncomment to display CCache statistics mkdir -p /server/build_docker cd /server/build_docker -cmake -G Ninja /server -DCMAKE_C_COMPILER=`which gcc-9` -DCMAKE_CXX_COMPILER=`which g++-9` +cmake -G Ninja /server "-DCMAKE_C_COMPILER=$(command -v gcc-9)" "-DCMAKE_CXX_COMPILER=$(command -v g++-9)" # Set the number of build jobs to the half of number of virtual CPU cores (rounded up). # By default, ninja use all virtual CPU cores, that leads to very high memory consumption without much improvement in build time. diff --git a/docker/packager/binary/build.sh b/docker/packager/binary/build.sh index fd70b03242b..300b70c4720 100755 --- a/docker/packager/binary/build.sh +++ b/docker/packager/binary/build.sh @@ -17,8 +17,8 @@ ccache --show-stats ||: ccache --zero-stats ||: ln -s /usr/lib/x86_64-linux-gnu/libOpenCL.so.1.0.0 /usr/lib/libOpenCL.so ||: rm -f CMakeCache.txt -cmake --debug-trycompile --verbose=1 -DCMAKE_VERBOSE_MAKEFILE=1 -LA -DCMAKE_BUILD_TYPE=$BUILD_TYPE -DSANITIZE=$SANITIZER -DENABLE_CHECK_HEAVY_BUILDS=1 $CMAKE_FLAGS .. -ninja $NINJA_FLAGS clickhouse-bundle +cmake --debug-trycompile --verbose=1 -DCMAKE_VERBOSE_MAKEFILE=1 -LA "-DCMAKE_BUILD_TYPE=$BUILD_TYPE" "-DSANITIZE=$SANITIZER" -DENABLE_CHECK_HEAVY_BUILDS=1 "$CMAKE_FLAGS" .. +ninja "$NINJA_FLAGS" clickhouse-bundle mv ./programs/clickhouse* /output mv ./src/unit_tests_dbms /output ||: # may not exist for some binary builds find . -name '*.so' -print -exec mv '{}' /output \; diff --git a/docker/packager/deb/build.sh b/docker/packager/deb/build.sh index fbaa0151c6b..a8072c5e78a 100755 --- a/docker/packager/deb/build.sh +++ b/docker/packager/deb/build.sh @@ -4,16 +4,16 @@ set -x -e ccache --show-stats ||: ccache --zero-stats ||: -build/release --no-pbuilder $ALIEN_PKGS | ts '%Y-%m-%d %H:%M:%S' +build/release --no-pbuilder "$ALIEN_PKGS" | ts '%Y-%m-%d %H:%M:%S' mv /*.deb /output -mv *.changes /output -mv *.buildinfo /output +mv -- *.changes /output +mv -- *.buildinfo /output mv /*.rpm /output ||: # if exists mv /*.tgz /output ||: # if exists if [ -n "$BINARY_OUTPUT" ] && { [ "$BINARY_OUTPUT" = "programs" ] || [ "$BINARY_OUTPUT" = "tests" ] ;} then - echo Place $BINARY_OUTPUT to output + echo "Place $BINARY_OUTPUT to output" mkdir /output/binary ||: # if exists mv /build/obj-*/programs/clickhouse* /output/binary if [ "$BINARY_OUTPUT" = "tests" ] diff --git a/docker/packager/unbundled/build.sh b/docker/packager/unbundled/build.sh index ca1217ac522..c94bd54d001 100755 --- a/docker/packager/unbundled/build.sh +++ b/docker/packager/unbundled/build.sh @@ -4,10 +4,10 @@ set -x -e ccache --show-stats ||: ccache --zero-stats ||: -build/release --no-pbuilder $ALIEN_PKGS | ts '%Y-%m-%d %H:%M:%S' +build/release --no-pbuilder "$ALIEN_PKGS" | ts '%Y-%m-%d %H:%M:%S' mv /*.deb /output -mv *.changes /output -mv *.buildinfo /output +mv -- *.changes /output +mv -- *.buildinfo /output mv /*.rpm /output ||: # if exists mv /*.tgz /output ||: # if exists diff --git a/docker/test/fuzzer/run-fuzzer.sh b/docker/test/fuzzer/run-fuzzer.sh index d35a13cc421..6429c1bc19c 100755 --- a/docker/test/fuzzer/run-fuzzer.sh +++ b/docker/test/fuzzer/run-fuzzer.sh @@ -58,7 +58,7 @@ function watchdog echo "Fuzzing run has timed out" killall clickhouse-client ||: - for x in {1..10} + for _ in {1..10} do if ! pgrep -f clickhouse-client then @@ -81,6 +81,9 @@ function fuzz echo Server started fuzzer_exit_code=0 + # SC2012: Use find instead of ls to better handle non-alphanumeric filenames. + # They are all alphanumeric. + # shellcheck disable=SC2012 ./clickhouse-client --query-fuzzer-runs=1000 \ < <(for f in $(ls ch/tests/queries/0_stateless/*.sql | sort -R); do cat "$f"; echo ';'; done) \ > >(tail -10000 > fuzzer.log) \ diff --git a/docker/test/integration/runner/dockerd-entrypoint.sh b/docker/test/integration/runner/dockerd-entrypoint.sh index cbdb7317b1e..c0255d3d706 100755 --- a/docker/test/integration/runner/dockerd-entrypoint.sh +++ b/docker/test/integration/runner/dockerd-entrypoint.sh @@ -7,7 +7,7 @@ set +e reties=0 while true; do docker info &>/dev/null && break - reties=$[$reties+1] + reties=$((reties+1)) if [[ $reties -ge 100 ]]; then # 10 sec max echo "Can't start docker daemon, timeout exceeded." >&2 exit 1; diff --git a/docker/test/performance-comparison/compare-releases.sh b/docker/test/performance-comparison/compare-releases.sh index a93a768f36b..456661362b3 100755 --- a/docker/test/performance-comparison/compare-releases.sh +++ b/docker/test/performance-comparison/compare-releases.sh @@ -9,7 +9,7 @@ right_version=${2} if [ "$left_version" == "" ] || [ "$right_version" == "" ] then - >&2 echo Usage: $(basename "$0") left_version right_version + >&2 echo "Usage: $(basename "$0") left_version right_version" exit 1 fi diff --git a/docker/test/performance-comparison/download.sh b/docker/test/performance-comparison/download.sh index ce953f59cd5..bd72547ec1c 100755 --- a/docker/test/performance-comparison/download.sh +++ b/docker/test/performance-comparison/download.sh @@ -10,7 +10,7 @@ mkdir left ||: left_pr=$1 left_sha=$2 -right_pr=$3 +# right_pr=$3 not used for now right_sha=$4 datasets=${CHPC_DATASETS:-"hits1 hits10 hits100 values"} diff --git a/docker/test/stateful/run.sh b/docker/test/stateful/run.sh index 87cc4054ee6..fa1d627bf57 100755 --- a/docker/test/stateful/run.sh +++ b/docker/test/stateful/run.sh @@ -26,12 +26,12 @@ function start() fi timeout 120 service clickhouse-server start sleep 0.5 - counter=$(($counter + 1)) + counter=$((counter + 1)) done } start -/s3downloader --dataset-names $DATASETS +/s3downloader --dataset-names "$DATASETS" chmod 777 -R /var/lib/clickhouse clickhouse-client --query "SHOW DATABASES" clickhouse-client --query "ATTACH DATABASE datasets ENGINE = Ordinary" @@ -43,8 +43,8 @@ clickhouse-client --query "RENAME TABLE datasets.hits_v1 TO test.hits" clickhouse-client --query "RENAME TABLE datasets.visits_v1 TO test.visits" clickhouse-client --query "SHOW TABLES FROM test" -if cat /usr/bin/clickhouse-test | grep -q -- "--use-skip-list"; then +if grep -q -- "--use-skip-list" /usr/bin/clickhouse-test ; then SKIP_LIST_OPT="--use-skip-list" fi -clickhouse-test --testname --shard --zookeeper --no-stateless "$SKIP_LIST_OPT" $ADDITIONAL_OPTIONS $SKIP_TESTS_OPTION 2>&1 | ts '%Y-%m-%d %H:%M:%S' | tee test_output/test_result.txt +clickhouse-test --testname --shard --zookeeper --no-stateless "$SKIP_LIST_OPT" "$ADDITIONAL_OPTIONS" "$SKIP_TESTS_OPTION" 2>&1 | ts '%Y-%m-%d %H:%M:%S' | tee test_output/test_result.txt diff --git a/docker/test/stateful_with_coverage/run.sh b/docker/test/stateful_with_coverage/run.sh index 7191745ec83..dc730f461bb 100755 --- a/docker/test/stateful_with_coverage/run.sh +++ b/docker/test/stateful_with_coverage/run.sh @@ -1,15 +1,15 @@ #!/bin/bash kill_clickhouse () { - kill `pgrep -u clickhouse` 2>/dev/null + kill "$(pgrep -u clickhouse)" 2>/dev/null - for i in {1..10} + for _ in {1..10} do - if ! kill -0 `pgrep -u clickhouse`; then + if ! kill -0 "$(pgrep -u clickhouse)"; then echo "No clickhouse process" break else - echo "Process" `pgrep -u clickhouse` "still alive" + echo "Process $(pgrep -u clickhouse) still alive" sleep 10 fi done @@ -20,19 +20,19 @@ start_clickhouse () { } wait_llvm_profdata () { - while kill -0 `pgrep llvm-profdata-10`; + while kill -0 "$(pgrep llvm-profdata-10)" do - echo "Waiting for profdata" `pgrep llvm-profdata-10` "still alive" + echo "Waiting for profdata $(pgrep llvm-profdata-10) still alive" sleep 3 done } merge_client_files_in_background () { - client_files=`ls /client_*profraw 2>/dev/null` - if [ ! -z "$client_files" ] + client_files=$(ls /client_*profraw 2>/dev/null) + if [ -n "$client_files" ] then - llvm-profdata-10 merge -sparse $client_files -o merged_client_`date +%s`.profraw - rm $client_files + llvm-profdata-10 merge -sparse "$client_files" -o "merged_client_$(date +%s).profraw" + rm "$client_files" fi } @@ -66,13 +66,13 @@ function start() fi timeout 120 service clickhouse-server start sleep 0.5 - counter=$(($counter + 1)) + counter=$((counter + 1)) done } start -if ! /s3downloader --dataset-names $DATASETS; then +if ! /s3downloader --dataset-names "$DATASETS"; then echo "Cannot download datatsets" exit 1 fi @@ -100,11 +100,11 @@ LLVM_PROFILE_FILE='client_%h_%p_%m.profraw' clickhouse-client --query "RENAME TA LLVM_PROFILE_FILE='client_%h_%p_%m.profraw' clickhouse-client --query "RENAME TABLE datasets.visits_v1 TO test.visits" LLVM_PROFILE_FILE='client_%h_%p_%m.profraw' clickhouse-client --query "SHOW TABLES FROM test" -if cat /usr/bin/clickhouse-test | grep -q -- "--use-skip-list"; then +if grep -q -- "--use-skip-list" /usr/bin/clickhouse-test; then SKIP_LIST_OPT="--use-skip-list" fi -LLVM_PROFILE_FILE='client_%h_%p_%m.profraw' clickhouse-test --testname --shard --zookeeper --no-stateless "$SKIP_LIST_OPT" $ADDITIONAL_OPTIONS $SKIP_TESTS_OPTION 2>&1 | ts '%Y-%m-%d %H:%M:%S' | tee test_output/test_result.txt +LLVM_PROFILE_FILE='client_%h_%p_%m.profraw' clickhouse-test --testname --shard --zookeeper --no-stateless "$SKIP_LIST_OPT" "$ADDITIONAL_OPTIONS" "$SKIP_TESTS_OPTION" 2>&1 | ts '%Y-%m-%d %H:%M:%S' | tee test_output/test_result.txt kill_clickhouse diff --git a/docker/test/stateless/run.sh b/docker/test/stateless/run.sh index 9f2bb9bf62d..8b8612d4211 100755 --- a/docker/test/stateless/run.sh +++ b/docker/test/stateless/run.sh @@ -13,8 +13,8 @@ dpkg -i package_folder/clickhouse-test_*.deb service clickhouse-server start && sleep 5 -if cat /usr/bin/clickhouse-test | grep -q -- "--use-skip-list"; then +if grep -q -- "--use-skip-list" /usr/bin/clickhouse-test; then SKIP_LIST_OPT="--use-skip-list" fi -clickhouse-test --testname --shard --zookeeper "$SKIP_LIST_OPT" $ADDITIONAL_OPTIONS $SKIP_TESTS_OPTION 2>&1 | ts '%Y-%m-%d %H:%M:%S' | tee test_output/test_result.txt +clickhouse-test --testname --shard --zookeeper "$SKIP_LIST_OPT" "$ADDITIONAL_OPTIONS" "$SKIP_TESTS_OPTION" 2>&1 | ts '%Y-%m-%d %H:%M:%S' | tee test_output/test_result.txt diff --git a/docker/test/stateless_unbundled/run.sh b/docker/test/stateless_unbundled/run.sh index 9f2bb9bf62d..8b8612d4211 100755 --- a/docker/test/stateless_unbundled/run.sh +++ b/docker/test/stateless_unbundled/run.sh @@ -13,8 +13,8 @@ dpkg -i package_folder/clickhouse-test_*.deb service clickhouse-server start && sleep 5 -if cat /usr/bin/clickhouse-test | grep -q -- "--use-skip-list"; then +if grep -q -- "--use-skip-list" /usr/bin/clickhouse-test; then SKIP_LIST_OPT="--use-skip-list" fi -clickhouse-test --testname --shard --zookeeper "$SKIP_LIST_OPT" $ADDITIONAL_OPTIONS $SKIP_TESTS_OPTION 2>&1 | ts '%Y-%m-%d %H:%M:%S' | tee test_output/test_result.txt +clickhouse-test --testname --shard --zookeeper "$SKIP_LIST_OPT" "$ADDITIONAL_OPTIONS" "$SKIP_TESTS_OPTION" 2>&1 | ts '%Y-%m-%d %H:%M:%S' | tee test_output/test_result.txt diff --git a/docker/test/stateless_with_coverage/run.sh b/docker/test/stateless_with_coverage/run.sh index 2f3f05a335a..338915c9d3a 100755 --- a/docker/test/stateless_with_coverage/run.sh +++ b/docker/test/stateless_with_coverage/run.sh @@ -1,24 +1,24 @@ #!/bin/bash kill_clickhouse () { - echo "clickhouse pids" `ps aux | grep clickhouse` | ts '%Y-%m-%d %H:%M:%S' - kill `pgrep -u clickhouse` 2>/dev/null + echo "clickhouse pids $(pgrep -u clickhouse)" | ts '%Y-%m-%d %H:%M:%S' + kill "$(pgrep -u clickhouse)" 2>/dev/null - for i in {1..10} + for _ in {1..10} do - if ! kill -0 `pgrep -u clickhouse`; then + if ! kill -0 "$(pgrep -u clickhouse)"; then echo "No clickhouse process" | ts '%Y-%m-%d %H:%M:%S' break else - echo "Process" `pgrep -u clickhouse` "still alive" | ts '%Y-%m-%d %H:%M:%S' + echo "Process $(pgrep -u clickhouse) still alive" | ts '%Y-%m-%d %H:%M:%S' sleep 10 fi done echo "Will try to send second kill signal for sure" - kill `pgrep -u clickhouse` 2>/dev/null + kill "$(pgrep -u clickhouse)" 2>/dev/null sleep 5 - echo "clickhouse pids" `ps aux | grep clickhouse` | ts '%Y-%m-%d %H:%M:%S' + echo "clickhouse pids $(pgrep -u clickhouse)" | ts '%Y-%m-%d %H:%M:%S' } start_clickhouse () { @@ -47,11 +47,11 @@ start_clickhouse sleep 10 -if cat /usr/bin/clickhouse-test | grep -q -- "--use-skip-list"; then +if grep -q -- "--use-skip-list" /usr/bin/clickhouse-test; then SKIP_LIST_OPT="--use-skip-list" fi -LLVM_PROFILE_FILE='client_coverage.profraw' clickhouse-test --testname --shard --zookeeper "$SKIP_LIST_OPT" $ADDITIONAL_OPTIONS $SKIP_TESTS_OPTION 2>&1 | ts '%Y-%m-%d %H:%M:%S' | tee test_output/test_result.txt +LLVM_PROFILE_FILE='client_coverage.profraw' clickhouse-test --testname --shard --zookeeper "$SKIP_LIST_OPT" "$ADDITIONAL_OPTIONS" "$SKIP_TESTS_OPTION" 2>&1 | ts '%Y-%m-%d %H:%M:%S' | tee test_output/test_result.txt kill_clickhouse diff --git a/docker/test/stress/run.sh b/docker/test/stress/run.sh index 14f0301d43e..bea246eb536 100755 --- a/docker/test/stress/run.sh +++ b/docker/test/stress/run.sh @@ -13,7 +13,7 @@ function stop() timeout 120 service clickhouse-server stop # Wait for process to disappear from processlist and also try to kill zombies. - while kill -9 $(pidof clickhouse-server) + while kill -9 "$(pidof clickhouse-server)" do echo "Killed clickhouse-server" sleep 0.5 @@ -35,7 +35,7 @@ function start() fi timeout 120 service clickhouse-server start sleep 0.5 - counter=$(($counter + 1)) + counter=$((counter + 1)) done } @@ -49,7 +49,7 @@ export ASAN_OPTIONS='malloc_context_size=10 verbosity=1 allocator_release_to_os_ start -/s3downloader --dataset-names $DATASETS +/s3downloader --dataset-names "$DATASETS" chmod 777 -R /var/lib/clickhouse clickhouse-client --query "ATTACH DATABASE IF NOT EXISTS datasets ENGINE = Ordinary" clickhouse-client --query "CREATE DATABASE IF NOT EXISTS test" diff --git a/docker/test/test_runner.sh b/docker/test/test_runner.sh index d9bca8f0037..cd6367b2964 100755 --- a/docker/test/test_runner.sh +++ b/docker/test/test_runner.sh @@ -2,17 +2,19 @@ set -e -x +# Not sure why shellcheck complains that rc is not assigned before it is referenced. +# shellcheck disable=SC2154 trap 'rc=$?; echo EXITED WITH: $rc; exit $rc' EXIT # CLI option to prevent rebuilding images, just re-run tests with images leftover from previuos time readonly NO_REBUILD_FLAG="--no-rebuild" -readonly CLICKHOUSE_DOCKER_DIR="$(realpath ${1})" +readonly CLICKHOUSE_DOCKER_DIR="$(realpath "${1}")" readonly CLICKHOUSE_PACKAGES_ARG="${2}" CLICKHOUSE_SERVER_IMAGE="${3}" -if [ ${CLICKHOUSE_PACKAGES_ARG} != ${NO_REBUILD_FLAG} ]; then - readonly CLICKHOUSE_PACKAGES_DIR="$(realpath ${2})" # or --no-rebuild +if [ "${CLICKHOUSE_PACKAGES_ARG}" != "${NO_REBUILD_FLAG}" ]; then + readonly CLICKHOUSE_PACKAGES_DIR="$(realpath "${2}")" # or --no-rebuild fi @@ -25,7 +27,7 @@ fi # TODO: optionally mount most recent clickhouse-test and queries directory from local machine -if [ ${CLICKHOUSE_PACKAGES_ARG} != ${NO_REBUILD_FLAG} ]; then +if [ "${CLICKHOUSE_PACKAGES_ARG}" != "${NO_REBUILD_FLAG}" ]; then docker build --network=host \ -f "${CLICKHOUSE_DOCKER_DIR}/test/stateless/clickhouse-statelest-test-runner.Dockerfile" \ --target clickhouse-test-runner-base \ @@ -49,7 +51,7 @@ fi if [ -z "${CLICKHOUSE_SERVER_IMAGE}" ]; then CLICKHOUSE_SERVER_IMAGE="yandex/clickhouse-server:local" - if [ ${CLICKHOUSE_PACKAGES_ARG} != ${NO_REBUILD_FLAG} ]; then + if [ "${CLICKHOUSE_PACKAGES_ARG}" != "${NO_REBUILD_FLAG}" ]; then docker build --network=host \ -f "${CLICKHOUSE_DOCKER_DIR}/server/local.Dockerfile" \ --target clickhouse-server-base \ diff --git a/docker/test/testflows/runner/dockerd-entrypoint.sh b/docker/test/testflows/runner/dockerd-entrypoint.sh index b10deaded08..1bac94a9df2 100755 --- a/docker/test/testflows/runner/dockerd-entrypoint.sh +++ b/docker/test/testflows/runner/dockerd-entrypoint.sh @@ -7,7 +7,7 @@ set +e reties=0 while true; do docker info &>/dev/null && break - reties=$[$reties+1] + reties=$((reties+1)) if [[ $reties -ge 100 ]]; then # 10 sec max echo "Can't start docker daemon, timeout exceeded." >&2 exit 1; diff --git a/utils/check-style/check-style b/utils/check-style/check-style index f745175a07f..3d2d99745e3 100755 --- a/utils/check-style/check-style +++ b/utils/check-style/check-style @@ -76,6 +76,9 @@ CURDIR=$(dirname "${BASH_SOURCE[0]}") # Check sh tests with Shellcheck (cd $ROOT_PATH/tests/queries/0_stateless/ && shellcheck --check-sourced --external-sources --severity info --exclude SC1071,SC2086 *.sh ../1_stateful/*.sh) +# Check docker scripts with shellcheck +find "$ROOT_PATH/docker" -executable -type f -exec file -F' ' --mime-type {} \; | awk -F' ' '$2==" text/x-shellscript" {print $1}' | xargs shellcheck + # There shouldn't be any docker containers outside docker directory find $ROOT_PATH -not -path $ROOT_PATH'/docker*' -not -path $ROOT_PATH'/contrib*' -name Dockerfile -type f 2>/dev/null | xargs --no-run-if-empty -n1 echo "Please move Dockerfile to docker directory:" From e45edd57f2320e9f6d701949361215a092ec5838 Mon Sep 17 00:00:00 2001 From: Alexander Kuzmenkov Date: Wed, 30 Sep 2020 20:19:32 +0300 Subject: [PATCH 246/321] Add reinterpretAsUUID function --- src/Common/Exception.h | 7 ++++-- src/Core/BaseSettings.h | 22 +++++++++++++------ src/Functions/reinterpretStringAs.cpp | 8 +++++-- .../queries/0_stateless/00396_uuid.reference | 1 + tests/queries/0_stateless/00396_uuid.sql | 6 +++++ 5 files changed, 33 insertions(+), 11 deletions(-) diff --git a/src/Common/Exception.h b/src/Common/Exception.h index 763b90048bb..d0de8d6a3f2 100644 --- a/src/Common/Exception.h +++ b/src/Common/Exception.h @@ -39,8 +39,11 @@ public: const char * name() const throw() override { return "DB::Exception"; } const char * what() const throw() override { return message().data(); } - /// Add something to the existing message. - void addMessage(const std::string & arg) { extendedMessage(arg); } + template + void addMessage(Fmt&&... fmt) + { + extendedMessage(fmt::format(std::forward(fmt)...)); + } std::string getStackTraceString() const; diff --git a/src/Core/BaseSettings.h b/src/Core/BaseSettings.h index 7de87b345c1..b193fdd4c93 100644 --- a/src/Core/BaseSettings.h +++ b/src/Core/BaseSettings.h @@ -390,13 +390,21 @@ String BaseSettings::valueToStringUtil(const std::string_view & name, c template Field BaseSettings::stringToValueUtil(const std::string_view & name, const String & str) { - const auto & accessor = Traits::Accessor::instance(); - if (size_t index = accessor.find(name); index != static_cast(-1)) - return accessor.stringToValueUtil(index, str); - if constexpr (Traits::allow_custom_settings) - return Field::restoreFromDump(str); - else - BaseSettingsHelpers::throwSettingNotFound(name); + try + { + const auto & accessor = Traits::Accessor::instance(); + if (size_t index = accessor.find(name); index != static_cast(-1)) + return accessor.stringToValueUtil(index, str); + if constexpr (Traits::allow_custom_settings) + return Field::restoreFromDump(str); + else + BaseSettingsHelpers::throwSettingNotFound(name); + } + catch (Exception & e) + { + e.addMessage("while parsing value '{}' for setting '{}'", str, name); + throw; + } } template diff --git a/src/Functions/reinterpretStringAs.cpp b/src/Functions/reinterpretStringAs.cpp index 71528b7cb61..5a2757a4284 100644 --- a/src/Functions/reinterpretStringAs.cpp +++ b/src/Functions/reinterpretStringAs.cpp @@ -5,6 +5,7 @@ #include #include #include +#include #include #include #include @@ -68,7 +69,7 @@ public: size_t offset = 0; for (size_t i = 0; i < size; ++i) { - ToFieldType value = 0; + ToFieldType value{}; memcpy(&value, &data_from[offset], std::min(static_cast(sizeof(ToFieldType)), offsets_from[i] - offset - 1)); vec_res[i] = value; offset = offsets_from[i]; @@ -90,7 +91,7 @@ public: size_t copy_size = std::min(step, sizeof(ToFieldType)); for (size_t i = 0; i < size; ++i) { - ToFieldType value = 0; + ToFieldType value{}; memcpy(&value, &data_from[offset], copy_size); vec_res[i] = value; offset += step; @@ -120,6 +121,7 @@ struct NameReinterpretAsFloat32 { static constexpr auto name = "reinterpretA struct NameReinterpretAsFloat64 { static constexpr auto name = "reinterpretAsFloat64"; }; struct NameReinterpretAsDate { static constexpr auto name = "reinterpretAsDate"; }; struct NameReinterpretAsDateTime { static constexpr auto name = "reinterpretAsDateTime"; }; +struct NameReinterpretAsUUID { static constexpr auto name = "reinterpretAsUUID"; }; using FunctionReinterpretAsUInt8 = FunctionReinterpretStringAs; using FunctionReinterpretAsUInt16 = FunctionReinterpretStringAs; @@ -133,6 +135,7 @@ using FunctionReinterpretAsFloat32 = FunctionReinterpretStringAs; using FunctionReinterpretAsDate = FunctionReinterpretStringAs; using FunctionReinterpretAsDateTime = FunctionReinterpretStringAs; +using FunctionReinterpretAsUUID = FunctionReinterpretStringAs; } @@ -150,6 +153,7 @@ void registerFunctionsReinterpretStringAs(FunctionFactory & factory) factory.registerFunction(); factory.registerFunction(); factory.registerFunction(); + factory.registerFunction(); } } diff --git a/tests/queries/0_stateless/00396_uuid.reference b/tests/queries/0_stateless/00396_uuid.reference index fe92b3684a6..d70322ec4c1 100644 --- a/tests/queries/0_stateless/00396_uuid.reference +++ b/tests/queries/0_stateless/00396_uuid.reference @@ -5,3 +5,4 @@ 01234567-89ab-cdef-0123-456789abcdef 01234567-89ab-cdef-0123-456789abcdef 01234567-89ab-cdef-0123-456789abcdef 01234567-89ab-cdef-0123-456789abcdef 01234567-89ab-cdef-0123-456789abcdef 01234567-89ab-cdef-0123-456789abcdef 3f1ed72e-f7fe-4459-9cbe-95fe9298f845 +1 diff --git a/tests/queries/0_stateless/00396_uuid.sql b/tests/queries/0_stateless/00396_uuid.sql index d671ce844e2..9d8b48bddb0 100644 --- a/tests/queries/0_stateless/00396_uuid.sql +++ b/tests/queries/0_stateless/00396_uuid.sql @@ -5,3 +5,9 @@ SELECT hex(UUIDStringToNum(materialize('01234567-89ab-cdef-0123-456789abcdef'))) SELECT '01234567-89ab-cdef-0123-456789abcdef' AS str, UUIDNumToString(UUIDStringToNum(str)), UUIDNumToString(UUIDStringToNum(toFixedString(str, 36))); SELECT materialize('01234567-89ab-cdef-0123-456789abcdef') AS str, UUIDNumToString(UUIDStringToNum(str)), UUIDNumToString(UUIDStringToNum(toFixedString(str, 36))); SELECT toString(toUUID('3f1ed72e-f7fe-4459-9cbe-95fe9298f845')); + +-- conversion back and forth to big-endian hex string +with generateUUIDv4() as uuid, + identity(lower(hex(reverse(reinterpretAsString(uuid))))) as str, + reinterpretAsUUID(reverse(unhex(str))) as uuid2 +select uuid = uuid2; From eb3669e82f9c5224eb27b4f986faeafdd95ca1fc Mon Sep 17 00:00:00 2001 From: Alexander Kuzmenkov Date: Wed, 30 Sep 2020 20:39:38 +0300 Subject: [PATCH 247/321] more fixes --- docker/test/fasttest/run.sh | 9 ++++++++- 1 file changed, 8 insertions(+), 1 deletion(-) diff --git a/docker/test/fasttest/run.sh b/docker/test/fasttest/run.sh index c4a9a0ea5f3..5e586402bd7 100755 --- a/docker/test/fasttest/run.sh +++ b/docker/test/fasttest/run.sh @@ -15,13 +15,20 @@ stage=${stage:-} # empty parameter. read -ra FASTTEST_CMAKE_FLAGS <<< "${FASTTEST_CMAKE_FLAGS:-}" - FASTTEST_WORKSPACE=$(readlink -f "${FASTTEST_WORKSPACE:-.}") FASTTEST_SOURCE=$(readlink -f "${FASTTEST_SOURCE:-$FASTTEST_WORKSPACE/ch}") FASTTEST_BUILD=$(readlink -f "${FASTTEST_BUILD:-${BUILD:-$FASTTEST_WORKSPACE/build}}") FASTTEST_DATA=$(readlink -f "${FASTTEST_DATA:-$FASTTEST_WORKSPACE/db-fasttest}") FASTTEST_OUTPUT=$(readlink -f "${FASTTEST_OUTPUT:-$FASTTEST_WORKSPACE}") +# Export these variables, so that all subsequent invocations of the script +# use them, and not try to guess them anew, which leads to weird effects. +export FASTTEST_WORKSPACE +export FASTTEST_SOURCE +export FASTTEST_BUILD +export FASTTEST_DATA +export FASTTEST_OUT + server_pid=none function stop_server From ac9f0ff4da147e55b07eda3c765f56bcbf93498d Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 30 Sep 2020 22:44:35 +0300 Subject: [PATCH 248/321] More strict check for version column --- src/Storages/MergeTree/MergeTreeData.cpp | 53 +++++++++++++++---- ...ersion_versioned_collapsing_merge_tree.sql | 5 ++ 2 files changed, 48 insertions(+), 10 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 83fc75d662e..0aee7a25875 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -1416,6 +1416,49 @@ void MergeTreeData::checkAlterIsPossible(const AlterCommands & commands, const S getPartitionIDFromQuery(command.partition, global_context); } + /// Some type changes for version column is allowed despite it's a part of sorting key + if (command.type == AlterCommand::MODIFY_COLUMN && command.column_name == merging_params.version_column) + { + auto new_type = command.data_type; + auto old_type = old_types[command.column_name]; + /// Check new type can be used as version + if (!new_type->canBeUsedAsVersion()) + throw Exception("Cannot alter version column " + backQuoteIfNeed(command.column_name) + + " to type " + new_type->getName() + + " because version column must be of an integer type or of type Date or DateTime" + , ErrorCodes::ALTER_OF_COLUMN_IS_FORBIDDEN); + + auto which_new_type = WhichDataType(new_type); + auto which_old_type = WhichDataType(old_type); + + /// Check alter to different sign or float -> int and so on + if ((which_old_type.isInt() && !which_new_type.isInt()) + || (which_old_type.isUInt() && !which_new_type.isUInt()) + || (which_old_type.isDate() && !which_new_type.isDate()) + || (which_old_type.isDateTime() && !which_new_type.isDateTime()) + || (which_old_type.isFloat() && !which_new_type.isFloat())) + { + throw Exception("Cannot alter version column " + backQuoteIfNeed(command.column_name) + + " from type " + old_type->getName() + + " to type " + new_type->getName() + " because new type will change sort order of version column." + + " The only possible conversion is expansion of the number of bytes of the current type." + , ErrorCodes::ALTER_OF_COLUMN_IS_FORBIDDEN); + } + + /// Check alter to smaller size: UInt64 -> UInt32 and so on + if (new_type->getSizeOfValueInMemory() < old_type->getSizeOfValueInMemory()) + { + throw Exception("Cannot alter version column " + backQuoteIfNeed(command.column_name) + + " from type " + old_type->getName() + + " to type " + new_type->getName() + " because new type is smaller than current in the number of bytes." + + " The only possible conversion is expansion of the number of bytes of the current type." + , ErrorCodes::ALTER_OF_COLUMN_IS_FORBIDDEN); + } + + /// Positive case, alter allowed + continue; + } + if (command.type == AlterCommand::MODIFY_ORDER_BY && !is_custom_partitioned) { throw Exception( @@ -1458,16 +1501,6 @@ void MergeTreeData::checkAlterIsPossible(const AlterCommands & commands, const S } else if (command.isRequireMutationStage(getInMemoryMetadata())) { - /// Type change for version column is allowed despite it's a part of sorting key - if (command.type == AlterCommand::MODIFY_COLUMN && command.column_name == merging_params.version_column) - { - if (!command.data_type->canBeUsedAsVersion()) - throw Exception("Cannot alter version column " + backQuoteIfNeed(command.column_name) + - " to type " + command.data_type->getName() + - " because version column must be of an integer type or of type Date or DateTime" - , ErrorCodes::ALTER_OF_COLUMN_IS_FORBIDDEN); - continue; - } /// This alter will override data on disk. Let's check that it doesn't /// modify immutable column. if (columns_alter_type_forbidden.count(command.column_name)) diff --git a/tests/queries/0_stateless/01511_alter_version_versioned_collapsing_merge_tree.sql b/tests/queries/0_stateless/01511_alter_version_versioned_collapsing_merge_tree.sql index 62cb4e77640..8f0b2d12ab0 100644 --- a/tests/queries/0_stateless/01511_alter_version_versioned_collapsing_merge_tree.sql +++ b/tests/queries/0_stateless/01511_alter_version_versioned_collapsing_merge_tree.sql @@ -37,5 +37,10 @@ INSERT INTO TABLE table_with_version VALUES(3, '3', 65555, -1); SELECT * FROM table_with_version FINAL ORDER BY key; ALTER TABLE table_with_version MODIFY COLUMN version String; --{serverError 524} +ALTER TABLE table_with_version MODIFY COLUMN version Int64; --{serverError 524} +ALTER TABLE table_with_version MODIFY COLUMN version UInt16; --{serverError 524} +ALTER TABLE table_with_version MODIFY COLUMN version Float64; --{serverError 524} +ALTER TABLE table_with_version MODIFY COLUMN version Date; --{serverError 524} +ALTER TABLE table_with_version MODIFY COLUMN version DateTime; --{serverError 524} DROP TABLE IF EXISTS table_with_version; From 97c2579fb9da90d9e5ee8d018abb12faa7801703 Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 30 Sep 2020 22:50:23 +0300 Subject: [PATCH 249/321] Use separate function --- src/Storages/MergeTree/MergeTreeData.cpp | 73 +++++++++++++----------- 1 file changed, 40 insertions(+), 33 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 0aee7a25875..3f09f3a7347 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -1330,6 +1330,44 @@ bool isSafeForPartitionKeyConversion(const IDataType * from, const IDataType * t return false; } +/// Special check for alters of VersionedCollapsingMergeTree version column +void checkVersionColumnTypesConversion(const IDataType * old_type, const IDataType * new_type, const String column_name) +{ + /// Check new type can be used as version + if (!new_type->canBeUsedAsVersion()) + throw Exception("Cannot alter version column " + backQuoteIfNeed(column_name) + + " to type " + new_type->getName() + + " because version column must be of an integer type or of type Date or DateTime" + , ErrorCodes::ALTER_OF_COLUMN_IS_FORBIDDEN); + + auto which_new_type = WhichDataType(new_type); + auto which_old_type = WhichDataType(old_type); + + /// Check alter to different sign or float -> int and so on + if ((which_old_type.isInt() && !which_new_type.isInt()) + || (which_old_type.isUInt() && !which_new_type.isUInt()) + || (which_old_type.isDate() && !which_new_type.isDate()) + || (which_old_type.isDateTime() && !which_new_type.isDateTime()) + || (which_old_type.isFloat() && !which_new_type.isFloat())) + { + throw Exception("Cannot alter version column " + backQuoteIfNeed(column_name) + + " from type " + old_type->getName() + + " to type " + new_type->getName() + " because new type will change sort order of version column." + + " The only possible conversion is expansion of the number of bytes of the current type." + , ErrorCodes::ALTER_OF_COLUMN_IS_FORBIDDEN); + } + + /// Check alter to smaller size: UInt64 -> UInt32 and so on + if (new_type->getSizeOfValueInMemory() < old_type->getSizeOfValueInMemory()) + { + throw Exception("Cannot alter version column " + backQuoteIfNeed(column_name) + + " from type " + old_type->getName() + + " to type " + new_type->getName() + " because new type is smaller than current in the number of bytes." + + " The only possible conversion is expansion of the number of bytes of the current type." + , ErrorCodes::ALTER_OF_COLUMN_IS_FORBIDDEN); + } +} + } void MergeTreeData::checkAlterIsPossible(const AlterCommands & commands, const Settings & settings) const @@ -1421,41 +1459,10 @@ void MergeTreeData::checkAlterIsPossible(const AlterCommands & commands, const S { auto new_type = command.data_type; auto old_type = old_types[command.column_name]; - /// Check new type can be used as version - if (!new_type->canBeUsedAsVersion()) - throw Exception("Cannot alter version column " + backQuoteIfNeed(command.column_name) + - " to type " + new_type->getName() + - " because version column must be of an integer type or of type Date or DateTime" - , ErrorCodes::ALTER_OF_COLUMN_IS_FORBIDDEN); - auto which_new_type = WhichDataType(new_type); - auto which_old_type = WhichDataType(old_type); + checkVersionColumnTypesConversion(old_type, new_type.get(), command.column_name); - /// Check alter to different sign or float -> int and so on - if ((which_old_type.isInt() && !which_new_type.isInt()) - || (which_old_type.isUInt() && !which_new_type.isUInt()) - || (which_old_type.isDate() && !which_new_type.isDate()) - || (which_old_type.isDateTime() && !which_new_type.isDateTime()) - || (which_old_type.isFloat() && !which_new_type.isFloat())) - { - throw Exception("Cannot alter version column " + backQuoteIfNeed(command.column_name) + - " from type " + old_type->getName() + - " to type " + new_type->getName() + " because new type will change sort order of version column." + - " The only possible conversion is expansion of the number of bytes of the current type." - , ErrorCodes::ALTER_OF_COLUMN_IS_FORBIDDEN); - } - - /// Check alter to smaller size: UInt64 -> UInt32 and so on - if (new_type->getSizeOfValueInMemory() < old_type->getSizeOfValueInMemory()) - { - throw Exception("Cannot alter version column " + backQuoteIfNeed(command.column_name) + - " from type " + old_type->getName() + - " to type " + new_type->getName() + " because new type is smaller than current in the number of bytes." + - " The only possible conversion is expansion of the number of bytes of the current type." - , ErrorCodes::ALTER_OF_COLUMN_IS_FORBIDDEN); - } - - /// Positive case, alter allowed + /// No other checks required continue; } From 0ac18a382ffb00358e1da53132c8772c80ba0bf9 Mon Sep 17 00:00:00 2001 From: Artem Zuikov Date: Wed, 30 Sep 2020 23:11:27 +0300 Subject: [PATCH 250/321] Fix JOIN with GROUP BY over StorageMerge (#15242) --- src/Storages/StorageMerge.cpp | 38 ++++++++++++++++--- ...83_merge_table_join_and_group_by.reference | 7 ++++ .../01483_merge_table_join_and_group_by.sql | 22 +++++++++++ 3 files changed, 61 insertions(+), 6 deletions(-) create mode 100644 tests/queries/0_stateless/01483_merge_table_join_and_group_by.reference create mode 100644 tests/queries/0_stateless/01483_merge_table_join_and_group_by.sql diff --git a/src/Storages/StorageMerge.cpp b/src/Storages/StorageMerge.cpp index 42ac9cb6a5b..bade0810320 100644 --- a/src/Storages/StorageMerge.cpp +++ b/src/Storages/StorageMerge.cpp @@ -11,6 +11,8 @@ #include #include #include +#include +#include #include #include #include @@ -41,19 +43,42 @@ namespace { /// Rewrite original query removing joined tables from it -void removeJoin(const ASTSelectQuery & select) +bool removeJoin(ASTSelectQuery & select) { const auto & tables = select.tables(); if (!tables || tables->children.size() < 2) - return; + return false; const auto & joined_table = tables->children[1]->as(); if (!joined_table.table_join) - return; + return false; /// The most simple temporary solution: leave only the first table in query. /// TODO: we also need to remove joined columns and related functions (taking in account aliases if any). tables->children.resize(1); + return true; +} + +void modifySelect(ASTSelectQuery & select, const TreeRewriterResult & rewriter_result) +{ + if (removeJoin(select)) + { + /// Also remove GROUP BY cause ExpressionAnalyzer would check if it has all aggregate columns but joined columns would be missed. + select.setExpression(ASTSelectQuery::Expression::GROUP_BY, {}); + + /// Replace select list to remove joined columns + auto select_list = std::make_shared(); + for (const auto & column : rewriter_result.required_source_columns) + select_list->children.emplace_back(std::make_shared(column.name)); + + select.setExpression(ASTSelectQuery::Expression::SELECT, select_list); + + /// TODO: keep WHERE/PREWHERE. We have to remove joined columns and their expressions but keep others. + select.setExpression(ASTSelectQuery::Expression::WHERE, {}); + select.setExpression(ASTSelectQuery::Expression::PREWHERE, {}); + select.setExpression(ASTSelectQuery::Expression::HAVING, {}); + select.setExpression(ASTSelectQuery::Expression::ORDER_BY, {}); + } } } @@ -265,7 +290,8 @@ Pipe StorageMerge::createSources( modified_query_info.query = query_info.query->clone(); /// Original query could contain JOIN but we need only the first joined table and its columns. - removeJoin(*modified_query_info.query->as()); + auto & modified_select = modified_query_info.query->as(); + modifySelect(modified_select, *query_info.syntax_analyzer_result); VirtualColumnUtils::rewriteEntityInAst(modified_query_info.query, "_table", table_name); @@ -282,7 +308,7 @@ Pipe StorageMerge::createSources( return pipe; } - auto storage_stage = storage->getQueryProcessingStage(*modified_context, QueryProcessingStage::Complete, query_info.query); + auto storage_stage = storage->getQueryProcessingStage(*modified_context, QueryProcessingStage::Complete, modified_query_info.query); if (processed_stage <= storage_stage) { /// If there are only virtual columns in query, you must request at least one other column. @@ -294,7 +320,7 @@ Pipe StorageMerge::createSources( } else if (processed_stage > storage_stage) { - modified_query_info.query->as()->replaceDatabaseAndTable(source_database, table_name); + modified_select.replaceDatabaseAndTable(source_database, table_name); /// Maximum permissible parallelism is streams_num modified_context->setSetting("max_threads", streams_num); diff --git a/tests/queries/0_stateless/01483_merge_table_join_and_group_by.reference b/tests/queries/0_stateless/01483_merge_table_join_and_group_by.reference new file mode 100644 index 00000000000..b2c3ea56b7f --- /dev/null +++ b/tests/queries/0_stateless/01483_merge_table_join_and_group_by.reference @@ -0,0 +1,7 @@ +0 1 +0 1 +0 1 +0 1 +1 +0 1 +0 1 diff --git a/tests/queries/0_stateless/01483_merge_table_join_and_group_by.sql b/tests/queries/0_stateless/01483_merge_table_join_and_group_by.sql new file mode 100644 index 00000000000..a6678ca9040 --- /dev/null +++ b/tests/queries/0_stateless/01483_merge_table_join_and_group_by.sql @@ -0,0 +1,22 @@ +DROP TABLE IF EXISTS a; +DROP TABLE IF EXISTS b; +DROP TABLE IF EXISTS m; + +CREATE TABLE a (key UInt32) ENGINE = MergeTree ORDER BY key; +CREATE TABLE b (key UInt32, ID UInt32) ENGINE = MergeTree ORDER BY key; +CREATE TABLE m (key UInt32) ENGINE = Merge(currentDatabase(), 'a'); + +INSERT INTO a VALUES (0); +INSERT INTO b VALUES (0, 1); + +SELECT * FROM m INNER JOIN b USING(key); +SELECT * FROM a INNER JOIN b USING(key) GROUP BY ID, key; +SELECT * FROM m INNER JOIN b USING(key) WHERE ID = 1; +SELECT * FROM m INNER JOIN b USING(key) GROUP BY ID, key; +SELECT ID FROM m INNER JOIN b USING(key) GROUP BY ID; +SELECT * FROM m INNER JOIN b USING(key) WHERE ID = 1 HAVING ID = 1 ORDER BY ID; +SELECT * FROM m INNER JOIN b USING(key) WHERE ID = 1 GROUP BY ID, key HAVING ID = 1 ORDER BY ID; + +DROP TABLE IF EXISTS a; +DROP TABLE IF EXISTS b; +DROP TABLE IF EXISTS m; From 0f6a6453fee773d4d38d3f85ba6b360958fc9435 Mon Sep 17 00:00:00 2001 From: Artem Zuikov Date: Wed, 30 Sep 2020 23:11:49 +0300 Subject: [PATCH 251/321] Remove multiple_joins_rewriter v1 (#15472) --- src/Core/Settings.h | 2 +- src/Interpreters/AsteriskSemantic.h | 41 --- src/Interpreters/InterpreterSelectQuery.cpp | 11 +- .../JoinToSubqueryTransformVisitor.cpp | 261 +--------------- .../JoinToSubqueryTransformVisitor.h | 6 +- .../TranslateQualifiedNamesVisitor.cpp | 25 +- src/Parsers/ASTAsterisk.h | 8 - src/Parsers/ASTQualifiedAsterisk.h | 8 - .../00847_multiple_join_same_column.reference | 12 - .../00847_multiple_join_same_column.sql | 22 -- .../00849_multiple_comma_join.reference | 292 ------------------ .../0_stateless/00849_multiple_comma_join.sql | 69 ----- .../00849_multiple_comma_join_2.sql | 1 - .../00854_multiple_join_asterisks.sql | 8 - .../01116_cross_count_asterisks.reference | 2 - .../01116_cross_count_asterisks.sql | 17 - ...rewrite_with_ambiguous_column_and_view.sql | 1 - ...multiple_joins_rewriter_v2_and_lambdas.sql | 2 - 18 files changed, 14 insertions(+), 774 deletions(-) delete mode 100644 src/Interpreters/AsteriskSemantic.h delete mode 100644 tests/queries/0_stateless/00849_multiple_comma_join.reference delete mode 100644 tests/queries/0_stateless/00849_multiple_comma_join.sql diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 19738d826c6..bc974433073 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -368,7 +368,6 @@ class IColumn; \ M(Bool, deduplicate_blocks_in_dependent_materialized_views, false, "Should deduplicate blocks for materialized views if the block is not a duplicate for the table. Use true to always deduplicate in dependent tables.", 0) \ M(Bool, use_compact_format_in_distributed_parts_names, false, "Changes format of directories names for distributed table insert parts.", 0) \ - M(UInt64, multiple_joins_rewriter_version, 2, "1 or 2. Second rewriter version knows about table columns and keep not clashed names as is.", 0) \ M(Bool, validate_polygons, true, "Throw exception if polygon is invalid in function pointInPolygon (e.g. self-tangent, self-intersecting). If the setting is false, the function will accept invalid polygons but may silently return wrong result.", 0) \ M(UInt64, max_parser_depth, DBMS_DEFAULT_MAX_PARSER_DEPTH, "Maximum parser depth (recursion depth of recursive descend parser).", 0) \ M(Seconds, temporary_live_view_timeout, DEFAULT_TEMPORARY_LIVE_VIEW_TIMEOUT_SEC, "Timeout after which temporary live view is deleted.", 0) \ @@ -399,6 +398,7 @@ class IColumn; M(UInt64, mark_cache_min_lifetime, 0, "Obsolete setting, does nothing. Will be removed after 2020-05-31", 0) \ M(Bool, partial_merge_join, false, "Obsolete. Use join_algorithm='prefer_partial_merge' instead.", 0) \ M(UInt64, max_memory_usage_for_all_queries, 0, "Obsolete. Will be removed after 2020-10-20", 0) \ + M(UInt64, multiple_joins_rewriter_version, 0, "Obsolete setting, does nothing. Will be removed after 2021-03-31", 0) \ \ M(Bool, force_optimize_skip_unused_shards_no_nested, false, "Obsolete setting, does nothing. Will be removed after 2020-12-01. Use force_optimize_skip_unused_shards_nesting instead.", 0) \ M(Bool, experimental_use_processors, true, "Obsolete setting, does nothing. Will be removed after 2020-11-29.", 0) \ diff --git a/src/Interpreters/AsteriskSemantic.h b/src/Interpreters/AsteriskSemantic.h deleted file mode 100644 index 1bd9ecedddd..00000000000 --- a/src/Interpreters/AsteriskSemantic.h +++ /dev/null @@ -1,41 +0,0 @@ -#pragma once - -#include - -#include -#include -#include - -namespace DB -{ - -struct AsteriskSemanticImpl -{ - using RevertedAliases = std::unordered_map>; - using RevertedAliasesPtr = std::shared_ptr; - - RevertedAliasesPtr aliases; /// map of aliases that should be set in phase of * expanding. -}; - - -struct AsteriskSemantic -{ - using RevertedAliases = AsteriskSemanticImpl::RevertedAliases; - using RevertedAliasesPtr = AsteriskSemanticImpl::RevertedAliasesPtr; - - static void setAliases(ASTAsterisk & node, const RevertedAliasesPtr & aliases) { node.semantic = makeSemantic(aliases); } - static void setAliases(ASTQualifiedAsterisk & node, const RevertedAliasesPtr & aliases) { node.semantic = makeSemantic(aliases); } - static void setAliases(ASTColumnsMatcher & node, const RevertedAliasesPtr & aliases) { node.semantic = makeSemantic(aliases); } - - static RevertedAliasesPtr getAliases(const ASTAsterisk & node) { return node.semantic ? node.semantic->aliases : nullptr; } - static RevertedAliasesPtr getAliases(const ASTQualifiedAsterisk & node) { return node.semantic ? node.semantic->aliases : nullptr; } - static RevertedAliasesPtr getAliases(const ASTColumnsMatcher & node) { return node.semantic ? node.semantic->aliases : nullptr; } - -private: - static std::shared_ptr makeSemantic(const RevertedAliasesPtr & aliases) - { - return std::make_shared(AsteriskSemanticImpl{aliases}); - } -}; - -} diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index 823808759a2..556070d0360 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -93,7 +93,6 @@ namespace ErrorCodes extern const int PARAMETER_OUT_OF_BOUND; extern const int INVALID_LIMIT_EXPRESSION; extern const int INVALID_WITH_FILL_EXPRESSION; - extern const int INVALID_SETTING_VALUE; } /// Assumes `storage` is set and the table filter (row-level security) is not empty. @@ -190,7 +189,7 @@ static Context getSubqueryContext(const Context & context) return subquery_context; } -static void rewriteMultipleJoins(ASTPtr & query, const TablesWithColumns & tables, const String & database, const Settings & settings) +static void rewriteMultipleJoins(ASTPtr & query, const TablesWithColumns & tables, const String & database) { ASTSelectQuery & select = query->as(); @@ -202,11 +201,7 @@ static void rewriteMultipleJoins(ASTPtr & query, const TablesWithColumns & table CrossToInnerJoinVisitor::Data cross_to_inner{tables, aliases, database}; CrossToInnerJoinVisitor(cross_to_inner).visit(query); - size_t rewriter_version = settings.multiple_joins_rewriter_version; - if (!rewriter_version || rewriter_version > 2) - throw Exception("Bad multiple_joins_rewriter_version setting value: " + settings.multiple_joins_rewriter_version.toString(), - ErrorCodes::INVALID_SETTING_VALUE); - JoinToSubqueryTransformVisitor::Data join_to_subs_data{tables, aliases, rewriter_version}; + JoinToSubqueryTransformVisitor::Data join_to_subs_data{tables, aliases}; JoinToSubqueryTransformVisitor(join_to_subs_data).visit(query); } @@ -271,7 +266,7 @@ InterpreterSelectQuery::InterpreterSelectQuery( /// Rewrite JOINs if (!has_input && joined_tables.tablesCount() > 1) { - rewriteMultipleJoins(query_ptr, joined_tables.tablesWithColumns(), context->getCurrentDatabase(), settings); + rewriteMultipleJoins(query_ptr, joined_tables.tablesWithColumns(), context->getCurrentDatabase()); joined_tables.reset(getSelectQuery()); joined_tables.resolveTables(); diff --git a/src/Interpreters/JoinToSubqueryTransformVisitor.cpp b/src/Interpreters/JoinToSubqueryTransformVisitor.cpp index 5f38f410e04..cdd7ec3ebf9 100644 --- a/src/Interpreters/JoinToSubqueryTransformVisitor.cpp +++ b/src/Interpreters/JoinToSubqueryTransformVisitor.cpp @@ -2,7 +2,6 @@ #include #include #include -#include #include #include #include @@ -11,6 +10,8 @@ #include #include #include +#include +#include #include #include #include @@ -127,169 +128,6 @@ private: } }; -/// Find columns with aliases to push them into rewritten subselects. -/// Normalize table aliases: table_name.column_name -> table_alias.column_name -/// Make aliases maps (alias -> column_name, column_name -> alias) -struct ColumnAliasesMatcher -{ - using Visitor = ConstInDepthNodeVisitor; - - struct Data - { - const std::vector tables; - bool public_names; - AsteriskSemantic::RevertedAliases rev_aliases; /// long_name -> aliases - std::unordered_map aliases; /// alias -> long_name - std::vector> compound_identifiers; - std::set allowed_long_names; /// original names allowed as aliases '--t.x as t.x' (select expressions only). - bool inside_function = false; - - explicit Data(const std::vector && tables_) - : tables(tables_) - , public_names(false) - {} - - void replaceIdentifiersWithAliases() - { - String hide_prefix = "--"; /// @note restriction: user should not use aliases like `--table.column` - - for (auto & [identifier, is_public] : compound_identifiers) - { - String long_name = identifier->name; - - auto it = rev_aliases.find(long_name); - if (it == rev_aliases.end()) - { - bool last_table = false; - { - if (auto best_table_pos = IdentifierSemantic::chooseTable(*identifier, tables)) - last_table = (*best_table_pos + 1 == tables.size()); - } - - if (!last_table) - { - String alias = hide_prefix + long_name; - aliases[alias] = long_name; - rev_aliases[long_name].push_back(alias); - - IdentifierSemantic::coverName(*identifier, alias); - if (is_public) - { - identifier->setAlias(long_name); - allowed_long_names.insert(long_name); - } - } - else if (is_public) - identifier->setAlias(long_name); /// prevent crop long to short name - } - else - { - if (it->second.empty()) - throw Exception("No alias for '" + long_name + "'", ErrorCodes::LOGICAL_ERROR); - - if (is_public && allowed_long_names.count(long_name)) - ; /// leave original name unchanged for correct output - else - IdentifierSemantic::coverName(*identifier, it->second[0]); - } - } - } - }; - - static bool needChildVisit(const ASTPtr & node, const ASTPtr &) - { - /// Do not go into subqueries. Function visits children itself. - if (node->as() || - node->as()) - return false; - return !node->as(); - } - - static void visit(const ASTPtr & ast, Data & data) - { - if (auto * t = ast->as()) - visit(*t, ast, data); - else if (auto * f = ast->as()) - visit(*f, ast, data); - - /// Do not allow asterisks but ignore them inside functions. I.e. allow 'count(*)'. - if (!data.inside_function && (ast->as() || ast->as())) - throw Exception("Multiple JOIN do not support asterisks for complex queries yet", ErrorCodes::NOT_IMPLEMENTED); - } - - static void visit(const ASTFunction &, const ASTPtr & ast, Data & data) - { - /// Grandchild case: Function -> (ExpressionList) -> Asterisk - data.inside_function = true; - Visitor visitor(data); - for (auto & child : ast->children) - visitor.visit(child); - data.inside_function = false; - } - - static void visit(const ASTIdentifier & const_node, const ASTPtr &, Data & data) - { - ASTIdentifier & node = const_cast(const_node); /// we know it's not const - if (node.isShort()) - return; - - bool last_table = false; - String long_name; - - if (auto table_pos = IdentifierSemantic::chooseTable(node, data.tables)) - { - const auto & table = data.tables[*table_pos]; - IdentifierSemantic::setColumnLongName(node, table); /// table_name.column_name -> table_alias.column_name - long_name = node.name; - if (&table == &data.tables.back()) - last_table = true; - } - - if (long_name.empty()) - throw Exception("Cannot refer column '" + node.name + "' to table", ErrorCodes::AMBIGUOUS_COLUMN_NAME); - - String alias = node.tryGetAlias(); - if (!alias.empty()) - { - data.aliases[alias] = long_name; - data.rev_aliases[long_name].push_back(alias); - - if (!last_table) - { - IdentifierSemantic::coverName(node, alias); - node.setAlias({}); - } - } - else if (node.compound()) - data.compound_identifiers.emplace_back(&node, data.public_names); - } -}; - -/// Attach additional semantic info to generated selects. -struct AppendSemanticVisitorData -{ - using TypeToVisit = ASTSelectQuery; - - AsteriskSemantic::RevertedAliasesPtr rev_aliases = {}; - bool done = false; - - void visit(ASTSelectQuery & select, ASTPtr &) - { - if (done || !rev_aliases || !select.select()) - return; - - for (auto & child : select.select()->children) - { - if (auto * node = child->as()) - AsteriskSemantic::setAliases(*node, rev_aliases); - if (auto * node = child->as()) - AsteriskSemantic::setAliases(*node, rev_aliases); - } - - done = true; - } -}; - /// Replaces table elements with pair. struct RewriteTablesVisitorData { @@ -371,9 +209,6 @@ bool needRewrite(ASTSelectQuery & select, std::vector; using RewriteVisitor = InDepthNodeVisitor; using ExtractAsterisksVisitor = ConstInDepthNodeVisitor; -using ColumnAliasesVisitor = ColumnAliasesMatcher::Visitor; -using AppendSemanticMatcher = OneTypeMatcher; -using AppendSemanticVisitor = InDepthNodeVisitor; /// V2 specific visitors @@ -718,12 +553,7 @@ bool JoinToSubqueryTransformMatcher::needChildVisit(ASTPtr & node, const ASTPtr void JoinToSubqueryTransformMatcher::visit(ASTPtr & ast, Data & data) { if (auto * t = ast->as()) - { - if (data.version == 1) - visitV1(*t, ast, data); - else - visitV2(*t, ast, data); - } + visit(*t, ast, data); } /// The reason for V2: not to alias columns without clashes. @@ -733,7 +563,7 @@ void JoinToSubqueryTransformMatcher::visit(ASTPtr & ast, Data & data) /// 3. Rewrite multiple JOINs with subqueries: /// SELECT ... FROM (SELECT `--.s`.*, ... FROM (...) AS `--.s` JOIN tableY ON ...) AS `--.s` JOIN tableZ ON ...' /// 4. Push down expressions of aliases used in ON section into expression list of first reletad subquery -void JoinToSubqueryTransformMatcher::visitV2(ASTSelectQuery & select, ASTPtr & ast, Data & data) +void JoinToSubqueryTransformMatcher::visit(ASTSelectQuery & select, ASTPtr & ast, Data & data) { std::vector table_expressions; if (!needRewrite<2>(select, table_expressions)) @@ -855,89 +685,6 @@ void JoinToSubqueryTransformMatcher::visitV2(ASTSelectQuery & select, ASTPtr & a data.done = true; } -void JoinToSubqueryTransformMatcher::visitV1(ASTSelectQuery & select, ASTPtr &, Data & data) -{ - using RevertedAliases = AsteriskSemantic::RevertedAliases; - - std::vector table_expressions; - if (!needRewrite(select, table_expressions)) - return; - - if (table_expressions.size() != data.tables.size()) - throw Exception("Inconsistent tables count in JOIN rewriter", ErrorCodes::LOGICAL_ERROR); - - bool has_subquery = false; - for (const auto & expr : table_expressions) - if (expr->subquery) - has_subquery = true; - - if (!has_subquery) - { - ExtractAsterisksVisitor::Data asterisks_data(data.tables); - ExtractAsterisksVisitor(asterisks_data).visit(select.select()); - if (asterisks_data.new_select_expression_list) - select.setExpression(ASTSelectQuery::Expression::SELECT, std::move(asterisks_data.new_select_expression_list)); - } - - ColumnAliasesVisitor::Data aliases_data(getDatabaseAndTables(select, "")); - if (select.select()) - { - /// TODO: there's a bug here. We need to publish only top-level ASTIdentifiers but visitor extracts all. - aliases_data.public_names = true; - ColumnAliasesVisitor(aliases_data).visit(select.select()); - aliases_data.public_names = false; - } - if (select.where()) - ColumnAliasesVisitor(aliases_data).visit(select.where()); - if (select.prewhere()) - ColumnAliasesVisitor(aliases_data).visit(select.prewhere()); - if (select.orderBy()) - ColumnAliasesVisitor(aliases_data).visit(select.orderBy()); - if (select.groupBy()) - ColumnAliasesVisitor(aliases_data).visit(select.groupBy()); - if (select.having()) - ColumnAliasesVisitor(aliases_data).visit(select.having()); - - /// JOIN sections - for (auto & child : select.tables()->children) - { - auto * table = child->as(); - if (table->table_join) - { - auto & join = table->table_join->as(); - if (join.on_expression) - ColumnAliasesVisitor(aliases_data).visit(join.on_expression); - } - } - - aliases_data.replaceIdentifiersWithAliases(); - - auto rev_aliases = std::make_shared(); - rev_aliases->swap(aliases_data.rev_aliases); - - auto & src_tables = select.tables()->children; - ASTPtr left_table = src_tables[0]; - - static ASTPtr subquery_template = makeSubqueryTemplate(); - - for (size_t i = 1; i < src_tables.size() - 1; ++i) - { - left_table = replaceJoin(left_table, src_tables[i], subquery_template->clone()); - if (!left_table) - throw Exception("Cannot replace tables with subselect", ErrorCodes::LOGICAL_ERROR); - - /// attach data to generated asterisk - AppendSemanticVisitor::Data semantic_data{rev_aliases, false}; - AppendSemanticVisitor(semantic_data).visit(left_table); - } - - /// replace tables in select with generated two-table join - RewriteVisitor::Data visitor_data{left_table, src_tables.back()}; - RewriteVisitor(visitor_data).visit(select.refTables()); - - data.done = true; -} - ASTPtr JoinToSubqueryTransformMatcher::replaceJoin(ASTPtr ast_left, ASTPtr ast_right, ASTPtr subquery_template) { const auto * left = ast_left->as(); diff --git a/src/Interpreters/JoinToSubqueryTransformVisitor.h b/src/Interpreters/JoinToSubqueryTransformVisitor.h index 643eb19b365..a024a168509 100644 --- a/src/Interpreters/JoinToSubqueryTransformVisitor.h +++ b/src/Interpreters/JoinToSubqueryTransformVisitor.h @@ -20,7 +20,6 @@ public: { const std::vector & tables; const Aliases & aliases; - size_t version = 1; bool done = false; }; @@ -43,10 +42,7 @@ private: /// TablesInSelectQueryElement [source1] /// TablesInSelectQueryElement [source2] /// - static void visitV1(ASTSelectQuery & select, ASTPtr & ast, Data & data); - - /// V2 uses information about tables' columns to rewrite queries. - static void visitV2(ASTSelectQuery & select, ASTPtr & ast, Data & data); + static void visit(ASTSelectQuery & select, ASTPtr & ast, Data & data); /// @return combined TablesInSelectQueryElement or nullptr if cannot rewrite static ASTPtr replaceJoin(ASTPtr left, ASTPtr right, ASTPtr subquery_template); diff --git a/src/Interpreters/TranslateQualifiedNamesVisitor.cpp b/src/Interpreters/TranslateQualifiedNamesVisitor.cpp index 74622c72865..9e0b6fdd196 100644 --- a/src/Interpreters/TranslateQualifiedNamesVisitor.cpp +++ b/src/Interpreters/TranslateQualifiedNamesVisitor.cpp @@ -2,7 +2,6 @@ #include #include -#include #include #include @@ -174,25 +173,11 @@ void TranslateQualifiedNamesMatcher::visit(ASTSelectQuery & select, const ASTPtr Visitor(data).visit(select.refHaving()); } -static void addIdentifier(ASTs & nodes, const DatabaseAndTableWithAlias & table, const String & column_name, - AsteriskSemantic::RevertedAliasesPtr aliases) +static void addIdentifier(ASTs & nodes, const DatabaseAndTableWithAlias & table, const String & column_name) { String table_name = table.getQualifiedNamePrefix(false); auto identifier = std::make_shared(std::vector{table_name, column_name}); - - bool added = false; - if (aliases && aliases->count(identifier->name)) - { - for (const String & alias : (*aliases)[identifier->name]) - { - nodes.push_back(identifier->clone()); - nodes.back()->setAlias(alias); - added = true; - } - } - - if (!added) - nodes.emplace_back(identifier); + nodes.emplace_back(identifier); } /// Replace *, alias.*, database.table.* with a list of columns. @@ -237,7 +222,7 @@ void TranslateQualifiedNamesMatcher::visit(ASTExpressionList & node, const ASTPt { if (first_table || !data.join_using_columns.count(column.name)) { - addIdentifier(node.children, table.table, column.name, AsteriskSemantic::getAliases(*asterisk)); + addIdentifier(node.children, table.table, column.name); } } @@ -264,7 +249,7 @@ void TranslateQualifiedNamesMatcher::visit(ASTExpressionList & node, const ASTPt { if (asterisk_pattern->isColumnMatching(column.name) && (first_table || !data.join_using_columns.count(column.name))) { - addIdentifier(node.children, table.table, column.name, AsteriskSemantic::getAliases(*asterisk_pattern)); + addIdentifier(node.children, table.table, column.name); } } @@ -287,7 +272,7 @@ void TranslateQualifiedNamesMatcher::visit(ASTExpressionList & node, const ASTPt { for (const auto & column : table.columns) { - addIdentifier(node.children, table.table, column.name, AsteriskSemantic::getAliases(*qualified_asterisk)); + addIdentifier(node.children, table.table, column.name); } break; } diff --git a/src/Parsers/ASTAsterisk.h b/src/Parsers/ASTAsterisk.h index 9c4c9a2df6d..027758ba48c 100644 --- a/src/Parsers/ASTAsterisk.h +++ b/src/Parsers/ASTAsterisk.h @@ -6,9 +6,6 @@ namespace DB { -struct AsteriskSemantic; -struct AsteriskSemanticImpl; - /** SELECT * is expanded to all visible columns of the source table. * Optional transformers can be attached to further manipulate these expanded columns. */ @@ -21,11 +18,6 @@ public: protected: void formatImpl(const FormatSettings & settings, FormatState &, FormatStateStacked) const override; - -private: - std::shared_ptr semantic; /// pimpl - - friend struct AsteriskSemantic; }; } diff --git a/src/Parsers/ASTQualifiedAsterisk.h b/src/Parsers/ASTQualifiedAsterisk.h index 2c3689d0ace..1b644532f53 100644 --- a/src/Parsers/ASTQualifiedAsterisk.h +++ b/src/Parsers/ASTQualifiedAsterisk.h @@ -6,9 +6,6 @@ namespace DB { -struct AsteriskSemantic; -struct AsteriskSemanticImpl; - /** Something like t.* * It will have qualifier as its child ASTIdentifier. * Optional transformers can be attached to further manipulate these expanded columns. @@ -27,11 +24,6 @@ public: protected: void formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override; - -private: - std::shared_ptr semantic; /// pimpl - - friend struct AsteriskSemantic; }; } diff --git a/tests/queries/0_stateless/00847_multiple_join_same_column.reference b/tests/queries/0_stateless/00847_multiple_join_same_column.reference index 4e813d5a677..91bd62ca5a3 100644 --- a/tests/queries/0_stateless/00847_multiple_join_same_column.reference +++ b/tests/queries/0_stateless/00847_multiple_join_same_column.reference @@ -31,18 +31,6 @@ y.b: 0 │ 1 │ 1 │ 1 │ │ 2 │ 2 │ 2 │ └─────┴─────┴─────┘ -┌─s.a─┬─s.a─┬─s_b─┬─s_b─┐ -│ 1 │ 1 │ 1 │ 1 │ -│ 0 │ 0 │ 0 │ 0 │ -└─────┴─────┴─────┴─────┘ -┌─y.a─┬─y.a─┬─y_b─┬─y_b─┐ -│ 1 │ 1 │ 1 │ 1 │ -│ 0 │ 0 │ 0 │ 0 │ -└─────┴─────┴─────┴─────┘ -┌─t_a─┬─t_a─┬─s_a─┬─s_a─┬─y_a─┬─y_a─┐ -│ 1 │ 1 │ 1 │ 1 │ 1 │ 1 │ -│ 2 │ 2 │ 0 │ 0 │ 0 │ 0 │ -└─────┴─────┴─────┴─────┴─────┴─────┘ ┌─s.a─┬─s.a─┬─s_b─┬─s.b─┐ │ 1 │ 1 │ 1 │ 1 │ │ 0 │ 0 │ 0 │ 0 │ diff --git a/tests/queries/0_stateless/00847_multiple_join_same_column.sql b/tests/queries/0_stateless/00847_multiple_join_same_column.sql index eae18dba7f3..c7f0c6383c2 100644 --- a/tests/queries/0_stateless/00847_multiple_join_same_column.sql +++ b/tests/queries/0_stateless/00847_multiple_join_same_column.sql @@ -39,28 +39,6 @@ left join y on y.b = s.b order by t.a format PrettyCompactNoEscapes; -set multiple_joins_rewriter_version = 1; - -select s.a, s.a, s.b as s_b, s.b from t -left join s on s.a = t.a -left join y on s.b = y.b -order by t.a -format PrettyCompactNoEscapes; - -select y.a, y.a, y.b as y_b, y.b from t -left join s on s.a = t.a -left join y on y.b = s.b -order by t.a -format PrettyCompactNoEscapes; - -select t.a, t.a as t_a, s.a, s.a as s_a, y.a, y.a as y_a from t -left join s on t.a = s.a -left join y on y.b = s.b -order by t.a -format PrettyCompactNoEscapes; - -set multiple_joins_rewriter_version = 2; - select s.a, s.a, s.b as s_b, s.b from t left join s on s.a = t.a left join y on s.b = y.b diff --git a/tests/queries/0_stateless/00849_multiple_comma_join.reference b/tests/queries/0_stateless/00849_multiple_comma_join.reference deleted file mode 100644 index f4db2238dd1..00000000000 --- a/tests/queries/0_stateless/00849_multiple_comma_join.reference +++ /dev/null @@ -1,292 +0,0 @@ -SELECT a -FROM t1_00849 -CROSS JOIN t2_00849 -SELECT a -FROM t1_00849 -ALL INNER JOIN t2_00849 ON a = t2_00849.a -WHERE a = t2_00849.a -SELECT a -FROM t1_00849 -ALL INNER JOIN t2_00849 ON b = t2_00849.b -WHERE b = t2_00849.b -SELECT `--t1_00849.a` AS `t1_00849.a` -FROM -( - SELECT - a AS `--t1_00849.a`, - b, - t2_00849.a AS `--t2_00849.a`, - t2_00849.b - FROM t1_00849 - ALL INNER JOIN t2_00849 ON `--t1_00849.a` = `--t2_00849.a` -) AS `--.s` -ALL INNER JOIN t3_00849 ON `--t1_00849.a` = a -WHERE (`--t1_00849.a` = `--t2_00849.a`) AND (`--t1_00849.a` = a) -SELECT `--t1_00849.a` AS `t1_00849.a` -FROM -( - SELECT - a AS `--t1_00849.a`, - b AS `--t1_00849.b`, - t2_00849.a, - t2_00849.b AS `--t2_00849.b` - FROM t1_00849 - ALL INNER JOIN t2_00849 ON `--t1_00849.b` = `--t2_00849.b` -) AS `--.s` -ALL INNER JOIN t3_00849 ON `--t1_00849.b` = b -WHERE (`--t1_00849.b` = `--t2_00849.b`) AND (`--t1_00849.b` = b) -SELECT `--t1_00849.a` AS `t1_00849.a` -FROM -( - SELECT - `--t1_00849.a`, - b, - `--t2_00849.a`, - `t2_00849.b`, - a AS `--t3_00849.a`, - t3_00849.b - FROM - ( - SELECT - a AS `--t1_00849.a`, - b, - t2_00849.a AS `--t2_00849.a`, - t2_00849.b - FROM t1_00849 - ALL INNER JOIN t2_00849 ON `--t1_00849.a` = `--t2_00849.a` - ) AS `--.s` - ALL INNER JOIN t3_00849 ON `--t1_00849.a` = `--t3_00849.a` -) AS `--.s` -ALL INNER JOIN t4_00849 ON `--t1_00849.a` = a -WHERE (`--t1_00849.a` = `--t2_00849.a`) AND (`--t1_00849.a` = `--t3_00849.a`) AND (`--t1_00849.a` = a) -SELECT `--t1_00849.a` AS `t1_00849.a` -FROM -( - SELECT - `--t1_00849.a`, - `--t1_00849.b`, - `t2_00849.a`, - `--t2_00849.b`, - a, - b AS `--t3_00849.b` - FROM - ( - SELECT - a AS `--t1_00849.a`, - b AS `--t1_00849.b`, - t2_00849.a, - t2_00849.b AS `--t2_00849.b` - FROM t1_00849 - ALL INNER JOIN t2_00849 ON `--t1_00849.b` = `--t2_00849.b` - ) AS `--.s` - ALL INNER JOIN t3_00849 ON `--t1_00849.b` = `--t3_00849.b` -) AS `--.s` -ALL INNER JOIN t4_00849 ON `--t1_00849.b` = b -WHERE (`--t1_00849.b` = `--t2_00849.b`) AND (`--t1_00849.b` = `--t3_00849.b`) AND (`--t1_00849.b` = b) -SELECT `--t1_00849.a` AS `t1_00849.a` -FROM -( - SELECT - `--t1_00849.a`, - b, - `--t2_00849.a`, - `t2_00849.b`, - a AS `--t3_00849.a`, - t3_00849.b - FROM - ( - SELECT - a AS `--t1_00849.a`, - b, - t2_00849.a AS `--t2_00849.a`, - t2_00849.b - FROM t1_00849 - ALL INNER JOIN t2_00849 ON `--t2_00849.a` = `--t1_00849.a` - ) AS `--.s` - ALL INNER JOIN t3_00849 ON `--t2_00849.a` = `--t3_00849.a` -) AS `--.s` -ALL INNER JOIN t4_00849 ON `--t2_00849.a` = a -WHERE (`--t2_00849.a` = `--t1_00849.a`) AND (`--t2_00849.a` = `--t3_00849.a`) AND (`--t2_00849.a` = a) -SELECT `--t1_00849.a` AS `t1_00849.a` -FROM -( - SELECT - `--t1_00849.a`, - b, - `--t2_00849.a`, - `t2_00849.b`, - a AS `--t3_00849.a`, - t3_00849.b - FROM - ( - SELECT - a AS `--t1_00849.a`, - b, - t2_00849.a AS `--t2_00849.a`, - t2_00849.b - FROM t1_00849 - CROSS JOIN t2_00849 - ) AS `--.s` - ALL INNER JOIN t3_00849 ON (`--t3_00849.a` = `--t1_00849.a`) AND (`--t3_00849.a` = `--t2_00849.a`) -) AS `--.s` -ALL INNER JOIN t4_00849 ON `--t3_00849.a` = a -WHERE (`--t3_00849.a` = `--t1_00849.a`) AND (`--t3_00849.a` = `--t2_00849.a`) AND (`--t3_00849.a` = a) -SELECT `--t1_00849.a` AS `t1_00849.a` -FROM -( - SELECT - `--t1_00849.a`, - b, - `--t2_00849.a`, - `t2_00849.b`, - a AS `--t3_00849.a`, - t3_00849.b - FROM - ( - SELECT - a AS `--t1_00849.a`, - b, - t2_00849.a AS `--t2_00849.a`, - t2_00849.b - FROM t1_00849 - CROSS JOIN t2_00849 - ) AS `--.s` - CROSS JOIN t3_00849 -) AS `--.s` -ALL INNER JOIN t4_00849 ON (a = `--t1_00849.a`) AND (a = `--t2_00849.a`) AND (a = `--t3_00849.a`) -WHERE (a = `--t1_00849.a`) AND (a = `--t2_00849.a`) AND (a = `--t3_00849.a`) -SELECT `--t1_00849.a` AS `t1_00849.a` -FROM -( - SELECT - `--t1_00849.a`, - b, - `--t2_00849.a`, - `t2_00849.b`, - a AS `--t3_00849.a`, - t3_00849.b - FROM - ( - SELECT - a AS `--t1_00849.a`, - b, - t2_00849.a AS `--t2_00849.a`, - t2_00849.b - FROM t1_00849 - ALL INNER JOIN t2_00849 ON `--t1_00849.a` = `--t2_00849.a` - ) AS `--.s` - ALL INNER JOIN t3_00849 ON `--t2_00849.a` = `--t3_00849.a` -) AS `--.s` -ALL INNER JOIN t4_00849 ON `--t3_00849.a` = a -WHERE (`--t1_00849.a` = `--t2_00849.a`) AND (`--t2_00849.a` = `--t3_00849.a`) AND (`--t3_00849.a` = a) -SELECT `--t1_00849.a` AS `t1_00849.a` -FROM -( - SELECT - `--t1_00849.a`, - b, - `t2_00849.a`, - `t2_00849.b`, - a, - t3_00849.b - FROM - ( - SELECT - a AS `--t1_00849.a`, - b, - t2_00849.a, - t2_00849.b - FROM t1_00849 - CROSS JOIN t2_00849 - ) AS `--.s` - CROSS JOIN t3_00849 -) AS `--.s` -CROSS JOIN t4_00849 -SELECT `--t1_00849.a` AS `t1_00849.a` -FROM -( - SELECT - `--t1_00849.a`, - b, - `t2_00849.a`, - `t2_00849.b`, - a, - t3_00849.b - FROM - ( - SELECT - a AS `--t1_00849.a`, - b, - t2_00849.a, - t2_00849.b - FROM t1_00849 - CROSS JOIN t2_00849 - ) AS `--.s` - CROSS JOIN t3_00849 -) AS `--.s` -CROSS JOIN t4_00849 -SELECT `--t1_00849.a` AS `t1_00849.a` -FROM -( - SELECT - a AS `--t1_00849.a`, - b, - t2_00849.a, - t2_00849.b - FROM t1_00849 - CROSS JOIN t2_00849 -) AS `--.s` -CROSS JOIN t3_00849 -SELECT `--t1_00849.a` AS `t1_00849.a` -FROM -( - SELECT - a AS `--t1_00849.a`, - b, - t2_00849.a AS `--t2_00849.a`, - t2_00849.b - FROM t1_00849 - ALL INNER JOIN t2_00849 ON `--t1_00849.a` = `--t2_00849.a` -) AS `--.s` -CROSS JOIN t3_00849 -SELECT * FROM t1, t2 -1 1 1 1 -1 1 1 \N -2 2 1 1 -2 2 1 \N -3 3 1 1 -3 3 1 \N -4 4 1 1 -4 4 1 \N -SELECT * FROM t1, t2 WHERE t1.a = t2.a -1 1 1 1 -1 1 1 \N -SELECT t1.a, t2.a FROM t1, t2 WHERE t1.b = t2.b -1 1 -SELECT t1.a, t2.b, t3.b FROM t1, t2, t3 WHERE t1.a = t2.a AND t1.a = t3.a -1 1 1 -1 1 \N -1 \N 1 -1 \N \N -SELECT t1.a, t2.b, t3.b FROM t1, t2, t3 WHERE t1.b = t2.b AND t1.b = t3.b -1 1 1 -SELECT t1.a, t2.b, t3.b, t4.b FROM t1, t2, t3, t4 WHERE t1.a = t2.a AND t1.a = t3.a AND t1.a = t4.a -1 1 1 1 -1 1 1 \N -1 1 \N 1 -1 1 \N \N -1 \N 1 1 -1 \N 1 \N -1 \N \N 1 -1 \N \N \N -SELECT t1.a, t2.b, t3.b, t4.b FROM t1, t2, t3, t4 WHERE t1.b = t2.b AND t1.b = t3.b AND t1.b = t4.b -1 1 1 1 -SELECT t1.a, t2.b, t3.b, t4.b FROM t1, t2, t3, t4 WHERE t1.a = t2.a AND t2.a = t3.a AND t3.a = t4.a -1 1 1 1 -1 1 1 \N -1 1 \N 1 -1 1 \N \N -1 \N 1 1 -1 \N 1 \N -1 \N \N 1 -1 \N \N \N diff --git a/tests/queries/0_stateless/00849_multiple_comma_join.sql b/tests/queries/0_stateless/00849_multiple_comma_join.sql deleted file mode 100644 index 5a8962c2277..00000000000 --- a/tests/queries/0_stateless/00849_multiple_comma_join.sql +++ /dev/null @@ -1,69 +0,0 @@ -SET enable_debug_queries = 1; -SET enable_optimize_predicate_expression = 0; -SET multiple_joins_rewriter_version = 1; - -DROP TABLE IF EXISTS t1_00849; -DROP TABLE IF EXISTS t2_00849; -DROP TABLE IF EXISTS t3_00849; -DROP TABLE IF EXISTS t4_00849; - -CREATE TABLE t1_00849 (a UInt32, b Nullable(Int32)) ENGINE = Memory; -CREATE TABLE t2_00849 (a UInt32, b Nullable(Int32)) ENGINE = Memory; -CREATE TABLE t3_00849 (a UInt32, b Nullable(Int32)) ENGINE = Memory; -CREATE TABLE t4_00849 (a UInt32, b Nullable(Int32)) ENGINE = Memory; - -ANALYZE SELECT t1_00849.a FROM t1_00849, t2_00849; -ANALYZE SELECT t1_00849.a FROM t1_00849, t2_00849 WHERE t1_00849.a = t2_00849.a; -ANALYZE SELECT t1_00849.a FROM t1_00849, t2_00849 WHERE t1_00849.b = t2_00849.b; -ANALYZE SELECT t1_00849.a FROM t1_00849, t2_00849, t3_00849 WHERE t1_00849.a = t2_00849.a AND t1_00849.a = t3_00849.a; -ANALYZE SELECT t1_00849.a FROM t1_00849, t2_00849, t3_00849 WHERE t1_00849.b = t2_00849.b AND t1_00849.b = t3_00849.b; -ANALYZE SELECT t1_00849.a FROM t1_00849, t2_00849, t3_00849, t4_00849 WHERE t1_00849.a = t2_00849.a AND t1_00849.a = t3_00849.a AND t1_00849.a = t4_00849.a; -ANALYZE SELECT t1_00849.a FROM t1_00849, t2_00849, t3_00849, t4_00849 WHERE t1_00849.b = t2_00849.b AND t1_00849.b = t3_00849.b AND t1_00849.b = t4_00849.b; - -ANALYZE SELECT t1_00849.a FROM t1_00849, t2_00849, t3_00849, t4_00849 WHERE t2_00849.a = t1_00849.a AND t2_00849.a = t3_00849.a AND t2_00849.a = t4_00849.a; -ANALYZE SELECT t1_00849.a FROM t1_00849, t2_00849, t3_00849, t4_00849 WHERE t3_00849.a = t1_00849.a AND t3_00849.a = t2_00849.a AND t3_00849.a = t4_00849.a; -ANALYZE SELECT t1_00849.a FROM t1_00849, t2_00849, t3_00849, t4_00849 WHERE t4_00849.a = t1_00849.a AND t4_00849.a = t2_00849.a AND t4_00849.a = t3_00849.a; -ANALYZE SELECT t1_00849.a FROM t1_00849, t2_00849, t3_00849, t4_00849 WHERE t1_00849.a = t2_00849.a AND t2_00849.a = t3_00849.a AND t3_00849.a = t4_00849.a; - -ANALYZE SELECT t1_00849.a FROM t1_00849, t2_00849, t3_00849, t4_00849; -ANALYZE SELECT t1_00849.a FROM t1_00849 CROSS JOIN t2_00849 CROSS JOIN t3_00849 CROSS JOIN t4_00849; - -ANALYZE SELECT t1_00849.a FROM t1_00849, t2_00849 CROSS JOIN t3_00849; -ANALYZE SELECT t1_00849.a FROM t1_00849 JOIN t2_00849 USING a CROSS JOIN t3_00849; -- { serverError 48 } -ANALYZE SELECT t1_00849.a FROM t1_00849 JOIN t2_00849 ON t1_00849.a = t2_00849.a CROSS JOIN t3_00849; - -INSERT INTO t1_00849 values (1,1), (2,2), (3,3), (4,4); -INSERT INTO t2_00849 values (1,1), (1, Null); -INSERT INTO t3_00849 values (1,1), (1, Null); -INSERT INTO t4_00849 values (1,1), (1, Null); - -SELECT 'SELECT * FROM t1, t2'; -SELECT * FROM t1_00849, t2_00849 -ORDER BY t1_00849.a, t2_00849.b; -SELECT 'SELECT * FROM t1, t2 WHERE t1.a = t2.a'; -SELECT * FROM t1_00849, t2_00849 WHERE t1_00849.a = t2_00849.a -ORDER BY t1_00849.a, t2_00849.b; -SELECT 'SELECT t1.a, t2.a FROM t1, t2 WHERE t1.b = t2.b'; -SELECT t1_00849.a, t2_00849.b FROM t1_00849, t2_00849 WHERE t1_00849.b = t2_00849.b; -SELECT 'SELECT t1.a, t2.b, t3.b FROM t1, t2, t3 WHERE t1.a = t2.a AND t1.a = t3.a'; -SELECT t1_00849.a, t2_00849.b, t3_00849.b FROM t1_00849, t2_00849, t3_00849 -WHERE t1_00849.a = t2_00849.a AND t1_00849.a = t3_00849.a -ORDER BY t2_00849.b, t3_00849.b; -SELECT 'SELECT t1.a, t2.b, t3.b FROM t1, t2, t3 WHERE t1.b = t2.b AND t1.b = t3.b'; -SELECT t1_00849.a, t2_00849.b, t3_00849.b FROM t1_00849, t2_00849, t3_00849 WHERE t1_00849.b = t2_00849.b AND t1_00849.b = t3_00849.b; -SELECT 'SELECT t1.a, t2.b, t3.b, t4.b FROM t1, t2, t3, t4 WHERE t1.a = t2.a AND t1.a = t3.a AND t1.a = t4.a'; -SELECT t1_00849.a, t2_00849.b, t3_00849.b, t4_00849.b FROM t1_00849, t2_00849, t3_00849, t4_00849 -WHERE t1_00849.a = t2_00849.a AND t1_00849.a = t3_00849.a AND t1_00849.a = t4_00849.a -ORDER BY t2_00849.b, t3_00849.b, t4_00849.b; -SELECT 'SELECT t1.a, t2.b, t3.b, t4.b FROM t1, t2, t3, t4 WHERE t1.b = t2.b AND t1.b = t3.b AND t1.b = t4.b'; -SELECT t1_00849.a, t2_00849.b, t3_00849.b, t4_00849.b FROM t1_00849, t2_00849, t3_00849, t4_00849 -WHERE t1_00849.b = t2_00849.b AND t1_00849.b = t3_00849.b AND t1_00849.b = t4_00849.b; -SELECT 'SELECT t1.a, t2.b, t3.b, t4.b FROM t1, t2, t3, t4 WHERE t1.a = t2.a AND t2.a = t3.a AND t3.a = t4.a'; -SELECT t1_00849.a, t2_00849.b, t3_00849.b, t4_00849.b FROM t1_00849, t2_00849, t3_00849, t4_00849 -WHERE t1_00849.a = t2_00849.a AND t2_00849.a = t3_00849.a AND t3_00849.a = t4_00849.a -ORDER BY t2_00849.b, t3_00849.b, t4_00849.b; - -DROP TABLE t1_00849; -DROP TABLE t2_00849; -DROP TABLE t3_00849; -DROP TABLE t4_00849; diff --git a/tests/queries/0_stateless/00849_multiple_comma_join_2.sql b/tests/queries/0_stateless/00849_multiple_comma_join_2.sql index de09bcc0b4d..710327ec0e4 100644 --- a/tests/queries/0_stateless/00849_multiple_comma_join_2.sql +++ b/tests/queries/0_stateless/00849_multiple_comma_join_2.sql @@ -1,6 +1,5 @@ SET enable_debug_queries = 1; SET enable_optimize_predicate_expression = 0; -SET multiple_joins_rewriter_version = 2; DROP TABLE IF EXISTS t1; DROP TABLE IF EXISTS t2; diff --git a/tests/queries/0_stateless/00854_multiple_join_asterisks.sql b/tests/queries/0_stateless/00854_multiple_join_asterisks.sql index 43aef45440c..c01ea721000 100644 --- a/tests/queries/0_stateless/00854_multiple_join_asterisks.sql +++ b/tests/queries/0_stateless/00854_multiple_join_asterisks.sql @@ -4,14 +4,6 @@ select t1.* from system.one t1 join system.one t2 on t1.dummy = t2.dummy join sy select t2.*, t3.* from system.one t1 join system.one t2 on t1.dummy = t2.dummy join system.one t3 ON t1.dummy = t3.dummy; select t1.dummy, t2.*, t3.dummy from system.one t1 join system.one t2 on t1.dummy = t2.dummy join system.one t3 ON t1.dummy = t3.dummy; -set multiple_joins_rewriter_version = 1; - -select t1.dummy, t2.*, t3.dummy from (select * from system.one) t1 -join system.one t2 on t1.dummy = t2.dummy -join system.one t3 ON t1.dummy = t3.dummy; -- { serverError 48 } - -set multiple_joins_rewriter_version = 2; - select t1.dummy, t2.*, t3.dummy from (select * from system.one) t1 join system.one t2 on t1.dummy = t2.dummy join system.one t3 ON t1.dummy = t3.dummy; diff --git a/tests/queries/0_stateless/01116_cross_count_asterisks.reference b/tests/queries/0_stateless/01116_cross_count_asterisks.reference index 8347b144a35..5f1d0ecea5d 100644 --- a/tests/queries/0_stateless/01116_cross_count_asterisks.reference +++ b/tests/queries/0_stateless/01116_cross_count_asterisks.reference @@ -1,4 +1,2 @@ 2 1 -2 -1 diff --git a/tests/queries/0_stateless/01116_cross_count_asterisks.sql b/tests/queries/0_stateless/01116_cross_count_asterisks.sql index 1fb8b0b0e66..aa5adaddae5 100644 --- a/tests/queries/0_stateless/01116_cross_count_asterisks.sql +++ b/tests/queries/0_stateless/01116_cross_count_asterisks.sql @@ -1,20 +1,3 @@ -SET multiple_joins_rewriter_version = 2; - -SELECT count(*) -FROM numbers(2) AS n1, numbers(3) AS n2, numbers(4) AS n3 -WHERE (n1.number = n2.number) AND (n2.number = n3.number); - -SELECT count(*) c FROM ( - SELECT count(*), count(*) as c - FROM numbers(2) AS n1, numbers(3) AS n2, numbers(4) AS n3 - WHERE (n1.number = n2.number) AND (n2.number = n3.number) - AND (SELECT count(*) FROM numbers(1)) = 1 -) -WHERE (SELECT count(*) FROM numbers(2)) = 2 -HAVING c IN(SELECT count(*) c FROM numbers(1)); - -SET multiple_joins_rewriter_version = 1; - SELECT count(*) FROM numbers(2) AS n1, numbers(3) AS n2, numbers(4) AS n3 WHERE (n1.number = n2.number) AND (n2.number = n3.number); diff --git a/tests/queries/0_stateless/01144_join_rewrite_with_ambiguous_column_and_view.sql b/tests/queries/0_stateless/01144_join_rewrite_with_ambiguous_column_and_view.sql index c90d01ff76d..ae844888407 100644 --- a/tests/queries/0_stateless/01144_join_rewrite_with_ambiguous_column_and_view.sql +++ b/tests/queries/0_stateless/01144_join_rewrite_with_ambiguous_column_and_view.sql @@ -11,7 +11,6 @@ INSERT INTO t1 (id, value1) VALUES (1, 'val11'); INSERT INTO t2 (id, value2) VALUES (1, 'val21'); INSERT INTO t3 (id, value3) VALUES (1, 'val31'); -SET multiple_joins_rewriter_version = 2; SET enable_optimize_predicate_expression = 1; SELECT t1.id, t2.id as id, t3.id as value diff --git a/tests/queries/0_stateless/01144_multiple_joins_rewriter_v2_and_lambdas.sql b/tests/queries/0_stateless/01144_multiple_joins_rewriter_v2_and_lambdas.sql index daf93649f89..6a5ba042c63 100644 --- a/tests/queries/0_stateless/01144_multiple_joins_rewriter_v2_and_lambdas.sql +++ b/tests/queries/0_stateless/01144_multiple_joins_rewriter_v2_and_lambdas.sql @@ -1,5 +1,3 @@ -set multiple_joins_rewriter_version = 2; - select arrayMap(x, y -> floor((y - x) / x, 3), l, r) diff_percent, test, query From c7e8021a70027f25cd40851790c82ddc39488c91 Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Thu, 1 Oct 2020 01:28:47 +0300 Subject: [PATCH 252/321] Update gRPC to the latest revision. --- .gitmodules | 1 - contrib/grpc | 2 +- 2 files changed, 1 insertion(+), 2 deletions(-) diff --git a/.gitmodules b/.gitmodules index 865a876b276..13ed2614f4d 100644 --- a/.gitmodules +++ b/.gitmodules @@ -107,7 +107,6 @@ [submodule "contrib/grpc"] path = contrib/grpc url = https://github.com/ClickHouse-Extras/grpc.git - branch = v1.25.0 [submodule "contrib/aws"] path = contrib/aws url = https://github.com/ClickHouse-Extras/aws-sdk-cpp.git diff --git a/contrib/grpc b/contrib/grpc index 8aea4e168e7..a6570b863cf 160000 --- a/contrib/grpc +++ b/contrib/grpc @@ -1 +1 @@ -Subproject commit 8aea4e168e78f3eb9828080740fc8cb73d53bf79 +Subproject commit a6570b863cf76c9699580ba51c7827d5bffaac43 From 8a3b024a6db52d40fdfbe4ea111533d742442329 Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Thu, 1 Oct 2020 01:26:02 +0300 Subject: [PATCH 253/321] Fix using external protobuf library for the build. --- cmake/find/protobuf.cmake | 81 +++---- cmake/protobuf_generate_cpp.cmake | 172 --------------- .../protobuf-cmake/protobuf_generate.cmake | 198 ++++++++++++++++++ 3 files changed, 241 insertions(+), 210 deletions(-) delete mode 100644 cmake/protobuf_generate_cpp.cmake create mode 100644 contrib/protobuf-cmake/protobuf_generate.cmake diff --git a/cmake/find/protobuf.cmake b/cmake/find/protobuf.cmake index a7769206e9f..eb9fbe3edef 100644 --- a/cmake/find/protobuf.cmake +++ b/cmake/find/protobuf.cmake @@ -1,57 +1,62 @@ option(ENABLE_PROTOBUF "Enable protobuf" ${ENABLE_LIBRARIES}) if(NOT ENABLE_PROTOBUF) - if(USE_INTERNAL_PROTOBUF_LIBRARY) - message (${RECONFIGURE_MESSAGE_LEVEL} "Can't use internal protobuf with ENABLE_PROTOBUF=OFF") - endif() - return() + if(USE_INTERNAL_PROTOBUF_LIBRARY) + message(${RECONFIGURE_MESSAGE_LEVEL} "Can't use internal protobuf with ENABLE_PROTOBUF=OFF") + endif() + return() endif() -option(USE_INTERNAL_PROTOBUF_LIBRARY "Set to FALSE to use system protobuf instead of bundled" ${NOT_UNBUNDLED}) +# Normally we use the internal protobuf library. +# You can set USE_INTERNAL_PROTOBUF_LIBRARY to OFF to force using the external protobuf library, which should be installed in the system in this case. +# The external protobuf library can be installed in the system by running +# sudo apt-get install libprotobuf-dev protobuf-compiler libprotoc-dev +option(USE_INTERNAL_PROTOBUF_LIBRARY "Set to FALSE to use system protobuf instead of bundled. (Experimental. Set to OFF on your own risk)" ${NOT_UNBUNDLED}) if(NOT EXISTS "${ClickHouse_SOURCE_DIR}/contrib/protobuf/cmake/CMakeLists.txt") - if(USE_INTERNAL_PROTOBUF_LIBRARY) - message(WARNING "submodule contrib/protobuf is missing. to fix try run: \n git submodule update --init --recursive") - message (${RECONFIGURE_MESSAGE_LEVEL} "Can't use internal protobuf") - set(USE_INTERNAL_PROTOBUF_LIBRARY 0) - endif() - set(MISSING_INTERNAL_PROTOBUF_LIBRARY 1) + if(USE_INTERNAL_PROTOBUF_LIBRARY) + message(WARNING "submodule contrib/protobuf is missing. to fix try run: \n git submodule update --init --recursive") + message(${RECONFIGURE_MESSAGE_LEVEL} "Can't use internal protobuf") + set(USE_INTERNAL_PROTOBUF_LIBRARY 0) + endif() + set(MISSING_INTERNAL_PROTOBUF_LIBRARY 1) endif() if(NOT USE_INTERNAL_PROTOBUF_LIBRARY) - find_package(Protobuf) - if (Protobuf_LIBRARY AND Protobuf_INCLUDE_DIR AND Protobuf_PROTOC_EXECUTABLE) - set(EXTERNAL_PROTOBUF_LIBRARY_FOUND 1) - set(USE_PROTOBUF 1) - else() - message (${RECONFIGURE_MESSAGE_LEVEL} "Can't find system protobuf") - set(EXTERNAL_PROTOBUF_LIBRARY_FOUND 0) - endif() + find_package(Protobuf) + if(NOT Protobuf_INCLUDE_DIR OR NOT Protobuf_LIBRARY) + message(${RECONFIGURE_MESSAGE_LEVEL} "Can't find system protobuf library") + set(EXTERNAL_PROTOBUF_LIBRARY_FOUND 0) + elseif(NOT Protobuf_PROTOC_EXECUTABLE) + message(${RECONFIGURE_MESSAGE_LEVEL} "Can't find system protobuf compiler") + set(EXTERNAL_PROTOBUF_LIBRARY_FOUND 0) + else() + set(EXTERNAL_PROTOBUF_LIBRARY_FOUND 1) + set(USE_PROTOBUF 1) + endif() endif() -if (NOT EXTERNAL_PROTOBUF_LIBRARY_FOUND AND NOT MISSING_INTERNAL_PROTOBUF_LIBRARY) - set(Protobuf_INCLUDE_DIR "${ClickHouse_SOURCE_DIR}/contrib/protobuf/src") +if(NOT EXTERNAL_PROTOBUF_LIBRARY_FOUND AND NOT MISSING_INTERNAL_PROTOBUF_LIBRARY) + set(Protobuf_INCLUDE_DIR "${ClickHouse_SOURCE_DIR}/contrib/protobuf/src") + set(Protobuf_LIBRARY libprotobuf) + set(Protobuf_PROTOC_EXECUTABLE "$") + set(Protobuf_PROTOC_LIBRARY libprotoc) - set(USE_PROTOBUF 1) - set(USE_INTERNAL_PROTOBUF_LIBRARY 1) - set(Protobuf_LIBRARY libprotobuf) - set(Protobuf_PROTOC_LIBRARY libprotoc) - set(Protobuf_LITE_LIBRARY libprotobuf-lite) + include("${ClickHouse_SOURCE_DIR}/contrib/protobuf-cmake/protobuf_generate.cmake") - set(Protobuf_PROTOC_EXECUTABLE "$") + set(USE_INTERNAL_PROTOBUF_LIBRARY 1) + set(USE_PROTOBUF 1) endif() if(OS_FREEBSD AND SANITIZE STREQUAL "address") - # ../contrib/protobuf/src/google/protobuf/arena_impl.h:45:10: fatal error: 'sanitizer/asan_interface.h' file not found - # #include - if(LLVM_INCLUDE_DIRS) - set(Protobuf_INCLUDE_DIR "${Protobuf_INCLUDE_DIR}" ${LLVM_INCLUDE_DIRS}) - else() - message (${RECONFIGURE_MESSAGE_LEVEL} "Can't use protobuf on FreeBSD with address sanitizer without LLVM") - set(USE_PROTOBUF 0) - endif() + # ../contrib/protobuf/src/google/protobuf/arena_impl.h:45:10: fatal error: 'sanitizer/asan_interface.h' file not found + # #include + if(LLVM_INCLUDE_DIRS) + set(Protobuf_INCLUDE_DIR "${Protobuf_INCLUDE_DIR}" ${LLVM_INCLUDE_DIRS}) + else() + message(${RECONFIGURE_MESSAGE_LEVEL} "Can't use protobuf on FreeBSD with address sanitizer without LLVM") + set(USE_PROTOBUF 0) + endif() endif() -include ("${ClickHouse_SOURCE_DIR}/cmake/protobuf_generate_cpp.cmake") - -message(STATUS "Using protobuf=${USE_PROTOBUF}: ${Protobuf_INCLUDE_DIR} : ${Protobuf_LIBRARY} : ${Protobuf_PROTOC_EXECUTABLE}") +message(STATUS "Using protobuf=${USE_PROTOBUF}: ${Protobuf_INCLUDE_DIR} : ${Protobuf_LIBRARY} : ${Protobuf_PROTOC_EXECUTABLE} : ${Protobuf_PROTOC_LIBRARY}") diff --git a/cmake/protobuf_generate_cpp.cmake b/cmake/protobuf_generate_cpp.cmake deleted file mode 100644 index cc2502e5eeb..00000000000 --- a/cmake/protobuf_generate_cpp.cmake +++ /dev/null @@ -1,172 +0,0 @@ -# This file declares functions adding custom commands for generating C++ files from *.proto files: -# function (protobuf_generate_cpp SRCS HDRS) -# function (protobuf_generate_grpc_cpp SRCS HDRS) - -if (NOT USE_PROTOBUF) - message (WARNING "Could not use protobuf_generate_cpp() without the protobuf library") - return() -endif() - -if (NOT DEFINED PROTOBUF_PROTOC_EXECUTABLE) - set (PROTOBUF_PROTOC_EXECUTABLE "$") -endif() - -if (NOT DEFINED GRPC_CPP_PLUGIN_EXECUTABLE) - set (GRPC_CPP_PLUGIN_EXECUTABLE $) -endif() - -if (NOT DEFINED PROTOBUF_GENERATE_CPP_APPEND_PATH) - set (PROTOBUF_GENERATE_CPP_APPEND_PATH TRUE) -endif() - - -function(protobuf_generate_cpp_impl SRCS HDRS MODES OUTPUT_FILE_EXTS PLUGIN) - if(NOT ARGN) - message(SEND_ERROR "Error: protobuf_generate_cpp() called without any proto files") - return() - endif() - - if(PROTOBUF_GENERATE_CPP_APPEND_PATH) - # Create an include path for each file specified - foreach(FIL ${ARGN}) - get_filename_component(ABS_FIL ${FIL} ABSOLUTE) - get_filename_component(ABS_PATH ${ABS_FIL} PATH) - list(FIND protobuf_include_path ${ABS_PATH} _contains_already) - if(${_contains_already} EQUAL -1) - list(APPEND protobuf_include_path -I ${ABS_PATH}) - endif() - endforeach() - else() - set(protobuf_include_path -I ${CMAKE_CURRENT_SOURCE_DIR}) - endif() - - if(DEFINED PROTOBUF_IMPORT_DIRS AND NOT DEFINED Protobuf_IMPORT_DIRS) - set(Protobuf_IMPORT_DIRS "${PROTOBUF_IMPORT_DIRS}") - endif() - - if(DEFINED Protobuf_IMPORT_DIRS) - foreach(DIR ${Protobuf_IMPORT_DIRS}) - get_filename_component(ABS_PATH ${DIR} ABSOLUTE) - list(FIND protobuf_include_path ${ABS_PATH} _contains_already) - if(${_contains_already} EQUAL -1) - list(APPEND protobuf_include_path -I ${ABS_PATH}) - endif() - endforeach() - endif() - - set (intermediate_dir ${CMAKE_CURRENT_BINARY_DIR}/intermediate) - file (MAKE_DIRECTORY ${intermediate_dir}) - - set (protoc_args) - foreach (mode ${MODES}) - list (APPEND protoc_args "--${mode}_out" ${intermediate_dir}) - endforeach() - if (PLUGIN) - list (APPEND protoc_args "--plugin=${PLUGIN}") - endif() - - set(srcs) - set(hdrs) - set(all_intermediate_outputs) - - foreach(input_name ${ARGN}) - get_filename_component(abs_name ${input_name} ABSOLUTE) - get_filename_component(name ${input_name} NAME_WE) - - set (intermediate_outputs) - foreach (ext ${OUTPUT_FILE_EXTS}) - set (filename "${name}${ext}") - set (output "${CMAKE_CURRENT_BINARY_DIR}/${filename}") - set (intermediate_output "${intermediate_dir}/${filename}") - list (APPEND intermediate_outputs "${intermediate_output}") - list (APPEND all_intermediate_outputs "${intermediate_output}") - - if (${ext} MATCHES ".*\\.h") - list(APPEND hdrs "${output}") - else() - list(APPEND srcs "${output}") - endif() - - add_custom_command( - OUTPUT ${output} - COMMAND ${CMAKE_COMMAND} -DPROTOBUF_GENERATE_CPP_SCRIPT_MODE=1 -DUSE_PROTOBUF=1 -DDIR=${CMAKE_CURRENT_BINARY_DIR} -DFILENAME=${filename} -DCOMPILER_ID=${CMAKE_CXX_COMPILER_ID} -P ${ClickHouse_SOURCE_DIR}/cmake/protobuf_generate_cpp.cmake - DEPENDS ${intermediate_output}) - endforeach() - - add_custom_command( - OUTPUT ${intermediate_outputs} - COMMAND ${Protobuf_PROTOC_EXECUTABLE} - ARGS ${protobuf_include_path} ${protoc_args} ${abs_name} - DEPENDS ${abs_name} ${Protobuf_PROTOC_EXECUTABLE} ${PLUGIN} - COMMENT "Running C++ protocol buffer compiler on ${name}" - VERBATIM ) - endforeach() - - set_source_files_properties(${srcs} ${hdrs} ${all_intermediate_outputs} PROPERTIES GENERATED TRUE) - set(${SRCS} ${srcs} PARENT_SCOPE) - set(${HDRS} ${hdrs} PARENT_SCOPE) -endfunction() - - -if (PROTOBUF_GENERATE_CPP_SCRIPT_MODE) - set (output "${DIR}/${FILENAME}") - set (intermediate_dir ${DIR}/intermediate) - set (intermediate_output "${intermediate_dir}/${FILENAME}") - - if (COMPILER_ID MATCHES "Clang") - set (pragma_push "#pragma clang diagnostic push\n") - set (pragma_pop "#pragma clang diagnostic pop\n") - set (pragma_disable_warnings "#pragma clang diagnostic ignored \"-Weverything\"\n") - elseif (COMPILER_ID MATCHES "GNU") - set (pragma_push "#pragma GCC diagnostic push\n") - set (pragma_pop "#pragma GCC diagnostic pop\n") - set (pragma_disable_warnings "#pragma GCC diagnostic ignored \"-Wall\"\n" - "#pragma GCC diagnostic ignored \"-Wextra\"\n" - "#pragma GCC diagnostic ignored \"-Warray-bounds\"\n" - "#pragma GCC diagnostic ignored \"-Wold-style-cast\"\n" - "#pragma GCC diagnostic ignored \"-Wshadow\"\n" - "#pragma GCC diagnostic ignored \"-Wsuggest-override\"\n" - "#pragma GCC diagnostic ignored \"-Wcast-qual\"\n" - "#pragma GCC diagnostic ignored \"-Wunused-parameter\"\n") - endif() - - if (${FILENAME} MATCHES ".*\\.h") - file(WRITE "${output}" - "#pragma once\n" - ${pragma_push} - ${pragma_disable_warnings} - "#include \"${intermediate_output}\"\n" - ${pragma_pop} - ) - else() - file(WRITE "${output}" - ${pragma_disable_warnings} - "#include \"${intermediate_output}\"\n" - ) - endif() - return() -endif() - - -function(protobuf_generate_cpp SRCS HDRS) - set (modes cpp) - set (output_file_exts ".pb.cc" ".pb.h") - set (plugin) - - protobuf_generate_cpp_impl(srcs hdrs "${modes}" "${output_file_exts}" "${plugin}" ${ARGN}) - - set(${SRCS} ${srcs} PARENT_SCOPE) - set(${HDRS} ${hdrs} PARENT_SCOPE) -endfunction() - - -function(protobuf_generate_grpc_cpp SRCS HDRS) - set (modes cpp grpc) - set (output_file_exts ".pb.cc" ".pb.h" ".grpc.pb.cc" ".grpc.pb.h") - set (plugin "protoc-gen-grpc=${GRPC_CPP_PLUGIN_EXECUTABLE}") - - protobuf_generate_cpp_impl(srcs hdrs "${modes}" "${output_file_exts}" "${plugin}" ${ARGN}) - - set(${SRCS} ${srcs} PARENT_SCOPE) - set(${HDRS} ${hdrs} PARENT_SCOPE) -endfunction() diff --git a/contrib/protobuf-cmake/protobuf_generate.cmake b/contrib/protobuf-cmake/protobuf_generate.cmake new file mode 100644 index 00000000000..fc1dfd9cc11 --- /dev/null +++ b/contrib/protobuf-cmake/protobuf_generate.cmake @@ -0,0 +1,198 @@ +# The code in this file was copied from https://github.com/Kitware/CMake/blob/master/Modules/FindProtobuf.cmake + +#[[ +Add custom commands to process ``.proto`` files to C++:: + +protobuf_generate_cpp ( + [DESCRIPTORS ] [EXPORT_MACRO ] [...]) + +``SRCS`` + Variable to define with autogenerated source files +``HDRS`` + Variable to define with autogenerated header files +``DESCRIPTORS`` + Variable to define with autogenerated descriptor files, if requested. +``EXPORT_MACRO`` + is a macro which should expand to ``__declspec(dllexport)`` or + ``__declspec(dllimport)`` depending on what is being compiled. +``ARGN`` + ``.proto`` files +#]] + +function(PROTOBUF_GENERATE_CPP SRCS HDRS) + cmake_parse_arguments(protobuf_generate_cpp "" "EXPORT_MACRO;DESCRIPTORS" "" ${ARGN}) + + set(_proto_files "${protobuf_generate_cpp_UNPARSED_ARGUMENTS}") + if(NOT _proto_files) + message(SEND_ERROR "Error: PROTOBUF_GENERATE_CPP() called without any proto files") + return() + endif() + + if(PROTOBUF_GENERATE_CPP_APPEND_PATH) + set(_append_arg APPEND_PATH) + endif() + + if(protobuf_generate_cpp_DESCRIPTORS) + set(_descriptors DESCRIPTORS) + endif() + + if(DEFINED PROTOBUF_IMPORT_DIRS AND NOT DEFINED Protobuf_IMPORT_DIRS) + set(Protobuf_IMPORT_DIRS "${PROTOBUF_IMPORT_DIRS}") + endif() + + if(DEFINED Protobuf_IMPORT_DIRS) + set(_import_arg IMPORT_DIRS ${Protobuf_IMPORT_DIRS}) + endif() + + set(_outvar) + protobuf_generate(${_append_arg} ${_descriptors} LANGUAGE cpp EXPORT_MACRO ${protobuf_generate_cpp_EXPORT_MACRO} OUT_VAR _outvar ${_import_arg} PROTOS ${_proto_files}) + + set(${SRCS}) + set(${HDRS}) + if(protobuf_generate_cpp_DESCRIPTORS) + set(${protobuf_generate_cpp_DESCRIPTORS}) + endif() + + foreach(_file ${_outvar}) + if(_file MATCHES "cc$") + list(APPEND ${SRCS} ${_file}) + elseif(_file MATCHES "desc$") + list(APPEND ${protobuf_generate_cpp_DESCRIPTORS} ${_file}) + else() + list(APPEND ${HDRS} ${_file}) + endif() + endforeach() + set(${SRCS} ${${SRCS}} PARENT_SCOPE) + set(${HDRS} ${${HDRS}} PARENT_SCOPE) + if(protobuf_generate_cpp_DESCRIPTORS) + set(${protobuf_generate_cpp_DESCRIPTORS} "${${protobuf_generate_cpp_DESCRIPTORS}}" PARENT_SCOPE) + endif() +endfunction() + +# By default have PROTOBUF_GENERATE_CPP macro pass -I to protoc +# for each directory where a proto file is referenced. +if(NOT DEFINED PROTOBUF_GENERATE_CPP_APPEND_PATH) + set(PROTOBUF_GENERATE_CPP_APPEND_PATH TRUE) +endif() + +function(protobuf_generate) + set(_options APPEND_PATH DESCRIPTORS) + set(_singleargs LANGUAGE OUT_VAR EXPORT_MACRO PROTOC_OUT_DIR) + if(COMMAND target_sources) + list(APPEND _singleargs TARGET) + endif() + set(_multiargs PROTOS IMPORT_DIRS GENERATE_EXTENSIONS) + + cmake_parse_arguments(protobuf_generate "${_options}" "${_singleargs}" "${_multiargs}" "${ARGN}") + + if(NOT protobuf_generate_PROTOS AND NOT protobuf_generate_TARGET) + message(SEND_ERROR "Error: protobuf_generate called without any targets or source files") + return() + endif() + + if(NOT protobuf_generate_OUT_VAR AND NOT protobuf_generate_TARGET) + message(SEND_ERROR "Error: protobuf_generate called without a target or output variable") + return() + endif() + + if(NOT protobuf_generate_LANGUAGE) + set(protobuf_generate_LANGUAGE cpp) + endif() + string(TOLOWER ${protobuf_generate_LANGUAGE} protobuf_generate_LANGUAGE) + + if(NOT protobuf_generate_PROTOC_OUT_DIR) + set(protobuf_generate_PROTOC_OUT_DIR ${CMAKE_CURRENT_BINARY_DIR}) + endif() + + if(protobuf_generate_EXPORT_MACRO AND protobuf_generate_LANGUAGE STREQUAL cpp) + set(_dll_export_decl "dllexport_decl=${protobuf_generate_EXPORT_MACRO}:") + endif() + + if(NOT protobuf_generate_GENERATE_EXTENSIONS) + if(protobuf_generate_LANGUAGE STREQUAL cpp) + set(protobuf_generate_GENERATE_EXTENSIONS .pb.h .pb.cc) + elseif(protobuf_generate_LANGUAGE STREQUAL python) + set(protobuf_generate_GENERATE_EXTENSIONS _pb2.py) + else() + message(SEND_ERROR "Error: protobuf_generate given unknown Language ${LANGUAGE}, please provide a value for GENERATE_EXTENSIONS") + return() + endif() + endif() + + if(protobuf_generate_TARGET) + get_target_property(_source_list ${protobuf_generate_TARGET} SOURCES) + foreach(_file ${_source_list}) + if(_file MATCHES "proto$") + list(APPEND protobuf_generate_PROTOS ${_file}) + endif() + endforeach() + endif() + + if(NOT protobuf_generate_PROTOS) + message(SEND_ERROR "Error: protobuf_generate could not find any .proto files") + return() + endif() + + if(protobuf_generate_APPEND_PATH) + # Create an include path for each file specified + foreach(_file ${protobuf_generate_PROTOS}) + get_filename_component(_abs_file ${_file} ABSOLUTE) + get_filename_component(_abs_path ${_abs_file} PATH) + list(FIND _protobuf_include_path ${_abs_path} _contains_already) + if(${_contains_already} EQUAL -1) + list(APPEND _protobuf_include_path -I ${_abs_path}) + endif() + endforeach() + else() + set(_protobuf_include_path -I ${CMAKE_CURRENT_SOURCE_DIR}) + endif() + + foreach(DIR ${protobuf_generate_IMPORT_DIRS}) + get_filename_component(ABS_PATH ${DIR} ABSOLUTE) + list(FIND _protobuf_include_path ${ABS_PATH} _contains_already) + if(${_contains_already} EQUAL -1) + list(APPEND _protobuf_include_path -I ${ABS_PATH}) + endif() + endforeach() + + set(_generated_srcs_all) + foreach(_proto ${protobuf_generate_PROTOS}) + get_filename_component(_abs_file ${_proto} ABSOLUTE) + get_filename_component(_abs_dir ${_abs_file} DIRECTORY) + get_filename_component(_basename ${_proto} NAME_WE) + file(RELATIVE_PATH _rel_dir ${CMAKE_CURRENT_SOURCE_DIR} ${_abs_dir}) + + set(_possible_rel_dir) + if (NOT protobuf_generate_APPEND_PATH) + set(_possible_rel_dir ${_rel_dir}/) + endif() + + set(_generated_srcs) + foreach(_ext ${protobuf_generate_GENERATE_EXTENSIONS}) + list(APPEND _generated_srcs "${protobuf_generate_PROTOC_OUT_DIR}/${_possible_rel_dir}${_basename}${_ext}") + endforeach() + + if(protobuf_generate_DESCRIPTORS AND protobuf_generate_LANGUAGE STREQUAL cpp) + set(_descriptor_file "${CMAKE_CURRENT_BINARY_DIR}/${_basename}.desc") + set(_dll_desc_out "--descriptor_set_out=${_descriptor_file}") + list(APPEND _generated_srcs ${_descriptor_file}) + endif() + list(APPEND _generated_srcs_all ${_generated_srcs}) + + add_custom_command( + OUTPUT ${_generated_srcs} + COMMAND protobuf::protoc + ARGS --${protobuf_generate_LANGUAGE}_out ${_dll_export_decl}${protobuf_generate_PROTOC_OUT_DIR} ${_dll_desc_out} ${_protobuf_include_path} ${_abs_file} + DEPENDS ${_abs_file} protobuf::protoc + COMMENT "Running ${protobuf_generate_LANGUAGE} protocol buffer compiler on ${_proto}" + VERBATIM ) + endforeach() + + set_source_files_properties(${_generated_srcs_all} PROPERTIES GENERATED TRUE) + if(protobuf_generate_OUT_VAR) + set(${protobuf_generate_OUT_VAR} ${_generated_srcs_all} PARENT_SCOPE) + endif() + if(protobuf_generate_TARGET) + target_sources(${protobuf_generate_TARGET} PRIVATE ${_generated_srcs_all}) + endif() +endfunction() From 8bc9ede711e69a1edfbb0f8732f2fb8226d676fd Mon Sep 17 00:00:00 2001 From: alesapin Date: Thu, 1 Oct 2020 09:53:47 +0300 Subject: [PATCH 254/321] Fix clang tidy --- src/Storages/MergeTree/MergeTreeData.cpp | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 3f09f3a7347..f2b26a928c1 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -1457,10 +1457,10 @@ void MergeTreeData::checkAlterIsPossible(const AlterCommands & commands, const S /// Some type changes for version column is allowed despite it's a part of sorting key if (command.type == AlterCommand::MODIFY_COLUMN && command.column_name == merging_params.version_column) { - auto new_type = command.data_type; - auto old_type = old_types[command.column_name]; + const IDataType * new_type = command.data_type.get(); + const IDataType * old_type = old_types[command.column_name]; - checkVersionColumnTypesConversion(old_type, new_type.get(), command.column_name); + checkVersionColumnTypesConversion(old_type, new_type, command.column_name); /// No other checks required continue; From 1df6b372885c49af6e025d9538e39c95068e2213 Mon Sep 17 00:00:00 2001 From: alex-zaitsev Date: Thu, 1 Oct 2020 09:57:07 +0300 Subject: [PATCH 255/321] Update RedShift benchmarks Few changes: 1) Grouped comparisons with open source on the left and commercial ones on the right 2) Removed old Mark's benchmark 3) Updated a link to Altinity RedShift benchmarks 4) Added a link to Brandon Harris new benchmark --- website/templates/index/performance.html | 17 ++++++++--------- 1 file changed, 8 insertions(+), 9 deletions(-) diff --git a/website/templates/index/performance.html b/website/templates/index/performance.html index 61cd31a06ca..75e2a43f902 100644 --- a/website/templates/index/performance.html +++ b/website/templates/index/performance.html @@ -18,27 +18,26 @@ From 38e50d10784f757e819c0a0eb28f9de1b59a798e Mon Sep 17 00:00:00 2001 From: alesapin Date: Thu, 1 Oct 2020 10:38:06 +0300 Subject: [PATCH 256/321] Revert "enable FastMemcpy properly" This reverts commit 6e239012279a22461014d508c5c8dffeaac2e3e9. --- CMakeLists.txt | 6 ------ base/glibc-compatibility/CMakeLists.txt | 2 ++ contrib/FastMemcpy/CMakeLists.txt | 2 ++ 3 files changed, 4 insertions(+), 6 deletions(-) diff --git a/CMakeLists.txt b/CMakeLists.txt index 0969d49d9a5..f4e230fbd93 100644 --- a/CMakeLists.txt +++ b/CMakeLists.txt @@ -369,12 +369,6 @@ endif () include(cmake/dbms_glob_sources.cmake) -if (${ENABLE_LIBRARIES} OR ${GLIBC_COMPATIBILITY}) - option (ENABLE_FASTMEMCPY "Enable FastMemcpy library (only internal)" ON) -else () - option (ENABLE_FASTMEMCPY "Enable FastMemcpy library (only internal)" OFF) -endif () - if (OS_LINUX OR OS_ANDROID) include(cmake/linux/default_libs.cmake) elseif (OS_DARWIN) diff --git a/base/glibc-compatibility/CMakeLists.txt b/base/glibc-compatibility/CMakeLists.txt index 1a76ef3df32..2bd4e20d3bc 100644 --- a/base/glibc-compatibility/CMakeLists.txt +++ b/base/glibc-compatibility/CMakeLists.txt @@ -1,4 +1,6 @@ if (GLIBC_COMPATIBILITY) + set (ENABLE_FASTMEMCPY ON) + enable_language(ASM) include(CheckIncludeFile) diff --git a/contrib/FastMemcpy/CMakeLists.txt b/contrib/FastMemcpy/CMakeLists.txt index 0f3072b4e05..8efe6d45dff 100644 --- a/contrib/FastMemcpy/CMakeLists.txt +++ b/contrib/FastMemcpy/CMakeLists.txt @@ -1,3 +1,5 @@ +option (ENABLE_FASTMEMCPY "Enable FastMemcpy library (only internal)" ${ENABLE_LIBRARIES}) + if (NOT OS_LINUX OR ARCH_AARCH64) set (ENABLE_FASTMEMCPY OFF) endif () From 94ab1313e3d331551af504808baca8bd1ec0aee0 Mon Sep 17 00:00:00 2001 From: alesapin Date: Thu, 1 Oct 2020 10:39:27 +0300 Subject: [PATCH 257/321] Add fno-builtin-memcpy --- CMakeLists.txt | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/CMakeLists.txt b/CMakeLists.txt index f4e230fbd93..32f37881578 100644 --- a/CMakeLists.txt +++ b/CMakeLists.txt @@ -305,9 +305,10 @@ if (COMPILER_CLANG) # completely. if (ENABLE_THINLTO AND NOT ENABLE_TESTS AND NOT SANITIZE) # Link time optimization - set (CMAKE_C_FLAGS_RELWITHDEBINFO "${CMAKE_C_FLAGS_RELWITHDEBINFO} -flto=thin") - set (CMAKE_CXX_FLAGS_RELWITHDEBINFO "${CMAKE_CXX_FLAGS_RELWITHDEBINFO} -flto=thin") - set (CMAKE_EXE_LINKER_FLAGS_RELWITHDEBINFO "${CMAKE_EXE_LINKER_FLAGS_RELWITHDEBINFO} -flto=thin") + # For some reason thin-lto use memcpy from glibc without -fno-builtin-memcpy + set (CMAKE_C_FLAGS_RELWITHDEBINFO "${CMAKE_C_FLAGS_RELWITHDEBINFO} -flto=thin -fno-builtin-memcpy") + set (CMAKE_CXX_FLAGS_RELWITHDEBINFO "${CMAKE_CXX_FLAGS_RELWITHDEBINFO} -flto=thin -fno-builtin-memcpy") + set (CMAKE_EXE_LINKER_FLAGS_RELWITHDEBINFO "${CMAKE_EXE_LINKER_FLAGS_RELWITHDEBINFO} -flto=thin -fno-builtin-memcpy") elseif (ENABLE_THINLTO) message (${RECONFIGURE_MESSAGE_LEVEL} "Cannot enable ThinLTO") endif () From b642bd494dea25f1e289a901225f6461832f529f Mon Sep 17 00:00:00 2001 From: Mark Papadakis Date: Thu, 1 Oct 2020 12:03:19 +0300 Subject: [PATCH 258/321] Update ActionsVisitor.cpp Refactoring: eliminated a local, throws if it matches the column name as opposed to iterating to the end of the container. --- src/Interpreters/ActionsVisitor.cpp | 15 +++++++-------- 1 file changed, 7 insertions(+), 8 deletions(-) diff --git a/src/Interpreters/ActionsVisitor.cpp b/src/Interpreters/ActionsVisitor.cpp index 1d524669fd9..2236b35d458 100644 --- a/src/Interpreters/ActionsVisitor.cpp +++ b/src/Interpreters/ActionsVisitor.cpp @@ -509,15 +509,14 @@ void ActionsMatcher::visit(const ASTIdentifier & identifier, const ASTPtr & ast, { /// The requested column is not in the block. /// If such a column exists in the table, then the user probably forgot to surround it with an aggregate function or add it to GROUP BY. - - bool found = false; - for (const auto & column_name_type : data.source_columns) - if (column_name_type.name == column_name.get(ast)) - found = true; - - if (found) - throw Exception("Column " + backQuote(column_name.get(ast)) + " is not under aggregate function and not in GROUP BY", + + for (const auto & column_name_type : data.source_columns) { + if (column_name_type.name == column_name.get(ast)) { + throw Exception("Column " + backQuote(column_name.get(ast)) + " is not under aggregate function and not in GROUP BY", ErrorCodes::NOT_AN_AGGREGATE); + } + } + /// Special check for WITH statement alias. Add alias action to be able to use this alias. if (identifier.prefer_alias_to_column_name && !identifier.alias.empty()) From d0dff5b738bacc146ab2df784cccb255094b4a28 Mon Sep 17 00:00:00 2001 From: Alexander Kuzmenkov Date: Thu, 1 Oct 2020 12:27:05 +0300 Subject: [PATCH 259/321] fixes --- docker/packager/binary/build.sh | 3 ++- docker/test/stateful/run.sh | 3 ++- docker/test/stateful_with_coverage/run.sh | 3 ++- docker/test/stress/run.sh | 3 ++- 4 files changed, 8 insertions(+), 4 deletions(-) diff --git a/docker/packager/binary/build.sh b/docker/packager/binary/build.sh index 300b70c4720..5df0392cb4d 100755 --- a/docker/packager/binary/build.sh +++ b/docker/packager/binary/build.sh @@ -18,7 +18,8 @@ ccache --zero-stats ||: ln -s /usr/lib/x86_64-linux-gnu/libOpenCL.so.1.0.0 /usr/lib/libOpenCL.so ||: rm -f CMakeCache.txt cmake --debug-trycompile --verbose=1 -DCMAKE_VERBOSE_MAKEFILE=1 -LA "-DCMAKE_BUILD_TYPE=$BUILD_TYPE" "-DSANITIZE=$SANITIZER" -DENABLE_CHECK_HEAVY_BUILDS=1 "$CMAKE_FLAGS" .. -ninja "$NINJA_FLAGS" clickhouse-bundle +# shellcheck disable=SC2086 # No quotes because I want it to expand to nothing if empty. +ninja $NINJA_FLAGS clickhouse-bundle mv ./programs/clickhouse* /output mv ./src/unit_tests_dbms /output ||: # may not exist for some binary builds find . -name '*.so' -print -exec mv '{}' /output \; diff --git a/docker/test/stateful/run.sh b/docker/test/stateful/run.sh index fa1d627bf57..98c9427d556 100755 --- a/docker/test/stateful/run.sh +++ b/docker/test/stateful/run.sh @@ -31,7 +31,8 @@ function start() } start -/s3downloader --dataset-names "$DATASETS" +# shellcheck disable=SC2086 # No quotes because I want to split it into words. +/s3downloader --dataset-names $DATASETS chmod 777 -R /var/lib/clickhouse clickhouse-client --query "SHOW DATABASES" clickhouse-client --query "ATTACH DATABASE datasets ENGINE = Ordinary" diff --git a/docker/test/stateful_with_coverage/run.sh b/docker/test/stateful_with_coverage/run.sh index dc730f461bb..b20e21efaf1 100755 --- a/docker/test/stateful_with_coverage/run.sh +++ b/docker/test/stateful_with_coverage/run.sh @@ -72,7 +72,8 @@ function start() start -if ! /s3downloader --dataset-names "$DATASETS"; then +# shellcheck disable=SC2086 # No quotes because I want to split it into words. +if ! /s3downloader --dataset-names $DATASETS; then echo "Cannot download datatsets" exit 1 fi diff --git a/docker/test/stress/run.sh b/docker/test/stress/run.sh index bea246eb536..ba529c401ea 100755 --- a/docker/test/stress/run.sh +++ b/docker/test/stress/run.sh @@ -49,7 +49,8 @@ export ASAN_OPTIONS='malloc_context_size=10 verbosity=1 allocator_release_to_os_ start -/s3downloader --dataset-names "$DATASETS" +# shellcheck disable=SC2086 # No quotes because I want to split it into words. +/s3downloader --dataset-names $DATASETS chmod 777 -R /var/lib/clickhouse clickhouse-client --query "ATTACH DATABASE IF NOT EXISTS datasets ENGINE = Ordinary" clickhouse-client --query "CREATE DATABASE IF NOT EXISTS test" From b6601cc11ae5d42e47c1ab70eaaabffa1bb8ce46 Mon Sep 17 00:00:00 2001 From: Ilya Yatsishin <2159081+qoega@users.noreply.github.com> Date: Thu, 1 Oct 2020 14:00:35 +0300 Subject: [PATCH 260/321] Update SECURITY.md --- SECURITY.md | 8 ++++++-- 1 file changed, 6 insertions(+), 2 deletions(-) diff --git a/SECURITY.md b/SECURITY.md index 67d8dc85c13..3f7a05e1c66 100644 --- a/SECURITY.md +++ b/SECURITY.md @@ -13,8 +13,12 @@ currently being supported with security updates: | 19.14 | :white_check_mark: | | 20.1 | :x: | | 20.3 | :white_check_mark: | -| 20.4 | :white_check_mark: | -| 20.5 | :white_check_mark: | +| 20.4 | :x: | +| 20.5 | :x: | +| 20.6 | :x: | +| 20.7 | :white_check_mark: | +| 20.8 | :white_check_mark: | +| 20.9 | :white_check_mark: | ## Reporting a Vulnerability From 99915fa444f24a40b91a2f14031fd5c22d792187 Mon Sep 17 00:00:00 2001 From: Alexander Kuzmenkov Date: Thu, 1 Oct 2020 14:00:58 +0300 Subject: [PATCH 261/321] remove unrelated changes --- src/Common/Exception.h | 7 ++----- src/Core/BaseSettings.h | 22 +++++++--------------- 2 files changed, 9 insertions(+), 20 deletions(-) diff --git a/src/Common/Exception.h b/src/Common/Exception.h index d0de8d6a3f2..763b90048bb 100644 --- a/src/Common/Exception.h +++ b/src/Common/Exception.h @@ -39,11 +39,8 @@ public: const char * name() const throw() override { return "DB::Exception"; } const char * what() const throw() override { return message().data(); } - template - void addMessage(Fmt&&... fmt) - { - extendedMessage(fmt::format(std::forward(fmt)...)); - } + /// Add something to the existing message. + void addMessage(const std::string & arg) { extendedMessage(arg); } std::string getStackTraceString() const; diff --git a/src/Core/BaseSettings.h b/src/Core/BaseSettings.h index b193fdd4c93..7de87b345c1 100644 --- a/src/Core/BaseSettings.h +++ b/src/Core/BaseSettings.h @@ -390,21 +390,13 @@ String BaseSettings::valueToStringUtil(const std::string_view & name, c template Field BaseSettings::stringToValueUtil(const std::string_view & name, const String & str) { - try - { - const auto & accessor = Traits::Accessor::instance(); - if (size_t index = accessor.find(name); index != static_cast(-1)) - return accessor.stringToValueUtil(index, str); - if constexpr (Traits::allow_custom_settings) - return Field::restoreFromDump(str); - else - BaseSettingsHelpers::throwSettingNotFound(name); - } - catch (Exception & e) - { - e.addMessage("while parsing value '{}' for setting '{}'", str, name); - throw; - } + const auto & accessor = Traits::Accessor::instance(); + if (size_t index = accessor.find(name); index != static_cast(-1)) + return accessor.stringToValueUtil(index, str); + if constexpr (Traits::allow_custom_settings) + return Field::restoreFromDump(str); + else + BaseSettingsHelpers::throwSettingNotFound(name); } template From c5d6e2e67cac238868df3522a2538a8c5843a63a Mon Sep 17 00:00:00 2001 From: Alexander Kuzmenkov Date: Thu, 1 Oct 2020 14:02:53 +0300 Subject: [PATCH 262/321] Add context to error messages --- src/Common/Exception.h | 7 +++++-- src/Core/BaseSettings.h | 22 +++++++++++++++------- 2 files changed, 20 insertions(+), 9 deletions(-) diff --git a/src/Common/Exception.h b/src/Common/Exception.h index 763b90048bb..d0de8d6a3f2 100644 --- a/src/Common/Exception.h +++ b/src/Common/Exception.h @@ -39,8 +39,11 @@ public: const char * name() const throw() override { return "DB::Exception"; } const char * what() const throw() override { return message().data(); } - /// Add something to the existing message. - void addMessage(const std::string & arg) { extendedMessage(arg); } + template + void addMessage(Fmt&&... fmt) + { + extendedMessage(fmt::format(std::forward(fmt)...)); + } std::string getStackTraceString() const; diff --git a/src/Core/BaseSettings.h b/src/Core/BaseSettings.h index 7de87b345c1..b193fdd4c93 100644 --- a/src/Core/BaseSettings.h +++ b/src/Core/BaseSettings.h @@ -390,13 +390,21 @@ String BaseSettings::valueToStringUtil(const std::string_view & name, c template Field BaseSettings::stringToValueUtil(const std::string_view & name, const String & str) { - const auto & accessor = Traits::Accessor::instance(); - if (size_t index = accessor.find(name); index != static_cast(-1)) - return accessor.stringToValueUtil(index, str); - if constexpr (Traits::allow_custom_settings) - return Field::restoreFromDump(str); - else - BaseSettingsHelpers::throwSettingNotFound(name); + try + { + const auto & accessor = Traits::Accessor::instance(); + if (size_t index = accessor.find(name); index != static_cast(-1)) + return accessor.stringToValueUtil(index, str); + if constexpr (Traits::allow_custom_settings) + return Field::restoreFromDump(str); + else + BaseSettingsHelpers::throwSettingNotFound(name); + } + catch (Exception & e) + { + e.addMessage("while parsing value '{}' for setting '{}'", str, name); + throw; + } } template From 0f92f60ab25df5c37563c9ace9e74a815a2a564d Mon Sep 17 00:00:00 2001 From: alesapin Date: Thu, 1 Oct 2020 14:35:56 +0300 Subject: [PATCH 263/321] Update ActionsVisitor.cpp --- src/Interpreters/ActionsVisitor.cpp | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/src/Interpreters/ActionsVisitor.cpp b/src/Interpreters/ActionsVisitor.cpp index 2236b35d458..a107c29717b 100644 --- a/src/Interpreters/ActionsVisitor.cpp +++ b/src/Interpreters/ActionsVisitor.cpp @@ -510,14 +510,15 @@ void ActionsMatcher::visit(const ASTIdentifier & identifier, const ASTPtr & ast, /// The requested column is not in the block. /// If such a column exists in the table, then the user probably forgot to surround it with an aggregate function or add it to GROUP BY. - for (const auto & column_name_type : data.source_columns) { - if (column_name_type.name == column_name.get(ast)) { + for (const auto & column_name_type : data.source_columns) + { + if (column_name_type.name == column_name.get(ast)) + { throw Exception("Column " + backQuote(column_name.get(ast)) + " is not under aggregate function and not in GROUP BY", ErrorCodes::NOT_AN_AGGREGATE); } } - /// Special check for WITH statement alias. Add alias action to be able to use this alias. if (identifier.prefer_alias_to_column_name && !identifier.alias.empty()) data.addAction(ExpressionAction::addAliases({{identifier.name, identifier.alias}})); From b98f4192e42157288e60d9c9cb22e04ce66ea411 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 1 Oct 2020 16:02:14 +0300 Subject: [PATCH 264/321] Update tsan_suppressions.txt --- tests/tsan_suppressions.txt | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/tsan_suppressions.txt b/tests/tsan_suppressions.txt index c07a31f2127..ccc36d876f7 100644 --- a/tests/tsan_suppressions.txt +++ b/tests/tsan_suppressions.txt @@ -1,4 +1,4 @@ # looks like a bug in clang-11 thread sanitizer, detects normal data race with random FD in this method race:DB::LazyPipeFDs::close # races in openSSL https://github.com/openssl/openssl/issues/11974 -race:evp_cipher_cache_constants +fun:evp_cipher_cache_constants From c20053351c0f09d772fcdeedf810f3165d08f19e Mon Sep 17 00:00:00 2001 From: Alexander Kuzmenkov Date: Thu, 1 Oct 2020 17:23:48 +0300 Subject: [PATCH 265/321] Make the test throw more comprehensive errors --- .../gtest_exception_on_incorrect_pipeline.cpp | 26 +++++++------------ 1 file changed, 10 insertions(+), 16 deletions(-) diff --git a/src/Processors/tests/gtest_exception_on_incorrect_pipeline.cpp b/src/Processors/tests/gtest_exception_on_incorrect_pipeline.cpp index 10a0fae2696..3d85ffede9a 100644 --- a/src/Processors/tests/gtest_exception_on_incorrect_pipeline.cpp +++ b/src/Processors/tests/gtest_exception_on_incorrect_pipeline.cpp @@ -49,21 +49,15 @@ TEST(Processors, PortsNotConnected) processors.emplace_back(std::move(source)); processors.emplace_back(std::move(sink)); - auto exec = [&]() + try { - - try - { - PipelineExecutor executor(processors); - executor.execute(1); - } - catch (DB::Exception & e) - { - std::cout << e.displayText() << std::endl; - ASSERT_TRUE(e.displayText().find("pipeline") != std::string::npos); - throw; - } - }; - - ASSERT_THROW(exec(), DB::Exception); + PipelineExecutor executor(processors); + executor.execute(1); + ASSERT_TRUE(false) << "Should have thrown."; + } + catch (DB::Exception & e) + { + std::cout << e.displayText() << std::endl; + ASSERT_TRUE(e.displayText().find("pipeline") != std::string::npos) << "Expected 'pipeline', got: " << e.displayText(); + } } From 6e07bfe7bc93721676ec0b7f99ef267e26acea80 Mon Sep 17 00:00:00 2001 From: Alexander Kuzmenkov Date: Thu, 1 Oct 2020 17:29:22 +0300 Subject: [PATCH 266/321] return the comment --- src/Common/Exception.h | 1 + 1 file changed, 1 insertion(+) diff --git a/src/Common/Exception.h b/src/Common/Exception.h index d0de8d6a3f2..09d923fe60e 100644 --- a/src/Common/Exception.h +++ b/src/Common/Exception.h @@ -39,6 +39,7 @@ public: const char * name() const throw() override { return "DB::Exception"; } const char * what() const throw() override { return message().data(); } + /// Add something to the existing message. template void addMessage(Fmt&&... fmt) { From 1829c8545c26527d82b4f54b7e787bd77ba72b28 Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Thu, 1 Oct 2020 19:17:11 +0300 Subject: [PATCH 267/321] Update SECURITY.md --- SECURITY.md | 1 - 1 file changed, 1 deletion(-) diff --git a/SECURITY.md b/SECURITY.md index 3f7a05e1c66..6d64dd2d24b 100644 --- a/SECURITY.md +++ b/SECURITY.md @@ -10,7 +10,6 @@ currently being supported with security updates: | 1.x | :x: | | 18.x | :x: | | 19.x | :x: | -| 19.14 | :white_check_mark: | | 20.1 | :x: | | 20.3 | :white_check_mark: | | 20.4 | :x: | From eb1d25335573e84165f969228625f0566666480e Mon Sep 17 00:00:00 2001 From: Nicolae Vartolomei Date: Thu, 1 Oct 2020 16:47:18 +0100 Subject: [PATCH 268/321] Throw an error when a single parameter is passed to ReplicatedMergeTree instead of ignoring it --- src/Storages/MergeTree/registerStorageMergeTree.cpp | 2 +- .../01512_create_replicate_merge_tree_one_arg.reference | 0 .../0_stateless/01512_create_replicate_merge_tree_one_arg.sql | 3 +++ 3 files changed, 4 insertions(+), 1 deletion(-) create mode 100644 tests/queries/0_stateless/01512_create_replicate_merge_tree_one_arg.reference create mode 100644 tests/queries/0_stateless/01512_create_replicate_merge_tree_one_arg.sql diff --git a/src/Storages/MergeTree/registerStorageMergeTree.cpp b/src/Storages/MergeTree/registerStorageMergeTree.cpp index 2b0d1a60657..f94425a81d3 100644 --- a/src/Storages/MergeTree/registerStorageMergeTree.cpp +++ b/src/Storages/MergeTree/registerStorageMergeTree.cpp @@ -446,7 +446,7 @@ static StoragePtr create(const StorageFactory::Arguments & args) "No replica name in config" + getMergeTreeVerboseHelp(is_extended_storage_def), ErrorCodes::NO_REPLICA_NAME_GIVEN); ++arg_num; } - else if (is_extended_storage_def && !has_arguments) + else if (is_extended_storage_def && arg_cnt == 0) { /// Try use default values if arguments are not specified. /// Note: {uuid} macro works for ON CLUSTER queries when database engine is Atomic. diff --git a/tests/queries/0_stateless/01512_create_replicate_merge_tree_one_arg.reference b/tests/queries/0_stateless/01512_create_replicate_merge_tree_one_arg.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/01512_create_replicate_merge_tree_one_arg.sql b/tests/queries/0_stateless/01512_create_replicate_merge_tree_one_arg.sql new file mode 100644 index 00000000000..f33c1534a44 --- /dev/null +++ b/tests/queries/0_stateless/01512_create_replicate_merge_tree_one_arg.sql @@ -0,0 +1,3 @@ +CREATE TABLE mt (v UInt8) ENGINE = ReplicatedMergeTree('/clickhouse/tables/test_01497/mt') + ORDER BY tuple() -- { serverError 36 } + From 409aaabc2f375d118c78a257f6441374d867b159 Mon Sep 17 00:00:00 2001 From: alesapin Date: Thu, 1 Oct 2020 22:22:14 +0300 Subject: [PATCH 269/321] Revert "Add fno-builtin-memcpy" This reverts commit 94ab1313e3d331551af504808baca8bd1ec0aee0. --- CMakeLists.txt | 7 +++---- 1 file changed, 3 insertions(+), 4 deletions(-) diff --git a/CMakeLists.txt b/CMakeLists.txt index 32f37881578..f4e230fbd93 100644 --- a/CMakeLists.txt +++ b/CMakeLists.txt @@ -305,10 +305,9 @@ if (COMPILER_CLANG) # completely. if (ENABLE_THINLTO AND NOT ENABLE_TESTS AND NOT SANITIZE) # Link time optimization - # For some reason thin-lto use memcpy from glibc without -fno-builtin-memcpy - set (CMAKE_C_FLAGS_RELWITHDEBINFO "${CMAKE_C_FLAGS_RELWITHDEBINFO} -flto=thin -fno-builtin-memcpy") - set (CMAKE_CXX_FLAGS_RELWITHDEBINFO "${CMAKE_CXX_FLAGS_RELWITHDEBINFO} -flto=thin -fno-builtin-memcpy") - set (CMAKE_EXE_LINKER_FLAGS_RELWITHDEBINFO "${CMAKE_EXE_LINKER_FLAGS_RELWITHDEBINFO} -flto=thin -fno-builtin-memcpy") + set (CMAKE_C_FLAGS_RELWITHDEBINFO "${CMAKE_C_FLAGS_RELWITHDEBINFO} -flto=thin") + set (CMAKE_CXX_FLAGS_RELWITHDEBINFO "${CMAKE_CXX_FLAGS_RELWITHDEBINFO} -flto=thin") + set (CMAKE_EXE_LINKER_FLAGS_RELWITHDEBINFO "${CMAKE_EXE_LINKER_FLAGS_RELWITHDEBINFO} -flto=thin") elseif (ENABLE_THINLTO) message (${RECONFIGURE_MESSAGE_LEVEL} "Cannot enable ThinLTO") endif () From 7c9bbe4c29dd356c1528283e49797007210354f6 Mon Sep 17 00:00:00 2001 From: Amos Bird Date: Fri, 2 Oct 2020 01:19:05 +0800 Subject: [PATCH 270/321] Neutrialize thinlto's memcpy libcall gen. (cherry picked from commit 8ffa4d395c6c920551f9822c37b3f909cfc32c6e) --- CMakeLists.txt | 7 ++++++- base/glibc-compatibility/CMakeLists.txt | 4 ++++ contrib/FastMemcpy/memcpy_wrapper.c | 2 +- 3 files changed, 11 insertions(+), 2 deletions(-) diff --git a/CMakeLists.txt b/CMakeLists.txt index f4e230fbd93..56277a1126d 100644 --- a/CMakeLists.txt +++ b/CMakeLists.txt @@ -513,7 +513,12 @@ endif () macro (add_executable target) # invoke built-in add_executable # explicitly acquire and interpose malloc symbols by clickhouse_malloc - _add_executable (${ARGV} $) + # if GLIBC_COMPATIBILITY is ON, also provide memcpy symbol explicitly to neutrialize thinlto's libcall generation. + if (GLIBC_COMPATIBILITY) + _add_executable (${ARGV} $ $) + else (GLIBC_COMPATIBILITY) + _add_executable (${ARGV} $) + endif () get_target_property (type ${target} TYPE) if (${type} STREQUAL EXECUTABLE) # operator::new/delete for executables (MemoryTracker stuff) diff --git a/base/glibc-compatibility/CMakeLists.txt b/base/glibc-compatibility/CMakeLists.txt index 2bd4e20d3bc..eb9d8db454d 100644 --- a/base/glibc-compatibility/CMakeLists.txt +++ b/base/glibc-compatibility/CMakeLists.txt @@ -27,6 +27,10 @@ if (GLIBC_COMPATIBILITY) list(APPEND glibc_compatibility_sources musl/getentropy.c) endif() + add_library (clickhouse_memcpy OBJECT + ${ClickHouse_SOURCE_DIR}/contrib/FastMemcpy/memcpy_wrapper.c + ) + # Need to omit frame pointers to match the performance of glibc set (CMAKE_C_FLAGS "${CMAKE_C_FLAGS} -fomit-frame-pointer") diff --git a/contrib/FastMemcpy/memcpy_wrapper.c b/contrib/FastMemcpy/memcpy_wrapper.c index 3caf8577d21..1f57345980a 100644 --- a/contrib/FastMemcpy/memcpy_wrapper.c +++ b/contrib/FastMemcpy/memcpy_wrapper.c @@ -1,4 +1,4 @@ -#include +#include "FastMemcpy.h" void * memcpy(void * __restrict destination, const void * __restrict source, size_t size) { From 65a1684ed7a61c110a94de4ceef0c32e11ed1959 Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Thu, 1 Oct 2020 01:28:23 +0300 Subject: [PATCH 271/321] Fix gRPC build scripts. --- cmake/Modules/FindgRPC.cmake | 330 +++++ cmake/find/grpc.cmake | 96 +- contrib/grpc-cmake/CMakeLists.txt | 1115 +---------------- .../grpc-cmake/protobuf_generate_grpc.cmake | 207 +++ docker/packager/unbundled/Dockerfile | 1 + 5 files changed, 632 insertions(+), 1117 deletions(-) create mode 100644 cmake/Modules/FindgRPC.cmake create mode 100644 contrib/grpc-cmake/protobuf_generate_grpc.cmake diff --git a/cmake/Modules/FindgRPC.cmake b/cmake/Modules/FindgRPC.cmake new file mode 100644 index 00000000000..671d207085b --- /dev/null +++ b/cmake/Modules/FindgRPC.cmake @@ -0,0 +1,330 @@ +#[[ +Defines the following variables: +``gRPC_FOUND`` + Whether the gRPC framework is found +``gRPC_INCLUDE_DIRS`` + The include directories of the gRPC framework, including the include directories of the C++ wrapper. +``gRPC_LIBRARIES`` + The libraries of the gRPC framework. +``gRPC_UNSECURE_LIBRARIES`` + The libraries of the gRPC framework without SSL. +``_gRPC_CPP_PLUGIN`` + The plugin for generating gRPC client and server C++ stubs from `.proto` files +``_gRPC_PYTHON_PLUGIN`` + The plugin for generating gRPC client and server Python stubs from `.proto` files + +The following :prop_tgt:`IMPORTED` targets are also defined: +``grpc++`` +``grpc++_unsecure`` +``grpc_cpp_plugin`` +``grpc_python_plugin`` + +Add custom commands to process ``.proto`` files to C++:: +protobuf_generate_grpc_cpp( + [DESCRIPTORS ] [EXPORT_MACRO ] [...]) + +``SRCS`` + Variable to define with autogenerated source files +``HDRS`` + Variable to define with autogenerated header files +``DESCRIPTORS`` + Variable to define with autogenerated descriptor files, if requested. +``EXPORT_MACRO`` + is a macro which should expand to ``__declspec(dllexport)`` or + ``__declspec(dllimport)`` depending on what is being compiled. +``ARGN`` + ``.proto`` files +#]] + +# Function to generate C++ files from .proto files. +# This function is a modified version of the function PROTOBUF_GENERATE_CPP() copied from https://github.com/Kitware/CMake/blob/master/Modules/FindProtobuf.cmake. +function(PROTOBUF_GENERATE_GRPC_CPP SRCS HDRS) + cmake_parse_arguments(protobuf_generate_grpc_cpp "" "EXPORT_MACRO;DESCRIPTORS" "" ${ARGN}) + + set(_proto_files "${protobuf_generate_grpc_cpp_UNPARSED_ARGUMENTS}") + if(NOT _proto_files) + message(SEND_ERROR "Error: PROTOBUF_GENERATE_GRPC_CPP() called without any proto files") + return() + endif() + + if(PROTOBUF_GENERATE_GRPC_CPP_APPEND_PATH) + set(_append_arg APPEND_PATH) + endif() + + if(protobuf_generate_grpc_cpp_DESCRIPTORS) + set(_descriptors DESCRIPTORS) + endif() + + if(DEFINED PROTOBUF_IMPORT_DIRS AND NOT DEFINED Protobuf_IMPORT_DIRS) + set(Protobuf_IMPORT_DIRS "${PROTOBUF_IMPORT_DIRS}") + endif() + + if(DEFINED Protobuf_IMPORT_DIRS) + set(_import_arg IMPORT_DIRS ${Protobuf_IMPORT_DIRS}) + endif() + + set(_outvar) + protobuf_generate_grpc(${_append_arg} ${_descriptors} LANGUAGE cpp EXPORT_MACRO ${protobuf_generate_cpp_EXPORT_MACRO} OUT_VAR _outvar ${_import_arg} PROTOS ${_proto_files}) + + set(${SRCS}) + set(${HDRS}) + if(protobuf_generate_grpc_cpp_DESCRIPTORS) + set(${protobuf_generate_grpc_cpp_DESCRIPTORS}) + endif() + + foreach(_file ${_outvar}) + if(_file MATCHES "cc$") + list(APPEND ${SRCS} ${_file}) + elseif(_file MATCHES "desc$") + list(APPEND ${protobuf_generate_grpc_cpp_DESCRIPTORS} ${_file}) + else() + list(APPEND ${HDRS} ${_file}) + endif() + endforeach() + set(${SRCS} ${${SRCS}} PARENT_SCOPE) + set(${HDRS} ${${HDRS}} PARENT_SCOPE) + if(protobuf_generate_grpc_cpp_DESCRIPTORS) + set(${protobuf_generate_grpc_cpp_DESCRIPTORS} "${${protobuf_generate_grpc_cpp_DESCRIPTORS}}" PARENT_SCOPE) + endif() +endfunction() + +# Helper function. +# This function is a modified version of the function protobuf_generate() copied from https://github.com/Kitware/CMake/blob/master/Modules/FindProtobuf.cmake. +function(protobuf_generate_grpc) + set(_options APPEND_PATH DESCRIPTORS) + set(_singleargs LANGUAGE OUT_VAR EXPORT_MACRO PROTOC_OUT_DIR) + if(COMMAND target_sources) + list(APPEND _singleargs TARGET) + endif() + set(_multiargs PROTOS IMPORT_DIRS GENERATE_EXTENSIONS) + + cmake_parse_arguments(protobuf_generate_grpc "${_options}" "${_singleargs}" "${_multiargs}" "${ARGN}") + + if(NOT protobuf_generate_grpc_PROTOS AND NOT protobuf_generate_grpc_TARGET) + message(SEND_ERROR "Error: protobuf_generate_grpc called without any targets or source files") + return() + endif() + + if(NOT protobuf_generate_grpc_OUT_VAR AND NOT protobuf_generate_grpc_TARGET) + message(SEND_ERROR "Error: protobuf_generate_grpc called without a target or output variable") + return() + endif() + + if(NOT protobuf_generate_grpc_LANGUAGE) + set(protobuf_generate_grpc_LANGUAGE cpp) + endif() + string(TOLOWER ${protobuf_generate_grpc_LANGUAGE} protobuf_generate_grpc_LANGUAGE) + + if(NOT protobuf_generate_grpc_PROTOC_OUT_DIR) + set(protobuf_generate_grpc_PROTOC_OUT_DIR ${CMAKE_CURRENT_BINARY_DIR}) + endif() + + if(protobuf_generate_grpc_EXPORT_MACRO AND protobuf_generate_grpc_LANGUAGE STREQUAL cpp) + set(_dll_export_decl "dllexport_decl=${protobuf_generate_grpc_EXPORT_MACRO}:") + endif() + + if(NOT protobuf_generate_grpc_GENERATE_EXTENSIONS) + if(protobuf_generate_grpc_LANGUAGE STREQUAL cpp) + set(protobuf_generate_grpc_GENERATE_EXTENSIONS .pb.h .pb.cc .grpc.pb.h .grpc.pb.cc) + elseif(protobuf_generate_grpc_LANGUAGE STREQUAL python) + set(protobuf_generate_grpc_GENERATE_EXTENSIONS _pb2.py) + else() + message(SEND_ERROR "Error: protobuf_generate_grpc given unknown Language ${LANGUAGE}, please provide a value for GENERATE_EXTENSIONS") + return() + endif() + endif() + + if(NOT protobuf_generate_grpc_PLUGIN) + if(protobuf_generate_grpc_LANGUAGE STREQUAL cpp) + set(protobuf_generate_grpc_PLUGIN "grpc_cpp_plugin") + elseif(protobuf_generate_grpc_LANGUAGE STREQUAL python) + set(protobuf_generate_grpc_PLUGIN "grpc_python_plugin") + else() + message(SEND_ERROR "Error: protobuf_generate_grpc given unknown Language ${LANGUAGE}, please provide a value for PLUGIN") + return() + endif() + endif() + + if(protobuf_generate_grpc_TARGET) + get_target_property(_source_list ${protobuf_generate_grpc_TARGET} SOURCES) + foreach(_file ${_source_list}) + if(_file MATCHES "proto$") + list(APPEND protobuf_generate_grpc_PROTOS ${_file}) + endif() + endforeach() + endif() + + if(NOT protobuf_generate_grpc_PROTOS) + message(SEND_ERROR "Error: protobuf_generate_grpc could not find any .proto files") + return() + endif() + + if(protobuf_generate_grpc_APPEND_PATH) + # Create an include path for each file specified + foreach(_file ${protobuf_generate_grpc_PROTOS}) + get_filename_component(_abs_file ${_file} ABSOLUTE) + get_filename_component(_abs_path ${_abs_file} PATH) + list(FIND _protobuf_include_path ${_abs_path} _contains_already) + if(${_contains_already} EQUAL -1) + list(APPEND _protobuf_include_path -I ${_abs_path}) + endif() + endforeach() + else() + set(_protobuf_include_path -I ${CMAKE_CURRENT_SOURCE_DIR}) + endif() + + foreach(DIR ${protobuf_generate_grpc_IMPORT_DIRS}) + get_filename_component(ABS_PATH ${DIR} ABSOLUTE) + list(FIND _protobuf_include_path ${ABS_PATH} _contains_already) + if(${_contains_already} EQUAL -1) + list(APPEND _protobuf_include_path -I ${ABS_PATH}) + endif() + endforeach() + + set(_generated_srcs_all) + foreach(_proto ${protobuf_generate_grpc_PROTOS}) + get_filename_component(_abs_file ${_proto} ABSOLUTE) + get_filename_component(_abs_dir ${_abs_file} DIRECTORY) + get_filename_component(_basename ${_proto} NAME_WE) + file(RELATIVE_PATH _rel_dir ${CMAKE_CURRENT_SOURCE_DIR} ${_abs_dir}) + + set(_possible_rel_dir) + if(NOT protobuf_generate_grpc_APPEND_PATH) + set(_possible_rel_dir ${_rel_dir}/) + endif() + + set(_generated_srcs) + foreach(_ext ${protobuf_generate_grpc_GENERATE_EXTENSIONS}) + list(APPEND _generated_srcs "${protobuf_generate_grpc_PROTOC_OUT_DIR}/${_possible_rel_dir}${_basename}${_ext}") + endforeach() + + if(protobuf_generate_grpc_DESCRIPTORS AND protobuf_generate_grpc_LANGUAGE STREQUAL cpp) + set(_descriptor_file "${CMAKE_CURRENT_BINARY_DIR}/${_basename}.desc") + set(_dll_desc_out "--descriptor_set_out=${_descriptor_file}") + list(APPEND _generated_srcs ${_descriptor_file}) + endif() + list(APPEND _generated_srcs_all ${_generated_srcs}) + + add_custom_command( + OUTPUT ${_generated_srcs} + COMMAND protobuf::protoc + ARGS --${protobuf_generate_grpc_LANGUAGE}_out ${_dll_export_decl}${protobuf_generate_grpc_PROTOC_OUT_DIR} + --grpc_out ${_dll_export_decl}${protobuf_generate_grpc_PROTOC_OUT_DIR} + --plugin=protoc-gen-grpc=$ + ${_dll_desc_out} ${_protobuf_include_path} ${_abs_file} + DEPENDS ${_abs_file} protobuf::protoc ${protobuf_generate_grpc_PLUGIN} + COMMENT "Running ${protobuf_generate_grpc_LANGUAGE} protocol buffer compiler on ${_proto}" + VERBATIM) + endforeach() + + set_source_files_properties(${_generated_srcs_all} PROPERTIES GENERATED TRUE) + if(protobuf_generate_grpc_OUT_VAR) + set(${protobuf_generate_grpc_OUT_VAR} ${_generated_srcs_all} PARENT_SCOPE) + endif() + if(protobuf_generate_grpc_TARGET) + target_sources(${protobuf_generate_grpc_TARGET} PRIVATE ${_generated_srcs_all}) + endif() +endfunction() + + +# Find the libraries. +if(gRPC_USE_STATIC_LIBS) + # Support preference of static libs by adjusting CMAKE_FIND_LIBRARY_SUFFIXES + set(_gRPC_ORIG_CMAKE_FIND_LIBRARY_SUFFIXES ${CMAKE_FIND_LIBRARY_SUFFIXES}) + if(WIN32) + set(CMAKE_FIND_LIBRARY_SUFFIXES .lib .a ${CMAKE_FIND_LIBRARY_SUFFIXES}) + else() + set(CMAKE_FIND_LIBRARY_SUFFIXES .a) + endif() +endif() + +find_library(gRPC_LIBRARY NAMES grpc) +find_library(gRPC_CPP_LIBRARY NAMES grpc++) +find_library(gRPC_UNSECURE_LIBRARY NAMES grpc_unsecure) +find_library(gRPC_CPP_UNSECURE_LIBRARY NAMES grpc++_unsecure) + +set(gRPC_LIBRARIES) +if(gRPC_USE_UNSECURE_LIBRARIES) + if(gRPC_UNSECURE_LIBRARY) + set(gRPC_LIBRARIES ${gRPC_LIBRARIES} ${gRPC_UNSECURE_LIBRARY}) + endif() + if(gRPC_CPP_UNSECURE_LIBRARY) + set(gRPC_LIBRARIES ${gRPC_LIBRARIES} ${gRPC_CPP_UNSECURE_LIBRARY}) + endif() +else() + if(gRPC_LIBRARY) + set(gRPC_LIBRARIES ${gRPC_LIBRARIES} ${gRPC_LIBRARY}) + endif() + if(gRPC_CPP_UNSECURE_LIBRARY) + set(gRPC_LIBRARIES ${gRPC_LIBRARIES} ${gRPC_CPP_LIBRARY}) + endif() +endif() + +# Restore the original find library ordering. +if(gRPC_USE_STATIC_LIBS) + set(CMAKE_FIND_LIBRARY_SUFFIXES ${_gRPC_ORIG_CMAKE_FIND_LIBRARY_SUFFIXES}) +endif() + +# Find the include directories. +find_path(gRPC_INCLUDE_DIR grpc/grpc.h) +find_path(gRPC_CPP_INCLUDE_DIR grpc++/grpc++.h) + +if(gRPC_INCLUDE_DIR AND gRPC_CPP_INCLUDE_DIR AND NOT(gRPC_INCLUDE_DIR STREQUAL gRPC_CPP_INCLUDE_DIR)) + set(gRPC_INCLUDE_DIRS ${gRPC_INCLUDE_DIR} ${gRPC_CPP_INCLUDE_DIR}) +elseif(gRPC_INCLUDE_DIR) + set(gRPC_INCLUDE_DIRS ${gRPC_INCLUDE_DIR}) +else() + set(gRPC_INCLUDE_DIRS ${gRPC_CPP_INCLUDE_DIR}) +endif() + +# Get full path to plugin. +find_program(_gRPC_CPP_PLUGIN + NAMES grpc_cpp_plugin + DOC "The plugin for generating gRPC client and server C++ stubs from `.proto` files") + +find_program(_gRPC_PYTHON_PLUGIN + NAMES grpc_python_plugin + DOC "The plugin for generating gRPC client and server Python stubs from `.proto` files") + +# Add imported targets. +if(gRPC_CPP_LIBRARY AND NOT TARGET grpc++) + add_library(grpc++ UNKNOWN IMPORTED) + set_target_properties(grpc++ PROPERTIES + IMPORTED_LOCATION "${gRPC_CPP_LIBRARY}") + set_target_properties(grpc++ PROPERTIES + INTERFACE_INCLUDE_DIRECTORIES ${gRPC_INCLUDE_DIRS}) +endif() + +if(gRPC_CPP_UNSECURE_LIBRARY AND NOT TARGET grpc++_unsecure) + add_library(grpc++_unsecure UNKNOWN IMPORTED) + set_target_properties(grpc++_unsecure PROPERTIES + IMPORTED_LOCATION "${gRPC_CPP_UNSECURE_LIBRARY}") + set_target_properties(grpc++_unsecure PROPERTIES + INTERFACE_INCLUDE_DIRECTORIES ${gRPC_INCLUDE_DIRS}) +endif() + +if(gRPC_CPP_PLUGIN AND NOT TARGET grpc_cpp_plugin) + add_executable(grpc_cpp_plugin IMPORTED) + set_target_properties(grpc_cpp_plugin PROPERTIES + IMPORTED_LOCATION "${gRPC_CPP_PLUGIN}") +endif() + +if(gRPC_PYTHON_PLUGIN AND NOT TARGET grpc_python_plugin) + add_executable(grpc_python_plugin IMPORTED) + set_target_properties(grpc_python_plugin PROPERTIES + IMPORTED_LOCATION "${gRPC_PYTHON_PLUGIN}") +endif() + +#include(FindPackageHandleStandardArgs.cmake) +FIND_PACKAGE_HANDLE_STANDARD_ARGS(gRPC + REQUIRED_VARS gRPC_LIBRARY gRPC_CPP_LIBRARY gRPC_UNSECURE_LIBRARY gRPC_CPP_UNSECURE_LIBRARY + gRPC_INCLUDE_DIR gRPC_CPP_INCLUDE_DIR _gRPC_CPP_PLUGIN _gRPC_PYTHON_PLUGIN) + +if(gRPC_FOUND) + if(gRPC_DEBUG) + message(STATUS "gRPC: INCLUDE_DIRS=${gRPC_INCLUDE_DIRS}") + message(STATUS "gRPC: LIBRARIES=${gRPC_LIBRARIES}") + message(STATUS "gRPC: CPP_PLUGIN=${_gRPC_CPP_PLUGIN}") + message(STATUS "gRPC: PYTHON_PLUGIN=${_gRPC_PYTHON_PLUGIN}") + endif() +endif() diff --git a/cmake/find/grpc.cmake b/cmake/find/grpc.cmake index 52d0e042573..fa283d98225 100644 --- a/cmake/find/grpc.cmake +++ b/cmake/find/grpc.cmake @@ -1,45 +1,65 @@ -option (ENABLE_GRPC "Use gRPC" ${ENABLE_LIBRARIES}) +option(ENABLE_GRPC "Use gRPC" ${ENABLE_LIBRARIES}) -if (NOT ENABLE_GRPC) - if (USE_INTERNAL_GRPC_LIBRARY) - message (${RECONFIGURE_MESSAGE_LEVEL} "Cannot use internal gRPC library with ENABLE_GRPC=OFF") - endif() - return() +if(NOT ENABLE_GRPC) + if(USE_INTERNAL_GRPC_LIBRARY) + message(${RECONFIGURE_MESSAGE_LEVEL} "Cannot use internal gRPC library with ENABLE_GRPC=OFF") + endif() + return() endif() -option (USE_INTERNAL_GRPC_LIBRARY - "Set to FALSE to use system gRPC library instead of bundled. (Experimental. Set to OFF on your own risk)" - ${NOT_UNBUNDLED}) - -if (NOT USE_INTERNAL_GRPC_LIBRARY) - find_package(grpc) - if (NOT GRPC_FOUND) - find_path(GRPC_INCLUDE_DIR grpcpp/grpcpp.h) - find_library(GRPC_LIBRARY grpc++) - endif () - - if (GRPC_INCLUDE_DIR AND GRPC_LIBRARY) - set (USE_GRPC ON) - else() - message (${RECONFIGURE_MESSAGE_LEVEL} "Can't find system gRPC") - endif() +if(NOT USE_PROTOBUF) + message(WARNING "Cannot use gRPC library without protobuf") endif() -if (NOT USE_GRPC) - if (NOT EXISTS "${ClickHouse_SOURCE_DIR}/contrib/grpc/include/grpc++/grpc++.h") - message (WARNING "submodule contrib/grpc is missing. To fix try run: \n git submodule update --init --recursive") - message (${RECONFIGURE_MESSAGE_LEVEL} "Can't find internal gRPC") - set (USE_INTERNAL_GRPC_LIBRARY OFF) - elseif (NOT USE_PROTOBUF) - message (WARNING "gRPC requires protobuf which is disabled") - message (${RECONFIGURE_MESSAGE_LEVEL} "Will not use internal gRPC without protobuf") - set (USE_INTERNAL_GRPC_LIBRARY OFF) - else() - set (GRPC_INCLUDE_DIR "${ClickHouse_SOURCE_DIR}/contrib/grpc/include") - set (GRPC_LIBRARY "libgrpc++") - set (USE_GRPC ON) - set (USE_INTERNAL_GRPC_LIBRARY ON) - endif() +# Normally we use the internal gRPC framework. +# You can set USE_INTERNAL_GRPC_LIBRARY to OFF to force using the external gRPC framework, which should be installed in the system in this case. +# The external gRPC framework can be installed in the system by running +# sudo apt-get install libgrpc++-dev protobuf-compiler-grpc +option(USE_INTERNAL_GRPC_LIBRARY "Set to FALSE to use system gRPC library instead of bundled. (Experimental. Set to OFF on your own risk)" ${NOT_UNBUNDLED}) + +if(NOT EXISTS "${ClickHouse_SOURCE_DIR}/contrib/grpc/CMakeLists.txt") + if(USE_INTERNAL_GRPC_LIBRARY) + message(WARNING "submodule contrib/grpc is missing. to fix try run: \n git submodule update --init --recursive") + message(${RECONFIGURE_MESSAGE_LEVEL} "Can't use internal grpc") + set(USE_INTERNAL_GRPC_LIBRARY 0) + endif() + set(MISSING_INTERNAL_GRPC_LIBRARY 1) endif() -message (STATUS "Using gRPC=${USE_GRPC}: ${GRPC_INCLUDE_DIR} : ${GRPC_LIBRARY}") +if(USE_SSL) + set(gRPC_USE_UNSECURE_LIBRARIES FALSE) +else() + set(gRPC_USE_UNSECURE_LIBRARIES TRUE) +endif() + +if(NOT USE_INTERNAL_GRPC_LIBRARY) + find_package(gRPC) + if(NOT gRPC_INCLUDE_DIRS OR NOT gRPC_LIBRARIES) + message(${RECONFIGURE_MESSAGE_LEVEL} "Can't find system gRPC library") + set(EXTERNAL_GRPC_LIBRARY_FOUND 0) + elseif(NOT _gRPC_CPP_PLUGIN) + message(${RECONFIGURE_MESSAGE_LEVEL} "Can't find system grcp_cpp_plugin") + set(EXTERNAL_GRPC_LIBRARY_FOUND 0) + else() + set(EXTERNAL_GRPC_LIBRARY_FOUND 1) + set(USE_GRPC 1) + endif() +endif() + +if(NOT EXTERNAL_GRPC_LIBRARY_FOUND AND NOT MISSING_INTERNAL_GRPC_LIBRARY) + set(gRPC_INCLUDE_DIRS "${ClickHouse_SOURCE_DIR}/contrib/grpc/include") + if(gRPC_USE_UNSECURE_LIBRARIES) + set(gRPC_LIBRARIES grpc_unsecure grpc++_unsecure) + else() + set(gRPC_LIBRARIES grpc grpc++) + endif() + set(_gRPC_CPP_PLUGIN $) + set(_gRPC_PROTOC_EXECUTABLE $) + + include("${ClickHouse_SOURCE_DIR}/contrib/grpc-cmake/protobuf_generate_grpc.cmake") + + set(USE_INTERNAL_GRPC_LIBRARY 1) + set(USE_GRPC 1) +endif() + +message(STATUS "Using gRPC=${USE_GRPC}: ${gRPC_INCLUDE_DIRS} : ${gRPC_LIBRARIES} : ${_gRPC_CPP_PLUGIN}") diff --git a/contrib/grpc-cmake/CMakeLists.txt b/contrib/grpc-cmake/CMakeLists.txt index a70c155da53..5ab70d83429 100644 --- a/contrib/grpc-cmake/CMakeLists.txt +++ b/contrib/grpc-cmake/CMakeLists.txt @@ -1,1092 +1,49 @@ -cmake_minimum_required(VERSION 3.5.1) +set(_gRPC_SOURCE_DIR "${ClickHouse_SOURCE_DIR}/contrib/grpc") +set(_gRPC_BINARY_DIR "${ClickHouse_BINARY_DIR}/contrib/grpc") - -set(GRPC_SOURCE_DIR ${ClickHouse_SOURCE_DIR}/contrib/grpc) -set(GRPC_INCLUDE_DIR ${GRPC_SOURCE_DIR}/include/) -set(GRPC_BINARY_DIR ${ClickHouse_BINARY_DIR}/contrib/grpc) - - -if(UNIX) - if(${CMAKE_SYSTEM_NAME} MATCHES "Linux") - set(_gRPC_PLATFORM_LINUX ON) - elseif(${CMAKE_SYSTEM_NAME} MATCHES "Darwin") - set(_gRPC_PLATFORM_MAC ON) - else() - set(_gRPC_PLATFORM_POSIX ON) - endif() +if(NOT RE2_INCLUDE_DIR) + message(FATAL_ERROR " grpc: The location of the \"re2\" library is unknown") endif() +set(gRPC_RE2_PROVIDER "clickhouse" CACHE STRING "" FORCE) +set(_gRPC_RE2_INCLUDE_DIR "${RE2_INCLUDE_DIR}") +set(_gRPC_RE2_LIBRARIES "${RE2_LIBRARY}") -if(_gRPC_PLATFORM_MAC) - set(_gRPC_ALLTARGETS_LIBRARIES ${CMAKE_DL_LIBS} m pthread) -elseif(UNIX) - set(_gRPC_ALLTARGETS_LIBRARIES ${CMAKE_DL_LIBS} rt m pthread) +if(NOT ZLIB_INCLUDE_DIRS) + message(FATAL_ERROR " grpc: The location of the \"zlib\" library is unknown") endif() +set(gRPC_ZLIB_PROVIDER "clickhouse" CACHE STRING "" FORCE) +set(_gRPC_ZLIB_INCLUDE_DIR "${ZLIB_INCLUDE_DIRS}") +set(_gRPC_ZLIB_LIBRARIES "${ZLIB_LIBRARIES}") - -# address_sorting.cmake -set(_gRPC_ADDRESS_SORTING_INCLUDE_DIR "${GRPC_SOURCE_DIR}/third_party/address_sorting/include") -set(_gRPC_ADDRESS_SORTING_LIBRARIES address_sorting) - -# cares.cmake -set(CARES_ROOT_DIR ${GRPC_SOURCE_DIR}/third_party/cares/cares) -set(CARES_BINARY_DIR ${GRPC_BINARY_DIR}/third_party/cares/cares) -set(CARES_SHARED ${BUILD_SHARED_LIBS} CACHE BOOL "" FORCE) -if(BUILD_SHARED_LIBS) - set(CARES_STATIC OFF CACHE BOOL "" FORCE) -else() - set(CARES_STATIC ON CACHE BOOL "" FORCE) +if(NOT Protobuf_INCLUDE_DIR OR NOT Protobuf_LIBRARY) + message(FATAL_ERROR " grpc: The location of the \"protobuf\" library is unknown") +elseif (NOT Protobuf_PROTOC_EXECUTABLE) + message(FATAL_ERROR " grpc: The location of the protobuf compiler is unknown") +elseif (NOT Protobuf_PROTOC_LIBRARY) + message(FATAL_ERROR " grpc: The location of the protobuf compiler's library is unknown") endif() -set(_gRPC_CARES_LIBRARIES c-ares) -add_subdirectory(${CARES_ROOT_DIR} ${CARES_BINARY_DIR}) +set(gRPC_PROTOBUF_PROVIDER "clickhouse" CACHE STRING "" FORCE) +set(_gRPC_PROTOBUF_WELLKNOWN_INCLUDE_DIR "${Protobuf_INCLUDE_DIR}") +set(_gRPC_PROTOBUF_LIBRARIES "${Protobuf_LIBRARY}") +set(_gRPC_PROTOBUF_PROTOC "protoc") +set(_gRPC_PROTOBUF_PROTOC_EXECUTABLE "${Protobuf_PROTOC_EXECUTABLE}") +set(_gRPC_PROTOBUF_PROTOC_LIBRARIES "${Protobuf_PROTOC_LIBRARY}") -# upb.cmake -set(UPB_ROOT_DIR ${GRPC_SOURCE_DIR}/third_party/upb) -set(_gRPC_UPB_INCLUDE_DIR "${UPB_ROOT_DIR}") -set(_gRPC_UPB_GRPC_GENERATED_DIR "${GRPC_SOURCE_DIR}/src/core/ext/upb-generated") -set(_gRPC_UPB_LIBRARIES upb) - -# protobuf.cmake -set(_gRPC_PROTOBUF_INCLUDE_DIR ${Protobuf_INCLUDE_DIR}) -set(_gRPC_PROTOBUF_LIBRARIES ${Protobuf_LIBRARY}) -set(_gRPC_PROTOBUF_PROTOC_LIBRARIES ${Protobuf_PROTOC_LIBRARY}) -set(_gRPC_PROTOBUF_PROTOC_EXECUTABLE ${Protobuf_PROTOC_EXECUTABLE}) - -# ssl.cmake +set(gRPC_SSL_PROVIDER "clickhouse" CACHE STRING "" FORCE) set(_gRPC_SSL_INCLUDE_DIR ${OPENSSL_INCLUDE_DIR}) -set(_gRPC_SSL_LIBRARIES ${OPENSSL_SSL_LIBRARY}) +set(_gRPC_SSL_LIBRARIES ${OPENSSL_LIBRARIES}) -# zlib.cmake -set(_gRPC_ZLIB_INCLUDE_DIR ${ZLIB_INCLUDE_DIR}) -set(_gRPC_ZLIB_LIBRARIES ${ZLIB_LIBRARIES}) +# We don't want to build C# extensions. +set(gRPC_BUILD_CSHARP_EXT OFF) +# We don't want to build abseil tests, so we temporarily switch BUILD_TESTING off. +set(_gRPC_ORIG_BUILD_TESTING ${BUILD_TESTING}) +set(BUILD_TESTING OFF) -add_library(address_sorting - ${GRPC_SOURCE_DIR}/third_party/address_sorting/address_sorting.c - ${GRPC_SOURCE_DIR}/third_party/address_sorting/address_sorting_posix.c - ${GRPC_SOURCE_DIR}/third_party/address_sorting/address_sorting_windows.c -) +add_subdirectory("${_gRPC_SOURCE_DIR}" "${_gRPC_BINARY_DIR}") -target_include_directories(address_sorting - PRIVATE ${GRPC_SOURCE_DIR} - PRIVATE ${_gRPC_ADDRESS_SORTING_INCLUDE_DIR} - PRIVATE ${_gRPC_CARES_INCLUDE_DIR} - PRIVATE ${_gRPC_PROTOBUF_INCLUDE_DIR} - PRIVATE ${_gRPC_SSL_INCLUDE_DIR} - PRIVATE ${_gRPC_UPB_GENERATED_DIR} - PRIVATE ${_gRPC_UPB_GRPC_GENERATED_DIR} - PRIVATE ${_gRPC_UPB_INCLUDE_DIR} - PRIVATE ${_gRPC_ZLIB_INCLUDE_DIR} -) -target_link_libraries(address_sorting - ${_gRPC_PROTOBUF_LIBRARIES} - ${_gRPC_ALLTARGETS_LIBRARIES} -) +set(BUILD_TESTING ${_gRPC_ORIG_BUILD_TESTING}) - -add_library(gpr - ${GRPC_SOURCE_DIR}/src/core/lib/gpr/alloc.cc - ${GRPC_SOURCE_DIR}/src/core/lib/gpr/atm.cc - ${GRPC_SOURCE_DIR}/src/core/lib/gpr/cpu_iphone.cc - ${GRPC_SOURCE_DIR}/src/core/lib/gpr/cpu_linux.cc - ${GRPC_SOURCE_DIR}/src/core/lib/gpr/cpu_posix.cc - ${GRPC_SOURCE_DIR}/src/core/lib/gpr/cpu_windows.cc - ${GRPC_SOURCE_DIR}/src/core/lib/gpr/env_linux.cc - ${GRPC_SOURCE_DIR}/src/core/lib/gpr/env_posix.cc - ${GRPC_SOURCE_DIR}/src/core/lib/gpr/env_windows.cc - ${GRPC_SOURCE_DIR}/src/core/lib/gpr/log.cc - ${GRPC_SOURCE_DIR}/src/core/lib/gpr/log_android.cc - ${GRPC_SOURCE_DIR}/src/core/lib/gpr/log_linux.cc - ${GRPC_SOURCE_DIR}/src/core/lib/gpr/log_posix.cc - ${GRPC_SOURCE_DIR}/src/core/lib/gpr/log_windows.cc - ${GRPC_SOURCE_DIR}/src/core/lib/gpr/murmur_hash.cc - ${GRPC_SOURCE_DIR}/src/core/lib/gpr/string.cc - ${GRPC_SOURCE_DIR}/src/core/lib/gpr/string_posix.cc - ${GRPC_SOURCE_DIR}/src/core/lib/gpr/string_util_windows.cc - ${GRPC_SOURCE_DIR}/src/core/lib/gpr/string_windows.cc - ${GRPC_SOURCE_DIR}/src/core/lib/gpr/sync.cc - ${GRPC_SOURCE_DIR}/src/core/lib/gpr/sync_posix.cc - ${GRPC_SOURCE_DIR}/src/core/lib/gpr/sync_windows.cc - ${GRPC_SOURCE_DIR}/src/core/lib/gpr/time.cc - ${GRPC_SOURCE_DIR}/src/core/lib/gpr/time_posix.cc - ${GRPC_SOURCE_DIR}/src/core/lib/gpr/time_precise.cc - ${GRPC_SOURCE_DIR}/src/core/lib/gpr/time_windows.cc - ${GRPC_SOURCE_DIR}/src/core/lib/gpr/tls_pthread.cc - ${GRPC_SOURCE_DIR}/src/core/lib/gpr/tmpfile_msys.cc - ${GRPC_SOURCE_DIR}/src/core/lib/gpr/tmpfile_posix.cc - ${GRPC_SOURCE_DIR}/src/core/lib/gpr/tmpfile_windows.cc - ${GRPC_SOURCE_DIR}/src/core/lib/gpr/wrap_memcpy.cc - ${GRPC_SOURCE_DIR}/src/core/lib/gprpp/arena.cc - ${GRPC_SOURCE_DIR}/src/core/lib/gprpp/fork.cc - ${GRPC_SOURCE_DIR}/src/core/lib/gprpp/global_config_env.cc - ${GRPC_SOURCE_DIR}/src/core/lib/gprpp/host_port.cc - ${GRPC_SOURCE_DIR}/src/core/lib/gprpp/mpscq.cc - ${GRPC_SOURCE_DIR}/src/core/lib/gprpp/thd_posix.cc - ${GRPC_SOURCE_DIR}/src/core/lib/gprpp/thd_windows.cc - ${GRPC_SOURCE_DIR}/src/core/lib/profiling/basic_timers.cc - ${GRPC_SOURCE_DIR}/src/core/lib/profiling/stap_timers.cc -) - -target_include_directories(gpr - PUBLIC ${GRPC_INCLUDE_DIR} - PRIVATE ${GRPC_SOURCE_DIR} - PRIVATE ${_gRPC_ADDRESS_SORTING_INCLUDE_DIR} - PRIVATE ${_gRPC_CARES_INCLUDE_DIR} - PRIVATE ${_gRPC_PROTOBUF_INCLUDE_DIR} - PRIVATE ${_gRPC_SSL_INCLUDE_DIR} - PRIVATE ${_gRPC_UPB_GENERATED_DIR} - PRIVATE ${_gRPC_UPB_GRPC_GENERATED_DIR} - PRIVATE ${_gRPC_UPB_INCLUDE_DIR} - PRIVATE ${_gRPC_ZLIB_INCLUDE_DIR} -) - -target_link_libraries(gpr - ${_gRPC_ALLTARGETS_LIBRARIES} - ${_gRPC_PROTOBUF_LIBRARIES} -) - -add_library(grpc - ${GRPC_SOURCE_DIR}/src/core/lib/surface/init.cc - ${GRPC_SOURCE_DIR}/src/core/lib/avl/avl.cc - ${GRPC_SOURCE_DIR}/src/core/lib/backoff/backoff.cc - ${GRPC_SOURCE_DIR}/src/core/lib/channel/channel_args.cc - ${GRPC_SOURCE_DIR}/src/core/lib/channel/channel_stack.cc - ${GRPC_SOURCE_DIR}/src/core/lib/channel/channel_stack_builder.cc - ${GRPC_SOURCE_DIR}/src/core/lib/channel/channel_trace.cc - ${GRPC_SOURCE_DIR}/src/core/lib/channel/channelz.cc - ${GRPC_SOURCE_DIR}/src/core/lib/channel/channelz_registry.cc - ${GRPC_SOURCE_DIR}/src/core/lib/channel/connected_channel.cc - ${GRPC_SOURCE_DIR}/src/core/lib/channel/handshaker.cc - ${GRPC_SOURCE_DIR}/src/core/lib/channel/handshaker_registry.cc - ${GRPC_SOURCE_DIR}/src/core/lib/channel/status_util.cc - ${GRPC_SOURCE_DIR}/src/core/lib/compression/compression.cc - ${GRPC_SOURCE_DIR}/src/core/lib/compression/compression_args.cc - ${GRPC_SOURCE_DIR}/src/core/lib/compression/compression_internal.cc - ${GRPC_SOURCE_DIR}/src/core/lib/compression/message_compress.cc - ${GRPC_SOURCE_DIR}/src/core/lib/compression/stream_compression.cc - ${GRPC_SOURCE_DIR}/src/core/lib/compression/stream_compression_gzip.cc - ${GRPC_SOURCE_DIR}/src/core/lib/compression/stream_compression_identity.cc - ${GRPC_SOURCE_DIR}/src/core/lib/debug/stats.cc - ${GRPC_SOURCE_DIR}/src/core/lib/debug/stats_data.cc - ${GRPC_SOURCE_DIR}/src/core/lib/http/format_request.cc - ${GRPC_SOURCE_DIR}/src/core/lib/http/httpcli.cc - ${GRPC_SOURCE_DIR}/src/core/lib/http/parser.cc - ${GRPC_SOURCE_DIR}/src/core/lib/iomgr/buffer_list.cc - ${GRPC_SOURCE_DIR}/src/core/lib/iomgr/call_combiner.cc - ${GRPC_SOURCE_DIR}/src/core/lib/iomgr/cfstream_handle.cc - ${GRPC_SOURCE_DIR}/src/core/lib/iomgr/combiner.cc - ${GRPC_SOURCE_DIR}/src/core/lib/iomgr/endpoint.cc - ${GRPC_SOURCE_DIR}/src/core/lib/iomgr/endpoint_cfstream.cc - ${GRPC_SOURCE_DIR}/src/core/lib/iomgr/endpoint_pair_posix.cc - ${GRPC_SOURCE_DIR}/src/core/lib/iomgr/endpoint_pair_uv.cc - ${GRPC_SOURCE_DIR}/src/core/lib/iomgr/endpoint_pair_windows.cc - ${GRPC_SOURCE_DIR}/src/core/lib/iomgr/error.cc - ${GRPC_SOURCE_DIR}/src/core/lib/iomgr/error_cfstream.cc - ${GRPC_SOURCE_DIR}/src/core/lib/iomgr/ev_epoll1_linux.cc - ${GRPC_SOURCE_DIR}/src/core/lib/iomgr/ev_epollex_linux.cc - ${GRPC_SOURCE_DIR}/src/core/lib/iomgr/ev_poll_posix.cc - ${GRPC_SOURCE_DIR}/src/core/lib/iomgr/ev_posix.cc - ${GRPC_SOURCE_DIR}/src/core/lib/iomgr/ev_windows.cc - ${GRPC_SOURCE_DIR}/src/core/lib/iomgr/exec_ctx.cc - ${GRPC_SOURCE_DIR}/src/core/lib/iomgr/executor.cc - ${GRPC_SOURCE_DIR}/src/core/lib/iomgr/executor/mpmcqueue.cc - ${GRPC_SOURCE_DIR}/src/core/lib/iomgr/executor/threadpool.cc - ${GRPC_SOURCE_DIR}/src/core/lib/iomgr/fork_posix.cc - ${GRPC_SOURCE_DIR}/src/core/lib/iomgr/fork_windows.cc - ${GRPC_SOURCE_DIR}/src/core/lib/iomgr/gethostname_fallback.cc - ${GRPC_SOURCE_DIR}/src/core/lib/iomgr/gethostname_host_name_max.cc - ${GRPC_SOURCE_DIR}/src/core/lib/iomgr/gethostname_sysconf.cc - ${GRPC_SOURCE_DIR}/src/core/lib/iomgr/grpc_if_nametoindex_posix.cc - ${GRPC_SOURCE_DIR}/src/core/lib/iomgr/grpc_if_nametoindex_unsupported.cc - ${GRPC_SOURCE_DIR}/src/core/lib/iomgr/internal_errqueue.cc - ${GRPC_SOURCE_DIR}/src/core/lib/iomgr/iocp_windows.cc - ${GRPC_SOURCE_DIR}/src/core/lib/iomgr/iomgr.cc - ${GRPC_SOURCE_DIR}/src/core/lib/iomgr/iomgr_custom.cc - ${GRPC_SOURCE_DIR}/src/core/lib/iomgr/iomgr_internal.cc - ${GRPC_SOURCE_DIR}/src/core/lib/iomgr/iomgr_posix.cc - ${GRPC_SOURCE_DIR}/src/core/lib/iomgr/iomgr_posix_cfstream.cc - ${GRPC_SOURCE_DIR}/src/core/lib/iomgr/iomgr_uv.cc - ${GRPC_SOURCE_DIR}/src/core/lib/iomgr/iomgr_windows.cc - ${GRPC_SOURCE_DIR}/src/core/lib/iomgr/is_epollexclusive_available.cc - ${GRPC_SOURCE_DIR}/src/core/lib/iomgr/load_file.cc - ${GRPC_SOURCE_DIR}/src/core/lib/iomgr/lockfree_event.cc - ${GRPC_SOURCE_DIR}/src/core/lib/iomgr/polling_entity.cc - ${GRPC_SOURCE_DIR}/src/core/lib/iomgr/pollset.cc - ${GRPC_SOURCE_DIR}/src/core/lib/iomgr/pollset_custom.cc - ${GRPC_SOURCE_DIR}/src/core/lib/iomgr/pollset_set.cc - ${GRPC_SOURCE_DIR}/src/core/lib/iomgr/pollset_set_custom.cc - ${GRPC_SOURCE_DIR}/src/core/lib/iomgr/pollset_set_windows.cc - ${GRPC_SOURCE_DIR}/src/core/lib/iomgr/pollset_uv.cc - ${GRPC_SOURCE_DIR}/src/core/lib/iomgr/pollset_windows.cc - ${GRPC_SOURCE_DIR}/src/core/lib/iomgr/resolve_address.cc - ${GRPC_SOURCE_DIR}/src/core/lib/iomgr/resolve_address_custom.cc - ${GRPC_SOURCE_DIR}/src/core/lib/iomgr/resolve_address_posix.cc - ${GRPC_SOURCE_DIR}/src/core/lib/iomgr/resolve_address_windows.cc - ${GRPC_SOURCE_DIR}/src/core/lib/iomgr/resource_quota.cc - ${GRPC_SOURCE_DIR}/src/core/lib/iomgr/sockaddr_utils.cc - ${GRPC_SOURCE_DIR}/src/core/lib/iomgr/socket_factory_posix.cc - ${GRPC_SOURCE_DIR}/src/core/lib/iomgr/socket_mutator.cc - ${GRPC_SOURCE_DIR}/src/core/lib/iomgr/socket_utils_common_posix.cc - ${GRPC_SOURCE_DIR}/src/core/lib/iomgr/socket_utils_linux.cc - ${GRPC_SOURCE_DIR}/src/core/lib/iomgr/socket_utils_posix.cc - ${GRPC_SOURCE_DIR}/src/core/lib/iomgr/socket_utils_uv.cc - ${GRPC_SOURCE_DIR}/src/core/lib/iomgr/socket_utils_windows.cc - ${GRPC_SOURCE_DIR}/src/core/lib/iomgr/socket_windows.cc - ${GRPC_SOURCE_DIR}/src/core/lib/iomgr/tcp_client.cc - ${GRPC_SOURCE_DIR}/src/core/lib/iomgr/tcp_client_cfstream.cc - ${GRPC_SOURCE_DIR}/src/core/lib/iomgr/tcp_client_custom.cc - ${GRPC_SOURCE_DIR}/src/core/lib/iomgr/tcp_client_posix.cc - ${GRPC_SOURCE_DIR}/src/core/lib/iomgr/tcp_client_windows.cc - ${GRPC_SOURCE_DIR}/src/core/lib/iomgr/tcp_custom.cc - ${GRPC_SOURCE_DIR}/src/core/lib/iomgr/tcp_posix.cc - ${GRPC_SOURCE_DIR}/src/core/lib/iomgr/tcp_server.cc - ${GRPC_SOURCE_DIR}/src/core/lib/iomgr/tcp_server_custom.cc - ${GRPC_SOURCE_DIR}/src/core/lib/iomgr/tcp_server_posix.cc - ${GRPC_SOURCE_DIR}/src/core/lib/iomgr/tcp_server_utils_posix_common.cc - ${GRPC_SOURCE_DIR}/src/core/lib/iomgr/tcp_server_utils_posix_ifaddrs.cc - ${GRPC_SOURCE_DIR}/src/core/lib/iomgr/tcp_server_utils_posix_noifaddrs.cc - ${GRPC_SOURCE_DIR}/src/core/lib/iomgr/tcp_server_windows.cc - ${GRPC_SOURCE_DIR}/src/core/lib/iomgr/tcp_uv.cc - ${GRPC_SOURCE_DIR}/src/core/lib/iomgr/tcp_windows.cc - ${GRPC_SOURCE_DIR}/src/core/lib/iomgr/time_averaged_stats.cc - ${GRPC_SOURCE_DIR}/src/core/lib/iomgr/timer.cc - ${GRPC_SOURCE_DIR}/src/core/lib/iomgr/timer_custom.cc - ${GRPC_SOURCE_DIR}/src/core/lib/iomgr/timer_generic.cc - ${GRPC_SOURCE_DIR}/src/core/lib/iomgr/timer_heap.cc - ${GRPC_SOURCE_DIR}/src/core/lib/iomgr/timer_manager.cc - ${GRPC_SOURCE_DIR}/src/core/lib/iomgr/timer_uv.cc - ${GRPC_SOURCE_DIR}/src/core/lib/iomgr/udp_server.cc - ${GRPC_SOURCE_DIR}/src/core/lib/iomgr/unix_sockets_posix.cc - ${GRPC_SOURCE_DIR}/src/core/lib/iomgr/unix_sockets_posix_noop.cc - ${GRPC_SOURCE_DIR}/src/core/lib/iomgr/wakeup_fd_eventfd.cc - ${GRPC_SOURCE_DIR}/src/core/lib/iomgr/wakeup_fd_nospecial.cc - ${GRPC_SOURCE_DIR}/src/core/lib/iomgr/wakeup_fd_pipe.cc - ${GRPC_SOURCE_DIR}/src/core/lib/iomgr/wakeup_fd_posix.cc - ${GRPC_SOURCE_DIR}/src/core/lib/json/json.cc - ${GRPC_SOURCE_DIR}/src/core/lib/json/json_reader.cc - ${GRPC_SOURCE_DIR}/src/core/lib/json/json_string.cc - ${GRPC_SOURCE_DIR}/src/core/lib/json/json_writer.cc - ${GRPC_SOURCE_DIR}/src/core/lib/slice/b64.cc - ${GRPC_SOURCE_DIR}/src/core/lib/slice/percent_encoding.cc - ${GRPC_SOURCE_DIR}/src/core/lib/slice/slice.cc - ${GRPC_SOURCE_DIR}/src/core/lib/slice/slice_buffer.cc - ${GRPC_SOURCE_DIR}/src/core/lib/slice/slice_intern.cc - ${GRPC_SOURCE_DIR}/src/core/lib/slice/slice_string_helpers.cc - ${GRPC_SOURCE_DIR}/src/core/lib/surface/api_trace.cc - ${GRPC_SOURCE_DIR}/src/core/lib/surface/byte_buffer.cc - ${GRPC_SOURCE_DIR}/src/core/lib/surface/byte_buffer_reader.cc - ${GRPC_SOURCE_DIR}/src/core/lib/surface/call.cc - ${GRPC_SOURCE_DIR}/src/core/lib/surface/call_details.cc - ${GRPC_SOURCE_DIR}/src/core/lib/surface/call_log_batch.cc - ${GRPC_SOURCE_DIR}/src/core/lib/surface/channel.cc - ${GRPC_SOURCE_DIR}/src/core/lib/surface/channel_init.cc - ${GRPC_SOURCE_DIR}/src/core/lib/surface/channel_ping.cc - ${GRPC_SOURCE_DIR}/src/core/lib/surface/channel_stack_type.cc - ${GRPC_SOURCE_DIR}/src/core/lib/surface/completion_queue.cc - ${GRPC_SOURCE_DIR}/src/core/lib/surface/completion_queue_factory.cc - ${GRPC_SOURCE_DIR}/src/core/lib/surface/event_string.cc - ${GRPC_SOURCE_DIR}/src/core/lib/surface/lame_client.cc - ${GRPC_SOURCE_DIR}/src/core/lib/surface/metadata_array.cc - ${GRPC_SOURCE_DIR}/src/core/lib/surface/server.cc - ${GRPC_SOURCE_DIR}/src/core/lib/surface/validate_metadata.cc - ${GRPC_SOURCE_DIR}/src/core/lib/surface/version.cc - ${GRPC_SOURCE_DIR}/src/core/lib/transport/bdp_estimator.cc - ${GRPC_SOURCE_DIR}/src/core/lib/transport/byte_stream.cc - ${GRPC_SOURCE_DIR}/src/core/lib/transport/connectivity_state.cc - ${GRPC_SOURCE_DIR}/src/core/lib/transport/error_utils.cc - ${GRPC_SOURCE_DIR}/src/core/lib/transport/metadata.cc - ${GRPC_SOURCE_DIR}/src/core/lib/transport/metadata_batch.cc - ${GRPC_SOURCE_DIR}/src/core/lib/transport/pid_controller.cc - ${GRPC_SOURCE_DIR}/src/core/lib/transport/static_metadata.cc - ${GRPC_SOURCE_DIR}/src/core/lib/transport/status_conversion.cc - ${GRPC_SOURCE_DIR}/src/core/lib/transport/status_metadata.cc - ${GRPC_SOURCE_DIR}/src/core/lib/transport/timeout_encoding.cc - ${GRPC_SOURCE_DIR}/src/core/lib/transport/transport.cc - ${GRPC_SOURCE_DIR}/src/core/lib/transport/transport_op_string.cc - ${GRPC_SOURCE_DIR}/src/core/lib/uri/uri_parser.cc - ${GRPC_SOURCE_DIR}/src/core/lib/debug/trace.cc - ${GRPC_SOURCE_DIR}/src/core/ext/transport/chttp2/server/secure/server_secure_chttp2.cc - ${GRPC_SOURCE_DIR}/src/core/ext/transport/chttp2/transport/bin_decoder.cc - ${GRPC_SOURCE_DIR}/src/core/ext/transport/chttp2/transport/bin_encoder.cc - ${GRPC_SOURCE_DIR}/src/core/ext/transport/chttp2/transport/chttp2_plugin.cc - ${GRPC_SOURCE_DIR}/src/core/ext/transport/chttp2/transport/chttp2_transport.cc - ${GRPC_SOURCE_DIR}/src/core/ext/transport/chttp2/transport/context_list.cc - ${GRPC_SOURCE_DIR}/src/core/ext/transport/chttp2/transport/flow_control.cc - ${GRPC_SOURCE_DIR}/src/core/ext/transport/chttp2/transport/frame_data.cc - ${GRPC_SOURCE_DIR}/src/core/ext/transport/chttp2/transport/frame_goaway.cc - ${GRPC_SOURCE_DIR}/src/core/ext/transport/chttp2/transport/frame_ping.cc - ${GRPC_SOURCE_DIR}/src/core/ext/transport/chttp2/transport/frame_rst_stream.cc - ${GRPC_SOURCE_DIR}/src/core/ext/transport/chttp2/transport/frame_settings.cc - ${GRPC_SOURCE_DIR}/src/core/ext/transport/chttp2/transport/frame_window_update.cc - ${GRPC_SOURCE_DIR}/src/core/ext/transport/chttp2/transport/hpack_encoder.cc - ${GRPC_SOURCE_DIR}/src/core/ext/transport/chttp2/transport/hpack_parser.cc - ${GRPC_SOURCE_DIR}/src/core/ext/transport/chttp2/transport/hpack_table.cc - ${GRPC_SOURCE_DIR}/src/core/ext/transport/chttp2/transport/http2_settings.cc - ${GRPC_SOURCE_DIR}/src/core/ext/transport/chttp2/transport/huffsyms.cc - ${GRPC_SOURCE_DIR}/src/core/ext/transport/chttp2/transport/incoming_metadata.cc - ${GRPC_SOURCE_DIR}/src/core/ext/transport/chttp2/transport/parsing.cc - ${GRPC_SOURCE_DIR}/src/core/ext/transport/chttp2/transport/stream_lists.cc - ${GRPC_SOURCE_DIR}/src/core/ext/transport/chttp2/transport/stream_map.cc - ${GRPC_SOURCE_DIR}/src/core/ext/transport/chttp2/transport/varint.cc - ${GRPC_SOURCE_DIR}/src/core/ext/transport/chttp2/transport/writing.cc - ${GRPC_SOURCE_DIR}/src/core/ext/transport/chttp2/alpn/alpn.cc - ${GRPC_SOURCE_DIR}/src/core/ext/filters/http/client/http_client_filter.cc - ${GRPC_SOURCE_DIR}/src/core/ext/filters/http/http_filters_plugin.cc - ${GRPC_SOURCE_DIR}/src/core/ext/filters/http/message_compress/message_compress_filter.cc - ${GRPC_SOURCE_DIR}/src/core/ext/filters/http/server/http_server_filter.cc - ${GRPC_SOURCE_DIR}/src/core/lib/http/httpcli_security_connector.cc - ${GRPC_SOURCE_DIR}/src/core/lib/security/context/security_context.cc - ${GRPC_SOURCE_DIR}/src/core/lib/security/credentials/alts/alts_credentials.cc - ${GRPC_SOURCE_DIR}/src/core/lib/security/credentials/composite/composite_credentials.cc - ${GRPC_SOURCE_DIR}/src/core/lib/security/credentials/credentials.cc - ${GRPC_SOURCE_DIR}/src/core/lib/security/credentials/credentials_metadata.cc - ${GRPC_SOURCE_DIR}/src/core/lib/security/credentials/fake/fake_credentials.cc - ${GRPC_SOURCE_DIR}/src/core/lib/security/credentials/google_default/credentials_generic.cc - ${GRPC_SOURCE_DIR}/src/core/lib/security/credentials/google_default/google_default_credentials.cc - ${GRPC_SOURCE_DIR}/src/core/lib/security/credentials/iam/iam_credentials.cc - ${GRPC_SOURCE_DIR}/src/core/lib/security/credentials/jwt/json_token.cc - ${GRPC_SOURCE_DIR}/src/core/lib/security/credentials/jwt/jwt_credentials.cc - ${GRPC_SOURCE_DIR}/src/core/lib/security/credentials/jwt/jwt_verifier.cc - ${GRPC_SOURCE_DIR}/src/core/lib/security/credentials/local/local_credentials.cc - ${GRPC_SOURCE_DIR}/src/core/lib/security/credentials/oauth2/oauth2_credentials.cc - ${GRPC_SOURCE_DIR}/src/core/lib/security/credentials/plugin/plugin_credentials.cc - ${GRPC_SOURCE_DIR}/src/core/lib/security/credentials/ssl/ssl_credentials.cc - ${GRPC_SOURCE_DIR}/src/core/lib/security/credentials/tls/grpc_tls_credentials_options.cc - ${GRPC_SOURCE_DIR}/src/core/lib/security/credentials/tls/spiffe_credentials.cc - ${GRPC_SOURCE_DIR}/src/core/lib/security/security_connector/alts/alts_security_connector.cc - ${GRPC_SOURCE_DIR}/src/core/lib/security/security_connector/fake/fake_security_connector.cc - ${GRPC_SOURCE_DIR}/src/core/lib/security/security_connector/load_system_roots_fallback.cc - ${GRPC_SOURCE_DIR}/src/core/lib/security/security_connector/load_system_roots_linux.cc - ${GRPC_SOURCE_DIR}/src/core/lib/security/security_connector/local/local_security_connector.cc - ${GRPC_SOURCE_DIR}/src/core/lib/security/security_connector/security_connector.cc - ${GRPC_SOURCE_DIR}/src/core/lib/security/security_connector/ssl/ssl_security_connector.cc - ${GRPC_SOURCE_DIR}/src/core/lib/security/security_connector/ssl_utils.cc - ${GRPC_SOURCE_DIR}/src/core/lib/security/security_connector/ssl_utils_config.cc - ${GRPC_SOURCE_DIR}/src/core/lib/security/security_connector/tls/spiffe_security_connector.cc - ${GRPC_SOURCE_DIR}/src/core/lib/security/transport/client_auth_filter.cc - ${GRPC_SOURCE_DIR}/src/core/lib/security/transport/secure_endpoint.cc - ${GRPC_SOURCE_DIR}/src/core/lib/security/transport/security_handshaker.cc - ${GRPC_SOURCE_DIR}/src/core/lib/security/transport/server_auth_filter.cc - ${GRPC_SOURCE_DIR}/src/core/lib/security/transport/target_authority_table.cc - ${GRPC_SOURCE_DIR}/src/core/lib/security/transport/tsi_error.cc - ${GRPC_SOURCE_DIR}/src/core/lib/security/util/json_util.cc - ${GRPC_SOURCE_DIR}/src/core/lib/surface/init_secure.cc - ${GRPC_SOURCE_DIR}/src/core/tsi/alts/crypt/aes_gcm.cc - ${GRPC_SOURCE_DIR}/src/core/tsi/alts/crypt/gsec.cc - ${GRPC_SOURCE_DIR}/src/core/tsi/alts/frame_protector/alts_counter.cc - ${GRPC_SOURCE_DIR}/src/core/tsi/alts/frame_protector/alts_crypter.cc - ${GRPC_SOURCE_DIR}/src/core/tsi/alts/frame_protector/alts_frame_protector.cc - ${GRPC_SOURCE_DIR}/src/core/tsi/alts/frame_protector/alts_record_protocol_crypter_common.cc - ${GRPC_SOURCE_DIR}/src/core/tsi/alts/frame_protector/alts_seal_privacy_integrity_crypter.cc - ${GRPC_SOURCE_DIR}/src/core/tsi/alts/frame_protector/alts_unseal_privacy_integrity_crypter.cc - ${GRPC_SOURCE_DIR}/src/core/tsi/alts/frame_protector/frame_handler.cc - ${GRPC_SOURCE_DIR}/src/core/tsi/alts/handshaker/alts_handshaker_client.cc - ${GRPC_SOURCE_DIR}/src/core/tsi/alts/handshaker/alts_shared_resource.cc - ${GRPC_SOURCE_DIR}/src/core/tsi/alts/handshaker/alts_tsi_handshaker.cc - ${GRPC_SOURCE_DIR}/src/core/tsi/alts/zero_copy_frame_protector/alts_grpc_integrity_only_record_protocol.cc - ${GRPC_SOURCE_DIR}/src/core/tsi/alts/zero_copy_frame_protector/alts_grpc_privacy_integrity_record_protocol.cc - ${GRPC_SOURCE_DIR}/src/core/tsi/alts/zero_copy_frame_protector/alts_grpc_record_protocol_common.cc - ${GRPC_SOURCE_DIR}/src/core/tsi/alts/zero_copy_frame_protector/alts_iovec_record_protocol.cc - ${GRPC_SOURCE_DIR}/src/core/tsi/alts/zero_copy_frame_protector/alts_zero_copy_grpc_protector.cc - ${GRPC_SOURCE_DIR}/src/core/lib/security/credentials/alts/check_gcp_environment.cc - ${GRPC_SOURCE_DIR}/src/core/lib/security/credentials/alts/check_gcp_environment_linux.cc - ${GRPC_SOURCE_DIR}/src/core/lib/security/credentials/alts/check_gcp_environment_no_op.cc - ${GRPC_SOURCE_DIR}/src/core/lib/security/credentials/alts/check_gcp_environment_windows.cc - ${GRPC_SOURCE_DIR}/src/core/lib/security/credentials/alts/grpc_alts_credentials_client_options.cc - ${GRPC_SOURCE_DIR}/src/core/lib/security/credentials/alts/grpc_alts_credentials_options.cc - ${GRPC_SOURCE_DIR}/src/core/lib/security/credentials/alts/grpc_alts_credentials_server_options.cc - ${GRPC_SOURCE_DIR}/src/core/tsi/alts/handshaker/alts_tsi_utils.cc - ${GRPC_SOURCE_DIR}/src/core/tsi/alts/handshaker/transport_security_common_api.cc - ${GRPC_SOURCE_DIR}/src/core/ext/upb-generated/src/proto/grpc/gcp/altscontext.upb.c - ${GRPC_SOURCE_DIR}/src/core/ext/upb-generated/src/proto/grpc/gcp/handshaker.upb.c - ${GRPC_SOURCE_DIR}/src/core/ext/upb-generated/src/proto/grpc/gcp/transport_security_common.upb.c - ${GRPC_SOURCE_DIR}/third_party/upb/upb/decode.c - ${GRPC_SOURCE_DIR}/third_party/upb/upb/encode.c - ${GRPC_SOURCE_DIR}/third_party/upb/upb/msg.c - ${GRPC_SOURCE_DIR}/third_party/upb/upb/port.c - ${GRPC_SOURCE_DIR}/third_party/upb/upb/table.c - ${GRPC_SOURCE_DIR}/third_party/upb/upb/upb.c - ${GRPC_SOURCE_DIR}/src/core/tsi/transport_security.cc - ${GRPC_SOURCE_DIR}/src/core/ext/transport/chttp2/client/insecure/channel_create.cc - ${GRPC_SOURCE_DIR}/src/core/ext/transport/chttp2/client/insecure/channel_create_posix.cc - ${GRPC_SOURCE_DIR}/src/core/ext/transport/chttp2/client/authority.cc - ${GRPC_SOURCE_DIR}/src/core/ext/transport/chttp2/client/chttp2_connector.cc - ${GRPC_SOURCE_DIR}/src/core/ext/filters/client_channel/backend_metric.cc - ${GRPC_SOURCE_DIR}/src/core/ext/filters/client_channel/backup_poller.cc - ${GRPC_SOURCE_DIR}/src/core/ext/filters/client_channel/channel_connectivity.cc - ${GRPC_SOURCE_DIR}/src/core/ext/filters/client_channel/client_channel.cc - ${GRPC_SOURCE_DIR}/src/core/ext/filters/client_channel/client_channel_channelz.cc - ${GRPC_SOURCE_DIR}/src/core/ext/filters/client_channel/client_channel_factory.cc - ${GRPC_SOURCE_DIR}/src/core/ext/filters/client_channel/client_channel_plugin.cc - ${GRPC_SOURCE_DIR}/src/core/ext/filters/client_channel/connector.cc - ${GRPC_SOURCE_DIR}/src/core/ext/filters/client_channel/global_subchannel_pool.cc - ${GRPC_SOURCE_DIR}/src/core/ext/filters/client_channel/health/health_check_client.cc - ${GRPC_SOURCE_DIR}/src/core/ext/filters/client_channel/http_connect_handshaker.cc - ${GRPC_SOURCE_DIR}/src/core/ext/filters/client_channel/http_proxy.cc - ${GRPC_SOURCE_DIR}/src/core/ext/filters/client_channel/lb_policy.cc - ${GRPC_SOURCE_DIR}/src/core/ext/filters/client_channel/lb_policy_registry.cc - ${GRPC_SOURCE_DIR}/src/core/ext/filters/client_channel/local_subchannel_pool.cc - ${GRPC_SOURCE_DIR}/src/core/ext/filters/client_channel/parse_address.cc - ${GRPC_SOURCE_DIR}/src/core/ext/filters/client_channel/proxy_mapper.cc - ${GRPC_SOURCE_DIR}/src/core/ext/filters/client_channel/proxy_mapper_registry.cc - ${GRPC_SOURCE_DIR}/src/core/ext/filters/client_channel/resolver.cc - ${GRPC_SOURCE_DIR}/src/core/ext/filters/client_channel/resolver_registry.cc - ${GRPC_SOURCE_DIR}/src/core/ext/filters/client_channel/resolver_result_parsing.cc - ${GRPC_SOURCE_DIR}/src/core/ext/filters/client_channel/resolving_lb_policy.cc - ${GRPC_SOURCE_DIR}/src/core/ext/filters/client_channel/retry_throttle.cc - ${GRPC_SOURCE_DIR}/src/core/ext/filters/client_channel/server_address.cc - ${GRPC_SOURCE_DIR}/src/core/ext/filters/client_channel/service_config.cc - ${GRPC_SOURCE_DIR}/src/core/ext/filters/client_channel/subchannel.cc - ${GRPC_SOURCE_DIR}/src/core/ext/filters/client_channel/subchannel_pool_interface.cc - ${GRPC_SOURCE_DIR}/src/core/ext/filters/deadline/deadline_filter.cc - ${GRPC_SOURCE_DIR}/src/core/ext/upb-generated/src/proto/grpc/health/v1/health.upb.c - ${GRPC_SOURCE_DIR}/src/core/ext/upb-generated/udpa/data/orca/v1/orca_load_report.upb.c - ${GRPC_SOURCE_DIR}/src/core/ext/upb-generated/gogoproto/gogo.upb.c - ${GRPC_SOURCE_DIR}/src/core/ext/upb-generated/validate/validate.upb.c - ${GRPC_SOURCE_DIR}/src/core/ext/upb-generated/google/api/annotations.upb.c - ${GRPC_SOURCE_DIR}/src/core/ext/upb-generated/google/api/http.upb.c - ${GRPC_SOURCE_DIR}/src/core/ext/upb-generated/google/protobuf/any.upb.c - ${GRPC_SOURCE_DIR}/src/core/ext/upb-generated/google/protobuf/descriptor.upb.c - ${GRPC_SOURCE_DIR}/src/core/ext/upb-generated/google/protobuf/duration.upb.c - ${GRPC_SOURCE_DIR}/src/core/ext/upb-generated/google/protobuf/empty.upb.c - ${GRPC_SOURCE_DIR}/src/core/ext/upb-generated/google/protobuf/struct.upb.c - ${GRPC_SOURCE_DIR}/src/core/ext/upb-generated/google/protobuf/timestamp.upb.c - ${GRPC_SOURCE_DIR}/src/core/ext/upb-generated/google/protobuf/wrappers.upb.c - ${GRPC_SOURCE_DIR}/src/core/ext/upb-generated/google/rpc/status.upb.c - ${GRPC_SOURCE_DIR}/src/core/tsi/fake_transport_security.cc - ${GRPC_SOURCE_DIR}/src/core/tsi/local_transport_security.cc - ${GRPC_SOURCE_DIR}/src/core/tsi/ssl/session_cache/ssl_session_boringssl.cc - ${GRPC_SOURCE_DIR}/src/core/tsi/ssl/session_cache/ssl_session_cache.cc - ${GRPC_SOURCE_DIR}/src/core/tsi/ssl/session_cache/ssl_session_openssl.cc - ${GRPC_SOURCE_DIR}/src/core/tsi/ssl_transport_security.cc - ${GRPC_SOURCE_DIR}/src/core/tsi/transport_security_grpc.cc - ${GRPC_SOURCE_DIR}/src/core/ext/transport/chttp2/server/chttp2_server.cc - ${GRPC_SOURCE_DIR}/src/core/ext/transport/chttp2/client/secure/secure_channel_create.cc - ${GRPC_SOURCE_DIR}/src/core/ext/transport/chttp2/server/insecure/server_chttp2.cc - ${GRPC_SOURCE_DIR}/src/core/ext/transport/chttp2/server/insecure/server_chttp2_posix.cc - ${GRPC_SOURCE_DIR}/src/core/ext/transport/inproc/inproc_plugin.cc - ${GRPC_SOURCE_DIR}/src/core/ext/transport/inproc/inproc_transport.cc - ${GRPC_SOURCE_DIR}/src/core/ext/filters/client_channel/lb_policy/grpclb/client_load_reporting_filter.cc - ${GRPC_SOURCE_DIR}/src/core/ext/filters/client_channel/lb_policy/grpclb/grpclb.cc - ${GRPC_SOURCE_DIR}/src/core/ext/filters/client_channel/lb_policy/grpclb/grpclb_channel_secure.cc - ${GRPC_SOURCE_DIR}/src/core/ext/filters/client_channel/lb_policy/grpclb/grpclb_client_stats.cc - ${GRPC_SOURCE_DIR}/src/core/ext/filters/client_channel/lb_policy/grpclb/load_balancer_api.cc - ${GRPC_SOURCE_DIR}/src/core/ext/upb-generated/src/proto/grpc/lb/v1/load_balancer.upb.c - ${GRPC_SOURCE_DIR}/src/core/ext/filters/client_channel/resolver/fake/fake_resolver.cc - ${GRPC_SOURCE_DIR}/src/core/ext/filters/client_channel/lb_policy/xds/xds.cc - ${GRPC_SOURCE_DIR}/src/core/ext/filters/client_channel/xds/xds_api.cc - ${GRPC_SOURCE_DIR}/src/core/ext/filters/client_channel/xds/xds_bootstrap.cc - ${GRPC_SOURCE_DIR}/src/core/ext/filters/client_channel/xds/xds_channel_secure.cc - ${GRPC_SOURCE_DIR}/src/core/ext/filters/client_channel/xds/xds_client.cc - ${GRPC_SOURCE_DIR}/src/core/ext/filters/client_channel/xds/xds_client_stats.cc - ${GRPC_SOURCE_DIR}/src/core/ext/upb-generated/envoy/api/v2/auth/cert.upb.c - ${GRPC_SOURCE_DIR}/src/core/ext/upb-generated/envoy/api/v2/cds.upb.c - ${GRPC_SOURCE_DIR}/src/core/ext/upb-generated/envoy/api/v2/cluster/circuit_breaker.upb.c - ${GRPC_SOURCE_DIR}/src/core/ext/upb-generated/envoy/api/v2/cluster/filter.upb.c - ${GRPC_SOURCE_DIR}/src/core/ext/upb-generated/envoy/api/v2/cluster/outlier_detection.upb.c - ${GRPC_SOURCE_DIR}/src/core/ext/upb-generated/envoy/api/v2/discovery.upb.c - ${GRPC_SOURCE_DIR}/src/core/ext/upb-generated/envoy/api/v2/eds.upb.c - ${GRPC_SOURCE_DIR}/src/core/ext/upb-generated/envoy/api/v2/endpoint/endpoint.upb.c - ${GRPC_SOURCE_DIR}/src/core/ext/upb-generated/envoy/api/v2/endpoint/load_report.upb.c - ${GRPC_SOURCE_DIR}/src/core/ext/upb-generated/envoy/service/discovery/v2/ads.upb.c - ${GRPC_SOURCE_DIR}/src/core/ext/upb-generated/envoy/service/load_stats/v2/lrs.upb.c - ${GRPC_SOURCE_DIR}/src/core/ext/upb-generated/envoy/api/v2/core/address.upb.c - ${GRPC_SOURCE_DIR}/src/core/ext/upb-generated/envoy/api/v2/core/base.upb.c - ${GRPC_SOURCE_DIR}/src/core/ext/upb-generated/envoy/api/v2/core/config_source.upb.c - ${GRPC_SOURCE_DIR}/src/core/ext/upb-generated/envoy/api/v2/core/grpc_service.upb.c - ${GRPC_SOURCE_DIR}/src/core/ext/upb-generated/envoy/api/v2/core/health_check.upb.c - ${GRPC_SOURCE_DIR}/src/core/ext/upb-generated/envoy/api/v2/core/http_uri.upb.c - ${GRPC_SOURCE_DIR}/src/core/ext/upb-generated/envoy/api/v2/core/protocol.upb.c - ${GRPC_SOURCE_DIR}/src/core/ext/upb-generated/envoy/type/percent.upb.c - ${GRPC_SOURCE_DIR}/src/core/ext/upb-generated/envoy/type/range.upb.c - ${GRPC_SOURCE_DIR}/src/core/ext/filters/client_channel/lb_policy/pick_first/pick_first.cc - ${GRPC_SOURCE_DIR}/src/core/ext/filters/client_channel/lb_policy/round_robin/round_robin.cc - ${GRPC_SOURCE_DIR}/src/core/ext/filters/client_channel/resolver/dns/c_ares/dns_resolver_ares.cc - ${GRPC_SOURCE_DIR}/src/core/ext/filters/client_channel/resolver/dns/c_ares/grpc_ares_ev_driver.cc - ${GRPC_SOURCE_DIR}/src/core/ext/filters/client_channel/resolver/dns/c_ares/grpc_ares_ev_driver_libuv.cc - ${GRPC_SOURCE_DIR}/src/core/ext/filters/client_channel/resolver/dns/c_ares/grpc_ares_ev_driver_posix.cc - ${GRPC_SOURCE_DIR}/src/core/ext/filters/client_channel/resolver/dns/c_ares/grpc_ares_ev_driver_windows.cc - ${GRPC_SOURCE_DIR}/src/core/ext/filters/client_channel/resolver/dns/c_ares/grpc_ares_wrapper.cc - ${GRPC_SOURCE_DIR}/src/core/ext/filters/client_channel/resolver/dns/c_ares/grpc_ares_wrapper_fallback.cc - ${GRPC_SOURCE_DIR}/src/core/ext/filters/client_channel/resolver/dns/c_ares/grpc_ares_wrapper_libuv.cc - ${GRPC_SOURCE_DIR}/src/core/ext/filters/client_channel/resolver/dns/c_ares/grpc_ares_wrapper_posix.cc - ${GRPC_SOURCE_DIR}/src/core/ext/filters/client_channel/resolver/dns/c_ares/grpc_ares_wrapper_windows.cc - ${GRPC_SOURCE_DIR}/src/core/ext/filters/client_channel/resolver/dns/dns_resolver_selection.cc - ${GRPC_SOURCE_DIR}/src/core/ext/filters/client_channel/resolver/dns/native/dns_resolver.cc - ${GRPC_SOURCE_DIR}/src/core/ext/filters/client_channel/resolver/sockaddr/sockaddr_resolver.cc - ${GRPC_SOURCE_DIR}/src/core/ext/filters/client_channel/resolver/xds/xds_resolver.cc - ${GRPC_SOURCE_DIR}/src/core/ext/filters/census/grpc_context.cc - ${GRPC_SOURCE_DIR}/src/core/ext/filters/client_idle/client_idle_filter.cc - ${GRPC_SOURCE_DIR}/src/core/ext/filters/max_age/max_age_filter.cc - ${GRPC_SOURCE_DIR}/src/core/ext/filters/message_size/message_size_filter.cc - ${GRPC_SOURCE_DIR}/src/core/ext/filters/http/client_authority_filter.cc - ${GRPC_SOURCE_DIR}/src/core/ext/filters/workarounds/workaround_cronet_compression_filter.cc - ${GRPC_SOURCE_DIR}/src/core/ext/filters/workarounds/workaround_utils.cc - ${GRPC_SOURCE_DIR}/src/core/plugin_registry/grpc_plugin_registry.cc -) - - -target_include_directories(grpc - PUBLIC ${GRPC_INCLUDE_DIR} - PRIVATE ${GRPC_SOURCE_DIR} - PRIVATE ${_gRPC_ADDRESS_SORTING_INCLUDE_DIR} - PRIVATE ${_gRPC_CARES_INCLUDE_DIR} - PRIVATE ${_gRPC_PROTOBUF_INCLUDE_DIR} - PRIVATE ${_gRPC_SSL_INCLUDE_DIR} - PRIVATE ${_gRPC_UPB_GENERATED_DIR} - PRIVATE ${_gRPC_UPB_GRPC_GENERATED_DIR} - PRIVATE ${_gRPC_UPB_INCLUDE_DIR} - PRIVATE ${_gRPC_ZLIB_INCLUDE_DIR} -) - -target_link_libraries(grpc - ${_gRPC_SSL_LIBRARIES} - ${_gRPC_ZLIB_LIBRARIES} - ${_gRPC_CARES_LIBRARIES} - ${_gRPC_ADDRESS_SORTING_LIBRARIES} - ${_gRPC_ALLTARGETS_LIBRARIES} - ${_gRPC_PROTOBUF_LIBRARIES} - gpr -) -if (_gRPC_PLATFORM_MAC) - target_link_libraries(grpc "-framework CoreFoundation") -endif() - - -add_library(grpc_unsecure - ${GRPC_SOURCE_DIR}/src/core/lib/surface/init.cc - ${GRPC_SOURCE_DIR}/src/core/lib/surface/init_unsecure.cc - ${GRPC_SOURCE_DIR}/src/core/lib/avl/avl.cc - ${GRPC_SOURCE_DIR}/src/core/lib/backoff/backoff.cc - ${GRPC_SOURCE_DIR}/src/core/lib/channel/channel_args.cc - ${GRPC_SOURCE_DIR}/src/core/lib/channel/channel_stack.cc - ${GRPC_SOURCE_DIR}/src/core/lib/channel/channel_stack_builder.cc - ${GRPC_SOURCE_DIR}/src/core/lib/channel/channel_trace.cc - ${GRPC_SOURCE_DIR}/src/core/lib/channel/channelz.cc - ${GRPC_SOURCE_DIR}/src/core/lib/channel/channelz_registry.cc - ${GRPC_SOURCE_DIR}/src/core/lib/channel/connected_channel.cc - ${GRPC_SOURCE_DIR}/src/core/lib/channel/handshaker.cc - ${GRPC_SOURCE_DIR}/src/core/lib/channel/handshaker_registry.cc - ${GRPC_SOURCE_DIR}/src/core/lib/channel/status_util.cc - ${GRPC_SOURCE_DIR}/src/core/lib/compression/compression.cc - ${GRPC_SOURCE_DIR}/src/core/lib/compression/compression_args.cc - ${GRPC_SOURCE_DIR}/src/core/lib/compression/compression_internal.cc - ${GRPC_SOURCE_DIR}/src/core/lib/compression/message_compress.cc - ${GRPC_SOURCE_DIR}/src/core/lib/compression/stream_compression.cc - ${GRPC_SOURCE_DIR}/src/core/lib/compression/stream_compression_gzip.cc - ${GRPC_SOURCE_DIR}/src/core/lib/compression/stream_compression_identity.cc - ${GRPC_SOURCE_DIR}/src/core/lib/debug/stats.cc - ${GRPC_SOURCE_DIR}/src/core/lib/debug/stats_data.cc - ${GRPC_SOURCE_DIR}/src/core/lib/http/format_request.cc - ${GRPC_SOURCE_DIR}/src/core/lib/http/httpcli.cc - ${GRPC_SOURCE_DIR}/src/core/lib/http/parser.cc - ${GRPC_SOURCE_DIR}/src/core/lib/iomgr/buffer_list.cc - ${GRPC_SOURCE_DIR}/src/core/lib/iomgr/call_combiner.cc - ${GRPC_SOURCE_DIR}/src/core/lib/iomgr/cfstream_handle.cc - ${GRPC_SOURCE_DIR}/src/core/lib/iomgr/combiner.cc - ${GRPC_SOURCE_DIR}/src/core/lib/iomgr/endpoint.cc - ${GRPC_SOURCE_DIR}/src/core/lib/iomgr/endpoint_cfstream.cc - ${GRPC_SOURCE_DIR}/src/core/lib/iomgr/endpoint_pair_posix.cc - ${GRPC_SOURCE_DIR}/src/core/lib/iomgr/endpoint_pair_uv.cc - ${GRPC_SOURCE_DIR}/src/core/lib/iomgr/endpoint_pair_windows.cc - ${GRPC_SOURCE_DIR}/src/core/lib/iomgr/error.cc - ${GRPC_SOURCE_DIR}/src/core/lib/iomgr/error_cfstream.cc - ${GRPC_SOURCE_DIR}/src/core/lib/iomgr/ev_epoll1_linux.cc - ${GRPC_SOURCE_DIR}/src/core/lib/iomgr/ev_epollex_linux.cc - ${GRPC_SOURCE_DIR}/src/core/lib/iomgr/ev_poll_posix.cc - ${GRPC_SOURCE_DIR}/src/core/lib/iomgr/ev_posix.cc - ${GRPC_SOURCE_DIR}/src/core/lib/iomgr/ev_windows.cc - ${GRPC_SOURCE_DIR}/src/core/lib/iomgr/exec_ctx.cc - ${GRPC_SOURCE_DIR}/src/core/lib/iomgr/executor.cc - ${GRPC_SOURCE_DIR}/src/core/lib/iomgr/executor/mpmcqueue.cc - ${GRPC_SOURCE_DIR}/src/core/lib/iomgr/executor/threadpool.cc - ${GRPC_SOURCE_DIR}/src/core/lib/iomgr/fork_posix.cc - ${GRPC_SOURCE_DIR}/src/core/lib/iomgr/fork_windows.cc - ${GRPC_SOURCE_DIR}/src/core/lib/iomgr/gethostname_fallback.cc - ${GRPC_SOURCE_DIR}/src/core/lib/iomgr/gethostname_host_name_max.cc - ${GRPC_SOURCE_DIR}/src/core/lib/iomgr/gethostname_sysconf.cc - ${GRPC_SOURCE_DIR}/src/core/lib/iomgr/grpc_if_nametoindex_posix.cc - ${GRPC_SOURCE_DIR}/src/core/lib/iomgr/grpc_if_nametoindex_unsupported.cc - ${GRPC_SOURCE_DIR}/src/core/lib/iomgr/internal_errqueue.cc - ${GRPC_SOURCE_DIR}/src/core/lib/iomgr/iocp_windows.cc - ${GRPC_SOURCE_DIR}/src/core/lib/iomgr/iomgr.cc - ${GRPC_SOURCE_DIR}/src/core/lib/iomgr/iomgr_custom.cc - ${GRPC_SOURCE_DIR}/src/core/lib/iomgr/iomgr_internal.cc - ${GRPC_SOURCE_DIR}/src/core/lib/iomgr/iomgr_posix.cc - ${GRPC_SOURCE_DIR}/src/core/lib/iomgr/iomgr_posix_cfstream.cc - ${GRPC_SOURCE_DIR}/src/core/lib/iomgr/iomgr_uv.cc - ${GRPC_SOURCE_DIR}/src/core/lib/iomgr/iomgr_windows.cc - ${GRPC_SOURCE_DIR}/src/core/lib/iomgr/is_epollexclusive_available.cc - ${GRPC_SOURCE_DIR}/src/core/lib/iomgr/load_file.cc - ${GRPC_SOURCE_DIR}/src/core/lib/iomgr/lockfree_event.cc - ${GRPC_SOURCE_DIR}/src/core/lib/iomgr/polling_entity.cc - ${GRPC_SOURCE_DIR}/src/core/lib/iomgr/pollset.cc - ${GRPC_SOURCE_DIR}/src/core/lib/iomgr/pollset_custom.cc - ${GRPC_SOURCE_DIR}/src/core/lib/iomgr/pollset_set.cc - ${GRPC_SOURCE_DIR}/src/core/lib/iomgr/pollset_set_custom.cc - ${GRPC_SOURCE_DIR}/src/core/lib/iomgr/pollset_set_windows.cc - ${GRPC_SOURCE_DIR}/src/core/lib/iomgr/pollset_uv.cc - ${GRPC_SOURCE_DIR}/src/core/lib/iomgr/pollset_windows.cc - ${GRPC_SOURCE_DIR}/src/core/lib/iomgr/resolve_address.cc - ${GRPC_SOURCE_DIR}/src/core/lib/iomgr/resolve_address_custom.cc - ${GRPC_SOURCE_DIR}/src/core/lib/iomgr/resolve_address_posix.cc - ${GRPC_SOURCE_DIR}/src/core/lib/iomgr/resolve_address_windows.cc - ${GRPC_SOURCE_DIR}/src/core/lib/iomgr/resource_quota.cc - ${GRPC_SOURCE_DIR}/src/core/lib/iomgr/sockaddr_utils.cc - ${GRPC_SOURCE_DIR}/src/core/lib/iomgr/socket_factory_posix.cc - ${GRPC_SOURCE_DIR}/src/core/lib/iomgr/socket_mutator.cc - ${GRPC_SOURCE_DIR}/src/core/lib/iomgr/socket_utils_common_posix.cc - ${GRPC_SOURCE_DIR}/src/core/lib/iomgr/socket_utils_linux.cc - ${GRPC_SOURCE_DIR}/src/core/lib/iomgr/socket_utils_posix.cc - ${GRPC_SOURCE_DIR}/src/core/lib/iomgr/socket_utils_uv.cc - ${GRPC_SOURCE_DIR}/src/core/lib/iomgr/socket_utils_windows.cc - ${GRPC_SOURCE_DIR}/src/core/lib/iomgr/socket_windows.cc - ${GRPC_SOURCE_DIR}/src/core/lib/iomgr/tcp_client.cc - ${GRPC_SOURCE_DIR}/src/core/lib/iomgr/tcp_client_cfstream.cc - ${GRPC_SOURCE_DIR}/src/core/lib/iomgr/tcp_client_custom.cc - ${GRPC_SOURCE_DIR}/src/core/lib/iomgr/tcp_client_posix.cc - ${GRPC_SOURCE_DIR}/src/core/lib/iomgr/tcp_client_windows.cc - ${GRPC_SOURCE_DIR}/src/core/lib/iomgr/tcp_custom.cc - ${GRPC_SOURCE_DIR}/src/core/lib/iomgr/tcp_posix.cc - ${GRPC_SOURCE_DIR}/src/core/lib/iomgr/tcp_server.cc - ${GRPC_SOURCE_DIR}/src/core/lib/iomgr/tcp_server_custom.cc - ${GRPC_SOURCE_DIR}/src/core/lib/iomgr/tcp_server_posix.cc - ${GRPC_SOURCE_DIR}/src/core/lib/iomgr/tcp_server_utils_posix_common.cc - ${GRPC_SOURCE_DIR}/src/core/lib/iomgr/tcp_server_utils_posix_ifaddrs.cc - ${GRPC_SOURCE_DIR}/src/core/lib/iomgr/tcp_server_utils_posix_noifaddrs.cc - ${GRPC_SOURCE_DIR}/src/core/lib/iomgr/tcp_server_windows.cc - ${GRPC_SOURCE_DIR}/src/core/lib/iomgr/tcp_uv.cc - ${GRPC_SOURCE_DIR}/src/core/lib/iomgr/tcp_windows.cc - ${GRPC_SOURCE_DIR}/src/core/lib/iomgr/time_averaged_stats.cc - ${GRPC_SOURCE_DIR}/src/core/lib/iomgr/timer.cc - ${GRPC_SOURCE_DIR}/src/core/lib/iomgr/timer_custom.cc - ${GRPC_SOURCE_DIR}/src/core/lib/iomgr/timer_generic.cc - ${GRPC_SOURCE_DIR}/src/core/lib/iomgr/timer_heap.cc - ${GRPC_SOURCE_DIR}/src/core/lib/iomgr/timer_manager.cc - ${GRPC_SOURCE_DIR}/src/core/lib/iomgr/timer_uv.cc - ${GRPC_SOURCE_DIR}/src/core/lib/iomgr/udp_server.cc - ${GRPC_SOURCE_DIR}/src/core/lib/iomgr/unix_sockets_posix.cc - ${GRPC_SOURCE_DIR}/src/core/lib/iomgr/unix_sockets_posix_noop.cc - ${GRPC_SOURCE_DIR}/src/core/lib/iomgr/wakeup_fd_eventfd.cc - ${GRPC_SOURCE_DIR}/src/core/lib/iomgr/wakeup_fd_nospecial.cc - ${GRPC_SOURCE_DIR}/src/core/lib/iomgr/wakeup_fd_pipe.cc - ${GRPC_SOURCE_DIR}/src/core/lib/iomgr/wakeup_fd_posix.cc - ${GRPC_SOURCE_DIR}/src/core/lib/json/json.cc - ${GRPC_SOURCE_DIR}/src/core/lib/json/json_reader.cc - ${GRPC_SOURCE_DIR}/src/core/lib/json/json_string.cc - ${GRPC_SOURCE_DIR}/src/core/lib/json/json_writer.cc - ${GRPC_SOURCE_DIR}/src/core/lib/slice/b64.cc - ${GRPC_SOURCE_DIR}/src/core/lib/slice/percent_encoding.cc - ${GRPC_SOURCE_DIR}/src/core/lib/slice/slice.cc - ${GRPC_SOURCE_DIR}/src/core/lib/slice/slice_buffer.cc - ${GRPC_SOURCE_DIR}/src/core/lib/slice/slice_intern.cc - ${GRPC_SOURCE_DIR}/src/core/lib/slice/slice_string_helpers.cc - ${GRPC_SOURCE_DIR}/src/core/lib/surface/api_trace.cc - ${GRPC_SOURCE_DIR}/src/core/lib/surface/byte_buffer.cc - ${GRPC_SOURCE_DIR}/src/core/lib/surface/byte_buffer_reader.cc - ${GRPC_SOURCE_DIR}/src/core/lib/surface/call.cc - ${GRPC_SOURCE_DIR}/src/core/lib/surface/call_details.cc - ${GRPC_SOURCE_DIR}/src/core/lib/surface/call_log_batch.cc - ${GRPC_SOURCE_DIR}/src/core/lib/surface/channel.cc - ${GRPC_SOURCE_DIR}/src/core/lib/surface/channel_init.cc - ${GRPC_SOURCE_DIR}/src/core/lib/surface/channel_ping.cc - ${GRPC_SOURCE_DIR}/src/core/lib/surface/channel_stack_type.cc - ${GRPC_SOURCE_DIR}/src/core/lib/surface/completion_queue.cc - ${GRPC_SOURCE_DIR}/src/core/lib/surface/completion_queue_factory.cc - ${GRPC_SOURCE_DIR}/src/core/lib/surface/event_string.cc - ${GRPC_SOURCE_DIR}/src/core/lib/surface/lame_client.cc - ${GRPC_SOURCE_DIR}/src/core/lib/surface/metadata_array.cc - ${GRPC_SOURCE_DIR}/src/core/lib/surface/server.cc - ${GRPC_SOURCE_DIR}/src/core/lib/surface/validate_metadata.cc - ${GRPC_SOURCE_DIR}/src/core/lib/surface/version.cc - ${GRPC_SOURCE_DIR}/src/core/lib/transport/bdp_estimator.cc - ${GRPC_SOURCE_DIR}/src/core/lib/transport/byte_stream.cc - ${GRPC_SOURCE_DIR}/src/core/lib/transport/connectivity_state.cc - ${GRPC_SOURCE_DIR}/src/core/lib/transport/error_utils.cc - ${GRPC_SOURCE_DIR}/src/core/lib/transport/metadata.cc - ${GRPC_SOURCE_DIR}/src/core/lib/transport/metadata_batch.cc - ${GRPC_SOURCE_DIR}/src/core/lib/transport/pid_controller.cc - ${GRPC_SOURCE_DIR}/src/core/lib/transport/static_metadata.cc - ${GRPC_SOURCE_DIR}/src/core/lib/transport/status_conversion.cc - ${GRPC_SOURCE_DIR}/src/core/lib/transport/status_metadata.cc - ${GRPC_SOURCE_DIR}/src/core/lib/transport/timeout_encoding.cc - ${GRPC_SOURCE_DIR}/src/core/lib/transport/transport.cc - ${GRPC_SOURCE_DIR}/src/core/lib/transport/transport_op_string.cc - ${GRPC_SOURCE_DIR}/src/core/lib/uri/uri_parser.cc - ${GRPC_SOURCE_DIR}/src/core/lib/debug/trace.cc - ${GRPC_SOURCE_DIR}/src/core/ext/transport/chttp2/server/insecure/server_chttp2.cc - ${GRPC_SOURCE_DIR}/src/core/ext/transport/chttp2/server/insecure/server_chttp2_posix.cc - ${GRPC_SOURCE_DIR}/src/core/ext/transport/chttp2/transport/bin_decoder.cc - ${GRPC_SOURCE_DIR}/src/core/ext/transport/chttp2/transport/bin_encoder.cc - ${GRPC_SOURCE_DIR}/src/core/ext/transport/chttp2/transport/chttp2_plugin.cc - ${GRPC_SOURCE_DIR}/src/core/ext/transport/chttp2/transport/chttp2_transport.cc - ${GRPC_SOURCE_DIR}/src/core/ext/transport/chttp2/transport/context_list.cc - ${GRPC_SOURCE_DIR}/src/core/ext/transport/chttp2/transport/flow_control.cc - ${GRPC_SOURCE_DIR}/src/core/ext/transport/chttp2/transport/frame_data.cc - ${GRPC_SOURCE_DIR}/src/core/ext/transport/chttp2/transport/frame_goaway.cc - ${GRPC_SOURCE_DIR}/src/core/ext/transport/chttp2/transport/frame_ping.cc - ${GRPC_SOURCE_DIR}/src/core/ext/transport/chttp2/transport/frame_rst_stream.cc - ${GRPC_SOURCE_DIR}/src/core/ext/transport/chttp2/transport/frame_settings.cc - ${GRPC_SOURCE_DIR}/src/core/ext/transport/chttp2/transport/frame_window_update.cc - ${GRPC_SOURCE_DIR}/src/core/ext/transport/chttp2/transport/hpack_encoder.cc - ${GRPC_SOURCE_DIR}/src/core/ext/transport/chttp2/transport/hpack_parser.cc - ${GRPC_SOURCE_DIR}/src/core/ext/transport/chttp2/transport/hpack_table.cc - ${GRPC_SOURCE_DIR}/src/core/ext/transport/chttp2/transport/http2_settings.cc - ${GRPC_SOURCE_DIR}/src/core/ext/transport/chttp2/transport/huffsyms.cc - ${GRPC_SOURCE_DIR}/src/core/ext/transport/chttp2/transport/incoming_metadata.cc - ${GRPC_SOURCE_DIR}/src/core/ext/transport/chttp2/transport/parsing.cc - ${GRPC_SOURCE_DIR}/src/core/ext/transport/chttp2/transport/stream_lists.cc - ${GRPC_SOURCE_DIR}/src/core/ext/transport/chttp2/transport/stream_map.cc - ${GRPC_SOURCE_DIR}/src/core/ext/transport/chttp2/transport/varint.cc - ${GRPC_SOURCE_DIR}/src/core/ext/transport/chttp2/transport/writing.cc - ${GRPC_SOURCE_DIR}/src/core/ext/transport/chttp2/alpn/alpn.cc - ${GRPC_SOURCE_DIR}/src/core/ext/filters/http/client/http_client_filter.cc - ${GRPC_SOURCE_DIR}/src/core/ext/filters/http/http_filters_plugin.cc - ${GRPC_SOURCE_DIR}/src/core/ext/filters/http/message_compress/message_compress_filter.cc - ${GRPC_SOURCE_DIR}/src/core/ext/filters/http/server/http_server_filter.cc - ${GRPC_SOURCE_DIR}/src/core/ext/transport/chttp2/server/chttp2_server.cc - ${GRPC_SOURCE_DIR}/src/core/ext/transport/chttp2/client/insecure/channel_create.cc - ${GRPC_SOURCE_DIR}/src/core/ext/transport/chttp2/client/insecure/channel_create_posix.cc - ${GRPC_SOURCE_DIR}/src/core/ext/transport/chttp2/client/authority.cc - ${GRPC_SOURCE_DIR}/src/core/ext/transport/chttp2/client/chttp2_connector.cc - ${GRPC_SOURCE_DIR}/src/core/ext/filters/client_channel/backend_metric.cc - ${GRPC_SOURCE_DIR}/src/core/ext/filters/client_channel/backup_poller.cc - ${GRPC_SOURCE_DIR}/src/core/ext/filters/client_channel/channel_connectivity.cc - ${GRPC_SOURCE_DIR}/src/core/ext/filters/client_channel/client_channel.cc - ${GRPC_SOURCE_DIR}/src/core/ext/filters/client_channel/client_channel_channelz.cc - ${GRPC_SOURCE_DIR}/src/core/ext/filters/client_channel/client_channel_factory.cc - ${GRPC_SOURCE_DIR}/src/core/ext/filters/client_channel/client_channel_plugin.cc - ${GRPC_SOURCE_DIR}/src/core/ext/filters/client_channel/connector.cc - ${GRPC_SOURCE_DIR}/src/core/ext/filters/client_channel/global_subchannel_pool.cc - ${GRPC_SOURCE_DIR}/src/core/ext/filters/client_channel/health/health_check_client.cc - ${GRPC_SOURCE_DIR}/src/core/ext/filters/client_channel/http_connect_handshaker.cc - ${GRPC_SOURCE_DIR}/src/core/ext/filters/client_channel/http_proxy.cc - ${GRPC_SOURCE_DIR}/src/core/ext/filters/client_channel/lb_policy.cc - ${GRPC_SOURCE_DIR}/src/core/ext/filters/client_channel/lb_policy_registry.cc - ${GRPC_SOURCE_DIR}/src/core/ext/filters/client_channel/local_subchannel_pool.cc - ${GRPC_SOURCE_DIR}/src/core/ext/filters/client_channel/parse_address.cc - ${GRPC_SOURCE_DIR}/src/core/ext/filters/client_channel/proxy_mapper.cc - ${GRPC_SOURCE_DIR}/src/core/ext/filters/client_channel/proxy_mapper_registry.cc - ${GRPC_SOURCE_DIR}/src/core/ext/filters/client_channel/resolver.cc - ${GRPC_SOURCE_DIR}/src/core/ext/filters/client_channel/resolver_registry.cc - ${GRPC_SOURCE_DIR}/src/core/ext/filters/client_channel/resolver_result_parsing.cc - ${GRPC_SOURCE_DIR}/src/core/ext/filters/client_channel/resolving_lb_policy.cc - ${GRPC_SOURCE_DIR}/src/core/ext/filters/client_channel/retry_throttle.cc - ${GRPC_SOURCE_DIR}/src/core/ext/filters/client_channel/server_address.cc - ${GRPC_SOURCE_DIR}/src/core/ext/filters/client_channel/service_config.cc - ${GRPC_SOURCE_DIR}/src/core/ext/filters/client_channel/subchannel.cc - ${GRPC_SOURCE_DIR}/src/core/ext/filters/client_channel/subchannel_pool_interface.cc - ${GRPC_SOURCE_DIR}/src/core/ext/filters/deadline/deadline_filter.cc - ${GRPC_SOURCE_DIR}/src/core/ext/upb-generated/src/proto/grpc/health/v1/health.upb.c - ${GRPC_SOURCE_DIR}/third_party/upb/upb/decode.c - ${GRPC_SOURCE_DIR}/third_party/upb/upb/encode.c - ${GRPC_SOURCE_DIR}/third_party/upb/upb/msg.c - ${GRPC_SOURCE_DIR}/third_party/upb/upb/port.c - ${GRPC_SOURCE_DIR}/third_party/upb/upb/table.c - ${GRPC_SOURCE_DIR}/third_party/upb/upb/upb.c - ${GRPC_SOURCE_DIR}/src/core/ext/upb-generated/udpa/data/orca/v1/orca_load_report.upb.c - ${GRPC_SOURCE_DIR}/src/core/ext/upb-generated/gogoproto/gogo.upb.c - ${GRPC_SOURCE_DIR}/src/core/ext/upb-generated/validate/validate.upb.c - ${GRPC_SOURCE_DIR}/src/core/ext/upb-generated/google/api/annotations.upb.c - ${GRPC_SOURCE_DIR}/src/core/ext/upb-generated/google/api/http.upb.c - ${GRPC_SOURCE_DIR}/src/core/ext/upb-generated/google/protobuf/any.upb.c - ${GRPC_SOURCE_DIR}/src/core/ext/upb-generated/google/protobuf/descriptor.upb.c - ${GRPC_SOURCE_DIR}/src/core/ext/upb-generated/google/protobuf/duration.upb.c - ${GRPC_SOURCE_DIR}/src/core/ext/upb-generated/google/protobuf/empty.upb.c - ${GRPC_SOURCE_DIR}/src/core/ext/upb-generated/google/protobuf/struct.upb.c - ${GRPC_SOURCE_DIR}/src/core/ext/upb-generated/google/protobuf/timestamp.upb.c - ${GRPC_SOURCE_DIR}/src/core/ext/upb-generated/google/protobuf/wrappers.upb.c - ${GRPC_SOURCE_DIR}/src/core/ext/upb-generated/google/rpc/status.upb.c - ${GRPC_SOURCE_DIR}/src/core/ext/transport/inproc/inproc_plugin.cc - ${GRPC_SOURCE_DIR}/src/core/ext/transport/inproc/inproc_transport.cc - ${GRPC_SOURCE_DIR}/src/core/ext/filters/client_channel/resolver/dns/c_ares/dns_resolver_ares.cc - ${GRPC_SOURCE_DIR}/src/core/ext/filters/client_channel/resolver/dns/c_ares/grpc_ares_ev_driver.cc - ${GRPC_SOURCE_DIR}/src/core/ext/filters/client_channel/resolver/dns/c_ares/grpc_ares_ev_driver_libuv.cc - ${GRPC_SOURCE_DIR}/src/core/ext/filters/client_channel/resolver/dns/c_ares/grpc_ares_ev_driver_posix.cc - ${GRPC_SOURCE_DIR}/src/core/ext/filters/client_channel/resolver/dns/c_ares/grpc_ares_ev_driver_windows.cc - ${GRPC_SOURCE_DIR}/src/core/ext/filters/client_channel/resolver/dns/c_ares/grpc_ares_wrapper.cc - ${GRPC_SOURCE_DIR}/src/core/ext/filters/client_channel/resolver/dns/c_ares/grpc_ares_wrapper_fallback.cc - ${GRPC_SOURCE_DIR}/src/core/ext/filters/client_channel/resolver/dns/c_ares/grpc_ares_wrapper_libuv.cc - ${GRPC_SOURCE_DIR}/src/core/ext/filters/client_channel/resolver/dns/c_ares/grpc_ares_wrapper_posix.cc - ${GRPC_SOURCE_DIR}/src/core/ext/filters/client_channel/resolver/dns/c_ares/grpc_ares_wrapper_windows.cc - ${GRPC_SOURCE_DIR}/src/core/ext/filters/client_channel/resolver/dns/dns_resolver_selection.cc - ${GRPC_SOURCE_DIR}/src/core/ext/filters/client_channel/resolver/dns/native/dns_resolver.cc - ${GRPC_SOURCE_DIR}/src/core/ext/filters/client_channel/resolver/sockaddr/sockaddr_resolver.cc - ${GRPC_SOURCE_DIR}/src/core/ext/filters/client_channel/resolver/fake/fake_resolver.cc - ${GRPC_SOURCE_DIR}/src/core/ext/filters/client_channel/resolver/xds/xds_resolver.cc - ${GRPC_SOURCE_DIR}/src/core/ext/filters/client_channel/xds/xds_api.cc - ${GRPC_SOURCE_DIR}/src/core/ext/filters/client_channel/xds/xds_bootstrap.cc - ${GRPC_SOURCE_DIR}/src/core/ext/filters/client_channel/xds/xds_channel.cc - ${GRPC_SOURCE_DIR}/src/core/ext/filters/client_channel/xds/xds_client.cc - ${GRPC_SOURCE_DIR}/src/core/ext/filters/client_channel/xds/xds_client_stats.cc - ${GRPC_SOURCE_DIR}/src/core/ext/upb-generated/envoy/api/v2/auth/cert.upb.c - ${GRPC_SOURCE_DIR}/src/core/ext/upb-generated/envoy/api/v2/cds.upb.c - ${GRPC_SOURCE_DIR}/src/core/ext/upb-generated/envoy/api/v2/cluster/circuit_breaker.upb.c - ${GRPC_SOURCE_DIR}/src/core/ext/upb-generated/envoy/api/v2/cluster/filter.upb.c - ${GRPC_SOURCE_DIR}/src/core/ext/upb-generated/envoy/api/v2/cluster/outlier_detection.upb.c - ${GRPC_SOURCE_DIR}/src/core/ext/upb-generated/envoy/api/v2/discovery.upb.c - ${GRPC_SOURCE_DIR}/src/core/ext/upb-generated/envoy/api/v2/eds.upb.c - ${GRPC_SOURCE_DIR}/src/core/ext/upb-generated/envoy/api/v2/endpoint/endpoint.upb.c - ${GRPC_SOURCE_DIR}/src/core/ext/upb-generated/envoy/api/v2/endpoint/load_report.upb.c - ${GRPC_SOURCE_DIR}/src/core/ext/upb-generated/envoy/service/discovery/v2/ads.upb.c - ${GRPC_SOURCE_DIR}/src/core/ext/upb-generated/envoy/service/load_stats/v2/lrs.upb.c - ${GRPC_SOURCE_DIR}/src/core/ext/upb-generated/envoy/api/v2/core/address.upb.c - ${GRPC_SOURCE_DIR}/src/core/ext/upb-generated/envoy/api/v2/core/base.upb.c - ${GRPC_SOURCE_DIR}/src/core/ext/upb-generated/envoy/api/v2/core/config_source.upb.c - ${GRPC_SOURCE_DIR}/src/core/ext/upb-generated/envoy/api/v2/core/grpc_service.upb.c - ${GRPC_SOURCE_DIR}/src/core/ext/upb-generated/envoy/api/v2/core/health_check.upb.c - ${GRPC_SOURCE_DIR}/src/core/ext/upb-generated/envoy/api/v2/core/http_uri.upb.c - ${GRPC_SOURCE_DIR}/src/core/ext/upb-generated/envoy/api/v2/core/protocol.upb.c - ${GRPC_SOURCE_DIR}/src/core/ext/upb-generated/envoy/type/percent.upb.c - ${GRPC_SOURCE_DIR}/src/core/ext/upb-generated/envoy/type/range.upb.c - ${GRPC_SOURCE_DIR}/src/core/ext/filters/client_channel/lb_policy/grpclb/client_load_reporting_filter.cc - ${GRPC_SOURCE_DIR}/src/core/ext/filters/client_channel/lb_policy/grpclb/grpclb.cc - ${GRPC_SOURCE_DIR}/src/core/ext/filters/client_channel/lb_policy/grpclb/grpclb_channel.cc - ${GRPC_SOURCE_DIR}/src/core/ext/filters/client_channel/lb_policy/grpclb/grpclb_client_stats.cc - ${GRPC_SOURCE_DIR}/src/core/ext/filters/client_channel/lb_policy/grpclb/load_balancer_api.cc - ${GRPC_SOURCE_DIR}/src/core/ext/upb-generated/src/proto/grpc/lb/v1/load_balancer.upb.c - ${GRPC_SOURCE_DIR}/src/core/ext/filters/client_channel/lb_policy/xds/xds.cc - ${GRPC_SOURCE_DIR}/src/core/ext/filters/client_channel/lb_policy/pick_first/pick_first.cc - ${GRPC_SOURCE_DIR}/src/core/ext/filters/client_channel/lb_policy/round_robin/round_robin.cc - ${GRPC_SOURCE_DIR}/src/core/ext/filters/census/grpc_context.cc - ${GRPC_SOURCE_DIR}/src/core/ext/filters/client_idle/client_idle_filter.cc - ${GRPC_SOURCE_DIR}/src/core/ext/filters/max_age/max_age_filter.cc - ${GRPC_SOURCE_DIR}/src/core/ext/filters/message_size/message_size_filter.cc - ${GRPC_SOURCE_DIR}/src/core/ext/filters/http/client_authority_filter.cc - ${GRPC_SOURCE_DIR}/src/core/ext/filters/workarounds/workaround_cronet_compression_filter.cc - ${GRPC_SOURCE_DIR}/src/core/ext/filters/workarounds/workaround_utils.cc - ${GRPC_SOURCE_DIR}/src/core/plugin_registry/grpc_unsecure_plugin_registry.cc -) - -target_include_directories(grpc_unsecure - PUBLIC ${GRPC_INCLUDE_DIR} - PRIVATE ${GRPC_SOURCE_DIR} - PRIVATE ${_gRPC_ADDRESS_SORTING_INCLUDE_DIR} - PRIVATE ${_gRPC_CARES_INCLUDE_DIR} - PRIVATE ${_gRPC_PROTOBUF_INCLUDE_DIR} - PRIVATE ${_gRPC_UPB_GENERATED_DIR} - PRIVATE ${_gRPC_UPB_GRPC_GENERATED_DIR} - PRIVATE ${_gRPC_UPB_INCLUDE_DIR} - PRIVATE ${_gRPC_ZLIB_INCLUDE_DIR} -) - -target_link_libraries(grpc_unsecure - ${_gRPC_BASELIB_LIBRARIES} - ${_gRPC_ZLIB_LIBRARIES} - ${_gRPC_CARES_LIBRARIES} - ${_gRPC_ADDRESS_SORTING_LIBRARIES} - ${_gRPC_ALLTARGETS_LIBRARIES} - ${_gRPC_PROTOBUF_LIBRARIES} - gpr -) - -if (_gRPC_PLATFORM_MAC) - target_link_libraries(grpc_unsecure "-framework CoreFoundation") -endif() - - -add_library(grpc++ - ${GRPC_SOURCE_DIR}/src/cpp/client/insecure_credentials.cc - ${GRPC_SOURCE_DIR}/src/cpp/client/secure_credentials.cc - ${GRPC_SOURCE_DIR}/src/cpp/common/auth_property_iterator.cc - ${GRPC_SOURCE_DIR}/src/cpp/common/secure_auth_context.cc - ${GRPC_SOURCE_DIR}/src/cpp/common/secure_channel_arguments.cc - ${GRPC_SOURCE_DIR}/src/cpp/common/secure_create_auth_context.cc - ${GRPC_SOURCE_DIR}/src/cpp/common/tls_credentials_options.cc - ${GRPC_SOURCE_DIR}/src/cpp/common/tls_credentials_options_util.cc - ${GRPC_SOURCE_DIR}/src/cpp/server/insecure_server_credentials.cc - ${GRPC_SOURCE_DIR}/src/cpp/server/secure_server_credentials.cc - ${GRPC_SOURCE_DIR}/src/cpp/client/channel_cc.cc - ${GRPC_SOURCE_DIR}/src/cpp/client/client_context.cc - ${GRPC_SOURCE_DIR}/src/cpp/client/client_interceptor.cc - ${GRPC_SOURCE_DIR}/src/cpp/client/create_channel.cc - ${GRPC_SOURCE_DIR}/src/cpp/client/create_channel_internal.cc - ${GRPC_SOURCE_DIR}/src/cpp/client/create_channel_posix.cc - ${GRPC_SOURCE_DIR}/src/cpp/client/credentials_cc.cc - ${GRPC_SOURCE_DIR}/src/cpp/client/generic_stub.cc - ${GRPC_SOURCE_DIR}/src/cpp/common/alarm.cc - ${GRPC_SOURCE_DIR}/src/cpp/common/channel_arguments.cc - ${GRPC_SOURCE_DIR}/src/cpp/common/channel_filter.cc - ${GRPC_SOURCE_DIR}/src/cpp/common/completion_queue_cc.cc - ${GRPC_SOURCE_DIR}/src/cpp/common/core_codegen.cc - ${GRPC_SOURCE_DIR}/src/cpp/common/resource_quota_cc.cc - ${GRPC_SOURCE_DIR}/src/cpp/common/rpc_method.cc - ${GRPC_SOURCE_DIR}/src/cpp/common/validate_service_config.cc - ${GRPC_SOURCE_DIR}/src/cpp/common/version_cc.cc - ${GRPC_SOURCE_DIR}/src/cpp/server/async_generic_service.cc - ${GRPC_SOURCE_DIR}/src/cpp/server/channel_argument_option.cc - ${GRPC_SOURCE_DIR}/src/cpp/server/create_default_thread_pool.cc - ${GRPC_SOURCE_DIR}/src/cpp/server/dynamic_thread_pool.cc - ${GRPC_SOURCE_DIR}/src/cpp/server/external_connection_acceptor_impl.cc - ${GRPC_SOURCE_DIR}/src/cpp/server/health/default_health_check_service.cc - ${GRPC_SOURCE_DIR}/src/cpp/server/health/health_check_service.cc - ${GRPC_SOURCE_DIR}/src/cpp/server/health/health_check_service_server_builder_option.cc - ${GRPC_SOURCE_DIR}/src/cpp/server/server_builder.cc - ${GRPC_SOURCE_DIR}/src/cpp/server/server_cc.cc - ${GRPC_SOURCE_DIR}/src/cpp/server/server_context.cc - ${GRPC_SOURCE_DIR}/src/cpp/server/server_credentials.cc - ${GRPC_SOURCE_DIR}/src/cpp/server/server_posix.cc - ${GRPC_SOURCE_DIR}/src/cpp/thread_manager/thread_manager.cc - ${GRPC_SOURCE_DIR}/src/cpp/util/byte_buffer_cc.cc - ${GRPC_SOURCE_DIR}/src/cpp/util/status.cc - ${GRPC_SOURCE_DIR}/src/cpp/util/string_ref.cc - ${GRPC_SOURCE_DIR}/src/cpp/util/time_cc.cc - ${GRPC_SOURCE_DIR}/src/core/ext/upb-generated/src/proto/grpc/health/v1/health.upb.c - ${GRPC_SOURCE_DIR}/third_party/upb/upb/decode.c - ${GRPC_SOURCE_DIR}/third_party/upb/upb/encode.c - ${GRPC_SOURCE_DIR}/third_party/upb/upb/msg.c - ${GRPC_SOURCE_DIR}/third_party/upb/upb/port.c - ${GRPC_SOURCE_DIR}/third_party/upb/upb/table.c - ${GRPC_SOURCE_DIR}/third_party/upb/upb/upb.c - ${GRPC_SOURCE_DIR}/src/cpp/codegen/codegen_init.cc -) - -target_include_directories(grpc++ - PUBLIC ${GRPC_INCLUDE_DIR} - PRIVATE ${GRPC_SOURCE_DIR} - PRIVATE ${_gRPC_ADDRESS_SORTING_INCLUDE_DIR} - PRIVATE ${_gRPC_CARES_INCLUDE_DIR} - PRIVATE ${_gRPC_PROTOBUF_INCLUDE_DIR} - PRIVATE ${_gRPC_SSL_INCLUDE_DIR} - PRIVATE ${_gRPC_UPB_GENERATED_DIR} - PRIVATE ${_gRPC_UPB_GRPC_GENERATED_DIR} - PRIVATE ${_gRPC_UPB_INCLUDE_DIR} - PRIVATE ${_gRPC_ZLIB_INCLUDE_DIR} -) - -target_link_libraries(grpc++ - ${_gRPC_SSL_LIBRARIES} - ${_gRPC_PROTOBUF_LIBRARIES} - ${_gRPC_ALLTARGETS_LIBRARIES} - grpc - gpr -) - - -add_library(grpc++_unsecure - ${GRPC_SOURCE_DIR}/src/cpp/client/insecure_credentials.cc - ${GRPC_SOURCE_DIR}/src/cpp/common/insecure_create_auth_context.cc - ${GRPC_SOURCE_DIR}/src/cpp/server/insecure_server_credentials.cc - ${GRPC_SOURCE_DIR}/src/cpp/client/channel_cc.cc - ${GRPC_SOURCE_DIR}/src/cpp/client/client_context.cc - ${GRPC_SOURCE_DIR}/src/cpp/client/client_interceptor.cc - ${GRPC_SOURCE_DIR}/src/cpp/client/create_channel.cc - ${GRPC_SOURCE_DIR}/src/cpp/client/create_channel_internal.cc - ${GRPC_SOURCE_DIR}/src/cpp/client/create_channel_posix.cc - ${GRPC_SOURCE_DIR}/src/cpp/client/credentials_cc.cc - ${GRPC_SOURCE_DIR}/src/cpp/client/generic_stub.cc - ${GRPC_SOURCE_DIR}/src/cpp/common/alarm.cc - ${GRPC_SOURCE_DIR}/src/cpp/common/channel_arguments.cc - ${GRPC_SOURCE_DIR}/src/cpp/common/channel_filter.cc - ${GRPC_SOURCE_DIR}/src/cpp/common/completion_queue_cc.cc - ${GRPC_SOURCE_DIR}/src/cpp/common/core_codegen.cc - ${GRPC_SOURCE_DIR}/src/cpp/common/resource_quota_cc.cc - ${GRPC_SOURCE_DIR}/src/cpp/common/rpc_method.cc - ${GRPC_SOURCE_DIR}/src/cpp/common/validate_service_config.cc - ${GRPC_SOURCE_DIR}/src/cpp/common/version_cc.cc - ${GRPC_SOURCE_DIR}/src/cpp/server/async_generic_service.cc - ${GRPC_SOURCE_DIR}/src/cpp/server/channel_argument_option.cc - ${GRPC_SOURCE_DIR}/src/cpp/server/create_default_thread_pool.cc - ${GRPC_SOURCE_DIR}/src/cpp/server/dynamic_thread_pool.cc - ${GRPC_SOURCE_DIR}/src/cpp/server/external_connection_acceptor_impl.cc - ${GRPC_SOURCE_DIR}/src/cpp/server/health/default_health_check_service.cc - ${GRPC_SOURCE_DIR}/src/cpp/server/health/health_check_service.cc - ${GRPC_SOURCE_DIR}/src/cpp/server/health/health_check_service_server_builder_option.cc - ${GRPC_SOURCE_DIR}/src/cpp/server/server_builder.cc - ${GRPC_SOURCE_DIR}/src/cpp/server/server_cc.cc - ${GRPC_SOURCE_DIR}/src/cpp/server/server_context.cc - ${GRPC_SOURCE_DIR}/src/cpp/server/server_credentials.cc - ${GRPC_SOURCE_DIR}/src/cpp/server/server_posix.cc - ${GRPC_SOURCE_DIR}/src/cpp/thread_manager/thread_manager.cc - ${GRPC_SOURCE_DIR}/src/cpp/util/byte_buffer_cc.cc - ${GRPC_SOURCE_DIR}/src/cpp/util/status.cc - ${GRPC_SOURCE_DIR}/src/cpp/util/string_ref.cc - ${GRPC_SOURCE_DIR}/src/cpp/util/time_cc.cc - ${GRPC_SOURCE_DIR}/src/core/ext/upb-generated/src/proto/grpc/health/v1/health.upb.c - ${GRPC_SOURCE_DIR}/third_party/upb/upb/decode.c - ${GRPC_SOURCE_DIR}/third_party/upb/upb/encode.c - ${GRPC_SOURCE_DIR}/third_party/upb/upb/msg.c - ${GRPC_SOURCE_DIR}/third_party/upb/upb/port.c - ${GRPC_SOURCE_DIR}/third_party/upb/upb/table.c - ${GRPC_SOURCE_DIR}/third_party/upb/upb/upb.c - ${GRPC_SOURCE_DIR}/src/cpp/codegen/codegen_init.cc -) - -target_include_directories(grpc++_unsecure - PUBLIC ${GRPC_INCLUDE_DIR} - PRIVATE ${GRPC_SOURCE_DIR} - PRIVATE ${_gRPC_ADDRESS_SORTING_INCLUDE_DIR} - PRIVATE ${_gRPC_CARES_INCLUDE_DIR} - PRIVATE ${_gRPC_PROTOBUF_INCLUDE_DIR} - PRIVATE ${_gRPC_UPB_GENERATED_DIR} - PRIVATE ${_gRPC_UPB_GRPC_GENERATED_DIR} - PRIVATE ${_gRPC_UPB_INCLUDE_DIR} - PRIVATE ${_gRPC_ZLIB_INCLUDE_DIR} - PRIVATE ${_gRPC_PROTO_GENS_DIR} -) - -target_link_libraries(grpc++_unsecure - ${_gRPC_BASELIB_LIBRARIES} - ${_gRPC_PROTOBUF_LIBRARIES} - ${_gRPC_ALLTARGETS_LIBRARIES} - gpr - grpc_unsecure -) - - -if (_gRPC_SSL_INCLUDE_DIR AND _gRPC_SSL_LIBRARIES) - add_library(libgrpc ALIAS grpc) - add_library(libgrpc++ ALIAS grpc++) -else() - add_library(libgrpc ALIAS grpc_unsecure) - add_library(libgrpc++ ALIAS grpc++_unsecure) -endif() - - -add_library(grpc_plugin_support - ${GRPC_SOURCE_DIR}/src/compiler/cpp_generator.cc -) - -target_include_directories(grpc_plugin_support - PRIVATE ${GRPC_SOURCE_DIR} - PUBLIC ${GRPC_INCLUDE_DIR} - PRIVATE ${CMAKE_CURRENT_SOURCE_DIR} - PRIVATE ${_gRPC_ADDRESS_SORTING_INCLUDE_DIR} - PRIVATE ${_gRPC_CARES_INCLUDE_DIR} - PRIVATE ${_gRPC_PROTOBUF_INCLUDE_DIR} - PRIVATE ${_gRPC_SSL_INCLUDE_DIR} - PRIVATE ${_gRPC_UPB_GENERATED_DIR} - PRIVATE ${_gRPC_UPB_GRPC_GENERATED_DIR} - PRIVATE ${_gRPC_UPB_INCLUDE_DIR} - PRIVATE ${_gRPC_ZLIB_INCLUDE_DIR} -) - -target_link_libraries(grpc_plugin_support - ${_gRPC_PROTOBUF_PROTOC_LIBRARIES} - ${_gRPC_PROTOBUF_LIBRARIES} - ${_gRPC_ALLTARGETS_LIBRARIES} -) - - -add_executable(grpc_cpp_plugin - ${GRPC_SOURCE_DIR}/src/compiler/cpp_plugin.cc -) - -target_include_directories(grpc_cpp_plugin - PRIVATE ${GRPC_SOURCE_DIR} - PUBLIC ${GRPC_INCLUDE_DIR} - PRIVATE ${_gRPC_ADDRESS_SORTING_INCLUDE_DIR} - PRIVATE ${_gRPC_CARES_INCLUDE_DIR} - PRIVATE ${_gRPC_PROTOBUF_INCLUDE_DIR} - PRIVATE ${_gRPC_SSL_INCLUDE_DIR} - PRIVATE ${_gRPC_UPB_GENERATED_DIR} - PRIVATE ${_gRPC_UPB_GRPC_GENERATED_DIR} - PRIVATE ${_gRPC_UPB_INCLUDE_DIR} - PRIVATE ${_gRPC_ZLIB_INCLUDE_DIR} -) - -target_link_libraries(grpc_cpp_plugin - ${_gRPC_PROTOBUF_PROTOC_LIBRARIES} - ${_gRPC_PROTOBUF_LIBRARIES} - ${_gRPC_ALLTARGETS_LIBRARIES} - grpc_plugin_support -) +# The contrib/grpc/CMakeLists.txt redefined the PROTOBUF_GENERATE_GRPC_CPP() function for its own purposes, +# so we need to redefine it back. +include("${ClickHouse_SOURCE_DIR}/contrib/grpc-cmake/protobuf_generate_grpc.cmake") diff --git a/contrib/grpc-cmake/protobuf_generate_grpc.cmake b/contrib/grpc-cmake/protobuf_generate_grpc.cmake new file mode 100644 index 00000000000..08d2976c26a --- /dev/null +++ b/contrib/grpc-cmake/protobuf_generate_grpc.cmake @@ -0,0 +1,207 @@ +#[[ +Add custom commands to process ``.proto`` files to C++:: +protobuf_generate_grpc_cpp( + [DESCRIPTORS ] [EXPORT_MACRO ] [...]) + +``SRCS`` + Variable to define with autogenerated source files +``HDRS`` + Variable to define with autogenerated header files +``DESCRIPTORS`` + Variable to define with autogenerated descriptor files, if requested. +``EXPORT_MACRO`` + is a macro which should expand to ``__declspec(dllexport)`` or + ``__declspec(dllimport)`` depending on what is being compiled. +``ARGN`` + ``.proto`` files +#]] + +# Function to generate C++ files from .proto files. +# This function is a modified version of the function PROTOBUF_GENERATE_CPP() copied from https://github.com/Kitware/CMake/blob/master/Modules/FindProtobuf.cmake. +function(PROTOBUF_GENERATE_GRPC_CPP SRCS HDRS) + cmake_parse_arguments(protobuf_generate_grpc_cpp "" "EXPORT_MACRO;DESCRIPTORS" "" ${ARGN}) + + set(_proto_files "${protobuf_generate_grpc_cpp_UNPARSED_ARGUMENTS}") + if(NOT _proto_files) + message(SEND_ERROR "Error: PROTOBUF_GENERATE_GRPC_CPP() called without any proto files") + return() + endif() + + if(PROTOBUF_GENERATE_GRPC_CPP_APPEND_PATH) + set(_append_arg APPEND_PATH) + endif() + + if(protobuf_generate_grpc_cpp_DESCRIPTORS) + set(_descriptors DESCRIPTORS) + endif() + + if(DEFINED PROTOBUF_IMPORT_DIRS AND NOT DEFINED Protobuf_IMPORT_DIRS) + set(Protobuf_IMPORT_DIRS "${PROTOBUF_IMPORT_DIRS}") + endif() + + if(DEFINED Protobuf_IMPORT_DIRS) + set(_import_arg IMPORT_DIRS ${Protobuf_IMPORT_DIRS}) + endif() + + set(_outvar) + protobuf_generate_grpc(${_append_arg} ${_descriptors} LANGUAGE cpp EXPORT_MACRO ${protobuf_generate_cpp_EXPORT_MACRO} OUT_VAR _outvar ${_import_arg} PROTOS ${_proto_files}) + + set(${SRCS}) + set(${HDRS}) + if(protobuf_generate_grpc_cpp_DESCRIPTORS) + set(${protobuf_generate_grpc_cpp_DESCRIPTORS}) + endif() + + foreach(_file ${_outvar}) + if(_file MATCHES "cc$") + list(APPEND ${SRCS} ${_file}) + elseif(_file MATCHES "desc$") + list(APPEND ${protobuf_generate_grpc_cpp_DESCRIPTORS} ${_file}) + else() + list(APPEND ${HDRS} ${_file}) + endif() + endforeach() + set(${SRCS} ${${SRCS}} PARENT_SCOPE) + set(${HDRS} ${${HDRS}} PARENT_SCOPE) + if(protobuf_generate_grpc_cpp_DESCRIPTORS) + set(${protobuf_generate_grpc_cpp_DESCRIPTORS} "${${protobuf_generate_grpc_cpp_DESCRIPTORS}}" PARENT_SCOPE) + endif() +endfunction() + +# Helper function. +# This function is a modified version of the function protobuf_generate() copied from https://github.com/Kitware/CMake/blob/master/Modules/FindProtobuf.cmake. +function(protobuf_generate_grpc) + set(_options APPEND_PATH DESCRIPTORS) + set(_singleargs LANGUAGE OUT_VAR EXPORT_MACRO PROTOC_OUT_DIR) + if(COMMAND target_sources) + list(APPEND _singleargs TARGET) + endif() + set(_multiargs PROTOS IMPORT_DIRS GENERATE_EXTENSIONS) + + cmake_parse_arguments(protobuf_generate_grpc "${_options}" "${_singleargs}" "${_multiargs}" "${ARGN}") + + if(NOT protobuf_generate_grpc_PROTOS AND NOT protobuf_generate_grpc_TARGET) + message(SEND_ERROR "Error: protobuf_generate_grpc called without any targets or source files") + return() + endif() + + if(NOT protobuf_generate_grpc_OUT_VAR AND NOT protobuf_generate_grpc_TARGET) + message(SEND_ERROR "Error: protobuf_generate_grpc called without a target or output variable") + return() + endif() + + if(NOT protobuf_generate_grpc_LANGUAGE) + set(protobuf_generate_grpc_LANGUAGE cpp) + endif() + string(TOLOWER ${protobuf_generate_grpc_LANGUAGE} protobuf_generate_grpc_LANGUAGE) + + if(NOT protobuf_generate_grpc_PROTOC_OUT_DIR) + set(protobuf_generate_grpc_PROTOC_OUT_DIR ${CMAKE_CURRENT_BINARY_DIR}) + endif() + + if(protobuf_generate_grpc_EXPORT_MACRO AND protobuf_generate_grpc_LANGUAGE STREQUAL cpp) + set(_dll_export_decl "dllexport_decl=${protobuf_generate_grpc_EXPORT_MACRO}:") + endif() + + if(NOT protobuf_generate_grpc_GENERATE_EXTENSIONS) + if(protobuf_generate_grpc_LANGUAGE STREQUAL cpp) + set(protobuf_generate_grpc_GENERATE_EXTENSIONS .pb.h .pb.cc .grpc.pb.h .grpc.pb.cc) + elseif(protobuf_generate_grpc_LANGUAGE STREQUAL python) + set(protobuf_generate_grpc_GENERATE_EXTENSIONS _pb2.py) + else() + message(SEND_ERROR "Error: protobuf_generate_grpc given unknown Language ${LANGUAGE}, please provide a value for GENERATE_EXTENSIONS") + return() + endif() + endif() + + if(NOT protobuf_generate_grpc_PLUGIN) + if(protobuf_generate_grpc_LANGUAGE STREQUAL cpp) + set(protobuf_generate_grpc_PLUGIN "grpc_cpp_plugin") + elseif(protobuf_generate_grpc_LANGUAGE STREQUAL python) + set(protobuf_generate_grpc_PLUGIN "grpc_python_plugin") + else() + message(SEND_ERROR "Error: protobuf_generate_grpc given unknown Language ${LANGUAGE}, please provide a value for PLUGIN") + return() + endif() + endif() + + if(protobuf_generate_grpc_TARGET) + get_target_property(_source_list ${protobuf_generate_grpc_TARGET} SOURCES) + foreach(_file ${_source_list}) + if(_file MATCHES "proto$") + list(APPEND protobuf_generate_grpc_PROTOS ${_file}) + endif() + endforeach() + endif() + + if(NOT protobuf_generate_grpc_PROTOS) + message(SEND_ERROR "Error: protobuf_generate_grpc could not find any .proto files") + return() + endif() + + if(protobuf_generate_grpc_APPEND_PATH) + # Create an include path for each file specified + foreach(_file ${protobuf_generate_grpc_PROTOS}) + get_filename_component(_abs_file ${_file} ABSOLUTE) + get_filename_component(_abs_path ${_abs_file} PATH) + list(FIND _protobuf_include_path ${_abs_path} _contains_already) + if(${_contains_already} EQUAL -1) + list(APPEND _protobuf_include_path -I ${_abs_path}) + endif() + endforeach() + else() + set(_protobuf_include_path -I ${CMAKE_CURRENT_SOURCE_DIR}) + endif() + + foreach(DIR ${protobuf_generate_grpc_IMPORT_DIRS}) + get_filename_component(ABS_PATH ${DIR} ABSOLUTE) + list(FIND _protobuf_include_path ${ABS_PATH} _contains_already) + if(${_contains_already} EQUAL -1) + list(APPEND _protobuf_include_path -I ${ABS_PATH}) + endif() + endforeach() + + set(_generated_srcs_all) + foreach(_proto ${protobuf_generate_grpc_PROTOS}) + get_filename_component(_abs_file ${_proto} ABSOLUTE) + get_filename_component(_abs_dir ${_abs_file} DIRECTORY) + get_filename_component(_basename ${_proto} NAME_WE) + file(RELATIVE_PATH _rel_dir ${CMAKE_CURRENT_SOURCE_DIR} ${_abs_dir}) + + set(_possible_rel_dir) + if(NOT protobuf_generate_grpc_APPEND_PATH) + set(_possible_rel_dir ${_rel_dir}/) + endif() + + set(_generated_srcs) + foreach(_ext ${protobuf_generate_grpc_GENERATE_EXTENSIONS}) + list(APPEND _generated_srcs "${protobuf_generate_grpc_PROTOC_OUT_DIR}/${_possible_rel_dir}${_basename}${_ext}") + endforeach() + + if(protobuf_generate_grpc_DESCRIPTORS AND protobuf_generate_grpc_LANGUAGE STREQUAL cpp) + set(_descriptor_file "${CMAKE_CURRENT_BINARY_DIR}/${_basename}.desc") + set(_dll_desc_out "--descriptor_set_out=${_descriptor_file}") + list(APPEND _generated_srcs ${_descriptor_file}) + endif() + list(APPEND _generated_srcs_all ${_generated_srcs}) + + add_custom_command( + OUTPUT ${_generated_srcs} + COMMAND protobuf::protoc + ARGS --${protobuf_generate_grpc_LANGUAGE}_out ${_dll_export_decl}${protobuf_generate_grpc_PROTOC_OUT_DIR} + --grpc_out ${_dll_export_decl}${protobuf_generate_grpc_PROTOC_OUT_DIR} + --plugin=protoc-gen-grpc=$ + ${_dll_desc_out} ${_protobuf_include_path} ${_abs_file} + DEPENDS ${_abs_file} protobuf::protoc ${protobuf_generate_grpc_PLUGIN} + COMMENT "Running ${protobuf_generate_grpc_LANGUAGE} protocol buffer compiler on ${_proto}" + VERBATIM) + endforeach() + + set_source_files_properties(${_generated_srcs_all} PROPERTIES GENERATED TRUE) + if(protobuf_generate_grpc_OUT_VAR) + set(${protobuf_generate_grpc_OUT_VAR} ${_generated_srcs_all} PARENT_SCOPE) + endif() + if(protobuf_generate_grpc_TARGET) + target_sources(${protobuf_generate_grpc_TARGET} PRIVATE ${_generated_srcs_all}) + endif() +endfunction() \ No newline at end of file diff --git a/docker/packager/unbundled/Dockerfile b/docker/packager/unbundled/Dockerfile index 4e2b6e3f2db..604f187cc58 100644 --- a/docker/packager/unbundled/Dockerfile +++ b/docker/packager/unbundled/Dockerfile @@ -51,6 +51,7 @@ RUN apt-get update \ protobuf-compiler \ libprotoc-dev \ libgrpc++-dev \ + protobuf-compiler-grpc \ rapidjson-dev \ libsnappy-dev \ libparquet-dev \ From 677a7e485f511314cd7bfbd349cf6d24ccb2be37 Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 2 Oct 2020 09:57:13 +0300 Subject: [PATCH 272/321] Don't add memcpy for sanitizers builds --- CMakeLists.txt | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/CMakeLists.txt b/CMakeLists.txt index 56277a1126d..aea5e0617a4 100644 --- a/CMakeLists.txt +++ b/CMakeLists.txt @@ -513,12 +513,13 @@ endif () macro (add_executable target) # invoke built-in add_executable # explicitly acquire and interpose malloc symbols by clickhouse_malloc - # if GLIBC_COMPATIBILITY is ON, also provide memcpy symbol explicitly to neutrialize thinlto's libcall generation. - if (GLIBC_COMPATIBILITY) + # if GLIBC_COMPATIBILITY is ON and not sanitizer build, provide memcpy symbol explicitly to neutrialize thinlto's libcall generation. + if (GLIBC_COMPATIBILITY AND NOT SANITIZE) _add_executable (${ARGV} $ $) - else (GLIBC_COMPATIBILITY) + else () _add_executable (${ARGV} $) endif () + get_target_property (type ${target} TYPE) if (${type} STREQUAL EXECUTABLE) # operator::new/delete for executables (MemoryTracker stuff) From 56141a1e6b49e2a48af28135a2f0fa76d38a7d56 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 2 Oct 2020 12:13:26 +0300 Subject: [PATCH 273/321] Update ActionsVisitor.cpp --- src/Interpreters/ActionsVisitor.cpp | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/src/Interpreters/ActionsVisitor.cpp b/src/Interpreters/ActionsVisitor.cpp index a107c29717b..b54ac0eb488 100644 --- a/src/Interpreters/ActionsVisitor.cpp +++ b/src/Interpreters/ActionsVisitor.cpp @@ -509,10 +509,10 @@ void ActionsMatcher::visit(const ASTIdentifier & identifier, const ASTPtr & ast, { /// The requested column is not in the block. /// If such a column exists in the table, then the user probably forgot to surround it with an aggregate function or add it to GROUP BY. - - for (const auto & column_name_type : data.source_columns) + + for (const auto & column_name_type : data.source_columns) { - if (column_name_type.name == column_name.get(ast)) + if (column_name_type.name == column_name.get(ast)) { throw Exception("Column " + backQuote(column_name.get(ast)) + " is not under aggregate function and not in GROUP BY", ErrorCodes::NOT_AN_AGGREGATE); From bde65557f17a767430b95aaabc31f9d1ba72be7c Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 2 Oct 2020 12:40:50 +0300 Subject: [PATCH 274/321] Fix bash --- docker/packager/deb/build.sh | 3 ++- docker/packager/unbundled/build.sh | 3 ++- 2 files changed, 4 insertions(+), 2 deletions(-) diff --git a/docker/packager/deb/build.sh b/docker/packager/deb/build.sh index a8072c5e78a..6450e21d289 100755 --- a/docker/packager/deb/build.sh +++ b/docker/packager/deb/build.sh @@ -4,7 +4,8 @@ set -x -e ccache --show-stats ||: ccache --zero-stats ||: -build/release --no-pbuilder "$ALIEN_PKGS" | ts '%Y-%m-%d %H:%M:%S' +read -ra ALIEN_PKGS <<< "${ALIEN_PKGS:-}" +build/release --no-pbuilder "${ALIEN_PKGS[@]}" | ts '%Y-%m-%d %H:%M:%S' mv /*.deb /output mv -- *.changes /output mv -- *.buildinfo /output diff --git a/docker/packager/unbundled/build.sh b/docker/packager/unbundled/build.sh index c94bd54d001..54575ab977c 100755 --- a/docker/packager/unbundled/build.sh +++ b/docker/packager/unbundled/build.sh @@ -4,7 +4,8 @@ set -x -e ccache --show-stats ||: ccache --zero-stats ||: -build/release --no-pbuilder "$ALIEN_PKGS" | ts '%Y-%m-%d %H:%M:%S' +read -ra ALIEN_PKGS <<< "${ALIEN_PKGS:-}" +build/release --no-pbuilder "${ALIEN_PKGS[@]}" | ts '%Y-%m-%d %H:%M:%S' mv /*.deb /output mv -- *.changes /output mv -- *.buildinfo /output From 3be7c7745e2e455478489b5970762ada4d434d62 Mon Sep 17 00:00:00 2001 From: Alexander Kuzmenkov Date: Fri, 2 Oct 2020 14:06:38 +0300 Subject: [PATCH 275/321] Use a separate overload for formatting with fmt --- src/Common/Exception.h | 21 +++++++++++++++------ 1 file changed, 15 insertions(+), 6 deletions(-) diff --git a/src/Common/Exception.h b/src/Common/Exception.h index 09d923fe60e..314c59cbf51 100644 --- a/src/Common/Exception.h +++ b/src/Common/Exception.h @@ -22,10 +22,14 @@ public: Exception() = default; Exception(const std::string & msg, int code); + Exception(int code, const std::string & message) + : Exception(message, code) + {} + // Format message with fmt::format, like the logging functions. - template - Exception(int code, Fmt&&... fmt) - : Exception(fmt::format(std::forward(fmt)...), code) + template + Exception(int code, const std::string & fmt, Args&&... args) + : Exception(fmt::format(fmt, std::forward(args)...), code) {} struct CreateFromPocoTag {}; @@ -40,10 +44,15 @@ public: const char * what() const throw() override { return message().data(); } /// Add something to the existing message. - template - void addMessage(Fmt&&... fmt) + template + void addMessage(const std::string& format, Args&&... args) { - extendedMessage(fmt::format(std::forward(fmt)...)); + extendedMessage(fmt::format(format, std::forward(args)...)); + } + + void addMessage(const std::string& message) + { + extendedMessage(message); } std::string getStackTraceString() const; From 17b86dec9678e8bdabd2f83bc2367758d8aa1b12 Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 2 Oct 2020 15:38:50 +0300 Subject: [PATCH 276/321] Recursive defaults --- programs/client/Client.cpp | 5 +- .../AddingDefaultBlockOutputStream.cpp | 2 +- .../AddingDefaultBlockOutputStream.h | 8 +- .../AddingDefaultsBlockInputStream.cpp | 14 ++-- .../AddingDefaultsBlockInputStream.h | 5 +- .../InputStreamFromASTInsertQuery.cpp | 6 +- .../PushingToViewsBlockOutputStream.cpp | 1 - src/Interpreters/InterpreterInsertQuery.cpp | 2 +- src/Interpreters/addMissingDefaults.cpp | 17 +++-- src/Interpreters/addMissingDefaults.h | 4 +- src/Interpreters/inplaceBlockConversions.cpp | 75 ++++++++++--------- src/Interpreters/inplaceBlockConversions.h | 6 +- .../Transforms/AddingMissedTransform.cpp | 6 +- .../Transforms/AddingMissedTransform.h | 6 +- src/Storages/ColumnsDescription.cpp | 8 ++ src/Storages/ColumnsDescription.h | 1 + src/Storages/MergeTree/IMergeTreeReader.cpp | 2 +- src/Storages/StorageBuffer.cpp | 2 +- src/Storages/StorageFile.cpp | 12 +-- src/Storages/StorageS3.cpp | 8 +- src/Storages/StorageURL.cpp | 6 +- 21 files changed, 104 insertions(+), 92 deletions(-) diff --git a/programs/client/Client.cpp b/programs/client/Client.cpp index d900eb17d78..df365a2089e 100644 --- a/programs/client/Client.cpp +++ b/programs/client/Client.cpp @@ -1555,9 +1555,8 @@ private: BlockInputStreamPtr block_input = context.getInputFormat( current_format, buf, sample, insert_format_max_block_size); - const auto & column_defaults = columns_description.getDefaults(); - if (!column_defaults.empty()) - block_input = std::make_shared(block_input, column_defaults, context); + if (columns_description.hasDefaults()) + block_input = std::make_shared(block_input, columns_description, context); BlockInputStreamPtr async_block_input = std::make_shared(block_input); diff --git a/src/DataStreams/AddingDefaultBlockOutputStream.cpp b/src/DataStreams/AddingDefaultBlockOutputStream.cpp index a8f78c434e7..74300a371fb 100644 --- a/src/DataStreams/AddingDefaultBlockOutputStream.cpp +++ b/src/DataStreams/AddingDefaultBlockOutputStream.cpp @@ -7,7 +7,7 @@ namespace DB void AddingDefaultBlockOutputStream::write(const Block & block) { - output->write(addMissingDefaults(block, output_block.getNamesAndTypesList(), column_defaults, context)); + output->write(addMissingDefaults(block, output_block.getNamesAndTypesList(), columns, context)); } void AddingDefaultBlockOutputStream::flush() diff --git a/src/DataStreams/AddingDefaultBlockOutputStream.h b/src/DataStreams/AddingDefaultBlockOutputStream.h index 89134920730..5b46c533f7f 100644 --- a/src/DataStreams/AddingDefaultBlockOutputStream.h +++ b/src/DataStreams/AddingDefaultBlockOutputStream.h @@ -2,7 +2,7 @@ #include #include -#include +#include namespace DB @@ -23,10 +23,10 @@ public: const BlockOutputStreamPtr & output_, const Block & header_, const Block & output_block_, - const ColumnDefaults & column_defaults_, + const ColumnsDescription & columns_, const Context & context_) : output(output_), header(header_), output_block(output_block_), - column_defaults(column_defaults_), context(context_) + columns(columns_), context(context_) { } @@ -43,7 +43,7 @@ private: const Block header; /// Blocks after this stream should have this structure const Block output_block; - const ColumnDefaults column_defaults; + const ColumnsDescription columns; const Context & context; }; diff --git a/src/DataStreams/AddingDefaultsBlockInputStream.cpp b/src/DataStreams/AddingDefaultsBlockInputStream.cpp index e958b579074..160d1b4fb76 100644 --- a/src/DataStreams/AddingDefaultsBlockInputStream.cpp +++ b/src/DataStreams/AddingDefaultsBlockInputStream.cpp @@ -127,11 +127,13 @@ static MutableColumnPtr mixColumns(const ColumnWithTypeAndName & col_read, } -AddingDefaultsBlockInputStream::AddingDefaultsBlockInputStream(const BlockInputStreamPtr & input, - const ColumnDefaults & column_defaults_, - const Context & context_) - : column_defaults(column_defaults_), - context(context_) +AddingDefaultsBlockInputStream::AddingDefaultsBlockInputStream( + const BlockInputStreamPtr & input, + const ColumnsDescription & columns_, + const Context & context_) + : columns(columns_) + , column_defaults(columns.getDefaults()) + , context(context_) { children.push_back(input); header = input->getHeader(); @@ -169,7 +171,7 @@ Block AddingDefaultsBlockInputStream::readImpl() if (!evaluate_block.columns()) evaluate_block.insert({ColumnConst::create(ColumnUInt8::create(1, 0), res.rows()), std::make_shared(), "_dummy"}); - evaluateMissingDefaults(evaluate_block, header.getNamesAndTypesList(), column_defaults, context, false); + evaluateMissingDefaults(evaluate_block, header.getNamesAndTypesList(), columns, context, false); std::unordered_map mixed_columns; diff --git a/src/DataStreams/AddingDefaultsBlockInputStream.h b/src/DataStreams/AddingDefaultsBlockInputStream.h index 10fa20f7ad1..b2efa5863ef 100644 --- a/src/DataStreams/AddingDefaultsBlockInputStream.h +++ b/src/DataStreams/AddingDefaultsBlockInputStream.h @@ -1,7 +1,7 @@ #pragma once #include -#include +#include namespace DB @@ -15,7 +15,7 @@ class AddingDefaultsBlockInputStream : public IBlockInputStream public: AddingDefaultsBlockInputStream( const BlockInputStreamPtr & input, - const ColumnDefaults & column_defaults_, + const ColumnsDescription & columns_, const Context & context_); String getName() const override { return "AddingDefaults"; } @@ -26,6 +26,7 @@ protected: private: Block header; + const ColumnsDescription columns; const ColumnDefaults column_defaults; const Context & context; }; diff --git a/src/DataStreams/InputStreamFromASTInsertQuery.cpp b/src/DataStreams/InputStreamFromASTInsertQuery.cpp index 19c6fe41eca..70d69227ac0 100644 --- a/src/DataStreams/InputStreamFromASTInsertQuery.cpp +++ b/src/DataStreams/InputStreamFromASTInsertQuery.cpp @@ -64,9 +64,9 @@ InputStreamFromASTInsertQuery::InputStreamFromASTInsertQuery( { StoragePtr storage = DatabaseCatalog::instance().getTable(ast_insert_query->table_id, context); auto metadata_snapshot = storage->getInMemoryMetadataPtr(); - auto column_defaults = metadata_snapshot->getColumns().getDefaults(); - if (!column_defaults.empty()) - res_stream = std::make_shared(res_stream, column_defaults, context); + const auto & columns = metadata_snapshot->getColumns(); + if (columns.hasDefaults()) + res_stream = std::make_shared(res_stream, columns, context); } } diff --git a/src/DataStreams/PushingToViewsBlockOutputStream.cpp b/src/DataStreams/PushingToViewsBlockOutputStream.cpp index 12ee22b20bb..dc28d59b532 100644 --- a/src/DataStreams/PushingToViewsBlockOutputStream.cpp +++ b/src/DataStreams/PushingToViewsBlockOutputStream.cpp @@ -1,4 +1,3 @@ -#include #include #include #include diff --git a/src/Interpreters/InterpreterInsertQuery.cpp b/src/Interpreters/InterpreterInsertQuery.cpp index 5985a7fb7d6..05707b66c0b 100644 --- a/src/Interpreters/InterpreterInsertQuery.cpp +++ b/src/Interpreters/InterpreterInsertQuery.cpp @@ -348,7 +348,7 @@ BlockIO InterpreterInsertQuery::execute() /// Actually we don't know structure of input blocks from query/table, /// because some clients break insertion protocol (columns != header) out = std::make_shared( - out, query_sample_block, out->getHeader(), metadata_snapshot->getColumns().getDefaults(), context); + out, query_sample_block, out->getHeader(), metadata_snapshot->getColumns(), context); /// It's important to squash blocks as early as possible (before other transforms), /// because other transforms may work inefficient if block size is small. diff --git a/src/Interpreters/addMissingDefaults.cpp b/src/Interpreters/addMissingDefaults.cpp index cbe6811ea9e..37a0812826b 100644 --- a/src/Interpreters/addMissingDefaults.cpp +++ b/src/Interpreters/addMissingDefaults.cpp @@ -6,16 +6,17 @@ #include #include #include -#include +#include namespace DB { -Block addMissingDefaults(const Block & block, - const NamesAndTypesList & required_columns, - const ColumnDefaults & column_defaults, - const Context & context) +Block addMissingDefaults( + const Block & block, + const NamesAndTypesList & required_columns, + const ColumnsDescription & columns, + const Context & context) { /// For missing columns of nested structure, you need to create not a column of empty arrays, but a column of arrays of correct lengths. /// First, remember the offset columns for all arrays in the block. @@ -49,7 +50,7 @@ Block addMissingDefaults(const Block & block, continue; } - if (column_defaults.count(column.name)) + if (columns.hasDefault(column.name)) continue; String offsets_name = Nested::extractTableName(column.name); @@ -72,8 +73,8 @@ Block addMissingDefaults(const Block & block, res.insert(ColumnWithTypeAndName(std::move(new_column), column.type, column.name)); } - /// Computes explicitly specified values (in column_defaults) by default and materialized columns. - evaluateMissingDefaults(res, required_columns, column_defaults, context); + /// Computes explicitly specified values by default and materialized columns. + evaluateMissingDefaults(res, required_columns, columns, context); return res; } diff --git a/src/Interpreters/addMissingDefaults.h b/src/Interpreters/addMissingDefaults.h index cc84f413b16..ed5d5ce61ff 100644 --- a/src/Interpreters/addMissingDefaults.h +++ b/src/Interpreters/addMissingDefaults.h @@ -10,7 +10,7 @@ namespace DB class Block; class Context; class NamesAndTypesList; -struct ColumnDefault; +class ColumnsDescription; /** Adds three types of columns into block * 1. Columns, that are missed inside request, but present in table without defaults (missed columns) @@ -21,7 +21,7 @@ struct ColumnDefault; Block addMissingDefaults( const Block & block, const NamesAndTypesList & required_columns, - const std::unordered_map & column_defaults, + const ColumnsDescription & columns, const Context & context); } diff --git a/src/Interpreters/inplaceBlockConversions.cpp b/src/Interpreters/inplaceBlockConversions.cpp index 19495b0ffed..00ddc5b9b32 100644 --- a/src/Interpreters/inplaceBlockConversions.cpp +++ b/src/Interpreters/inplaceBlockConversions.cpp @@ -2,7 +2,6 @@ #include #include -#include #include #include #include @@ -14,6 +13,8 @@ #include #include #include +#include +#include namespace DB @@ -22,46 +23,46 @@ namespace DB namespace { -ASTPtr defaultRequiredExpressions(Block & block, const NamesAndTypesList & required_columns, const ColumnDefaults & column_defaults) +void addDefaultRequiredExpressionsRecursively(Block & block, const String & required_column, const ColumnsDescription & columns, ASTPtr default_expr_list_accum, NameSet & added_columns) +{ + checkStackSize(); + if (block.has(required_column) || added_columns.count(required_column)) + return; + + auto column_default = columns.getDefault(required_column); + + if (column_default) + { + /// expressions must be cloned to prevent modification by the ExpressionAnalyzer + auto column_default_expr = column_default->expression->clone(); + + /// Our default may depend on columns with ALIAS as default expr which not present in block + /// we can easily add them from column_defaults struct + RequiredSourceColumnsVisitor::Data columns_context; + RequiredSourceColumnsVisitor(columns_context).visit(column_default_expr); + NameSet required_columns_names = columns_context.requiredColumns(); + + auto cast_func = makeASTFunction("CAST", column_default_expr, std::make_shared(columns.get(required_column).type->getName())); + default_expr_list_accum->children.emplace_back(setAlias(cast_func, required_column)); + added_columns.emplace(required_column); + + for (const auto & required_column_name : required_columns_names) + addDefaultRequiredExpressionsRecursively(block, required_column_name, columns, default_expr_list_accum, added_columns); + } + +} + +ASTPtr defaultRequiredExpressions(Block & block, const NamesAndTypesList & required_columns, const ColumnsDescription & columns) { ASTPtr default_expr_list = std::make_shared(); + NameSet added_columns; for (const auto & column : required_columns) - { - if (block.has(column.name)) - continue; - - const auto it = column_defaults.find(column.name); - - if (it != column_defaults.end()) - { - /// expressions must be cloned to prevent modification by the ExpressionAnalyzer - auto column_default_expr = it->second.expression->clone(); - - /// Our default may depend on columns with ALIAS as default expr which not present in block - /// we can easily add them from column_defaults struct - RequiredSourceColumnsVisitor::Data columns_context; - RequiredSourceColumnsVisitor(columns_context).visit(column_default_expr); - NameSet required_columns_names = columns_context.requiredColumns(); - - for (const auto & required_column_name : required_columns_names) - { - /// If we have such default column and it's alias than we should - /// add it into default_expression_list - if (auto rit = column_defaults.find(required_column_name); - rit != column_defaults.end() && rit->second.kind == ColumnDefaultKind::Alias) - { - default_expr_list->children.emplace_back(setAlias(rit->second.expression->clone(), required_column_name)); - } - } - - auto cast_func = makeASTFunction("CAST", column_default_expr, std::make_shared(column.type->getName())); - default_expr_list->children.emplace_back(setAlias(cast_func, it->first)); - } - } + addDefaultRequiredExpressionsRecursively(block, column.name, columns, default_expr_list, added_columns); if (default_expr_list->children.empty()) return nullptr; + return default_expr_list; } @@ -161,13 +162,13 @@ void performRequiredConversions(Block & block, const NamesAndTypesList & require void evaluateMissingDefaults(Block & block, const NamesAndTypesList & required_columns, - const ColumnDefaults & column_defaults, + const ColumnsDescription & columns, const Context & context, bool save_unneeded_columns) { - if (column_defaults.empty()) + if (!columns.hasDefaults()) return; - ASTPtr default_expr_list = defaultRequiredExpressions(block, required_columns, column_defaults); + ASTPtr default_expr_list = defaultRequiredExpressions(block, required_columns, columns); executeExpressionsOnBlock(block, default_expr_list, save_unneeded_columns, required_columns, context); } diff --git a/src/Interpreters/inplaceBlockConversions.h b/src/Interpreters/inplaceBlockConversions.h index b86a23bde13..066975ab4bc 100644 --- a/src/Interpreters/inplaceBlockConversions.h +++ b/src/Interpreters/inplaceBlockConversions.h @@ -10,13 +10,13 @@ namespace DB class Block; class Context; class NamesAndTypesList; -struct ColumnDefault; +class ColumnsDescription; /// Adds missing defaults to block according to required_columns -/// using column_defaults map +/// using columns description void evaluateMissingDefaults(Block & block, const NamesAndTypesList & required_columns, - const std::unordered_map & column_defaults, + const ColumnsDescription & columns, const Context & context, bool save_unneeded_columns = true); /// Tries to convert columns in block to required_columns diff --git a/src/Processors/Transforms/AddingMissedTransform.cpp b/src/Processors/Transforms/AddingMissedTransform.cpp index 75823fc8a63..1344cce22a7 100644 --- a/src/Processors/Transforms/AddingMissedTransform.cpp +++ b/src/Processors/Transforms/AddingMissedTransform.cpp @@ -8,10 +8,10 @@ namespace DB AddingMissedTransform::AddingMissedTransform( Block header_, Block result_header_, - const ColumnDefaults & column_defaults_, + const ColumnsDescription & columns_, const Context & context_) : ISimpleTransform(std::move(header_), std::move(result_header_), false) - , column_defaults(column_defaults_), context(context_) + , columns(columns_), context(context_) { } @@ -20,7 +20,7 @@ void AddingMissedTransform::transform(Chunk & chunk) auto num_rows = chunk.getNumRows(); Block src = getInputPort().getHeader().cloneWithColumns(chunk.detachColumns()); - auto res = addMissingDefaults(src, getOutputPort().getHeader().getNamesAndTypesList(), column_defaults, context); + auto res = addMissingDefaults(src, getOutputPort().getHeader().getNamesAndTypesList(), columns, context); chunk.setColumns(res.getColumns(), num_rows); } diff --git a/src/Processors/Transforms/AddingMissedTransform.h b/src/Processors/Transforms/AddingMissedTransform.h index b9baa1622ef..561f908acef 100644 --- a/src/Processors/Transforms/AddingMissedTransform.h +++ b/src/Processors/Transforms/AddingMissedTransform.h @@ -1,7 +1,7 @@ #pragma once #include -#include +#include namespace DB @@ -20,7 +20,7 @@ public: AddingMissedTransform( Block header_, Block result_header_, - const ColumnDefaults & column_defaults_, + const ColumnsDescription & columns_, const Context & context_); String getName() const override { return "AddingMissed"; } @@ -28,7 +28,7 @@ public: private: void transform(Chunk &) override; - const ColumnDefaults column_defaults; + const ColumnsDescription columns; const Context & context; }; diff --git a/src/Storages/ColumnsDescription.cpp b/src/Storages/ColumnsDescription.cpp index 48cde6b6aa9..9bbc1653848 100644 --- a/src/Storages/ColumnsDescription.cpp +++ b/src/Storages/ColumnsDescription.cpp @@ -378,6 +378,14 @@ bool ColumnsDescription::hasPhysical(const String & column_name) const } +bool ColumnsDescription::hasDefaults() const +{ + for (const auto & column : columns) + if (column.default_desc.expression) + return true; + return false; +} + ColumnDefaults ColumnsDescription::getDefaults() const { ColumnDefaults ret; diff --git a/src/Storages/ColumnsDescription.h b/src/Storages/ColumnsDescription.h index 8d1009b7263..1df6c6ae67b 100644 --- a/src/Storages/ColumnsDescription.h +++ b/src/Storages/ColumnsDescription.h @@ -109,6 +109,7 @@ public: ColumnDefaults getDefaults() const; /// TODO: remove bool hasDefault(const String & column_name) const; + bool hasDefaults() const; std::optional getDefault(const String & column_name) const; /// Does column has non default specified compression codec diff --git a/src/Storages/MergeTree/IMergeTreeReader.cpp b/src/Storages/MergeTree/IMergeTreeReader.cpp index 5740eef6810..4379ac09af0 100644 --- a/src/Storages/MergeTree/IMergeTreeReader.cpp +++ b/src/Storages/MergeTree/IMergeTreeReader.cpp @@ -180,7 +180,7 @@ void IMergeTreeReader::evaluateMissingDefaults(Block additional_columns, Columns additional_columns.insert({res_columns[pos], name_and_type->type, name_and_type->name}); } - DB::evaluateMissingDefaults(additional_columns, columns, metadata_snapshot->getColumns().getDefaults(), storage.global_context); + DB::evaluateMissingDefaults(additional_columns, columns, metadata_snapshot->getColumns(), storage.global_context); /// Move columns from block. name_and_type = columns.begin(); diff --git a/src/Storages/StorageBuffer.cpp b/src/Storages/StorageBuffer.cpp index 14f188275e5..4b038692b77 100644 --- a/src/Storages/StorageBuffer.cpp +++ b/src/Storages/StorageBuffer.cpp @@ -224,7 +224,7 @@ Pipe StorageBuffer::read( pipe_from_dst.addSimpleTransform([&](const Block & stream_header) { return std::make_shared(stream_header, header_after_adding_defaults, - metadata_snapshot->getColumns().getDefaults(), context); + metadata_snapshot->getColumns(), context); }); pipe_from_dst.addSimpleTransform([&](const Block & stream_header) diff --git a/src/Storages/StorageFile.cpp b/src/Storages/StorageFile.cpp index cc47047dc78..8f7e1563a62 100644 --- a/src/Storages/StorageFile.cpp +++ b/src/Storages/StorageFile.cpp @@ -234,12 +234,12 @@ public: const Context & context_, UInt64 max_block_size_, FilesInfoPtr files_info_, - ColumnDefaults column_defaults_) + ColumnsDescription columns_description_) : SourceWithProgress(getHeader(metadata_snapshot_, files_info_->need_path_column, files_info_->need_file_column)) , storage(std::move(storage_)) , metadata_snapshot(metadata_snapshot_) , files_info(std::move(files_info_)) - , column_defaults(std::move(column_defaults_)) + , columns_description(std::move(columns_description_)) , context(context_) , max_block_size(max_block_size_) { @@ -314,8 +314,8 @@ public: reader = FormatFactory::instance().getInput( storage->format_name, *read_buf, metadata_snapshot->getSampleBlock(), context, max_block_size); - if (!column_defaults.empty()) - reader = std::make_shared(reader, column_defaults, context); + if (columns_description.hasDefaults()) + reader = std::make_shared(reader, columns_description, context); reader->readPrefix(); } @@ -366,7 +366,7 @@ private: std::unique_ptr read_buf; BlockInputStreamPtr reader; - ColumnDefaults column_defaults; + ColumnsDescription columns_description; const Context & context; /// TODO Untangle potential issues with context lifetime. UInt64 max_block_size; @@ -417,7 +417,7 @@ Pipe StorageFile::read( for (size_t i = 0; i < num_streams; ++i) pipes.emplace_back(std::make_shared( - this_ptr, metadata_snapshot, context, max_block_size, files_info, metadata_snapshot->getColumns().getDefaults())); + this_ptr, metadata_snapshot, context, max_block_size, files_info, metadata_snapshot->getColumns())); return Pipe::unitePipes(std::move(pipes)); } diff --git a/src/Storages/StorageS3.cpp b/src/Storages/StorageS3.cpp index 469350d6eaa..ceca1405857 100644 --- a/src/Storages/StorageS3.cpp +++ b/src/Storages/StorageS3.cpp @@ -71,7 +71,7 @@ namespace String name_, const Block & sample_block, const Context & context, - const ColumnDefaults & column_defaults, + const ColumnsDescription & columns, UInt64 max_block_size, const CompressionMethod compression_method, const std::shared_ptr & client, @@ -86,8 +86,8 @@ namespace read_buf = wrapReadBufferWithCompressionMethod(std::make_unique(client, bucket, key), compression_method); reader = FormatFactory::instance().getInput(format, *read_buf, sample_block, context, max_block_size); - if (!column_defaults.empty()) - reader = std::make_shared(reader, column_defaults, context); + if (columns.hasDefaults()) + reader = std::make_shared(reader, columns, context); } String getName() const override @@ -312,7 +312,7 @@ Pipe StorageS3::read( getName(), metadata_snapshot->getSampleBlock(), context, - metadata_snapshot->getColumns().getDefaults(), + metadata_snapshot->getColumns(), max_block_size, chooseCompressionMethod(uri.endpoint, compression_method), client, diff --git a/src/Storages/StorageURL.cpp b/src/Storages/StorageURL.cpp index c2f7bfd18d2..a6c8f1b39f8 100644 --- a/src/Storages/StorageURL.cpp +++ b/src/Storages/StorageURL.cpp @@ -61,7 +61,7 @@ namespace String name_, const Block & sample_block, const Context & context, - const ColumnDefaults & column_defaults, + const ColumnsDescription & columns, UInt64 max_block_size, const ConnectionTimeouts & timeouts, const CompressionMethod compression_method) @@ -81,7 +81,7 @@ namespace compression_method); reader = FormatFactory::instance().getInput(format, *read_buf, sample_block, context, max_block_size); - reader = std::make_shared(reader, column_defaults, context); + reader = std::make_shared(reader, columns, context); } String getName() const override @@ -201,7 +201,7 @@ Pipe IStorageURLBase::read( getName(), getHeaderBlock(column_names, metadata_snapshot), context, - metadata_snapshot->getColumns().getDefaults(), + metadata_snapshot->getColumns(), max_block_size, ConnectionTimeouts::getHTTPTimeouts(context), chooseCompressionMethod(request_uri.getPath(), compression_method))); From 7d41afa342b7a279e27d44439d2001e60347d935 Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 2 Oct 2020 15:51:01 +0300 Subject: [PATCH 277/321] More defaults on defaults --- ...3_defaults_on_defaults_no_column.reference | 7 ++++ .../01513_defaults_on_defaults_no_column.sql | 36 +++++++++++++++++++ 2 files changed, 43 insertions(+) create mode 100644 tests/queries/0_stateless/01513_defaults_on_defaults_no_column.reference create mode 100644 tests/queries/0_stateless/01513_defaults_on_defaults_no_column.sql diff --git a/tests/queries/0_stateless/01513_defaults_on_defaults_no_column.reference b/tests/queries/0_stateless/01513_defaults_on_defaults_no_column.reference new file mode 100644 index 00000000000..10eb384579e --- /dev/null +++ b/tests/queries/0_stateless/01513_defaults_on_defaults_no_column.reference @@ -0,0 +1,7 @@ +1 +1 +1 +1 +1 [] [] [] 0 +CREATE TABLE default.defaults_on_defaults\n(\n `key` UInt64,\n `Arr.C1` Array(UInt32) DEFAULT emptyArrayUInt32(),\n `Arr.C2` Array(UInt32) DEFAULT arrayResize(emptyArrayUInt32(), length(Arr.C1)),\n `Arr.C3` Array(UInt32) ALIAS arrayResize(emptyArrayUInt32(), length(Arr.C2)),\n `Arr.C4` Array(UInt32) DEFAULT arrayResize(emptyArrayUInt32(), length(Arr.C3)),\n `ArrLen` UInt64 DEFAULT length(Arr.C4)\n)\nENGINE = MergeTree()\nORDER BY tuple()\nSETTINGS index_granularity = 8192 +1 diff --git a/tests/queries/0_stateless/01513_defaults_on_defaults_no_column.sql b/tests/queries/0_stateless/01513_defaults_on_defaults_no_column.sql new file mode 100644 index 00000000000..65b7b99f70b --- /dev/null +++ b/tests/queries/0_stateless/01513_defaults_on_defaults_no_column.sql @@ -0,0 +1,36 @@ +DROP TABLE IF EXISTS defaults_on_defaults; +CREATE TABLE defaults_on_defaults ( + key UInt64 +) +ENGINE = MergeTree() +ORDER BY tuple(); + +INSERT INTO defaults_on_defaults values (1); + +ALTER TABLE defaults_on_defaults ADD COLUMN `Arr.C1` Array(UInt32) DEFAULT emptyArrayUInt32(); + +ALTER TABLE defaults_on_defaults ADD COLUMN `Arr.C2` Array(UInt32) DEFAULT arrayResize(emptyArrayUInt32(), length(Arr.C1)); + +ALTER TABLE defaults_on_defaults ADD COLUMN `Arr.C3` Array(UInt32) ALIAS arrayResize(emptyArrayUInt32(), length(Arr.C2)); + +SELECT 1 from defaults_on_defaults where length(`Arr.C2`) = 0; + +SELECT 1 from defaults_on_defaults where length(`Arr.C3`) = 0; + +ALTER TABLE defaults_on_defaults ADD COLUMN `Arr.C4` Array(UInt32) DEFAULT arrayResize(emptyArrayUInt32(), length(Arr.C3)); + +SELECT 1 from defaults_on_defaults where length(`Arr.C4`) = 0; + +ALTER TABLE defaults_on_defaults ADD COLUMN `ArrLen` UInt64 DEFAULT length(Arr.C4); + +SELECT 1 from defaults_on_defaults where ArrLen = 0; + +SELECT * from defaults_on_defaults where ArrLen = 0; + +SHOW CREATE TABLE defaults_on_defaults; + +OPTIMIZE TABLE defaults_on_defaults FINAL; + +SELECT 1 from defaults_on_defaults where length(`Arr.C4`) = 0; + +DROP TABLE IF EXISTS defaults_on_defaults; From ef9728da794d15bd6eae180c11eb136f55b55387 Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 2 Oct 2020 15:54:51 +0300 Subject: [PATCH 278/321] Fix comment --- src/Interpreters/inplaceBlockConversions.cpp | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/src/Interpreters/inplaceBlockConversions.cpp b/src/Interpreters/inplaceBlockConversions.cpp index 00ddc5b9b32..849512adcc1 100644 --- a/src/Interpreters/inplaceBlockConversions.cpp +++ b/src/Interpreters/inplaceBlockConversions.cpp @@ -23,6 +23,7 @@ namespace DB namespace { +/// Add all required expressions for missing columns calculation void addDefaultRequiredExpressionsRecursively(Block & block, const String & required_column, const ColumnsDescription & columns, ASTPtr default_expr_list_accum, NameSet & added_columns) { checkStackSize(); @@ -36,8 +37,8 @@ void addDefaultRequiredExpressionsRecursively(Block & block, const String & requ /// expressions must be cloned to prevent modification by the ExpressionAnalyzer auto column_default_expr = column_default->expression->clone(); - /// Our default may depend on columns with ALIAS as default expr which not present in block - /// we can easily add them from column_defaults struct + /// Our default may depend on columns with default expr which not present in block + /// we have to add them to block too RequiredSourceColumnsVisitor::Data columns_context; RequiredSourceColumnsVisitor(columns_context).visit(column_default_expr); NameSet required_columns_names = columns_context.requiredColumns(); @@ -49,7 +50,6 @@ void addDefaultRequiredExpressionsRecursively(Block & block, const String & requ for (const auto & required_column_name : required_columns_names) addDefaultRequiredExpressionsRecursively(block, required_column_name, columns, default_expr_list_accum, added_columns); } - } ASTPtr defaultRequiredExpressions(Block & block, const NamesAndTypesList & required_columns, const ColumnsDescription & columns) From 21deb6812c49a32bb92568bd9297c4d7bf52d50a Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Fri, 2 Oct 2020 16:46:20 +0300 Subject: [PATCH 279/321] Drop unused code for numeric_limits in MergeTreeDataSelectExecutor (#15519) --- .../MergeTree/MergeTreeDataSelectExecutor.cpp | 22 ------------------- 1 file changed, 22 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp index 8b5b337bcec..9128694c793 100644 --- a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp +++ b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp @@ -21,28 +21,6 @@ #include #include -/// Allow to use __uint128_t as a template parameter for boost::rational. -// https://stackoverflow.com/questions/41198673/uint128-t-not-working-with-clang-and-libstdc -#if 0 -#if !defined(__GLIBCXX_BITSIZE_INT_N_0) && defined(__SIZEOF_INT128__) -namespace std -{ - template <> - struct numeric_limits<__uint128_t> - { - static constexpr bool is_specialized = true; - static constexpr bool is_signed = false; - static constexpr bool is_integer = true; - static constexpr int radix = 2; - static constexpr int digits = 128; - static constexpr int digits10 = 38; - static constexpr __uint128_t min () { return 0; } // used in boost 1.65.1+ - static constexpr __uint128_t max () { return __uint128_t(0) - 1; } // used in boost 1.68.0+ - }; -} -#endif -#endif - #include #include #include From ecdf0ed1a8637e7c29d1f5b96d01432f19954f2d Mon Sep 17 00:00:00 2001 From: Artem Zuikov Date: Fri, 2 Oct 2020 16:47:29 +0300 Subject: [PATCH 280/321] update en and ru docs: add 128/256 types (#15515) --- docs/en/sql-reference/data-types/decimal.md | 7 ++-- docs/en/sql-reference/data-types/int-uint.md | 9 ++++-- .../functions/type-conversion-functions.md | 32 +++++++++++-------- docs/ru/sql-reference/data-types/decimal.md | 7 ++-- docs/ru/sql-reference/data-types/int-uint.md | 23 +++++++------ .../functions/type-conversion-functions.md | 32 +++++++++++-------- 6 files changed, 69 insertions(+), 41 deletions(-) diff --git a/docs/en/sql-reference/data-types/decimal.md b/docs/en/sql-reference/data-types/decimal.md index 7902394e3ee..fe80e735667 100644 --- a/docs/en/sql-reference/data-types/decimal.md +++ b/docs/en/sql-reference/data-types/decimal.md @@ -3,25 +3,27 @@ toc_priority: 42 toc_title: Decimal --- -# Decimal(P, S), Decimal32(S), Decimal64(S), Decimal128(S) {#decimalp-s-decimal32s-decimal64s-decimal128s} +# Decimal(P, S), Decimal32(S), Decimal64(S), Decimal128(S), Decimal256(S) {#decimalp-s-decimal32s-decimal64s-decimal128s} Signed fixed-point numbers that keep precision during add, subtract and multiply operations. For division least significant digits are discarded (not rounded). ## Parameters {#parameters} -- P - precision. Valid range: \[ 1 : 38 \]. Determines how many decimal digits number can have (including fraction). +- P - precision. Valid range: \[ 1 : 76 \]. Determines how many decimal digits number can have (including fraction). - S - scale. Valid range: \[ 0 : P \]. Determines how many decimal digits fraction can have. Depending on P parameter value Decimal(P, S) is a synonym for: - P from \[ 1 : 9 \] - for Decimal32(S) - P from \[ 10 : 18 \] - for Decimal64(S) - P from \[ 19 : 38 \] - for Decimal128(S) +- P from \[ 39 : 76 \] - for Decimal256(S) ## Decimal Value Ranges {#decimal-value-ranges} - Decimal32(S) - ( -1 \* 10^(9 - S), 1 \* 10^(9 - S) ) - Decimal64(S) - ( -1 \* 10^(18 - S), 1 \* 10^(18 - S) ) - Decimal128(S) - ( -1 \* 10^(38 - S), 1 \* 10^(38 - S) ) +- Decimal256(S) - ( -1 \* 10^(76 - S), 1 \* 10^(76 - S) ) For example, Decimal32(4) can contain numbers from -99999.9999 to 99999.9999 with 0.0001 step. @@ -38,6 +40,7 @@ Binary operations on Decimal result in wider result type (with any order of argu - `Decimal64(S1) Decimal32(S2) -> Decimal64(S)` - `Decimal128(S1) Decimal32(S2) -> Decimal128(S)` - `Decimal128(S1) Decimal64(S2) -> Decimal128(S)` +- `Decimal256(S1) Decimal<32|64|128>(S2) -> Decimal256(S)` Rules for scale: diff --git a/docs/en/sql-reference/data-types/int-uint.md b/docs/en/sql-reference/data-types/int-uint.md index 84ef120af7d..4c2377a634b 100644 --- a/docs/en/sql-reference/data-types/int-uint.md +++ b/docs/en/sql-reference/data-types/int-uint.md @@ -1,9 +1,9 @@ --- toc_priority: 40 -toc_title: UInt8, UInt16, UInt32, UInt64, Int8, Int16, Int32, Int64 +toc_title: UInt8, UInt16, UInt32, UInt64, UInt256, Int8, Int16, Int32, Int64, Int128, Int256 --- -# UInt8, UInt16, UInt32, UInt64, Int8, Int16, Int32, Int64 {#uint8-uint16-uint32-uint64-int8-int16-int32-int64} +# UInt8, UInt16, UInt32, UInt64, UInt256, Int8, Int16, Int32, Int64, Int128, Int256 {#uint8-uint16-uint32-uint64-int8-int16-int32-int64} Fixed-length integers, with or without a sign. @@ -13,6 +13,8 @@ Fixed-length integers, with or without a sign. - Int16 - \[-32768 : 32767\] - Int32 - \[-2147483648 : 2147483647\] - Int64 - \[-9223372036854775808 : 9223372036854775807\] +- Int128 - \[-170141183460469231731687303715884105728 : 170141183460469231731687303715884105727\] +- Int256 - \[-57896044618658097711785492504343953926634992332820282019728792003956564819968 : 57896044618658097711785492504343953926634992332820282019728792003956564819967\] ## Uint Ranges {#uint-ranges} @@ -20,5 +22,8 @@ Fixed-length integers, with or without a sign. - UInt16 - \[0 : 65535\] - UInt32 - \[0 : 4294967295\] - UInt64 - \[0 : 18446744073709551615\] +- UInt256 - \[0 : 115792089237316195423570985008687907853269984665640564039457584007913129639935\] + +UInt128 is not supported yet. [Original article](https://clickhouse.tech/docs/en/data_types/int_uint/) diff --git a/docs/en/sql-reference/functions/type-conversion-functions.md b/docs/en/sql-reference/functions/type-conversion-functions.md index e5c321041c2..d4518c1f45b 100644 --- a/docs/en/sql-reference/functions/type-conversion-functions.md +++ b/docs/en/sql-reference/functions/type-conversion-functions.md @@ -11,7 +11,7 @@ When you convert a value from one to another data type, you should remember that ClickHouse has the [same behavior as C++ programs](https://en.cppreference.com/w/cpp/language/implicit_conversion). -## toInt(8\|16\|32\|64) {#toint8163264} +## toInt(8\|16\|32\|64\|128\|256) {#toint8163264} Converts an input value to the [Int](../../sql-reference/data-types/int-uint.md) data type. This function family includes: @@ -19,6 +19,8 @@ Converts an input value to the [Int](../../sql-reference/data-types/int-uint.md) - `toInt16(expr)` — Results in the `Int16` data type. - `toInt32(expr)` — Results in the `Int32` data type. - `toInt64(expr)` — Results in the `Int64` data type. +- `toInt128(expr)` — Results in the `Int128` data type. +- `toInt256(expr)` — Results in the `Int256` data type. **Parameters** @@ -26,7 +28,7 @@ Converts an input value to the [Int](../../sql-reference/data-types/int-uint.md) **Returned value** -Integer value in the `Int8`, `Int16`, `Int32`, or `Int64` data type. +Integer value in the `Int8`, `Int16`, `Int32`, `Int64`, `Int128` or `Int256` data type. Functions use [rounding towards zero](https://en.wikipedia.org/wiki/Rounding#Rounding_towards_zero), meaning they truncate fractional digits of numbers. @@ -44,9 +46,9 @@ SELECT toInt64(nan), toInt32(32), toInt16('16'), toInt8(8.8) └──────────────────────┴─────────────┴───────────────┴─────────────┘ ``` -## toInt(8\|16\|32\|64)OrZero {#toint8163264orzero} +## toInt(8\|16\|32\|64\|128\|256)OrZero {#toint8163264orzero} -It takes an argument of type String and tries to parse it into Int (8 \| 16 \| 32 \| 64). If failed, returns 0. +It takes an argument of type String and tries to parse it into Int (8 \| 16 \| 32 \| 64 \| 128 \| 256). If failed, returns 0. **Example** @@ -60,9 +62,9 @@ select toInt64OrZero('123123'), toInt8OrZero('123qwe123') └─────────────────────────┴───────────────────────────┘ ``` -## toInt(8\|16\|32\|64)OrNull {#toint8163264ornull} +## toInt(8\|16\|32\|64\|128\|256)OrNull {#toint8163264ornull} -It takes an argument of type String and tries to parse it into Int (8 \| 16 \| 32 \| 64). If failed, returns NULL. +It takes an argument of type String and tries to parse it into Int (8 \| 16 \| 32 \| 64 \| 128 \| 256). If failed, returns NULL. **Example** @@ -76,7 +78,7 @@ select toInt64OrNull('123123'), toInt8OrNull('123qwe123') └─────────────────────────┴───────────────────────────┘ ``` -## toUInt(8\|16\|32\|64) {#touint8163264} +## toUInt(8\|16\|32\|64\|256) {#touint8163264} Converts an input value to the [UInt](../../sql-reference/data-types/int-uint.md) data type. This function family includes: @@ -84,6 +86,7 @@ Converts an input value to the [UInt](../../sql-reference/data-types/int-uint.md - `toUInt16(expr)` — Results in the `UInt16` data type. - `toUInt32(expr)` — Results in the `UInt32` data type. - `toUInt64(expr)` — Results in the `UInt64` data type. +- `toUInt256(expr)` — Results in the `UInt256` data type. **Parameters** @@ -91,7 +94,7 @@ Converts an input value to the [UInt](../../sql-reference/data-types/int-uint.md **Returned value** -Integer value in the `UInt8`, `UInt16`, `UInt32`, or `UInt64` data type. +Integer value in the `UInt8`, `UInt16`, `UInt32`, `UInt64` or `UInt256` data type. Functions use [rounding towards zero](https://en.wikipedia.org/wiki/Rounding#Rounding_towards_zero), meaning they truncate fractional digits of numbers. @@ -109,9 +112,9 @@ SELECT toUInt64(nan), toUInt32(-32), toUInt16('16'), toUInt8(8.8) └─────────────────────┴───────────────┴────────────────┴──────────────┘ ``` -## toUInt(8\|16\|32\|64)OrZero {#touint8163264orzero} +## toUInt(8\|16\|32\|64\|256)OrZero {#touint8163264orzero} -## toUInt(8\|16\|32\|64)OrNull {#touint8163264ornull} +## toUInt(8\|16\|32\|64\|256)OrNull {#touint8163264ornull} ## toFloat(32\|64) {#tofloat3264} @@ -131,21 +134,23 @@ SELECT toUInt64(nan), toUInt32(-32), toUInt16('16'), toUInt8(8.8) ## toDateTimeOrNull {#todatetimeornull} -## toDecimal(32\|64\|128) {#todecimal3264128} +## toDecimal(32\|64\|128\|256) {#todecimal3264128} Converts `value` to the [Decimal](../../sql-reference/data-types/decimal.md) data type with precision of `S`. The `value` can be a number or a string. The `S` (scale) parameter specifies the number of decimal places. - `toDecimal32(value, S)` - `toDecimal64(value, S)` - `toDecimal128(value, S)` +- `toDecimal256(value, S)` -## toDecimal(32\|64\|128)OrNull {#todecimal3264128ornull} +## toDecimal(32\|64\|128\|256)OrNull {#todecimal3264128ornull} Converts an input string to a [Nullable(Decimal(P,S))](../../sql-reference/data-types/decimal.md) data type value. This family of functions include: - `toDecimal32OrNull(expr, S)` — Results in `Nullable(Decimal32(S))` data type. - `toDecimal64OrNull(expr, S)` — Results in `Nullable(Decimal64(S))` data type. - `toDecimal128OrNull(expr, S)` — Results in `Nullable(Decimal128(S))` data type. +- `toDecimal256OrNull(expr, S)` — Results in `Nullable(Decimal256(S))` data type. These functions should be used instead of `toDecimal*()` functions, if you prefer to get a `NULL` value instead of an exception in the event of an input value parsing error. @@ -183,13 +188,14 @@ SELECT toDecimal32OrNull(toString(-1.111), 2) AS val, toTypeName(val) └──────┴────────────────────────────────────────────────────┘ ``` -## toDecimal(32\|64\|128)OrZero {#todecimal3264128orzero} +## toDecimal(32\|64\|128\|256)OrZero {#todecimal3264128orzero} Converts an input value to the [Decimal(P,S)](../../sql-reference/data-types/decimal.md) data type. This family of functions include: - `toDecimal32OrZero( expr, S)` — Results in `Decimal32(S)` data type. - `toDecimal64OrZero( expr, S)` — Results in `Decimal64(S)` data type. - `toDecimal128OrZero( expr, S)` — Results in `Decimal128(S)` data type. +- `toDecimal256OrZero( expr, S)` — Results in `Decimal256(S)` data type. These functions should be used instead of `toDecimal*()` functions, if you prefer to get a `0` value instead of an exception in the event of an input value parsing error. diff --git a/docs/ru/sql-reference/data-types/decimal.md b/docs/ru/sql-reference/data-types/decimal.md index f7d51c97552..5e22fcccadc 100644 --- a/docs/ru/sql-reference/data-types/decimal.md +++ b/docs/ru/sql-reference/data-types/decimal.md @@ -1,22 +1,24 @@ -# Decimal(P, S), Decimal32(S), Decimal64(S), Decimal128(S) {#decimalp-s-decimal32s-decimal64s-decimal128s} +# Decimal(P, S), Decimal32(S), Decimal64(S), Decimal128(S), Decimal256(S) {#decimalp-s-decimal32s-decimal64s-decimal128s} Знаковые дробные числа с сохранением точности операций сложения, умножения и вычитания. Для деления осуществляется отбрасывание (не округление) знаков, не попадающих в младший десятичный разряд. ## Параметры {#parametry} -- P - precision. Значение из диапазона \[ 1 : 38 \]. Определяет, сколько десятичных знаков (с учетом дробной части) может содержать число. +- P - precision. Значение из диапазона \[ 1 : 76 \]. Определяет, сколько десятичных знаков (с учетом дробной части) может содержать число. - S - scale. Значение из диапазона \[ 0 : P \]. Определяет, сколько десятичных знаков содержится в дробной части числа. В зависимости от параметра P Decimal(P, S) является синонимом: - P из \[ 1 : 9 \] - для Decimal32(S) - P из \[ 10 : 18 \] - для Decimal64(S) - P из \[ 19 : 38 \] - для Decimal128(S) +- P из \[ 39 : 76 \] - для Decimal256(S) ## Диапазоны Decimal {#diapazony-decimal} - Decimal32(S) - ( -1 \* 10^(9 - S), 1 \* 10^(9 - S) ) - Decimal64(S) - ( -1 \* 10^(18 - S), 1 \* 10^(18 - S) ) - Decimal128(S) - ( -1 \* 10^(38 - S), 1 \* 10^(38 - S) ) +- Decimal256(S) - ( -1 \* 10^(76 - S), 1 \* 10^(76 - S) ) Например, Decimal32(4) содержит числа от -99999.9999 до 99999.9999 c шагом 0.0001. @@ -32,6 +34,7 @@ - `Decimal64(S1) Decimal32(S2) -> Decimal64(S)` - `Decimal128(S1) Decimal32(S2) -> Decimal128(S)` - `Decimal128(S1) Decimal64(S2) -> Decimal128(S)` +- `Decimal256(S1) Decimal<32|64|128>(S2) -> Decimal256(S)` Для размера дробной части (scale) результата действуют следующие правила: diff --git a/docs/ru/sql-reference/data-types/int-uint.md b/docs/ru/sql-reference/data-types/int-uint.md index f885c3f7dba..3a33c95e4c3 100644 --- a/docs/ru/sql-reference/data-types/int-uint.md +++ b/docs/ru/sql-reference/data-types/int-uint.md @@ -1,19 +1,24 @@ -# UInt8, UInt16, UInt32, UInt64, Int8, Int16, Int32, Int64 {#uint8-uint16-uint32-uint64-int8-int16-int32-int64} +# UInt8, UInt16, UInt32, UInt64, UInt256, Int8, Int16, Int32, Int64, Int128, Int256 {#uint8-uint16-uint32-uint64-int8-int16-int32-int64} Целые числа фиксированной длины, без знака или со знаком. ## Диапазоны Int {#int-ranges} -- Int8 - \[ -128 : 127 \] -- Int16 - \[ -32768 : 32767 \] -- Int32 - \[ -2147483648 : 2147483647 \] -- Int64 - \[ -9223372036854775808 : 9223372036854775807 \] +- Int8 - \[-128 : 127\] +- Int16 - \[-32768 : 32767\] +- Int32 - \[-2147483648 : 2147483647\] +- Int64 - \[-9223372036854775808 : 9223372036854775807\] +- Int128 - \[-170141183460469231731687303715884105728 : 170141183460469231731687303715884105727\] +- Int256 - \[-57896044618658097711785492504343953926634992332820282019728792003956564819968 : 57896044618658097711785492504343953926634992332820282019728792003956564819967\] ## Диапазоны Uint {#uint-ranges} -- UInt8 - \[ 0 : 255 \] -- UInt16 - \[ 0 : 65535 \] -- UInt32 - \[ 0 : 4294967295 \] -- UInt64 - \[ 0 : 18446744073709551615 \] +- UInt8 - \[0 : 255\] +- UInt16 - \[0 : 65535\] +- UInt32 - \[0 : 4294967295\] +- UInt64 - \[0 : 18446744073709551615\] +- UInt256 - \[0 : 115792089237316195423570985008687907853269984665640564039457584007913129639935\] + +UInt128 пока не реализован. [Оригинальная статья](https://clickhouse.tech/docs/ru/data_types/int_uint/) diff --git a/docs/ru/sql-reference/functions/type-conversion-functions.md b/docs/ru/sql-reference/functions/type-conversion-functions.md index 9ceb105095d..0440d70902c 100644 --- a/docs/ru/sql-reference/functions/type-conversion-functions.md +++ b/docs/ru/sql-reference/functions/type-conversion-functions.md @@ -6,7 +6,7 @@ Поведение ClickHouse при конвертировании похоже на [поведение C++ программ](https://en.cppreference.com/w/cpp/language/implicit_conversion). -## toInt(8\|16\|32\|64) {#toint8163264} +## toInt(8\|16\|32\|64\|128\|256) {#toint8163264} Преобразует входное значение к типу [Int](../../sql-reference/functions/type-conversion-functions.md). Семейство функций включает: @@ -14,6 +14,8 @@ - `toInt16(expr)` — возвращает значение типа `Int16`. - `toInt32(expr)` — возвращает значение типа `Int32`. - `toInt64(expr)` — возвращает значение типа `Int64`. +- `toInt128(expr)` — возвращает значение типа `Int128`. +- `toInt256(expr)` — возвращает значение типа `Int256`. **Параметры** @@ -21,7 +23,7 @@ **Возвращаемое значение** -Целое число типа `Int8`, `Int16`, `Int32` или `Int64`. +Целое число типа `Int8`, `Int16`, `Int32`, `Int64`, `Int128` или `Int256`. Функции используют [округление к нулю](https://en.wikipedia.org/wiki/Rounding#Rounding_towards_zero), т.е. обрезают дробную часть числа. @@ -39,9 +41,9 @@ SELECT toInt64(nan), toInt32(32), toInt16('16'), toInt8(8.8) └──────────────────────┴─────────────┴───────────────┴─────────────┘ ``` -## toInt(8\|16\|32\|64)OrZero {#toint8163264orzero} +## toInt(8\|16\|32\|64\|128\|256)OrZero {#toint8163264orzero} -Принимает аргумент типа String и пытается его распарсить в Int(8\|16\|32\|64). Если не удалось - возвращает 0. +Принимает аргумент типа String и пытается его распарсить в Int(8\|16\|32\|64\|128\|256). Если не удалось - возвращает 0. **Пример** @@ -55,9 +57,9 @@ select toInt64OrZero('123123'), toInt8OrZero('123qwe123') └─────────────────────────┴───────────────────────────┘ ``` -## toInt(8\|16\|32\|64)OrNull {#toint8163264ornull} +## toInt(8\|16\|32\|64\|128\|256)OrNull {#toint8163264ornull} -Принимает аргумент типа String и пытается его распарсить в Int(8\|16\|32\|64). Если не удалось - возвращает NULL. +Принимает аргумент типа String и пытается его распарсить в Int(8\|16\|32\|64\|128\|256). Если не удалось - возвращает NULL. **Пример** @@ -71,7 +73,7 @@ select toInt64OrNull('123123'), toInt8OrNull('123qwe123') └─────────────────────────┴───────────────────────────┘ ``` -## toUInt(8\|16\|32\|64) {#touint8163264} +## toUInt(8\|16\|32\|64\|256) {#touint8163264} Преобраует входное значение к типу [UInt](../../sql-reference/functions/type-conversion-functions.md). Семейство функций включает: @@ -79,6 +81,7 @@ select toInt64OrNull('123123'), toInt8OrNull('123qwe123') - `toUInt16(expr)` — возвращает значение типа `UInt16`. - `toUInt32(expr)` — возвращает значение типа `UInt32`. - `toUInt64(expr)` — возвращает значение типа `UInt64`. +- `toUInt256(expr)` — возвращает значение типа `UInt256`. **Параметры** @@ -86,7 +89,7 @@ select toInt64OrNull('123123'), toInt8OrNull('123qwe123') **Возвращаемое значение** -Целое число типа `UInt8`, `UInt16`, `UInt32` или `UInt64`. +Целое число типа `UInt8`, `UInt16`, `UInt32`, `UInt64` или `UInt256`. Функции используют [округление к нулю](https://en.wikipedia.org/wiki/Rounding#Rounding_towards_zero), т.е. обрезают дробную часть числа. @@ -104,9 +107,9 @@ SELECT toUInt64(nan), toUInt32(-32), toUInt16('16'), toUInt8(8.8) └─────────────────────┴───────────────┴────────────────┴──────────────┘ ``` -## toUInt(8\|16\|32\|64)OrZero {#touint8163264orzero} +## toUInt(8\|16\|32\|64\|256)OrZero {#touint8163264orzero} -## toUInt(8\|16\|32\|64)OrNull {#touint8163264ornull} +## toUInt(8\|16\|32\|64\|256)OrNull {#touint8163264ornull} ## toFloat(32\|64) {#tofloat3264} @@ -126,21 +129,23 @@ SELECT toUInt64(nan), toUInt32(-32), toUInt16('16'), toUInt8(8.8) ## toDateTimeOrNull {#todatetimeornull} -## toDecimal(32\|64\|128) {#todecimal3264128} +## toDecimal(32\|64\|128\|256) {#todecimal3264128} Преобразует `value` к типу данных [Decimal](../../sql-reference/functions/type-conversion-functions.md) с точностью `S`. `value` может быть числом или строкой. Параметр `S` (scale) задаёт число десятичных знаков. - `toDecimal32(value, S)` - `toDecimal64(value, S)` - `toDecimal128(value, S)` +- `toDecimal256(value, S)` -## toDecimal(32\|64\|128)OrNull {#todecimal3264128ornull} +## toDecimal(32\|64\|128\|256)OrNull {#todecimal3264128ornull} Преобразует входную строку в значение с типом данных [Nullable (Decimal (P, S))](../../sql-reference/functions/type-conversion-functions.md). Семейство функций включает в себя: - `toDecimal32OrNull(expr, S)` — Возвращает значение типа `Nullable(Decimal32(S))`. - `toDecimal64OrNull(expr, S)` — Возвращает значение типа `Nullable(Decimal64(S))`. - `toDecimal128OrNull(expr, S)` — Возвращает значение типа `Nullable(Decimal128(S))`. +- `toDecimal256OrNull(expr, S)` — Возвращает значение типа `Nullable(Decimal256(S))`. Эти функции следует использовать вместо функций `toDecimal*()`, если при ошибке обработки входного значения вы хотите получать `NULL` вместо исключения. @@ -178,13 +183,14 @@ SELECT toDecimal32OrNull(toString(-1.111), 2) AS val, toTypeName(val) └──────┴────────────────────────────────────────────────────┘ ``` -## toDecimal(32\|64\|128)OrZero {#todecimal3264128orzero} +## toDecimal(32\|64\|128\|256)OrZero {#todecimal3264128orzero} Преобразует тип входного значения в [Decimal (P, S)](../../sql-reference/functions/type-conversion-functions.md). Семейство функций включает в себя: - `toDecimal32OrZero( expr, S)` — возвращает значение типа `Decimal32(S)`. - `toDecimal64OrZero( expr, S)` — возвращает значение типа `Decimal64(S)`. - `toDecimal128OrZero( expr, S)` — возвращает значение типа `Decimal128(S)`. +- `toDecimal256OrZero( expr, S)` — возвращает значение типа `Decimal256(S)`. Эти функции следует использовать вместо функций `toDecimal*()`, если при ошибке обработки входного значения вы хотите получать `0` вместо исключения. From d35ac3c3fb6e690569de28817305c0df8598bdb5 Mon Sep 17 00:00:00 2001 From: nvartolomei Date: Fri, 2 Oct 2020 15:10:19 +0100 Subject: [PATCH 281/321] Add missing entry to client packet stringify (noticed while reading the code) --- src/Core/Protocol.h | 1 + 1 file changed, 1 insertion(+) diff --git a/src/Core/Protocol.h b/src/Core/Protocol.h index a370a29dac8..f383e509751 100644 --- a/src/Core/Protocol.h +++ b/src/Core/Protocol.h @@ -146,6 +146,7 @@ namespace Protocol "Ping", "TablesStatusRequest", "KeepAlive", + "Scalar", }; return packet <= MAX ? data[packet] From 8c0581c503553bc9dfba48d583e5c191c3cac400 Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 2 Oct 2020 17:27:47 +0300 Subject: [PATCH 282/321] Fix ilike cache --- src/Functions/MatchImpl.h | 5 +---- src/Functions/Regexps.h | 15 +++++++++------ 2 files changed, 10 insertions(+), 10 deletions(-) diff --git a/src/Functions/MatchImpl.h b/src/Functions/MatchImpl.h index 54ceb05645d..fee8201f3f4 100644 --- a/src/Functions/MatchImpl.h +++ b/src/Functions/MatchImpl.h @@ -141,10 +141,7 @@ struct MatchImpl { size_t size = offsets.size(); - constexpr int flags = case_insensitive ? - Regexps::Regexp::RE_CASELESS : 0; - - auto regexp = Regexps::get(pattern, flags); + auto regexp = Regexps::get(pattern); std::string required_substring; bool is_trivial; diff --git a/src/Functions/Regexps.h b/src/Functions/Regexps.h index cbfbbf7107d..d9df4218056 100644 --- a/src/Functions/Regexps.h +++ b/src/Functions/Regexps.h @@ -58,21 +58,24 @@ namespace Regexps * You must hold the ownership while using the object. * In destructor, it returns the object back to the Pool for further reuse. */ - template - inline Pool::Pointer get(const std::string & pattern, int flags = 0) + template + inline Pool::Pointer get(const std::string & pattern) { /// C++11 has thread-safe function-local statics on most modern compilers. static Pool known_regexps; /// Different variables for different pattern parameters. - return known_regexps.get(pattern, [flags, &pattern] + return known_regexps.get(pattern, [&pattern] { - int flags_final = flags | OptimizedRegularExpression::RE_DOT_NL; + int flags = OptimizedRegularExpression::RE_DOT_NL; if (no_capture) - flags_final |= OptimizedRegularExpression::RE_NO_CAPTURE; + flags |= OptimizedRegularExpression::RE_NO_CAPTURE; + + if (case_insensitive) + flags |= Regexps::Regexp::RE_CASELESS; ProfileEvents::increment(ProfileEvents::RegexpCreated); - return new Regexp{createRegexp(pattern, flags_final)}; + return new Regexp{createRegexp(pattern, flags)}; }); } } From 7d758667221510564827b4d39e73ae479048707f Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 2 Oct 2020 17:29:20 +0300 Subject: [PATCH 283/321] Add tests --- tests/queries/0_stateless/01513_ilike_like_cache.reference | 4 ++++ tests/queries/0_stateless/01513_ilike_like_cache.sql | 5 +++++ 2 files changed, 9 insertions(+) create mode 100644 tests/queries/0_stateless/01513_ilike_like_cache.reference create mode 100644 tests/queries/0_stateless/01513_ilike_like_cache.sql diff --git a/tests/queries/0_stateless/01513_ilike_like_cache.reference b/tests/queries/0_stateless/01513_ilike_like_cache.reference new file mode 100644 index 00000000000..533dce9c3d6 --- /dev/null +++ b/tests/queries/0_stateless/01513_ilike_like_cache.reference @@ -0,0 +1,4 @@ +1 +1 +1 +0 diff --git a/tests/queries/0_stateless/01513_ilike_like_cache.sql b/tests/queries/0_stateless/01513_ilike_like_cache.sql new file mode 100644 index 00000000000..dc761771336 --- /dev/null +++ b/tests/queries/0_stateless/01513_ilike_like_cache.sql @@ -0,0 +1,5 @@ +SELECT 'hello' like 'hell%'; +SELECT 'HELLO' ilike 'hell%'; + +SELECT 'world' ilike 'Wo%Ld'; +SELECT 'world' like 'Wo%Ld'; From d2f75f850887d2b9d9a4875663d6e8e484ebd8f9 Mon Sep 17 00:00:00 2001 From: feng lv Date: Fri, 2 Oct 2020 23:13:28 +0800 Subject: [PATCH 284/321] fix --- .../Formats/Impl/PrettyBlockOutputFormat.cpp | 39 +++++++++++++++++++ .../Impl/PrettyCompactBlockOutputFormat.cpp | 10 +---- .../Impl/PrettySpaceBlockOutputFormat.cpp | 14 +++++++ 3 files changed, 55 insertions(+), 8 deletions(-) diff --git a/src/Processors/Formats/Impl/PrettyBlockOutputFormat.cpp b/src/Processors/Formats/Impl/PrettyBlockOutputFormat.cpp index 43fff9a66ff..b07cf83eb85 100644 --- a/src/Processors/Formats/Impl/PrettyBlockOutputFormat.cpp +++ b/src/Processors/Formats/Impl/PrettyBlockOutputFormat.cpp @@ -200,9 +200,20 @@ void PrettyBlockOutputFormat::write(const Chunk & chunk, PortKind port_kind) std::string middle_values_separator_s = middle_values_separator.str(); std::string bottom_separator_s = bottom_separator.str(); + if (format_settings.pretty.output_format_pretty_row_numbers) + { + /// Write left blank + writeString(String(row_number_width, ' '), out); + } /// Output the block writeString(top_separator_s, out); + if (format_settings.pretty.output_format_pretty_row_numbers) + { + /// Write left blank + writeString(String(row_number_width, ' '), out); + } + /// Names writeCString(grid_symbols.bold_bar, out); writeCString(" ", out); @@ -242,12 +253,35 @@ void PrettyBlockOutputFormat::write(const Chunk & chunk, PortKind port_kind) writeCString(grid_symbols.bold_bar, out); writeCString("\n", out); + if (format_settings.pretty.output_format_pretty_row_numbers) + { + /// Write left blank + writeString(String(row_number_width, ' '), out); + } writeString(middle_names_separator_s, out); for (size_t i = 0; i < num_rows && total_rows + i < max_rows; ++i) { if (i != 0) + { + if (format_settings.pretty.output_format_pretty_row_numbers) + { + /// Write left blank + writeString(String(row_number_width, ' '), out); + } writeString(middle_values_separator_s, out); + } + + if (format_settings.pretty.output_format_pretty_row_numbers) + { + // Write row number; + auto row_num_string = std::to_string(i + 1) + ". "; + for (size_t j = 0; j < row_number_width - row_num_string.size(); ++j) + { + writeCString(" ", out); + } + writeString(row_num_string, out); + } writeCString(grid_symbols.bar, out); @@ -266,6 +300,11 @@ void PrettyBlockOutputFormat::write(const Chunk & chunk, PortKind port_kind) writeCString("\n", out); } + if (format_settings.pretty.output_format_pretty_row_numbers) + { + /// Write left blank + writeString(String(row_number_width, ' '), out); + } writeString(bottom_separator_s, out); total_rows += num_rows; diff --git a/src/Processors/Formats/Impl/PrettyCompactBlockOutputFormat.cpp b/src/Processors/Formats/Impl/PrettyCompactBlockOutputFormat.cpp index 13000a7b9d3..e832dd83388 100644 --- a/src/Processors/Formats/Impl/PrettyCompactBlockOutputFormat.cpp +++ b/src/Processors/Formats/Impl/PrettyCompactBlockOutputFormat.cpp @@ -72,10 +72,7 @@ void PrettyCompactBlockOutputFormat::writeHeader( if (format_settings.pretty.output_format_pretty_row_numbers) { /// Write left blank - for (size_t i = 0; i < row_number_width; ++i) - { - writeCString(" ", out); - } + writeString(String(row_number_width, ' '), out); } const GridSymbols & grid_symbols = format_settings.pretty.charset == FormatSettings::Pretty::Charset::UTF8 ? @@ -129,10 +126,7 @@ void PrettyCompactBlockOutputFormat::writeBottom(const Widths & max_widths) if (format_settings.pretty.output_format_pretty_row_numbers) { /// Write left blank - for (size_t i = 0; i < row_number_width; ++i) - { - writeCString(" ", out); - } + writeString(String(row_number_width, ' '), out); } const GridSymbols & grid_symbols = format_settings.pretty.charset == FormatSettings::Pretty::Charset::UTF8 ? diff --git a/src/Processors/Formats/Impl/PrettySpaceBlockOutputFormat.cpp b/src/Processors/Formats/Impl/PrettySpaceBlockOutputFormat.cpp index 6b42ea57e1f..f1a5cbac8e0 100644 --- a/src/Processors/Formats/Impl/PrettySpaceBlockOutputFormat.cpp +++ b/src/Processors/Formats/Impl/PrettySpaceBlockOutputFormat.cpp @@ -29,6 +29,10 @@ void PrettySpaceBlockOutputFormat::write(const Chunk & chunk, PortKind port_kind Widths name_widths; calculateWidths(header, chunk, widths, max_widths, name_widths); + if (format_settings.pretty.output_format_pretty_row_numbers) + { + writeString(String(row_number_width, ' '), out); + } /// Names for (size_t i = 0; i < num_columns; ++i) { @@ -66,6 +70,16 @@ void PrettySpaceBlockOutputFormat::write(const Chunk & chunk, PortKind port_kind for (size_t row = 0; row < num_rows && total_rows + row < max_rows; ++row) { + if (format_settings.pretty.output_format_pretty_row_numbers) + { + // Write row number; + auto row_num_string = std::to_string(row + 1) + ". "; + for (size_t i = 0; i < row_number_width - row_num_string.size(); ++i) + { + writeCString(" ", out); + } + writeString(row_num_string, out); + } for (size_t column = 0; column < num_columns; ++column) { if (column != 0) From ac1f0b9947c35fbae522c5414fd89a281dd3609b Mon Sep 17 00:00:00 2001 From: feng lv Date: Fri, 2 Oct 2020 23:34:14 +0800 Subject: [PATCH 285/321] add test --- ...output_format_pretty_row_numbers.reference | 212 ++++++++++++++++++ ...01509_output_format_pretty_row_numbers.sql | 15 ++ 2 files changed, 227 insertions(+) create mode 100644 tests/queries/0_stateless/01509_output_format_pretty_row_numbers.reference create mode 100644 tests/queries/0_stateless/01509_output_format_pretty_row_numbers.sql diff --git a/tests/queries/0_stateless/01509_output_format_pretty_row_numbers.reference b/tests/queries/0_stateless/01509_output_format_pretty_row_numbers.reference new file mode 100644 index 00000000000..9010c371fdf --- /dev/null +++ b/tests/queries/0_stateless/01509_output_format_pretty_row_numbers.reference @@ -0,0 +1,212 @@ +┏━━━━━━━━┓ +┃ number ┃ +┡━━━━━━━━┩ +│ 0 │ +├────────┤ +│ 1 │ +├────────┤ +│ 2 │ +├────────┤ +│ 3 │ +├────────┤ +│ 4 │ +├────────┤ +│ 5 │ +├────────┤ +│ 6 │ +├────────┤ +│ 7 │ +├────────┤ +│ 8 │ +├────────┤ +│ 9 │ +└────────┘ +┌─number─┐ +│ 0 │ +│ 1 │ +│ 2 │ +│ 3 │ +│ 4 │ +│ 5 │ +│ 6 │ +│ 7 │ +│ 8 │ +│ 9 │ +└────────┘ +┌─number─┐ +│ 0 │ +│ 1 │ +│ 2 │ +│ 3 │ +│ 4 │ +│ 5 │ +│ 6 │ +│ 7 │ +│ 8 │ +│ 9 │ +└────────┘ +┏━━━━━━━━┓ +┃ number ┃ +┡━━━━━━━━┩ +│ 0 │ +├────────┤ +│ 1 │ +├────────┤ +│ 2 │ +├────────┤ +│ 3 │ +├────────┤ +│ 4 │ +├────────┤ +│ 5 │ +├────────┤ +│ 6 │ +├────────┤ +│ 7 │ +├────────┤ +│ 8 │ +├────────┤ +│ 9 │ +└────────┘ +┌─number─┐ +│ 0 │ +│ 1 │ +│ 2 │ +│ 3 │ +│ 4 │ +│ 5 │ +│ 6 │ +│ 7 │ +│ 8 │ +│ 9 │ +└────────┘ + number + + 0 + 1 + 2 + 3 + 4 + 5 + 6 + 7 + 8 + 9 + number + + 0 + 1 + 2 + 3 + 4 + 5 + 6 + 7 + 8 + 9 + ┏━━━━━━━━┓ + ┃ number ┃ + ┡━━━━━━━━┩ + 1. │ 0 │ + ├────────┤ + 2. │ 1 │ + ├────────┤ + 3. │ 2 │ + ├────────┤ + 4. │ 3 │ + ├────────┤ + 5. │ 4 │ + ├────────┤ + 6. │ 5 │ + ├────────┤ + 7. │ 6 │ + ├────────┤ + 8. │ 7 │ + ├────────┤ + 9. │ 8 │ + ├────────┤ +10. │ 9 │ + └────────┘ + ┌─number─┐ + 1. │ 0 │ + 2. │ 1 │ + 3. │ 2 │ + 4. │ 3 │ + 5. │ 4 │ + 6. │ 5 │ + 7. │ 6 │ + 8. │ 7 │ + 9. │ 8 │ +10. │ 9 │ + └────────┘ + ┌─number─┐ + 1. │ 0 │ + 2. │ 1 │ + 3. │ 2 │ + 4. │ 3 │ + 5. │ 4 │ + 6. │ 5 │ + 7. │ 6 │ + 8. │ 7 │ + 9. │ 8 │ +10. │ 9 │ + └────────┘ + ┏━━━━━━━━┓ + ┃ number ┃ + ┡━━━━━━━━┩ + 1. │ 0 │ + ├────────┤ + 2. │ 1 │ + ├────────┤ + 3. │ 2 │ + ├────────┤ + 4. │ 3 │ + ├────────┤ + 5. │ 4 │ + ├────────┤ + 6. │ 5 │ + ├────────┤ + 7. │ 6 │ + ├────────┤ + 8. │ 7 │ + ├────────┤ + 9. │ 8 │ + ├────────┤ +10. │ 9 │ + └────────┘ + ┌─number─┐ + 1. │ 0 │ + 2. │ 1 │ + 3. │ 2 │ + 4. │ 3 │ + 5. │ 4 │ + 6. │ 5 │ + 7. │ 6 │ + 8. │ 7 │ + 9. │ 8 │ +10. │ 9 │ + └────────┘ + number + + 1. 0 + 2. 1 + 3. 2 + 4. 3 + 5. 4 + 6. 5 + 7. 6 + 8. 7 + 9. 8 +10. 9 + number + + 1. 0 + 2. 1 + 3. 2 + 4. 3 + 5. 4 + 6. 5 + 7. 6 + 8. 7 + 9. 8 +10. 9 diff --git a/tests/queries/0_stateless/01509_output_format_pretty_row_numbers.sql b/tests/queries/0_stateless/01509_output_format_pretty_row_numbers.sql new file mode 100644 index 00000000000..3c0eeaf0e53 --- /dev/null +++ b/tests/queries/0_stateless/01509_output_format_pretty_row_numbers.sql @@ -0,0 +1,15 @@ +SELECT * FROM numbers(10) FORMAT Pretty; +SELECT * FROM numbers(10) FORMAT PrettyCompact; +SELECT * FROM numbers(10) FORMAT PrettyCompactMonoBlock; +SELECT * FROM numbers(10) FORMAT PrettyNoEscapes; +SELECT * FROM numbers(10) FORMAT PrettyCompactNoEscapes; +SELECT * FROM numbers(10) FORMAT PrettySpaceNoEscapes; +SELECT * FROM numbers(10) FORMAT PrettySpace; +SET output_format_pretty_row_numbers=1; +SELECT * FROM numbers(10) FORMAT Pretty; +SELECT * FROM numbers(10) FORMAT PrettyCompact; +SELECT * FROM numbers(10) FORMAT PrettyCompactMonoBlock; +SELECT * FROM numbers(10) FORMAT PrettyNoEscapes; +SELECT * FROM numbers(10) FORMAT PrettyCompactNoEscapes; +SELECT * FROM numbers(10) FORMAT PrettySpaceNoEscapes; +SELECT * FROM numbers(10) FORMAT PrettySpace; From 9cb3c743bd0ec28942b018c5853accfc1b1259b0 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Fri, 2 Oct 2020 19:54:07 +0300 Subject: [PATCH 286/321] Convert to python3 (#15007) --- benchmark/greenplum/result_parser.py | 6 +- debian/control | 2 +- docker/builder/Dockerfile | 8 +- docker/packager/packager | 2 +- docker/test/fasttest/Dockerfile | 8 +- docker/test/integration/base/Dockerfile | 2 +- docker/test/integration/runner/Dockerfile | 6 +- docker/test/performance-comparison/report.py | 8 +- docker/test/stateful/Dockerfile | 2 +- docker/test/stateful/s3downloader | 6 +- docker/test/stateful_with_coverage/Dockerfile | 2 +- .../test/stateful_with_coverage/s3downloader | 4 +- docker/test/stateless/Dockerfile | 8 +- docker/test/stateless_pytest/Dockerfile | 8 +- docker/test/stateless_unbundled/Dockerfile | 8 +- .../test/stateless_with_coverage/Dockerfile | 8 +- docker/test/stress/Dockerfile | 8 +- docker/test/stress/stress | 2 +- docs/en/development/build.md | 2 +- docs/es/development/build.md | 2 +- docs/fa/development/build.md | 2 +- docs/fr/development/build.md | 2 +- docs/ja/development/build.md | 2 +- docs/tools/easy_diff.py | 2 +- docs/tools/github.py | 4 +- docs/tools/mdx_clickhouse.py | 4 +- docs/tools/nav.py | 2 +- docs/tools/purge_cache_for_changed_files.py | 6 +- docs/tools/single_page.py | 2 +- docs/tools/test.py | 3 +- docs/tools/translate/typograph_ru.py | 2 +- docs/tr/development/build.md | 2 +- release | 2 +- src/Storages/tests/active_parts.py | 8 +- tests/clickhouse-test | 90 +- .../catboost/helpers/server.py | 2 +- .../catboost/helpers/server_with_models.py | 8 +- .../external_models/catboost/helpers/table.py | 10 +- .../external_models/catboost/helpers/train.py | 4 +- .../test_apply_catboost_model/test.py | 52 +- tests/integration/README.md | 6 +- tests/integration/helpers/client.py | 18 +- tests/integration/helpers/cluster.py | 146 +- tests/integration/helpers/external_sources.py | 14 +- tests/integration/helpers/hdfs_api.py | 23 +- tests/integration/helpers/http_server.py | 16 +- tests/integration/helpers/network.py | 2 +- tests/integration/helpers/test_tools.py | 15 +- tests/integration/helpers/uclient.py | 2 +- tests/integration/helpers/uexpect.py | 7 +- tests/integration/runner | 4 +- .../test_adaptive_granularity/test.py | 4 +- tests/integration/test_cluster_copier/test.py | 22 +- .../test_cluster_copier/trivial_test.py | 18 +- .../test.py | 6 +- .../test_concurrent_ttl_merges/test.py | 4 +- .../test_config_substitutions/test.py | 2 +- .../test.py | 2 +- .../test.py | 2 +- .../test_cross_replication/test.py | 8 +- .../test_delayed_replica_failover/test.py | 4 +- .../common.py | 12 +- .../test.py | 4 +- .../test_dictionaries_redis/test.py | 12 +- .../generate_dictionaries.py | 16 +- .../test_dictionaries_select_all/test.py | 10 +- .../test_default_reading.py | 2 +- .../test_default_string.py | 2 +- .../test_dict_get.py | 2 +- .../test_dict_get_or_default.py | 2 +- .../http_server.py | 12 +- .../test_dictionary_custom_settings/test.py | 2 +- tests/integration/test_disk_types/test.py | 2 +- .../test_distributed_ddl/cluster.py | 12 +- .../integration/test_distributed_ddl/test.py | 22 +- .../test_replicated_alter.py | 18 +- .../test.py | 2 +- .../test_distributed_load_balancing/test.py | 4 +- .../test_distributed_over_distributed/test.py | 4 +- .../test_distributed_over_live_view/test.py | 22 +- .../test.py | 4 +- tests/integration/test_drop_replica/test.py | 2 +- .../test.py | 2 +- .../test_format_avro_confluent/test.py | 2 +- .../integration/test_grant_and_revoke/test.py | 4 +- .../test_graphite_merge_tree/test.py | 2 +- tests/integration/test_host_ip_change/test.py | 6 +- .../test_http_handlers_config/test.py | 30 +- .../test_https_replication/test.py | 2 +- .../test.py | 7 +- .../test_live_view_over_distributed/test.py | 11 +- .../materialize_with_ddl.py | 2 +- .../test_materialize_mysql_database/test.py | 10 +- .../test.py | 12 +- .../test_merge_table_over_distributed/test.py | 6 +- tests/integration/test_multiple_disks/test.py | 4 +- .../test_mutations_hardlinks/test.py | 2 +- .../test_mutations_with_merge_tree/test.py | 4 +- .../test_mysql_database_engine/test.py | 4 +- tests/integration/test_mysql_protocol/test.py | 55 +- .../test_no_local_metadata_node/test.py | 2 +- .../test_non_default_compression/test.py | 2 +- .../integration/test_odbc_interaction/test.py | 14 +- tests/integration/test_partition/test.py | 4 +- .../test_parts_delete_zookeeper/test.py | 2 +- .../test_postgresql_protocol/test.py | 12 +- .../test_prometheus_endpoint/test.py | 2 +- tests/integration/test_quota/test.py | 96 +- tests/integration/test_random_inserts/test.py | 6 +- .../test_recompression_ttl/test.py | 2 +- .../integration/test_recovery_replica/test.py | 2 +- .../test.py | 3 +- .../integration/test_reload_zookeeper/test.py | 2 +- .../test.py | 2 +- tests/integration/test_rename_column/test.py | 2 +- .../test_replicated_merge_tree_s3/test.py | 4 +- .../test_replicated_mutations/test.py | 26 +- .../test_replicated_parse_zk_metadata/test.py | 2 +- .../test.py | 2 +- tests/integration/test_role/test.py | 12 +- .../fake_sentry_server.py | 15 +- .../test_send_crash_reports/test.py | 4 +- .../test_settings_constraints/test.py | 2 +- .../integration/test_settings_profile/test.py | 14 +- tests/integration/test_storage_hdfs/test.py | 12 +- tests/integration/test_storage_kafka/test.py | 131 +- .../test_storage_kerberized_kafka/test.py | 6 +- tests/integration/test_storage_mysql/test.py | 6 +- .../integration/test_storage_rabbitmq/test.py | 22 +- .../test_storage_s3/s3_mock/mock_s3.py | 2 +- tests/integration/test_storage_s3/test.py | 20 +- tests/integration/test_system_queries/test.py | 9 +- tests/integration/test_ttl_move/test.py | 66 +- tests/integration/test_ttl_replicated/test.py | 2 +- .../test_user_ip_restrictions/test.py | 6 +- .../test_user_zero_database_access.py | 6 +- .../integration/test_zookeeper_config/test.py | 4 +- .../accurate_comparisons.sh | 2 +- .../0_stateless/00386_long_in_pk.python | 8 +- tests/queries/0_stateless/00386_long_in_pk.sh | 2 +- ...411_long_accurate_number_comparison.python | 18 +- ...1_long_accurate_number_comparison_float.sh | 2 +- ...11_long_accurate_number_comparison_int1.sh | 2 +- ...11_long_accurate_number_comparison_int2.sh | 2 +- ...11_long_accurate_number_comparison_int3.sh | 2 +- ...11_long_accurate_number_comparison_int4.sh | 2 +- tests/queries/0_stateless/00565_enum_order.sh | 2 +- .../0_stateless/00612_http_max_query_size.sh | 10 +- .../0_stateless/00646_url_engine.python | 59 +- tests/queries/0_stateless/00646_url_engine.sh | 2 +- .../00921_datetime64_compatibility.python | 35 +- .../00921_datetime64_compatibility.reference | 1402 ++++++++--------- .../00960_live_view_watch_events_live.py | 2 +- .../00962_temporary_live_view_watch_live.py | 2 +- ...y_live_view_watch_live_timeout.py.disabled | 2 +- .../00964_live_view_watch_events_heartbeat.py | 2 +- .../00965_live_view_watch_heartbeat.py | 2 +- .../00966_live_view_watch_events_http.py | 2 +- .../0_stateless/00967_live_view_watch_http.py | 2 +- ...0_live_view_watch_events_http_heartbeat.py | 2 +- .../00971_live_view_watch_http_heartbeat.py | 2 +- ...9_live_view_watch_continuous_aggregates.py | 2 +- .../0_stateless/00979_live_view_watch_live.py | 2 +- .../00979_live_view_watch_live_moving_avg.py | 2 +- ...0979_live_view_watch_live_with_subquery.py | 2 +- .../queries/0_stateless/00990_hasToken.python | 4 +- tests/queries/0_stateless/00990_hasToken.sh | 2 +- .../00991_live_view_watch_event_live.python | 4 +- ...991_live_view_watch_event_live.sh.disabled | 2 +- .../00991_live_view_watch_http.python | 4 +- .../00991_live_view_watch_http.sh.disabled | 2 +- ...ry_live_view_watch_events_heartbeat.python | 4 +- ...ve_view_watch_events_heartbeat.sh.disabled | 2 +- ...0991_temporary_live_view_watch_live.python | 4 +- ...temporary_live_view_watch_live.sh.disabled | 2 +- .../01246_insert_into_watch_live_view.py | 2 +- .../queries/0_stateless/helpers/httpclient.py | 2 +- .../0_stateless/helpers/httpechoserver.py | 8 +- .../queries/0_stateless/helpers/httpexpect.py | 14 +- tests/queries/0_stateless/helpers/uexpect.py | 6 +- tests/queries/conftest.py | 6 +- tests/queries/query_test.py | 4 +- tests/queries/server.py | 12 +- tests/queries/shell_config.sh | 2 +- tests/testflows/helpers/cluster.py | 2 +- tests/testflows/ldap/tests/common.py | 6 +- .../testflows/rbac/tests/privileges/insert.py | 6 +- .../testflows/rbac/tests/privileges/select.py | 2 +- tests/testflows/runner | 4 +- utils/build/build_debian.sh | 2 +- utils/github-hook/hook.py | 10 +- utils/github/backport.py | 4 +- utils/github/parser.py | 2 +- utils/github/query.py | 2 +- utils/junit_to_html/junit_to_html | 4 +- utils/kafka/manage.py | 4 +- utils/make_changelog.py | 26 +- utils/release/push_packages | 8 +- utils/release/release_lib.sh | 4 +- utils/s3tools/s3uploader | 4 +- utils/simple-backport/format-changelog.py | 4 +- utils/test_history/test-history | 8 +- utils/upload_test_results/upload_test_results | 6 +- 203 files changed, 1632 insertions(+), 1633 deletions(-) diff --git a/benchmark/greenplum/result_parser.py b/benchmark/greenplum/result_parser.py index ea178cf77a4..8af20d265a0 100755 --- a/benchmark/greenplum/result_parser.py +++ b/benchmark/greenplum/result_parser.py @@ -1,6 +1,6 @@ -#!/usr/bin/env python +#!/usr/bin/env python3 + -from __future__ import print_function import sys import json @@ -99,7 +99,7 @@ def gen_html_json(options, arguments): tuples = read_stats_file(options, arguments[1]) print('{') print('"system: GreenPlum(x2),') - print('"version": "%s",' % '4.3.9.1') + print(('"version": "%s",' % '4.3.9.1')) print('"data_size": 10000000,') print('"time": "",') print('"comments": "",') diff --git a/debian/control b/debian/control index 1014b8b0a3c..12d69d9fff6 100644 --- a/debian/control +++ b/debian/control @@ -62,5 +62,5 @@ Description: debugging symbols for clickhouse-common-static Package: clickhouse-test Priority: optional Architecture: all -Depends: ${shlibs:Depends}, ${misc:Depends}, clickhouse-client, bash, expect, python, python-lxml, python-termcolor, python-requests, curl, perl, sudo, openssl, netcat-openbsd, telnet, brotli, bsdutils +Depends: ${shlibs:Depends}, ${misc:Depends}, clickhouse-client, bash, expect, python3, python3-lxml, python3-termcolor, python3-requests, curl, perl, sudo, openssl, netcat-openbsd, telnet, brotli, bsdutils Description: ClickHouse tests diff --git a/docker/builder/Dockerfile b/docker/builder/Dockerfile index d4a121d13eb..68245a92c58 100644 --- a/docker/builder/Dockerfile +++ b/docker/builder/Dockerfile @@ -25,10 +25,10 @@ RUN apt-get update \ ninja-build \ perl \ pkg-config \ - python \ - python-lxml \ - python-requests \ - python-termcolor \ + python3 \ + python3-lxml \ + python3-requests \ + python3-termcolor \ tzdata \ llvm-${LLVM_VERSION} \ clang-${LLVM_VERSION} \ diff --git a/docker/packager/packager b/docker/packager/packager index 0a14102ec04..6d075195003 100755 --- a/docker/packager/packager +++ b/docker/packager/packager @@ -1,4 +1,4 @@ -#!/usr/bin/env python +#!/usr/bin/env python3 #-*- coding: utf-8 -*- import subprocess import os diff --git a/docker/test/fasttest/Dockerfile b/docker/test/fasttest/Dockerfile index 9b4bb574f8f..65ec078d3ca 100644 --- a/docker/test/fasttest/Dockerfile +++ b/docker/test/fasttest/Dockerfile @@ -52,10 +52,10 @@ RUN apt-get update \ moreutils \ ninja-build \ psmisc \ - python \ - python-lxml \ - python-requests \ - python-termcolor \ + python3 \ + python3-lxml \ + python3-requests \ + python3-termcolor \ qemu-user-static \ rename \ software-properties-common \ diff --git a/docker/test/integration/base/Dockerfile b/docker/test/integration/base/Dockerfile index 3e4e88965e0..b6a46f6d934 100644 --- a/docker/test/integration/base/Dockerfile +++ b/docker/test/integration/base/Dockerfile @@ -4,7 +4,7 @@ FROM yandex/clickhouse-test-base RUN apt-get update \ && env DEBIAN_FRONTEND=noninteractive apt-get -y install \ tzdata \ - python \ + python3 \ libreadline-dev \ libicu-dev \ bsdutils \ diff --git a/docker/test/integration/runner/Dockerfile b/docker/test/integration/runner/Dockerfile index bfbe8da816f..795d0d371f6 100644 --- a/docker/test/integration/runner/Dockerfile +++ b/docker/test/integration/runner/Dockerfile @@ -16,13 +16,13 @@ RUN apt-get update \ iproute2 \ module-init-tools \ cgroupfs-mount \ - python-pip \ + python3-pip \ tzdata \ libreadline-dev \ libicu-dev \ bsdutils \ curl \ - python-pika \ + python3-pika \ liblua5.1-dev \ luajit \ libssl-dev \ @@ -37,7 +37,7 @@ RUN apt-get update \ ENV TZ=Europe/Moscow RUN ln -snf /usr/share/zoneinfo/$TZ /etc/localtime && echo $TZ > /etc/timezone -RUN pip install urllib3==1.23 pytest docker-compose==1.22.0 docker dicttoxml kazoo PyMySQL psycopg2==2.7.5 pymongo tzlocal kafka-python protobuf redis aerospike pytest-timeout minio rpm-confluent-schemaregistry grpcio grpcio-tools cassandra-driver +RUN python3 -m pip install urllib3==1.23 pytest docker-compose==1.22.0 docker dicttoxml kazoo PyMySQL psycopg2==2.7.5 pymongo tzlocal kafka-python protobuf redis aerospike pytest-timeout minio grpcio grpcio-tools cassandra-driver confluent-kafka avro ENV DOCKER_CHANNEL stable ENV DOCKER_VERSION 17.09.1-ce diff --git a/docker/test/performance-comparison/report.py b/docker/test/performance-comparison/report.py index 69015c2ce1a..1f55300661b 100755 --- a/docker/test/performance-comparison/report.py +++ b/docker/test/performance-comparison/report.py @@ -312,7 +312,7 @@ def add_errors_explained(): if args.report == 'main': - print(header_template.format()) + print((header_template.format())) add_tested_commits() @@ -571,14 +571,14 @@ if args.report == 'main': status = 'failure' message = 'Errors while building the report.' - print(""" + print((""" - """.format(status=status, message=message)) + """.format(status=status, message=message))) elif args.report == 'all-queries': - print(header_template.format()) + print((header_template.format())) add_tested_commits() diff --git a/docker/test/stateful/Dockerfile b/docker/test/stateful/Dockerfile index 8a7aca80653..07aad75a2ea 100644 --- a/docker/test/stateful/Dockerfile +++ b/docker/test/stateful/Dockerfile @@ -4,7 +4,7 @@ FROM yandex/clickhouse-stateless-test RUN apt-get update -y \ && env DEBIAN_FRONTEND=noninteractive \ apt-get install --yes --no-install-recommends \ - python-requests \ + python3-requests \ llvm-9 COPY s3downloader /s3downloader diff --git a/docker/test/stateful/s3downloader b/docker/test/stateful/s3downloader index fb49931f022..363ece8dac6 100755 --- a/docker/test/stateful/s3downloader +++ b/docker/test/stateful/s3downloader @@ -1,4 +1,4 @@ -#!/usr/bin/env python +#!/usr/bin/env python3 # -*- coding: utf-8 -*- import os import sys @@ -29,7 +29,7 @@ def dowload_with_progress(url, path): logging.info("Downloading from %s to temp path %s", url, path) for i in range(RETRIES_COUNT): try: - with open(path, 'w') as f: + with open(path, 'wb') as f: response = requests.get(url, stream=True) response.raise_for_status() total_length = response.headers.get('content-length') @@ -74,7 +74,7 @@ if __name__ == "__main__": parser = argparse.ArgumentParser( description="Simple tool for dowloading datasets for clickhouse from S3") - parser.add_argument('--dataset-names', required=True, nargs='+', choices=AVAILABLE_DATASETS.keys()) + parser.add_argument('--dataset-names', required=True, nargs='+', choices=list(AVAILABLE_DATASETS.keys())) parser.add_argument('--url-prefix', default=DEFAULT_URL) parser.add_argument('--clickhouse-data-path', default='/var/lib/clickhouse/') diff --git a/docker/test/stateful_with_coverage/Dockerfile b/docker/test/stateful_with_coverage/Dockerfile index 839eea5cdc1..f5d66ed5013 100644 --- a/docker/test/stateful_with_coverage/Dockerfile +++ b/docker/test/stateful_with_coverage/Dockerfile @@ -6,7 +6,7 @@ RUN echo "deb [trusted=yes] http://apt.llvm.org/bionic/ llvm-toolchain-bionic-9 RUN apt-get update -y \ && env DEBIAN_FRONTEND=noninteractive \ apt-get install --yes --no-install-recommends \ - python-requests + python3-requests COPY s3downloader /s3downloader COPY run.sh /run.sh diff --git a/docker/test/stateful_with_coverage/s3downloader b/docker/test/stateful_with_coverage/s3downloader index fb49931f022..a27c03a70f0 100755 --- a/docker/test/stateful_with_coverage/s3downloader +++ b/docker/test/stateful_with_coverage/s3downloader @@ -1,4 +1,4 @@ -#!/usr/bin/env python +#!/usr/bin/env python3 # -*- coding: utf-8 -*- import os import sys @@ -74,7 +74,7 @@ if __name__ == "__main__": parser = argparse.ArgumentParser( description="Simple tool for dowloading datasets for clickhouse from S3") - parser.add_argument('--dataset-names', required=True, nargs='+', choices=AVAILABLE_DATASETS.keys()) + parser.add_argument('--dataset-names', required=True, nargs='+', choices=list(AVAILABLE_DATASETS.keys())) parser.add_argument('--url-prefix', default=DEFAULT_URL) parser.add_argument('--clickhouse-data-path', default='/var/lib/clickhouse/') diff --git a/docker/test/stateless/Dockerfile b/docker/test/stateless/Dockerfile index 516d8d5842b..33eb1c29103 100644 --- a/docker/test/stateless/Dockerfile +++ b/docker/test/stateless/Dockerfile @@ -12,10 +12,10 @@ RUN apt-get update -y \ ncdu \ netcat-openbsd \ openssl \ - python \ - python-lxml \ - python-requests \ - python-termcolor \ + python3 \ + python3-lxml \ + python3-requests \ + python3-termcolor \ qemu-user-static \ sudo \ telnet \ diff --git a/docker/test/stateless_pytest/Dockerfile b/docker/test/stateless_pytest/Dockerfile index 596e2686f49..674a6003187 100644 --- a/docker/test/stateless_pytest/Dockerfile +++ b/docker/test/stateless_pytest/Dockerfile @@ -3,10 +3,10 @@ FROM yandex/clickhouse-test-base RUN apt-get update -y && \ apt-get install -y --no-install-recommends \ - python-pip \ - python-setuptools + python3-pip \ + python3-setuptools -RUN pip install \ +RUN python3 -m pip install \ pytest \ pytest-html \ pytest-timeout \ @@ -17,4 +17,4 @@ CMD dpkg -i package_folder/clickhouse-common-static_*.deb; \ dpkg -i package_folder/clickhouse-server_*.deb; \ dpkg -i package_folder/clickhouse-client_*.deb; \ dpkg -i package_folder/clickhouse-test_*.deb; \ - python -m pytest /usr/share/clickhouse-test/queries -n $(nproc) --html=test_output/report.html --self-contained-html + python3 -m pytest /usr/share/clickhouse-test/queries -n $(nproc) --html=test_output/report.html --self-contained-html diff --git a/docker/test/stateless_unbundled/Dockerfile b/docker/test/stateless_unbundled/Dockerfile index cb8cd158e5f..f2fd28e4078 100644 --- a/docker/test/stateless_unbundled/Dockerfile +++ b/docker/test/stateless_unbundled/Dockerfile @@ -54,10 +54,10 @@ RUN apt-get --allow-unauthenticated update -y \ perl \ pigz \ pkg-config \ - python \ - python-lxml \ - python-requests \ - python-termcolor \ + python3 \ + python3-lxml \ + python3-requests \ + python3-termcolor \ qemu-user-static \ sudo \ telnet \ diff --git a/docker/test/stateless_with_coverage/Dockerfile b/docker/test/stateless_with_coverage/Dockerfile index b76989de1cf..1d6a85adf9e 100644 --- a/docker/test/stateless_with_coverage/Dockerfile +++ b/docker/test/stateless_with_coverage/Dockerfile @@ -12,10 +12,10 @@ RUN apt-get update -y \ fakeroot \ debhelper \ expect \ - python \ - python-lxml \ - python-termcolor \ - python-requests \ + python3 \ + python3-lxml \ + python3-termcolor \ + python3-requests \ sudo \ openssl \ ncdu \ diff --git a/docker/test/stress/Dockerfile b/docker/test/stress/Dockerfile index 6855a632df4..e1df32ec3d7 100644 --- a/docker/test/stress/Dockerfile +++ b/docker/test/stress/Dockerfile @@ -10,10 +10,10 @@ RUN apt-get update -y \ debhelper \ parallel \ expect \ - python \ - python-lxml \ - python-termcolor \ - python-requests \ + python3 \ + python3-lxml \ + python3-termcolor \ + python3-requests \ curl \ sudo \ openssl \ diff --git a/docker/test/stress/stress b/docker/test/stress/stress index a36adda3aad..a81391d56a7 100755 --- a/docker/test/stress/stress +++ b/docker/test/stress/stress @@ -1,4 +1,4 @@ -#!/usr/bin/env python +#!/usr/bin/env python3 # -*- coding: utf-8 -*- from multiprocessing import cpu_count from subprocess import Popen, check_call diff --git a/docs/en/development/build.md b/docs/en/development/build.md index 1c84342baf9..145dd99e8de 100644 --- a/docs/en/development/build.md +++ b/docs/en/development/build.md @@ -116,7 +116,7 @@ ninja Example for Fedora Rawhide: ``` bash sudo yum update -yum --nogpg install git cmake make gcc-c++ python2 +yum --nogpg install git cmake make gcc-c++ python3 git clone --recursive https://github.com/ClickHouse/ClickHouse.git mkdir build && cd build cmake ../ClickHouse diff --git a/docs/es/development/build.md b/docs/es/development/build.md index 861ceb53e51..0b190242f8c 100644 --- a/docs/es/development/build.md +++ b/docs/es/development/build.md @@ -102,7 +102,7 @@ Ejemplo de OpenSUSE Tumbleweed: Ejemplo de Fedora Rawhide: sudo yum update - yum --nogpg install git cmake make gcc-c++ python2 + yum --nogpg install git cmake make gcc-c++ python3 git clone --recursive https://github.com/ClickHouse/ClickHouse.git mkdir build && cd build cmake ../ClickHouse diff --git a/docs/fa/development/build.md b/docs/fa/development/build.md index 8b9d8bba822..7ba21c0fa93 100644 --- a/docs/fa/development/build.md +++ b/docs/fa/development/build.md @@ -103,7 +103,7 @@ $ cd .. به عنوان مثال برای فدورا پوست دباغی نشده: sudo yum update - yum --nogpg install git cmake make gcc-c++ python2 + yum --nogpg install git cmake make gcc-c++ python3 git clone --recursive https://github.com/ClickHouse/ClickHouse.git mkdir build && cd build cmake ../ClickHouse diff --git a/docs/fr/development/build.md b/docs/fr/development/build.md index a57e0f20337..d05f0999720 100644 --- a/docs/fr/development/build.md +++ b/docs/fr/development/build.md @@ -102,7 +102,7 @@ Exemple Pour openSUSE Tumbleweed: Exemple Pour Fedora Rawhide: sudo yum update - yum --nogpg install git cmake make gcc-c++ python2 + yum --nogpg install git cmake make gcc-c++ python3 git clone --recursive https://github.com/ClickHouse/ClickHouse.git mkdir build && cd build cmake ../ClickHouse diff --git a/docs/ja/development/build.md b/docs/ja/development/build.md index c1e92efd441..035d5e7efb1 100644 --- a/docs/ja/development/build.md +++ b/docs/ja/development/build.md @@ -102,7 +102,7 @@ OpenSUSEタンブルウィードの例: Fedora Rawhideの例: sudo yum update - yum --nogpg install git cmake make gcc-c++ python2 + yum --nogpg install git cmake make gcc-c++ python3 git clone --recursive https://github.com/ClickHouse/ClickHouse.git mkdir build && cd build cmake ../ClickHouse diff --git a/docs/tools/easy_diff.py b/docs/tools/easy_diff.py index e4b5b675525..22d305d3da3 100755 --- a/docs/tools/easy_diff.py +++ b/docs/tools/easy_diff.py @@ -1,4 +1,4 @@ -#!/usr/bin/env python +#!/usr/bin/env python3 # -*- coding: utf-8 -*- import os, sys diff --git a/docs/tools/github.py b/docs/tools/github.py index d007752dca9..29184f5e567 100644 --- a/docs/tools/github.py +++ b/docs/tools/github.py @@ -71,8 +71,8 @@ def choose_latest_releases(args): logging.fatal('Unexpected GitHub response: %s', str(candidates)) sys.exit(1) - logging.info('Found LTS releases: %s', ', '.join(seen_lts.keys())) - logging.info('Found stable releases: %s', ', '.join(seen_stable.keys())) + logging.info('Found LTS releases: %s', ', '.join(list(seen_lts.keys()))) + logging.info('Found stable releases: %s', ', '.join(list(seen_stable.keys()))) return sorted(list(seen_lts.items()) + list(seen_stable.items())) diff --git a/docs/tools/mdx_clickhouse.py b/docs/tools/mdx_clickhouse.py index 80ecf829341..b4c255066ff 100755 --- a/docs/tools/mdx_clickhouse.py +++ b/docs/tools/mdx_clickhouse.py @@ -1,6 +1,6 @@ -#!/usr/bin/env python +#!/usr/bin/env python3 # -*- coding: utf-8 -*- -from __future__ import unicode_literals + import datetime import os diff --git a/docs/tools/nav.py b/docs/tools/nav.py index d14791a5b9b..291797a1633 100644 --- a/docs/tools/nav.py +++ b/docs/tools/nav.py @@ -59,7 +59,7 @@ def build_docs_nav(lang, args): _, _, nav = build_nav_entry(docs_dir, args) result = [] index_key = None - for key, value in nav.items(): + for key, value in list(nav.items()): if key and value: if value == 'index.md': index_key = key diff --git a/docs/tools/purge_cache_for_changed_files.py b/docs/tools/purge_cache_for_changed_files.py index 761697770b7..6cfc9d18a57 100644 --- a/docs/tools/purge_cache_for_changed_files.py +++ b/docs/tools/purge_cache_for_changed_files.py @@ -59,7 +59,7 @@ def convert_to_dicts(changed_files, batch_size): def post_data(prepared_batches, token): headers = {"Authorization": "Bearer {}".format(token)} for batch in prepared_batches: - print("Pugring cache for", ", ".join(batch["files"])) + print(("Pugring cache for", ", ".join(batch["files"]))) response = requests.post(CLOUDFLARE_URL, json=batch, headers=headers) response.raise_for_status() time.sleep(3) @@ -71,8 +71,8 @@ if __name__ == "__main__": raise Exception("Env variable CLOUDFLARE_TOKEN is empty") base_domain = os.getenv("BASE_DOMAIN", "https://content.clickhouse.tech/") changed_files = collect_changed_files() - print("Found", len(changed_files), "changed files") + print(("Found", len(changed_files), "changed files")) filtered_files = filter_and_transform_changed_files(changed_files, base_domain) - print("Files rest after filtering", len(filtered_files)) + print(("Files rest after filtering", len(filtered_files))) prepared_batches = convert_to_dicts(filtered_files, 25) post_data(prepared_batches, token) diff --git a/docs/tools/single_page.py b/docs/tools/single_page.py index b074cd42329..004409fe281 100644 --- a/docs/tools/single_page.py +++ b/docs/tools/single_page.py @@ -15,7 +15,7 @@ import website def recursive_values(item): if isinstance(item, dict): - for _, value in item.items(): + for _, value in list(item.items()): yield from recursive_values(value) elif isinstance(item, list): for value in item: diff --git a/docs/tools/test.py b/docs/tools/test.py index d963d34df08..7d11157c986 100755 --- a/docs/tools/test.py +++ b/docs/tools/test.py @@ -1,6 +1,5 @@ #!/usr/bin/env python3 -# -*- coding: utf-8 -*- -from __future__ import unicode_literals + import logging import os import sys diff --git a/docs/tools/translate/typograph_ru.py b/docs/tools/translate/typograph_ru.py index db20109efff..2d970cf2a2e 100644 --- a/docs/tools/translate/typograph_ru.py +++ b/docs/tools/translate/typograph_ru.py @@ -42,4 +42,4 @@ def typograph(text): if __name__ == '__main__': import sys - print(typograph(sys.stdin.read())) + print((typograph(sys.stdin.read()))) diff --git a/docs/tr/development/build.md b/docs/tr/development/build.md index 18ef2cd66ae..90cb166094d 100644 --- a/docs/tr/development/build.md +++ b/docs/tr/development/build.md @@ -102,7 +102,7 @@ OpenSUSE Tumbleweed için örnek: Fedora Rawhide için örnek: sudo yum update - yum --nogpg install git cmake make gcc-c++ python2 + yum --nogpg install git cmake make gcc-c++ python3 git clone --recursive https://github.com/ClickHouse/ClickHouse.git mkdir build && cd build cmake ../ClickHouse diff --git a/release b/release index b446ceca0d5..f2052840cb0 100755 --- a/release +++ b/release @@ -66,7 +66,7 @@ do shift elif [[ $1 == '--fast' ]]; then # Wrong but fast pbuilder mode: create base package with all depends - EXTRAPACKAGES="$EXTRAPACKAGES debhelper cmake ninja-build gcc-8 g++-8 libc6-dev libicu-dev libreadline-dev psmisc bash expect python python-lxml python-termcolor python-requests curl perl sudo openssl netcat-openbsd" + EXTRAPACKAGES="$EXTRAPACKAGES debhelper cmake ninja-build gcc-8 g++-8 libc6-dev libicu-dev libreadline-dev psmisc bash expect python3 python3-lxml python3-termcolor python3-requests curl perl sudo openssl netcat-openbsd" shift elif [[ $1 == '--rpm' ]]; then MAKE_RPM=1 diff --git a/src/Storages/tests/active_parts.py b/src/Storages/tests/active_parts.py index 6872baf6368..a818a76017d 100644 --- a/src/Storages/tests/active_parts.py +++ b/src/Storages/tests/active_parts.py @@ -24,16 +24,16 @@ for s in sys.stdin.read().split(): parts[m1].append((i1, i2, l, s)) for m, ps in sorted(parts.items()): - ps.sort(key=lambda (i1, i2, l, s): (i1, -i2, -l)) + ps.sort(key=lambda i1_i2_l_s: (i1_i2_l_s[0], -i1_i2_l_s[1], -i1_i2_l_s[2])) (x2, y2, l2, s2) = (-1, -1, -1, -1) for x1, y1, l1, s1 in ps: if x1 >= x2 and y1 <= y2 and l1 < l2 and (x1, y1) != (x2, y2): # 2 contains 1 pass elif x1 > y2: # 1 is to the right of 2 if x1 != y2 + 1 and y2 != -1: - print # to see the missing numbers + print() # to see the missing numbers (x2, y2, l2, s2) = (x1, y1, l1, s1) - print s1 + print(s1) else: raise Exception('invalid parts intersection: ' + s1 + ' and ' + s2) - print + print() diff --git a/tests/clickhouse-test b/tests/clickhouse-test index 2a9c95eb830..348df2f93be 100755 --- a/tests/clickhouse-test +++ b/tests/clickhouse-test @@ -1,5 +1,5 @@ -#!/usr/bin/env python2 -from __future__ import print_function +#!/usr/bin/env python3 + import sys import os import os.path @@ -23,7 +23,7 @@ try: except ImportError: termcolor = None from random import random -import commands +import subprocess import multiprocessing from contextlib import closing @@ -99,7 +99,7 @@ def remove_control_characters(s): """ def str_to_int(s, default, base=10): if int(s, base) < 0x10000: - return unichr(int(s, base)) + return chr(int(s, base)) return default s = re.sub(r"&#(\d+);?", lambda c: str_to_int(c.group(1), c.group(0)), s) s = re.sub(r"&#[xX]([0-9a-fA-F]+);?", lambda c: str_to_int(c.group(1), c.group(0), base=16), s) @@ -129,8 +129,8 @@ def run_single_test(args, ext, server_logs_level, client_options, case_file, std return ''.join(random.choice(alphabet) for _ in range(length)) database = 'test_{suffix}'.format(suffix=random_str()) - clickhouse_proc_create = Popen(shlex.split(args.client), stdin=PIPE, stdout=PIPE, stderr=PIPE) - clickhouse_proc_create.communicate("CREATE DATABASE " + database + get_db_engine(args)) + clickhouse_proc_create = Popen(shlex.split(args.client), stdin=PIPE, stdout=PIPE, stderr=PIPE, universal_newlines=True) + clickhouse_proc_create.communicate(("CREATE DATABASE " + database + get_db_engine(args))) os.environ["CLICKHOUSE_DATABASE"] = database @@ -157,8 +157,8 @@ def run_single_test(args, ext, server_logs_level, client_options, case_file, std sleep(0.01) if not args.database: - clickhouse_proc_create = Popen(shlex.split(args.client), stdin=PIPE, stdout=PIPE, stderr=PIPE) - clickhouse_proc_create.communicate("DROP DATABASE " + database) + clickhouse_proc_create = Popen(shlex.split(args.client), stdin=PIPE, stdout=PIPE, stderr=PIPE, universal_newlines=True) + clickhouse_proc_create.communicate(("DROP DATABASE " + database)) total_time = (datetime.now() - start_time).total_seconds() @@ -166,10 +166,10 @@ def run_single_test(args, ext, server_logs_level, client_options, case_file, std os.system("LC_ALL=C sed -i -e 's/{test_db}/default/g' {file}".format(test_db=database, file=stdout_file)) os.system("LC_ALL=C sed -i -e 's/{test_db}/default/g' {file}".format(test_db=database, file=stderr_file)) - stdout = open(stdout_file, 'r').read() if os.path.exists(stdout_file) else '' - stdout = unicode(stdout, errors='replace', encoding='utf-8') - stderr = open(stderr_file, 'r').read() if os.path.exists(stderr_file) else '' - stderr = unicode(stderr, errors='replace', encoding='utf-8') + stdout = open(stdout_file, 'rb').read() if os.path.exists(stdout_file) else b'' + stdout = str(stdout, errors='replace', encoding='utf-8') + stderr = open(stderr_file, 'rb').read() if os.path.exists(stderr_file) else b'' + stderr = str(stderr, errors='replace', encoding='utf-8') return proc, stdout, stderr, total_time @@ -300,8 +300,8 @@ def run_tests_array(all_tests_with_params): else: if args.testname: - clickhouse_proc = Popen(shlex.split(args.client), stdin=PIPE, stdout=PIPE, stderr=PIPE) - clickhouse_proc.communicate("SELECT 'Running test {suite}/{case} from pid={pid}';".format(pid = os.getpid(), case = case, suite = suite)) + clickhouse_proc = Popen(shlex.split(args.client), stdin=PIPE, stdout=PIPE, stderr=PIPE, universal_newlines=True) + clickhouse_proc.communicate(("SELECT 'Running test {suite}/{case} from pid={pid}';".format(pid = os.getpid(), case = case, suite = suite))) if clickhouse_proc.returncode != 0: failures += 1 @@ -342,7 +342,7 @@ def run_tests_array(all_tests_with_params): print(" - return code {}".format(proc.returncode)) if stderr: - print(stderr.encode('utf-8')) + print(stderr) # Stop on fatal errors like segmentation fault. They are send to client via logs. if ' ' in stderr: @@ -360,22 +360,22 @@ def run_tests_array(all_tests_with_params): failures_chain += 1 print(MSG_FAIL, end='') print_test_time(total_time) - print(" - having stderror:\n{}".format(stderr.encode('utf-8'))) + print(" - having stderror:\n{}".format(stderr)) elif 'Exception' in stdout: failures += 1 failures_chain += 1 print(MSG_FAIL, end='') print_test_time(total_time) - print(" - having exception:\n{}".format(stdout.encode('utf-8'))) + print(" - having exception:\n{}".format(stdout)) elif not os.path.isfile(reference_file): print(MSG_UNKNOWN, end='') print_test_time(total_time) print(" - no reference file") else: - result_is_different = subprocess.call(['diff', '-q', reference_file, stdout_file], stdout = PIPE) + result_is_different = subprocess.call(['diff', '-q', reference_file, stdout_file], stdout=PIPE) if result_is_different: - diff = Popen(['diff', '-U', str(args.unified), reference_file, stdout_file], stdout = PIPE).communicate()[0] + diff = Popen(['diff', '-U', str(args.unified), reference_file, stdout_file], stdout=PIPE, universal_newlines=True).communicate()[0] failures += 1 print(MSG_FAIL, end='') print_test_time(total_time) @@ -419,9 +419,9 @@ def check_server_started(client, retry_count): sys.stdout.flush() while retry_count > 0: clickhouse_proc = Popen(shlex.split(client), stdin=PIPE, stdout=PIPE, stderr=PIPE) - (stdout, stderr) = clickhouse_proc.communicate("SELECT 1") + (stdout, stderr) = clickhouse_proc.communicate(b"SELECT 1") - if clickhouse_proc.returncode == 0 and stdout.startswith("1"): + if clickhouse_proc.returncode == 0 and stdout.startswith(b"1"): print(" OK") sys.stdout.flush() return True @@ -468,46 +468,46 @@ class BuildFlags(object): def collect_build_flags(client): clickhouse_proc = Popen(shlex.split(client), stdin=PIPE, stdout=PIPE, stderr=PIPE) - (stdout, stderr) = clickhouse_proc.communicate("SELECT value FROM system.build_options WHERE name = 'CXX_FLAGS'") + (stdout, stderr) = clickhouse_proc.communicate(b"SELECT value FROM system.build_options WHERE name = 'CXX_FLAGS'") result = [] if clickhouse_proc.returncode == 0: - if '-fsanitize=thread' in stdout: + if b'-fsanitize=thread' in stdout: result.append(BuildFlags.THREAD) - elif '-fsanitize=address' in stdout: + elif b'-fsanitize=address' in stdout: result.append(BuildFlags.ADDRESS) - elif '-fsanitize=undefined' in stdout: + elif b'-fsanitize=undefined' in stdout: result.append(BuildFlags.UNDEFINED) - elif '-fsanitize=memory' in stdout: + elif b'-fsanitize=memory' in stdout: result.append(BuildFlags.MEMORY) else: raise Exception("Cannot get inforamtion about build from server errorcode {}, stderr {}".format(clickhouse_proc.returncode, stderr)) clickhouse_proc = Popen(shlex.split(client), stdin=PIPE, stdout=PIPE, stderr=PIPE) - (stdout, stderr) = clickhouse_proc.communicate("SELECT value FROM system.build_options WHERE name = 'BUILD_TYPE'") + (stdout, stderr) = clickhouse_proc.communicate(b"SELECT value FROM system.build_options WHERE name = 'BUILD_TYPE'") if clickhouse_proc.returncode == 0: - if 'Debug' in stdout: + if b'Debug' in stdout: result.append(BuildFlags.DEBUG) - elif 'RelWithDebInfo' in stdout or 'Release' in stdout: + elif b'RelWithDebInfo' in stdout or b'Release' in stdout: result.append(BuildFlags.RELEASE) else: raise Exception("Cannot get inforamtion about build from server errorcode {}, stderr {}".format(clickhouse_proc.returncode, stderr)) clickhouse_proc = Popen(shlex.split(client), stdin=PIPE, stdout=PIPE, stderr=PIPE) - (stdout, stderr) = clickhouse_proc.communicate("SELECT value FROM system.build_options WHERE name = 'UNBUNDLED'") + (stdout, stderr) = clickhouse_proc.communicate(b"SELECT value FROM system.build_options WHERE name = 'UNBUNDLED'") if clickhouse_proc.returncode == 0: - if 'ON' in stdout or '1' in stdout: + if b'ON' in stdout or b'1' in stdout: result.append(BuildFlags.UNBUNDLED) else: raise Exception("Cannot get inforamtion about build from server errorcode {}, stderr {}".format(clickhouse_proc.returncode, stderr)) clickhouse_proc = Popen(shlex.split(client), stdin=PIPE, stdout=PIPE, stderr=PIPE) - (stdout, stderr) = clickhouse_proc.communicate("SELECT value FROM system.settings WHERE name = 'default_database_engine'") + (stdout, stderr) = clickhouse_proc.communicate(b"SELECT value FROM system.settings WHERE name = 'default_database_engine'") if clickhouse_proc.returncode == 0: - if 'Ordinary' in stdout: + if b'Ordinary' in stdout: result.append(BuildFlags.DATABASE_ORDINARY) else: raise Exception("Cannot get inforamtion about build from server errorcode {}, stderr {}".format(clickhouse_proc.returncode, stderr)) @@ -523,11 +523,11 @@ def main(args): def is_data_present(): clickhouse_proc = Popen(shlex.split(args.client), stdin=PIPE, stdout=PIPE, stderr=PIPE) - (stdout, stderr) = clickhouse_proc.communicate("EXISTS TABLE test.hits") + (stdout, stderr) = clickhouse_proc.communicate(b"EXISTS TABLE test.hits") if clickhouse_proc.returncode != 0: raise CalledProcessError(clickhouse_proc.returncode, args.client, stderr) - return stdout.startswith('1') + return stdout.startswith(b'1') if not check_server_started(args.client, args.server_check_retries): raise Exception("clickhouse-server is not responding. Cannot execute 'SELECT 1' query.") @@ -562,7 +562,7 @@ def main(args): stop_time = time() + args.global_time_limit if args.zookeeper is None: - code, out = commands.getstatusoutput(args.extract_from_config + " --try --config " + args.configserver + ' --key zookeeper | grep . | wc -l') + code, out = subprocess.getstatusoutput(args.extract_from_config + " --try --config " + args.configserver + ' --key zookeeper | grep . | wc -l') try: if int(out) > 0: args.zookeeper = True @@ -572,18 +572,18 @@ def main(args): args.zookeeper = False if args.shard is None: - code, out = commands.getstatusoutput(args.extract_from_config + " --try --config " + args.configserver + ' --key listen_host | grep -E "127.0.0.2|::"') + code, out = subprocess.getstatusoutput(args.extract_from_config + " --try --config " + args.configserver + ' --key listen_host | grep -E "127.0.0.2|::"') if out: args.shard = True else: args.shard = False if args.database and args.database != "test": - clickhouse_proc_create = Popen(shlex.split(args.client), stdin=PIPE, stdout=PIPE, stderr=PIPE) - clickhouse_proc_create.communicate("CREATE DATABASE IF NOT EXISTS " + args.database + get_db_engine(args)) + clickhouse_proc_create = Popen(shlex.split(args.client), stdin=PIPE, stdout=PIPE, stderr=PIPE, universal_newlines=True) + clickhouse_proc_create.communicate(("CREATE DATABASE IF NOT EXISTS " + args.database + get_db_engine(args))) - clickhouse_proc_create = Popen(shlex.split(args.client), stdin=PIPE, stdout=PIPE, stderr=PIPE) - clickhouse_proc_create.communicate("CREATE DATABASE IF NOT EXISTS test" + get_db_engine(args)) + clickhouse_proc_create = Popen(shlex.split(args.client), stdin=PIPE, stdout=PIPE, stderr=PIPE, universal_newlines=True) + clickhouse_proc_create.communicate(("CREATE DATABASE IF NOT EXISTS test" + get_db_engine(args))) def is_test_from_dir(suite_dir, case): case_file = os.path.join(suite_dir, case) @@ -595,14 +595,14 @@ def main(args): return random() if -1 == item.find('_'): - return 99998 + return 99998, '' prefix, suffix = item.split('_', 1) try: return int(prefix), suffix except ValueError: - return 99997 + return 99997, '' total_tests_run = 0 for suite in sorted(os.listdir(base_dir), key=sute_key_func): @@ -650,7 +650,7 @@ def main(args): return 99997 all_tests = os.listdir(suite_dir) - all_tests = filter(lambda case: is_test_from_dir(suite_dir, case), all_tests) + all_tests = [case for case in all_tests if is_test_from_dir(suite_dir, case)] if args.test: all_tests = [t for t in all_tests if any([re.search(r, t) for r in args.test])] all_tests.sort(key=key_func) @@ -670,7 +670,7 @@ def main(args): if jobs > run_total: run_total = jobs - batch_size = len(all_tests) / jobs + batch_size = len(all_tests) // jobs all_tests_array = [] for i in range(0, len(all_tests), batch_size): all_tests_array.append((all_tests[i:i+batch_size], suite, suite_dir, suite_tmp_dir, run_total)) diff --git a/tests/external_models/catboost/helpers/server.py b/tests/external_models/catboost/helpers/server.py index 265898d8d41..8248b16e6df 100644 --- a/tests/external_models/catboost/helpers/server.py +++ b/tests/external_models/catboost/helpers/server.py @@ -37,7 +37,7 @@ class ClickHouseServer: s.connect(('localhost', port)) except socket.error as socketerror: - print "Error: ", socketerror + print("Error: ", socketerror) raise def shutdown(self, timeout=10): diff --git a/tests/external_models/catboost/helpers/server_with_models.py b/tests/external_models/catboost/helpers/server_with_models.py index ad9feea99fe..e00da7b7027 100644 --- a/tests/external_models/catboost/helpers/server_with_models.py +++ b/tests/external_models/catboost/helpers/server_with_models.py @@ -1,6 +1,6 @@ -from server import ClickHouseServer -from client import ClickHouseClient -from table import ClickHouseTable +from .server import ClickHouseServer +from .client import ClickHouseClient +from .table import ClickHouseTable import os import errno from shutil import rmtree @@ -140,7 +140,7 @@ class ClickHouseServerWithCatboostModels: if not os.path.exists(self.models_dir): os.makedirs(self.models_dir) - for name, model in self.models.items(): + for name, model in list(self.models.items()): model_path = os.path.join(self.models_dir, name + '.cbm') config_path = os.path.join(self.models_dir, name + '_model.xml') params = { diff --git a/tests/external_models/catboost/helpers/table.py b/tests/external_models/catboost/helpers/table.py index e6b05ac7b7b..5f9b828c9f3 100644 --- a/tests/external_models/catboost/helpers/table.py +++ b/tests/external_models/catboost/helpers/table.py @@ -1,5 +1,5 @@ -from server import ClickHouseServer -from client import ClickHouseClient +from .server import ClickHouseServer +from .client import ClickHouseClient from pandas import DataFrame import os import threading @@ -40,7 +40,7 @@ class ClickHouseTable: column_types = list(self.df.dtypes) column_names = list(self.df) schema = ', '.join((name + ' ' + self._convert(str(t)) for name, t in zip(column_names, column_types))) - print 'schema:', schema + print('schema:', schema) create_query = 'create table test.{} (date Date DEFAULT today(), {}) engine = MergeTree(date, (date), 8192)' self.client.query(create_query.format(self.table_name, schema)) @@ -58,10 +58,10 @@ class ClickHouseTable: result = self.client.query(query.format(model_name, columns, self.table_name)) def parse_row(row): - values = tuple(map(float, filter(len, map(str.strip, row.replace('(', '').replace(')', '').split(','))))) + values = tuple(map(float, list(filter(len, list(map(str.strip, row.replace('(', '').replace(')', '').split(','))))))) return values if len(values) != 1 else values[0] - return tuple(map(parse_row, filter(len, map(str.strip, result.split('\n'))))) + return tuple(map(parse_row, list(filter(len, list(map(str.strip, result.split('\n'))))))) def _drop_table(self): self.client.query('drop table test.{}'.format(self.table_name)) diff --git a/tests/external_models/catboost/helpers/train.py b/tests/external_models/catboost/helpers/train.py index df81d719553..34a6f8e958b 100644 --- a/tests/external_models/catboost/helpers/train.py +++ b/tests/external_models/catboost/helpers/train.py @@ -19,10 +19,10 @@ def train_catboost_model(df, target, cat_features, params, verbose=True): if not isinstance(df, DataFrame): raise Exception('DataFrame object expected, but got ' + repr(df)) - print 'features:', df.columns.tolist() + print('features:', df.columns.tolist()) cat_features_index = list(df.columns.get_loc(feature) for feature in cat_features) - print 'cat features:', cat_features_index + print('cat features:', cat_features_index) model = CatBoostClassifier(**params) model.fit(df, target, cat_features=cat_features_index, verbose=verbose) return model diff --git a/tests/external_models/catboost/test_apply_catboost_model/test.py b/tests/external_models/catboost/test_apply_catboost_model/test.py index 00b9fe0dce1..d266393bf48 100644 --- a/tests/external_models/catboost/test_apply_catboost_model/test.py +++ b/tests/external_models/catboost/test_apply_catboost_model/test.py @@ -23,7 +23,7 @@ def check_predictions(test_name, target, pred_python, pred_ch, acc_threshold): acc = 1 - np.sum(np.abs(ch_class - np.array(target))) / (len(target) + .0) assert acc >= acc_threshold - print test_name, 'accuracy: {:.10f}'.format(acc) + print(test_name, 'accuracy: {:.10f}'.format(acc)) def test_apply_float_features_only(): @@ -52,9 +52,9 @@ def test_apply_float_features_only(): train_target = get_target(train_df) test_target = get_target(test_df) - print - print 'train target', train_target - print 'test target', test_target + print() + print('train target', train_target) + print('test target', test_target) params = { 'iterations': 4, @@ -71,8 +71,8 @@ def test_apply_float_features_only(): with server: pred_ch = (np.array(server.apply_model(name, test_df, [])) > 0).astype(int) - print 'python predictions', pred_python - print 'clickhouse predictions', pred_ch + print('python predictions', pred_python) + print('clickhouse predictions', pred_ch) check_predictions(name, test_target, pred_python, pred_ch, 0.9) @@ -105,9 +105,9 @@ def test_apply_float_features_with_string_cat_features(): train_target = get_target(train_df) test_target = get_target(test_df) - print - print 'train target', train_target - print 'test target', test_target + print() + print('train target', train_target) + print('test target', test_target) params = { 'iterations': 6, @@ -124,8 +124,8 @@ def test_apply_float_features_with_string_cat_features(): with server: pred_ch = (np.array(server.apply_model(name, test_df, [])) > 0).astype(int) - print 'python predictions', pred_python - print 'clickhouse predictions', pred_ch + print('python predictions', pred_python) + print('clickhouse predictions', pred_ch) check_predictions(name, test_target, pred_python, pred_ch, 0.9) @@ -158,9 +158,9 @@ def test_apply_float_features_with_int_cat_features(): train_target = get_target(train_df) test_target = get_target(test_df) - print - print 'train target', train_target - print 'test target', test_target + print() + print('train target', train_target) + print('test target', test_target) params = { 'iterations': 6, @@ -177,8 +177,8 @@ def test_apply_float_features_with_int_cat_features(): with server: pred_ch = (np.array(server.apply_model(name, test_df, [])) > 0).astype(int) - print 'python predictions', pred_python - print 'clickhouse predictions', pred_ch + print('python predictions', pred_python) + print('clickhouse predictions', pred_ch) check_predictions(name, test_target, pred_python, pred_ch, 0.9) @@ -211,9 +211,9 @@ def test_apply_float_features_with_mixed_cat_features(): train_target = get_target(train_df) test_target = get_target(test_df) - print - print 'train target', train_target - print 'test target', test_target + print() + print('train target', train_target) + print('test target', test_target) params = { 'iterations': 6, @@ -230,8 +230,8 @@ def test_apply_float_features_with_mixed_cat_features(): with server: pred_ch = (np.array(server.apply_model(name, test_df, [])) > 0).astype(int) - print 'python predictions', pred_python - print 'clickhouse predictions', pred_ch + print('python predictions', pred_python) + print('clickhouse predictions', pred_ch) check_predictions(name, test_target, pred_python, pred_ch, 0.9) @@ -269,9 +269,9 @@ def test_apply_multiclass(): train_target = get_target(train_df) test_target = get_target(test_df) - print - print 'train target', train_target - print 'test target', test_target + print() + print('train target', train_target) + print('test target', test_target) params = { 'iterations': 10, @@ -288,7 +288,7 @@ def test_apply_multiclass(): with server: pred_ch = np.argmax(np.array(server.apply_model(name, test_df, [])), axis=1) - print 'python predictions', pred_python - print 'clickhouse predictions', pred_ch + print('python predictions', pred_python) + print('clickhouse predictions', pred_ch) check_predictions(name, test_target, pred_python, pred_ch, 0.9) diff --git a/tests/integration/README.md b/tests/integration/README.md index a3eb577d609..94a6000d707 100644 --- a/tests/integration/README.md +++ b/tests/integration/README.md @@ -12,11 +12,11 @@ You must install latest Docker from https://docs.docker.com/engine/installation/linux/docker-ce/ubuntu/#set-up-the-repository Don't use Docker from your system repository. -* [pip](https://pypi.python.org/pypi/pip) and `libpq-dev`. To install: `sudo apt-get install python-pip libpq-dev zlib1g-dev libcrypto++-dev libssl-dev` +* [pip](https://pypi.python.org/pypi/pip) and `libpq-dev`. To install: `sudo apt-get install python3-pip libpq-dev zlib1g-dev libcrypto++-dev libssl-dev` * [py.test](https://docs.pytest.org/) testing framework. To install: `sudo -H pip install pytest` -* [docker-compose](https://docs.docker.com/compose/) and additional python libraries. To install: `sudo -H pip install urllib3==1.23 pytest docker-compose==1.22.0 docker dicttoxml kazoo PyMySQL psycopg2==2.7.5 pymongo tzlocal kafka-python protobuf redis aerospike pytest-timeout minio rpm-confluent-schemaregistry` +* [docker-compose](https://docs.docker.com/compose/) and additional python libraries. To install: `sudo -H pip install urllib3==1.23 pytest docker-compose==1.22.0 docker dicttoxml kazoo PyMySQL psycopg2==2.7.5 pymongo tzlocal kafka-python protobuf redis aerospike pytest-timeout minio confluent-kafka avro -(highly not recommended) If you really want to use OS packages on modern debian/ubuntu instead of "pip": `sudo apt install -y docker docker-compose python-pytest python-dicttoxml python-docker python-pymysql python-pymongo python-tzlocal python-kazoo python-psycopg2 python-kafka python-pytest-timeout python-minio` +(highly not recommended) If you really want to use OS packages on modern debian/ubuntu instead of "pip": `sudo apt install -y docker docker-compose python3-pytest python3-dicttoxml python3-docker python3-pymysql python3-pymongo python3-tzlocal python3-kazoo python3-psycopg2 kafka-python python3-pytest-timeout python3-minio` If you want to run the tests under a non-privileged user, you must add this user to `docker` group: `sudo usermod -aG docker $USER` and re-login. (You must close all your sessions (for example, restart your computer)) diff --git a/tests/integration/helpers/client.py b/tests/integration/helpers/client.py index deffa20753f..a43440e41f7 100644 --- a/tests/integration/helpers/client.py +++ b/tests/integration/helpers/client.py @@ -31,7 +31,7 @@ class Client: command += ['--query', sql] if settings is not None: - for setting, value in settings.iteritems(): + for setting, value in settings.items(): command += ['--' + setting, str(value)] if user is not None: @@ -67,7 +67,7 @@ class QueryRuntimeException(Exception): class CommandRequest: def __init__(self, command, stdin=None, timeout=None, ignore_error=False): # Write data to tmp file to avoid PIPEs and execution blocking - stdin_file = tempfile.TemporaryFile() + stdin_file = tempfile.TemporaryFile(mode='w+') stdin_file.write(stdin) stdin_file.seek(0) self.stdout_file = tempfile.TemporaryFile() @@ -80,7 +80,7 @@ class CommandRequest: # can print some debug information there env = {} env["TSAN_OPTIONS"] = "verbosity=0" - self.process = sp.Popen(command, stdin=stdin_file, stdout=self.stdout_file, stderr=self.stderr_file, env=env) + self.process = sp.Popen(command, stdin=stdin_file, stdout=self.stdout_file, stderr=self.stderr_file, env=env, universal_newlines=True) self.timer = None self.process_finished_before_timeout = True @@ -98,8 +98,8 @@ class CommandRequest: self.stdout_file.seek(0) self.stderr_file.seek(0) - stdout = self.stdout_file.read() - stderr = self.stderr_file.read() + stdout = self.stdout_file.read().decode() + stderr = self.stderr_file.read().decode() if self.timer is not None and not self.process_finished_before_timeout and not self.ignore_error: raise QueryTimeoutExceedException('Client timed out!') @@ -115,8 +115,8 @@ class CommandRequest: self.stdout_file.seek(0) self.stderr_file.seek(0) - stdout = self.stdout_file.read() - stderr = self.stderr_file.read() + stdout = self.stdout_file.read().decode() + stderr = self.stderr_file.read().decode() if self.timer is not None and not self.process_finished_before_timeout and not self.ignore_error: raise QueryTimeoutExceedException('Client timed out!') @@ -131,8 +131,8 @@ class CommandRequest: self.stdout_file.seek(0) self.stderr_file.seek(0) - stdout = self.stdout_file.read() - stderr = self.stderr_file.read() + stdout = self.stdout_file.read().decode() + stderr = self.stderr_file.read().decode() if self.timer is not None and not self.process_finished_before_timeout and not self.ignore_error: raise QueryTimeoutExceedException('Client timed out!') diff --git a/tests/integration/helpers/cluster.py b/tests/integration/helpers/cluster.py index d99d8a17844..6b24bc30460 100644 --- a/tests/integration/helpers/cluster.py +++ b/tests/integration/helpers/cluster.py @@ -1,6 +1,6 @@ import base64 import errno -import httplib +import http.client import logging import os import os.path as p @@ -12,7 +12,7 @@ import socket import subprocess import time import traceback -import urllib +import urllib.parse import cassandra.cluster import docker @@ -21,7 +21,7 @@ import pymongo import pymysql import requests import xml.dom.minidom -from confluent.schemaregistry.client import CachedSchemaRegistryClient +from confluent_kafka.avro.cached_schema_registry_client import CachedSchemaRegistryClient from dicttoxml import dicttoxml from kazoo.client import KazooClient from kazoo.exceptions import KazooException @@ -41,7 +41,7 @@ SANITIZER_SIGN = "==================" def _create_env_file(path, variables, fname=DEFAULT_ENV_NAME): full_path = os.path.join(path, fname) with open(full_path, 'w') as f: - for var, value in variables.items(): + for var, value in list(variables.items()): f.write("=".join([var, value]) + "\n") return full_path @@ -76,7 +76,7 @@ def get_docker_compose_path(): if os.path.exists(os.path.dirname('/compose/')): return os.path.dirname('/compose/') # default in docker runner container else: - print("Fallback docker_compose_path to LOCAL_DOCKER_COMPOSE_DIR: {}".format(LOCAL_DOCKER_COMPOSE_DIR)) + print(("Fallback docker_compose_path to LOCAL_DOCKER_COMPOSE_DIR: {}".format(LOCAL_DOCKER_COMPOSE_DIR))) return LOCAL_DOCKER_COMPOSE_DIR @@ -91,8 +91,8 @@ class ClickHouseCluster: def __init__(self, base_path, name=None, base_config_dir=None, server_bin_path=None, client_bin_path=None, odbc_bridge_bin_path=None, zookeeper_config_path=None, custom_dockerd_host=None): - for param in os.environ.keys(): - print "ENV %40s %s" % (param, os.environ[param]) + for param in list(os.environ.keys()): + print("ENV %40s %s" % (param, os.environ[param])) self.base_dir = p.dirname(base_path) self.name = name if name is not None else '' @@ -160,7 +160,7 @@ class ClickHouseCluster: self.docker_client = None self.is_up = False - print "CLUSTER INIT base_config_dir:{}".format(self.base_config_dir) + print("CLUSTER INIT base_config_dir:{}".format(self.base_config_dir)) def get_client_cmd(self): cmd = self.client_bin_path @@ -386,7 +386,7 @@ class ClickHouseCluster: def get_instance_ip(self, instance_name): docker_id = self.get_instance_docker_id(instance_name) handle = self.docker_client.containers.get(docker_id) - return handle.attrs['NetworkSettings']['Networks'].values()[0]['IPAddress'] + return list(handle.attrs['NetworkSettings']['Networks'].values())[0]['IPAddress'] def get_container_id(self, instance_name): docker_id = self.get_instance_docker_id(instance_name) @@ -395,22 +395,21 @@ class ClickHouseCluster: def get_container_logs(self, instance_name): container_id = self.get_container_id(instance_name) - return self.docker_client.api.logs(container_id) + return self.docker_client.api.logs(container_id).decode() def exec_in_container(self, container_id, cmd, detach=False, nothrow=False, **kwargs): exec_id = self.docker_client.api.exec_create(container_id, cmd, **kwargs) output = self.docker_client.api.exec_start(exec_id, detach=detach) - output = output.decode('utf8') exit_code = self.docker_client.api.exec_inspect(exec_id)['ExitCode'] if exit_code: container_info = self.docker_client.api.inspect_container(container_id) image_id = container_info.get('Image') image_info = self.docker_client.api.inspect_image(image_id) - print("Command failed in container {}: ".format(container_id)) + print(("Command failed in container {}: ".format(container_id))) pprint.pprint(container_info) print("") - print("Container {} uses image {}: ".format(container_id, image_id)) + print(("Container {} uses image {}: ".format(container_id, image_id))) pprint.pprint(image_info) print("") message = 'Cmd "{}" failed in container {}. Return code {}. Output: {}'.format(' '.join(cmd), container_id, @@ -419,14 +418,17 @@ class ClickHouseCluster: print(message) else: raise Exception(message) + if not detach: + return output.decode() return output def copy_file_to_container(self, container_id, local_path, dest_path): - with open(local_path, 'r') as fdata: + with open(local_path, "r") as fdata: data = fdata.read() - encoded_data = base64.b64encode(data) + encodedBytes = base64.b64encode(data.encode("utf-8")) + encodedStr = str(encodedBytes, "utf-8") self.exec_in_container(container_id, - ["bash", "-c", "echo {} | base64 --decode > {}".format(encoded_data, dest_path)], + ["bash", "-c", "echo {} | base64 --decode > {}".format(encodedStr, dest_path)], user='root') def wait_mysql_to_start(self, timeout=60): @@ -435,10 +437,10 @@ class ClickHouseCluster: try: conn = pymysql.connect(user='root', password='clickhouse', host='127.0.0.1', port=3308) conn.close() - print "Mysql Started" + print("Mysql Started") return except Exception as ex: - print "Can't connect to MySQL " + str(ex) + print("Can't connect to MySQL " + str(ex)) time.sleep(0.5) subprocess_call(['docker-compose', 'ps', '--services', '--all']) @@ -451,10 +453,10 @@ class ClickHouseCluster: conn_string = "host='localhost' user='postgres' password='mysecretpassword'" conn = psycopg2.connect(conn_string) conn.close() - print "Postgres Started" + print("Postgres Started") return except Exception as ex: - print "Can't connect to Postgres " + str(ex) + print("Can't connect to Postgres " + str(ex)) time.sleep(0.5) raise Exception("Cannot wait Postgres container") @@ -466,10 +468,10 @@ class ClickHouseCluster: for instance in ['zoo1', 'zoo2', 'zoo3']: conn = self.get_kazoo_client(instance) conn.get_children('/') - print "All instances of ZooKeeper started" + print("All instances of ZooKeeper started") return except Exception as ex: - print "Can't connect to ZooKeeper " + str(ex) + print("Can't connect to ZooKeeper " + str(ex)) time.sleep(0.5) raise Exception("Cannot wait ZooKeeper container") @@ -480,10 +482,10 @@ class ClickHouseCluster: while time.time() - start < timeout: try: hdfs_api.write_data("/somefilewithrandomname222", "1") - print "Connected to HDFS and SafeMode disabled! " + print("Connected to HDFS and SafeMode disabled! ") return except Exception as ex: - print "Can't connect to HDFS " + str(ex) + print("Can't connect to HDFS " + str(ex)) time.sleep(1) raise Exception("Can't wait HDFS to start") @@ -496,10 +498,10 @@ class ClickHouseCluster: while time.time() - start < timeout: try: connection.list_database_names() - print "Connected to Mongo dbs:", connection.list_database_names() + print("Connected to Mongo dbs:", connection.database_names()) return except Exception as ex: - print "Can't connect to Mongo " + str(ex) + print("Can't connect to Mongo " + str(ex)) time.sleep(1) def wait_minio_to_start(self, timeout=30, secure=False): @@ -519,12 +521,12 @@ class ClickHouseCluster: minio_client.make_bucket(self.minio_bucket) - print("S3 bucket '%s' created", self.minio_bucket) + print(("S3 bucket '%s' created", self.minio_bucket)) self.minio_client = minio_client return except Exception as ex: - print("Can't connect to Minio: %s", str(ex)) + print(("Can't connect to Minio: %s", str(ex))) time.sleep(1) raise Exception("Can't wait Minio to start") @@ -539,7 +541,7 @@ class ClickHouseCluster: print("Connected to SchemaRegistry") return except Exception as ex: - print("Can't connect to SchemaRegistry: %s", str(ex)) + print(("Can't connect to SchemaRegistry: %s", str(ex))) time.sleep(1) def wait_cassandra_to_start(self, timeout=30): @@ -555,7 +557,7 @@ class ClickHouseCluster: time.sleep(1) def start(self, destroy_dirs=True): - print "Cluster start called. is_up={}, destroy_dirs={}".format(self.is_up, destroy_dirs) + print("Cluster start called. is_up={}, destroy_dirs={}".format(self.is_up, destroy_dirs)) if self.is_up: return @@ -571,11 +573,11 @@ class ClickHouseCluster: try: if destroy_dirs and p.exists(self.instances_dir): - print("Removing instances dir %s", self.instances_dir) + print(("Removing instances dir %s", self.instances_dir)) shutil.rmtree(self.instances_dir) - for instance in self.instances.values(): - print('Setup directory for instance: {} destroy_dirs: {}'.format(instance.name, destroy_dirs)) + for instance in list(self.instances.values()): + print(('Setup directory for instance: {} destroy_dirs: {}'.format(instance.name, destroy_dirs))) instance.create_dir(destroy_dir=destroy_dirs) self.docker_client = docker.from_env(version=self.docker_api_version) @@ -676,12 +678,12 @@ class ClickHouseCluster: self.wait_cassandra_to_start() clickhouse_start_cmd = self.base_cmd + ['up', '-d', '--no-recreate'] - print("Trying to create ClickHouse instance by command %s", ' '.join(map(str, clickhouse_start_cmd))) + print(("Trying to create ClickHouse instance by command %s", ' '.join(map(str, clickhouse_start_cmd)))) subprocess_check_call(clickhouse_start_cmd) print("ClickHouse instance created") start_deadline = time.time() + 20.0 # seconds - for instance in self.instances.itervalues(): + for instance in self.instances.values(): instance.docker_client = self.docker_client instance.ip_address = self.get_instance_ip(instance.name) @@ -693,10 +695,10 @@ class ClickHouseCluster: self.is_up = True - except BaseException, e: - print "Failed to start cluster: " - print str(e) - print traceback.print_exc() + except BaseException as e: + print("Failed to start cluster: ") + print(str(e)) + print(traceback.print_exc()) raise def shutdown(self, kill=True): @@ -705,7 +707,7 @@ class ClickHouseCluster: try: subprocess.check_call(self.base_cmd + ['logs'], stdout=f) except Exception as e: - print "Unable to get logs from docker." + print("Unable to get logs from docker.") f.seek(0) for line in f: if SANITIZER_SIGN in line: @@ -716,18 +718,18 @@ class ClickHouseCluster: try: subprocess_check_call(self.base_cmd + ['kill']) except Exception as e: - print "Kill command failed durung shutdown. {}".format(repr(e)) + print("Kill command failed durung shutdown. {}".format(repr(e))) try: subprocess_check_call(self.base_cmd + ['down', '--volumes', '--remove-orphans']) except Exception as e: - print "Down + remove orphans failed durung shutdown. {}".format(repr(e)) + print("Down + remove orphans failed durung shutdown. {}".format(repr(e))) self.is_up = False self.docker_client = None - for instance in self.instances.values(): + for instance in list(self.instances.values()): instance.docker_client = None instance.ip_address = None instance.client = None @@ -769,7 +771,7 @@ class ClickHouseCluster: kazoo_callback(self.get_kazoo_client(zoo_instance_name)) return except KazooException as e: - print repr(e) + print(repr(e)) time.sleep(sleep_for) kazoo_callback(self.get_kazoo_client(zoo_instance_name)) @@ -922,7 +924,7 @@ class ClickHouseInstance: return result time.sleep(sleep_time) except Exception as ex: - print "Retry {} got exception {}".format(i + 1, ex) + print("Retry {} got exception {}".format(i + 1, ex)) time.sleep(sleep_time) if result is not None: @@ -954,28 +956,30 @@ class ClickHouseInstance: params["query"] = sql - auth = "" + auth = None if user and password: - auth = "{}:{}@".format(user, password) + auth = requests.auth.HTTPBasicAuth(user, password) elif user: - auth = "{}@".format(user) + auth = requests.auth.HTTPBasicAuth(user, '') + url = "http://" + self.ip_address + ":8123/?" + urllib.parse.urlencode(params) - url = "http://" + auth + self.ip_address + ":8123/?" + urllib.urlencode(params) - - open_result = urllib.urlopen(url, data) + if data: + r = requests.post(url, data, auth=auth) + else: + r = requests.get(url, auth=auth) def http_code_and_message(): - return str(open_result.getcode()) + " " + httplib.responses[ - open_result.getcode()] + ": " + open_result.read() + code = r.status_code + return str(code) + " " + http.client.responses[code] + ": " + r.text if expect_fail_and_get_error: - if open_result.getcode() == 200: - raise Exception("ClickHouse HTTP server is expected to fail, but succeeded: " + open_result.read()) + if r.ok: + raise Exception("ClickHouse HTTP server is expected to fail, but succeeded: " + r.text) return http_code_and_message() else: - if open_result.getcode() != 200: + if not r.ok: raise Exception("ClickHouse HTTP server returned " + http_code_and_message()) - return open_result.read() + return r.text # Connects to the instance via HTTP interface, sends a query and returns the answer def http_request(self, url, method='GET', params=None, data=None, headers=None): @@ -1161,9 +1165,9 @@ class ClickHouseInstance: def _create_odbc_config_file(self): with open(self.odbc_ini_path.split(':')[0], 'w') as f: - for driver_setup in self.odbc_drivers.values(): + for driver_setup in list(self.odbc_drivers.values()): f.write("[{}]\n".format(driver_setup["DSN"])) - for key, value in driver_setup.items(): + for key, value in list(driver_setup.items()): if key != "DSN": f.write(key + "=" + value + "\n") @@ -1183,16 +1187,16 @@ class ClickHouseInstance: instance_config_dir = p.abspath(p.join(self.path, 'configs')) os.makedirs(instance_config_dir) - print "Copy common default production configuration from {}".format(self.base_config_dir) + print("Copy common default production configuration from {}".format(self.base_config_dir)) shutil.copyfile(p.join(self.base_config_dir, 'config.xml'), p.join(instance_config_dir, 'config.xml')) shutil.copyfile(p.join(self.base_config_dir, 'users.xml'), p.join(instance_config_dir, 'users.xml')) - print "Create directory for configuration generated in this helper" + print("Create directory for configuration generated in this helper") # used by all utils with any config conf_d_dir = p.abspath(p.join(instance_config_dir, 'conf.d')) os.mkdir(conf_d_dir) - print "Create directory for common tests configuration" + print("Create directory for common tests configuration") # used by server with main config.xml self.config_d_dir = p.abspath(p.join(instance_config_dir, 'config.d')) os.mkdir(self.config_d_dir) @@ -1201,14 +1205,14 @@ class ClickHouseInstance: dictionaries_dir = p.abspath(p.join(instance_config_dir, 'dictionaries')) os.mkdir(dictionaries_dir) - print "Copy common configuration from helpers" + print("Copy common configuration from helpers") # The file is named with 0_ prefix to be processed before other configuration overloads. shutil.copy(p.join(HELPERS_DIR, '0_common_instance_config.xml'), self.config_d_dir) shutil.copy(p.join(HELPERS_DIR, '0_common_instance_users.xml'), users_d_dir) if len(self.custom_dictionaries_paths): shutil.copy(p.join(HELPERS_DIR, '0_common_enable_dictionaries.xml'), self.config_d_dir) - print "Generate and write macros file" + print("Generate and write macros file") macros = self.macros.copy() macros['instance'] = self.name with open(p.join(conf_d_dir, 'macros.xml'), 'w') as macros_config: @@ -1222,7 +1226,7 @@ class ClickHouseInstance: shutil.copytree(self.kerberos_secrets_dir, p.abspath(p.join(self.path, 'secrets'))) # Copy config.d configs - print "Copy custom test config files {} to {}".format(self.custom_main_config_paths, self.config_d_dir) + print("Copy custom test config files {} to {}".format(self.custom_main_config_paths, self.config_d_dir)) for path in self.custom_main_config_paths: shutil.copy(path, self.config_d_dir) @@ -1235,16 +1239,16 @@ class ClickHouseInstance: shutil.copy(path, dictionaries_dir) db_dir = p.abspath(p.join(self.path, 'database')) - print "Setup database dir {}".format(db_dir) + print("Setup database dir {}".format(db_dir)) if self.clickhouse_path_dir is not None: - print "Database files taken from {}".format(self.clickhouse_path_dir) + print("Database files taken from {}".format(self.clickhouse_path_dir)) shutil.copytree(self.clickhouse_path_dir, db_dir) - print "Database copied from {} to {}".format(self.clickhouse_path_dir, db_dir) + print("Database copied from {} to {}".format(self.clickhouse_path_dir, db_dir)) else: os.mkdir(db_dir) logs_dir = p.abspath(p.join(self.path, 'logs')) - print "Setup logs dir {}".format(logs_dir) + print("Setup logs dir {}".format(logs_dir)) os.mkdir(logs_dir) depends_on = [] @@ -1272,7 +1276,7 @@ class ClickHouseInstance: env_file = _create_env_file(os.path.dirname(self.docker_compose_path), self.env_variables) - print "Env {} stored in {}".format(self.env_variables, env_file) + print("Env {} stored in {}".format(self.env_variables, env_file)) odbc_ini_path = "" if self.odbc_ini_path: @@ -1284,7 +1288,7 @@ class ClickHouseInstance: if self.stay_alive: entrypoint_cmd = CLICKHOUSE_STAY_ALIVE_COMMAND - print "Entrypoint cmd: {}".format(entrypoint_cmd) + print("Entrypoint cmd: {}".format(entrypoint_cmd)) networks = app_net = ipv4_address = ipv6_address = net_aliases = net_alias1 = "" if self.ipv4_address is not None or self.ipv6_address is not None or self.hostname != self.name: diff --git a/tests/integration/helpers/external_sources.py b/tests/integration/helpers/external_sources.py index a52cf7a02d8..47de9dd0caf 100644 --- a/tests/integration/helpers/external_sources.py +++ b/tests/integration/helpers/external_sources.py @@ -176,7 +176,7 @@ class SourceMongo(ExternalSource): to_insert = [] for row in data: row_dict = {} - for cell_name, cell_value in row.data.items(): + for cell_name, cell_value in list(row.data.items()): row_dict[cell_name] = self.converters[cell_name](cell_value) to_insert.append(row_dict) @@ -387,7 +387,7 @@ class SourceHTTPBase(ExternalSource): self.node.exec_in_container([ "bash", "-c", - "python2 /http_server.py --data-path={tbl} --schema={schema} --host={host} --port={port} --cert-path=/fake_cert.pem".format( + "python3 /http_server.py --data-path={tbl} --schema={schema} --host={host} --port={port} --cert-path=/fake_cert.pem".format( tbl=path, schema=self._get_schema(), host=self.docker_hostname, port=self.http_port) ], detach=True) self.ordered_names = structure.get_ordered_names() @@ -573,12 +573,14 @@ class SourceAerospike(ExternalSource): def _flush_aerospike_db(self): keys = [] - def handle_record((key, metadata, record)): - print("Handle record {} {}".format(key, record)) + def handle_record(xxx_todo_changeme): + (key, metadata, record) = xxx_todo_changeme + print(("Handle record {} {}".format(key, record))) keys.append(key) - def print_record((key, metadata, record)): - print("Print record {} {}".format(key, record)) + def print_record(xxx_todo_changeme1): + (key, metadata, record) = xxx_todo_changeme1 + print(("Print record {} {}".format(key, record))) scan = self.client.scan(self.namespace, self.set) scan.foreach(handle_record) diff --git a/tests/integration/helpers/hdfs_api.py b/tests/integration/helpers/hdfs_api.py index 70111045ad2..d3afa9be837 100644 --- a/tests/integration/helpers/hdfs_api.py +++ b/tests/integration/helpers/hdfs_api.py @@ -1,5 +1,5 @@ # -*- coding: utf-8 -*- -import StringIO +import io import gzip import subprocess from tempfile import NamedTemporaryFile @@ -14,7 +14,7 @@ class HDFSApi(object): self.http_data_port = "50075" self.user = user - def read_data(self, path): + def read_data(self, path, universal_newlines=True): response = requests.get( "http://{host}:{port}/webhdfs/v1{path}?op=OPEN".format(host=self.host, port=self.http_proxy_port, path=path), allow_redirects=False) @@ -27,7 +27,10 @@ class HDFSApi(object): if response_data.status_code != 200: response_data.raise_for_status() - return response_data.content + if universal_newlines: + return response_data.text + else: + return response_data.content # Requests can't put file def _curl_to_put(self, filename, path, params): @@ -35,12 +38,14 @@ class HDFSApi(object): port=self.http_data_port, path=path, params=params) cmd = "curl -s -i -X PUT -T {fname} '{url}'".format(fname=filename, url=url) - output = subprocess.check_output(cmd, shell=True) + output = subprocess.check_output(cmd, shell=True, universal_newlines=True) return output def write_data(self, path, content): - named_file = NamedTemporaryFile() + named_file = NamedTemporaryFile(mode='wb+') fpath = named_file.name + if isinstance(content, str): + content = content.encode() named_file.write(content) named_file.flush() response = requests.put( @@ -58,10 +63,12 @@ class HDFSApi(object): raise Exception("Can't create file on hdfs:\n {}".format(output)) def write_gzip_data(self, path, content): - out = StringIO.StringIO() - with gzip.GzipFile(fileobj=out, mode="w") as f: + if isinstance(content, str): + content = content.encode() + out = io.BytesIO() + with gzip.GzipFile(fileobj=out, mode="wb") as f: f.write(content) self.write_data(path, out.getvalue()) def read_gzip_data(self, path): - return gzip.GzipFile(fileobj=StringIO.StringIO(self.read_data(path))).read() + return gzip.GzipFile(fileobj=io.BytesIO(self.read_data(path, universal_newlines=False))).read().decode() diff --git a/tests/integration/helpers/http_server.py b/tests/integration/helpers/http_server.py index 83e134606e3..e62096dd33f 100644 --- a/tests/integration/helpers/http_server.py +++ b/tests/integration/helpers/http_server.py @@ -3,7 +3,7 @@ import argparse import csv import socket import ssl -from BaseHTTPServer import BaseHTTPRequestHandler, HTTPServer +from http.server import BaseHTTPRequestHandler, HTTPServer # Decorator used to see if authentication works for external dictionary who use a HTTP source. @@ -29,7 +29,7 @@ def start_server(server_address, data_path, schema, cert_path, address_family): @check_auth def do_POST(self): ids = self.__read_and_decode_post_ids() - print "ids=", ids + print("ids=", ids) self.__send_headers() self.__send_data(ids) @@ -43,26 +43,26 @@ def start_server(server_address, data_path, schema, cert_path, address_family): reader = csv.reader(fl, delimiter='\t') for row in reader: if not only_ids or (row[0] in only_ids): - self.wfile.write('\t'.join(row) + '\n') + self.wfile.write(('\t'.join(row) + '\n').encode()) def __read_and_decode_post_ids(self): data = self.__read_and_decode_post_data() - return filter(None, data.split()) + return [_f for _f in data.split() if _f] def __read_and_decode_post_data(self): transfer_encoding = self.headers.get("Transfer-encoding") decoded = ""; if transfer_encoding == "chunked": while True: - s = self.rfile.readline() + s = self.rfile.readline().decode() chunk_length = int(s, 16) if not chunk_length: break - decoded += self.rfile.read(chunk_length) - self.rfile.readline() + decoded += self.rfile.read(chunk_length).decode() + self.rfile.readline().decode() else: content_length = int(self.headers.get("Content-Length", 0)) - decoded = self.rfile.read(content_length) + decoded = self.rfile.read(content_length).decode() return decoded if address_family == "ipv6": diff --git a/tests/integration/helpers/network.py b/tests/integration/helpers/network.py index f6505e81c91..a237f7d3cc7 100644 --- a/tests/integration/helpers/network.py +++ b/tests/integration/helpers/network.py @@ -183,7 +183,7 @@ class _NetworkManager: exit_code = self._docker_client.api.exec_inspect(handle)['ExitCode'] if exit_code != 0: - print output + print(output) raise subprocess.CalledProcessError(exit_code, cmd) return output diff --git a/tests/integration/helpers/test_tools.py b/tests/integration/helpers/test_tools.py index 9fbffe41819..75ae8f67f7a 100644 --- a/tests/integration/helpers/test_tools.py +++ b/tests/integration/helpers/test_tools.py @@ -1,14 +1,15 @@ import difflib import time +from io import IOBase class TSV: """Helper to get pretty diffs between expected and actual tab-separated value files""" def __init__(self, contents): - if isinstance(contents, file): + if isinstance(contents, IOBase): raw_lines = contents.readlines() - elif isinstance(contents, str) or isinstance(contents, unicode): + elif isinstance(contents, str) or isinstance(contents, str): raw_lines = contents.splitlines(True) elif isinstance(contents, list): raw_lines = ['\t'.join(map(str, l)) if isinstance(l, list) else str(l) for l in contents] @@ -29,7 +30,7 @@ class TSV: return self != TSV(other) return self.lines != other.lines - def diff(self, other, n1=None, n2=None): + def diff(self, other, n1='', n2=''): if not isinstance(other, TSV): return self.diff(TSV(other), n1=n1, n2=n2) return list(line.rstrip() for line in difflib.unified_diff(self.lines, other.lines, fromfile=n1, tofile=n2))[2:] @@ -45,14 +46,14 @@ class TSV: def assert_eq_with_retry(instance, query, expectation, retry_count=20, sleep_time=0.5, stdin=None, timeout=None, settings=None, user=None, ignore_error=False): expectation_tsv = TSV(expectation) - for i in xrange(retry_count): + for i in range(retry_count): try: if TSV(instance.query(query, user=user, stdin=stdin, timeout=timeout, settings=settings, ignore_error=ignore_error)) == expectation_tsv: break time.sleep(sleep_time) except Exception as ex: - print "assert_eq_with_retry retry {} exception {}".format(i + 1, ex) + print(("assert_eq_with_retry retry {} exception {}".format(i + 1, ex))) time.sleep(sleep_time) else: val = TSV(instance.query(query, user=user, stdin=stdin, timeout=timeout, settings=settings, @@ -66,13 +67,13 @@ def assert_logs_contain(instance, substring): raise AssertionError("'{}' not found in logs".format(substring)) def assert_logs_contain_with_retry(instance, substring, retry_count=20, sleep_time=0.5): - for i in xrange(retry_count): + for i in range(retry_count): try: if instance.contains_in_log(substring): break time.sleep(sleep_time) except Exception as ex: - print "contains_in_log_with_retry retry {} exception {}".format(i + 1, ex) + print("contains_in_log_with_retry retry {} exception {}".format(i + 1, ex)) time.sleep(sleep_time) else: raise AssertionError("'{}' not found in logs".format(substring)) diff --git a/tests/integration/helpers/uclient.py b/tests/integration/helpers/uclient.py index 098e17a38da..538722580af 100644 --- a/tests/integration/helpers/uclient.py +++ b/tests/integration/helpers/uclient.py @@ -6,7 +6,7 @@ CURDIR = os.path.dirname(os.path.realpath(__file__)) sys.path.insert(0, os.path.join(CURDIR)) -import uexpect +from . import uexpect prompt = ':\) ' end_of_block = r'.*\r\n.*\r\n' diff --git a/tests/integration/helpers/uexpect.py b/tests/integration/helpers/uexpect.py index 873d9a749e0..cd26e3ddbd3 100644 --- a/tests/integration/helpers/uexpect.py +++ b/tests/integration/helpers/uexpect.py @@ -15,7 +15,7 @@ import os import pty import re import time -from Queue import Queue, Empty +from queue import Queue, Empty from subprocess import Popen from threading import Thread, Event @@ -118,7 +118,7 @@ class IO(object): return self.write(data + eol) def write(self, data): - return os.write(self.master, data) + return os.write(self.master, data.encode()) def expect(self, pattern, timeout=None, escape=False): self.match = None @@ -201,7 +201,8 @@ def spawn(command): def reader(process, out, queue, kill_event): while True: try: - data = os.read(out, 65536) + # TODO: there are some issues with 1<<16 buffer size + data = os.read(out, 1<<17).decode(errors='replace') queue.put(data) except: if kill_event.is_set(): diff --git a/tests/integration/runner b/tests/integration/runner index 78d93af2929..dbcb6f21732 100755 --- a/tests/integration/runner +++ b/tests/integration/runner @@ -1,4 +1,4 @@ -#!/usr/bin/env python +#!/usr/bin/env python3 #-*- coding: utf-8 -*- import subprocess import os @@ -188,5 +188,5 @@ if __name__ == "__main__": command=args.command ) - print("Running pytest container as: '" + cmd + "'.") + print(("Running pytest container as: '" + cmd + "'.")) subprocess.check_call(cmd, shell=True) diff --git a/tests/integration/test_adaptive_granularity/test.py b/tests/integration/test_adaptive_granularity/test.py index ec3169bb995..12bfc22d7d9 100644 --- a/tests/integration/test_adaptive_granularity/test.py +++ b/tests/integration/test_adaptive_granularity/test.py @@ -371,7 +371,7 @@ def test_version_update_two_nodes(start_dynamic_cluster): node12.query("SYSTEM SYNC REPLICA table_with_default_granularity_new", timeout=120) break except Exception as ex: - print("Exception during replica sync", ex) + print(("Exception during replica sync", ex)) node11.query("SYSTEM RESTART REPLICA table_with_default_granularity_new") node12.query("SYSTEM RESTART REPLICA table_with_default_granularity_new") time.sleep(2 * i) @@ -386,7 +386,7 @@ def test_version_update_two_nodes(start_dynamic_cluster): node12.query("SYSTEM SYNC REPLICA table_with_default_granularity", timeout=120) break except Exception as ex: - print("Exception during replica sync", ex) + print(("Exception during replica sync", ex)) node11.query("SYSTEM RESTART REPLICA table_with_default_granularity") node12.query("SYSTEM RESTART REPLICA table_with_default_granularity") time.sleep(2 * i) diff --git a/tests/integration/test_cluster_copier/test.py b/tests/integration/test_cluster_copier/test.py index 88dac06f158..6a922dbfca7 100644 --- a/tests/integration/test_cluster_copier/test.py +++ b/tests/integration/test_cluster_copier/test.py @@ -50,8 +50,8 @@ def started_cluster(): } } - for cluster_name, shards in clusters_schema.iteritems(): - for shard_name, replicas in shards.iteritems(): + for cluster_name, shards in clusters_schema.items(): + for shard_name, replicas in shards.items(): for replica_name in replicas: name = "s{}_{}_{}".format(cluster_name, shard_name, replica_name) cluster.add_instance(name, @@ -235,16 +235,16 @@ def execute_task(task, cmd_options): task.start() zk = cluster.get_kazoo_client('zoo1') - print "Use ZooKeeper server: {}:{}".format(zk.hosts[0][0], zk.hosts[0][1]) + print("Use ZooKeeper server: {}:{}".format(zk.hosts[0][0], zk.hosts[0][1])) try: zk.delete("/clickhouse-copier", recursive=True) except kazoo.exceptions.NoNodeError: - print "No node /clickhouse-copier. It is Ok in first test." + print("No node /clickhouse-copier. It is Ok in first test.") zk_task_path = task.zk_task_path zk.ensure_path(zk_task_path) - zk.create(zk_task_path + "/description", task.copier_task_config) + zk.create(zk_task_path + "/description", task.copier_task_config.encode()) # Run cluster-copier processes on each node docker_api = docker.from_env().api @@ -256,19 +256,19 @@ def execute_task(task, cmd_options): '--base-dir', '/var/log/clickhouse-server/copier'] cmd += cmd_options - copiers = random.sample(cluster.instances.keys(), 3) + copiers = random.sample(list(cluster.instances.keys()), 3) for instance_name in copiers: instance = cluster.instances[instance_name] container = instance.get_docker_handle() instance.copy_file_to_container(os.path.join(CURRENT_TEST_DIR, "configs/config-copier.xml"), "/etc/clickhouse-server/config-copier.xml") - print "Copied copier config to {}".format(instance.name) + print("Copied copier config to {}".format(instance.name)) exec_id = docker_api.exec_create(container.id, cmd, stderr=True) output = docker_api.exec_start(exec_id).decode('utf8') print(output) copiers_exec_ids.append(exec_id) - print "Copier for {} ({}) has started".format(instance.name, instance.ip_address) + print("Copier for {} ({}) has started".format(instance.name, instance.ip_address)) # Wait for copiers stopping and check their return codes for exec_id, instance_name in zip(copiers_exec_ids, copiers): @@ -362,6 +362,6 @@ def test_no_arg(started_cluster): if __name__ == '__main__': with contextmanager(started_cluster)() as cluster: - for name, instance in cluster.instances.items(): - print name, instance.ip_address - raw_input("Cluster created, press any key to destroy...") + for name, instance in list(cluster.instances.items()): + print(name, instance.ip_address) + input("Cluster created, press any key to destroy...") diff --git a/tests/integration/test_cluster_copier/trivial_test.py b/tests/integration/test_cluster_copier/trivial_test.py index 035faf0bb9f..8a43440ac90 100644 --- a/tests/integration/test_cluster_copier/trivial_test.py +++ b/tests/integration/test_cluster_copier/trivial_test.py @@ -27,8 +27,8 @@ def started_cluster(): cluster = ClickHouseCluster(__file__) - for cluster_name, shards in clusters_schema.iteritems(): - for shard_name, replicas in shards.iteritems(): + for cluster_name, shards in clusters_schema.items(): + for shard_name, replicas in shards.items(): for replica_name in replicas: name = "s{}_{}_{}".format(cluster_name, shard_name, replica_name) cluster.add_instance(name, @@ -83,7 +83,7 @@ def execute_task(task, cmd_options): task.start() zk = cluster.get_kazoo_client('zoo1') - print "Use ZooKeeper server: {}:{}".format(zk.hosts[0][0], zk.hosts[0][1]) + print("Use ZooKeeper server: {}:{}".format(zk.hosts[0][0], zk.hosts[0][1])) zk_task_path = task.zk_task_path zk.ensure_path(zk_task_path) @@ -101,16 +101,16 @@ def execute_task(task, cmd_options): print(cmd) - for instance_name, instance in cluster.instances.iteritems(): + for instance_name, instance in cluster.instances.items(): container = instance.get_docker_handle() exec_id = docker_api.exec_create(container.id, cmd, stderr=True) docker_api.exec_start(exec_id, detach=True) copiers_exec_ids.append(exec_id) - print "Copier for {} ({}) has started".format(instance.name, instance.ip_address) + print("Copier for {} ({}) has started".format(instance.name, instance.ip_address)) # Wait for copiers stopping and check their return codes - for exec_id, instance in zip(copiers_exec_ids, cluster.instances.itervalues()): + for exec_id, instance in zip(copiers_exec_ids, iter(cluster.instances.values())): while True: res = docker_api.exec_inspect(exec_id) if not res['Running']: @@ -175,6 +175,6 @@ def test_trivial_copy_with_move_fault(started_cluster, use_sample_offset): if __name__ == '__main__': with contextmanager(started_cluster)() as cluster: - for name, instance in cluster.instances.items(): - print name, instance.ip_address - raw_input("Cluster created, press any key to destroy...") + for name, instance in list(cluster.instances.items()): + print(name, instance.ip_address) + input("Cluster created, press any key to destroy...") diff --git a/tests/integration/test_concurrent_queries_for_user_restriction/test.py b/tests/integration/test_concurrent_queries_for_user_restriction/test.py index e287eb763ce..279e0dfe439 100644 --- a/tests/integration/test_concurrent_queries_for_user_restriction/test.py +++ b/tests/integration/test_concurrent_queries_for_user_restriction/test.py @@ -26,14 +26,14 @@ def test_exception_message(started_cluster): assert node1.query("select number from nums order by number") == "0\n1\n" def node_busy(_): - for i in xrange(10): + for i in range(10): node1.query("select sleep(2)", user='default') busy_pool = Pool(3) - busy_pool.map_async(node_busy, xrange(3)) + busy_pool.map_async(node_busy, range(3)) time.sleep(1) # wait a little until polling starts try: assert node2.query("select number from remote('node1', 'default', 'nums')", user='good') == "0\n1\n" except Exception as ex: - print ex.message + print(ex.message) assert False, "Exception thrown while max_concurrent_queries_for_user is not exceeded" diff --git a/tests/integration/test_concurrent_ttl_merges/test.py b/tests/integration/test_concurrent_ttl_merges/test.py index f77ae5996d1..f067e65f58a 100644 --- a/tests/integration/test_concurrent_ttl_merges/test.py +++ b/tests/integration/test_concurrent_ttl_merges/test.py @@ -66,7 +66,7 @@ def test_no_ttl_merges_in_busy_pool(started_cluster): node1.query("ALTER TABLE test_ttl UPDATE data = data + 1 WHERE sleepEachRow(1) = 0") while count_running_mutations(node1, "test_ttl") < 6: - print "Mutations count", count_running_mutations(node1, "test_ttl") + print("Mutations count", count_running_mutations(node1, "test_ttl")) assert count_ttl_merges_in_background_pool(node1, "test_ttl") == 0 time.sleep(0.5) @@ -74,7 +74,7 @@ def test_no_ttl_merges_in_busy_pool(started_cluster): rows_count = [] while count_running_mutations(node1, "test_ttl") == 6: - print "Mutations count after start TTL", count_running_mutations(node1, "test_ttl") + print("Mutations count after start TTL", count_running_mutations(node1, "test_ttl")) rows_count.append(int(node1.query("SELECT count() FROM test_ttl").strip())) time.sleep(0.5) diff --git a/tests/integration/test_config_substitutions/test.py b/tests/integration/test_config_substitutions/test.py index 3a2d0d98281..565cd1c0e97 100644 --- a/tests/integration/test_config_substitutions/test.py +++ b/tests/integration/test_config_substitutions/test.py @@ -19,7 +19,7 @@ node6 = cluster.add_instance('node6', user_configs=['configs/config_include_from def start_cluster(): try: def create_zk_roots(zk): - zk.create(path="/setting/max_query_size", value="77777", makepath=True) + zk.create(path="/setting/max_query_size", value=b"77777", makepath=True) cluster.add_zookeeper_startup_command(create_zk_roots) diff --git a/tests/integration/test_consistant_parts_after_move_partition/test.py b/tests/integration/test_consistant_parts_after_move_partition/test.py index 05e721ee5ea..2070c8cb3f8 100644 --- a/tests/integration/test_consistant_parts_after_move_partition/test.py +++ b/tests/integration/test_consistant_parts_after_move_partition/test.py @@ -33,7 +33,7 @@ def start_cluster(): initialize_database([node1, node2], 1) yield cluster except Exception as ex: - print ex + print(ex) finally: cluster.shutdown() diff --git a/tests/integration/test_consistent_parts_after_clone_replica/test.py b/tests/integration/test_consistent_parts_after_clone_replica/test.py index 60b91bcb282..784f94397af 100644 --- a/tests/integration/test_consistent_parts_after_clone_replica/test.py +++ b/tests/integration/test_consistent_parts_after_clone_replica/test.py @@ -29,7 +29,7 @@ def start_cluster(): fill_nodes([node1, node2], 1) yield cluster except Exception as ex: - print ex + print(ex) finally: cluster.shutdown() diff --git a/tests/integration/test_cross_replication/test.py b/tests/integration/test_cross_replication/test.py index 9171fea5547..8a118934c93 100644 --- a/tests/integration/test_cross_replication/test.py +++ b/tests/integration/test_cross_replication/test.py @@ -83,11 +83,11 @@ def test(started_cluster): assert_eq_with_retry(node2, "SELECT * FROM distributed ORDER BY id", expected_from_distributed) with pytest.raises(Exception): - print node3.query_with_retry("SELECT * FROM distributed ORDER BY id", retry_count=5) + print(node3.query_with_retry("SELECT * FROM distributed ORDER BY id", retry_count=5)) if __name__ == '__main__': with contextmanager(started_cluster)() as cluster: - for name, instance in cluster.instances.items(): - print name, instance.ip_address - raw_input("Cluster created, press any key to destroy...") + for name, instance in list(cluster.instances.items()): + print(name, instance.ip_address) + input("Cluster created, press any key to destroy...") diff --git a/tests/integration/test_delayed_replica_failover/test.py b/tests/integration/test_delayed_replica_failover/test.py index f657edae6fb..18184c3304d 100644 --- a/tests/integration/test_delayed_replica_failover/test.py +++ b/tests/integration/test_delayed_replica_failover/test.py @@ -98,12 +98,12 @@ SELECT sum(x) FROM distributed SETTINGS # If we forbid stale replicas, the query must fail. with pytest.raises(Exception): - print instance_with_dist_table.query(''' + print(instance_with_dist_table.query(''' SELECT count() FROM distributed SETTINGS load_balancing='in_order', max_replica_delay_for_distributed_queries=1, fallback_to_stale_replicas_for_distributed_queries=0 -''') +''')) # Now partition off the remote replica of the local shard and test that failover still works. pm.partition_instances(node_1_1, node_1_2, port=9000) diff --git a/tests/integration/test_dictionaries_all_layouts_separate_sources/common.py b/tests/integration/test_dictionaries_all_layouts_separate_sources/common.py index 0411b5d9475..ef6d133893a 100644 --- a/tests/integration/test_dictionaries_all_layouts_separate_sources/common.py +++ b/tests/integration/test_dictionaries_all_layouts_separate_sources/common.py @@ -113,12 +113,12 @@ class SimpleLayoutTester: self.layout_to_dictionary[layout] = get_dict(source_, Layout(layout), self.fields) def prepare(self, cluster_): - for _, dictionary in self.layout_to_dictionary.items(): + for _, dictionary in list(self.layout_to_dictionary.items()): dictionary.prepare_source(cluster_) dictionary.load_data(self.data) def execute(self, layout_name, node): - if not self.layout_to_dictionary.has_key(layout_name): + if layout_name not in self.layout_to_dictionary: raise RuntimeError("Source doesn't support layout: {}".format(layout_name)) dct = self.layout_to_dictionary[layout_name] @@ -170,12 +170,12 @@ class ComplexLayoutTester: self.layout_to_dictionary[layout] = get_dict(source_, Layout(layout), self.fields) def prepare(self, cluster_): - for _, dictionary in self.layout_to_dictionary.items(): + for _, dictionary in list(self.layout_to_dictionary.items()): dictionary.prepare_source(cluster_) dictionary.load_data(self.data) def execute(self, layout_name, node): - if not self.layout_to_dictionary.has_key(layout_name): + if layout_name not in self.layout_to_dictionary: raise RuntimeError("Source doesn't support layout: {}".format(layout_name)) dct = self.layout_to_dictionary[layout_name] @@ -213,13 +213,13 @@ class RangedLayoutTester: self.layout_to_dictionary[layout] = get_dict(source_, Layout(layout), self.fields) def prepare(self, cluster_): - for _, dictionary in self.layout_to_dictionary.items(): + for _, dictionary in list(self.layout_to_dictionary.items()): dictionary.prepare_source(cluster_) dictionary.load_data(self.data) def execute(self, layout_name, node): - if not self.layout_to_dictionary.has_key(layout_name): + if layout_name not in self.layout_to_dictionary: raise RuntimeError("Source doesn't support layout: {}".format(layout_name)) dct = self.layout_to_dictionary[layout_name] diff --git a/tests/integration/test_dictionaries_complex_key_cache_string/test.py b/tests/integration/test_dictionaries_complex_key_cache_string/test.py index c8969aee63e..a01e60af47d 100644 --- a/tests/integration/test_dictionaries_complex_key_cache_string/test.py +++ b/tests/integration/test_dictionaries_complex_key_cache_string/test.py @@ -42,7 +42,7 @@ def test_memory_consumption(cluster): allocated_first = int(node.query("select bytes_allocated from system.dictionaries where name = 'radars'").strip()) alloc_array = [] - for i in xrange(5): + for i in range(5): node.query("select dictGetString('radars', 'client_id', tuple(toString(number))) from numbers(0, 5000)") allocated = int(node.query("select bytes_allocated from system.dictionaries where name = 'radars'").strip()) @@ -51,7 +51,7 @@ def test_memory_consumption(cluster): # size doesn't grow assert all(allocated_first >= a for a in alloc_array) - for i in xrange(5): + for i in range(5): node.query("select dictGetString('radars', 'client_id', tuple(toString(number))) from numbers(0, 5000)") allocated = int(node.query("select bytes_allocated from system.dictionaries where name = 'radars'").strip()) diff --git a/tests/integration/test_dictionaries_redis/test.py b/tests/integration/test_dictionaries_redis/test.py index 385580816e0..d08734af547 100644 --- a/tests/integration/test_dictionaries_redis/test.py +++ b/tests/integration/test_dictionaries_redis/test.py @@ -106,7 +106,7 @@ def setup_module(module): for source in sources: for layout in LAYOUTS: if not source.compatible_with_layout(layout): - print "Source", source.name, "incompatible with layout", layout.name + print("Source", source.name, "incompatible with layout", layout.name) continue fields = KEY_FIELDS[layout.layout_type] + [field] @@ -128,9 +128,9 @@ def started_cluster(): assert len(FIELDS) == len(VALUES) for dicts in DICTIONARIES: for dictionary in dicts: - print "Preparing", dictionary.name + print("Preparing", dictionary.name) dictionary.prepare_source(cluster) - print "Prepared" + print("Prepared") yield cluster @@ -138,9 +138,9 @@ def started_cluster(): cluster.shutdown() -@pytest.mark.parametrize("id", range(len(FIELDS))) +@pytest.mark.parametrize("id", list(range(len(FIELDS)))) def test_redis_dictionaries(started_cluster, id): - print 'id:', id + print('id:', id) dicts = DICTIONARIES[id] values = VALUES[id] @@ -173,7 +173,7 @@ def test_redis_dictionaries(started_cluster, id): node.query("system reload dictionary {}".format(dct.name)) for query, answer in queries_with_answers: - print query + print(query) assert node.query(query) == str(answer) + '\n' # Checks, that dictionaries can be reloaded. diff --git a/tests/integration/test_dictionaries_select_all/generate_dictionaries.py b/tests/integration/test_dictionaries_select_all/generate_dictionaries.py index 5c92d0d67e8..109ecea438e 100644 --- a/tests/integration/test_dictionaries_select_all/generate_dictionaries.py +++ b/tests/integration/test_dictionaries_select_all/generate_dictionaries.py @@ -1,5 +1,6 @@ import difflib import os +from functools import reduce files = ['key_simple.tsv', 'key_complex_integers.tsv', 'key_complex_mixed.tsv'] @@ -78,8 +79,9 @@ def generate_dictionaries(path, structure): ''' dictionary_skeleton = \ - dictionary_skeleton % reduce(lambda xml, (type, default): xml + attribute_skeleton % (type, type, default), - zip(types, implicit_defaults), '') + dictionary_skeleton % reduce( + lambda xml, type_default: xml + attribute_skeleton % (type_default[0], type_default[0], type_default[1]), + list(zip(types, implicit_defaults)), '') source_clickhouse = ''' @@ -195,7 +197,7 @@ class DictionaryTestTable: String_ String, Date_ Date, DateTime_ DateTime, Parent UInt64''' - self.names_and_types = map(str.split, self.structure.split(',')) + self.names_and_types = list(map(str.split, self.structure.split(','))) self.keys_names_and_types = self.names_and_types[:6] self.values_names_and_types = self.names_and_types[6:] self.source_file_name = source_file_name @@ -223,10 +225,10 @@ class DictionaryTestTable: def make_tuple(line): row = tuple(line.split('\t')) self.rows.append(row) - return '(' + ','.join(map(wrap_value, zip(row, types))) + ')' + return '(' + ','.join(map(wrap_value, list(zip(row, types)))) + ')' values = ','.join(map(make_tuple, lines)) - print query % (self.structure, values) + print(query % (self.structure, values)) instance.query(query % (self.structure, values)) def get_structure_for_keys(self, keys, enable_parent=True): @@ -245,7 +247,7 @@ class DictionaryTestTable: for row in rows: key = '\t'.join(row[:len(keys)]) value = '\t'.join(row[len(keys):]) - if key in lines_map.keys(): + if key in list(lines_map.keys()): pattern_value = lines_map[key] del lines_map[key] if not value == pattern_value: @@ -256,7 +258,7 @@ class DictionaryTestTable: diff.append((key + '\t' + value, '')) if add_not_found_rows: - for key, value in lines_map.items(): + for key, value in list(lines_map.items()): diff.append(('', key + '\t' + value)) if not diff: diff --git a/tests/integration/test_dictionaries_select_all/test.py b/tests/integration/test_dictionaries_select_all/test.py index 5b8d39a7a63..5331f51f4c7 100644 --- a/tests/integration/test_dictionaries_select_all/test.py +++ b/tests/integration/test_dictionaries_select_all/test.py @@ -4,7 +4,7 @@ import pytest from helpers.cluster import ClickHouseCluster from helpers.test_tools import TSV -from generate_dictionaries import generate_structure, generate_dictionaries, DictionaryTestTable +from .generate_dictionaries import generate_structure, generate_dictionaries, DictionaryTestTable SCRIPT_DIR = os.path.dirname(os.path.realpath(__file__)) @@ -32,7 +32,7 @@ def started_cluster(): cluster.start() test_table.create_clickhouse_source(instance) for line in TSV(instance.query('select name from system.dictionaries')).lines: - print line, + print(line, end=' ') yield cluster @@ -72,7 +72,7 @@ def test_select_all(dictionary_structure): result = TSV(query('select * from test.{0}'.format(name))) diff = test_table.compare_by_keys(keys, result.lines, use_parent, add_not_found_rows=True) - print test_table.process_diff(diff) + print(test_table.process_diff(diff)) assert not diff @@ -103,7 +103,7 @@ def test_select_all_from_cached(cached_dictionary_structure): for i in range(4): result = TSV(query('select * from test.{0}'.format(name))) diff = test_table.compare_by_keys(keys, result.lines, use_parent, add_not_found_rows=False) - print test_table.process_diff(diff) + print(test_table.process_diff(diff)) assert not diff key = [] @@ -120,5 +120,5 @@ def test_select_all_from_cached(cached_dictionary_structure): result = TSV(query('select * from test.{0}'.format(name))) diff = test_table.compare_by_keys(keys, result.lines, use_parent, add_not_found_rows=True) - print test_table.process_diff(diff) + print(test_table.process_diff(diff)) assert not diff diff --git a/tests/integration/test_dictionary_allow_read_expired_keys/test_default_reading.py b/tests/integration/test_dictionary_allow_read_expired_keys/test_default_reading.py index 1266c37dcd3..416bbe089aa 100644 --- a/tests/integration/test_dictionary_allow_read_expired_keys/test_default_reading.py +++ b/tests/integration/test_dictionary_allow_read_expired_keys/test_default_reading.py @@ -1,4 +1,4 @@ -from __future__ import print_function + import time diff --git a/tests/integration/test_dictionary_allow_read_expired_keys/test_default_string.py b/tests/integration/test_dictionary_allow_read_expired_keys/test_default_string.py index 2aa6fb448ca..caabdf12c66 100644 --- a/tests/integration/test_dictionary_allow_read_expired_keys/test_default_string.py +++ b/tests/integration/test_dictionary_allow_read_expired_keys/test_default_string.py @@ -1,4 +1,4 @@ -from __future__ import print_function + import os import random diff --git a/tests/integration/test_dictionary_allow_read_expired_keys/test_dict_get.py b/tests/integration/test_dictionary_allow_read_expired_keys/test_dict_get.py index 9de0b3be4eb..7097bd15bb7 100644 --- a/tests/integration/test_dictionary_allow_read_expired_keys/test_dict_get.py +++ b/tests/integration/test_dictionary_allow_read_expired_keys/test_dict_get.py @@ -1,4 +1,4 @@ -from __future__ import print_function + import time diff --git a/tests/integration/test_dictionary_allow_read_expired_keys/test_dict_get_or_default.py b/tests/integration/test_dictionary_allow_read_expired_keys/test_dict_get_or_default.py index 31f0e469555..2aecb8691fb 100644 --- a/tests/integration/test_dictionary_allow_read_expired_keys/test_dict_get_or_default.py +++ b/tests/integration/test_dictionary_allow_read_expired_keys/test_dict_get_or_default.py @@ -1,4 +1,4 @@ -from __future__ import print_function + import time diff --git a/tests/integration/test_dictionary_custom_settings/http_server.py b/tests/integration/test_dictionary_custom_settings/http_server.py index 20487ccf447..bd5ce22dbac 100644 --- a/tests/integration/test_dictionary_custom_settings/http_server.py +++ b/tests/integration/test_dictionary_custom_settings/http_server.py @@ -3,7 +3,7 @@ import argparse import csv import socket import ssl -from BaseHTTPServer import BaseHTTPRequestHandler, HTTPServer +from http.server import BaseHTTPRequestHandler, HTTPServer # Decorator used to see if authentication works for external dictionary who use a HTTP source. @@ -29,7 +29,7 @@ def start_server(server_address, data_path, schema, cert_path, address_family): @check_auth def do_POST(self): ids = self.__read_and_decode_post_ids() - print "ids=", ids + print("ids=", ids) self.__send_headers() self.__send_data(ids) @@ -43,11 +43,11 @@ def start_server(server_address, data_path, schema, cert_path, address_family): reader = csv.reader(fl, delimiter='\t') for row in reader: if not only_ids or (row[0] in only_ids): - self.wfile.write('\t'.join(row) + '\n') + self.wfile.write(('\t'.join(row) + '\n').encode()) def __read_and_decode_post_ids(self): data = self.__read_and_decode_post_data() - return filter(None, data.split()) + return [_f for _f in data.split() if _f] def __read_and_decode_post_data(self): transfer_encoding = self.headers.get("Transfer-encoding") @@ -58,11 +58,11 @@ def start_server(server_address, data_path, schema, cert_path, address_family): chunk_length = int(s, 16) if not chunk_length: break - decoded += self.rfile.read(chunk_length) + decoded += self.rfile.read(chunk_length).decode() self.rfile.readline() else: content_length = int(self.headers.get("Content-Length", 0)) - decoded = self.rfile.read(content_length) + decoded = self.rfile.read(content_length).decode() return decoded if address_family == "ipv6": diff --git a/tests/integration/test_dictionary_custom_settings/test.py b/tests/integration/test_dictionary_custom_settings/test.py index 022822c8a80..aa6a16afb51 100644 --- a/tests/integration/test_dictionary_custom_settings/test.py +++ b/tests/integration/test_dictionary_custom_settings/test.py @@ -26,7 +26,7 @@ def prepare(): node.exec_in_container([ "bash", "-c", - "python2 /http_server.py --data-path={tbl} --schema=http --host=localhost --port=5555".format( + "python3 /http_server.py --data-path={tbl} --schema=http --host=localhost --port=5555".format( tbl=path) ], detach=True) diff --git a/tests/integration/test_disk_types/test.py b/tests/integration/test_disk_types/test.py index a97b90af27d..c748653bc82 100644 --- a/tests/integration/test_disk_types/test.py +++ b/tests/integration/test_disk_types/test.py @@ -33,5 +33,5 @@ def test_different_types(cluster): def test_select_by_type(cluster): node = cluster.instances["node"] - for name, disk_type in disk_types.items(): + for name, disk_type in list(disk_types.items()): assert node.query("SELECT name FROM system.disks WHERE type='" + disk_type + "'") == name + "\n" diff --git a/tests/integration/test_distributed_ddl/cluster.py b/tests/integration/test_distributed_ddl/cluster.py index efd6ce7e65c..811eb94bad4 100644 --- a/tests/integration/test_distributed_ddl/cluster.py +++ b/tests/integration/test_distributed_ddl/cluster.py @@ -26,12 +26,12 @@ class ClickHouseClusterWithDDLHelpers(ClickHouseCluster): main_configs += [os.path.join(self.test_config_dir, f) for f in ["server.crt", "server.key", "dhparam.pem", "config.d/ssl_conf.xml"]] - for i in xrange(4): + for i in range(4): self.add_instance( 'ch{}'.format(i + 1), main_configs=main_configs, user_configs=user_configs, - macros={"layer": 0, "shard": i / 2 + 1, "replica": i % 2 + 1}, + macros={"layer": 0, "shard": i // 2 + 1, "replica": i % 2 + 1}, with_zookeeper=True) self.start() @@ -62,11 +62,11 @@ class ClickHouseClusterWithDDLHelpers(ClickHouseCluster): self.ddl_check_query(instance, "CREATE DATABASE IF NOT EXISTS test ON CLUSTER 'cluster'") except Exception as e: - print e + print(e) raise def sync_replicas(self, table, timeout=5): - for instance in self.instances.values(): + for instance in list(self.instances.values()): instance.query("SYSTEM SYNC REPLICA {}".format(table), timeout=timeout) def check_all_hosts_successfully_executed(self, tsv_content, num_hosts=None): @@ -90,7 +90,7 @@ class ClickHouseClusterWithDDLHelpers(ClickHouseCluster): def replace_domains_to_ip_addresses_in_cluster_config(self, instances_to_replace): clusters_config = open(p.join(self.base_dir, '{}/config.d/clusters.xml'.format(self.test_config_dir))).read() - for inst_name, inst in self.instances.items(): + for inst_name, inst in list(self.instances.items()): clusters_config = clusters_config.replace(inst_name, str(inst.ip_address)) for inst_name in instances_to_replace: @@ -113,7 +113,7 @@ class ClickHouseClusterWithDDLHelpers(ClickHouseCluster): Make retries in case of UNKNOWN_STATUS_OF_INSERT or zkutil::KeeperException errors """ - for i in xrange(100): + for i in range(100): try: instance.query(query_insert) return diff --git a/tests/integration/test_distributed_ddl/test.py b/tests/integration/test_distributed_ddl/test.py index 9f01fa7ed5b..f0e78dfec41 100755 --- a/tests/integration/test_distributed_ddl/test.py +++ b/tests/integration/test_distributed_ddl/test.py @@ -27,7 +27,7 @@ def test_cluster(request): # Check query log to ensure that DDL queries are not executed twice time.sleep(1.5) - for instance in cluster.instances.values(): + for instance in list(cluster.instances.values()): cluster.ddl_check_there_are_no_dublicates(instance) cluster.pm_random_drops.heal_all() @@ -133,12 +133,12 @@ CREATE TABLE IF NOT EXISTS all_merge_64 ON CLUSTER '{cluster}' (p Date, i Int64, ENGINE = Distributed('{cluster}', default, merge, i) """) - for i in xrange(0, 4, 2): + for i in range(0, 4, 2): k = (i / 2) * 2 test_cluster.instances['ch{}'.format(i + 1)].query("INSERT INTO merge (i) VALUES ({})({})".format(k, k + 1)) assert TSV(instance.query("SELECT i FROM all_merge_32 ORDER BY i")) == TSV( - ''.join(['{}\n'.format(x) for x in xrange(4)])) + ''.join(['{}\n'.format(x) for x in range(4)])) time.sleep(5) test_cluster.ddl_check_query(instance, "ALTER TABLE merge ON CLUSTER '{cluster}' MODIFY COLUMN i Int64") @@ -147,19 +147,19 @@ ENGINE = Distributed('{cluster}', default, merge, i) "ALTER TABLE merge ON CLUSTER '{cluster}' ADD COLUMN s String DEFAULT toString(i) FORMAT TSV") assert TSV(instance.query("SELECT i, s FROM all_merge_64 ORDER BY i")) == TSV( - ''.join(['{}\t{}\n'.format(x, x) for x in xrange(4)])) + ''.join(['{}\t{}\n'.format(x, x) for x in range(4)])) - for i in xrange(0, 4, 2): + for i in range(0, 4, 2): k = (i / 2) * 2 + 4 test_cluster.instances['ch{}'.format(i + 1)].query( "INSERT INTO merge (p, i) VALUES (31, {})(31, {})".format(k, k + 1)) assert TSV(instance.query("SELECT i, s FROM all_merge_64 ORDER BY i")) == TSV( - ''.join(['{}\t{}\n'.format(x, x) for x in xrange(8)])) + ''.join(['{}\t{}\n'.format(x, x) for x in range(8)])) test_cluster.ddl_check_query(instance, "ALTER TABLE merge ON CLUSTER '{cluster}' DETACH PARTITION 197002") assert TSV(instance.query("SELECT i, s FROM all_merge_64 ORDER BY i")) == TSV( - ''.join(['{}\t{}\n'.format(x, x) for x in xrange(4)])) + ''.join(['{}\t{}\n'.format(x, x) for x in range(4)])) test_cluster.ddl_check_query(instance, "DROP TABLE merge ON CLUSTER '{cluster}'") test_cluster.ddl_check_query(instance, "DROP TABLE all_merge_32 ON CLUSTER '{cluster}'") @@ -170,7 +170,7 @@ def test_macro(test_cluster): instance = test_cluster.instances['ch2'] test_cluster.ddl_check_query(instance, "CREATE TABLE tab ON CLUSTER '{cluster}' (value UInt8) ENGINE = Memory") - for i in xrange(4): + for i in range(4): test_cluster.insert_reliable(test_cluster.instances['ch{}'.format(i + 1)], "INSERT INTO tab VALUES ({})".format(i)) @@ -359,6 +359,6 @@ def test_replicated_without_arguments(test_cluster): if __name__ == '__main__': with contextmanager(test_cluster)() as ctx_cluster: - for name, instance in ctx_cluster.instances.items(): - print name, instance.ip_address - raw_input("Cluster created, press any key to destroy...") + for name, instance in list(ctx_cluster.instances.items()): + print(name, instance.ip_address) + input("Cluster created, press any key to destroy...") diff --git a/tests/integration/test_distributed_ddl/test_replicated_alter.py b/tests/integration/test_distributed_ddl/test_replicated_alter.py index 840803f61ef..bd95f5660b7 100644 --- a/tests/integration/test_distributed_ddl/test_replicated_alter.py +++ b/tests/integration/test_distributed_ddl/test_replicated_alter.py @@ -26,7 +26,7 @@ def test_cluster(request): # Check query log to ensure that DDL queries are not executed twice time.sleep(1.5) - for instance in cluster.instances.values(): + for instance in list(cluster.instances.values()): cluster.ddl_check_there_are_no_dublicates(instance) cluster.pm_random_drops.heal_all() @@ -59,36 +59,36 @@ CREATE TABLE IF NOT EXISTS all_merge_64 ON CLUSTER cluster (p Date, i Int64, s S ENGINE = Distributed(cluster, default, merge_for_alter, i) """) - for i in xrange(4): - k = (i / 2) * 2 + for i in range(4): + k = (i // 2) * 2 test_cluster.insert_reliable(test_cluster.instances['ch{}'.format(i + 1)], "INSERT INTO merge_for_alter (i) VALUES ({})({})".format(k, k + 1)) test_cluster.sync_replicas("merge_for_alter") assert TSV(instance.query("SELECT i FROM all_merge_32 ORDER BY i")) == TSV( - ''.join(['{}\n'.format(x) for x in xrange(4)])) + ''.join(['{}\n'.format(x) for x in range(4)])) test_cluster.ddl_check_query(instance, "ALTER TABLE merge_for_alter ON CLUSTER cluster MODIFY COLUMN i Int64") test_cluster.ddl_check_query(instance, "ALTER TABLE merge_for_alter ON CLUSTER cluster ADD COLUMN s String DEFAULT toString(i)") assert TSV(instance.query("SELECT i, s FROM all_merge_64 ORDER BY i")) == TSV( - ''.join(['{}\t{}\n'.format(x, x) for x in xrange(4)])) + ''.join(['{}\t{}\n'.format(x, x) for x in range(4)])) - for i in xrange(4): - k = (i / 2) * 2 + 4 + for i in range(4): + k = (i // 2) * 2 + 4 test_cluster.insert_reliable(test_cluster.instances['ch{}'.format(i + 1)], "INSERT INTO merge_for_alter (p, i) VALUES (31, {})(31, {})".format(k, k + 1)) test_cluster.sync_replicas("merge_for_alter") assert TSV(instance.query("SELECT i, s FROM all_merge_64 ORDER BY i")) == TSV( - ''.join(['{}\t{}\n'.format(x, x) for x in xrange(8)])) + ''.join(['{}\t{}\n'.format(x, x) for x in range(8)])) test_cluster.ddl_check_query(instance, "ALTER TABLE merge_for_alter ON CLUSTER cluster DETACH PARTITION 197002") assert TSV(instance.query("SELECT i, s FROM all_merge_64 ORDER BY i")) == TSV( - ''.join(['{}\t{}\n'.format(x, x) for x in xrange(4)])) + ''.join(['{}\t{}\n'.format(x, x) for x in range(4)])) test_cluster.ddl_check_query(instance, "DROP TABLE merge_for_alter ON CLUSTER cluster") diff --git a/tests/integration/test_distributed_inter_server_secret/test.py b/tests/integration/test_distributed_inter_server_secret/test.py index b39f9dec861..bd9e6d111ca 100644 --- a/tests/integration/test_distributed_inter_server_secret/test.py +++ b/tests/integration/test_distributed_inter_server_secret/test.py @@ -25,7 +25,7 @@ users = pytest.mark.parametrize('user,password', [ ]) def bootstrap(): - for n in cluster.instances.values(): + for n in list(cluster.instances.values()): n.query('DROP TABLE IF EXISTS data') n.query('DROP TABLE IF EXISTS dist') n.query('CREATE TABLE data (key Int) Engine=Memory()') diff --git a/tests/integration/test_distributed_load_balancing/test.py b/tests/integration/test_distributed_load_balancing/test.py index b227c57fb04..e7b86a210bd 100644 --- a/tests/integration/test_distributed_load_balancing/test.py +++ b/tests/integration/test_distributed_load_balancing/test.py @@ -18,7 +18,7 @@ queries = nodes * 5 def bootstrap(): - for n in cluster.instances.values(): + for n in list(cluster.instances.values()): # At startup, server loads configuration files. # # However ConfigReloader does not know about already loaded files @@ -90,7 +90,7 @@ def get_node(query_node, table='dist', *args, **kwargs): query_node.query('SELECT * FROM ' + table, *args, **kwargs) - for n in cluster.instances.values(): + for n in list(cluster.instances.values()): n.query('SYSTEM FLUSH LOGS') rows = query_node.query(""" diff --git a/tests/integration/test_distributed_over_distributed/test.py b/tests/integration/test_distributed_over_distributed/test.py index 716bc66d629..410a03a6af1 100644 --- a/tests/integration/test_distributed_over_distributed/test.py +++ b/tests/integration/test_distributed_over_distributed/test.py @@ -1,7 +1,7 @@ # This test is a subset of the 01223_dist_on_dist. # (just in case, with real separate instances). -from __future__ import print_function + import pytest from helpers.cluster import ClickHouseCluster @@ -51,7 +51,7 @@ def started_cluster(): cluster.shutdown() -@pytest.mark.parametrize("node", NODES.values()) +@pytest.mark.parametrize("node", list(NODES.values())) @pytest.mark.parametrize("source", ["distributed_over_distributed_table", "cluster('test_cluster', default, distributed_table)"]) class TestDistributedOverDistributedSuite: diff --git a/tests/integration/test_distributed_over_live_view/test.py b/tests/integration/test_distributed_over_live_view/test.py index 9e62aaad982..78b90024ebf 100644 --- a/tests/integration/test_distributed_over_live_view/test.py +++ b/tests/integration/test_distributed_over_live_view/test.py @@ -1,4 +1,4 @@ -from __future__ import print_function + import sys import time @@ -9,6 +9,9 @@ from helpers.uclient import client, prompt, end_of_block cluster = ClickHouseCluster(__file__) +# log = sys.stdout +log = None + NODES = {'node' + str(i): cluster.add_instance( 'node' + str(i), main_configs=['configs/remote_servers.xml'], @@ -63,12 +66,11 @@ def poll_query(node, query, expected, timeout): pass assert node.query(query) == expected -@pytest.mark.parametrize("node", NODES.values()[:1]) +@pytest.mark.parametrize("node", list(NODES.values())[:1]) @pytest.mark.parametrize("source", ["lv_over_distributed_table"]) class TestLiveViewOverDistributedSuite: def test_distributed_over_live_view_order_by_node(self, started_cluster, node, source): - log = sys.stdout - node0, node1 = NODES.values() + node0, node1 = list(NODES.values()) select_query = "SELECT * FROM distributed_over_lv ORDER BY node, key FORMAT CSV" select_query_dist_table = "SELECT * FROM distributed_table ORDER BY node, key FORMAT CSV" @@ -118,8 +120,7 @@ class TestLiveViewOverDistributedSuite: client1.expect(prompt) def test_distributed_over_live_view_order_by_key(self, started_cluster, node, source): - log = sys.stdout - node0, node1 = NODES.values() + node0, node1 = list(NODES.values()) select_query = "SELECT * FROM distributed_over_lv ORDER BY key, node FORMAT CSV" select_count_query = "SELECT count() FROM distributed_over_lv" @@ -160,8 +161,7 @@ class TestLiveViewOverDistributedSuite: client1.expect(prompt) def test_distributed_over_live_view_group_by_node(self, started_cluster, node, source): - log = sys.stdout - node0, node1 = NODES.values() + node0, node1 = list(NODES.values()) select_query = "SELECT node, SUM(value) FROM distributed_over_lv GROUP BY node ORDER BY node FORMAT CSV" @@ -204,8 +204,7 @@ class TestLiveViewOverDistributedSuite: client1.expect(prompt) def test_distributed_over_live_view_group_by_key(self, started_cluster, node, source): - log = sys.stdout - node0, node1 = NODES.values() + node0, node1 = list(NODES.values()) select_query = "SELECT key, SUM(value) FROM distributed_over_lv GROUP BY key ORDER BY key FORMAT CSV" @@ -249,8 +248,7 @@ class TestLiveViewOverDistributedSuite: client1.expect(prompt) def test_distributed_over_live_view_sum(self, started_cluster, node, source): - log = sys.stdout - node0, node1 = NODES.values() + node0, node1 = list(NODES.values()) with client(name="client1> ", log=log, command=" ".join(node0.client.command)) as client1, \ client(name="client2> ", log=log, command=" ".join(node1.client.command)) as client2: diff --git a/tests/integration/test_distributed_respect_user_timeouts/test.py b/tests/integration/test_distributed_respect_user_timeouts/test.py index e5d9d0c1857..c19323b2049 100644 --- a/tests/integration/test_distributed_respect_user_timeouts/test.py +++ b/tests/integration/test_distributed_respect_user_timeouts/test.py @@ -103,7 +103,7 @@ def started_cluster(request): try: cluster.start() - for node_id, node in NODES.items(): + for node_id, node in list(NODES.items()): node.query(CREATE_TABLES_SQL) node.query(INSERT_SQL_TEMPLATE.format(node_id=node_id)) @@ -155,7 +155,7 @@ def test_reconnect(started_cluster, node_name, first_user, query_base): with PartitionManager() as pm: # Break the connection. - pm.partition_instances(*NODES.values()) + pm.partition_instances(*list(NODES.values())) # Now it shouldn't: _check_timeout_and_exception(node, first_user, query_base, query) diff --git a/tests/integration/test_drop_replica/test.py b/tests/integration/test_drop_replica/test.py index fac8802b2f9..f3af9dcb980 100644 --- a/tests/integration/test_drop_replica/test.py +++ b/tests/integration/test_drop_replica/test.py @@ -65,7 +65,7 @@ def start_cluster(): yield cluster except Exception as ex: - print ex + print(ex) finally: cluster.shutdown() diff --git a/tests/integration/test_fetch_partition_from_auxiliary_zookeeper/test.py b/tests/integration/test_fetch_partition_from_auxiliary_zookeeper/test.py index 9ad56d4fb17..f9c10d68fe3 100644 --- a/tests/integration/test_fetch_partition_from_auxiliary_zookeeper/test.py +++ b/tests/integration/test_fetch_partition_from_auxiliary_zookeeper/test.py @@ -1,4 +1,4 @@ -from __future__ import print_function + import pytest from helpers.client import QueryRuntimeException diff --git a/tests/integration/test_format_avro_confluent/test.py b/tests/integration/test_format_avro_confluent/test.py index 67d15305333..cc006801735 100644 --- a/tests/integration/test_format_avro_confluent/test.py +++ b/tests/integration/test_format_avro_confluent/test.py @@ -3,7 +3,7 @@ import logging import avro.schema import pytest -from confluent.schemaregistry.serializers import MessageSerializer +from confluent_kafka.avro.serializer.message_serializer import MessageSerializer from helpers.cluster import ClickHouseCluster, ClickHouseInstance logging.getLogger().setLevel(logging.INFO) diff --git a/tests/integration/test_grant_and_revoke/test.py b/tests/integration/test_grant_and_revoke/test.py index 073578edaa5..0404120907d 100644 --- a/tests/integration/test_grant_and_revoke/test.py +++ b/tests/integration/test_grant_and_revoke/test.py @@ -226,8 +226,8 @@ def test_introspection(): assert instance.query( "SELECT * from system.grants WHERE user_name IN ('A', 'B') ORDER BY user_name, access_type, grant_option") == \ - TSV([["A", "\N", "SELECT", "test", "table", "\N", 0, 0], - ["B", "\N", "CREATE", "\N", "\N", "\N", 0, 1]]) + TSV([["A", "\\N", "SELECT", "test", "table", "\\N", 0, 0], + ["B", "\\N", "CREATE", "\\N", "\\N", "\\N", 0, 1]]) def test_current_database(): diff --git a/tests/integration/test_graphite_merge_tree/test.py b/tests/integration/test_graphite_merge_tree/test.py index 319fdb816ff..502004d2dfe 100644 --- a/tests/integration/test_graphite_merge_tree/test.py +++ b/tests/integration/test_graphite_merge_tree/test.py @@ -301,7 +301,7 @@ CREATE TABLE test.graphite2 "AND table='graphite2'")) if parts == 1: break - print('Parts', parts) + print(('Parts', parts)) assert TSV( q("SELECT value, timestamp, date, updated FROM test.graphite2") diff --git a/tests/integration/test_host_ip_change/test.py b/tests/integration/test_host_ip_change/test.py index 951af699a5f..7525914e803 100644 --- a/tests/integration/test_host_ip_change/test.py +++ b/tests/integration/test_host_ip_change/test.py @@ -35,7 +35,7 @@ def cluster_without_dns_cache_update(): yield cluster except Exception as ex: - print ex + print(ex) finally: cluster.shutdown() @@ -90,7 +90,7 @@ def cluster_with_dns_cache_update(): yield cluster except Exception as ex: - print ex + print(ex) finally: cluster.shutdown() @@ -117,7 +117,7 @@ def test_ip_change_update_dns_cache(cluster_with_dns_cache_update): curl_result = node4.exec_in_container(["bash", "-c", "curl -s 'node3:8123'"]) assert curl_result == 'Ok.\n' cat_resolv = node4.exec_in_container(["bash", "-c", "cat /etc/resolv.conf"]) - print("RESOLV {}".format(cat_resolv)) + print(("RESOLV {}".format(cat_resolv))) assert_eq_with_retry(node4, "SELECT * FROM remote('node3', 'system', 'one')", "0", sleep_time=0.5) diff --git a/tests/integration/test_http_handlers_config/test.py b/tests/integration/test_http_handlers_config/test.py index 06602ba3ca3..818a1e54640 100644 --- a/tests/integration/test_http_handlers_config/test.py +++ b/tests/integration/test_http_handlers_config/test.py @@ -1,6 +1,6 @@ import contextlib import os -import urllib +import urllib.request, urllib.parse, urllib.error from helpers.cluster import ClickHouseCluster @@ -22,7 +22,7 @@ class SimpleCluster: def test_dynamic_query_handler(): with contextlib.closing( SimpleCluster(ClickHouseCluster(__file__), "dynamic_handler", "test_dynamic_handler")) as cluster: - test_query = urllib.quote_plus('SELECT * FROM system.settings WHERE name = \'max_threads\'') + test_query = urllib.parse.quote_plus('SELECT * FROM system.settings WHERE name = \'max_threads\'') assert 404 == cluster.instance.http_request('?max_threads=1', method='GET', headers={'XXX': 'xxx'}).status_code @@ -54,11 +54,11 @@ def test_predefined_query_handler(): assert 500 == cluster.instance.http_request('test_predefined_handler_get?max_threads=1', method='GET', headers={'XXX': 'xxx'}).status_code - assert 'max_threads\t1\n' == cluster.instance.http_request( + assert b'max_threads\t1\n' == cluster.instance.http_request( 'test_predefined_handler_get?max_threads=1&setting_name=max_threads', method='GET', headers={'XXX': 'xxx'}).content - assert 'max_threads\t1\nmax_alter_threads\t1\n' == cluster.instance.http_request( + assert b'max_threads\t1\nmax_alter_threads\t1\n' == cluster.instance.http_request( 'query_param_with_url/max_threads?max_threads=1&max_alter_threads=1', headers={'XXX': 'max_alter_threads'}).content @@ -79,7 +79,7 @@ def test_fixed_static_handler(): assert 'text/html; charset=UTF-8' == \ cluster.instance.http_request('test_get_fixed_static_handler', method='GET', headers={'XXX': 'xxx'}).headers['Content-Type'] - assert 'Test get static handler and fix content' == cluster.instance.http_request( + assert b'Test get static handler and fix content' == cluster.instance.http_request( 'test_get_fixed_static_handler', method='GET', headers={'XXX': 'xxx'}).content @@ -100,7 +100,7 @@ def test_config_static_handler(): assert 'text/plain; charset=UTF-8' == \ cluster.instance.http_request('test_get_config_static_handler', method='GET', headers={'XXX': 'xxx'}).headers['Content-Type'] - assert 'Test get static handler and config content' == cluster.instance.http_request( + assert b'Test get static handler and config content' == cluster.instance.http_request( 'test_get_config_static_handler', method='GET', headers={'XXX': 'xxx'}).content @@ -126,7 +126,7 @@ def test_absolute_path_static_handler(): assert 'text/html; charset=UTF-8' == \ cluster.instance.http_request('test_get_absolute_path_static_handler', method='GET', headers={'XXX': 'xxx'}).headers['Content-Type'] - assert 'Absolute Path File\n' == cluster.instance.http_request( + assert b'Absolute Path File\n' == cluster.instance.http_request( 'test_get_absolute_path_static_handler', method='GET', headers={'XXX': 'xxx'}).content @@ -152,7 +152,7 @@ def test_relative_path_static_handler(): assert 'text/html; charset=UTF-8' == \ cluster.instance.http_request('test_get_relative_path_static_handler', method='GET', headers={'XXX': 'xxx'}).headers['Content-Type'] - assert 'Relative Path File\n' == cluster.instance.http_request( + assert b'Relative Path File\n' == cluster.instance.http_request( 'test_get_relative_path_static_handler', method='GET', headers={'XXX': 'xxx'}).content @@ -160,19 +160,19 @@ def test_defaults_http_handlers(): with contextlib.closing( SimpleCluster(ClickHouseCluster(__file__), "defaults_handlers", "test_defaults_handlers")) as cluster: assert 200 == cluster.instance.http_request('', method='GET').status_code - assert 'Default server response' == cluster.instance.http_request('', method='GET').content + assert b'Default server response' == cluster.instance.http_request('', method='GET').content assert 200 == cluster.instance.http_request('ping', method='GET').status_code - assert 'Ok.\n' == cluster.instance.http_request('ping', method='GET').content + assert b'Ok.\n' == cluster.instance.http_request('ping', method='GET').content assert 200 == cluster.instance.http_request('replicas_status', method='get').status_code - assert 'Ok.\n' == cluster.instance.http_request('replicas_status', method='get').content + assert b'Ok.\n' == cluster.instance.http_request('replicas_status', method='get').content assert 200 == cluster.instance.http_request('replicas_status?verbose=1', method='get').status_code - assert '' == cluster.instance.http_request('replicas_status?verbose=1', method='get').content + assert b'' == cluster.instance.http_request('replicas_status?verbose=1', method='get').content assert 200 == cluster.instance.http_request('?query=SELECT+1', method='GET').status_code - assert '1\n' == cluster.instance.http_request('?query=SELECT+1', method='GET').content + assert b'1\n' == cluster.instance.http_request('?query=SELECT+1', method='GET').content def test_prometheus_handler(): @@ -186,7 +186,7 @@ def test_prometheus_handler(): headers={'XXX': 'xxx'}).status_code assert 200 == cluster.instance.http_request('test_prometheus', method='GET', headers={'XXX': 'xxx'}).status_code - assert 'ClickHouseProfileEvents_Query' in cluster.instance.http_request('test_prometheus', method='GET', + assert b'ClickHouseProfileEvents_Query' in cluster.instance.http_request('test_prometheus', method='GET', headers={'XXX': 'xxx'}).content @@ -203,5 +203,5 @@ def test_replicas_status_handler(): assert 200 == cluster.instance.http_request('test_replicas_status', method='GET', headers={'XXX': 'xxx'}).status_code - assert 'Ok.\n' == cluster.instance.http_request('test_replicas_status', method='GET', + assert b'Ok.\n' == cluster.instance.http_request('test_replicas_status', method='GET', headers={'XXX': 'xxx'}).content diff --git a/tests/integration/test_https_replication/test.py b/tests/integration/test_https_replication/test.py index 84c2744923d..dbb6874718c 100644 --- a/tests/integration/test_https_replication/test.py +++ b/tests/integration/test_https_replication/test.py @@ -75,7 +75,7 @@ def test_replication_after_partition(both_https_cluster): closing_pool = Pool(1) inserting_pool = Pool(5) cres = closing_pool.map_async(close, [random.randint(1, 3) for _ in range(10)]) - ires = inserting_pool.map_async(insert_data_and_check, range(100)) + ires = inserting_pool.map_async(insert_data_and_check, list(range(100))) cres.wait() ires.wait() diff --git a/tests/integration/test_insert_into_distributed_sync_async/test.py b/tests/integration/test_insert_into_distributed_sync_async/test.py index 30c80e50c43..372ed04cd2c 100755 --- a/tests/integration/test_insert_into_distributed_sync_async/test.py +++ b/tests/integration/test_insert_into_distributed_sync_async/test.py @@ -1,4 +1,3 @@ -#!/usr/bin/env python2 import os import sys from contextlib import contextmanager @@ -119,6 +118,6 @@ def test_async_inserts_into_local_shard(started_cluster): if __name__ == '__main__': with contextmanager(started_cluster)() as cluster: - for name, instance in cluster.instances.items(): - print name, instance.ip_address - raw_input("Cluster created, press any key to destroy...") + for name, instance in list(cluster.instances.items()): + print(name, instance.ip_address) + input("Cluster created, press any key to destroy...") diff --git a/tests/integration/test_live_view_over_distributed/test.py b/tests/integration/test_live_view_over_distributed/test.py index 67ff4d8dfe7..a21eeb772e5 100644 --- a/tests/integration/test_live_view_over_distributed/test.py +++ b/tests/integration/test_live_view_over_distributed/test.py @@ -1,4 +1,4 @@ -from __future__ import print_function + import sys @@ -7,6 +7,8 @@ from helpers.cluster import ClickHouseCluster from helpers.uclient import client, prompt, end_of_block cluster = ClickHouseCluster(__file__) +# log = sys.stdout +log = None NODES = {'node' + str(i): cluster.add_instance( 'node' + str(i), @@ -55,7 +57,7 @@ def started_cluster(): cluster.shutdown() -@pytest.mark.parametrize("node", NODES.values()[:1]) +@pytest.mark.parametrize("node", list(NODES.values())[:1]) @pytest.mark.parametrize("source", ["lv_over_distributed_table"]) class TestLiveViewOverDistributedSuite: def test_select_with_order_by_node(self, started_cluster, node, source): @@ -87,7 +89,6 @@ node2\t1\t11 == "22\n" def test_watch_live_view_order_by_node(self, started_cluster, node, source): - log = sys.stdout command = " ".join(node.client.command) args = dict(log=log, command=command) @@ -130,7 +131,6 @@ node2\t1\t11 client1.expect('"node3",3,3,3') def test_watch_live_view_order_by_key(self, started_cluster, node, source): - log = sys.stdout command = " ".join(node.client.command) args = dict(log=log, command=command) @@ -173,7 +173,6 @@ node2\t1\t11 client1.expect('"node3",3,3,3') def test_watch_live_view_group_by_node(self, started_cluster, node, source): - log = sys.stdout command = " ".join(node.client.command) args = dict(log=log, command=command) @@ -208,7 +207,6 @@ node2\t1\t11 client1.expect('"node3",3,3') def test_watch_live_view_group_by_key(self, started_cluster, node, source): - log = sys.stdout command = " ".join(node.client.command) args = dict(log=log, command=command) sep = ' \xe2\x94\x82' @@ -245,7 +243,6 @@ node2\t1\t11 client1.expect('3,3,3') def test_watch_live_view_sum(self, started_cluster, node, source): - log = sys.stdout command = " ".join(node.client.command) args = dict(log=log, command=command) diff --git a/tests/integration/test_materialize_mysql_database/materialize_with_ddl.py b/tests/integration/test_materialize_mysql_database/materialize_with_ddl.py index 813e72fe7a7..3990f7dbd33 100644 --- a/tests/integration/test_materialize_mysql_database/materialize_with_ddl.py +++ b/tests/integration/test_materialize_mysql_database/materialize_with_ddl.py @@ -11,7 +11,7 @@ def check_query(clickhouse_node, query, result_set, retry_count=3, interval_seco if result_set == lastest_result: return - print lastest_result + print(lastest_result) time.sleep(interval_seconds) assert lastest_result == result_set diff --git a/tests/integration/test_materialize_mysql_database/test.py b/tests/integration/test_materialize_mysql_database/test.py index 81a69dd7c54..237df999c62 100644 --- a/tests/integration/test_materialize_mysql_database/test.py +++ b/tests/integration/test_materialize_mysql_database/test.py @@ -6,7 +6,7 @@ import pymysql.cursors import pytest from helpers.cluster import ClickHouseCluster, get_docker_compose_path -import materialize_with_ddl +from . import materialize_with_ddl DOCKER_COMPOSE_PATH = get_docker_compose_path() @@ -50,10 +50,10 @@ class MySQLNodeInstance: while time.time() - start < timeout: try: self.alloc_connection() - print "Mysql Started" + print("Mysql Started") return except Exception as ex: - print "Can't connect to MySQL " + str(ex) + print("Can't connect to MySQL " + str(ex)) time.sleep(0.5) subprocess.check_call(['docker-compose', 'ps', '--services', 'all']) @@ -119,8 +119,8 @@ def test_materialize_database_ddl_with_mysql_5_7(started_cluster, started_mysql_ materialize_with_ddl.alter_modify_column_with_materialize_mysql_database(clickhouse_node, started_mysql_5_7, "mysql1") except: - print(clickhouse_node.query( - "select '\n', thread_id, query_id, arrayStringConcat(arrayMap(x -> concat(demangle(addressToSymbol(x)), '\n ', addressToLine(x)), trace), '\n') AS sym from system.stack_trace format TSVRaw")) + print((clickhouse_node.query( + "select '\n', thread_id, query_id, arrayStringConcat(arrayMap(x -> concat(demangle(addressToSymbol(x)), '\n ', addressToLine(x)), trace), '\n') AS sym from system.stack_trace format TSVRaw"))) raise diff --git a/tests/integration/test_max_http_connections_for_replication/test.py b/tests/integration/test_max_http_connections_for_replication/test.py index 5ef45c2a893..2dc4e2a8810 100644 --- a/tests/integration/test_max_http_connections_for_replication/test.py +++ b/tests/integration/test_max_http_connections_for_replication/test.py @@ -44,12 +44,12 @@ def start_small_cluster(): def test_single_endpoint_connections_count(start_small_cluster): def task(count): - print("Inserting ten times from {}".format(count)) - for i in xrange(count, count + 10): + print(("Inserting ten times from {}".format(count))) + for i in range(count, count + 10): node1.query("insert into test_table values ('2017-06-16', {}, 0)".format(i)) p = Pool(10) - p.map(task, xrange(0, 100, 10)) + p.map(task, range(0, 100, 10)) assert_eq_with_retry(node1, "select count() from test_table", "100") assert_eq_with_retry(node2, "select count() from test_table", "100") @@ -97,17 +97,17 @@ def start_big_cluster(): def test_multiple_endpoint_connections_count(start_big_cluster): def task(count): - print("Inserting ten times from {}".format(count)) + print(("Inserting ten times from {}".format(count))) if (count / 10) % 2 == 1: node = node3 else: node = node4 - for i in xrange(count, count + 10): + for i in range(count, count + 10): node.query("insert into test_table values ('2017-06-16', {}, 0)".format(i)) p = Pool(10) - p.map(task, xrange(0, 100, 10)) + p.map(task, range(0, 100, 10)) assert_eq_with_retry(node3, "select count() from test_table", "100") assert_eq_with_retry(node4, "select count() from test_table", "100") diff --git a/tests/integration/test_merge_table_over_distributed/test.py b/tests/integration/test_merge_table_over_distributed/test.py index 2e73bd09ded..ab294867126 100644 --- a/tests/integration/test_merge_table_over_distributed/test.py +++ b/tests/integration/test_merge_table_over_distributed/test.py @@ -68,6 +68,6 @@ def test_select_table_name_from_merge_over_distributed(started_cluster): if __name__ == '__main__': with contextmanager(started_cluster)() as cluster: - for name, instance in cluster.instances.items(): - print name, instance.ip_address - raw_input("Cluster created, press any key to destroy...") + for name, instance in list(cluster.instances.items()): + print(name, instance.ip_address) + input("Cluster created, press any key to destroy...") diff --git a/tests/integration/test_multiple_disks/test.py b/tests/integration/test_multiple_disks/test.py index d12c7f7b357..5058bcf368e 100644 --- a/tests/integration/test_multiple_disks/test.py +++ b/tests/integration/test_multiple_disks/test.py @@ -954,7 +954,7 @@ def test_mutate_to_another_disk(start_cluster, name, engine): if node1.query("SELECT latest_fail_reason FROM system.mutations WHERE table = '{}'".format(name)) == "": assert node1.query("SELECT sum(endsWith(s1, 'x')) FROM {}".format(name)) == "25\n" else: # mutation failed, let's try on another disk - print "Mutation failed" + print("Mutation failed") node1.query("OPTIMIZE TABLE {} FINAL".format(name)) node1.query("ALTER TABLE {} UPDATE s1 = concat(s1, 'x') WHERE 1".format(name)) retry = 20 @@ -1114,7 +1114,7 @@ def test_download_appropriate_disk(start_cluster): for _ in range(10): try: - print "Syncing replica" + print("Syncing replica") node2.query("SYSTEM SYNC REPLICA replicated_table_for_download") break except: diff --git a/tests/integration/test_mutations_hardlinks/test.py b/tests/integration/test_mutations_hardlinks/test.py index b1e538b123b..7ac7fe12108 100644 --- a/tests/integration/test_mutations_hardlinks/test.py +++ b/tests/integration/test_mutations_hardlinks/test.py @@ -122,7 +122,7 @@ def test_delete_and_drop_mutation(started_cluster): if int(result.strip()) == 2: break except: - print "Result", result + print("Result", result) pass time.sleep(0.5) diff --git a/tests/integration/test_mutations_with_merge_tree/test.py b/tests/integration/test_mutations_with_merge_tree/test.py index 25bc0df8e7c..65eaee21577 100644 --- a/tests/integration/test_mutations_with_merge_tree/test.py +++ b/tests/integration/test_mutations_with_merge_tree/test.py @@ -44,8 +44,8 @@ def test_mutations_with_merge_background_task(started_cluster): all_done = True break - print instance_test_mutations.query( - "SELECT mutation_id, command, parts_to_do, is_done FROM system.mutations WHERE table = 'test_mutations_with_ast_elements' SETTINGS force_index_by_date = 0, force_primary_key = 0 FORMAT TSVWithNames") + print(instance_test_mutations.query( + "SELECT mutation_id, command, parts_to_do, is_done FROM system.mutations WHERE table = 'test_mutations_with_ast_elements' SETTINGS force_index_by_date = 0, force_primary_key = 0 FORMAT TSVWithNames")) assert all_done diff --git a/tests/integration/test_mysql_database_engine/test.py b/tests/integration/test_mysql_database_engine/test.py index a8824b383ab..23424c550b2 100644 --- a/tests/integration/test_mysql_database_engine/test.py +++ b/tests/integration/test_mysql_database_engine/test.py @@ -44,7 +44,7 @@ class MySQLNodeInstance: res = "\n".join(rows) return res - if isinstance(execution_query, (str, bytes, unicode)): + if isinstance(execution_query, (str, bytes)): return execute(execution_query) else: return [execute(q) for q in execution_query] @@ -256,7 +256,7 @@ def test_mysql_types(started_cluster, case_name, mysql_type, expected_ch_type, m res = node.query(query, **kwargs) return res if isinstance(res, int) else res.rstrip('\n\r') - if isinstance(query, (str, bytes, unicode)): + if isinstance(query, (str, bytes)): return do_execute(query) else: return [do_execute(q) for q in query] diff --git a/tests/integration/test_mysql_protocol/test.py b/tests/integration/test_mysql_protocol/test.py index 3e737dc2644..04cbef59af7 100644 --- a/tests/integration/test_mysql_protocol/test.py +++ b/tests/integration/test_mysql_protocol/test.py @@ -98,7 +98,7 @@ def test_mysql_client(mysql_client, server_address): -e "SELECT 1;" '''.format(host=server_address, port=server_port), demux=True) - assert stdout == '\n'.join(['1', '1', '']) + assert stdout.decode() == '\n'.join(['1', '1', '']) code, (stdout, stderr) = mysql_client.exec_run(''' mysql --protocol tcp -h {host} -P {port} default -u default --password=123 @@ -106,13 +106,13 @@ def test_mysql_client(mysql_client, server_address): -e "SELECT 'тест' as b;" '''.format(host=server_address, port=server_port), demux=True) - assert stdout == '\n'.join(['a', '1', 'b', 'тест', '']) + assert stdout.decode() == '\n'.join(['a', '1', 'b', 'тест', '']) code, (stdout, stderr) = mysql_client.exec_run(''' mysql --protocol tcp -h {host} -P {port} default -u default --password=abc -e "select 1 as a;" '''.format(host=server_address, port=server_port), demux=True) - assert stderr == 'mysql: [Warning] Using a password on the command line interface can be insecure.\n' \ + assert stderr.decode() == 'mysql: [Warning] Using a password on the command line interface can be insecure.\n' \ 'ERROR 516 (00000): default: Authentication failed: password is incorrect or there is no user with such name\n' code, (stdout, stderr) = mysql_client.exec_run(''' @@ -122,8 +122,8 @@ def test_mysql_client(mysql_client, server_address): -e "use system2;" '''.format(host=server_address, port=server_port), demux=True) - assert stdout == 'count()\n1\n' - assert stderr[0:182] == "mysql: [Warning] Using a password on the command line interface can be insecure.\n" \ + assert stdout.decode() == 'count()\n1\n' + assert stderr[0:182].decode() == "mysql: [Warning] Using a password on the command line interface can be insecure.\n" \ "ERROR 81 (00000) at line 1: Code: 81, e.displayText() = DB::Exception: Database system2 doesn't exist" code, (stdout, stderr) = mysql_client.exec_run(''' @@ -140,7 +140,7 @@ def test_mysql_client(mysql_client, server_address): -e "SELECT * FROM tmp ORDER BY tmp_column;" '''.format(host=server_address, port=server_port), demux=True) - assert stdout == '\n'.join(['column', '0', '0', '1', '1', '5', '5', 'tmp_column', '0', '1', '']) + assert stdout.decode() == '\n'.join(['column', '0', '0', '1', '1', '5', '5', 'tmp_column', '0', '1', '']) def test_mysql_client_exception(mysql_client, server_address): @@ -150,7 +150,7 @@ def test_mysql_client_exception(mysql_client, server_address): -e "CREATE TABLE default.t1_remote_mysql AS mysql('127.0.0.1:10086','default','t1_local','default','');" '''.format(host=server_address, port=server_port), demux=True) - assert stderr[0:266] == "mysql: [Warning] Using a password on the command line interface can be insecure.\n" \ + assert stderr[0:266].decode() == "mysql: [Warning] Using a password on the command line interface can be insecure.\n" \ "ERROR 1000 (00000) at line 1: Poco::Exception. Code: 1000, e.code() = 2002, e.displayText() = mysqlxx::ConnectionFailed: Can't connect to MySQL server on '127.0.0.1' (115) ((nullptr):0)" @@ -188,14 +188,14 @@ def test_mysql_replacement_query(mysql_client, server_address): --password=123 -e "select database();" '''.format(host=server_address, port=server_port), demux=True) assert code == 0 - assert stdout == 'database()\ndefault\n' + assert stdout.decode() == 'database()\ndefault\n' code, (stdout, stderr) = mysql_client.exec_run(''' mysql --protocol tcp -h {host} -P {port} default -u default --password=123 -e "select DATABASE();" '''.format(host=server_address, port=server_port), demux=True) assert code == 0 - assert stdout == 'DATABASE()\ndefault\n' + assert stdout.decode() == 'DATABASE()\ndefault\n' def test_mysql_explain(mysql_client, server_address): @@ -238,6 +238,7 @@ def test_mysql_federated(mysql_server, server_address): node.query('''INSERT INTO mysql_federated.test VALUES (0), (1), (5)''', settings={"password": "123"}) def check_retryable_error_in_stderr(stderr): + stderr = stderr.decode() return ("Can't connect to local MySQL server through socket" in stderr or "MySQL server has gone away" in stderr or "Server shutdown in progress" in stderr) @@ -252,8 +253,8 @@ def test_mysql_federated(mysql_server, server_address): '''.format(host=server_address, port=server_port), demux=True) if code != 0: - print("stdout", stdout) - print("stderr", stderr) + print(("stdout", stdout)) + print(("stderr", stderr)) if try_num + 1 < retries and check_retryable_error_in_stderr(stderr): time.sleep(1) continue @@ -266,14 +267,14 @@ def test_mysql_federated(mysql_server, server_address): '''.format(host=server_address, port=server_port), demux=True) if code != 0: - print("stdout", stdout) - print("stderr", stderr) + print(("stdout", stdout)) + print(("stderr", stderr)) if try_num + 1 < retries and check_retryable_error_in_stderr(stderr): time.sleep(1) continue assert code == 0 - assert stdout == '\n'.join(['col', '0', '1', '5', '']) + assert stdout.decode() == '\n'.join(['col', '0', '1', '5', '']) code, (stdout, stderr) = mysql_server.exec_run(''' mysql @@ -282,14 +283,14 @@ def test_mysql_federated(mysql_server, server_address): '''.format(host=server_address, port=server_port), demux=True) if code != 0: - print("stdout", stdout) - print("stderr", stderr) + print(("stdout", stdout)) + print(("stderr", stderr)) if try_num + 1 < retries and check_retryable_error_in_stderr(stderr): time.sleep(1) continue assert code == 0 - assert stdout == '\n'.join(['col', '0', '0', '1', '1', '5', '5', '']) + assert stdout.decode() == '\n'.join(['col', '0', '0', '1', '1', '5', '5', '']) def test_mysql_set_variables(mysql_client, server_address): @@ -362,7 +363,7 @@ def test_python_client(server_address): def test_golang_client(server_address, golang_container): # type: (str, Container) -> None - with open(os.path.join(SCRIPT_DIR, 'golang.reference')) as fp: + with open(os.path.join(SCRIPT_DIR, 'golang.reference'), 'rb') as fp: reference = fp.read() code, (stdout, stderr) = golang_container.exec_run( @@ -370,7 +371,7 @@ def test_golang_client(server_address, golang_container): 'abc'.format(host=server_address, port=server_port), demux=True) assert code == 1 - assert stderr == "Error 81: Database abc doesn't exist\n" + assert stderr.decode() == "Error 81: Database abc doesn't exist\n" code, (stdout, stderr) = golang_container.exec_run( './main --host {host} --port {port} --user default --password 123 --database ' @@ -391,31 +392,31 @@ def test_php_client(server_address, php_container): code, (stdout, stderr) = php_container.exec_run( 'php -f test.php {host} {port} default 123'.format(host=server_address, port=server_port), demux=True) assert code == 0 - assert stdout == 'tables\n' + assert stdout.decode() == 'tables\n' code, (stdout, stderr) = php_container.exec_run( 'php -f test_ssl.php {host} {port} default 123'.format(host=server_address, port=server_port), demux=True) assert code == 0 - assert stdout == 'tables\n' + assert stdout.decode() == 'tables\n' code, (stdout, stderr) = php_container.exec_run( 'php -f test.php {host} {port} user_with_double_sha1 abacaba'.format(host=server_address, port=server_port), demux=True) assert code == 0 - assert stdout == 'tables\n' + assert stdout.decode() == 'tables\n' code, (stdout, stderr) = php_container.exec_run( 'php -f test_ssl.php {host} {port} user_with_double_sha1 abacaba'.format(host=server_address, port=server_port), demux=True) assert code == 0 - assert stdout == 'tables\n' + assert stdout.decode() == 'tables\n' def test_mysqljs_client(server_address, nodejs_container): code, (_, stderr) = nodejs_container.exec_run( 'node test.js {host} {port} user_with_sha256 abacaba'.format(host=server_address, port=server_port), demux=True) assert code == 1 - assert 'MySQL is requesting the sha256_password authentication method, which is not supported.' in stderr + assert 'MySQL is requesting the sha256_password authentication method, which is not supported.' in stderr.decode() code, (_, stderr) = nodejs_container.exec_run( 'node test.js {host} {port} user_with_empty_password ""'.format(host=server_address, port=server_port), @@ -449,21 +450,21 @@ def test_java_client(server_address, java_container): 'java JavaConnectorTest --host {host} --port {port} --user user_with_empty_password --database ' 'default'.format(host=server_address, port=server_port), demux=True) assert code == 0 - assert stdout == reference + assert stdout.decode() == reference # non-empty password passed. code, (stdout, stderr) = java_container.exec_run( 'java JavaConnectorTest --host {host} --port {port} --user default --password 123 --database ' 'default'.format(host=server_address, port=server_port), demux=True) assert code == 0 - assert stdout == reference + assert stdout.decode() == reference # double-sha1 password passed. code, (stdout, stderr) = java_container.exec_run( 'java JavaConnectorTest --host {host} --port {port} --user user_with_double_sha1 --password abacaba --database ' 'default'.format(host=server_address, port=server_port), demux=True) assert code == 0 - assert stdout == reference + assert stdout.decode() == reference def test_types(server_address): diff --git a/tests/integration/test_no_local_metadata_node/test.py b/tests/integration/test_no_local_metadata_node/test.py index ae69f5e1384..f976cc005bd 100644 --- a/tests/integration/test_no_local_metadata_node/test.py +++ b/tests/integration/test_no_local_metadata_node/test.py @@ -47,7 +47,7 @@ def test_table_start_without_metadata(start_cluster): node1.query("DETACH TABLE test") - zk_cli.set("/clickhouse/table/test_table/replicas/1/metadata", "") + zk_cli.set("/clickhouse/table/test_table/replicas/1/metadata", b"") node1.query("ATTACH TABLE test") diff --git a/tests/integration/test_non_default_compression/test.py b/tests/integration/test_non_default_compression/test.py index 4706d8efbdd..03210e47081 100644 --- a/tests/integration/test_non_default_compression/test.py +++ b/tests/integration/test_non_default_compression/test.py @@ -82,7 +82,7 @@ def test_preconfigured_custom_codec(start_cluster): assert node3.query( "SELECT max(length(data)) from compression_codec_multiple_with_key GROUP BY data ORDER BY max(length(data)) DESC LIMIT 1") == "10000\n" - for i in xrange(10): + for i in range(10): node3.query( "INSERT INTO compression_codec_multiple_with_key VALUES(toDate('2018-10-12'), {}, '{}', 88.88)".format(i, ''.join( diff --git a/tests/integration/test_odbc_interaction/test.py b/tests/integration/test_odbc_interaction/test.py index f527b4cc66e..028137cef12 100644 --- a/tests/integration/test_odbc_interaction/test.py +++ b/tests/integration/test_odbc_interaction/test.py @@ -60,7 +60,7 @@ def started_cluster(): cluster.start() sqlite_db = node1.odbc_drivers["SQLite3"]["Database"] - print "sqlite data received" + print("sqlite data received") node1.exec_in_container( ["bash", "-c", "echo 'CREATE TABLE t1(x INTEGER PRIMARY KEY ASC, y, z);' | sqlite3 {}".format(sqlite_db)], privileged=True, user='root') @@ -73,18 +73,18 @@ def started_cluster(): node1.exec_in_container( ["bash", "-c", "echo 'CREATE TABLE t4(X INTEGER PRIMARY KEY ASC, Y, Z);' | sqlite3 {}".format(sqlite_db)], privileged=True, user='root') - print "sqlite tables created" + print("sqlite tables created") mysql_conn = get_mysql_conn() - print "mysql connection received" + print("mysql connection received") ## create mysql db and table create_mysql_db(mysql_conn, 'clickhouse') - print "mysql database created" + print("mysql database created") postgres_conn = get_postgres_conn() - print "postgres connection received" + print("postgres connection received") create_postgres_db(postgres_conn, 'clickhouse') - print "postgres db created" + print("postgres db created") cursor = postgres_conn.cursor() cursor.execute( @@ -259,7 +259,7 @@ def test_postgres_odbc_hached_dictionary_no_tty_pipe_overflow(started_cluster): conn = get_postgres_conn() cursor = conn.cursor() cursor.execute("insert into clickhouse.test_table values(3, 'xxx')") - for i in xrange(100): + for i in range(100): try: node1.query("system reload dictionary postgres_odbc_hashed", timeout=5) except Exception as ex: diff --git a/tests/integration/test_partition/test.py b/tests/integration/test_partition/test.py index 679c6fb8c5b..b5facb5f4b2 100644 --- a/tests/integration/test_partition/test.py +++ b/tests/integration/test_partition/test.py @@ -52,7 +52,7 @@ def partition_complex_assert_columns_txt(): for part_name in parts.lines: path_to_columns = path_to_parts + part_name + '/columns.txt' # 2 header lines + 3 columns - assert exec_bash('cat {} | wc -l'.format(path_to_columns)) == u'5\n' + assert exec_bash('cat {} | wc -l'.format(path_to_columns)) == '5\n' def partition_complex_assert_checksums(): @@ -145,7 +145,7 @@ def cannot_attach_active_part_table(started_cluster): def test_cannot_attach_active_part(cannot_attach_active_part_table): error = instance.client.query_and_get_error("ALTER TABLE test.attach_active ATTACH PART '../1_2_2_0'") - print error + print(error) assert 0 <= error.find('Invalid part name') res = q("SElECT name FROM system.parts WHERE table='attach_active' AND database='test' ORDER BY name") diff --git a/tests/integration/test_parts_delete_zookeeper/test.py b/tests/integration/test_parts_delete_zookeeper/test.py index 7489b2411f9..8a4aafaa55c 100644 --- a/tests/integration/test_parts_delete_zookeeper/test.py +++ b/tests/integration/test_parts_delete_zookeeper/test.py @@ -26,7 +26,7 @@ def start_cluster(): yield cluster except Exception as ex: - print ex + print(ex) finally: cluster.shutdown() diff --git a/tests/integration/test_postgresql_protocol/test.py b/tests/integration/test_postgresql_protocol/test.py index 513bb75fcab..52c911cb939 100644 --- a/tests/integration/test_postgresql_protocol/test.py +++ b/tests/integration/test_postgresql_protocol/test.py @@ -1,6 +1,6 @@ # -*- coding: utf-8 -*- -from __future__ import print_function + import datetime import decimal @@ -81,10 +81,10 @@ def test_psql_client(psql_client, server_address): cmd_prefix += "--no-align --field-separator=' ' " code, (stdout, stderr) = psql_client.exec_run(cmd_prefix + '-c "SELECT 1 as a"', demux=True) - assert stdout == '\n'.join(['a', '1', '(1 row)', '']) + assert stdout.decode() == '\n'.join(['a', '1', '(1 row)', '']) code, (stdout, stderr) = psql_client.exec_run(cmd_prefix + '''-c "SELECT 'колонка' as a"''', demux=True) - assert stdout == '\n'.join(['a', 'колонка', '(1 row)', '']) + assert stdout.decode() == '\n'.join(['a', 'колонка', '(1 row)', '']) code, (stdout, stderr) = psql_client.exec_run( cmd_prefix + '-c ' + @@ -98,7 +98,7 @@ def test_psql_client(psql_client, server_address): ''', demux=True ) - assert stdout == '\n'.join(['column', '0', '0', '1', '1', '5', '5', '(6 rows)', '']) + assert stdout.decode() == '\n'.join(['column', '0', '0', '1', '1', '5', '5', '(6 rows)', '']) code, (stdout, stderr) = psql_client.exec_run( cmd_prefix + '-c ' + @@ -110,7 +110,7 @@ def test_psql_client(psql_client, server_address): ''', demux=True ) - assert stdout == '\n'.join(['tmp_column', '0', '1', '(2 rows)', '']) + assert stdout.decode() == '\n'.join(['tmp_column', '0', '1', '(2 rows)', '']) def test_python_client(server_address): @@ -157,4 +157,4 @@ def test_java_client(server_address, java_container): 'default'.format(host=server_address, port=server_port), demux=True) print(stdout, stderr, file=sys.stderr) assert code == 0 - assert stdout == reference + assert stdout.decode() == reference diff --git a/tests/integration/test_prometheus_endpoint/test.py b/tests/integration/test_prometheus_endpoint/test.py index 909dbf139b9..06276803c3d 100644 --- a/tests/integration/test_prometheus_endpoint/test.py +++ b/tests/integration/test_prometheus_endpoint/test.py @@ -1,4 +1,4 @@ -from __future__ import print_function + import re import time diff --git a/tests/integration/test_quota/test.py b/tests/integration/test_quota/test.py index 5d2a4acffe6..0614150ee07 100644 --- a/tests/integration/test_quota/test.py +++ b/tests/integration/test_quota/test.py @@ -15,14 +15,14 @@ instance = cluster.add_instance('instance', user_configs=["configs/users.d/assig def check_system_quotas(canonical): canonical_tsv = TSV(canonical) r = TSV(instance.query("SELECT * FROM system.quotas ORDER BY name")) - print("system_quotas: {},\ncanonical: {}".format(r, TSV(canonical_tsv))) + print(("system_quotas: {},\ncanonical: {}".format(r, TSV(canonical_tsv)))) assert r == canonical_tsv def system_quota_limits(canonical): canonical_tsv = TSV(canonical) r = TSV(instance.query("SELECT * FROM system.quota_limits ORDER BY quota_name, duration")) - print("system_quota_limits: {},\ncanonical: {}".format(r, TSV(canonical_tsv))) + print(("system_quota_limits: {},\ncanonical: {}".format(r, TSV(canonical_tsv)))) assert r == canonical_tsv @@ -32,7 +32,7 @@ def system_quota_usage(canonical): "result_bytes, max_result_bytes, read_rows, max_read_rows, read_bytes, max_read_bytes, max_execution_time " \ "FROM system.quota_usage ORDER BY duration" r = TSV(instance.query(query)) - print("system_quota_usage: {},\ncanonical: {}".format(r, TSV(canonical_tsv))) + print(("system_quota_usage: {},\ncanonical: {}".format(r, TSV(canonical_tsv)))) assert r == canonical_tsv @@ -42,7 +42,7 @@ def system_quotas_usage(canonical): "result_bytes, max_result_bytes, read_rows, max_read_rows, read_bytes, max_read_bytes, max_execution_time " \ "FROM system.quotas_usage ORDER BY quota_name, quota_key, duration" r = TSV(instance.query(query)) - print("system_quotas_usage: {},\ncanonical: {}".format(r, TSV(canonical_tsv))) + print(("system_quotas_usage: {},\ncanonical: {}".format(r, TSV(canonical_tsv)))) assert r == canonical_tsv @@ -81,18 +81,18 @@ def reset_quotas_and_usage_info(): def test_quota_from_users_xml(): check_system_quotas([["myQuota", "e651da9c-a748-8703-061a-7e5e5096dae7", "users.xml", "['user_name']", [31556952], 0, "['default']", "[]"]]) - system_quota_limits([["myQuota", 31556952, 0, 1000, "\N", "\N", "\N", 1000, "\N", "\N"]]) - system_quota_usage([["myQuota", "default", 31556952, 0, 1000, 0, "\N", 0, "\N", 0, "\N", 0, 1000, 0, "\N", "\N"]]) + system_quota_limits([["myQuota", 31556952, 0, 1000, "\\N", "\\N", "\\N", 1000, "\\N", "\\N"]]) + system_quota_usage([["myQuota", "default", 31556952, 0, 1000, 0, "\\N", 0, "\\N", 0, "\\N", 0, 1000, 0, "\\N", "\\N"]]) system_quotas_usage( - [["myQuota", "default", 1, 31556952, 0, 1000, 0, "\N", 0, "\N", 0, "\N", 0, 1000, 0, "\N", "\N"]]) + [["myQuota", "default", 1, 31556952, 0, 1000, 0, "\\N", 0, "\\N", 0, "\\N", 0, 1000, 0, "\\N", "\\N"]]) instance.query("SELECT * from test_table") system_quota_usage( - [["myQuota", "default", 31556952, 1, 1000, 0, "\N", 50, "\N", 200, "\N", 50, 1000, 200, "\N", "\N"]]) + [["myQuota", "default", 31556952, 1, 1000, 0, "\\N", 50, "\\N", 200, "\\N", 50, 1000, 200, "\\N", "\\N"]]) instance.query("SELECT COUNT() from test_table") system_quota_usage( - [["myQuota", "default", 31556952, 2, 1000, 0, "\N", 51, "\N", 208, "\N", 50, 1000, 200, "\N", "\N"]]) + [["myQuota", "default", 31556952, 2, 1000, 0, "\\N", 51, "\\N", 208, "\\N", 50, 1000, 200, "\\N", "\\N"]]) def test_simpliest_quota(): @@ -102,11 +102,11 @@ def test_simpliest_quota(): "['default']", "[]"]]) system_quota_limits("") system_quota_usage( - [["myQuota", "default", "\N", "\N", "\N", "\N", "\N", "\N", "\N", "\N", "\N", "\N", "\N", "\N", "\N", "\N"]]) + [["myQuota", "default", "\\N", "\\N", "\\N", "\\N", "\\N", "\\N", "\\N", "\\N", "\\N", "\\N", "\\N", "\\N", "\\N", "\\N"]]) instance.query("SELECT * from test_table") system_quota_usage( - [["myQuota", "default", "\N", "\N", "\N", "\N", "\N", "\N", "\N", "\N", "\N", "\N", "\N", "\N", "\N", "\N"]]) + [["myQuota", "default", "\\N", "\\N", "\\N", "\\N", "\\N", "\\N", "\\N", "\\N", "\\N", "\\N", "\\N", "\\N", "\\N", "\\N"]]) def test_tracking_quota(): @@ -114,16 +114,16 @@ def test_tracking_quota(): copy_quota_xml('tracking.xml') check_system_quotas([["myQuota", "e651da9c-a748-8703-061a-7e5e5096dae7", "users.xml", "['user_name']", "[31556952]", 0, "['default']", "[]"]]) - system_quota_limits([["myQuota", 31556952, 0, "\N", "\N", "\N", "\N", "\N", "\N", "\N"]]) - system_quota_usage([["myQuota", "default", 31556952, 0, "\N", 0, "\N", 0, "\N", 0, "\N", 0, "\N", 0, "\N", "\N"]]) + system_quota_limits([["myQuota", 31556952, 0, "\\N", "\\N", "\\N", "\\N", "\\N", "\\N", "\\N"]]) + system_quota_usage([["myQuota", "default", 31556952, 0, "\\N", 0, "\\N", 0, "\\N", 0, "\\N", 0, "\\N", 0, "\\N", "\\N"]]) instance.query("SELECT * from test_table") system_quota_usage( - [["myQuota", "default", 31556952, 1, "\N", 0, "\N", 50, "\N", 200, "\N", 50, "\N", 200, "\N", "\N"]]) + [["myQuota", "default", 31556952, 1, "\\N", 0, "\\N", 50, "\\N", 200, "\\N", 50, "\\N", 200, "\\N", "\\N"]]) instance.query("SELECT COUNT() from test_table") system_quota_usage( - [["myQuota", "default", 31556952, 2, "\N", 0, "\N", 51, "\N", 208, "\N", 50, "\N", 200, "\N", "\N"]]) + [["myQuota", "default", 31556952, 2, "\\N", 0, "\\N", 51, "\\N", 208, "\\N", 50, "\\N", 200, "\\N", "\\N"]]) def test_exceed_quota(): @@ -131,55 +131,55 @@ def test_exceed_quota(): copy_quota_xml('tiny_limits.xml') check_system_quotas([["myQuota", "e651da9c-a748-8703-061a-7e5e5096dae7", "users.xml", "['user_name']", "[31556952]", 0, "['default']", "[]"]]) - system_quota_limits([["myQuota", 31556952, 0, 1, 1, 1, "\N", 1, "\N", "\N"]]) - system_quota_usage([["myQuota", "default", 31556952, 0, 1, 0, 1, 0, 1, 0, "\N", 0, 1, 0, "\N", "\N"]]) + system_quota_limits([["myQuota", 31556952, 0, 1, 1, 1, "\\N", 1, "\\N", "\\N"]]) + system_quota_usage([["myQuota", "default", 31556952, 0, 1, 0, 1, 0, 1, 0, "\\N", 0, 1, 0, "\\N", "\\N"]]) assert re.search("Quota.*has\ been\ exceeded", instance.query_and_get_error("SELECT * from test_table")) - system_quota_usage([["myQuota", "default", 31556952, 1, 1, 1, 1, 0, 1, 0, "\N", 50, 1, 0, "\N", "\N"]]) + system_quota_usage([["myQuota", "default", 31556952, 1, 1, 1, 1, 0, 1, 0, "\\N", 50, 1, 0, "\\N", "\\N"]]) # Change quota, now the limits are enough to execute queries. copy_quota_xml('normal_limits.xml') check_system_quotas([["myQuota", "e651da9c-a748-8703-061a-7e5e5096dae7", "users.xml", "['user_name']", "[31556952]", 0, "['default']", "[]"]]) - system_quota_limits([["myQuota", 31556952, 0, 1000, "\N", "\N", "\N", 1000, "\N", "\N"]]) - system_quota_usage([["myQuota", "default", 31556952, 1, 1000, 1, "\N", 0, "\N", 0, "\N", 50, 1000, 0, "\N", "\N"]]) + system_quota_limits([["myQuota", 31556952, 0, 1000, "\\N", "\\N", "\\N", 1000, "\\N", "\\N"]]) + system_quota_usage([["myQuota", "default", 31556952, 1, 1000, 1, "\\N", 0, "\\N", 0, "\\N", 50, 1000, 0, "\\N", "\\N"]]) instance.query("SELECT * from test_table") system_quota_usage( - [["myQuota", "default", 31556952, 2, 1000, 1, "\N", 50, "\N", 200, "\N", 100, 1000, 200, "\N", "\N"]]) + [["myQuota", "default", 31556952, 2, 1000, 1, "\\N", 50, "\\N", 200, "\\N", 100, 1000, 200, "\\N", "\\N"]]) def test_add_remove_interval(): check_system_quotas([["myQuota", "e651da9c-a748-8703-061a-7e5e5096dae7", "users.xml", "['user_name']", [31556952], 0, "['default']", "[]"]]) - system_quota_limits([["myQuota", 31556952, 0, 1000, "\N", "\N", "\N", 1000, "\N", "\N"]]) - system_quota_usage([["myQuota", "default", 31556952, 0, 1000, 0, "\N", 0, "\N", 0, "\N", 0, 1000, 0, "\N", "\N"]]) + system_quota_limits([["myQuota", 31556952, 0, 1000, "\\N", "\\N", "\\N", 1000, "\\N", "\\N"]]) + system_quota_usage([["myQuota", "default", 31556952, 0, 1000, 0, "\\N", 0, "\\N", 0, "\\N", 0, 1000, 0, "\\N", "\\N"]]) # Add interval. copy_quota_xml('two_intervals.xml') check_system_quotas([["myQuota", "e651da9c-a748-8703-061a-7e5e5096dae7", "users.xml", "['user_name']", "[31556952,63113904]", 0, "['default']", "[]"]]) - system_quota_limits([["myQuota", 31556952, 0, 1000, "\N", "\N", "\N", 1000, "\N", "\N"], - ["myQuota", 63113904, 1, "\N", "\N", "\N", 30000, "\N", 20000, 120]]) - system_quota_usage([["myQuota", "default", 31556952, 0, 1000, 0, "\N", 0, "\N", 0, "\N", 0, 1000, 0, "\N", "\N"], - ["myQuota", "default", 63113904, 0, "\N", 0, "\N", 0, "\N", 0, 30000, 0, "\N", 0, 20000, 120]]) + system_quota_limits([["myQuota", 31556952, 0, 1000, "\\N", "\\N", "\\N", 1000, "\\N", "\\N"], + ["myQuota", 63113904, 1, "\\N", "\\N", "\\N", 30000, "\\N", 20000, 120]]) + system_quota_usage([["myQuota", "default", 31556952, 0, 1000, 0, "\\N", 0, "\\N", 0, "\\N", 0, 1000, 0, "\\N", "\\N"], + ["myQuota", "default", 63113904, 0, "\\N", 0, "\\N", 0, "\\N", 0, 30000, 0, "\\N", 0, 20000, 120]]) instance.query("SELECT * from test_table") system_quota_usage( - [["myQuota", "default", 31556952, 1, 1000, 0, "\N", 50, "\N", 200, "\N", 50, 1000, 200, "\N", "\N"], - ["myQuota", "default", 63113904, 1, "\N", 0, "\N", 50, "\N", 200, 30000, 50, "\N", 200, 20000, 120]]) + [["myQuota", "default", 31556952, 1, 1000, 0, "\\N", 50, "\\N", 200, "\\N", 50, 1000, 200, "\\N", "\\N"], + ["myQuota", "default", 63113904, 1, "\\N", 0, "\\N", 50, "\\N", 200, 30000, 50, "\\N", 200, 20000, 120]]) # Remove interval. copy_quota_xml('normal_limits.xml') check_system_quotas([["myQuota", "e651da9c-a748-8703-061a-7e5e5096dae7", "users.xml", "['user_name']", [31556952], 0, "['default']", "[]"]]) - system_quota_limits([["myQuota", 31556952, 0, 1000, "\N", "\N", "\N", 1000, "\N", "\N"]]) + system_quota_limits([["myQuota", 31556952, 0, 1000, "\\N", "\\N", "\\N", 1000, "\\N", "\\N"]]) system_quota_usage( - [["myQuota", "default", 31556952, 1, 1000, 0, "\N", 50, "\N", 200, "\N", 50, 1000, 200, "\N", "\N"]]) + [["myQuota", "default", 31556952, 1, 1000, 0, "\\N", 50, "\\N", 200, "\\N", 50, 1000, 200, "\\N", "\\N"]]) instance.query("SELECT * from test_table") system_quota_usage( - [["myQuota", "default", 31556952, 2, 1000, 0, "\N", 100, "\N", 400, "\N", 100, 1000, 400, "\N", "\N"]]) + [["myQuota", "default", 31556952, 2, 1000, 0, "\\N", 100, "\\N", 400, "\\N", 100, 1000, 400, "\\N", "\\N"]]) # Remove all intervals. copy_quota_xml('simpliest.xml') @@ -187,26 +187,26 @@ def test_add_remove_interval(): "['default']", "[]"]]) system_quota_limits("") system_quota_usage( - [["myQuota", "default", "\N", "\N", "\N", "\N", "\N", "\N", "\N", "\N", "\N", "\N", "\N", "\N", "\N", "\N"]]) + [["myQuota", "default", "\\N", "\\N", "\\N", "\\N", "\\N", "\\N", "\\N", "\\N", "\\N", "\\N", "\\N", "\\N", "\\N", "\\N"]]) instance.query("SELECT * from test_table") system_quota_usage( - [["myQuota", "default", "\N", "\N", "\N", "\N", "\N", "\N", "\N", "\N", "\N", "\N", "\N", "\N", "\N", "\N"]]) + [["myQuota", "default", "\\N", "\\N", "\\N", "\\N", "\\N", "\\N", "\\N", "\\N", "\\N", "\\N", "\\N", "\\N", "\\N", "\\N"]]) # Add one interval back. copy_quota_xml('normal_limits.xml') check_system_quotas([["myQuota", "e651da9c-a748-8703-061a-7e5e5096dae7", "users.xml", "['user_name']", [31556952], 0, "['default']", "[]"]]) - system_quota_limits([["myQuota", 31556952, 0, 1000, "\N", "\N", "\N", 1000, "\N", "\N"]]) - system_quota_usage([["myQuota", "default", 31556952, 0, 1000, 0, "\N", 0, "\N", 0, "\N", 0, 1000, 0, "\N", "\N"]]) + system_quota_limits([["myQuota", 31556952, 0, 1000, "\\N", "\\N", "\\N", 1000, "\\N", "\\N"]]) + system_quota_usage([["myQuota", "default", 31556952, 0, 1000, 0, "\\N", 0, "\\N", 0, "\\N", 0, 1000, 0, "\\N", "\\N"]]) def test_add_remove_quota(): check_system_quotas([["myQuota", "e651da9c-a748-8703-061a-7e5e5096dae7", "users.xml", "['user_name']", [31556952], 0, "['default']", "[]"]]) - system_quota_limits([["myQuota", 31556952, 0, 1000, "\N", "\N", "\N", 1000, "\N", "\N"]]) + system_quota_limits([["myQuota", 31556952, 0, 1000, "\\N", "\\N", "\\N", 1000, "\\N", "\\N"]]) system_quotas_usage( - [["myQuota", "default", 1, 31556952, 0, 1000, 0, "\N", 0, "\N", 0, "\N", 0, 1000, 0, "\N", "\N"]]) + [["myQuota", "default", 1, 31556952, 0, 1000, 0, "\\N", 0, "\\N", 0, "\\N", 0, 1000, 0, "\\N", "\\N"]]) # Add quota. copy_quota_xml('two_quotas.xml') @@ -214,19 +214,19 @@ def test_add_remove_quota(): 0, "['default']", "[]"], ["myQuota2", "4590510c-4d13-bf21-ec8a-c2187b092e73", "users.xml", "['client_key','user_name']", "[3600,2629746]", 0, "[]", "[]"]]) - system_quota_limits([["myQuota", 31556952, 0, 1000, "\N", "\N", "\N", 1000, "\N", "\N"], - ["myQuota2", 3600, 1, "\N", "\N", 4000, 400000, 4000, 400000, 60], - ["myQuota2", 2629746, 0, "\N", "\N", "\N", "\N", "\N", "\N", 1800]]) + system_quota_limits([["myQuota", 31556952, 0, 1000, "\\N", "\\N", "\\N", 1000, "\\N", "\\N"], + ["myQuota2", 3600, 1, "\\N", "\\N", 4000, 400000, 4000, 400000, 60], + ["myQuota2", 2629746, 0, "\\N", "\\N", "\\N", "\\N", "\\N", "\\N", 1800]]) system_quotas_usage( - [["myQuota", "default", 1, 31556952, 0, 1000, 0, "\N", 0, "\N", 0, "\N", 0, 1000, 0, "\N", "\N"]]) + [["myQuota", "default", 1, 31556952, 0, 1000, 0, "\\N", 0, "\\N", 0, "\\N", 0, 1000, 0, "\\N", "\\N"]]) # Drop quota. copy_quota_xml('normal_limits.xml') check_system_quotas([["myQuota", "e651da9c-a748-8703-061a-7e5e5096dae7", "users.xml", "['user_name']", "[31556952]", 0, "['default']", "[]"]]) - system_quota_limits([["myQuota", 31556952, 0, 1000, "\N", "\N", "\N", 1000, "\N", "\N"]]) + system_quota_limits([["myQuota", 31556952, 0, 1000, "\\N", "\\N", "\\N", 1000, "\\N", "\\N"]]) system_quotas_usage( - [["myQuota", "default", 1, 31556952, 0, 1000, 0, "\N", 0, "\N", 0, "\N", 0, 1000, 0, "\N", "\N"]]) + [["myQuota", "default", 1, 31556952, 0, 1000, 0, "\\N", 0, "\\N", 0, "\\N", 0, 1000, 0, "\\N", "\\N"]]) # Drop all quotas. copy_quota_xml('no_quotas.xml') @@ -238,15 +238,15 @@ def test_add_remove_quota(): copy_quota_xml('normal_limits.xml') check_system_quotas([["myQuota", "e651da9c-a748-8703-061a-7e5e5096dae7", "users.xml", "['user_name']", "[31556952]", 0, "['default']", "[]"]]) - system_quota_limits([["myQuota", 31556952, 0, 1000, "\N", "\N", "\N", 1000, "\N", "\N"]]) + system_quota_limits([["myQuota", 31556952, 0, 1000, "\\N", "\\N", "\\N", 1000, "\\N", "\\N"]]) system_quotas_usage( - [["myQuota", "default", 1, 31556952, 0, 1000, 0, "\N", 0, "\N", 0, "\N", 0, 1000, 0, "\N", "\N"]]) + [["myQuota", "default", 1, 31556952, 0, 1000, 0, "\\N", 0, "\\N", 0, "\\N", 0, 1000, 0, "\\N", "\\N"]]) def test_reload_users_xml_by_timer(): check_system_quotas([["myQuota", "e651da9c-a748-8703-061a-7e5e5096dae7", "users.xml", "['user_name']", "[31556952]", 0, "['default']", "[]"]]) - system_quota_limits([["myQuota", 31556952, 0, 1000, "\N", "\N", "\N", 1000, "\N", "\N"]]) + system_quota_limits([["myQuota", 31556952, 0, 1000, "\\N", "\\N", "\\N", 1000, "\\N", "\\N"]]) time.sleep(1) # The modification time of the 'quota.xml' file should be different, # because config files are reload by timer only when the modification time is changed. @@ -255,7 +255,7 @@ def test_reload_users_xml_by_timer(): ["myQuota", "e651da9c-a748-8703-061a-7e5e5096dae7", "users.xml", ['user_name'], "[31556952]", 0, "['default']", "[]"]]) assert_eq_with_retry(instance, "SELECT * FROM system.quota_limits", - [["myQuota", 31556952, 0, 1, 1, 1, "\N", 1, "\N", "\N"]]) + [["myQuota", 31556952, 0, 1, 1, 1, "\\N", 1, "\\N", "\\N"]]) def test_dcl_introspection(): diff --git a/tests/integration/test_random_inserts/test.py b/tests/integration/test_random_inserts/test.py index f43581b6482..a06649dba52 100644 --- a/tests/integration/test_random_inserts/test.py +++ b/tests/integration/test_random_inserts/test.py @@ -55,7 +55,7 @@ def test_random_inserts(started_cluster): cmd = ['/bin/bash', bash_script, node.ip_address, str(min_timestamp), str(max_timestamp), str(cluster.get_client_cmd())] inserters.append(CommandRequest(cmd, timeout=DURATION_SECONDS * 2, stdin='')) - print node.name, node.ip_address + print(node.name, node.ip_address) for inserter in inserters: inserter.get_answer() @@ -105,8 +105,8 @@ class Runner: self.total_inserted += 2 * x + 1 self.mtx.release() - except Exception, e: - print 'Exception:', e + except Exception as e: + print('Exception:', e) x += 2 self.stop_ev.wait(0.1 + random.random() / 10) diff --git a/tests/integration/test_recompression_ttl/test.py b/tests/integration/test_recompression_ttl/test.py index 9a96151d04a..e74ae928b51 100644 --- a/tests/integration/test_recompression_ttl/test.py +++ b/tests/integration/test_recompression_ttl/test.py @@ -16,7 +16,7 @@ def started_cluster(): yield cluster except Exception as ex: - print ex + print(ex) finally: cluster.shutdown() diff --git a/tests/integration/test_recovery_replica/test.py b/tests/integration/test_recovery_replica/test.py index 74e773cfb83..9320a176f1e 100644 --- a/tests/integration/test_recovery_replica/test.py +++ b/tests/integration/test_recovery_replica/test.py @@ -31,7 +31,7 @@ def start_cluster(): yield cluster except Exception as ex: - print ex + print(ex) finally: cluster.shutdown() diff --git a/tests/integration/test_reload_max_table_size_to_drop/test.py b/tests/integration/test_reload_max_table_size_to_drop/test.py index d6bdcc83945..5f2083d742e 100644 --- a/tests/integration/test_reload_max_table_size_to_drop/test.py +++ b/tests/integration/test_reload_max_table_size_to_drop/test.py @@ -34,8 +34,7 @@ def test_reload_max_table_size_to_drop(start_cluster): config = open(CONFIG_PATH, 'r') config_lines = config.readlines() config.close() - config_lines = map(lambda line: line.replace("1", "1000000"), - config_lines) + config_lines = [line.replace("1", "1000000") for line in config_lines] config = open(CONFIG_PATH, 'w') config.writelines(config_lines) config.close() diff --git a/tests/integration/test_reload_zookeeper/test.py b/tests/integration/test_reload_zookeeper/test.py index 66df5a1a126..35958c95417 100644 --- a/tests/integration/test_reload_zookeeper/test.py +++ b/tests/integration/test_reload_zookeeper/test.py @@ -65,7 +65,7 @@ def test_reload_zookeeper(start_cluster): print("All instances of ZooKeeper started") return except Exception as ex: - print("Can't connect to ZooKeeper " + str(ex)) + print(("Can't connect to ZooKeeper " + str(ex))) time.sleep(0.5) node.query("INSERT INTO test_table(date, id) select today(), number FROM numbers(1000)") diff --git a/tests/integration/test_reloading_storage_configuration/test.py b/tests/integration/test_reloading_storage_configuration/test.py index 0c3139c7fdd..edcba4e8a60 100644 --- a/tests/integration/test_reloading_storage_configuration/test.py +++ b/tests/integration/test_reloading_storage_configuration/test.py @@ -82,7 +82,7 @@ def add_policy(node, name, volumes): root = tree.getroot() new_policy = ET.Element(name) new_volumes = ET.Element("volumes") - for volume, disks in volumes.items(): + for volume, disks in list(volumes.items()): new_volume = ET.Element(volume) for disk in disks: new_disk = ET.Element("disk") diff --git a/tests/integration/test_rename_column/test.py b/tests/integration/test_rename_column/test.py index b7ab8a75ba5..0e5cc9f5d9d 100644 --- a/tests/integration/test_rename_column/test.py +++ b/tests/integration/test_rename_column/test.py @@ -1,4 +1,4 @@ -from __future__ import print_function + import random import time diff --git a/tests/integration/test_replicated_merge_tree_s3/test.py b/tests/integration/test_replicated_merge_tree_s3/test.py index 4d19793d0b2..3b3540ef1b8 100644 --- a/tests/integration/test_replicated_merge_tree_s3/test.py +++ b/tests/integration/test_replicated_merge_tree_s3/test.py @@ -63,13 +63,13 @@ def create_table(cluster, additional_settings=None): create_table_statement += "," create_table_statement += additional_settings - cluster.instances.values()[0].query(create_table_statement) + list(cluster.instances.values())[0].query(create_table_statement) @pytest.fixture(autouse=True) def drop_table(cluster): yield - for node in cluster.instances.values(): + for node in list(cluster.instances.values()): node.query("DROP TABLE IF EXISTS s3_test") minio = cluster.minio_client diff --git a/tests/integration/test_replicated_mutations/test.py b/tests/integration/test_replicated_mutations/test.py index cf3cef3a9e6..40a2b15ffaf 100644 --- a/tests/integration/test_replicated_mutations/test.py +++ b/tests/integration/test_replicated_mutations/test.py @@ -90,7 +90,7 @@ class Runner: i += 1 try: - print 'thread {}: insert for {}: {}'.format(thread_num, date_str, ','.join(str(x) for x in xs)) + print('thread {}: insert for {}: {}'.format(thread_num, date_str, ','.join(str(x) for x in xs))) random.choice(self.nodes).query("INSERT INTO test_mutations FORMAT TSV", payload) with self.mtx: @@ -99,8 +99,8 @@ class Runner: self.total_inserted_xs += sum(xs) self.total_inserted_rows += len(xs) - except Exception, e: - print 'Exception while inserting,', e + except Exception as e: + print('Exception while inserting,', e) self.exceptions.append(e) finally: with self.mtx: @@ -128,7 +128,7 @@ class Runner: continue try: - print 'thread {}: delete {} * {}'.format(thread_num, to_delete_count, x) + print('thread {}: delete {} * {}'.format(thread_num, to_delete_count, x)) random.choice(self.nodes).query("ALTER TABLE test_mutations DELETE WHERE x = {}".format(x)) with self.mtx: @@ -137,8 +137,8 @@ class Runner: self.total_deleted_xs += to_delete_count * x self.total_deleted_rows += to_delete_count - except Exception, e: - print 'Exception while deleting,', e + except Exception as e: + print('Exception while deleting,', e) finally: with self.mtx: self.currently_deleting_xs.remove(x) @@ -186,10 +186,10 @@ def test_mutations(started_cluster): all_done = wait_for_mutations(nodes, runner.total_mutations) - print "Total mutations: ", runner.total_mutations + print("Total mutations: ", runner.total_mutations) for node in nodes: - print node.query( - "SELECT mutation_id, command, parts_to_do, is_done FROM system.mutations WHERE table = 'test_mutations' FORMAT TSVWithNames") + print(node.query( + "SELECT mutation_id, command, parts_to_do, is_done FROM system.mutations WHERE table = 'test_mutations' FORMAT TSVWithNames")) assert all_done expected_sum = runner.total_inserted_xs - runner.total_deleted_xs @@ -233,10 +233,10 @@ def test_mutations_dont_prevent_merges(started_cluster, nodes): t.join() for node in nodes: - print node.query( - "SELECT mutation_id, command, parts_to_do, is_done FROM system.mutations WHERE table = 'test_mutations' FORMAT TSVWithNames") - print node.query( - "SELECT partition, count(name), sum(active), sum(active*rows) FROM system.parts WHERE table ='test_mutations' GROUP BY partition FORMAT TSVWithNames") + print(node.query( + "SELECT mutation_id, command, parts_to_do, is_done FROM system.mutations WHERE table = 'test_mutations' FORMAT TSVWithNames")) + print(node.query( + "SELECT partition, count(name), sum(active), sum(active*rows) FROM system.parts WHERE table ='test_mutations' GROUP BY partition FORMAT TSVWithNames")) assert all_done assert all([str(e).find("Too many parts") < 0 for e in runner.exceptions]) diff --git a/tests/integration/test_replicated_parse_zk_metadata/test.py b/tests/integration/test_replicated_parse_zk_metadata/test.py index 4bdd77393b3..d8b6685ddcd 100644 --- a/tests/integration/test_replicated_parse_zk_metadata/test.py +++ b/tests/integration/test_replicated_parse_zk_metadata/test.py @@ -33,7 +33,7 @@ def test_replicated_engine_parse_metadata_on_attach(): # and successfully accepted by the server. # # This metadata was obtain from the server without #11325 - zk.set('/ch/tables/default/data/replicas/node/metadata', """ + zk.set('/ch/tables/default/data/replicas/node/metadata', b""" metadata format version: 1 date column: sampling expression: diff --git a/tests/integration/test_replication_without_zookeeper/test.py b/tests/integration/test_replication_without_zookeeper/test.py index 90f060d991a..26347b47d36 100644 --- a/tests/integration/test_replication_without_zookeeper/test.py +++ b/tests/integration/test_replication_without_zookeeper/test.py @@ -23,7 +23,7 @@ def start_cluster(): yield cluster except Exception as ex: - print ex + print(ex) finally: cluster.shutdown() diff --git a/tests/integration/test_role/test.py b/tests/integration/test_role/test.py index 31fddd3df8c..ccd3477ed72 100644 --- a/tests/integration/test_role/test.py +++ b/tests/integration/test_role/test.py @@ -184,15 +184,15 @@ def test_introspection(): assert instance.query( "SELECT * from system.grants WHERE user_name IN ('A', 'B') OR role_name IN ('R1', 'R2') ORDER BY user_name, role_name, access_type, grant_option") == \ - TSV([["A", "\N", "SELECT", "test", "table", "\N", 0, 0], - ["B", "\N", "CREATE", "\N", "\N", "\N", 0, 1], - ["\N", "R2", "SELECT", "test", "table", "\N", 0, 0], - ["\N", "R2", "SELECT", "test", "table", "x", 1, 0]]) + TSV([["A", "\\N", "SELECT", "test", "table", "\\N", 0, 0], + ["B", "\\N", "CREATE", "\\N", "\\N", "\\N", 0, 1], + ["\\N", "R2", "SELECT", "test", "table", "\\N", 0, 0], + ["\\N", "R2", "SELECT", "test", "table", "x", 1, 0]]) assert instance.query( "SELECT * from system.role_grants WHERE user_name IN ('A', 'B') OR role_name IN ('R1', 'R2') ORDER BY user_name, role_name, granted_role_name") == \ - TSV([["A", "\N", "R1", 1, 0], - ["B", "\N", "R2", 1, 1]]) + TSV([["A", "\\N", "R1", 1, 0], + ["B", "\\N", "R2", 1, 1]]) assert instance.query("SELECT * from system.current_roles ORDER BY role_name", user='A') == TSV([["R1", 0, 1]]) assert instance.query("SELECT * from system.current_roles ORDER BY role_name", user='B') == TSV([["R2", 1, 1]]) diff --git a/tests/integration/test_send_crash_reports/fake_sentry_server.py b/tests/integration/test_send_crash_reports/fake_sentry_server.py index 49463bdb133..fa40f642e41 100644 --- a/tests/integration/test_send_crash_reports/fake_sentry_server.py +++ b/tests/integration/test_send_crash_reports/fake_sentry_server.py @@ -1,17 +1,18 @@ -import BaseHTTPServer +import http.server RESULT_PATH = '/result.txt' -class SentryHandler(BaseHTTPServer.BaseHTTPRequestHandler): +class SentryHandler(http.server.BaseHTTPRequestHandler): def do_POST(self): post_data = self.__read_and_decode_post_data() with open(RESULT_PATH, 'w') as f: + content_length = self.headers.get("content-length") if self.headers.get("content-type") != "application/x-sentry-envelope": f.write("INCORRECT_CONTENT_TYPE") - elif self.headers.get("content-length") < 3000: - f.write("INCORRECT_CONTENT_LENGTH") - elif '"http://6f33034cfe684dd7a3ab9875e57b1c8d@localhost:9500/5226277"' not in post_data: + elif int(content_length) < 200: + f.write("INCORRECT_CONTENT_LENGTH:" + content_length + '\n' + post_data.decode()) + elif b'"http://6f33034cfe684dd7a3ab9875e57b1c8d@localhost:9500/5226277"' not in post_data: f.write('INCORRECT_POST_DATA') else: f.write("OK") @@ -19,7 +20,7 @@ class SentryHandler(BaseHTTPServer.BaseHTTPRequestHandler): def __read_and_decode_post_data(self): transfer_encoding = self.headers.get("transfer-Encoding") - decoded = "" + decoded = b"" if transfer_encoding == "chunked": while True: s = self.rfile.readline() @@ -37,7 +38,7 @@ class SentryHandler(BaseHTTPServer.BaseHTTPRequestHandler): if __name__ == "__main__": with open(RESULT_PATH, 'w') as f: f.write("INITIAL_STATE") - httpd = BaseHTTPServer.HTTPServer(("localhost", 9500,), SentryHandler) + httpd = http.server.HTTPServer(("localhost", 9500,), SentryHandler) try: httpd.serve_forever() finally: diff --git a/tests/integration/test_send_crash_reports/test.py b/tests/integration/test_send_crash_reports/test.py index 4c832d9d67c..a3c35ca1537 100644 --- a/tests/integration/test_send_crash_reports/test.py +++ b/tests/integration/test_send_crash_reports/test.py @@ -5,7 +5,7 @@ import helpers.cluster import helpers.test_tools import pytest -import fake_sentry_server +from . import fake_sentry_server SCRIPT_DIR = os.path.dirname(os.path.realpath(__file__)) @@ -25,7 +25,7 @@ def started_node(): def test_send_segfault(started_node, ): started_node.copy_file_to_container(os.path.join(SCRIPT_DIR, "fake_sentry_server.py"), "/fake_sentry_server.py") - started_node.exec_in_container(["bash", "-c", "python2 /fake_sentry_server.py"], detach=True, user="root") + started_node.exec_in_container(["bash", "-c", "python3 /fake_sentry_server.py > /fake_sentry_server.log 2>&1"], detach=True, user="root") time.sleep(0.5) started_node.exec_in_container(["bash", "-c", "pkill -11 clickhouse"], user="root") diff --git a/tests/integration/test_settings_constraints/test.py b/tests/integration/test_settings_constraints/test.py index 90e639685f0..18c80d9c1da 100644 --- a/tests/integration/test_settings_constraints/test.py +++ b/tests/integration/test_settings_constraints/test.py @@ -116,7 +116,7 @@ def assert_query_settings(instance, query, settings, result=None, exception=None # session level settings queries = "" - for k, v in settings.items(): + for k, v in list(settings.items()): queries += "SET {}={};\n".format(k, v) queries += query diff --git a/tests/integration/test_settings_profile/test.py b/tests/integration/test_settings_profile/test.py index f7901dc1fe6..5345ef9a474 100644 --- a/tests/integration/test_settings_profile/test.py +++ b/tests/integration/test_settings_profile/test.py @@ -59,7 +59,7 @@ def test_smoke(): "SET max_memory_usage = 120000000", user="robin") assert system_settings_profile("xyz") == [["xyz", "local directory", 1, 0, "['robin']", "[]"]] assert system_settings_profile_elements(profile_name="xyz") == [ - ["xyz", "\N", "\N", 0, "max_memory_usage", 100000001, 90000000, 110000000, "\N", "\N"]] + ["xyz", "\\N", "\\N", 0, "max_memory_usage", 100000001, 90000000, 110000000, "\\N", "\\N"]] instance.query("ALTER SETTINGS PROFILE xyz TO NONE") assert instance.query( @@ -81,7 +81,7 @@ def test_smoke(): assert "Setting max_memory_usage shouldn't be greater than 110000000" in instance.query_and_get_error( "SET max_memory_usage = 120000000", user="robin") assert system_settings_profile_elements(user_name="robin") == [ - ["\N", "robin", "\N", 0, "\N", "\N", "\N", "\N", "\N", "xyz"]] + ["\\N", "robin", "\\N", 0, "\\N", "\\N", "\\N", "\\N", "\\N", "xyz"]] instance.query("ALTER USER robin SETTINGS NONE") assert instance.query("SHOW CREATE USER robin") == "CREATE USER robin\n" @@ -108,10 +108,10 @@ def test_settings_from_granted_role(): "SET max_memory_usage = 120000000", user="robin") assert system_settings_profile("xyz") == [["xyz", "local directory", 2, 0, "[]", "[]"]] assert system_settings_profile_elements(profile_name="xyz") == [ - ["xyz", "\N", "\N", 0, "max_memory_usage", 100000001, "\N", 110000000, "\N", "\N"], - ["xyz", "\N", "\N", 1, "max_ast_depth", 2000, "\N", "\N", "\N", "\N"]] + ["xyz", "\\N", "\\N", 0, "max_memory_usage", 100000001, "\\N", 110000000, "\\N", "\\N"], + ["xyz", "\\N", "\\N", 1, "max_ast_depth", 2000, "\\N", "\\N", "\\N", "\\N"]] assert system_settings_profile_elements(role_name="worker") == [ - ["\N", "\N", "worker", 0, "\N", "\N", "\N", "\N", "\N", "xyz"]] + ["\\N", "\\N", "worker", 0, "\\N", "\\N", "\\N", "\\N", "\\N", "xyz"]] instance.query("REVOKE worker FROM robin") assert instance.query("SELECT value FROM system.settings WHERE name = 'max_memory_usage'", @@ -159,10 +159,10 @@ def test_inheritance(): assert system_settings_profile("xyz") == [["xyz", "local directory", 1, 0, "[]", "[]"]] assert system_settings_profile_elements(profile_name="xyz") == [ - ["xyz", "\N", "\N", 0, "max_memory_usage", 100000002, "\N", "\N", 1, "\N"]] + ["xyz", "\\N", "\\N", 0, "max_memory_usage", 100000002, "\\N", "\\N", 1, "\\N"]] assert system_settings_profile("alpha") == [["alpha", "local directory", 1, 0, "['robin']", "[]"]] assert system_settings_profile_elements(profile_name="alpha") == [ - ["alpha", "\N", "\N", 0, "\N", "\N", "\N", "\N", "\N", "xyz"]] + ["alpha", "\\N", "\\N", 0, "\\N", "\\N", "\\N", "\\N", "\\N", "xyz"]] assert system_settings_profile_elements(user_name="robin") == [] diff --git a/tests/integration/test_storage_hdfs/test.py b/tests/integration/test_storage_hdfs/test.py index ed2a4e0140d..996df28ac81 100644 --- a/tests/integration/test_storage_hdfs/test.py +++ b/tests/integration/test_storage_hdfs/test.py @@ -58,21 +58,21 @@ def test_read_write_storage_with_globs(started_cluster): node1.query("insert into HDFSStorageWithEnum values (1, 'NEW', 4.2)") assert False, "Exception have to be thrown" except Exception as ex: - print ex + print(ex) assert "in readonly mode" in str(ex) try: node1.query("insert into HDFSStorageWithQuestionMark values (1, 'NEW', 4.2)") assert False, "Exception have to be thrown" except Exception as ex: - print ex + print(ex) assert "in readonly mode" in str(ex) try: node1.query("insert into HDFSStorageWithAsterisk values (1, 'NEW', 4.2)") assert False, "Exception have to be thrown" except Exception as ex: - print ex + print(ex) assert "in readonly mode" in str(ex) @@ -104,20 +104,20 @@ def test_bad_hdfs_uri(started_cluster): node1.query( "create table BadStorage1 (id UInt32, name String, weight Float64) ENGINE = HDFS('hads:hgsdfs100500:9000/other_storage', 'TSV')") except Exception as ex: - print ex + print(ex) assert "Illegal HDFS URI" in str(ex) try: node1.query( "create table BadStorage2 (id UInt32, name String, weight Float64) ENGINE = HDFS('hdfs://hdfs100500:9000/other_storage', 'TSV')") except Exception as ex: - print ex + print(ex) assert "Unable to create builder to connect to HDFS" in str(ex) try: node1.query( "create table BadStorage3 (id UInt32, name String, weight Float64) ENGINE = HDFS('hdfs://hdfs1:9000/<>', 'TSV')") except Exception as ex: - print ex + print(ex) assert "Unable to open HDFS file" in str(ex) diff --git a/tests/integration/test_storage_kafka/test.py b/tests/integration/test_storage_kafka/test.py index 90422bf98e9..ae7eed55532 100644 --- a/tests/integration/test_storage_kafka/test.py +++ b/tests/integration/test_storage_kafka/test.py @@ -7,10 +7,11 @@ import threading import time import avro.schema +from confluent_kafka.avro.cached_schema_registry_client import CachedSchemaRegistryClient +from confluent_kafka.avro.serializer.message_serializer import MessageSerializer + import kafka.errors import pytest -from confluent.schemaregistry.client import CachedSchemaRegistryClient -from confluent.schemaregistry.serializers.MessageSerializer import MessageSerializer from google.protobuf.internal.encoder import _VarintBytes from helpers.client import QueryRuntimeException from helpers.cluster import ClickHouseCluster @@ -28,7 +29,7 @@ libprotoc 3.0.0 # to create kafka_pb2.py protoc --python_out=. kafka.proto """ -import kafka_pb2 +from . import kafka_pb2 # TODO: add test for run-time offset update in CH, if we manually update it on Kafka side. # TODO: add test for SELECT LIMIT is working. @@ -69,40 +70,38 @@ def wait_kafka_is_available(max_retries=50): print("Waiting for Kafka to start up") time.sleep(1) +def producer_serializer(x): + return x.encode() if isinstance(x, str) else x def kafka_produce(topic, messages, timestamp=None): - producer = KafkaProducer(bootstrap_servers="localhost:9092") + producer = KafkaProducer(bootstrap_servers="localhost:9092", value_serializer=producer_serializer) for message in messages: producer.send(topic=topic, value=message, timestamp_ms=timestamp) producer.flush() - -# print ("Produced {} messages for topic {}".format(len(messages), topic)) - - def kafka_consume(topic): consumer = KafkaConsumer(bootstrap_servers="localhost:9092", auto_offset_reset="earliest") consumer.subscribe(topics=(topic)) - for toppar, messages in consumer.poll(5000).items(): + for toppar, messages in list(consumer.poll(5000).items()): if toppar.topic == topic: for message in messages: - yield message.value + yield message.value.decode() consumer.unsubscribe() consumer.close() def kafka_produce_protobuf_messages(topic, start_index, num_messages): - data = '' + data = b'' for i in range(start_index, start_index + num_messages): msg = kafka_pb2.KeyValuePair() msg.key = i msg.value = str(i) serialized_msg = msg.SerializeToString() data = data + _VarintBytes(len(serialized_msg)) + serialized_msg - producer = KafkaProducer(bootstrap_servers="localhost:9092") + producer = KafkaProducer(bootstrap_servers="localhost:9092", value_serializer=producer_serializer) producer.send(topic=topic, value=data) producer.flush() - print("Produced {} messages for topic {}".format(num_messages, topic)) + print(("Produced {} messages for topic {}".format(num_messages, topic))) def avro_confluent_message(schema_registry_client, value): @@ -285,9 +284,9 @@ def test_kafka_formats(kafka_cluster): # clickhouse-client ... | xxd -ps -c 200 | tr -d '\n' | sed 's/\(..\)/\\x\1/g' 'Native': { 'data_sample': [ - '\x05\x01\x02\x69\x64\x05\x49\x6e\x74\x36\x34\x00\x00\x00\x00\x00\x00\x00\x00\x07\x62\x6c\x6f\x63\x6b\x4e\x6f\x06\x55\x49\x6e\x74\x31\x36\x00\x00\x04\x76\x61\x6c\x31\x06\x53\x74\x72\x69\x6e\x67\x02\x41\x4d\x04\x76\x61\x6c\x32\x07\x46\x6c\x6f\x61\x74\x33\x32\x00\x00\x00\x3f\x04\x76\x61\x6c\x33\x05\x55\x49\x6e\x74\x38\x01', - '\x05\x0f\x02\x69\x64\x05\x49\x6e\x74\x36\x34\x01\x00\x00\x00\x00\x00\x00\x00\x02\x00\x00\x00\x00\x00\x00\x00\x03\x00\x00\x00\x00\x00\x00\x00\x04\x00\x00\x00\x00\x00\x00\x00\x05\x00\x00\x00\x00\x00\x00\x00\x06\x00\x00\x00\x00\x00\x00\x00\x07\x00\x00\x00\x00\x00\x00\x00\x08\x00\x00\x00\x00\x00\x00\x00\x09\x00\x00\x00\x00\x00\x00\x00\x0a\x00\x00\x00\x00\x00\x00\x00\x0b\x00\x00\x00\x00\x00\x00\x00\x0c\x00\x00\x00\x00\x00\x00\x00\x0d\x00\x00\x00\x00\x00\x00\x00\x0e\x00\x00\x00\x00\x00\x00\x00\x0f\x00\x00\x00\x00\x00\x00\x00\x07\x62\x6c\x6f\x63\x6b\x4e\x6f\x06\x55\x49\x6e\x74\x31\x36\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x04\x76\x61\x6c\x31\x06\x53\x74\x72\x69\x6e\x67\x02\x41\x4d\x02\x41\x4d\x02\x41\x4d\x02\x41\x4d\x02\x41\x4d\x02\x41\x4d\x02\x41\x4d\x02\x41\x4d\x02\x41\x4d\x02\x41\x4d\x02\x41\x4d\x02\x41\x4d\x02\x41\x4d\x02\x41\x4d\x02\x41\x4d\x04\x76\x61\x6c\x32\x07\x46\x6c\x6f\x61\x74\x33\x32\x00\x00\x00\x3f\x00\x00\x00\x3f\x00\x00\x00\x3f\x00\x00\x00\x3f\x00\x00\x00\x3f\x00\x00\x00\x3f\x00\x00\x00\x3f\x00\x00\x00\x3f\x00\x00\x00\x3f\x00\x00\x00\x3f\x00\x00\x00\x3f\x00\x00\x00\x3f\x00\x00\x00\x3f\x00\x00\x00\x3f\x00\x00\x00\x3f\x04\x76\x61\x6c\x33\x05\x55\x49\x6e\x74\x38\x01\x01\x01\x01\x01\x01\x01\x01\x01\x01\x01\x01\x01\x01\x01', - '\x05\x01\x02\x69\x64\x05\x49\x6e\x74\x36\x34\x00\x00\x00\x00\x00\x00\x00\x00\x07\x62\x6c\x6f\x63\x6b\x4e\x6f\x06\x55\x49\x6e\x74\x31\x36\x00\x00\x04\x76\x61\x6c\x31\x06\x53\x74\x72\x69\x6e\x67\x02\x41\x4d\x04\x76\x61\x6c\x32\x07\x46\x6c\x6f\x61\x74\x33\x32\x00\x00\x00\x3f\x04\x76\x61\x6c\x33\x05\x55\x49\x6e\x74\x38\x01', + b'\x05\x01\x02\x69\x64\x05\x49\x6e\x74\x36\x34\x00\x00\x00\x00\x00\x00\x00\x00\x07\x62\x6c\x6f\x63\x6b\x4e\x6f\x06\x55\x49\x6e\x74\x31\x36\x00\x00\x04\x76\x61\x6c\x31\x06\x53\x74\x72\x69\x6e\x67\x02\x41\x4d\x04\x76\x61\x6c\x32\x07\x46\x6c\x6f\x61\x74\x33\x32\x00\x00\x00\x3f\x04\x76\x61\x6c\x33\x05\x55\x49\x6e\x74\x38\x01', + b'\x05\x0f\x02\x69\x64\x05\x49\x6e\x74\x36\x34\x01\x00\x00\x00\x00\x00\x00\x00\x02\x00\x00\x00\x00\x00\x00\x00\x03\x00\x00\x00\x00\x00\x00\x00\x04\x00\x00\x00\x00\x00\x00\x00\x05\x00\x00\x00\x00\x00\x00\x00\x06\x00\x00\x00\x00\x00\x00\x00\x07\x00\x00\x00\x00\x00\x00\x00\x08\x00\x00\x00\x00\x00\x00\x00\x09\x00\x00\x00\x00\x00\x00\x00\x0a\x00\x00\x00\x00\x00\x00\x00\x0b\x00\x00\x00\x00\x00\x00\x00\x0c\x00\x00\x00\x00\x00\x00\x00\x0d\x00\x00\x00\x00\x00\x00\x00\x0e\x00\x00\x00\x00\x00\x00\x00\x0f\x00\x00\x00\x00\x00\x00\x00\x07\x62\x6c\x6f\x63\x6b\x4e\x6f\x06\x55\x49\x6e\x74\x31\x36\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x04\x76\x61\x6c\x31\x06\x53\x74\x72\x69\x6e\x67\x02\x41\x4d\x02\x41\x4d\x02\x41\x4d\x02\x41\x4d\x02\x41\x4d\x02\x41\x4d\x02\x41\x4d\x02\x41\x4d\x02\x41\x4d\x02\x41\x4d\x02\x41\x4d\x02\x41\x4d\x02\x41\x4d\x02\x41\x4d\x02\x41\x4d\x04\x76\x61\x6c\x32\x07\x46\x6c\x6f\x61\x74\x33\x32\x00\x00\x00\x3f\x00\x00\x00\x3f\x00\x00\x00\x3f\x00\x00\x00\x3f\x00\x00\x00\x3f\x00\x00\x00\x3f\x00\x00\x00\x3f\x00\x00\x00\x3f\x00\x00\x00\x3f\x00\x00\x00\x3f\x00\x00\x00\x3f\x00\x00\x00\x3f\x00\x00\x00\x3f\x00\x00\x00\x3f\x00\x00\x00\x3f\x04\x76\x61\x6c\x33\x05\x55\x49\x6e\x74\x38\x01\x01\x01\x01\x01\x01\x01\x01\x01\x01\x01\x01\x01\x01\x01', + b'\x05\x01\x02\x69\x64\x05\x49\x6e\x74\x36\x34\x00\x00\x00\x00\x00\x00\x00\x00\x07\x62\x6c\x6f\x63\x6b\x4e\x6f\x06\x55\x49\x6e\x74\x31\x36\x00\x00\x04\x76\x61\x6c\x31\x06\x53\x74\x72\x69\x6e\x67\x02\x41\x4d\x04\x76\x61\x6c\x32\x07\x46\x6c\x6f\x61\x74\x33\x32\x00\x00\x00\x3f\x04\x76\x61\x6c\x33\x05\x55\x49\x6e\x74\x38\x01', # '' # On empty message exception happens: DB::Exception: Attempt to read after eof # /src/IO/VarInt.h:122: DB::throwReadAfterEOF() @ 0x15c34487 in /usr/bin/clickhouse @@ -301,9 +300,9 @@ def test_kafka_formats(kafka_cluster): }, 'MsgPack': { 'data_sample': [ - '\x00\x00\xa2\x41\x4d\xca\x3f\x00\x00\x00\x01', - '\x01\x00\xa2\x41\x4d\xca\x3f\x00\x00\x00\x01\x02\x00\xa2\x41\x4d\xca\x3f\x00\x00\x00\x01\x03\x00\xa2\x41\x4d\xca\x3f\x00\x00\x00\x01\x04\x00\xa2\x41\x4d\xca\x3f\x00\x00\x00\x01\x05\x00\xa2\x41\x4d\xca\x3f\x00\x00\x00\x01\x06\x00\xa2\x41\x4d\xca\x3f\x00\x00\x00\x01\x07\x00\xa2\x41\x4d\xca\x3f\x00\x00\x00\x01\x08\x00\xa2\x41\x4d\xca\x3f\x00\x00\x00\x01\x09\x00\xa2\x41\x4d\xca\x3f\x00\x00\x00\x01\x0a\x00\xa2\x41\x4d\xca\x3f\x00\x00\x00\x01\x0b\x00\xa2\x41\x4d\xca\x3f\x00\x00\x00\x01\x0c\x00\xa2\x41\x4d\xca\x3f\x00\x00\x00\x01\x0d\x00\xa2\x41\x4d\xca\x3f\x00\x00\x00\x01\x0e\x00\xa2\x41\x4d\xca\x3f\x00\x00\x00\x01\x0f\x00\xa2\x41\x4d\xca\x3f\x00\x00\x00\x01', - '\x00\x00\xa2\x41\x4d\xca\x3f\x00\x00\x00\x01', + b'\x00\x00\xa2\x41\x4d\xca\x3f\x00\x00\x00\x01', + b'\x01\x00\xa2\x41\x4d\xca\x3f\x00\x00\x00\x01\x02\x00\xa2\x41\x4d\xca\x3f\x00\x00\x00\x01\x03\x00\xa2\x41\x4d\xca\x3f\x00\x00\x00\x01\x04\x00\xa2\x41\x4d\xca\x3f\x00\x00\x00\x01\x05\x00\xa2\x41\x4d\xca\x3f\x00\x00\x00\x01\x06\x00\xa2\x41\x4d\xca\x3f\x00\x00\x00\x01\x07\x00\xa2\x41\x4d\xca\x3f\x00\x00\x00\x01\x08\x00\xa2\x41\x4d\xca\x3f\x00\x00\x00\x01\x09\x00\xa2\x41\x4d\xca\x3f\x00\x00\x00\x01\x0a\x00\xa2\x41\x4d\xca\x3f\x00\x00\x00\x01\x0b\x00\xa2\x41\x4d\xca\x3f\x00\x00\x00\x01\x0c\x00\xa2\x41\x4d\xca\x3f\x00\x00\x00\x01\x0d\x00\xa2\x41\x4d\xca\x3f\x00\x00\x00\x01\x0e\x00\xa2\x41\x4d\xca\x3f\x00\x00\x00\x01\x0f\x00\xa2\x41\x4d\xca\x3f\x00\x00\x00\x01', + b'\x00\x00\xa2\x41\x4d\xca\x3f\x00\x00\x00\x01', # '' # On empty message exception happens: Unexpected end of file while parsing msgpack object.: (at row 1) # coming from Processors/Formats/Impl/MsgPackRowInputFormat.cpp:170 @@ -311,9 +310,9 @@ def test_kafka_formats(kafka_cluster): }, 'RowBinary': { 'data_sample': [ - '\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x02\x41\x4d\x00\x00\x00\x3f\x01', - '\x01\x00\x00\x00\x00\x00\x00\x00\x00\x00\x02\x41\x4d\x00\x00\x00\x3f\x01\x02\x00\x00\x00\x00\x00\x00\x00\x00\x00\x02\x41\x4d\x00\x00\x00\x3f\x01\x03\x00\x00\x00\x00\x00\x00\x00\x00\x00\x02\x41\x4d\x00\x00\x00\x3f\x01\x04\x00\x00\x00\x00\x00\x00\x00\x00\x00\x02\x41\x4d\x00\x00\x00\x3f\x01\x05\x00\x00\x00\x00\x00\x00\x00\x00\x00\x02\x41\x4d\x00\x00\x00\x3f\x01\x06\x00\x00\x00\x00\x00\x00\x00\x00\x00\x02\x41\x4d\x00\x00\x00\x3f\x01\x07\x00\x00\x00\x00\x00\x00\x00\x00\x00\x02\x41\x4d\x00\x00\x00\x3f\x01\x08\x00\x00\x00\x00\x00\x00\x00\x00\x00\x02\x41\x4d\x00\x00\x00\x3f\x01\x09\x00\x00\x00\x00\x00\x00\x00\x00\x00\x02\x41\x4d\x00\x00\x00\x3f\x01\x0a\x00\x00\x00\x00\x00\x00\x00\x00\x00\x02\x41\x4d\x00\x00\x00\x3f\x01\x0b\x00\x00\x00\x00\x00\x00\x00\x00\x00\x02\x41\x4d\x00\x00\x00\x3f\x01\x0c\x00\x00\x00\x00\x00\x00\x00\x00\x00\x02\x41\x4d\x00\x00\x00\x3f\x01\x0d\x00\x00\x00\x00\x00\x00\x00\x00\x00\x02\x41\x4d\x00\x00\x00\x3f\x01\x0e\x00\x00\x00\x00\x00\x00\x00\x00\x00\x02\x41\x4d\x00\x00\x00\x3f\x01\x0f\x00\x00\x00\x00\x00\x00\x00\x00\x00\x02\x41\x4d\x00\x00\x00\x3f\x01', - '\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x02\x41\x4d\x00\x00\x00\x3f\x01', + b'\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x02\x41\x4d\x00\x00\x00\x3f\x01', + b'\x01\x00\x00\x00\x00\x00\x00\x00\x00\x00\x02\x41\x4d\x00\x00\x00\x3f\x01\x02\x00\x00\x00\x00\x00\x00\x00\x00\x00\x02\x41\x4d\x00\x00\x00\x3f\x01\x03\x00\x00\x00\x00\x00\x00\x00\x00\x00\x02\x41\x4d\x00\x00\x00\x3f\x01\x04\x00\x00\x00\x00\x00\x00\x00\x00\x00\x02\x41\x4d\x00\x00\x00\x3f\x01\x05\x00\x00\x00\x00\x00\x00\x00\x00\x00\x02\x41\x4d\x00\x00\x00\x3f\x01\x06\x00\x00\x00\x00\x00\x00\x00\x00\x00\x02\x41\x4d\x00\x00\x00\x3f\x01\x07\x00\x00\x00\x00\x00\x00\x00\x00\x00\x02\x41\x4d\x00\x00\x00\x3f\x01\x08\x00\x00\x00\x00\x00\x00\x00\x00\x00\x02\x41\x4d\x00\x00\x00\x3f\x01\x09\x00\x00\x00\x00\x00\x00\x00\x00\x00\x02\x41\x4d\x00\x00\x00\x3f\x01\x0a\x00\x00\x00\x00\x00\x00\x00\x00\x00\x02\x41\x4d\x00\x00\x00\x3f\x01\x0b\x00\x00\x00\x00\x00\x00\x00\x00\x00\x02\x41\x4d\x00\x00\x00\x3f\x01\x0c\x00\x00\x00\x00\x00\x00\x00\x00\x00\x02\x41\x4d\x00\x00\x00\x3f\x01\x0d\x00\x00\x00\x00\x00\x00\x00\x00\x00\x02\x41\x4d\x00\x00\x00\x3f\x01\x0e\x00\x00\x00\x00\x00\x00\x00\x00\x00\x02\x41\x4d\x00\x00\x00\x3f\x01\x0f\x00\x00\x00\x00\x00\x00\x00\x00\x00\x02\x41\x4d\x00\x00\x00\x3f\x01', + b'\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x02\x41\x4d\x00\x00\x00\x3f\x01', # '' # On empty message exception happens: DB::Exception: Cannot read all data. Bytes read: 0. Bytes expected: 8. # /src/IO/ReadBuffer.h:157: DB::ReadBuffer::readStrict(char*, unsigned long) @ 0x15c6894d in /usr/bin/clickhouse @@ -325,9 +324,9 @@ def test_kafka_formats(kafka_cluster): }, 'RowBinaryWithNamesAndTypes': { 'data_sample': [ - '\x05\x02\x69\x64\x07\x62\x6c\x6f\x63\x6b\x4e\x6f\x04\x76\x61\x6c\x31\x04\x76\x61\x6c\x32\x04\x76\x61\x6c\x33\x05\x49\x6e\x74\x36\x34\x06\x55\x49\x6e\x74\x31\x36\x06\x53\x74\x72\x69\x6e\x67\x07\x46\x6c\x6f\x61\x74\x33\x32\x05\x55\x49\x6e\x74\x38\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x02\x41\x4d\x00\x00\x00\x3f\x01', - '\x05\x02\x69\x64\x07\x62\x6c\x6f\x63\x6b\x4e\x6f\x04\x76\x61\x6c\x31\x04\x76\x61\x6c\x32\x04\x76\x61\x6c\x33\x05\x49\x6e\x74\x36\x34\x06\x55\x49\x6e\x74\x31\x36\x06\x53\x74\x72\x69\x6e\x67\x07\x46\x6c\x6f\x61\x74\x33\x32\x05\x55\x49\x6e\x74\x38\x01\x00\x00\x00\x00\x00\x00\x00\x00\x00\x02\x41\x4d\x00\x00\x00\x3f\x01\x02\x00\x00\x00\x00\x00\x00\x00\x00\x00\x02\x41\x4d\x00\x00\x00\x3f\x01\x03\x00\x00\x00\x00\x00\x00\x00\x00\x00\x02\x41\x4d\x00\x00\x00\x3f\x01\x04\x00\x00\x00\x00\x00\x00\x00\x00\x00\x02\x41\x4d\x00\x00\x00\x3f\x01\x05\x00\x00\x00\x00\x00\x00\x00\x00\x00\x02\x41\x4d\x00\x00\x00\x3f\x01\x06\x00\x00\x00\x00\x00\x00\x00\x00\x00\x02\x41\x4d\x00\x00\x00\x3f\x01\x07\x00\x00\x00\x00\x00\x00\x00\x00\x00\x02\x41\x4d\x00\x00\x00\x3f\x01\x08\x00\x00\x00\x00\x00\x00\x00\x00\x00\x02\x41\x4d\x00\x00\x00\x3f\x01\x09\x00\x00\x00\x00\x00\x00\x00\x00\x00\x02\x41\x4d\x00\x00\x00\x3f\x01\x0a\x00\x00\x00\x00\x00\x00\x00\x00\x00\x02\x41\x4d\x00\x00\x00\x3f\x01\x0b\x00\x00\x00\x00\x00\x00\x00\x00\x00\x02\x41\x4d\x00\x00\x00\x3f\x01\x0c\x00\x00\x00\x00\x00\x00\x00\x00\x00\x02\x41\x4d\x00\x00\x00\x3f\x01\x0d\x00\x00\x00\x00\x00\x00\x00\x00\x00\x02\x41\x4d\x00\x00\x00\x3f\x01\x0e\x00\x00\x00\x00\x00\x00\x00\x00\x00\x02\x41\x4d\x00\x00\x00\x3f\x01\x0f\x00\x00\x00\x00\x00\x00\x00\x00\x00\x02\x41\x4d\x00\x00\x00\x3f\x01', - '\x05\x02\x69\x64\x07\x62\x6c\x6f\x63\x6b\x4e\x6f\x04\x76\x61\x6c\x31\x04\x76\x61\x6c\x32\x04\x76\x61\x6c\x33\x05\x49\x6e\x74\x36\x34\x06\x55\x49\x6e\x74\x31\x36\x06\x53\x74\x72\x69\x6e\x67\x07\x46\x6c\x6f\x61\x74\x33\x32\x05\x55\x49\x6e\x74\x38\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x02\x41\x4d\x00\x00\x00\x3f\x01', + b'\x05\x02\x69\x64\x07\x62\x6c\x6f\x63\x6b\x4e\x6f\x04\x76\x61\x6c\x31\x04\x76\x61\x6c\x32\x04\x76\x61\x6c\x33\x05\x49\x6e\x74\x36\x34\x06\x55\x49\x6e\x74\x31\x36\x06\x53\x74\x72\x69\x6e\x67\x07\x46\x6c\x6f\x61\x74\x33\x32\x05\x55\x49\x6e\x74\x38\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x02\x41\x4d\x00\x00\x00\x3f\x01', + b'\x05\x02\x69\x64\x07\x62\x6c\x6f\x63\x6b\x4e\x6f\x04\x76\x61\x6c\x31\x04\x76\x61\x6c\x32\x04\x76\x61\x6c\x33\x05\x49\x6e\x74\x36\x34\x06\x55\x49\x6e\x74\x31\x36\x06\x53\x74\x72\x69\x6e\x67\x07\x46\x6c\x6f\x61\x74\x33\x32\x05\x55\x49\x6e\x74\x38\x01\x00\x00\x00\x00\x00\x00\x00\x00\x00\x02\x41\x4d\x00\x00\x00\x3f\x01\x02\x00\x00\x00\x00\x00\x00\x00\x00\x00\x02\x41\x4d\x00\x00\x00\x3f\x01\x03\x00\x00\x00\x00\x00\x00\x00\x00\x00\x02\x41\x4d\x00\x00\x00\x3f\x01\x04\x00\x00\x00\x00\x00\x00\x00\x00\x00\x02\x41\x4d\x00\x00\x00\x3f\x01\x05\x00\x00\x00\x00\x00\x00\x00\x00\x00\x02\x41\x4d\x00\x00\x00\x3f\x01\x06\x00\x00\x00\x00\x00\x00\x00\x00\x00\x02\x41\x4d\x00\x00\x00\x3f\x01\x07\x00\x00\x00\x00\x00\x00\x00\x00\x00\x02\x41\x4d\x00\x00\x00\x3f\x01\x08\x00\x00\x00\x00\x00\x00\x00\x00\x00\x02\x41\x4d\x00\x00\x00\x3f\x01\x09\x00\x00\x00\x00\x00\x00\x00\x00\x00\x02\x41\x4d\x00\x00\x00\x3f\x01\x0a\x00\x00\x00\x00\x00\x00\x00\x00\x00\x02\x41\x4d\x00\x00\x00\x3f\x01\x0b\x00\x00\x00\x00\x00\x00\x00\x00\x00\x02\x41\x4d\x00\x00\x00\x3f\x01\x0c\x00\x00\x00\x00\x00\x00\x00\x00\x00\x02\x41\x4d\x00\x00\x00\x3f\x01\x0d\x00\x00\x00\x00\x00\x00\x00\x00\x00\x02\x41\x4d\x00\x00\x00\x3f\x01\x0e\x00\x00\x00\x00\x00\x00\x00\x00\x00\x02\x41\x4d\x00\x00\x00\x3f\x01\x0f\x00\x00\x00\x00\x00\x00\x00\x00\x00\x02\x41\x4d\x00\x00\x00\x3f\x01', + b'\x05\x02\x69\x64\x07\x62\x6c\x6f\x63\x6b\x4e\x6f\x04\x76\x61\x6c\x31\x04\x76\x61\x6c\x32\x04\x76\x61\x6c\x33\x05\x49\x6e\x74\x36\x34\x06\x55\x49\x6e\x74\x31\x36\x06\x53\x74\x72\x69\x6e\x67\x07\x46\x6c\x6f\x61\x74\x33\x32\x05\x55\x49\x6e\x74\x38\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x02\x41\x4d\x00\x00\x00\x3f\x01', # '' # !!! On empty message segfault: Address not mapped to object # /contrib/FastMemcpy/FastMemcpy.h:666: memcpy_fast @ 0x21742d65 in /usr/bin/clickhouse @@ -340,9 +339,9 @@ def test_kafka_formats(kafka_cluster): }, 'Protobuf': { 'data_sample': [ - '\x0b\x1a\x02\x41\x4d\x25\x00\x00\x00\x3f\x28\x01', - '\x0d\x08\x01\x1a\x02\x41\x4d\x25\x00\x00\x00\x3f\x28\x01\x0d\x08\x02\x1a\x02\x41\x4d\x25\x00\x00\x00\x3f\x28\x01\x0d\x08\x03\x1a\x02\x41\x4d\x25\x00\x00\x00\x3f\x28\x01\x0d\x08\x04\x1a\x02\x41\x4d\x25\x00\x00\x00\x3f\x28\x01\x0d\x08\x05\x1a\x02\x41\x4d\x25\x00\x00\x00\x3f\x28\x01\x0d\x08\x06\x1a\x02\x41\x4d\x25\x00\x00\x00\x3f\x28\x01\x0d\x08\x07\x1a\x02\x41\x4d\x25\x00\x00\x00\x3f\x28\x01\x0d\x08\x08\x1a\x02\x41\x4d\x25\x00\x00\x00\x3f\x28\x01\x0d\x08\x09\x1a\x02\x41\x4d\x25\x00\x00\x00\x3f\x28\x01\x0d\x08\x0a\x1a\x02\x41\x4d\x25\x00\x00\x00\x3f\x28\x01\x0d\x08\x0b\x1a\x02\x41\x4d\x25\x00\x00\x00\x3f\x28\x01\x0d\x08\x0c\x1a\x02\x41\x4d\x25\x00\x00\x00\x3f\x28\x01\x0d\x08\x0d\x1a\x02\x41\x4d\x25\x00\x00\x00\x3f\x28\x01\x0d\x08\x0e\x1a\x02\x41\x4d\x25\x00\x00\x00\x3f\x28\x01\x0d\x08\x0f\x1a\x02\x41\x4d\x25\x00\x00\x00\x3f\x28\x01', - '\x0b\x1a\x02\x41\x4d\x25\x00\x00\x00\x3f\x28\x01', + b'\x0b\x1a\x02\x41\x4d\x25\x00\x00\x00\x3f\x28\x01', + b'\x0d\x08\x01\x1a\x02\x41\x4d\x25\x00\x00\x00\x3f\x28\x01\x0d\x08\x02\x1a\x02\x41\x4d\x25\x00\x00\x00\x3f\x28\x01\x0d\x08\x03\x1a\x02\x41\x4d\x25\x00\x00\x00\x3f\x28\x01\x0d\x08\x04\x1a\x02\x41\x4d\x25\x00\x00\x00\x3f\x28\x01\x0d\x08\x05\x1a\x02\x41\x4d\x25\x00\x00\x00\x3f\x28\x01\x0d\x08\x06\x1a\x02\x41\x4d\x25\x00\x00\x00\x3f\x28\x01\x0d\x08\x07\x1a\x02\x41\x4d\x25\x00\x00\x00\x3f\x28\x01\x0d\x08\x08\x1a\x02\x41\x4d\x25\x00\x00\x00\x3f\x28\x01\x0d\x08\x09\x1a\x02\x41\x4d\x25\x00\x00\x00\x3f\x28\x01\x0d\x08\x0a\x1a\x02\x41\x4d\x25\x00\x00\x00\x3f\x28\x01\x0d\x08\x0b\x1a\x02\x41\x4d\x25\x00\x00\x00\x3f\x28\x01\x0d\x08\x0c\x1a\x02\x41\x4d\x25\x00\x00\x00\x3f\x28\x01\x0d\x08\x0d\x1a\x02\x41\x4d\x25\x00\x00\x00\x3f\x28\x01\x0d\x08\x0e\x1a\x02\x41\x4d\x25\x00\x00\x00\x3f\x28\x01\x0d\x08\x0f\x1a\x02\x41\x4d\x25\x00\x00\x00\x3f\x28\x01', + b'\x0b\x1a\x02\x41\x4d\x25\x00\x00\x00\x3f\x28\x01', # '' # On empty message exception: Attempt to read after eof # /src/IO/ReadBuffer.h:184: DB::ReadBuffer::throwReadAfterEOF() @ 0x15c9699b in /usr/bin/clickhouse @@ -355,9 +354,9 @@ def test_kafka_formats(kafka_cluster): }, 'ORC': { 'data_sample': [ - '\x4f\x52\x43\x11\x00\x00\x0a\x06\x12\x04\x08\x01\x50\x00\x2b\x00\x00\x0a\x13\x0a\x03\x00\x00\x00\x12\x0c\x08\x01\x12\x06\x08\x00\x10\x00\x18\x00\x50\x00\x30\x00\x00\xe3\x12\xe7\x62\x65\x00\x01\x21\x3e\x0e\x46\x25\x0e\x2e\x46\x03\x21\x46\x03\x09\xa6\x00\x06\x00\x32\x00\x00\xe3\x92\xe4\x62\x65\x00\x01\x21\x01\x0e\x46\x25\x2e\x2e\x26\x47\x5f\x21\x20\x96\x60\x09\x60\x00\x00\x36\x00\x00\xe3\x92\xe1\x62\x65\x00\x01\x21\x61\x0e\x46\x23\x5e\x2e\x46\x03\x21\x66\x03\x3d\x53\x29\x10\x11\xc0\x00\x00\x2b\x00\x00\x0a\x13\x0a\x03\x00\x00\x00\x12\x0c\x08\x01\x12\x06\x08\x02\x10\x02\x18\x02\x50\x00\x05\x00\x00\xff\x00\x03\x00\x00\x30\x07\x00\x00\x40\x00\x80\x05\x00\x00\x41\x4d\x07\x00\x00\x42\x00\x80\x03\x00\x00\x0a\x07\x00\x00\x42\x00\x80\x05\x00\x00\xff\x01\x88\x00\x00\x4d\xca\xc1\x0a\x80\x30\x0c\x03\xd0\x2e\x6b\xcb\x98\x17\xf1\x14\x50\xfc\xff\xcf\xb4\x66\x1e\x3c\x84\x47\x9a\xce\x1c\xb9\x1b\xb7\xf9\xda\x48\x09\x9e\xb2\xf3\x92\xce\x5b\x86\xf6\x56\x7f\x21\x41\x2f\x51\xa6\x7a\xd7\x1d\xe5\xea\xae\x3d\xca\xd5\x83\x71\x60\xd8\x17\xfc\x62\x0f\xa8\x00\x00\xe3\x4a\xe6\x62\xe1\x60\x0c\x60\xe0\xe2\xe3\x60\x14\x62\xe3\x60\x10\x60\x90\x60\x08\x60\x88\x60\xe5\x12\xe0\x60\x54\xe2\xe0\x62\x34\x10\x62\x34\x90\x60\x02\x8a\x70\x71\x09\x01\x45\xb8\xb8\x98\x1c\x7d\x85\x80\x58\x82\x05\x28\xc6\xcd\x25\xca\xc1\x68\xc4\x0b\x52\xc5\x6c\xa0\x67\x2a\x05\x22\xc0\x4a\x21\x86\x31\x09\x30\x81\xb5\xb2\x02\x00\x36\x01\x00\x25\x8c\xbd\x0a\xc2\x30\x14\x85\x73\x6f\x92\xf6\x92\x6a\x09\x01\x21\x64\x92\x4e\x75\x91\x58\x71\xc9\x64\x27\x5d\x2c\x1d\x5d\xfd\x59\xc4\x42\x37\x5f\xc0\x17\xe8\x23\x9b\xc6\xe1\x3b\x70\x0f\xdf\xb9\xc4\xf5\x17\x5d\x41\x5c\x4f\x60\x37\xeb\x53\x0d\x55\x4d\x0b\x23\x01\xb9\x90\x2e\xbf\x0f\xe3\xe3\xdd\x8d\x0e\x5f\x4f\x27\x3e\xb7\x61\x97\xb2\x49\xb9\xaf\x90\x20\x92\x27\x32\x2a\x6b\xf4\xf3\x0d\x1e\x82\x20\xe8\x59\x28\x09\x4c\x46\x4c\x33\xcb\x7a\x76\x95\x41\x47\x9f\x14\x78\x03\xde\x62\x6c\x54\x30\xb1\x51\x0a\xdb\x8b\x89\x58\x11\xbb\x22\xac\x08\x9a\xe5\x6c\x71\xbf\x3d\xb8\x39\x92\xfa\x7f\x86\x1a\xd3\x54\x1e\xa7\xee\xcc\x7e\x08\x9e\x01\x10\x01\x18\x80\x80\x10\x22\x02\x00\x0c\x28\x57\x30\x06\x82\xf4\x03\x03\x4f\x52\x43\x18', - '\x4f\x52\x43\x11\x00\x00\x0a\x06\x12\x04\x08\x0f\x50\x00\x2b\x00\x00\x0a\x13\x0a\x03\x00\x00\x00\x12\x0c\x08\x0f\x12\x06\x08\x00\x10\x00\x18\x00\x50\x00\x30\x00\x00\xe3\x12\xe7\x62\x65\x00\x01\x21\x3e\x0e\x7e\x25\x0e\x2e\x46\x43\x21\x46\x4b\x09\xad\x00\x06\x00\x33\x00\x00\x0a\x17\x0a\x03\x00\x00\x00\x12\x10\x08\x0f\x22\x0a\x0a\x02\x41\x4d\x12\x02\x41\x4d\x18\x3c\x50\x00\x3a\x00\x00\xe3\x92\xe1\x62\x65\x00\x01\x21\x61\x0e\x7e\x23\x5e\x2e\x46\x03\x21\x66\x03\x3d\x53\x29\x66\x73\x3d\xd3\x00\x06\x00\x2b\x00\x00\x0a\x13\x0a\x03\x00\x00\x00\x12\x0c\x08\x0f\x12\x06\x08\x02\x10\x02\x18\x1e\x50\x00\x05\x00\x00\x0c\x00\x2b\x00\x00\x31\x32\x33\x34\x35\x36\x37\x38\x39\x31\x30\x31\x31\x31\x32\x31\x33\x31\x34\x31\x35\x09\x00\x00\x06\x01\x03\x02\x09\x00\x00\xc0\x0e\x00\x00\x07\x00\x00\x42\x00\x80\x05\x00\x00\x41\x4d\x0a\x00\x00\xe3\xe2\x42\x01\x00\x09\x00\x00\xc0\x0e\x02\x00\x05\x00\x00\x0c\x01\x94\x00\x00\x2d\xca\xc1\x0e\x80\x30\x08\x03\xd0\xc1\x60\x2e\xf3\x62\x76\x6a\xe2\x0e\xfe\xff\x57\x5a\x3b\x0f\xe4\x51\xe8\x68\xbd\x5d\x05\xe7\xf8\x34\x40\x3a\x6e\x59\xb1\x64\xe0\x91\xa9\xbf\xb1\x97\xd2\x95\x9d\x1e\xca\x55\x3a\x6d\xb4\xd2\xdd\x0b\x74\x9a\x74\xf7\x12\x39\xbd\x97\x7f\x7c\x06\xbb\xa6\x8d\x97\x17\xb4\x00\x00\xe3\x4a\xe6\x62\xe1\xe0\x0f\x60\xe0\xe2\xe3\xe0\x17\x62\xe3\x60\x10\x60\x90\x60\x08\x60\x88\x60\xe5\x12\xe0\xe0\x57\xe2\xe0\x62\x34\x14\x62\xb4\x94\xd0\x02\x8a\xc8\x73\x09\x01\x45\xb8\xb8\x98\x1c\x7d\x85\x80\x58\xc2\x06\x28\x26\xc4\x25\xca\xc1\x6f\xc4\xcb\xc5\x68\x20\xc4\x6c\xa0\x67\x2a\xc5\x6c\xae\x67\x0a\x14\xe6\x87\x1a\xc6\x24\xc0\x24\x21\x07\x32\x0c\x00\x4a\x01\x00\xe3\x60\x16\x58\xc3\x24\xc5\xcd\xc1\x2c\x30\x89\x51\xc2\x4b\xc1\x57\x83\x5f\x49\x83\x83\x47\x88\x95\x91\x89\x99\x85\x55\x8a\x3d\x29\x27\x3f\x39\xdb\x2f\x5f\x8a\x29\x33\x45\x8a\xa5\x2c\x31\xc7\x10\x4c\x1a\x81\x49\x63\x25\x26\x0e\x46\x20\x66\x07\x63\x36\x0e\x3e\x0d\x26\x03\x10\x9f\xd1\x80\xdf\x8a\x85\x83\x3f\x80\xc1\x8a\x8f\x83\x5f\x88\x8d\x83\x41\x80\x41\x82\x21\x80\x21\x82\xd5\x4a\x80\x83\x5f\x89\x83\x8b\xd1\x50\x88\xd1\x52\x42\x0b\x28\x22\x6f\x25\x04\x14\xe1\xe2\x62\x72\xf4\x15\x02\x62\x09\x1b\xa0\x98\x90\x95\x28\x07\xbf\x11\x2f\x17\xa3\x81\x10\xb3\x81\x9e\xa9\x14\xb3\xb9\x9e\x29\x50\x98\x1f\x6a\x18\x93\x00\x93\x84\x1c\xc8\x30\x87\x09\x7e\x1e\x0c\x00\x08\xa8\x01\x10\x01\x18\x80\x80\x10\x22\x02\x00\x0c\x28\x5d\x30\x06\x82\xf4\x03\x03\x4f\x52\x43\x18', - '\x4f\x52\x43\x11\x00\x00\x0a\x06\x12\x04\x08\x01\x50\x00\x2b\x00\x00\x0a\x13\x0a\x03\x00\x00\x00\x12\x0c\x08\x01\x12\x06\x08\x00\x10\x00\x18\x00\x50\x00\x30\x00\x00\xe3\x12\xe7\x62\x65\x00\x01\x21\x3e\x0e\x46\x25\x0e\x2e\x46\x03\x21\x46\x03\x09\xa6\x00\x06\x00\x32\x00\x00\xe3\x92\xe4\x62\x65\x00\x01\x21\x01\x0e\x46\x25\x2e\x2e\x26\x47\x5f\x21\x20\x96\x60\x09\x60\x00\x00\x36\x00\x00\xe3\x92\xe1\x62\x65\x00\x01\x21\x61\x0e\x46\x23\x5e\x2e\x46\x03\x21\x66\x03\x3d\x53\x29\x10\x11\xc0\x00\x00\x2b\x00\x00\x0a\x13\x0a\x03\x00\x00\x00\x12\x0c\x08\x01\x12\x06\x08\x02\x10\x02\x18\x02\x50\x00\x05\x00\x00\xff\x00\x03\x00\x00\x30\x07\x00\x00\x40\x00\x80\x05\x00\x00\x41\x4d\x07\x00\x00\x42\x00\x80\x03\x00\x00\x0a\x07\x00\x00\x42\x00\x80\x05\x00\x00\xff\x01\x88\x00\x00\x4d\xca\xc1\x0a\x80\x30\x0c\x03\xd0\x2e\x6b\xcb\x98\x17\xf1\x14\x50\xfc\xff\xcf\xb4\x66\x1e\x3c\x84\x47\x9a\xce\x1c\xb9\x1b\xb7\xf9\xda\x48\x09\x9e\xb2\xf3\x92\xce\x5b\x86\xf6\x56\x7f\x21\x41\x2f\x51\xa6\x7a\xd7\x1d\xe5\xea\xae\x3d\xca\xd5\x83\x71\x60\xd8\x17\xfc\x62\x0f\xa8\x00\x00\xe3\x4a\xe6\x62\xe1\x60\x0c\x60\xe0\xe2\xe3\x60\x14\x62\xe3\x60\x10\x60\x90\x60\x08\x60\x88\x60\xe5\x12\xe0\x60\x54\xe2\xe0\x62\x34\x10\x62\x34\x90\x60\x02\x8a\x70\x71\x09\x01\x45\xb8\xb8\x98\x1c\x7d\x85\x80\x58\x82\x05\x28\xc6\xcd\x25\xca\xc1\x68\xc4\x0b\x52\xc5\x6c\xa0\x67\x2a\x05\x22\xc0\x4a\x21\x86\x31\x09\x30\x81\xb5\xb2\x02\x00\x36\x01\x00\x25\x8c\xbd\x0a\xc2\x30\x14\x85\x73\x6f\x92\xf6\x92\x6a\x09\x01\x21\x64\x92\x4e\x75\x91\x58\x71\xc9\x64\x27\x5d\x2c\x1d\x5d\xfd\x59\xc4\x42\x37\x5f\xc0\x17\xe8\x23\x9b\xc6\xe1\x3b\x70\x0f\xdf\xb9\xc4\xf5\x17\x5d\x41\x5c\x4f\x60\x37\xeb\x53\x0d\x55\x4d\x0b\x23\x01\xb9\x90\x2e\xbf\x0f\xe3\xe3\xdd\x8d\x0e\x5f\x4f\x27\x3e\xb7\x61\x97\xb2\x49\xb9\xaf\x90\x20\x92\x27\x32\x2a\x6b\xf4\xf3\x0d\x1e\x82\x20\xe8\x59\x28\x09\x4c\x46\x4c\x33\xcb\x7a\x76\x95\x41\x47\x9f\x14\x78\x03\xde\x62\x6c\x54\x30\xb1\x51\x0a\xdb\x8b\x89\x58\x11\xbb\x22\xac\x08\x9a\xe5\x6c\x71\xbf\x3d\xb8\x39\x92\xfa\x7f\x86\x1a\xd3\x54\x1e\xa7\xee\xcc\x7e\x08\x9e\x01\x10\x01\x18\x80\x80\x10\x22\x02\x00\x0c\x28\x57\x30\x06\x82\xf4\x03\x03\x4f\x52\x43\x18', + b'\x4f\x52\x43\x11\x00\x00\x0a\x06\x12\x04\x08\x01\x50\x00\x2b\x00\x00\x0a\x13\x0a\x03\x00\x00\x00\x12\x0c\x08\x01\x12\x06\x08\x00\x10\x00\x18\x00\x50\x00\x30\x00\x00\xe3\x12\xe7\x62\x65\x00\x01\x21\x3e\x0e\x46\x25\x0e\x2e\x46\x03\x21\x46\x03\x09\xa6\x00\x06\x00\x32\x00\x00\xe3\x92\xe4\x62\x65\x00\x01\x21\x01\x0e\x46\x25\x2e\x2e\x26\x47\x5f\x21\x20\x96\x60\x09\x60\x00\x00\x36\x00\x00\xe3\x92\xe1\x62\x65\x00\x01\x21\x61\x0e\x46\x23\x5e\x2e\x46\x03\x21\x66\x03\x3d\x53\x29\x10\x11\xc0\x00\x00\x2b\x00\x00\x0a\x13\x0a\x03\x00\x00\x00\x12\x0c\x08\x01\x12\x06\x08\x02\x10\x02\x18\x02\x50\x00\x05\x00\x00\xff\x00\x03\x00\x00\x30\x07\x00\x00\x40\x00\x80\x05\x00\x00\x41\x4d\x07\x00\x00\x42\x00\x80\x03\x00\x00\x0a\x07\x00\x00\x42\x00\x80\x05\x00\x00\xff\x01\x88\x00\x00\x4d\xca\xc1\x0a\x80\x30\x0c\x03\xd0\x2e\x6b\xcb\x98\x17\xf1\x14\x50\xfc\xff\xcf\xb4\x66\x1e\x3c\x84\x47\x9a\xce\x1c\xb9\x1b\xb7\xf9\xda\x48\x09\x9e\xb2\xf3\x92\xce\x5b\x86\xf6\x56\x7f\x21\x41\x2f\x51\xa6\x7a\xd7\x1d\xe5\xea\xae\x3d\xca\xd5\x83\x71\x60\xd8\x17\xfc\x62\x0f\xa8\x00\x00\xe3\x4a\xe6\x62\xe1\x60\x0c\x60\xe0\xe2\xe3\x60\x14\x62\xe3\x60\x10\x60\x90\x60\x08\x60\x88\x60\xe5\x12\xe0\x60\x54\xe2\xe0\x62\x34\x10\x62\x34\x90\x60\x02\x8a\x70\x71\x09\x01\x45\xb8\xb8\x98\x1c\x7d\x85\x80\x58\x82\x05\x28\xc6\xcd\x25\xca\xc1\x68\xc4\x0b\x52\xc5\x6c\xa0\x67\x2a\x05\x22\xc0\x4a\x21\x86\x31\x09\x30\x81\xb5\xb2\x02\x00\x36\x01\x00\x25\x8c\xbd\x0a\xc2\x30\x14\x85\x73\x6f\x92\xf6\x92\x6a\x09\x01\x21\x64\x92\x4e\x75\x91\x58\x71\xc9\x64\x27\x5d\x2c\x1d\x5d\xfd\x59\xc4\x42\x37\x5f\xc0\x17\xe8\x23\x9b\xc6\xe1\x3b\x70\x0f\xdf\xb9\xc4\xf5\x17\x5d\x41\x5c\x4f\x60\x37\xeb\x53\x0d\x55\x4d\x0b\x23\x01\xb9\x90\x2e\xbf\x0f\xe3\xe3\xdd\x8d\x0e\x5f\x4f\x27\x3e\xb7\x61\x97\xb2\x49\xb9\xaf\x90\x20\x92\x27\x32\x2a\x6b\xf4\xf3\x0d\x1e\x82\x20\xe8\x59\x28\x09\x4c\x46\x4c\x33\xcb\x7a\x76\x95\x41\x47\x9f\x14\x78\x03\xde\x62\x6c\x54\x30\xb1\x51\x0a\xdb\x8b\x89\x58\x11\xbb\x22\xac\x08\x9a\xe5\x6c\x71\xbf\x3d\xb8\x39\x92\xfa\x7f\x86\x1a\xd3\x54\x1e\xa7\xee\xcc\x7e\x08\x9e\x01\x10\x01\x18\x80\x80\x10\x22\x02\x00\x0c\x28\x57\x30\x06\x82\xf4\x03\x03\x4f\x52\x43\x18', + b'\x4f\x52\x43\x11\x00\x00\x0a\x06\x12\x04\x08\x0f\x50\x00\x2b\x00\x00\x0a\x13\x0a\x03\x00\x00\x00\x12\x0c\x08\x0f\x12\x06\x08\x00\x10\x00\x18\x00\x50\x00\x30\x00\x00\xe3\x12\xe7\x62\x65\x00\x01\x21\x3e\x0e\x7e\x25\x0e\x2e\x46\x43\x21\x46\x4b\x09\xad\x00\x06\x00\x33\x00\x00\x0a\x17\x0a\x03\x00\x00\x00\x12\x10\x08\x0f\x22\x0a\x0a\x02\x41\x4d\x12\x02\x41\x4d\x18\x3c\x50\x00\x3a\x00\x00\xe3\x92\xe1\x62\x65\x00\x01\x21\x61\x0e\x7e\x23\x5e\x2e\x46\x03\x21\x66\x03\x3d\x53\x29\x66\x73\x3d\xd3\x00\x06\x00\x2b\x00\x00\x0a\x13\x0a\x03\x00\x00\x00\x12\x0c\x08\x0f\x12\x06\x08\x02\x10\x02\x18\x1e\x50\x00\x05\x00\x00\x0c\x00\x2b\x00\x00\x31\x32\x33\x34\x35\x36\x37\x38\x39\x31\x30\x31\x31\x31\x32\x31\x33\x31\x34\x31\x35\x09\x00\x00\x06\x01\x03\x02\x09\x00\x00\xc0\x0e\x00\x00\x07\x00\x00\x42\x00\x80\x05\x00\x00\x41\x4d\x0a\x00\x00\xe3\xe2\x42\x01\x00\x09\x00\x00\xc0\x0e\x02\x00\x05\x00\x00\x0c\x01\x94\x00\x00\x2d\xca\xc1\x0e\x80\x30\x08\x03\xd0\xc1\x60\x2e\xf3\x62\x76\x6a\xe2\x0e\xfe\xff\x57\x5a\x3b\x0f\xe4\x51\xe8\x68\xbd\x5d\x05\xe7\xf8\x34\x40\x3a\x6e\x59\xb1\x64\xe0\x91\xa9\xbf\xb1\x97\xd2\x95\x9d\x1e\xca\x55\x3a\x6d\xb4\xd2\xdd\x0b\x74\x9a\x74\xf7\x12\x39\xbd\x97\x7f\x7c\x06\xbb\xa6\x8d\x97\x17\xb4\x00\x00\xe3\x4a\xe6\x62\xe1\xe0\x0f\x60\xe0\xe2\xe3\xe0\x17\x62\xe3\x60\x10\x60\x90\x60\x08\x60\x88\x60\xe5\x12\xe0\xe0\x57\xe2\xe0\x62\x34\x14\x62\xb4\x94\xd0\x02\x8a\xc8\x73\x09\x01\x45\xb8\xb8\x98\x1c\x7d\x85\x80\x58\xc2\x06\x28\x26\xc4\x25\xca\xc1\x6f\xc4\xcb\xc5\x68\x20\xc4\x6c\xa0\x67\x2a\xc5\x6c\xae\x67\x0a\x14\xe6\x87\x1a\xc6\x24\xc0\x24\x21\x07\x32\x0c\x00\x4a\x01\x00\xe3\x60\x16\x58\xc3\x24\xc5\xcd\xc1\x2c\x30\x89\x51\xc2\x4b\xc1\x57\x83\x5f\x49\x83\x83\x47\x88\x95\x91\x89\x99\x85\x55\x8a\x3d\x29\x27\x3f\x39\xdb\x2f\x5f\x8a\x29\x33\x45\x8a\xa5\x2c\x31\xc7\x10\x4c\x1a\x81\x49\x63\x25\x26\x0e\x46\x20\x66\x07\x63\x36\x0e\x3e\x0d\x26\x03\x10\x9f\xd1\x80\xdf\x8a\x85\x83\x3f\x80\xc1\x8a\x8f\x83\x5f\x88\x8d\x83\x41\x80\x41\x82\x21\x80\x21\x82\xd5\x4a\x80\x83\x5f\x89\x83\x8b\xd1\x50\x88\xd1\x52\x42\x0b\x28\x22\x6f\x25\x04\x14\xe1\xe2\x62\x72\xf4\x15\x02\x62\x09\x1b\xa0\x98\x90\x95\x28\x07\xbf\x11\x2f\x17\xa3\x81\x10\xb3\x81\x9e\xa9\x14\xb3\xb9\x9e\x29\x50\x98\x1f\x6a\x18\x93\x00\x93\x84\x1c\xc8\x30\x87\x09\x7e\x1e\x0c\x00\x08\xa8\x01\x10\x01\x18\x80\x80\x10\x22\x02\x00\x0c\x28\x5d\x30\x06\x82\xf4\x03\x03\x4f\x52\x43\x18', + b'\x4f\x52\x43\x11\x00\x00\x0a\x06\x12\x04\x08\x01\x50\x00\x2b\x00\x00\x0a\x13\x0a\x03\x00\x00\x00\x12\x0c\x08\x01\x12\x06\x08\x00\x10\x00\x18\x00\x50\x00\x30\x00\x00\xe3\x12\xe7\x62\x65\x00\x01\x21\x3e\x0e\x46\x25\x0e\x2e\x46\x03\x21\x46\x03\x09\xa6\x00\x06\x00\x32\x00\x00\xe3\x92\xe4\x62\x65\x00\x01\x21\x01\x0e\x46\x25\x2e\x2e\x26\x47\x5f\x21\x20\x96\x60\x09\x60\x00\x00\x36\x00\x00\xe3\x92\xe1\x62\x65\x00\x01\x21\x61\x0e\x46\x23\x5e\x2e\x46\x03\x21\x66\x03\x3d\x53\x29\x10\x11\xc0\x00\x00\x2b\x00\x00\x0a\x13\x0a\x03\x00\x00\x00\x12\x0c\x08\x01\x12\x06\x08\x02\x10\x02\x18\x02\x50\x00\x05\x00\x00\xff\x00\x03\x00\x00\x30\x07\x00\x00\x40\x00\x80\x05\x00\x00\x41\x4d\x07\x00\x00\x42\x00\x80\x03\x00\x00\x0a\x07\x00\x00\x42\x00\x80\x05\x00\x00\xff\x01\x88\x00\x00\x4d\xca\xc1\x0a\x80\x30\x0c\x03\xd0\x2e\x6b\xcb\x98\x17\xf1\x14\x50\xfc\xff\xcf\xb4\x66\x1e\x3c\x84\x47\x9a\xce\x1c\xb9\x1b\xb7\xf9\xda\x48\x09\x9e\xb2\xf3\x92\xce\x5b\x86\xf6\x56\x7f\x21\x41\x2f\x51\xa6\x7a\xd7\x1d\xe5\xea\xae\x3d\xca\xd5\x83\x71\x60\xd8\x17\xfc\x62\x0f\xa8\x00\x00\xe3\x4a\xe6\x62\xe1\x60\x0c\x60\xe0\xe2\xe3\x60\x14\x62\xe3\x60\x10\x60\x90\x60\x08\x60\x88\x60\xe5\x12\xe0\x60\x54\xe2\xe0\x62\x34\x10\x62\x34\x90\x60\x02\x8a\x70\x71\x09\x01\x45\xb8\xb8\x98\x1c\x7d\x85\x80\x58\x82\x05\x28\xc6\xcd\x25\xca\xc1\x68\xc4\x0b\x52\xc5\x6c\xa0\x67\x2a\x05\x22\xc0\x4a\x21\x86\x31\x09\x30\x81\xb5\xb2\x02\x00\x36\x01\x00\x25\x8c\xbd\x0a\xc2\x30\x14\x85\x73\x6f\x92\xf6\x92\x6a\x09\x01\x21\x64\x92\x4e\x75\x91\x58\x71\xc9\x64\x27\x5d\x2c\x1d\x5d\xfd\x59\xc4\x42\x37\x5f\xc0\x17\xe8\x23\x9b\xc6\xe1\x3b\x70\x0f\xdf\xb9\xc4\xf5\x17\x5d\x41\x5c\x4f\x60\x37\xeb\x53\x0d\x55\x4d\x0b\x23\x01\xb9\x90\x2e\xbf\x0f\xe3\xe3\xdd\x8d\x0e\x5f\x4f\x27\x3e\xb7\x61\x97\xb2\x49\xb9\xaf\x90\x20\x92\x27\x32\x2a\x6b\xf4\xf3\x0d\x1e\x82\x20\xe8\x59\x28\x09\x4c\x46\x4c\x33\xcb\x7a\x76\x95\x41\x47\x9f\x14\x78\x03\xde\x62\x6c\x54\x30\xb1\x51\x0a\xdb\x8b\x89\x58\x11\xbb\x22\xac\x08\x9a\xe5\x6c\x71\xbf\x3d\xb8\x39\x92\xfa\x7f\x86\x1a\xd3\x54\x1e\xa7\xee\xcc\x7e\x08\x9e\x01\x10\x01\x18\x80\x80\x10\x22\x02\x00\x0c\x28\x57\x30\x06\x82\xf4\x03\x03\x4f\x52\x43\x18', # '' # On empty message exception: IOError: File size too small, Stack trace (when copying this message, always include the lines below): # /src/Processors/Formats/Impl/ORCBlockInputFormat.cpp:36: DB::ORCBlockInputFormat::generate() @ 0x1df282a6 in /usr/bin/clickhouse @@ -366,9 +365,9 @@ def test_kafka_formats(kafka_cluster): }, 'CapnProto': { 'data_sample': [ - '\x00\x00\x00\x00\x05\x00\x00\x00\x00\x00\x00\x00\x02\x00\x01\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x01\x00\x00\x00\x00\x3f\x01\x00\x00\x00\x1a\x00\x00\x00\x41\x4d\x00\x00\x00\x00\x00\x00', - '\x00\x00\x00\x00\x05\x00\x00\x00\x00\x00\x00\x00\x02\x00\x01\x00\x01\x00\x00\x00\x00\x00\x00\x00\x00\x00\x01\x00\x00\x00\x00\x3f\x01\x00\x00\x00\x1a\x00\x00\x00\x41\x4d\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x05\x00\x00\x00\x00\x00\x00\x00\x02\x00\x01\x00\x02\x00\x00\x00\x00\x00\x00\x00\x00\x00\x01\x00\x00\x00\x00\x3f\x01\x00\x00\x00\x1a\x00\x00\x00\x41\x4d\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x05\x00\x00\x00\x00\x00\x00\x00\x02\x00\x01\x00\x03\x00\x00\x00\x00\x00\x00\x00\x00\x00\x01\x00\x00\x00\x00\x3f\x01\x00\x00\x00\x1a\x00\x00\x00\x41\x4d\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x05\x00\x00\x00\x00\x00\x00\x00\x02\x00\x01\x00\x04\x00\x00\x00\x00\x00\x00\x00\x00\x00\x01\x00\x00\x00\x00\x3f\x01\x00\x00\x00\x1a\x00\x00\x00\x41\x4d\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x05\x00\x00\x00\x00\x00\x00\x00\x02\x00\x01\x00\x05\x00\x00\x00\x00\x00\x00\x00\x00\x00\x01\x00\x00\x00\x00\x3f\x01\x00\x00\x00\x1a\x00\x00\x00\x41\x4d\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x05\x00\x00\x00\x00\x00\x00\x00\x02\x00\x01\x00\x06\x00\x00\x00\x00\x00\x00\x00\x00\x00\x01\x00\x00\x00\x00\x3f\x01\x00\x00\x00\x1a\x00\x00\x00\x41\x4d\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x05\x00\x00\x00\x00\x00\x00\x00\x02\x00\x01\x00\x07\x00\x00\x00\x00\x00\x00\x00\x00\x00\x01\x00\x00\x00\x00\x3f\x01\x00\x00\x00\x1a\x00\x00\x00\x41\x4d\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x05\x00\x00\x00\x00\x00\x00\x00\x02\x00\x01\x00\x08\x00\x00\x00\x00\x00\x00\x00\x00\x00\x01\x00\x00\x00\x00\x3f\x01\x00\x00\x00\x1a\x00\x00\x00\x41\x4d\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x05\x00\x00\x00\x00\x00\x00\x00\x02\x00\x01\x00\x09\x00\x00\x00\x00\x00\x00\x00\x00\x00\x01\x00\x00\x00\x00\x3f\x01\x00\x00\x00\x1a\x00\x00\x00\x41\x4d\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x05\x00\x00\x00\x00\x00\x00\x00\x02\x00\x01\x00\x0a\x00\x00\x00\x00\x00\x00\x00\x00\x00\x01\x00\x00\x00\x00\x3f\x01\x00\x00\x00\x1a\x00\x00\x00\x41\x4d\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x05\x00\x00\x00\x00\x00\x00\x00\x02\x00\x01\x00\x0b\x00\x00\x00\x00\x00\x00\x00\x00\x00\x01\x00\x00\x00\x00\x3f\x01\x00\x00\x00\x1a\x00\x00\x00\x41\x4d\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x05\x00\x00\x00\x00\x00\x00\x00\x02\x00\x01\x00\x0c\x00\x00\x00\x00\x00\x00\x00\x00\x00\x01\x00\x00\x00\x00\x3f\x01\x00\x00\x00\x1a\x00\x00\x00\x41\x4d\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x05\x00\x00\x00\x00\x00\x00\x00\x02\x00\x01\x00\x0d\x00\x00\x00\x00\x00\x00\x00\x00\x00\x01\x00\x00\x00\x00\x3f\x01\x00\x00\x00\x1a\x00\x00\x00\x41\x4d\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x05\x00\x00\x00\x00\x00\x00\x00\x02\x00\x01\x00\x0e\x00\x00\x00\x00\x00\x00\x00\x00\x00\x01\x00\x00\x00\x00\x3f\x01\x00\x00\x00\x1a\x00\x00\x00\x41\x4d\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x05\x00\x00\x00\x00\x00\x00\x00\x02\x00\x01\x00\x0f\x00\x00\x00\x00\x00\x00\x00\x00\x00\x01\x00\x00\x00\x00\x3f\x01\x00\x00\x00\x1a\x00\x00\x00\x41\x4d\x00\x00\x00\x00\x00\x00', - '\x00\x00\x00\x00\x05\x00\x00\x00\x00\x00\x00\x00\x02\x00\x01\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x01\x00\x00\x00\x00\x3f\x01\x00\x00\x00\x1a\x00\x00\x00\x41\x4d\x00\x00\x00\x00\x00\x00', + b'\x00\x00\x00\x00\x05\x00\x00\x00\x00\x00\x00\x00\x02\x00\x01\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x01\x00\x00\x00\x00\x3f\x01\x00\x00\x00\x1a\x00\x00\x00\x41\x4d\x00\x00\x00\x00\x00\x00', + b'\x00\x00\x00\x00\x05\x00\x00\x00\x00\x00\x00\x00\x02\x00\x01\x00\x01\x00\x00\x00\x00\x00\x00\x00\x00\x00\x01\x00\x00\x00\x00\x3f\x01\x00\x00\x00\x1a\x00\x00\x00\x41\x4d\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x05\x00\x00\x00\x00\x00\x00\x00\x02\x00\x01\x00\x02\x00\x00\x00\x00\x00\x00\x00\x00\x00\x01\x00\x00\x00\x00\x3f\x01\x00\x00\x00\x1a\x00\x00\x00\x41\x4d\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x05\x00\x00\x00\x00\x00\x00\x00\x02\x00\x01\x00\x03\x00\x00\x00\x00\x00\x00\x00\x00\x00\x01\x00\x00\x00\x00\x3f\x01\x00\x00\x00\x1a\x00\x00\x00\x41\x4d\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x05\x00\x00\x00\x00\x00\x00\x00\x02\x00\x01\x00\x04\x00\x00\x00\x00\x00\x00\x00\x00\x00\x01\x00\x00\x00\x00\x3f\x01\x00\x00\x00\x1a\x00\x00\x00\x41\x4d\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x05\x00\x00\x00\x00\x00\x00\x00\x02\x00\x01\x00\x05\x00\x00\x00\x00\x00\x00\x00\x00\x00\x01\x00\x00\x00\x00\x3f\x01\x00\x00\x00\x1a\x00\x00\x00\x41\x4d\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x05\x00\x00\x00\x00\x00\x00\x00\x02\x00\x01\x00\x06\x00\x00\x00\x00\x00\x00\x00\x00\x00\x01\x00\x00\x00\x00\x3f\x01\x00\x00\x00\x1a\x00\x00\x00\x41\x4d\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x05\x00\x00\x00\x00\x00\x00\x00\x02\x00\x01\x00\x07\x00\x00\x00\x00\x00\x00\x00\x00\x00\x01\x00\x00\x00\x00\x3f\x01\x00\x00\x00\x1a\x00\x00\x00\x41\x4d\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x05\x00\x00\x00\x00\x00\x00\x00\x02\x00\x01\x00\x08\x00\x00\x00\x00\x00\x00\x00\x00\x00\x01\x00\x00\x00\x00\x3f\x01\x00\x00\x00\x1a\x00\x00\x00\x41\x4d\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x05\x00\x00\x00\x00\x00\x00\x00\x02\x00\x01\x00\x09\x00\x00\x00\x00\x00\x00\x00\x00\x00\x01\x00\x00\x00\x00\x3f\x01\x00\x00\x00\x1a\x00\x00\x00\x41\x4d\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x05\x00\x00\x00\x00\x00\x00\x00\x02\x00\x01\x00\x0a\x00\x00\x00\x00\x00\x00\x00\x00\x00\x01\x00\x00\x00\x00\x3f\x01\x00\x00\x00\x1a\x00\x00\x00\x41\x4d\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x05\x00\x00\x00\x00\x00\x00\x00\x02\x00\x01\x00\x0b\x00\x00\x00\x00\x00\x00\x00\x00\x00\x01\x00\x00\x00\x00\x3f\x01\x00\x00\x00\x1a\x00\x00\x00\x41\x4d\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x05\x00\x00\x00\x00\x00\x00\x00\x02\x00\x01\x00\x0c\x00\x00\x00\x00\x00\x00\x00\x00\x00\x01\x00\x00\x00\x00\x3f\x01\x00\x00\x00\x1a\x00\x00\x00\x41\x4d\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x05\x00\x00\x00\x00\x00\x00\x00\x02\x00\x01\x00\x0d\x00\x00\x00\x00\x00\x00\x00\x00\x00\x01\x00\x00\x00\x00\x3f\x01\x00\x00\x00\x1a\x00\x00\x00\x41\x4d\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x05\x00\x00\x00\x00\x00\x00\x00\x02\x00\x01\x00\x0e\x00\x00\x00\x00\x00\x00\x00\x00\x00\x01\x00\x00\x00\x00\x3f\x01\x00\x00\x00\x1a\x00\x00\x00\x41\x4d\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x05\x00\x00\x00\x00\x00\x00\x00\x02\x00\x01\x00\x0f\x00\x00\x00\x00\x00\x00\x00\x00\x00\x01\x00\x00\x00\x00\x3f\x01\x00\x00\x00\x1a\x00\x00\x00\x41\x4d\x00\x00\x00\x00\x00\x00', + b'\x00\x00\x00\x00\x05\x00\x00\x00\x00\x00\x00\x00\x02\x00\x01\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x01\x00\x00\x00\x00\x3f\x01\x00\x00\x00\x1a\x00\x00\x00\x41\x4d\x00\x00\x00\x00\x00\x00', # '' # On empty message exception: Cannot read all data. Bytes read: 0. Bytes expected: 4. # /src/IO/ReadBuffer.h:157: DB::ReadBuffer::readStrict(char*, unsigned long) @ 0x15c6894d in /usr/bin/clickhouse @@ -404,30 +403,30 @@ def test_kafka_formats(kafka_cluster): # /src/DataStreams/copyData.cpp:63: DB::copyData(DB::IBlockInputStream&, DB::IBlockOutputStream&, std::__1::atomic*) @ 0x1c9e9fc7 in /usr/bin/clickhouse # /src/Storages/Kafka/StorageKafka.cpp:565: DB::StorageKafka::streamToViews() @ 0x1d8cc3fa in /usr/bin/clickhouse # # 'data_sample' : [ - # # '\x50\x41\x52\x31\x15\x04\x15\x10\x15\x14\x4c\x15\x02\x15\x04\x12\x00\x00\x08\x1c\x00\x00\x00\x00\x00\x00\x00\x00\x15\x00\x15\x06\x15\x0a\x2c\x15\x02\x15\x04\x15\x06\x15\x06\x1c\x18\x08\x00\x00\x00\x00\x00\x00\x00\x00\x18\x08\x00\x00\x00\x00\x00\x00\x00\x00\x16\x00\x28\x08\x00\x00\x00\x00\x00\x00\x00\x00\x18\x08\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x03\x08\x01\x02\x00\x26\xbc\x01\x1c\x15\x04\x19\x35\x04\x00\x06\x19\x18\x02\x69\x64\x15\x02\x16\x02\x16\xac\x01\x16\xb4\x01\x26\x38\x26\x08\x1c\x18\x08\x00\x00\x00\x00\x00\x00\x00\x00\x18\x08\x00\x00\x00\x00\x00\x00\x00\x00\x16\x00\x28\x08\x00\x00\x00\x00\x00\x00\x00\x00\x18\x08\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x15\x04\x15\x08\x15\x0c\x4c\x15\x02\x15\x04\x12\x00\x00\x04\x0c\x00\x00\x00\x00\x15\x00\x15\x06\x15\x0a\x2c\x15\x02\x15\x04\x15\x06\x15\x06\x1c\x36\x00\x28\x04\x00\x00\x00\x00\x18\x04\x00\x00\x00\x00\x00\x00\x00\x03\x08\x01\x02\x00\x26\xc8\x03\x1c\x15\x02\x19\x35\x04\x00\x06\x19\x18\x07\x62\x6c\x6f\x63\x6b\x4e\x6f\x15\x02\x16\x02\x16\x6c\x16\x74\x26\xfc\x02\x26\xd4\x02\x1c\x36\x00\x28\x04\x00\x00\x00\x00\x18\x04\x00\x00\x00\x00\x00\x00\x00\x15\x04\x15\x0c\x15\x10\x4c\x15\x02\x15\x04\x12\x00\x00\x06\x14\x02\x00\x00\x00\x41\x4d\x15\x00\x15\x06\x15\x0a\x2c\x15\x02\x15\x04\x15\x06\x15\x06\x1c\x36\x00\x28\x02\x41\x4d\x18\x02\x41\x4d\x00\x00\x00\x03\x08\x01\x02\x00\x26\xa2\x05\x1c\x15\x0c\x19\x35\x04\x00\x06\x19\x18\x04\x76\x61\x6c\x31\x15\x02\x16\x02\x16\x68\x16\x70\x26\xde\x04\x26\xb2\x04\x1c\x36\x00\x28\x02\x41\x4d\x18\x02\x41\x4d\x00\x00\x00\x15\x04\x15\x08\x15\x0c\x4c\x15\x02\x15\x04\x12\x00\x00\x04\x0c\x00\x00\x00\x3f\x15\x00\x15\x06\x15\x0a\x2c\x15\x02\x15\x04\x15\x06\x15\x06\x1c\x18\x04\x00\x00\x00\x3f\x18\x04\x00\x00\x00\x3f\x16\x00\x28\x04\x00\x00\x00\x3f\x18\x04\x00\x00\x00\x3f\x00\x00\x00\x03\x08\x01\x02\x00\x26\x8a\x07\x1c\x15\x08\x19\x35\x04\x00\x06\x19\x18\x04\x76\x61\x6c\x32\x15\x02\x16\x02\x16\x84\x01\x16\x8c\x01\x26\xa6\x06\x26\xfe\x05\x1c\x18\x04\x00\x00\x00\x3f\x18\x04\x00\x00\x00\x3f\x16\x00\x28\x04\x00\x00\x00\x3f\x18\x04\x00\x00\x00\x3f\x00\x00\x00\x15\x04\x15\x08\x15\x0c\x4c\x15\x02\x15\x04\x12\x00\x00\x04\x0c\x01\x00\x00\x00\x15\x00\x15\x06\x15\x0a\x2c\x15\x02\x15\x04\x15\x06\x15\x06\x1c\x36\x00\x28\x04\x01\x00\x00\x00\x18\x04\x01\x00\x00\x00\x00\x00\x00\x03\x08\x01\x02\x00\x26\xfe\x08\x1c\x15\x02\x19\x35\x04\x00\x06\x19\x18\x04\x76\x61\x6c\x33\x15\x02\x16\x02\x16\x6c\x16\x74\x26\xb2\x08\x26\x8a\x08\x1c\x36\x00\x28\x04\x01\x00\x00\x00\x18\x04\x01\x00\x00\x00\x00\x00\x00\x15\x02\x19\x6c\x35\x00\x18\x06\x73\x63\x68\x65\x6d\x61\x15\x0a\x00\x15\x04\x25\x00\x18\x02\x69\x64\x00\x15\x02\x25\x00\x18\x07\x62\x6c\x6f\x63\x6b\x4e\x6f\x25\x18\x4c\xac\x13\x10\x12\x00\x00\x00\x15\x0c\x25\x00\x18\x04\x76\x61\x6c\x31\x25\x00\x4c\x1c\x00\x00\x00\x15\x08\x25\x00\x18\x04\x76\x61\x6c\x32\x00\x15\x02\x25\x00\x18\x04\x76\x61\x6c\x33\x25\x16\x4c\xac\x13\x08\x12\x00\x00\x00\x16\x02\x19\x1c\x19\x5c\x26\xbc\x01\x1c\x15\x04\x19\x35\x04\x00\x06\x19\x18\x02\x69\x64\x15\x02\x16\x02\x16\xac\x01\x16\xb4\x01\x26\x38\x26\x08\x1c\x18\x08\x00\x00\x00\x00\x00\x00\x00\x00\x18\x08\x00\x00\x00\x00\x00\x00\x00\x00\x16\x00\x28\x08\x00\x00\x00\x00\x00\x00\x00\x00\x18\x08\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x26\xc8\x03\x1c\x15\x02\x19\x35\x04\x00\x06\x19\x18\x07\x62\x6c\x6f\x63\x6b\x4e\x6f\x15\x02\x16\x02\x16\x6c\x16\x74\x26\xfc\x02\x26\xd4\x02\x1c\x36\x00\x28\x04\x00\x00\x00\x00\x18\x04\x00\x00\x00\x00\x00\x00\x00\x26\xa2\x05\x1c\x15\x0c\x19\x35\x04\x00\x06\x19\x18\x04\x76\x61\x6c\x31\x15\x02\x16\x02\x16\x68\x16\x70\x26\xde\x04\x26\xb2\x04\x1c\x36\x00\x28\x02\x41\x4d\x18\x02\x41\x4d\x00\x00\x00\x26\x8a\x07\x1c\x15\x08\x19\x35\x04\x00\x06\x19\x18\x04\x76\x61\x6c\x32\x15\x02\x16\x02\x16\x84\x01\x16\x8c\x01\x26\xa6\x06\x26\xfe\x05\x1c\x18\x04\x00\x00\x00\x3f\x18\x04\x00\x00\x00\x3f\x16\x00\x28\x04\x00\x00\x00\x3f\x18\x04\x00\x00\x00\x3f\x00\x00\x00\x26\xfe\x08\x1c\x15\x02\x19\x35\x04\x00\x06\x19\x18\x04\x76\x61\x6c\x33\x15\x02\x16\x02\x16\x6c\x16\x74\x26\xb2\x08\x26\x8a\x08\x1c\x36\x00\x28\x04\x01\x00\x00\x00\x18\x04\x01\x00\x00\x00\x00\x00\x00\x16\x98\x05\x16\x02\x00\x28\x22\x70\x61\x72\x71\x75\x65\x74\x2d\x63\x70\x70\x20\x76\x65\x72\x73\x69\x6f\x6e\x20\x31\x2e\x35\x2e\x31\x2d\x53\x4e\x41\x50\x53\x48\x4f\x54\x19\x5c\x1c\x00\x00\x1c\x00\x00\x1c\x00\x00\x1c\x00\x00\x1c\x00\x00\x00\xc4\x01\x00\x00\x50\x41\x52\x31', - # # '\x50\x41\x52\x31\x15\x04\x15\xf0\x01\x15\x90\x01\x4c\x15\x1e\x15\x04\x12\x00\x00\x78\x04\x01\x00\x09\x01\x00\x02\x09\x07\x04\x00\x03\x0d\x08\x00\x04\x0d\x08\x00\x05\x0d\x08\x00\x06\x0d\x08\x00\x07\x0d\x08\x00\x08\x0d\x08\x00\x09\x0d\x08\x00\x0a\x0d\x08\x00\x0b\x0d\x08\x00\x0c\x0d\x08\x00\x0d\x0d\x08\x3c\x0e\x00\x00\x00\x00\x00\x00\x00\x0f\x00\x00\x00\x00\x00\x00\x00\x15\x00\x15\x14\x15\x18\x2c\x15\x1e\x15\x04\x15\x06\x15\x06\x1c\x18\x08\x0f\x00\x00\x00\x00\x00\x00\x00\x18\x08\x01\x00\x00\x00\x00\x00\x00\x00\x16\x00\x28\x08\x0f\x00\x00\x00\x00\x00\x00\x00\x18\x08\x01\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x0a\x24\x04\x05\x10\x32\x54\x76\x98\xba\xdc\x0e\x26\xca\x02\x1c\x15\x04\x19\x35\x04\x00\x06\x19\x18\x02\x69\x64\x15\x02\x16\x1e\x16\x9e\x03\x16\xc2\x02\x26\xb8\x01\x26\x08\x1c\x18\x08\x0f\x00\x00\x00\x00\x00\x00\x00\x18\x08\x01\x00\x00\x00\x00\x00\x00\x00\x16\x00\x28\x08\x0f\x00\x00\x00\x00\x00\x00\x00\x18\x08\x01\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x15\x04\x15\x08\x15\x0c\x4c\x15\x02\x15\x04\x12\x00\x00\x04\x0c\x00\x00\x00\x00\x15\x00\x15\x06\x15\x0a\x2c\x15\x1e\x15\x04\x15\x06\x15\x06\x1c\x36\x00\x28\x04\x00\x00\x00\x00\x18\x04\x00\x00\x00\x00\x00\x00\x00\x03\x08\x01\x1e\x00\x26\xd8\x04\x1c\x15\x02\x19\x35\x04\x00\x06\x19\x18\x07\x62\x6c\x6f\x63\x6b\x4e\x6f\x15\x02\x16\x1e\x16\x6c\x16\x74\x26\x8c\x04\x26\xe4\x03\x1c\x36\x00\x28\x04\x00\x00\x00\x00\x18\x04\x00\x00\x00\x00\x00\x00\x00\x15\x04\x15\x0c\x15\x10\x4c\x15\x02\x15\x04\x12\x00\x00\x06\x14\x02\x00\x00\x00\x41\x4d\x15\x00\x15\x06\x15\x0a\x2c\x15\x1e\x15\x04\x15\x06\x15\x06\x1c\x36\x00\x28\x02\x41\x4d\x18\x02\x41\x4d\x00\x00\x00\x03\x08\x01\x1e\x00\x26\xb2\x06\x1c\x15\x0c\x19\x35\x04\x00\x06\x19\x18\x04\x76\x61\x6c\x31\x15\x02\x16\x1e\x16\x68\x16\x70\x26\xee\x05\x26\xc2\x05\x1c\x36\x00\x28\x02\x41\x4d\x18\x02\x41\x4d\x00\x00\x00\x15\x04\x15\x08\x15\x0c\x4c\x15\x02\x15\x04\x12\x00\x00\x04\x0c\x00\x00\x00\x3f\x15\x00\x15\x06\x15\x0a\x2c\x15\x1e\x15\x04\x15\x06\x15\x06\x1c\x18\x04\x00\x00\x00\x3f\x18\x04\x00\x00\x00\x3f\x16\x00\x28\x04\x00\x00\x00\x3f\x18\x04\x00\x00\x00\x3f\x00\x00\x00\x03\x08\x01\x1e\x00\x26\x9a\x08\x1c\x15\x08\x19\x35\x04\x00\x06\x19\x18\x04\x76\x61\x6c\x32\x15\x02\x16\x1e\x16\x84\x01\x16\x8c\x01\x26\xb6\x07\x26\x8e\x07\x1c\x18\x04\x00\x00\x00\x3f\x18\x04\x00\x00\x00\x3f\x16\x00\x28\x04\x00\x00\x00\x3f\x18\x04\x00\x00\x00\x3f\x00\x00\x00\x15\x04\x15\x08\x15\x0c\x4c\x15\x02\x15\x04\x12\x00\x00\x04\x0c\x01\x00\x00\x00\x15\x00\x15\x06\x15\x0a\x2c\x15\x1e\x15\x04\x15\x06\x15\x06\x1c\x36\x00\x28\x04\x01\x00\x00\x00\x18\x04\x01\x00\x00\x00\x00\x00\x00\x03\x08\x01\x1e\x00\x26\x8e\x0a\x1c\x15\x02\x19\x35\x04\x00\x06\x19\x18\x04\x76\x61\x6c\x33\x15\x02\x16\x1e\x16\x6c\x16\x74\x26\xc2\x09\x26\x9a\x09\x1c\x36\x00\x28\x04\x01\x00\x00\x00\x18\x04\x01\x00\x00\x00\x00\x00\x00\x15\x02\x19\x6c\x35\x00\x18\x06\x73\x63\x68\x65\x6d\x61\x15\x0a\x00\x15\x04\x25\x00\x18\x02\x69\x64\x00\x15\x02\x25\x00\x18\x07\x62\x6c\x6f\x63\x6b\x4e\x6f\x25\x18\x4c\xac\x13\x10\x12\x00\x00\x00\x15\x0c\x25\x00\x18\x04\x76\x61\x6c\x31\x25\x00\x4c\x1c\x00\x00\x00\x15\x08\x25\x00\x18\x04\x76\x61\x6c\x32\x00\x15\x02\x25\x00\x18\x04\x76\x61\x6c\x33\x25\x16\x4c\xac\x13\x08\x12\x00\x00\x00\x16\x1e\x19\x1c\x19\x5c\x26\xca\x02\x1c\x15\x04\x19\x35\x04\x00\x06\x19\x18\x02\x69\x64\x15\x02\x16\x1e\x16\x9e\x03\x16\xc2\x02\x26\xb8\x01\x26\x08\x1c\x18\x08\x0f\x00\x00\x00\x00\x00\x00\x00\x18\x08\x01\x00\x00\x00\x00\x00\x00\x00\x16\x00\x28\x08\x0f\x00\x00\x00\x00\x00\x00\x00\x18\x08\x01\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x26\xd8\x04\x1c\x15\x02\x19\x35\x04\x00\x06\x19\x18\x07\x62\x6c\x6f\x63\x6b\x4e\x6f\x15\x02\x16\x1e\x16\x6c\x16\x74\x26\x8c\x04\x26\xe4\x03\x1c\x36\x00\x28\x04\x00\x00\x00\x00\x18\x04\x00\x00\x00\x00\x00\x00\x00\x26\xb2\x06\x1c\x15\x0c\x19\x35\x04\x00\x06\x19\x18\x04\x76\x61\x6c\x31\x15\x02\x16\x1e\x16\x68\x16\x70\x26\xee\x05\x26\xc2\x05\x1c\x36\x00\x28\x02\x41\x4d\x18\x02\x41\x4d\x00\x00\x00\x26\x9a\x08\x1c\x15\x08\x19\x35\x04\x00\x06\x19\x18\x04\x76\x61\x6c\x32\x15\x02\x16\x1e\x16\x84\x01\x16\x8c\x01\x26\xb6\x07\x26\x8e\x07\x1c\x18\x04\x00\x00\x00\x3f\x18\x04\x00\x00\x00\x3f\x16\x00\x28\x04\x00\x00\x00\x3f\x18\x04\x00\x00\x00\x3f\x00\x00\x00\x26\x8e\x0a\x1c\x15\x02\x19\x35\x04\x00\x06\x19\x18\x04\x76\x61\x6c\x33\x15\x02\x16\x1e\x16\x6c\x16\x74\x26\xc2\x09\x26\x9a\x09\x1c\x36\x00\x28\x04\x01\x00\x00\x00\x18\x04\x01\x00\x00\x00\x00\x00\x00\x16\xa6\x06\x16\x1e\x00\x28\x22\x70\x61\x72\x71\x75\x65\x74\x2d\x63\x70\x70\x20\x76\x65\x72\x73\x69\x6f\x6e\x20\x31\x2e\x35\x2e\x31\x2d\x53\x4e\x41\x50\x53\x48\x4f\x54\x19\x5c\x1c\x00\x00\x1c\x00\x00\x1c\x00\x00\x1c\x00\x00\x1c\x00\x00\x00\xc5\x01\x00\x00\x50\x41\x52\x31', - # # '\x50\x41\x52\x31\x15\x04\x15\x10\x15\x14\x4c\x15\x02\x15\x04\x12\x00\x00\x08\x1c\x00\x00\x00\x00\x00\x00\x00\x00\x15\x00\x15\x06\x15\x0a\x2c\x15\x02\x15\x04\x15\x06\x15\x06\x1c\x18\x08\x00\x00\x00\x00\x00\x00\x00\x00\x18\x08\x00\x00\x00\x00\x00\x00\x00\x00\x16\x00\x28\x08\x00\x00\x00\x00\x00\x00\x00\x00\x18\x08\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x03\x08\x01\x02\x00\x26\xbc\x01\x1c\x15\x04\x19\x35\x04\x00\x06\x19\x18\x02\x69\x64\x15\x02\x16\x02\x16\xac\x01\x16\xb4\x01\x26\x38\x26\x08\x1c\x18\x08\x00\x00\x00\x00\x00\x00\x00\x00\x18\x08\x00\x00\x00\x00\x00\x00\x00\x00\x16\x00\x28\x08\x00\x00\x00\x00\x00\x00\x00\x00\x18\x08\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x15\x04\x15\x08\x15\x0c\x4c\x15\x02\x15\x04\x12\x00\x00\x04\x0c\x00\x00\x00\x00\x15\x00\x15\x06\x15\x0a\x2c\x15\x02\x15\x04\x15\x06\x15\x06\x1c\x36\x00\x28\x04\x00\x00\x00\x00\x18\x04\x00\x00\x00\x00\x00\x00\x00\x03\x08\x01\x02\x00\x26\xc8\x03\x1c\x15\x02\x19\x35\x04\x00\x06\x19\x18\x07\x62\x6c\x6f\x63\x6b\x4e\x6f\x15\x02\x16\x02\x16\x6c\x16\x74\x26\xfc\x02\x26\xd4\x02\x1c\x36\x00\x28\x04\x00\x00\x00\x00\x18\x04\x00\x00\x00\x00\x00\x00\x00\x15\x04\x15\x0c\x15\x10\x4c\x15\x02\x15\x04\x12\x00\x00\x06\x14\x02\x00\x00\x00\x41\x4d\x15\x00\x15\x06\x15\x0a\x2c\x15\x02\x15\x04\x15\x06\x15\x06\x1c\x36\x00\x28\x02\x41\x4d\x18\x02\x41\x4d\x00\x00\x00\x03\x08\x01\x02\x00\x26\xa2\x05\x1c\x15\x0c\x19\x35\x04\x00\x06\x19\x18\x04\x76\x61\x6c\x31\x15\x02\x16\x02\x16\x68\x16\x70\x26\xde\x04\x26\xb2\x04\x1c\x36\x00\x28\x02\x41\x4d\x18\x02\x41\x4d\x00\x00\x00\x15\x04\x15\x08\x15\x0c\x4c\x15\x02\x15\x04\x12\x00\x00\x04\x0c\x00\x00\x00\x3f\x15\x00\x15\x06\x15\x0a\x2c\x15\x02\x15\x04\x15\x06\x15\x06\x1c\x18\x04\x00\x00\x00\x3f\x18\x04\x00\x00\x00\x3f\x16\x00\x28\x04\x00\x00\x00\x3f\x18\x04\x00\x00\x00\x3f\x00\x00\x00\x03\x08\x01\x02\x00\x26\x8a\x07\x1c\x15\x08\x19\x35\x04\x00\x06\x19\x18\x04\x76\x61\x6c\x32\x15\x02\x16\x02\x16\x84\x01\x16\x8c\x01\x26\xa6\x06\x26\xfe\x05\x1c\x18\x04\x00\x00\x00\x3f\x18\x04\x00\x00\x00\x3f\x16\x00\x28\x04\x00\x00\x00\x3f\x18\x04\x00\x00\x00\x3f\x00\x00\x00\x15\x04\x15\x08\x15\x0c\x4c\x15\x02\x15\x04\x12\x00\x00\x04\x0c\x01\x00\x00\x00\x15\x00\x15\x06\x15\x0a\x2c\x15\x02\x15\x04\x15\x06\x15\x06\x1c\x36\x00\x28\x04\x01\x00\x00\x00\x18\x04\x01\x00\x00\x00\x00\x00\x00\x03\x08\x01\x02\x00\x26\xfe\x08\x1c\x15\x02\x19\x35\x04\x00\x06\x19\x18\x04\x76\x61\x6c\x33\x15\x02\x16\x02\x16\x6c\x16\x74\x26\xb2\x08\x26\x8a\x08\x1c\x36\x00\x28\x04\x01\x00\x00\x00\x18\x04\x01\x00\x00\x00\x00\x00\x00\x15\x02\x19\x6c\x35\x00\x18\x06\x73\x63\x68\x65\x6d\x61\x15\x0a\x00\x15\x04\x25\x00\x18\x02\x69\x64\x00\x15\x02\x25\x00\x18\x07\x62\x6c\x6f\x63\x6b\x4e\x6f\x25\x18\x4c\xac\x13\x10\x12\x00\x00\x00\x15\x0c\x25\x00\x18\x04\x76\x61\x6c\x31\x25\x00\x4c\x1c\x00\x00\x00\x15\x08\x25\x00\x18\x04\x76\x61\x6c\x32\x00\x15\x02\x25\x00\x18\x04\x76\x61\x6c\x33\x25\x16\x4c\xac\x13\x08\x12\x00\x00\x00\x16\x02\x19\x1c\x19\x5c\x26\xbc\x01\x1c\x15\x04\x19\x35\x04\x00\x06\x19\x18\x02\x69\x64\x15\x02\x16\x02\x16\xac\x01\x16\xb4\x01\x26\x38\x26\x08\x1c\x18\x08\x00\x00\x00\x00\x00\x00\x00\x00\x18\x08\x00\x00\x00\x00\x00\x00\x00\x00\x16\x00\x28\x08\x00\x00\x00\x00\x00\x00\x00\x00\x18\x08\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x26\xc8\x03\x1c\x15\x02\x19\x35\x04\x00\x06\x19\x18\x07\x62\x6c\x6f\x63\x6b\x4e\x6f\x15\x02\x16\x02\x16\x6c\x16\x74\x26\xfc\x02\x26\xd4\x02\x1c\x36\x00\x28\x04\x00\x00\x00\x00\x18\x04\x00\x00\x00\x00\x00\x00\x00\x26\xa2\x05\x1c\x15\x0c\x19\x35\x04\x00\x06\x19\x18\x04\x76\x61\x6c\x31\x15\x02\x16\x02\x16\x68\x16\x70\x26\xde\x04\x26\xb2\x04\x1c\x36\x00\x28\x02\x41\x4d\x18\x02\x41\x4d\x00\x00\x00\x26\x8a\x07\x1c\x15\x08\x19\x35\x04\x00\x06\x19\x18\x04\x76\x61\x6c\x32\x15\x02\x16\x02\x16\x84\x01\x16\x8c\x01\x26\xa6\x06\x26\xfe\x05\x1c\x18\x04\x00\x00\x00\x3f\x18\x04\x00\x00\x00\x3f\x16\x00\x28\x04\x00\x00\x00\x3f\x18\x04\x00\x00\x00\x3f\x00\x00\x00\x26\xfe\x08\x1c\x15\x02\x19\x35\x04\x00\x06\x19\x18\x04\x76\x61\x6c\x33\x15\x02\x16\x02\x16\x6c\x16\x74\x26\xb2\x08\x26\x8a\x08\x1c\x36\x00\x28\x04\x01\x00\x00\x00\x18\x04\x01\x00\x00\x00\x00\x00\x00\x16\x98\x05\x16\x02\x00\x28\x22\x70\x61\x72\x71\x75\x65\x74\x2d\x63\x70\x70\x20\x76\x65\x72\x73\x69\x6f\x6e\x20\x31\x2e\x35\x2e\x31\x2d\x53\x4e\x41\x50\x53\x48\x4f\x54\x19\x5c\x1c\x00\x00\x1c\x00\x00\x1c\x00\x00\x1c\x00\x00\x1c\x00\x00\x00\xc4\x01\x00\x00\x50\x41\x52\x31', + # # b'\x50\x41\x52\x31\x15\x04\x15\x10\x15\x14\x4c\x15\x02\x15\x04\x12\x00\x00\x08\x1c\x00\x00\x00\x00\x00\x00\x00\x00\x15\x00\x15\x06\x15\x0a\x2c\x15\x02\x15\x04\x15\x06\x15\x06\x1c\x18\x08\x00\x00\x00\x00\x00\x00\x00\x00\x18\x08\x00\x00\x00\x00\x00\x00\x00\x00\x16\x00\x28\x08\x00\x00\x00\x00\x00\x00\x00\x00\x18\x08\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x03\x08\x01\x02\x00\x26\xbc\x01\x1c\x15\x04\x19\x35\x04\x00\x06\x19\x18\x02\x69\x64\x15\x02\x16\x02\x16\xac\x01\x16\xb4\x01\x26\x38\x26\x08\x1c\x18\x08\x00\x00\x00\x00\x00\x00\x00\x00\x18\x08\x00\x00\x00\x00\x00\x00\x00\x00\x16\x00\x28\x08\x00\x00\x00\x00\x00\x00\x00\x00\x18\x08\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x15\x04\x15\x08\x15\x0c\x4c\x15\x02\x15\x04\x12\x00\x00\x04\x0c\x00\x00\x00\x00\x15\x00\x15\x06\x15\x0a\x2c\x15\x02\x15\x04\x15\x06\x15\x06\x1c\x36\x00\x28\x04\x00\x00\x00\x00\x18\x04\x00\x00\x00\x00\x00\x00\x00\x03\x08\x01\x02\x00\x26\xc8\x03\x1c\x15\x02\x19\x35\x04\x00\x06\x19\x18\x07\x62\x6c\x6f\x63\x6b\x4e\x6f\x15\x02\x16\x02\x16\x6c\x16\x74\x26\xfc\x02\x26\xd4\x02\x1c\x36\x00\x28\x04\x00\x00\x00\x00\x18\x04\x00\x00\x00\x00\x00\x00\x00\x15\x04\x15\x0c\x15\x10\x4c\x15\x02\x15\x04\x12\x00\x00\x06\x14\x02\x00\x00\x00\x41\x4d\x15\x00\x15\x06\x15\x0a\x2c\x15\x02\x15\x04\x15\x06\x15\x06\x1c\x36\x00\x28\x02\x41\x4d\x18\x02\x41\x4d\x00\x00\x00\x03\x08\x01\x02\x00\x26\xa2\x05\x1c\x15\x0c\x19\x35\x04\x00\x06\x19\x18\x04\x76\x61\x6c\x31\x15\x02\x16\x02\x16\x68\x16\x70\x26\xde\x04\x26\xb2\x04\x1c\x36\x00\x28\x02\x41\x4d\x18\x02\x41\x4d\x00\x00\x00\x15\x04\x15\x08\x15\x0c\x4c\x15\x02\x15\x04\x12\x00\x00\x04\x0c\x00\x00\x00\x3f\x15\x00\x15\x06\x15\x0a\x2c\x15\x02\x15\x04\x15\x06\x15\x06\x1c\x18\x04\x00\x00\x00\x3f\x18\x04\x00\x00\x00\x3f\x16\x00\x28\x04\x00\x00\x00\x3f\x18\x04\x00\x00\x00\x3f\x00\x00\x00\x03\x08\x01\x02\x00\x26\x8a\x07\x1c\x15\x08\x19\x35\x04\x00\x06\x19\x18\x04\x76\x61\x6c\x32\x15\x02\x16\x02\x16\x84\x01\x16\x8c\x01\x26\xa6\x06\x26\xfe\x05\x1c\x18\x04\x00\x00\x00\x3f\x18\x04\x00\x00\x00\x3f\x16\x00\x28\x04\x00\x00\x00\x3f\x18\x04\x00\x00\x00\x3f\x00\x00\x00\x15\x04\x15\x08\x15\x0c\x4c\x15\x02\x15\x04\x12\x00\x00\x04\x0c\x01\x00\x00\x00\x15\x00\x15\x06\x15\x0a\x2c\x15\x02\x15\x04\x15\x06\x15\x06\x1c\x36\x00\x28\x04\x01\x00\x00\x00\x18\x04\x01\x00\x00\x00\x00\x00\x00\x03\x08\x01\x02\x00\x26\xfe\x08\x1c\x15\x02\x19\x35\x04\x00\x06\x19\x18\x04\x76\x61\x6c\x33\x15\x02\x16\x02\x16\x6c\x16\x74\x26\xb2\x08\x26\x8a\x08\x1c\x36\x00\x28\x04\x01\x00\x00\x00\x18\x04\x01\x00\x00\x00\x00\x00\x00\x15\x02\x19\x6c\x35\x00\x18\x06\x73\x63\x68\x65\x6d\x61\x15\x0a\x00\x15\x04\x25\x00\x18\x02\x69\x64\x00\x15\x02\x25\x00\x18\x07\x62\x6c\x6f\x63\x6b\x4e\x6f\x25\x18\x4c\xac\x13\x10\x12\x00\x00\x00\x15\x0c\x25\x00\x18\x04\x76\x61\x6c\x31\x25\x00\x4c\x1c\x00\x00\x00\x15\x08\x25\x00\x18\x04\x76\x61\x6c\x32\x00\x15\x02\x25\x00\x18\x04\x76\x61\x6c\x33\x25\x16\x4c\xac\x13\x08\x12\x00\x00\x00\x16\x02\x19\x1c\x19\x5c\x26\xbc\x01\x1c\x15\x04\x19\x35\x04\x00\x06\x19\x18\x02\x69\x64\x15\x02\x16\x02\x16\xac\x01\x16\xb4\x01\x26\x38\x26\x08\x1c\x18\x08\x00\x00\x00\x00\x00\x00\x00\x00\x18\x08\x00\x00\x00\x00\x00\x00\x00\x00\x16\x00\x28\x08\x00\x00\x00\x00\x00\x00\x00\x00\x18\x08\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x26\xc8\x03\x1c\x15\x02\x19\x35\x04\x00\x06\x19\x18\x07\x62\x6c\x6f\x63\x6b\x4e\x6f\x15\x02\x16\x02\x16\x6c\x16\x74\x26\xfc\x02\x26\xd4\x02\x1c\x36\x00\x28\x04\x00\x00\x00\x00\x18\x04\x00\x00\x00\x00\x00\x00\x00\x26\xa2\x05\x1c\x15\x0c\x19\x35\x04\x00\x06\x19\x18\x04\x76\x61\x6c\x31\x15\x02\x16\x02\x16\x68\x16\x70\x26\xde\x04\x26\xb2\x04\x1c\x36\x00\x28\x02\x41\x4d\x18\x02\x41\x4d\x00\x00\x00\x26\x8a\x07\x1c\x15\x08\x19\x35\x04\x00\x06\x19\x18\x04\x76\x61\x6c\x32\x15\x02\x16\x02\x16\x84\x01\x16\x8c\x01\x26\xa6\x06\x26\xfe\x05\x1c\x18\x04\x00\x00\x00\x3f\x18\x04\x00\x00\x00\x3f\x16\x00\x28\x04\x00\x00\x00\x3f\x18\x04\x00\x00\x00\x3f\x00\x00\x00\x26\xfe\x08\x1c\x15\x02\x19\x35\x04\x00\x06\x19\x18\x04\x76\x61\x6c\x33\x15\x02\x16\x02\x16\x6c\x16\x74\x26\xb2\x08\x26\x8a\x08\x1c\x36\x00\x28\x04\x01\x00\x00\x00\x18\x04\x01\x00\x00\x00\x00\x00\x00\x16\x98\x05\x16\x02\x00\x28\x22\x70\x61\x72\x71\x75\x65\x74\x2d\x63\x70\x70\x20\x76\x65\x72\x73\x69\x6f\x6e\x20\x31\x2e\x35\x2e\x31\x2d\x53\x4e\x41\x50\x53\x48\x4f\x54\x19\x5c\x1c\x00\x00\x1c\x00\x00\x1c\x00\x00\x1c\x00\x00\x1c\x00\x00\x00\xc4\x01\x00\x00\x50\x41\x52\x31', + # # b'\x50\x41\x52\x31\x15\x04\x15\xf0\x01\x15\x90\x01\x4c\x15\x1e\x15\x04\x12\x00\x00\x78\x04\x01\x00\x09\x01\x00\x02\x09\x07\x04\x00\x03\x0d\x08\x00\x04\x0d\x08\x00\x05\x0d\x08\x00\x06\x0d\x08\x00\x07\x0d\x08\x00\x08\x0d\x08\x00\x09\x0d\x08\x00\x0a\x0d\x08\x00\x0b\x0d\x08\x00\x0c\x0d\x08\x00\x0d\x0d\x08\x3c\x0e\x00\x00\x00\x00\x00\x00\x00\x0f\x00\x00\x00\x00\x00\x00\x00\x15\x00\x15\x14\x15\x18\x2c\x15\x1e\x15\x04\x15\x06\x15\x06\x1c\x18\x08\x0f\x00\x00\x00\x00\x00\x00\x00\x18\x08\x01\x00\x00\x00\x00\x00\x00\x00\x16\x00\x28\x08\x0f\x00\x00\x00\x00\x00\x00\x00\x18\x08\x01\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x0a\x24\x04\x05\x10\x32\x54\x76\x98\xba\xdc\x0e\x26\xca\x02\x1c\x15\x04\x19\x35\x04\x00\x06\x19\x18\x02\x69\x64\x15\x02\x16\x1e\x16\x9e\x03\x16\xc2\x02\x26\xb8\x01\x26\x08\x1c\x18\x08\x0f\x00\x00\x00\x00\x00\x00\x00\x18\x08\x01\x00\x00\x00\x00\x00\x00\x00\x16\x00\x28\x08\x0f\x00\x00\x00\x00\x00\x00\x00\x18\x08\x01\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x15\x04\x15\x08\x15\x0c\x4c\x15\x02\x15\x04\x12\x00\x00\x04\x0c\x00\x00\x00\x00\x15\x00\x15\x06\x15\x0a\x2c\x15\x1e\x15\x04\x15\x06\x15\x06\x1c\x36\x00\x28\x04\x00\x00\x00\x00\x18\x04\x00\x00\x00\x00\x00\x00\x00\x03\x08\x01\x1e\x00\x26\xd8\x04\x1c\x15\x02\x19\x35\x04\x00\x06\x19\x18\x07\x62\x6c\x6f\x63\x6b\x4e\x6f\x15\x02\x16\x1e\x16\x6c\x16\x74\x26\x8c\x04\x26\xe4\x03\x1c\x36\x00\x28\x04\x00\x00\x00\x00\x18\x04\x00\x00\x00\x00\x00\x00\x00\x15\x04\x15\x0c\x15\x10\x4c\x15\x02\x15\x04\x12\x00\x00\x06\x14\x02\x00\x00\x00\x41\x4d\x15\x00\x15\x06\x15\x0a\x2c\x15\x1e\x15\x04\x15\x06\x15\x06\x1c\x36\x00\x28\x02\x41\x4d\x18\x02\x41\x4d\x00\x00\x00\x03\x08\x01\x1e\x00\x26\xb2\x06\x1c\x15\x0c\x19\x35\x04\x00\x06\x19\x18\x04\x76\x61\x6c\x31\x15\x02\x16\x1e\x16\x68\x16\x70\x26\xee\x05\x26\xc2\x05\x1c\x36\x00\x28\x02\x41\x4d\x18\x02\x41\x4d\x00\x00\x00\x15\x04\x15\x08\x15\x0c\x4c\x15\x02\x15\x04\x12\x00\x00\x04\x0c\x00\x00\x00\x3f\x15\x00\x15\x06\x15\x0a\x2c\x15\x1e\x15\x04\x15\x06\x15\x06\x1c\x18\x04\x00\x00\x00\x3f\x18\x04\x00\x00\x00\x3f\x16\x00\x28\x04\x00\x00\x00\x3f\x18\x04\x00\x00\x00\x3f\x00\x00\x00\x03\x08\x01\x1e\x00\x26\x9a\x08\x1c\x15\x08\x19\x35\x04\x00\x06\x19\x18\x04\x76\x61\x6c\x32\x15\x02\x16\x1e\x16\x84\x01\x16\x8c\x01\x26\xb6\x07\x26\x8e\x07\x1c\x18\x04\x00\x00\x00\x3f\x18\x04\x00\x00\x00\x3f\x16\x00\x28\x04\x00\x00\x00\x3f\x18\x04\x00\x00\x00\x3f\x00\x00\x00\x15\x04\x15\x08\x15\x0c\x4c\x15\x02\x15\x04\x12\x00\x00\x04\x0c\x01\x00\x00\x00\x15\x00\x15\x06\x15\x0a\x2c\x15\x1e\x15\x04\x15\x06\x15\x06\x1c\x36\x00\x28\x04\x01\x00\x00\x00\x18\x04\x01\x00\x00\x00\x00\x00\x00\x03\x08\x01\x1e\x00\x26\x8e\x0a\x1c\x15\x02\x19\x35\x04\x00\x06\x19\x18\x04\x76\x61\x6c\x33\x15\x02\x16\x1e\x16\x6c\x16\x74\x26\xc2\x09\x26\x9a\x09\x1c\x36\x00\x28\x04\x01\x00\x00\x00\x18\x04\x01\x00\x00\x00\x00\x00\x00\x15\x02\x19\x6c\x35\x00\x18\x06\x73\x63\x68\x65\x6d\x61\x15\x0a\x00\x15\x04\x25\x00\x18\x02\x69\x64\x00\x15\x02\x25\x00\x18\x07\x62\x6c\x6f\x63\x6b\x4e\x6f\x25\x18\x4c\xac\x13\x10\x12\x00\x00\x00\x15\x0c\x25\x00\x18\x04\x76\x61\x6c\x31\x25\x00\x4c\x1c\x00\x00\x00\x15\x08\x25\x00\x18\x04\x76\x61\x6c\x32\x00\x15\x02\x25\x00\x18\x04\x76\x61\x6c\x33\x25\x16\x4c\xac\x13\x08\x12\x00\x00\x00\x16\x1e\x19\x1c\x19\x5c\x26\xca\x02\x1c\x15\x04\x19\x35\x04\x00\x06\x19\x18\x02\x69\x64\x15\x02\x16\x1e\x16\x9e\x03\x16\xc2\x02\x26\xb8\x01\x26\x08\x1c\x18\x08\x0f\x00\x00\x00\x00\x00\x00\x00\x18\x08\x01\x00\x00\x00\x00\x00\x00\x00\x16\x00\x28\x08\x0f\x00\x00\x00\x00\x00\x00\x00\x18\x08\x01\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x26\xd8\x04\x1c\x15\x02\x19\x35\x04\x00\x06\x19\x18\x07\x62\x6c\x6f\x63\x6b\x4e\x6f\x15\x02\x16\x1e\x16\x6c\x16\x74\x26\x8c\x04\x26\xe4\x03\x1c\x36\x00\x28\x04\x00\x00\x00\x00\x18\x04\x00\x00\x00\x00\x00\x00\x00\x26\xb2\x06\x1c\x15\x0c\x19\x35\x04\x00\x06\x19\x18\x04\x76\x61\x6c\x31\x15\x02\x16\x1e\x16\x68\x16\x70\x26\xee\x05\x26\xc2\x05\x1c\x36\x00\x28\x02\x41\x4d\x18\x02\x41\x4d\x00\x00\x00\x26\x9a\x08\x1c\x15\x08\x19\x35\x04\x00\x06\x19\x18\x04\x76\x61\x6c\x32\x15\x02\x16\x1e\x16\x84\x01\x16\x8c\x01\x26\xb6\x07\x26\x8e\x07\x1c\x18\x04\x00\x00\x00\x3f\x18\x04\x00\x00\x00\x3f\x16\x00\x28\x04\x00\x00\x00\x3f\x18\x04\x00\x00\x00\x3f\x00\x00\x00\x26\x8e\x0a\x1c\x15\x02\x19\x35\x04\x00\x06\x19\x18\x04\x76\x61\x6c\x33\x15\x02\x16\x1e\x16\x6c\x16\x74\x26\xc2\x09\x26\x9a\x09\x1c\x36\x00\x28\x04\x01\x00\x00\x00\x18\x04\x01\x00\x00\x00\x00\x00\x00\x16\xa6\x06\x16\x1e\x00\x28\x22\x70\x61\x72\x71\x75\x65\x74\x2d\x63\x70\x70\x20\x76\x65\x72\x73\x69\x6f\x6e\x20\x31\x2e\x35\x2e\x31\x2d\x53\x4e\x41\x50\x53\x48\x4f\x54\x19\x5c\x1c\x00\x00\x1c\x00\x00\x1c\x00\x00\x1c\x00\x00\x1c\x00\x00\x00\xc5\x01\x00\x00\x50\x41\x52\x31', + # # b'\x50\x41\x52\x31\x15\x04\x15\x10\x15\x14\x4c\x15\x02\x15\x04\x12\x00\x00\x08\x1c\x00\x00\x00\x00\x00\x00\x00\x00\x15\x00\x15\x06\x15\x0a\x2c\x15\x02\x15\x04\x15\x06\x15\x06\x1c\x18\x08\x00\x00\x00\x00\x00\x00\x00\x00\x18\x08\x00\x00\x00\x00\x00\x00\x00\x00\x16\x00\x28\x08\x00\x00\x00\x00\x00\x00\x00\x00\x18\x08\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x03\x08\x01\x02\x00\x26\xbc\x01\x1c\x15\x04\x19\x35\x04\x00\x06\x19\x18\x02\x69\x64\x15\x02\x16\x02\x16\xac\x01\x16\xb4\x01\x26\x38\x26\x08\x1c\x18\x08\x00\x00\x00\x00\x00\x00\x00\x00\x18\x08\x00\x00\x00\x00\x00\x00\x00\x00\x16\x00\x28\x08\x00\x00\x00\x00\x00\x00\x00\x00\x18\x08\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x15\x04\x15\x08\x15\x0c\x4c\x15\x02\x15\x04\x12\x00\x00\x04\x0c\x00\x00\x00\x00\x15\x00\x15\x06\x15\x0a\x2c\x15\x02\x15\x04\x15\x06\x15\x06\x1c\x36\x00\x28\x04\x00\x00\x00\x00\x18\x04\x00\x00\x00\x00\x00\x00\x00\x03\x08\x01\x02\x00\x26\xc8\x03\x1c\x15\x02\x19\x35\x04\x00\x06\x19\x18\x07\x62\x6c\x6f\x63\x6b\x4e\x6f\x15\x02\x16\x02\x16\x6c\x16\x74\x26\xfc\x02\x26\xd4\x02\x1c\x36\x00\x28\x04\x00\x00\x00\x00\x18\x04\x00\x00\x00\x00\x00\x00\x00\x15\x04\x15\x0c\x15\x10\x4c\x15\x02\x15\x04\x12\x00\x00\x06\x14\x02\x00\x00\x00\x41\x4d\x15\x00\x15\x06\x15\x0a\x2c\x15\x02\x15\x04\x15\x06\x15\x06\x1c\x36\x00\x28\x02\x41\x4d\x18\x02\x41\x4d\x00\x00\x00\x03\x08\x01\x02\x00\x26\xa2\x05\x1c\x15\x0c\x19\x35\x04\x00\x06\x19\x18\x04\x76\x61\x6c\x31\x15\x02\x16\x02\x16\x68\x16\x70\x26\xde\x04\x26\xb2\x04\x1c\x36\x00\x28\x02\x41\x4d\x18\x02\x41\x4d\x00\x00\x00\x15\x04\x15\x08\x15\x0c\x4c\x15\x02\x15\x04\x12\x00\x00\x04\x0c\x00\x00\x00\x3f\x15\x00\x15\x06\x15\x0a\x2c\x15\x02\x15\x04\x15\x06\x15\x06\x1c\x18\x04\x00\x00\x00\x3f\x18\x04\x00\x00\x00\x3f\x16\x00\x28\x04\x00\x00\x00\x3f\x18\x04\x00\x00\x00\x3f\x00\x00\x00\x03\x08\x01\x02\x00\x26\x8a\x07\x1c\x15\x08\x19\x35\x04\x00\x06\x19\x18\x04\x76\x61\x6c\x32\x15\x02\x16\x02\x16\x84\x01\x16\x8c\x01\x26\xa6\x06\x26\xfe\x05\x1c\x18\x04\x00\x00\x00\x3f\x18\x04\x00\x00\x00\x3f\x16\x00\x28\x04\x00\x00\x00\x3f\x18\x04\x00\x00\x00\x3f\x00\x00\x00\x15\x04\x15\x08\x15\x0c\x4c\x15\x02\x15\x04\x12\x00\x00\x04\x0c\x01\x00\x00\x00\x15\x00\x15\x06\x15\x0a\x2c\x15\x02\x15\x04\x15\x06\x15\x06\x1c\x36\x00\x28\x04\x01\x00\x00\x00\x18\x04\x01\x00\x00\x00\x00\x00\x00\x03\x08\x01\x02\x00\x26\xfe\x08\x1c\x15\x02\x19\x35\x04\x00\x06\x19\x18\x04\x76\x61\x6c\x33\x15\x02\x16\x02\x16\x6c\x16\x74\x26\xb2\x08\x26\x8a\x08\x1c\x36\x00\x28\x04\x01\x00\x00\x00\x18\x04\x01\x00\x00\x00\x00\x00\x00\x15\x02\x19\x6c\x35\x00\x18\x06\x73\x63\x68\x65\x6d\x61\x15\x0a\x00\x15\x04\x25\x00\x18\x02\x69\x64\x00\x15\x02\x25\x00\x18\x07\x62\x6c\x6f\x63\x6b\x4e\x6f\x25\x18\x4c\xac\x13\x10\x12\x00\x00\x00\x15\x0c\x25\x00\x18\x04\x76\x61\x6c\x31\x25\x00\x4c\x1c\x00\x00\x00\x15\x08\x25\x00\x18\x04\x76\x61\x6c\x32\x00\x15\x02\x25\x00\x18\x04\x76\x61\x6c\x33\x25\x16\x4c\xac\x13\x08\x12\x00\x00\x00\x16\x02\x19\x1c\x19\x5c\x26\xbc\x01\x1c\x15\x04\x19\x35\x04\x00\x06\x19\x18\x02\x69\x64\x15\x02\x16\x02\x16\xac\x01\x16\xb4\x01\x26\x38\x26\x08\x1c\x18\x08\x00\x00\x00\x00\x00\x00\x00\x00\x18\x08\x00\x00\x00\x00\x00\x00\x00\x00\x16\x00\x28\x08\x00\x00\x00\x00\x00\x00\x00\x00\x18\x08\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x26\xc8\x03\x1c\x15\x02\x19\x35\x04\x00\x06\x19\x18\x07\x62\x6c\x6f\x63\x6b\x4e\x6f\x15\x02\x16\x02\x16\x6c\x16\x74\x26\xfc\x02\x26\xd4\x02\x1c\x36\x00\x28\x04\x00\x00\x00\x00\x18\x04\x00\x00\x00\x00\x00\x00\x00\x26\xa2\x05\x1c\x15\x0c\x19\x35\x04\x00\x06\x19\x18\x04\x76\x61\x6c\x31\x15\x02\x16\x02\x16\x68\x16\x70\x26\xde\x04\x26\xb2\x04\x1c\x36\x00\x28\x02\x41\x4d\x18\x02\x41\x4d\x00\x00\x00\x26\x8a\x07\x1c\x15\x08\x19\x35\x04\x00\x06\x19\x18\x04\x76\x61\x6c\x32\x15\x02\x16\x02\x16\x84\x01\x16\x8c\x01\x26\xa6\x06\x26\xfe\x05\x1c\x18\x04\x00\x00\x00\x3f\x18\x04\x00\x00\x00\x3f\x16\x00\x28\x04\x00\x00\x00\x3f\x18\x04\x00\x00\x00\x3f\x00\x00\x00\x26\xfe\x08\x1c\x15\x02\x19\x35\x04\x00\x06\x19\x18\x04\x76\x61\x6c\x33\x15\x02\x16\x02\x16\x6c\x16\x74\x26\xb2\x08\x26\x8a\x08\x1c\x36\x00\x28\x04\x01\x00\x00\x00\x18\x04\x01\x00\x00\x00\x00\x00\x00\x16\x98\x05\x16\x02\x00\x28\x22\x70\x61\x72\x71\x75\x65\x74\x2d\x63\x70\x70\x20\x76\x65\x72\x73\x69\x6f\x6e\x20\x31\x2e\x35\x2e\x31\x2d\x53\x4e\x41\x50\x53\x48\x4f\x54\x19\x5c\x1c\x00\x00\x1c\x00\x00\x1c\x00\x00\x1c\x00\x00\x1c\x00\x00\x00\xc4\x01\x00\x00\x50\x41\x52\x31', # # '' # # ], # }, # 'Avro' : { # 'data_sample' : [ - # '\x4f\x62\x6a\x01\x04\x16\x61\x76\x72\x6f\x2e\x73\x63\x68\x65\x6d\x61\x82\x03\x7b\x22\x74\x79\x70\x65\x22\x3a\x22\x72\x65\x63\x6f\x72\x64\x22\x2c\x22\x6e\x61\x6d\x65\x22\x3a\x22\x72\x6f\x77\x22\x2c\x22\x66\x69\x65\x6c\x64\x73\x22\x3a\x5b\x7b\x22\x6e\x61\x6d\x65\x22\x3a\x22\x69\x64\x22\x2c\x22\x74\x79\x70\x65\x22\x3a\x22\x6c\x6f\x6e\x67\x22\x7d\x2c\x7b\x22\x6e\x61\x6d\x65\x22\x3a\x22\x62\x6c\x6f\x63\x6b\x4e\x6f\x22\x2c\x22\x74\x79\x70\x65\x22\x3a\x22\x69\x6e\x74\x22\x7d\x2c\x7b\x22\x6e\x61\x6d\x65\x22\x3a\x22\x76\x61\x6c\x31\x22\x2c\x22\x74\x79\x70\x65\x22\x3a\x22\x73\x74\x72\x69\x6e\x67\x22\x7d\x2c\x7b\x22\x6e\x61\x6d\x65\x22\x3a\x22\x76\x61\x6c\x32\x22\x2c\x22\x74\x79\x70\x65\x22\x3a\x22\x66\x6c\x6f\x61\x74\x22\x7d\x2c\x7b\x22\x6e\x61\x6d\x65\x22\x3a\x22\x76\x61\x6c\x33\x22\x2c\x22\x74\x79\x70\x65\x22\x3a\x22\x69\x6e\x74\x22\x7d\x5d\x7d\x14\x61\x76\x72\x6f\x2e\x63\x6f\x64\x65\x63\x08\x6e\x75\x6c\x6c\x00\x8d\x1f\xf2\x17\x71\xa4\x2e\xe4\xc9\x0a\x23\x67\x12\xaa\xc6\xc0\x02\x14\x00\x00\x04\x41\x4d\x00\x00\x00\x3f\x02\x8d\x1f\xf2\x17\x71\xa4\x2e\xe4\xc9\x0a\x23\x67\x12\xaa\xc6\xc0', - # '\x4f\x62\x6a\x01\x04\x16\x61\x76\x72\x6f\x2e\x73\x63\x68\x65\x6d\x61\x82\x03\x7b\x22\x74\x79\x70\x65\x22\x3a\x22\x72\x65\x63\x6f\x72\x64\x22\x2c\x22\x6e\x61\x6d\x65\x22\x3a\x22\x72\x6f\x77\x22\x2c\x22\x66\x69\x65\x6c\x64\x73\x22\x3a\x5b\x7b\x22\x6e\x61\x6d\x65\x22\x3a\x22\x69\x64\x22\x2c\x22\x74\x79\x70\x65\x22\x3a\x22\x6c\x6f\x6e\x67\x22\x7d\x2c\x7b\x22\x6e\x61\x6d\x65\x22\x3a\x22\x62\x6c\x6f\x63\x6b\x4e\x6f\x22\x2c\x22\x74\x79\x70\x65\x22\x3a\x22\x69\x6e\x74\x22\x7d\x2c\x7b\x22\x6e\x61\x6d\x65\x22\x3a\x22\x76\x61\x6c\x31\x22\x2c\x22\x74\x79\x70\x65\x22\x3a\x22\x73\x74\x72\x69\x6e\x67\x22\x7d\x2c\x7b\x22\x6e\x61\x6d\x65\x22\x3a\x22\x76\x61\x6c\x32\x22\x2c\x22\x74\x79\x70\x65\x22\x3a\x22\x66\x6c\x6f\x61\x74\x22\x7d\x2c\x7b\x22\x6e\x61\x6d\x65\x22\x3a\x22\x76\x61\x6c\x33\x22\x2c\x22\x74\x79\x70\x65\x22\x3a\x22\x69\x6e\x74\x22\x7d\x5d\x7d\x14\x61\x76\x72\x6f\x2e\x63\x6f\x64\x65\x63\x08\x6e\x75\x6c\x6c\x00\xeb\x9d\x51\x82\xf2\x11\x3d\x0b\xc5\x92\x97\xb2\x07\x6d\x72\x5a\x1e\xac\x02\x02\x00\x04\x41\x4d\x00\x00\x00\x3f\x02\x04\x00\x04\x41\x4d\x00\x00\x00\x3f\x02\x06\x00\x04\x41\x4d\x00\x00\x00\x3f\x02\x08\x00\x04\x41\x4d\x00\x00\x00\x3f\x02\x0a\x00\x04\x41\x4d\x00\x00\x00\x3f\x02\x0c\x00\x04\x41\x4d\x00\x00\x00\x3f\x02\x0e\x00\x04\x41\x4d\x00\x00\x00\x3f\x02\x10\x00\x04\x41\x4d\x00\x00\x00\x3f\x02\x12\x00\x04\x41\x4d\x00\x00\x00\x3f\x02\x14\x00\x04\x41\x4d\x00\x00\x00\x3f\x02\x16\x00\x04\x41\x4d\x00\x00\x00\x3f\x02\x18\x00\x04\x41\x4d\x00\x00\x00\x3f\x02\x1a\x00\x04\x41\x4d\x00\x00\x00\x3f\x02\x1c\x00\x04\x41\x4d\x00\x00\x00\x3f\x02\x1e\x00\x04\x41\x4d\x00\x00\x00\x3f\x02\xeb\x9d\x51\x82\xf2\x11\x3d\x0b\xc5\x92\x97\xb2\x07\x6d\x72\x5a', - # '\x4f\x62\x6a\x01\x04\x16\x61\x76\x72\x6f\x2e\x73\x63\x68\x65\x6d\x61\x82\x03\x7b\x22\x74\x79\x70\x65\x22\x3a\x22\x72\x65\x63\x6f\x72\x64\x22\x2c\x22\x6e\x61\x6d\x65\x22\x3a\x22\x72\x6f\x77\x22\x2c\x22\x66\x69\x65\x6c\x64\x73\x22\x3a\x5b\x7b\x22\x6e\x61\x6d\x65\x22\x3a\x22\x69\x64\x22\x2c\x22\x74\x79\x70\x65\x22\x3a\x22\x6c\x6f\x6e\x67\x22\x7d\x2c\x7b\x22\x6e\x61\x6d\x65\x22\x3a\x22\x62\x6c\x6f\x63\x6b\x4e\x6f\x22\x2c\x22\x74\x79\x70\x65\x22\x3a\x22\x69\x6e\x74\x22\x7d\x2c\x7b\x22\x6e\x61\x6d\x65\x22\x3a\x22\x76\x61\x6c\x31\x22\x2c\x22\x74\x79\x70\x65\x22\x3a\x22\x73\x74\x72\x69\x6e\x67\x22\x7d\x2c\x7b\x22\x6e\x61\x6d\x65\x22\x3a\x22\x76\x61\x6c\x32\x22\x2c\x22\x74\x79\x70\x65\x22\x3a\x22\x66\x6c\x6f\x61\x74\x22\x7d\x2c\x7b\x22\x6e\x61\x6d\x65\x22\x3a\x22\x76\x61\x6c\x33\x22\x2c\x22\x74\x79\x70\x65\x22\x3a\x22\x69\x6e\x74\x22\x7d\x5d\x7d\x14\x61\x76\x72\x6f\x2e\x63\x6f\x64\x65\x63\x08\x6e\x75\x6c\x6c\x00\x73\x65\x4f\x7c\xd9\x33\xe1\x18\xdd\x30\xe8\x22\x2a\x58\x20\x6f\x02\x14\x00\x00\x04\x41\x4d\x00\x00\x00\x3f\x02\x73\x65\x4f\x7c\xd9\x33\xe1\x18\xdd\x30\xe8\x22\x2a\x58\x20\x6f', + # b'\x4f\x62\x6a\x01\x04\x16\x61\x76\x72\x6f\x2e\x73\x63\x68\x65\x6d\x61\x82\x03\x7b\x22\x74\x79\x70\x65\x22\x3a\x22\x72\x65\x63\x6f\x72\x64\x22\x2c\x22\x6e\x61\x6d\x65\x22\x3a\x22\x72\x6f\x77\x22\x2c\x22\x66\x69\x65\x6c\x64\x73\x22\x3a\x5b\x7b\x22\x6e\x61\x6d\x65\x22\x3a\x22\x69\x64\x22\x2c\x22\x74\x79\x70\x65\x22\x3a\x22\x6c\x6f\x6e\x67\x22\x7d\x2c\x7b\x22\x6e\x61\x6d\x65\x22\x3a\x22\x62\x6c\x6f\x63\x6b\x4e\x6f\x22\x2c\x22\x74\x79\x70\x65\x22\x3a\x22\x69\x6e\x74\x22\x7d\x2c\x7b\x22\x6e\x61\x6d\x65\x22\x3a\x22\x76\x61\x6c\x31\x22\x2c\x22\x74\x79\x70\x65\x22\x3a\x22\x73\x74\x72\x69\x6e\x67\x22\x7d\x2c\x7b\x22\x6e\x61\x6d\x65\x22\x3a\x22\x76\x61\x6c\x32\x22\x2c\x22\x74\x79\x70\x65\x22\x3a\x22\x66\x6c\x6f\x61\x74\x22\x7d\x2c\x7b\x22\x6e\x61\x6d\x65\x22\x3a\x22\x76\x61\x6c\x33\x22\x2c\x22\x74\x79\x70\x65\x22\x3a\x22\x69\x6e\x74\x22\x7d\x5d\x7d\x14\x61\x76\x72\x6f\x2e\x63\x6f\x64\x65\x63\x08\x6e\x75\x6c\x6c\x00\x8d\x1f\xf2\x17\x71\xa4\x2e\xe4\xc9\x0a\x23\x67\x12\xaa\xc6\xc0\x02\x14\x00\x00\x04\x41\x4d\x00\x00\x00\x3f\x02\x8d\x1f\xf2\x17\x71\xa4\x2e\xe4\xc9\x0a\x23\x67\x12\xaa\xc6\xc0', + # b'\x4f\x62\x6a\x01\x04\x16\x61\x76\x72\x6f\x2e\x73\x63\x68\x65\x6d\x61\x82\x03\x7b\x22\x74\x79\x70\x65\x22\x3a\x22\x72\x65\x63\x6f\x72\x64\x22\x2c\x22\x6e\x61\x6d\x65\x22\x3a\x22\x72\x6f\x77\x22\x2c\x22\x66\x69\x65\x6c\x64\x73\x22\x3a\x5b\x7b\x22\x6e\x61\x6d\x65\x22\x3a\x22\x69\x64\x22\x2c\x22\x74\x79\x70\x65\x22\x3a\x22\x6c\x6f\x6e\x67\x22\x7d\x2c\x7b\x22\x6e\x61\x6d\x65\x22\x3a\x22\x62\x6c\x6f\x63\x6b\x4e\x6f\x22\x2c\x22\x74\x79\x70\x65\x22\x3a\x22\x69\x6e\x74\x22\x7d\x2c\x7b\x22\x6e\x61\x6d\x65\x22\x3a\x22\x76\x61\x6c\x31\x22\x2c\x22\x74\x79\x70\x65\x22\x3a\x22\x73\x74\x72\x69\x6e\x67\x22\x7d\x2c\x7b\x22\x6e\x61\x6d\x65\x22\x3a\x22\x76\x61\x6c\x32\x22\x2c\x22\x74\x79\x70\x65\x22\x3a\x22\x66\x6c\x6f\x61\x74\x22\x7d\x2c\x7b\x22\x6e\x61\x6d\x65\x22\x3a\x22\x76\x61\x6c\x33\x22\x2c\x22\x74\x79\x70\x65\x22\x3a\x22\x69\x6e\x74\x22\x7d\x5d\x7d\x14\x61\x76\x72\x6f\x2e\x63\x6f\x64\x65\x63\x08\x6e\x75\x6c\x6c\x00\xeb\x9d\x51\x82\xf2\x11\x3d\x0b\xc5\x92\x97\xb2\x07\x6d\x72\x5a\x1e\xac\x02\x02\x00\x04\x41\x4d\x00\x00\x00\x3f\x02\x04\x00\x04\x41\x4d\x00\x00\x00\x3f\x02\x06\x00\x04\x41\x4d\x00\x00\x00\x3f\x02\x08\x00\x04\x41\x4d\x00\x00\x00\x3f\x02\x0a\x00\x04\x41\x4d\x00\x00\x00\x3f\x02\x0c\x00\x04\x41\x4d\x00\x00\x00\x3f\x02\x0e\x00\x04\x41\x4d\x00\x00\x00\x3f\x02\x10\x00\x04\x41\x4d\x00\x00\x00\x3f\x02\x12\x00\x04\x41\x4d\x00\x00\x00\x3f\x02\x14\x00\x04\x41\x4d\x00\x00\x00\x3f\x02\x16\x00\x04\x41\x4d\x00\x00\x00\x3f\x02\x18\x00\x04\x41\x4d\x00\x00\x00\x3f\x02\x1a\x00\x04\x41\x4d\x00\x00\x00\x3f\x02\x1c\x00\x04\x41\x4d\x00\x00\x00\x3f\x02\x1e\x00\x04\x41\x4d\x00\x00\x00\x3f\x02\xeb\x9d\x51\x82\xf2\x11\x3d\x0b\xc5\x92\x97\xb2\x07\x6d\x72\x5a', + # b'\x4f\x62\x6a\x01\x04\x16\x61\x76\x72\x6f\x2e\x73\x63\x68\x65\x6d\x61\x82\x03\x7b\x22\x74\x79\x70\x65\x22\x3a\x22\x72\x65\x63\x6f\x72\x64\x22\x2c\x22\x6e\x61\x6d\x65\x22\x3a\x22\x72\x6f\x77\x22\x2c\x22\x66\x69\x65\x6c\x64\x73\x22\x3a\x5b\x7b\x22\x6e\x61\x6d\x65\x22\x3a\x22\x69\x64\x22\x2c\x22\x74\x79\x70\x65\x22\x3a\x22\x6c\x6f\x6e\x67\x22\x7d\x2c\x7b\x22\x6e\x61\x6d\x65\x22\x3a\x22\x62\x6c\x6f\x63\x6b\x4e\x6f\x22\x2c\x22\x74\x79\x70\x65\x22\x3a\x22\x69\x6e\x74\x22\x7d\x2c\x7b\x22\x6e\x61\x6d\x65\x22\x3a\x22\x76\x61\x6c\x31\x22\x2c\x22\x74\x79\x70\x65\x22\x3a\x22\x73\x74\x72\x69\x6e\x67\x22\x7d\x2c\x7b\x22\x6e\x61\x6d\x65\x22\x3a\x22\x76\x61\x6c\x32\x22\x2c\x22\x74\x79\x70\x65\x22\x3a\x22\x66\x6c\x6f\x61\x74\x22\x7d\x2c\x7b\x22\x6e\x61\x6d\x65\x22\x3a\x22\x76\x61\x6c\x33\x22\x2c\x22\x74\x79\x70\x65\x22\x3a\x22\x69\x6e\x74\x22\x7d\x5d\x7d\x14\x61\x76\x72\x6f\x2e\x63\x6f\x64\x65\x63\x08\x6e\x75\x6c\x6c\x00\x73\x65\x4f\x7c\xd9\x33\xe1\x18\xdd\x30\xe8\x22\x2a\x58\x20\x6f\x02\x14\x00\x00\x04\x41\x4d\x00\x00\x00\x3f\x02\x73\x65\x4f\x7c\xd9\x33\xe1\x18\xdd\x30\xe8\x22\x2a\x58\x20\x6f', # ], # }, 'AvroConfluent': { 'data_sample': [ avro_confluent_message(cluster.schema_registry_client, - {'id': 0L, 'blockNo': 0, 'val1': unicode('AM'), 'val2': 0.5, "val3": 1}), + {'id': 0, 'blockNo': 0, 'val1': str('AM'), 'val2': 0.5, "val3": 1}), - ''.join(map(lambda id: avro_confluent_message(cluster.schema_registry_client, - {'id': id, 'blockNo': 0, 'val1': unicode('AM'), - 'val2': 0.5, "val3": 1}), range(1, 16))), + b''.join([avro_confluent_message(cluster.schema_registry_client, + {'id': id, 'blockNo': 0, 'val1': str('AM'), + 'val2': 0.5, "val3": 1}) for id in range(1, 16)]), avro_confluent_message(cluster.schema_registry_client, - {'id': 0L, 'blockNo': 0, 'val1': unicode('AM'), 'val2': 0.5, "val3": 1}), + {'id': 0, 'blockNo': 0, 'val1': str('AM'), 'val2': 0.5, "val3": 1}), ], 'extra_settings': ", format_avro_schema_registry_url='http://{}:{}'".format( cluster.schema_registry_host, @@ -479,8 +478,8 @@ def test_kafka_formats(kafka_cluster): # }, } - for format_name, format_opts in all_formats.items(): - print('Set up {}'.format(format_name)) + for format_name, format_opts in list(all_formats.items()): + print(('Set up {}'.format(format_name))) topic_name = 'format_tests_{}'.format(format_name) data_sample = format_opts['data_sample'] data_prefix = [] @@ -513,8 +512,8 @@ def test_kafka_formats(kafka_cluster): time.sleep(12) - for format_name, format_opts in all_formats.items(): - print('Checking {}'.format(format_name)) + for format_name, format_opts in list(all_formats.items()): + print(('Checking {}'.format(format_name))) topic_name = 'format_tests_{}'.format(format_name) # shift offsets by 1 if format supports empty value offsets = [1, 2, 3] if format_opts.get('supports_empty_value', False) else [0, 1, 2] @@ -588,7 +587,7 @@ def kafka_cluster(): global kafka_id cluster.start() kafka_id = instance.cluster.kafka_docker_id - print("kafka_id is {}".format(kafka_id)) + print(("kafka_id is {}".format(kafka_id))) yield cluster finally: @@ -638,7 +637,7 @@ kafka_topic_old old kafka_check_result(result, True) members = describe_consumer_group('old') - assert members[0]['client_id'] == u'ClickHouse-instance-test-kafka' + assert members[0]['client_id'] == 'ClickHouse-instance-test-kafka' # text_desc = kafka_cluster.exec_in_container(kafka_cluster.get_container_id('kafka1'),"kafka-consumer-groups --bootstrap-server localhost:9092 --describe --members --group old --verbose")) @@ -679,7 +678,7 @@ def test_kafka_settings_new_syntax(kafka_cluster): kafka_check_result(result, True) members = describe_consumer_group('new') - assert members[0]['client_id'] == u'instance test 1234' + assert members[0]['client_id'] == 'instance test 1234' @pytest.mark.timeout(180) @@ -1124,7 +1123,7 @@ def test_kafka_flush_on_big_message(kafka_cluster): while not received: try: offsets = client.list_consumer_group_offsets('flush') - for topic, offset in offsets.items(): + for topic, offset in list(offsets.items()): if topic.topic == 'flush' and offset.offset == kafka_messages: received = True break @@ -1407,19 +1406,19 @@ def test_kafka_virtual_columns2(kafka_cluster): SELECT value, _key, _topic, _partition, _offset, toUnixTimestamp(_timestamp), toUnixTimestamp64Milli(_timestamp_ms), _headers.name, _headers.value FROM test.kafka; ''') - producer = KafkaProducer(bootstrap_servers="localhost:9092") + producer = KafkaProducer(bootstrap_servers="localhost:9092", value_serializer=producer_serializer, key_serializer=producer_serializer) producer.send(topic='virt2_0', value=json.dumps({'value': 1}), partition=0, key='k1', timestamp_ms=1577836801001, headers=[('content-encoding', b'base64')]) producer.send(topic='virt2_0', value=json.dumps({'value': 2}), partition=0, key='k2', timestamp_ms=1577836802002, - headers=[('empty_value', ''), ('', 'empty name'), ('', ''), ('repetition', '1'), ('repetition', '2')]) + headers=[('empty_value', b''), ('', b'empty name'), ('', b''), ('repetition', b'1'), ('repetition', b'2')]) producer.flush() time.sleep(1) producer.send(topic='virt2_0', value=json.dumps({'value': 3}), partition=1, key='k3', timestamp_ms=1577836803003, - headers=[('b', 'b'), ('a', 'a')]) + headers=[('b', b'b'), ('a', b'a')]) producer.send(topic='virt2_0', value=json.dumps({'value': 4}), partition=1, key='k4', timestamp_ms=1577836804004, - headers=[('a', 'a'), ('b', 'b')]) + headers=[('a', b'a'), ('b', b'b')]) producer.flush() time.sleep(1) @@ -1436,8 +1435,8 @@ def test_kafka_virtual_columns2(kafka_cluster): members = describe_consumer_group('virt2') # pprint.pprint(members) - members[0]['client_id'] = u'ClickHouse-instance-test-kafka-0' - members[1]['client_id'] = u'ClickHouse-instance-test-kafka-1' + members[0]['client_id'] = 'ClickHouse-instance-test-kafka-0' + members[1]['client_id'] = 'ClickHouse-instance-test-kafka-1' result = instance.query("SELECT * FROM test.view ORDER BY value", ignore_error=True) @@ -1717,7 +1716,7 @@ def test_kafka_rebalance(kafka_cluster): for consumer_index in range(NUMBER_OF_CONSURRENT_CONSUMERS): table_name = 'kafka_consumer{}'.format(consumer_index) - print("Setting up {}".format(table_name)) + print(("Setting up {}".format(table_name))) instance.query(''' DROP TABLE IF EXISTS test.{0}; @@ -1744,14 +1743,14 @@ def test_kafka_rebalance(kafka_cluster): # kafka_cluster.open_bash_shell('instance') while int( instance.query("SELECT count() FROM test.destination WHERE _consumed_by='{}'".format(table_name))) == 0: - print("Waiting for test.kafka_consumer{} to start consume".format(consumer_index)) + print(("Waiting for test.kafka_consumer{} to start consume".format(consumer_index))) time.sleep(1) cancel.set() # I leave last one working by intent (to finish consuming after all rebalances) for consumer_index in range(NUMBER_OF_CONSURRENT_CONSUMERS - 1): - print("Dropping test.kafka_consumer{}".format(consumer_index)) + print(("Dropping test.kafka_consumer{}".format(consumer_index))) instance.query('DROP TABLE IF EXISTS test.kafka_consumer{}'.format(consumer_index)) while int(instance.query( "SELECT count() FROM system.tables WHERE database='test' AND name='kafka_consumer{}'".format( @@ -1766,9 +1765,9 @@ def test_kafka_rebalance(kafka_cluster): if messages_consumed >= msg_index[0]: break time.sleep(1) - print("Waiting for finishing consuming (have {}, should be {})".format(messages_consumed, msg_index[0])) + print(("Waiting for finishing consuming (have {}, should be {})".format(messages_consumed, msg_index[0]))) - print(instance.query('SELECT count(), uniqExact(key), max(key) + 1 FROM test.destination')) + print((instance.query('SELECT count(), uniqExact(key), max(key) + 1 FROM test.destination'))) # Some queries to debug... # SELECT * FROM test.destination where key in (SELECT key FROM test.destination group by key having count() <> 1) @@ -1793,7 +1792,7 @@ def test_kafka_rebalance(kafka_cluster): result = int(instance.query('SELECT count() == uniqExact(key) FROM test.destination')) for consumer_index in range(NUMBER_OF_CONSURRENT_CONSUMERS): - print("kafka_consumer{}".format(consumer_index)) + print(("kafka_consumer{}".format(consumer_index))) table_name = 'kafka_consumer{}'.format(consumer_index) instance.query(''' DROP TABLE IF EXISTS test.{0}; @@ -2253,5 +2252,5 @@ def test_kafka_csv_with_thread_per_consumer(kafka_cluster): if __name__ == '__main__': cluster.start() - raw_input("Cluster created, press any key to destroy...") + input("Cluster created, press any key to destroy...") cluster.shutdown() diff --git a/tests/integration/test_storage_kerberized_kafka/test.py b/tests/integration/test_storage_kerberized_kafka/test.py index ec23d340977..59fb043b546 100644 --- a/tests/integration/test_storage_kerberized_kafka/test.py +++ b/tests/integration/test_storage_kerberized_kafka/test.py @@ -56,8 +56,10 @@ def wait_kafka_is_available(max_retries=50): time.sleep(1) +def producer_serializer(x): + return x.encode() if isinstance(x, str) else x def kafka_produce(topic, messages, timestamp=None): - producer = KafkaProducer(bootstrap_servers="localhost:9093") + producer = KafkaProducer(bootstrap_servers="localhost:9093", value_serializer=producer_serializer) for message in messages: producer.send(topic=topic, value=message, timestamp_ms=timestamp) producer.flush() @@ -142,5 +144,5 @@ def test_kafka_json_as_string_no_kdc(kafka_cluster): if __name__ == '__main__': cluster.start() - raw_input("Cluster created, press any key to destroy...") + input("Cluster created, press any key to destroy...") cluster.shutdown() diff --git a/tests/integration/test_storage_mysql/test.py b/tests/integration/test_storage_mysql/test.py index 83ef1e6c86a..87033381e2c 100644 --- a/tests/integration/test_storage_mysql/test.py +++ b/tests/integration/test_storage_mysql/test.py @@ -179,6 +179,6 @@ def create_mysql_table(conn, tableName): if __name__ == '__main__': with contextmanager(started_cluster)() as cluster: - for name, instance in cluster.instances.items(): - print name, instance.ip_address - raw_input("Cluster created, press any key to destroy...") + for name, instance in list(cluster.instances.items()): + print(name, instance.ip_address) + input("Cluster created, press any key to destroy...") diff --git a/tests/integration/test_storage_rabbitmq/test.py b/tests/integration/test_storage_rabbitmq/test.py index 4d892eaa72c..ab44d0ebea0 100644 --- a/tests/integration/test_storage_rabbitmq/test.py +++ b/tests/integration/test_storage_rabbitmq/test.py @@ -13,7 +13,7 @@ from helpers.client import QueryRuntimeException from helpers.cluster import ClickHouseCluster from helpers.test_tools import TSV -import rabbitmq_pb2 +from . import rabbitmq_pb2 cluster = ClickHouseCluster(__file__) instance = cluster.add_instance('instance', @@ -103,7 +103,7 @@ def rabbitmq_cluster(): global rabbitmq_id cluster.start() rabbitmq_id = instance.cluster.rabbitmq_docker_id - print("rabbitmq_id is {}".format(rabbitmq_id)) + print(("rabbitmq_id is {}".format(rabbitmq_id))) instance.query('CREATE DATABASE test') yield cluster @@ -957,7 +957,7 @@ def test_rabbitmq_direct_exchange(rabbitmq_cluster): num_tables = 5 for consumer_id in range(num_tables): - print("Setting up table {}".format(consumer_id)) + print(("Setting up table {}".format(consumer_id))) instance.query(''' DROP TABLE IF EXISTS test.direct_exchange_{0}; DROP TABLE IF EXISTS test.direct_exchange_{0}_mv; @@ -1030,7 +1030,7 @@ def test_rabbitmq_fanout_exchange(rabbitmq_cluster): num_tables = 5 for consumer_id in range(num_tables): - print("Setting up table {}".format(consumer_id)) + print(("Setting up table {}".format(consumer_id))) instance.query(''' DROP TABLE IF EXISTS test.fanout_exchange_{0}; DROP TABLE IF EXISTS test.fanout_exchange_{0}_mv; @@ -1097,7 +1097,7 @@ def test_rabbitmq_topic_exchange(rabbitmq_cluster): num_tables = 5 for consumer_id in range(num_tables): - print("Setting up table {}".format(consumer_id)) + print(("Setting up table {}".format(consumer_id))) instance.query(''' DROP TABLE IF EXISTS test.topic_exchange_{0}; DROP TABLE IF EXISTS test.topic_exchange_{0}_mv; @@ -1116,7 +1116,7 @@ def test_rabbitmq_topic_exchange(rabbitmq_cluster): '''.format(consumer_id)) for consumer_id in range(num_tables): - print("Setting up table {}".format(num_tables + consumer_id)) + print(("Setting up table {}".format(num_tables + consumer_id))) instance.query(''' DROP TABLE IF EXISTS test.topic_exchange_{0}; DROP TABLE IF EXISTS test.topic_exchange_{0}_mv; @@ -1195,7 +1195,7 @@ def test_rabbitmq_hash_exchange(rabbitmq_cluster): num_tables = 4 for consumer_id in range(num_tables): table_name = 'rabbitmq_consumer{}'.format(consumer_id) - print("Setting up {}".format(table_name)) + print(("Setting up {}".format(table_name))) instance.query(''' DROP TABLE IF EXISTS test.{0}; DROP TABLE IF EXISTS test.{0}_mv; @@ -1353,7 +1353,7 @@ def test_rabbitmq_headers_exchange(rabbitmq_cluster): num_tables_to_receive = 2 for consumer_id in range(num_tables_to_receive): - print("Setting up table {}".format(consumer_id)) + print(("Setting up table {}".format(consumer_id))) instance.query(''' DROP TABLE IF EXISTS test.headers_exchange_{0}; DROP TABLE IF EXISTS test.headers_exchange_{0}_mv; @@ -1372,7 +1372,7 @@ def test_rabbitmq_headers_exchange(rabbitmq_cluster): num_tables_to_ignore = 2 for consumer_id in range(num_tables_to_ignore): - print("Setting up table {}".format(consumer_id + num_tables_to_receive)) + print(("Setting up table {}".format(consumer_id + num_tables_to_receive))) instance.query(''' DROP TABLE IF EXISTS test.headers_exchange_{0}; DROP TABLE IF EXISTS test.headers_exchange_{0}_mv; @@ -1570,7 +1570,7 @@ def test_rabbitmq_many_consumers_to_each_queue(rabbitmq_cluster): num_tables = 4 for table_id in range(num_tables): - print("Setting up table {}".format(table_id)) + print(("Setting up table {}".format(table_id))) instance.query(''' DROP TABLE IF EXISTS test.many_consumers_{0}; DROP TABLE IF EXISTS test.many_consumers_{0}_mv; @@ -1864,5 +1864,5 @@ def test_rabbitmq_commit_on_block_write(rabbitmq_cluster): if __name__ == '__main__': cluster.start() - raw_input("Cluster created, press any key to destroy...") + input("Cluster created, press any key to destroy...") cluster.shutdown() diff --git a/tests/integration/test_storage_s3/s3_mock/mock_s3.py b/tests/integration/test_storage_s3/s3_mock/mock_s3.py index 5b422f6a73a..d76fe4ac36d 100644 --- a/tests/integration/test_storage_s3/s3_mock/mock_s3.py +++ b/tests/integration/test_storage_s3/s3_mock/mock_s3.py @@ -11,7 +11,7 @@ def infinite_redirect(_path): @route('/<_bucket>/<_path>') def server(_bucket, _path): for name in request.headers: - if name == 'Authorization' and request.headers[name] == u'Bearer TOKEN': + if name == 'Authorization' and request.headers[name] == 'Bearer TOKEN': return '1, 2, 3' abort(403) diff --git a/tests/integration/test_storage_s3/test.py b/tests/integration/test_storage_s3/test.py index f752e72c677..42dbeda4717 100644 --- a/tests/integration/test_storage_s3/test.py +++ b/tests/integration/test_storage_s3/test.py @@ -2,8 +2,8 @@ import gzip import json import logging import os +import io import random -import StringIO import threading import time @@ -61,18 +61,20 @@ def prepare_s3_bucket(cluster): def put_s3_file_content(cluster, bucket, filename, data): - buf = StringIO.StringIO(data) + buf = io.BytesIO(data) cluster.minio_client.put_object(bucket, filename, buf, len(data)) # Returns content of given S3 file as string. -def get_s3_file_content(cluster, bucket, filename): +def get_s3_file_content(cluster, bucket, filename, decode=True): # type: (ClickHouseCluster, str) -> str data = cluster.minio_client.get_object(bucket, filename) - data_str = "" + data_str = b"" for chunk in data.stream(): data_str += chunk + if decode: + return data_str.decode() return data_str @@ -231,7 +233,7 @@ def test_multipart_put(cluster, maybe_auth, positive): one_line_length = 6 # 3 digits, 2 commas, 1 line separator. # Generate data having size more than one part - int_data = [[1, 2, 3] for i in range(csv_size_bytes / one_line_length)] + int_data = [[1, 2, 3] for i in range(csv_size_bytes // one_line_length)] csv_data = "".join(["{},{},{}\n".format(x, y, z) for x, y, z in int_data]) assert len(csv_data) > min_part_size_bytes @@ -377,9 +379,9 @@ def test_storage_s3_get_gzip(cluster): "Norman Ortega,33", "" ] - buf = StringIO.StringIO() + buf = io.BytesIO() compressed = gzip.GzipFile(fileobj=buf, mode="wb") - compressed.write("\n".join(data)) + compressed.write(("\n".join(data)).encode()) compressed.close() put_s3_file_content(cluster, bucket, filename, buf.getvalue()) @@ -459,9 +461,9 @@ def test_storage_s3_put_gzip(cluster): run_query(instance, "SELECT sum(id) FROM {}".format(name)).splitlines() == ["708"] - buf = StringIO.StringIO(get_s3_file_content(cluster, bucket, filename)) + buf = io.BytesIO(get_s3_file_content(cluster, bucket, filename, decode=False)) f = gzip.GzipFile(fileobj=buf, mode="rb") - uncompressed_content = f.read() + uncompressed_content = f.read().decode() assert sum([ int(i.split(',')[1]) for i in uncompressed_content.splitlines() ]) == 708 finally: run_query(instance, "DROP TABLE {}".format(name)) diff --git a/tests/integration/test_system_queries/test.py b/tests/integration/test_system_queries/test.py index 18a164da805..7f5bce97805 100644 --- a/tests/integration/test_system_queries/test.py +++ b/tests/integration/test_system_queries/test.py @@ -26,9 +26,6 @@ def started_cluster(): instance = cluster.instances['ch1'] instance.query('CREATE DATABASE dictionaries ENGINE = Dictionary') instance.query('CREATE TABLE dictionary_source (id UInt64, value UInt8) ENGINE = Memory') - print instance.query('SELECT * FROM system.dictionaries FORMAT Vertical') - print "Started ", instance.ip_address - yield cluster finally: @@ -136,6 +133,6 @@ def test_SYSTEM_FLUSH_LOGS(started_cluster): if __name__ == '__main__': with contextmanager(started_cluster)() as cluster: - for name, instance in cluster.instances.items(): - print name, instance.ip_address - raw_input("Cluster created, press any key to destroy...") + for name, instance in list(cluster.instances.items()): + print(name, instance.ip_address) + input("Cluster created, press any key to destroy...") diff --git a/tests/integration/test_ttl_move/test.py b/tests/integration/test_ttl_move/test.py index 377ee0e5d75..751d15b8313 100644 --- a/tests/integration/test_ttl_move/test.py +++ b/tests/integration/test_ttl_move/test.py @@ -37,14 +37,6 @@ def started_cluster(): cluster.shutdown() -def get_random_string(length): - symbols = bytes(string.ascii_uppercase + string.digits) - result_list = bytearray([0]) * length - for i in range(length): - result_list[i] = random.choice(symbols) - return str(result_list) - - def get_used_disks_for_table(node, table_name, partition=None): if partition is None: suffix = "" @@ -150,8 +142,8 @@ def test_inserts_to_disk_work(started_cluster, name, engine, positive): data = [] # 10MB in total for i in range(10): - data.append(("'{}'".format(get_random_string(1024 * 1024)), "toDateTime({})".format( - time.time() - 1 if i > 0 or positive else time.time() + 300))) # 1MB row + data.append(("randomPrintableASCII(1024*1024)", "toDateTime({})".format( + time.time() - 1 if i > 0 or positive else time.time() + 300))) node1.query("INSERT INTO {} (s1, d1) VALUES {}".format(name, ",".join(["(" + ",".join(x) + ")" for x in data]))) used_disks = get_used_disks_for_table(node1, name) @@ -199,7 +191,7 @@ def test_moves_work_after_storage_policy_change(started_cluster, name, engine): data = [] # 10MB in total for i in range(10): - data.append(("'{}'".format(get_random_string(1024 * 1024)), "toDateTime({})".format(time_1))) # 1MB row + data.append(("randomPrintableASCII(1024*1024)", "toDateTime({})".format(time_1))) node1.query("INSERT INTO {} (s1, d1) VALUES {}".format(name, ",".join(["(" + ",".join(x) + ")" for x in data]))) used_disks = get_used_disks_for_table(node1, name) @@ -247,8 +239,8 @@ def test_moves_to_disk_work(started_cluster, name, engine, positive): data = [] # 10MB in total for i in range(10): - data.append(("'{}'".format(get_random_string(1024 * 1024)), - "toDateTime({})".format(time_1 if i > 0 or positive else time_2))) # 1MB row + data.append(("randomPrintableASCII(1024*1024)", + "toDateTime({})".format(time_1 if i > 0 or positive else time_2))) node1.query("INSERT INTO {} (s1, d1) VALUES {}".format(name, ",".join(["(" + ",".join(x) + ")" for x in data]))) used_disks = get_used_disks_for_table(node1, name) @@ -295,7 +287,7 @@ def test_moves_to_volume_work(started_cluster, name, engine): data = [] # 10MB in total for i in range(5): data.append( - (str(p), "'{}'".format(get_random_string(1024 * 1024)), "toDateTime({})".format(time_1))) # 1MB row + (str(p), "randomPrintableASCII(1024*1024)", "toDateTime({})".format(time_1))) node1.query( "INSERT INTO {} (p1, s1, d1) VALUES {}".format(name, ",".join(["(" + ",".join(x) + ")" for x in data]))) @@ -342,8 +334,8 @@ def test_inserts_to_volume_work(started_cluster, name, engine, positive): for p in range(2): data = [] # 20MB in total for i in range(10): - data.append((str(p), "'{}'".format(get_random_string(1024 * 1024)), "toDateTime({})".format( - time.time() - 1 if i > 0 or positive else time.time() + 300))) # 1MB row + data.append((str(p), "randomPrintableASCII(1024*1024)", "toDateTime({})".format( + time.time() - 1 if i > 0 or positive else time.time() + 300))) node1.query( "INSERT INTO {} (p1, s1, d1) VALUES {}".format(name, ",".join(["(" + ",".join(x) + ")" for x in data]))) @@ -376,9 +368,9 @@ def test_moves_to_disk_eventually_work(started_cluster, name, engine): data = [] # 35MB in total for i in range(35): - data.append(get_random_string(1024 * 1024)) # 1MB row + data.append("randomPrintableASCII(1024*1024)") - node1.query("INSERT INTO {} VALUES {}".format(name_temp, ",".join(["('" + x + "')" for x in data]))) + node1.query("INSERT INTO {} VALUES {}".format(name_temp, ",".join(["(" + x + ")" for x in data]))) used_disks = get_used_disks_for_table(node1, name_temp) assert set(used_disks) == {"jbod2"} @@ -395,7 +387,7 @@ def test_moves_to_disk_eventually_work(started_cluster, name, engine): data = [] # 10MB in total for i in range(10): data.append( - ("'{}'".format(get_random_string(1024 * 1024)), "toDateTime({})".format(time.time() - 1))) # 1MB row + ("randomPrintableASCII(1024*1024)", "toDateTime({})".format(time.time() - 1))) node1.query("INSERT INTO {} (s1, d1) VALUES {}".format(name, ",".join(["(" + ",".join(x) + ")" for x in data]))) used_disks = get_used_disks_for_table(node1, name) @@ -431,8 +423,7 @@ def test_replicated_download_ttl_info(started_cluster): node1.query("SYSTEM STOP MOVES {}".format(name)) - node2.query("INSERT INTO {} (s1, d1) VALUES ('{}', toDateTime({}))".format(name, get_random_string(1024 * 1024), - time.time() - 100)) + node2.query("INSERT INTO {} (s1, d1) VALUES (randomPrintableASCII(1024*1024), toDateTime({}))".format(name, time.time() - 100)) assert set(get_used_disks_for_table(node2, name)) == {"external"} time.sleep(1) @@ -482,8 +473,8 @@ def test_merges_to_disk_work(started_cluster, name, engine, positive): for _ in range(2): data = [] # 16MB in total for i in range(8): - data.append(("'{}'".format(get_random_string(1024 * 1024)), - "toDateTime({})".format(time_1 if i > 0 or positive else time_2))) # 1MB row + data.append(("randomPrintableASCII(1024*1024)", + "toDateTime({})".format(time_1 if i > 0 or positive else time_2))) node1.query( "INSERT INTO {} (s1, d1) VALUES {}".format(name, ",".join(["(" + ",".join(x) + ")" for x in data]))) @@ -530,9 +521,9 @@ def test_merges_with_full_disk_work(started_cluster, name, engine): data = [] # 35MB in total for i in range(35): - data.append(get_random_string(1024 * 1024)) # 1MB row + data.append("randomPrintableASCII(1024*1024)") - node1.query("INSERT INTO {} VALUES {}".format(name_temp, ",".join(["('" + x + "')" for x in data]))) + node1.query("INSERT INTO {} VALUES {}".format(name_temp, ",".join(["(" + x + ")" for x in data]))) used_disks = get_used_disks_for_table(node1, name_temp) assert set(used_disks) == {"jbod2"} @@ -555,7 +546,7 @@ def test_merges_with_full_disk_work(started_cluster, name, engine): for _ in range(2): data = [] # 12MB in total for i in range(6): - data.append(("'{}'".format(get_random_string(1024 * 1024)), "toDateTime({})".format(time_1))) # 1MB row + data.append(("randomPrintableASCII(1024*1024)", "toDateTime({})".format(time_1))) # 1MB row node1.query( "INSERT INTO {} (s1, d1) VALUES {}".format(name, ",".join(["(" + ",".join(x) + ")" for x in data]))) @@ -612,7 +603,7 @@ def test_moves_after_merges_work(started_cluster, name, engine, positive): for _ in range(2): data = [] # 14MB in total for i in range(7): - data.append(("'{}'".format(get_random_string(1024 * 1024)), + data.append(("randomPrintableASCII(1024*1024)", "toDateTime({})".format(time_1 if i > 0 or positive else time_2))) # 1MB row node1.query( @@ -674,7 +665,7 @@ def test_ttls_do_not_work_after_alter(started_cluster, name, engine, positive, b data = [] # 10MB in total for i in range(10): data.append( - ("'{}'".format(get_random_string(1024 * 1024)), "toDateTime({})".format(time.time() - 1))) # 1MB row + ("randomPrintableASCII(1024*1024)", "toDateTime({})".format(time.time() - 1))) # 1MB row node1.query("INSERT INTO {} (s1, d1) VALUES {}".format(name, ",".join(["(" + ",".join(x) + ")" for x in data]))) used_disks = get_used_disks_for_table(node1, name) @@ -706,7 +697,7 @@ def test_materialize_ttl_in_partition(started_cluster, name, engine): data = [] # 5MB in total for i in range(5): - data.append((str(i), "'{}'".format(get_random_string(1024 * 1024)), + data.append((str(i), "randomPrintableASCII(1024*1024)", "toDateTime({})".format(time.time() - 1))) # 1MB row node1.query( "INSERT INTO {} (p1, s1, d1) VALUES {}".format(name, ",".join(["(" + ",".join(x) + ")" for x in data]))) @@ -808,9 +799,8 @@ limitations under the License.""" now = time.time() for i in range(2): p1 = p - s1 = get_random_string(1024 * 1024) # 1MB d1 = now - 1 if i > 0 or positive else now + 300 - data.append("({}, '{}', toDateTime({}))".format(p1, s1, d1)) + data.append("({}, randomPrintableASCII(1024*1024), toDateTime({}))".format(p1, d1)) node1.query("INSERT INTO {name} (p1, s1, d1) VALUES {values}".format(name=name, values=",".join(data))) used_disks = get_used_disks_for_table(node1, name) @@ -970,7 +960,7 @@ def test_double_move_while_select(started_cluster, name, positive): """.format(name=name)) node1.query( - "INSERT INTO {name} VALUES (1, '{string}')".format(name=name, string=get_random_string(10 * 1024 * 1024))) + "INSERT INTO {name} VALUES (1, randomPrintableASCII(10*1024*1024))".format(name=name)) parts = node1.query( "SELECT name FROM system.parts WHERE table = '{name}' AND active = 1".format(name=name)).splitlines() @@ -991,11 +981,11 @@ def test_double_move_while_select(started_cluster, name, positive): # Fill jbod1 to force ClickHouse to make move of partition 1 to external. node1.query( - "INSERT INTO {name} VALUES (2, '{string}')".format(name=name, string=get_random_string(9 * 1024 * 1024))) + "INSERT INTO {name} VALUES (2, randomPrintableASCII(9*1024*1024))".format(name=name)) node1.query( - "INSERT INTO {name} VALUES (3, '{string}')".format(name=name, string=get_random_string(9 * 1024 * 1024))) + "INSERT INTO {name} VALUES (3, randomPrintableASCII(9*1024*1024))".format(name=name)) node1.query( - "INSERT INTO {name} VALUES (4, '{string}')".format(name=name, string=get_random_string(9 * 1024 * 1024))) + "INSERT INTO {name} VALUES (4, randomPrintableASCII(9*1024*1024))".format(name=name)) time.sleep(1) @@ -1059,9 +1049,8 @@ limitations under the License.""" data = [] # 6MB in total now = time.time() for i in range(2): - s1 = get_random_string(1024 * 1024) # 1MB d1 = now - 1 if positive else now + 300 - data.append("('{}', toDateTime({}))".format(s1, d1)) + data.append("(randomPrintableASCII(1024*1024), toDateTime({}))".format(d1)) values = ",".join(data) node1.query("INSERT INTO {name} (s1, d1) VALUES {values}".format(name=name, values=values)) @@ -1126,8 +1115,7 @@ def test_disabled_ttl_move_on_insert(started_cluster, name, dest_type, engine): data = [] # 10MB in total for i in range(10): - data.append(("'{}'".format(get_random_string(1024 * 1024)), "toDateTime({})".format( - time.time() - 1))) # 1MB row + data.append(("randomPrintableASCII(1024*1024)", "toDateTime({})".format(time.time() - 1))) node1.query("INSERT INTO {} (s1, d1) VALUES {}".format(name, ",".join(["(" + ",".join(x) + ")" for x in data]))) diff --git a/tests/integration/test_ttl_replicated/test.py b/tests/integration/test_ttl_replicated/test.py index 13fb779a6e6..cbf13c20220 100644 --- a/tests/integration/test_ttl_replicated/test.py +++ b/tests/integration/test_ttl_replicated/test.py @@ -18,7 +18,7 @@ def started_cluster(): yield cluster except Exception as ex: - print ex + print(ex) finally: cluster.shutdown() diff --git a/tests/integration/test_user_ip_restrictions/test.py b/tests/integration/test_user_ip_restrictions/test.py index 1f28fbde069..a7344fd1a45 100644 --- a/tests/integration/test_user_ip_restrictions/test.py +++ b/tests/integration/test_user_ip_restrictions/test.py @@ -63,7 +63,7 @@ def test_ipv4(setup_cluster): except AssertionError: raise except Exception as ex: - print ex + print(ex) def test_ipv6(setup_cluster): @@ -72,7 +72,7 @@ def test_ipv6(setup_cluster): ["bash", "-c", "/usr/bin/clickhouse client --host 2001:3984:3989::1:1000 --query 'select 1'"], privileged=True, user='root') except Exception as ex: - print ex + print(ex) assert False, "allowed client with 2001:3984:3989:0:0:0:1:1111 cannot connect to server with allowed mask '2001:3984:3989:0:0:0:0:0/112'" try: @@ -90,4 +90,4 @@ def test_ipv6(setup_cluster): except AssertionError: raise except Exception as ex: - print ex + print(ex) diff --git a/tests/integration/test_user_zero_database_access/test_user_zero_database_access.py b/tests/integration/test_user_zero_database_access/test_user_zero_database_access.py index c5ea7ed60a0..dd3789cde57 100644 --- a/tests/integration/test_user_zero_database_access/test_user_zero_database_access.py +++ b/tests/integration/test_user_zero_database_access/test_user_zero_database_access.py @@ -24,7 +24,7 @@ def test_user_zero_database_access(start_cluster): except AssertionError: raise except Exception as ex: - print ex + print(ex) try: node.exec_in_container( @@ -47,7 +47,7 @@ def test_user_zero_database_access(start_cluster): except AssertionError: raise except Exception as ex: - print ex + print(ex) try: node.exec_in_container( @@ -57,7 +57,7 @@ def test_user_zero_database_access(start_cluster): except AssertionError: raise except Exception as ex: - print ex + print(ex) try: node.exec_in_container( diff --git a/tests/integration/test_zookeeper_config/test.py b/tests/integration/test_zookeeper_config/test.py index 9bc206d8da4..eb5ab2da98f 100644 --- a/tests/integration/test_zookeeper_config/test.py +++ b/tests/integration/test_zookeeper_config/test.py @@ -1,4 +1,4 @@ -from __future__ import print_function + import time from os import path as p, unlink @@ -147,7 +147,7 @@ def test_secure_connection(): cluster = ClickHouseCluster(__file__, zookeeper_config_path='configs/zookeeper_config_with_ssl.xml') - docker_compose = NamedTemporaryFile(delete=False) + docker_compose = NamedTemporaryFile(mode='w+', delete=False) docker_compose.write( "version: '2.3'\nservices:\n" + diff --git a/tests/perf_drafts/accurate_comparisons/accurate_comparisons.sh b/tests/perf_drafts/accurate_comparisons/accurate_comparisons.sh index 2cd040cec63..cfd73b5b942 100755 --- a/tests/perf_drafts/accurate_comparisons/accurate_comparisons.sh +++ b/tests/perf_drafts/accurate_comparisons/accurate_comparisons.sh @@ -8,7 +8,7 @@ clickhouse-client -q "INSERT INTO test.comparisons SELECT toInt64(rand64()) + nu function test_cmp { echo -n "$1 : " echo "SELECT count() FROM test.comparisons WHERE ($1)" | clickhouse-benchmark --max_threads=1 -i 20 -d 0 --json test.json 1>&2 2>/dev/null - python2 -c "import json; print '%.3f' % float(json.load(open('test.json'))['query_time_percentiles']['0'])" + python3 -c "import json; print '%.3f' % float(json.load(open('test.json'))['query_time_percentiles']['0'])" rm test.json } diff --git a/tests/queries/0_stateless/00386_long_in_pk.python b/tests/queries/0_stateless/00386_long_in_pk.python index f189233d299..ab5fc50d8e3 100644 --- a/tests/queries/0_stateless/00386_long_in_pk.python +++ b/tests/queries/0_stateless/00386_long_in_pk.python @@ -1,4 +1,4 @@ -#!/usr/bin/env python +#!/usr/bin/env python3 def gen_queries(): create_template = 'create table tab_00386 (a Int8, b String, c Tuple(Int8), d Tuple(Tuple(Int8)), e Tuple(Int8, String), f Tuple(Tuple(Int8, String))) engine = MergeTree order by ({}) partition by {}' @@ -45,9 +45,9 @@ def main(): for q in gen_queries(): resp = requests.post(url, data=q) if resp.status_code != 200 or resp.text.strip() not in ('1', ''): - print 'Query:', q - print 'Code:', resp.status_code - print resp.text + print('Query:', q) + print('Code:', resp.status_code) + print(resp.text) break if __name__ == "__main__": diff --git a/tests/queries/0_stateless/00386_long_in_pk.sh b/tests/queries/0_stateless/00386_long_in_pk.sh index 414a3dce32d..8cad8f93a13 100755 --- a/tests/queries/0_stateless/00386_long_in_pk.sh +++ b/tests/queries/0_stateless/00386_long_in_pk.sh @@ -5,5 +5,5 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # We should have correct env vars from shell_config.sh to run this test -python "$CURDIR"/00386_long_in_pk.python +python3 "$CURDIR"/00386_long_in_pk.python diff --git a/tests/queries/0_stateless/00411_long_accurate_number_comparison.python b/tests/queries/0_stateless/00411_long_accurate_number_comparison.python index 47c5d7f3d5b..3c8a8f2ea25 100644 --- a/tests/queries/0_stateless/00411_long_accurate_number_comparison.python +++ b/tests/queries/0_stateless/00411_long_accurate_number_comparison.python @@ -1,9 +1,9 @@ -#!/usr/bin/env python -from __future__ import print_function -import os, itertools, urllib, urllib2, sys +#!/usr/bin/env python3 + +import os, itertools, urllib.request, urllib.parse, urllib.error, urllib.request, urllib.error, urllib.parse, sys def get_ch_answer(query): - return urllib.urlopen(os.environ.get('CLICKHOUSE_URL', 'http://localhost:' + os.environ.get('CLICKHOUSE_PORT_HTTP', '8123') ), data=query).read() + return urllib.request.urlopen(os.environ.get('CLICKHOUSE_URL', 'http://localhost:' + os.environ.get('CLICKHOUSE_PORT_HTTP', '8123') ), data=query.encode()).read().decode() def check_answers(query, answer): ch_answer = get_ch_answer(query) @@ -83,9 +83,9 @@ def test_pair(v1, v2): answers += a if TEST_WITH_CASTING: - for t1 in TYPES.iterkeys(): + for t1 in TYPES.keys(): if inside_range(v1, t1): - for t2 in TYPES.iterkeys(): + for t2 in TYPES.keys(): if inside_range(v2, t2): q, a = test_operators(v1, v2, 'to{}({})'.format(t1, v1), 'to{}({})'.format(t2, v2)) query += ', ' + q @@ -108,7 +108,7 @@ def test_float_pair(i, f): answers += a if TEST_WITH_CASTING: - for t1 in TYPES.iterkeys(): + for t1 in TYPES.keys(): if inside_range(i, t1): q, a = test_operators(i, f, 'to{}({})'.format(t1, i), f_str) query += ', ' + q @@ -127,9 +127,9 @@ def main(): num_int_tests = len(list(itertools.combinations(VALUES, 2))) num_parts = 4 - for part in xrange(0, num_parts): + for part in range(0, num_parts): if 'int' + str(part + 1) in sys.argv[1:]: - for (v1, v2) in itertools.islice(itertools.combinations(VALUES, 2), part * num_int_tests / num_parts, (part + 1) * num_int_tests / num_parts): + for (v1, v2) in itertools.islice(itertools.combinations(VALUES, 2), part * num_int_tests // num_parts, (part + 1) * num_int_tests // num_parts): q, a = test_pair(v1, v2) if GENERATE_TEST_FILES: sql_file.write(q + ";\n") diff --git a/tests/queries/0_stateless/00411_long_accurate_number_comparison_float.sh b/tests/queries/0_stateless/00411_long_accurate_number_comparison_float.sh index 50a6cd386ae..17d0c7564e3 100755 --- a/tests/queries/0_stateless/00411_long_accurate_number_comparison_float.sh +++ b/tests/queries/0_stateless/00411_long_accurate_number_comparison_float.sh @@ -5,4 +5,4 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # We should have correct env vars from shell_config.sh to run this test -python "$CURDIR"/00411_long_accurate_number_comparison.python float +python3 "$CURDIR"/00411_long_accurate_number_comparison.python float diff --git a/tests/queries/0_stateless/00411_long_accurate_number_comparison_int1.sh b/tests/queries/0_stateless/00411_long_accurate_number_comparison_int1.sh index 8aba65d8911..43d9d550ddf 100755 --- a/tests/queries/0_stateless/00411_long_accurate_number_comparison_int1.sh +++ b/tests/queries/0_stateless/00411_long_accurate_number_comparison_int1.sh @@ -5,4 +5,4 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # We should have correct env vars from shell_config.sh to run this test -python "$CURDIR"/00411_long_accurate_number_comparison.python int1 +python3 "$CURDIR"/00411_long_accurate_number_comparison.python int1 diff --git a/tests/queries/0_stateless/00411_long_accurate_number_comparison_int2.sh b/tests/queries/0_stateless/00411_long_accurate_number_comparison_int2.sh index 93001184144..34aaf9ef7ed 100755 --- a/tests/queries/0_stateless/00411_long_accurate_number_comparison_int2.sh +++ b/tests/queries/0_stateless/00411_long_accurate_number_comparison_int2.sh @@ -5,4 +5,4 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # We should have correct env vars from shell_config.sh to run this test -python "$CURDIR"/00411_long_accurate_number_comparison.python int2 +python3 "$CURDIR"/00411_long_accurate_number_comparison.python int2 diff --git a/tests/queries/0_stateless/00411_long_accurate_number_comparison_int3.sh b/tests/queries/0_stateless/00411_long_accurate_number_comparison_int3.sh index 08ef31863b5..139792944ee 100755 --- a/tests/queries/0_stateless/00411_long_accurate_number_comparison_int3.sh +++ b/tests/queries/0_stateless/00411_long_accurate_number_comparison_int3.sh @@ -5,4 +5,4 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # We should have correct env vars from shell_config.sh to run this test -python "$CURDIR"/00411_long_accurate_number_comparison.python int3 +python3 "$CURDIR"/00411_long_accurate_number_comparison.python int3 diff --git a/tests/queries/0_stateless/00411_long_accurate_number_comparison_int4.sh b/tests/queries/0_stateless/00411_long_accurate_number_comparison_int4.sh index a268066725b..f57099e77ca 100755 --- a/tests/queries/0_stateless/00411_long_accurate_number_comparison_int4.sh +++ b/tests/queries/0_stateless/00411_long_accurate_number_comparison_int4.sh @@ -5,4 +5,4 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # We should have correct env vars from shell_config.sh to run this test -python "$CURDIR"/00411_long_accurate_number_comparison.python int4 +python3 "$CURDIR"/00411_long_accurate_number_comparison.python int4 diff --git a/tests/queries/0_stateless/00565_enum_order.sh b/tests/queries/0_stateless/00565_enum_order.sh index d7d06b49108..2851bcaaca2 100755 --- a/tests/queries/0_stateless/00565_enum_order.sh +++ b/tests/queries/0_stateless/00565_enum_order.sh @@ -42,7 +42,7 @@ QUERY='INSERT INTO "test_log"("date", "datetime", "path", "gtid", "query_serial" "new_fields"."is_null", "record_source_type", "record_source_timestamp", "deleted") FORMAT TabSeparated' QUERY="$(tr -d '\n' <<<"$QUERY")" echo "$QUERY" -URL=$(python -c 'print "'"${CLICKHOUSE_URL}"'&query=" + __import__("urllib").quote('"'''$QUERY'''"')') +URL=$(python3 -c 'import urllib.parse; print("'"${CLICKHOUSE_URL}"'&query=" + urllib.parse.quote('"'''$QUERY'''"'))') set +e for _ in 1 2 3; do diff --git a/tests/queries/0_stateless/00612_http_max_query_size.sh b/tests/queries/0_stateless/00612_http_max_query_size.sh index a50155edab0..78ae4eba1dc 100755 --- a/tests/queries/0_stateless/00612_http_max_query_size.sh +++ b/tests/queries/0_stateless/00612_http_max_query_size.sh @@ -28,20 +28,20 @@ if not url.startswith('http'): q = 'select sum(number) from (select * from system.numbers limit 10000000) where number = 0' def gen_data(q): - yield q - yield ''.join([' '] * (1024 - len(q))) + yield q.encode() + yield (''.join([' '] * (1024 - len(q)))).encode() pattern = ''' or toString(number) = '{}'\n''' for i in range(1, 4 * 1024): - yield pattern.format(str(i).zfill(1024 - len(pattern) + 2)) + yield pattern.format(str(i).zfill(1024 - len(pattern) + 2)).encode() s = requests.Session() resp = s.post(url + '&max_query_size={}'.format(1 << 21), timeout=1, data=gen_data(q), stream=True, headers = {'Connection': 'close'}) for line in resp.iter_lines(): - print line -" | python | grep -o "Max query size exceeded" + print(line) +" | python3 | grep -o "Max query size exceeded" echo - diff --git a/tests/queries/0_stateless/00646_url_engine.python b/tests/queries/0_stateless/00646_url_engine.python index c03ce51c9a9..6a70fac21f3 100644 --- a/tests/queries/0_stateless/00646_url_engine.python +++ b/tests/queries/0_stateless/00646_url_engine.python @@ -1,14 +1,21 @@ -#!/usr/bin/env python -from __future__ import print_function +#!/usr/bin/env python3 + +import socket import csv import sys -import time import tempfile import threading -import os, urllib +import os +import traceback +import urllib.request import subprocess from io import StringIO -from BaseHTTPServer import BaseHTTPRequestHandler, HTTPServer +from http.server import BaseHTTPRequestHandler, HTTPServer + +def get_local_port(host): + with socket.socket() as fd: + fd.bind((host, 0)) + return fd.getsockname()[1] CLICKHOUSE_HOST = os.environ.get('CLICKHOUSE_HOST', '127.0.0.1') CLICKHOUSE_PORT_HTTP = os.environ.get('CLICKHOUSE_PORT_HTTP', '8123') @@ -21,7 +28,7 @@ CLICKHOUSE_PORT_HTTP = os.environ.get('CLICKHOUSE_PORT_HTTP', '8123') # IP-address of this host accessible from outside world. HTTP_SERVER_HOST = subprocess.check_output(['hostname', '-i']).decode('utf-8').strip() -HTTP_SERVER_PORT = 55123 +HTTP_SERVER_PORT = get_local_port(HTTP_SERVER_HOST) # IP address and port of the HTTP server started from this script. HTTP_SERVER_ADDRESS = (HTTP_SERVER_HOST, HTTP_SERVER_PORT) @@ -30,7 +37,7 @@ CSV_DATA = os.path.join(tempfile._get_default_tempdir(), next(tempfile._get_cand def get_ch_answer(query): url = os.environ.get('CLICKHOUSE_URL', 'http://{host}:{port}'.format(host=CLICKHOUSE_HOST, port=CLICKHOUSE_PORT_HTTP)) - return urllib.urlopen(url, data=query).read() + return urllib.request.urlopen(url, data=query.encode()).read().decode() def check_answers(query, answer): ch_answer = get_ch_answer(query) @@ -51,7 +58,7 @@ class CSVHTTPServer(BaseHTTPRequestHandler): with open(CSV_DATA, 'r') as fl: reader = csv.reader(fl, delimiter=',') for row in reader: - self.wfile.write(', '.join(row) + '\n') + self.wfile.write((', '.join(row) + '\n').encode()) return def read_chunk(self): @@ -77,14 +84,13 @@ class CSVHTTPServer(BaseHTTPRequestHandler): if not chunk: break data += chunk - text = "" with StringIO(data) as fl: reader = csv.reader(fl, delimiter=',') with open(CSV_DATA, 'a') as d: for row in reader: d.write(','.join(row) + '\n') self._set_headers() - self.wfile.write("ok") + self.wfile.write(b"ok") def log_message(self, format, *args): return @@ -93,7 +99,7 @@ def start_server(requests_amount): httpd = HTTPServer(HTTP_SERVER_ADDRESS, CSVHTTPServer) def real_func(): - for i in xrange(requests_amount): + for i in range(requests_amount): httpd.handle_request() t = threading.Thread(target=real_func) @@ -113,7 +119,7 @@ def test_select(table_name="", schema="str String,numuint UInt32,numint Int32,do get_ch_answer("drop table if exists {}".format(table_name)) get_ch_answer("create table {} ({}) engine=URL('{}', 'CSV')".format(table_name, schema, HTTP_SERVER_URL_STR)) - for i in xrange(len(requests)): + for i in range(len(requests)): tbl = table_name if not tbl: tbl = "url('{addr}', 'CSV', '{schema}')".format(addr=HTTP_SERVER_URL_STR, schema=schema) @@ -137,7 +143,7 @@ def test_insert(table_name="", schema="str String,numuint UInt32,numint Int32,do get_ch_answer(req.format(tbl=tbl)) - for i in xrange(len(requests_select)): + for i in range(len(requests_select)): tbl = table_name if not tbl: tbl = "url('{addr}', 'CSV', '{schema}')".format(addr=HTTP_SERVER_URL_STR, schema=schema) @@ -161,7 +167,7 @@ def main(): ] select_requests = { - "select distinct numuint from {tbl} order by numuint": '\n'.join([str(i) for i in xrange(11)]), + "select distinct numuint from {tbl} order by numuint": '\n'.join([str(i) for i in range(11)]), "select count(*) from {tbl}": '12', 'select double, count(*) from {tbl} group by double': "7.7\t2\n9.9\t10" } @@ -169,27 +175,24 @@ def main(): t = start_server(len(select_only_requests) * 2 + (len(insert_requests) + len(select_requests)) * 2) t.start() # test table with url engine - test_select(table_name="test_table_select", requests=select_only_requests.keys(), answers=select_only_requests.values(), test_data=test_data) + test_select(table_name="test_table_select", requests=list(select_only_requests.keys()), answers=list(select_only_requests.values()), test_data=test_data) # test table function url - test_select(requests=select_only_requests.keys(), answers=select_only_requests.values(), test_data=test_data) + test_select(requests=list(select_only_requests.keys()), answers=list(select_only_requests.values()), test_data=test_data) #test insert into table with url engine - test_insert(table_name="test_table_insert", requests_insert=insert_requests, requests_select=select_requests.keys(), answers=select_requests.values()) + test_insert(table_name="test_table_insert", requests_insert=insert_requests, requests_select=list(select_requests.keys()), answers=list(select_requests.values())) #test insert into table function url - test_insert(requests_insert=insert_requests, requests_select=select_requests.keys(), answers=select_requests.values()) + test_insert(requests_insert=insert_requests, requests_select=list(select_requests.keys()), answers=list(select_requests.values())) t.join() print("PASSED") if __name__ == "__main__": - exception_text = '' - for i in range(1, 5): - try: - main() - break - except Exception as ex: - exception_text = str(ex) - time.sleep(0.1) + try: + main() + except Exception as ex: + exc_type, exc_value, exc_traceback = sys.exc_info() + traceback.print_tb(exc_traceback, file=sys.stderr) + print(ex, file=sys.stderr) + sys.stderr.flush() - if exception_text: - print("Exception: {}".format(exception_text), file=sys.stderr) os._exit(1) diff --git a/tests/queries/0_stateless/00646_url_engine.sh b/tests/queries/0_stateless/00646_url_engine.sh index ee2e2521cf8..bf20e0c1222 100755 --- a/tests/queries/0_stateless/00646_url_engine.sh +++ b/tests/queries/0_stateless/00646_url_engine.sh @@ -5,4 +5,4 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # We should have correct env vars from shell_config.sh to run this test -python "$CURDIR"/00646_url_engine.python +python3 "$CURDIR"/00646_url_engine.python diff --git a/tests/queries/0_stateless/00921_datetime64_compatibility.python b/tests/queries/0_stateless/00921_datetime64_compatibility.python index 54630755f05..bf0ae8a72ac 100755 --- a/tests/queries/0_stateless/00921_datetime64_compatibility.python +++ b/tests/queries/0_stateless/00921_datetime64_compatibility.python @@ -1,4 +1,4 @@ -#!/usr/bin/env python +#!/usr/bin/env python3 # encoding: utf-8 import re @@ -88,8 +88,7 @@ formatDateTime(N, '%C %d %D %e %F %H %I %j %m %M %p %R %S %T %u %V %w %y %Y %%') # Expanded later to cartesian product of all arguments. # NOTE: {N} to be turned into N after str.format() for keys (format string), but not for list of values! -extra_ops =\ -[ +extra_ops = [ # With same type: ( ['N {op} N'], @@ -161,37 +160,35 @@ extra_ops =\ # Expand extra_ops here for funcs, args in extra_ops: - args_keys = args.keys() - for args_vals in itertools.product(*args.values()): + args_keys = list(args.keys()) + for args_vals in itertools.product(*list(args.values())): for func in funcs: - result_func = func.format(**dict(zip(args_keys, args_vals))) + result_func = func.format(**dict(list(zip(args_keys, args_vals)))) FUNCTIONS.append(result_func) # filter out empty lines and commented out lines COMMENTED_OUT_LINE_RE = re.compile(r"^\s*#") -FUNCTIONS = list(filter(lambda f: len(f) != 0 and COMMENTED_OUT_LINE_RE.match(f) == None, FUNCTIONS)) +FUNCTIONS = list([f for f in FUNCTIONS if len(f) != 0 and COMMENTED_OUT_LINE_RE.match(f) == None]) TYPES = ['D', 'DT', 'DT64'] -if sys.version_info[0] > 2: - escape_string_codec = 'unicode_escape' -else: - escape_string_codec = 'string-escape' - def escape_string(s): - return s.encode(escape_string_codec).decode('utf-8') + if sys.version_info[0] > 2: + return s.encode('unicode_escape').decode('utf-8').replace("'", "\\'") + else: + return s.encode('string-escape').decode('utf-8') def execute_functions_for_types(functions, types): # TODO: use string.Template here to allow lines that do not contain type, like: SELECT CAST(toDateTime64(1234567890), 'DateTime64') for func in functions: - print("""SELECT 'SELECT {func}';""".format(func=escape_string(func))) + print(("""SELECT 'SELECT {func}';""".format(func=escape_string(func)))) for dt in types: prologue = "\ WITH \ toDateTime64('2019-09-16 19:20:11.234', 3, 'Europe/Minsk') as DT64, \ toDateTime('2019-09-16 19:20:11', 'Europe/Minsk') as DT, \ toDate('2019-09-16') as D, {X} as N".format(X=dt) - print("""{prologue} SELECT toTypeName(r), {func} as r FORMAT CSV;""".format(prologue=prologue, func=func)) + print(("""{prologue} SELECT toTypeName(r), {func} as r FORMAT CSV;""".format(prologue=prologue, func=func))) print("""SELECT '------------------------------------------';""") def main(): @@ -210,22 +207,22 @@ def main(): types = TYPES if args.functions_re: - functions = list(filter(lambda f : args.functions_re.search(f), functions)) + functions = list([f for f in functions if args.functions_re.search(f)]) if len(functions) == 0: print("functions list is empty") return -1 if args.types_re: - types = list(filter(lambda t : args.types_re.match(t), types)) + types = list([t for t in types if args.types_re.match(t)]) if len(types) == 0: print("types list is empty") return -1 if args.list_functions: - print("\n".join(functions)) + print(("\n".join(functions))) return 0 execute_functions_for_types(functions, types) if __name__ == '__main__': - exit(main()) \ No newline at end of file + exit(main()) diff --git a/tests/queries/0_stateless/00921_datetime64_compatibility.reference b/tests/queries/0_stateless/00921_datetime64_compatibility.reference index 079469c8c2d..1a909c8c754 100644 --- a/tests/queries/0_stateless/00921_datetime64_compatibility.reference +++ b/tests/queries/0_stateless/00921_datetime64_compatibility.reference @@ -430,66 +430,6 @@ Code: 43: Illegal types DateTime('Europe/Minsk') and Date of arguments of functi Code: 43: Illegal types DateTime('Europe/Minsk') and DateTime64(3, 'Europe/Minsk') of arguments of function minus. ------------------------------------------ -SELECT N != DT -"UInt8",1 -"UInt8",0 -"UInt8",1 ------------------------------------------- -SELECT DT != N -"UInt8",1 -"UInt8",0 -"UInt8",1 ------------------------------------------- -SELECT N == DT -"UInt8",0 -"UInt8",1 -"UInt8",0 ------------------------------------------- -SELECT DT == N -"UInt8",0 -"UInt8",1 -"UInt8",0 ------------------------------------------- -SELECT N < DT -"UInt8",1 -"UInt8",0 -"UInt8",0 ------------------------------------------- -SELECT DT < N -"UInt8",0 -"UInt8",0 -"UInt8",1 ------------------------------------------- -SELECT N <= DT -"UInt8",1 -"UInt8",1 -"UInt8",0 ------------------------------------------- -SELECT DT <= N -"UInt8",0 -"UInt8",1 -"UInt8",1 ------------------------------------------- -SELECT N > DT -"UInt8",0 -"UInt8",0 -"UInt8",1 ------------------------------------------- -SELECT DT > N -"UInt8",1 -"UInt8",0 -"UInt8",0 ------------------------------------------- -SELECT N >= DT -"UInt8",0 -"UInt8",1 -"UInt8",1 ------------------------------------------- -SELECT DT >= N -"UInt8",1 -"UInt8",1 -"UInt8",0 ------------------------------------------- SELECT N - D "Int32",0 @@ -504,66 +444,6 @@ Code: 43: Illegal types Date and DateTime('Europe/Minsk') of arguments of functi Code: 43: Illegal types Date and DateTime64(3, 'Europe/Minsk') of arguments of function minus. ------------------------------------------ -SELECT N != D -"UInt8",0 -"UInt8",1 -"UInt8",1 ------------------------------------------- -SELECT D != N -"UInt8",0 -"UInt8",1 -"UInt8",1 ------------------------------------------- -SELECT N == D -"UInt8",1 -"UInt8",0 -"UInt8",0 ------------------------------------------- -SELECT D == N -"UInt8",1 -"UInt8",0 -"UInt8",0 ------------------------------------------- -SELECT N < D -"UInt8",0 -"UInt8",0 -"UInt8",0 ------------------------------------------- -SELECT D < N -"UInt8",0 -"UInt8",1 -"UInt8",1 ------------------------------------------- -SELECT N <= D -"UInt8",1 -"UInt8",0 -"UInt8",0 ------------------------------------------- -SELECT D <= N -"UInt8",1 -"UInt8",1 -"UInt8",1 ------------------------------------------- -SELECT N > D -"UInt8",0 -"UInt8",1 -"UInt8",1 ------------------------------------------- -SELECT D > N -"UInt8",0 -"UInt8",0 -"UInt8",0 ------------------------------------------- -SELECT N >= D -"UInt8",1 -"UInt8",1 -"UInt8",1 ------------------------------------------- -SELECT D >= N -"UInt8",1 -"UInt8",0 -"UInt8",0 ------------------------------------------- SELECT N - DT64 Code: 43: Illegal types Date and DateTime64(3, 'Europe/Minsk') of arguments of function minus. @@ -580,6 +460,26 @@ Code: 43: Illegal types DateTime64(3, 'Europe/Minsk') and DateTime('Europe/Minsk Code: 43: Illegal types DateTime64(3, 'Europe/Minsk') and DateTime64(3, 'Europe/Minsk') of arguments of function minus. ------------------------------------------ +SELECT N != DT +"UInt8",1 +"UInt8",0 +"UInt8",1 +------------------------------------------ +SELECT DT != N +"UInt8",1 +"UInt8",0 +"UInt8",1 +------------------------------------------ +SELECT N != D +"UInt8",0 +"UInt8",1 +"UInt8",1 +------------------------------------------ +SELECT D != N +"UInt8",0 +"UInt8",1 +"UInt8",1 +------------------------------------------ SELECT N != DT64 "UInt8",1 "UInt8",1 @@ -590,6 +490,26 @@ SELECT DT64 != N "UInt8",1 "UInt8",0 ------------------------------------------ +SELECT N == DT +"UInt8",0 +"UInt8",1 +"UInt8",0 +------------------------------------------ +SELECT DT == N +"UInt8",0 +"UInt8",1 +"UInt8",0 +------------------------------------------ +SELECT N == D +"UInt8",1 +"UInt8",0 +"UInt8",0 +------------------------------------------ +SELECT D == N +"UInt8",1 +"UInt8",0 +"UInt8",0 +------------------------------------------ SELECT N == DT64 "UInt8",0 "UInt8",0 @@ -600,6 +520,26 @@ SELECT DT64 == N "UInt8",0 "UInt8",1 ------------------------------------------ +SELECT N < DT +"UInt8",1 +"UInt8",0 +"UInt8",0 +------------------------------------------ +SELECT DT < N +"UInt8",0 +"UInt8",0 +"UInt8",1 +------------------------------------------ +SELECT N < D +"UInt8",0 +"UInt8",0 +"UInt8",0 +------------------------------------------ +SELECT D < N +"UInt8",0 +"UInt8",1 +"UInt8",1 +------------------------------------------ SELECT N < DT64 "UInt8",1 "UInt8",1 @@ -610,6 +550,26 @@ SELECT DT64 < N "UInt8",0 "UInt8",0 ------------------------------------------ +SELECT N <= DT +"UInt8",1 +"UInt8",1 +"UInt8",0 +------------------------------------------ +SELECT DT <= N +"UInt8",0 +"UInt8",1 +"UInt8",1 +------------------------------------------ +SELECT N <= D +"UInt8",1 +"UInt8",0 +"UInt8",0 +------------------------------------------ +SELECT D <= N +"UInt8",1 +"UInt8",1 +"UInt8",1 +------------------------------------------ SELECT N <= DT64 "UInt8",1 "UInt8",1 @@ -620,6 +580,26 @@ SELECT DT64 <= N "UInt8",0 "UInt8",1 ------------------------------------------ +SELECT N > DT +"UInt8",0 +"UInt8",0 +"UInt8",1 +------------------------------------------ +SELECT DT > N +"UInt8",1 +"UInt8",0 +"UInt8",0 +------------------------------------------ +SELECT N > D +"UInt8",0 +"UInt8",1 +"UInt8",1 +------------------------------------------ +SELECT D > N +"UInt8",0 +"UInt8",0 +"UInt8",0 +------------------------------------------ SELECT N > DT64 "UInt8",0 "UInt8",0 @@ -630,6 +610,26 @@ SELECT DT64 > N "UInt8",1 "UInt8",0 ------------------------------------------ +SELECT N >= DT +"UInt8",0 +"UInt8",1 +"UInt8",1 +------------------------------------------ +SELECT DT >= N +"UInt8",1 +"UInt8",1 +"UInt8",0 +------------------------------------------ +SELECT N >= D +"UInt8",1 +"UInt8",1 +"UInt8",1 +------------------------------------------ +SELECT D >= N +"UInt8",1 +"UInt8",0 +"UInt8",0 +------------------------------------------ SELECT N >= DT64 "UInt8",0 "UInt8",0 @@ -650,6 +650,76 @@ SELECT toUInt8(1) + N "DateTime('Europe/Minsk')","2019-09-16 19:20:12" "DateTime64(3, 'Europe/Minsk')","2019-09-16 19:20:12.234" ------------------------------------------ +SELECT N + toInt8(-1) +"Date","2019-09-15" +"DateTime('Europe/Minsk')","2019-09-16 19:20:10" +"DateTime64(3, 'Europe/Minsk')","2019-09-16 19:20:10.234" +------------------------------------------ +SELECT toInt8(-1) + N +"Date","2019-09-15" +"DateTime('Europe/Minsk')","2019-09-16 19:20:10" +"DateTime64(3, 'Europe/Minsk')","2019-09-16 19:20:10.234" +------------------------------------------ +SELECT N + toUInt16(1) +"Date","2019-09-17" +"DateTime('Europe/Minsk')","2019-09-16 19:20:12" +"DateTime64(3, 'Europe/Minsk')","2019-09-16 19:20:12.234" +------------------------------------------ +SELECT toUInt16(1) + N +"Date","2019-09-17" +"DateTime('Europe/Minsk')","2019-09-16 19:20:12" +"DateTime64(3, 'Europe/Minsk')","2019-09-16 19:20:12.234" +------------------------------------------ +SELECT N + toInt16(-1) +"Date","2019-09-15" +"DateTime('Europe/Minsk')","2019-09-16 19:20:10" +"DateTime64(3, 'Europe/Minsk')","2019-09-16 19:20:10.234" +------------------------------------------ +SELECT toInt16(-1) + N +"Date","2019-09-15" +"DateTime('Europe/Minsk')","2019-09-16 19:20:10" +"DateTime64(3, 'Europe/Minsk')","2019-09-16 19:20:10.234" +------------------------------------------ +SELECT N + toUInt32(1) +"Date","2019-09-17" +"DateTime('Europe/Minsk')","2019-09-16 19:20:12" +"DateTime64(3, 'Europe/Minsk')","2019-09-16 19:20:12.234" +------------------------------------------ +SELECT toUInt32(1) + N +"Date","2019-09-17" +"DateTime('Europe/Minsk')","2019-09-16 19:20:12" +"DateTime64(3, 'Europe/Minsk')","2019-09-16 19:20:12.234" +------------------------------------------ +SELECT N + toInt32(-1) +"Date","2019-09-15" +"DateTime('Europe/Minsk')","2019-09-16 19:20:10" +"DateTime64(3, 'Europe/Minsk')","2019-09-16 19:20:10.234" +------------------------------------------ +SELECT toInt32(-1) + N +"Date","2019-09-15" +"DateTime('Europe/Minsk')","2019-09-16 19:20:10" +"DateTime64(3, 'Europe/Minsk')","2019-09-16 19:20:10.234" +------------------------------------------ +SELECT N + toUInt64(1) +"Date","2019-09-17" +"DateTime('Europe/Minsk')","2019-09-16 19:20:12" +"DateTime64(3, 'Europe/Minsk')","2019-09-16 19:20:12.234" +------------------------------------------ +SELECT toUInt64(1) + N +"Date","2019-09-17" +"DateTime('Europe/Minsk')","2019-09-16 19:20:12" +"DateTime64(3, 'Europe/Minsk')","2019-09-16 19:20:12.234" +------------------------------------------ +SELECT N + toInt64(-1) +"Date","2019-09-15" +"DateTime('Europe/Minsk')","2019-09-16 19:20:10" +"DateTime64(3, 'Europe/Minsk')","2019-09-16 19:20:10.234" +------------------------------------------ +SELECT toInt64(-1) + N +"Date","2019-09-15" +"DateTime('Europe/Minsk')","2019-09-16 19:20:10" +"DateTime64(3, 'Europe/Minsk')","2019-09-16 19:20:10.234" +------------------------------------------ SELECT N - toUInt8(1) "Date","2019-09-15" "DateTime('Europe/Minsk')","2019-09-16 19:20:10" @@ -661,6 +731,97 @@ Code: 43: Wrong order of arguments for function minus: argument of type Interval Code: 43: Wrong order of arguments for function minus: argument of type Interval cannot be first.. +Code: 43: Wrong order of arguments for function minus: argument of type Interval cannot be first.. +------------------------------------------ +SELECT N - toInt8(-1) +"Date","2019-09-17" +"DateTime('Europe/Minsk')","2019-09-16 19:20:12" +"DateTime64(3, 'Europe/Minsk')","2019-09-16 19:20:12.234" +------------------------------------------ +SELECT toInt8(-1) - N + +Code: 43: Wrong order of arguments for function minus: argument of type Interval cannot be first.. + +Code: 43: Wrong order of arguments for function minus: argument of type Interval cannot be first.. + +Code: 43: Wrong order of arguments for function minus: argument of type Interval cannot be first.. +------------------------------------------ +SELECT N - toUInt16(1) +"Date","2019-09-15" +"DateTime('Europe/Minsk')","2019-09-16 19:20:10" +"DateTime64(3, 'Europe/Minsk')","2019-09-16 19:20:10.234" +------------------------------------------ +SELECT toUInt16(1) - N + +Code: 43: Wrong order of arguments for function minus: argument of type Interval cannot be first.. + +Code: 43: Wrong order of arguments for function minus: argument of type Interval cannot be first.. + +Code: 43: Wrong order of arguments for function minus: argument of type Interval cannot be first.. +------------------------------------------ +SELECT N - toInt16(-1) +"Date","2019-09-17" +"DateTime('Europe/Minsk')","2019-09-16 19:20:12" +"DateTime64(3, 'Europe/Minsk')","2019-09-16 19:20:12.234" +------------------------------------------ +SELECT toInt16(-1) - N + +Code: 43: Wrong order of arguments for function minus: argument of type Interval cannot be first.. + +Code: 43: Wrong order of arguments for function minus: argument of type Interval cannot be first.. + +Code: 43: Wrong order of arguments for function minus: argument of type Interval cannot be first.. +------------------------------------------ +SELECT N - toUInt32(1) +"Date","2019-09-15" +"DateTime('Europe/Minsk')","2019-09-16 19:20:10" +"DateTime64(3, 'Europe/Minsk')","2019-09-16 19:20:10.234" +------------------------------------------ +SELECT toUInt32(1) - N + +Code: 43: Wrong order of arguments for function minus: argument of type Interval cannot be first.. + +Code: 43: Wrong order of arguments for function minus: argument of type Interval cannot be first.. + +Code: 43: Wrong order of arguments for function minus: argument of type Interval cannot be first.. +------------------------------------------ +SELECT N - toInt32(-1) +"Date","2019-09-17" +"DateTime('Europe/Minsk')","2019-09-16 19:20:12" +"DateTime64(3, 'Europe/Minsk')","2019-09-16 19:20:12.234" +------------------------------------------ +SELECT toInt32(-1) - N + +Code: 43: Wrong order of arguments for function minus: argument of type Interval cannot be first.. + +Code: 43: Wrong order of arguments for function minus: argument of type Interval cannot be first.. + +Code: 43: Wrong order of arguments for function minus: argument of type Interval cannot be first.. +------------------------------------------ +SELECT N - toUInt64(1) +"Date","2019-09-15" +"DateTime('Europe/Minsk')","2019-09-16 19:20:10" +"DateTime64(3, 'Europe/Minsk')","2019-09-16 19:20:10.234" +------------------------------------------ +SELECT toUInt64(1) - N + +Code: 43: Wrong order of arguments for function minus: argument of type Interval cannot be first.. + +Code: 43: Wrong order of arguments for function minus: argument of type Interval cannot be first.. + +Code: 43: Wrong order of arguments for function minus: argument of type Interval cannot be first.. +------------------------------------------ +SELECT N - toInt64(-1) +"Date","2019-09-17" +"DateTime('Europe/Minsk')","2019-09-16 19:20:12" +"DateTime64(3, 'Europe/Minsk')","2019-09-16 19:20:12.234" +------------------------------------------ +SELECT toInt64(-1) - N + +Code: 43: Wrong order of arguments for function minus: argument of type Interval cannot be first.. + +Code: 43: Wrong order of arguments for function minus: argument of type Interval cannot be first.. + Code: 43: Wrong order of arguments for function minus: argument of type Interval cannot be first.. ------------------------------------------ SELECT N == toUInt8(1) @@ -675,6 +836,90 @@ Code: 43: Illegal types of arguments (UInt8, Date) of function equals. "UInt8",0 "UInt8",0 ------------------------------------------ +SELECT N == toInt8(-1) + +Code: 43: Illegal types of arguments (Date, Int8) of function equals. +"UInt8",0 +"UInt8",0 +------------------------------------------ +SELECT toInt8(-1) == N + +Code: 43: Illegal types of arguments (Int8, Date) of function equals. +"UInt8",0 +"UInt8",0 +------------------------------------------ +SELECT N == toUInt16(1) + +Code: 43: Illegal types of arguments (Date, UInt16) of function equals. +"UInt8",0 +"UInt8",0 +------------------------------------------ +SELECT toUInt16(1) == N + +Code: 43: Illegal types of arguments (UInt16, Date) of function equals. +"UInt8",0 +"UInt8",0 +------------------------------------------ +SELECT N == toInt16(-1) + +Code: 43: Illegal types of arguments (Date, Int16) of function equals. +"UInt8",0 +"UInt8",0 +------------------------------------------ +SELECT toInt16(-1) == N + +Code: 43: Illegal types of arguments (Int16, Date) of function equals. +"UInt8",0 +"UInt8",0 +------------------------------------------ +SELECT N == toUInt32(1) + +Code: 43: Illegal types of arguments (Date, UInt32) of function equals. +"UInt8",0 +"UInt8",0 +------------------------------------------ +SELECT toUInt32(1) == N + +Code: 43: Illegal types of arguments (UInt32, Date) of function equals. +"UInt8",0 +"UInt8",0 +------------------------------------------ +SELECT N == toInt32(-1) + +Code: 43: Illegal types of arguments (Date, Int32) of function equals. +"UInt8",0 +"UInt8",0 +------------------------------------------ +SELECT toInt32(-1) == N + +Code: 43: Illegal types of arguments (Int32, Date) of function equals. +"UInt8",0 +"UInt8",0 +------------------------------------------ +SELECT N == toUInt64(1) + +Code: 43: Illegal types of arguments (Date, UInt64) of function equals. +"UInt8",0 +"UInt8",0 +------------------------------------------ +SELECT toUInt64(1) == N + +Code: 43: Illegal types of arguments (UInt64, Date) of function equals. +"UInt8",0 +"UInt8",0 +------------------------------------------ +SELECT N == toInt64(-1) + +Code: 43: Illegal types of arguments (Date, Int64) of function equals. +"UInt8",0 +"UInt8",0 +------------------------------------------ +SELECT toInt64(-1) == N + +Code: 43: Illegal types of arguments (Int64, Date) of function equals. +"UInt8",0 +"UInt8",0 +------------------------------------------ SELECT N != toUInt8(1) Code: 43: Illegal types of arguments (Date, UInt8) of function notEquals. @@ -687,6 +932,90 @@ Code: 43: Illegal types of arguments (UInt8, Date) of function notEquals. "UInt8",1 "UInt8",1 ------------------------------------------ +SELECT N != toInt8(-1) + +Code: 43: Illegal types of arguments (Date, Int8) of function notEquals. +"UInt8",1 +"UInt8",1 +------------------------------------------ +SELECT toInt8(-1) != N + +Code: 43: Illegal types of arguments (Int8, Date) of function notEquals. +"UInt8",1 +"UInt8",1 +------------------------------------------ +SELECT N != toUInt16(1) + +Code: 43: Illegal types of arguments (Date, UInt16) of function notEquals. +"UInt8",1 +"UInt8",1 +------------------------------------------ +SELECT toUInt16(1) != N + +Code: 43: Illegal types of arguments (UInt16, Date) of function notEquals. +"UInt8",1 +"UInt8",1 +------------------------------------------ +SELECT N != toInt16(-1) + +Code: 43: Illegal types of arguments (Date, Int16) of function notEquals. +"UInt8",1 +"UInt8",1 +------------------------------------------ +SELECT toInt16(-1) != N + +Code: 43: Illegal types of arguments (Int16, Date) of function notEquals. +"UInt8",1 +"UInt8",1 +------------------------------------------ +SELECT N != toUInt32(1) + +Code: 43: Illegal types of arguments (Date, UInt32) of function notEquals. +"UInt8",1 +"UInt8",1 +------------------------------------------ +SELECT toUInt32(1) != N + +Code: 43: Illegal types of arguments (UInt32, Date) of function notEquals. +"UInt8",1 +"UInt8",1 +------------------------------------------ +SELECT N != toInt32(-1) + +Code: 43: Illegal types of arguments (Date, Int32) of function notEquals. +"UInt8",1 +"UInt8",1 +------------------------------------------ +SELECT toInt32(-1) != N + +Code: 43: Illegal types of arguments (Int32, Date) of function notEquals. +"UInt8",1 +"UInt8",1 +------------------------------------------ +SELECT N != toUInt64(1) + +Code: 43: Illegal types of arguments (Date, UInt64) of function notEquals. +"UInt8",1 +"UInt8",1 +------------------------------------------ +SELECT toUInt64(1) != N + +Code: 43: Illegal types of arguments (UInt64, Date) of function notEquals. +"UInt8",1 +"UInt8",1 +------------------------------------------ +SELECT N != toInt64(-1) + +Code: 43: Illegal types of arguments (Date, Int64) of function notEquals. +"UInt8",1 +"UInt8",1 +------------------------------------------ +SELECT toInt64(-1) != N + +Code: 43: Illegal types of arguments (Int64, Date) of function notEquals. +"UInt8",1 +"UInt8",1 +------------------------------------------ SELECT N < toUInt8(1) Code: 43: Illegal types of arguments (Date, UInt8) of function less. @@ -699,89 +1028,6 @@ Code: 43: Illegal types of arguments (UInt8, Date) of function less. "UInt8",1 "UInt8",1 ------------------------------------------ -SELECT N <= toUInt8(1) - -Code: 43: Illegal types of arguments (Date, UInt8) of function lessOrEquals. -"UInt8",0 -"UInt8",0 ------------------------------------------- -SELECT toUInt8(1) <= N - -Code: 43: Illegal types of arguments (UInt8, Date) of function lessOrEquals. -"UInt8",1 -"UInt8",1 ------------------------------------------- -SELECT N > toUInt8(1) - -Code: 43: Illegal types of arguments (Date, UInt8) of function greater. -"UInt8",1 -"UInt8",1 ------------------------------------------- -SELECT toUInt8(1) > N - -Code: 43: Illegal types of arguments (UInt8, Date) of function greater. -"UInt8",0 -"UInt8",0 ------------------------------------------- -SELECT N >= toUInt8(1) - -Code: 43: Illegal types of arguments (Date, UInt8) of function greaterOrEquals. -"UInt8",1 -"UInt8",1 ------------------------------------------- -SELECT toUInt8(1) >= N - -Code: 43: Illegal types of arguments (UInt8, Date) of function greaterOrEquals. -"UInt8",0 -"UInt8",0 ------------------------------------------- -SELECT N + toInt8(-1) -"Date","2019-09-15" -"DateTime('Europe/Minsk')","2019-09-16 19:20:10" -"DateTime64(3, 'Europe/Minsk')","2019-09-16 19:20:10.234" ------------------------------------------- -SELECT toInt8(-1) + N -"Date","2019-09-15" -"DateTime('Europe/Minsk')","2019-09-16 19:20:10" -"DateTime64(3, 'Europe/Minsk')","2019-09-16 19:20:10.234" ------------------------------------------- -SELECT N - toInt8(-1) -"Date","2019-09-17" -"DateTime('Europe/Minsk')","2019-09-16 19:20:12" -"DateTime64(3, 'Europe/Minsk')","2019-09-16 19:20:12.234" ------------------------------------------- -SELECT toInt8(-1) - N - -Code: 43: Wrong order of arguments for function minus: argument of type Interval cannot be first.. - -Code: 43: Wrong order of arguments for function minus: argument of type Interval cannot be first.. - -Code: 43: Wrong order of arguments for function minus: argument of type Interval cannot be first.. ------------------------------------------- -SELECT N == toInt8(-1) - -Code: 43: Illegal types of arguments (Date, Int8) of function equals. -"UInt8",0 -"UInt8",0 ------------------------------------------- -SELECT toInt8(-1) == N - -Code: 43: Illegal types of arguments (Int8, Date) of function equals. -"UInt8",0 -"UInt8",0 ------------------------------------------- -SELECT N != toInt8(-1) - -Code: 43: Illegal types of arguments (Date, Int8) of function notEquals. -"UInt8",1 -"UInt8",1 ------------------------------------------- -SELECT toInt8(-1) != N - -Code: 43: Illegal types of arguments (Int8, Date) of function notEquals. -"UInt8",1 -"UInt8",1 ------------------------------------------- SELECT N < toInt8(-1) Code: 43: Illegal types of arguments (Date, Int8) of function less. @@ -794,89 +1040,6 @@ Code: 43: Illegal types of arguments (Int8, Date) of function less. "UInt8",1 "UInt8",1 ------------------------------------------ -SELECT N <= toInt8(-1) - -Code: 43: Illegal types of arguments (Date, Int8) of function lessOrEquals. -"UInt8",0 -"UInt8",0 ------------------------------------------- -SELECT toInt8(-1) <= N - -Code: 43: Illegal types of arguments (Int8, Date) of function lessOrEquals. -"UInt8",1 -"UInt8",1 ------------------------------------------- -SELECT N > toInt8(-1) - -Code: 43: Illegal types of arguments (Date, Int8) of function greater. -"UInt8",1 -"UInt8",1 ------------------------------------------- -SELECT toInt8(-1) > N - -Code: 43: Illegal types of arguments (Int8, Date) of function greater. -"UInt8",0 -"UInt8",0 ------------------------------------------- -SELECT N >= toInt8(-1) - -Code: 43: Illegal types of arguments (Date, Int8) of function greaterOrEquals. -"UInt8",1 -"UInt8",1 ------------------------------------------- -SELECT toInt8(-1) >= N - -Code: 43: Illegal types of arguments (Int8, Date) of function greaterOrEquals. -"UInt8",0 -"UInt8",0 ------------------------------------------- -SELECT N + toUInt16(1) -"Date","2019-09-17" -"DateTime('Europe/Minsk')","2019-09-16 19:20:12" -"DateTime64(3, 'Europe/Minsk')","2019-09-16 19:20:12.234" ------------------------------------------- -SELECT toUInt16(1) + N -"Date","2019-09-17" -"DateTime('Europe/Minsk')","2019-09-16 19:20:12" -"DateTime64(3, 'Europe/Minsk')","2019-09-16 19:20:12.234" ------------------------------------------- -SELECT N - toUInt16(1) -"Date","2019-09-15" -"DateTime('Europe/Minsk')","2019-09-16 19:20:10" -"DateTime64(3, 'Europe/Minsk')","2019-09-16 19:20:10.234" ------------------------------------------- -SELECT toUInt16(1) - N - -Code: 43: Wrong order of arguments for function minus: argument of type Interval cannot be first.. - -Code: 43: Wrong order of arguments for function minus: argument of type Interval cannot be first.. - -Code: 43: Wrong order of arguments for function minus: argument of type Interval cannot be first.. ------------------------------------------- -SELECT N == toUInt16(1) - -Code: 43: Illegal types of arguments (Date, UInt16) of function equals. -"UInt8",0 -"UInt8",0 ------------------------------------------- -SELECT toUInt16(1) == N - -Code: 43: Illegal types of arguments (UInt16, Date) of function equals. -"UInt8",0 -"UInt8",0 ------------------------------------------- -SELECT N != toUInt16(1) - -Code: 43: Illegal types of arguments (Date, UInt16) of function notEquals. -"UInt8",1 -"UInt8",1 ------------------------------------------- -SELECT toUInt16(1) != N - -Code: 43: Illegal types of arguments (UInt16, Date) of function notEquals. -"UInt8",1 -"UInt8",1 ------------------------------------------- SELECT N < toUInt16(1) Code: 43: Illegal types of arguments (Date, UInt16) of function less. @@ -889,89 +1052,6 @@ Code: 43: Illegal types of arguments (UInt16, Date) of function less. "UInt8",1 "UInt8",1 ------------------------------------------ -SELECT N <= toUInt16(1) - -Code: 43: Illegal types of arguments (Date, UInt16) of function lessOrEquals. -"UInt8",0 -"UInt8",0 ------------------------------------------- -SELECT toUInt16(1) <= N - -Code: 43: Illegal types of arguments (UInt16, Date) of function lessOrEquals. -"UInt8",1 -"UInt8",1 ------------------------------------------- -SELECT N > toUInt16(1) - -Code: 43: Illegal types of arguments (Date, UInt16) of function greater. -"UInt8",1 -"UInt8",1 ------------------------------------------- -SELECT toUInt16(1) > N - -Code: 43: Illegal types of arguments (UInt16, Date) of function greater. -"UInt8",0 -"UInt8",0 ------------------------------------------- -SELECT N >= toUInt16(1) - -Code: 43: Illegal types of arguments (Date, UInt16) of function greaterOrEquals. -"UInt8",1 -"UInt8",1 ------------------------------------------- -SELECT toUInt16(1) >= N - -Code: 43: Illegal types of arguments (UInt16, Date) of function greaterOrEquals. -"UInt8",0 -"UInt8",0 ------------------------------------------- -SELECT N + toInt16(-1) -"Date","2019-09-15" -"DateTime('Europe/Minsk')","2019-09-16 19:20:10" -"DateTime64(3, 'Europe/Minsk')","2019-09-16 19:20:10.234" ------------------------------------------- -SELECT toInt16(-1) + N -"Date","2019-09-15" -"DateTime('Europe/Minsk')","2019-09-16 19:20:10" -"DateTime64(3, 'Europe/Minsk')","2019-09-16 19:20:10.234" ------------------------------------------- -SELECT N - toInt16(-1) -"Date","2019-09-17" -"DateTime('Europe/Minsk')","2019-09-16 19:20:12" -"DateTime64(3, 'Europe/Minsk')","2019-09-16 19:20:12.234" ------------------------------------------- -SELECT toInt16(-1) - N - -Code: 43: Wrong order of arguments for function minus: argument of type Interval cannot be first.. - -Code: 43: Wrong order of arguments for function minus: argument of type Interval cannot be first.. - -Code: 43: Wrong order of arguments for function minus: argument of type Interval cannot be first.. ------------------------------------------- -SELECT N == toInt16(-1) - -Code: 43: Illegal types of arguments (Date, Int16) of function equals. -"UInt8",0 -"UInt8",0 ------------------------------------------- -SELECT toInt16(-1) == N - -Code: 43: Illegal types of arguments (Int16, Date) of function equals. -"UInt8",0 -"UInt8",0 ------------------------------------------- -SELECT N != toInt16(-1) - -Code: 43: Illegal types of arguments (Date, Int16) of function notEquals. -"UInt8",1 -"UInt8",1 ------------------------------------------- -SELECT toInt16(-1) != N - -Code: 43: Illegal types of arguments (Int16, Date) of function notEquals. -"UInt8",1 -"UInt8",1 ------------------------------------------- SELECT N < toInt16(-1) Code: 43: Illegal types of arguments (Date, Int16) of function less. @@ -984,89 +1064,6 @@ Code: 43: Illegal types of arguments (Int16, Date) of function less. "UInt8",1 "UInt8",1 ------------------------------------------ -SELECT N <= toInt16(-1) - -Code: 43: Illegal types of arguments (Date, Int16) of function lessOrEquals. -"UInt8",0 -"UInt8",0 ------------------------------------------- -SELECT toInt16(-1) <= N - -Code: 43: Illegal types of arguments (Int16, Date) of function lessOrEquals. -"UInt8",1 -"UInt8",1 ------------------------------------------- -SELECT N > toInt16(-1) - -Code: 43: Illegal types of arguments (Date, Int16) of function greater. -"UInt8",1 -"UInt8",1 ------------------------------------------- -SELECT toInt16(-1) > N - -Code: 43: Illegal types of arguments (Int16, Date) of function greater. -"UInt8",0 -"UInt8",0 ------------------------------------------- -SELECT N >= toInt16(-1) - -Code: 43: Illegal types of arguments (Date, Int16) of function greaterOrEquals. -"UInt8",1 -"UInt8",1 ------------------------------------------- -SELECT toInt16(-1) >= N - -Code: 43: Illegal types of arguments (Int16, Date) of function greaterOrEquals. -"UInt8",0 -"UInt8",0 ------------------------------------------- -SELECT N + toUInt32(1) -"Date","2019-09-17" -"DateTime('Europe/Minsk')","2019-09-16 19:20:12" -"DateTime64(3, 'Europe/Minsk')","2019-09-16 19:20:12.234" ------------------------------------------- -SELECT toUInt32(1) + N -"Date","2019-09-17" -"DateTime('Europe/Minsk')","2019-09-16 19:20:12" -"DateTime64(3, 'Europe/Minsk')","2019-09-16 19:20:12.234" ------------------------------------------- -SELECT N - toUInt32(1) -"Date","2019-09-15" -"DateTime('Europe/Minsk')","2019-09-16 19:20:10" -"DateTime64(3, 'Europe/Minsk')","2019-09-16 19:20:10.234" ------------------------------------------- -SELECT toUInt32(1) - N - -Code: 43: Wrong order of arguments for function minus: argument of type Interval cannot be first.. - -Code: 43: Wrong order of arguments for function minus: argument of type Interval cannot be first.. - -Code: 43: Wrong order of arguments for function minus: argument of type Interval cannot be first.. ------------------------------------------- -SELECT N == toUInt32(1) - -Code: 43: Illegal types of arguments (Date, UInt32) of function equals. -"UInt8",0 -"UInt8",0 ------------------------------------------- -SELECT toUInt32(1) == N - -Code: 43: Illegal types of arguments (UInt32, Date) of function equals. -"UInt8",0 -"UInt8",0 ------------------------------------------- -SELECT N != toUInt32(1) - -Code: 43: Illegal types of arguments (Date, UInt32) of function notEquals. -"UInt8",1 -"UInt8",1 ------------------------------------------- -SELECT toUInt32(1) != N - -Code: 43: Illegal types of arguments (UInt32, Date) of function notEquals. -"UInt8",1 -"UInt8",1 ------------------------------------------- SELECT N < toUInt32(1) Code: 43: Illegal types of arguments (Date, UInt32) of function less. @@ -1079,89 +1076,6 @@ Code: 43: Illegal types of arguments (UInt32, Date) of function less. "UInt8",1 "UInt8",1 ------------------------------------------ -SELECT N <= toUInt32(1) - -Code: 43: Illegal types of arguments (Date, UInt32) of function lessOrEquals. -"UInt8",0 -"UInt8",0 ------------------------------------------- -SELECT toUInt32(1) <= N - -Code: 43: Illegal types of arguments (UInt32, Date) of function lessOrEquals. -"UInt8",1 -"UInt8",1 ------------------------------------------- -SELECT N > toUInt32(1) - -Code: 43: Illegal types of arguments (Date, UInt32) of function greater. -"UInt8",1 -"UInt8",1 ------------------------------------------- -SELECT toUInt32(1) > N - -Code: 43: Illegal types of arguments (UInt32, Date) of function greater. -"UInt8",0 -"UInt8",0 ------------------------------------------- -SELECT N >= toUInt32(1) - -Code: 43: Illegal types of arguments (Date, UInt32) of function greaterOrEquals. -"UInt8",1 -"UInt8",1 ------------------------------------------- -SELECT toUInt32(1) >= N - -Code: 43: Illegal types of arguments (UInt32, Date) of function greaterOrEquals. -"UInt8",0 -"UInt8",0 ------------------------------------------- -SELECT N + toInt32(-1) -"Date","2019-09-15" -"DateTime('Europe/Minsk')","2019-09-16 19:20:10" -"DateTime64(3, 'Europe/Minsk')","2019-09-16 19:20:10.234" ------------------------------------------- -SELECT toInt32(-1) + N -"Date","2019-09-15" -"DateTime('Europe/Minsk')","2019-09-16 19:20:10" -"DateTime64(3, 'Europe/Minsk')","2019-09-16 19:20:10.234" ------------------------------------------- -SELECT N - toInt32(-1) -"Date","2019-09-17" -"DateTime('Europe/Minsk')","2019-09-16 19:20:12" -"DateTime64(3, 'Europe/Minsk')","2019-09-16 19:20:12.234" ------------------------------------------- -SELECT toInt32(-1) - N - -Code: 43: Wrong order of arguments for function minus: argument of type Interval cannot be first.. - -Code: 43: Wrong order of arguments for function minus: argument of type Interval cannot be first.. - -Code: 43: Wrong order of arguments for function minus: argument of type Interval cannot be first.. ------------------------------------------- -SELECT N == toInt32(-1) - -Code: 43: Illegal types of arguments (Date, Int32) of function equals. -"UInt8",0 -"UInt8",0 ------------------------------------------- -SELECT toInt32(-1) == N - -Code: 43: Illegal types of arguments (Int32, Date) of function equals. -"UInt8",0 -"UInt8",0 ------------------------------------------- -SELECT N != toInt32(-1) - -Code: 43: Illegal types of arguments (Date, Int32) of function notEquals. -"UInt8",1 -"UInt8",1 ------------------------------------------- -SELECT toInt32(-1) != N - -Code: 43: Illegal types of arguments (Int32, Date) of function notEquals. -"UInt8",1 -"UInt8",1 ------------------------------------------- SELECT N < toInt32(-1) Code: 43: Illegal types of arguments (Date, Int32) of function less. @@ -1174,89 +1088,6 @@ Code: 43: Illegal types of arguments (Int32, Date) of function less. "UInt8",1 "UInt8",1 ------------------------------------------ -SELECT N <= toInt32(-1) - -Code: 43: Illegal types of arguments (Date, Int32) of function lessOrEquals. -"UInt8",0 -"UInt8",0 ------------------------------------------- -SELECT toInt32(-1) <= N - -Code: 43: Illegal types of arguments (Int32, Date) of function lessOrEquals. -"UInt8",1 -"UInt8",1 ------------------------------------------- -SELECT N > toInt32(-1) - -Code: 43: Illegal types of arguments (Date, Int32) of function greater. -"UInt8",1 -"UInt8",1 ------------------------------------------- -SELECT toInt32(-1) > N - -Code: 43: Illegal types of arguments (Int32, Date) of function greater. -"UInt8",0 -"UInt8",0 ------------------------------------------- -SELECT N >= toInt32(-1) - -Code: 43: Illegal types of arguments (Date, Int32) of function greaterOrEquals. -"UInt8",1 -"UInt8",1 ------------------------------------------- -SELECT toInt32(-1) >= N - -Code: 43: Illegal types of arguments (Int32, Date) of function greaterOrEquals. -"UInt8",0 -"UInt8",0 ------------------------------------------- -SELECT N + toUInt64(1) -"Date","2019-09-17" -"DateTime('Europe/Minsk')","2019-09-16 19:20:12" -"DateTime64(3, 'Europe/Minsk')","2019-09-16 19:20:12.234" ------------------------------------------- -SELECT toUInt64(1) + N -"Date","2019-09-17" -"DateTime('Europe/Minsk')","2019-09-16 19:20:12" -"DateTime64(3, 'Europe/Minsk')","2019-09-16 19:20:12.234" ------------------------------------------- -SELECT N - toUInt64(1) -"Date","2019-09-15" -"DateTime('Europe/Minsk')","2019-09-16 19:20:10" -"DateTime64(3, 'Europe/Minsk')","2019-09-16 19:20:10.234" ------------------------------------------- -SELECT toUInt64(1) - N - -Code: 43: Wrong order of arguments for function minus: argument of type Interval cannot be first.. - -Code: 43: Wrong order of arguments for function minus: argument of type Interval cannot be first.. - -Code: 43: Wrong order of arguments for function minus: argument of type Interval cannot be first.. ------------------------------------------- -SELECT N == toUInt64(1) - -Code: 43: Illegal types of arguments (Date, UInt64) of function equals. -"UInt8",0 -"UInt8",0 ------------------------------------------- -SELECT toUInt64(1) == N - -Code: 43: Illegal types of arguments (UInt64, Date) of function equals. -"UInt8",0 -"UInt8",0 ------------------------------------------- -SELECT N != toUInt64(1) - -Code: 43: Illegal types of arguments (Date, UInt64) of function notEquals. -"UInt8",1 -"UInt8",1 ------------------------------------------- -SELECT toUInt64(1) != N - -Code: 43: Illegal types of arguments (UInt64, Date) of function notEquals. -"UInt8",1 -"UInt8",1 ------------------------------------------- SELECT N < toUInt64(1) Code: 43: Illegal types of arguments (Date, UInt64) of function less. @@ -1269,89 +1100,6 @@ Code: 43: Illegal types of arguments (UInt64, Date) of function less. "UInt8",1 "UInt8",1 ------------------------------------------ -SELECT N <= toUInt64(1) - -Code: 43: Illegal types of arguments (Date, UInt64) of function lessOrEquals. -"UInt8",0 -"UInt8",0 ------------------------------------------- -SELECT toUInt64(1) <= N - -Code: 43: Illegal types of arguments (UInt64, Date) of function lessOrEquals. -"UInt8",1 -"UInt8",1 ------------------------------------------- -SELECT N > toUInt64(1) - -Code: 43: Illegal types of arguments (Date, UInt64) of function greater. -"UInt8",1 -"UInt8",1 ------------------------------------------- -SELECT toUInt64(1) > N - -Code: 43: Illegal types of arguments (UInt64, Date) of function greater. -"UInt8",0 -"UInt8",0 ------------------------------------------- -SELECT N >= toUInt64(1) - -Code: 43: Illegal types of arguments (Date, UInt64) of function greaterOrEquals. -"UInt8",1 -"UInt8",1 ------------------------------------------- -SELECT toUInt64(1) >= N - -Code: 43: Illegal types of arguments (UInt64, Date) of function greaterOrEquals. -"UInt8",0 -"UInt8",0 ------------------------------------------- -SELECT N + toInt64(-1) -"Date","2019-09-15" -"DateTime('Europe/Minsk')","2019-09-16 19:20:10" -"DateTime64(3, 'Europe/Minsk')","2019-09-16 19:20:10.234" ------------------------------------------- -SELECT toInt64(-1) + N -"Date","2019-09-15" -"DateTime('Europe/Minsk')","2019-09-16 19:20:10" -"DateTime64(3, 'Europe/Minsk')","2019-09-16 19:20:10.234" ------------------------------------------- -SELECT N - toInt64(-1) -"Date","2019-09-17" -"DateTime('Europe/Minsk')","2019-09-16 19:20:12" -"DateTime64(3, 'Europe/Minsk')","2019-09-16 19:20:12.234" ------------------------------------------- -SELECT toInt64(-1) - N - -Code: 43: Wrong order of arguments for function minus: argument of type Interval cannot be first.. - -Code: 43: Wrong order of arguments for function minus: argument of type Interval cannot be first.. - -Code: 43: Wrong order of arguments for function minus: argument of type Interval cannot be first.. ------------------------------------------- -SELECT N == toInt64(-1) - -Code: 43: Illegal types of arguments (Date, Int64) of function equals. -"UInt8",0 -"UInt8",0 ------------------------------------------- -SELECT toInt64(-1) == N - -Code: 43: Illegal types of arguments (Int64, Date) of function equals. -"UInt8",0 -"UInt8",0 ------------------------------------------- -SELECT N != toInt64(-1) - -Code: 43: Illegal types of arguments (Date, Int64) of function notEquals. -"UInt8",1 -"UInt8",1 ------------------------------------------- -SELECT toInt64(-1) != N - -Code: 43: Illegal types of arguments (Int64, Date) of function notEquals. -"UInt8",1 -"UInt8",1 ------------------------------------------- SELECT N < toInt64(-1) Code: 43: Illegal types of arguments (Date, Int64) of function less. @@ -1364,6 +1112,90 @@ Code: 43: Illegal types of arguments (Int64, Date) of function less. "UInt8",1 "UInt8",1 ------------------------------------------ +SELECT N <= toUInt8(1) + +Code: 43: Illegal types of arguments (Date, UInt8) of function lessOrEquals. +"UInt8",0 +"UInt8",0 +------------------------------------------ +SELECT toUInt8(1) <= N + +Code: 43: Illegal types of arguments (UInt8, Date) of function lessOrEquals. +"UInt8",1 +"UInt8",1 +------------------------------------------ +SELECT N <= toInt8(-1) + +Code: 43: Illegal types of arguments (Date, Int8) of function lessOrEquals. +"UInt8",0 +"UInt8",0 +------------------------------------------ +SELECT toInt8(-1) <= N + +Code: 43: Illegal types of arguments (Int8, Date) of function lessOrEquals. +"UInt8",1 +"UInt8",1 +------------------------------------------ +SELECT N <= toUInt16(1) + +Code: 43: Illegal types of arguments (Date, UInt16) of function lessOrEquals. +"UInt8",0 +"UInt8",0 +------------------------------------------ +SELECT toUInt16(1) <= N + +Code: 43: Illegal types of arguments (UInt16, Date) of function lessOrEquals. +"UInt8",1 +"UInt8",1 +------------------------------------------ +SELECT N <= toInt16(-1) + +Code: 43: Illegal types of arguments (Date, Int16) of function lessOrEquals. +"UInt8",0 +"UInt8",0 +------------------------------------------ +SELECT toInt16(-1) <= N + +Code: 43: Illegal types of arguments (Int16, Date) of function lessOrEquals. +"UInt8",1 +"UInt8",1 +------------------------------------------ +SELECT N <= toUInt32(1) + +Code: 43: Illegal types of arguments (Date, UInt32) of function lessOrEquals. +"UInt8",0 +"UInt8",0 +------------------------------------------ +SELECT toUInt32(1) <= N + +Code: 43: Illegal types of arguments (UInt32, Date) of function lessOrEquals. +"UInt8",1 +"UInt8",1 +------------------------------------------ +SELECT N <= toInt32(-1) + +Code: 43: Illegal types of arguments (Date, Int32) of function lessOrEquals. +"UInt8",0 +"UInt8",0 +------------------------------------------ +SELECT toInt32(-1) <= N + +Code: 43: Illegal types of arguments (Int32, Date) of function lessOrEquals. +"UInt8",1 +"UInt8",1 +------------------------------------------ +SELECT N <= toUInt64(1) + +Code: 43: Illegal types of arguments (Date, UInt64) of function lessOrEquals. +"UInt8",0 +"UInt8",0 +------------------------------------------ +SELECT toUInt64(1) <= N + +Code: 43: Illegal types of arguments (UInt64, Date) of function lessOrEquals. +"UInt8",1 +"UInt8",1 +------------------------------------------ SELECT N <= toInt64(-1) Code: 43: Illegal types of arguments (Date, Int64) of function lessOrEquals. @@ -1376,6 +1208,90 @@ Code: 43: Illegal types of arguments (Int64, Date) of function lessOrEquals. "UInt8",1 "UInt8",1 ------------------------------------------ +SELECT N > toUInt8(1) + +Code: 43: Illegal types of arguments (Date, UInt8) of function greater. +"UInt8",1 +"UInt8",1 +------------------------------------------ +SELECT toUInt8(1) > N + +Code: 43: Illegal types of arguments (UInt8, Date) of function greater. +"UInt8",0 +"UInt8",0 +------------------------------------------ +SELECT N > toInt8(-1) + +Code: 43: Illegal types of arguments (Date, Int8) of function greater. +"UInt8",1 +"UInt8",1 +------------------------------------------ +SELECT toInt8(-1) > N + +Code: 43: Illegal types of arguments (Int8, Date) of function greater. +"UInt8",0 +"UInt8",0 +------------------------------------------ +SELECT N > toUInt16(1) + +Code: 43: Illegal types of arguments (Date, UInt16) of function greater. +"UInt8",1 +"UInt8",1 +------------------------------------------ +SELECT toUInt16(1) > N + +Code: 43: Illegal types of arguments (UInt16, Date) of function greater. +"UInt8",0 +"UInt8",0 +------------------------------------------ +SELECT N > toInt16(-1) + +Code: 43: Illegal types of arguments (Date, Int16) of function greater. +"UInt8",1 +"UInt8",1 +------------------------------------------ +SELECT toInt16(-1) > N + +Code: 43: Illegal types of arguments (Int16, Date) of function greater. +"UInt8",0 +"UInt8",0 +------------------------------------------ +SELECT N > toUInt32(1) + +Code: 43: Illegal types of arguments (Date, UInt32) of function greater. +"UInt8",1 +"UInt8",1 +------------------------------------------ +SELECT toUInt32(1) > N + +Code: 43: Illegal types of arguments (UInt32, Date) of function greater. +"UInt8",0 +"UInt8",0 +------------------------------------------ +SELECT N > toInt32(-1) + +Code: 43: Illegal types of arguments (Date, Int32) of function greater. +"UInt8",1 +"UInt8",1 +------------------------------------------ +SELECT toInt32(-1) > N + +Code: 43: Illegal types of arguments (Int32, Date) of function greater. +"UInt8",0 +"UInt8",0 +------------------------------------------ +SELECT N > toUInt64(1) + +Code: 43: Illegal types of arguments (Date, UInt64) of function greater. +"UInt8",1 +"UInt8",1 +------------------------------------------ +SELECT toUInt64(1) > N + +Code: 43: Illegal types of arguments (UInt64, Date) of function greater. +"UInt8",0 +"UInt8",0 +------------------------------------------ SELECT N > toInt64(-1) Code: 43: Illegal types of arguments (Date, Int64) of function greater. @@ -1388,6 +1304,90 @@ Code: 43: Illegal types of arguments (Int64, Date) of function greater. "UInt8",0 "UInt8",0 ------------------------------------------ +SELECT N >= toUInt8(1) + +Code: 43: Illegal types of arguments (Date, UInt8) of function greaterOrEquals. +"UInt8",1 +"UInt8",1 +------------------------------------------ +SELECT toUInt8(1) >= N + +Code: 43: Illegal types of arguments (UInt8, Date) of function greaterOrEquals. +"UInt8",0 +"UInt8",0 +------------------------------------------ +SELECT N >= toInt8(-1) + +Code: 43: Illegal types of arguments (Date, Int8) of function greaterOrEquals. +"UInt8",1 +"UInt8",1 +------------------------------------------ +SELECT toInt8(-1) >= N + +Code: 43: Illegal types of arguments (Int8, Date) of function greaterOrEquals. +"UInt8",0 +"UInt8",0 +------------------------------------------ +SELECT N >= toUInt16(1) + +Code: 43: Illegal types of arguments (Date, UInt16) of function greaterOrEquals. +"UInt8",1 +"UInt8",1 +------------------------------------------ +SELECT toUInt16(1) >= N + +Code: 43: Illegal types of arguments (UInt16, Date) of function greaterOrEquals. +"UInt8",0 +"UInt8",0 +------------------------------------------ +SELECT N >= toInt16(-1) + +Code: 43: Illegal types of arguments (Date, Int16) of function greaterOrEquals. +"UInt8",1 +"UInt8",1 +------------------------------------------ +SELECT toInt16(-1) >= N + +Code: 43: Illegal types of arguments (Int16, Date) of function greaterOrEquals. +"UInt8",0 +"UInt8",0 +------------------------------------------ +SELECT N >= toUInt32(1) + +Code: 43: Illegal types of arguments (Date, UInt32) of function greaterOrEquals. +"UInt8",1 +"UInt8",1 +------------------------------------------ +SELECT toUInt32(1) >= N + +Code: 43: Illegal types of arguments (UInt32, Date) of function greaterOrEquals. +"UInt8",0 +"UInt8",0 +------------------------------------------ +SELECT N >= toInt32(-1) + +Code: 43: Illegal types of arguments (Date, Int32) of function greaterOrEquals. +"UInt8",1 +"UInt8",1 +------------------------------------------ +SELECT toInt32(-1) >= N + +Code: 43: Illegal types of arguments (Int32, Date) of function greaterOrEquals. +"UInt8",0 +"UInt8",0 +------------------------------------------ +SELECT N >= toUInt64(1) + +Code: 43: Illegal types of arguments (Date, UInt64) of function greaterOrEquals. +"UInt8",1 +"UInt8",1 +------------------------------------------ +SELECT toUInt64(1) >= N + +Code: 43: Illegal types of arguments (UInt64, Date) of function greaterOrEquals. +"UInt8",0 +"UInt8",0 +------------------------------------------ SELECT N >= toInt64(-1) Code: 43: Illegal types of arguments (Date, Int64) of function greaterOrEquals. diff --git a/tests/queries/0_stateless/00960_live_view_watch_events_live.py b/tests/queries/0_stateless/00960_live_view_watch_events_live.py index 3349175dee8..780c31f20ca 100755 --- a/tests/queries/0_stateless/00960_live_view_watch_events_live.py +++ b/tests/queries/0_stateless/00960_live_view_watch_events_live.py @@ -1,4 +1,4 @@ -#!/usr/bin/env python +#!/usr/bin/env python3 import os import sys import signal diff --git a/tests/queries/0_stateless/00962_temporary_live_view_watch_live.py b/tests/queries/0_stateless/00962_temporary_live_view_watch_live.py index c6a5251fbee..073c8c88e2b 100755 --- a/tests/queries/0_stateless/00962_temporary_live_view_watch_live.py +++ b/tests/queries/0_stateless/00962_temporary_live_view_watch_live.py @@ -1,4 +1,4 @@ -#!/usr/bin/env python +#!/usr/bin/env python3 import os import sys import signal diff --git a/tests/queries/0_stateless/00963_temporary_live_view_watch_live_timeout.py.disabled b/tests/queries/0_stateless/00963_temporary_live_view_watch_live_timeout.py.disabled index 525e7022156..cfd218d5983 100755 --- a/tests/queries/0_stateless/00963_temporary_live_view_watch_live_timeout.py.disabled +++ b/tests/queries/0_stateless/00963_temporary_live_view_watch_live_timeout.py.disabled @@ -1,4 +1,4 @@ -#!/usr/bin/env python +#!/usr/bin/env python3 import os import sys import signal diff --git a/tests/queries/0_stateless/00964_live_view_watch_events_heartbeat.py b/tests/queries/0_stateless/00964_live_view_watch_events_heartbeat.py index 794d04a429c..03b598fe46f 100755 --- a/tests/queries/0_stateless/00964_live_view_watch_events_heartbeat.py +++ b/tests/queries/0_stateless/00964_live_view_watch_events_heartbeat.py @@ -1,4 +1,4 @@ -#!/usr/bin/env python +#!/usr/bin/env python3 import os import sys import signal diff --git a/tests/queries/0_stateless/00965_live_view_watch_heartbeat.py b/tests/queries/0_stateless/00965_live_view_watch_heartbeat.py index 1d557f09f4d..15c4e763634 100755 --- a/tests/queries/0_stateless/00965_live_view_watch_heartbeat.py +++ b/tests/queries/0_stateless/00965_live_view_watch_heartbeat.py @@ -1,4 +1,4 @@ -#!/usr/bin/env python +#!/usr/bin/env python3 import os import sys import signal diff --git a/tests/queries/0_stateless/00966_live_view_watch_events_http.py b/tests/queries/0_stateless/00966_live_view_watch_events_http.py index 72ab3ea8818..3d407ec5602 100755 --- a/tests/queries/0_stateless/00966_live_view_watch_events_http.py +++ b/tests/queries/0_stateless/00966_live_view_watch_events_http.py @@ -1,4 +1,4 @@ -#!/usr/bin/env python +#!/usr/bin/env python3 import os import sys diff --git a/tests/queries/0_stateless/00967_live_view_watch_http.py b/tests/queries/0_stateless/00967_live_view_watch_http.py index e2f33971c3d..d26bb5402e7 100755 --- a/tests/queries/0_stateless/00967_live_view_watch_http.py +++ b/tests/queries/0_stateless/00967_live_view_watch_http.py @@ -1,4 +1,4 @@ -#!/usr/bin/env python +#!/usr/bin/env python3 import os import sys diff --git a/tests/queries/0_stateless/00970_live_view_watch_events_http_heartbeat.py b/tests/queries/0_stateless/00970_live_view_watch_events_http_heartbeat.py index 8435cdc147a..76e5cdbe88d 100755 --- a/tests/queries/0_stateless/00970_live_view_watch_events_http_heartbeat.py +++ b/tests/queries/0_stateless/00970_live_view_watch_events_http_heartbeat.py @@ -1,4 +1,4 @@ -#!/usr/bin/env python +#!/usr/bin/env python3 import os import sys diff --git a/tests/queries/0_stateless/00971_live_view_watch_http_heartbeat.py b/tests/queries/0_stateless/00971_live_view_watch_http_heartbeat.py index 2317d705efe..5da36ab2d5f 100755 --- a/tests/queries/0_stateless/00971_live_view_watch_http_heartbeat.py +++ b/tests/queries/0_stateless/00971_live_view_watch_http_heartbeat.py @@ -1,4 +1,4 @@ -#!/usr/bin/env python +#!/usr/bin/env python3 import os import sys diff --git a/tests/queries/0_stateless/00979_live_view_watch_continuous_aggregates.py b/tests/queries/0_stateless/00979_live_view_watch_continuous_aggregates.py index 3a67226da80..7924aa15d0c 100755 --- a/tests/queries/0_stateless/00979_live_view_watch_continuous_aggregates.py +++ b/tests/queries/0_stateless/00979_live_view_watch_continuous_aggregates.py @@ -1,4 +1,4 @@ -#!/usr/bin/env python +#!/usr/bin/env python3 import os import sys import signal diff --git a/tests/queries/0_stateless/00979_live_view_watch_live.py b/tests/queries/0_stateless/00979_live_view_watch_live.py index 04ca070c969..2fc70352dbf 100755 --- a/tests/queries/0_stateless/00979_live_view_watch_live.py +++ b/tests/queries/0_stateless/00979_live_view_watch_live.py @@ -1,4 +1,4 @@ -#!/usr/bin/env python +#!/usr/bin/env python3 import os import sys import signal diff --git a/tests/queries/0_stateless/00979_live_view_watch_live_moving_avg.py b/tests/queries/0_stateless/00979_live_view_watch_live_moving_avg.py index ccc824c4d20..54f9ef694d6 100755 --- a/tests/queries/0_stateless/00979_live_view_watch_live_moving_avg.py +++ b/tests/queries/0_stateless/00979_live_view_watch_live_moving_avg.py @@ -1,4 +1,4 @@ -#!/usr/bin/env python +#!/usr/bin/env python3 import os import sys import signal diff --git a/tests/queries/0_stateless/00979_live_view_watch_live_with_subquery.py b/tests/queries/0_stateless/00979_live_view_watch_live_with_subquery.py index 809b8b0342a..2918bed82e8 100755 --- a/tests/queries/0_stateless/00979_live_view_watch_live_with_subquery.py +++ b/tests/queries/0_stateless/00979_live_view_watch_live_with_subquery.py @@ -1,4 +1,4 @@ -#!/usr/bin/env python +#!/usr/bin/env python3 import os import sys import signal diff --git a/tests/queries/0_stateless/00990_hasToken.python b/tests/queries/0_stateless/00990_hasToken.python index cd2a284655f..7d3775adc9d 100755 --- a/tests/queries/0_stateless/00990_hasToken.python +++ b/tests/queries/0_stateless/00990_hasToken.python @@ -1,4 +1,4 @@ -#!/usr/bin/env python +#!/usr/bin/env python3 # encoding: utf-8 import re @@ -126,7 +126,7 @@ def create_cases(case_sensitive_func, case_insensitive_func, table_row_template, def main(): def query(x): - print x + print(x) CONST_QUERY = Template("""SELECT ${func}('${haystack}', '${needle}'), ' expecting ', ${match};""") TABLE_QUERY = Template("""WITH '${needle}' as n diff --git a/tests/queries/0_stateless/00990_hasToken.sh b/tests/queries/0_stateless/00990_hasToken.sh index 0b88ac0007f..4b42a570e99 100755 --- a/tests/queries/0_stateless/00990_hasToken.sh +++ b/tests/queries/0_stateless/00990_hasToken.sh @@ -5,4 +5,4 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # We should have correct env vars from shell_config.sh to run this test -python "$CURDIR"/00990_hasToken.python | ${CLICKHOUSE_CLIENT} --max_query_size 1048576 -nm +python3 "$CURDIR"/00990_hasToken.python | ${CLICKHOUSE_CLIENT} --max_query_size 1048576 -nm diff --git a/tests/queries/0_stateless/00991_live_view_watch_event_live.python b/tests/queries/0_stateless/00991_live_view_watch_event_live.python index 782671cdfaf..901d388ec01 100644 --- a/tests/queries/0_stateless/00991_live_view_watch_event_live.python +++ b/tests/queries/0_stateless/00991_live_view_watch_event_live.python @@ -1,8 +1,8 @@ -#!/usr/bin/env python +#!/usr/bin/env python3 import subprocess import threading -import Queue as queue +import queue as queue import os import sys import signal diff --git a/tests/queries/0_stateless/00991_live_view_watch_event_live.sh.disabled b/tests/queries/0_stateless/00991_live_view_watch_event_live.sh.disabled index 10e4e98b2e3..0548d24a80b 100755 --- a/tests/queries/0_stateless/00991_live_view_watch_event_live.sh.disabled +++ b/tests/queries/0_stateless/00991_live_view_watch_event_live.sh.disabled @@ -3,4 +3,4 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) . $CURDIR/../shell_config.sh -python $CURDIR/00991_live_view_watch_event_live.python +python3 $CURDIR/00991_live_view_watch_event_live.python diff --git a/tests/queries/0_stateless/00991_live_view_watch_http.python b/tests/queries/0_stateless/00991_live_view_watch_http.python index 938547ca0cb..d5a1e6e8ed9 100755 --- a/tests/queries/0_stateless/00991_live_view_watch_http.python +++ b/tests/queries/0_stateless/00991_live_view_watch_http.python @@ -1,8 +1,8 @@ -#!/usr/bin/env python +#!/usr/bin/env python3 import subprocess import threading -import Queue as queue +import queue as queue import os import sys diff --git a/tests/queries/0_stateless/00991_live_view_watch_http.sh.disabled b/tests/queries/0_stateless/00991_live_view_watch_http.sh.disabled index 88cce77f595..d441ab94473 100755 --- a/tests/queries/0_stateless/00991_live_view_watch_http.sh.disabled +++ b/tests/queries/0_stateless/00991_live_view_watch_http.sh.disabled @@ -3,4 +3,4 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) . $CURDIR/../shell_config.sh -python $CURDIR/00991_live_view_watch_http.python +python3 $CURDIR/00991_live_view_watch_http.python diff --git a/tests/queries/0_stateless/00991_temporary_live_view_watch_events_heartbeat.python b/tests/queries/0_stateless/00991_temporary_live_view_watch_events_heartbeat.python index 3fd3b38be16..8ddb1a1ea81 100644 --- a/tests/queries/0_stateless/00991_temporary_live_view_watch_events_heartbeat.python +++ b/tests/queries/0_stateless/00991_temporary_live_view_watch_events_heartbeat.python @@ -1,8 +1,8 @@ -#!/usr/bin/env python +#!/usr/bin/env python3 import subprocess import threading -import Queue as queue +import queue as queue import os import sys import signal diff --git a/tests/queries/0_stateless/00991_temporary_live_view_watch_events_heartbeat.sh.disabled b/tests/queries/0_stateless/00991_temporary_live_view_watch_events_heartbeat.sh.disabled index f7aa13d52b3..ebe492fafce 100755 --- a/tests/queries/0_stateless/00991_temporary_live_view_watch_events_heartbeat.sh.disabled +++ b/tests/queries/0_stateless/00991_temporary_live_view_watch_events_heartbeat.sh.disabled @@ -3,4 +3,4 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) . $CURDIR/../shell_config.sh -python $CURDIR/00991_temporary_live_view_watch_events_heartbeat.python +python3 $CURDIR/00991_temporary_live_view_watch_events_heartbeat.python diff --git a/tests/queries/0_stateless/00991_temporary_live_view_watch_live.python b/tests/queries/0_stateless/00991_temporary_live_view_watch_live.python index a0a1fae0188..a417cdf2937 100644 --- a/tests/queries/0_stateless/00991_temporary_live_view_watch_live.python +++ b/tests/queries/0_stateless/00991_temporary_live_view_watch_live.python @@ -1,8 +1,8 @@ -#!/usr/bin/env python +#!/usr/bin/env python3 import subprocess import threading -import Queue as queue +import queue as queue import os import sys import signal diff --git a/tests/queries/0_stateless/00991_temporary_live_view_watch_live.sh.disabled b/tests/queries/0_stateless/00991_temporary_live_view_watch_live.sh.disabled index 4d01d1c3a8e..91efff53abe 100755 --- a/tests/queries/0_stateless/00991_temporary_live_view_watch_live.sh.disabled +++ b/tests/queries/0_stateless/00991_temporary_live_view_watch_live.sh.disabled @@ -3,4 +3,4 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) . $CURDIR/../shell_config.sh -python $CURDIR/00991_temporary_live_view_watch_live.python +python3 $CURDIR/00991_temporary_live_view_watch_live.python diff --git a/tests/queries/0_stateless/01246_insert_into_watch_live_view.py b/tests/queries/0_stateless/01246_insert_into_watch_live_view.py index 7f65a7135d5..0f7c6965b7b 100755 --- a/tests/queries/0_stateless/01246_insert_into_watch_live_view.py +++ b/tests/queries/0_stateless/01246_insert_into_watch_live_view.py @@ -1,4 +1,4 @@ -#!/usr/bin/env python +#!/usr/bin/env python3 import os import sys import time diff --git a/tests/queries/0_stateless/helpers/httpclient.py b/tests/queries/0_stateless/helpers/httpclient.py index a42fad2cbc3..6fb6edff142 100644 --- a/tests/queries/0_stateless/helpers/httpclient.py +++ b/tests/queries/0_stateless/helpers/httpclient.py @@ -8,7 +8,7 @@ sys.path.insert(0, os.path.join(CURDIR)) import httpexpect def client(request, name='', log=None): - client = httpexpect.spawn({'host':'localhost','port':8123}, request) + client = httpexpect.spawn({'host':'localhost','port':8123,'timeout':1}, request) client.logger(log, prefix=name) client.timeout(20) return client diff --git a/tests/queries/0_stateless/helpers/httpechoserver.py b/tests/queries/0_stateless/helpers/httpechoserver.py index 94f94d07f67..a1176c5e72d 100644 --- a/tests/queries/0_stateless/helpers/httpechoserver.py +++ b/tests/queries/0_stateless/helpers/httpechoserver.py @@ -1,12 +1,12 @@ -#!/usr/bin/env python -from __future__ import print_function +#!/usr/bin/env python3 + import sys import os import time import subprocess import threading from io import StringIO, SEEK_END -from BaseHTTPServer import BaseHTTPRequestHandler, HTTPServer +from http.server import BaseHTTPRequestHandler, HTTPServer CLICKHOUSE_HOST = os.environ.get('CLICKHOUSE_HOST', '127.0.0.1') CLICKHOUSE_PORT_HTTP = os.environ.get('CLICKHOUSE_PORT_HTTP', '8123') @@ -72,7 +72,7 @@ def start_server(requests_amount, test_data="Hello,2,-2,7.7\nWorld,2,-5,8.8"): httpd = HTTPServer(HTTP_SERVER_ADDRESS, EchoCSVHTTPServer) def real_func(): - for i in xrange(requests_amount): + for i in range(requests_amount): httpd.handle_request() t = threading.Thread(target=real_func) diff --git a/tests/queries/0_stateless/helpers/httpexpect.py b/tests/queries/0_stateless/helpers/httpexpect.py index e440dafce4e..788e57499a8 100644 --- a/tests/queries/0_stateless/helpers/httpexpect.py +++ b/tests/queries/0_stateless/helpers/httpexpect.py @@ -13,7 +13,7 @@ # limitations under the License. import os import sys -import httplib +import http.client CURDIR = os.path.dirname(os.path.realpath(__file__)) sys.path.insert(0, CURDIR) @@ -21,7 +21,7 @@ sys.path.insert(0, CURDIR) import uexpect from threading import Thread, Event -from Queue import Queue, Empty +from queue import Queue, Empty class IO(uexpect.IO): def __init__(self, connection, response, queue, reader): @@ -45,15 +45,15 @@ def reader(response, queue, kill_event): try: if kill_event.is_set(): break - data = response.read(1) + data = response.read(1).decode() queue.put(data) - except Exception, e: + except Exception as e: if kill_event.is_set(): break raise def spawn(connection, request): - connection = httplib.HTTPConnection(**connection) + connection = http.client.HTTPConnection(**connection) connection.request(**request) response = connection.getresponse() @@ -66,8 +66,8 @@ def spawn(connection, request): return IO(connection, response, queue, reader={'thread':thread, 'kill_event':reader_kill_event}) if __name__ == '__main__': - with http({'host':'localhost','port':8123},{'method':'GET', 'url':'?query=SELECT%201'}) as client: + with spawn({'host':'localhost','port':8123},{'method':'GET', 'url':'?query=SELECT%201'}) as client: client.logger(sys.stdout) client.timeout(2) - print client.response.status, client.response.reason + print(client.response.status, client.response.reason) client.expect('1\n') diff --git a/tests/queries/0_stateless/helpers/uexpect.py b/tests/queries/0_stateless/helpers/uexpect.py index b71dd2f81a0..7a633facc95 100644 --- a/tests/queries/0_stateless/helpers/uexpect.py +++ b/tests/queries/0_stateless/helpers/uexpect.py @@ -19,7 +19,7 @@ import re from threading import Thread, Event from subprocess import Popen -from Queue import Queue, Empty +from queue import Queue, Empty class TimeoutError(Exception): def __init__(self, timeout): @@ -117,7 +117,7 @@ class IO(object): return self.write(data + eol) def write(self, data): - return os.write(self.master, data) + return os.write(self.master, data.encode()) def expect(self, pattern, timeout=None, escape=False): self.match = None @@ -198,7 +198,7 @@ def spawn(command): def reader(process, out, queue, kill_event): while True: try: - data = os.read(out, 65536) + data = os.read(out, 65536).decode(errors='replace') queue.put(data) except: if kill_event.is_set(): diff --git a/tests/queries/conftest.py b/tests/queries/conftest.py index e0f47f63451..85b7250af0c 100644 --- a/tests/queries/conftest.py +++ b/tests/queries/conftest.py @@ -4,7 +4,7 @@ import os import sys import tempfile -from server import ServerThread +from .server import ServerThread def pytest_addoption(parser): @@ -44,9 +44,9 @@ def standalone_server(bin_prefix, tmp_path): if wait_result is not None: with open(os.path.join(server.log_dir, 'server', 'stdout.txt'), 'r') as f: - print >> sys.stderr, f.read() + print(f.read(), file=sys.stderr) with open(os.path.join(server.log_dir, 'server', 'stderr.txt'), 'r') as f: - print >> sys.stderr, f.read() + print(f.read(), file=sys.stderr) pytest.fail('Server died unexpectedly with code {code}'.format(code=server._proc.returncode), pytrace=False) yield server diff --git a/tests/queries/query_test.py b/tests/queries/query_test.py index e31a8ded265..bac38d0334e 100644 --- a/tests/queries/query_test.py +++ b/tests/queries/query_test.py @@ -14,11 +14,11 @@ def run_client(bin_prefix, port, query, reference, replace_map={}): result, error = client.communicate(query) assert client.returncode is not None, "Client should exit after processing all queries" - for old, new in replace_map.iteritems(): + for old, new in replace_map.items(): result = result.replace(old, new) if client.returncode != 0: - print >> sys.stderr, error + print(error, file=sys.stderr) pytest.fail('Client died unexpectedly with code {code}'.format(code=client.returncode), pytrace=False) elif result != reference: pytest.fail("Query output doesn't match reference:{eol}{diff}".format( diff --git a/tests/queries/server.py b/tests/queries/server.py index 31f92281f97..cb0755db6ae 100644 --- a/tests/queries/server.py +++ b/tests/queries/server.py @@ -56,7 +56,7 @@ class ServerThread(threading.Thread): while retries: self._choose_ports_and_args() - print 'Start clickhouse-server with args:', self._args + print('Start clickhouse-server with args:', self._args) self._proc = subprocess.Popen([self._bin] + self._args, shell=False, stdout=subprocess.PIPE, stderr=subprocess.PIPE) while self._proc.poll() is None: @@ -64,11 +64,11 @@ class ServerThread(threading.Thread): time.sleep(ServerThread.DEFAULT_SERVER_DELAY) s = socket.create_connection(('localhost', self.tcp_port), ServerThread.DEFAULT_CONNECTION_TIMEOUT) s.sendall('G') # trigger expected "bad" HELLO response - print 'Successful server response:', s.recv(1024) # FIXME: read whole buffered response + print('Successful server response:', s.recv(1024)) # FIXME: read whole buffered response s.shutdown(socket.SHUT_RDWR) s.close() except Exception as e: - print >> sys.stderr, 'Failed to connect to server:', e + print('Failed to connect to server:', e, file=sys.stderr) continue else: break @@ -76,8 +76,8 @@ class ServerThread(threading.Thread): # If process has died then try to fetch output before releasing lock if self._proc.returncode is not None: stdout, stderr = self._proc.communicate() - print >> sys.stderr, stdout - print >> sys.stderr, stderr + print(stdout, file=sys.stderr) + print(stderr, file=sys.stderr) if self._proc.returncode == 70: # Address already in use retries -= 1 @@ -101,7 +101,7 @@ class ServerThread(threading.Thread): if self._proc.returncode is None: self._proc.terminate() self.join() - print 'Stop clickhouse-server' + print('Stop clickhouse-server') ServerThread.DEFAULT_SERVER_CONFIG = \ diff --git a/tests/queries/shell_config.sh b/tests/queries/shell_config.sh index 8c66d79b5b1..1fe199be48f 100644 --- a/tests/queries/shell_config.sh +++ b/tests/queries/shell_config.sh @@ -73,5 +73,5 @@ function clickhouse_client_removed_host_parameter() { # removing only `--host=value` and `--host value` (removing '-hvalue' feels to dangerous) with python regex. # bash regex magic is arcane, but version dependant and weak; sed or awk are not really portable. - $(echo "$CLICKHOUSE_CLIENT" | python -c "import sys, re; print re.sub('--host(\s+|=)[^\s]+', '', sys.stdin.read())") "$@" + $(echo "$CLICKHOUSE_CLIENT" | python3 -c "import sys, re; print(re.sub('--host(\s+|=)[^\s]+', '', sys.stdin.read()))") "$@" } diff --git a/tests/testflows/helpers/cluster.py b/tests/testflows/helpers/cluster.py index 8288e700e3b..8fda8ac43d8 100644 --- a/tests/testflows/helpers/cluster.py +++ b/tests/testflows/helpers/cluster.py @@ -210,7 +210,7 @@ class Cluster(object): self.down() finally: with self.lock: - for shell in self._bash.values(): + for shell in list(self._bash.values()): shell.__exit__(type, value, traceback) def node(self, name): diff --git a/tests/testflows/ldap/tests/common.py b/tests/testflows/ldap/tests/common.py index eb21923e930..c065576c9d4 100644 --- a/tests/testflows/ldap/tests/common.py +++ b/tests/testflows/ldap/tests/common.py @@ -129,9 +129,9 @@ def create_ldap_servers_config_content(servers, config_d_dir="/etc/clickhouse-se xml_servers = root.find("ldap_servers") xml_servers.append(xmltree.Comment(text=f"LDAP servers {uid}")) - for _name, server in servers.items(): + for _name, server in list(servers.items()): xml_server = xmltree.Element(_name) - for key, value in server.items(): + for key, value in list(server.items()): xml_append(xml_server, key, value) xml_servers.append(xml_server) @@ -288,7 +288,7 @@ def add_user_to_ldap(cn, userpassword, givenname=None, homedirectory=None, sn=No } lines = [] - for key, value in user.items(): + for key, value in list(user.items()): if key.startswith("_"): continue elif key == "objectclass": diff --git a/tests/testflows/rbac/tests/privileges/insert.py b/tests/testflows/rbac/tests/privileges/insert.py index 5cb78c6a8a5..ad0ba90c8e3 100755 --- a/tests/testflows/rbac/tests/privileges/insert.py +++ b/tests/testflows/rbac/tests/privileges/insert.py @@ -65,9 +65,9 @@ def role(node, role): def input_output_equality_check(node, input_columns, input_data): data_list = [x.strip("'") for x in input_data.split(",")] - input_dict = dict(zip(input_columns.split(","), data_list)) + input_dict = dict(list(zip(input_columns.split(","), data_list))) output_dict = json.loads(node.query(f"select {input_columns} from merge_tree format JSONEachRow").output) - output_dict = {k:str(v) for (k,v) in output_dict.items()} + output_dict = {k:str(v) for (k,v) in list(output_dict.items())} return input_dict == output_dict @TestScenario @@ -509,7 +509,7 @@ def revoke_privilege_from_role_via_role_with_grant_option(self, table_type, node RQ_SRS_006_RBAC_Privileges_Insert("1.0"), ) @Examples("table_type", [ - (table_type, Requirements(requirement)) for table_type, requirement in table_requirements.items() + (table_type, Requirements(requirement)) for table_type, requirement in list(table_requirements.items()) ]) @Name("insert") def feature(self, table_type, node="clickhouse1"): diff --git a/tests/testflows/rbac/tests/privileges/select.py b/tests/testflows/rbac/tests/privileges/select.py index e74a63614d5..a2d3092f1c3 100644 --- a/tests/testflows/rbac/tests/privileges/select.py +++ b/tests/testflows/rbac/tests/privileges/select.py @@ -477,7 +477,7 @@ def revoke_privilege_from_role_via_role_with_grant_option(self, table_type, node RQ_SRS_006_RBAC_Privileges_Select("1.0"), ) @Examples("table_type", [ - (table_type, Requirements(requirement)) for table_type, requirement in table_requirements.items() + (table_type, Requirements(requirement)) for table_type, requirement in list(table_requirements.items()) ]) @Name("select") def feature(self, table_type, node="clickhouse1"): diff --git a/tests/testflows/runner b/tests/testflows/runner index 6522c8f47f7..0acc3a25945 100755 --- a/tests/testflows/runner +++ b/tests/testflows/runner @@ -1,4 +1,4 @@ -#!/usr/bin/env python +#!/usr/bin/env python3 #-*- coding: utf-8 -*- import subprocess import os @@ -117,6 +117,6 @@ if __name__ == "__main__": command=args.command ) - print("Running testflows container as: '" + cmd + "'.") + print(("Running testflows container as: '" + cmd + "'.")) # testflows return non zero error code on failed tests subprocess.call(cmd, shell=True) diff --git a/utils/build/build_debian.sh b/utils/build/build_debian.sh index 4ae54b0d29f..c6cfe6ce4b8 100755 --- a/utils/build/build_debian.sh +++ b/utils/build/build_debian.sh @@ -11,7 +11,7 @@ sudo apt install -y git bash cmake ninja-build gcc-8 g++-8 libicu-dev libreadlin #sudo apt install -y libboost-program-options-dev libboost-system-dev libboost-filesystem-dev libboost-thread-dev libboost-regex-dev libboost-iostreams-dev zlib1g-dev liblz4-dev libdouble-conversion-dev libzstd-dev libre2-dev librdkafka-dev libcapnp-dev libpoco-dev libgoogle-perftools-dev libunwind-dev googletest libcctz-dev # install testing only stuff if you want: -sudo apt install -y expect python python-lxml python-termcolor python-requests curl perl sudo openssl netcat-openbsd telnet +sudo apt install -y expect python3 python3-lxml python3-termcolor python3-requests curl perl sudo openssl netcat-openbsd telnet BASE_DIR=$(dirname $0) && [ -f "$BASE_DIR/../../CMakeLists.txt" ] && ROOT_DIR=$BASE_DIR/../.. && cd $ROOT_DIR diff --git a/utils/github-hook/hook.py b/utils/github-hook/hook.py index 22eb85867c3..1ea65f3c3ab 100644 --- a/utils/github-hook/hook.py +++ b/utils/github-hook/hook.py @@ -12,7 +12,7 @@ API_URL = 'https://api.github.com/repos/ClickHouse/ClickHouse/' def _reverse_dict_with_list(source): result = {} - for key, value in source.items(): + for key, value in list(source.items()): for elem in value: result[elem] = key return result @@ -48,14 +48,14 @@ def set_labels_for_pr(pull_request_number, labels, headers): response.raise_for_status() break except Exception as ex: - print("Exception", ex) + print(("Exception", ex)) time.sleep(0.2) def get_required_labels_from_desc(description, current_labels): result = set([]) # find first matching category - for marker, labels in MARKER_TO_LABEL.items(): + for marker, labels in list(MARKER_TO_LABEL.items()): if marker in description: if not any(label in current_labels for label in labels): result.add(labels[0]) @@ -282,9 +282,9 @@ class ClickHouseInserter(object): response.raise_for_status() break except Exception as ex: - print("Cannot insert with exception %s", str(ex)) + print(("Cannot insert with exception %s", str(ex))) if response: - print("Response text %s", response.text) + print(("Response text %s", response.text)) time.sleep(0.1) else: raise Exception("Cannot insert data into clickhouse") diff --git a/utils/github/backport.py b/utils/github/backport.py index 3a33c3a563f..f303be23ac4 100644 --- a/utils/github/backport.py +++ b/utils/github/backport.py @@ -78,14 +78,14 @@ class Backport: backport_map[pr['number']].remove(matched_backported.group(1)) logging.info('\tskipping %s because it\'s already backported manually', matched_backported.group(1)) - for pr, branches in backport_map.items(): + for pr, branches in list(backport_map.items()): logging.info('PR #%s needs to be backported to:', pr) for branch in branches: logging.info('\t%s, and the status is: %s', branch, run_cherrypick(self._token, pr, branch)) # print API costs logging.info('\nGitHub API total costs per query:') - for name, value in self._gh.api_costs.items(): + for name, value in list(self._gh.api_costs.items()): logging.info('%s : %s', name, value) diff --git a/utils/github/parser.py b/utils/github/parser.py index 2f00cac9bb4..570410ba23d 100644 --- a/utils/github/parser.py +++ b/utils/github/parser.py @@ -57,4 +57,4 @@ class Description: if not category: print('Cannot find category in pr description') else: - print('Unknown category: ' + category) + print(('Unknown category: ' + category)) diff --git a/utils/github/query.py b/utils/github/query.py index 9182b001dfe..ac3ce5bffa9 100644 --- a/utils/github/query.py +++ b/utils/github/query.py @@ -517,7 +517,7 @@ class Query: if not is_mutation: import inspect caller = inspect.getouterframes(inspect.currentframe(), 2)[1][3] - if caller not in self.api_costs.keys(): + if caller not in list(self.api_costs.keys()): self.api_costs[caller] = 0 self.api_costs[caller] += result['data']['rateLimit']['cost'] diff --git a/utils/junit_to_html/junit_to_html b/utils/junit_to_html/junit_to_html index cf50e7df00a..132763c7d4c 100755 --- a/utils/junit_to_html/junit_to_html +++ b/utils/junit_to_html/junit_to_html @@ -1,4 +1,4 @@ -#!/usr/bin/env python +#!/usr/bin/env python3 # -*- coding: utf-8 -*- import os import lxml.etree as etree @@ -82,5 +82,5 @@ if __name__ == "__main__": result_path = args.result_dir else: result_path = os.path.dirname(junit_path) - print "junit_path: {}, result_path: {}, export cases:{}, export suites: {}".format(junit_path, result_path, args.export_cases, args.export_suites) + print("junit_path: {}, result_path: {}, export cases:{}, export suites: {}".format(junit_path, result_path, args.export_cases, args.export_suites)) _convert_junit_to_html(junit_path, result_path, args.export_cases, args.export_suites) diff --git a/utils/kafka/manage.py b/utils/kafka/manage.py index 01847c7675b..7458bdceb74 100755 --- a/utils/kafka/manage.py +++ b/utils/kafka/manage.py @@ -30,9 +30,9 @@ def main(): client = kafka.KafkaAdminClient(**config) if args.create: - print(client.create_topics(args.create)) + print((client.create_topics(args.create))) elif args.delete: - print(client.delete_topics(args.delete)) + print((client.delete_topics(args.delete))) client.close() return 0 diff --git a/utils/make_changelog.py b/utils/make_changelog.py index fb8fa037001..5a5c82e5ab6 100755 --- a/utils/make_changelog.py +++ b/utils/make_changelog.py @@ -1,6 +1,6 @@ -#!/usr/bin/env python +#!/usr/bin/env python3 # Note: should work with python 2 and 3 -from __future__ import print_function + import requests import json @@ -201,10 +201,10 @@ def get_users_info(pull_requests, commits_info, token, max_retries, retry_timeou resp = github_api_get_json(query, token, max_retries, retry_timeout) users[user] = resp - for pull_request in pull_requests.values(): + for pull_request in list(pull_requests.values()): update_user(pull_request['user']) - for commit_info in commits_info.values(): + for commit_info in list(commits_info.values()): if 'committer' in commit_info and commit_info['committer'] is not None and 'login' in commit_info['committer']: update_user(commit_info['committer']['login']) else: @@ -216,7 +216,7 @@ def get_users_info(pull_requests, commits_info, token, max_retries, retry_timeou # List of unknown commits -> text description. def process_unknown_commits(commits, commits_info, users): - pattern = u'Commit: [{}]({})\nAuthor: {}\nMessage: {}' + pattern = 'Commit: [{}]({})\nAuthor: {}\nMessage: {}' texts = [] @@ -262,7 +262,7 @@ def process_unknown_commits(commits, commits_info, users): # Returns False if the PR should not be mentioned changelog. def parse_one_pull_request(item): description = item['description'] - lines = [line for line in map(lambda x: x.strip(), description.split('\n') if description else []) if line] + lines = [line for line in [x.strip() for x in description.split('\n') if description else []] if line] lines = [re.sub(r'\s+', ' ', l) for l in lines] cat_pos = None @@ -310,17 +310,17 @@ def parse_one_pull_request(item): def process_pull_requests(pull_requests, users, repo): groups = {} - for id, item in pull_requests.items(): + for id, item in list(pull_requests.items()): if not parse_one_pull_request(item): continue - pattern = u"{} [#{}]({}) ({})" + pattern = "{} [#{}]({}) ({})" link = 'https://github.com/{}/pull/{}'.format(repo, id) author = 'author not found' if item['user'] in users: # TODO get user from any commit if no user name on github user = users[item['user']] - author = u'[{}]({})'.format(user['name'] or user['login'], user['html_url']) + author = '[{}]({})'.format(user['name'] or user['login'], user['html_url']) cat = item['category'] if cat not in groups: @@ -336,9 +336,9 @@ def process_pull_requests(pull_requests, users, repo): return name.lower() texts = [] - for group, text in sorted(groups.items(), key = lambda kv: categories_sort_key(kv[0])): - items = [u'* {}'.format(pr) for pr in text] - texts.append(u'### {}\n{}'.format(group if group else u'[No category]', '\n'.join(items))) + for group, text in sorted(list(groups.items()), key = lambda kv: categories_sort_key(kv[0])): + items = ['* {}'.format(pr) for pr in text] + texts.append('### {}\n{}'.format(group if group else '[No category]', '\n'.join(items))) return '\n\n'.join(texts) @@ -456,7 +456,7 @@ def make_changelog(new_tag, prev_tag, pull_requests_nums, repo, repo_folder, sta logging.info('Found %d users.', len(users)) save_state(state_file, state) - changelog = u'{}\n\n{}'.format(process_pull_requests(pull_requests, users, repo), process_unknown_commits(unknown_commits, commits_info, users)) + changelog = '{}\n\n{}'.format(process_pull_requests(pull_requests, users, repo), process_unknown_commits(unknown_commits, commits_info, users)) # Substitute links to issues changelog = re.sub(r'(?> ${PACKAGE}-$VERSION_FULL-2.spec - #echo "Requires: python2-termcolor" >> ${PACKAGE}-$VERSION-2.spec + echo "Requires: python3" >> ${PACKAGE}-$VERSION_FULL-2.spec + #echo "Requires: python3-termcolor" >> ${PACKAGE}-$VERSION-2.spec cat ${PACKAGE}-$VERSION_FULL-2.spec_tmp >> ${PACKAGE}-$VERSION_FULL-2.spec rpm_pack diff --git a/utils/s3tools/s3uploader b/utils/s3tools/s3uploader index 4e8722e0851..a21263c1094 100755 --- a/utils/s3tools/s3uploader +++ b/utils/s3tools/s3uploader @@ -1,4 +1,4 @@ -#!/usr/bin/env python2 +#!/usr/bin/env python3 # -*- coding: utf-8 -*- import os import logging @@ -32,7 +32,7 @@ class S3API(object): chunkcount = int(math.ceil(filesize / chunksize)) def call_back(x, y): - print "Uploaded {}/{} bytes".format(x, y) + print("Uploaded {}/{} bytes".format(x, y)) try: for i in range(chunkcount + 1): logging.info("Uploading chunk %s of %s", i, chunkcount + 1) diff --git a/utils/simple-backport/format-changelog.py b/utils/simple-backport/format-changelog.py index 5dff4f1c5e8..e0fe4912d5d 100755 --- a/utils/simple-backport/format-changelog.py +++ b/utils/simple-backport/format-changelog.py @@ -18,7 +18,7 @@ args = parser.parse_args() def parse_one_pull_request(item): description = item['body'] # Don't skip empty lines because they delimit parts of description - lines = [line for line in map(lambda x: x.strip(), description.split('\n') if description else [])] + lines = [line for line in [x.strip() for x in description.split('\n') if description else []]] lines = [re.sub(r'\s+', ' ', l) for l in lines] category = '' @@ -102,7 +102,7 @@ for line in args.file: users[user_id] = json.loads(open(f'user{user_id}.json').read()) def print_category(category): - print("#### " + category) + print(("#### " + category)) print() for pr in category_to_pr[category]: user = users[pr["user"]["id"]] diff --git a/utils/test_history/test-history b/utils/test_history/test-history index dca62625c9f..fdd6c36e9dc 100755 --- a/utils/test_history/test-history +++ b/utils/test_history/test-history @@ -1,8 +1,8 @@ -#!/usr/bin/env python +#!/usr/bin/env python3 # -*- coding: utf-8 -*- # Note: should work with python 2 and 3 -from __future__ import print_function + from github import Github import datetime import tabulate @@ -31,7 +31,7 @@ def _filter_statuses(statuses): filt = {} for status in sorted(statuses, key=lambda x: x.updated_at): filt[status.context] = status - return filt.values() + return list(filt.values()) def get_filtered_statuses(commit): @@ -76,7 +76,7 @@ if __name__ == "__main__": all_data = [] for num, commit in enumerate(commits): commit_sha, commit_modified, check_results = process_one_commit(commit) - mapped_keys = [key for key in check_results.keys() if args.substr in key] + mapped_keys = [key for key in list(check_results.keys()) if args.substr in key] if len(mapped_keys) > len(longest_header): longest_header = mapped_keys all_data.append((commit_modified, commit_sha, check_results)) diff --git a/utils/upload_test_results/upload_test_results b/utils/upload_test_results/upload_test_results index 058a73d8081..5916d0d85e8 100755 --- a/utils/upload_test_results/upload_test_results +++ b/utils/upload_test_results/upload_test_results @@ -1,4 +1,4 @@ -#!/usr/bin/env python +#!/usr/bin/env python3 import requests import argparse @@ -96,7 +96,7 @@ def upload_request(sha, pr, file, q_type, user, password, ca_cert, host, db): 'X-ClickHouse-Key': password, } - print query; + print(query); res = requests.post( url, @@ -121,7 +121,7 @@ if __name__ == "__main__": args = parser.parse_args() - print(upload_request(args.sha, args.pr, args.file, args.type, args.user, args.password, args.ca_cert, args.host, args.db)) + print((upload_request(args.sha, args.pr, args.file, args.type, args.user, args.password, args.ca_cert, args.host, args.db))) From f6f4285348451a79203d9cb6b07514e5347cd3c7 Mon Sep 17 00:00:00 2001 From: Pavel Kruglov Date: Fri, 2 Oct 2020 20:07:54 +0300 Subject: [PATCH 287/321] Improve quantileTDigest performance --- src/AggregateFunctions/QuantileTDigest.h | 157 ++++++++++++++++------- 1 file changed, 114 insertions(+), 43 deletions(-) diff --git a/src/AggregateFunctions/QuantileTDigest.h b/src/AggregateFunctions/QuantileTDigest.h index b90979c02b9..fc957adb739 100644 --- a/src/AggregateFunctions/QuantileTDigest.h +++ b/src/AggregateFunctions/QuantileTDigest.h @@ -36,7 +36,7 @@ namespace ErrorCodes * uses asin, which slows down the algorithm a bit. */ template -class QuantileTDigest +class TDigest { using Value = Float32; using Count = Float32; @@ -86,20 +86,12 @@ class QuantileTDigest /// The memory will be allocated to several elements at once, so that the state occupies 64 bytes. static constexpr size_t bytes_in_arena = 128 - sizeof(PODArray) - sizeof(Count) - sizeof(UInt32); - using Summary = PODArrayWithStackMemory; + using Centroids = PODArrayWithStackMemory; - Summary summary; + Centroids centroids; Count count = 0; UInt32 unmerged = 0; - /** Linear interpolation at the point x on the line (x1, y1)..(x2, y2) - */ - static Value interpolate(Value x, Value x1, Value y1, Value x2, Value y2) - { - double k = (x - x1) / (x2 - x1); - return y1 + k * (y2 - y1); - } - struct RadixSortTraits { using Element = Centroid; @@ -122,13 +114,14 @@ class QuantileTDigest */ void addCentroid(const Centroid & c) { - summary.push_back(c); + centroids.push_back(c); count += c.count; ++unmerged; if (unmerged >= params.max_unmerged) compress(); } +public: /** Performs compression of accumulated centroids * When merging, the invariant is retained to the maximum size of each * centroid that does not exceed `4 q (1 - q) \ delta N`. @@ -137,16 +130,16 @@ class QuantileTDigest { if (unmerged > 0) { - RadixSort::executeLSD(summary.data(), summary.size()); + RadixSort::executeLSD(centroids.data(), centroids.size()); - if (summary.size() > 3) + if (centroids.size() > 3) { /// A pair of consecutive bars of the histogram. - auto l = summary.begin(); + auto l = centroids.begin(); auto r = std::next(l); Count sum = 0; - while (r != summary.end()) + while (r != centroids.end()) { // we use quantile which gives us the smallest error @@ -188,14 +181,13 @@ class QuantileTDigest } /// At the end of the loop, all values to the right of l were "eaten". - summary.resize(l - summary.begin() + 1); + centroids.resize(l - centroids.begin() + 1); } unmerged = 0; } } -public: /** Adds to the digest a change in `x` with a weight of `cnt` (default 1) */ void add(T x, UInt64 cnt = 1) @@ -203,17 +195,17 @@ public: addCentroid(Centroid(Value(x), Count(cnt))); } - void merge(const QuantileTDigest & other) + void merge(const TDigest & other) { - for (const auto & c : other.summary) + for (const auto & c : other.centroids) addCentroid(c); } void serialize(WriteBuffer & buf) { compress(); - writeVarUInt(summary.size(), buf); - buf.write(reinterpret_cast(summary.data()), summary.size() * sizeof(summary[0])); + writeVarUInt(centroids.size(), buf); + buf.write(reinterpret_cast(centroids.data()), centroids.size() * sizeof(centroids[0])); } void deserialize(ReadBuffer & buf) @@ -222,36 +214,112 @@ public: readVarUInt(size, buf); if (size > params.max_unmerged) - throw Exception("Too large t-digest summary size", ErrorCodes::TOO_LARGE_ARRAY_SIZE); + throw Exception("Too large t-digest centroids size", ErrorCodes::TOO_LARGE_ARRAY_SIZE); - summary.resize(size); - buf.read(reinterpret_cast(summary.data()), size * sizeof(summary[0])); + centroids.resize(size); + buf.read(reinterpret_cast(centroids.data()), size * sizeof(centroids[0])); count = 0; - for (const auto & c : summary) + for (const auto & c : centroids) count += c.count; } + Count getCount() + { + return count; + } + + const Centroids & getCentroids() const + { + return centroids; + } + + void reset() + { + centroids.resize(0); + count = 0; + unmerged = 0; + } +}; + +template +class QuantileTDigest { + using Value = Float32; + using Count = Float32; + + /** We store two t-digests. When an amount of elements in sub_tdigest become more than merge_threshold + * we merge sub_tdigest in main_tdigest and reset sub_tdigest. This method is needed to decrease an amount of + * centroids in t-digest (experiments show that after merge_threshold the size of t-digest significantly grows, + * but merging two big t-digest decreases it). + */ + TDigest main_tdigest; + TDigest sub_tdigest; + size_t merge_threshold = 1e7; + + /** Linear interpolation at the point x on the line (x1, y1)..(x2, y2) + */ + static Value interpolate(Value x, Value x1, Value y1, Value x2, Value y2) + { + double k = (x - x1) / (x2 - x1); + return y1 + k * (y2 - y1); + } + + void mergeTDigests() + { + main_tdigest.merge(sub_tdigest); + sub_tdigest.reset(); + } + +public: + void add(T x, UInt64 cnt = 1) + { + if (sub_tdigest.getCount() >= merge_threshold) + mergeTDigests(); + sub_tdigest.add(x, cnt); + } + + void merge(const QuantileTDigest & other) + { + mergeTDigests(); + main_tdigest.merge(other.main_tdigest); + main_tdigest.merge(other.sub_tdigest); + } + + void serialize(WriteBuffer & buf) + { + mergeTDigests(); + main_tdigest.serialize(buf); + } + + void deserialize(ReadBuffer & buf) + { + sub_tdigest.reset(); + main_tdigest.deserialize(buf); + } + /** Calculates the quantile q [0, 1] based on the digest. * For an empty digest returns NaN. */ template ResultType getImpl(Float64 level) { - if (summary.empty()) + mergeTDigests(); + + auto & centroids = main_tdigest.getCentroids(); + if (centroids.empty()) return std::is_floating_point_v ? NAN : 0; - compress(); + main_tdigest.compress(); - if (summary.size() == 1) - return summary.front().mean; + if (centroids.size() == 1) + return centroids.front().mean; - Float64 x = level * count; + Float64 x = level * main_tdigest.getCount(); Float64 prev_x = 0; Count sum = 0; - Value prev_mean = summary.front().mean; + Value prev_mean = centroids.front().mean; - for (const auto & c : summary) + for (const auto & c : centroids) { Float64 current_x = sum + c.count * 0.5; @@ -263,7 +331,7 @@ public: prev_x = current_x; } - return summary.back().mean; + return centroids.back().mean; } /** Get multiple quantiles (`size` parts). @@ -274,29 +342,32 @@ public: template void getManyImpl(const Float64 * levels, const size_t * levels_permutation, size_t size, ResultType * result) { - if (summary.empty()) + mergeTDigests(); + + auto & centroids = main_tdigest.getCentroids(); + if (centroids.empty()) { for (size_t result_num = 0; result_num < size; ++result_num) result[result_num] = std::is_floating_point_v ? NAN : 0; return; } - compress(); + main_tdigest.compress(); - if (summary.size() == 1) + if (centroids.size() == 1) { for (size_t result_num = 0; result_num < size; ++result_num) - result[result_num] = summary.front().mean; + result[result_num] = centroids.front().mean; return; } - Float64 x = levels[levels_permutation[0]] * count; + Float64 x = levels[levels_permutation[0]] * main_tdigest.getCount(); Float64 prev_x = 0; Count sum = 0; - Value prev_mean = summary.front().mean; + Value prev_mean = centroids.front().mean; size_t result_num = 0; - for (const auto & c : summary) + for (const auto & c : centroids) { Float64 current_x = sum + c.count * 0.5; @@ -308,7 +379,7 @@ public: if (result_num >= size) return; - x = levels[levels_permutation[result_num]] * count; + x = levels[levels_permutation[result_num]] * main_tdigest.getCount(); } sum += c.count; @@ -316,7 +387,7 @@ public: prev_x = current_x; } - auto rest_of_results = summary.back().mean; + auto rest_of_results = centroids.back().mean; for (; result_num < size; ++result_num) result[levels_permutation[result_num]] = rest_of_results; } From 7acd85d2b1ed02957950a9768ae212a23766863e Mon Sep 17 00:00:00 2001 From: Pavel Kruglov Date: Fri, 2 Oct 2020 22:20:35 +0300 Subject: [PATCH 288/321] Fix style error --- src/AggregateFunctions/QuantileTDigest.h | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/AggregateFunctions/QuantileTDigest.h b/src/AggregateFunctions/QuantileTDigest.h index fc957adb739..02d43ede66d 100644 --- a/src/AggregateFunctions/QuantileTDigest.h +++ b/src/AggregateFunctions/QuantileTDigest.h @@ -243,7 +243,8 @@ public: }; template -class QuantileTDigest { +class QuantileTDigest +{ using Value = Float32; using Count = Float32; From d9ac79c09418e284c234f2e0eeff5f99b6f9fd1f Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Fri, 2 Oct 2020 22:17:18 +0300 Subject: [PATCH 289/321] Regression for memory usage for optimize_aggregation_in_order --- ...ptimize_aggregation_in_order_memory.reference | 0 ...1513_optimize_aggregation_in_order_memory.sql | 16 ++++++++++++++++ 2 files changed, 16 insertions(+) create mode 100644 tests/queries/0_stateless/01513_optimize_aggregation_in_order_memory.reference create mode 100644 tests/queries/0_stateless/01513_optimize_aggregation_in_order_memory.sql diff --git a/tests/queries/0_stateless/01513_optimize_aggregation_in_order_memory.reference b/tests/queries/0_stateless/01513_optimize_aggregation_in_order_memory.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/01513_optimize_aggregation_in_order_memory.sql b/tests/queries/0_stateless/01513_optimize_aggregation_in_order_memory.sql new file mode 100644 index 00000000000..38920262fba --- /dev/null +++ b/tests/queries/0_stateless/01513_optimize_aggregation_in_order_memory.sql @@ -0,0 +1,16 @@ +drop table if exists data_01513; +create table data_01513 (key String) engine=MergeTree() order by key; +-- 10e3 groups, 1e3 keys each +insert into data_01513 select number%10e3 from numbers(toUInt64(2e6)); +-- reduce number of parts to 1 +optimize table data_01513 final; + +-- this is enough to trigger non-reusable Chunk in Arena. +set max_memory_usage='500M'; +set max_threads=1; +set max_block_size=500; + +select key, groupArray(repeat('a', 200)), count() from data_01513 group by key format Null; -- { serverError 241; } +select key, groupArray(repeat('a', 200)), count() from data_01513 group by key format Null settings optimize_aggregation_in_order=1; +-- for WITH TOTALS previous groups should be kept. +select key, groupArray(repeat('a', 200)), count() from data_01513 group by key with totals format Null settings optimize_aggregation_in_order=1; -- { serverError 241; } From 2a2f858365168f0bf74ef4da52d67c62eda6eefd Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Fri, 2 Oct 2020 22:17:18 +0300 Subject: [PATCH 290/321] Significantly reduce memory usage in AggregatingInOrderTransform Clean the aggregates pools (Arena's objects) between flushes, this will reduce memory usage significantly (since Arena is not intended for memory reuse in the already full Chunk's) Before this patch you cannot run SELECT FROM huge_table GROUP BY primary_key SETTINGS optimize_aggregation_in_order=1 (and the whole point of optimize_aggregation_in_order got lost), while after, this should be possible. --- .../AggregatingInOrderTransform.cpp | 24 +++++++++++++++++++ .../Transforms/AggregatingTransform.h | 9 +++++++ 2 files changed, 33 insertions(+) diff --git a/src/Processors/Transforms/AggregatingInOrderTransform.cpp b/src/Processors/Transforms/AggregatingInOrderTransform.cpp index 7edeff65ec8..0db95bc3b20 100644 --- a/src/Processors/Transforms/AggregatingInOrderTransform.cpp +++ b/src/Processors/Transforms/AggregatingInOrderTransform.cpp @@ -4,6 +4,12 @@ namespace DB { +namespace ErrorCodes +{ + extern const int LOGICAL_ERROR; +} + + AggregatingInOrderTransform::AggregatingInOrderTransform( Block header, AggregatingTransformParamsPtr params_, const SortDescription & group_by_description_, size_t res_block_size_) @@ -140,6 +146,24 @@ void AggregatingInOrderTransform::consume(Chunk chunk) block_end_reached = true; need_generate = true; cur_block_size = 0; + + /// Arenas cannot be destroyed here, since later, in FinalizingSimpleTransform + /// there will be finalizeChunk(), but even after + /// finalizeChunk() we cannot destroy arena, since some memory + /// from Arena still in use, so we attach it to the Chunk to + /// remove it once it will be consumed. + if (params->final) + { + if (variants.aggregates_pools.size() != 1) + throw Exception("Too much arenas", ErrorCodes::LOGICAL_ERROR); + + Arenas arenas(1, std::make_shared()); + std::swap(variants.aggregates_pools, arenas); + variants.aggregates_pool = variants.aggregates_pools.at(0).get(); + + chunk.setChunkInfo(std::make_shared(std::move(arenas))); + } + return; } diff --git a/src/Processors/Transforms/AggregatingTransform.h b/src/Processors/Transforms/AggregatingTransform.h index 235d01ebc77..ab18992151a 100644 --- a/src/Processors/Transforms/AggregatingTransform.h +++ b/src/Processors/Transforms/AggregatingTransform.h @@ -8,6 +8,15 @@ namespace DB { +class AggregatedArenasChunkInfo : public ChunkInfo +{ +public: + Arenas arenas; + AggregatedArenasChunkInfo(Arenas arenas_) + : arenas(std::move(arenas_)) + {} +}; + class AggregatedChunkInfo : public ChunkInfo { public: From f25c1742b83a6f3894b37bb2766c8250faf6605b Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sat, 3 Oct 2020 01:28:46 +0300 Subject: [PATCH 291/321] Pass through *_for_user settings via Distributed with cluster-secure In cluster-secure case the user on shards (remote) is equal to the user on the initiator, so those settings can be safely applied. --- .../ClusterProxy/executeQuery.cpp | 22 ++++-- src/Interpreters/ClusterProxy/executeQuery.h | 13 +++- src/Storages/getStructureOfRemoteTable.cpp | 75 ++++++++++--------- src/Storages/getStructureOfRemoteTable.h | 6 -- .../test.py | 24 ++++++ 5 files changed, 86 insertions(+), 54 deletions(-) diff --git a/src/Interpreters/ClusterProxy/executeQuery.cpp b/src/Interpreters/ClusterProxy/executeQuery.cpp index 1ebd3009ff7..eb636395fd9 100644 --- a/src/Interpreters/ClusterProxy/executeQuery.cpp +++ b/src/Interpreters/ClusterProxy/executeQuery.cpp @@ -15,18 +15,24 @@ namespace DB namespace ClusterProxy { -Context removeUserRestrictionsFromSettings(const Context & context, const Settings & settings, Poco::Logger * log) +Context updateSettingsForCluster(const Cluster & cluster, const Context & context, const Settings & settings, Poco::Logger * log) { Settings new_settings = settings; new_settings.queue_max_wait_ms = Cluster::saturate(new_settings.queue_max_wait_ms, settings.max_execution_time); - /// Does not matter on remote servers, because queries are sent under different user. - new_settings.max_concurrent_queries_for_user = 0; - new_settings.max_memory_usage_for_user = 0; + /// If "secret" (in remote_servers) is not in use, + /// user on the shard is not the same as the user on the initiator, + /// hence per-user limits should not be applied. + if (cluster.getSecret().empty()) + { + /// Does not matter on remote servers, because queries are sent under different user. + new_settings.max_concurrent_queries_for_user = 0; + new_settings.max_memory_usage_for_user = 0; - /// Set as unchanged to avoid sending to remote server. - new_settings.max_concurrent_queries_for_user.changed = false; - new_settings.max_memory_usage_for_user.changed = false; + /// Set as unchanged to avoid sending to remote server. + new_settings.max_concurrent_queries_for_user.changed = false; + new_settings.max_memory_usage_for_user.changed = false; + } if (settings.force_optimize_skip_unused_shards_nesting && settings.force_optimize_skip_unused_shards) { @@ -84,7 +90,7 @@ Pipe executeQuery( const std::string query = queryToString(query_ast); - Context new_context = removeUserRestrictionsFromSettings(context, settings, log); + Context new_context = updateSettingsForCluster(*cluster, context, settings, log); ThrottlerPtr user_level_throttler; if (auto * process_list_element = context.getProcessListElement()) diff --git a/src/Interpreters/ClusterProxy/executeQuery.h b/src/Interpreters/ClusterProxy/executeQuery.h index f0d9539770d..3a3fca4d54c 100644 --- a/src/Interpreters/ClusterProxy/executeQuery.h +++ b/src/Interpreters/ClusterProxy/executeQuery.h @@ -18,9 +18,16 @@ namespace ClusterProxy class IStreamFactory; -/// removes different restrictions (like max_concurrent_queries_for_user, max_memory_usage_for_user, etc.) -/// from settings and creates new context with them -Context removeUserRestrictionsFromSettings(const Context & context, const Settings & settings, Poco::Logger * log = nullptr); +/// Update settings for Distributed query. +/// +/// - Removes different restrictions (like max_concurrent_queries_for_user, max_memory_usage_for_user, etc.) +/// (but only if cluster does not have secret, since if it has, the user is the same) +/// - Update some settings depends on force_optimize_skip_unused_shards and: +/// - force_optimize_skip_unused_shards_nesting +/// - optimize_skip_unused_shards_nesting +/// +/// @return new Context with adjusted settings +Context updateSettingsForCluster(const Cluster & cluster, const Context & context, const Settings & settings, Poco::Logger * log = nullptr); /// Execute a distributed query, creating a vector of BlockInputStreams, from which the result can be read. /// `stream_factory` object encapsulates the logic of creating streams for a different type of query diff --git a/src/Storages/getStructureOfRemoteTable.cpp b/src/Storages/getStructureOfRemoteTable.cpp index 7dfee90588a..6a5ef58a326 100644 --- a/src/Storages/getStructureOfRemoteTable.cpp +++ b/src/Storages/getStructureOfRemoteTable.cpp @@ -25,43 +25,8 @@ namespace ErrorCodes } -ColumnsDescription getStructureOfRemoteTable( - const Cluster & cluster, - const StorageID & table_id, - const Context & context, - const ASTPtr & table_func_ptr) -{ - const auto & shards_info = cluster.getShardsInfo(); - - std::string fail_messages; - - for (const auto & shard_info : shards_info) - { - try - { - const auto & res = getStructureOfRemoteTableInShard(shard_info, table_id, context, table_func_ptr); - - /// Expect at least some columns. - /// This is a hack to handle the empty block case returned by Connection when skip_unavailable_shards is set. - if (res.empty()) - continue; - - return res; - } - catch (const NetException &) - { - std::string fail_message = getCurrentExceptionMessage(false); - fail_messages += fail_message + '\n'; - continue; - } - } - - throw NetException( - "All attempts to get table structure failed. Log: \n\n" + fail_messages + "\n", - ErrorCodes::NO_REMOTE_SHARD_AVAILABLE); -} - ColumnsDescription getStructureOfRemoteTableInShard( + const Cluster & cluster, const Cluster::ShardInfo & shard_info, const StorageID & table_id, const Context & context, @@ -96,7 +61,7 @@ ColumnsDescription getStructureOfRemoteTableInShard( ColumnsDescription res; - auto new_context = ClusterProxy::removeUserRestrictionsFromSettings(context, context.getSettingsRef()); + auto new_context = ClusterProxy::updateSettingsForCluster(cluster, context, context.getSettingsRef()); /// Expect only needed columns from the result of DESC TABLE. NOTE 'comment' column is ignored for compatibility reasons. Block sample_block @@ -151,4 +116,40 @@ ColumnsDescription getStructureOfRemoteTableInShard( return res; } +ColumnsDescription getStructureOfRemoteTable( + const Cluster & cluster, + const StorageID & table_id, + const Context & context, + const ASTPtr & table_func_ptr) +{ + const auto & shards_info = cluster.getShardsInfo(); + + std::string fail_messages; + + for (const auto & shard_info : shards_info) + { + try + { + const auto & res = getStructureOfRemoteTableInShard(cluster, shard_info, table_id, context, table_func_ptr); + + /// Expect at least some columns. + /// This is a hack to handle the empty block case returned by Connection when skip_unavailable_shards is set. + if (res.empty()) + continue; + + return res; + } + catch (const NetException &) + { + std::string fail_message = getCurrentExceptionMessage(false); + fail_messages += fail_message + '\n'; + continue; + } + } + + throw NetException( + "All attempts to get table structure failed. Log: \n\n" + fail_messages + "\n", + ErrorCodes::NO_REMOTE_SHARD_AVAILABLE); +} + } diff --git a/src/Storages/getStructureOfRemoteTable.h b/src/Storages/getStructureOfRemoteTable.h index fa7c80c2800..af418144cb0 100644 --- a/src/Storages/getStructureOfRemoteTable.h +++ b/src/Storages/getStructureOfRemoteTable.h @@ -19,10 +19,4 @@ ColumnsDescription getStructureOfRemoteTable( const Context & context, const ASTPtr & table_func_ptr = nullptr); -ColumnsDescription getStructureOfRemoteTableInShard( - const Cluster::ShardInfo & shard_info, - const StorageID & table_id, - const Context & context, - const ASTPtr & table_func_ptr = nullptr); - } diff --git a/tests/integration/test_distributed_inter_server_secret/test.py b/tests/integration/test_distributed_inter_server_secret/test.py index bd9e6d111ca..ec6a71b93ab 100644 --- a/tests/integration/test_distributed_inter_server_secret/test.py +++ b/tests/integration/test_distributed_inter_server_secret/test.py @@ -149,4 +149,28 @@ def test_user_secure_cluster(user, password): assert get_query_user_info(n1, id_) == [user, user] assert get_query_user_info(n2, id_) == [user, user] +# settings in the protocol cannot be used since they will be applied to early +# and it will not even enter execution of distributed query +@users +def test_per_user_settings_insecure_cluster(user, password): + id_ = 'query-settings-dist_insecure-' + user + query_with_id(n1, id_, """ + SELECT * FROM dist_insecure + SETTINGS + prefer_localhost_replica=0, + max_memory_usage_for_user=100, + max_untracked_memory=0 + """, user=user, password=password) +@users +def test_per_user_settings_secure_cluster(user, password): + id_ = 'query-settings-dist_secure-' + user + with pytest.raises(QueryRuntimeException): + query_with_id(n1, id_, """ + SELECT * FROM dist_secure + SETTINGS + prefer_localhost_replica=0, + max_memory_usage_for_user=100, + max_untracked_memory=0 + """, user=user, password=password) + # TODO: check user for INSERT From 6608de470dab766da276b4ed6a70751b33b1e86e Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 3 Oct 2020 03:01:32 +0300 Subject: [PATCH 292/321] Step 1: handle EINTR in statvfs --- src/Common/filesystemHelpers.h | 7 ++++++- 1 file changed, 6 insertions(+), 1 deletion(-) diff --git a/src/Common/filesystemHelpers.h b/src/Common/filesystemHelpers.h index f97f91d2647..59af1adeb15 100644 --- a/src/Common/filesystemHelpers.h +++ b/src/Common/filesystemHelpers.h @@ -6,6 +6,7 @@ #include #include #include +#include #include #include @@ -34,8 +35,12 @@ String getFilesystemName([[maybe_unused]] const String & mount_point); inline struct statvfs getStatVFS(const String & path) { struct statvfs fs; - if (statvfs(path.c_str(), &fs) != 0) + while (statvfs(path.c_str(), &fs) != 0) + { + if (errno == EINTR) + continue; throwFromErrnoWithPath("Could not calculate available disk space (statvfs)", path, ErrorCodes::CANNOT_STATVFS); + } return fs; } From 2441be4b660a2e8261d725bd4604d3eb956a27cd Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 3 Oct 2020 03:02:47 +0300 Subject: [PATCH 293/321] Step 2: move code to .cpp --- src/Common/filesystemHelpers.cpp | 14 ++++++++++++++ src/Common/filesystemHelpers.h | 12 +----------- 2 files changed, 15 insertions(+), 11 deletions(-) diff --git a/src/Common/filesystemHelpers.cpp b/src/Common/filesystemHelpers.cpp index e722fbc9c0f..845c25e8ed2 100644 --- a/src/Common/filesystemHelpers.cpp +++ b/src/Common/filesystemHelpers.cpp @@ -18,6 +18,20 @@ namespace ErrorCodes extern const int NOT_IMPLEMENTED; } + +struct statvfs getStatVFS(const String & path) +{ + struct statvfs fs; + while (statvfs(path.c_str(), &fs) != 0) + { + if (errno == EINTR) + continue; + throwFromErrnoWithPath("Could not calculate available disk space (statvfs)", path, ErrorCodes::CANNOT_STATVFS); + } + return fs; +} + + bool enoughSpaceInDirectory(const std::string & path [[maybe_unused]], size_t data_size [[maybe_unused]]) { #if POCO_VERSION >= 0x01090000 diff --git a/src/Common/filesystemHelpers.h b/src/Common/filesystemHelpers.h index 59af1adeb15..bea16c146ba 100644 --- a/src/Common/filesystemHelpers.h +++ b/src/Common/filesystemHelpers.h @@ -32,16 +32,6 @@ std::filesystem::path getMountPoint(std::filesystem::path absolute_path); #endif String getFilesystemName([[maybe_unused]] const String & mount_point); -inline struct statvfs getStatVFS(const String & path) -{ - struct statvfs fs; - while (statvfs(path.c_str(), &fs) != 0) - { - if (errno == EINTR) - continue; - throwFromErrnoWithPath("Could not calculate available disk space (statvfs)", path, ErrorCodes::CANNOT_STATVFS); - } - return fs; -} +struct statvfs getStatVFS(const String & path); } From b6ca8a06765f764b08f76c8a9b48c854b030aaff Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 3 Oct 2020 03:05:54 +0300 Subject: [PATCH 294/321] Step 3: check other usages, add a comment --- src/Common/filesystemHelpers.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Common/filesystemHelpers.cpp b/src/Common/filesystemHelpers.cpp index 845c25e8ed2..0e5f5d0f61e 100644 --- a/src/Common/filesystemHelpers.cpp +++ b/src/Common/filesystemHelpers.cpp @@ -60,7 +60,7 @@ std::filesystem::path getMountPoint(std::filesystem::path absolute_path) const auto get_device_id = [](const std::filesystem::path & p) { struct stat st; - if (stat(p.c_str(), &st)) + if (stat(p.c_str(), &st)) /// NOTE: man stat does not list EINTR as possible error throwFromErrnoWithPath("Cannot stat " + p.string(), p.string(), ErrorCodes::SYSTEM_ERROR); return st.st_dev; }; From d426ba3074f9c0e8ac22890a4c4fcfddd3068fc1 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 3 Oct 2020 03:07:55 +0300 Subject: [PATCH 295/321] Step 4: cleanup --- src/Common/filesystemHelpers.cpp | 1 + src/Common/filesystemHelpers.h | 1 - 2 files changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Common/filesystemHelpers.cpp b/src/Common/filesystemHelpers.cpp index 0e5f5d0f61e..94b54f9c6ba 100644 --- a/src/Common/filesystemHelpers.cpp +++ b/src/Common/filesystemHelpers.cpp @@ -5,6 +5,7 @@ # include # include #endif +#include #include #include #include diff --git a/src/Common/filesystemHelpers.h b/src/Common/filesystemHelpers.h index bea16c146ba..e4cb3dab6e0 100644 --- a/src/Common/filesystemHelpers.h +++ b/src/Common/filesystemHelpers.h @@ -6,7 +6,6 @@ #include #include #include -#include #include #include From 3ca7fca48a07ac634fddac472362ffe0f280a96d Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Sat, 3 Oct 2020 03:42:33 +0300 Subject: [PATCH 296/321] Update DiskS3.cpp --- src/Disks/S3/DiskS3.cpp | 49 +++++++++++++++++++++-------------------- 1 file changed, 25 insertions(+), 24 deletions(-) diff --git a/src/Disks/S3/DiskS3.cpp b/src/Disks/S3/DiskS3.cpp index fc4ff60d69d..13721da629a 100644 --- a/src/Disks/S3/DiskS3.cpp +++ b/src/Disks/S3/DiskS3.cpp @@ -36,32 +36,33 @@ namespace ErrorCodes extern const int NOT_IMPLEMENTED; } -class DiskS3::AwsS3KeyKeeper : public std::list> -{ -public: - void addKey(const String & key); - -private: - /// limit for one DeleteObject request - /// see https://docs.aws.amazon.com/AmazonS3/latest/API/API_DeleteObjects.html - const static size_t chunk_limit = 1000; -}; - -void DiskS3::AwsS3KeyKeeper::addKey(const String & key) -{ - if (empty() || back().size() >= chunk_limit) - { /// add one more chunk - push_back(value_type()); - back().reserve(chunk_limit); - } - - Aws::S3::Model::ObjectIdentifier obj; - obj.SetKey(key); - back().push_back(obj); -} - namespace { + /// Helper class to collect keys into chunks of maximum size (to prepare batch requests to AWS API) + class DiskS3::AwsS3KeyKeeper : public std::list> + { + public: + void addKey(const String & key); + + private: + /// limit for one DeleteObject request + /// see https://docs.aws.amazon.com/AmazonS3/latest/API/API_DeleteObjects.html + const static size_t chunk_limit = 1000; + }; + + void DiskS3::AwsS3KeyKeeper::addKey(const String & key) + { + if (empty() || back().size() >= chunk_limit) + { /// add one more chunk + push_back(value_type()); + back().reserve(chunk_limit); + } + + Aws::S3::Model::ObjectIdentifier obj; + obj.SetKey(key); + back().push_back(obj); + } + String getRandomName() { std::uniform_int_distribution distribution('a', 'z'); From 5e37ba433594c5e44f070600c23ca493124bd11e Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Sat, 3 Oct 2020 03:47:45 +0300 Subject: [PATCH 297/321] Update DiskS3.cpp --- src/Disks/S3/DiskS3.cpp | 45 +++++++++++++++++++++-------------------- 1 file changed, 23 insertions(+), 22 deletions(-) diff --git a/src/Disks/S3/DiskS3.cpp b/src/Disks/S3/DiskS3.cpp index 13721da629a..2705040841c 100644 --- a/src/Disks/S3/DiskS3.cpp +++ b/src/Disks/S3/DiskS3.cpp @@ -36,33 +36,34 @@ namespace ErrorCodes extern const int NOT_IMPLEMENTED; } -namespace + +/// Helper class to collect keys into chunks of maximum size (to prepare batch requests to AWS API) +class DiskS3::AwsS3KeyKeeper : public std::list> { - /// Helper class to collect keys into chunks of maximum size (to prepare batch requests to AWS API) - class DiskS3::AwsS3KeyKeeper : public std::list> - { - public: - void addKey(const String & key); +public: + void addKey(const String & key); - private: - /// limit for one DeleteObject request - /// see https://docs.aws.amazon.com/AmazonS3/latest/API/API_DeleteObjects.html - const static size_t chunk_limit = 1000; - }; +private: + /// limit for one DeleteObject request + /// see https://docs.aws.amazon.com/AmazonS3/latest/API/API_DeleteObjects.html + const static size_t chunk_limit = 1000; +}; - void DiskS3::AwsS3KeyKeeper::addKey(const String & key) - { - if (empty() || back().size() >= chunk_limit) - { /// add one more chunk - push_back(value_type()); - back().reserve(chunk_limit); - } - - Aws::S3::Model::ObjectIdentifier obj; - obj.SetKey(key); - back().push_back(obj); +void DiskS3::AwsS3KeyKeeper::addKey(const String & key) +{ + if (empty() || back().size() >= chunk_limit) + { /// add one more chunk + push_back(value_type()); + back().reserve(chunk_limit); } + Aws::S3::Model::ObjectIdentifier obj; + obj.SetKey(key); + back().push_back(obj); +} + +namespace +{ String getRandomName() { std::uniform_int_distribution distribution('a', 'z'); From 84583faa43836cc1f6c2e457163f530a429b1847 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sat, 3 Oct 2020 11:01:59 +0300 Subject: [PATCH 298/321] Fix test_distributed_inter_server_secret under ASAN And also cover both cases: - settings from DDL - settings from TCP protocol --- .../test.py | 61 ++++++++++++++----- 1 file changed, 46 insertions(+), 15 deletions(-) diff --git a/tests/integration/test_distributed_inter_server_secret/test.py b/tests/integration/test_distributed_inter_server_secret/test.py index ec6a71b93ab..b1daf2271d0 100644 --- a/tests/integration/test_distributed_inter_server_secret/test.py +++ b/tests/integration/test_distributed_inter_server_secret/test.py @@ -79,6 +79,20 @@ def get_query_user_info(node, query_pattern): type = 'QueryFinish' """.format(query_pattern)).strip().split('\t') +# @return -- settings +def get_query_setting_on_shard(node, query_pattern, setting): + node.query("SYSTEM FLUSH LOGS") + return node.query(""" + SELECT (arrayFilter(x -> ((x.1) = '{}'), arrayZip(Settings.Names, Settings.Values))[1]).2 + FROM system.query_log + WHERE + query LIKE '%{}%' AND + NOT is_initial_query AND + query NOT LIKE '%system.query_log%' AND + type = 'QueryFinish' + LIMIT 1 + """.format(setting, query_pattern)).strip() + def test_insecure(): n1.query('SELECT * FROM dist_insecure') @@ -149,28 +163,45 @@ def test_user_secure_cluster(user, password): assert get_query_user_info(n1, id_) == [user, user] assert get_query_user_info(n2, id_) == [user, user] -# settings in the protocol cannot be used since they will be applied to early -# and it will not even enter execution of distributed query @users -def test_per_user_settings_insecure_cluster(user, password): - id_ = 'query-settings-dist_insecure-' + user +def test_per_user_inline_settings_insecure_cluster(user, password): + id_ = 'query-ddl-settings-dist_insecure-' + user query_with_id(n1, id_, """ SELECT * FROM dist_insecure SETTINGS prefer_localhost_replica=0, - max_memory_usage_for_user=100, + max_memory_usage_for_user=1e9, max_untracked_memory=0 """, user=user, password=password) + assert get_query_setting_on_shard(n1, id_, 'max_memory_usage_for_user') == '' @users -def test_per_user_settings_secure_cluster(user, password): - id_ = 'query-settings-dist_secure-' + user - with pytest.raises(QueryRuntimeException): - query_with_id(n1, id_, """ - SELECT * FROM dist_secure - SETTINGS - prefer_localhost_replica=0, - max_memory_usage_for_user=100, - max_untracked_memory=0 - """, user=user, password=password) +def test_per_user_inline_settings_secure_cluster(user, password): + id_ = 'query-ddl-settings-dist_secure-' + user + query_with_id(n1, id_, """ + SELECT * FROM dist_secure + SETTINGS + prefer_localhost_replica=0, + max_memory_usage_for_user=1e9, + max_untracked_memory=0 + """, user=user, password=password) + assert int(get_query_setting_on_shard(n1, id_, 'max_memory_usage_for_user')) == int(1e9) +@users +def test_per_user_protocol_settings_insecure_cluster(user, password): + id_ = 'query-protocol-settings-dist_insecure-' + user + query_with_id(n1, id_, 'SELECT * FROM dist_insecure', user=user, password=password, settings={ + 'prefer_localhost_replica': 0, + 'max_memory_usage_for_user': int(1e9), + 'max_untracked_memory': 0, + }) + assert get_query_setting_on_shard(n1, id_, 'max_memory_usage_for_user') == '' +@users +def test_per_user_protocol_settings_secure_cluster(user, password): + id_ = 'query-protocol-settings-dist_secure-' + user + query_with_id(n1, id_, 'SELECT * FROM dist_secure', user=user, password=password, settings={ + 'prefer_localhost_replica': 0, + 'max_memory_usage_for_user': int(1e9), + 'max_untracked_memory': 0, + }) + assert int(get_query_setting_on_shard(n1, id_, 'max_memory_usage_for_user')) == int(1e9) # TODO: check user for INSERT From 4d6402525a4e3433a58b302eea3ef76673aa4128 Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Sat, 3 Oct 2020 15:57:12 +0300 Subject: [PATCH 299/321] Update filesystemHelpers.h --- src/Common/filesystemHelpers.h | 4 ---- 1 file changed, 4 deletions(-) diff --git a/src/Common/filesystemHelpers.h b/src/Common/filesystemHelpers.h index e4cb3dab6e0..f534b61808a 100644 --- a/src/Common/filesystemHelpers.h +++ b/src/Common/filesystemHelpers.h @@ -12,10 +12,6 @@ namespace DB { -namespace ErrorCodes -{ - extern const int CANNOT_STATVFS; -} using TemporaryFile = Poco::TemporaryFile; From ae61f8f7a6b90da59522301e696d06ac6540c5ad Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Sat, 3 Oct 2020 15:57:35 +0300 Subject: [PATCH 300/321] Update filesystemHelpers.cpp --- src/Common/filesystemHelpers.cpp | 3 +++ 1 file changed, 3 insertions(+) diff --git a/src/Common/filesystemHelpers.cpp b/src/Common/filesystemHelpers.cpp index 94b54f9c6ba..51a66fba3aa 100644 --- a/src/Common/filesystemHelpers.cpp +++ b/src/Common/filesystemHelpers.cpp @@ -10,13 +10,16 @@ #include #include + namespace DB { + namespace ErrorCodes { extern const int LOGICAL_ERROR; extern const int SYSTEM_ERROR; extern const int NOT_IMPLEMENTED; + extern const int CANNOT_STATVFS; } From a4c16459a7f26a23c8682228fb479a12a48c797c Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Sat, 3 Oct 2020 16:55:27 +0300 Subject: [PATCH 301/321] Update settings.md --- docs/en/operations/settings/settings.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index dd55ea454c2..decaf6b9029 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -1146,7 +1146,7 @@ See also: Write to quorum timeout in milliseconds. If the timeout has passed and no write has taken place yet, ClickHouse will generate an exception and the client must repeat the query to write the same block to the same or any other replica. -Default value: 600000 milliseconds. +Default value: 600000 milliseconds (ten minutes). See also: From 561f3613b24f705eee719fcfa873b2a0d5b000cc Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Sat, 3 Oct 2020 17:04:47 +0300 Subject: [PATCH 302/321] Update type-conversion-functions.md --- docs/en/sql-reference/functions/type-conversion-functions.md | 2 +- 1 file changed, 1 insertion(+), 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 bbf8b7adc8e..858c647ec05 100644 --- a/docs/en/sql-reference/functions/type-conversion-functions.md +++ b/docs/en/sql-reference/functions/type-conversion-functions.md @@ -746,7 +746,7 @@ formatRow(format, x, y, ...) **Returned value** -- A formatted string terminated with the new line symbol. +- A formatted string (for text formats it's usually terminated with the new line character). **Example** From eb67696353158120932999b46e9620b341470e99 Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Sat, 3 Oct 2020 17:05:25 +0300 Subject: [PATCH 303/321] Update type-conversion-functions.md --- docs/ru/sql-reference/functions/type-conversion-functions.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/sql-reference/functions/type-conversion-functions.md b/docs/ru/sql-reference/functions/type-conversion-functions.md index f4c811c0656..20f69346d2d 100644 --- a/docs/ru/sql-reference/functions/type-conversion-functions.md +++ b/docs/ru/sql-reference/functions/type-conversion-functions.md @@ -734,7 +734,7 @@ formatRow(format, x, y, ...) **Возвращаемое значение** -- Отформатированная строка, которая заканчивается символом новой строки. +- Отформатированная строка (в текстовых форматах обычно с завершающим переводом строки). **Пример** From e219162b820ddf8170f951e59d0a2202c36785e9 Mon Sep 17 00:00:00 2001 From: tavplubix Date: Sat, 3 Oct 2020 21:09:22 +0300 Subject: [PATCH 304/321] Update FunctionsJSON.h --- src/Functions/FunctionsJSON.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Functions/FunctionsJSON.h b/src/Functions/FunctionsJSON.h index 40498353195..a5c88cad1d6 100644 --- a/src/Functions/FunctionsJSON.h +++ b/src/Functions/FunctionsJSON.h @@ -279,7 +279,7 @@ public: String getName() const override { return Name::name; } bool isVariadic() const override { return true; } size_t getNumberOfArguments() const override { return 0; } - bool useDefaultImplementationForConstants() const override { return false; } + bool useDefaultImplementationForConstants() const override { return true; } DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override { From 9c09050e7c8ff8477dc52f96943e9de74624997b Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sun, 4 Oct 2020 01:38:16 +0300 Subject: [PATCH 305/321] Cleanup server instructions startup checking code --- programs/main.cpp | 26 ++++++++++++++------------ 1 file changed, 14 insertions(+), 12 deletions(-) diff --git a/programs/main.cpp b/programs/main.cpp index b91bd732f21..3afa78b0aec 100644 --- a/programs/main.cpp +++ b/programs/main.cpp @@ -238,8 +238,9 @@ void checkRequiredInstructionsImpl(volatile InstructionFail & fail) } /// This function is safe to use in static initializers. -void writeError(const char * data, size_t size) +void writeError(const char * data) { + size_t size = strlen(data); while (size != 0) { ssize_t res = ::write(STDERR_FILENO, data, size); @@ -272,8 +273,7 @@ void checkRequiredInstructions() /// Typical implementation of strlen is using SSE4.2 or AVX2. /// But this is not the case because it's compiler builtin and is executed at compile time. - const char * msg = "Can not set signal handler\n"; - writeError(msg, strlen(msg)); + writeError("Can not set signal handler\n"); _Exit(1); } @@ -281,12 +281,9 @@ void checkRequiredInstructions() if (sigsetjmp(jmpbuf, 1)) { - const char * msg1 = "Instruction check fail. The CPU does not support "; - writeError(msg1, strlen(msg1)); - const char * msg2 = instructionFailToString(fail); - writeError(msg2, strlen(msg2)); - const char * msg3 = " instruction set.\n"; - writeError(msg3, strlen(msg3)); + writeError("Instruction check fail. The CPU does not support "); + writeError(instructionFailToString(fail)); + writeError(" instruction set.\n"); _Exit(1); } @@ -294,13 +291,18 @@ void checkRequiredInstructions() if (sigaction(signal, &sa_old, nullptr)) { - const char * msg = "Can not set signal handler\n"; - writeError(msg, strlen(msg)); + writeError("Can not set signal handler\n"); _Exit(1); } } -struct Checker { Checker() { checkRequiredInstructions(); } } checker; +struct Checker +{ + Checker() + { + checkRequiredInstructions(); + } +} checker; } From caf3156fb7c63df2a0213b5f5d5e1b503d66a93d Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sun, 4 Oct 2020 01:48:24 +0300 Subject: [PATCH 306/321] Check MADV_DONTNEED (for jemalloc), maybe broken under qemu jemalloc relies on working MADV_DONTNEED (that fact that after madvise(MADV_DONTNEED) returns success, after subsequent access to those pages they will be zeroed). However qemu does not support this, yet [1], and you can get very tricky assert if you will run clickhouse-server under qemu: : ../contrib/jemalloc/src/extent.c:1195: Failed assertion: "p[i] == 0" [1]: https://patchwork.kernel.org/patch/10576637/ But after this patch you will get pretty error: $ qemu-x86_64-static programs/clickhouse MADV_DONTNEED does not zeroed page. jemalloc will be broken --- programs/main.cpp | 46 ++++++++++++++++++++++++++++++++++++++++++++++ 1 file changed, 46 insertions(+) diff --git a/programs/main.cpp b/programs/main.cpp index 3afa78b0aec..3f7395aea32 100644 --- a/programs/main.cpp +++ b/programs/main.cpp @@ -2,6 +2,10 @@ #include #include +#ifdef __linux__ +#include +#endif + #include #include #include @@ -296,10 +300,52 @@ void checkRequiredInstructions() } } +#ifdef __linux__ +/// clickhouse uses jemalloc as a production allocator +/// and jemalloc relies on working MADV_DONTNEED, +/// which doesn't work under qemu +/// +/// but do this only under for linux, since only it return zeroed pages after MADV_DONTNEED +/// (and jemalloc assumes this too, see contrib/jemalloc-cmake/include_linux_x86_64/jemalloc/internal/jemalloc_internal_defs.h.in) +void checkRequiredMadviseFlags() +{ + size_t size = 1 << 16; + void * addr = mmap(nullptr, size, PROT_READ|PROT_WRITE, MAP_PRIVATE|MAP_ANONYMOUS, -1, 0); + if (addr == MAP_FAILED) + { + writeError("Can not mmap pages for MADV_DONTNEED check\n"); + _Exit(1); + } + memset(addr, 'A', size); + + if (!madvise(addr, size, MADV_DONTNEED)) + { + /// Suboptimal, but should be simple. + for (size_t i = 0; i < size; ++i) + { + if (reinterpret_cast(addr)[i] != 0) + { + writeError("MADV_DONTNEED does not zeroed page. jemalloc will be broken\n"); + _Exit(1); + } + } + } + + if (munmap(addr, size)) + { + writeError("Can not munmap pages for MADV_DONTNEED check\n"); + _Exit(1); + } +} +#endif + struct Checker { Checker() { +#ifdef __linux__ + checkRequiredMadviseFlags(); +#endif checkRequiredInstructions(); } } checker; From 2a6874e065d0291ad3adbd5f43af2024f456285a Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sun, 4 Oct 2020 02:01:40 +0300 Subject: [PATCH 307/321] Run MADV_DONTNEED after SSE check to fix 01103_check_cpu_instructions_at_startup --- programs/main.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/programs/main.cpp b/programs/main.cpp index 3f7395aea32..de9f26afbd3 100644 --- a/programs/main.cpp +++ b/programs/main.cpp @@ -343,10 +343,10 @@ struct Checker { Checker() { + checkRequiredInstructions(); #ifdef __linux__ checkRequiredMadviseFlags(); #endif - checkRequiredInstructions(); } } checker; From d4e7c90306373fa9de4710b472c768957251fa0e Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sun, 4 Oct 2020 02:02:57 +0300 Subject: [PATCH 308/321] Update expectations for 01103_check_cpu_instructions_at_startup --- .../01103_check_cpu_instructions_at_startup.reference | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/01103_check_cpu_instructions_at_startup.reference b/tests/queries/0_stateless/01103_check_cpu_instructions_at_startup.reference index 03ed07cf1a4..bcc7aebeae8 100644 --- a/tests/queries/0_stateless/01103_check_cpu_instructions_at_startup.reference +++ b/tests/queries/0_stateless/01103_check_cpu_instructions_at_startup.reference @@ -2,4 +2,4 @@ Instruction check fail. The CPU does not support SSSE3 instruction set. Instruction check fail. The CPU does not support SSE4.1 instruction set. Instruction check fail. The CPU does not support SSE4.2 instruction set. Instruction check fail. The CPU does not support POPCNT instruction set. -1 +MADV_DONTNEED does not zeroed page. jemalloc will be broken From b3fa86191e4f85ad3ff4d72d6370d56b80be868c Mon Sep 17 00:00:00 2001 From: alesapin Date: Sun, 4 Oct 2020 13:23:25 +0300 Subject: [PATCH 309/321] Fix binary build --- docker/packager/binary/build.sh | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/docker/packager/binary/build.sh b/docker/packager/binary/build.sh index 5df0392cb4d..8b552320a29 100755 --- a/docker/packager/binary/build.sh +++ b/docker/packager/binary/build.sh @@ -17,7 +17,9 @@ ccache --show-stats ||: ccache --zero-stats ||: ln -s /usr/lib/x86_64-linux-gnu/libOpenCL.so.1.0.0 /usr/lib/libOpenCL.so ||: rm -f CMakeCache.txt -cmake --debug-trycompile --verbose=1 -DCMAKE_VERBOSE_MAKEFILE=1 -LA "-DCMAKE_BUILD_TYPE=$BUILD_TYPE" "-DSANITIZE=$SANITIZER" -DENABLE_CHECK_HEAVY_BUILDS=1 "$CMAKE_FLAGS" .. + +read -ra CMAKE_FLAGS <<< "${CMAKE_FLAGS:-}" +cmake --debug-trycompile --verbose=1 -DCMAKE_VERBOSE_MAKEFILE=1 -LA "-DCMAKE_BUILD_TYPE=$BUILD_TYPE" "-DSANITIZE=$SANITIZER" -DENABLE_CHECK_HEAVY_BUILDS=1 "${CMAKE_FLAGS[@]}" .. # shellcheck disable=SC2086 # No quotes because I want it to expand to nothing if empty. ninja $NINJA_FLAGS clickhouse-bundle mv ./programs/clickhouse* /output From 9e3ff349eb2a1a1c61fe3323e4fd84ea34b19f12 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sun, 4 Oct 2020 11:24:36 +0300 Subject: [PATCH 310/321] Ensure that there will be no strlen() calls for SSE checks --- programs/main.cpp | 32 +++++++++++++++++++------------- 1 file changed, 19 insertions(+), 13 deletions(-) diff --git a/programs/main.cpp b/programs/main.cpp index de9f26afbd3..fc6908f4095 100644 --- a/programs/main.cpp +++ b/programs/main.cpp @@ -10,6 +10,7 @@ #include #include #include +#include #include /// pair #if !defined(ARCADIA_BUILD) @@ -61,6 +62,7 @@ int mainEntryClickHouseStatus(int argc, char ** argv); int mainEntryClickHouseRestart(int argc, char ** argv); #endif +#define ARRAY_SIZE(a) (sizeof(a)/sizeof((a)[0])) namespace { @@ -154,28 +156,29 @@ enum class InstructionFail AVX512 = 8 }; -const char * instructionFailToString(InstructionFail fail) +std::pair instructionFailToString(InstructionFail fail) { switch (fail) { +#define ret(x) return std::make_pair(x, ARRAY_SIZE(x) - 1) case InstructionFail::NONE: - return "NONE"; + ret("NONE"); case InstructionFail::SSE3: - return "SSE3"; + ret("SSE3"); case InstructionFail::SSSE3: - return "SSSE3"; + ret("SSSE3"); case InstructionFail::SSE4_1: - return "SSE4.1"; + ret("SSE4.1"); case InstructionFail::SSE4_2: - return "SSE4.2"; + ret("SSE4.2"); case InstructionFail::POPCNT: - return "POPCNT"; + ret("POPCNT"); case InstructionFail::AVX: - return "AVX"; + ret("AVX"); case InstructionFail::AVX2: - return "AVX2"; + ret("AVX2"); case InstructionFail::AVX512: - return "AVX512"; + ret("AVX512"); } __builtin_unreachable(); } @@ -242,9 +245,8 @@ void checkRequiredInstructionsImpl(volatile InstructionFail & fail) } /// This function is safe to use in static initializers. -void writeError(const char * data) +void writeErrorLen(const char * data, size_t size) { - size_t size = strlen(data); while (size != 0) { ssize_t res = ::write(STDERR_FILENO, data, size); @@ -259,6 +261,10 @@ void writeError(const char * data) } } } +/// Macros to avoid using strlen(), since it may fail if SSE is not supported. +#define writeError(data) \ + static_assert(__builtin_constant_p(data)); \ + writeErrorLen(data, ARRAY_SIZE(data) - 1) /// Check SSE and others instructions availability. Calls exit on fail. /// This function must be called as early as possible, even before main, because static initializers may use unavailable instructions. @@ -286,7 +292,7 @@ void checkRequiredInstructions() if (sigsetjmp(jmpbuf, 1)) { writeError("Instruction check fail. The CPU does not support "); - writeError(instructionFailToString(fail)); + std::apply(writeErrorLen, instructionFailToString(fail)); writeError(" instruction set.\n"); _Exit(1); } From 8ff92a16ab298ee7caa363f6b6ae50083b003173 Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Mon, 5 Oct 2020 08:01:08 +0300 Subject: [PATCH 311/321] Update main.cpp --- programs/main.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/programs/main.cpp b/programs/main.cpp index fc6908f4095..30cc3770354 100644 --- a/programs/main.cpp +++ b/programs/main.cpp @@ -262,9 +262,9 @@ void writeErrorLen(const char * data, size_t size) } } /// Macros to avoid using strlen(), since it may fail if SSE is not supported. -#define writeError(data) \ +#define writeError(data) do { \ static_assert(__builtin_constant_p(data)); \ - writeErrorLen(data, ARRAY_SIZE(data) - 1) + writeErrorLen(data, ARRAY_SIZE(data) - 1) } while (false) /// Check SSE and others instructions availability. Calls exit on fail. /// This function must be called as early as possible, even before main, because static initializers may use unavailable instructions. From 674d8d44b9e2542e142339c3002da3fb4f3eeb34 Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Mon, 5 Oct 2020 09:28:52 +0300 Subject: [PATCH 312/321] Update main.cpp --- programs/main.cpp | 8 +++++--- 1 file changed, 5 insertions(+), 3 deletions(-) diff --git a/programs/main.cpp b/programs/main.cpp index 30cc3770354..fad2d35f3bd 100644 --- a/programs/main.cpp +++ b/programs/main.cpp @@ -262,9 +262,11 @@ void writeErrorLen(const char * data, size_t size) } } /// Macros to avoid using strlen(), since it may fail if SSE is not supported. -#define writeError(data) do { \ - static_assert(__builtin_constant_p(data)); \ - writeErrorLen(data, ARRAY_SIZE(data) - 1) } while (false) +#define writeError(data) do \ + { \ + static_assert(__builtin_constant_p(data)); \ + writeErrorLen(data, ARRAY_SIZE(data) - 1); \ + } while (false) /// Check SSE and others instructions availability. Calls exit on fail. /// This function must be called as early as possible, even before main, because static initializers may use unavailable instructions. From 9caf46c2735c5891c47610393a4ae5fd20f99e79 Mon Sep 17 00:00:00 2001 From: alesapin Date: Mon, 5 Oct 2020 10:06:38 +0300 Subject: [PATCH 313/321] Update build.sh --- docker/packager/binary/build.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docker/packager/binary/build.sh b/docker/packager/binary/build.sh index 8b552320a29..8acbe271b1f 100755 --- a/docker/packager/binary/build.sh +++ b/docker/packager/binary/build.sh @@ -17,7 +17,7 @@ ccache --show-stats ||: ccache --zero-stats ||: ln -s /usr/lib/x86_64-linux-gnu/libOpenCL.so.1.0.0 /usr/lib/libOpenCL.so ||: rm -f CMakeCache.txt - +# Read cmake arguments into array (possibly empty) read -ra CMAKE_FLAGS <<< "${CMAKE_FLAGS:-}" cmake --debug-trycompile --verbose=1 -DCMAKE_VERBOSE_MAKEFILE=1 -LA "-DCMAKE_BUILD_TYPE=$BUILD_TYPE" "-DSANITIZE=$SANITIZER" -DENABLE_CHECK_HEAVY_BUILDS=1 "${CMAKE_FLAGS[@]}" .. # shellcheck disable=SC2086 # No quotes because I want it to expand to nothing if empty. From 5281288bf635d0c74bb7c08ed9360246cbf71c35 Mon Sep 17 00:00:00 2001 From: alesapin Date: Mon, 5 Oct 2020 10:23:18 +0300 Subject: [PATCH 314/321] More strict check to add clickhouse_memcpy --- CMakeLists.txt | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/CMakeLists.txt b/CMakeLists.txt index aea5e0617a4..b2ac9dfa730 100644 --- a/CMakeLists.txt +++ b/CMakeLists.txt @@ -514,7 +514,8 @@ macro (add_executable target) # invoke built-in add_executable # explicitly acquire and interpose malloc symbols by clickhouse_malloc # if GLIBC_COMPATIBILITY is ON and not sanitizer build, provide memcpy symbol explicitly to neutrialize thinlto's libcall generation. - if (GLIBC_COMPATIBILITY AND NOT SANITIZE) + # `flto=thin` used only for release build, probably this `if` should be more strict. + if (GLIBC_COMPATIBILITY AND NOT SANITIZE AND NOT ARCH_AMD64) _add_executable (${ARGV} $ $) else () _add_executable (${ARGV} $) From 3fbbb6bef9f5ff876fec986613ba351e9c1e7276 Mon Sep 17 00:00:00 2001 From: alesapin Date: Mon, 5 Oct 2020 12:39:03 +0300 Subject: [PATCH 315/321] Fix flag --- CMakeLists.txt | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/CMakeLists.txt b/CMakeLists.txt index b2ac9dfa730..55e0770bf77 100644 --- a/CMakeLists.txt +++ b/CMakeLists.txt @@ -515,7 +515,7 @@ macro (add_executable target) # explicitly acquire and interpose malloc symbols by clickhouse_malloc # if GLIBC_COMPATIBILITY is ON and not sanitizer build, provide memcpy symbol explicitly to neutrialize thinlto's libcall generation. # `flto=thin` used only for release build, probably this `if` should be more strict. - if (GLIBC_COMPATIBILITY AND NOT SANITIZE AND NOT ARCH_AMD64) + if (GLIBC_COMPATIBILITY AND NOT SANITIZE AND NOT ARCH_AARCH64) _add_executable (${ARGV} $ $) else () _add_executable (${ARGV} $) From 8f41b8773765907a795000ac8f1ddbdc6bbed2cc Mon Sep 17 00:00:00 2001 From: alesapin Date: Mon, 5 Oct 2020 12:57:28 +0300 Subject: [PATCH 316/321] Better fix --- CMakeLists.txt | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/CMakeLists.txt b/CMakeLists.txt index 55e0770bf77..9fe56641fcf 100644 --- a/CMakeLists.txt +++ b/CMakeLists.txt @@ -515,7 +515,7 @@ macro (add_executable target) # explicitly acquire and interpose malloc symbols by clickhouse_malloc # if GLIBC_COMPATIBILITY is ON and not sanitizer build, provide memcpy symbol explicitly to neutrialize thinlto's libcall generation. # `flto=thin` used only for release build, probably this `if` should be more strict. - if (GLIBC_COMPATIBILITY AND NOT SANITIZE AND NOT ARCH_AARCH64) + if (GLIBC_COMPATIBILITY AND ENABLE_THINLTO) _add_executable (${ARGV} $ $) else () _add_executable (${ARGV} $) From a644512fce4b28eb635b8d9beaabc8af7002160b Mon Sep 17 00:00:00 2001 From: alesapin Date: Mon, 5 Oct 2020 12:58:23 +0300 Subject: [PATCH 317/321] Better comment --- CMakeLists.txt | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/CMakeLists.txt b/CMakeLists.txt index 9fe56641fcf..cb9134d2ac0 100644 --- a/CMakeLists.txt +++ b/CMakeLists.txt @@ -513,8 +513,7 @@ endif () macro (add_executable target) # invoke built-in add_executable # explicitly acquire and interpose malloc symbols by clickhouse_malloc - # if GLIBC_COMPATIBILITY is ON and not sanitizer build, provide memcpy symbol explicitly to neutrialize thinlto's libcall generation. - # `flto=thin` used only for release build, probably this `if` should be more strict. + # if GLIBC_COMPATIBILITY is ON and ENABLE_THINLTO is on than provide memcpy symbol explicitly to neutrialize thinlto's libcall generation. if (GLIBC_COMPATIBILITY AND ENABLE_THINLTO) _add_executable (${ARGV} $ $) else () From 0c73b3b99fca6b6573bb03aae179b1fb48e7c7bb Mon Sep 17 00:00:00 2001 From: alesapin Date: Mon, 5 Oct 2020 13:20:06 +0300 Subject: [PATCH 318/321] Fix woboq codebrowser image --- docker/test/codebrowser/Dockerfile | 25 +++---------------------- 1 file changed, 3 insertions(+), 22 deletions(-) diff --git a/docker/test/codebrowser/Dockerfile b/docker/test/codebrowser/Dockerfile index cb3462cad0e..241fc643251 100644 --- a/docker/test/codebrowser/Dockerfile +++ b/docker/test/codebrowser/Dockerfile @@ -1,27 +1,8 @@ # docker build --network=host -t yandex/clickhouse-codebrowser . # docker run --volume=path_to_repo:/repo_folder --volume=path_to_result:/test_output yandex/clickhouse-codebrowser -FROM ubuntu:18.04 +FROM yandex/clickhouse-binary-builder -RUN apt-get --allow-unauthenticated update -y \ - && env DEBIAN_FRONTEND=noninteractive \ - apt-get --allow-unauthenticated install --yes --no-install-recommends \ - bash \ - sudo \ - wget \ - software-properties-common \ - ca-certificates \ - apt-transport-https \ - build-essential \ - gpg-agent \ - git - -RUN wget -nv -O - https://apt.kitware.com/keys/kitware-archive-latest.asc | sudo apt-key add - -RUN sudo apt-add-repository 'deb https://apt.kitware.com/ubuntu/ bionic main' -RUN sudo echo "deb [trusted=yes] http://apt.llvm.org/bionic/ llvm-toolchain-bionic-8 main" >> /etc/apt/sources.list - -RUN sudo apt-get --yes --allow-unauthenticated update -# To build woboq -RUN sudo apt-get --yes --allow-unauthenticated install cmake clang-8 libllvm8 libclang-8-dev +RUN apt-get update && apt-get --yes --allow-unauthenticated install clang-8 libllvm8 libclang-8-dev # repo versions doesn't work correctly with C++17 # also we push reports to s3, so we add index.html to subfolder urls @@ -40,7 +21,7 @@ ENV SHA=nosha ENV DATA="data" CMD mkdir -p $BUILD_DIRECTORY && cd $BUILD_DIRECTORY && \ - cmake $SOURCE_DIRECTORY -DCMAKE_CXX_COMPILER=/usr/bin/clang\+\+-8 -DCMAKE_C_COMPILER=/usr/bin/clang-8 -DCMAKE_EXPORT_COMPILE_COMMANDS=ON && \ + cmake $SOURCE_DIRECTORY -DCMAKE_CXX_COMPILER=/usr/bin/clang\+\+-11 -DCMAKE_C_COMPILER=/usr/bin/clang-11 -DCMAKE_EXPORT_COMPILE_COMMANDS=ON && \ mkdir -p $HTML_RESULT_DIRECTORY && \ $CODEGEN -b $BUILD_DIRECTORY -a -o $HTML_RESULT_DIRECTORY -p ClickHouse:$SOURCE_DIRECTORY:$SHA -d $DATA && \ cp -r $STATIC_DATA $HTML_RESULT_DIRECTORY/ &&\ From a940298eb4b336080f2a942b738510bbad10db20 Mon Sep 17 00:00:00 2001 From: alesapin Date: Mon, 5 Oct 2020 13:54:36 +0300 Subject: [PATCH 319/321] One more fix --- docker/test/codebrowser/Dockerfile | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docker/test/codebrowser/Dockerfile b/docker/test/codebrowser/Dockerfile index 241fc643251..94657835dfe 100644 --- a/docker/test/codebrowser/Dockerfile +++ b/docker/test/codebrowser/Dockerfile @@ -8,7 +8,7 @@ RUN apt-get update && apt-get --yes --allow-unauthenticated install clang-8 libl # also we push reports to s3, so we add index.html to subfolder urls # https://github.com/ClickHouse-Extras/woboq_codebrowser/commit/37e15eaf377b920acb0b48dbe82471be9203f76b RUN git clone https://github.com/ClickHouse-Extras/woboq_codebrowser -RUN cd woboq_codebrowser && cmake . -DCMAKE_BUILD_TYPE=Release && make -j +RUN cd woboq_codebrowser && cmake . -DCMAKE_BUILD_TYPE=Release -DCMAKE_CXX_COMPILER=clang++-8 -DCMAKE_C_COMPILER=clang-8 && make -j ENV CODEGEN=/woboq_codebrowser/generator/codebrowser_generator ENV CODEINDEX=/woboq_codebrowser/indexgenerator/codebrowser_indexgenerator From 0b67ee8e54e971062aa3950a357220a93498f068 Mon Sep 17 00:00:00 2001 From: alesapin Date: Mon, 5 Oct 2020 14:35:23 +0300 Subject: [PATCH 320/321] Trying with clang-9 --- docker/images.json | 3 ++- docker/test/codebrowser/Dockerfile | 5 +++-- 2 files changed, 5 insertions(+), 3 deletions(-) diff --git a/docker/images.json b/docker/images.json index e9e91864e1e..f5b10a14313 100644 --- a/docker/images.json +++ b/docker/images.json @@ -9,7 +9,8 @@ "name": "yandex/clickhouse-binary-builder", "dependent": [ "docker/test/split_build_smoke_test", - "docker/test/pvs" + "docker/test/pvs", + "docker/test/codebrowser" ] }, "docker/packager/unbundled": { diff --git a/docker/test/codebrowser/Dockerfile b/docker/test/codebrowser/Dockerfile index 94657835dfe..e03f94a85e0 100644 --- a/docker/test/codebrowser/Dockerfile +++ b/docker/test/codebrowser/Dockerfile @@ -2,13 +2,14 @@ # docker run --volume=path_to_repo:/repo_folder --volume=path_to_result:/test_output yandex/clickhouse-codebrowser FROM yandex/clickhouse-binary-builder -RUN apt-get update && apt-get --yes --allow-unauthenticated install clang-8 libllvm8 libclang-8-dev +RUN apt-get update && apt-get --yes --allow-unauthenticated install clang-9 libllvm9 libclang-9-dev # repo versions doesn't work correctly with C++17 # also we push reports to s3, so we add index.html to subfolder urls # https://github.com/ClickHouse-Extras/woboq_codebrowser/commit/37e15eaf377b920acb0b48dbe82471be9203f76b RUN git clone https://github.com/ClickHouse-Extras/woboq_codebrowser -RUN cd woboq_codebrowser && cmake . -DCMAKE_BUILD_TYPE=Release -DCMAKE_CXX_COMPILER=clang++-8 -DCMAKE_C_COMPILER=clang-8 && make -j + +RUN cd woboq_codebrowser && cmake . -DCMAKE_BUILD_TYPE=Release -DCMAKE_CXX_COMPILER=clang\+\+-9 -DCMAKE_C_COMPILER=clang-9 && make -j ENV CODEGEN=/woboq_codebrowser/generator/codebrowser_generator ENV CODEINDEX=/woboq_codebrowser/indexgenerator/codebrowser_indexgenerator From a77d232411c20795dee16032e412c7f5f597bdc4 Mon Sep 17 00:00:00 2001 From: olgarev <56617294+olgarev@users.noreply.github.com> Date: Mon, 5 Oct 2020 16:06:16 +0300 Subject: [PATCH 321/321] DOCSUP-2407: Documented the bayesAB function (#15599) * Docs for the bayesAB function, english. * Note edited. Co-authored-by: Olga Revyakina --- .../functions/machine-learning-functions.md | 79 +++++++++++++++++++ 1 file changed, 79 insertions(+) diff --git a/docs/en/sql-reference/functions/machine-learning-functions.md b/docs/en/sql-reference/functions/machine-learning-functions.md index 9de3854c3e3..8627fc26bad 100644 --- a/docs/en/sql-reference/functions/machine-learning-functions.md +++ b/docs/en/sql-reference/functions/machine-learning-functions.md @@ -16,3 +16,82 @@ The [stochasticLinearRegression](../../sql-reference/aggregate-functions/referen ## stochasticLogisticRegression {#stochastic-logistic-regression} The [stochasticLogisticRegression](../../sql-reference/aggregate-functions/reference/stochasticlogisticregression.md#agg_functions-stochasticlogisticregression) aggregate function implements stochastic gradient descent method for binary classification problem. Uses `evalMLMethod` to predict on new data. + +## bayesAB {#bayesab} + +Compares test groups (variants) and calculates for each group the probability to be the best one. The first group is used as a control group. + +**Syntax** + +``` sql +bayesAB(distribution_name, higher_is_better, variant_names, x, y) +``` + +**Parameters** + +- `distribution_name` — Name of the probability distribution. [String](../../sql-reference/data-types/string.md). Possible values: + + - `beta` for [Beta distribution](https://en.wikipedia.org/wiki/Beta_distribution) + - `gamma` for [Gamma distribution](https://en.wikipedia.org/wiki/Gamma_distribution) + +- `higher_is_better` — Boolean flag. [Boolean](../../sql-reference/data-types/boolean.md). Possible values: + + - `0` - lower values are considered to be better than higher + - `1` - higher values are considered to be better than lower + +- `variant_names` - Variant names. [Array](../../sql-reference/data-types/array.md)([String](../../sql-reference/data-types/string.md)). + +- `x` - Numbers of tests for the corresponding variants. [Array](../../sql-reference/data-types/array.md)([Float64](../../sql-reference/data-types/float.md)). + +- `y` - Numbers of successful tests for the corresponding variants. [Array](../../sql-reference/data-types/array.md)([Float64](../../sql-reference/data-types/float.md)). + +!!! note "Note" + All three arrays must have the same size. All `x` and `y` values must be non-negative constant numbers. `y` cannot be larger than `x`. + +**Returned values** + +For each variant the function calculates: +- `beats_control` - long-term probability to out-perform the first (control) variant +- `to_be_best` - long-term probability to out-perform all other variants + +Type: JSON. + +**Example** + +Query: + +``` sql +SELECT bayesAB('beta', 1, ['Control', 'A', 'B'], [3000., 3000., 3000.], [100., 90., 110.]) FORMAT PrettySpace; +``` + +Result: + +``` text +{ + "data":[ + { + "variant_name":"Control", + "x":3000, + "y":100, + "beats_control":0, + "to_be_best":0.22619 + }, + { + "variant_name":"A", + "x":3000, + "y":90, + "beats_control":0.23469, + "to_be_best":0.04671 + }, + { + "variant_name":"B", + "x":3000, + "y":110, + "beats_control":0.7580899999999999, + "to_be_best":0.7271 + } + ] +} +``` + +[Original article](https://clickhouse.tech/docs/en/query_language/functions/machine-learning-functions/)