From c65fec341ce246a4c95c0b2c1256762ba6308785 Mon Sep 17 00:00:00 2001 From: Vasily Nemkov Date: Tue, 23 Oct 2018 10:16:28 +0300 Subject: [PATCH 01/31] Test for MySQL as range_hashed dictionary source --- .../generate_and_test.py | 74 ++++++++++++++++--- 1 file changed, 64 insertions(+), 10 deletions(-) diff --git a/dbms/tests/external_dictionaries/generate_and_test.py b/dbms/tests/external_dictionaries/generate_and_test.py index 98b67345326..db2f4fe5e9c 100755 --- a/dbms/tests/external_dictionaries/generate_and_test.py +++ b/dbms/tests/external_dictionaries/generate_and_test.py @@ -145,6 +145,15 @@ def generate_structure(args): # [ 'executable_flat' + base_name, 3, True ] ]) + if not args.no_mysql: + for range_hashed_range_type in range_hashed_range_types: + base_name = 'range_hashed_' + range_hashed_range_type + dictionaries.extend([ + ['mysql_' + base_name, 3, False], + # [ 'clickhouse_' + base_name, 3, True ], + # [ 'executable_flat' + base_name, 3, True ] + ]) + files = [ 'key_simple.tsv', 'key_complex_integers.tsv', 'key_complex_mixed.tsv', 'key_range_hashed_{range_hashed_range_type}.tsv' ] @@ -206,6 +215,20 @@ range_hashed_dictGet_values = { ("toDateTime('2015-11-19 23:59:59')", "toDateTime('2015-10-26 00:00:01')", "toDateTime('2018-09-14 00:00:00')")], } +range_hashed_mysql_column_types = { + 'UInt8': 'tinyint unsigned', + 'UInt16': 'smallint unsigned', + 'UInt32': 'int unsigned', + 'UInt64': 'bigint unsigned', + 'Int8': 'tinyint', + 'Int16': 'smallint', + 'Int32': 'int', + 'Int64': 'bigint', + # default type (Date) for compatibility with older versions: + '': 'date', + 'Date': 'date', + 'DateTime': 'datetime', +} def dump_report(destination, suite, test_case, report): if destination is not None: @@ -268,6 +291,24 @@ def generate_data(args): query = file_source_query % comma_separated(chain(keys, columns(), ['Parent'] if 1 == len(keys) else [])) call([args.client, '--port', args.port, '--query', query], 'generated/' + file) + if not args.no_mysql: + print 'Creating MySQL table for "{0}"...'.format(range_hashed_range_type) + table_name = "test.dictionary_source_" + range_hashed_range_type + col_type = range_hashed_mysql_column_types[range_hashed_range_type] + subprocess.check_call('echo "' + 'create database if not exists test;' + 'drop table if exists {table_name};' + 'create table {table_name} (' + 'id tinyint unsigned, StartDate {col_type}, EndDate {col_type}, ' + 'UInt8_ tinyint unsigned, UInt16_ smallint unsigned, UInt32_ int unsigned, UInt64_ bigint unsigned, ' + 'Int8_ tinyint, Int16_ smallint, Int32_ int, Int64_ bigint, ' + 'Float32_ float, Float64_ double, ' + 'String_ text, Date_ date, DateTime_ datetime, Parent bigint unsigned, UUID_ varchar(36)' + ');' + 'load data local infile \'{0}/generated/{file}\' into table {table_name};" | mysql $MYSQL_OPTIONS --local-infile=1' + .format(prefix, table_name=table_name, col_type=col_type, file=file), shell=True) + + # create MySQL table from complete_query if not args.no_mysql: print 'Creating MySQL table' @@ -384,7 +425,7 @@ def generate_dictionaries(args): root test - dictionary_source
+ dictionary_source{key_type}
''' @@ -568,14 +609,15 @@ def generate_dictionaries(args): ]) if not args.no_mysql: + source_mysql_default = source_mysql.format(key_type="") sources_and_layouts.extend([ - [ source_mysql, layout_flat ], - [ source_mysql, layout_hashed ], - [ source_mysql, layout_cache ], - [ source_mysql, layout_complex_key_hashed ], - [ source_mysql, layout_complex_key_cache ], - [ source_mysql, layout_complex_key_hashed ], - [ source_mysql, layout_complex_key_cache ], + [ source_mysql_default, layout_flat ], + [ source_mysql_default, layout_hashed ], + [ source_mysql_default, layout_cache ], + [ source_mysql_default, layout_complex_key_hashed ], + [ source_mysql_default, layout_complex_key_cache ], + [ source_mysql_default, layout_complex_key_hashed ], + [ source_mysql_default, layout_complex_key_cache ], ]) if not args.no_mongo: @@ -619,10 +661,22 @@ def generate_dictionaries(args): # [ source_executable, layout_range_hashed ] ]) + if not args.no_mysql: + for range_hashed_range_type in range_hashed_range_types: + key_type = "_" + range_hashed_range_type + source_mysql_typed = source_mysql.format(key_type=key_type) + sources_and_layouts.extend([ + [source_mysql_typed, + (layout_range_hashed, range_hashed_range_type)], + ]) + + dict_name_filter = args.filter.split('/')[0] for (name, key_idx, has_parent), (source, layout) in zip(dictionaries, sources_and_layouts): + if args.filter and not fnmatch.fnmatch(name, dict_name_filter): + continue + filename = os.path.join(args.generated, 'dictionary_%s.xml' % name) key = keys[key_idx] - if key_idx == 3: layout, range_hashed_range_type = layout # Wrap non-empty type (default) with tag. @@ -670,7 +724,7 @@ def run_tests(args): global SERVER_DIED print "{0:100}".format('Dictionary: ' + dict + ' Name: ' + name + ": "), - if args.filter and not fnmatch.fnmatch(dict, args.filter) and not fnmatch.fnmatch(name, args.filter): + if args.filter and not fnmatch.fnmatch(dict + "/" + name, args.filter): print " ... skipped due to filter." return From 070d75c7c9932198d01a65095450e4bdede8cf1c Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 24 Oct 2018 14:31:23 +0300 Subject: [PATCH 02/31] Don't unfold non deterministic expressions --- dbms/src/Interpreters/ExpressionActions.cpp | 8 +++++++- ...mpile_non_deterministic_function.reference | 4 ++++ ...746_compile_non_deterministic_function.sql | 20 +++++++++++++++++++ 3 files changed, 31 insertions(+), 1 deletion(-) create mode 100644 dbms/tests/queries/0_stateless/00746_compile_non_deterministic_function.reference create mode 100644 dbms/tests/queries/0_stateless/00746_compile_non_deterministic_function.sql diff --git a/dbms/src/Interpreters/ExpressionActions.cpp b/dbms/src/Interpreters/ExpressionActions.cpp index b84c007d790..4b5ccfde219 100644 --- a/dbms/src/Interpreters/ExpressionActions.cpp +++ b/dbms/src/Interpreters/ExpressionActions.cpp @@ -206,8 +206,14 @@ void ExpressionAction::prepare(Block & sample_block, const Settings & settings) if (auto * prepared_function = dynamic_cast(function.get())) prepared_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. - if (all_const && function_base->isSuitableForConstantFolding()) + /// 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()); diff --git a/dbms/tests/queries/0_stateless/00746_compile_non_deterministic_function.reference b/dbms/tests/queries/0_stateless/00746_compile_non_deterministic_function.reference new file mode 100644 index 00000000000..3750f104e7f --- /dev/null +++ b/dbms/tests/queries/0_stateless/00746_compile_non_deterministic_function.reference @@ -0,0 +1,4 @@ +2 +0 +0 +3 diff --git a/dbms/tests/queries/0_stateless/00746_compile_non_deterministic_function.sql b/dbms/tests/queries/0_stateless/00746_compile_non_deterministic_function.sql new file mode 100644 index 00000000000..3e83a2284c9 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00746_compile_non_deterministic_function.sql @@ -0,0 +1,20 @@ +SET compile_expressions = 1; +SET min_count_to_compile = 1; + +DROP TABLE IF EXISTS test.time_table; + +CREATE TABLE test.time_table(timecol DateTime, value Int32) ENGINE = MergeTree order by tuple(); + +INSERT INTO test.time_table VALUES (now() - 5, 5), (now() - 3, 3); + +SELECT COUNT() from test.time_table WHERE value < now() - 1 AND value != 0 AND modulo(value, 2) != 0 AND timecol < now() - 1; + +SELECT sleep(3); + +INSERT INTO test.time_table VALUES (now(), 101); + +SELECT sleep(3); + +SELECT COUNT() from test.time_table WHERE value < now() - 1 AND value != 0 AND modulo(value, 2) != 0 AND timecol < now() - 1; + +DROP TABLE IF EXISTS test.time_table; From 54a1f0c2a64c71797ade3f0dfea6708d95051c90 Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 24 Oct 2018 14:36:02 +0300 Subject: [PATCH 03/31] enable compiled expressions byt default --- dbms/src/Interpreters/Settings.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/src/Interpreters/Settings.h b/dbms/src/Interpreters/Settings.h index 1890d5a7834..6eb85e9c4df 100644 --- a/dbms/src/Interpreters/Settings.h +++ b/dbms/src/Interpreters/Settings.h @@ -74,7 +74,7 @@ struct Settings M(SettingFloat, totals_auto_threshold, 0.5, "The threshold for totals_mode = 'auto'.") \ \ M(SettingBool, compile, false, "Whether query compilation is enabled.") \ - M(SettingBool, compile_expressions, false, "Compile some scalar functions and operators to native code.") \ + M(SettingBool, compile_expressions, true, "Compile some scalar functions and operators to native code.") \ M(SettingUInt64, min_count_to_compile, 3, "The number of structurally identical queries before they are compiled.") \ M(SettingUInt64, group_by_two_level_threshold, 100000, "From what number of keys, a two-level aggregation starts. 0 - the threshold is not set.") \ M(SettingUInt64, group_by_two_level_threshold_bytes, 100000000, "From what size of the aggregation state in bytes, a two-level aggregation begins to be used. 0 - the threshold is not set. Two-level aggregation is used when at least one of the thresholds is triggered.") \ From 58f14b0db2e117e0283d99d6f584115434be3c50 Mon Sep 17 00:00:00 2001 From: chertus Date: Wed, 24 Oct 2018 18:31:07 +0300 Subject: [PATCH 04/31] USE database for ON CLUSTER DDLs [CLICKHOUSE-4036] --- .../Interpreters/AddDefaultDatabaseVisitor.h | 65 +++++++++++++++++++ dbms/src/Interpreters/DDLWorker.cpp | 8 +++ dbms/src/Parsers/ASTAlterQuery.cpp | 12 ---- dbms/src/Parsers/ASTAlterQuery.h | 5 +- dbms/src/Parsers/ASTCreateQuery.h | 9 +-- dbms/src/Parsers/ASTDropQuery.cpp | 12 ---- dbms/src/Parsers/ASTDropQuery.h | 5 +- dbms/src/Parsers/ASTKillQueryQuery.cpp | 10 --- dbms/src/Parsers/ASTKillQueryQuery.h | 5 +- dbms/src/Parsers/ASTOptimizeQuery.cpp | 13 ---- dbms/src/Parsers/ASTOptimizeQuery.h | 5 +- dbms/src/Parsers/ASTQueryWithOnCluster.h | 21 ++++++ 12 files changed, 111 insertions(+), 59 deletions(-) create mode 100644 dbms/src/Interpreters/AddDefaultDatabaseVisitor.h diff --git a/dbms/src/Interpreters/AddDefaultDatabaseVisitor.h b/dbms/src/Interpreters/AddDefaultDatabaseVisitor.h new file mode 100644 index 00000000000..87d45aa0ade --- /dev/null +++ b/dbms/src/Interpreters/AddDefaultDatabaseVisitor.h @@ -0,0 +1,65 @@ +#pragma once + +#include +#include +#include + +namespace DB +{ + +/// Visits AST nodes, add default database to DDLs if not set. +class AddDefaultDatabaseVisitor +{ +public: + AddDefaultDatabaseVisitor(const String & default_database_) + : default_database(default_database_) + {} + + void visit(ASTPtr & ast) const + { + visitChildren(ast); + + if (!tryVisit(ast) && + !tryVisit(ast)) + {} + } + +private: + const String default_database; + + void visit(ASTQueryWithTableAndOutput * node, ASTPtr &) const + { + if (node->database.empty()) + node->database = default_database; + } + + void visit(ASTRenameQuery * node, ASTPtr &) const + { + for (ASTRenameQuery::Element & elem : node->elements) + { + if (elem.from.database.empty()) + elem.from.database = default_database; + if (elem.to.database.empty()) + elem.to.database = default_database; + } + } + + void visitChildren(ASTPtr & ast) const + { + for (auto & child : ast->children) + visit(child); + } + + template + bool tryVisit(ASTPtr & ast) const + { + if (T * t = dynamic_cast(ast.get())) + { + visit(t, ast); + return true; + } + return false; + } +}; + +} diff --git a/dbms/src/Interpreters/DDLWorker.cpp b/dbms/src/Interpreters/DDLWorker.cpp index 8fa02c91f6a..46401f999b7 100644 --- a/dbms/src/Interpreters/DDLWorker.cpp +++ b/dbms/src/Interpreters/DDLWorker.cpp @@ -12,6 +12,7 @@ #include #include #include +#include #include #include #include @@ -1159,6 +1160,13 @@ BlockIO executeDDLQueryOnCluster(const ASTPtr & query_ptr_, const Context & cont } } + const String & default_database = context.getCurrentDatabase(); + if (!default_database.empty()) + { + AddDefaultDatabaseVisitor visitor(context.getCurrentDatabase()); + visitor.visit(query_ptr); + } + query->cluster = context.getMacros()->expand(query->cluster); ClusterPtr cluster = context.getCluster(query->cluster); DDLWorker & ddl_worker = context.getDDLWorker(); diff --git a/dbms/src/Parsers/ASTAlterQuery.cpp b/dbms/src/Parsers/ASTAlterQuery.cpp index 7081b512247..8869e873c71 100644 --- a/dbms/src/Parsers/ASTAlterQuery.cpp +++ b/dbms/src/Parsers/ASTAlterQuery.cpp @@ -187,18 +187,6 @@ ASTPtr ASTAlterQuery::clone() const return res; } -ASTPtr ASTAlterQuery::getRewrittenASTWithoutOnCluster(const std::string & new_database) const -{ - auto query_ptr = clone(); - auto & query = static_cast(*query_ptr); - - query.cluster.clear(); - if (query.database.empty()) - query.database = new_database; - - return query_ptr; -} - void ASTAlterQuery::formatQueryImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const { frame.need_parens = false; diff --git a/dbms/src/Parsers/ASTAlterQuery.h b/dbms/src/Parsers/ASTAlterQuery.h index ced92bfb9bd..9bfcdae2427 100644 --- a/dbms/src/Parsers/ASTAlterQuery.h +++ b/dbms/src/Parsers/ASTAlterQuery.h @@ -122,7 +122,10 @@ public: ASTPtr clone() const override; - ASTPtr getRewrittenASTWithoutOnCluster(const std::string & new_database) const override; + ASTPtr getRewrittenASTWithoutOnCluster(const std::string & new_database) const override + { + return removeOnCluster(clone(), new_database); + } protected: void formatQueryImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override; diff --git a/dbms/src/Parsers/ASTCreateQuery.h b/dbms/src/Parsers/ASTCreateQuery.h index 100aee8e4f0..bbc7e35bff6 100644 --- a/dbms/src/Parsers/ASTCreateQuery.h +++ b/dbms/src/Parsers/ASTCreateQuery.h @@ -112,14 +112,7 @@ public: ASTPtr getRewrittenASTWithoutOnCluster(const std::string & new_database) const override { - auto query_ptr = clone(); - ASTCreateQuery & query = static_cast(*query_ptr); - - query.cluster.clear(); - if (query.database.empty()) - query.database = new_database; - - return query_ptr; + return removeOnCluster(clone(), new_database); } protected: diff --git a/dbms/src/Parsers/ASTDropQuery.cpp b/dbms/src/Parsers/ASTDropQuery.cpp index aa47829519a..6b6b9b0bec2 100644 --- a/dbms/src/Parsers/ASTDropQuery.cpp +++ b/dbms/src/Parsers/ASTDropQuery.cpp @@ -29,18 +29,6 @@ ASTPtr ASTDropQuery::clone() const return res; } -ASTPtr ASTDropQuery::getRewrittenASTWithoutOnCluster(const std::string & new_database) const -{ - auto query_ptr = clone(); - auto & query = static_cast(*query_ptr); - - query.cluster.clear(); - if (query.database.empty()) - query.database = new_database; - - return query_ptr; -} - void ASTDropQuery::formatQueryImpl(const FormatSettings & settings, FormatState &, FormatStateStacked) const { settings.ostr << (settings.hilite ? hilite_keyword : ""); diff --git a/dbms/src/Parsers/ASTDropQuery.h b/dbms/src/Parsers/ASTDropQuery.h index 0916eb757a0..83b5d28e38b 100644 --- a/dbms/src/Parsers/ASTDropQuery.h +++ b/dbms/src/Parsers/ASTDropQuery.h @@ -26,7 +26,10 @@ public: String getID() const override; ASTPtr clone() const override; - ASTPtr getRewrittenASTWithoutOnCluster(const std::string & new_database) const override; + ASTPtr getRewrittenASTWithoutOnCluster(const std::string & new_database) const override + { + return removeOnCluster(clone(), new_database); + } protected: void formatQueryImpl(const FormatSettings & settings, FormatState &, FormatStateStacked) const override; diff --git a/dbms/src/Parsers/ASTKillQueryQuery.cpp b/dbms/src/Parsers/ASTKillQueryQuery.cpp index 0f3e5406fdd..a8b351cdb39 100644 --- a/dbms/src/Parsers/ASTKillQueryQuery.cpp +++ b/dbms/src/Parsers/ASTKillQueryQuery.cpp @@ -8,16 +8,6 @@ String ASTKillQueryQuery::getID() const return "KillQueryQuery_" + (where_expression ? where_expression->getID() : "") + "_" + String(sync ? "SYNC" : "ASYNC"); } -ASTPtr ASTKillQueryQuery::getRewrittenASTWithoutOnCluster(const std::string & /*new_database*/) const -{ - auto query_ptr = clone(); - ASTKillQueryQuery & query = static_cast(*query_ptr); - - query.cluster.clear(); - - return query_ptr; -} - void ASTKillQueryQuery::formatQueryImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const { settings.ostr << (settings.hilite ? hilite_keyword : "") << "KILL QUERY "; diff --git a/dbms/src/Parsers/ASTKillQueryQuery.h b/dbms/src/Parsers/ASTKillQueryQuery.h index 0592062ccd6..491bd3aecd2 100644 --- a/dbms/src/Parsers/ASTKillQueryQuery.h +++ b/dbms/src/Parsers/ASTKillQueryQuery.h @@ -24,7 +24,10 @@ public: void formatQueryImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override; - ASTPtr getRewrittenASTWithoutOnCluster(const std::string &new_database) const override; + ASTPtr getRewrittenASTWithoutOnCluster(const std::string &) const override + { + return removeOnCluster(clone()); + } }; } diff --git a/dbms/src/Parsers/ASTOptimizeQuery.cpp b/dbms/src/Parsers/ASTOptimizeQuery.cpp index dd37b665173..5e95dc41795 100644 --- a/dbms/src/Parsers/ASTOptimizeQuery.cpp +++ b/dbms/src/Parsers/ASTOptimizeQuery.cpp @@ -3,19 +3,6 @@ namespace DB { - -ASTPtr ASTOptimizeQuery::getRewrittenASTWithoutOnCluster(const std::string & new_database) const -{ - auto query_ptr = clone(); - ASTOptimizeQuery & query = static_cast(*query_ptr); - - query.cluster.clear(); - if (query.database.empty()) - query.database = new_database; - - return query_ptr; -} - void ASTOptimizeQuery::formatQueryImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const { settings.ostr << (settings.hilite ? hilite_keyword : "") << "OPTIMIZE TABLE " << (settings.hilite ? hilite_none : "") diff --git a/dbms/src/Parsers/ASTOptimizeQuery.h b/dbms/src/Parsers/ASTOptimizeQuery.h index 269ea5b19ff..d228a8c905f 100644 --- a/dbms/src/Parsers/ASTOptimizeQuery.h +++ b/dbms/src/Parsers/ASTOptimizeQuery.h @@ -40,7 +40,10 @@ public: void formatQueryImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override; - ASTPtr getRewrittenASTWithoutOnCluster(const std::string &new_database) const override; + ASTPtr getRewrittenASTWithoutOnCluster(const std::string &new_database) const override + { + return removeOnCluster(clone(), new_database); + } }; } diff --git a/dbms/src/Parsers/ASTQueryWithOnCluster.h b/dbms/src/Parsers/ASTQueryWithOnCluster.h index f850dd9014c..373307700aa 100644 --- a/dbms/src/Parsers/ASTQueryWithOnCluster.h +++ b/dbms/src/Parsers/ASTQueryWithOnCluster.h @@ -28,6 +28,27 @@ public: static bool parse(Pos & pos, std::string & cluster_str, Expected & expected); virtual ~ASTQueryWithOnCluster() = default; + +protected: + template + static ASTPtr removeOnCluster(ASTPtr query_ptr, const std::string & new_database) + { + T & query = static_cast(*query_ptr); + + query.cluster.clear(); + if (query.database.empty()) + query.database = new_database; + + return query_ptr; + } + + template + static ASTPtr removeOnCluster(ASTPtr query_ptr) + { + T & query = static_cast(*query_ptr); + query.cluster.clear(); + return query_ptr; + } }; } From a844d10a9aee5ffc8e52755871103c58ff1b3bdb Mon Sep 17 00:00:00 2001 From: chertus Date: Wed, 24 Oct 2018 18:39:37 +0300 Subject: [PATCH 05/31] minor fix --- dbms/src/Interpreters/DDLWorker.cpp | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/dbms/src/Interpreters/DDLWorker.cpp b/dbms/src/Interpreters/DDLWorker.cpp index 46401f999b7..3ee01c1571a 100644 --- a/dbms/src/Interpreters/DDLWorker.cpp +++ b/dbms/src/Interpreters/DDLWorker.cpp @@ -1160,10 +1160,10 @@ BlockIO executeDDLQueryOnCluster(const ASTPtr & query_ptr_, const Context & cont } } - const String & default_database = context.getCurrentDatabase(); - if (!default_database.empty()) + const String & current_database = context.getCurrentDatabase(); + if (!current_database.empty()) { - AddDefaultDatabaseVisitor visitor(context.getCurrentDatabase()); + AddDefaultDatabaseVisitor visitor(current_database); visitor.visit(query_ptr); } @@ -1193,7 +1193,7 @@ BlockIO executeDDLQueryOnCluster(const ASTPtr & query_ptr_, const Context & cont } for (const String & database : databases_to_check_access_rights) - context.checkDatabaseAccessRights(database.empty() ? context.getCurrentDatabase() : database); + context.checkDatabaseAccessRights(database.empty() ? current_database : database); String node_path = ddl_worker.enqueueQuery(entry); From 7a24e9b8ff5a33253f0fb8c946ee424f4fe1dc30 Mon Sep 17 00:00:00 2001 From: chertus Date: Wed, 24 Oct 2018 18:41:28 +0300 Subject: [PATCH 06/31] one more minor fix --- dbms/src/Interpreters/AddDefaultDatabaseVisitor.h | 1 - 1 file changed, 1 deletion(-) diff --git a/dbms/src/Interpreters/AddDefaultDatabaseVisitor.h b/dbms/src/Interpreters/AddDefaultDatabaseVisitor.h index 87d45aa0ade..a9aed99c10f 100644 --- a/dbms/src/Interpreters/AddDefaultDatabaseVisitor.h +++ b/dbms/src/Interpreters/AddDefaultDatabaseVisitor.h @@ -1,6 +1,5 @@ #pragma once -#include #include #include From 8aa0c423aad50567d1043fe8fab392d433696bff Mon Sep 17 00:00:00 2001 From: Vasily Nemkov Date: Thu, 25 Oct 2018 07:16:41 +0300 Subject: [PATCH 07/31] Fix for the case when there is no filter. --- dbms/tests/external_dictionaries/generate_and_test.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/tests/external_dictionaries/generate_and_test.py b/dbms/tests/external_dictionaries/generate_and_test.py index db2f4fe5e9c..a2cab7006a7 100755 --- a/dbms/tests/external_dictionaries/generate_and_test.py +++ b/dbms/tests/external_dictionaries/generate_and_test.py @@ -670,7 +670,7 @@ def generate_dictionaries(args): (layout_range_hashed, range_hashed_range_type)], ]) - dict_name_filter = args.filter.split('/')[0] + dict_name_filter = args.filter.split('/')[0] if args.filter else None for (name, key_idx, has_parent), (source, layout) in zip(dictionaries, sources_and_layouts): if args.filter and not fnmatch.fnmatch(name, dict_name_filter): continue From 7ebc44a0c877c3ad097111c4ec31d3cdb8444a9c Mon Sep 17 00:00:00 2001 From: Vasily Nemkov Date: Thu, 25 Oct 2018 07:42:02 +0300 Subject: [PATCH 08/31] Fixed test case for UUID_ dictionary attribute. --- dbms/tests/external_dictionaries/generate_and_test.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/tests/external_dictionaries/generate_and_test.py b/dbms/tests/external_dictionaries/generate_and_test.py index a2cab7006a7..7722d9b34b2 100755 --- a/dbms/tests/external_dictionaries/generate_and_test.py +++ b/dbms/tests/external_dictionaries/generate_and_test.py @@ -303,7 +303,7 @@ def generate_data(args): 'UInt8_ tinyint unsigned, UInt16_ smallint unsigned, UInt32_ int unsigned, UInt64_ bigint unsigned, ' 'Int8_ tinyint, Int16_ smallint, Int32_ int, Int64_ bigint, ' 'Float32_ float, Float64_ double, ' - 'String_ text, Date_ date, DateTime_ datetime, Parent bigint unsigned, UUID_ varchar(36)' + 'String_ text, Date_ date, DateTime_ datetime, UUID_ varchar(36)' ');' 'load data local infile \'{0}/generated/{file}\' into table {table_name};" | mysql $MYSQL_OPTIONS --local-infile=1' .format(prefix, table_name=table_name, col_type=col_type, file=file), shell=True) From f2533a9653531c8381e5dd9365e3ec1e9f5838ed Mon Sep 17 00:00:00 2001 From: SuperBot Date: Thu, 25 Oct 2018 17:10:53 +0800 Subject: [PATCH 09/31] Add a much more simple JOIN example. --- docs/en/query_language/select.md | 36 ++++++++++++++++++++++++++++++++ 1 file changed, 36 insertions(+) diff --git a/docs/en/query_language/select.md b/docs/en/query_language/select.md index 53164e67d80..14603d1066d 100644 --- a/docs/en/query_language/select.md +++ b/docs/en/query_language/select.md @@ -346,6 +346,42 @@ The table name can be specified instead of a subquery. This is equivalent to the All columns that are not needed for the JOIN are deleted from the subquery. +Example: + +``` sql +SELECT + endpoint, + metric, + value, + idcName +FROM datapoints +ALL INNER JOIN +( + SELECT + ips AS endpoint, + idcName + FROM cmdb +) USING (endpoint) +LIMIT 10 +``` + +``` +┌─endpoint─────┬─metric────────────┬───────value─┬─idcName─┐ +│ 192.168.1.100 │ disk.total │ 50465866000 │ office │ +│ 192.168.1.100 │ disk.total │ 494927870 │ office │ +│ 192.168.1.100 │ disk.used.percent │ 48 │ office │ +│ 192.168.1.100 │ disk.used.percent │ 50 │ office │ +│ 192.168.1.100 │ disk.util │ 0 │ office │ +│ 192.168.1.100 │ disk.util │ 0 │ office │ +│ 192.168.1.100 │ disk.util │ 0 │ office │ +│ 192.168.1.100 │ disk.util │ 0 │ office │ +│ 192.168.1.100 │ load.15min │ 0.02 │ office │ +│ 192.168.1.100 │ load.1min │ 0.11 │ office │ +└──────────────┴───────────────────┴─────────────┴─────────┘ +``` + +Notice the column in `USING`(endpoint) must be in former `SELECT` column list and subquery's. + There are several types of JOINs: `INNER` or `LEFT` type:If INNER is specified, the result will contain only those rows that have a matching row in the right table. From e8f9e3c2a2ae78af3218eb5fde6e9f0d60e4c9f4 Mon Sep 17 00:00:00 2001 From: chertus Date: Thu, 25 Oct 2018 12:40:30 +0300 Subject: [PATCH 10/31] better fix for distributed DDL [CLICKHOUSE-4036] [CLICKHOUSE-4015] --- dbms/src/Interpreters/DDLWorker.cpp | 74 ++++++++++++------- dbms/src/Interpreters/DDLWorker.h | 2 +- .../Interpreters/InterpreterAlterQuery.cpp | 2 +- .../Interpreters/InterpreterCreateQuery.cpp | 2 +- .../Interpreters/InterpreterRenameQuery.cpp | 2 +- 5 files changed, 51 insertions(+), 31 deletions(-) diff --git a/dbms/src/Interpreters/DDLWorker.cpp b/dbms/src/Interpreters/DDLWorker.cpp index 3ee01c1571a..02e48a838bd 100644 --- a/dbms/src/Interpreters/DDLWorker.cpp +++ b/dbms/src/Interpreters/DDLWorker.cpp @@ -40,6 +40,7 @@ namespace DB namespace ErrorCodes { + extern const int LOGICAL_ERROR; extern const int UNKNOWN_ELEMENT_IN_CONFIG; extern const int INVALID_CONFIG_PARAMETER; extern const int UNKNOWN_FORMAT_VERSION; @@ -1136,7 +1137,7 @@ private: }; -BlockIO executeDDLQueryOnCluster(const ASTPtr & query_ptr_, const Context & context, const NameSet & query_databases) +BlockIO executeDDLQueryOnCluster(const ASTPtr & query_ptr_, const Context & context, NameSet && query_databases) { /// Remove FORMAT and INTO OUTFILE if exists ASTPtr query_ptr = query_ptr_->clone(); @@ -1160,41 +1161,60 @@ BlockIO executeDDLQueryOnCluster(const ASTPtr & query_ptr_, const Context & cont } } + query->cluster = context.getMacros()->expand(query->cluster); + ClusterPtr cluster = context.getCluster(query->cluster); + DDLWorker & ddl_worker = context.getDDLWorker(); + + /// Check database access rights, assume that all servers have the same users config + NameSet databases_to_access; const String & current_database = context.getCurrentDatabase(); - if (!current_database.empty()) + + Cluster::AddressesWithFailover shards = cluster->getShardsAddresses(); + + std::vector hosts; + bool use_shard_default_db = false; + bool use_local_default_db = false; + for (const auto & shard : shards) + { + for (const auto & addr : shard) + { + hosts.emplace_back(addr); + + /// Expand empty database name to shards' default (o current) database name + for (const String & database : query_databases) + { + if (database.empty()) + { + bool has_shard_default_db = !addr.default_database.empty(); + use_shard_default_db |= has_shard_default_db; + use_local_default_db |= !has_shard_default_db; + databases_to_access.emplace(has_shard_default_db ? addr.default_database : current_database ); + } + else + databases_to_access.emplace(database); + } + } + } + + if (use_shard_default_db && use_local_default_db) + throw Exception("Mixed local default DB and shard default DB in DDL query", ErrorCodes::NOT_IMPLEMENTED); + + if (databases_to_access.empty()) + throw Exception("No databases to access in distributed DDL query", ErrorCodes::LOGICAL_ERROR); + + for (const String & database : databases_to_access) + context.checkDatabaseAccessRights(database); + + if (use_local_default_db) { AddDefaultDatabaseVisitor visitor(current_database); visitor.visit(query_ptr); } - query->cluster = context.getMacros()->expand(query->cluster); - ClusterPtr cluster = context.getCluster(query->cluster); - DDLWorker & ddl_worker = context.getDDLWorker(); - DDLLogEntry entry; + entry.hosts = std::move(hosts); entry.query = queryToString(query_ptr); entry.initiator = ddl_worker.getCommonHostID(); - - /// Check database access rights, assume that all servers have the same users config - NameSet databases_to_check_access_rights; - - Cluster::AddressesWithFailover shards = cluster->getShardsAddresses(); - - for (const auto & shard : shards) - { - for (const auto & addr : shard) - { - entry.hosts.emplace_back(addr); - - /// Expand empty database name to shards' default database name - for (const String & database : query_databases) - databases_to_check_access_rights.emplace(database.empty() ? addr.default_database : database); - } - } - - for (const String & database : databases_to_check_access_rights) - context.checkDatabaseAccessRights(database.empty() ? current_database : database); - String node_path = ddl_worker.enqueueQuery(entry); BlockIO io; diff --git a/dbms/src/Interpreters/DDLWorker.h b/dbms/src/Interpreters/DDLWorker.h index 2fba83b7356..d3872b8ac95 100644 --- a/dbms/src/Interpreters/DDLWorker.h +++ b/dbms/src/Interpreters/DDLWorker.h @@ -20,7 +20,7 @@ struct DDLTask; /// Pushes distributed DDL query to the queue -BlockIO executeDDLQueryOnCluster(const ASTPtr & query_ptr, const Context & context, const NameSet & query_databases); +BlockIO executeDDLQueryOnCluster(const ASTPtr & query_ptr, const Context & context, NameSet && query_databases); class DDLWorker diff --git a/dbms/src/Interpreters/InterpreterAlterQuery.cpp b/dbms/src/Interpreters/InterpreterAlterQuery.cpp index 1d20f163925..31eedff6d11 100644 --- a/dbms/src/Interpreters/InterpreterAlterQuery.cpp +++ b/dbms/src/Interpreters/InterpreterAlterQuery.cpp @@ -27,7 +27,7 @@ BlockIO InterpreterAlterQuery::execute() auto & alter = typeid_cast(*query_ptr); if (!alter.cluster.empty()) - return executeDDLQueryOnCluster(query_ptr, context, {alter.table}); + return executeDDLQueryOnCluster(query_ptr, context, {alter.database}); const String & table_name = alter.table; String database_name = alter.database.empty() ? context.getCurrentDatabase() : alter.database; diff --git a/dbms/src/Interpreters/InterpreterCreateQuery.cpp b/dbms/src/Interpreters/InterpreterCreateQuery.cpp index c1c4c6f0706..dc7225fbbaa 100644 --- a/dbms/src/Interpreters/InterpreterCreateQuery.cpp +++ b/dbms/src/Interpreters/InterpreterCreateQuery.cpp @@ -487,7 +487,7 @@ BlockIO InterpreterCreateQuery::createTable(ASTCreateQuery & create) if (!create.to_table.empty()) databases.emplace(create.to_database); - return executeDDLQueryOnCluster(query_ptr, context, databases); + return executeDDLQueryOnCluster(query_ptr, context, std::move(databases)); } String path = context.getPath(); diff --git a/dbms/src/Interpreters/InterpreterRenameQuery.cpp b/dbms/src/Interpreters/InterpreterRenameQuery.cpp index 52bacb970c0..abeb7ff5c1f 100644 --- a/dbms/src/Interpreters/InterpreterRenameQuery.cpp +++ b/dbms/src/Interpreters/InterpreterRenameQuery.cpp @@ -47,7 +47,7 @@ BlockIO InterpreterRenameQuery::execute() databases.emplace(elem.to.database); } - return executeDDLQueryOnCluster(query_ptr, context, databases); + return executeDDLQueryOnCluster(query_ptr, context, std::move(databases)); } String path = context.getPath(); From 4fa0e71e536ac930faa15f70adb5cb2b615fdcd7 Mon Sep 17 00:00:00 2001 From: SaltTan Date: Wed, 24 Oct 2018 12:49:12 -0300 Subject: [PATCH 11/31] Update CHANGELOG_RU.md ClickHouse release 18.14.10 --- CHANGELOG_RU.md | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/CHANGELOG_RU.md b/CHANGELOG_RU.md index f3c50637454..99c7b489967 100644 --- a/CHANGELOG_RU.md +++ b/CHANGELOG_RU.md @@ -1,3 +1,8 @@ +## ClickHouse release 18.14.10, 2018-10-23 + +* Настройка `compile_expressions` (JIT компиляция выражений) выключена по-умолчанию. [#3410](https://github.com/yandex/ClickHouse/pull/3410) +* Настройка `enable_optimize_predicate_expression` выключена по-умолчанию. + ## ClickHouse release 18.14.9, 2018-10-16 ### Новые возможности: From db2d65939cbce28c2f663f9aa7e52a8f9a79945d Mon Sep 17 00:00:00 2001 From: Alex Zatelepin Date: Thu, 25 Oct 2018 14:26:07 +0300 Subject: [PATCH 12/31] Update CHANGELOG_RU.md --- CHANGELOG_RU.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/CHANGELOG_RU.md b/CHANGELOG_RU.md index 99c7b489967..30fb526f3a6 100644 --- a/CHANGELOG_RU.md +++ b/CHANGELOG_RU.md @@ -1,7 +1,7 @@ ## ClickHouse release 18.14.10, 2018-10-23 -* Настройка `compile_expressions` (JIT компиляция выражений) выключена по-умолчанию. [#3410](https://github.com/yandex/ClickHouse/pull/3410) -* Настройка `enable_optimize_predicate_expression` выключена по-умолчанию. +* Настройка `compile_expressions` (JIT компиляция выражений) выключена по умолчанию. [#3410](https://github.com/yandex/ClickHouse/pull/3410) +* Настройка `enable_optimize_predicate_expression` выключена по умолчанию. ## ClickHouse release 18.14.9, 2018-10-16 From 19a69004f0c872a3c62565f21d6d8bfc98fc3bc5 Mon Sep 17 00:00:00 2001 From: chertus Date: Thu, 25 Oct 2018 16:53:25 +0300 Subject: [PATCH 13/31] fix crash with non const scale in toDecimal() --- dbms/src/Functions/FunctionsConversion.h | 3 +++ dbms/tests/queries/0_stateless/00700_decimal_casts.sql | 4 ++++ 2 files changed, 7 insertions(+) diff --git a/dbms/src/Functions/FunctionsConversion.h b/dbms/src/Functions/FunctionsConversion.h index 9335e5eee57..f9802cd5178 100644 --- a/dbms/src/Functions/FunctionsConversion.h +++ b/dbms/src/Functions/FunctionsConversion.h @@ -784,6 +784,9 @@ public: } else if constexpr (to_decimal) { + if (!arguments[1].column) + throw Exception("Second argument for function " + getName() + " must be constant", ErrorCodes::ILLEGAL_COLUMN); + UInt64 scale = extractToDecimalScale(arguments[1]); if constexpr (std::is_same_v) diff --git a/dbms/tests/queries/0_stateless/00700_decimal_casts.sql b/dbms/tests/queries/0_stateless/00700_decimal_casts.sql index 8ebc98c129e..14593b50b5e 100644 --- a/dbms/tests/queries/0_stateless/00700_decimal_casts.sql +++ b/dbms/tests/queries/0_stateless/00700_decimal_casts.sql @@ -239,4 +239,8 @@ SELECT toUInt64('2147483649') AS x, toDecimal32(x, 0); -- { serverError 407 } SELECT toUInt64('9223372036854775807') AS x, toDecimal64(x, 0); SELECT toUInt64('9223372036854775809') AS x, toDecimal64(x, 0); -- { serverError 407 } +SELECT toDecimal32(0, rowNumberInBlock()); -- { serverError 44 } +SELECT toDecimal64(0, rowNumberInBlock()); -- { serverError 44 } +SELECT toDecimal128(0, rowNumberInBlock()); -- { serverError 44 } + DROP TABLE IF EXISTS test.decimal; From c3159af89884a2a4cee2d12df93dfd22bbf1578c Mon Sep 17 00:00:00 2001 From: alesapin Date: Thu, 25 Oct 2018 17:54:00 +0300 Subject: [PATCH 14/31] Enlarge constant default column if it's empty --- dbms/src/Functions/IFunction.cpp | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/dbms/src/Functions/IFunction.cpp b/dbms/src/Functions/IFunction.cpp index 8d6142bc287..2157f6d7251 100644 --- a/dbms/src/Functions/IFunction.cpp +++ b/dbms/src/Functions/IFunction.cpp @@ -288,7 +288,10 @@ bool PreparedFunctionImpl::defaultImplementationForConstantArguments(Block & blo const ColumnWithTypeAndName & column = block.getByPosition(args[arg_num]); if (arguments_to_remain_constants.end() != std::find(arguments_to_remain_constants.begin(), arguments_to_remain_constants.end(), arg_num)) - temporary_block.insert(column); + if (column.column->empty()) + temporary_block.insert({column.column->cloneResized(1), column.type, column.name}); + else + temporary_block.insert(column); else { have_converted_columns = true; From 713c44c3db58698b7bdbc6c20be589434db37b32 Mon Sep 17 00:00:00 2001 From: proller Date: Thu, 25 Oct 2018 18:23:41 +0300 Subject: [PATCH 15/31] Build fixes (#3466) --- cmake/find_odbc.cmake | 2 +- cmake/find_poco.cmake | 4 ++-- dbms/src/Client/Connection.cpp | 8 +++++++- dbms/tests/clickhouse-test | 2 +- ...long_sql_fuzzy.reference => 00746_sql_fuzzy.reference} | 0 dbms/tests/queries/0_stateless/00746_sql_fuzzy.sh | 2 +- dbms/tests/queries/bugs/fuzzy.sql | 1 + 7 files changed, 13 insertions(+), 6 deletions(-) rename dbms/tests/queries/0_stateless/{00675_long_sql_fuzzy.reference => 00746_sql_fuzzy.reference} (100%) diff --git a/cmake/find_odbc.cmake b/cmake/find_odbc.cmake index 95acf40b2b4..d89e3b532d8 100644 --- a/cmake/find_odbc.cmake +++ b/cmake/find_odbc.cmake @@ -86,4 +86,4 @@ if (ENABLE_ODBC) endif () endif () -message (STATUS "Using odbc: ${ODBC_INCLUDE_DIRECTORIES} : ${ODBC_LIBRARIES}") +message (STATUS "Using odbc=${ODBC_FOUND}: ${ODBC_INCLUDE_DIRECTORIES} : ${ODBC_LIBRARIES}") diff --git a/cmake/find_poco.cmake b/cmake/find_poco.cmake index 79d5930bc9c..d8468e5306d 100644 --- a/cmake/find_poco.cmake +++ b/cmake/find_poco.cmake @@ -116,10 +116,10 @@ endif () if (Poco_MongoDB_LIBRARY) set (USE_POCO_MONGODB 1) endif () -if (Poco_DataODBC_LIBRARY) +if (Poco_DataODBC_LIBRARY AND ODBC_FOUND) set (USE_POCO_DATAODBC 1) endif () -if (Poco_SQLODBC_LIBRARY) +if (Poco_SQLODBC_LIBRARY AND ODBC_FOUND) set (USE_POCO_SQLODBC 1) endif () diff --git a/dbms/src/Client/Connection.cpp b/dbms/src/Client/Connection.cpp index 688ba11612d..ce6246fba3a 100644 --- a/dbms/src/Client/Connection.cpp +++ b/dbms/src/Client/Connection.cpp @@ -80,7 +80,13 @@ void Connection::connect() if (timeouts.tcp_keep_alive_timeout.totalSeconds()) { socket->setKeepAlive(true); - socket->setOption(IPPROTO_TCP, TCP_KEEPIDLE, timeouts.tcp_keep_alive_timeout); + socket->setOption(IPPROTO_TCP, +#if defined(TCP_KEEPALIVE) + TCP_KEEPALIVE +#else + TCP_KEEPIDLE // __APPLE__ +#endif + , timeouts.tcp_keep_alive_timeout); } in = std::make_shared(*socket); diff --git a/dbms/tests/clickhouse-test b/dbms/tests/clickhouse-test index bf71c18c5c7..b9f4c82af62 100755 --- a/dbms/tests/clickhouse-test +++ b/dbms/tests/clickhouse-test @@ -242,7 +242,7 @@ def main(args): stderr_element = et.Element("system-err") stderr_element.text = et.CDATA(stderr) report_testcase.append(stderr_element) - print(stderr) + print(stderr.encode('utf-8')) if args.stop and ('Connection refused' in stderr or 'Attempt to read after eof' in stderr) and not 'Received exception from server' in stderr: SERVER_DIED = True diff --git a/dbms/tests/queries/0_stateless/00675_long_sql_fuzzy.reference b/dbms/tests/queries/0_stateless/00746_sql_fuzzy.reference similarity index 100% rename from dbms/tests/queries/0_stateless/00675_long_sql_fuzzy.reference rename to dbms/tests/queries/0_stateless/00746_sql_fuzzy.reference diff --git a/dbms/tests/queries/0_stateless/00746_sql_fuzzy.sh b/dbms/tests/queries/0_stateless/00746_sql_fuzzy.sh index 6bce694ae36..913963c7318 100755 --- a/dbms/tests/queries/0_stateless/00746_sql_fuzzy.sh +++ b/dbms/tests/queries/0_stateless/00746_sql_fuzzy.sh @@ -12,7 +12,7 @@ $CLICKHOUSE_CLIENT -q "select name from system.table_functions format TSV;" > $S # This is short run for ordinary tests. # if you want long run use: env SQL_FUZZY_RUNS=100000 clickhouse-test sql_fuzzy -for SQL_FUZZY_RUN in $(seq ${SQL_FUZZY_RUNS:=100}); do +for SQL_FUZZY_RUN in $(seq ${SQL_FUZZY_RUNS:=10}); do env SQL_FUZZY_RUN=$SQL_FUZZY_RUN $CURDIR/00746_sql_fuzzy.pl | $CLICKHOUSE_CLIENT -n --ignore-error >/dev/null 2>&1 if [[ `$CLICKHOUSE_CLIENT -q "SELECT 'Still alive'"` != 'Still alive' ]]; then break diff --git a/dbms/tests/queries/bugs/fuzzy.sql b/dbms/tests/queries/bugs/fuzzy.sql index 671d496e688..52008e27ca7 100644 --- a/dbms/tests/queries/bugs/fuzzy.sql +++ b/dbms/tests/queries/bugs/fuzzy.sql @@ -6,3 +6,4 @@ SELECT extractURLParameter('?_', '\0'); SELECT extractURLParameter('ZiqSZeh?', '\0') SELECT globalNotIn(['"wh'], [NULL]); SELECT globalIn([''], [NULL]) +SELECT ( SELECT toDecimal128([], rowNumberInBlock()) ) , lcm('', [[(CAST(('>A') AS String))]]); From 3a9311d3ad5f3be37c5e5fb115d74fa27803ffd7 Mon Sep 17 00:00:00 2001 From: alesapin Date: Thu, 25 Oct 2018 19:02:20 +0300 Subject: [PATCH 16/31] Now default block can have size more than one --- dbms/src/Functions/IFunction.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/src/Functions/IFunction.cpp b/dbms/src/Functions/IFunction.cpp index 2157f6d7251..d2e39c836c9 100644 --- a/dbms/src/Functions/IFunction.cpp +++ b/dbms/src/Functions/IFunction.cpp @@ -314,7 +314,7 @@ bool PreparedFunctionImpl::defaultImplementationForConstantArguments(Block & blo executeWithoutLowCardinalityColumns(temporary_block, temporary_argument_numbers, arguments_size, temporary_block.rows()); - block.getByPosition(result).column = ColumnConst::create(temporary_block.getByPosition(arguments_size).column, input_rows_count); + block.getByPosition(result).column = ColumnConst::create(temporary_block.getByPosition(arguments_size).column->cloneResized(1), input_rows_count); return true; } From a4dd489165dba5572ec868cd02a62bc7af3343dd Mon Sep 17 00:00:00 2001 From: alesapin Date: Thu, 25 Oct 2018 19:52:28 +0300 Subject: [PATCH 17/31] Do not build compiled functions --- dbms/src/Functions/IFunction.cpp | 10 +++++++++- dbms/src/Interpreters/ExpressionActions.cpp | 3 ++- 2 files changed, 11 insertions(+), 2 deletions(-) diff --git a/dbms/src/Functions/IFunction.cpp b/dbms/src/Functions/IFunction.cpp index d2e39c836c9..e0d1081246a 100644 --- a/dbms/src/Functions/IFunction.cpp +++ b/dbms/src/Functions/IFunction.cpp @@ -314,7 +314,15 @@ bool PreparedFunctionImpl::defaultImplementationForConstantArguments(Block & blo executeWithoutLowCardinalityColumns(temporary_block, temporary_argument_numbers, arguments_size, temporary_block.rows()); - block.getByPosition(result).column = ColumnConst::create(temporary_block.getByPosition(arguments_size).column->cloneResized(1), input_rows_count); + ColumnPtr result_column; + /// extremely rare case, when we have function with completely const arguments + /// but some of them produced by non isDeterministic function + if (temporary_block.getByPosition(arguments_size).column->size() > 1) + result_column = temporary_block.getByPosition(arguments_size).column->cloneResized(1); + else + result_column = temporary_block.getByPosition(arguments_size).column; + + block.getByPosition(result).column = ColumnConst::create(result_column, input_rows_count); return true; } diff --git a/dbms/src/Interpreters/ExpressionActions.cpp b/dbms/src/Interpreters/ExpressionActions.cpp index 4b5ccfde219..1daf0bc3729 100644 --- a/dbms/src/Interpreters/ExpressionActions.cpp +++ b/dbms/src/Interpreters/ExpressionActions.cpp @@ -677,7 +677,8 @@ void ExpressionActions::addImpl(ExpressionAction action, Names & new_names) new_names.push_back(action.result_name); new_names.insert(new_names.end(), action.array_joined_columns.begin(), action.array_joined_columns.end()); - if (action.type == ExpressionAction::APPLY_FUNCTION) + /// Compiled functions are custom functions and them don't need building + if (action.type == ExpressionAction::APPLY_FUNCTION && !action.is_function_compiled) { if (sample_block.has(action.result_name)) throw Exception("Column '" + action.result_name + "' already exists", ErrorCodes::DUPLICATE_COLUMN); From 20b20d04f947f229f51a1b0565be040ae4d500f8 Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 26 Oct 2018 12:22:33 +0300 Subject: [PATCH 18/31] Include from env --- dbms/src/Common/Config/ConfigProcessor.cpp | 2 ++ dbms/tests/integration/test_config_substitutions/test.py | 2 ++ 2 files changed, 4 insertions(+) diff --git a/dbms/src/Common/Config/ConfigProcessor.cpp b/dbms/src/Common/Config/ConfigProcessor.cpp index 8cefc16500b..eebf9b75a35 100644 --- a/dbms/src/Common/Config/ConfigProcessor.cpp +++ b/dbms/src/Common/Config/ConfigProcessor.cpp @@ -462,6 +462,8 @@ XMLDocumentPtr ConfigProcessor::processConfig( std::string include_from_path; if (node) { + /// if we include_from env or zk. + doIncludesRecursive(config, nullptr, node, zk_node_cache, contributing_zk_paths); include_from_path = node->innerText(); } else diff --git a/dbms/tests/integration/test_config_substitutions/test.py b/dbms/tests/integration/test_config_substitutions/test.py index 5adeca2b1e9..8472f85a285 100644 --- a/dbms/tests/integration/test_config_substitutions/test.py +++ b/dbms/tests/integration/test_config_substitutions/test.py @@ -9,6 +9,7 @@ node2 = cluster.add_instance('node2', user_configs=['configs/config_env.xml'], e node3 = cluster.add_instance('node3', user_configs=['configs/config_zk.xml'], with_zookeeper=True) node4 = cluster.add_instance('node4', user_configs=['configs/config_incl.xml'], main_configs=['configs/max_query_size.xml']) # include value 77777 node5 = cluster.add_instance('node5', user_configs=['configs/config_allow_databases.xml']) +node6 = cluster.add_instance('node6', user_configs=['configs/config_include_from_env.xml'], env_variables={"INCLUDE_FROM_ENV": "/etc/clickhouse-server/config.d/max_query_size.xml"}, main_configs=['configs/max_query_size.xml']) @pytest.fixture(scope="module") def start_cluster(): @@ -27,6 +28,7 @@ def test_config(start_cluster): assert node2.query("select value from system.settings where name = 'max_query_size'") == "55555\n" assert node3.query("select value from system.settings where name = 'max_query_size'") == "77777\n" assert node4.query("select value from system.settings where name = 'max_query_size'") == "99999\n" + assert node6.query("select value from system.settings where name = 'max_query_size'") == "99999\n" def test_allow_databases(start_cluster): node5.query("CREATE DATABASE db1") From 32f6745ae839a6c1d1a0e6511db8d3e4c8dd3f95 Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 26 Oct 2018 12:23:29 +0300 Subject: [PATCH 19/31] Config file --- .../configs/config_include_from_env.xml | 15 +++++++++++++++ 1 file changed, 15 insertions(+) create mode 100644 dbms/tests/integration/test_config_substitutions/configs/config_include_from_env.xml diff --git a/dbms/tests/integration/test_config_substitutions/configs/config_include_from_env.xml b/dbms/tests/integration/test_config_substitutions/configs/config_include_from_env.xml new file mode 100644 index 00000000000..71e11235749 --- /dev/null +++ b/dbms/tests/integration/test_config_substitutions/configs/config_include_from_env.xml @@ -0,0 +1,15 @@ + + + + + + + + + + + default + default + + + From 95c618b163410e83c10ea63d5c50ae92c2e900cd Mon Sep 17 00:00:00 2001 From: chertus Date: Fri, 26 Oct 2018 18:13:02 +0300 Subject: [PATCH 20/31] ExpressionAnalyzer: remove duplicated code --- dbms/src/Interpreters/ExpressionAnalyzer.cpp | 51 ++----------------- .../PredicateExpressionsOptimizer.cpp | 28 ++-------- .../PredicateExpressionsOptimizer.h | 2 - dbms/src/Interpreters/evaluateQualified.cpp | 32 ++++++++++++ dbms/src/Interpreters/evaluateQualified.h | 4 ++ 5 files changed, 44 insertions(+), 73 deletions(-) diff --git a/dbms/src/Interpreters/ExpressionAnalyzer.cpp b/dbms/src/Interpreters/ExpressionAnalyzer.cpp index f90868d7a36..b889336b620 100644 --- a/dbms/src/Interpreters/ExpressionAnalyzer.cpp +++ b/dbms/src/Interpreters/ExpressionAnalyzer.cpp @@ -275,55 +275,14 @@ bool ExpressionAnalyzer::isRemoteStorage() const } -static std::vector getTableExpressions(const ASTPtr & query) -{ - ASTSelectQuery * select_query = typeid_cast(query.get()); - - std::vector table_expressions; - - if (select_query && select_query->tables) - { - for (const auto & element : select_query->tables->children) - { - ASTTablesInSelectQueryElement & select_element = static_cast(*element); - - if (select_element.table_expression) - table_expressions.emplace_back(static_cast(*select_element.table_expression)); - } - } - - return table_expressions; -} - void ExpressionAnalyzer::translateQualifiedNames() { if (!select_query || !select_query->tables || select_query->tables->children.empty()) return; - std::vector tables; - std::vector tables_expression = getTableExpressions(query); + std::vector tables = getDatabaseAndTableWithAliases(select_query, context.getCurrentDatabase()); LogAST log; - - for (const auto & table_expression : tables_expression) - { - auto table = getTableNameWithAliasFromTableExpression(table_expression, context.getCurrentDatabase()); - - { /// debug print - size_t depth = 0; - DumpASTNode dump(table_expression, log.stream(), depth, "getTableNames"); - if (table_expression.database_and_table_name) - DumpASTNode(*table_expression.database_and_table_name, log.stream(), depth); - if (table_expression.table_function) - DumpASTNode(*table_expression.table_function, log.stream(), depth); - if (table_expression.subquery) - DumpASTNode(*table_expression.subquery, log.stream(), depth); - dump.print("getTableNameWithAlias", table.database + '.' + table.table + ' ' + table.alias); - } - - tables.emplace_back(table); - } - TranslateQualifiedNamesVisitor visitor(source_columns, tables, log.stream()); visitor.visit(query); } @@ -602,13 +561,13 @@ void ExpressionAnalyzer::normalizeTree() TableNamesAndColumnNames table_names_and_column_names; if (select_query && select_query->tables && !select_query->tables->children.empty()) { - std::vector tables_expression = getTableExpressions(query); + std::vector tables_expression = getSelectTablesExpression(select_query); bool first = true; - for (const auto & table_expression : tables_expression) + for (const auto * table_expression : tables_expression) { - const auto table_name = getTableNameWithAliasFromTableExpression(table_expression, context.getCurrentDatabase()); - NamesAndTypesList names_and_types = getNamesAndTypeListFromTableExpression(table_expression, context); + const auto table_name = getTableNameWithAliasFromTableExpression(*table_expression, context.getCurrentDatabase()); + NamesAndTypesList names_and_types = getNamesAndTypeListFromTableExpression(*table_expression, context); if (!first) { diff --git a/dbms/src/Interpreters/PredicateExpressionsOptimizer.cpp b/dbms/src/Interpreters/PredicateExpressionsOptimizer.cpp index 1e4bb996f90..222b9dd5131 100644 --- a/dbms/src/Interpreters/PredicateExpressionsOptimizer.cpp +++ b/dbms/src/Interpreters/PredicateExpressionsOptimizer.cpp @@ -44,11 +44,8 @@ bool PredicateExpressionsOptimizer::optimizeImpl( /// split predicate with `and` PredicateExpressions outer_predicate_expressions = splitConjunctionPredicate(outer_expression); - std::vector tables_expression = getSelectTablesExpression(ast_select); - std::vector database_and_table_with_aliases; - for (const auto & table_expression : tables_expression) - database_and_table_with_aliases.emplace_back( - getTableNameWithAliasFromTableExpression(*table_expression, context.getCurrentDatabase())); + std::vector database_and_table_with_aliases = + getDatabaseAndTableWithAliases(ast_select, context.getCurrentDatabase()); bool is_rewrite_subquery = false; for (const auto & outer_predicate : outer_predicate_expressions) @@ -336,7 +333,7 @@ ASTs PredicateExpressionsOptimizer::evaluateAsterisk(ASTSelectQuery * select_que if (!select_query->tables || select_query->tables->children.empty()) return {}; - std::vector tables_expression = getSelectTablesExpression(select_query); + std::vector tables_expression = getSelectTablesExpression(select_query); if (const auto qualified_asterisk = typeid_cast(asterisk.get())) { @@ -406,25 +403,6 @@ ASTs PredicateExpressionsOptimizer::evaluateAsterisk(ASTSelectQuery * select_que return projection_columns; } -std::vector PredicateExpressionsOptimizer::getSelectTablesExpression(ASTSelectQuery * select_query) -{ - if (!select_query->tables) - return {}; - - std::vector tables_expression; - const ASTTablesInSelectQuery & tables_in_select_query = static_cast(*select_query->tables); - - for (const auto & child : tables_in_select_query.children) - { - ASTTablesInSelectQueryElement * tables_element = static_cast(child.get()); - - if (tables_element->table_expression) - tables_expression.emplace_back(static_cast(tables_element->table_expression.get())); - } - - return tables_expression; -} - void PredicateExpressionsOptimizer::cleanExpressionAlias(ASTPtr & expression) { const auto my_alias = expression->tryGetAlias(); diff --git a/dbms/src/Interpreters/PredicateExpressionsOptimizer.h b/dbms/src/Interpreters/PredicateExpressionsOptimizer.h index c1a02a7df18..d75d9135f8a 100644 --- a/dbms/src/Interpreters/PredicateExpressionsOptimizer.h +++ b/dbms/src/Interpreters/PredicateExpressionsOptimizer.h @@ -105,8 +105,6 @@ private: ASTs getSelectQueryProjectionColumns(ASTPtr & ast); - std::vector getSelectTablesExpression(ASTSelectQuery * select_query); - ASTs evaluateAsterisk(ASTSelectQuery * select_query, const ASTPtr & asterisk); void cleanExpressionAlias(ASTPtr & expression); diff --git a/dbms/src/Interpreters/evaluateQualified.cpp b/dbms/src/Interpreters/evaluateQualified.cpp index 205885011d1..7f3002edad1 100644 --- a/dbms/src/Interpreters/evaluateQualified.cpp +++ b/dbms/src/Interpreters/evaluateQualified.cpp @@ -5,6 +5,7 @@ #include #include #include +#include namespace DB { @@ -164,4 +165,35 @@ void DatabaseAndTableWithAlias::makeQualifiedName(const ASTPtr & ast) const } } +std::vector getSelectTablesExpression(const ASTSelectQuery * select_query) +{ + if (!select_query->tables) + return {}; + + std::vector tables_expression; + + for (const auto & child : select_query->tables->children) + { + ASTTablesInSelectQueryElement * tables_element = static_cast(child.get()); + + if (tables_element->table_expression) + tables_expression.emplace_back(static_cast(tables_element->table_expression.get())); + } + + return tables_expression; +} + +std::vector getDatabaseAndTableWithAliases(const ASTSelectQuery * select_query, const String & current_database) +{ + std::vector tables_expression = getSelectTablesExpression(select_query); + + std::vector database_and_table_with_aliases; + database_and_table_with_aliases.reserve(tables_expression.size()); + + for (const auto & table_expression : tables_expression) + database_and_table_with_aliases.emplace_back(getTableNameWithAliasFromTableExpression(*table_expression, current_database)); + + return database_and_table_with_aliases; +} + } diff --git a/dbms/src/Interpreters/evaluateQualified.h b/dbms/src/Interpreters/evaluateQualified.h index 94833190d81..39dcf77fbfd 100644 --- a/dbms/src/Interpreters/evaluateQualified.h +++ b/dbms/src/Interpreters/evaluateQualified.h @@ -9,6 +9,7 @@ namespace DB class IAST; using ASTPtr = std::shared_ptr; +class ASTSelectQuery; class ASTIdentifier; struct ASTTableExpression; @@ -36,4 +37,7 @@ size_t getNumComponentsToStripInOrderToTranslateQualifiedName(const ASTIdentifie std::pair getDatabaseAndTableNameFromIdentifier(const ASTIdentifier & identifier); +std::vector getSelectTablesExpression(const ASTSelectQuery * select_query); +std::vector getDatabaseAndTableWithAliases(const ASTSelectQuery * select_query, const String & current_database); + } From e4e38f71e12dab5b7404d917747d18a3eb4a4a42 Mon Sep 17 00:00:00 2001 From: Alexey Zatelepin Date: Fri, 26 Oct 2018 18:26:07 +0300 Subject: [PATCH 21/31] allow empty header in RemoteBlockOutputStream #3411 The bug with inserts to Distributed tables was introduced in https://github.com/yandex/ClickHouse/pull/3171 It added a workaround specifically for inserting in the Native format without specifying the list of columns. Native (as opposed to other formats) historically supports this. To signal that the input block structure shouldn't conform to any fixed header in this case, the remote server started sending empty header block. This commit adds support for empty headers to RemoteBlockOutputStream. --- dbms/src/DataStreams/RemoteBlockOutputStream.cpp | 6 ++---- 1 file changed, 2 insertions(+), 4 deletions(-) diff --git a/dbms/src/DataStreams/RemoteBlockOutputStream.cpp b/dbms/src/DataStreams/RemoteBlockOutputStream.cpp index f1e68a6a0c1..ff5fc75f1c4 100644 --- a/dbms/src/DataStreams/RemoteBlockOutputStream.cpp +++ b/dbms/src/DataStreams/RemoteBlockOutputStream.cpp @@ -33,9 +33,6 @@ RemoteBlockOutputStream::RemoteBlockOutputStream(Connection & connection_, const if (Protocol::Server::Data == packet.type) { header = packet.block; - - if (!header) - throw Exception("Logical error: empty block received as table structure", ErrorCodes::LOGICAL_ERROR); break; } else if (Protocol::Server::Exception == packet.type) @@ -58,7 +55,8 @@ RemoteBlockOutputStream::RemoteBlockOutputStream(Connection & connection_, const void RemoteBlockOutputStream::write(const Block & block) { - assertBlocksHaveEqualStructure(block, header, "RemoteBlockOutputStream"); + if (header) + assertBlocksHaveEqualStructure(block, header, "RemoteBlockOutputStream"); try { From f22779ad15c6be57a7d2ff2b9423b796f091725c Mon Sep 17 00:00:00 2001 From: hotid Date: Fri, 26 Oct 2018 20:14:43 +0300 Subject: [PATCH 22/31] stop generating empty WHERE () section in mysql queries --- dbms/src/Storages/transformQueryForExternalDatabase.cpp | 7 ++++++- 1 file changed, 6 insertions(+), 1 deletion(-) diff --git a/dbms/src/Storages/transformQueryForExternalDatabase.cpp b/dbms/src/Storages/transformQueryForExternalDatabase.cpp index 2615e8d0d92..43d20d7a4c0 100644 --- a/dbms/src/Storages/transformQueryForExternalDatabase.cpp +++ b/dbms/src/Storages/transformQueryForExternalDatabase.cpp @@ -119,6 +119,7 @@ String transformQueryForExternalDatabase( { if (function->name == "and") { + bool compatibleFound = false; auto new_function_and = std::make_shared(); auto new_function_and_arguments = std::make_shared(); new_function_and->arguments = new_function_and_arguments; @@ -126,9 +127,13 @@ String transformQueryForExternalDatabase( for (const auto & elem : function->arguments->children) if (isCompatible(*elem)) + { new_function_and_arguments->children.push_back(elem); + compatibleFound = true; + } - select->where_expression = std::move(new_function_and); + if (compatibleFound) + select->where_expression = std::move(new_function_and); } } } From c35c979285109f566a00dcc33de555625264888f Mon Sep 17 00:00:00 2001 From: proller Date: Fri, 26 Oct 2018 20:43:50 +0300 Subject: [PATCH 23/31] CLICKHOUSE-4085 system.contributors (#3452) * CLICKHOUSE-4085 system.contributors * fi * Fix random --- .gitignore | 3 ++ dbms/CMakeLists.txt | 6 ---- dbms/src/Common/config_build.h | 4 --- dbms/src/Storages/System/CMakeLists.txt | 11 ++++++ .../System/StorageSystemBuildOptions.cpp | 6 ++-- ...torageSystemBuildOptions.generated.cpp.in} | 2 -- .../System/StorageSystemContributors.cpp | 36 +++++++++++++++++++ .../System/StorageSystemContributors.h | 34 ++++++++++++++++++ .../System/StorageSystemContributors.sh | 19 ++++++++++ .../Storages/System/attachSystemTables.cpp | 6 ++++ .../0_stateless/00747_contributors.reference | 1 + .../0_stateless/00747_contributors.sql | 2 ++ release | 2 ++ 13 files changed, 118 insertions(+), 14 deletions(-) delete mode 100644 dbms/src/Common/config_build.h rename dbms/src/{Common/config_build.cpp.in => Storages/System/StorageSystemBuildOptions.generated.cpp.in} (98%) create mode 100644 dbms/src/Storages/System/StorageSystemContributors.cpp create mode 100644 dbms/src/Storages/System/StorageSystemContributors.h create mode 100755 dbms/src/Storages/System/StorageSystemContributors.sh create mode 100644 dbms/tests/queries/0_stateless/00747_contributors.reference create mode 100644 dbms/tests/queries/0_stateless/00747_contributors.sql diff --git a/.gitignore b/.gitignore index 6c0865d1959..6e969a8459c 100644 --- a/.gitignore +++ b/.gitignore @@ -9,7 +9,10 @@ # auto generated files *.logrt +dbms/src/Storages/System/StorageSystemContributors.generated.cpp + /build +/build_* /docs/build /docs/edit /docs/tools/venv/ diff --git a/dbms/CMakeLists.txt b/dbms/CMakeLists.txt index 0cb5824e1fc..91171eea685 100644 --- a/dbms/CMakeLists.txt +++ b/dbms/CMakeLists.txt @@ -6,18 +6,12 @@ include(${CMAKE_CURRENT_SOURCE_DIR}/cmake/find_vectorclass.cmake) set (CONFIG_VERSION ${CMAKE_CURRENT_BINARY_DIR}/src/Common/config_version.h) set (CONFIG_COMMON ${CMAKE_CURRENT_BINARY_DIR}/src/Common/config.h) -set (CONFIG_BUILD ${CMAKE_CURRENT_BINARY_DIR}/src/Common/config_build.cpp) include (cmake/version.cmake) message (STATUS "Will build ${VERSION_FULL}") configure_file (${CMAKE_CURRENT_SOURCE_DIR}/src/Common/config.h.in ${CONFIG_COMMON}) configure_file (${CMAKE_CURRENT_SOURCE_DIR}/src/Common/config_version.h.in ${CONFIG_VERSION}) -get_property (BUILD_COMPILE_DEFINITIONS DIRECTORY ${ClickHouse_SOURCE_DIR} PROPERTY COMPILE_DEFINITIONS) -get_property (BUILD_INCLUDE_DIRECTORIES DIRECTORY ${ClickHouse_SOURCE_DIR} PROPERTY INCLUDE_DIRECTORIES) -string (TIMESTAMP BUILD_DATE "%Y-%m-%d" UTC) -configure_file (${CMAKE_CURRENT_SOURCE_DIR}/src/Common/config_build.cpp.in ${CONFIG_BUILD}) - if (NOT MSVC) set (CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} -Wextra") endif () diff --git a/dbms/src/Common/config_build.h b/dbms/src/Common/config_build.h deleted file mode 100644 index 93e9ba35a8d..00000000000 --- a/dbms/src/Common/config_build.h +++ /dev/null @@ -1,4 +0,0 @@ -#pragma once -#include - -extern const char * auto_config_build[]; diff --git a/dbms/src/Storages/System/CMakeLists.txt b/dbms/src/Storages/System/CMakeLists.txt index 907fbc2907a..ed12cf6f78c 100644 --- a/dbms/src/Storages/System/CMakeLists.txt +++ b/dbms/src/Storages/System/CMakeLists.txt @@ -1,5 +1,16 @@ +if (NOT EXISTS ${CMAKE_CURRENT_SOURCE_DIR}/StorageSystemContributors.generated.cpp) + execute_process(COMMAND ${CMAKE_CURRENT_SOURCE_DIR}/StorageSystemContributors.sh) +endif() + +set (CONFIG_BUILD ${CMAKE_CURRENT_BINARY_DIR}/StorageSystemBuildOptions.generated.cpp) +get_property (BUILD_COMPILE_DEFINITIONS DIRECTORY ${ClickHouse_SOURCE_DIR} PROPERTY COMPILE_DEFINITIONS) +get_property (BUILD_INCLUDE_DIRECTORIES DIRECTORY ${ClickHouse_SOURCE_DIR} PROPERTY INCLUDE_DIRECTORIES) +string (TIMESTAMP BUILD_DATE "%Y-%m-%d" UTC) +configure_file (StorageSystemBuildOptions.generated.cpp.in ${CONFIG_BUILD}) include(${ClickHouse_SOURCE_DIR}/cmake/dbms_glob_sources.cmake) add_headers_and_sources(storages_system .) +list (APPEND storages_system_sources ${CONFIG_BUILD}) add_library(clickhouse_storages_system ${LINK_MODE} ${storages_system_headers} ${storages_system_sources}) target_link_libraries(clickhouse_storages_system dbms) +target_include_directories(clickhouse_storages_system PRIVATE ${CMAKE_CURRENT_SOURCE_DIR}) diff --git a/dbms/src/Storages/System/StorageSystemBuildOptions.cpp b/dbms/src/Storages/System/StorageSystemBuildOptions.cpp index 2a8ffc947be..b85cc9cf9f7 100644 --- a/dbms/src/Storages/System/StorageSystemBuildOptions.cpp +++ b/dbms/src/Storages/System/StorageSystemBuildOptions.cpp @@ -1,7 +1,9 @@ -#include +#include "StorageSystemBuildOptions.h" + #include #include -#include + +extern const char * auto_config_build[]; namespace DB { diff --git a/dbms/src/Common/config_build.cpp.in b/dbms/src/Storages/System/StorageSystemBuildOptions.generated.cpp.in similarity index 98% rename from dbms/src/Common/config_build.cpp.in rename to dbms/src/Storages/System/StorageSystemBuildOptions.generated.cpp.in index 9e74ea43e00..57c418d7f98 100644 --- a/dbms/src/Common/config_build.cpp.in +++ b/dbms/src/Storages/System/StorageSystemBuildOptions.generated.cpp.in @@ -1,7 +1,5 @@ // .cpp autogenerated by cmake -#include - const char * auto_config_build[] { "VERSION_FULL", "@VERSION_FULL@", diff --git a/dbms/src/Storages/System/StorageSystemContributors.cpp b/dbms/src/Storages/System/StorageSystemContributors.cpp new file mode 100644 index 00000000000..99c720e0f97 --- /dev/null +++ b/dbms/src/Storages/System/StorageSystemContributors.cpp @@ -0,0 +1,36 @@ +#if __has_include("StorageSystemContributors.generated.cpp") + +#include "StorageSystemContributors.h" + +#include +#include +#include +#include + + +extern const char * auto_contributors[]; + +namespace DB +{ +NamesAndTypesList StorageSystemContributors::getNamesAndTypes() +{ + return { + {"name", std::make_shared()}, + }; +} + +void StorageSystemContributors::fillData(MutableColumns & res_columns, const Context &, const SelectQueryInfo &) const +{ + std::vector contributors; + for (auto it = auto_contributors; *it; ++it) + contributors.emplace_back(*it); + + pcg64 rng(randomSeed()); + std::shuffle(contributors.begin(), contributors.end(), rng); + + for (auto & it : contributors) + res_columns[0]->insert(String(it)); +} +} + +#endif diff --git a/dbms/src/Storages/System/StorageSystemContributors.h b/dbms/src/Storages/System/StorageSystemContributors.h new file mode 100644 index 00000000000..b62895d5788 --- /dev/null +++ b/dbms/src/Storages/System/StorageSystemContributors.h @@ -0,0 +1,34 @@ +#pragma once + +#if __has_include("StorageSystemContributors.generated.cpp") + +#include +#include + + +namespace DB +{ +class Context; + + +/** System table "contributors" with list of clickhouse contributors + */ +class StorageSystemContributors : public ext::shared_ptr_helper, + public IStorageSystemOneBlock +{ +protected: + void fillData(MutableColumns & res_columns, const Context & context, const SelectQueryInfo & query_info) const override; + + using IStorageSystemOneBlock::IStorageSystemOneBlock; + +public: + std::string getName() const override + { + return "SystemContributors"; + } + + static NamesAndTypesList getNamesAndTypes(); +}; +} + +#endif diff --git a/dbms/src/Storages/System/StorageSystemContributors.sh b/dbms/src/Storages/System/StorageSystemContributors.sh new file mode 100755 index 00000000000..44b4730a6f4 --- /dev/null +++ b/dbms/src/Storages/System/StorageSystemContributors.sh @@ -0,0 +1,19 @@ +#!/usr/bin/env bash + +CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) + +CONTRIBUTORS_FILE=${CONTRIBUTORS_FILE=$CUR_DIR/StorageSystemContributors.generated.cpp} + +git shortlog --summary | perl -lnE 's/^\s+\d+\s+(.+)/"$1",/; next unless $1; say $_' > $CONTRIBUTORS_FILE.tmp + +# If git history not available - dont make target file +if [ ! -s $CONTRIBUTORS_FILE.tmp ]; then + exit +fi + +echo "// autogenerated by $0" > $CONTRIBUTORS_FILE +echo "const char * auto_contributors[] {" >> $CONTRIBUTORS_FILE +cat $CONTRIBUTORS_FILE.tmp >> $CONTRIBUTORS_FILE +echo "nullptr };" >> $CONTRIBUTORS_FILE + +rm $CONTRIBUTORS_FILE.tmp diff --git a/dbms/src/Storages/System/attachSystemTables.cpp b/dbms/src/Storages/System/attachSystemTables.cpp index 10448948942..facaa6c4fd3 100644 --- a/dbms/src/Storages/System/attachSystemTables.cpp +++ b/dbms/src/Storages/System/attachSystemTables.cpp @@ -32,6 +32,9 @@ #include #include #include +#if __has_include("StorageSystemContributors.generated.cpp") +# include +#endif namespace DB @@ -56,6 +59,9 @@ void attachSystemTablesLocal(IDatabase & system_database) system_database.attachTable("data_type_families", StorageSystemDataTypeFamilies::create("data_type_families")); system_database.attachTable("collations", StorageSystemCollations::create("collations")); system_database.attachTable("table_engines", StorageSystemTableEngines::create("table_engines")); +#if __has_include("StorageSystemContributors.generated.cpp") + system_database.attachTable("contributors", StorageSystemContributors::create("contributors")); +#endif } void attachSystemTablesServer(IDatabase & system_database, bool has_zookeeper) diff --git a/dbms/tests/queries/0_stateless/00747_contributors.reference b/dbms/tests/queries/0_stateless/00747_contributors.reference new file mode 100644 index 00000000000..9766475a418 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00747_contributors.reference @@ -0,0 +1 @@ +ok diff --git a/dbms/tests/queries/0_stateless/00747_contributors.sql b/dbms/tests/queries/0_stateless/00747_contributors.sql new file mode 100644 index 00000000000..ec75d877841 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00747_contributors.sql @@ -0,0 +1,2 @@ +-- Normally table should contain 250+ contributors. But when fast git clone used (--depth=X) (Travis build) table will contain only <=X contributors +SELECT if ((SELECT count(*) FROM system.contributors) > 1, 'ok', 'fail'); diff --git a/release b/release index 23bfd6f2dd6..2a76b9bbdd4 100755 --- a/release +++ b/release @@ -107,6 +107,8 @@ echo -e "\nCurrent version is $VERSION_STRING" gen_changelog "$VERSION_STRING" "" "$AUTHOR" "" +$CURDIR/dbms/src/Storages/System/StorageSystemContributors.sh + if [ -z "$USE_PBUILDER" ] ; then DEB_CC=${DEB_CC:=`which gcc-7 gcc-8 gcc | head -n1`} DEB_CXX=${DEB_CXX:=`which g++-7 g++-8 g++ | head -n1`} From aa0f812b1ef73bf704f3ec4203ef67d8aedd3fe0 Mon Sep 17 00:00:00 2001 From: Vasily Nemkov Date: Sun, 28 Oct 2018 08:15:57 +0300 Subject: [PATCH 24/31] Tests for ClickHouse as range_hashed dictionary source --- .../generate_and_test.py | 65 ++++++++++++++----- 1 file changed, 49 insertions(+), 16 deletions(-) diff --git a/dbms/tests/external_dictionaries/generate_and_test.py b/dbms/tests/external_dictionaries/generate_and_test.py index 7722d9b34b2..2774ccaa6d3 100755 --- a/dbms/tests/external_dictionaries/generate_and_test.py +++ b/dbms/tests/external_dictionaries/generate_and_test.py @@ -141,7 +141,7 @@ def generate_structure(args): base_name = 'range_hashed_' + range_hashed_range_type dictionaries.extend([ [ 'file_' + base_name, 3, False ], - # [ 'clickhouse_' + base_name, 3, True ], + [ 'clickhouse_' + base_name, 3, False ], # [ 'executable_flat' + base_name, 3, True ] ]) @@ -150,8 +150,6 @@ def generate_structure(args): base_name = 'range_hashed_' + range_hashed_range_type dictionaries.extend([ ['mysql_' + base_name, 3, False], - # [ 'clickhouse_' + base_name, 3, True ], - # [ 'executable_flat' + base_name, 3, True ] ]) @@ -230,6 +228,22 @@ range_hashed_mysql_column_types = { 'DateTime': 'datetime', } +range_hashed_clickhouse_column_types = { + 'UInt8': 'UInt8', + 'UInt16': 'UInt16', + 'UInt32': 'UInt32', + 'UInt64': 'UInt64', + 'Int8': 'Int8', + 'Int16': 'Int16', + 'Int32': 'Int32', + 'Int64': 'Int64', + # default type (Date) for compatibility with older versions: + '': 'Date', + 'Date': 'Date', + 'DateTime': 'DateTime', +} + + def dump_report(destination, suite, test_case, report): if destination is not None: destination_file = os.path.join(destination, suite, test_case + ".xml") @@ -291,9 +305,26 @@ def generate_data(args): query = file_source_query % comma_separated(chain(keys, columns(), ['Parent'] if 1 == len(keys) else [])) call([args.client, '--port', args.port, '--query', query], 'generated/' + file) + table_name = "test.dictionary_source_" + range_hashed_range_type + col_type = range_hashed_clickhouse_column_types[range_hashed_range_type] + + source_tsv_full_path = "{0}/generated/{1}".format(prefix, file) + print 'Creating Clickhouse table for "{0}" range_hashed dictionary...'.format(range_hashed_range_type) + system('cat {source} | {ch} --port={port} -m -n --query "' + 'create database if not exists test;' + 'drop table if exists {table_name};' + 'create table {table_name} (' + 'id UInt64, StartDate {col_type}, EndDate {col_type},' + 'UInt8_ UInt8, UInt16_ UInt16, UInt32_ UInt32, UInt64_ UInt64,' + 'Int8_ Int8, Int16_ Int16, Int32_ Int32, Int64_ Int64,' + 'Float32_ Float32, Float64_ Float64,' + 'String_ String,' + 'Date_ Date, DateTime_ DateTime, UUID_ UUID' + ') engine=Log; insert into {table_name} format TabSeparated' + '"'.format(table_name=table_name, col_type=col_type, source=source_tsv_full_path, ch=args.client, port=args.port)) + if not args.no_mysql: - print 'Creating MySQL table for "{0}"...'.format(range_hashed_range_type) - table_name = "test.dictionary_source_" + range_hashed_range_type + print 'Creating MySQL table for "{0}" range_hashed dictionary...'.format(range_hashed_range_type) col_type = range_hashed_mysql_column_types[range_hashed_range_type] subprocess.check_call('echo "' 'create database if not exists test;' @@ -305,8 +336,8 @@ def generate_data(args): 'Float32_ float, Float64_ double, ' 'String_ text, Date_ date, DateTime_ datetime, UUID_ varchar(36)' ');' - 'load data local infile \'{0}/generated/{file}\' into table {table_name};" | mysql $MYSQL_OPTIONS --local-infile=1' - .format(prefix, table_name=table_name, col_type=col_type, file=file), shell=True) + 'load data local infile \'{source}\' into table {table_name};" | mysql $MYSQL_OPTIONS --local-infile=1' + .format(prefix, table_name=table_name, col_type=col_type, source=source_tsv_full_path), shell=True) # create MySQL table from complete_query @@ -406,7 +437,7 @@ def generate_dictionaries(args): default test - dictionary_source
+ dictionary_source{key_type}
''' % args.port @@ -559,33 +590,34 @@ def generate_dictionaries(args): ''' + source_clickhouse_deafult = source_clickhouse.format(key_type="") sources_and_layouts = [ # Simple key dictionaries [ source_file % (generated_prefix + files[0]), layout_flat], - [ source_clickhouse, layout_flat ], + [ source_clickhouse_deafult, layout_flat ], [ source_executable % (generated_prefix + files[0]), layout_flat ], [ source_file % (generated_prefix + files[0]), layout_hashed], - [ source_clickhouse, layout_hashed ], + [ source_clickhouse_deafult, layout_hashed ], [ source_executable % (generated_prefix + files[0]), layout_hashed ], - [ source_clickhouse, layout_cache ], + [ source_clickhouse_deafult, layout_cache ], [ source_executable_cache % (generated_prefix + files[0]), layout_cache ], # Complex key dictionaries with (UInt8, UInt8) key [ source_file % (generated_prefix + files[1]), layout_complex_key_hashed], - [ source_clickhouse, layout_complex_key_hashed ], + [ source_clickhouse_deafult, layout_complex_key_hashed ], [ source_executable % (generated_prefix + files[1]), layout_complex_key_hashed ], - [ source_clickhouse, layout_complex_key_cache ], + [ source_clickhouse_deafult, layout_complex_key_cache ], [ source_executable_cache % (generated_prefix + files[1]), layout_complex_key_cache ], # Complex key dictionaries with (String, UInt8) key [ source_file % (generated_prefix + files[2]), layout_complex_key_hashed], - [ source_clickhouse, layout_complex_key_hashed ], + [ source_clickhouse_deafult, layout_complex_key_hashed ], [ source_executable % (generated_prefix + files[2]), layout_complex_key_hashed ], - [ source_clickhouse, layout_complex_key_cache ], + [ source_clickhouse_deafult, layout_complex_key_cache ], [ source_executable_cache % (generated_prefix + files[2]), layout_complex_key_cache ], ] @@ -655,9 +687,10 @@ def generate_dictionaries(args): ]) for range_hashed_range_type in range_hashed_range_types: + key_type = "_" + range_hashed_range_type sources_and_layouts.extend([ [ source_file % (generated_prefix + (files[3].format(range_hashed_range_type=range_hashed_range_type))), (layout_range_hashed, range_hashed_range_type) ], - # [ source_clickhouse, layout_range_hashed ], + [ source_clickhouse.format(key_type=key_type), (layout_range_hashed, range_hashed_range_type) ], # [ source_executable, layout_range_hashed ] ]) From 2a0c967b6bbf8954e8ba60ca1612df8a155f8a2c Mon Sep 17 00:00:00 2001 From: Alexey Zatelepin Date: Mon, 29 Oct 2018 14:32:28 +0300 Subject: [PATCH 25/31] add RU changelog for v18.14.11 --- CHANGELOG_RU.md | 9 +++++++++ 1 file changed, 9 insertions(+) diff --git a/CHANGELOG_RU.md b/CHANGELOG_RU.md index 30fb526f3a6..2569c655c9b 100644 --- a/CHANGELOG_RU.md +++ b/CHANGELOG_RU.md @@ -1,3 +1,12 @@ +## ClickHouse release 18.14.11, 2018-10-29 + +### Исправления ошибок: + +* Исправлена ошибка `Block structure mismatch in UNION stream: different number of columns` в запросах с LIMIT. [#2156](https://github.com/yandex/ClickHouse/issues/2156) +* Исправлены ошибки при слиянии данных в таблицах, содержащих массивы внутри Nested структур. [#3397](https://github.com/yandex/ClickHouse/pull/3397) +* Исправлен неправильный результат запросов при выключенной настройке `merge_tree_uniform_read_distribution` (включена по умолчанию). [#3429](https://github.com/yandex/ClickHouse/pull/3429) +* Исправлена ошибка при вставке в Distributed таблицу в формате Native. [#3411](https://github.com/yandex/ClickHouse/issues/3411) + ## ClickHouse release 18.14.10, 2018-10-23 * Настройка `compile_expressions` (JIT компиляция выражений) выключена по умолчанию. [#3410](https://github.com/yandex/ClickHouse/pull/3410) From 444b6e1e43a5bb9809b2dd01681c71b1c17fcb26 Mon Sep 17 00:00:00 2001 From: alesapin Date: Mon, 29 Oct 2018 16:18:41 +0300 Subject: [PATCH 26/31] Add validation of array data --- dbms/src/Formats/ValuesRowInputStream.cpp | 30 +++++++++++++++++------ 1 file changed, 23 insertions(+), 7 deletions(-) diff --git a/dbms/src/Formats/ValuesRowInputStream.cpp b/dbms/src/Formats/ValuesRowInputStream.cpp index 559ac658a6a..13d013a8ac9 100644 --- a/dbms/src/Formats/ValuesRowInputStream.cpp +++ b/dbms/src/Formats/ValuesRowInputStream.cpp @@ -2,6 +2,7 @@ #include #include #include +#include #include #include #include @@ -29,6 +30,20 @@ namespace ErrorCodes } +bool is_array_type_compatible(const DataTypeArray & type, const Field & value) +{ + if (type.getNestedType()->isNullable()) + return true; + + const Array & array = DB::get(value); + size_t size = array.size(); + for (size_t i = 0; i < size; ++i) + if (array[i].isNull()) + return false; + + return true; +} + ValuesRowInputStream::ValuesRowInputStream(ReadBuffer & istr_, const Block & header_, const Context & context_, const FormatSettings & format_settings) : istr(istr_), header(header_), context(std::make_unique(context_)), format_settings(format_settings) { @@ -116,14 +131,15 @@ bool ValuesRowInputStream::read(MutableColumns & columns) std::pair value_raw = evaluateConstantExpression(ast, *context); Field value = convertFieldToType(value_raw.first, type, value_raw.second.get()); - if (value.isNull()) + const auto * array_type = typeid_cast(&type); + + /// Check that we are indeed allowed to insert a NULL. + if ((value.isNull() && !type.isNullable()) || (array_type && !is_array_type_compatible(*array_type, value))) { - /// Check that we are indeed allowed to insert a NULL. - if (!type.isNullable()) - throw Exception{"Expression returns value " + applyVisitor(FieldVisitorToString(), value) - + ", that is out of range of type " + type.getName() - + ", at: " + String(prev_istr_position, std::min(SHOW_CHARS_ON_SYNTAX_ERROR, istr.buffer().end() - prev_istr_position)), - ErrorCodes::VALUE_IS_OUT_OF_RANGE_OF_DATA_TYPE}; + throw Exception{"Expression returns value " + applyVisitor(FieldVisitorToString(), value) + + ", that is out of range of type " + type.getName() + + ", at: " + String(prev_istr_position, std::min(SHOW_CHARS_ON_SYNTAX_ERROR, istr.buffer().end() - prev_istr_position)), + ErrorCodes::VALUE_IS_OUT_OF_RANGE_OF_DATA_TYPE}; } columns[i]->insert(value); From fc99b7d389b5135bcbfb96ada592e574e88827f6 Mon Sep 17 00:00:00 2001 From: Alex Zatelepin Date: Mon, 29 Oct 2018 17:17:58 +0300 Subject: [PATCH 27/31] style fixes --- dbms/src/Storages/transformQueryForExternalDatabase.cpp | 8 +++++--- 1 file changed, 5 insertions(+), 3 deletions(-) diff --git a/dbms/src/Storages/transformQueryForExternalDatabase.cpp b/dbms/src/Storages/transformQueryForExternalDatabase.cpp index 43d20d7a4c0..d143cb32ff8 100644 --- a/dbms/src/Storages/transformQueryForExternalDatabase.cpp +++ b/dbms/src/Storages/transformQueryForExternalDatabase.cpp @@ -119,20 +119,22 @@ String transformQueryForExternalDatabase( { if (function->name == "and") { - bool compatibleFound = false; + bool compatible_found = false; auto new_function_and = std::make_shared(); auto new_function_and_arguments = std::make_shared(); new_function_and->arguments = new_function_and_arguments; new_function_and->children.push_back(new_function_and_arguments); for (const auto & elem : function->arguments->children) + { if (isCompatible(*elem)) { new_function_and_arguments->children.push_back(elem); - compatibleFound = true; + compatible_found = true; } + } - if (compatibleFound) + if (compatible_found) select->where_expression = std::move(new_function_and); } } From 6efd0f6bf7ab8787122cb9622b970610545b7ab6 Mon Sep 17 00:00:00 2001 From: Alexey Zatelepin Date: Mon, 29 Oct 2018 19:21:01 +0300 Subject: [PATCH 28/31] Remove incorrect statement from docs Column that is in the USING clause is not required to be present in the column list of the left side of the join (but it must be present in the table definition). This reverts commit f2533a9653531c8381e5dd9365e3ec1e9f5838ed. --- docs/en/query_language/select.md | 36 -------------------------------- 1 file changed, 36 deletions(-) diff --git a/docs/en/query_language/select.md b/docs/en/query_language/select.md index 14603d1066d..53164e67d80 100644 --- a/docs/en/query_language/select.md +++ b/docs/en/query_language/select.md @@ -346,42 +346,6 @@ The table name can be specified instead of a subquery. This is equivalent to the All columns that are not needed for the JOIN are deleted from the subquery. -Example: - -``` sql -SELECT - endpoint, - metric, - value, - idcName -FROM datapoints -ALL INNER JOIN -( - SELECT - ips AS endpoint, - idcName - FROM cmdb -) USING (endpoint) -LIMIT 10 -``` - -``` -┌─endpoint─────┬─metric────────────┬───────value─┬─idcName─┐ -│ 192.168.1.100 │ disk.total │ 50465866000 │ office │ -│ 192.168.1.100 │ disk.total │ 494927870 │ office │ -│ 192.168.1.100 │ disk.used.percent │ 48 │ office │ -│ 192.168.1.100 │ disk.used.percent │ 50 │ office │ -│ 192.168.1.100 │ disk.util │ 0 │ office │ -│ 192.168.1.100 │ disk.util │ 0 │ office │ -│ 192.168.1.100 │ disk.util │ 0 │ office │ -│ 192.168.1.100 │ disk.util │ 0 │ office │ -│ 192.168.1.100 │ load.15min │ 0.02 │ office │ -│ 192.168.1.100 │ load.1min │ 0.11 │ office │ -└──────────────┴───────────────────┴─────────────┴─────────┘ -``` - -Notice the column in `USING`(endpoint) must be in former `SELECT` column list and subquery's. - There are several types of JOINs: `INNER` or `LEFT` type:If INNER is specified, the result will contain only those rows that have a matching row in the right table. From 31bc680ac5f4bb1d0360a8ba4696fa84bb47d6ab Mon Sep 17 00:00:00 2001 From: Alexey Zatelepin Date: Mon, 29 Oct 2018 21:00:36 +0300 Subject: [PATCH 29/31] ErrorCodes::SYNTAX_ERROR now causes HTTP_BAD_REQUEST --- dbms/programs/server/HTTPHandler.cpp | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/dbms/programs/server/HTTPHandler.cpp b/dbms/programs/server/HTTPHandler.cpp index 1a2f321fa00..9d9324b9a3e 100644 --- a/dbms/programs/server/HTTPHandler.cpp +++ b/dbms/programs/server/HTTPHandler.cpp @@ -63,6 +63,8 @@ namespace ErrorCodes extern const int TOO_BIG_AST; extern const int UNEXPECTED_AST_STRUCTURE; + extern const int SYNTAX_ERROR; + extern const int UNKNOWN_TABLE; extern const int UNKNOWN_FUNCTION; extern const int UNKNOWN_IDENTIFIER; @@ -109,6 +111,8 @@ static Poco::Net::HTTPResponse::HTTPStatus exceptionCodeToHTTPStatus(int excepti exception_code == ErrorCodes::TOO_BIG_AST || exception_code == ErrorCodes::UNEXPECTED_AST_STRUCTURE) return HTTPResponse::HTTP_BAD_REQUEST; + else if (exception_code == ErrorCodes::SYNTAX_ERROR) + return HTTPResponse::HTTP_BAD_REQUEST; else if (exception_code == ErrorCodes::UNKNOWN_TABLE || exception_code == ErrorCodes::UNKNOWN_FUNCTION || exception_code == ErrorCodes::UNKNOWN_IDENTIFIER || From 604ba01ab5bd397830ea5f196551a7f9e6c5dde4 Mon Sep 17 00:00:00 2001 From: Alexey Zatelepin Date: Mon, 29 Oct 2018 21:33:51 +0300 Subject: [PATCH 30/31] fix using clickhouse-client docker image as a "binary" https://github.com/yandex/ClickHouse/pull/3195#issuecomment-434027727 --- docker/client/Dockerfile | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docker/client/Dockerfile b/docker/client/Dockerfile index e1780db7c6f..3c78bb71978 100644 --- a/docker/client/Dockerfile +++ b/docker/client/Dockerfile @@ -26,4 +26,4 @@ ENV LANG en_US.UTF-8 ENV LANGUAGE en_US:en ENV LC_ALL en_US.UTF-8 -CMD ["/usr/bin/clickhouse-client"] +ENTRYPOINT ["/usr/bin/clickhouse-client"] From d538f706794fd24b89d8e47546dbb6843a163f99 Mon Sep 17 00:00:00 2001 From: abyss7 <5627721+abyss7@users.noreply.github.com> Date: Tue, 30 Oct 2018 17:05:44 +0300 Subject: [PATCH 31/31] Fix build and tests on Fedora (#3496) * Fix some tests and build on Fedora 28 * Update contrib/ssl * Try `sudo` first, then without `sudo`. --- .gitignore | 3 +++ contrib/ssl | 2 +- dbms/tests/CMakeLists.txt | 2 +- .../queries/0_stateless/00417_system_build_options.sh | 2 +- dbms/tests/queries/0_stateless/00428_partition.sh | 8 ++++++-- libs/libcommon/include/common/readline_use.h | 1 - 6 files changed, 12 insertions(+), 6 deletions(-) diff --git a/.gitignore b/.gitignore index 6e969a8459c..bc598cf0d0d 100644 --- a/.gitignore +++ b/.gitignore @@ -246,3 +246,6 @@ website/presentations website/package-lock.json .DS_Store */.DS_Store + +# Ignore files for locally disabled tests +/dbms/tests/queries/**/*.disabled diff --git a/contrib/ssl b/contrib/ssl index de02224a42c..919f6f1331d 160000 --- a/contrib/ssl +++ b/contrib/ssl @@ -1 +1 @@ -Subproject commit de02224a42c69e3d8c9112c82018816f821878d0 +Subproject commit 919f6f1331d500bfdd26f8bbbf88e92c0119879b diff --git a/dbms/tests/CMakeLists.txt b/dbms/tests/CMakeLists.txt index 3f42f94158b..6a983134937 100644 --- a/dbms/tests/CMakeLists.txt +++ b/dbms/tests/CMakeLists.txt @@ -27,7 +27,7 @@ if (ENABLE_TESTS) # maybe add --no-long ? # if you want disable some tests: env TEST_OPT0='--skip compile' - add_test(NAME with_server COMMAND bash -c "env BUILD_DIR=${ClickHouse_BINARY_DIR} INTERNAL_COMPILER_BIN_ROOT=${INTERNAL_COMPILER_BIN_ROOT} ${CMAKE_CURRENT_SOURCE_DIR}/clickhouse-test-server") + add_test(NAME with_server COMMAND bash -c "env BUILD_DIR=${ClickHouse_BINARY_DIR} ${CMAKE_CURRENT_SOURCE_DIR}/clickhouse-test-server") endif () if (ENABLE_TEST_INTEGRATION) diff --git a/dbms/tests/queries/0_stateless/00417_system_build_options.sh b/dbms/tests/queries/0_stateless/00417_system_build_options.sh index 4de22e36194..4c4b5276a1b 100755 --- a/dbms/tests/queries/0_stateless/00417_system_build_options.sh +++ b/dbms/tests/queries/0_stateless/00417_system_build_options.sh @@ -3,4 +3,4 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) . $CURDIR/../shell_config.sh -$CLICKHOUSE_CLIENT --query="SELECT * FROM system.build_options" | perl -lnE 'print $1 if /(BUILD_DATE|BUILD_TYPE|CXX_COMPILER|CXX_FLAGS|LINK_FLAGS)\s+\S+/'; +$CLICKHOUSE_CLIENT --query="SELECT * FROM system.build_options" | perl -lnE 'print $1 if /(BUILD_DATE|BUILD_TYPE|CXX_COMPILER)\s+\S+/ || /(CXX_FLAGS|LINK_FLAGS)/'; diff --git a/dbms/tests/queries/0_stateless/00428_partition.sh b/dbms/tests/queries/0_stateless/00428_partition.sh index 31e92603024..ce6ad9e1cd8 100755 --- a/dbms/tests/queries/0_stateless/00428_partition.sh +++ b/dbms/tests/queries/0_stateless/00428_partition.sh @@ -18,14 +18,18 @@ $chl "INSERT INTO test.partition_428 (p, k) VALUES(toDate(31), 1)" $chl "INSERT INTO test.partition_428 (p, k) VALUES(toDate(1), 2)" for part in `$chl "SELECT name FROM system.parts WHERE database='test' AND table='partition_428'"`; do - sudo -n cat $ch_dir/data/test/partition_428/$part/columns.txt | wc -l # 2 header lines + 3 columns + # 2 header lines + 3 columns + (sudo -n cat $ch_dir/data/test/partition_428/$part/columns.txt 2>/dev/null || \ + cat $ch_dir/data/test/partition_428/$part/columns.txt) | wc -l done $chl "ALTER TABLE test.partition_428 DETACH PARTITION 197001" $chl "ALTER TABLE test.partition_428 ATTACH PARTITION 197001" for part in `$chl "SELECT name FROM system.parts WHERE database='test' AND table='partition_428'"`; do - sudo -n cat $ch_dir/data/test/partition_428/$part/columns.txt | wc -l # 2 header lines + 3 columns + # 2 header lines + 3 columns + (sudo -n cat $ch_dir/data/test/partition_428/$part/columns.txt 2>/dev/null || \ + cat $ch_dir/data/test/partition_428/$part/columns.txt) | wc -l done $chl "ALTER TABLE test.partition_428 MODIFY COLUMN v1 Int8" diff --git a/libs/libcommon/include/common/readline_use.h b/libs/libcommon/include/common/readline_use.h index 97622b26839..549676ef9b2 100644 --- a/libs/libcommon/include/common/readline_use.h +++ b/libs/libcommon/include/common/readline_use.h @@ -10,7 +10,6 @@ #include #elif USE_LIBEDIT #include - #include // Y_IGNORE #else #include #include