diff --git a/.gitignore b/.gitignore index 6c0865d1959..bc598cf0d0d 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/ @@ -243,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/CHANGELOG_RU.md b/CHANGELOG_RU.md index f3c50637454..2569c655c9b 100644 --- a/CHANGELOG_RU.md +++ b/CHANGELOG_RU.md @@ -1,3 +1,17 @@ +## 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) +* Настройка `enable_optimize_predicate_expression` выключена по умолчанию. + ## ClickHouse release 18.14.9, 2018-10-16 ### Новые возможности: 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/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/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/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 || 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/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/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/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 { 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); 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/src/Functions/IFunction.cpp b/dbms/src/Functions/IFunction.cpp index 8d6142bc287..e0d1081246a 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; @@ -311,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, 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/AddDefaultDatabaseVisitor.h b/dbms/src/Interpreters/AddDefaultDatabaseVisitor.h new file mode 100644 index 00000000000..a9aed99c10f --- /dev/null +++ b/dbms/src/Interpreters/AddDefaultDatabaseVisitor.h @@ -0,0 +1,64 @@ +#pragma once + +#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..02e48a838bd 100644 --- a/dbms/src/Interpreters/DDLWorker.cpp +++ b/dbms/src/Interpreters/DDLWorker.cpp @@ -12,6 +12,7 @@ #include #include #include +#include #include #include #include @@ -39,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; @@ -1135,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(); @@ -1163,30 +1165,56 @@ BlockIO executeDDLQueryOnCluster(const ASTPtr & query_ptr_, const Context & cont ClusterPtr cluster = context.getCluster(query->cluster); DDLWorker & ddl_worker = context.getDDLWorker(); - DDLLogEntry entry; - 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; + NameSet databases_to_access; + const String & current_database = context.getCurrentDatabase(); 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) { - entry.hosts.emplace_back(addr); + hosts.emplace_back(addr); - /// Expand empty database name to shards' default database name + /// Expand empty database name to shards' default (o current) database name for (const String & database : query_databases) - databases_to_check_access_rights.emplace(database.empty() ? addr.default_database : database); + { + 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); + } } } - for (const String & database : databases_to_check_access_rights) - context.checkDatabaseAccessRights(database.empty() ? context.getCurrentDatabase() : 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); + } + + DDLLogEntry entry; + entry.hosts = std::move(hosts); + entry.query = queryToString(query_ptr); + entry.initiator = ddl_worker.getCommonHostID(); 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/ExpressionActions.cpp b/dbms/src/Interpreters/ExpressionActions.cpp index b84c007d790..1daf0bc3729 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()); @@ -671,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); 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/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(); 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/Settings.h b/dbms/src/Interpreters/Settings.h index af4e3d51575..13c800ed296 100644 --- a/dbms/src/Interpreters/Settings.h +++ b/dbms/src/Interpreters/Settings.h @@ -75,7 +75,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.") \ 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); + } 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; + } }; } 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/src/Storages/transformQueryForExternalDatabase.cpp b/dbms/src/Storages/transformQueryForExternalDatabase.cpp index 2615e8d0d92..d143cb32ff8 100644 --- a/dbms/src/Storages/transformQueryForExternalDatabase.cpp +++ b/dbms/src/Storages/transformQueryForExternalDatabase.cpp @@ -119,16 +119,23 @@ String transformQueryForExternalDatabase( { if (function->name == "and") { + 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); + compatible_found = true; + } + } - select->where_expression = std::move(new_function_and); + if (compatible_found) + select->where_expression = std::move(new_function_and); } } } 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/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/external_dictionaries/generate_and_test.py b/dbms/tests/external_dictionaries/generate_and_test.py index 98b67345326..2774ccaa6d3 100755 --- a/dbms/tests/external_dictionaries/generate_and_test.py +++ b/dbms/tests/external_dictionaries/generate_and_test.py @@ -141,10 +141,17 @@ 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 ] ]) + 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], + ]) + files = [ 'key_simple.tsv', 'key_complex_integers.tsv', 'key_complex_mixed.tsv', 'key_range_hashed_{range_hashed_range_type}.tsv' ] @@ -206,6 +213,36 @@ 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', +} + +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: @@ -268,6 +305,41 @@ 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}" 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;' + '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, UUID_ varchar(36)' + ');' + '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 if not args.no_mysql: print 'Creating MySQL table' @@ -365,7 +437,7 @@ def generate_dictionaries(args): default test - dictionary_source
+ dictionary_source{key_type}
''' % args.port @@ -384,7 +456,7 @@ def generate_dictionaries(args): root test - dictionary_source
+ dictionary_source{key_type}
''' @@ -518,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 ], ] @@ -568,14 +641,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: @@ -613,16 +687,29 @@ 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 ] ]) + 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] 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 + 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 +757,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 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 + + + 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") 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/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; 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; 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/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/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))]]); 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"] 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 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`}