From 8f7c2b58218db084ed38f53ee0de52777146c56c Mon Sep 17 00:00:00 2001 From: Sabyanin Maxim Date: Mon, 1 Oct 2018 23:16:50 +0300 Subject: [PATCH 001/124] first try --- dbms/src/Parsers/ASTColumnDeclaration.h | 11 ++++++++++ dbms/src/Parsers/ParserCreateQuery.h | 22 +++++++++++++++++-- dbms/src/Parsers/ParserTablePropertiesQuery.h | 1 + 3 files changed, 32 insertions(+), 2 deletions(-) diff --git a/dbms/src/Parsers/ASTColumnDeclaration.h b/dbms/src/Parsers/ASTColumnDeclaration.h index 308e9b66526..845fe175155 100644 --- a/dbms/src/Parsers/ASTColumnDeclaration.h +++ b/dbms/src/Parsers/ASTColumnDeclaration.h @@ -15,6 +15,7 @@ public: ASTPtr type; String default_specifier; ASTPtr default_expression; + ASTPtr comment_expression; String getID() const override { return "ColumnDeclaration_" + name; } @@ -35,6 +36,11 @@ public: res->children.push_back(res->default_expression); } + if (comment_expression) { + res->comment_expression = comment_expression->clone(); + res->children.push_back(res->comment_expression); // TODO: понять, зачем это нужно. + } + return res; } @@ -56,6 +62,11 @@ protected: settings.ostr << ' ' << (settings.hilite ? hilite_keyword : "") << default_specifier << (settings.hilite ? hilite_none : "") << ' '; default_expression->formatImpl(settings, state, frame); } + + // TODO: понять, почему не отрицание + if (comment_expression) { + comment_expression->formatImpl(settings, state, frame); + } } }; diff --git a/dbms/src/Parsers/ParserCreateQuery.h b/dbms/src/Parsers/ParserCreateQuery.h index 75ce5b80537..72286c155e4 100644 --- a/dbms/src/Parsers/ParserCreateQuery.h +++ b/dbms/src/Parsers/ParserCreateQuery.h @@ -111,6 +111,7 @@ bool IParserColumnDeclaration::parseImpl(Pos & pos, ASTPtr & node, E ParserKeyword s_default{"DEFAULT"}; ParserKeyword s_materialized{"MATERIALIZED"}; ParserKeyword s_alias{"ALIAS"}; + ParserKeyword s_comment{"COMMENT"}; ParserTernaryOperatorExpression expr_parser; /// mandatory column name @@ -119,13 +120,14 @@ bool IParserColumnDeclaration::parseImpl(Pos & pos, ASTPtr & node, E return false; /** column name should be followed by type name if it - * is not immediately followed by {DEFAULT, MATERIALIZED, ALIAS} + * is not immediately followed by {DEFAULT, MATERIALIZED, ALIAS, COMMENT} */ ASTPtr type; const auto fallback_pos = pos; if (!s_default.check(pos, expected) && !s_materialized.check(pos, expected) && - !s_alias.check(pos, expected)) + !s_alias.check(pos, expected) && + !s_comment.check(pos, expected)) { type_parser.parse(pos, type, expected); } @@ -149,6 +151,17 @@ bool IParserColumnDeclaration::parseImpl(Pos & pos, ASTPtr & node, E else if (!type) return false; /// reject sole column name without type + String comment_specifier; + ASTPtr comment_expression; + pos_before_specifier = pos; + if (s_comment.ignore(pos, expected)) + { + comment_specifier = Poco::toUpper(std::string{pos_before_specifier->begin, pos_specifier->end}); + if (!expr_parser.parse(pos, comment_expression, expected)) { + return false; + } + } + const auto column_declaration = std::make_shared(); node = column_declaration; column_declaration->name = typeid_cast(*name).name; @@ -165,6 +178,11 @@ bool IParserColumnDeclaration::parseImpl(Pos & pos, ASTPtr & node, E column_declaration->children.push_back(std::move(default_expression)); } + if (comment_expression) { + column_declaration->comment_expression = comment_expression; + column_declaration->children.push_back(std::move(comment_expression)); + } + return true; } diff --git a/dbms/src/Parsers/ParserTablePropertiesQuery.h b/dbms/src/Parsers/ParserTablePropertiesQuery.h index 1bc4a06f161..c5321dd472b 100644 --- a/dbms/src/Parsers/ParserTablePropertiesQuery.h +++ b/dbms/src/Parsers/ParserTablePropertiesQuery.h @@ -8,6 +8,7 @@ namespace DB { +// TODO: возможно тут тоже надо разобраться /** Query (EXISTS | SHOW CREATE) [TABLE] [db.]name [FORMAT format] */ class ParserTablePropertiesQuery : public IParserBase From 5ff30ebb3d35f42aa20dba21a1eb4ed2724d39e9 Mon Sep 17 00:00:00 2001 From: Alexander Krasheninnikov Date: Wed, 10 Oct 2018 04:04:07 +0300 Subject: [PATCH 002/124] Initial implementation --- .gitmodules | 3 + contrib/CMakeLists.txt | 3 + contrib/base64 | 1 + contrib/base64-cmake/CMakeLists.txt | 39 +++++ contrib/base64-cmake/config.h | 8 + dbms/src/Functions/CMakeLists.txt | 4 +- dbms/src/Functions/FunctionBase64Conversion.h | 144 ++++++++++++++++++ dbms/src/Functions/base64Decode.cpp | 12 ++ dbms/src/Functions/base64Encode.cpp | 12 ++ .../src/Functions/registerFunctionsString.cpp | 4 + .../000732_base64_functions.reference | 15 ++ .../0_stateless/000732_base64_functions.sql | 5 + 12 files changed, 249 insertions(+), 1 deletion(-) create mode 160000 contrib/base64 create mode 100644 contrib/base64-cmake/CMakeLists.txt create mode 100644 contrib/base64-cmake/config.h create mode 100644 dbms/src/Functions/FunctionBase64Conversion.h create mode 100644 dbms/src/Functions/base64Decode.cpp create mode 100644 dbms/src/Functions/base64Encode.cpp create mode 100644 dbms/tests/queries/0_stateless/000732_base64_functions.reference create mode 100644 dbms/tests/queries/0_stateless/000732_base64_functions.sql diff --git a/.gitmodules b/.gitmodules index c43b754dba8..be97e7c4672 100644 --- a/.gitmodules +++ b/.gitmodules @@ -46,3 +46,6 @@ [submodule "contrib/unixodbc"] path = contrib/unixodbc url = https://github.com/ClickHouse-Extras/UnixODBC.git +[submodule "contrib/base64"] + path = contrib/base64 + url = git@github.com:aklomp/base64.git diff --git a/contrib/CMakeLists.txt b/contrib/CMakeLists.txt index 1c653f86808..373398333e4 100644 --- a/contrib/CMakeLists.txt +++ b/contrib/CMakeLists.txt @@ -174,3 +174,6 @@ if (USE_INTERNAL_GTEST_LIBRARY) target_compile_definitions (gtest INTERFACE GTEST_HAS_POSIX_RE=0) target_include_directories (gtest SYSTEM INTERFACE ${ClickHouse_SOURCE_DIR}/contrib/googletest/include) endif () + + +add_subdirectory (base64-cmake) \ No newline at end of file diff --git a/contrib/base64 b/contrib/base64 new file mode 160000 index 00000000000..a27c565d1b6 --- /dev/null +++ b/contrib/base64 @@ -0,0 +1 @@ +Subproject commit a27c565d1b6c676beaf297fe503c4518185666f7 diff --git a/contrib/base64-cmake/CMakeLists.txt b/contrib/base64-cmake/CMakeLists.txt new file mode 100644 index 00000000000..1e8d477ca09 --- /dev/null +++ b/contrib/base64-cmake/CMakeLists.txt @@ -0,0 +1,39 @@ +SET(LIBRARY_DIR ${ClickHouse_SOURCE_DIR}/contrib/base64) + +macro(add_glob cur_list) + file(GLOB __tmp RELATIVE ${CMAKE_CURRENT_SOURCE_DIR} ${ARGN}) + list(APPEND ${cur_list} ${__tmp}) +endmacro() + +macro(add_headers_and_sources prefix common_path) + add_glob(${prefix}_headers RELATIVE ${CMAKE_CURRENT_SOURCE_DIR} ${common_path}/*.h) + add_glob(${prefix}_sources ${common_path}/*.cpp ${common_path}/*.h ${common_path}/codec.c) +endmacro() + + +MACRO(SUBDIRLIST result curdir) + FILE(GLOB children RELATIVE ${curdir} ${curdir}/*) + SET(dirlist "") + FOREACH(child ${children}) + IF(IS_DIRECTORY ${curdir}/${child}) + LIST(APPEND dirlist ${child}) + ENDIF() + ENDFOREACH() + SET(${result} ${dirlist}) +ENDMACRO() + +add_headers_and_sources(base64 ${LIBRARY_DIR}/include/) +add_headers_and_sources(base64 ${LIBRARY_DIR}/lib/) + +SUBDIRLIST(SUBDIRS ${LIBRARY_DIR}/lib/arch/) +FOREACH(subdir ${SUBDIRS}) + add_headers_and_sources(base64 ${LIBRARY_DIR}/lib/arch/${subdir}) +ENDFOREACH() + +list(APPEND base64_sources ../base64/lib/lib.c ../base64/lib/codec_choose.c) + +add_library (base64 ${LINK_MODE} ${base64_sources} config.h) + +# todo fix +target_compile_definitions(base64 PUBLIC SSE41_CFLAGS=-msse4.1 SSE42_CFLAGS=-msse4.2) +target_include_directories(base64 PRIVATE ${LIBRARY_DIR}/include .) diff --git a/contrib/base64-cmake/config.h b/contrib/base64-cmake/config.h new file mode 100644 index 00000000000..f90c0dd6120 --- /dev/null +++ b/contrib/base64-cmake/config.h @@ -0,0 +1,8 @@ +#define HAVE_AVX2 0 +#define HAVE_NEON32 0 +#define HAVE_NEON64 0 +#define HAVE_SSSE3 0 +#define HAVE_SSE41 1 +#define HAVE_SSE42 1 +#define HAVE_AVX 0 +#define HAVE_FAST_UNALIGNED_ACCESS 1 diff --git a/dbms/src/Functions/CMakeLists.txt b/dbms/src/Functions/CMakeLists.txt index 2e0c6615676..f67af9b617e 100644 --- a/dbms/src/Functions/CMakeLists.txt +++ b/dbms/src/Functions/CMakeLists.txt @@ -52,10 +52,12 @@ list(REMOVE_ITEM clickhouse_functions_headers IFunction.h FunctionFactory.h Func add_library(clickhouse_functions ${clickhouse_functions_sources}) -target_link_libraries(clickhouse_functions PUBLIC dbms PRIVATE ${CONSISTENT_HASHING_LIBRARY} consistent-hashing-sumbur ${FARMHASH_LIBRARIES} ${METROHASH_LIBRARIES} murmurhash) +target_link_libraries(clickhouse_functions PUBLIC dbms PRIVATE ${CONSISTENT_HASHING_LIBRARY} consistent-hashing-sumbur ${FARMHASH_LIBRARIES} ${METROHASH_LIBRARIES} murmurhash base64) target_include_directories (clickhouse_functions SYSTEM BEFORE PUBLIC ${DIVIDE_INCLUDE_DIR}) +target_include_directories (clickhouse_functions SYSTEM PRIVATE ${ClickHouse_SOURCE_DIR}/contrib/base64/include) + if (CONSISTENT_HASHING_INCLUDE_DIR) target_include_directories (clickhouse_functions PRIVATE ${CONSISTENT_HASHING_INCLUDE_DIR}) endif () diff --git a/dbms/src/Functions/FunctionBase64Conversion.h b/dbms/src/Functions/FunctionBase64Conversion.h new file mode 100644 index 00000000000..146814d9ea2 --- /dev/null +++ b/dbms/src/Functions/FunctionBase64Conversion.h @@ -0,0 +1,144 @@ +#include +#include +#include +#include +#include +#include +#include +#include + + +namespace DB +{ +using namespace GatherUtils; + +namespace ErrorCodes +{ + extern const int ILLEGAL_COLUMN; + extern const int ILLEGAL_TYPE_OF_ARGUMENT; + extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; + extern const int INCORRECT_DATA; +} + +struct Base64Encode +{ + static constexpr auto name = "base64Encode"; + static constexpr auto buffer_size_multiplier = 5.0 / 3.0; +}; + +struct Base64Decode +{ + static constexpr auto name = "base64Decode"; + static constexpr auto buffer_size_multiplier = 3.0 / 4.0; +}; + + +template +class FunctionBase64Conversion : public IFunction +{ +public: + static constexpr auto name = Func::name; + + static FunctionPtr create(const Context &) + { + return std::make_shared(); + } + + String getName() const override + { + return Func::name; + } + + size_t getNumberOfArguments() const override + { + return 1; + } + + bool useDefaultImplementationForConstants() const override { return true; } + + DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override + { + size_t number_of_arguments = arguments.size(); + + if (number_of_arguments != 1) + throw Exception("Function " + getName() + " needs exactly one argument of type String", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); + + if (!WhichDataType(arguments[0].type).isString()) + throw Exception( + "Illegal type " + arguments[0].type->getName() + " of 1 argument of function " + getName() + ". Must be String.", + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + + return std::make_shared(); + } + + void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override + { + const ColumnPtr column_string = block.getByPosition(arguments[0]).column; + const ColumnString * input = checkAndGetColumn(column_string.get()); + + if (!input) + throw Exception( + "Illegal column " + block.getByPosition(arguments[0]).column->getName() + " of first argument of function " + getName(), + ErrorCodes::ILLEGAL_COLUMN); + + auto dst_column = ColumnString::create(); + auto & dst_data = dst_column->getChars(); + auto & dst_offsets = dst_column->getOffsets(); + + size_t reserve = ceil(input->getChars().size() * Func::buffer_size_multiplier + input->size()); + dst_data.resize(reserve); + dst_offsets.resize(input_rows_count); + + const ColumnString::Offsets & src_offsets = input->getOffsets(); + + auto source = reinterpret_cast(input->getChars().data()); + auto dst = reinterpret_cast(dst_data.data()); + auto dst_pos = dst; + + size_t src_offset_prev = 0; + + int codec = getCodec(); + for (size_t row = 0; row < input_rows_count; ++row) + { + size_t srclen = src_offsets[row] - src_offset_prev - 1; + size_t outlen = 0; + + if constexpr (std::is_same_v) + { + base64_encode(source, srclen , dst_pos, &outlen, codec); + } + else + { + if (!base64_decode(source, srclen, dst_pos, &outlen, codec)) + { + throw Exception("Failed to " + getName() + " input '" + String(source, srclen) + "'", ErrorCodes::INCORRECT_DATA); + } + } + + source += srclen + 1; + dst_pos += outlen + 1; + + dst_offsets[row] = dst_pos - dst; + src_offset_prev = src_offsets[row]; + } + + dst_data.resize(dst_pos - dst); + + block.getByPosition(result).column = std::move(dst_column); + } + +private: + static int getCodec() + { +#if __SSE4_2__ + return BASE64_FORCE_SSE42; +#elif __SSE4_1__ + return BASE64_FORCE_SSE41; +#else + return BASE64_FORCE_PLAIN; +#endif + + } +}; + +} diff --git a/dbms/src/Functions/base64Decode.cpp b/dbms/src/Functions/base64Decode.cpp new file mode 100644 index 00000000000..5eb107a59b0 --- /dev/null +++ b/dbms/src/Functions/base64Decode.cpp @@ -0,0 +1,12 @@ +#include +#include +#include + +namespace DB +{ + +void registerFunctionBase64Decode(FunctionFactory &factory) +{ + factory.registerFunction>(); +} +} \ No newline at end of file diff --git a/dbms/src/Functions/base64Encode.cpp b/dbms/src/Functions/base64Encode.cpp new file mode 100644 index 00000000000..cca00f1fed5 --- /dev/null +++ b/dbms/src/Functions/base64Encode.cpp @@ -0,0 +1,12 @@ +#include +#include +#include + +namespace DB +{ + +void registerFunctionBase64Encode(FunctionFactory &factory) +{ + factory.registerFunction>(); +} +} \ No newline at end of file diff --git a/dbms/src/Functions/registerFunctionsString.cpp b/dbms/src/Functions/registerFunctionsString.cpp index 9adf3a52824..998bc948409 100644 --- a/dbms/src/Functions/registerFunctionsString.cpp +++ b/dbms/src/Functions/registerFunctionsString.cpp @@ -19,6 +19,8 @@ void registerFunctionSubstringUTF8(FunctionFactory &); void registerFunctionAppendTrailingCharIfAbsent(FunctionFactory &); void registerFunctionStartsWith(FunctionFactory &); void registerFunctionEndsWith(FunctionFactory &); +void registerFunctionBase64Encode(FunctionFactory &); +void registerFunctionBase64Decode(FunctionFactory &); void registerFunctionsString(FunctionFactory & factory) { @@ -38,6 +40,8 @@ void registerFunctionsString(FunctionFactory & factory) registerFunctionAppendTrailingCharIfAbsent(factory); registerFunctionStartsWith(factory); registerFunctionEndsWith(factory); + registerFunctionBase64Encode(factory); + registerFunctionBase64Decode(factory); } } diff --git a/dbms/tests/queries/0_stateless/000732_base64_functions.reference b/dbms/tests/queries/0_stateless/000732_base64_functions.reference new file mode 100644 index 00000000000..e6fb8f43b11 --- /dev/null +++ b/dbms/tests/queries/0_stateless/000732_base64_functions.reference @@ -0,0 +1,15 @@ + +Zg== +Zm8= +Zm9v +Zm9vYg== +Zm9vYmE= +Zm9vYmFy + +f +fo +foo +foob +fooba +foobar +1 1 diff --git a/dbms/tests/queries/0_stateless/000732_base64_functions.sql b/dbms/tests/queries/0_stateless/000732_base64_functions.sql new file mode 100644 index 00000000000..7778eef02c7 --- /dev/null +++ b/dbms/tests/queries/0_stateless/000732_base64_functions.sql @@ -0,0 +1,5 @@ +SET send_logs_level = 'none'; +SELECT base64Encode(val) FROM (select arrayJoin(['', 'f', 'fo', 'foo', 'foob', 'fooba', 'foobar']) val); +SELECT base64Decode(val) FROM (select arrayJoin(['', 'Zg==', 'Zm8=', 'Zm9v', 'Zm9vYg==', 'Zm9vYmE=', 'Zm9vYmFy']) val); +SELECT base64Decode(base64Encode('foo')) = 'foo', base64Encode(base64Decode('Zm9v')) == 'Zm9v'; +SELECT base64Decode('Zm9vYmF=Zm9v'); -- { serverError 117 } \ No newline at end of file From d9a0432c6cd9b9457813fd5895d39239462837e6 Mon Sep 17 00:00:00 2001 From: Alexey Zatelepin Date: Tue, 9 Oct 2018 21:32:44 +0300 Subject: [PATCH 003/124] don't quote single-field partition keys #3109 [#CLICKHOUSE-4054] --- dbms/src/Storages/MergeTree/MergeTreeData.cpp | 2 +- .../Storages/MergeTree/MergeTreePartition.cpp | 32 +++++++++++-------- .../Storages/MergeTree/MergeTreePartition.h | 2 +- .../Storages/System/StorageSystemParts.cpp | 2 +- .../System/StorageSystemPartsColumns.cpp | 2 +- .../00502_custom_partitioning_local.reference | 24 +++++++------- ...artitioning_replicated_zookeeper.reference | 24 +++++++------- 7 files changed, 46 insertions(+), 42 deletions(-) diff --git a/dbms/src/Storages/MergeTree/MergeTreeData.cpp b/dbms/src/Storages/MergeTree/MergeTreeData.cpp index 0a55bfa97e5..eddf3fbb339 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeData.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeData.cpp @@ -2184,7 +2184,7 @@ String MergeTreeData::getPartitionIDFromQuery(const ASTPtr & ast, const Context { WriteBufferFromOwnString buf; writeCString("Parsed partition value: ", buf); - partition.serializeTextQuoted(*this, buf, format_settings); + partition.serializeText(*this, buf, format_settings); writeCString(" doesn't match partition value for an existing part with the same partition ID: ", buf); writeString(existing_part_in_partition->name, buf); throw Exception(buf.str(), ErrorCodes::INVALID_PARTITION_VALUE); diff --git a/dbms/src/Storages/MergeTree/MergeTreePartition.cpp b/dbms/src/Storages/MergeTree/MergeTreePartition.cpp index 6064625d49e..d4aad4230b0 100644 --- a/dbms/src/Storages/MergeTree/MergeTreePartition.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreePartition.cpp @@ -77,32 +77,36 @@ String MergeTreePartition::getID(const MergeTreeData & storage) const return result; } -void MergeTreePartition::serializeTextQuoted(const MergeTreeData & storage, WriteBuffer & out, const FormatSettings & format_settings) const +void MergeTreePartition::serializeText(const MergeTreeData & storage, WriteBuffer & out, const FormatSettings & format_settings) const { size_t key_size = storage.partition_key_sample.columns(); if (key_size == 0) { writeCString("tuple()", out); - return; } - - if (key_size > 1) - writeChar('(', out); - - for (size_t i = 0; i < key_size; ++i) + else if (key_size == 1) { - if (i > 0) - writeCString(", ", out); - - const DataTypePtr & type = storage.partition_key_sample.getByPosition(i).type; + const DataTypePtr & type = storage.partition_key_sample.getByPosition(0).type; auto column = type->createColumn(); - column->insert(value[i]); - type->serializeTextQuoted(*column, 0, out, format_settings); + column->insert(value[0]); + type->serializeText(*column, 0, out, format_settings); } + else + { + writeChar('(', out); + for (size_t i = 0; i < key_size; ++i) + { + if (i > 0) + writeCString(", ", out); - if (key_size > 1) + const DataTypePtr & type = storage.partition_key_sample.getByPosition(i).type; + auto column = type->createColumn(); + column->insert(value[i]); + type->serializeTextQuoted(*column, 0, out, format_settings); + } writeChar(')', out); + } } void MergeTreePartition::load(const MergeTreeData & storage, const String & part_path) diff --git a/dbms/src/Storages/MergeTree/MergeTreePartition.h b/dbms/src/Storages/MergeTree/MergeTreePartition.h index cb5493cc8bd..425828c2cc1 100644 --- a/dbms/src/Storages/MergeTree/MergeTreePartition.h +++ b/dbms/src/Storages/MergeTree/MergeTreePartition.h @@ -26,7 +26,7 @@ public: String getID(const MergeTreeData & storage) const; - void serializeTextQuoted(const MergeTreeData & storage, WriteBuffer & out, const FormatSettings & format_settings) const; + void serializeText(const MergeTreeData & storage, WriteBuffer & out, const FormatSettings & format_settings) const; void load(const MergeTreeData & storage, const String & part_path); void store(const MergeTreeData & storage, const String & part_path, MergeTreeDataPartChecksums & checksums) const; diff --git a/dbms/src/Storages/System/StorageSystemParts.cpp b/dbms/src/Storages/System/StorageSystemParts.cpp index 71c15004e66..597a4e2cc30 100644 --- a/dbms/src/Storages/System/StorageSystemParts.cpp +++ b/dbms/src/Storages/System/StorageSystemParts.cpp @@ -61,7 +61,7 @@ void StorageSystemParts::processNextStorage(MutableColumns & columns, const Stor size_t i = 0; { WriteBufferFromOwnString out; - part->partition.serializeTextQuoted(*info.data, out, format_settings); + part->partition.serializeText(*info.data, out, format_settings); columns[i++]->insert(out.str()); } columns[i++]->insert(part->name); diff --git a/dbms/src/Storages/System/StorageSystemPartsColumns.cpp b/dbms/src/Storages/System/StorageSystemPartsColumns.cpp index 435b3dd287f..de882ee4581 100644 --- a/dbms/src/Storages/System/StorageSystemPartsColumns.cpp +++ b/dbms/src/Storages/System/StorageSystemPartsColumns.cpp @@ -106,7 +106,7 @@ void StorageSystemPartsColumns::processNextStorage(MutableColumns & columns, con size_t j = 0; { WriteBufferFromOwnString out; - part->partition.serializeTextQuoted(*info.data, out, format_settings); + part->partition.serializeText(*info.data, out, format_settings); columns[j++]->insert(out.str()); } columns[j++]->insert(part->name); diff --git a/dbms/tests/queries/0_stateless/00502_custom_partitioning_local.reference b/dbms/tests/queries/0_stateless/00502_custom_partitioning_local.reference index f3811ddcf1d..b8370d9e27a 100644 --- a/dbms/tests/queries/0_stateless/00502_custom_partitioning_local.reference +++ b/dbms/tests/queries/0_stateless/00502_custom_partitioning_local.reference @@ -10,12 +10,12 @@ Sum after DETACH PARTITION: 0 *** Partitioned by week *** Parts before OPTIMIZE: -\'1999-12-27\' 19991227_1_1_0 -\'2000-01-03\' 20000103_2_2_0 -\'2000-01-03\' 20000103_3_3_0 +1999-12-27 19991227_1_1_0 +2000-01-03 20000103_2_2_0 +2000-01-03 20000103_3_3_0 Parts after OPTIMIZE: -\'1999-12-27\' 19991227_1_1_0 -\'2000-01-03\' 20000103_2_3_1 +1999-12-27 19991227_1_1_0 +2000-01-03 20000103_2_3_1 Sum before DROP PARTITION: 15 Sum after DROP PARTITION: @@ -37,14 +37,14 @@ Sum after DETACH PARTITION: 9 *** Partitioned by String *** Parts before OPTIMIZE: -\'bbb\' 7d878f3d88441d2b3dc371e2a3050f6d_2_2_0 -\'bbb\' 7d878f3d88441d2b3dc371e2a3050f6d_3_3_0 -\'aaa\' 9b50856126a8a6064f11f027d455bf58_1_1_0 -\'aaa\' 9b50856126a8a6064f11f027d455bf58_4_4_0 +bbb 7d878f3d88441d2b3dc371e2a3050f6d_2_2_0 +bbb 7d878f3d88441d2b3dc371e2a3050f6d_3_3_0 +aaa 9b50856126a8a6064f11f027d455bf58_1_1_0 +aaa 9b50856126a8a6064f11f027d455bf58_4_4_0 Parts after OPTIMIZE: -\'bbb\' 7d878f3d88441d2b3dc371e2a3050f6d_2_2_0 -\'bbb\' 7d878f3d88441d2b3dc371e2a3050f6d_3_3_0 -\'aaa\' 9b50856126a8a6064f11f027d455bf58_1_4_1 +bbb 7d878f3d88441d2b3dc371e2a3050f6d_2_2_0 +bbb 7d878f3d88441d2b3dc371e2a3050f6d_3_3_0 +aaa 9b50856126a8a6064f11f027d455bf58_1_4_1 Sum before DROP PARTITION: 15 Sum after DROP PARTITION: diff --git a/dbms/tests/queries/0_stateless/00502_custom_partitioning_replicated_zookeeper.reference b/dbms/tests/queries/0_stateless/00502_custom_partitioning_replicated_zookeeper.reference index 313e84db2d7..8b6d0ed1384 100644 --- a/dbms/tests/queries/0_stateless/00502_custom_partitioning_replicated_zookeeper.reference +++ b/dbms/tests/queries/0_stateless/00502_custom_partitioning_replicated_zookeeper.reference @@ -10,12 +10,12 @@ Sum after DETACH PARTITION: 0 *** Partitioned by week *** Parts before OPTIMIZE: -\'1999-12-27\' 19991227_0_0_0 -\'2000-01-03\' 20000103_0_0_0 -\'2000-01-03\' 20000103_1_1_0 +1999-12-27 19991227_0_0_0 +2000-01-03 20000103_0_0_0 +2000-01-03 20000103_1_1_0 Parts after OPTIMIZE: -\'1999-12-27\' 19991227_0_0_0 -\'2000-01-03\' 20000103_0_1_1 +1999-12-27 19991227_0_0_0 +2000-01-03 20000103_0_1_1 Sum before DROP PARTITION: 15 Sum after DROP PARTITION: @@ -37,14 +37,14 @@ Sum after DETACH PARTITION: 9 *** Partitioned by String *** Parts before OPTIMIZE: -\'bbb\' 7d878f3d88441d2b3dc371e2a3050f6d_0_0_0 -\'bbb\' 7d878f3d88441d2b3dc371e2a3050f6d_1_1_0 -\'aaa\' 9b50856126a8a6064f11f027d455bf58_0_0_0 -\'aaa\' 9b50856126a8a6064f11f027d455bf58_1_1_0 +bbb 7d878f3d88441d2b3dc371e2a3050f6d_0_0_0 +bbb 7d878f3d88441d2b3dc371e2a3050f6d_1_1_0 +aaa 9b50856126a8a6064f11f027d455bf58_0_0_0 +aaa 9b50856126a8a6064f11f027d455bf58_1_1_0 Parts after OPTIMIZE: -\'bbb\' 7d878f3d88441d2b3dc371e2a3050f6d_0_0_0 -\'bbb\' 7d878f3d88441d2b3dc371e2a3050f6d_1_1_0 -\'aaa\' 9b50856126a8a6064f11f027d455bf58_0_1_1 +bbb 7d878f3d88441d2b3dc371e2a3050f6d_0_0_0 +bbb 7d878f3d88441d2b3dc371e2a3050f6d_1_1_0 +aaa 9b50856126a8a6064f11f027d455bf58_0_1_1 Sum before DROP PARTITION: 15 Sum after DROP PARTITION: From 41281ad39757eea0caefbd2e2b8682c4d96d2a24 Mon Sep 17 00:00:00 2001 From: Alexandr Krasheninnikov Date: Thu, 11 Oct 2018 19:22:50 +0300 Subject: [PATCH 004/124] Make compilation on-demand --- CMakeLists.txt | 1 + cmake/find_base64.cmake | 12 ++++ contrib/CMakeLists.txt | 4 +- contrib/base64-cmake/.gitignore | 1 + contrib/base64-cmake/CMakeLists.txt | 56 ++++++++----------- .../{config.h => config-header.tpl} | 5 +- dbms/src/Common/config.h.in | 1 + dbms/src/Common/config_build.cpp.in | 1 + dbms/src/Functions/CMakeLists.txt | 8 ++- dbms/src/Functions/FunctionBase64Conversion.h | 16 ++++-- dbms/src/Functions/base64Decode.cpp | 6 +- dbms/src/Functions/base64Encode.cpp | 6 +- .../src/Functions/registerFunctionsString.cpp | 4 ++ 13 files changed, 72 insertions(+), 49 deletions(-) create mode 100644 cmake/find_base64.cmake create mode 100644 contrib/base64-cmake/.gitignore rename contrib/base64-cmake/{config.h => config-header.tpl} (62%) diff --git a/CMakeLists.txt b/CMakeLists.txt index 4dbc414e079..ca7a55cd33b 100644 --- a/CMakeLists.txt +++ b/CMakeLists.txt @@ -255,6 +255,7 @@ include (cmake/find_capnp.cmake) include (cmake/find_llvm.cmake) include (cmake/find_cpuid.cmake) include (cmake/find_consistent-hashing.cmake) +include (cmake/find_base64.cmake) if (ENABLE_TESTS) include (cmake/find_gtest.cmake) endif () diff --git a/cmake/find_base64.cmake b/cmake/find_base64.cmake new file mode 100644 index 00000000000..ad71d11c1b4 --- /dev/null +++ b/cmake/find_base64.cmake @@ -0,0 +1,12 @@ +option (ENABLE_BASE64 "Enable base64" ON) + +if (ENABLE_BASE64) + if (NOT EXISTS "${ClickHouse_SOURCE_DIR}/contrib/base64") + message (WARNING "submodule contrib/base64 is missing. to fix try run: \n git submodule update --init --recursive") + else() + set (BASE64_INCLUDE_DIR ${ClickHouse_SOURCE_DIR}/contrib/base64/include) + set (BASE64_LIBRARY base64) + set (USE_BASE64 1) + endif() +endif () + diff --git a/contrib/CMakeLists.txt b/contrib/CMakeLists.txt index 373398333e4..a0320589802 100644 --- a/contrib/CMakeLists.txt +++ b/contrib/CMakeLists.txt @@ -176,4 +176,6 @@ if (USE_INTERNAL_GTEST_LIBRARY) endif () -add_subdirectory (base64-cmake) \ No newline at end of file +if (USE_BASE64) + add_subdirectory (base64-cmake) +endif() \ No newline at end of file diff --git a/contrib/base64-cmake/.gitignore b/contrib/base64-cmake/.gitignore new file mode 100644 index 00000000000..299bb98e503 --- /dev/null +++ b/contrib/base64-cmake/.gitignore @@ -0,0 +1 @@ +config.h \ No newline at end of file diff --git a/contrib/base64-cmake/CMakeLists.txt b/contrib/base64-cmake/CMakeLists.txt index 1e8d477ca09..8bf0316b461 100644 --- a/contrib/base64-cmake/CMakeLists.txt +++ b/contrib/base64-cmake/CMakeLists.txt @@ -1,39 +1,31 @@ SET(LIBRARY_DIR ${ClickHouse_SOURCE_DIR}/contrib/base64) -macro(add_glob cur_list) - file(GLOB __tmp RELATIVE ${CMAKE_CURRENT_SOURCE_DIR} ${ARGN}) - list(APPEND ${cur_list} ${__tmp}) -endmacro() +# write config.h file, to include it in application +file(READ config-header.tpl header) +file(WRITE config.h ${header}) +file(APPEND config.h "#define HAVE_SSE41 ${HAVE_SSE41}\n") +file(APPEND config.h "#define HAVE_SSE42 ${HAVE_SSE42}\n") -macro(add_headers_and_sources prefix common_path) - add_glob(${prefix}_headers RELATIVE ${CMAKE_CURRENT_SOURCE_DIR} ${common_path}/*.h) - add_glob(${prefix}_sources ${common_path}/*.cpp ${common_path}/*.h ${common_path}/codec.c) -endmacro() +set(HAVE_FAST_UNALIGNED_ACCESS 0) +if (${HAVE_SSE41} OR ${HAVE_SSE42}) + set(HAVE_FAST_UNALIGNED_ACCESS 1) +endif () +file(APPEND config.h "#define HAVE_FAST_UNALIGNED_ACCESS " ${HAVE_FAST_UNALIGNED_ACCESS} "\n") -MACRO(SUBDIRLIST result curdir) - FILE(GLOB children RELATIVE ${curdir} ${curdir}/*) - SET(dirlist "") - FOREACH(child ${children}) - IF(IS_DIRECTORY ${curdir}/${child}) - LIST(APPEND dirlist ${child}) - ENDIF() - ENDFOREACH() - SET(${result} ${dirlist}) -ENDMACRO() +add_library(base64 ${LINK_MODE} + ${LIBRARY_DIR}/lib/lib.c + ${LIBRARY_DIR}/lib/codec_choose.c + ${LIBRARY_DIR}/lib/arch/avx/codec.c + ${LIBRARY_DIR}/lib/arch/avx2/codec.c + ${LIBRARY_DIR}/lib/arch/generic/codec.c + ${LIBRARY_DIR}/lib/arch/neon32/codec.c + ${LIBRARY_DIR}/lib/arch/neon64/codec.c + ${LIBRARY_DIR}/lib/arch/sse41/codec.c + ${LIBRARY_DIR}/lib/arch/sse42/codec.c + ${LIBRARY_DIR}/lib/arch/ssse3/codec.c -add_headers_and_sources(base64 ${LIBRARY_DIR}/include/) -add_headers_and_sources(base64 ${LIBRARY_DIR}/lib/) + ${LIBRARY_DIR}/lib/codecs.h + config.h) -SUBDIRLIST(SUBDIRS ${LIBRARY_DIR}/lib/arch/) -FOREACH(subdir ${SUBDIRS}) - add_headers_and_sources(base64 ${LIBRARY_DIR}/lib/arch/${subdir}) -ENDFOREACH() - -list(APPEND base64_sources ../base64/lib/lib.c ../base64/lib/codec_choose.c) - -add_library (base64 ${LINK_MODE} ${base64_sources} config.h) - -# todo fix -target_compile_definitions(base64 PUBLIC SSE41_CFLAGS=-msse4.1 SSE42_CFLAGS=-msse4.2) -target_include_directories(base64 PRIVATE ${LIBRARY_DIR}/include .) +target_include_directories(base64 PRIVATE ${LIBRARY_DIR}/include .) \ No newline at end of file diff --git a/contrib/base64-cmake/config.h b/contrib/base64-cmake/config-header.tpl similarity index 62% rename from contrib/base64-cmake/config.h rename to contrib/base64-cmake/config-header.tpl index f90c0dd6120..76f251d875e 100644 --- a/contrib/base64-cmake/config.h +++ b/contrib/base64-cmake/config-header.tpl @@ -1,8 +1,5 @@ +#define HAVE_AVX 0 #define HAVE_AVX2 0 #define HAVE_NEON32 0 #define HAVE_NEON64 0 #define HAVE_SSSE3 0 -#define HAVE_SSE41 1 -#define HAVE_SSE42 1 -#define HAVE_AVX 0 -#define HAVE_FAST_UNALIGNED_ACCESS 1 diff --git a/dbms/src/Common/config.h.in b/dbms/src/Common/config.h.in index af12cc525f7..44aa9e9773d 100644 --- a/dbms/src/Common/config.h.in +++ b/dbms/src/Common/config.h.in @@ -15,3 +15,4 @@ #cmakedefine01 USE_POCO_MONGODB #cmakedefine01 USE_POCO_NETSSL #cmakedefine01 CLICKHOUSE_SPLIT_BINARY +#cmakedefine01 USE_BASE64 diff --git a/dbms/src/Common/config_build.cpp.in b/dbms/src/Common/config_build.cpp.in index 9e74ea43e00..df6cbd033cc 100644 --- a/dbms/src/Common/config_build.cpp.in +++ b/dbms/src/Common/config_build.cpp.in @@ -40,6 +40,7 @@ const char * auto_config_build[] "USE_POCO_DATAODBC", "@USE_POCO_DATAODBC@", "USE_POCO_MONGODB", "@USE_POCO_MONGODB@", "USE_POCO_NETSSL", "@USE_POCO_NETSSL@", + "USE_BASE64", "@USE_BASE64@", nullptr, nullptr }; diff --git a/dbms/src/Functions/CMakeLists.txt b/dbms/src/Functions/CMakeLists.txt index f67af9b617e..84a2916656b 100644 --- a/dbms/src/Functions/CMakeLists.txt +++ b/dbms/src/Functions/CMakeLists.txt @@ -52,12 +52,10 @@ list(REMOVE_ITEM clickhouse_functions_headers IFunction.h FunctionFactory.h Func add_library(clickhouse_functions ${clickhouse_functions_sources}) -target_link_libraries(clickhouse_functions PUBLIC dbms PRIVATE ${CONSISTENT_HASHING_LIBRARY} consistent-hashing-sumbur ${FARMHASH_LIBRARIES} ${METROHASH_LIBRARIES} murmurhash base64) +target_link_libraries(clickhouse_functions PUBLIC dbms PRIVATE ${CONSISTENT_HASHING_LIBRARY} consistent-hashing-sumbur ${FARMHASH_LIBRARIES} ${METROHASH_LIBRARIES} murmurhash ${BASE64_LIBRARY}) target_include_directories (clickhouse_functions SYSTEM BEFORE PUBLIC ${DIVIDE_INCLUDE_DIR}) -target_include_directories (clickhouse_functions SYSTEM PRIVATE ${ClickHouse_SOURCE_DIR}/contrib/base64/include) - if (CONSISTENT_HASHING_INCLUDE_DIR) target_include_directories (clickhouse_functions PRIVATE ${CONSISTENT_HASHING_INCLUDE_DIR}) endif () @@ -83,3 +81,7 @@ endif () if (USE_EMBEDDED_COMPILER) target_include_directories (clickhouse_functions SYSTEM BEFORE PUBLIC ${LLVM_INCLUDE_DIRS}) endif () + +if (USE_BASE64) + target_include_directories (clickhouse_functions SYSTEM PRIVATE ${BASE64_INCLUDE_DIR}) +endif() \ No newline at end of file diff --git a/dbms/src/Functions/FunctionBase64Conversion.h b/dbms/src/Functions/FunctionBase64Conversion.h index 146814d9ea2..25dabff7af7 100644 --- a/dbms/src/Functions/FunctionBase64Conversion.h +++ b/dbms/src/Functions/FunctionBase64Conversion.h @@ -5,6 +5,9 @@ #include #include #include +#include + +#if USE_BASE64 #include @@ -54,14 +57,18 @@ public: return 1; } - bool useDefaultImplementationForConstants() const override { return true; } + bool useDefaultImplementationForConstants() const override + { + return true; + } DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override { size_t number_of_arguments = arguments.size(); if (number_of_arguments != 1) - throw Exception("Function " + getName() + " needs exactly one argument of type String", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); + throw Exception( + "Function " + getName() + " needs exactly one argument of type String", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); if (!WhichDataType(arguments[0].type).isString()) throw Exception( @@ -105,7 +112,7 @@ public: if constexpr (std::is_same_v) { - base64_encode(source, srclen , dst_pos, &outlen, codec); + base64_encode(source, srclen, dst_pos, &outlen, codec); } else { @@ -137,8 +144,7 @@ private: #else return BASE64_FORCE_PLAIN; #endif - } }; - } +#endif \ No newline at end of file diff --git a/dbms/src/Functions/base64Decode.cpp b/dbms/src/Functions/base64Decode.cpp index 5eb107a59b0..ee6ceadebad 100644 --- a/dbms/src/Functions/base64Decode.cpp +++ b/dbms/src/Functions/base64Decode.cpp @@ -1,6 +1,7 @@ +#include +#if USE_BASE64 #include #include -#include namespace DB { @@ -9,4 +10,5 @@ void registerFunctionBase64Decode(FunctionFactory &factory) { factory.registerFunction>(); } -} \ No newline at end of file +} +#endif \ No newline at end of file diff --git a/dbms/src/Functions/base64Encode.cpp b/dbms/src/Functions/base64Encode.cpp index cca00f1fed5..5bf7dd19693 100644 --- a/dbms/src/Functions/base64Encode.cpp +++ b/dbms/src/Functions/base64Encode.cpp @@ -1,6 +1,7 @@ +#include +#if USE_BASE64 #include #include -#include namespace DB { @@ -9,4 +10,5 @@ void registerFunctionBase64Encode(FunctionFactory &factory) { factory.registerFunction>(); } -} \ No newline at end of file +} +#endif \ No newline at end of file diff --git a/dbms/src/Functions/registerFunctionsString.cpp b/dbms/src/Functions/registerFunctionsString.cpp index 998bc948409..48e1cf89f9f 100644 --- a/dbms/src/Functions/registerFunctionsString.cpp +++ b/dbms/src/Functions/registerFunctionsString.cpp @@ -19,8 +19,10 @@ void registerFunctionSubstringUTF8(FunctionFactory &); void registerFunctionAppendTrailingCharIfAbsent(FunctionFactory &); void registerFunctionStartsWith(FunctionFactory &); void registerFunctionEndsWith(FunctionFactory &); +#if USE_BASE64 void registerFunctionBase64Encode(FunctionFactory &); void registerFunctionBase64Decode(FunctionFactory &); +#endif void registerFunctionsString(FunctionFactory & factory) { @@ -40,8 +42,10 @@ void registerFunctionsString(FunctionFactory & factory) registerFunctionAppendTrailingCharIfAbsent(factory); registerFunctionStartsWith(factory); registerFunctionEndsWith(factory); +#if USE_BASE64 registerFunctionBase64Encode(factory); registerFunctionBase64Decode(factory); +#endif } } From c1b8f7cc3106bbc039330343a4822963d11f0a9d Mon Sep 17 00:00:00 2001 From: Alexandr Krasheninnikov Date: Thu, 11 Oct 2018 19:37:09 +0300 Subject: [PATCH 005/124] Change repository clone method --- .gitmodules | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.gitmodules b/.gitmodules index be97e7c4672..a9e50ab8f6d 100644 --- a/.gitmodules +++ b/.gitmodules @@ -48,4 +48,4 @@ url = https://github.com/ClickHouse-Extras/UnixODBC.git [submodule "contrib/base64"] path = contrib/base64 - url = git@github.com:aklomp/base64.git + url = https://github.com/aklomp/base64.git From 468ca70683d4a0d5ef9653fa184d18721b8ce5ff Mon Sep 17 00:00:00 2001 From: Alexandr Krasheninnikov Date: Fri, 12 Oct 2018 11:46:53 +0300 Subject: [PATCH 006/124] Minor review fixes --- dbms/src/Functions/FunctionBase64Conversion.h | 11 ++--------- dbms/src/Functions/base64Decode.cpp | 2 +- dbms/src/Functions/base64Encode.cpp | 2 +- 3 files changed, 4 insertions(+), 11 deletions(-) diff --git a/dbms/src/Functions/FunctionBase64Conversion.h b/dbms/src/Functions/FunctionBase64Conversion.h index 25dabff7af7..fe7ea7cb206 100644 --- a/dbms/src/Functions/FunctionBase64Conversion.h +++ b/dbms/src/Functions/FunctionBase64Conversion.h @@ -1,3 +1,5 @@ +#include +#if USE_BASE64 #include #include #include @@ -5,9 +7,6 @@ #include #include #include -#include - -#if USE_BASE64 #include @@ -64,12 +63,6 @@ public: DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override { - size_t number_of_arguments = arguments.size(); - - if (number_of_arguments != 1) - throw Exception( - "Function " + getName() + " needs exactly one argument of type String", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); - if (!WhichDataType(arguments[0].type).isString()) throw Exception( "Illegal type " + arguments[0].type->getName() + " of 1 argument of function " + getName() + ". Must be String.", diff --git a/dbms/src/Functions/base64Decode.cpp b/dbms/src/Functions/base64Decode.cpp index ee6ceadebad..2adfc04463a 100644 --- a/dbms/src/Functions/base64Decode.cpp +++ b/dbms/src/Functions/base64Decode.cpp @@ -6,7 +6,7 @@ namespace DB { -void registerFunctionBase64Decode(FunctionFactory &factory) +void registerFunctionBase64Decode(FunctionFactory & factory) { factory.registerFunction>(); } diff --git a/dbms/src/Functions/base64Encode.cpp b/dbms/src/Functions/base64Encode.cpp index 5bf7dd19693..38b9cdcddf7 100644 --- a/dbms/src/Functions/base64Encode.cpp +++ b/dbms/src/Functions/base64Encode.cpp @@ -6,7 +6,7 @@ namespace DB { -void registerFunctionBase64Encode(FunctionFactory &factory) +void registerFunctionBase64Encode(FunctionFactory & factory) { factory.registerFunction>(); } From ab050c8466f9104cce0e08f697570cdb9de39b93 Mon Sep 17 00:00:00 2001 From: Sabyanin Maxim Date: Sun, 14 Oct 2018 18:30:06 +0300 Subject: [PATCH 007/124] add comment section in column declaration. add comment column type in alter method. add comment section in system.columns. --- dbms/programs/server/TCPHandler.cpp | 2 +- .../Interpreters/InterpreterCreateQuery.cpp | 29 ++++- .../src/Interpreters/InterpreterCreateQuery.h | 2 +- dbms/src/Parsers/ASTAlterQuery.cpp | 7 + dbms/src/Parsers/ASTAlterQuery.h | 5 + dbms/src/Parsers/ASTColumnDeclaration.h | 7 +- dbms/src/Parsers/ParserAlterQuery.cpp | 11 ++ dbms/src/Parsers/ParserAlterQuery.h | 1 + dbms/src/Parsers/ParserCreateQuery.h | 13 +- dbms/src/Storages/AlterCommands.cpp | 23 ++++ dbms/src/Storages/AlterCommands.h | 6 +- dbms/src/Storages/ColumnComment.cpp | 7 + dbms/src/Storages/ColumnComment.h | 19 +++ dbms/src/Storages/ColumnsDescription.cpp | 121 +++++++++++++----- dbms/src/Storages/ColumnsDescription.h | 9 +- dbms/src/Storages/IStorage.h | 16 ++- dbms/src/Storages/MergeTree/MergeTreeData.cpp | 5 + .../Storages/System/StorageSystemColumns.cpp | 18 +++ .../00725_comment_columns.reference | 28 ++++ .../0_stateless/00725_comment_columns.sql | 60 +++++++++ 20 files changed, 331 insertions(+), 58 deletions(-) create mode 100644 dbms/src/Storages/ColumnComment.cpp create mode 100644 dbms/src/Storages/ColumnComment.h create mode 100644 dbms/tests/queries/0_stateless/00725_comment_columns.reference create mode 100644 dbms/tests/queries/0_stateless/00725_comment_columns.sql diff --git a/dbms/programs/server/TCPHandler.cpp b/dbms/programs/server/TCPHandler.cpp index da2a5bbea2b..0005d1433ab 100644 --- a/dbms/programs/server/TCPHandler.cpp +++ b/dbms/programs/server/TCPHandler.cpp @@ -716,7 +716,7 @@ bool TCPHandler::receiveData() { NamesAndTypesList columns = block.getNamesAndTypesList(); storage = StorageMemory::create(external_table_name, - ColumnsDescription{columns, NamesAndTypesList{}, NamesAndTypesList{}, ColumnDefaults{}}); + ColumnsDescription{columns, NamesAndTypesList{}, NamesAndTypesList{}, ColumnDefaults{}, ColumnComments{}}); storage->startup(); query_context.addExternalTable(external_table_name, storage); } diff --git a/dbms/src/Interpreters/InterpreterCreateQuery.cpp b/dbms/src/Interpreters/InterpreterCreateQuery.cpp index 337ad39abef..2f8716b87a9 100644 --- a/dbms/src/Interpreters/InterpreterCreateQuery.cpp +++ b/dbms/src/Interpreters/InterpreterCreateQuery.cpp @@ -166,13 +166,15 @@ BlockIO InterpreterCreateQuery::createDatabase(ASTCreateQuery & create) using ColumnsAndDefaults = std::pair; +using ParsedColumns = std::tuple; /// AST to the list of columns with types. Columns of Nested type are expanded into a list of real columns. -static ColumnsAndDefaults parseColumns(const ASTExpressionList & column_list_ast, const Context & context) +static ParsedColumns parseColumns(const ASTExpressionList & column_list_ast, const Context & context) { /// list of table columns in correct order NamesAndTypesList columns{}; ColumnDefaults defaults{}; + ColumnComments comments{}; /// Columns requiring type-deduction or default_expression type-check std::vector> defaulted_columns{}; @@ -216,6 +218,11 @@ static ColumnsAndDefaults parseColumns(const ASTExpressionList & column_list_ast else default_expr_list->children.emplace_back(setAlias(col_decl.default_expression->clone(), col_decl.name)); } + + if (col_decl.comment_expression) + { + comments.emplace(col_decl.name, ColumnComment{col_decl.comment_expression}); + } } /// set missing types and wrap default_expression's in a conversion-function if necessary @@ -261,7 +268,7 @@ static ColumnsAndDefaults parseColumns(const ASTExpressionList & column_list_ast } } - return {Nested::flatten(columns), defaults}; + return {Nested::flatten(columns), defaults, comments}; } @@ -329,11 +336,17 @@ ASTPtr InterpreterCreateQuery::formatColumns(const ColumnsDescription & columns) column_declaration->type = parseQuery(storage_p, pos, end, "data type", 0); column_declaration->type->owned_string = type_name; - const auto it = columns.defaults.find(column.name); - if (it != std::end(columns.defaults)) + const auto defaults_it = columns.defaults.find(column.name); + if (defaults_it != std::end(columns.defaults)) { - column_declaration->default_specifier = toString(it->second.kind); - column_declaration->default_expression = it->second.expression->clone(); + column_declaration->default_specifier = toString(defaults_it->second.kind); + column_declaration->default_expression = defaults_it->second.expression->clone(); + } + + const auto comments_it = columns.comments.find(column.name); + if (comments_it != std::end(columns.comments)) + { + column_declaration->comment_expression = comments_it->second.expression->clone(); } columns_list->children.push_back(column_declaration_ptr); @@ -347,11 +360,13 @@ ColumnsDescription InterpreterCreateQuery::getColumnsDescription(const ASTExpres { ColumnsDescription res; - auto && columns_and_defaults = parseColumns(columns, context); + auto && parsed_columns = parseColumns(columns, context); + auto columns_and_defaults = std::make_pair(std::move(std::get<0>(parsed_columns)), std::move(std::get<1>(parsed_columns))); res.materialized = removeAndReturnColumns(columns_and_defaults, ColumnDefaultKind::Materialized); res.aliases = removeAndReturnColumns(columns_and_defaults, ColumnDefaultKind::Alias); res.ordinary = std::move(columns_and_defaults.first); res.defaults = std::move(columns_and_defaults.second); + res.comments = std::move(std::get<2>(parsed_columns)); if (res.ordinary.size() + res.materialized.size() == 0) throw Exception{"Cannot CREATE table without physical columns", ErrorCodes::EMPTY_LIST_OF_COLUMNS_PASSED}; diff --git a/dbms/src/Interpreters/InterpreterCreateQuery.h b/dbms/src/Interpreters/InterpreterCreateQuery.h index 2f186764866..e450ae0728e 100644 --- a/dbms/src/Interpreters/InterpreterCreateQuery.h +++ b/dbms/src/Interpreters/InterpreterCreateQuery.h @@ -44,7 +44,7 @@ public: internal = internal_; } - /// Obtain information about columns, their types and default values, for case when columns in CREATE query is specified explicitly. + /// Obtain information about columns, their types, default values and column comments, for case when columns in CREATE query is specified explicitly. static ColumnsDescription getColumnsDescription(const ASTExpressionList & columns, const Context & context); /// Check that column types are allowed for usage in table according to settings. static void checkSupportedTypes(const ColumnsDescription & columns, const Context & context); diff --git a/dbms/src/Parsers/ASTAlterQuery.cpp b/dbms/src/Parsers/ASTAlterQuery.cpp index 7081b512247..827906a4294 100644 --- a/dbms/src/Parsers/ASTAlterQuery.cpp +++ b/dbms/src/Parsers/ASTAlterQuery.cpp @@ -141,6 +141,13 @@ void ASTAlterCommand::formatImpl( settings.ostr << (settings.hilite ? hilite_keyword : "") << " WHERE " << (settings.hilite ? hilite_none : ""); predicate->formatImpl(settings, state, frame); } + else if (type == ASTAlterCommand::COMMENT_COLUMN) + { + settings.ostr << (settings.hilite ? hilite_keyword : "") << indent_str << "COMMENT COLUMN " << (settings.hilite ? hilite_none : ""); + column->formatImpl(settings, state, frame); + settings.ostr << " "; + comment->formatImpl(settings, state, frame); + } else throw Exception("Unexpected type of ALTER", ErrorCodes::UNEXPECTED_AST_STRUCTURE); } diff --git a/dbms/src/Parsers/ASTAlterQuery.h b/dbms/src/Parsers/ASTAlterQuery.h index c79f9ba8b2f..949d4ecf851 100644 --- a/dbms/src/Parsers/ASTAlterQuery.h +++ b/dbms/src/Parsers/ASTAlterQuery.h @@ -14,6 +14,7 @@ namespace DB * DROP COLUMN col_drop [FROM PARTITION partition], * MODIFY COLUMN col_name type, * DROP PARTITION partition, + * COMMENT_COLUMN col_name 'comment', */ class ASTAlterCommand : public IAST @@ -25,6 +26,7 @@ public: DROP_COLUMN, MODIFY_COLUMN, MODIFY_PRIMARY_KEY, + COMMENT_COLUMN, DROP_PARTITION, ATTACH_PARTITION, @@ -66,6 +68,9 @@ public: /// A list of expressions of the form `column = expr` for the UPDATE command. ASTPtr update_assignments; + /// A column comment + ASTPtr comment; + bool detach = false; /// true for DETACH PARTITION bool part = false; /// true for ATTACH PART diff --git a/dbms/src/Parsers/ASTColumnDeclaration.h b/dbms/src/Parsers/ASTColumnDeclaration.h index 845fe175155..ea078ad0052 100644 --- a/dbms/src/Parsers/ASTColumnDeclaration.h +++ b/dbms/src/Parsers/ASTColumnDeclaration.h @@ -5,7 +5,7 @@ namespace DB { -/** Name, type, default-specifier, default-expression. +/** Name, type, default-specifier, default-expression, comment-expression. * The type is optional if default-expression is specified. */ class ASTColumnDeclaration : public IAST @@ -38,13 +38,12 @@ public: if (comment_expression) { res->comment_expression = comment_expression->clone(); - res->children.push_back(res->comment_expression); // TODO: понять, зачем это нужно. + res->children.push_back(res->comment_expression); } return res; } -protected: void formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override { frame.need_parens = false; @@ -63,8 +62,8 @@ protected: default_expression->formatImpl(settings, state, frame); } - // TODO: понять, почему не отрицание if (comment_expression) { + settings.ostr << ' ' << (settings.hilite ? hilite_keyword : "") << "COMMENT" << (settings.hilite ? hilite_none : "") << ' '; comment_expression->formatImpl(settings, state, frame); } } diff --git a/dbms/src/Parsers/ParserAlterQuery.cpp b/dbms/src/Parsers/ParserAlterQuery.cpp index 56eaddb38ee..65446e5a4db 100644 --- a/dbms/src/Parsers/ParserAlterQuery.cpp +++ b/dbms/src/Parsers/ParserAlterQuery.cpp @@ -24,6 +24,7 @@ bool ParserAlterCommand::parseImpl(Pos & pos, ASTPtr & node, Expected & expected ParserKeyword s_clear_column("CLEAR COLUMN"); ParserKeyword s_modify_column("MODIFY COLUMN"); ParserKeyword s_modify_primary_key("MODIFY PRIMARY KEY"); + ParserKeyword s_comment_column("COMMENT COLUMN"); ParserKeyword s_attach_partition("ATTACH PARTITION"); ParserKeyword s_detach_partition("DETACH PARTITION"); @@ -220,6 +221,16 @@ bool ParserAlterCommand::parseImpl(Pos & pos, ASTPtr & node, Expected & expected command->type = ASTAlterCommand::UPDATE; } + else if (s_comment_column.ignore(pos, expected)) + { + if (!parser_name.parse(pos, command->column, expected)) + return false; + + if (!parser_string_literal.parse(pos, command->comment, expected)) + return false; + + command->type = ASTAlterCommand::COMMENT_COLUMN; + } else return false; diff --git a/dbms/src/Parsers/ParserAlterQuery.h b/dbms/src/Parsers/ParserAlterQuery.h index 46908ae135d..c7b21ca15d3 100644 --- a/dbms/src/Parsers/ParserAlterQuery.h +++ b/dbms/src/Parsers/ParserAlterQuery.h @@ -13,6 +13,7 @@ namespace DB * [CLEAR COLUMN col_to_clear [IN PARTITION partition],] * [MODIFY COLUMN col_to_modify type, ...] * [MODIFY PRIMARY KEY (a, b, c...)] + * [COMMENT COLUMN col_name string] * [DROP|DETACH|ATTACH PARTITION|PART partition, ...] * [FETCH PARTITION partition FROM ...] * [FREEZE PARTITION] diff --git a/dbms/src/Parsers/ParserCreateQuery.h b/dbms/src/Parsers/ParserCreateQuery.h index 72286c155e4..3931b228c29 100644 --- a/dbms/src/Parsers/ParserCreateQuery.h +++ b/dbms/src/Parsers/ParserCreateQuery.h @@ -113,6 +113,7 @@ bool IParserColumnDeclaration::parseImpl(Pos & pos, ASTPtr & node, E ParserKeyword s_alias{"ALIAS"}; ParserKeyword s_comment{"COMMENT"}; ParserTernaryOperatorExpression expr_parser; + ParserStringLiteral string_literal_parser; /// mandatory column name ASTPtr name; @@ -120,14 +121,13 @@ bool IParserColumnDeclaration::parseImpl(Pos & pos, ASTPtr & node, E return false; /** column name should be followed by type name if it - * is not immediately followed by {DEFAULT, MATERIALIZED, ALIAS, COMMENT} + * is not immediately followed by {DEFAULT, MATERIALIZED, ALIAS} */ ASTPtr type; const auto fallback_pos = pos; if (!s_default.check(pos, expected) && !s_materialized.check(pos, expected) && - !s_alias.check(pos, expected) && - !s_comment.check(pos, expected)) + !s_alias.check(pos, expected)) { type_parser.parse(pos, type, expected); } @@ -151,15 +151,10 @@ bool IParserColumnDeclaration::parseImpl(Pos & pos, ASTPtr & node, E else if (!type) return false; /// reject sole column name without type - String comment_specifier; ASTPtr comment_expression; - pos_before_specifier = pos; if (s_comment.ignore(pos, expected)) { - comment_specifier = Poco::toUpper(std::string{pos_before_specifier->begin, pos_specifier->end}); - if (!expr_parser.parse(pos, comment_expression, expected)) { - return false; - } + string_literal_parser.parse(pos, comment_expression, expected); } const auto column_declaration = std::make_shared(); diff --git a/dbms/src/Storages/AlterCommands.cpp b/dbms/src/Storages/AlterCommands.cpp index 9e6d525f685..5b9b6f3b0db 100644 --- a/dbms/src/Storages/AlterCommands.cpp +++ b/dbms/src/Storages/AlterCommands.cpp @@ -90,6 +90,15 @@ std::optional AlterCommand::parse(const ASTAlterCommand * command_ command.primary_key = command_ast->primary_key; return command; } + else if (command_ast->type == ASTAlterCommand::COMMENT_COLUMN) + { + AlterCommand command; + command.type = COMMENT_COLUMN; + const auto & ast_identifier = typeid_cast(*command_ast->column); + command.column_name = ast_identifier.name; + command.comment_expression = command_ast->comment; + return command; + } else return {}; } @@ -237,6 +246,11 @@ void AlterCommand::apply(ColumnsDescription & columns_description) const /// This have no relation to changing the list of columns. /// TODO Check that all columns exist, that only columns with constant defaults are added. } + else if (type == COMMENT_COLUMN) + { + + columns_description.comments[column_name].expression = comment_expression; + } else throw Exception("Wrong parameter type in ALTER query", ErrorCodes::LOGICAL_ERROR); } @@ -353,6 +367,15 @@ void AlterCommands::validate(const IStorage & table, const Context & context) throw Exception("Wrong column name. Cannot find column " + command.column_name + " to drop", ErrorCodes::ILLEGAL_COLUMN); } + else if (command.type == AlterCommand::COMMENT_COLUMN) + { + const auto column_it = std::find_if(std::begin(all_columns), std::end(all_columns), + std::bind(namesEqual, std::cref(command.column_name), std::placeholders::_1)); + if (column_it == std::end(all_columns)) + { + throw Exception{"Wrong column name. Cannot find column " + command.column_name + " to comment", ErrorCodes::ILLEGAL_COLUMN}; + } + } } /** Existing defaulted columns may require default expression extensions with a type conversion, diff --git a/dbms/src/Storages/AlterCommands.h b/dbms/src/Storages/AlterCommands.h index 888bd64f03e..1bbf3ddbee0 100644 --- a/dbms/src/Storages/AlterCommands.h +++ b/dbms/src/Storages/AlterCommands.h @@ -21,6 +21,7 @@ struct AlterCommand DROP_COLUMN, MODIFY_COLUMN, MODIFY_PRIMARY_KEY, + COMMENT_COLUMN, }; Type type; @@ -35,6 +36,7 @@ struct AlterCommand ColumnDefaultKind default_kind{}; ASTPtr default_expression{}; + ASTPtr comment_expression; /// For ADD - after which column to add a new one. If an empty string, add to the end. To add to the beginning now it is impossible. String after_column; @@ -45,9 +47,9 @@ struct AlterCommand AlterCommand() = default; AlterCommand(const Type type, const String & column_name, const DataTypePtr & data_type, const ColumnDefaultKind default_kind, const ASTPtr & default_expression, - const String & after_column = String{}) + const String & after_column = String{}, const ASTPtr & comment_expression = nullptr) : type{type}, column_name{column_name}, data_type{data_type}, default_kind{default_kind}, - default_expression{default_expression}, after_column{after_column} + default_expression{default_expression}, comment_expression(comment_expression), after_column{after_column} {} static std::optional parse(const ASTAlterCommand * command); diff --git a/dbms/src/Storages/ColumnComment.cpp b/dbms/src/Storages/ColumnComment.cpp new file mode 100644 index 00000000000..076ec1e19df --- /dev/null +++ b/dbms/src/Storages/ColumnComment.cpp @@ -0,0 +1,7 @@ +#include +#include + +bool DB::operator== (const DB::ColumnComment& lhs, const DB::ColumnComment& rhs) +{ + return queryToString(lhs.expression) == queryToString(rhs.expression); +} diff --git a/dbms/src/Storages/ColumnComment.h b/dbms/src/Storages/ColumnComment.h new file mode 100644 index 00000000000..33e44fb5188 --- /dev/null +++ b/dbms/src/Storages/ColumnComment.h @@ -0,0 +1,19 @@ +#pragma once + +#include +#include + +#include + +namespace DB +{ + +struct ColumnComment { + ASTPtr expression; +}; + +bool operator== (const ColumnComment& lhs, const ColumnComment& rhs); + +using ColumnComments = std::unordered_map; + +} diff --git a/dbms/src/Storages/ColumnsDescription.cpp b/dbms/src/Storages/ColumnsDescription.cpp index cb67d01a4ea..e3e76f6012a 100644 --- a/dbms/src/Storages/ColumnsDescription.cpp +++ b/dbms/src/Storages/ColumnsDescription.cpp @@ -75,22 +75,32 @@ String ColumnsDescription::toString() const { for (const auto & column : columns) { - const auto it = defaults.find(column.name); + const auto defaults_it = defaults.find(column.name); + const auto comments_it = comments.find(column.name); writeBackQuotedString(column.name, buf); writeChar(' ', buf); writeText(column.type->getName(), buf); - if (it == std::end(defaults)) - { - writeChar('\n', buf); - continue; - } - else - writeChar('\t', buf); - writeText(DB::toString(it->second.kind), buf); - writeChar('\t', buf); - writeText(queryToString(it->second.expression), buf); + const bool exist_comment = comments_it != std::end(comments) && !comments_it->second.expression; + if (defaults_it != std::end(defaults)) + { + writeChar('\t', buf); + writeText(DB::toString(defaults_it->second.kind), buf); + writeChar('\t', buf); + writeText(queryToString(defaults_it->second.expression), buf); + } + else if (exist_comment) + { + writeChar('\t', buf); + } + + if (exist_comment) + { + writeChar('\t', buf); + writeText(queryToString(comments_it->second.expression), buf); + } + writeChar('\n', buf); } }; @@ -102,6 +112,55 @@ String ColumnsDescription::toString() const return buf.str(); } +struct ParsedDefaultInfo +{ + ColumnDefaultKind default_kind; + ASTPtr default_expr_str; +}; + +std::optional parseDefaulfInfo(ReadBufferFromString & buf) +{ + if (*buf.position() == '\n') + { + return {}; + } + + assertChar('\t', buf); + if (*buf.position() == '\t') + { + assertChar('\t', buf); + return {}; + } + + String default_kind_str; + readText(default_kind_str, buf); + const auto default_kind = columnDefaultKindFromString(default_kind_str); + assertChar('\t', buf); + + ParserExpression expr_parser; + String default_expr_str; + readText(default_expr_str, buf); + const char * begin = default_expr_str.data(); + const auto end = begin + default_expr_str.size(); + ASTPtr default_expr = parseQuery(expr_parser, begin, end, "default_expression", 0); + return ParsedDefaultInfo{default_kind, std::move(default_expr)}; +} + +ASTPtr parseCommentExpr(ReadBufferFromString& buf) +{ + if (*buf.position() == '\n') + { + return {}; + } + + ParserExpression parser_expr; + String comment_expr_str; + readText(comment_expr_str, buf); + const char * begin = comment_expr_str.data(); + const auto end = begin + comment_expr_str.size(); + ASTPtr comment_expr = parseQuery(parser_expr, begin, end, "comment_expression", 0); + return comment_expr; +} ColumnsDescription ColumnsDescription::parse(const String & str) { @@ -132,29 +191,31 @@ ColumnsDescription ColumnsDescription::parse(const String & str) result.ordinary.emplace_back(column_name, std::move(type)); continue; } + assertChar('\t', buf); - String default_kind_str; - readText(default_kind_str, buf); - const auto default_kind = columnDefaultKindFromString(default_kind_str); - assertChar('\t', buf); + const auto default_info = parseDefaulfInfo(buf); + if (default_info) + { + const auto & default_kind = default_info->default_kind; + const auto & default_expr = default_info->default_expr_str; + if (ColumnDefaultKind::Default == default_kind) + result.ordinary.emplace_back(column_name, std::move(type)); + else if (ColumnDefaultKind::Materialized == default_kind) + result.materialized.emplace_back(column_name, std::move(type)); + else if (ColumnDefaultKind::Alias == default_kind) + result.aliases.emplace_back(column_name, std::move(type)); + + result.defaults.emplace(column_name, ColumnDefault{default_kind, default_expr}); + } + + const auto comment_expr = parseCommentExpr(buf); + if (comment_expr) + { + result.comments.emplace(column_name, ColumnComment{comment_expr}); + } - String default_expr_str; - readText(default_expr_str, buf); assertChar('\n', buf); - - const char * begin = default_expr_str.data(); - const auto end = begin + default_expr_str.size(); - ASTPtr default_expr = parseQuery(expr_parser, begin, end, "default expression", 0); - - if (ColumnDefaultKind::Default == default_kind) - result.ordinary.emplace_back(column_name, std::move(type)); - else if (ColumnDefaultKind::Materialized == default_kind) - result.materialized.emplace_back(column_name, std::move(type)); - else if (ColumnDefaultKind::Alias == default_kind) - result.aliases.emplace_back(column_name, std::move(type)); - - result.defaults.emplace(column_name, ColumnDefault{default_kind, default_expr}); } assertEOF(buf); diff --git a/dbms/src/Storages/ColumnsDescription.h b/dbms/src/Storages/ColumnsDescription.h index 288d2712b3b..69738d83861 100644 --- a/dbms/src/Storages/ColumnsDescription.h +++ b/dbms/src/Storages/ColumnsDescription.h @@ -2,6 +2,7 @@ #include #include +#include #include #include @@ -15,6 +16,7 @@ struct ColumnsDescription NamesAndTypesList materialized; NamesAndTypesList aliases; ColumnDefaults defaults; + ColumnComments comments; ColumnsDescription() = default; @@ -22,11 +24,13 @@ struct ColumnsDescription NamesAndTypesList ordinary_, NamesAndTypesList materialized_, NamesAndTypesList aliases_, - ColumnDefaults defaults_) + ColumnDefaults defaults_, + ColumnComments comments_ = {}) : ordinary(std::move(ordinary_)) , materialized(std::move(materialized_)) , aliases(std::move(aliases_)) , defaults(std::move(defaults_)) + , comments(std::move(comments_)) {} explicit ColumnsDescription(NamesAndTypesList ordinary_) : ordinary(std::move(ordinary_)) {} @@ -36,7 +40,8 @@ struct ColumnsDescription return ordinary == other.ordinary && materialized == other.materialized && aliases == other.aliases - && defaults == other.defaults; + && defaults == other.defaults + && comments == other.comments; } bool operator!=(const ColumnsDescription & other) const { return !(*this == other); } diff --git a/dbms/src/Storages/IStorage.h b/dbms/src/Storages/IStorage.h index d256a5628f7..d889190ce45 100644 --- a/dbms/src/Storages/IStorage.h +++ b/dbms/src/Storages/IStorage.h @@ -4,6 +4,8 @@ #include #include #include +#include +#include #include #include #include @@ -233,9 +235,19 @@ public: * This method must fully execute the ALTER query, taking care of the locks itself. * To update the table metadata on disk, this method should call InterpreterAlterQuery::updateMetadata. */ - virtual void alter(const AlterCommands & /*params*/, const String & /*database_name*/, const String & /*table_name*/, const Context & /*context*/) + virtual void alter(const AlterCommands & params, const String & database_name, const String & table_name, const Context & context) { - throw Exception("Method alter is not supported by storage " + getName(), ErrorCodes::NOT_IMPLEMENTED); + for (const auto & param : params) + { + if (param.type != AlterCommand::Type::COMMENT_COLUMN) + throw Exception("Method alter only supports change comment of column for storage " + getName(), ErrorCodes::NOT_IMPLEMENTED); + } + + auto lock = lockStructureForAlter(__PRETTY_FUNCTION__); + auto new_columns = getColumns(); + params.apply(new_columns); + context.getDatabase(database_name)->alterTable(context, table_name, new_columns, {}); + setColumns(std::move(new_columns)); } /** Execute CLEAR COLUMN ... IN PARTITION query which removes column from given partition. */ diff --git a/dbms/src/Storages/MergeTree/MergeTreeData.cpp b/dbms/src/Storages/MergeTree/MergeTreeData.cpp index 0a55bfa97e5..f8c21a2d675 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeData.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeData.cpp @@ -909,6 +909,11 @@ void MergeTreeData::checkAlter(const AlterCommands & commands) for (const AlterCommand & command : commands) { + if (command.type == AlterCommand::COMMENT_COLUMN) + { + continue; + } + if (columns_alter_forbidden.count(command.column_name)) throw Exception("trying to ALTER key column " + command.column_name, ErrorCodes::ILLEGAL_COLUMN); diff --git a/dbms/src/Storages/System/StorageSystemColumns.cpp b/dbms/src/Storages/System/StorageSystemColumns.cpp index 40802f16466..b546a275f0b 100644 --- a/dbms/src/Storages/System/StorageSystemColumns.cpp +++ b/dbms/src/Storages/System/StorageSystemColumns.cpp @@ -37,6 +37,7 @@ StorageSystemColumns::StorageSystemColumns(const std::string & name_) { "data_compressed_bytes", std::make_shared() }, { "data_uncompressed_bytes", std::make_shared() }, { "marks_bytes", std::make_shared() }, + { "comment", std::make_shared() }, })); } @@ -81,6 +82,7 @@ protected: NamesAndTypesList columns; ColumnDefaults column_defaults; + ColumnComments column_comments; MergeTreeData::ColumnSizeByName column_sizes; { @@ -106,6 +108,7 @@ protected: columns = storage->getColumns().getAll(); column_defaults = storage->getColumns().defaults; + column_comments = storage->getColumns().comments; /** Info about sizes of columns for tables of MergeTree family. * NOTE: It is possible to add getter for this info to IStorage interface. @@ -174,6 +177,21 @@ protected: } } + { + const auto it = column_comments.find(column.name); + if (it == std::end(column_comments)) + { + if (columns_mask[src_index++]) + res_columns[res_index++]->insertDefault(); + } + else + { + const auto & literal = typeid_cast(it->second.expression.get()); + if (columns_mask[src_index++]) + res_columns[res_index++]->insert(literal->value.get()); + } + } + ++rows_count; } } diff --git a/dbms/tests/queries/0_stateless/00725_comment_columns.reference b/dbms/tests/queries/0_stateless/00725_comment_columns.reference new file mode 100644 index 00000000000..8d7837d8a31 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00725_comment_columns.reference @@ -0,0 +1,28 @@ +CREATE TABLE default.check_query_comment_column ( first_column UInt8 DEFAULT 1 COMMENT \'first comment\', fourth_column UInt8 COMMENT \'fourth comment\', fifth_column UInt8, second_column UInt8 MATERIALIZED first_column COMMENT \'second comment\', third_column UInt8 ALIAS second_column COMMENT \'third comment\') ENGINE = TinyLog +┌─table──────────────────────┬─name──────────┬─comment────────┐ +│ check_query_comment_column │ first_column │ first comment │ +│ check_query_comment_column │ fourth_column │ fourth comment │ +│ check_query_comment_column │ fifth_column │ │ +│ check_query_comment_column │ second_column │ second comment │ +│ check_query_comment_column │ third_column │ third comment │ +└────────────────────────────┴───────────────┴────────────────┘ +CREATE TABLE default.check_query_comment_column ( first_column UInt8 DEFAULT 1 COMMENT \'another first column\', fourth_column UInt8 COMMENT \'another fourth column\', fifth_column UInt8 COMMENT \'another fifth column\', second_column UInt8 MATERIALIZED first_column COMMENT \'another second column\', third_column UInt8 ALIAS second_column COMMENT \'another third column\') ENGINE = TinyLog +┌─table──────────────────────┬─name──────────┬─comment───────────────┐ +│ check_query_comment_column │ first_column │ another first column │ +│ check_query_comment_column │ fourth_column │ another fourth column │ +│ check_query_comment_column │ fifth_column │ another fifth column │ +│ check_query_comment_column │ second_column │ another second column │ +│ check_query_comment_column │ third_column │ another third column │ +└────────────────────────────┴───────────────┴───────────────────────┘ +CREATE TABLE default.check_query_comment_column ( first_column Date COMMENT \'first comment\', second_column UInt8 COMMENT \'second comment\', third_column UInt8 COMMENT \'third comment\') ENGINE = MergeTree(first_column, (second_column, second_column), 8192) +┌─table──────────────────────┬─name──────────┬─comment────────┐ +│ check_query_comment_column │ first_column │ first comment │ +│ check_query_comment_column │ second_column │ second comment │ +│ check_query_comment_column │ third_column │ third comment │ +└────────────────────────────┴───────────────┴────────────────┘ +CREATE TABLE default.check_query_comment_column ( first_column Date COMMENT \'another first comment\', second_column UInt8 COMMENT \'another second comment\', third_column UInt8 COMMENT \'another third comment\') ENGINE = MergeTree(first_column, (second_column, second_column), 8192) +┌─table──────────────────────┬─name──────────┬─comment────────────────┐ +│ check_query_comment_column │ first_column │ another first comment │ +│ check_query_comment_column │ second_column │ another second comment │ +│ check_query_comment_column │ third_column │ another third comment │ +└────────────────────────────┴───────────────┴────────────────────────┘ diff --git a/dbms/tests/queries/0_stateless/00725_comment_columns.sql b/dbms/tests/queries/0_stateless/00725_comment_columns.sql new file mode 100644 index 00000000000..6c2bd2305b1 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00725_comment_columns.sql @@ -0,0 +1,60 @@ +DROP TABLE IF EXISTS check_query_comment_column; + +CREATE TABLE check_query_comment_column + ( + first_column UInt8 DEFAULT 1 COMMENT 'first comment', + second_column UInt8 MATERIALIZED first_column COMMENT 'second comment', + third_column UInt8 ALIAS second_column COMMENT 'third comment', + fourth_column UInt8 COMMENT 'fourth comment', + fifth_column UInt8 + ) ENGINE = TinyLog; + +SHOW CREATE TABLE check_query_comment_column; + +SELECT table, name, comment +FROM system.columns +WHERE table = 'check_query_comment_column' +FORMAT PrettyCompactNoEscapes; + +ALTER TABLE check_query_comment_column + COMMENT COLUMN first_column 'another first column', + COMMENT COLUMN second_column 'another second column', + COMMENT COLUMN third_column 'another third column', + COMMENT COLUMN fourth_column 'another fourth column', + COMMENT COLUMN fifth_column 'another fifth column'; + +SHOW CREATE TABLE check_query_comment_column; + +SELECT table, name, comment +FROM system.columns +WHERE table = 'check_query_comment_column' +FORMAT PrettyCompactNoEscapes; + +DROP TABLE IF EXISTS check_query_comment_column; + + +CREATE TABLE check_query_comment_column + ( + first_column Date COMMENT 'first comment', + second_column UInt8 COMMENT 'second comment', + third_column UInt8 COMMENT 'third comment' + ) ENGINE = MergeTree(first_column, (second_column, second_column), 8192); + +SHOW CREATE TABLE check_query_comment_column; + +SELECT table, name, comment +FROM system.columns +WHERE table = 'check_query_comment_column' +FORMAT PrettyCompactNoEscapes; + +ALTER TABLE check_query_comment_column + COMMENT COLUMN first_column 'another first comment', + COMMENT COLUMN second_column 'another second comment', + COMMENT COLUMN third_column 'another third comment'; + +SHOW CREATE TABLE check_query_comment_column; + +SELECT table, name, comment +FROM system.columns +WHERE table = 'check_query_comment_column' +FORMAT PrettyCompactNoEscapes; \ No newline at end of file From 6d783ef013cdd1b919f5f30626f0eb2cc35cbbd7 Mon Sep 17 00:00:00 2001 From: Ivan Lezhankin Date: Wed, 17 Oct 2018 14:45:14 +0300 Subject: [PATCH 008/124] Move-away "uniqCombined" as a separate aggregated function with HLL precision --- dbms/scripts/test_uniq_functions.sh | 2 +- .../AggregateFunctionUniq.cpp | 3 - .../AggregateFunctionUniq.h | 88 ---- .../AggregateFunctionUniqCombined.cpp | 90 ++++ .../AggregateFunctionUniqCombined.h | 429 ++++++++++++++++++ .../registerAggregateFunctions.cpp | 2 + .../tests/performance/test_hits/test_hits.xml | 4 +- .../00212_shard_aggregate_function_uniq.sql | 26 +- .../0_stateless/00264_uniq_many_args.sql | 10 +- .../0_stateless/00700_decimal_aggregates.sql | 2 +- .../00146_aggregate_function_uniq.sql | 2 +- .../query_language/agg_functions/reference.md | 2 +- .../query_language/agg_functions/reference.md | 2 +- 13 files changed, 546 insertions(+), 116 deletions(-) create mode 100644 dbms/src/AggregateFunctions/AggregateFunctionUniqCombined.cpp create mode 100644 dbms/src/AggregateFunctions/AggregateFunctionUniqCombined.h diff --git a/dbms/scripts/test_uniq_functions.sh b/dbms/scripts/test_uniq_functions.sh index 9a4b6f20433..f7e2083610b 100755 --- a/dbms/scripts/test_uniq_functions.sh +++ b/dbms/scripts/test_uniq_functions.sh @@ -6,6 +6,6 @@ do do n=$(( 10**p * i )) echo -n "$n " - clickhouse-client -q "select uniqHLL12(number), uniq(number), uniqCombined(number) from numbers($n);" + clickhouse-client -q "select uniqHLL12(number), uniq(number), uniqCombined(15)(number) from numbers($n);" done done diff --git a/dbms/src/AggregateFunctions/AggregateFunctionUniq.cpp b/dbms/src/AggregateFunctions/AggregateFunctionUniq.cpp index 77b6c9cfb97..6b63a719b8f 100644 --- a/dbms/src/AggregateFunctions/AggregateFunctionUniq.cpp +++ b/dbms/src/AggregateFunctions/AggregateFunctionUniq.cpp @@ -130,9 +130,6 @@ void registerAggregateFunctionsUniq(AggregateFunctionFactory & factory) factory.registerFunction("uniqExact", createAggregateFunctionUniq>); - - factory.registerFunction("uniqCombined", - createAggregateFunctionUniq>); } } diff --git a/dbms/src/AggregateFunctions/AggregateFunctionUniq.h b/dbms/src/AggregateFunctions/AggregateFunctionUniq.h index 140928959a3..980d62b40ec 100644 --- a/dbms/src/AggregateFunctions/AggregateFunctionUniq.h +++ b/dbms/src/AggregateFunctions/AggregateFunctionUniq.h @@ -22,7 +22,6 @@ #include #include -#include #include @@ -124,46 +123,6 @@ struct AggregateFunctionUniqExactData static String getName() { return "uniqExact"; } }; -template -struct AggregateFunctionUniqCombinedData -{ - using Key = UInt32; - using Set = CombinedCardinalityEstimator< - Key, - HashSet>, - 16, - 14, - 17, - TrivialHash, - UInt32, - HyperLogLogBiasEstimator, - HyperLogLogMode::FullFeatured>; - - Set set; - - static String getName() { return "uniqCombined"; } -}; - -template <> -struct AggregateFunctionUniqCombinedData -{ - using Key = UInt64; - using Set = CombinedCardinalityEstimator< - Key, - HashSet>, - 16, - 14, - 17, - TrivialHash, - UInt64, - HyperLogLogBiasEstimator, - HyperLogLogMode::FullFeatured>; - - Set set; - - static String getName() { return "uniqCombined"; } -}; - namespace detail { @@ -199,39 +158,6 @@ template <> struct AggregateFunctionUniqTraits } }; -/** Hash function for uniqCombined. - */ -template struct AggregateFunctionUniqCombinedTraits -{ - static UInt32 hash(T x) { return static_cast(intHash64(x)); } -}; - -template <> struct AggregateFunctionUniqCombinedTraits -{ - static UInt32 hash(UInt128 x) - { - return sipHash64(x); - } -}; - -template <> struct AggregateFunctionUniqCombinedTraits -{ - static UInt32 hash(Float32 x) - { - UInt64 res = ext::bit_cast(x); - return static_cast(intHash64(res)); - } -}; - -template <> struct AggregateFunctionUniqCombinedTraits -{ - static UInt32 hash(Float64 x) - { - UInt64 res = ext::bit_cast(x); - return static_cast(intHash64(res)); - } -}; - /** The structure for the delegation work to add one element to the `uniq` aggregate functions. * Used for partial specialization to add strings. @@ -255,19 +181,6 @@ struct OneAdder data.set.insert(CityHash_v1_0_2::CityHash64(value.data, value.size)); } } - else if constexpr (std::is_same_v>) - { - if constexpr (!std::is_same_v) - { - const auto & value = static_cast &>(column).getData()[row_num]; - data.set.insert(AggregateFunctionUniqCombinedTraits::hash(value)); - } - else - { - StringRef value = column.getDataAt(row_num); - data.set.insert(CityHash_v1_0_2::CityHash64(value.data, value.size)); - } - } else if constexpr (std::is_same_v>) { if constexpr (!std::is_same_v) @@ -387,5 +300,4 @@ public: const char * getHeaderFilePath() const override { return __FILE__; } }; - } diff --git a/dbms/src/AggregateFunctions/AggregateFunctionUniqCombined.cpp b/dbms/src/AggregateFunctions/AggregateFunctionUniqCombined.cpp new file mode 100644 index 00000000000..cdaf5b90c70 --- /dev/null +++ b/dbms/src/AggregateFunctions/AggregateFunctionUniqCombined.cpp @@ -0,0 +1,90 @@ +#include + +#include +#include + +#include +#include + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int ILLEGAL_TYPE_OF_ARGUMENT; + extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; + extern const int ARGUMENT_OUT_OF_BOUND; +} + +namespace +{ + +AggregateFunctionPtr createAggregateFunctionUniqCombined( + const std::string & name, const DataTypes & argument_types, const Array & params) +{ + UInt8 precision = 17; /// default value - must correlate with default ctor of |AggregateFunctionUniqCombinedData| + + if (!params.empty()) + { + if (params.size() != 1) + throw Exception( + "Aggregate function " + name + " requires one parameter or less.", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); + + UInt64 precision_param = applyVisitor(FieldVisitorConvertToNumber(), params[0]); + + // This range is hardcoded into |AggregateFunctionUniqCombinedData| + if (precision_param > 20 || precision_param < 12) + throw Exception( + "Parameter for aggregate function " + name + "is out or range: [12, 20].", ErrorCodes::ARGUMENT_OUT_OF_BOUND); + + precision = precision_param; + } + + if (argument_types.empty()) + throw Exception("Incorrect number of arguments for aggregate function " + name, ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); + + /// We use exact hash function if the user wants it; + /// or if the arguments are not contiguous in memory, because only exact hash function have support for this case. + bool use_exact_hash_function = !isAllArgumentsContiguousInMemory(argument_types); + + if (argument_types.size() == 1) + { + const IDataType & argument_type = *argument_types[0]; + + AggregateFunctionPtr res(createWithNumericType(*argument_types[0], precision)); + + WhichDataType which(argument_type); + if (res) + return res; + else if (which.isDate()) + return std::make_shared>(precision); + else if (which.isDateTime()) + return std::make_shared>(precision); + else if (which.isStringOrFixedString()) + return std::make_shared>(precision); + else if (which.isUUID()) + return std::make_shared>(precision); + else if (which.isTuple()) + { + if (use_exact_hash_function) + return std::make_shared>(argument_types, precision); + else + return std::make_shared>(argument_types, precision); + } + } + + /// "Variadic" method also works as a fallback generic case for single argument. + if (use_exact_hash_function) + return std::make_shared>(argument_types, precision); + else + return std::make_shared>(argument_types, precision); +} + +} // namespace + +void registerAggregateFunctionUniqCombined(AggregateFunctionFactory & factory) +{ + factory.registerFunction("uniqCombined", createAggregateFunctionUniqCombined); +} + +} // namespace DB diff --git a/dbms/src/AggregateFunctions/AggregateFunctionUniqCombined.h b/dbms/src/AggregateFunctions/AggregateFunctionUniqCombined.h new file mode 100644 index 00000000000..8d8a7c6745d --- /dev/null +++ b/dbms/src/AggregateFunctions/AggregateFunctionUniqCombined.h @@ -0,0 +1,429 @@ +#pragma once + +#include + +#include +#include +#include + +#include +#include +#include + +#include + +#include +#include +#include +#include + +#include + +namespace DB +{ +namespace detail +{ + /** Hash function for uniqCombined. + */ + template + struct AggregateFunctionUniqCombinedTraits + { + static UInt32 hash(T x) + { + return static_cast(intHash64(x)); + } + }; + + template <> + struct AggregateFunctionUniqCombinedTraits + { + static UInt32 hash(UInt128 x) + { + return sipHash64(x); + } + }; + + template <> + struct AggregateFunctionUniqCombinedTraits + { + static UInt32 hash(Float32 x) + { + UInt64 res = ext::bit_cast(x); + return static_cast(intHash64(res)); + } + }; + + template <> + struct AggregateFunctionUniqCombinedTraits + { + static UInt32 hash(Float64 x) + { + UInt64 res = ext::bit_cast(x); + return static_cast(intHash64(res)); + } + }; + +} // namespace detail + + +template +struct __attribute__((__packed__)) AggregateFunctionUniqCombinedDataWithKey +{ + template + using Set = CombinedCardinalityEstimator>, + 16, + K - 3, + K, + TrivialHash, + Key, + HyperLogLogBiasEstimator, + HyperLogLogMode::FullFeatured>; + + mutable UInt8 inited = 0; + union + { + Set<12> set_12; + Set<13> set_13; + Set<14> set_14; + Set<15> set_15; + Set<16> set_16; + Set<17> set_17; + Set<18> set_18; + Set<19> set_19; + Set<20> set_20; + }; + + AggregateFunctionUniqCombinedDataWithKey() : set_17() {} + + ~AggregateFunctionUniqCombinedDataWithKey() + { + switch (inited) + { + case 12: + set_12.~CombinedCardinalityEstimator(); + break; + case 13: + set_13.~CombinedCardinalityEstimator(); + break; + case 14: + set_14.~CombinedCardinalityEstimator(); + break; + case 15: + set_15.~CombinedCardinalityEstimator(); + break; + case 16: + set_16.~CombinedCardinalityEstimator(); + break; + case 0: + case 17: + set_17.~CombinedCardinalityEstimator(); + break; + case 18: + set_18.~CombinedCardinalityEstimator(); + break; + case 19: + set_19.~CombinedCardinalityEstimator(); + break; + case 20: + set_20.~CombinedCardinalityEstimator(); + break; + } + } + + void init(UInt8 precision) const + { + if (inited || precision == 17) + return; + + // TODO: assert "inited == precision" + + set_17.~CombinedCardinalityEstimator(); + + switch (precision) + { + case 12: + new (&set_12) Set<12>; + break; + case 13: + new (&set_13) Set<13>; + break; + case 14: + new (&set_14) Set<14>; + break; + case 15: + new (&set_15) Set<15>; + break; + case 16: + new (&set_16) Set<16>; + break; + case 18: + new (&set_18) Set<18>; + break; + case 19: + new (&set_19) Set<19>; + break; + case 20: + new (&set_20) Set<20>; + break; + } + inited = precision; + } + +#define SET_METHOD(method) \ + switch (inited) \ + { \ + case 12: \ + set_12.method; \ + break; \ + case 13: \ + set_13.method; \ + break; \ + case 14: \ + set_14.method; \ + break; \ + case 15: \ + set_15.method; \ + break; \ + case 16: \ + set_16.method; \ + break; \ + case 17: \ + set_17.method; \ + break; \ + case 18: \ + set_18.method; \ + break; \ + case 19: \ + set_19.method; \ + break; \ + case 20: \ + set_20.method; \ + break; \ + } + +#define SET_RETURN_METHOD(method) \ + switch (inited) \ + { \ + case 12: \ + return set_12.method; \ + case 13: \ + return set_13.method; \ + case 14: \ + return set_14.method; \ + case 15: \ + return set_15.method; \ + case 16: \ + return set_16.method; \ + case 18: \ + return set_18.method; \ + case 19: \ + return set_19.method; \ + case 20: \ + return set_20.method; \ + case 17: \ + default: \ + return set_17.method; \ + } + + void insert(Key value, UInt8 precision) + { + init(precision); + SET_METHOD(insert(value)); + } + + void merge(const AggregateFunctionUniqCombinedDataWithKey & rhs, UInt8 precision) + { + init(precision); + switch (inited) + { + case 12: + set_12.merge(rhs.set_12); + break; + case 13: + set_13.merge(rhs.set_13); + break; + case 14: + set_14.merge(rhs.set_14); + break; + case 15: + set_15.merge(rhs.set_15); + break; + case 16: + set_16.merge(rhs.set_16); + break; + case 17: + set_17.merge(rhs.set_17); + break; + case 18: + set_18.merge(rhs.set_18); + break; + case 19: + set_19.merge(rhs.set_19); + break; + case 20: + set_20.merge(rhs.set_20); + break; + } + } + + void write(DB::WriteBuffer & out, UInt8 precision) const + { + init(precision); + SET_METHOD(write(out)); + } + + void read(DB::ReadBuffer & in, UInt8 precision) + { + init(precision); + SET_METHOD(read(in)); + } + + UInt32 size(UInt8 precision) const + { + init(precision); + SET_RETURN_METHOD(size()); + } + +#undef SET_METHOD +#undef SET_RETURN_METHOD +}; + + +template +struct __attribute__((__packed__)) AggregateFunctionUniqCombinedData : public AggregateFunctionUniqCombinedDataWithKey +{ +}; + + +template <> +struct __attribute__((__packed__)) AggregateFunctionUniqCombinedData : public AggregateFunctionUniqCombinedDataWithKey +{ +}; + + +template +class AggregateFunctionUniqCombined final + : public IAggregateFunctionDataHelper, AggregateFunctionUniqCombined> +{ +private: + const UInt8 precision; + +public: + explicit AggregateFunctionUniqCombined(UInt8 precision) : precision(precision) {} + + String getName() const override + { + return "uniqCombined"; + } + + DataTypePtr getReturnType() const override + { + return std::make_shared(); + } + + void add(AggregateDataPtr place, const IColumn ** columns, size_t row_num, Arena *) const override + { + if constexpr (!std::is_same_v) + { + const auto & value = static_cast &>(*columns[0]).getData()[row_num]; + this->data(place).insert(detail::AggregateFunctionUniqCombinedTraits::hash(value), precision); + } + else + { + StringRef value = columns[0]->getDataAt(row_num); + this->data(place).insert(CityHash_v1_0_2::CityHash64(value.data, value.size), precision); + } + } + + void merge(AggregateDataPtr place, ConstAggregateDataPtr rhs, Arena *) const override + { + this->data(place).merge(this->data(rhs), precision); + } + + void serialize(ConstAggregateDataPtr place, WriteBuffer & buf) const override + { + this->data(place).write(buf, precision); + } + + void deserialize(AggregateDataPtr place, ReadBuffer & buf, Arena *) const override + { + this->data(place).read(buf, precision); + } + + void insertResultInto(ConstAggregateDataPtr place, IColumn & to) const override + { + static_cast(to).getData().push_back(this->data(place).size(precision)); + } + + const char * getHeaderFilePath() const override + { + return __FILE__; + } +}; + +/** For multiple arguments. To compute, hashes them. + * You can pass multiple arguments as is; You can also pass one argument - a tuple. + * But (for the possibility of efficient implementation), you can not pass several arguments, among which there are tuples. + */ +template +class AggregateFunctionUniqCombinedVariadic final : public IAggregateFunctionDataHelper, + AggregateFunctionUniqCombinedVariadic> +{ +private: + size_t num_args = 0; + UInt8 precision; + +public: + AggregateFunctionUniqCombinedVariadic(const DataTypes & arguments, UInt8 precision) : precision(precision) + { + if (argument_is_tuple) + num_args = typeid_cast(*arguments[0]).getElements().size(); + else + num_args = arguments.size(); + } + + String getName() const override + { + return "uniqCombined"; + } + + DataTypePtr getReturnType() const override + { + return std::make_shared(); + } + + void add(AggregateDataPtr place, const IColumn ** columns, size_t row_num, Arena *) const override + { + this->data(place).insert(typename AggregateFunctionUniqCombinedData::Set<12>::value_type( + UniqVariadicHash::apply(num_args, columns, row_num)), + precision); + } + + void merge(AggregateDataPtr place, ConstAggregateDataPtr rhs, Arena *) const override + { + this->data(place).merge(this->data(rhs), precision); + } + + void serialize(ConstAggregateDataPtr place, WriteBuffer & buf) const override + { + this->data(place).write(buf, precision); + } + + void deserialize(AggregateDataPtr place, ReadBuffer & buf, Arena *) const override + { + this->data(place).read(buf, precision); + } + + void insertResultInto(ConstAggregateDataPtr place, IColumn & to) const override + { + static_cast(to).getData().push_back(this->data(place).size(precision)); + } + + const char * getHeaderFilePath() const override + { + return __FILE__; + } +}; + +} // namespace DB diff --git a/dbms/src/AggregateFunctions/registerAggregateFunctions.cpp b/dbms/src/AggregateFunctions/registerAggregateFunctions.cpp index 3517ad57a73..800beda1d53 100644 --- a/dbms/src/AggregateFunctions/registerAggregateFunctions.cpp +++ b/dbms/src/AggregateFunctions/registerAggregateFunctions.cpp @@ -21,6 +21,7 @@ void registerAggregateFunctionsStatisticsSimple(AggregateFunctionFactory &); void registerAggregateFunctionSum(AggregateFunctionFactory &); void registerAggregateFunctionSumMap(AggregateFunctionFactory &); void registerAggregateFunctionsUniq(AggregateFunctionFactory &); +void registerAggregateFunctionUniqCombined(AggregateFunctionFactory &); void registerAggregateFunctionUniqUpTo(AggregateFunctionFactory &); void registerAggregateFunctionTopK(AggregateFunctionFactory &); void registerAggregateFunctionsBitwise(AggregateFunctionFactory &); @@ -55,6 +56,7 @@ void registerAggregateFunctions() registerAggregateFunctionSum(factory); registerAggregateFunctionSumMap(factory); registerAggregateFunctionsUniq(factory); + registerAggregateFunctionUniqCombined(factory); registerAggregateFunctionUniqUpTo(factory); registerAggregateFunctionTopK(factory); registerAggregateFunctionsBitwise(factory); diff --git a/dbms/tests/performance/test_hits/test_hits.xml b/dbms/tests/performance/test_hits/test_hits.xml index c9e30227ff0..eea308fdd64 100644 --- a/dbms/tests/performance/test_hits/test_hits.xml +++ b/dbms/tests/performance/test_hits/test_hits.xml @@ -86,8 +86,8 @@ PageCharset тоже почти всегда непуст, но его сред SELECT uniq(UserID) FROM test.hits SETTINGS max_threads = 1 SELECT uniq(UserID) FROM test.hits -SELECT uniqCombined(UserID) FROM test.hits SETTINGS max_threads = 1 -SELECT uniqCombined(UserID) FROM test.hits +SELECT uniqCombined(15)(UserID) FROM test.hits SETTINGS max_threads = 1 +SELECT uniqCombined(15)(UserID) FROM test.hits SELECT uniqExact(UserID) FROM test.hits SETTINGS max_threads = 1 SELECT uniqExact(UserID) FROM test.hits diff --git a/dbms/tests/queries/0_stateless/00212_shard_aggregate_function_uniq.sql b/dbms/tests/queries/0_stateless/00212_shard_aggregate_function_uniq.sql index 53b5ec0001b..b3d82b71685 100644 --- a/dbms/tests/queries/0_stateless/00212_shard_aggregate_function_uniq.sql +++ b/dbms/tests/queries/0_stateless/00212_shard_aggregate_function_uniq.sql @@ -20,20 +20,20 @@ SELECT uniqHLL12(dummy) FROM remote('127.0.0.{2,3}', system.one); /* uniqCombined */ -SELECT Y, uniqCombined(X) FROM (SELECT number AS X, (3*X*X - 7*X + 11) % 37 AS Y FROM system.numbers LIMIT 15) GROUP BY Y; -SELECT Y, uniqCombined(X) FROM (SELECT number AS X, (3*X*X - 7*X + 11) % 37 AS Y FROM system.numbers LIMIT 3000) GROUP BY Y; -SELECT Y, uniqCombined(X) FROM (SELECT number AS X, (3*X*X - 7*X + 11) % 37 AS Y FROM system.numbers LIMIT 1000000) GROUP BY Y; +SELECT Y, uniqCombined(15)(X) FROM (SELECT number AS X, (3*X*X - 7*X + 11) % 37 AS Y FROM system.numbers LIMIT 15) GROUP BY Y; +SELECT Y, uniqCombined(15)(X) FROM (SELECT number AS X, (3*X*X - 7*X + 11) % 37 AS Y FROM system.numbers LIMIT 3000) GROUP BY Y; +SELECT Y, uniqCombined(15)(X) FROM (SELECT number AS X, (3*X*X - 7*X + 11) % 37 AS Y FROM system.numbers LIMIT 1000000) GROUP BY Y; -SELECT Y, uniqCombined(X) FROM (SELECT number AS X, round(1/(1 + (3*X*X - 7*X + 11) % 37), 3) AS Y FROM system.numbers LIMIT 15) GROUP BY Y; -SELECT Y, uniqCombined(X) FROM (SELECT number AS X, round(1/(1 + (3*X*X - 7*X + 11) % 37), 3) AS Y FROM system.numbers LIMIT 3000) GROUP BY Y; -SELECT Y, uniqCombined(X) FROM (SELECT number AS X, round(1/(1 + (3*X*X - 7*X + 11) % 37), 3) AS Y FROM system.numbers LIMIT 1000000) GROUP BY Y; +SELECT Y, uniqCombined(15)(X) FROM (SELECT number AS X, round(1/(1 + (3*X*X - 7*X + 11) % 37), 3) AS Y FROM system.numbers LIMIT 15) GROUP BY Y; +SELECT Y, uniqCombined(15)(X) FROM (SELECT number AS X, round(1/(1 + (3*X*X - 7*X + 11) % 37), 3) AS Y FROM system.numbers LIMIT 3000) GROUP BY Y; +SELECT Y, uniqCombined(15)(X) FROM (SELECT number AS X, round(1/(1 + (3*X*X - 7*X + 11) % 37), 3) AS Y FROM system.numbers LIMIT 1000000) GROUP BY Y; -SELECT Y, uniqCombined(X) FROM (SELECT number AS X, round(toFloat32(1/(1 + (3*X*X - 7*X + 11) % 37)), 3) AS Y FROM system.numbers LIMIT 15) GROUP BY Y; -SELECT Y, uniqCombined(X) FROM (SELECT number AS X, round(toFloat32(1/(1 + (3*X*X - 7*X + 11) % 37)), 3) AS Y FROM system.numbers LIMIT 3000) GROUP BY Y; -SELECT Y, uniqCombined(X) FROM (SELECT number AS X, round(toFloat32(1/(1 + (3*X*X - 7*X + 11) % 37)), 3) AS Y FROM system.numbers LIMIT 1000000) GROUP BY Y; +SELECT Y, uniqCombined(15)(X) FROM (SELECT number AS X, round(toFloat32(1/(1 + (3*X*X - 7*X + 11) % 37)), 3) AS Y FROM system.numbers LIMIT 15) GROUP BY Y; +SELECT Y, uniqCombined(15)(X) FROM (SELECT number AS X, round(toFloat32(1/(1 + (3*X*X - 7*X + 11) % 37)), 3) AS Y FROM system.numbers LIMIT 3000) GROUP BY Y; +SELECT Y, uniqCombined(15)(X) FROM (SELECT number AS X, round(toFloat32(1/(1 + (3*X*X - 7*X + 11) % 37)), 3) AS Y FROM system.numbers LIMIT 1000000) GROUP BY Y; -SELECT Y, uniqCombined(Z) FROM (SELECT number AS X, IPv4NumToString(toUInt32(X)) AS Z, (3*X*X - 7*X + 11) % 37 AS Y FROM system.numbers LIMIT 15) GROUP BY Y; -SELECT Y, uniqCombined(Z) FROM (SELECT number AS X, IPv4NumToString(toUInt32(X)) AS Z, (3*X*X - 7*X + 11) % 37 AS Y FROM system.numbers LIMIT 3000) GROUP BY Y; -SELECT Y, uniqCombined(Z) FROM (SELECT number AS X, IPv4NumToString(toUInt32(X)) AS Z, (3*X*X - 7*X + 11) % 37 AS Y FROM system.numbers LIMIT 1000000) GROUP BY Y; +SELECT Y, uniqCombined(15)(Z) FROM (SELECT number AS X, IPv4NumToString(toUInt32(X)) AS Z, (3*X*X - 7*X + 11) % 37 AS Y FROM system.numbers LIMIT 15) GROUP BY Y; +SELECT Y, uniqCombined(15)(Z) FROM (SELECT number AS X, IPv4NumToString(toUInt32(X)) AS Z, (3*X*X - 7*X + 11) % 37 AS Y FROM system.numbers LIMIT 3000) GROUP BY Y; +SELECT Y, uniqCombined(15)(Z) FROM (SELECT number AS X, IPv4NumToString(toUInt32(X)) AS Z, (3*X*X - 7*X + 11) % 37 AS Y FROM system.numbers LIMIT 1000000) GROUP BY Y; -SELECT uniqCombined(dummy) FROM remote('127.0.0.{2,3}', system.one); +SELECT uniqCombined(15)(dummy) FROM remote('127.0.0.{2,3}', system.one); diff --git a/dbms/tests/queries/0_stateless/00264_uniq_many_args.sql b/dbms/tests/queries/0_stateless/00264_uniq_many_args.sql index 2b24e68910c..a94ec6e8f0a 100644 --- a/dbms/tests/queries/0_stateless/00264_uniq_many_args.sql +++ b/dbms/tests/queries/0_stateless/00264_uniq_many_args.sql @@ -1,12 +1,12 @@ -SELECT +SELECT uniq(x), uniq((x)), uniq(x, y), uniq((x, y)), uniq(x, y, z), uniq((x, y, z)), - uniqCombined(x), uniqCombined((x)), uniqCombined(x, y), uniqCombined((x, y)), uniqCombined(x, y, z), uniqCombined((x, y, z)), + uniqCombined(15)(x), uniqCombined(15)((x)), uniqCombined(15)(x, y), uniqCombined(15)((x, y)), uniqCombined(15)(x, y, z), uniqCombined(15)((x, y, z)), uniqHLL12(x), uniqHLL12((x)), uniqHLL12(x, y), uniqHLL12((x, y)), uniqHLL12(x, y, z), uniqHLL12((x, y, z)), uniqExact(x), uniqExact((x)), uniqExact(x, y), uniqExact((x, y)), uniqExact(x, y, z), uniqExact((x, y, z)), uniqUpTo(5)(x), uniqUpTo(5)((x)), uniqUpTo(5)(x, y), uniqUpTo(5)((x, y)), uniqUpTo(5)(x, y, z), uniqUpTo(5)((x, y, z)) FROM ( - SELECT + SELECT number % 10 AS x, intDiv(number, 10) % 10 AS y, toString(intDiv(number, 100) % 10) AS z @@ -16,14 +16,14 @@ FROM SELECT k, uniq(x), uniq((x)), uniq(x, y), uniq((x, y)), uniq(x, y, z), uniq((x, y, z)), - uniqCombined(x), uniqCombined((x)), uniqCombined(x, y), uniqCombined((x, y)), uniqCombined(x, y, z), uniqCombined((x, y, z)), + uniqCombined(15)(x), uniqCombined(15)((x)), uniqCombined(15)(x, y), uniqCombined(15)((x, y)), uniqCombined(15)(x, y, z), uniqCombined(15)((x, y, z)), uniqHLL12(x), uniqHLL12((x)), uniqHLL12(x, y), uniqHLL12((x, y)), uniqHLL12(x, y, z), uniqHLL12((x, y, z)), uniqExact(x), uniqExact((x)), uniqExact(x, y), uniqExact((x, y)), uniqExact(x, y, z), uniqExact((x, y, z)), uniqUpTo(5)(x), uniqUpTo(5)((x)), uniqUpTo(5)(x, y), uniqUpTo(5)((x, y)), uniqUpTo(5)(x, y, z), uniqUpTo(5)((x, y, z)), count() AS c FROM ( - SELECT + SELECT (number + 0x8ffcbd8257219a26) * 0x66bb3430c06d2353 % 131 AS k, number % 10 AS x, intDiv(number, 10) % 10 AS y, diff --git a/dbms/tests/queries/0_stateless/00700_decimal_aggregates.sql b/dbms/tests/queries/0_stateless/00700_decimal_aggregates.sql index e10f665ad02..ba097ada98b 100644 --- a/dbms/tests/queries/0_stateless/00700_decimal_aggregates.sql +++ b/dbms/tests/queries/0_stateless/00700_decimal_aggregates.sql @@ -28,7 +28,7 @@ SELECT avg(a), avg(b), avg(c) FROM test.decimal WHERE a > 0; SELECT avg(a), avg(b), avg(c) FROM test.decimal WHERE a < 0; SELECT (uniq(a), uniq(b), uniq(c)), - (uniqCombined(a), uniqCombined(b), uniqCombined(c)), + (uniqCombined(15)(a), uniqCombined(15)(b), uniqCombined(15)(c)), (uniqExact(a), uniqExact(b), uniqExact(c)), (uniqHLL12(a), uniqHLL12(b), uniqHLL12(c)) FROM (SELECT * FROM test.decimal ORDER BY a); diff --git a/dbms/tests/queries/1_stateful/00146_aggregate_function_uniq.sql b/dbms/tests/queries/1_stateful/00146_aggregate_function_uniq.sql index fd3fde7636d..0f860948a3a 100644 --- a/dbms/tests/queries/1_stateful/00146_aggregate_function_uniq.sql +++ b/dbms/tests/queries/1_stateful/00146_aggregate_function_uniq.sql @@ -1,3 +1,3 @@ SELECT RegionID, uniqHLL12(WatchID) AS X FROM remote('127.0.0.{1,2}', test, hits) GROUP BY RegionID HAVING X > 100000 ORDER BY RegionID ASC; -SELECT RegionID, uniqCombined(WatchID) AS X FROM remote('127.0.0.{1,2}', test, hits) GROUP BY RegionID HAVING X > 100000 ORDER BY RegionID ASC; +SELECT RegionID, uniqCombined(15)(WatchID) AS X FROM remote('127.0.0.{1,2}', test, hits) GROUP BY RegionID HAVING X > 100000 ORDER BY RegionID ASC; SELECT abs(uniq(WatchID) - uniqExact(WatchID)) FROM test.hits; diff --git a/docs/en/query_language/agg_functions/reference.md b/docs/en/query_language/agg_functions/reference.md index fd0bb213933..491a710fe5b 100644 --- a/docs/en/query_language/agg_functions/reference.md +++ b/docs/en/query_language/agg_functions/reference.md @@ -146,7 +146,7 @@ The result is determinate (it doesn't depend on the order of query processing). This function provides excellent accuracy even for data sets with extremely high cardinality (over 10 billion elements). It is recommended for default use. -## uniqCombined(x) +## uniqCombined(HLL_precision)(x) Calculates the approximate number of different values of the argument. Works for numbers, strings, dates, date-with-time, and for multiple arguments and tuple arguments. diff --git a/docs/ru/query_language/agg_functions/reference.md b/docs/ru/query_language/agg_functions/reference.md index 1c700851ef0..d42f216c6ea 100644 --- a/docs/ru/query_language/agg_functions/reference.md +++ b/docs/ru/query_language/agg_functions/reference.md @@ -156,7 +156,7 @@ GROUP BY timeslot Данная функция обеспечивает отличную точность даже для множеств огромной кардинальности (10B+ элементов) и рекомендуется к использованию по умолчанию. -## uniqCombined(x) +## uniqCombined(HLL_precision)(x) Приближённо вычисляет количество различных значений аргумента. Работает для чисел, строк, дат, дат-с-временем, для нескольких аргументов и аргументов-кортежей. From e16081d4d4cc23aea65ce56b4d9f4ad3d110c465 Mon Sep 17 00:00:00 2001 From: Ivan Lezhankin Date: Thu, 18 Oct 2018 14:15:47 +0300 Subject: [PATCH 009/124] Fix initialization of precision. --- .../src/AggregateFunctions/AggregateFunctionUniqCombined.h | 7 ++++++- 1 file changed, 6 insertions(+), 1 deletion(-) diff --git a/dbms/src/AggregateFunctions/AggregateFunctionUniqCombined.h b/dbms/src/AggregateFunctions/AggregateFunctionUniqCombined.h index 8d8a7c6745d..7d8bd487345 100644 --- a/dbms/src/AggregateFunctions/AggregateFunctionUniqCombined.h +++ b/dbms/src/AggregateFunctions/AggregateFunctionUniqCombined.h @@ -133,9 +133,14 @@ struct __attribute__((__packed__)) AggregateFunctionUniqCombinedDataWithKey void init(UInt8 precision) const { - if (inited || precision == 17) + if (inited) return; + if (precision == 17) { + inited = precision; + return; + } + // TODO: assert "inited == precision" set_17.~CombinedCardinalityEstimator(); From d588120677264a4849b9fe8b88a004eaaa46b4b4 Mon Sep 17 00:00:00 2001 From: Ivan Lezhankin Date: Thu, 18 Oct 2018 17:47:21 +0300 Subject: [PATCH 010/124] Keep default value for HLL precision in tests - to not break them. --- dbms/scripts/test_uniq_functions.sh | 2 +- .../tests/performance/test_hits/test_hits.xml | 4 +-- .../00212_shard_aggregate_function_uniq.sql | 26 +++++++++---------- .../0_stateless/00264_uniq_many_args.sql | 4 +-- .../0_stateless/00700_decimal_aggregates.sql | 2 +- .../00146_aggregate_function_uniq.sql | 2 +- 6 files changed, 20 insertions(+), 20 deletions(-) diff --git a/dbms/scripts/test_uniq_functions.sh b/dbms/scripts/test_uniq_functions.sh index f7e2083610b..2a3becbcfa3 100755 --- a/dbms/scripts/test_uniq_functions.sh +++ b/dbms/scripts/test_uniq_functions.sh @@ -6,6 +6,6 @@ do do n=$(( 10**p * i )) echo -n "$n " - clickhouse-client -q "select uniqHLL12(number), uniq(number), uniqCombined(15)(number) from numbers($n);" + clickhouse-client -q "select uniqHLL12(number), uniq(number), uniqCombined(17)(number) from numbers($n);" done done diff --git a/dbms/tests/performance/test_hits/test_hits.xml b/dbms/tests/performance/test_hits/test_hits.xml index eea308fdd64..e98c6c8e625 100644 --- a/dbms/tests/performance/test_hits/test_hits.xml +++ b/dbms/tests/performance/test_hits/test_hits.xml @@ -86,8 +86,8 @@ PageCharset тоже почти всегда непуст, но его сред SELECT uniq(UserID) FROM test.hits SETTINGS max_threads = 1 SELECT uniq(UserID) FROM test.hits -SELECT uniqCombined(15)(UserID) FROM test.hits SETTINGS max_threads = 1 -SELECT uniqCombined(15)(UserID) FROM test.hits +SELECT uniqCombined(17)(UserID) FROM test.hits SETTINGS max_threads = 1 +SELECT uniqCombined(17)(UserID) FROM test.hits SELECT uniqExact(UserID) FROM test.hits SETTINGS max_threads = 1 SELECT uniqExact(UserID) FROM test.hits diff --git a/dbms/tests/queries/0_stateless/00212_shard_aggregate_function_uniq.sql b/dbms/tests/queries/0_stateless/00212_shard_aggregate_function_uniq.sql index b3d82b71685..596b7e816b0 100644 --- a/dbms/tests/queries/0_stateless/00212_shard_aggregate_function_uniq.sql +++ b/dbms/tests/queries/0_stateless/00212_shard_aggregate_function_uniq.sql @@ -20,20 +20,20 @@ SELECT uniqHLL12(dummy) FROM remote('127.0.0.{2,3}', system.one); /* uniqCombined */ -SELECT Y, uniqCombined(15)(X) FROM (SELECT number AS X, (3*X*X - 7*X + 11) % 37 AS Y FROM system.numbers LIMIT 15) GROUP BY Y; -SELECT Y, uniqCombined(15)(X) FROM (SELECT number AS X, (3*X*X - 7*X + 11) % 37 AS Y FROM system.numbers LIMIT 3000) GROUP BY Y; -SELECT Y, uniqCombined(15)(X) FROM (SELECT number AS X, (3*X*X - 7*X + 11) % 37 AS Y FROM system.numbers LIMIT 1000000) GROUP BY Y; +SELECT Y, uniqCombined(17)(X) FROM (SELECT number AS X, (3*X*X - 7*X + 11) % 37 AS Y FROM system.numbers LIMIT 15) GROUP BY Y; +SELECT Y, uniqCombined(17)(X) FROM (SELECT number AS X, (3*X*X - 7*X + 11) % 37 AS Y FROM system.numbers LIMIT 3000) GROUP BY Y; +SELECT Y, uniqCombined(17)(X) FROM (SELECT number AS X, (3*X*X - 7*X + 11) % 37 AS Y FROM system.numbers LIMIT 1000000) GROUP BY Y; -SELECT Y, uniqCombined(15)(X) FROM (SELECT number AS X, round(1/(1 + (3*X*X - 7*X + 11) % 37), 3) AS Y FROM system.numbers LIMIT 15) GROUP BY Y; -SELECT Y, uniqCombined(15)(X) FROM (SELECT number AS X, round(1/(1 + (3*X*X - 7*X + 11) % 37), 3) AS Y FROM system.numbers LIMIT 3000) GROUP BY Y; -SELECT Y, uniqCombined(15)(X) FROM (SELECT number AS X, round(1/(1 + (3*X*X - 7*X + 11) % 37), 3) AS Y FROM system.numbers LIMIT 1000000) GROUP BY Y; +SELECT Y, uniqCombined(17)(X) FROM (SELECT number AS X, round(1/(1 + (3*X*X - 7*X + 11) % 37), 3) AS Y FROM system.numbers LIMIT 15) GROUP BY Y; +SELECT Y, uniqCombined(17)(X) FROM (SELECT number AS X, round(1/(1 + (3*X*X - 7*X + 11) % 37), 3) AS Y FROM system.numbers LIMIT 3000) GROUP BY Y; +SELECT Y, uniqCombined(17)(X) FROM (SELECT number AS X, round(1/(1 + (3*X*X - 7*X + 11) % 37), 3) AS Y FROM system.numbers LIMIT 1000000) GROUP BY Y; -SELECT Y, uniqCombined(15)(X) FROM (SELECT number AS X, round(toFloat32(1/(1 + (3*X*X - 7*X + 11) % 37)), 3) AS Y FROM system.numbers LIMIT 15) GROUP BY Y; -SELECT Y, uniqCombined(15)(X) FROM (SELECT number AS X, round(toFloat32(1/(1 + (3*X*X - 7*X + 11) % 37)), 3) AS Y FROM system.numbers LIMIT 3000) GROUP BY Y; -SELECT Y, uniqCombined(15)(X) FROM (SELECT number AS X, round(toFloat32(1/(1 + (3*X*X - 7*X + 11) % 37)), 3) AS Y FROM system.numbers LIMIT 1000000) GROUP BY Y; +SELECT Y, uniqCombined(17)(X) FROM (SELECT number AS X, round(toFloat32(1/(1 + (3*X*X - 7*X + 11) % 37)), 3) AS Y FROM system.numbers LIMIT 15) GROUP BY Y; +SELECT Y, uniqCombined(17)(X) FROM (SELECT number AS X, round(toFloat32(1/(1 + (3*X*X - 7*X + 11) % 37)), 3) AS Y FROM system.numbers LIMIT 3000) GROUP BY Y; +SELECT Y, uniqCombined(17)(X) FROM (SELECT number AS X, round(toFloat32(1/(1 + (3*X*X - 7*X + 11) % 37)), 3) AS Y FROM system.numbers LIMIT 1000000) GROUP BY Y; -SELECT Y, uniqCombined(15)(Z) FROM (SELECT number AS X, IPv4NumToString(toUInt32(X)) AS Z, (3*X*X - 7*X + 11) % 37 AS Y FROM system.numbers LIMIT 15) GROUP BY Y; -SELECT Y, uniqCombined(15)(Z) FROM (SELECT number AS X, IPv4NumToString(toUInt32(X)) AS Z, (3*X*X - 7*X + 11) % 37 AS Y FROM system.numbers LIMIT 3000) GROUP BY Y; -SELECT Y, uniqCombined(15)(Z) FROM (SELECT number AS X, IPv4NumToString(toUInt32(X)) AS Z, (3*X*X - 7*X + 11) % 37 AS Y FROM system.numbers LIMIT 1000000) GROUP BY Y; +SELECT Y, uniqCombined(17)(Z) FROM (SELECT number AS X, IPv4NumToString(toUInt32(X)) AS Z, (3*X*X - 7*X + 11) % 37 AS Y FROM system.numbers LIMIT 15) GROUP BY Y; +SELECT Y, uniqCombined(17)(Z) FROM (SELECT number AS X, IPv4NumToString(toUInt32(X)) AS Z, (3*X*X - 7*X + 11) % 37 AS Y FROM system.numbers LIMIT 3000) GROUP BY Y; +SELECT Y, uniqCombined(17)(Z) FROM (SELECT number AS X, IPv4NumToString(toUInt32(X)) AS Z, (3*X*X - 7*X + 11) % 37 AS Y FROM system.numbers LIMIT 1000000) GROUP BY Y; -SELECT uniqCombined(15)(dummy) FROM remote('127.0.0.{2,3}', system.one); +SELECT uniqCombined(17)(dummy) FROM remote('127.0.0.{2,3}', system.one); diff --git a/dbms/tests/queries/0_stateless/00264_uniq_many_args.sql b/dbms/tests/queries/0_stateless/00264_uniq_many_args.sql index a94ec6e8f0a..0a813bab006 100644 --- a/dbms/tests/queries/0_stateless/00264_uniq_many_args.sql +++ b/dbms/tests/queries/0_stateless/00264_uniq_many_args.sql @@ -1,6 +1,6 @@ SELECT uniq(x), uniq((x)), uniq(x, y), uniq((x, y)), uniq(x, y, z), uniq((x, y, z)), - uniqCombined(15)(x), uniqCombined(15)((x)), uniqCombined(15)(x, y), uniqCombined(15)((x, y)), uniqCombined(15)(x, y, z), uniqCombined(15)((x, y, z)), + uniqCombined(17)(x), uniqCombined(17)((x)), uniqCombined(17)(x, y), uniqCombined(17)((x, y)), uniqCombined(17)(x, y, z), uniqCombined(17)((x, y, z)), uniqHLL12(x), uniqHLL12((x)), uniqHLL12(x, y), uniqHLL12((x, y)), uniqHLL12(x, y, z), uniqHLL12((x, y, z)), uniqExact(x), uniqExact((x)), uniqExact(x, y), uniqExact((x, y)), uniqExact(x, y, z), uniqExact((x, y, z)), uniqUpTo(5)(x), uniqUpTo(5)((x)), uniqUpTo(5)(x, y), uniqUpTo(5)((x, y)), uniqUpTo(5)(x, y, z), uniqUpTo(5)((x, y, z)) @@ -16,7 +16,7 @@ FROM SELECT k, uniq(x), uniq((x)), uniq(x, y), uniq((x, y)), uniq(x, y, z), uniq((x, y, z)), - uniqCombined(15)(x), uniqCombined(15)((x)), uniqCombined(15)(x, y), uniqCombined(15)((x, y)), uniqCombined(15)(x, y, z), uniqCombined(15)((x, y, z)), + uniqCombined(17)(x), uniqCombined(17)((x)), uniqCombined(17)(x, y), uniqCombined(17)((x, y)), uniqCombined(17)(x, y, z), uniqCombined(17)((x, y, z)), uniqHLL12(x), uniqHLL12((x)), uniqHLL12(x, y), uniqHLL12((x, y)), uniqHLL12(x, y, z), uniqHLL12((x, y, z)), uniqExact(x), uniqExact((x)), uniqExact(x, y), uniqExact((x, y)), uniqExact(x, y, z), uniqExact((x, y, z)), uniqUpTo(5)(x), uniqUpTo(5)((x)), uniqUpTo(5)(x, y), uniqUpTo(5)((x, y)), uniqUpTo(5)(x, y, z), uniqUpTo(5)((x, y, z)), diff --git a/dbms/tests/queries/0_stateless/00700_decimal_aggregates.sql b/dbms/tests/queries/0_stateless/00700_decimal_aggregates.sql index ba097ada98b..1795398babb 100644 --- a/dbms/tests/queries/0_stateless/00700_decimal_aggregates.sql +++ b/dbms/tests/queries/0_stateless/00700_decimal_aggregates.sql @@ -28,7 +28,7 @@ SELECT avg(a), avg(b), avg(c) FROM test.decimal WHERE a > 0; SELECT avg(a), avg(b), avg(c) FROM test.decimal WHERE a < 0; SELECT (uniq(a), uniq(b), uniq(c)), - (uniqCombined(15)(a), uniqCombined(15)(b), uniqCombined(15)(c)), + (uniqCombined(17)(a), uniqCombined(17)(b), uniqCombined(17)(c)), (uniqExact(a), uniqExact(b), uniqExact(c)), (uniqHLL12(a), uniqHLL12(b), uniqHLL12(c)) FROM (SELECT * FROM test.decimal ORDER BY a); diff --git a/dbms/tests/queries/1_stateful/00146_aggregate_function_uniq.sql b/dbms/tests/queries/1_stateful/00146_aggregate_function_uniq.sql index 0f860948a3a..1200e312652 100644 --- a/dbms/tests/queries/1_stateful/00146_aggregate_function_uniq.sql +++ b/dbms/tests/queries/1_stateful/00146_aggregate_function_uniq.sql @@ -1,3 +1,3 @@ SELECT RegionID, uniqHLL12(WatchID) AS X FROM remote('127.0.0.{1,2}', test, hits) GROUP BY RegionID HAVING X > 100000 ORDER BY RegionID ASC; -SELECT RegionID, uniqCombined(15)(WatchID) AS X FROM remote('127.0.0.{1,2}', test, hits) GROUP BY RegionID HAVING X > 100000 ORDER BY RegionID ASC; +SELECT RegionID, uniqCombined(17)(WatchID) AS X FROM remote('127.0.0.{1,2}', test, hits) GROUP BY RegionID HAVING X > 100000 ORDER BY RegionID ASC; SELECT abs(uniq(WatchID) - uniqExact(WatchID)) FROM test.hits; From 45e6fd8cd25f7328590d69428138ed3095ef8e8f Mon Sep 17 00:00:00 2001 From: Ivan Lezhankin Date: Thu, 18 Oct 2018 18:23:42 +0300 Subject: [PATCH 011/124] Fix compilation with gcc and code style --- .../AggregateFunctionUniqCombined.h | 23 ++++++++++--------- 1 file changed, 12 insertions(+), 11 deletions(-) diff --git a/dbms/src/AggregateFunctions/AggregateFunctionUniqCombined.h b/dbms/src/AggregateFunctions/AggregateFunctionUniqCombined.h index 7d8bd487345..d655ab3626e 100644 --- a/dbms/src/AggregateFunctions/AggregateFunctionUniqCombined.h +++ b/dbms/src/AggregateFunctions/AggregateFunctionUniqCombined.h @@ -67,7 +67,7 @@ namespace detail template -struct __attribute__((__packed__)) AggregateFunctionUniqCombinedDataWithKey +struct AggregateFunctionUniqCombinedDataWithKey { template using Set = CombinedCardinalityEstimator set_12; - Set<13> set_13; - Set<14> set_14; - Set<15> set_15; - Set<16> set_16; - Set<17> set_17; - Set<18> set_18; - Set<19> set_19; - Set<20> set_20; + mutable Set<12> set_12; + mutable Set<13> set_13; + mutable Set<14> set_14; + mutable Set<15> set_15; + mutable Set<16> set_16; + mutable Set<17> set_17; + mutable Set<18> set_18; + mutable Set<19> set_19; + mutable Set<20> set_20; }; AggregateFunctionUniqCombinedDataWithKey() : set_17() {} @@ -136,7 +136,8 @@ struct __attribute__((__packed__)) AggregateFunctionUniqCombinedDataWithKey if (inited) return; - if (precision == 17) { + if (precision == 17) + { inited = precision; return; } From 28ea773b9ee17eaa3030e2bac0fd6ad5fd77a870 Mon Sep 17 00:00:00 2001 From: Ivan Lezhankin Date: Thu, 18 Oct 2018 21:38:45 +0300 Subject: [PATCH 012/124] Use a single "place" for a default precision constant. Also update stateless tests. --- dbms/scripts/test_uniq_functions.sh | 2 +- .../AggregateFunctionUniqCombined.cpp | 2 +- .../AggregateFunctionUniqCombined.h | 72 +++--- ...12_shard_aggregate_function_uniq.reference | 209 ++++++++++++++++++ .../00212_shard_aggregate_function_uniq.sql | 18 ++ .../00264_uniq_many_args.reference | 22 +- .../0_stateless/00264_uniq_many_args.sql | 2 + .../00700_decimal_aggregates.reference | 2 +- .../0_stateless/00700_decimal_aggregates.sql | 1 + .../00146_aggregate_function_uniq.sql | 2 +- 10 files changed, 290 insertions(+), 42 deletions(-) diff --git a/dbms/scripts/test_uniq_functions.sh b/dbms/scripts/test_uniq_functions.sh index 2a3becbcfa3..9a4b6f20433 100755 --- a/dbms/scripts/test_uniq_functions.sh +++ b/dbms/scripts/test_uniq_functions.sh @@ -6,6 +6,6 @@ do do n=$(( 10**p * i )) echo -n "$n " - clickhouse-client -q "select uniqHLL12(number), uniq(number), uniqCombined(17)(number) from numbers($n);" + clickhouse-client -q "select uniqHLL12(number), uniq(number), uniqCombined(number) from numbers($n);" done done diff --git a/dbms/src/AggregateFunctions/AggregateFunctionUniqCombined.cpp b/dbms/src/AggregateFunctions/AggregateFunctionUniqCombined.cpp index cdaf5b90c70..e148c9157cd 100644 --- a/dbms/src/AggregateFunctions/AggregateFunctionUniqCombined.cpp +++ b/dbms/src/AggregateFunctions/AggregateFunctionUniqCombined.cpp @@ -22,7 +22,7 @@ namespace AggregateFunctionPtr createAggregateFunctionUniqCombined( const std::string & name, const DataTypes & argument_types, const Array & params) { - UInt8 precision = 17; /// default value - must correlate with default ctor of |AggregateFunctionUniqCombinedData| + UInt8 precision = detail::UNIQ_COMBINED_DEFAULT_PRECISION; if (!params.empty()) { diff --git a/dbms/src/AggregateFunctions/AggregateFunctionUniqCombined.h b/dbms/src/AggregateFunctions/AggregateFunctionUniqCombined.h index d655ab3626e..c4e1a571aee 100644 --- a/dbms/src/AggregateFunctions/AggregateFunctionUniqCombined.h +++ b/dbms/src/AggregateFunctions/AggregateFunctionUniqCombined.h @@ -19,12 +19,16 @@ #include +#define DEFAULT_HLL_PRECISION 17 + namespace DB { namespace detail { + const UInt8 UNIQ_COMBINED_DEFAULT_PRECISION = DEFAULT_HLL_PRECISION; + /** Hash function for uniqCombined. - */ + */ template struct AggregateFunctionUniqCombinedTraits { @@ -94,12 +98,19 @@ struct AggregateFunctionUniqCombinedDataWithKey mutable Set<20> set_20; }; - AggregateFunctionUniqCombinedDataWithKey() : set_17() {} +#define PASTE(x, y) x##y +#define EVAL(x, y) PASTE(x, y) +#define DEFAULT_SET EVAL(set_, DEFAULT_HLL_PRECISION) + + AggregateFunctionUniqCombinedDataWithKey() : DEFAULT_SET() {} ~AggregateFunctionUniqCombinedDataWithKey() { switch (inited) { + case 0: + DEFAULT_SET.~CombinedCardinalityEstimator(); + break; case 12: set_12.~CombinedCardinalityEstimator(); break; @@ -115,7 +126,6 @@ struct AggregateFunctionUniqCombinedDataWithKey case 16: set_16.~CombinedCardinalityEstimator(); break; - case 0: case 17: set_17.~CombinedCardinalityEstimator(); break; @@ -136,7 +146,7 @@ struct AggregateFunctionUniqCombinedDataWithKey if (inited) return; - if (precision == 17) + if (precision == DEFAULT_HLL_PRECISION) { inited = precision; return; @@ -144,7 +154,7 @@ struct AggregateFunctionUniqCombinedDataWithKey // TODO: assert "inited == precision" - set_17.~CombinedCardinalityEstimator(); + DEFAULT_SET.~CombinedCardinalityEstimator(); switch (precision) { @@ -163,6 +173,9 @@ struct AggregateFunctionUniqCombinedDataWithKey case 16: new (&set_16) Set<16>; break; + case 17: + new (&set_17) Set<17>; + break; case 18: new (&set_18) Set<18>; break; @@ -208,28 +221,29 @@ struct AggregateFunctionUniqCombinedDataWithKey break; \ } -#define SET_RETURN_METHOD(method) \ - switch (inited) \ - { \ - case 12: \ - return set_12.method; \ - case 13: \ - return set_13.method; \ - case 14: \ - return set_14.method; \ - case 15: \ - return set_15.method; \ - case 16: \ - return set_16.method; \ - case 18: \ - return set_18.method; \ - case 19: \ - return set_19.method; \ - case 20: \ - return set_20.method; \ - case 17: \ - default: \ - return set_17.method; \ +#define SET_RETURN_METHOD(method) \ + switch (inited) \ + { \ + case 12: \ + return set_12.method; \ + case 13: \ + return set_13.method; \ + case 14: \ + return set_14.method; \ + case 15: \ + return set_15.method; \ + case 16: \ + return set_16.method; \ + case 17: \ + return set_17.method; \ + case 18: \ + return set_18.method; \ + case 19: \ + return set_19.method; \ + case 20: \ + return set_20.method; \ + default: \ + return DEFAULT_SET.method; \ } void insert(Key value, UInt8 precision) @@ -293,6 +307,10 @@ struct AggregateFunctionUniqCombinedDataWithKey #undef SET_METHOD #undef SET_RETURN_METHOD +#undef PASTE +#undef EVAL +#undef DEFAULT_SET +#undef DEFAULT_HLL_PRECISION }; diff --git a/dbms/tests/queries/0_stateless/00212_shard_aggregate_function_uniq.reference b/dbms/tests/queries/0_stateless/00212_shard_aggregate_function_uniq.reference index 8c94b8a5d58..c7c4f8ebacd 100644 --- a/dbms/tests/queries/0_stateless/00212_shard_aggregate_function_uniq.reference +++ b/dbms/tests/queries/0_stateless/00212_shard_aggregate_function_uniq.reference @@ -259,6 +259,58 @@ 31 53948 35 53931 36 53982 +1 1 +3 1 +6 1 +7 1 +9 1 +11 1 +14 1 +17 1 +19 1 +20 2 +26 1 +31 1 +35 1 +36 1 +0 162 +1 162 +3 162 +6 162 +7 163 +9 163 +10 81 +11 163 +13 162 +14 162 +17 162 +19 162 +20 162 +21 162 +22 162 +26 162 +31 162 +35 162 +36 162 +0 53901 +1 54056 +3 53999 +6 54129 +7 54213 +9 53853 +10 26975 +11 54064 +13 53963 +14 53997 +17 54129 +19 53923 +20 53958 +21 54117 +22 54150 +26 54047 +31 53948 +35 53931 +36 53982 0.125 1 0.5 1 0.05 1 @@ -311,6 +363,110 @@ 0.043 54150 0.037 54047 0.071 53963 +0.125 1 +0.5 1 +0.05 1 +0.143 1 +0.056 1 +0.048 2 +0.083 1 +0.25 1 +0.1 1 +0.028 1 +0.027 1 +0.031 1 +0.067 1 +0.037 1 +0.045 162 +0.125 163 +0.5 162 +0.05 162 +0.143 162 +0.091 81 +0.056 162 +0.048 162 +0.083 163 +0.25 162 +1 162 +0.1 163 +0.028 162 +0.027 162 +0.031 162 +0.067 162 +0.043 162 +0.037 162 +0.071 162 +0.045 54117 +0.125 54213 +0.5 54056 +0.05 53923 +0.143 54129 +0.091 26975 +0.056 54129 +0.048 53958 +0.083 54064 +0.25 53999 +1 53901 +0.1 53853 +0.028 53931 +0.027 53982 +0.031 53948 +0.067 53997 +0.043 54150 +0.037 54047 +0.071 53963 +0.5 1 +0.05 1 +0.25 1 +0.048 2 +0.083 1 +0.125 1 +0.031 1 +0.143 1 +0.028 1 +0.067 1 +0.027 1 +0.056 1 +0.037 1 +0.1 1 +0.5 162 +0.05 162 +0.25 162 +0.048 162 +0.091 81 +0.043 162 +0.071 162 +0.083 163 +0.125 163 +0.031 162 +0.143 162 +0.028 162 +0.067 162 +0.045 162 +0.027 162 +0.056 162 +0.037 162 +0.1 163 +1 162 +0.5 54056 +0.05 53923 +0.25 53999 +0.048 53958 +0.091 26975 +0.043 54150 +0.071 53963 +0.083 54064 +0.125 54213 +0.031 53948 +0.143 54129 +0.028 53931 +0.067 53997 +0.045 54117 +0.027 53982 +0.056 54129 +0.037 54047 +0.1 53853 +1 53901 0.5 1 0.05 1 0.25 1 @@ -415,4 +571,57 @@ 31 54074 35 54153 36 53999 +1 1 +3 1 +6 1 +7 1 +9 1 +11 1 +14 1 +17 1 +19 1 +20 2 +26 1 +31 1 +35 1 +36 1 +0 162 +1 162 +3 162 +6 162 +7 163 +9 163 +10 81 +11 163 +13 162 +14 162 +17 162 +19 162 +20 162 +21 162 +22 162 +26 162 +31 162 +35 162 +36 162 +0 54195 +1 54086 +3 54127 +6 54173 +7 53969 +9 54257 +10 26985 +11 53880 +13 54105 +14 54043 +17 54176 +19 53913 +20 54088 +21 53991 +22 54112 +26 54136 +31 54074 +35 54153 +36 53999 +1 1 diff --git a/dbms/tests/queries/0_stateless/00212_shard_aggregate_function_uniq.sql b/dbms/tests/queries/0_stateless/00212_shard_aggregate_function_uniq.sql index 596b7e816b0..a2ab15775c5 100644 --- a/dbms/tests/queries/0_stateless/00212_shard_aggregate_function_uniq.sql +++ b/dbms/tests/queries/0_stateless/00212_shard_aggregate_function_uniq.sql @@ -20,20 +20,38 @@ SELECT uniqHLL12(dummy) FROM remote('127.0.0.{2,3}', system.one); /* uniqCombined */ +SELECT Y, uniqCombined(X) FROM (SELECT number AS X, (3*X*X - 7*X + 11) % 37 AS Y FROM system.numbers LIMIT 15) GROUP BY Y; +SELECT Y, uniqCombined(X) FROM (SELECT number AS X, (3*X*X - 7*X + 11) % 37 AS Y FROM system.numbers LIMIT 3000) GROUP BY Y; +SELECT Y, uniqCombined(X) FROM (SELECT number AS X, (3*X*X - 7*X + 11) % 37 AS Y FROM system.numbers LIMIT 1000000) GROUP BY Y; + SELECT Y, uniqCombined(17)(X) FROM (SELECT number AS X, (3*X*X - 7*X + 11) % 37 AS Y FROM system.numbers LIMIT 15) GROUP BY Y; SELECT Y, uniqCombined(17)(X) FROM (SELECT number AS X, (3*X*X - 7*X + 11) % 37 AS Y FROM system.numbers LIMIT 3000) GROUP BY Y; SELECT Y, uniqCombined(17)(X) FROM (SELECT number AS X, (3*X*X - 7*X + 11) % 37 AS Y FROM system.numbers LIMIT 1000000) GROUP BY Y; +SELECT Y, uniqCombined(X) FROM (SELECT number AS X, round(1/(1 + (3*X*X - 7*X + 11) % 37), 3) AS Y FROM system.numbers LIMIT 15) GROUP BY Y; +SELECT Y, uniqCombined(X) FROM (SELECT number AS X, round(1/(1 + (3*X*X - 7*X + 11) % 37), 3) AS Y FROM system.numbers LIMIT 3000) GROUP BY Y; +SELECT Y, uniqCombined(X) FROM (SELECT number AS X, round(1/(1 + (3*X*X - 7*X + 11) % 37), 3) AS Y FROM system.numbers LIMIT 1000000) GROUP BY Y; + SELECT Y, uniqCombined(17)(X) FROM (SELECT number AS X, round(1/(1 + (3*X*X - 7*X + 11) % 37), 3) AS Y FROM system.numbers LIMIT 15) GROUP BY Y; SELECT Y, uniqCombined(17)(X) FROM (SELECT number AS X, round(1/(1 + (3*X*X - 7*X + 11) % 37), 3) AS Y FROM system.numbers LIMIT 3000) GROUP BY Y; SELECT Y, uniqCombined(17)(X) FROM (SELECT number AS X, round(1/(1 + (3*X*X - 7*X + 11) % 37), 3) AS Y FROM system.numbers LIMIT 1000000) GROUP BY Y; +SELECT Y, uniqCombined(X) FROM (SELECT number AS X, round(toFloat32(1/(1 + (3*X*X - 7*X + 11) % 37)), 3) AS Y FROM system.numbers LIMIT 15) GROUP BY Y; +SELECT Y, uniqCombined(X) FROM (SELECT number AS X, round(toFloat32(1/(1 + (3*X*X - 7*X + 11) % 37)), 3) AS Y FROM system.numbers LIMIT 3000) GROUP BY Y; +SELECT Y, uniqCombined(X) FROM (SELECT number AS X, round(toFloat32(1/(1 + (3*X*X - 7*X + 11) % 37)), 3) AS Y FROM system.numbers LIMIT 1000000) GROUP BY Y; + SELECT Y, uniqCombined(17)(X) FROM (SELECT number AS X, round(toFloat32(1/(1 + (3*X*X - 7*X + 11) % 37)), 3) AS Y FROM system.numbers LIMIT 15) GROUP BY Y; SELECT Y, uniqCombined(17)(X) FROM (SELECT number AS X, round(toFloat32(1/(1 + (3*X*X - 7*X + 11) % 37)), 3) AS Y FROM system.numbers LIMIT 3000) GROUP BY Y; SELECT Y, uniqCombined(17)(X) FROM (SELECT number AS X, round(toFloat32(1/(1 + (3*X*X - 7*X + 11) % 37)), 3) AS Y FROM system.numbers LIMIT 1000000) GROUP BY Y; +SELECT Y, uniqCombined(Z) FROM (SELECT number AS X, IPv4NumToString(toUInt32(X)) AS Z, (3*X*X - 7*X + 11) % 37 AS Y FROM system.numbers LIMIT 15) GROUP BY Y; +SELECT Y, uniqCombined(Z) FROM (SELECT number AS X, IPv4NumToString(toUInt32(X)) AS Z, (3*X*X - 7*X + 11) % 37 AS Y FROM system.numbers LIMIT 3000) GROUP BY Y; +SELECT Y, uniqCombined(Z) FROM (SELECT number AS X, IPv4NumToString(toUInt32(X)) AS Z, (3*X*X - 7*X + 11) % 37 AS Y FROM system.numbers LIMIT 1000000) GROUP BY Y; + SELECT Y, uniqCombined(17)(Z) FROM (SELECT number AS X, IPv4NumToString(toUInt32(X)) AS Z, (3*X*X - 7*X + 11) % 37 AS Y FROM system.numbers LIMIT 15) GROUP BY Y; SELECT Y, uniqCombined(17)(Z) FROM (SELECT number AS X, IPv4NumToString(toUInt32(X)) AS Z, (3*X*X - 7*X + 11) % 37 AS Y FROM system.numbers LIMIT 3000) GROUP BY Y; SELECT Y, uniqCombined(17)(Z) FROM (SELECT number AS X, IPv4NumToString(toUInt32(X)) AS Z, (3*X*X - 7*X + 11) % 37 AS Y FROM system.numbers LIMIT 1000000) GROUP BY Y; +SELECT uniqCombined(dummy) FROM remote('127.0.0.{2,3}', system.one); + SELECT uniqCombined(17)(dummy) FROM remote('127.0.0.{2,3}', system.one); diff --git a/dbms/tests/queries/0_stateless/00264_uniq_many_args.reference b/dbms/tests/queries/0_stateless/00264_uniq_many_args.reference index 758b4d6b8e2..12aeed56e8b 100644 --- a/dbms/tests/queries/0_stateless/00264_uniq_many_args.reference +++ b/dbms/tests/queries/0_stateless/00264_uniq_many_args.reference @@ -1,11 +1,11 @@ -10 10 100 100 1000 1000 10 10 100 100 1000 1000 10 10 101 101 1006 1006 10 10 100 100 1000 1000 6 6 6 6 6 6 -17 10 10 100 100 610 610 10 10 100 100 610 610 10 10 101 101 616 616 10 10 100 100 610 610 6 6 6 6 6 6 766 -52 10 10 100 100 608 608 10 10 100 100 608 608 10 10 101 101 611 611 10 10 100 100 608 608 6 6 6 6 6 6 766 -5 10 10 100 100 609 609 10 10 100 100 609 609 10 10 101 101 608 608 10 10 100 100 609 609 6 6 6 6 6 6 765 -9 10 10 100 100 608 608 10 10 100 100 608 608 10 10 101 101 618 618 10 10 100 100 608 608 6 6 6 6 6 6 765 -13 10 10 100 100 607 607 10 10 100 100 607 607 10 10 101 101 610 610 10 10 100 100 607 607 6 6 6 6 6 6 765 -46 10 10 100 100 607 607 10 10 100 100 607 607 10 10 101 101 611 611 10 10 100 100 607 607 6 6 6 6 6 6 765 -48 10 10 100 100 609 609 10 10 100 100 609 609 10 10 101 101 617 617 10 10 100 100 609 609 6 6 6 6 6 6 765 -50 10 10 100 100 608 608 10 10 100 100 608 608 10 10 101 101 614 614 10 10 100 100 608 608 6 6 6 6 6 6 765 -54 10 10 100 100 609 609 10 10 100 100 609 609 10 10 101 101 615 615 10 10 100 100 609 609 6 6 6 6 6 6 765 -56 10 10 100 100 608 608 10 10 100 100 608 608 10 10 101 101 614 614 10 10 100 100 608 608 6 6 6 6 6 6 765 +10 10 100 100 1000 1000 10 10 100 100 1000 1000 10 10 100 100 1000 1000 10 10 101 101 1006 1006 10 10 100 100 1000 1000 6 6 6 6 6 6 +17 10 10 100 100 610 610 10 10 100 100 610 610 10 10 100 100 610 610 10 10 101 101 616 616 10 10 100 100 610 610 6 6 6 6 6 6 766 +52 10 10 100 100 608 608 10 10 100 100 608 608 10 10 100 100 608 608 10 10 101 101 611 611 10 10 100 100 608 608 6 6 6 6 6 6 766 +5 10 10 100 100 609 609 10 10 100 100 609 609 10 10 100 100 609 609 10 10 101 101 608 608 10 10 100 100 609 609 6 6 6 6 6 6 765 +9 10 10 100 100 608 608 10 10 100 100 608 608 10 10 100 100 608 608 10 10 101 101 618 618 10 10 100 100 608 608 6 6 6 6 6 6 765 +13 10 10 100 100 607 607 10 10 100 100 607 607 10 10 100 100 607 607 10 10 101 101 610 610 10 10 100 100 607 607 6 6 6 6 6 6 765 +46 10 10 100 100 607 607 10 10 100 100 607 607 10 10 100 100 607 607 10 10 101 101 611 611 10 10 100 100 607 607 6 6 6 6 6 6 765 +48 10 10 100 100 609 609 10 10 100 100 609 609 10 10 100 100 609 609 10 10 101 101 617 617 10 10 100 100 609 609 6 6 6 6 6 6 765 +50 10 10 100 100 608 608 10 10 100 100 608 608 10 10 100 100 608 608 10 10 101 101 614 614 10 10 100 100 608 608 6 6 6 6 6 6 765 +54 10 10 100 100 609 609 10 10 100 100 609 609 10 10 100 100 609 609 10 10 101 101 615 615 10 10 100 100 609 609 6 6 6 6 6 6 765 +56 10 10 100 100 608 608 10 10 100 100 608 608 10 10 100 100 608 608 10 10 101 101 614 614 10 10 100 100 608 608 6 6 6 6 6 6 765 diff --git a/dbms/tests/queries/0_stateless/00264_uniq_many_args.sql b/dbms/tests/queries/0_stateless/00264_uniq_many_args.sql index 0a813bab006..847d753a36b 100644 --- a/dbms/tests/queries/0_stateless/00264_uniq_many_args.sql +++ b/dbms/tests/queries/0_stateless/00264_uniq_many_args.sql @@ -1,5 +1,6 @@ SELECT uniq(x), uniq((x)), uniq(x, y), uniq((x, y)), uniq(x, y, z), uniq((x, y, z)), + uniqCombined(x), uniqCombined((x)), uniqCombined(x, y), uniqCombined((x, y)), uniqCombined(x, y, z), uniqCombined((x, y, z)), uniqCombined(17)(x), uniqCombined(17)((x)), uniqCombined(17)(x, y), uniqCombined(17)((x, y)), uniqCombined(17)(x, y, z), uniqCombined(17)((x, y, z)), uniqHLL12(x), uniqHLL12((x)), uniqHLL12(x, y), uniqHLL12((x, y)), uniqHLL12(x, y, z), uniqHLL12((x, y, z)), uniqExact(x), uniqExact((x)), uniqExact(x, y), uniqExact((x, y)), uniqExact(x, y, z), uniqExact((x, y, z)), @@ -16,6 +17,7 @@ FROM SELECT k, uniq(x), uniq((x)), uniq(x, y), uniq((x, y)), uniq(x, y, z), uniq((x, y, z)), + uniqCombined(x), uniqCombined((x)), uniqCombined(x, y), uniqCombined((x, y)), uniqCombined(x, y, z), uniqCombined((x, y, z)), uniqCombined(17)(x), uniqCombined(17)((x)), uniqCombined(17)(x, y), uniqCombined(17)((x, y)), uniqCombined(17)(x, y, z), uniqCombined(17)((x, y, z)), uniqHLL12(x), uniqHLL12((x)), uniqHLL12(x, y), uniqHLL12((x, y)), uniqHLL12(x, y, z), uniqHLL12((x, y, z)), uniqExact(x), uniqExact((x)), uniqExact(x, y), uniqExact((x, y)), uniqExact(x, y, z), uniqExact((x, y, z)), diff --git a/dbms/tests/queries/0_stateless/00700_decimal_aggregates.reference b/dbms/tests/queries/0_stateless/00700_decimal_aggregates.reference index 3b1c6f9099d..1a7c7fbdbb0 100644 --- a/dbms/tests/queries/0_stateless/00700_decimal_aggregates.reference +++ b/dbms/tests/queries/0_stateless/00700_decimal_aggregates.reference @@ -8,7 +8,7 @@ 0.0000 0.00000000 0.00000000 25.5000 8.49999999 5.10000000 -25.5000 -8.49999999 -5.10000000 -(101,101,101) (101,101,101) (101,101,101) (102,100,101) +(101,101,101) (101,101,101) (101,101,101) (101,101,101) (102,100,101) 5 5 5 10 10 10 -50.0000 -50.0000 -16.66666666 -16.66666666 -10.00000000 -10.00000000 diff --git a/dbms/tests/queries/0_stateless/00700_decimal_aggregates.sql b/dbms/tests/queries/0_stateless/00700_decimal_aggregates.sql index 1795398babb..951e1384e93 100644 --- a/dbms/tests/queries/0_stateless/00700_decimal_aggregates.sql +++ b/dbms/tests/queries/0_stateless/00700_decimal_aggregates.sql @@ -28,6 +28,7 @@ SELECT avg(a), avg(b), avg(c) FROM test.decimal WHERE a > 0; SELECT avg(a), avg(b), avg(c) FROM test.decimal WHERE a < 0; SELECT (uniq(a), uniq(b), uniq(c)), + (uniqCombined(a), uniqCombined(b), uniqCombined(c)), (uniqCombined(17)(a), uniqCombined(17)(b), uniqCombined(17)(c)), (uniqExact(a), uniqExact(b), uniqExact(c)), (uniqHLL12(a), uniqHLL12(b), uniqHLL12(c)) diff --git a/dbms/tests/queries/1_stateful/00146_aggregate_function_uniq.sql b/dbms/tests/queries/1_stateful/00146_aggregate_function_uniq.sql index 1200e312652..fd3fde7636d 100644 --- a/dbms/tests/queries/1_stateful/00146_aggregate_function_uniq.sql +++ b/dbms/tests/queries/1_stateful/00146_aggregate_function_uniq.sql @@ -1,3 +1,3 @@ SELECT RegionID, uniqHLL12(WatchID) AS X FROM remote('127.0.0.{1,2}', test, hits) GROUP BY RegionID HAVING X > 100000 ORDER BY RegionID ASC; -SELECT RegionID, uniqCombined(17)(WatchID) AS X FROM remote('127.0.0.{1,2}', test, hits) GROUP BY RegionID HAVING X > 100000 ORDER BY RegionID ASC; +SELECT RegionID, uniqCombined(WatchID) AS X FROM remote('127.0.0.{1,2}', test, hits) GROUP BY RegionID HAVING X > 100000 ORDER BY RegionID ASC; SELECT abs(uniq(WatchID) - uniqExact(WatchID)) FROM test.hits; From 19ed7b6ae9d099d091ab01f5b8ca0a6a92b923d8 Mon Sep 17 00:00:00 2001 From: Ivan Lezhankin Date: Mon, 22 Oct 2018 13:00:37 +0300 Subject: [PATCH 013/124] Minor optimizations --- .../AggregateFunctionUniqCombined.h | 129 +++++++++--------- 1 file changed, 61 insertions(+), 68 deletions(-) diff --git a/dbms/src/AggregateFunctions/AggregateFunctionUniqCombined.h b/dbms/src/AggregateFunctions/AggregateFunctionUniqCombined.h index c4e1a571aee..234f24719c5 100644 --- a/dbms/src/AggregateFunctions/AggregateFunctionUniqCombined.h +++ b/dbms/src/AggregateFunctions/AggregateFunctionUniqCombined.h @@ -73,6 +73,8 @@ namespace detail template struct AggregateFunctionUniqCombinedDataWithKey { + using value_type = Key; + template using Set = CombinedCardinalityEstimator>, @@ -87,6 +89,7 @@ struct AggregateFunctionUniqCombinedDataWithKey mutable UInt8 inited = 0; union { + mutable char stub[sizeof(Set<20>)]; mutable Set<12> set_12; mutable Set<13> set_13; mutable Set<14> set_14; @@ -98,19 +101,12 @@ struct AggregateFunctionUniqCombinedDataWithKey mutable Set<20> set_20; }; -#define PASTE(x, y) x##y -#define EVAL(x, y) PASTE(x, y) -#define DEFAULT_SET EVAL(set_, DEFAULT_HLL_PRECISION) - - AggregateFunctionUniqCombinedDataWithKey() : DEFAULT_SET() {} + AggregateFunctionUniqCombinedDataWithKey() : stub() {} ~AggregateFunctionUniqCombinedDataWithKey() { switch (inited) { - case 0: - DEFAULT_SET.~CombinedCardinalityEstimator(); - break; case 12: set_12.~CombinedCardinalityEstimator(); break; @@ -141,21 +137,13 @@ struct AggregateFunctionUniqCombinedDataWithKey } } - void init(UInt8 precision) const + void ALWAYS_INLINE init(UInt8 precision) const { if (inited) return; - if (precision == DEFAULT_HLL_PRECISION) - { - inited = precision; - return; - } - // TODO: assert "inited == precision" - DEFAULT_SET.~CombinedCardinalityEstimator(); - switch (precision) { case 12: @@ -221,40 +209,39 @@ struct AggregateFunctionUniqCombinedDataWithKey break; \ } -#define SET_RETURN_METHOD(method) \ - switch (inited) \ - { \ - case 12: \ - return set_12.method; \ - case 13: \ - return set_13.method; \ - case 14: \ - return set_14.method; \ - case 15: \ - return set_15.method; \ - case 16: \ - return set_16.method; \ - case 17: \ - return set_17.method; \ - case 18: \ - return set_18.method; \ - case 19: \ - return set_19.method; \ - case 20: \ - return set_20.method; \ - default: \ - return DEFAULT_SET.method; \ +#define SET_RETURN_METHOD(method) \ + switch (inited) \ + { \ + case 12: \ + return set_12.method; \ + case 13: \ + return set_13.method; \ + case 14: \ + return set_14.method; \ + case 15: \ + return set_15.method; \ + case 16: \ + return set_16.method; \ + case 17: \ + return set_17.method; \ + case 18: \ + return set_18.method; \ + case 19: \ + return set_19.method; \ + case 20: \ + return set_20.method; \ + default: \ + /* TODO: UNREACHABLE! */ \ + return set_12.method; \ } - void insert(Key value, UInt8 precision) + void ALWAYS_INLINE insert(Key value) { - init(precision); SET_METHOD(insert(value)); } - void merge(const AggregateFunctionUniqCombinedDataWithKey & rhs, UInt8 precision) + void ALWAYS_INLINE merge(const AggregateFunctionUniqCombinedDataWithKey & rhs) { - init(precision); switch (inited) { case 12: @@ -287,41 +274,35 @@ struct AggregateFunctionUniqCombinedDataWithKey } } - void write(DB::WriteBuffer & out, UInt8 precision) const + void ALWAYS_INLINE write(DB::WriteBuffer & out) const { - init(precision); SET_METHOD(write(out)); } - void read(DB::ReadBuffer & in, UInt8 precision) + void ALWAYS_INLINE read(DB::ReadBuffer & in) { - init(precision); SET_METHOD(read(in)); } - UInt32 size(UInt8 precision) const + UInt32 ALWAYS_INLINE size() const { - init(precision); SET_RETURN_METHOD(size()); } #undef SET_METHOD #undef SET_RETURN_METHOD -#undef PASTE -#undef EVAL -#undef DEFAULT_SET #undef DEFAULT_HLL_PRECISION }; template -struct __attribute__((__packed__)) AggregateFunctionUniqCombinedData : public AggregateFunctionUniqCombinedDataWithKey +struct AggregateFunctionUniqCombinedData : public AggregateFunctionUniqCombinedDataWithKey { }; template <> -struct __attribute__((__packed__)) AggregateFunctionUniqCombinedData : public AggregateFunctionUniqCombinedDataWithKey +struct AggregateFunctionUniqCombinedData : public AggregateFunctionUniqCombinedDataWithKey { }; @@ -346,38 +327,44 @@ public: return std::make_shared(); } + void create(AggregateDataPtr place) const override + { + IAggregateFunctionDataHelper, AggregateFunctionUniqCombined>::create(place); + this->data(place).init(precision); + } + void add(AggregateDataPtr place, const IColumn ** columns, size_t row_num, Arena *) const override { if constexpr (!std::is_same_v) { const auto & value = static_cast &>(*columns[0]).getData()[row_num]; - this->data(place).insert(detail::AggregateFunctionUniqCombinedTraits::hash(value), precision); + this->data(place).insert(detail::AggregateFunctionUniqCombinedTraits::hash(value)); } else { StringRef value = columns[0]->getDataAt(row_num); - this->data(place).insert(CityHash_v1_0_2::CityHash64(value.data, value.size), precision); + this->data(place).insert(CityHash_v1_0_2::CityHash64(value.data, value.size)); } } void merge(AggregateDataPtr place, ConstAggregateDataPtr rhs, Arena *) const override { - this->data(place).merge(this->data(rhs), precision); + this->data(place).merge(this->data(rhs)); } void serialize(ConstAggregateDataPtr place, WriteBuffer & buf) const override { - this->data(place).write(buf, precision); + this->data(place).write(buf); } void deserialize(AggregateDataPtr place, ReadBuffer & buf, Arena *) const override { - this->data(place).read(buf, precision); + this->data(place).read(buf); } void insertResultInto(ConstAggregateDataPtr place, IColumn & to) const override { - static_cast(to).getData().push_back(this->data(place).size(precision)); + static_cast(to).getData().push_back(this->data(place).size()); } const char * getHeaderFilePath() const override @@ -417,31 +404,37 @@ public: return std::make_shared(); } + void create(AggregateDataPtr place) const override + { + IAggregateFunctionDataHelper, + AggregateFunctionUniqCombinedVariadic>::create(place); + this->data(place).init(precision); + } + void add(AggregateDataPtr place, const IColumn ** columns, size_t row_num, Arena *) const override { - this->data(place).insert(typename AggregateFunctionUniqCombinedData::Set<12>::value_type( - UniqVariadicHash::apply(num_args, columns, row_num)), - precision); + this->data(place).insert(typename AggregateFunctionUniqCombinedData::value_type( + UniqVariadicHash::apply(num_args, columns, row_num))); } void merge(AggregateDataPtr place, ConstAggregateDataPtr rhs, Arena *) const override { - this->data(place).merge(this->data(rhs), precision); + this->data(place).merge(this->data(rhs)); } void serialize(ConstAggregateDataPtr place, WriteBuffer & buf) const override { - this->data(place).write(buf, precision); + this->data(place).write(buf); } void deserialize(AggregateDataPtr place, ReadBuffer & buf, Arena *) const override { - this->data(place).read(buf, precision); + this->data(place).read(buf); } void insertResultInto(ConstAggregateDataPtr place, IColumn & to) const override { - static_cast(to).getData().push_back(this->data(place).size(precision)); + static_cast(to).getData().push_back(this->data(place).size()); } const char * getHeaderFilePath() const override From 2a53716043f4f32b972ea18e5a46a9e8c48bae53 Mon Sep 17 00:00:00 2001 From: Ivan Lezhankin Date: Mon, 22 Oct 2018 20:18:08 +0300 Subject: [PATCH 014/124] Move the precision template argument one level up. --- .../AggregateFunctionUniqCombined.cpp | 143 +++++---- .../AggregateFunctionUniqCombined.h | 277 ++---------------- 2 files changed, 113 insertions(+), 307 deletions(-) diff --git a/dbms/src/AggregateFunctions/AggregateFunctionUniqCombined.cpp b/dbms/src/AggregateFunctions/AggregateFunctionUniqCombined.cpp index e148c9157cd..1ad628588d6 100644 --- a/dbms/src/AggregateFunctions/AggregateFunctionUniqCombined.cpp +++ b/dbms/src/AggregateFunctions/AggregateFunctionUniqCombined.cpp @@ -8,7 +8,6 @@ namespace DB { - namespace ErrorCodes { extern const int ILLEGAL_TYPE_OF_ARGUMENT; @@ -18,67 +17,105 @@ namespace ErrorCodes namespace { - -AggregateFunctionPtr createAggregateFunctionUniqCombined( - const std::string & name, const DataTypes & argument_types, const Array & params) -{ - UInt8 precision = detail::UNIQ_COMBINED_DEFAULT_PRECISION; - - if (!params.empty()) + template + struct WithK { - if (params.size() != 1) - throw Exception( - "Aggregate function " + name + " requires one parameter or less.", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); + template + using AggregateFunction = AggregateFunctionUniqCombined; - UInt64 precision_param = applyVisitor(FieldVisitorConvertToNumber(), params[0]); + template + using AggregateFunctionVariadic = AggregateFunctionUniqCombinedVariadic; + }; - // This range is hardcoded into |AggregateFunctionUniqCombinedData| - if (precision_param > 20 || precision_param < 12) - throw Exception( - "Parameter for aggregate function " + name + "is out or range: [12, 20].", ErrorCodes::ARGUMENT_OUT_OF_BOUND); - - precision = precision_param; - } - - if (argument_types.empty()) - throw Exception("Incorrect number of arguments for aggregate function " + name, ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); - - /// We use exact hash function if the user wants it; - /// or if the arguments are not contiguous in memory, because only exact hash function have support for this case. - bool use_exact_hash_function = !isAllArgumentsContiguousInMemory(argument_types); - - if (argument_types.size() == 1) + template + AggregateFunctionPtr createAggregateFunctionWithK(const DataTypes & argument_types) { - const IDataType & argument_type = *argument_types[0]; + /// We use exact hash function if the user wants it; + /// or if the arguments are not contiguous in memory, because only exact hash function have support for this case. + bool use_exact_hash_function = !isAllArgumentsContiguousInMemory(argument_types); - AggregateFunctionPtr res(createWithNumericType(*argument_types[0], precision)); - - WhichDataType which(argument_type); - if (res) - return res; - else if (which.isDate()) - return std::make_shared>(precision); - else if (which.isDateTime()) - return std::make_shared>(precision); - else if (which.isStringOrFixedString()) - return std::make_shared>(precision); - else if (which.isUUID()) - return std::make_shared>(precision); - else if (which.isTuple()) + if (argument_types.size() == 1) { - if (use_exact_hash_function) - return std::make_shared>(argument_types, precision); - else - return std::make_shared>(argument_types, precision); + const IDataType & argument_type = *argument_types[0]; + + AggregateFunctionPtr res(createWithNumericType::template AggregateFunction>(*argument_types[0])); + + WhichDataType which(argument_type); + if (res) + return res; + else if (which.isDate()) + return std::make_shared::template AggregateFunction>(); + else if (which.isDateTime()) + return std::make_shared::template AggregateFunction>(); + else if (which.isStringOrFixedString()) + return std::make_shared::template AggregateFunction>(); + else if (which.isUUID()) + return std::make_shared::template AggregateFunction>(); + else if (which.isTuple()) + { + if (use_exact_hash_function) + return std::make_shared::template AggregateFunctionVariadic>(argument_types); + else + return std::make_shared::template AggregateFunctionVariadic>(argument_types); + } } + + /// "Variadic" method also works as a fallback generic case for a single argument. + if (use_exact_hash_function) + return std::make_shared::template AggregateFunctionVariadic>(argument_types); + else + return std::make_shared::template AggregateFunctionVariadic>(argument_types); } - /// "Variadic" method also works as a fallback generic case for single argument. - if (use_exact_hash_function) - return std::make_shared>(argument_types, precision); - else - return std::make_shared>(argument_types, precision); -} + AggregateFunctionPtr createAggregateFunctionUniqCombined( + const std::string & name, const DataTypes & argument_types, const Array & params) + { + UInt8 precision = 17; + + if (!params.empty()) + { + if (params.size() != 1) + throw Exception( + "Aggregate function " + name + " requires one parameter or less.", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); + + UInt64 precision_param = applyVisitor(FieldVisitorConvertToNumber(), params[0]); + + // This range is hardcoded below + if (precision_param > 20 || precision_param < 12) + throw Exception( + "Parameter for aggregate function " + name + "is out or range: [12, 20].", ErrorCodes::ARGUMENT_OUT_OF_BOUND); + + precision = precision_param; + } + + if (argument_types.empty()) + throw Exception("Incorrect number of arguments for aggregate function " + name, ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); + + switch (precision) + { + case 12: + return createAggregateFunctionWithK<12>(argument_types); + case 13: + return createAggregateFunctionWithK<13>(argument_types); + case 14: + return createAggregateFunctionWithK<14>(argument_types); + case 15: + return createAggregateFunctionWithK<15>(argument_types); + case 16: + return createAggregateFunctionWithK<16>(argument_types); + case 17: + return createAggregateFunctionWithK<17>(argument_types); + case 18: + return createAggregateFunctionWithK<18>(argument_types); + case 19: + return createAggregateFunctionWithK<19>(argument_types); + case 20: + return createAggregateFunctionWithK<20>(argument_types); + } + + // TODO: not reached! + return {}; + } } // namespace diff --git a/dbms/src/AggregateFunctions/AggregateFunctionUniqCombined.h b/dbms/src/AggregateFunctions/AggregateFunctionUniqCombined.h index 234f24719c5..1e855cb8f5f 100644 --- a/dbms/src/AggregateFunctions/AggregateFunctionUniqCombined.h +++ b/dbms/src/AggregateFunctions/AggregateFunctionUniqCombined.h @@ -19,14 +19,10 @@ #include -#define DEFAULT_HLL_PRECISION 17 - namespace DB { namespace detail { - const UInt8 UNIQ_COMBINED_DEFAULT_PRECISION = DEFAULT_HLL_PRECISION; - /** Hash function for uniqCombined. */ template @@ -70,12 +66,9 @@ namespace detail } // namespace detail -template +template struct AggregateFunctionUniqCombinedDataWithKey { - using value_type = Key; - - template using Set = CombinedCardinalityEstimator>, 16, @@ -86,237 +79,27 @@ struct AggregateFunctionUniqCombinedDataWithKey HyperLogLogBiasEstimator, HyperLogLogMode::FullFeatured>; - mutable UInt8 inited = 0; - union - { - mutable char stub[sizeof(Set<20>)]; - mutable Set<12> set_12; - mutable Set<13> set_13; - mutable Set<14> set_14; - mutable Set<15> set_15; - mutable Set<16> set_16; - mutable Set<17> set_17; - mutable Set<18> set_18; - mutable Set<19> set_19; - mutable Set<20> set_20; - }; - - AggregateFunctionUniqCombinedDataWithKey() : stub() {} - - ~AggregateFunctionUniqCombinedDataWithKey() - { - switch (inited) - { - case 12: - set_12.~CombinedCardinalityEstimator(); - break; - case 13: - set_13.~CombinedCardinalityEstimator(); - break; - case 14: - set_14.~CombinedCardinalityEstimator(); - break; - case 15: - set_15.~CombinedCardinalityEstimator(); - break; - case 16: - set_16.~CombinedCardinalityEstimator(); - break; - case 17: - set_17.~CombinedCardinalityEstimator(); - break; - case 18: - set_18.~CombinedCardinalityEstimator(); - break; - case 19: - set_19.~CombinedCardinalityEstimator(); - break; - case 20: - set_20.~CombinedCardinalityEstimator(); - break; - } - } - - void ALWAYS_INLINE init(UInt8 precision) const - { - if (inited) - return; - - // TODO: assert "inited == precision" - - switch (precision) - { - case 12: - new (&set_12) Set<12>; - break; - case 13: - new (&set_13) Set<13>; - break; - case 14: - new (&set_14) Set<14>; - break; - case 15: - new (&set_15) Set<15>; - break; - case 16: - new (&set_16) Set<16>; - break; - case 17: - new (&set_17) Set<17>; - break; - case 18: - new (&set_18) Set<18>; - break; - case 19: - new (&set_19) Set<19>; - break; - case 20: - new (&set_20) Set<20>; - break; - } - inited = precision; - } - -#define SET_METHOD(method) \ - switch (inited) \ - { \ - case 12: \ - set_12.method; \ - break; \ - case 13: \ - set_13.method; \ - break; \ - case 14: \ - set_14.method; \ - break; \ - case 15: \ - set_15.method; \ - break; \ - case 16: \ - set_16.method; \ - break; \ - case 17: \ - set_17.method; \ - break; \ - case 18: \ - set_18.method; \ - break; \ - case 19: \ - set_19.method; \ - break; \ - case 20: \ - set_20.method; \ - break; \ - } - -#define SET_RETURN_METHOD(method) \ - switch (inited) \ - { \ - case 12: \ - return set_12.method; \ - case 13: \ - return set_13.method; \ - case 14: \ - return set_14.method; \ - case 15: \ - return set_15.method; \ - case 16: \ - return set_16.method; \ - case 17: \ - return set_17.method; \ - case 18: \ - return set_18.method; \ - case 19: \ - return set_19.method; \ - case 20: \ - return set_20.method; \ - default: \ - /* TODO: UNREACHABLE! */ \ - return set_12.method; \ - } - - void ALWAYS_INLINE insert(Key value) - { - SET_METHOD(insert(value)); - } - - void ALWAYS_INLINE merge(const AggregateFunctionUniqCombinedDataWithKey & rhs) - { - switch (inited) - { - case 12: - set_12.merge(rhs.set_12); - break; - case 13: - set_13.merge(rhs.set_13); - break; - case 14: - set_14.merge(rhs.set_14); - break; - case 15: - set_15.merge(rhs.set_15); - break; - case 16: - set_16.merge(rhs.set_16); - break; - case 17: - set_17.merge(rhs.set_17); - break; - case 18: - set_18.merge(rhs.set_18); - break; - case 19: - set_19.merge(rhs.set_19); - break; - case 20: - set_20.merge(rhs.set_20); - break; - } - } - - void ALWAYS_INLINE write(DB::WriteBuffer & out) const - { - SET_METHOD(write(out)); - } - - void ALWAYS_INLINE read(DB::ReadBuffer & in) - { - SET_METHOD(read(in)); - } - - UInt32 ALWAYS_INLINE size() const - { - SET_RETURN_METHOD(size()); - } - -#undef SET_METHOD -#undef SET_RETURN_METHOD -#undef DEFAULT_HLL_PRECISION + Set set; }; -template -struct AggregateFunctionUniqCombinedData : public AggregateFunctionUniqCombinedDataWithKey +template +struct AggregateFunctionUniqCombinedData : public AggregateFunctionUniqCombinedDataWithKey { }; -template <> -struct AggregateFunctionUniqCombinedData : public AggregateFunctionUniqCombinedDataWithKey +template +struct AggregateFunctionUniqCombinedData : public AggregateFunctionUniqCombinedDataWithKey { }; -template +template class AggregateFunctionUniqCombined final - : public IAggregateFunctionDataHelper, AggregateFunctionUniqCombined> + : public IAggregateFunctionDataHelper, AggregateFunctionUniqCombined> { -private: - const UInt8 precision; - public: - explicit AggregateFunctionUniqCombined(UInt8 precision) : precision(precision) {} - String getName() const override { return "uniqCombined"; @@ -327,44 +110,38 @@ public: return std::make_shared(); } - void create(AggregateDataPtr place) const override - { - IAggregateFunctionDataHelper, AggregateFunctionUniqCombined>::create(place); - this->data(place).init(precision); - } - void add(AggregateDataPtr place, const IColumn ** columns, size_t row_num, Arena *) const override { if constexpr (!std::is_same_v) { const auto & value = static_cast &>(*columns[0]).getData()[row_num]; - this->data(place).insert(detail::AggregateFunctionUniqCombinedTraits::hash(value)); + this->data(place).set.insert(detail::AggregateFunctionUniqCombinedTraits::hash(value)); } else { StringRef value = columns[0]->getDataAt(row_num); - this->data(place).insert(CityHash_v1_0_2::CityHash64(value.data, value.size)); + this->data(place).set.insert(CityHash_v1_0_2::CityHash64(value.data, value.size)); } } void merge(AggregateDataPtr place, ConstAggregateDataPtr rhs, Arena *) const override { - this->data(place).merge(this->data(rhs)); + this->data(place).set.merge(this->data(rhs).set); } void serialize(ConstAggregateDataPtr place, WriteBuffer & buf) const override { - this->data(place).write(buf); + this->data(place).set.write(buf); } void deserialize(AggregateDataPtr place, ReadBuffer & buf, Arena *) const override { - this->data(place).read(buf); + this->data(place).set.read(buf); } void insertResultInto(ConstAggregateDataPtr place, IColumn & to) const override { - static_cast(to).getData().push_back(this->data(place).size()); + static_cast(to).getData().push_back(this->data(place).set.size()); } const char * getHeaderFilePath() const override @@ -377,16 +154,15 @@ public: * You can pass multiple arguments as is; You can also pass one argument - a tuple. * But (for the possibility of efficient implementation), you can not pass several arguments, among which there are tuples. */ -template -class AggregateFunctionUniqCombinedVariadic final : public IAggregateFunctionDataHelper, - AggregateFunctionUniqCombinedVariadic> +template +class AggregateFunctionUniqCombinedVariadic final : public IAggregateFunctionDataHelper, + AggregateFunctionUniqCombinedVariadic> { private: size_t num_args = 0; - UInt8 precision; public: - AggregateFunctionUniqCombinedVariadic(const DataTypes & arguments, UInt8 precision) : precision(precision) + explicit AggregateFunctionUniqCombinedVariadic(const DataTypes & arguments) { if (argument_is_tuple) num_args = typeid_cast(*arguments[0]).getElements().size(); @@ -404,37 +180,30 @@ public: return std::make_shared(); } - void create(AggregateDataPtr place) const override - { - IAggregateFunctionDataHelper, - AggregateFunctionUniqCombinedVariadic>::create(place); - this->data(place).init(precision); - } - void add(AggregateDataPtr place, const IColumn ** columns, size_t row_num, Arena *) const override { - this->data(place).insert(typename AggregateFunctionUniqCombinedData::value_type( + this->data(place).set.insert(typename AggregateFunctionUniqCombinedData::Set::value_type( UniqVariadicHash::apply(num_args, columns, row_num))); } void merge(AggregateDataPtr place, ConstAggregateDataPtr rhs, Arena *) const override { - this->data(place).merge(this->data(rhs)); + this->data(place).set.merge(this->data(rhs).set); } void serialize(ConstAggregateDataPtr place, WriteBuffer & buf) const override { - this->data(place).write(buf); + this->data(place).set.write(buf); } void deserialize(AggregateDataPtr place, ReadBuffer & buf, Arena *) const override { - this->data(place).read(buf); + this->data(place).set.read(buf); } void insertResultInto(ConstAggregateDataPtr place, IColumn & to) const override { - static_cast(to).getData().push_back(this->data(place).size()); + static_cast(to).getData().push_back(this->data(place).set.size()); } const char * getHeaderFilePath() const override From c8868cffaa6214584af13d5b335afc413dd603f9 Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Mon, 22 Oct 2018 22:34:39 +0300 Subject: [PATCH 015/124] Update AggregateFunctionUniqCombined.cpp --- dbms/src/AggregateFunctions/AggregateFunctionUniqCombined.cpp | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/dbms/src/AggregateFunctions/AggregateFunctionUniqCombined.cpp b/dbms/src/AggregateFunctions/AggregateFunctionUniqCombined.cpp index 1ad628588d6..3c747d94e63 100644 --- a/dbms/src/AggregateFunctions/AggregateFunctionUniqCombined.cpp +++ b/dbms/src/AggregateFunctions/AggregateFunctionUniqCombined.cpp @@ -30,8 +30,7 @@ namespace template AggregateFunctionPtr createAggregateFunctionWithK(const DataTypes & argument_types) { - /// We use exact hash function if the user wants it; - /// or if the arguments are not contiguous in memory, because only exact hash function have support for this case. + /// We use exact hash function if the arguments are not contiguous in memory, because only exact hash function has support for this case. bool use_exact_hash_function = !isAllArgumentsContiguousInMemory(argument_types); if (argument_types.size() == 1) From f90783d885a040aef48aed0725d86b2bba9a0092 Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Mon, 22 Oct 2018 22:36:32 +0300 Subject: [PATCH 016/124] Update AggregateFunctionUniqCombined.cpp --- dbms/src/AggregateFunctions/AggregateFunctionUniqCombined.cpp | 1 + 1 file changed, 1 insertion(+) diff --git a/dbms/src/AggregateFunctions/AggregateFunctionUniqCombined.cpp b/dbms/src/AggregateFunctions/AggregateFunctionUniqCombined.cpp index 3c747d94e63..0430c8872f1 100644 --- a/dbms/src/AggregateFunctions/AggregateFunctionUniqCombined.cpp +++ b/dbms/src/AggregateFunctions/AggregateFunctionUniqCombined.cpp @@ -69,6 +69,7 @@ namespace AggregateFunctionPtr createAggregateFunctionUniqCombined( const std::string & name, const DataTypes & argument_types, const Array & params) { + /// Reasonable default value, selected to be comparable in quality with "uniq" aggregate function. UInt8 precision = 17; if (!params.empty()) From bd148bb082c99a622b9ccdc85ec676c23ffea139 Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Mon, 22 Oct 2018 22:37:55 +0300 Subject: [PATCH 017/124] Update AggregateFunctionUniqCombined.cpp --- dbms/src/AggregateFunctions/AggregateFunctionUniqCombined.cpp | 1 + 1 file changed, 1 insertion(+) diff --git a/dbms/src/AggregateFunctions/AggregateFunctionUniqCombined.cpp b/dbms/src/AggregateFunctions/AggregateFunctionUniqCombined.cpp index 0430c8872f1..0ad4a093ed5 100644 --- a/dbms/src/AggregateFunctions/AggregateFunctionUniqCombined.cpp +++ b/dbms/src/AggregateFunctions/AggregateFunctionUniqCombined.cpp @@ -69,6 +69,7 @@ namespace AggregateFunctionPtr createAggregateFunctionUniqCombined( const std::string & name, const DataTypes & argument_types, const Array & params) { + /// log2 of the number of cells in HyperLogLog. /// Reasonable default value, selected to be comparable in quality with "uniq" aggregate function. UInt8 precision = 17; From fd86a6bea5b51aca607889819b2dff9c0e120503 Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Mon, 22 Oct 2018 22:38:46 +0300 Subject: [PATCH 018/124] Update AggregateFunctionUniqCombined.cpp --- dbms/src/AggregateFunctions/AggregateFunctionUniqCombined.cpp | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/dbms/src/AggregateFunctions/AggregateFunctionUniqCombined.cpp b/dbms/src/AggregateFunctions/AggregateFunctionUniqCombined.cpp index 0ad4a093ed5..90b84d3b927 100644 --- a/dbms/src/AggregateFunctions/AggregateFunctionUniqCombined.cpp +++ b/dbms/src/AggregateFunctions/AggregateFunctionUniqCombined.cpp @@ -114,8 +114,7 @@ namespace return createAggregateFunctionWithK<20>(argument_types); } - // TODO: not reached! - return {}; + __builtin_unreachable(); } } // namespace From 9296ef50089b8427734f8a9b0e59b6ba98b2b0d0 Mon Sep 17 00:00:00 2001 From: Ivan Lezhankin Date: Tue, 23 Oct 2018 17:59:24 +0300 Subject: [PATCH 019/124] Temporarily disable precalculated bias estimations for precisions other than 17. --- dbms/scripts/gen-bias-data.py | 2 +- .../AggregateFunctionUniqCombined.h | 12 ++++++++++-- 2 files changed, 11 insertions(+), 3 deletions(-) diff --git a/dbms/scripts/gen-bias-data.py b/dbms/scripts/gen-bias-data.py index 7edc9948e76..034cfcca7dd 100755 --- a/dbms/scripts/gen-bias-data.py +++ b/dbms/scripts/gen-bias-data.py @@ -1,4 +1,4 @@ -#!/usr/bin/python3.4 +#!/usr/bin/python3 # -*- coding: utf-8 -*- import sys diff --git a/dbms/src/AggregateFunctions/AggregateFunctionUniqCombined.h b/dbms/src/AggregateFunctions/AggregateFunctionUniqCombined.h index 1e855cb8f5f..fac220b7bbb 100644 --- a/dbms/src/AggregateFunctions/AggregateFunctionUniqCombined.h +++ b/dbms/src/AggregateFunctions/AggregateFunctionUniqCombined.h @@ -68,12 +68,20 @@ namespace detail template struct AggregateFunctionUniqCombinedDataWithKey +{ + using Set = CombinedCardinalityEstimator>, 16, K - 3, K, TrivialHash, Key>; + + Set set; +}; + +template +struct AggregateFunctionUniqCombinedDataWithKey { using Set = CombinedCardinalityEstimator>, 16, - K - 3, - K, + 14, + 17, TrivialHash, Key, HyperLogLogBiasEstimator, From aa6a69b6350e668014b6183e170666543173694f Mon Sep 17 00:00:00 2001 From: Ivan Lezhankin Date: Wed, 24 Oct 2018 17:28:23 +0300 Subject: [PATCH 020/124] Add TODO comment --- dbms/src/AggregateFunctions/AggregateFunctionUniqCombined.h | 2 ++ 1 file changed, 2 insertions(+) diff --git a/dbms/src/AggregateFunctions/AggregateFunctionUniqCombined.h b/dbms/src/AggregateFunctions/AggregateFunctionUniqCombined.h index fac220b7bbb..f15c4e7e453 100644 --- a/dbms/src/AggregateFunctions/AggregateFunctionUniqCombined.h +++ b/dbms/src/AggregateFunctions/AggregateFunctionUniqCombined.h @@ -69,6 +69,8 @@ namespace detail template struct AggregateFunctionUniqCombinedDataWithKey { + // TODO(ilezhankin): pre-generate values for |UniqCombinedBiasData|, + // at the moment gen-bias-data.py script doesn't work. using Set = CombinedCardinalityEstimator>, 16, K - 3, K, TrivialHash, Key>; Set set; From 68138a76f8191db8fb21601989b56702a1492e4c Mon Sep 17 00:00:00 2001 From: Ivan Lezhankin Date: Wed, 24 Oct 2018 21:05:55 +0300 Subject: [PATCH 021/124] Add tests for different HLL precisions --- ...12_shard_aggregate_function_uniq.reference | 418 ++++++++++++++++++ .../00212_shard_aggregate_function_uniq.sql | 36 ++ 2 files changed, 454 insertions(+) diff --git a/dbms/tests/queries/0_stateless/00212_shard_aggregate_function_uniq.reference b/dbms/tests/queries/0_stateless/00212_shard_aggregate_function_uniq.reference index c7c4f8ebacd..7ca0f2fb7be 100644 --- a/dbms/tests/queries/0_stateless/00212_shard_aggregate_function_uniq.reference +++ b/dbms/tests/queries/0_stateless/00212_shard_aggregate_function_uniq.reference @@ -292,6 +292,58 @@ 31 162 35 162 36 162 +0 54708 +1 53721 +3 53226 +6 54532 +7 52275 +9 53417 +10 26931 +11 54428 +13 53409 +14 53188 +17 55120 +19 54123 +20 53293 +21 53928 +22 53827 +26 53920 +31 53763 +35 54635 +36 53155 +1 1 +3 1 +6 1 +7 1 +9 1 +11 1 +14 1 +17 1 +19 1 +20 2 +26 1 +31 1 +35 1 +36 1 +0 162 +1 162 +3 162 +6 162 +7 163 +9 163 +10 81 +11 163 +13 162 +14 162 +17 162 +19 162 +20 162 +21 162 +22 162 +26 162 +31 162 +35 162 +36 162 0 53901 1 54056 3 53999 @@ -311,6 +363,58 @@ 31 53948 35 53931 36 53982 +1 1 +3 1 +6 1 +7 1 +9 1 +11 1 +14 1 +17 1 +19 1 +20 2 +26 1 +31 1 +35 1 +36 1 +0 162 +1 162 +3 162 +6 162 +7 163 +9 163 +10 81 +11 163 +13 162 +14 162 +17 162 +19 162 +20 162 +21 162 +22 162 +26 162 +31 162 +35 162 +36 162 +0 54054 +1 54054 +3 54054 +6 54054 +7 54053 +9 54053 +10 27027 +11 54055 +13 54054 +14 54054 +17 54054 +19 54053 +20 54053 +21 54054 +22 54053 +26 54053 +31 54054 +35 54054 +36 54054 0.125 1 0.5 1 0.05 1 @@ -396,6 +500,58 @@ 0.043 162 0.037 162 0.071 162 +0.045 53928 +0.125 52275 +0.5 53721 +0.05 54123 +0.143 54532 +0.091 26931 +0.056 55120 +0.048 53293 +0.083 54428 +0.25 53226 +1 54708 +0.1 53417 +0.028 54635 +0.027 53155 +0.031 53763 +0.067 53188 +0.043 53827 +0.037 53920 +0.071 53409 +0.125 1 +0.5 1 +0.05 1 +0.143 1 +0.056 1 +0.048 2 +0.083 1 +0.25 1 +0.1 1 +0.028 1 +0.027 1 +0.031 1 +0.067 1 +0.037 1 +0.045 162 +0.125 163 +0.5 162 +0.05 162 +0.143 162 +0.091 81 +0.056 162 +0.048 162 +0.083 163 +0.25 162 +1 162 +0.1 163 +0.028 162 +0.027 162 +0.031 162 +0.067 162 +0.043 162 +0.037 162 +0.071 162 0.045 54117 0.125 54213 0.5 54056 @@ -415,6 +571,58 @@ 0.043 54150 0.037 54047 0.071 53963 +0.125 1 +0.5 1 +0.05 1 +0.143 1 +0.056 1 +0.048 2 +0.083 1 +0.25 1 +0.1 1 +0.028 1 +0.027 1 +0.031 1 +0.067 1 +0.037 1 +0.045 162 +0.125 163 +0.5 162 +0.05 162 +0.143 162 +0.091 81 +0.056 162 +0.048 162 +0.083 163 +0.25 162 +1 162 +0.1 163 +0.028 162 +0.027 162 +0.031 162 +0.067 162 +0.043 162 +0.037 162 +0.071 162 +0.045 54054 +0.125 54053 +0.5 54054 +0.05 54053 +0.143 54054 +0.091 27027 +0.056 54054 +0.048 54053 +0.083 54055 +0.25 54054 +1 54054 +0.1 54053 +0.028 54054 +0.027 54054 +0.031 54054 +0.067 54054 +0.043 54053 +0.037 54053 +0.071 54054 0.5 1 0.05 1 0.25 1 @@ -500,6 +708,58 @@ 0.037 162 0.1 163 1 162 +0.5 53721 +0.05 54123 +0.25 53226 +0.048 53293 +0.091 26931 +0.043 53827 +0.071 53409 +0.083 54428 +0.125 52275 +0.031 53763 +0.143 54532 +0.028 54635 +0.067 53188 +0.045 53928 +0.027 53155 +0.056 55120 +0.037 53920 +0.1 53417 +1 54708 +0.5 1 +0.05 1 +0.25 1 +0.048 2 +0.083 1 +0.125 1 +0.031 1 +0.143 1 +0.028 1 +0.067 1 +0.027 1 +0.056 1 +0.037 1 +0.1 1 +0.5 162 +0.05 162 +0.25 162 +0.048 162 +0.091 81 +0.043 162 +0.071 162 +0.083 163 +0.125 163 +0.031 162 +0.143 162 +0.028 162 +0.067 162 +0.045 162 +0.027 162 +0.056 162 +0.037 162 +0.1 163 +1 162 0.5 54056 0.05 53923 0.25 53999 @@ -519,6 +779,58 @@ 0.037 54047 0.1 53853 1 53901 +0.5 1 +0.05 1 +0.25 1 +0.048 2 +0.083 1 +0.125 1 +0.031 1 +0.143 1 +0.028 1 +0.067 1 +0.027 1 +0.056 1 +0.037 1 +0.1 1 +0.5 162 +0.05 162 +0.25 162 +0.048 162 +0.091 81 +0.043 162 +0.071 162 +0.083 163 +0.125 163 +0.031 162 +0.143 162 +0.028 162 +0.067 162 +0.045 162 +0.027 162 +0.056 162 +0.037 162 +0.1 163 +1 162 +0.5 54054 +0.05 54053 +0.25 54054 +0.048 54053 +0.091 27027 +0.043 54053 +0.071 54054 +0.083 54055 +0.125 54053 +0.031 54054 +0.143 54054 +0.028 54054 +0.067 54054 +0.045 54054 +0.027 54054 +0.056 54054 +0.037 54053 +0.1 54053 +1 54054 1 1 3 1 6 1 @@ -604,6 +916,58 @@ 31 162 35 162 36 162 +0 52613 +1 54468 +3 53824 +6 54441 +7 54543 +9 51908 +10 26964 +11 54013 +13 53178 +14 54113 +17 54662 +19 54697 +20 53279 +21 55301 +22 53693 +26 53873 +31 55200 +35 54808 +36 53051 +1 1 +3 1 +6 1 +7 1 +9 1 +11 1 +14 1 +17 1 +19 1 +20 2 +26 1 +31 1 +35 1 +36 1 +0 162 +1 162 +3 162 +6 162 +7 163 +9 163 +10 81 +11 163 +13 162 +14 162 +17 162 +19 162 +20 162 +21 162 +22 162 +26 162 +31 162 +35 162 +36 162 0 54195 1 54086 3 54127 @@ -623,5 +987,59 @@ 31 54074 35 54153 36 53999 +1 1 +3 1 +6 1 +7 1 +9 1 +11 1 +14 1 +17 1 +19 1 +20 2 +26 1 +31 1 +35 1 +36 1 +0 162 +1 162 +3 162 +6 162 +7 163 +9 163 +10 81 +11 163 +13 162 +14 162 +17 162 +19 162 +20 162 +21 162 +22 162 +26 162 +31 162 +35 162 +36 162 +0 54054 +1 54054 +3 54054 +6 54054 +7 54054 +9 54054 +10 27027 +11 54055 +13 54054 +14 54054 +17 54054 +19 54054 +20 54054 +21 54054 +22 54054 +26 54054 +31 54054 +35 54054 +36 54054 +1 +1 1 1 diff --git a/dbms/tests/queries/0_stateless/00212_shard_aggregate_function_uniq.sql b/dbms/tests/queries/0_stateless/00212_shard_aggregate_function_uniq.sql index a2ab15775c5..ae54831b1af 100644 --- a/dbms/tests/queries/0_stateless/00212_shard_aggregate_function_uniq.sql +++ b/dbms/tests/queries/0_stateless/00212_shard_aggregate_function_uniq.sql @@ -24,34 +24,70 @@ SELECT Y, uniqCombined(X) FROM (SELECT number AS X, (3*X*X - 7*X + 11) % 37 AS Y SELECT Y, uniqCombined(X) FROM (SELECT number AS X, (3*X*X - 7*X + 11) % 37 AS Y FROM system.numbers LIMIT 3000) GROUP BY Y; SELECT Y, uniqCombined(X) FROM (SELECT number AS X, (3*X*X - 7*X + 11) % 37 AS Y FROM system.numbers LIMIT 1000000) GROUP BY Y; +SELECT Y, uniqCombined(12)(X) FROM (SELECT number AS X, (3*X*X - 7*X + 11) % 37 AS Y FROM system.numbers LIMIT 15) GROUP BY Y; +SELECT Y, uniqCombined(12)(X) FROM (SELECT number AS X, (3*X*X - 7*X + 11) % 37 AS Y FROM system.numbers LIMIT 3000) GROUP BY Y; +SELECT Y, uniqCombined(12)(X) FROM (SELECT number AS X, (3*X*X - 7*X + 11) % 37 AS Y FROM system.numbers LIMIT 1000000) GROUP BY Y; + SELECT Y, uniqCombined(17)(X) FROM (SELECT number AS X, (3*X*X - 7*X + 11) % 37 AS Y FROM system.numbers LIMIT 15) GROUP BY Y; SELECT Y, uniqCombined(17)(X) FROM (SELECT number AS X, (3*X*X - 7*X + 11) % 37 AS Y FROM system.numbers LIMIT 3000) GROUP BY Y; SELECT Y, uniqCombined(17)(X) FROM (SELECT number AS X, (3*X*X - 7*X + 11) % 37 AS Y FROM system.numbers LIMIT 1000000) GROUP BY Y; +SELECT Y, uniqCombined(20)(X) FROM (SELECT number AS X, (3*X*X - 7*X + 11) % 37 AS Y FROM system.numbers LIMIT 15) GROUP BY Y; +SELECT Y, uniqCombined(20)(X) FROM (SELECT number AS X, (3*X*X - 7*X + 11) % 37 AS Y FROM system.numbers LIMIT 3000) GROUP BY Y; +SELECT Y, uniqCombined(20)(X) FROM (SELECT number AS X, (3*X*X - 7*X + 11) % 37 AS Y FROM system.numbers LIMIT 1000000) GROUP BY Y; + SELECT Y, uniqCombined(X) FROM (SELECT number AS X, round(1/(1 + (3*X*X - 7*X + 11) % 37), 3) AS Y FROM system.numbers LIMIT 15) GROUP BY Y; SELECT Y, uniqCombined(X) FROM (SELECT number AS X, round(1/(1 + (3*X*X - 7*X + 11) % 37), 3) AS Y FROM system.numbers LIMIT 3000) GROUP BY Y; SELECT Y, uniqCombined(X) FROM (SELECT number AS X, round(1/(1 + (3*X*X - 7*X + 11) % 37), 3) AS Y FROM system.numbers LIMIT 1000000) GROUP BY Y; +SELECT Y, uniqCombined(12)(X) FROM (SELECT number AS X, round(1/(1 + (3*X*X - 7*X + 11) % 37), 3) AS Y FROM system.numbers LIMIT 15) GROUP BY Y; +SELECT Y, uniqCombined(12)(X) FROM (SELECT number AS X, round(1/(1 + (3*X*X - 7*X + 11) % 37), 3) AS Y FROM system.numbers LIMIT 3000) GROUP BY Y; +SELECT Y, uniqCombined(12)(X) FROM (SELECT number AS X, round(1/(1 + (3*X*X - 7*X + 11) % 37), 3) AS Y FROM system.numbers LIMIT 1000000) GROUP BY Y; + SELECT Y, uniqCombined(17)(X) FROM (SELECT number AS X, round(1/(1 + (3*X*X - 7*X + 11) % 37), 3) AS Y FROM system.numbers LIMIT 15) GROUP BY Y; SELECT Y, uniqCombined(17)(X) FROM (SELECT number AS X, round(1/(1 + (3*X*X - 7*X + 11) % 37), 3) AS Y FROM system.numbers LIMIT 3000) GROUP BY Y; SELECT Y, uniqCombined(17)(X) FROM (SELECT number AS X, round(1/(1 + (3*X*X - 7*X + 11) % 37), 3) AS Y FROM system.numbers LIMIT 1000000) GROUP BY Y; +SELECT Y, uniqCombined(20)(X) FROM (SELECT number AS X, round(1/(1 + (3*X*X - 7*X + 11) % 37), 3) AS Y FROM system.numbers LIMIT 15) GROUP BY Y; +SELECT Y, uniqCombined(20)(X) FROM (SELECT number AS X, round(1/(1 + (3*X*X - 7*X + 11) % 37), 3) AS Y FROM system.numbers LIMIT 3000) GROUP BY Y; +SELECT Y, uniqCombined(20)(X) FROM (SELECT number AS X, round(1/(1 + (3*X*X - 7*X + 11) % 37), 3) AS Y FROM system.numbers LIMIT 1000000) GROUP BY Y; + SELECT Y, uniqCombined(X) FROM (SELECT number AS X, round(toFloat32(1/(1 + (3*X*X - 7*X + 11) % 37)), 3) AS Y FROM system.numbers LIMIT 15) GROUP BY Y; SELECT Y, uniqCombined(X) FROM (SELECT number AS X, round(toFloat32(1/(1 + (3*X*X - 7*X + 11) % 37)), 3) AS Y FROM system.numbers LIMIT 3000) GROUP BY Y; SELECT Y, uniqCombined(X) FROM (SELECT number AS X, round(toFloat32(1/(1 + (3*X*X - 7*X + 11) % 37)), 3) AS Y FROM system.numbers LIMIT 1000000) GROUP BY Y; +SELECT Y, uniqCombined(12)(X) FROM (SELECT number AS X, round(toFloat32(1/(1 + (3*X*X - 7*X + 11) % 37)), 3) AS Y FROM system.numbers LIMIT 15) GROUP BY Y; +SELECT Y, uniqCombined(12)(X) FROM (SELECT number AS X, round(toFloat32(1/(1 + (3*X*X - 7*X + 11) % 37)), 3) AS Y FROM system.numbers LIMIT 3000) GROUP BY Y; +SELECT Y, uniqCombined(12)(X) FROM (SELECT number AS X, round(toFloat32(1/(1 + (3*X*X - 7*X + 11) % 37)), 3) AS Y FROM system.numbers LIMIT 1000000) GROUP BY Y; + SELECT Y, uniqCombined(17)(X) FROM (SELECT number AS X, round(toFloat32(1/(1 + (3*X*X - 7*X + 11) % 37)), 3) AS Y FROM system.numbers LIMIT 15) GROUP BY Y; SELECT Y, uniqCombined(17)(X) FROM (SELECT number AS X, round(toFloat32(1/(1 + (3*X*X - 7*X + 11) % 37)), 3) AS Y FROM system.numbers LIMIT 3000) GROUP BY Y; SELECT Y, uniqCombined(17)(X) FROM (SELECT number AS X, round(toFloat32(1/(1 + (3*X*X - 7*X + 11) % 37)), 3) AS Y FROM system.numbers LIMIT 1000000) GROUP BY Y; +SELECT Y, uniqCombined(20)(X) FROM (SELECT number AS X, round(toFloat32(1/(1 + (3*X*X - 7*X + 11) % 37)), 3) AS Y FROM system.numbers LIMIT 15) GROUP BY Y; +SELECT Y, uniqCombined(20)(X) FROM (SELECT number AS X, round(toFloat32(1/(1 + (3*X*X - 7*X + 11) % 37)), 3) AS Y FROM system.numbers LIMIT 3000) GROUP BY Y; +SELECT Y, uniqCombined(20)(X) FROM (SELECT number AS X, round(toFloat32(1/(1 + (3*X*X - 7*X + 11) % 37)), 3) AS Y FROM system.numbers LIMIT 1000000) GROUP BY Y; + SELECT Y, uniqCombined(Z) FROM (SELECT number AS X, IPv4NumToString(toUInt32(X)) AS Z, (3*X*X - 7*X + 11) % 37 AS Y FROM system.numbers LIMIT 15) GROUP BY Y; SELECT Y, uniqCombined(Z) FROM (SELECT number AS X, IPv4NumToString(toUInt32(X)) AS Z, (3*X*X - 7*X + 11) % 37 AS Y FROM system.numbers LIMIT 3000) GROUP BY Y; SELECT Y, uniqCombined(Z) FROM (SELECT number AS X, IPv4NumToString(toUInt32(X)) AS Z, (3*X*X - 7*X + 11) % 37 AS Y FROM system.numbers LIMIT 1000000) GROUP BY Y; +SELECT Y, uniqCombined(12)(Z) FROM (SELECT number AS X, IPv4NumToString(toUInt32(X)) AS Z, (3*X*X - 7*X + 11) % 37 AS Y FROM system.numbers LIMIT 15) GROUP BY Y; +SELECT Y, uniqCombined(12)(Z) FROM (SELECT number AS X, IPv4NumToString(toUInt32(X)) AS Z, (3*X*X - 7*X + 11) % 37 AS Y FROM system.numbers LIMIT 3000) GROUP BY Y; +SELECT Y, uniqCombined(12)(Z) FROM (SELECT number AS X, IPv4NumToString(toUInt32(X)) AS Z, (3*X*X - 7*X + 11) % 37 AS Y FROM system.numbers LIMIT 1000000) GROUP BY Y; + SELECT Y, uniqCombined(17)(Z) FROM (SELECT number AS X, IPv4NumToString(toUInt32(X)) AS Z, (3*X*X - 7*X + 11) % 37 AS Y FROM system.numbers LIMIT 15) GROUP BY Y; SELECT Y, uniqCombined(17)(Z) FROM (SELECT number AS X, IPv4NumToString(toUInt32(X)) AS Z, (3*X*X - 7*X + 11) % 37 AS Y FROM system.numbers LIMIT 3000) GROUP BY Y; SELECT Y, uniqCombined(17)(Z) FROM (SELECT number AS X, IPv4NumToString(toUInt32(X)) AS Z, (3*X*X - 7*X + 11) % 37 AS Y FROM system.numbers LIMIT 1000000) GROUP BY Y; +SELECT Y, uniqCombined(20)(Z) FROM (SELECT number AS X, IPv4NumToString(toUInt32(X)) AS Z, (3*X*X - 7*X + 11) % 37 AS Y FROM system.numbers LIMIT 15) GROUP BY Y; +SELECT Y, uniqCombined(20)(Z) FROM (SELECT number AS X, IPv4NumToString(toUInt32(X)) AS Z, (3*X*X - 7*X + 11) % 37 AS Y FROM system.numbers LIMIT 3000) GROUP BY Y; +SELECT Y, uniqCombined(20)(Z) FROM (SELECT number AS X, IPv4NumToString(toUInt32(X)) AS Z, (3*X*X - 7*X + 11) % 37 AS Y FROM system.numbers LIMIT 1000000) GROUP BY Y; + SELECT uniqCombined(dummy) FROM remote('127.0.0.{2,3}', system.one); +SELECT uniqCombined(12)(dummy) FROM remote('127.0.0.{2,3}', system.one); + SELECT uniqCombined(17)(dummy) FROM remote('127.0.0.{2,3}', system.one); + +SELECT uniqCombined(20)(dummy) FROM remote('127.0.0.{2,3}', system.one); From 0592081f2fe4b550c28d06dc8b017b34b64bdbbf Mon Sep 17 00:00:00 2001 From: Ivan Lezhankin Date: Thu, 25 Oct 2018 16:17:29 +0300 Subject: [PATCH 022/124] Reduce maximum allowable size of |HashSet|. --- .../AggregateFunctions/AggregateFunctionUniqCombined.h | 8 ++++++-- 1 file changed, 6 insertions(+), 2 deletions(-) diff --git a/dbms/src/AggregateFunctions/AggregateFunctionUniqCombined.h b/dbms/src/AggregateFunctions/AggregateFunctionUniqCombined.h index f15c4e7e453..99cc3a93890 100644 --- a/dbms/src/AggregateFunctions/AggregateFunctionUniqCombined.h +++ b/dbms/src/AggregateFunctions/AggregateFunctionUniqCombined.h @@ -71,7 +71,11 @@ struct AggregateFunctionUniqCombinedDataWithKey { // TODO(ilezhankin): pre-generate values for |UniqCombinedBiasData|, // at the moment gen-bias-data.py script doesn't work. - using Set = CombinedCardinalityEstimator>, 16, K - 3, K, TrivialHash, Key>; + + // We want to migrate from |HashSet| to |HyperLogLogCounter| when the sizes in memory become almost equal. + // The size per element in |HashSet| is sizeof(Key)*2 bytes, and the overall size of |HyperLogLogCounter| is 2^K * 6 bits. + // For Key=UInt32 we can calculate: 2^X * 4 * 2 ≤ 2^(K-3) * 6 ⇒ X ≤ K-4. + using Set = CombinedCardinalityEstimator>, 16, K - 4, K, TrivialHash, Key>; Set set; }; @@ -82,7 +86,7 @@ struct AggregateFunctionUniqCombinedDataWithKey using Set = CombinedCardinalityEstimator>, 16, - 14, + 13, 17, TrivialHash, Key, From 73e9f5057dc1eb6c10455d6a85af5c6d15e13206 Mon Sep 17 00:00:00 2001 From: Sabyanin Maxim Date: Sat, 27 Oct 2018 23:39:59 +0300 Subject: [PATCH 023/124] fix code style checker errors --- dbms/src/Parsers/ASTColumnDeclaration.h | 6 ++++-- dbms/src/Parsers/ParserCreateQuery.h | 3 ++- dbms/src/Storages/ColumnComment.h | 3 ++- 3 files changed, 8 insertions(+), 4 deletions(-) diff --git a/dbms/src/Parsers/ASTColumnDeclaration.h b/dbms/src/Parsers/ASTColumnDeclaration.h index ea078ad0052..0b4aeddb796 100644 --- a/dbms/src/Parsers/ASTColumnDeclaration.h +++ b/dbms/src/Parsers/ASTColumnDeclaration.h @@ -36,7 +36,8 @@ public: res->children.push_back(res->default_expression); } - if (comment_expression) { + if (comment_expression) + { res->comment_expression = comment_expression->clone(); res->children.push_back(res->comment_expression); } @@ -62,7 +63,8 @@ public: default_expression->formatImpl(settings, state, frame); } - if (comment_expression) { + if (comment_expression) + { settings.ostr << ' ' << (settings.hilite ? hilite_keyword : "") << "COMMENT" << (settings.hilite ? hilite_none : "") << ' '; comment_expression->formatImpl(settings, state, frame); } diff --git a/dbms/src/Parsers/ParserCreateQuery.h b/dbms/src/Parsers/ParserCreateQuery.h index 3931b228c29..d5ae385e69e 100644 --- a/dbms/src/Parsers/ParserCreateQuery.h +++ b/dbms/src/Parsers/ParserCreateQuery.h @@ -173,7 +173,8 @@ bool IParserColumnDeclaration::parseImpl(Pos & pos, ASTPtr & node, E column_declaration->children.push_back(std::move(default_expression)); } - if (comment_expression) { + if (comment_expression) + { column_declaration->comment_expression = comment_expression; column_declaration->children.push_back(std::move(comment_expression)); } diff --git a/dbms/src/Storages/ColumnComment.h b/dbms/src/Storages/ColumnComment.h index 33e44fb5188..2e5553a8464 100644 --- a/dbms/src/Storages/ColumnComment.h +++ b/dbms/src/Storages/ColumnComment.h @@ -8,7 +8,8 @@ namespace DB { -struct ColumnComment { +struct ColumnComment +{ ASTPtr expression; }; From 6ab45d081c4f9b77a85b3865fd6988c3f3002f7e Mon Sep 17 00:00:00 2001 From: Alexandr Krasheninnikov Date: Fri, 2 Nov 2018 22:06:05 +0300 Subject: [PATCH 024/124] Arch-based compilation --- cmake/test_cpu.cmake | 34 ++++++++++++++- contrib/base64-cmake/CMakeLists.txt | 27 ++++++++++-- contrib/base64-cmake/config-header.tpl | 3 -- dbms/src/Functions/FunctionBase64Conversion.h | 43 ++++++++++++++----- .../src/Functions/registerFunctionsString.cpp | 2 + dbms/src/Functions/tryBase64Decode.cpp | 14 ++++++ .../000732_base64_functions.reference | 1 + .../0_stateless/000732_base64_functions.sql | 1 + .../functions/string_functions.md | 8 ++++ .../functions/string_functions.md | 9 ++++ 10 files changed, 124 insertions(+), 18 deletions(-) create mode 100644 dbms/src/Functions/tryBase64Decode.cpp diff --git a/cmake/test_cpu.cmake b/cmake/test_cpu.cmake index 6894c58703d..c360de5b962 100644 --- a/cmake/test_cpu.cmake +++ b/cmake/test_cpu.cmake @@ -45,6 +45,38 @@ if (HAVE_SSE42) set (COMPILER_FLAGS "${COMPILER_FLAGS} ${TEST_FLAG}") endif () +set (TEST_FLAG "-mssse3") +set (CMAKE_REQUIRED_FLAGS "${TEST_FLAG} -O0") +check_cxx_source_compiles(" + #include + int main() { + __m64 a = _mm_abs_pi8(__m64()); + (void)a; + return 0; + } +" HAVE_SSSE3) + +set (TEST_FLAG "-mavx") +set (CMAKE_REQUIRED_FLAGS "${TEST_FLAG} -O0") +check_cxx_source_compiles(" + #include + int main() { + auto a = _mm256_insert_epi8(__m256i(), 0, 0); + (void)a; + return 0; + } +" HAVE_AVX) + +set (TEST_FLAG "-mavx2") +set (CMAKE_REQUIRED_FLAGS "${TEST_FLAG} -O0") +check_cxx_source_compiles(" + #include + int main() { + auto a = _mm256_add_epi16(__m256i(), __m256i()); + (void)a; + return 0; + } +" HAVE_AVX2) # gcc -dM -E -mpopcnt - < /dev/null | sort > gcc-dump-popcnt #define __POPCNT__ 1 @@ -65,5 +97,3 @@ if (HAVE_POPCNT AND NOT ARCH_AARCH64) endif () cmake_pop_check_state () - -# TODO: add here sse3 test if you want use it diff --git a/contrib/base64-cmake/CMakeLists.txt b/contrib/base64-cmake/CMakeLists.txt index 8bf0316b461..09abb3b02b9 100644 --- a/contrib/base64-cmake/CMakeLists.txt +++ b/contrib/base64-cmake/CMakeLists.txt @@ -1,13 +1,33 @@ SET(LIBRARY_DIR ${ClickHouse_SOURCE_DIR}/contrib/base64) +set(base64_compile_instructions "") +LIST(LENGTH base64_compile_instructions 0) +macro(cast_to_bool var instruction) + if (HAVE_${var}) + set(base64_${var} 1) + set(base64_${var}_opt ${instruction}) + else() + set(base64_${var} 0) + endif() +endmacro() + +cast_to_bool(SSSE3 "-mssse3") +cast_to_bool(SSE41 "-msse4.1") +cast_to_bool(SSE42 "-msse4.2") +cast_to_bool(AVX "-mavx") +cast_to_bool(AVX2 "-mavx2") + # write config.h file, to include it in application file(READ config-header.tpl header) file(WRITE config.h ${header}) -file(APPEND config.h "#define HAVE_SSE41 ${HAVE_SSE41}\n") -file(APPEND config.h "#define HAVE_SSE42 ${HAVE_SSE42}\n") +file(APPEND config.h "#define HAVE_SSSE3 ${base64_SSSE3}\n") +file(APPEND config.h "#define HAVE_SSE41 ${base64_SSE41}\n") +file(APPEND config.h "#define HAVE_SSE42 ${base64_SSE42}\n") +file(APPEND config.h "#define HAVE_AVX ${base64_AVX}\n") +file(APPEND config.h "#define HAVE_AVX2 ${base64_AVX2}\n") set(HAVE_FAST_UNALIGNED_ACCESS 0) -if (${HAVE_SSE41} OR ${HAVE_SSE42}) +if (${base64_SSSE3} OR ${base64_SSE41} OR ${base64_SSE42} OR ${base64_AVX} OR ${base64_AVX2}) set(HAVE_FAST_UNALIGNED_ACCESS 1) endif () @@ -28,4 +48,5 @@ add_library(base64 ${LINK_MODE} ${LIBRARY_DIR}/lib/codecs.h config.h) +target_compile_options(base64 PRIVATE ${base64_SSSE3_opt} ${base64_SSE41_opt} ${base64_SSE42_opt} ${base64_AVX_opt} ${base64_AVX2_opt}) target_include_directories(base64 PRIVATE ${LIBRARY_DIR}/include .) \ No newline at end of file diff --git a/contrib/base64-cmake/config-header.tpl b/contrib/base64-cmake/config-header.tpl index 76f251d875e..c978ca0c082 100644 --- a/contrib/base64-cmake/config-header.tpl +++ b/contrib/base64-cmake/config-header.tpl @@ -1,5 +1,2 @@ -#define HAVE_AVX 0 -#define HAVE_AVX2 0 #define HAVE_NEON32 0 #define HAVE_NEON64 0 -#define HAVE_SSSE3 0 diff --git a/dbms/src/Functions/FunctionBase64Conversion.h b/dbms/src/Functions/FunctionBase64Conversion.h index fe7ea7cb206..6f3169f1f49 100644 --- a/dbms/src/Functions/FunctionBase64Conversion.h +++ b/dbms/src/Functions/FunctionBase64Conversion.h @@ -25,15 +25,31 @@ namespace ErrorCodes struct Base64Encode { static constexpr auto name = "base64Encode"; - static constexpr auto buffer_size_multiplier = 5.0 / 3.0; + static size_t getBufferSize(size_t string_length, size_t string_count) + { + return ( ( string_length - string_count ) / 3 + string_count ) * 4 + string_count ; + } }; struct Base64Decode { static constexpr auto name = "base64Decode"; - static constexpr auto buffer_size_multiplier = 3.0 / 4.0; + + static size_t getBufferSize(size_t string_length, size_t string_count) + { + return ( ( string_length - string_count) / 4 + string_count) * 3 + string_count; + } }; +struct TryBase64Decode +{ + static constexpr auto name = "tryBase64Decode"; + + static size_t getBufferSize(size_t string_length, size_t string_count) + { + return Base64Decode::getBufferSize(string_length, string_count); + } +}; template class FunctionBase64Conversion : public IFunction @@ -85,7 +101,7 @@ public: auto & dst_data = dst_column->getChars(); auto & dst_offsets = dst_column->getOffsets(); - size_t reserve = ceil(input->getChars().size() * Func::buffer_size_multiplier + input->size()); + size_t reserve = Func::getBufferSize(input->getChars().size(), input->size()); dst_data.resize(reserve); dst_offsets.resize(input_rows_count); @@ -107,13 +123,26 @@ public: { base64_encode(source, srclen, dst_pos, &outlen, codec); } - else + else if constexpr (std::is_same_v) { if (!base64_decode(source, srclen, dst_pos, &outlen, codec)) { throw Exception("Failed to " + getName() + " input '" + String(source, srclen) + "'", ErrorCodes::INCORRECT_DATA); } } + else + { + // during decoding character array can be partially polluted + // if fail, revert back and clean + auto savepoint = dst_pos; + if (!base64_decode(source, srclen, dst_pos, &outlen, codec)) + { + outlen = 0; + dst_pos = savepoint; + // clean the symbol + dst_pos[0] = 0; + } + } source += srclen + 1; dst_pos += outlen + 1; @@ -130,13 +159,7 @@ public: private: static int getCodec() { -#if __SSE4_2__ - return BASE64_FORCE_SSE42; -#elif __SSE4_1__ - return BASE64_FORCE_SSE41; -#else return BASE64_FORCE_PLAIN; -#endif } }; } diff --git a/dbms/src/Functions/registerFunctionsString.cpp b/dbms/src/Functions/registerFunctionsString.cpp index 48e1cf89f9f..5d4b05acdf6 100644 --- a/dbms/src/Functions/registerFunctionsString.cpp +++ b/dbms/src/Functions/registerFunctionsString.cpp @@ -22,6 +22,7 @@ void registerFunctionEndsWith(FunctionFactory &); #if USE_BASE64 void registerFunctionBase64Encode(FunctionFactory &); void registerFunctionBase64Decode(FunctionFactory &); +void registerFunctionTryBase64Decode(FunctionFactory &); #endif void registerFunctionsString(FunctionFactory & factory) @@ -45,6 +46,7 @@ void registerFunctionsString(FunctionFactory & factory) #if USE_BASE64 registerFunctionBase64Encode(factory); registerFunctionBase64Decode(factory); + registerFunctionTryBase64Decode(factory); #endif } diff --git a/dbms/src/Functions/tryBase64Decode.cpp b/dbms/src/Functions/tryBase64Decode.cpp new file mode 100644 index 00000000000..17b09d67eda --- /dev/null +++ b/dbms/src/Functions/tryBase64Decode.cpp @@ -0,0 +1,14 @@ +#include +#if USE_BASE64 +#include +#include + +namespace DB +{ + +void registerFunctionTryBase64Decode(FunctionFactory & factory) +{ + factory.registerFunction>(); +} +} +#endif \ No newline at end of file diff --git a/dbms/tests/queries/0_stateless/000732_base64_functions.reference b/dbms/tests/queries/0_stateless/000732_base64_functions.reference index e6fb8f43b11..b22ae4e7e24 100644 --- a/dbms/tests/queries/0_stateless/000732_base64_functions.reference +++ b/dbms/tests/queries/0_stateless/000732_base64_functions.reference @@ -13,3 +13,4 @@ foob fooba foobar 1 1 + diff --git a/dbms/tests/queries/0_stateless/000732_base64_functions.sql b/dbms/tests/queries/0_stateless/000732_base64_functions.sql index 7778eef02c7..a2dd4c4b41b 100644 --- a/dbms/tests/queries/0_stateless/000732_base64_functions.sql +++ b/dbms/tests/queries/0_stateless/000732_base64_functions.sql @@ -2,4 +2,5 @@ SET send_logs_level = 'none'; SELECT base64Encode(val) FROM (select arrayJoin(['', 'f', 'fo', 'foo', 'foob', 'fooba', 'foobar']) val); SELECT base64Decode(val) FROM (select arrayJoin(['', 'Zg==', 'Zm8=', 'Zm9v', 'Zm9vYg==', 'Zm9vYmE=', 'Zm9vYmFy']) val); SELECT base64Decode(base64Encode('foo')) = 'foo', base64Encode(base64Decode('Zm9v')) == 'Zm9v'; +SELECT tryBase64Decode('Zm9vYmF=Zm9v'); SELECT base64Decode('Zm9vYmF=Zm9v'); -- { serverError 117 } \ No newline at end of file diff --git a/docs/en/query_language/functions/string_functions.md b/docs/en/query_language/functions/string_functions.md index 61e33040aa5..7c36dfb785e 100644 --- a/docs/en/query_language/functions/string_functions.md +++ b/docs/en/query_language/functions/string_functions.md @@ -74,3 +74,11 @@ If the 's' string is non-empty and does not contain the 'c' character at the end Returns the string 's' that was converted from the encoding in 'from' to the encoding in 'to'. +## base64Encode(s) +Encodes 's' string into base64 + +## base64Decode(s) +Decode base64-encoded string 's' into original string. In case of failure raises an exception. + +## tryBase64Decode(s) +Similar to base64Decode, but in case of error an empty string would be returned. \ No newline at end of file diff --git a/docs/ru/query_language/functions/string_functions.md b/docs/ru/query_language/functions/string_functions.md index 43842b59468..83ccaec70be 100644 --- a/docs/ru/query_language/functions/string_functions.md +++ b/docs/ru/query_language/functions/string_functions.md @@ -58,3 +58,12 @@ ## convertCharset(s, from, to) Возвращает сконвертированную из кодировки from в кодировку to строку s. + +## base64Encode(s) +Производит кодирование строки s в base64-представление. + +## base64Decode(s) +Декодирует base64-представление s в исходную строку. При невозможности декодирования выбрасывает исключение + +## tryBase64Decode(s) +Функционал аналогичен base64Decode, но при невозможности декодирования возвращает пустую строку. \ No newline at end of file From bdcdc15b1eb398bf715e05fb3652a21650bbab68 Mon Sep 17 00:00:00 2001 From: Sabyanin Maxim Date: Tue, 6 Nov 2018 16:26:43 +0300 Subject: [PATCH 025/124] fixed comment test. remove ColumnComment struct. style fixes --- .../Interpreters/InterpreterCreateQuery.cpp | 6 ++-- .../Interpreters/InterpreterDescribeQuery.cpp | 24 +++++++++++--- dbms/src/Parsers/ASTAlterQuery.cpp | 2 +- dbms/src/Parsers/ASTColumnDeclaration.h | 13 ++------ dbms/src/Parsers/ParserCreateQuery.h | 4 ++- dbms/src/Parsers/ParserTablePropertiesQuery.h | 1 - dbms/src/Storages/AlterCommands.cpp | 7 ++-- dbms/src/Storages/AlterCommands.h | 6 ++-- dbms/src/Storages/ColumnComment.cpp | 7 ---- dbms/src/Storages/ColumnComment.h | 9 +---- dbms/src/Storages/ColumnsDescription.cpp | 23 +++++++------ .../Storages/System/StorageSystemColumns.cpp | 3 +- .../00725_comment_columns.reference | 8 ++--- .../0_stateless/00725_comment_columns.sql | 33 ++++++++++--------- 14 files changed, 72 insertions(+), 74 deletions(-) delete mode 100644 dbms/src/Storages/ColumnComment.cpp diff --git a/dbms/src/Interpreters/InterpreterCreateQuery.cpp b/dbms/src/Interpreters/InterpreterCreateQuery.cpp index 29f613307a9..aeaf48f4617 100644 --- a/dbms/src/Interpreters/InterpreterCreateQuery.cpp +++ b/dbms/src/Interpreters/InterpreterCreateQuery.cpp @@ -220,9 +220,9 @@ static ParsedColumns parseColumns(const ASTExpressionList & column_list_ast, con default_expr_list->children.emplace_back(setAlias(col_decl.default_expression->clone(), col_decl.name)); } - if (col_decl.comment_expression) + if (!col_decl.comment.empty()) { - comments.emplace(col_decl.name, ColumnComment{col_decl.comment_expression}); + comments.emplace(col_decl.name, col_decl.comment); } } @@ -351,7 +351,7 @@ ASTPtr InterpreterCreateQuery::formatColumns(const ColumnsDescription & columns) const auto comments_it = columns.comments.find(column.name); if (comments_it != std::end(columns.comments)) { - column_declaration->comment_expression = comments_it->second.expression->clone(); + column_declaration->comment = comments_it->second; } columns_list->children.push_back(column_declaration_ptr); diff --git a/dbms/src/Interpreters/InterpreterDescribeQuery.cpp b/dbms/src/Interpreters/InterpreterDescribeQuery.cpp index 536c554e996..6ddd9d93319 100644 --- a/dbms/src/Interpreters/InterpreterDescribeQuery.cpp +++ b/dbms/src/Interpreters/InterpreterDescribeQuery.cpp @@ -45,6 +45,10 @@ Block InterpreterDescribeQuery::getSampleBlock() col.name = "default_expression"; block.insert(col); + // TODO: may be it unneeded + col.name = "comment_expression"; + block.insert(col); + return block; } @@ -55,6 +59,7 @@ BlockInputStreamPtr InterpreterDescribeQuery::executeImpl() NamesAndTypesList columns; ColumnDefaults column_defaults; + ColumnComments column_comments; StoragePtr table; auto table_expression = typeid_cast(ast.table_expression.get()); @@ -101,6 +106,7 @@ BlockInputStreamPtr InterpreterDescribeQuery::executeImpl() auto table_lock = table->lockStructure(false, __PRETTY_FUNCTION__); columns = table->getColumns().getAll(); column_defaults = table->getColumns().defaults; + column_comments = table->getColumns().comments; } Block sample_block = getSampleBlock(); @@ -111,16 +117,26 @@ BlockInputStreamPtr InterpreterDescribeQuery::executeImpl() res_columns[0]->insert(column.name); res_columns[1]->insert(column.type->getName()); - const auto it = column_defaults.find(column.name); - if (it == std::end(column_defaults)) + const auto defaults_it = column_defaults.find(column.name); + if (defaults_it == std::end(column_defaults)) { res_columns[2]->insertDefault(); res_columns[3]->insertDefault(); } else { - res_columns[2]->insert(toString(it->second.kind)); - res_columns[3]->insert(queryToString(it->second.expression)); + res_columns[2]->insert(toString(defaults_it->second.kind)); + res_columns[3]->insert(queryToString(defaults_it->second.expression)); + } + + const auto comments_it = column_comments.find(column.name); + if (comments_it == std::end(column_comments)) + { + res_columns[4]->insertDefault(); + } + else + { + res_columns[4]->insert(comments_it->second); } } diff --git a/dbms/src/Parsers/ASTAlterQuery.cpp b/dbms/src/Parsers/ASTAlterQuery.cpp index ec43d7c76dc..eee67223496 100644 --- a/dbms/src/Parsers/ASTAlterQuery.cpp +++ b/dbms/src/Parsers/ASTAlterQuery.cpp @@ -146,7 +146,7 @@ void ASTAlterCommand::formatImpl( settings.ostr << (settings.hilite ? hilite_keyword : "") << indent_str << "COMMENT COLUMN " << (settings.hilite ? hilite_none : ""); column->formatImpl(settings, state, frame); settings.ostr << " "; - comment->formatImpl(settings, state, frame); + // comment->formatImpl(settings, state, frame); } else throw Exception("Unexpected type of ALTER", ErrorCodes::UNEXPECTED_AST_STRUCTURE); diff --git a/dbms/src/Parsers/ASTColumnDeclaration.h b/dbms/src/Parsers/ASTColumnDeclaration.h index 0b4aeddb796..0680e69d300 100644 --- a/dbms/src/Parsers/ASTColumnDeclaration.h +++ b/dbms/src/Parsers/ASTColumnDeclaration.h @@ -15,7 +15,7 @@ public: ASTPtr type; String default_specifier; ASTPtr default_expression; - ASTPtr comment_expression; + String comment; String getID() const override { return "ColumnDeclaration_" + name; } @@ -36,12 +36,6 @@ public: res->children.push_back(res->default_expression); } - if (comment_expression) - { - res->comment_expression = comment_expression->clone(); - res->children.push_back(res->comment_expression); - } - return res; } @@ -63,10 +57,9 @@ public: default_expression->formatImpl(settings, state, frame); } - if (comment_expression) + if (!comment.empty()) { - settings.ostr << ' ' << (settings.hilite ? hilite_keyword : "") << "COMMENT" << (settings.hilite ? hilite_none : "") << ' '; - comment_expression->formatImpl(settings, state, frame); + settings.ostr << ' ' << (settings.hilite ? hilite_none : "") << "COMMENT" << (settings.hilite ? hilite_none : "") << ' ' << comment; } } }; diff --git a/dbms/src/Parsers/ParserCreateQuery.h b/dbms/src/Parsers/ParserCreateQuery.h index d5ae385e69e..6705a38c63a 100644 --- a/dbms/src/Parsers/ParserCreateQuery.h +++ b/dbms/src/Parsers/ParserCreateQuery.h @@ -6,6 +6,7 @@ #include #include #include +#include #include #include #include @@ -175,7 +176,8 @@ bool IParserColumnDeclaration::parseImpl(Pos & pos, ASTPtr & node, E if (comment_expression) { - column_declaration->comment_expression = comment_expression; + auto & literal_value = typeid_cast(*comment_expression).value; + column_declaration->comment = literal_value.safeGet(); column_declaration->children.push_back(std::move(comment_expression)); } diff --git a/dbms/src/Parsers/ParserTablePropertiesQuery.h b/dbms/src/Parsers/ParserTablePropertiesQuery.h index f97babe194a..c9f8abf8f19 100644 --- a/dbms/src/Parsers/ParserTablePropertiesQuery.h +++ b/dbms/src/Parsers/ParserTablePropertiesQuery.h @@ -7,7 +7,6 @@ namespace DB { -// TODO: возможно тут тоже надо разобраться /** Query (EXISTS | SHOW CREATE) [TABLE] [db.]name [FORMAT format] */ class ParserTablePropertiesQuery : public IParserBase diff --git a/dbms/src/Storages/AlterCommands.cpp b/dbms/src/Storages/AlterCommands.cpp index 5b9b6f3b0db..fa0127a26b7 100644 --- a/dbms/src/Storages/AlterCommands.cpp +++ b/dbms/src/Storages/AlterCommands.cpp @@ -94,9 +94,10 @@ std::optional AlterCommand::parse(const ASTAlterCommand * command_ { AlterCommand command; command.type = COMMENT_COLUMN; - const auto & ast_identifier = typeid_cast(*command_ast->column); + const auto & ast_identifier = typeid_cast(*command_ast->column); command.column_name = ast_identifier.name; - command.comment_expression = command_ast->comment; + const auto & ast_comment = typeid_cast(*command_ast->comment); + command.comment = ast_comment.value.get(); return command; } else @@ -249,7 +250,7 @@ void AlterCommand::apply(ColumnsDescription & columns_description) const else if (type == COMMENT_COLUMN) { - columns_description.comments[column_name].expression = comment_expression; + columns_description.comments[column_name] = comment; } else throw Exception("Wrong parameter type in ALTER query", ErrorCodes::LOGICAL_ERROR); diff --git a/dbms/src/Storages/AlterCommands.h b/dbms/src/Storages/AlterCommands.h index 1bbf3ddbee0..7c56740a2fe 100644 --- a/dbms/src/Storages/AlterCommands.h +++ b/dbms/src/Storages/AlterCommands.h @@ -36,7 +36,7 @@ struct AlterCommand ColumnDefaultKind default_kind{}; ASTPtr default_expression{}; - ASTPtr comment_expression; + String comment; /// For ADD - after which column to add a new one. If an empty string, add to the end. To add to the beginning now it is impossible. String after_column; @@ -47,9 +47,9 @@ struct AlterCommand AlterCommand() = default; AlterCommand(const Type type, const String & column_name, const DataTypePtr & data_type, const ColumnDefaultKind default_kind, const ASTPtr & default_expression, - const String & after_column = String{}, const ASTPtr & comment_expression = nullptr) + const String & after_column = String{}, const String & comment = "") : type{type}, column_name{column_name}, data_type{data_type}, default_kind{default_kind}, - default_expression{default_expression}, comment_expression(comment_expression), after_column{after_column} + default_expression{default_expression}, comment(comment), after_column{after_column} {} static std::optional parse(const ASTAlterCommand * command); diff --git a/dbms/src/Storages/ColumnComment.cpp b/dbms/src/Storages/ColumnComment.cpp deleted file mode 100644 index 076ec1e19df..00000000000 --- a/dbms/src/Storages/ColumnComment.cpp +++ /dev/null @@ -1,7 +0,0 @@ -#include -#include - -bool DB::operator== (const DB::ColumnComment& lhs, const DB::ColumnComment& rhs) -{ - return queryToString(lhs.expression) == queryToString(rhs.expression); -} diff --git a/dbms/src/Storages/ColumnComment.h b/dbms/src/Storages/ColumnComment.h index 2e5553a8464..521d1fd9d17 100644 --- a/dbms/src/Storages/ColumnComment.h +++ b/dbms/src/Storages/ColumnComment.h @@ -8,13 +8,6 @@ namespace DB { -struct ColumnComment -{ - ASTPtr expression; -}; - -bool operator== (const ColumnComment& lhs, const ColumnComment& rhs); - -using ColumnComments = std::unordered_map; +using ColumnComments = std::unordered_map; } diff --git a/dbms/src/Storages/ColumnsDescription.cpp b/dbms/src/Storages/ColumnsDescription.cpp index e3e76f6012a..da540258329 100644 --- a/dbms/src/Storages/ColumnsDescription.cpp +++ b/dbms/src/Storages/ColumnsDescription.cpp @@ -1,4 +1,5 @@ #include +#include #include #include #include @@ -10,6 +11,7 @@ #include #include #include +#include #include #include @@ -82,7 +84,7 @@ String ColumnsDescription::toString() const writeChar(' ', buf); writeText(column.type->getName(), buf); - const bool exist_comment = comments_it != std::end(comments) && !comments_it->second.expression; + const bool exist_comment = comments_it != std::end(comments); if (defaults_it != std::end(defaults)) { writeChar('\t', buf); @@ -98,7 +100,7 @@ String ColumnsDescription::toString() const if (exist_comment) { writeChar('\t', buf); - writeText(queryToString(comments_it->second.expression), buf); + writeText(comments_it->second, buf); } writeChar('\n', buf); @@ -146,7 +148,7 @@ std::optional parseDefaulfInfo(ReadBufferFromString & buf) return ParsedDefaultInfo{default_kind, std::move(default_expr)}; } -ASTPtr parseCommentExpr(ReadBufferFromString& buf) +String parseComment(ReadBufferFromString& buf) { if (*buf.position() == '\n') { @@ -154,12 +156,9 @@ ASTPtr parseCommentExpr(ReadBufferFromString& buf) } ParserExpression parser_expr; - String comment_expr_str; - readText(comment_expr_str, buf); - const char * begin = comment_expr_str.data(); - const auto end = begin + comment_expr_str.size(); - ASTPtr comment_expr = parseQuery(parser_expr, begin, end, "comment_expression", 0); - return comment_expr; + String comment; + readText(comment, buf); // This is wrong may be + return comment; } ColumnsDescription ColumnsDescription::parse(const String & str) @@ -209,10 +208,10 @@ ColumnsDescription ColumnsDescription::parse(const String & str) result.defaults.emplace(column_name, ColumnDefault{default_kind, default_expr}); } - const auto comment_expr = parseCommentExpr(buf); - if (comment_expr) + const auto comment = parseComment(buf); + if (!comment.empty()) { - result.comments.emplace(column_name, ColumnComment{comment_expr}); + result.comments.emplace(column_name, comment); } assertChar('\n', buf); diff --git a/dbms/src/Storages/System/StorageSystemColumns.cpp b/dbms/src/Storages/System/StorageSystemColumns.cpp index b546a275f0b..45ffb73f9af 100644 --- a/dbms/src/Storages/System/StorageSystemColumns.cpp +++ b/dbms/src/Storages/System/StorageSystemColumns.cpp @@ -186,9 +186,8 @@ protected: } else { - const auto & literal = typeid_cast(it->second.expression.get()); if (columns_mask[src_index++]) - res_columns[res_index++]->insert(literal->value.get()); + res_columns[res_index++]->insert(it->second); } } diff --git a/dbms/tests/queries/0_stateless/00725_comment_columns.reference b/dbms/tests/queries/0_stateless/00725_comment_columns.reference index 8d7837d8a31..19dac99b721 100644 --- a/dbms/tests/queries/0_stateless/00725_comment_columns.reference +++ b/dbms/tests/queries/0_stateless/00725_comment_columns.reference @@ -1,4 +1,4 @@ -CREATE TABLE default.check_query_comment_column ( first_column UInt8 DEFAULT 1 COMMENT \'first comment\', fourth_column UInt8 COMMENT \'fourth comment\', fifth_column UInt8, second_column UInt8 MATERIALIZED first_column COMMENT \'second comment\', third_column UInt8 ALIAS second_column COMMENT \'third comment\') ENGINE = TinyLog +CREATE TABLE test.check_query_comment_column ( first_column UInt8 DEFAULT 1 COMMENT \'first comment\', fourth_column UInt8 COMMENT \'fourth comment\', fifth_column UInt8, second_column UInt8 MATERIALIZED first_column COMMENT \'second comment\', third_column UInt8 ALIAS second_column COMMENT \'third comment\') ENGINE = TinyLog ┌─table──────────────────────┬─name──────────┬─comment────────┐ │ check_query_comment_column │ first_column │ first comment │ │ check_query_comment_column │ fourth_column │ fourth comment │ @@ -6,7 +6,7 @@ CREATE TABLE default.check_query_comment_column ( first_column UInt8 DEFAULT 1 C │ check_query_comment_column │ second_column │ second comment │ │ check_query_comment_column │ third_column │ third comment │ └────────────────────────────┴───────────────┴────────────────┘ -CREATE TABLE default.check_query_comment_column ( first_column UInt8 DEFAULT 1 COMMENT \'another first column\', fourth_column UInt8 COMMENT \'another fourth column\', fifth_column UInt8 COMMENT \'another fifth column\', second_column UInt8 MATERIALIZED first_column COMMENT \'another second column\', third_column UInt8 ALIAS second_column COMMENT \'another third column\') ENGINE = TinyLog +CREATE TABLE test.check_query_comment_column ( first_column UInt8 DEFAULT 1 COMMENT \'another first column\', fourth_column UInt8 COMMENT \'another fourth column\', fifth_column UInt8 COMMENT \'another fifth column\', second_column UInt8 MATERIALIZED first_column COMMENT \'another second column\', third_column UInt8 ALIAS second_column COMMENT \'another third column\') ENGINE = TinyLog ┌─table──────────────────────┬─name──────────┬─comment───────────────┐ │ check_query_comment_column │ first_column │ another first column │ │ check_query_comment_column │ fourth_column │ another fourth column │ @@ -14,13 +14,13 @@ CREATE TABLE default.check_query_comment_column ( first_column UInt8 DEFAULT 1 C │ check_query_comment_column │ second_column │ another second column │ │ check_query_comment_column │ third_column │ another third column │ └────────────────────────────┴───────────────┴───────────────────────┘ -CREATE TABLE default.check_query_comment_column ( first_column Date COMMENT \'first comment\', second_column UInt8 COMMENT \'second comment\', third_column UInt8 COMMENT \'third comment\') ENGINE = MergeTree(first_column, (second_column, second_column), 8192) +CREATE TABLE test.check_query_comment_column ( first_column Date COMMENT \'first comment\', second_column UInt8 COMMENT \'second comment\', third_column UInt8 COMMENT \'third comment\') ENGINE = MergeTree(first_column, (second_column, second_column), 8192) ┌─table──────────────────────┬─name──────────┬─comment────────┐ │ check_query_comment_column │ first_column │ first comment │ │ check_query_comment_column │ second_column │ second comment │ │ check_query_comment_column │ third_column │ third comment │ └────────────────────────────┴───────────────┴────────────────┘ -CREATE TABLE default.check_query_comment_column ( first_column Date COMMENT \'another first comment\', second_column UInt8 COMMENT \'another second comment\', third_column UInt8 COMMENT \'another third comment\') ENGINE = MergeTree(first_column, (second_column, second_column), 8192) +CREATE TABLE test.check_query_comment_column ( first_column Date COMMENT \'another first comment\', second_column UInt8 COMMENT \'another second comment\', third_column UInt8 COMMENT \'another third comment\') ENGINE = MergeTree(first_column, (second_column, second_column), 8192) ┌─table──────────────────────┬─name──────────┬─comment────────────────┐ │ check_query_comment_column │ first_column │ another first comment │ │ check_query_comment_column │ second_column │ another second comment │ diff --git a/dbms/tests/queries/0_stateless/00725_comment_columns.sql b/dbms/tests/queries/0_stateless/00725_comment_columns.sql index 6c2bd2305b1..afcb1b34eab 100644 --- a/dbms/tests/queries/0_stateless/00725_comment_columns.sql +++ b/dbms/tests/queries/0_stateless/00725_comment_columns.sql @@ -1,6 +1,7 @@ -DROP TABLE IF EXISTS check_query_comment_column; +CREATE DATABASE IF NOT EXISTS test; +DROP TABLE IF EXISTS test.check_query_comment_column; -CREATE TABLE check_query_comment_column +CREATE TABLE test.check_query_comment_column ( first_column UInt8 DEFAULT 1 COMMENT 'first comment', second_column UInt8 MATERIALIZED first_column COMMENT 'second comment', @@ -9,52 +10,54 @@ CREATE TABLE check_query_comment_column fifth_column UInt8 ) ENGINE = TinyLog; -SHOW CREATE TABLE check_query_comment_column; +SHOW CREATE TABLE test.check_query_comment_column; SELECT table, name, comment FROM system.columns -WHERE table = 'check_query_comment_column' +WHERE table = 'check_query_comment_column' AND database = 'test' FORMAT PrettyCompactNoEscapes; -ALTER TABLE check_query_comment_column +ALTER TABLE test.check_query_comment_column COMMENT COLUMN first_column 'another first column', COMMENT COLUMN second_column 'another second column', COMMENT COLUMN third_column 'another third column', COMMENT COLUMN fourth_column 'another fourth column', COMMENT COLUMN fifth_column 'another fifth column'; -SHOW CREATE TABLE check_query_comment_column; +SHOW CREATE TABLE test.check_query_comment_column; SELECT table, name, comment FROM system.columns -WHERE table = 'check_query_comment_column' +WHERE table = 'check_query_comment_column' AND database = 'test' FORMAT PrettyCompactNoEscapes; -DROP TABLE IF EXISTS check_query_comment_column; +DROP TABLE IF EXISTS test.check_query_comment_column; -CREATE TABLE check_query_comment_column +CREATE TABLE test.check_query_comment_column ( first_column Date COMMENT 'first comment', second_column UInt8 COMMENT 'second comment', third_column UInt8 COMMENT 'third comment' ) ENGINE = MergeTree(first_column, (second_column, second_column), 8192); -SHOW CREATE TABLE check_query_comment_column; +SHOW CREATE TABLE test.check_query_comment_column; SELECT table, name, comment FROM system.columns -WHERE table = 'check_query_comment_column' +WHERE table = 'check_query_comment_column' AND database = 'test' FORMAT PrettyCompactNoEscapes; -ALTER TABLE check_query_comment_column +ALTER TABLE test.check_query_comment_column COMMENT COLUMN first_column 'another first comment', COMMENT COLUMN second_column 'another second comment', COMMENT COLUMN third_column 'another third comment'; -SHOW CREATE TABLE check_query_comment_column; +SHOW CREATE TABLE test.check_query_comment_column; SELECT table, name, comment FROM system.columns -WHERE table = 'check_query_comment_column' -FORMAT PrettyCompactNoEscapes; \ No newline at end of file +WHERE table = 'check_query_comment_column' and database = 'test' +FORMAT PrettyCompactNoEscapes; + +DROP table test.check_query_comment_column; \ No newline at end of file From f8f2779f7b227a5bca98d08b7d3069674bd060bc Mon Sep 17 00:00:00 2001 From: Sabyanin Maxim Date: Thu, 8 Nov 2018 15:03:42 +0300 Subject: [PATCH 026/124] make ColumnComment just a hashmap --- .../src/Interpreters/InterpreterCreateQuery.cpp | 8 +++++--- dbms/src/Parsers/ASTAlterQuery.cpp | 6 +++--- dbms/src/Parsers/ASTColumnDeclaration.h | 13 ++++++++++--- dbms/src/Parsers/ParserCreateQuery.h | 3 +-- dbms/src/Storages/ColumnComment.h | 4 ++-- dbms/src/Storages/ColumnsDescription.cpp | 17 ++++++++--------- dbms/src/Storages/ColumnsDescription.h | 2 +- 7 files changed, 30 insertions(+), 23 deletions(-) diff --git a/dbms/src/Interpreters/InterpreterCreateQuery.cpp b/dbms/src/Interpreters/InterpreterCreateQuery.cpp index 7551eb9ea23..d7a223194b5 100644 --- a/dbms/src/Interpreters/InterpreterCreateQuery.cpp +++ b/dbms/src/Interpreters/InterpreterCreateQuery.cpp @@ -6,6 +6,7 @@ #include #include +#include #include #include @@ -220,9 +221,10 @@ static ParsedColumns parseColumns(const ASTExpressionList & column_list_ast, con default_expr_list->children.emplace_back(setAlias(col_decl.default_expression->clone(), col_decl.name)); } - if (!col_decl.comment.empty()) + if (col_decl.comment) { - comments.emplace(col_decl.name, col_decl.comment); + auto comment_literal = typeid_cast(*col_decl.comment); + comments.emplace(col_decl.name, comment_literal.value.get()); } } @@ -351,7 +353,7 @@ ASTPtr InterpreterCreateQuery::formatColumns(const ColumnsDescription & columns) const auto comments_it = columns.comments.find(column.name); if (comments_it != std::end(columns.comments)) { - column_declaration->comment = comments_it->second; + column_declaration->comment = std::make_shared(Field(comments_it->second)); } columns_list->children.push_back(column_declaration_ptr); diff --git a/dbms/src/Parsers/ASTAlterQuery.cpp b/dbms/src/Parsers/ASTAlterQuery.cpp index eee67223496..65f281a3f71 100644 --- a/dbms/src/Parsers/ASTAlterQuery.cpp +++ b/dbms/src/Parsers/ASTAlterQuery.cpp @@ -143,10 +143,10 @@ void ASTAlterCommand::formatImpl( } else if (type == ASTAlterCommand::COMMENT_COLUMN) { - settings.ostr << (settings.hilite ? hilite_keyword : "") << indent_str << "COMMENT COLUMN " << (settings.hilite ? hilite_none : ""); + settings.ostr << (settings.hilite ? hilite_keyword : "") << indent_str << "COMMENT COLUMN " << (settings.hilite ? hilite_identifier : ""); column->formatImpl(settings, state, frame); - settings.ostr << " "; - // comment->formatImpl(settings, state, frame); + settings.ostr << " " << (settings.hilite ? hilite_none : ""); + comment->formatImpl(settings, state, frame); } else throw Exception("Unexpected type of ALTER", ErrorCodes::UNEXPECTED_AST_STRUCTURE); diff --git a/dbms/src/Parsers/ASTColumnDeclaration.h b/dbms/src/Parsers/ASTColumnDeclaration.h index 0680e69d300..010c845b244 100644 --- a/dbms/src/Parsers/ASTColumnDeclaration.h +++ b/dbms/src/Parsers/ASTColumnDeclaration.h @@ -15,7 +15,7 @@ public: ASTPtr type; String default_specifier; ASTPtr default_expression; - String comment; + ASTPtr comment; String getID() const override { return "ColumnDeclaration_" + name; } @@ -36,6 +36,12 @@ public: res->children.push_back(res->default_expression); } + if (comment) + { + res->comment = comment->clone(); + res->children.push_back(res->comment); + } + return res; } @@ -57,9 +63,10 @@ public: default_expression->formatImpl(settings, state, frame); } - if (!comment.empty()) + if (comment) { - settings.ostr << ' ' << (settings.hilite ? hilite_none : "") << "COMMENT" << (settings.hilite ? hilite_none : "") << ' ' << comment; + settings.ostr << ' ' << (settings.hilite ? hilite_keyword : "") << "COMMENT " << (settings.hilite ? hilite_none : "") << ' '; + comment->formatImpl(settings, state, frame); } } }; diff --git a/dbms/src/Parsers/ParserCreateQuery.h b/dbms/src/Parsers/ParserCreateQuery.h index 6705a38c63a..7d2bdb382cd 100644 --- a/dbms/src/Parsers/ParserCreateQuery.h +++ b/dbms/src/Parsers/ParserCreateQuery.h @@ -176,8 +176,7 @@ bool IParserColumnDeclaration::parseImpl(Pos & pos, ASTPtr & node, E if (comment_expression) { - auto & literal_value = typeid_cast(*comment_expression).value; - column_declaration->comment = literal_value.safeGet(); + column_declaration->comment = comment_expression; column_declaration->children.push_back(std::move(comment_expression)); } diff --git a/dbms/src/Storages/ColumnComment.h b/dbms/src/Storages/ColumnComment.h index 521d1fd9d17..0fc4e7e2742 100644 --- a/dbms/src/Storages/ColumnComment.h +++ b/dbms/src/Storages/ColumnComment.h @@ -1,10 +1,10 @@ #pragma once +#include + #include #include -#include - namespace DB { diff --git a/dbms/src/Storages/ColumnsDescription.cpp b/dbms/src/Storages/ColumnsDescription.cpp index da540258329..7ac7b07d955 100644 --- a/dbms/src/Storages/ColumnsDescription.cpp +++ b/dbms/src/Storages/ColumnsDescription.cpp @@ -100,7 +100,7 @@ String ColumnsDescription::toString() const if (exist_comment) { writeChar('\t', buf); - writeText(comments_it->second, buf); + writeText(queryToString(ASTLiteral(Field(comments_it->second))), buf); } writeChar('\n', buf); @@ -120,7 +120,7 @@ struct ParsedDefaultInfo ASTPtr default_expr_str; }; -std::optional parseDefaulfInfo(ReadBufferFromString & buf) +std::optional parseDefaultInfo(ReadBufferFromString & buf) { if (*buf.position() == '\n') { @@ -142,9 +142,7 @@ std::optional parseDefaulfInfo(ReadBufferFromString & buf) ParserExpression expr_parser; String default_expr_str; readText(default_expr_str, buf); - const char * begin = default_expr_str.data(); - const auto end = begin + default_expr_str.size(); - ASTPtr default_expr = parseQuery(expr_parser, begin, end, "default_expression", 0); + ASTPtr default_expr = parseQuery(expr_parser, default_expr_str, "default_expression", 0); return ParsedDefaultInfo{default_kind, std::move(default_expr)}; } @@ -156,9 +154,10 @@ String parseComment(ReadBufferFromString& buf) } ParserExpression parser_expr; - String comment; - readText(comment, buf); // This is wrong may be - return comment; + String comment_expr_str; + readText(comment_expr_str, buf); // This is wrong may be + ASTPtr comment_expr = parseQuery(parser_expr, comment_expr_str, "comment expression", 0); + return typeid_cast(*comment_expr).value.get(); } ColumnsDescription ColumnsDescription::parse(const String & str) @@ -193,7 +192,7 @@ ColumnsDescription ColumnsDescription::parse(const String & str) assertChar('\t', buf); - const auto default_info = parseDefaulfInfo(buf); + const auto default_info = parseDefaultInfo(buf); if (default_info) { const auto & default_kind = default_info->default_kind; diff --git a/dbms/src/Storages/ColumnsDescription.h b/dbms/src/Storages/ColumnsDescription.h index 69738d83861..0fe88140ba1 100644 --- a/dbms/src/Storages/ColumnsDescription.h +++ b/dbms/src/Storages/ColumnsDescription.h @@ -24,7 +24,7 @@ struct ColumnsDescription NamesAndTypesList ordinary_, NamesAndTypesList materialized_, NamesAndTypesList aliases_, - ColumnDefaults defaults_, + ColumnDefaults defaults_ = {}, ColumnComments comments_ = {}) : ordinary(std::move(ordinary_)) , materialized(std::move(materialized_)) From b869cfed9a3dc65e4a3541db4d31603df6a770aa Mon Sep 17 00:00:00 2001 From: Sabyanin Maxim Date: Mon, 12 Nov 2018 18:45:35 +0300 Subject: [PATCH 027/124] fixed couple of mistakes. --- .../Interpreters/InterpreterDescribeQuery.cpp | 1 - dbms/src/Parsers/ASTAlterQuery.cpp | 2 +- dbms/src/Parsers/ASTColumnDeclaration.h | 2 +- dbms/src/Storages/ColumnComment.h | 13 --- dbms/src/Storages/ColumnsDescription.h | 7 +- .../System/StorageSystemPartsBase.cpp | 3 +- .../0_stateless/00030_alter_table.reference | 52 ++++----- .../00061_merge_tree_alter.reference | 106 +++++++++--------- .../00079_defaulted_columns.reference | 54 ++++----- .../00168_buffer_defaults.reference | 12 +- .../0_stateless/00294_shard_enums.reference | 40 +++---- .../0_stateless/00415_into_outfile.reference | 2 +- ...c_table_functions_and_subqueries.reference | 46 ++++---- .../00557_alter_null_storage_tables.reference | 6 +- .../queries/0_stateless/00642_cast.reference | 4 +- ...ult_database_on_client_reconnect.reference | 2 +- .../00700_decimal_defaults.reference | 12 +- 17 files changed, 176 insertions(+), 188 deletions(-) delete mode 100644 dbms/src/Storages/ColumnComment.h diff --git a/dbms/src/Interpreters/InterpreterDescribeQuery.cpp b/dbms/src/Interpreters/InterpreterDescribeQuery.cpp index 6ddd9d93319..db33383b76d 100644 --- a/dbms/src/Interpreters/InterpreterDescribeQuery.cpp +++ b/dbms/src/Interpreters/InterpreterDescribeQuery.cpp @@ -45,7 +45,6 @@ Block InterpreterDescribeQuery::getSampleBlock() col.name = "default_expression"; block.insert(col); - // TODO: may be it unneeded col.name = "comment_expression"; block.insert(col); diff --git a/dbms/src/Parsers/ASTAlterQuery.cpp b/dbms/src/Parsers/ASTAlterQuery.cpp index 65f281a3f71..de379fe5e3d 100644 --- a/dbms/src/Parsers/ASTAlterQuery.cpp +++ b/dbms/src/Parsers/ASTAlterQuery.cpp @@ -143,7 +143,7 @@ void ASTAlterCommand::formatImpl( } else if (type == ASTAlterCommand::COMMENT_COLUMN) { - settings.ostr << (settings.hilite ? hilite_keyword : "") << indent_str << "COMMENT COLUMN " << (settings.hilite ? hilite_identifier : ""); + settings.ostr << (settings.hilite ? hilite_keyword : "") << indent_str << "COMMENT COLUMN " << (settings.hilite ? hilite_none : ""); column->formatImpl(settings, state, frame); settings.ostr << " " << (settings.hilite ? hilite_none : ""); comment->formatImpl(settings, state, frame); diff --git a/dbms/src/Parsers/ASTColumnDeclaration.h b/dbms/src/Parsers/ASTColumnDeclaration.h index 010c845b244..870472fcb30 100644 --- a/dbms/src/Parsers/ASTColumnDeclaration.h +++ b/dbms/src/Parsers/ASTColumnDeclaration.h @@ -65,7 +65,7 @@ public: if (comment) { - settings.ostr << ' ' << (settings.hilite ? hilite_keyword : "") << "COMMENT " << (settings.hilite ? hilite_none : "") << ' '; + settings.ostr << ' ' << (settings.hilite ? hilite_keyword : "") << "COMMENT" << (settings.hilite ? hilite_none : "") << ' '; comment->formatImpl(settings, state, frame); } } diff --git a/dbms/src/Storages/ColumnComment.h b/dbms/src/Storages/ColumnComment.h deleted file mode 100644 index 0fc4e7e2742..00000000000 --- a/dbms/src/Storages/ColumnComment.h +++ /dev/null @@ -1,13 +0,0 @@ -#pragma once - -#include - -#include -#include - -namespace DB -{ - -using ColumnComments = std::unordered_map; - -} diff --git a/dbms/src/Storages/ColumnsDescription.h b/dbms/src/Storages/ColumnsDescription.h index 0fe88140ba1..89457421364 100644 --- a/dbms/src/Storages/ColumnsDescription.h +++ b/dbms/src/Storages/ColumnsDescription.h @@ -2,7 +2,6 @@ #include #include -#include #include #include @@ -10,6 +9,8 @@ namespace DB { +using ColumnComments = std::unordered_map; + struct ColumnsDescription { NamesAndTypesList ordinary; @@ -24,8 +25,8 @@ struct ColumnsDescription NamesAndTypesList ordinary_, NamesAndTypesList materialized_, NamesAndTypesList aliases_, - ColumnDefaults defaults_ = {}, - ColumnComments comments_ = {}) + ColumnDefaults defaults_, + ColumnComments comments_) : ordinary(std::move(ordinary_)) , materialized(std::move(materialized_)) , aliases(std::move(aliases_)) diff --git a/dbms/src/Storages/System/StorageSystemPartsBase.cpp b/dbms/src/Storages/System/StorageSystemPartsBase.cpp index 8cbf497e43c..9d7b2745721 100644 --- a/dbms/src/Storages/System/StorageSystemPartsBase.cpp +++ b/dbms/src/Storages/System/StorageSystemPartsBase.cpp @@ -1,3 +1,4 @@ +#include #include #include #include @@ -306,7 +307,7 @@ StorageSystemPartsBase::StorageSystemPartsBase(std::string name_, NamesAndTypesL add_alias("bytes", "bytes_on_disk"); add_alias("marks_size", "marks_bytes"); - setColumns(ColumnsDescription(std::move(columns_), {}, std::move(aliases), std::move(defaults))); + setColumns(ColumnsDescription(std::move(columns_), {}, std::move(aliases), std::move(defaults), ColumnComments{})); } } diff --git a/dbms/tests/queries/0_stateless/00030_alter_table.reference b/dbms/tests/queries/0_stateless/00030_alter_table.reference index ac2c0af78f4..05022b485f7 100644 --- a/dbms/tests/queries/0_stateless/00030_alter_table.reference +++ b/dbms/tests/queries/0_stateless/00030_alter_table.reference @@ -1,27 +1,27 @@ -CounterID UInt32 -StartDate Date -UserID UInt32 -VisitID UInt32 -NestedColumn.A Array(UInt8) -NestedColumn.S Array(String) -ToDrop UInt32 -Added0 UInt32 -Added1 UInt32 -Added2 UInt32 -AddedNested1.A Array(UInt32) -AddedNested1.B Array(UInt64) -AddedNested1.C Array(String) -AddedNested2.A Array(UInt32) -AddedNested2.B Array(UInt64) -CounterID UInt32 -StartDate Date -UserID UInt32 -VisitID UInt32 -Added0 String -Added1 UInt32 -Added2 UInt32 -AddedNested1.A Array(UInt32) -AddedNested1.C Array(String) -AddedNested2.A Array(UInt32) -AddedNested2.B Array(UInt64) +CounterID UInt32 +StartDate Date +UserID UInt32 +VisitID UInt32 +NestedColumn.A Array(UInt8) +NestedColumn.S Array(String) +ToDrop UInt32 +Added0 UInt32 +Added1 UInt32 +Added2 UInt32 +AddedNested1.A Array(UInt32) +AddedNested1.B Array(UInt64) +AddedNested1.C Array(String) +AddedNested2.A Array(UInt32) +AddedNested2.B Array(UInt64) +CounterID UInt32 +StartDate Date +UserID UInt32 +VisitID UInt32 +Added0 String +Added1 UInt32 +Added2 UInt32 +AddedNested1.A Array(UInt32) +AddedNested1.C Array(String) +AddedNested2.A Array(UInt32) +AddedNested2.B Array(UInt64) 1 2014-01-01 2 3 0 0 [] [] [] [] diff --git a/dbms/tests/queries/0_stateless/00061_merge_tree_alter.reference b/dbms/tests/queries/0_stateless/00061_merge_tree_alter.reference index 48a896785fd..9545f7b4200 100644 --- a/dbms/tests/queries/0_stateless/00061_merge_tree_alter.reference +++ b/dbms/tests/queries/0_stateless/00061_merge_tree_alter.reference @@ -1,56 +1,56 @@ -d Date -k UInt64 -i32 Int32 +d Date +k UInt64 +i32 Int32 CREATE TABLE test.alter ( d Date, k UInt64, i32 Int32) ENGINE = MergeTree(d, k, 8192) 2015-01-01 10 42 -d Date -k UInt64 -i32 Int32 -n.ui8 Array(UInt8) -n.s Array(String) +d Date +k UInt64 +i32 Int32 +n.ui8 Array(UInt8) +n.s Array(String) CREATE TABLE test.alter ( d Date, k UInt64, i32 Int32, `n.ui8` Array(UInt8), `n.s` Array(String)) ENGINE = MergeTree(d, k, 8192) 2015-01-01 8 40 [1,2,3] ['12','13','14'] 2015-01-01 10 42 [] [] -d Date -k UInt64 -i32 Int32 -n.ui8 Array(UInt8) -n.s Array(String) -n.d Array(Date) +d Date +k UInt64 +i32 Int32 +n.ui8 Array(UInt8) +n.s Array(String) +n.d Array(Date) CREATE TABLE test.alter ( d Date, k UInt64, i32 Int32, `n.ui8` Array(UInt8), `n.s` Array(String), `n.d` Array(Date)) ENGINE = MergeTree(d, k, 8192) 2015-01-01 7 39 [10,20,30] ['120','130','140'] ['2000-01-01','2000-01-01','2000-01-03'] 2015-01-01 8 40 [1,2,3] ['12','13','14'] ['0000-00-00','0000-00-00','0000-00-00'] 2015-01-01 10 42 [] [] [] -d Date -k UInt64 -i32 Int32 -n.ui8 Array(UInt8) -n.s Array(String) -n.d Array(Date) -s String DEFAULT \'0\' +d Date +k UInt64 +i32 Int32 +n.ui8 Array(UInt8) +n.s Array(String) +n.d Array(Date) +s String DEFAULT \'0\' CREATE TABLE test.alter ( d Date, k UInt64, i32 Int32, `n.ui8` Array(UInt8), `n.s` Array(String), `n.d` Array(Date), s String DEFAULT \'0\') ENGINE = MergeTree(d, k, 8192) 2015-01-01 6 38 [10,20,30] ['asd','qwe','qwe'] ['2000-01-01','2000-01-01','2000-01-03'] 100500 2015-01-01 7 39 [10,20,30] ['120','130','140'] ['2000-01-01','2000-01-01','2000-01-03'] 0 2015-01-01 8 40 [1,2,3] ['12','13','14'] ['0000-00-00','0000-00-00','0000-00-00'] 0 2015-01-01 10 42 [] [] [] 0 -d Date -k UInt64 -i32 Int32 -n.ui8 Array(UInt8) -n.s Array(String) -s Int64 +d Date +k UInt64 +i32 Int32 +n.ui8 Array(UInt8) +n.s Array(String) +s Int64 CREATE TABLE test.alter ( d Date, k UInt64, i32 Int32, `n.ui8` Array(UInt8), `n.s` Array(String), s Int64) ENGINE = MergeTree(d, k, 8192) 2015-01-01 6 38 [10,20,30] ['asd','qwe','qwe'] 100500 2015-01-01 7 39 [10,20,30] ['120','130','140'] 0 2015-01-01 8 40 [1,2,3] ['12','13','14'] 0 2015-01-01 10 42 [] [] 0 -d Date -k UInt64 -i32 Int32 -n.ui8 Array(UInt8) -n.s Array(String) -s UInt32 -n.d Array(Date) +d Date +k UInt64 +i32 Int32 +n.ui8 Array(UInt8) +n.s Array(String) +s UInt32 +n.d Array(Date) CREATE TABLE test.alter ( d Date, k UInt64, i32 Int32, `n.ui8` Array(UInt8), `n.s` Array(String), s UInt32, `n.d` Array(Date)) ENGINE = MergeTree(d, k, 8192) 2015-01-01 6 38 [10,20,30] ['asd','qwe','qwe'] 100500 ['0000-00-00','0000-00-00','0000-00-00'] 2015-01-01 7 39 [10,20,30] ['120','130','140'] 0 ['0000-00-00','0000-00-00','0000-00-00'] @@ -60,40 +60,40 @@ CREATE TABLE test.alter ( d Date, k UInt64, i32 Int32, `n.ui8` Array(UInt8), 2015-01-01 7 39 [10,20,30] ['120','130','140'] 0 ['0000-00-00','0000-00-00','0000-00-00'] 2015-01-01 8 40 [1,2,3] ['12','13','14'] 0 ['0000-00-00','0000-00-00','0000-00-00'] 2015-01-01 10 42 [] [] 0 [] -d Date -k UInt64 -i32 Int32 -n.s Array(String) -s UInt32 +d Date +k UInt64 +i32 Int32 +n.s Array(String) +s UInt32 CREATE TABLE test.alter ( d Date, k UInt64, i32 Int32, `n.s` Array(String), s UInt32) ENGINE = MergeTree(d, k, 8192) 2015-01-01 6 38 ['asd','qwe','qwe'] 100500 2015-01-01 7 39 ['120','130','140'] 0 2015-01-01 8 40 ['12','13','14'] 0 2015-01-01 10 42 [] 0 -d Date -k UInt64 -i32 Int32 -s UInt32 +d Date +k UInt64 +i32 Int32 +s UInt32 CREATE TABLE test.alter ( d Date, k UInt64, i32 Int32, s UInt32) ENGINE = MergeTree(d, k, 8192) 2015-01-01 6 38 100500 2015-01-01 7 39 0 2015-01-01 8 40 0 2015-01-01 10 42 0 -d Date -k UInt64 -i32 Int32 -s UInt32 -n.s Array(String) -n.d Array(Date) +d Date +k UInt64 +i32 Int32 +s UInt32 +n.s Array(String) +n.d Array(Date) CREATE TABLE test.alter ( d Date, k UInt64, i32 Int32, s UInt32, `n.s` Array(String), `n.d` Array(Date)) ENGINE = MergeTree(d, k, 8192) 2015-01-01 6 38 100500 [] [] 2015-01-01 7 39 0 [] [] 2015-01-01 8 40 0 [] [] 2015-01-01 10 42 0 [] [] -d Date -k UInt64 -i32 Int32 -s UInt32 +d Date +k UInt64 +i32 Int32 +s UInt32 CREATE TABLE test.alter ( d Date, k UInt64, i32 Int32, s UInt32) ENGINE = MergeTree(d, k, 8192) 2015-01-01 6 38 100500 2015-01-01 7 39 0 diff --git a/dbms/tests/queries/0_stateless/00079_defaulted_columns.reference b/dbms/tests/queries/0_stateless/00079_defaulted_columns.reference index 0665588a532..135da6b3681 100644 --- a/dbms/tests/queries/0_stateless/00079_defaulted_columns.reference +++ b/dbms/tests/queries/0_stateless/00079_defaulted_columns.reference @@ -1,42 +1,42 @@ -col1 UInt8 DEFAULT 0 -col1 UInt32 -col2 UInt64 DEFAULT col1 + 1 -col3 UInt64 MATERIALIZED col1 + 2 -col4 UInt64 ALIAS col1 + 3 +col1 UInt8 DEFAULT 0 +col1 UInt32 +col2 UInt64 DEFAULT col1 + 1 +col3 UInt64 MATERIALIZED col1 + 2 +col4 UInt64 ALIAS col1 + 3 10 11 12 13 99 -payload String -date Date MATERIALIZED today() -key UInt64 MATERIALIZED 0 * rand() +payload String +date Date MATERIALIZED today() +key UInt64 MATERIALIZED 0 * rand() hello clickhouse -payload String -date Date MATERIALIZED today() -key UInt64 MATERIALIZED 0 * rand() -payload_length UInt64 MATERIALIZED length(payload) +payload String +date Date MATERIALIZED today() +key UInt64 MATERIALIZED 0 * rand() +payload_length UInt64 MATERIALIZED length(payload) hello clickhouse 16 hello clickhouse 16 some string 11 hello clickhouse 16 some string 11 -payload String -payload_length UInt64 DEFAULT length(payload) -date Date MATERIALIZED today() -key UInt64 MATERIALIZED 0 * rand() +payload String +payload_length UInt64 DEFAULT length(payload) +date Date MATERIALIZED today() +key UInt64 MATERIALIZED 0 * rand() hello clickhouse 16 some string 11 -payload String -payload_length UInt16 DEFAULT length(payload) % 65535 -date Date MATERIALIZED today() -key UInt64 MATERIALIZED 0 * rand() +payload String +payload_length UInt16 DEFAULT length(payload) % 65535 +date Date MATERIALIZED today() +key UInt64 MATERIALIZED 0 * rand() hello clickhouse 16 some string 11 -payload String -payload_length UInt16 DEFAULT CAST(length(payload), \'UInt16\') -date Date MATERIALIZED today() -key UInt64 MATERIALIZED 0 * rand() -payload String -date Date MATERIALIZED today() -key UInt64 MATERIALIZED 0 * rand() +payload String +payload_length UInt16 DEFAULT CAST(length(payload), \'UInt16\') +date Date MATERIALIZED today() +key UInt64 MATERIALIZED 0 * rand() +payload String +date Date MATERIALIZED today() +key UInt64 MATERIALIZED 0 * rand() hello clickhouse some string diff --git a/dbms/tests/queries/0_stateless/00168_buffer_defaults.reference b/dbms/tests/queries/0_stateless/00168_buffer_defaults.reference index 3f70f0ee2c4..aa378b78ba9 100644 --- a/dbms/tests/queries/0_stateless/00168_buffer_defaults.reference +++ b/dbms/tests/queries/0_stateless/00168_buffer_defaults.reference @@ -1,9 +1,9 @@ -EventDate Date -UTCEventTime DateTime -MoscowEventDate Date DEFAULT toDate(UTCEventTime) -EventDate Date -UTCEventTime DateTime -MoscowEventDate Date DEFAULT toDate(UTCEventTime) +EventDate Date +UTCEventTime DateTime +MoscowEventDate Date DEFAULT toDate(UTCEventTime) +EventDate Date +UTCEventTime DateTime +MoscowEventDate Date DEFAULT toDate(UTCEventTime) 2015-06-09 2015-06-09 01:02:03 2015-06-09 2015-06-09 2015-06-09 01:02:03 2015-06-09 2015-06-09 2015-06-09 01:02:03 2015-06-09 diff --git a/dbms/tests/queries/0_stateless/00294_shard_enums.reference b/dbms/tests/queries/0_stateless/00294_shard_enums.reference index 0f354d57230..28dc7781253 100644 --- a/dbms/tests/queries/0_stateless/00294_shard_enums.reference +++ b/dbms/tests/queries/0_stateless/00294_shard_enums.reference @@ -1,28 +1,28 @@ -d Date DEFAULT CAST(\'2015-12-29\', \'Date\') -k UInt8 DEFAULT 0 -e Enum8(\'hello\' = 1, \'world\' = 2) -sign Enum8(\'minus\' = -1, \'plus\' = 1) -letter Enum16(\'*\' = -256, \'a\' = 0, \'b\' = 1, \'c\' = 2) +d Date DEFAULT CAST(\'2015-12-29\', \'Date\') +k UInt8 DEFAULT 0 +e Enum8(\'hello\' = 1, \'world\' = 2) +sign Enum8(\'minus\' = -1, \'plus\' = 1) +letter Enum16(\'*\' = -256, \'a\' = 0, \'b\' = 1, \'c\' = 2) 2015-12-29 0 hello minus * -d Date DEFAULT CAST(\'2015-12-29\', \'Date\') -k UInt8 DEFAULT 0 -e Enum8(\'hello\' = 1, \'world\' = 2, \'!\' = 3) -sign Enum8(\'minus\' = -1, \'plus\' = 1) -letter Enum16(\'*\' = -256, \'a\' = 0, \'b\' = 1, \'c\' = 2) +d Date DEFAULT CAST(\'2015-12-29\', \'Date\') +k UInt8 DEFAULT 0 +e Enum8(\'hello\' = 1, \'world\' = 2, \'!\' = 3) +sign Enum8(\'minus\' = -1, \'plus\' = 1) +letter Enum16(\'*\' = -256, \'a\' = 0, \'b\' = 1, \'c\' = 2) 2015-12-29 0 hello minus * 2015-12-29 0 ! plus b -d Date DEFAULT CAST(\'2015-12-29\', \'Date\') -k UInt8 DEFAULT 0 -e Enum16(\'hello\' = 1, \'world\' = 2, \'!\' = 3) -sign Enum16(\'minus\' = -1, \'plus\' = 1) -letter Enum16(\'no letter\' = -256, \'a\' = 0, \'b\' = 1, \'c\' = 2) +d Date DEFAULT CAST(\'2015-12-29\', \'Date\') +k UInt8 DEFAULT 0 +e Enum16(\'hello\' = 1, \'world\' = 2, \'!\' = 3) +sign Enum16(\'minus\' = -1, \'plus\' = 1) +letter Enum16(\'no letter\' = -256, \'a\' = 0, \'b\' = 1, \'c\' = 2) 2015-12-29 0 hello minus no letter 2015-12-29 0 ! plus b -d Date DEFAULT CAST(\'2015-12-29\', \'Date\') -k UInt8 DEFAULT 0 -e Enum8(\'hello\' = 1, \'world\' = 2, \'!\' = 3) -sign Enum8(\'minus\' = -1, \'plus\' = 1) -letter Enum16(\'no letter\' = -256, \'a\' = 0, \'b\' = 1, \'c\' = 2) +d Date DEFAULT CAST(\'2015-12-29\', \'Date\') +k UInt8 DEFAULT 0 +e Enum8(\'hello\' = 1, \'world\' = 2, \'!\' = 3) +sign Enum8(\'minus\' = -1, \'plus\' = 1) +letter Enum16(\'no letter\' = -256, \'a\' = 0, \'b\' = 1, \'c\' = 2) 2015-12-29 0 hello minus no letter 2015-12-29 0 ! plus b 2015-12-29 0 world minus c diff --git a/dbms/tests/queries/0_stateless/00415_into_outfile.reference b/dbms/tests/queries/0_stateless/00415_into_outfile.reference index 0aa3993f4c0..1fc1ceac892 100644 --- a/dbms/tests/queries/0_stateless/00415_into_outfile.reference +++ b/dbms/tests/queries/0_stateless/00415_into_outfile.reference @@ -6,7 +6,7 @@ performing test: union_all performing test: bad_union_all query failed performing test: describe_table -dummy UInt8 +dummy UInt8 performing test: clickhouse-local 2 3 performing test: http diff --git a/dbms/tests/queries/0_stateless/00515_shard_desc_table_functions_and_subqueries.reference b/dbms/tests/queries/0_stateless/00515_shard_desc_table_functions_and_subqueries.reference index f24dcad7c8d..5ccb1694c92 100644 --- a/dbms/tests/queries/0_stateless/00515_shard_desc_table_functions_and_subqueries.reference +++ b/dbms/tests/queries/0_stateless/00515_shard_desc_table_functions_and_subqueries.reference @@ -1,30 +1,30 @@ -date Date -val UInt64 -val2 UInt8 DEFAULT 42 -val3 UInt8 DEFAULT CAST(val2 + 1, \'UInt8\') -val4 UInt64 ALIAS val +date Date +val UInt64 +val2 UInt8 DEFAULT 42 +val3 UInt8 DEFAULT CAST(val2 + 1, \'UInt8\') +val4 UInt64 ALIAS val - -date Date -val UInt64 -val2 UInt8 DEFAULT 42 -val3 UInt8 DEFAULT CAST(val2 + 1, \'UInt8\') -val4 UInt64 ALIAS val +date Date +val UInt64 +val2 UInt8 DEFAULT 42 +val3 UInt8 DEFAULT CAST(val2 + 1, \'UInt8\') +val4 UInt64 ALIAS val - -date Date -val UInt64 -val2 UInt8 DEFAULT 42 -val3 UInt8 DEFAULT CAST(val2 + 1, \'UInt8\') -val4 UInt64 ALIAS val +date Date +val UInt64 +val2 UInt8 DEFAULT 42 +val3 UInt8 DEFAULT CAST(val2 + 1, \'UInt8\') +val4 UInt64 ALIAS val - -date Date -val UInt64 -val2 UInt8 DEFAULT 42 -val3 UInt8 DEFAULT CAST(val2 + 1, \'UInt8\') -val4 UInt64 ALIAS val +date Date +val UInt64 +val2 UInt8 DEFAULT 42 +val3 UInt8 DEFAULT CAST(val2 + 1, \'UInt8\') +val4 UInt64 ALIAS val - -1 UInt8 +1 UInt8 - -1 UInt8 +1 UInt8 - -number UInt64 +number UInt64 - diff --git a/dbms/tests/queries/0_stateless/00557_alter_null_storage_tables.reference b/dbms/tests/queries/0_stateless/00557_alter_null_storage_tables.reference index c9634180c5c..35790c754de 100644 --- a/dbms/tests/queries/0_stateless/00557_alter_null_storage_tables.reference +++ b/dbms/tests/queries/0_stateless/00557_alter_null_storage_tables.reference @@ -1,3 +1,3 @@ -x UInt8 -x Int64 DEFAULT toInt64(y) -y String +x UInt8 +x Int64 DEFAULT toInt64(y) +y String diff --git a/dbms/tests/queries/0_stateless/00642_cast.reference b/dbms/tests/queries/0_stateless/00642_cast.reference index 296f1351c16..56f79769cc7 100644 --- a/dbms/tests/queries/0_stateless/00642_cast.reference +++ b/dbms/tests/queries/0_stateless/00642_cast.reference @@ -8,6 +8,6 @@ hello hello 1970-01-01 00:00:01 CREATE TABLE test.cast ( x UInt8, e Enum8('hello' = 1, 'world' = 2) DEFAULT CAST(x, 'Enum8(\'hello\' = 1, \'world\' = 2)')) ENGINE = MergeTree ORDER BY e SETTINGS index_granularity = 8192 -x UInt8 -e Enum8(\'hello\' = 1, \'world\' = 2) DEFAULT CAST(x, \'Enum8(\\\'hello\\\' = 1, \\\'world\\\' = 2)\') +x UInt8 +e Enum8(\'hello\' = 1, \'world\' = 2) DEFAULT CAST(x, \'Enum8(\\\'hello\\\' = 1, \\\'world\\\' = 2)\') 1 hello diff --git a/dbms/tests/queries/0_stateless/00651_default_database_on_client_reconnect.reference b/dbms/tests/queries/0_stateless/00651_default_database_on_client_reconnect.reference index 4482012335a..8ed1dd9ba87 100644 --- a/dbms/tests/queries/0_stateless/00651_default_database_on_client_reconnect.reference +++ b/dbms/tests/queries/0_stateless/00651_default_database_on_client_reconnect.reference @@ -1 +1 @@ -val UInt64 +val UInt64 diff --git a/dbms/tests/queries/0_stateless/00700_decimal_defaults.reference b/dbms/tests/queries/0_stateless/00700_decimal_defaults.reference index 30500d7c53f..635f40a11fe 100644 --- a/dbms/tests/queries/0_stateless/00700_decimal_defaults.reference +++ b/dbms/tests/queries/0_stateless/00700_decimal_defaults.reference @@ -1,9 +1,9 @@ -a Decimal(9, 4) DEFAULT CAST(0, \'Decimal(9, 4)\') -b Decimal(18, 4) DEFAULT CAST(a / 2, \'Decimal(18, 4)\') -c Decimal(38, 4) DEFAULT CAST(b / 3, \'Decimal(38, 4)\') -d Decimal(9, 4) MATERIALIZED a + toDecimal32(\'0.2\', 1) -e Decimal(18, 4) ALIAS b * 2 -f Decimal(38, 4) ALIAS c * 6 +a Decimal(9, 4) DEFAULT CAST(0, \'Decimal(9, 4)\') +b Decimal(18, 4) DEFAULT CAST(a / 2, \'Decimal(18, 4)\') +c Decimal(38, 4) DEFAULT CAST(b / 3, \'Decimal(38, 4)\') +d Decimal(9, 4) MATERIALIZED a + toDecimal32(\'0.2\', 1) +e Decimal(18, 4) ALIAS b * 2 +f Decimal(38, 4) ALIAS c * 6 0.0000 0.0000 0.0000 1.0000 0.5000 0.1666 2.0000 1.0000 0.3333 From befaea63d2c023d3cd32e6dfe5f038a51de3c28d Mon Sep 17 00:00:00 2001 From: Sabyanin Maxim Date: Tue, 13 Nov 2018 15:08:07 +0300 Subject: [PATCH 028/124] add is_mutable method to AlterCommand. change comment column test --- dbms/src/Parsers/ParserCreateQuery.h | 21 +++++++++++------ dbms/src/Storages/AlterCommands.cpp | 23 +++++++++++++++++++ dbms/src/Storages/AlterCommands.h | 2 ++ dbms/src/Storages/IStorage.h | 4 ++-- .../0_stateless/00725_comment_columns.sql | 16 ++++++++++++- 5 files changed, 56 insertions(+), 10 deletions(-) diff --git a/dbms/src/Parsers/ParserCreateQuery.h b/dbms/src/Parsers/ParserCreateQuery.h index 7d2bdb382cd..2d1dc3f9cdf 100644 --- a/dbms/src/Parsers/ParserCreateQuery.h +++ b/dbms/src/Parsers/ParserCreateQuery.h @@ -128,17 +128,26 @@ bool IParserColumnDeclaration::parseImpl(Pos & pos, ASTPtr & node, E const auto fallback_pos = pos; if (!s_default.check(pos, expected) && !s_materialized.check(pos, expected) && - !s_alias.check(pos, expected)) + !s_alias.check(pos, expected) && + !s_comment.check(pos, expected)) { type_parser.parse(pos, type, expected); } else pos = fallback_pos; - /// parse {DEFAULT, MATERIALIZED, ALIAS} + /// parse {DEFAULT, MATERIALIZED, ALIAS, COMMENT} String default_specifier; ASTPtr default_expression; + ASTPtr comment_expression; Pos pos_before_specifier = pos; + if (!s_default.ignore(pos, expected) && + !s_materialized.ignore(pos, expected) && + !s_alias.ignore(pos, expected) && + !s_comment.ignore(pos, expected) && + !type) + return false; /// reject sole column name without type + if (s_default.ignore(pos, expected) || s_materialized.ignore(pos, expected) || s_alias.ignore(pos, expected)) @@ -149,14 +158,12 @@ bool IParserColumnDeclaration::parseImpl(Pos & pos, ASTPtr & node, E if (!expr_parser.parse(pos, default_expression, expected)) return false; } - else if (!type) - return false; /// reject sole column name without type - - ASTPtr comment_expression; - if (s_comment.ignore(pos, expected)) + else if (s_comment.ignore(pos, expected)) { string_literal_parser.parse(pos, comment_expression, expected); } + else if (!type) // TODO: тут надо очень хорошо подумать. есть проблема с тем, что для modify column имя колонки и коммент ок, а для создания таблицы не ок. + return false; /// reject sole column name without type const auto column_declaration = std::make_shared(); node = column_declaration; diff --git a/dbms/src/Storages/AlterCommands.cpp b/dbms/src/Storages/AlterCommands.cpp index 9e8239b79b3..a16e813961b 100644 --- a/dbms/src/Storages/AlterCommands.cpp +++ b/dbms/src/Storages/AlterCommands.cpp @@ -188,6 +188,20 @@ void AlterCommand::apply(ColumnsDescription & columns_description) const } else if (type == MODIFY_COLUMN) { + if (!is_mutable()) + { + auto & comments = columns_description.comments; + if (comment.empty()) + { + if (auto it = comments.find(column_name); it != comments.end()) + comments.erase(it); + } + else + columns_description.comments[column_name] = comment; + + return; + } + const auto default_it = columns_description.defaults.find(column_name); const auto had_default_expr = default_it != std::end(columns_description.defaults); const auto old_default_kind = had_default_expr ? default_it->second.kind : ColumnDefaultKind{}; @@ -256,6 +270,15 @@ void AlterCommand::apply(ColumnsDescription & columns_description) const throw Exception("Wrong parameter type in ALTER query", ErrorCodes::LOGICAL_ERROR); } +bool AlterCommand::is_mutable() const +{ + if (type == COMMENT_COLUMN) + return false; + if (type == MODIFY_COLUMN) + return data_type.get() || default_expression; + + return true; +} void AlterCommands::apply(ColumnsDescription & columns_description) const { diff --git a/dbms/src/Storages/AlterCommands.h b/dbms/src/Storages/AlterCommands.h index 7c56740a2fe..f6d7861dcce 100644 --- a/dbms/src/Storages/AlterCommands.h +++ b/dbms/src/Storages/AlterCommands.h @@ -56,6 +56,8 @@ struct AlterCommand void apply(ColumnsDescription & columns_description) const; + /// Checks that not only metadata touched by that command + bool is_mutable() const; }; class IStorage; diff --git a/dbms/src/Storages/IStorage.h b/dbms/src/Storages/IStorage.h index 9e2f9741613..4b4837abc5c 100644 --- a/dbms/src/Storages/IStorage.h +++ b/dbms/src/Storages/IStorage.h @@ -239,8 +239,8 @@ public: { for (const auto & param : params) { - if (param.type != AlterCommand::Type::COMMENT_COLUMN) - throw Exception("Method alter only supports change comment of column for storage " + getName(), ErrorCodes::NOT_IMPLEMENTED); + if (param.is_mutable()) + throw Exception("Method alter supports only change comment of column for storage " + getName(), ErrorCodes::NOT_IMPLEMENTED); } auto lock = lockStructureForAlter(__PRETTY_FUNCTION__); diff --git a/dbms/tests/queries/0_stateless/00725_comment_columns.sql b/dbms/tests/queries/0_stateless/00725_comment_columns.sql index afcb1b34eab..ada4e4e1983 100644 --- a/dbms/tests/queries/0_stateless/00725_comment_columns.sql +++ b/dbms/tests/queries/0_stateless/00725_comment_columns.sql @@ -60,4 +60,18 @@ FROM system.columns WHERE table = 'check_query_comment_column' and database = 'test' FORMAT PrettyCompactNoEscapes; -DROP table test.check_query_comment_column; \ No newline at end of file +DROP TABLE IF test.check_query_comment_column; + +CREATE TABLE test.check_query_comment_column + ( + first_column UInt8 COMMENT 'first comment' + ) ENGINE = TinyLog; + +ALTER TABLE test.check_query_comment_column MODIFY COLUMN first_column COMMENT 'another comment'; + +SELECT table, name, comment +FROM system.columns +WHERE table = 'check_query_comment_column' and database = 'test' +FORMAT PrettyCompactNoEscapes; + +DROP TABLE IF EXISTS test.check_query_comment_column; From 050abdf2b65bc1ddea86ce070fd3ebd34fde47c2 Mon Sep 17 00:00:00 2001 From: Alexandr Krasheninnikov Date: Wed, 14 Nov 2018 12:07:20 +0300 Subject: [PATCH 029/124] Auto detect of codec --- dbms/src/Functions/FunctionBase64Conversion.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/src/Functions/FunctionBase64Conversion.h b/dbms/src/Functions/FunctionBase64Conversion.h index 6f3169f1f49..418de6e59c8 100644 --- a/dbms/src/Functions/FunctionBase64Conversion.h +++ b/dbms/src/Functions/FunctionBase64Conversion.h @@ -159,7 +159,7 @@ public: private: static int getCodec() { - return BASE64_FORCE_PLAIN; + return 0; } }; } From 035049d1a864cb0799108319c6999b6a5027438c Mon Sep 17 00:00:00 2001 From: Sabyanin Maxim Date: Thu, 15 Nov 2018 01:46:39 +0300 Subject: [PATCH 030/124] make modify column comment work --- dbms/src/Parsers/IParser.h | 8 ++++ dbms/src/Parsers/ParserAlterQuery.cpp | 3 +- dbms/src/Parsers/ParserCreateQuery.h | 55 ++++++++++++++------------- dbms/src/Storages/AlterCommands.cpp | 8 +++- dbms/src/Storages/AlterCommands.h | 5 ++- 5 files changed, 48 insertions(+), 31 deletions(-) diff --git a/dbms/src/Parsers/IParser.h b/dbms/src/Parsers/IParser.h index 1bee3005fdd..b300e99c4cf 100644 --- a/dbms/src/Parsers/IParser.h +++ b/dbms/src/Parsers/IParser.h @@ -86,6 +86,14 @@ public: return true; } + /* The same, but never move the position and do not write the result to node. + */ + bool check_without_moving(Pos pos, Expected & expected) + { + ASTPtr node; + return parse(pos, node, expected); + } + virtual ~IParser() {} }; diff --git a/dbms/src/Parsers/ParserAlterQuery.cpp b/dbms/src/Parsers/ParserAlterQuery.cpp index 65446e5a4db..a85c385dc7e 100644 --- a/dbms/src/Parsers/ParserAlterQuery.cpp +++ b/dbms/src/Parsers/ParserAlterQuery.cpp @@ -47,6 +47,7 @@ bool ParserAlterCommand::parseImpl(Pos & pos, ASTPtr & node, Expected & expected ParserCompoundIdentifier parser_name; ParserStringLiteral parser_string_literal; ParserCompoundColumnDeclaration parser_col_decl; + ParserCompoundColumnDeclaration parser_modify_col_decl(false); ParserPartition parser_partition; ParserExpression parser_exp_elem; ParserList parser_assignment_list( @@ -181,7 +182,7 @@ bool ParserAlterCommand::parseImpl(Pos & pos, ASTPtr & node, Expected & expected } else if (s_modify_column.ignore(pos, expected)) { - if (!parser_col_decl.parse(pos, command->col_decl, expected)) + if (!parser_modify_col_decl.parse(pos, command->col_decl, expected)) return false; command->type = ASTAlterCommand::MODIFY_COLUMN; diff --git a/dbms/src/Parsers/ParserCreateQuery.h b/dbms/src/Parsers/ParserCreateQuery.h index 2d1dc3f9cdf..59a8540b84f 100644 --- a/dbms/src/Parsers/ParserCreateQuery.h +++ b/dbms/src/Parsers/ParserCreateQuery.h @@ -96,9 +96,16 @@ protected: template class IParserColumnDeclaration : public IParserBase { +public: + explicit IParserColumnDeclaration(bool require_type_ = true) : require_type(require_type_) + { + } + protected: const char * getName() const { return "column declaration"; } bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected); + + bool require_type = true; }; using ParserColumnDeclaration = IParserColumnDeclaration; @@ -122,35 +129,24 @@ bool IParserColumnDeclaration::parseImpl(Pos & pos, ASTPtr & node, E return false; /** column name should be followed by type name if it - * is not immediately followed by {DEFAULT, MATERIALIZED, ALIAS} + * is not immediately followed by {DEFAULT, MATERIALIZED, ALIAS, COMMENT} */ ASTPtr type; - const auto fallback_pos = pos; - if (!s_default.check(pos, expected) && - !s_materialized.check(pos, expected) && - !s_alias.check(pos, expected) && - !s_comment.check(pos, expected)) - { - type_parser.parse(pos, type, expected); - } - else - pos = fallback_pos; - - /// parse {DEFAULT, MATERIALIZED, ALIAS, COMMENT} String default_specifier; ASTPtr default_expression; ASTPtr comment_expression; - Pos pos_before_specifier = pos; - if (!s_default.ignore(pos, expected) && - !s_materialized.ignore(pos, expected) && - !s_alias.ignore(pos, expected) && - !s_comment.ignore(pos, expected) && - !type) - return false; /// reject sole column name without type - if (s_default.ignore(pos, expected) || - s_materialized.ignore(pos, expected) || - s_alias.ignore(pos, expected)) + if (!s_default.check_without_moving(pos, expected) && + !s_materialized.check_without_moving(pos, expected) && + !s_alias.check_without_moving(pos, expected) && + !s_comment.check_without_moving(pos, expected)) + { + if (!type_parser.parse(pos, type, expected)) + return false; + } + + Pos pos_before_specifier = pos; + if (s_default.ignore(pos, expected) || s_materialized.ignore(pos, expected) || s_alias.ignore(pos, expected)) { default_specifier = Poco::toUpper(std::string{pos_before_specifier->begin, pos_before_specifier->end}); @@ -158,12 +154,17 @@ bool IParserColumnDeclaration::parseImpl(Pos & pos, ASTPtr & node, E if (!expr_parser.parse(pos, default_expression, expected)) return false; } - else if (s_comment.ignore(pos, expected)) + + if (require_type && !type && !default_expression) + return false; /// reject column name without type + + + if (s_comment.ignore(pos, expected)) { - string_literal_parser.parse(pos, comment_expression, expected); + /// should be followed by a string literal + if (!string_literal_parser.parse(pos, comment_expression, expected)) + return false; } - else if (!type) // TODO: тут надо очень хорошо подумать. есть проблема с тем, что для modify column имя колонки и коммент ок, а для создания таблицы не ок. - return false; /// reject sole column name without type const auto column_declaration = std::make_shared(); node = column_declaration; diff --git a/dbms/src/Storages/AlterCommands.cpp b/dbms/src/Storages/AlterCommands.cpp index a16e813961b..3c75f806ad1 100644 --- a/dbms/src/Storages/AlterCommands.cpp +++ b/dbms/src/Storages/AlterCommands.cpp @@ -81,6 +81,12 @@ std::optional AlterCommand::parse(const ASTAlterCommand * command_ command.default_expression = ast_col_decl.default_expression; } + if (ast_col_decl.comment) + { + const auto & ast_comment = typeid_cast(*ast_col_decl.comment); + command.comment = ast_comment.value.get(); + } + return command; } else if (command_ast->type == ASTAlterCommand::MODIFY_PRIMARY_KEY) @@ -276,7 +282,7 @@ bool AlterCommand::is_mutable() const return false; if (type == MODIFY_COLUMN) return data_type.get() || default_expression; - + // TODO: возможно, здесь нужно дополнить return true; } diff --git a/dbms/src/Storages/AlterCommands.h b/dbms/src/Storages/AlterCommands.h index f6d7861dcce..ac6d74d249a 100644 --- a/dbms/src/Storages/AlterCommands.h +++ b/dbms/src/Storages/AlterCommands.h @@ -22,9 +22,10 @@ struct AlterCommand MODIFY_COLUMN, MODIFY_PRIMARY_KEY, COMMENT_COLUMN, + UKNOWN_TYPE, }; - Type type; + Type type = UKNOWN_TYPE; String column_name; @@ -47,7 +48,7 @@ struct AlterCommand AlterCommand() = default; AlterCommand(const Type type, const String & column_name, const DataTypePtr & data_type, const ColumnDefaultKind default_kind, const ASTPtr & default_expression, - const String & after_column = String{}, const String & comment = "") + const String & after_column = String{}, const String & comment = "") // TODO: разобраться здесь с параметром по умолчанию : type{type}, column_name{column_name}, data_type{data_type}, default_kind{default_kind}, default_expression{default_expression}, comment(comment), after_column{after_column} {} From d9e6789f781693261eeec06cd28f49b314fe0b81 Mon Sep 17 00:00:00 2001 From: Sabyanin Maxim Date: Thu, 15 Nov 2018 16:12:27 +0300 Subject: [PATCH 031/124] make MergeTree work with comments --- dbms/src/Storages/AlterCommands.cpp | 11 +++ dbms/src/Storages/AlterCommands.h | 1 + dbms/src/Storages/MergeTree/MergeTreeData.cpp | 2 +- dbms/src/Storages/StorageMergeTree.cpp | 10 +++ .../00725_comment_columns.reference | 66 +++++++++-------- .../0_stateless/00725_comment_columns.sql | 70 +++++++++++-------- 6 files changed, 103 insertions(+), 57 deletions(-) diff --git a/dbms/src/Storages/AlterCommands.cpp b/dbms/src/Storages/AlterCommands.cpp index 3c75f806ad1..6748f56a5ab 100644 --- a/dbms/src/Storages/AlterCommands.cpp +++ b/dbms/src/Storages/AlterCommands.cpp @@ -477,4 +477,15 @@ void AlterCommands::validate(const IStorage & table, const Context & context) } } +bool AlterCommands::is_mutable() const +{ + for (const auto & param : *this) + { + if (param.is_mutable()) + return true; + } + + return false; +} + } diff --git a/dbms/src/Storages/AlterCommands.h b/dbms/src/Storages/AlterCommands.h index ac6d74d249a..e870e2f9762 100644 --- a/dbms/src/Storages/AlterCommands.h +++ b/dbms/src/Storages/AlterCommands.h @@ -70,6 +70,7 @@ public: void apply(ColumnsDescription & columns_description) const; void validate(const IStorage & table, const Context & context); + bool is_mutable() const; }; } diff --git a/dbms/src/Storages/MergeTree/MergeTreeData.cpp b/dbms/src/Storages/MergeTree/MergeTreeData.cpp index 012991db465..a9126dc79a9 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeData.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeData.cpp @@ -908,7 +908,7 @@ void MergeTreeData::checkAlter(const AlterCommands & commands) for (const AlterCommand & command : commands) { - if (command.type == AlterCommand::COMMENT_COLUMN) + if (!command.is_mutable()) { continue; } diff --git a/dbms/src/Storages/StorageMergeTree.cpp b/dbms/src/Storages/StorageMergeTree.cpp index 0e926218a05..2686427fa74 100644 --- a/dbms/src/Storages/StorageMergeTree.cpp +++ b/dbms/src/Storages/StorageMergeTree.cpp @@ -188,6 +188,16 @@ void StorageMergeTree::alter( const String & table_name, const Context & context) { + if (!params.is_mutable()) + { + auto table_soft_lock = lockStructureForAlter(__PRETTY_FUNCTION__); + auto new_columns = getColumns(); + params.apply(new_columns); + context.getDatabase(database_name)->alterTable(context, table_name, new_columns, {}); + setColumns(std::move(new_columns)); + return; + } + /// NOTE: Here, as in ReplicatedMergeTree, you can do ALTER which does not block the writing of data for a long time. auto merge_blocker = merger_mutator.actions_blocker.cancel(); diff --git a/dbms/tests/queries/0_stateless/00725_comment_columns.reference b/dbms/tests/queries/0_stateless/00725_comment_columns.reference index 19dac99b721..ca4edfb3122 100644 --- a/dbms/tests/queries/0_stateless/00725_comment_columns.reference +++ b/dbms/tests/queries/0_stateless/00725_comment_columns.reference @@ -1,28 +1,38 @@ -CREATE TABLE test.check_query_comment_column ( first_column UInt8 DEFAULT 1 COMMENT \'first comment\', fourth_column UInt8 COMMENT \'fourth comment\', fifth_column UInt8, second_column UInt8 MATERIALIZED first_column COMMENT \'second comment\', third_column UInt8 ALIAS second_column COMMENT \'third comment\') ENGINE = TinyLog -┌─table──────────────────────┬─name──────────┬─comment────────┐ -│ check_query_comment_column │ first_column │ first comment │ -│ check_query_comment_column │ fourth_column │ fourth comment │ -│ check_query_comment_column │ fifth_column │ │ -│ check_query_comment_column │ second_column │ second comment │ -│ check_query_comment_column │ third_column │ third comment │ -└────────────────────────────┴───────────────┴────────────────┘ -CREATE TABLE test.check_query_comment_column ( first_column UInt8 DEFAULT 1 COMMENT \'another first column\', fourth_column UInt8 COMMENT \'another fourth column\', fifth_column UInt8 COMMENT \'another fifth column\', second_column UInt8 MATERIALIZED first_column COMMENT \'another second column\', third_column UInt8 ALIAS second_column COMMENT \'another third column\') ENGINE = TinyLog -┌─table──────────────────────┬─name──────────┬─comment───────────────┐ -│ check_query_comment_column │ first_column │ another first column │ -│ check_query_comment_column │ fourth_column │ another fourth column │ -│ check_query_comment_column │ fifth_column │ another fifth column │ -│ check_query_comment_column │ second_column │ another second column │ -│ check_query_comment_column │ third_column │ another third column │ -└────────────────────────────┴───────────────┴───────────────────────┘ -CREATE TABLE test.check_query_comment_column ( first_column Date COMMENT \'first comment\', second_column UInt8 COMMENT \'second comment\', third_column UInt8 COMMENT \'third comment\') ENGINE = MergeTree(first_column, (second_column, second_column), 8192) -┌─table──────────────────────┬─name──────────┬─comment────────┐ -│ check_query_comment_column │ first_column │ first comment │ -│ check_query_comment_column │ second_column │ second comment │ -│ check_query_comment_column │ third_column │ third comment │ -└────────────────────────────┴───────────────┴────────────────┘ -CREATE TABLE test.check_query_comment_column ( first_column Date COMMENT \'another first comment\', second_column UInt8 COMMENT \'another second comment\', third_column UInt8 COMMENT \'another third comment\') ENGINE = MergeTree(first_column, (second_column, second_column), 8192) -┌─table──────────────────────┬─name──────────┬─comment────────────────┐ -│ check_query_comment_column │ first_column │ another first comment │ -│ check_query_comment_column │ second_column │ another second comment │ -│ check_query_comment_column │ third_column │ another third comment │ -└────────────────────────────┴───────────────┴────────────────────────┘ +CREATE TABLE test.check_query_comment_column ( first_column UInt8 DEFAULT 1 COMMENT \'comment 1\', fourth_column UInt8 COMMENT \'comment 4\', fifth_column UInt8, second_column UInt8 MATERIALIZED first_column COMMENT \'comment 2\', third_column UInt8 ALIAS second_column COMMENT \'comment 3\') ENGINE = TinyLog +first_column UInt8 DEFAULT 1 comment 1 +fourth_column UInt8 comment 4 +fifth_column UInt8 +second_column UInt8 MATERIALIZED first_column comment 2 +third_column UInt8 ALIAS second_column comment 3 +┌─table──────────────────────┬─name──────────┬─comment───┐ +│ check_query_comment_column │ first_column │ comment 1 │ +│ check_query_comment_column │ fourth_column │ comment 4 │ +│ check_query_comment_column │ fifth_column │ │ +│ check_query_comment_column │ second_column │ comment 2 │ +│ check_query_comment_column │ third_column │ comment 3 │ +└────────────────────────────┴───────────────┴───────────┘ +CREATE TABLE test.check_query_comment_column ( first_column UInt8 DEFAULT 1 COMMENT \'comment 1_1\', fourth_column UInt8 COMMENT \'comment 4_1\', fifth_column UInt8 COMMENT \'comment 5_1\', second_column UInt8 MATERIALIZED first_column COMMENT \'comment 2_1\', third_column UInt8 ALIAS second_column COMMENT \'comment 3_1\') ENGINE = TinyLog +┌─table──────────────────────┬─name──────────┬─comment─────┐ +│ check_query_comment_column │ first_column │ comment 1_2 │ +│ check_query_comment_column │ fourth_column │ comment 4_2 │ +│ check_query_comment_column │ fifth_column │ comment 5_2 │ +│ check_query_comment_column │ second_column │ comment 2_2 │ +│ check_query_comment_column │ third_column │ comment 3_2 │ +└────────────────────────────┴───────────────┴─────────────┘ +CREATE TABLE test.check_query_comment_column ( first_column UInt8 DEFAULT 1 COMMENT \'comment 1_2\', fourth_column UInt8 COMMENT \'comment 4_2\', fifth_column UInt8 COMMENT \'comment 5_2\', second_column UInt8 MATERIALIZED first_column COMMENT \'comment 2_2\', third_column UInt8 ALIAS second_column COMMENT \'comment 3_2\') ENGINE = TinyLog +CREATE TABLE test.check_query_comment_column ( first_column UInt8 COMMENT \'comment 1\', second_column UInt8 COMMENT \'comment 2\', third_column UInt8 COMMENT \'comment 3\') ENGINE = MergeTree() PARTITION BY second_column ORDER BY first_column SAMPLE BY first_column SETTINGS index_granularity = 8192 +first_column UInt8 comment 1 +second_column UInt8 comment 2 +third_column UInt8 comment 3 +┌─table──────────────────────┬─name──────────┬─comment───┐ +│ check_query_comment_column │ first_column │ comment 1 │ +│ check_query_comment_column │ second_column │ comment 2 │ +│ check_query_comment_column │ third_column │ comment 3 │ +└────────────────────────────┴───────────────┴───────────┘ +CREATE TABLE test.check_query_comment_column ( first_column UInt8 COMMENT \'comment 1_2\', second_column UInt8 COMMENT \'comment 2_2\', third_column UInt8 COMMENT \'comment 3_2\') ENGINE = MergeTree() PARTITION BY second_column ORDER BY first_column SAMPLE BY first_column SETTINGS index_granularity = 8192 +CREATE TABLE test.check_query_comment_column ( first_column UInt8 COMMENT \'comment 1_3\', second_column UInt8 COMMENT \'comment 2_3\', third_column UInt8 COMMENT \'comment 3_3\') ENGINE = MergeTree() PARTITION BY second_column ORDER BY first_column SAMPLE BY first_column SETTINGS index_granularity = 8192 +┌─table──────────────────────┬─name──────────┬─comment─────┐ +│ check_query_comment_column │ first_column │ comment 1_3 │ +│ check_query_comment_column │ second_column │ comment 2_3 │ +│ check_query_comment_column │ third_column │ comment 3_3 │ +└────────────────────────────┴───────────────┴─────────────┘ diff --git a/dbms/tests/queries/0_stateless/00725_comment_columns.sql b/dbms/tests/queries/0_stateless/00725_comment_columns.sql index ada4e4e1983..97667616983 100644 --- a/dbms/tests/queries/0_stateless/00725_comment_columns.sql +++ b/dbms/tests/queries/0_stateless/00725_comment_columns.sql @@ -1,16 +1,18 @@ CREATE DATABASE IF NOT EXISTS test; DROP TABLE IF EXISTS test.check_query_comment_column; +-- Check COMMENT COLUMN and MODIFY COLUMN statements with simple engine CREATE TABLE test.check_query_comment_column ( - first_column UInt8 DEFAULT 1 COMMENT 'first comment', - second_column UInt8 MATERIALIZED first_column COMMENT 'second comment', - third_column UInt8 ALIAS second_column COMMENT 'third comment', - fourth_column UInt8 COMMENT 'fourth comment', + first_column UInt8 DEFAULT 1 COMMENT 'comment 1', + second_column UInt8 MATERIALIZED first_column COMMENT 'comment 2', + third_column UInt8 ALIAS second_column COMMENT 'comment 3', + fourth_column UInt8 COMMENT 'comment 4', fifth_column UInt8 ) ENGINE = TinyLog; SHOW CREATE TABLE test.check_query_comment_column; +DESCRIBE TABLE test.check_query_comment_column; SELECT table, name, comment FROM system.columns @@ -18,30 +20,42 @@ WHERE table = 'check_query_comment_column' AND database = 'test' FORMAT PrettyCompactNoEscapes; ALTER TABLE test.check_query_comment_column - COMMENT COLUMN first_column 'another first column', - COMMENT COLUMN second_column 'another second column', - COMMENT COLUMN third_column 'another third column', - COMMENT COLUMN fourth_column 'another fourth column', - COMMENT COLUMN fifth_column 'another fifth column'; + COMMENT COLUMN first_column 'comment 1_1', + COMMENT COLUMN second_column 'comment 2_1', + COMMENT COLUMN third_column 'comment 3_1', + COMMENT COLUMN fourth_column 'comment 4_1', + COMMENT COLUMN fifth_column 'comment 5_1'; SHOW CREATE TABLE test.check_query_comment_column; +ALTER TABLE test.check_query_comment_column + MODIFY COLUMN first_column COMMENT 'comment 1_2', + MODIFY COLUMN second_column COMMENT 'comment 2_2', + MODIFY COLUMN third_column COMMENT 'comment 3_2', + MODIFY COLUMN fourth_column COMMENT 'comment 4_2', + MODIFY COLUMN fifth_column COMMENT 'comment 5_2'; + SELECT table, name, comment FROM system.columns WHERE table = 'check_query_comment_column' AND database = 'test' FORMAT PrettyCompactNoEscapes; +SHOW CREATE TABLE test.check_query_comment_column; DROP TABLE IF EXISTS test.check_query_comment_column; - +-- Check `ALTER TABLE table_name COMMENT COLUMN 'comment'` statement with MergeTree engine CREATE TABLE test.check_query_comment_column ( - first_column Date COMMENT 'first comment', - second_column UInt8 COMMENT 'second comment', - third_column UInt8 COMMENT 'third comment' - ) ENGINE = MergeTree(first_column, (second_column, second_column), 8192); + first_column UInt8 COMMENT 'comment 1', + second_column UInt8 COMMENT 'comment 2', + third_column UInt8 COMMENT 'comment 3' + ) ENGINE = MergeTree() + ORDER BY first_column + PARTITION BY second_column + SAMPLE BY first_column; SHOW CREATE TABLE test.check_query_comment_column; +DESCRIBE TABLE test.check_query_comment_column; SELECT table, name, comment FROM system.columns @@ -49,25 +63,23 @@ WHERE table = 'check_query_comment_column' AND database = 'test' FORMAT PrettyCompactNoEscapes; ALTER TABLE test.check_query_comment_column - COMMENT COLUMN first_column 'another first comment', - COMMENT COLUMN second_column 'another second comment', - COMMENT COLUMN third_column 'another third comment'; + COMMENT COLUMN first_column 'comment 1_2', + COMMENT COLUMN second_column 'comment 2_2', + COMMENT COLUMN third_column 'comment 3_2'; SHOW CREATE TABLE test.check_query_comment_column; -SELECT table, name, comment -FROM system.columns -WHERE table = 'check_query_comment_column' and database = 'test' -FORMAT PrettyCompactNoEscapes; +ALTER TABLE test.check_query_comment_column + MODIFY COLUMN first_column COMMENT 'comment 1_3', + MODIFY COLUMN second_column COMMENT 'comment 2_3', + MODIFY COLUMN third_column COMMENT 'comment 3_3'; -DROP TABLE IF test.check_query_comment_column; +SHOW CREATE TABLE test.check_query_comment_column; -CREATE TABLE test.check_query_comment_column - ( - first_column UInt8 COMMENT 'first comment' - ) ENGINE = TinyLog; - -ALTER TABLE test.check_query_comment_column MODIFY COLUMN first_column COMMENT 'another comment'; +ALTER TABLE test.check_query_comment_column + MODIFY COLUMN first_column DEFAULT 1 COMMENT 'comment 1_3', + MODIFY COLUMN second_column COMMENT 'comment 2_3', -- We can't change default value of partition key. + MODIFY COLUMN third_column DEFAULT 1 COMMENT 'comment 3_3'; SELECT table, name, comment FROM system.columns @@ -75,3 +87,5 @@ WHERE table = 'check_query_comment_column' and database = 'test' FORMAT PrettyCompactNoEscapes; DROP TABLE IF EXISTS test.check_query_comment_column; + +-- TODO: add here tests with ReplicatedMergeTree From 49dc9939d4497d5ed610fc6f346a530bdc03495c Mon Sep 17 00:00:00 2001 From: Sabyanin Maxim Date: Fri, 16 Nov 2018 15:01:52 +0300 Subject: [PATCH 032/124] fix serializing of ColumnsDescription --- dbms/src/Storages/ColumnsDescription.cpp | 112 +++++++++++------------ 1 file changed, 52 insertions(+), 60 deletions(-) diff --git a/dbms/src/Storages/ColumnsDescription.cpp b/dbms/src/Storages/ColumnsDescription.cpp index 7ac7b07d955..087c6dbf605 100644 --- a/dbms/src/Storages/ColumnsDescription.cpp +++ b/dbms/src/Storages/ColumnsDescription.cpp @@ -1,5 +1,6 @@ #include #include +#include #include #include #include @@ -110,29 +111,17 @@ String ColumnsDescription::toString() const write_columns(ordinary); write_columns(materialized); write_columns(aliases); - return buf.str(); } -struct ParsedDefaultInfo -{ - ColumnDefaultKind default_kind; - ASTPtr default_expr_str; -}; - -std::optional parseDefaultInfo(ReadBufferFromString & buf) +std::optional parseDefaultInfo(ReadBufferFromString & buf) { if (*buf.position() == '\n') - { return {}; - } assertChar('\t', buf); if (*buf.position() == '\t') - { - assertChar('\t', buf); return {}; - } String default_kind_str; readText(default_kind_str, buf); @@ -143,21 +132,63 @@ std::optional parseDefaultInfo(ReadBufferFromString & buf) String default_expr_str; readText(default_expr_str, buf); ASTPtr default_expr = parseQuery(expr_parser, default_expr_str, "default_expression", 0); - return ParsedDefaultInfo{default_kind, std::move(default_expr)}; + return ColumnDefault{default_kind, std::move(default_expr)}; } String parseComment(ReadBufferFromString& buf) { if (*buf.position() == '\n') - { return {}; + + assertChar('\t', buf); + ParserStringLiteral string_literal_parser; + String comment_expr_str; + readText(comment_expr_str, buf); + ASTPtr comment_expr = parseQuery(string_literal_parser, comment_expr_str, "comment expression", 0); + return typeid_cast(*comment_expr).value.get(); +} + +void parseColumn(ReadBufferFromString & buf, ColumnsDescription & result, const DataTypeFactory & data_type_factory) +{ + String column_name; + readBackQuotedStringWithSQLStyle(column_name, buf); + assertChar(' ', buf); + + String type_name; + readText(type_name, buf); + auto type = data_type_factory.get(type_name); + if (*buf.position() == '\n') + { + assertChar('\n', buf); + result.ordinary.emplace_back(column_name, std::move(type)); + return; } - ParserExpression parser_expr; - String comment_expr_str; - readText(comment_expr_str, buf); // This is wrong may be - ASTPtr comment_expr = parseQuery(parser_expr, comment_expr_str, "comment expression", 0); - return typeid_cast(*comment_expr).value.get(); + auto column_default = parseDefaultInfo(buf); + if (column_default) + { + switch (column_default->kind) + { + case ColumnDefaultKind::Default: + result.ordinary.emplace_back(column_name, std::move(type)); + break; + case ColumnDefaultKind::Materialized: + result.materialized.emplace_back(column_name, std::move(type)); + break; + case ColumnDefaultKind::Alias: + result.aliases.emplace_back(column_name, std::move(type)); + } + + result.defaults.emplace(column_name, std::move(*column_default)); + } + + auto comment = parseComment(buf); + if (!comment.empty()) + { + result.comments.emplace(column_name, std::move(comment)); + } + + assertChar('\n', buf); } ColumnsDescription ColumnsDescription::parse(const String & str) @@ -175,49 +206,10 @@ ColumnsDescription ColumnsDescription::parse(const String & str) ColumnsDescription result; for (size_t i = 0; i < count; ++i) { - String column_name; - readBackQuotedStringWithSQLStyle(column_name, buf); - assertChar(' ', buf); - - String type_name; - readText(type_name, buf); - auto type = data_type_factory.get(type_name); - if (*buf.position() == '\n') - { - assertChar('\n', buf); - - result.ordinary.emplace_back(column_name, std::move(type)); - continue; - } - - assertChar('\t', buf); - - const auto default_info = parseDefaultInfo(buf); - if (default_info) - { - const auto & default_kind = default_info->default_kind; - const auto & default_expr = default_info->default_expr_str; - if (ColumnDefaultKind::Default == default_kind) - result.ordinary.emplace_back(column_name, std::move(type)); - else if (ColumnDefaultKind::Materialized == default_kind) - result.materialized.emplace_back(column_name, std::move(type)); - else if (ColumnDefaultKind::Alias == default_kind) - result.aliases.emplace_back(column_name, std::move(type)); - - result.defaults.emplace(column_name, ColumnDefault{default_kind, default_expr}); - } - - const auto comment = parseComment(buf); - if (!comment.empty()) - { - result.comments.emplace(column_name, comment); - } - - assertChar('\n', buf); + parseColumn(buf, result, data_type_factory); } assertEOF(buf); - return result; } From 849c9018eb852bb7f0e54718248634d5debee723 Mon Sep 17 00:00:00 2001 From: Pavlo Bashynskiy Date: Tue, 20 Nov 2018 00:07:34 +0200 Subject: [PATCH 033/124] Running daemon in custom environment --- debian/clickhouse-server.init | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/debian/clickhouse-server.init b/debian/clickhouse-server.init index 92978c72257..0df35f55952 100755 --- a/debian/clickhouse-server.init +++ b/debian/clickhouse-server.init @@ -14,6 +14,7 @@ CLICKHOUSE_GROUP=${CLICKHOUSE_USER} SHELL=/bin/bash PROGRAM=clickhouse-server GENERIC_PROGRAM=clickhouse +CLICKHOUSE_PROGRAM_ENV="" EXTRACT_FROM_CONFIG=${GENERIC_PROGRAM}-extract-from-config SYSCONFDIR=/etc/$PROGRAM CLICKHOUSE_LOGDIR=/var/log/clickhouse-server @@ -168,7 +169,7 @@ start() if ! is_running; then # Lock should not be held while running child process, so we release the lock. Note: obviously, there is race condition. # But clickhouse-server has protection from simultaneous runs with same data directory. - su -s $SHELL ${CLICKHOUSE_USER} -c "$FLOCK -u 9; exec -a \"$PROGRAM\" \"$BINDIR/$PROGRAM\" --daemon --pid-file=\"$CLICKHOUSE_PIDFILE\" --config-file=\"$CLICKHOUSE_CONFIG\"" + su -s $SHELL ${CLICKHOUSE_USER} -c "$FLOCK -u 9; $CLICKHOUSE_PROGRAM_ENV exec -a \"$PROGRAM\" \"$BINDIR/$PROGRAM\" --daemon --pid-file=\"$CLICKHOUSE_PIDFILE\" --config-file=\"$CLICKHOUSE_CONFIG\"" EXIT_STATUS=$? if [ $EXIT_STATUS -ne 0 ]; then break From 06187e9a9845e8efa2d302001bad9fd3abfd5bc4 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 21 Nov 2018 20:27:13 +0300 Subject: [PATCH 034/124] Comments [#CLICKHOUSE-2] --- libs/libdaemon/include/daemon/BaseDaemon.h | 29 ++++++++-------------- 1 file changed, 10 insertions(+), 19 deletions(-) diff --git a/libs/libdaemon/include/daemon/BaseDaemon.h b/libs/libdaemon/include/daemon/BaseDaemon.h index de0ce496850..815b03c3b27 100644 --- a/libs/libdaemon/include/daemon/BaseDaemon.h +++ b/libs/libdaemon/include/daemon/BaseDaemon.h @@ -27,28 +27,19 @@ namespace Poco { class TaskManager; } -/// \brief Базовый класс для демонов +/// \brief Base class for applications that can run as deamons. /// /// \code -/// # Список возможных опций командной строки обрабатываемых демоном: -/// # --config-file или --config - имя файла конфигурации. По умолчанию - config.xml -/// # --pid-file - имя PID файла. По умолчанию - pid -/// # --log-file - имя лог файла -/// # --errorlog-file - имя лог файла, в который будут помещаться только ошибки -/// # --daemon - запустить в режиме демона; если не указан - логгирование будет вестись на консоль -/// --daemon --config-file=localfile.xml --pid-file=pid.pid --log-file=log.log --errorlog-file=error.log +/// # Some possible command line options: +/// # --config-file, -C or --config - path to configuration file. By default - config.xml in the current directory. +/// # --log-file +/// # --errorlog-file +/// # --daemon - run as daemon; without this option, the program will be attached to the terminal and also output logs to stderr. +/// --daemon --config-file=localfile.xml --log-file=log.log --errorlog-file=error.log /// \endcode /// -/// Если неперехваченное исключение выкинуто в других потоках (не Task-и), то по-умолчанию -/// используется KillingErrorHandler, который вызывает std::terminate. -/// -/// Кроме того, класс позволяет достаточно гибко управлять журналированием. В методе initialize() вызывается метод -/// buildLoggers() который и строит нужные логгеры. Эта функция ожидает увидеть в конфигурации определённые теги -/// заключённые в секции "logger". -/// Если нужно журналирование на консоль, нужно просто не использовать тег "log" или использовать --console. -/// Теги уровней вывода использовать можно в любом случае - - +/// You can configure different log options for different loggers used inside program +/// by providing subsections to "logger" in configuration file. class BaseDaemon : public Poco::Util::ServerApplication { friend class SignalListener; @@ -182,7 +173,7 @@ protected: std::unique_ptr task_manager; - /// Создание и автоматическое удаление pid файла. + /// RAII wrapper for pid file. struct PID { std::string file; From 4d96e88058d9a08921df67d99d80303a47f045fc Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Wed, 21 Nov 2018 21:30:29 +0300 Subject: [PATCH 035/124] Fix distributed group by LowCardinality(Nullable). #3615 --- dbms/src/Columns/ColumnUnique.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/src/Columns/ColumnUnique.h b/dbms/src/Columns/ColumnUnique.h index be5d71c3e29..1f3366fceaa 100644 --- a/dbms/src/Columns/ColumnUnique.h +++ b/dbms/src/Columns/ColumnUnique.h @@ -305,7 +305,7 @@ StringRef ColumnUnique::serializeValueIntoArena(size_t n, Arena & ar size_t nested_size = 0; - if (n == getNullValueIndex()) + if (n != getNullValueIndex()) nested_size = column_holder->serializeValueIntoArena(n, arena, begin).size; return StringRef(pos, sizeof(null_flag) + nested_size); From eac03c1c830631d1eb1b00a4b65d108961b62e3c Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Wed, 21 Nov 2018 21:39:24 +0300 Subject: [PATCH 036/124] Added test. #3615 --- ..._low_cardinaliry_distributed_group_by.reference | 10 ++++++++++ .../00717_low_cardinaliry_distributed_group_by.sql | 12 ++++++++++++ dbms/tests/server-test.xml | 14 ++++++++++++++ 3 files changed, 36 insertions(+) create mode 100644 dbms/tests/queries/0_stateless/00717_low_cardinaliry_distributed_group_by.reference create mode 100644 dbms/tests/queries/0_stateless/00717_low_cardinaliry_distributed_group_by.sql diff --git a/dbms/tests/queries/0_stateless/00717_low_cardinaliry_distributed_group_by.reference b/dbms/tests/queries/0_stateless/00717_low_cardinaliry_distributed_group_by.reference new file mode 100644 index 00000000000..97833514d88 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00717_low_cardinaliry_distributed_group_by.reference @@ -0,0 +1,10 @@ +0 2 +1 2 +2 2 +3 2 +4 2 +5 2 +6 2 +7 2 +8 2 +9 2 diff --git a/dbms/tests/queries/0_stateless/00717_low_cardinaliry_distributed_group_by.sql b/dbms/tests/queries/0_stateless/00717_low_cardinaliry_distributed_group_by.sql new file mode 100644 index 00000000000..db3ea53490a --- /dev/null +++ b/dbms/tests/queries/0_stateless/00717_low_cardinaliry_distributed_group_by.sql @@ -0,0 +1,12 @@ +SET allow_experimental_low_cardinality_type = 1; +DROP TABLE IF EXISTS test.test_low_null_float; +DROP TABLE IF EXISTS test.dist; + +CREATE TABLE test.test_low_null_float (a LowCardinality(Nullable(Float64))) ENGINE = Memory; +CREATE TABLE test.dist (a LowCardinality(Nullable(Float64))) ENGINE = Distributed('test_cluster_two_shards_localhost', 'test', 'test_low_null_float', rand()); + +INSERT INTO test.dist (a) SELECT number FROM system.numbers LIMIT 1000000; +SELECT a, count() FROM test.dist GROUP BY a ORDER BY a ASC, count() ASC LIMIT 10; + +DROP TABLE IF EXISTS test.test_low_null_float; +DROP TABLE IF EXISTS test.dist; diff --git a/dbms/tests/server-test.xml b/dbms/tests/server-test.xml index cb9d63aa94c..82b76f62fa4 100644 --- a/dbms/tests/server-test.xml +++ b/dbms/tests/server-test.xml @@ -61,6 +61,20 @@ + + + + localhost + 59000 + + + + + localhost + 59000 + + + From 1d40c2c87173638a7203c24ede5cf1bef2301d3a Mon Sep 17 00:00:00 2001 From: proller Date: Wed, 21 Nov 2018 22:20:27 +0300 Subject: [PATCH 037/124] Better link and deps (#3631) * try fix 1 * Better deps * fix * fix * fix * Update CMakeLists.txt * Update CMakeLists.txt * fix * fix * fix * disable old * Update find_odbc.cmake --- cmake/find_odbc.cmake | 8 ++- cmake/find_poco.cmake | 4 +- cmake/lib_name.cmake | 1 - cmake/print_include_directories.cmake | 3 ++ contrib/cityhash102/CMakeLists.txt | 5 +- dbms/CMakeLists.txt | 51 +++++++++++------- dbms/programs/CMakeLists.txt | 26 ++++----- dbms/programs/benchmark/CMakeLists.txt | 4 +- dbms/programs/clang/CMakeLists.txt | 4 +- dbms/programs/client/CMakeLists.txt | 5 +- dbms/programs/compressor/CMakeLists.txt | 4 +- dbms/programs/copier/CMakeLists.txt | 2 +- .../extract-from-config/CMakeLists.txt | 4 +- dbms/programs/format/CMakeLists.txt | 4 +- dbms/programs/local/CMakeLists.txt | 4 +- dbms/programs/obfuscator/CMakeLists.txt | 4 +- dbms/programs/odbc-bridge/CMakeLists.txt | 12 ++--- .../programs/odbc-bridge/tests/CMakeLists.txt | 2 +- dbms/programs/performance-test/CMakeLists.txt | 4 +- dbms/programs/server/CMakeLists.txt | 8 ++- dbms/src/AggregateFunctions/CMakeLists.txt | 2 +- dbms/src/Common/Config/CMakeLists.txt | 5 +- dbms/src/Common/ZooKeeper/CMakeLists.txt | 3 +- .../src/Common/ZooKeeper/tests/CMakeLists.txt | 16 +++--- dbms/src/Common/tests/CMakeLists.txt | 48 ++++++++--------- dbms/src/Core/tests/CMakeLists.txt | 12 ++--- dbms/src/DataStreams/tests/CMakeLists.txt | 12 ++--- dbms/src/DataTypes/tests/CMakeLists.txt | 6 +-- dbms/src/Formats/tests/CMakeLists.txt | 4 +- dbms/src/Functions/tests/CMakeLists.txt | 2 +- dbms/src/IO/tests/CMakeLists.txt | 54 +++++++++---------- dbms/src/Interpreters/tests/CMakeLists.txt | 34 ++++++------ dbms/src/Parsers/CMakeLists.txt | 10 ++-- dbms/src/Parsers/tests/CMakeLists.txt | 15 +++--- dbms/src/Storages/System/CMakeLists.txt | 3 +- dbms/src/Storages/tests/CMakeLists.txt | 20 +++---- dbms/src/TableFunctions/CMakeLists.txt | 2 +- libs/libcommon/CMakeLists.txt | 12 +++-- libs/libcommon/cmake/find_jemalloc.cmake | 2 +- libs/libdaemon/CMakeLists.txt | 4 +- libs/libmysqlxx/CMakeLists.txt | 13 ++++- utils/check-marks/CMakeLists.txt | 2 +- utils/compressor/CMakeLists.txt | 8 +-- utils/config-processor/CMakeLists.txt | 2 +- utils/corrector_utf8/CMakeLists.txt | 5 +- utils/fill-factor/CMakeLists.txt | 2 +- utils/iotest/CMakeLists.txt | 6 +-- utils/test-data-generator/CMakeLists.txt | 4 +- utils/wikistat-loader/CMakeLists.txt | 2 +- utils/zookeeper-cli/CMakeLists.txt | 2 +- .../CMakeLists.txt | 2 +- utils/zookeeper-dump-tree/CMakeLists.txt | 2 +- utils/zookeeper-remove-by-list/CMakeLists.txt | 2 +- 53 files changed, 253 insertions(+), 224 deletions(-) diff --git a/cmake/find_odbc.cmake b/cmake/find_odbc.cmake index d89e3b532d8..32a410c6f11 100644 --- a/cmake/find_odbc.cmake +++ b/cmake/find_odbc.cmake @@ -71,10 +71,10 @@ if (ENABLE_ODBC) ) # MinGW find usually fails - if(MINGW) + if (MINGW) set(ODBC_INCLUDE_DIRECTORIES ".") set(ODBC_LIBRARIES odbc32) - endif() + endif () include(FindPackageHandleStandardArgs) find_package_handle_standard_args(ODBC @@ -82,6 +82,10 @@ if (ENABLE_ODBC) ODBC_INCLUDE_DIRECTORIES ODBC_LIBRARIES) + if (USE_STATIC_LIBRARIES) + list(APPEND ODBC_LIBRARIES ${LTDL_LIBRARY}) + endif () + mark_as_advanced(ODBC_FOUND ODBC_LIBRARIES ODBC_INCLUDE_DIRECTORIES) endif () endif () diff --git a/cmake/find_poco.cmake b/cmake/find_poco.cmake index d8468e5306d..012f269d48d 100644 --- a/cmake/find_poco.cmake +++ b/cmake/find_poco.cmake @@ -93,8 +93,8 @@ elseif (NOT MISSING_INTERNAL_POCO_LIBRARY) endif () if (OPENSSL_FOUND AND (NOT DEFINED ENABLE_POCO_NETSSL OR ENABLE_POCO_NETSSL)) - set (Poco_NetSSL_LIBRARY PocoNetSSL) - set (Poco_Crypto_LIBRARY PocoCrypto) + set (Poco_NetSSL_LIBRARY PocoNetSSL ${OPENSSL_LIBRARIES}) + set (Poco_Crypto_LIBRARY PocoCrypto ${OPENSSL_LIBRARIES}) endif () if (USE_STATIC_LIBRARIES AND USE_INTERNAL_ZLIB_LIBRARY) diff --git a/cmake/lib_name.cmake b/cmake/lib_name.cmake index b49276fc279..5c919b263e6 100644 --- a/cmake/lib_name.cmake +++ b/cmake/lib_name.cmake @@ -1,5 +1,4 @@ set(DIVIDE_INCLUDE_DIR ${ClickHouse_SOURCE_DIR}/contrib/libdivide) -set(CITYHASH_CONTRIB_INCLUDE_DIR ${ClickHouse_SOURCE_DIR}/contrib/cityhash102/include) set(COMMON_INCLUDE_DIR ${ClickHouse_SOURCE_DIR}/libs/libcommon/include ${ClickHouse_BINARY_DIR}/libs/libcommon/include) set(DBMS_INCLUDE_DIR ${ClickHouse_SOURCE_DIR}/dbms/src ${ClickHouse_BINARY_DIR}/dbms/src) set(DOUBLE_CONVERSION_CONTRIB_INCLUDE_DIR ${ClickHouse_SOURCE_DIR}/contrib/double-conversion) diff --git a/cmake/print_include_directories.cmake b/cmake/print_include_directories.cmake index 41c4773cfa3..c4c5d00c54f 100644 --- a/cmake/print_include_directories.cmake +++ b/cmake/print_include_directories.cmake @@ -10,6 +10,9 @@ list(APPEND dirs ${dirs1}) get_property (dirs1 TARGET common PROPERTY INCLUDE_DIRECTORIES) list(APPEND dirs ${dirs1}) +get_property (dirs1 TARGET cityhash PROPERTY INCLUDE_DIRECTORIES) +list(APPEND dirs ${dirs1}) + if (USE_INTERNAL_BOOST_LIBRARY) get_property (dirs1 TARGET ${Boost_PROGRAM_OPTIONS_LIBRARY} PROPERTY INCLUDE_DIRECTORIES) list(APPEND dirs ${dirs1}) diff --git a/contrib/cityhash102/CMakeLists.txt b/contrib/cityhash102/CMakeLists.txt index eafa6f2025f..c3f53a8f878 100644 --- a/contrib/cityhash102/CMakeLists.txt +++ b/contrib/cityhash102/CMakeLists.txt @@ -1,9 +1,8 @@ add_library(cityhash src/city.cc - include/citycrc.h include/city.h src/config.h) -target_include_directories (cityhash BEFORE PUBLIC include) -target_include_directories (cityhash PRIVATE src) +target_include_directories(cityhash BEFORE PUBLIC include) +target_include_directories(cityhash PRIVATE src) diff --git a/dbms/CMakeLists.txt b/dbms/CMakeLists.txt index 3edcbbee5f3..eebd3504192 100644 --- a/dbms/CMakeLists.txt +++ b/dbms/CMakeLists.txt @@ -119,7 +119,7 @@ endif () if (USE_EMBEDDED_COMPILER) llvm_libs_all(REQUIRED_LLVM_LIBRARIES) - target_link_libraries (dbms ${REQUIRED_LLVM_LIBRARIES}) + target_link_libraries (dbms PRIVATE ${REQUIRED_LLVM_LIBRARIES}) target_include_directories (dbms SYSTEM BEFORE PUBLIC ${LLVM_INCLUDE_DIRS}) endif () @@ -150,33 +150,48 @@ if (NOT ARCH_ARM AND CPUID_LIBRARY) endif() target_link_libraries (clickhouse_common_io + PUBLIC common + PRIVATE string_utils widechar_width ${LINK_LIBRARIES_ONLY_ON_X86_64} ${LZ4_LIBRARY} ${ZSTD_LIBRARY} ${DOUBLE_CONVERSION_LIBRARIES} + pocoext + PUBLIC ${Poco_Net_LIBRARY} ${Poco_Util_LIBRARY} ${Poco_Foundation_LIBRARY} + ${RE2_LIBRARY} + ${RE2_ST_LIBRARY} + ${CITYHASH_LIBRARIES} + PRIVATE ${ZLIB_LIBRARIES} ${EXECINFO_LIBRARY} ${ELF_LIBRARY} + PUBLIC ${Boost_SYSTEM_LIBRARY} + PRIVATE apple_rt ${CMAKE_DL_LIBS} ) target_link_libraries (dbms + PRIVATE clickhouse_parsers clickhouse_common_config + PUBLIC clickhouse_common_io + pocoext + PUBLIC ${MYSQLXX_LIBRARY} - ${RE2_LIBRARY} - ${RE2_ST_LIBRARY} + PRIVATE ${BTRIE_LIBRARIES} ${Boost_PROGRAM_OPTIONS_LIBRARY} + PUBLIC + ${Boost_SYSTEM_LIBRARY} ) if (NOT USE_INTERNAL_RE2_LIBRARY) @@ -193,8 +208,8 @@ if (Poco_SQL_FOUND AND NOT USE_INTERNAL_POCO_LIBRARY) endif() if (USE_POCO_SQLODBC) - target_link_libraries (clickhouse_common_io ${Poco_SQL_LIBRARY}) - target_link_libraries (dbms ${Poco_SQLODBC_LIBRARY} ${Poco_SQL_LIBRARY}) + target_link_libraries (clickhouse_common_io PRIVATE ${Poco_SQL_LIBRARY}) + target_link_libraries (dbms PRIVATE ${Poco_SQLODBC_LIBRARY} ${Poco_SQL_LIBRARY}) if (NOT USE_INTERNAL_POCO_LIBRARY) target_include_directories (clickhouse_common_io SYSTEM PRIVATE ${ODBC_INCLUDE_DIRECTORIES} ${Poco_SQL_INCLUDE_DIR}) target_include_directories (dbms SYSTEM PRIVATE ${ODBC_INCLUDE_DIRECTORIES} ${Poco_SQLODBC_INCLUDE_DIR} PUBLIC ${Poco_SQL_INCLUDE_DIR}) @@ -208,48 +223,44 @@ if (Poco_Data_FOUND) endif() if (USE_POCO_DATAODBC) - target_link_libraries (clickhouse_common_io ${Poco_Data_LIBRARY}) - target_link_libraries (dbms ${Poco_DataODBC_LIBRARY}) + target_link_libraries (clickhouse_common_io PRIVATE ${Poco_Data_LIBRARY}) + target_link_libraries (dbms PRIVATE ${Poco_DataODBC_LIBRARY}) if (NOT USE_INTERNAL_POCO_LIBRARY) target_include_directories (dbms SYSTEM PRIVATE ${ODBC_INCLUDE_DIRECTORIES} ${Poco_DataODBC_INCLUDE_DIR}) endif() endif() if (USE_POCO_MONGODB) - target_link_libraries (dbms ${Poco_MongoDB_LIBRARY}) + target_link_libraries (dbms PRIVATE ${Poco_MongoDB_LIBRARY}) endif() if (USE_POCO_NETSSL) - target_link_libraries (clickhouse_common_io ${Poco_NetSSL_LIBRARY} ${Poco_Crypto_LIBRARY}) - target_link_libraries (dbms ${Poco_NetSSL_LIBRARY} ${Poco_Crypto_LIBRARY}) + target_link_libraries (clickhouse_common_io PRIVATE ${Poco_NetSSL_LIBRARY} ${Poco_Crypto_LIBRARY}) + target_link_libraries (dbms PRIVATE ${Poco_NetSSL_LIBRARY} ${Poco_Crypto_LIBRARY}) endif() -target_link_libraries (dbms ${Poco_Foundation_LIBRARY}) +target_link_libraries (dbms PRIVATE ${Poco_Foundation_LIBRARY}) if (USE_ICU) - target_link_libraries (dbms ${ICU_LIBS}) + target_link_libraries (dbms PRIVATE ${ICU_LIBS}) target_include_directories (dbms SYSTEM PRIVATE ${ICU_INCLUDE_DIR}) endif () if (USE_CAPNP) - target_link_libraries (dbms ${CAPNP_LIBRARY}) + target_link_libraries (dbms PRIVATE ${CAPNP_LIBRARY}) if (NOT USE_INTERNAL_CAPNP_LIBRARY) target_include_directories (dbms SYSTEM BEFORE PRIVATE ${CAPNP_INCLUDE_DIR}) endif () endif () if (USE_RDKAFKA) - target_link_libraries (dbms ${RDKAFKA_LIBRARY}) + target_link_libraries (dbms PRIVATE ${RDKAFKA_LIBRARY}) if (NOT USE_INTERNAL_RDKAFKA_LIBRARY) target_include_directories (dbms SYSTEM BEFORE PRIVATE ${RDKAFKA_INCLUDE_DIR}) endif () endif () -target_link_libraries(dbms ${OPENSSL_CRYPTO_LIBRARY}) - -target_link_libraries (dbms - Threads::Threads -) +target_link_libraries(dbms PRIVATE ${OPENSSL_CRYPTO_LIBRARY} Threads::Threads) target_include_directories (dbms SYSTEM BEFORE PRIVATE ${DIVIDE_INCLUDE_DIR}) target_include_directories (dbms SYSTEM BEFORE PRIVATE ${SPARCEHASH_INCLUDE_DIR}) @@ -285,6 +296,6 @@ if (ENABLE_TESTS) # attach all dbms gtest sources grep_gtest_sources(${ClickHouse_SOURCE_DIR}/dbms dbms_gtest_sources) add_executable(unit_tests_dbms ${dbms_gtest_sources}) - target_link_libraries(unit_tests_dbms gtest_main dbms) + target_link_libraries(unit_tests_dbms PRIVATE gtest_main dbms clickhouse_common_zookeeper) add_check(unit_tests_dbms) endif () diff --git a/dbms/programs/CMakeLists.txt b/dbms/programs/CMakeLists.txt index 136616ca44b..441b39d9966 100644 --- a/dbms/programs/CMakeLists.txt +++ b/dbms/programs/CMakeLists.txt @@ -48,45 +48,45 @@ else () link_directories (${LLVM_LIBRARY_DIRS}) endif () add_executable (clickhouse main.cpp) - target_link_libraries (clickhouse clickhouse_common_io) + target_link_libraries (clickhouse PRIVATE clickhouse_common_io) target_include_directories (clickhouse BEFORE PRIVATE ${COMMON_INCLUDE_DIR}) target_include_directories (clickhouse PRIVATE ${CMAKE_CURRENT_BINARY_DIR}) if (ENABLE_CLICKHOUSE_SERVER) - target_link_libraries (clickhouse clickhouse-server-lib) + target_link_libraries (clickhouse PRIVATE clickhouse-server-lib) endif () if (ENABLE_CLICKHOUSE_CLIENT) - target_link_libraries (clickhouse clickhouse-client-lib) + target_link_libraries (clickhouse PRIVATE clickhouse-client-lib) endif () if (ENABLE_CLICKHOUSE_LOCAL) - target_link_libraries (clickhouse clickhouse-local-lib) + target_link_libraries (clickhouse PRIVATE clickhouse-local-lib) endif () if (ENABLE_CLICKHOUSE_BENCHMARK) - target_link_libraries (clickhouse clickhouse-benchmark-lib) + target_link_libraries (clickhouse PRIVATE clickhouse-benchmark-lib) endif () if (ENABLE_CLICKHOUSE_PERFORMANCE) - target_link_libraries (clickhouse clickhouse-performance-test-lib) + target_link_libraries (clickhouse PRIVATE clickhouse-performance-test-lib) endif () if (ENABLE_CLICKHOUSE_COPIER) - target_link_libraries (clickhouse clickhouse-copier-lib) + target_link_libraries (clickhouse PRIVATE clickhouse-copier-lib) endif () if (ENABLE_CLICKHOUSE_EXTRACT_FROM_CONFIG) - target_link_libraries (clickhouse clickhouse-extract-from-config-lib) + target_link_libraries (clickhouse PRIVATE clickhouse-extract-from-config-lib) endif () if (ENABLE_CLICKHOUSE_COMPRESSOR) - target_link_libraries (clickhouse clickhouse-compressor-lib) + target_link_libraries (clickhouse PRIVATE clickhouse-compressor-lib) endif () if (ENABLE_CLICKHOUSE_FORMAT) - target_link_libraries (clickhouse clickhouse-format-lib) + target_link_libraries (clickhouse PRIVATE clickhouse-format-lib) endif () if (ENABLE_CLICKHOUSE_OBFUSCATOR) - target_link_libraries (clickhouse clickhouse-obfuscator-lib) + target_link_libraries (clickhouse PRIVATE clickhouse-obfuscator-lib) endif () if (USE_EMBEDDED_COMPILER) - target_link_libraries (clickhouse clickhouse-compiler-lib) + target_link_libraries (clickhouse PRIVATE clickhouse-compiler-lib) endif () if (ENABLE_CLICKHOUSE_ODBC_BRIDGE) - target_link_libraries (clickhouse clickhouse-odbc-bridge-lib) + target_link_libraries (clickhouse PRIVATE clickhouse-odbc-bridge-lib) endif() set (CLICKHOUSE_BUNDLE) diff --git a/dbms/programs/benchmark/CMakeLists.txt b/dbms/programs/benchmark/CMakeLists.txt index 85af0750715..c41c46edeb8 100644 --- a/dbms/programs/benchmark/CMakeLists.txt +++ b/dbms/programs/benchmark/CMakeLists.txt @@ -1,8 +1,8 @@ add_library (clickhouse-benchmark-lib ${LINK_MODE} Benchmark.cpp) -target_link_libraries (clickhouse-benchmark-lib clickhouse-client-lib clickhouse_common_io ${Boost_PROGRAM_OPTIONS_LIBRARY}) +target_link_libraries (clickhouse-benchmark-lib PRIVATE clickhouse-client-lib clickhouse_common_io ${Boost_PROGRAM_OPTIONS_LIBRARY}) target_include_directories (clickhouse-benchmark-lib SYSTEM PRIVATE ${PCG_RANDOM_INCLUDE_DIR}) if (CLICKHOUSE_SPLIT_BINARY) add_executable (clickhouse-benchmark clickhouse-benchmark.cpp) - target_link_libraries (clickhouse-benchmark clickhouse-benchmark-lib clickhouse_aggregate_functions) + target_link_libraries (clickhouse-benchmark PRIVATE clickhouse-benchmark-lib clickhouse_aggregate_functions) endif () diff --git a/dbms/programs/clang/CMakeLists.txt b/dbms/programs/clang/CMakeLists.txt index ca06f27546d..dec21ac611e 100644 --- a/dbms/programs/clang/CMakeLists.txt +++ b/dbms/programs/clang/CMakeLists.txt @@ -6,9 +6,9 @@ if (CLICKHOUSE_SPLIT_BINARY) if (USE_EMBEDDED_COMPILER) link_directories (${LLVM_LIBRARY_DIRS}) add_executable (clickhouse-clang clickhouse-clang.cpp) - target_link_libraries (clickhouse-clang clickhouse-compiler-lib) + target_link_libraries (clickhouse-clang PRIVATE clickhouse-compiler-lib) add_executable (clickhouse-lld clickhouse-lld.cpp) - target_link_libraries (clickhouse-lld clickhouse-compiler-lib) + target_link_libraries (clickhouse-lld PRIVATE clickhouse-compiler-lib) install (TARGETS clickhouse-clang clickhouse-lld RUNTIME DESTINATION ${CMAKE_INSTALL_BINDIR} COMPONENT clickhouse) endif () endif () diff --git a/dbms/programs/client/CMakeLists.txt b/dbms/programs/client/CMakeLists.txt index 65353094c26..f3dd518e891 100644 --- a/dbms/programs/client/CMakeLists.txt +++ b/dbms/programs/client/CMakeLists.txt @@ -1,13 +1,12 @@ add_library (clickhouse-client-lib ${LINK_MODE} Client.cpp) -target_link_libraries (clickhouse-client-lib clickhouse_common_io clickhouse_functions clickhouse_aggregate_functions ${LINE_EDITING_LIBS} ${Boost_PROGRAM_OPTIONS_LIBRARY}) +target_link_libraries (clickhouse-client-lib PRIVATE clickhouse_common_io clickhouse_functions clickhouse_aggregate_functions ${LINE_EDITING_LIBS} ${Boost_PROGRAM_OPTIONS_LIBRARY}) if (READLINE_INCLUDE_DIR) target_include_directories (clickhouse-client-lib SYSTEM PRIVATE ${READLINE_INCLUDE_DIR}) endif () if (CLICKHOUSE_SPLIT_BINARY) add_executable (clickhouse-client clickhouse-client.cpp) - target_link_libraries (clickhouse-client clickhouse-client-lib) + target_link_libraries (clickhouse-client PRIVATE clickhouse-client-lib) endif () install (FILES clickhouse-client.xml DESTINATION ${CLICKHOUSE_ETC_DIR}/clickhouse-client COMPONENT clickhouse-client RENAME config.xml) - diff --git a/dbms/programs/compressor/CMakeLists.txt b/dbms/programs/compressor/CMakeLists.txt index 5c9c11072c9..7aa2cad5708 100644 --- a/dbms/programs/compressor/CMakeLists.txt +++ b/dbms/programs/compressor/CMakeLists.txt @@ -1,8 +1,8 @@ add_library (clickhouse-compressor-lib ${LINK_MODE} Compressor.cpp) -target_link_libraries (clickhouse-compressor-lib clickhouse_common_io ${Boost_PROGRAM_OPTIONS_LIBRARY}) +target_link_libraries (clickhouse-compressor-lib PRIVATE clickhouse_common_io ${Boost_PROGRAM_OPTIONS_LIBRARY}) if (CLICKHOUSE_SPLIT_BINARY) # Also in utils add_executable (clickhouse-compressor clickhouse-compressor.cpp) - target_link_libraries (clickhouse-compressor clickhouse-compressor-lib) + target_link_libraries (clickhouse-compressor PRIVATE clickhouse-compressor-lib) endif () diff --git a/dbms/programs/copier/CMakeLists.txt b/dbms/programs/copier/CMakeLists.txt index e8583dba440..ed3e55208aa 100644 --- a/dbms/programs/copier/CMakeLists.txt +++ b/dbms/programs/copier/CMakeLists.txt @@ -1,5 +1,5 @@ add_library (clickhouse-copier-lib ${LINK_MODE} ClusterCopier.cpp) -target_link_libraries (clickhouse-copier-lib clickhouse-server-lib clickhouse_functions clickhouse_aggregate_functions) +target_link_libraries (clickhouse-copier-lib PRIVATE clickhouse-server-lib clickhouse_functions clickhouse_aggregate_functions daemon) if (CLICKHOUSE_SPLIT_BINARY) add_executable (clickhouse-copier clickhouse-copier.cpp) diff --git a/dbms/programs/extract-from-config/CMakeLists.txt b/dbms/programs/extract-from-config/CMakeLists.txt index c31b0e8cec9..62253649368 100644 --- a/dbms/programs/extract-from-config/CMakeLists.txt +++ b/dbms/programs/extract-from-config/CMakeLists.txt @@ -1,7 +1,7 @@ add_library (clickhouse-extract-from-config-lib ${LINK_MODE} ExtractFromConfig.cpp) -target_link_libraries (clickhouse-extract-from-config-lib clickhouse_common_config clickhouse_common_io ${Boost_PROGRAM_OPTIONS_LIBRARY}) +target_link_libraries (clickhouse-extract-from-config-lib PRIVATE clickhouse_common_config clickhouse_common_io ${Boost_PROGRAM_OPTIONS_LIBRARY}) if (CLICKHOUSE_SPLIT_BINARY) add_executable (clickhouse-extract-from-config clickhouse-extract-from-config.cpp) - target_link_libraries (clickhouse-extract-from-config clickhouse-extract-from-config-lib) + target_link_libraries (clickhouse-extract-from-config PRIVATE clickhouse-extract-from-config-lib) endif () diff --git a/dbms/programs/format/CMakeLists.txt b/dbms/programs/format/CMakeLists.txt index f53cd11bee7..53d09e82621 100644 --- a/dbms/programs/format/CMakeLists.txt +++ b/dbms/programs/format/CMakeLists.txt @@ -1,6 +1,6 @@ add_library (clickhouse-format-lib ${LINK_MODE} Format.cpp) -target_link_libraries (clickhouse-format-lib dbms clickhouse_common_io ${Boost_PROGRAM_OPTIONS_LIBRARY}) +target_link_libraries (clickhouse-format-lib PRIVATE dbms clickhouse_common_io clickhouse_parsers ${Boost_PROGRAM_OPTIONS_LIBRARY}) if (CLICKHOUSE_SPLIT_BINARY) add_executable (clickhouse-format clickhouse-format.cpp) - target_link_libraries (clickhouse-format clickhouse-format-lib) + target_link_libraries (clickhouse-format PRIVATE clickhouse-format-lib) endif () diff --git a/dbms/programs/local/CMakeLists.txt b/dbms/programs/local/CMakeLists.txt index 9680ceaf1c7..07729d68563 100644 --- a/dbms/programs/local/CMakeLists.txt +++ b/dbms/programs/local/CMakeLists.txt @@ -1,7 +1,7 @@ add_library (clickhouse-local-lib ${LINK_MODE} LocalServer.cpp) -target_link_libraries (clickhouse-local-lib clickhouse_common_io clickhouse-server-lib clickhouse_functions clickhouse_aggregate_functions clickhouse_table_functions ${Boost_PROGRAM_OPTIONS_LIBRARY}) +target_link_libraries (clickhouse-local-lib PRIVATE clickhouse_common_io clickhouse-server-lib clickhouse_functions clickhouse_aggregate_functions clickhouse_table_functions ${Boost_PROGRAM_OPTIONS_LIBRARY}) if (CLICKHOUSE_SPLIT_BINARY) add_executable (clickhouse-local clickhouse-local.cpp) - target_link_libraries (clickhouse-local clickhouse-local-lib) + target_link_libraries (clickhouse-local PRIVATE clickhouse-local-lib) endif () diff --git a/dbms/programs/obfuscator/CMakeLists.txt b/dbms/programs/obfuscator/CMakeLists.txt index 5ee6ace5a02..73c3f01e9cb 100644 --- a/dbms/programs/obfuscator/CMakeLists.txt +++ b/dbms/programs/obfuscator/CMakeLists.txt @@ -1,8 +1,8 @@ add_library (clickhouse-obfuscator-lib ${LINK_MODE} Obfuscator.cpp) -target_link_libraries (clickhouse-obfuscator-lib dbms ${Boost_PROGRAM_OPTIONS_LIBRARY}) +target_link_libraries (clickhouse-obfuscator-lib PRIVATE dbms ${Boost_PROGRAM_OPTIONS_LIBRARY}) if (CLICKHOUSE_SPLIT_BINARY) add_executable (clickhouse-obfuscator clickhouse-obfuscator.cpp) set_target_properties(clickhouse-obfuscator PROPERTIES RUNTIME_OUTPUT_DIRECTORY ..) - target_link_libraries (clickhouse-obfuscator clickhouse-obfuscator-lib) + target_link_libraries (clickhouse-obfuscator PRIVATE clickhouse-obfuscator-lib) endif () diff --git a/dbms/programs/odbc-bridge/CMakeLists.txt b/dbms/programs/odbc-bridge/CMakeLists.txt index d13a2866e77..a57c8c9c8cf 100644 --- a/dbms/programs/odbc-bridge/CMakeLists.txt +++ b/dbms/programs/odbc-bridge/CMakeLists.txt @@ -9,23 +9,23 @@ add_library (clickhouse-odbc-bridge-lib ${LINK_MODE} validateODBCConnectionString.cpp ) -target_link_libraries (clickhouse-odbc-bridge-lib clickhouse_common_io daemon dbms) +target_link_libraries (clickhouse-odbc-bridge-lib PRIVATE clickhouse_common_io daemon dbms) target_include_directories (clickhouse-odbc-bridge-lib PUBLIC ${ClickHouse_SOURCE_DIR}/libs/libdaemon/include) if (USE_POCO_SQLODBC) - target_link_libraries (clickhouse-odbc-bridge-lib ${Poco_SQLODBC_LIBRARY}) + target_link_libraries (clickhouse-odbc-bridge-lib PRIVATE ${Poco_SQLODBC_LIBRARY}) target_include_directories (clickhouse-odbc-bridge-lib SYSTEM PRIVATE ${ODBC_INCLUDE_DIRECTORIES} ${Poco_SQLODBC_INCLUDE_DIR}) endif () if (Poco_SQL_FOUND) - target_link_libraries (clickhouse-odbc-bridge-lib ${Poco_SQL_LIBRARY}) + target_link_libraries (clickhouse-odbc-bridge-lib PRIVATE ${Poco_SQL_LIBRARY}) endif () if (USE_POCO_DATAODBC) - target_link_libraries (clickhouse-odbc-bridge-lib ${Poco_DataODBC_LIBRARY}) + target_link_libraries (clickhouse-odbc-bridge-lib PRIVATE ${Poco_DataODBC_LIBRARY}) target_include_directories (clickhouse-odbc-bridge-lib SYSTEM PRIVATE ${ODBC_INCLUDE_DIRECTORIES} ${Poco_DataODBC_INCLUDE_DIR}) endif() if (Poco_Data_FOUND) - target_link_libraries (clickhouse-odbc-bridge-lib ${Poco_Data_LIBRARY}) + target_link_libraries (clickhouse-odbc-bridge-lib PRIVATE ${Poco_Data_LIBRARY}) endif () @@ -35,5 +35,5 @@ endif () if (CLICKHOUSE_SPLIT_BINARY) add_executable (clickhouse-odbc-bridge odbc-bridge.cpp) - target_link_libraries (clickhouse-odbc-bridge clickhouse-odbc-bridge-lib) + target_link_libraries (clickhouse-odbc-bridge PRIVATE clickhouse-odbc-bridge-lib) endif () diff --git a/dbms/programs/odbc-bridge/tests/CMakeLists.txt b/dbms/programs/odbc-bridge/tests/CMakeLists.txt index 5240a917429..5211c39d111 100644 --- a/dbms/programs/odbc-bridge/tests/CMakeLists.txt +++ b/dbms/programs/odbc-bridge/tests/CMakeLists.txt @@ -1,2 +1,2 @@ add_executable (validate-odbc-connection-string validate-odbc-connection-string.cpp) -target_link_libraries (validate-odbc-connection-string clickhouse-odbc-bridge-lib) +target_link_libraries (validate-odbc-connection-string PRIVATE clickhouse-odbc-bridge-lib clickhouse_common_io) diff --git a/dbms/programs/performance-test/CMakeLists.txt b/dbms/programs/performance-test/CMakeLists.txt index 31796cd9d74..adad45025c3 100644 --- a/dbms/programs/performance-test/CMakeLists.txt +++ b/dbms/programs/performance-test/CMakeLists.txt @@ -1,8 +1,8 @@ add_library (clickhouse-performance-test-lib ${LINK_MODE} PerformanceTest.cpp) -target_link_libraries (clickhouse-performance-test-lib clickhouse_common_io dbms ${Boost_PROGRAM_OPTIONS_LIBRARY}) +target_link_libraries (clickhouse-performance-test-lib PRIVATE dbms clickhouse_common_io ${Boost_PROGRAM_OPTIONS_LIBRARY}) target_include_directories (clickhouse-performance-test-lib SYSTEM PRIVATE ${PCG_RANDOM_INCLUDE_DIR}) if (CLICKHOUSE_SPLIT_BINARY) add_executable (clickhouse-performance-test clickhouse-performance-test.cpp) - target_link_libraries (clickhouse-performance-test clickhouse-performance-test-lib dbms) + target_link_libraries (clickhouse-performance-test PRIVATE clickhouse-performance-test-lib) endif () diff --git a/dbms/programs/server/CMakeLists.txt b/dbms/programs/server/CMakeLists.txt index bc6683d6e92..d8caa07b743 100644 --- a/dbms/programs/server/CMakeLists.txt +++ b/dbms/programs/server/CMakeLists.txt @@ -10,12 +10,16 @@ add_library (clickhouse-server-lib ${LINK_MODE} TCPHandler.cpp ) -target_link_libraries (clickhouse-server-lib clickhouse_common_io daemon clickhouse_storages_system clickhouse_functions clickhouse_aggregate_functions clickhouse_table_functions) +target_link_libraries (clickhouse-server-lib PRIVATE clickhouse_common_io daemon clickhouse_storages_system clickhouse_functions clickhouse_aggregate_functions clickhouse_table_functions ${Poco_Net_LIBRARY}) +if (USE_POCO_NETSSL) + target_link_libraries (clickhouse-server-lib PRIVATE ${Poco_NetSSL_LIBRARY} ${Poco_Crypto_LIBRARY}) +endif () + target_include_directories (clickhouse-server-lib PUBLIC ${ClickHouse_SOURCE_DIR}/libs/libdaemon/include) if (CLICKHOUSE_SPLIT_BINARY) add_executable (clickhouse-server clickhouse-server.cpp) - target_link_libraries (clickhouse-server clickhouse-server-lib) + target_link_libraries (clickhouse-server PRIVATE clickhouse-server-lib) install (TARGETS clickhouse-server ${CLICKHOUSE_ALL_TARGETS} RUNTIME DESTINATION ${CMAKE_INSTALL_BINDIR} COMPONENT clickhouse) endif () diff --git a/dbms/src/AggregateFunctions/CMakeLists.txt b/dbms/src/AggregateFunctions/CMakeLists.txt index ef2665d0d3f..56cc66d3913 100644 --- a/dbms/src/AggregateFunctions/CMakeLists.txt +++ b/dbms/src/AggregateFunctions/CMakeLists.txt @@ -20,5 +20,5 @@ list(REMOVE_ITEM clickhouse_aggregate_functions_headers ) add_library(clickhouse_aggregate_functions ${LINK_MODE} ${clickhouse_aggregate_functions_sources}) -target_link_libraries(clickhouse_aggregate_functions dbms) +target_link_libraries(clickhouse_aggregate_functions PRIVATE dbms) target_include_directories (clickhouse_aggregate_functions BEFORE PRIVATE ${COMMON_INCLUDE_DIR}) diff --git a/dbms/src/Common/Config/CMakeLists.txt b/dbms/src/Common/Config/CMakeLists.txt index 614e70e757b..a1bb2790fdf 100644 --- a/dbms/src/Common/Config/CMakeLists.txt +++ b/dbms/src/Common/Config/CMakeLists.txt @@ -1,9 +1,8 @@ - include(${ClickHouse_SOURCE_DIR}/cmake/dbms_glob_sources.cmake) add_headers_and_sources(clickhouse_common_config .) add_library(clickhouse_common_config ${LINK_MODE} ${clickhouse_common_config_headers} ${clickhouse_common_config_sources}) -target_link_libraries (clickhouse_common_config clickhouse_common_zookeeper string_utils ${Poco_XML_LIBRARY} ${Poco_Util_LIBRARY}) -target_include_directories (clickhouse_common_config PRIVATE ${DBMS_INCLUDE_DIR}) +target_link_libraries(clickhouse_common_config PUBLIC common PRIVATE clickhouse_common_zookeeper string_utils PUBLIC ${Poco_XML_LIBRARY} ${Poco_Util_LIBRARY}) +target_include_directories(clickhouse_common_config PUBLIC ${DBMS_INCLUDE_DIR}) diff --git a/dbms/src/Common/ZooKeeper/CMakeLists.txt b/dbms/src/Common/ZooKeeper/CMakeLists.txt index 2969d9a91e9..1f69f0af1ec 100644 --- a/dbms/src/Common/ZooKeeper/CMakeLists.txt +++ b/dbms/src/Common/ZooKeeper/CMakeLists.txt @@ -4,7 +4,8 @@ add_headers_and_sources(clickhouse_common_zookeeper .) add_library(clickhouse_common_zookeeper ${LINK_MODE} ${clickhouse_common_zookeeper_headers} ${clickhouse_common_zookeeper_sources}) -target_link_libraries (clickhouse_common_zookeeper clickhouse_common_io) +target_link_libraries (clickhouse_common_zookeeper PUBLIC clickhouse_common_io common PRIVATE string_utils PUBLIC ${Poco_Util_LIBRARY}) +target_include_directories(clickhouse_common_zookeeper PUBLIC ${DBMS_INCLUDE_DIR}) if (ENABLE_TESTS) add_subdirectory (tests) diff --git a/dbms/src/Common/ZooKeeper/tests/CMakeLists.txt b/dbms/src/Common/ZooKeeper/tests/CMakeLists.txt index a24948240f6..06716e49918 100644 --- a/dbms/src/Common/ZooKeeper/tests/CMakeLists.txt +++ b/dbms/src/Common/ZooKeeper/tests/CMakeLists.txt @@ -1,23 +1,23 @@ add_executable(zkutil_test_commands zkutil_test_commands.cpp) -target_link_libraries(zkutil_test_commands clickhouse_common_zookeeper) +target_link_libraries(zkutil_test_commands PRIVATE clickhouse_common_zookeeper) add_executable(zkutil_test_commands_new_lib zkutil_test_commands_new_lib.cpp) -target_link_libraries(zkutil_test_commands_new_lib clickhouse_common_zookeeper) +target_link_libraries(zkutil_test_commands_new_lib PRIVATE clickhouse_common_zookeeper) add_executable(zkutil_test_lock zkutil_test_lock.cpp) -target_link_libraries(zkutil_test_lock clickhouse_common_zookeeper) +target_link_libraries(zkutil_test_lock PRIVATE clickhouse_common_zookeeper) add_executable(zkutil_expiration_test zkutil_expiration_test.cpp) -target_link_libraries(zkutil_expiration_test clickhouse_common_zookeeper) +target_link_libraries(zkutil_expiration_test PRIVATE clickhouse_common_zookeeper) add_executable(zkutil_test_async zkutil_test_async.cpp) -target_link_libraries(zkutil_test_async clickhouse_common_zookeeper) +target_link_libraries(zkutil_test_async PRIVATE clickhouse_common_zookeeper) add_executable(zkutil_zookeeper_holder zkutil_zookeeper_holder.cpp) -target_link_libraries(zkutil_zookeeper_holder clickhouse_common_zookeeper) +target_link_libraries(zkutil_zookeeper_holder PRIVATE clickhouse_common_zookeeper) add_executable (zk_many_watches_reconnect zk_many_watches_reconnect.cpp) -target_link_libraries (zk_many_watches_reconnect clickhouse_common_zookeeper clickhouse_common_config) +target_link_libraries (zk_many_watches_reconnect PRIVATE clickhouse_common_zookeeper clickhouse_common_config) add_executable (zookeeper_impl zookeeper_impl.cpp) -target_link_libraries (zookeeper_impl clickhouse_common_zookeeper) +target_link_libraries (zookeeper_impl PRIVATE clickhouse_common_zookeeper) diff --git a/dbms/src/Common/tests/CMakeLists.txt b/dbms/src/Common/tests/CMakeLists.txt index f4d01e85bd2..802963bf404 100644 --- a/dbms/src/Common/tests/CMakeLists.txt +++ b/dbms/src/Common/tests/CMakeLists.txt @@ -1,73 +1,73 @@ add_executable (hashes_test hashes_test.cpp) -target_link_libraries (hashes_test dbms) +target_link_libraries (hashes_test PRIVATE dbms) add_executable (sip_hash sip_hash.cpp) -target_link_libraries (sip_hash clickhouse_common_io) +target_link_libraries (sip_hash PRIVATE clickhouse_common_io) add_executable (sip_hash_perf sip_hash_perf.cpp) -target_link_libraries (sip_hash_perf clickhouse_common_io) +target_link_libraries (sip_hash_perf PRIVATE clickhouse_common_io) add_executable (auto_array auto_array.cpp) -target_link_libraries (auto_array clickhouse_common_io) +target_link_libraries (auto_array PRIVATE clickhouse_common_io) add_executable (lru_cache lru_cache.cpp) -target_link_libraries (lru_cache clickhouse_common_io) +target_link_libraries (lru_cache PRIVATE clickhouse_common_io) add_executable (hash_table hash_table.cpp) -target_link_libraries (hash_table clickhouse_common_io) +target_link_libraries (hash_table PRIVATE clickhouse_common_io) add_executable (small_table small_table.cpp) -target_link_libraries (small_table clickhouse_common_io) +target_link_libraries (small_table PRIVATE clickhouse_common_io) add_executable (parallel_aggregation parallel_aggregation.cpp) -target_link_libraries (parallel_aggregation clickhouse_common_io) +target_link_libraries (parallel_aggregation PRIVATE clickhouse_common_io) add_executable (parallel_aggregation2 parallel_aggregation2.cpp) -target_link_libraries (parallel_aggregation2 clickhouse_common_io) +target_link_libraries (parallel_aggregation2 PRIVATE clickhouse_common_io) add_executable (int_hashes_perf int_hashes_perf.cpp AvalancheTest.cpp Random.cpp) -target_link_libraries (int_hashes_perf clickhouse_common_io) +target_link_libraries (int_hashes_perf PRIVATE clickhouse_common_io) add_executable (simple_cache simple_cache.cpp) target_include_directories (simple_cache PRIVATE ${DBMS_INCLUDE_DIR}) -target_link_libraries (simple_cache common) +target_link_libraries (simple_cache PRIVATE common) add_executable (compact_array compact_array.cpp) -target_link_libraries (compact_array clickhouse_common_io ${Boost_FILESYSTEM_LIBRARY}) +target_link_libraries (compact_array PRIVATE clickhouse_common_io ${Boost_FILESYSTEM_LIBRARY}) add_executable (radix_sort radix_sort.cpp) -target_link_libraries (radix_sort clickhouse_common_io) +target_link_libraries (radix_sort PRIVATE clickhouse_common_io) add_executable (shell_command_test shell_command_test.cpp) -target_link_libraries (shell_command_test clickhouse_common_io) +target_link_libraries (shell_command_test PRIVATE clickhouse_common_io) add_executable (arena_with_free_lists arena_with_free_lists.cpp) -target_link_libraries (arena_with_free_lists clickhouse_common_io) +target_link_libraries (arena_with_free_lists PRIVATE clickhouse_common_io) add_executable (pod_array pod_array.cpp) -target_link_libraries (pod_array clickhouse_common_io) +target_link_libraries (pod_array PRIVATE clickhouse_common_io) add_executable (thread_creation_latency thread_creation_latency.cpp) -target_link_libraries (thread_creation_latency clickhouse_common_io) +target_link_libraries (thread_creation_latency PRIVATE clickhouse_common_io) add_executable (thread_pool thread_pool.cpp) -target_link_libraries (thread_pool clickhouse_common_io) +target_link_libraries (thread_pool PRIVATE clickhouse_common_io) add_executable (array_cache array_cache.cpp) -target_link_libraries (array_cache clickhouse_common_io) +target_link_libraries (array_cache PRIVATE clickhouse_common_io) add_executable (space_saving space_saving.cpp) -target_link_libraries (space_saving clickhouse_common_io) +target_link_libraries (space_saving PRIVATE clickhouse_common_io) add_executable (integer_hash_tables_and_hashes integer_hash_tables_and_hashes.cpp) target_include_directories (integer_hash_tables_and_hashes SYSTEM BEFORE PRIVATE ${SPARCEHASH_INCLUDE_DIR}) -target_link_libraries (integer_hash_tables_and_hashes clickhouse_common_io) +target_link_libraries (integer_hash_tables_and_hashes PRIVATE clickhouse_common_io) add_executable (allocator allocator.cpp) -target_link_libraries (allocator clickhouse_common_io) +target_link_libraries (allocator PRIVATE clickhouse_common_io) add_executable (cow_columns cow_columns.cpp) -target_link_libraries (cow_columns clickhouse_common_io) +target_link_libraries (cow_columns PRIVATE clickhouse_common_io) add_executable (stopwatch stopwatch.cpp) -target_link_libraries (stopwatch clickhouse_common_io) +target_link_libraries (stopwatch PRIVATE clickhouse_common_io) diff --git a/dbms/src/Core/tests/CMakeLists.txt b/dbms/src/Core/tests/CMakeLists.txt index 26e89341bbb..4748c61e182 100644 --- a/dbms/src/Core/tests/CMakeLists.txt +++ b/dbms/src/Core/tests/CMakeLists.txt @@ -1,18 +1,18 @@ add_executable (exception exception.cpp) -target_link_libraries (exception clickhouse_common_io) +target_link_libraries (exception PRIVATE clickhouse_common_io) add_executable (string_pool string_pool.cpp) -target_link_libraries (string_pool clickhouse_common_io) +target_link_libraries (string_pool PRIVATE clickhouse_common_io) target_include_directories (string_pool SYSTEM BEFORE PRIVATE ${SPARCEHASH_INCLUDE_DIR}) add_executable (field field.cpp) -target_link_libraries (field dbms) +target_link_libraries (field PRIVATE dbms) add_executable (move_field move_field.cpp) -target_link_libraries (move_field clickhouse_common_io) +target_link_libraries (move_field PRIVATE clickhouse_common_io) add_executable (rvo_test rvo_test.cpp) -target_link_libraries (rvo_test Threads::Threads) +target_link_libraries (rvo_test PRIVATE Threads::Threads) add_executable (string_ref_hash string_ref_hash.cpp) -target_link_libraries (string_ref_hash clickhouse_common_io) +target_link_libraries (string_ref_hash PRIVATE clickhouse_common_io) diff --git a/dbms/src/DataStreams/tests/CMakeLists.txt b/dbms/src/DataStreams/tests/CMakeLists.txt index 3dbbc0edbe3..3f6e154927f 100644 --- a/dbms/src/DataStreams/tests/CMakeLists.txt +++ b/dbms/src/DataStreams/tests/CMakeLists.txt @@ -1,16 +1,16 @@ -set(SRCS ) +set(SRCS) add_executable (expression_stream expression_stream.cpp ${SRCS}) -target_link_libraries (expression_stream dbms clickhouse_storages_system) +target_link_libraries (expression_stream PRIVATE dbms clickhouse_storages_system clickhouse_parsers) add_executable (filter_stream filter_stream.cpp ${SRCS}) -target_link_libraries (filter_stream dbms clickhouse_storages_system) +target_link_libraries (filter_stream PRIVATE dbms clickhouse_storages_system clickhouse_parsers clickhouse_common_io) add_executable (union_stream2 union_stream2.cpp ${SRCS}) -target_link_libraries (union_stream2 dbms) +target_link_libraries (union_stream2 PRIVATE dbms) add_executable (collapsing_sorted_stream collapsing_sorted_stream.cpp ${SRCS}) -target_link_libraries (collapsing_sorted_stream dbms) +target_link_libraries (collapsing_sorted_stream PRIVATE dbms) add_executable (finish_sorting_stream finish_sorting_stream.cpp ${SRCS}) -target_link_libraries (finish_sorting_stream dbms) +target_link_libraries (finish_sorting_stream PRIVATE dbms) diff --git a/dbms/src/DataTypes/tests/CMakeLists.txt b/dbms/src/DataTypes/tests/CMakeLists.txt index 6186c7dfef4..c2afc6eb2fe 100644 --- a/dbms/src/DataTypes/tests/CMakeLists.txt +++ b/dbms/src/DataTypes/tests/CMakeLists.txt @@ -1,10 +1,10 @@ set(SRCS ) add_executable (data_types_number_fixed data_types_number_fixed.cpp ${SRCS}) -target_link_libraries (data_types_number_fixed dbms) +target_link_libraries (data_types_number_fixed PRIVATE dbms) add_executable (data_type_string data_type_string.cpp ${SRCS}) -target_link_libraries (data_type_string dbms) +target_link_libraries (data_type_string PRIVATE dbms) add_executable (data_type_get_common_type data_type_get_common_type.cpp ${SRCS}) -target_link_libraries (data_type_get_common_type dbms gtest_main) +target_link_libraries (data_type_get_common_type PRIVATE dbms gtest_main) diff --git a/dbms/src/Formats/tests/CMakeLists.txt b/dbms/src/Formats/tests/CMakeLists.txt index de94fb4d4f3..e12fa0f02fb 100644 --- a/dbms/src/Formats/tests/CMakeLists.txt +++ b/dbms/src/Formats/tests/CMakeLists.txt @@ -1,7 +1,7 @@ set(SRCS ) add_executable (tab_separated_streams tab_separated_streams.cpp ${SRCS}) -target_link_libraries (tab_separated_streams dbms) +target_link_libraries (tab_separated_streams PRIVATE dbms) add_executable (block_row_transforms block_row_transforms.cpp ${SRCS}) -target_link_libraries (block_row_transforms dbms) +target_link_libraries (block_row_transforms PRIVATE dbms) diff --git a/dbms/src/Functions/tests/CMakeLists.txt b/dbms/src/Functions/tests/CMakeLists.txt index b0fd9d1642d..91bfa4bc276 100644 --- a/dbms/src/Functions/tests/CMakeLists.txt +++ b/dbms/src/Functions/tests/CMakeLists.txt @@ -1,2 +1,2 @@ add_executable (number_traits number_traits.cpp) -target_link_libraries (number_traits dbms) +target_link_libraries (number_traits PRIVATE dbms) diff --git a/dbms/src/IO/tests/CMakeLists.txt b/dbms/src/IO/tests/CMakeLists.txt index f75ad8523f1..75398ff9801 100644 --- a/dbms/src/IO/tests/CMakeLists.txt +++ b/dbms/src/IO/tests/CMakeLists.txt @@ -1,84 +1,84 @@ add_executable (read_buffer read_buffer.cpp) -target_link_libraries (read_buffer clickhouse_common_io) +target_link_libraries (read_buffer PRIVATE clickhouse_common_io) add_executable (read_buffer_perf read_buffer_perf.cpp) -target_link_libraries (read_buffer_perf clickhouse_common_io) +target_link_libraries (read_buffer_perf PRIVATE clickhouse_common_io) add_executable (read_float_perf read_float_perf.cpp) -target_link_libraries (read_float_perf clickhouse_common_io) +target_link_libraries (read_float_perf PRIVATE clickhouse_common_io) add_executable (write_buffer write_buffer.cpp) -target_link_libraries (write_buffer clickhouse_common_io) +target_link_libraries (write_buffer PRIVATE clickhouse_common_io) add_executable (write_buffer_perf write_buffer_perf.cpp) -target_link_libraries (write_buffer_perf clickhouse_common_io) +target_link_libraries (write_buffer_perf PRIVATE clickhouse_common_io) add_executable (valid_utf8_perf valid_utf8_perf.cpp) -target_link_libraries (valid_utf8_perf clickhouse_common_io) +target_link_libraries (valid_utf8_perf PRIVATE clickhouse_common_io) add_executable (valid_utf8 valid_utf8.cpp) -target_link_libraries (valid_utf8 clickhouse_common_io) +target_link_libraries (valid_utf8 PRIVATE clickhouse_common_io) add_executable (compressed_buffer compressed_buffer.cpp) -target_link_libraries (compressed_buffer clickhouse_common_io) +target_link_libraries (compressed_buffer PRIVATE clickhouse_common_io) add_executable (var_uint var_uint.cpp) -target_link_libraries (var_uint clickhouse_common_io) +target_link_libraries (var_uint PRIVATE clickhouse_common_io) add_executable (read_escaped_string read_escaped_string.cpp) -target_link_libraries (read_escaped_string clickhouse_common_io) +target_link_libraries (read_escaped_string PRIVATE clickhouse_common_io) add_executable (async_write async_write.cpp) -target_link_libraries (async_write clickhouse_common_io) +target_link_libraries (async_write PRIVATE clickhouse_common_io) add_executable (parse_int_perf parse_int_perf.cpp) -target_link_libraries (parse_int_perf clickhouse_common_io) +target_link_libraries (parse_int_perf PRIVATE clickhouse_common_io) add_executable (parse_int_perf2 parse_int_perf2.cpp) -target_link_libraries (parse_int_perf2 clickhouse_common_io) +target_link_libraries (parse_int_perf2 PRIVATE clickhouse_common_io) add_executable (read_write_int read_write_int.cpp) -target_link_libraries (read_write_int clickhouse_common_io) +target_link_libraries (read_write_int PRIVATE clickhouse_common_io) add_executable (mempbrk mempbrk.cpp) -target_link_libraries (mempbrk clickhouse_common_io) +target_link_libraries (mempbrk PRIVATE clickhouse_common_io) add_executable (cached_compressed_read_buffer cached_compressed_read_buffer.cpp) -target_link_libraries (cached_compressed_read_buffer clickhouse_common_io) +target_link_libraries (cached_compressed_read_buffer PRIVATE clickhouse_common_io) add_executable (o_direct_and_dirty_pages o_direct_and_dirty_pages.cpp) -target_link_libraries (o_direct_and_dirty_pages clickhouse_common_io) +target_link_libraries (o_direct_and_dirty_pages PRIVATE clickhouse_common_io) add_executable (hashing_write_buffer hashing_write_buffer.cpp) -target_link_libraries (hashing_write_buffer clickhouse_common_io) +target_link_libraries (hashing_write_buffer PRIVATE clickhouse_common_io) add_check(hashing_write_buffer) add_executable (hashing_read_buffer hashing_read_buffer.cpp) -target_link_libraries (hashing_read_buffer clickhouse_common_io) +target_link_libraries (hashing_read_buffer PRIVATE clickhouse_common_io) add_check (hashing_read_buffer) add_executable (io_operators operators.cpp) -target_link_libraries (io_operators clickhouse_common_io) +target_link_libraries (io_operators PRIVATE clickhouse_common_io) if (OS_LINUX) add_executable(write_buffer_aio write_buffer_aio.cpp) - target_link_libraries (write_buffer_aio clickhouse_common_io ${Boost_FILESYSTEM_LIBRARY}) + target_link_libraries (write_buffer_aio PRIVATE clickhouse_common_io ${Boost_FILESYSTEM_LIBRARY}) add_executable(read_buffer_aio read_buffer_aio.cpp) - target_link_libraries (read_buffer_aio clickhouse_common_io ${Boost_FILESYSTEM_LIBRARY}) + target_link_libraries (read_buffer_aio PRIVATE clickhouse_common_io ${Boost_FILESYSTEM_LIBRARY}) endif () add_executable (zlib_buffers zlib_buffers.cpp) -target_link_libraries (zlib_buffers clickhouse_common_io) +target_link_libraries (zlib_buffers PRIVATE clickhouse_common_io) add_executable (limit_read_buffer limit_read_buffer.cpp) -target_link_libraries (limit_read_buffer clickhouse_common_io) +target_link_libraries (limit_read_buffer PRIVATE clickhouse_common_io) add_executable (limit_read_buffer2 limit_read_buffer2.cpp) -target_link_libraries (limit_read_buffer2 clickhouse_common_io) +target_link_libraries (limit_read_buffer2 PRIVATE clickhouse_common_io) add_executable (parse_date_time_best_effort parse_date_time_best_effort.cpp) -target_link_libraries (parse_date_time_best_effort clickhouse_common_io) +target_link_libraries (parse_date_time_best_effort PRIVATE clickhouse_common_io) add_executable (zlib_ng_bug zlib_ng_bug.cpp) -target_link_libraries (zlib_ng_bug ${Poco_Foundation_LIBRARY}) +target_link_libraries (zlib_ng_bug PRIVATE ${Poco_Foundation_LIBRARY}) diff --git a/dbms/src/Interpreters/tests/CMakeLists.txt b/dbms/src/Interpreters/tests/CMakeLists.txt index 01fc72730dc..7660527cd87 100644 --- a/dbms/src/Interpreters/tests/CMakeLists.txt +++ b/dbms/src/Interpreters/tests/CMakeLists.txt @@ -1,58 +1,58 @@ add_executable (expression expression.cpp) -target_link_libraries (expression dbms) +target_link_libraries (expression PRIVATE dbms clickhouse_parsers) add_executable (create_query create_query.cpp) -target_link_libraries (create_query dbms) +target_link_libraries (create_query PRIVATE dbms clickhouse_parsers) add_executable (select_query select_query.cpp) -target_link_libraries (select_query clickhouse_storages_system dbms) +target_link_libraries (select_query PRIVATE clickhouse_storages_system dbms clickhouse_common_io) add_executable (aggregate aggregate.cpp) -target_link_libraries (aggregate dbms) +target_link_libraries (aggregate PRIVATE dbms) add_executable (hash_map hash_map.cpp) target_include_directories (hash_map SYSTEM BEFORE PRIVATE ${SPARCEHASH_INCLUDE_DIR}) -target_link_libraries (hash_map dbms) +target_link_libraries (hash_map PRIVATE dbms) add_executable (hash_map3 hash_map3.cpp) -target_link_libraries (hash_map3 dbms ${FARMHASH_LIBRARIES} ${METROHASH_LIBRARIES}) +target_link_libraries (hash_map3 PRIVATE dbms ${FARMHASH_LIBRARIES} ${METROHASH_LIBRARIES}) add_executable (hash_map_string hash_map_string.cpp) target_include_directories (hash_map_string SYSTEM BEFORE PRIVATE ${SPARCEHASH_INCLUDE_DIR}) -target_link_libraries (hash_map_string dbms) +target_link_libraries (hash_map_string PRIVATE dbms) add_executable (hash_map_string_2 hash_map_string_2.cpp) -target_link_libraries (hash_map_string_2 dbms) +target_link_libraries (hash_map_string_2 PRIVATE dbms) add_executable (hash_map_string_3 hash_map_string_3.cpp) -target_link_libraries (hash_map_string_3 dbms ${FARMHASH_LIBRARIES} ${METROHASH_LIBRARIES}) +target_link_libraries (hash_map_string_3 PRIVATE dbms ${FARMHASH_LIBRARIES} ${METROHASH_LIBRARIES}) add_executable (hash_map_string_small hash_map_string_small.cpp) target_include_directories (hash_map_string_small SYSTEM BEFORE PRIVATE ${SPARCEHASH_INCLUDE_DIR}) -target_link_libraries (hash_map_string_small dbms) +target_link_libraries (hash_map_string_small PRIVATE dbms) add_executable (two_level_hash_map two_level_hash_map.cpp) target_include_directories (two_level_hash_map SYSTEM BEFORE PRIVATE ${SPARCEHASH_INCLUDE_DIR}) -target_link_libraries (two_level_hash_map dbms) +target_link_libraries (two_level_hash_map PRIVATE dbms) add_executable (compiler_test compiler_test.cpp) -target_link_libraries (compiler_test dbms) +target_link_libraries (compiler_test PRIVATE dbms) add_executable (logical_expressions_optimizer logical_expressions_optimizer.cpp) -target_link_libraries (logical_expressions_optimizer dbms) +target_link_libraries (logical_expressions_optimizer PRIVATE dbms clickhouse_parsers) add_executable (in_join_subqueries_preprocessor in_join_subqueries_preprocessor.cpp) -target_link_libraries (in_join_subqueries_preprocessor dbms) +target_link_libraries (in_join_subqueries_preprocessor PRIVATE dbms clickhouse_parsers) add_check(in_join_subqueries_preprocessor) add_executable (expression_analyzer expression_analyzer.cpp) -target_link_libraries (expression_analyzer dbms clickhouse_storages_system) +target_link_libraries (expression_analyzer PRIVATE dbms clickhouse_storages_system clickhouse_parsers clickhouse_common_io) add_check(expression_analyzer) add_executable (users users.cpp) -target_link_libraries (users dbms ${Boost_FILESYSTEM_LIBRARY}) +target_link_libraries (users PRIVATE dbms clickhouse_common_config ${Boost_FILESYSTEM_LIBRARY}) if (OS_LINUX) add_executable (internal_iotop internal_iotop.cpp) - target_link_libraries (internal_iotop dbms) + target_link_libraries (internal_iotop PRIVATE dbms) endif () diff --git a/dbms/src/Parsers/CMakeLists.txt b/dbms/src/Parsers/CMakeLists.txt index 90f314266fa..790e151dab3 100644 --- a/dbms/src/Parsers/CMakeLists.txt +++ b/dbms/src/Parsers/CMakeLists.txt @@ -1,9 +1,9 @@ include(${ClickHouse_SOURCE_DIR}/cmake/dbms_glob_sources.cmake) add_headers_and_sources(clickhouse_parsers .) add_library(clickhouse_parsers ${LINK_MODE} ${clickhouse_parsers_headers} ${clickhouse_parsers_sources}) -target_link_libraries (clickhouse_parsers clickhouse_common_io) -target_include_directories (clickhouse_parsers PUBLIC ${DBMS_INCLUDE_DIR}) +target_link_libraries(clickhouse_parsers PUBLIC clickhouse_common_io) +target_include_directories(clickhouse_parsers PUBLIC ${DBMS_INCLUDE_DIR}) -if (ENABLE_TESTS) - add_subdirectory (tests) -endif () +if(ENABLE_TESTS) + add_subdirectory(tests) +endif() diff --git a/dbms/src/Parsers/tests/CMakeLists.txt b/dbms/src/Parsers/tests/CMakeLists.txt index 4f87424bf89..3e1d6ae559f 100644 --- a/dbms/src/Parsers/tests/CMakeLists.txt +++ b/dbms/src/Parsers/tests/CMakeLists.txt @@ -1,11 +1,10 @@ -set(SRCS ) +set(SRCS) -add_executable (lexer lexer.cpp ${SRCS}) -target_link_libraries (lexer clickhouse_parsers) +add_executable(lexer lexer.cpp ${SRCS}) +target_link_libraries(lexer PRIVATE clickhouse_parsers) -add_executable (select_parser select_parser.cpp ${SRCS}) -target_link_libraries (select_parser clickhouse_parsers) - -add_executable (create_parser create_parser.cpp ${SRCS}) -target_link_libraries (create_parser clickhouse_parsers) +add_executable(select_parser select_parser.cpp ${SRCS}) +target_link_libraries(select_parser PRIVATE clickhouse_parsers) +add_executable(create_parser create_parser.cpp ${SRCS}) +target_link_libraries(create_parser PRIVATE clickhouse_parsers) diff --git a/dbms/src/Storages/System/CMakeLists.txt b/dbms/src/Storages/System/CMakeLists.txt index b4783ffa315..418c25de5fd 100644 --- a/dbms/src/Storages/System/CMakeLists.txt +++ b/dbms/src/Storages/System/CMakeLists.txt @@ -12,5 +12,4 @@ 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}) +target_link_libraries(clickhouse_storages_system PRIVATE dbms common string_utils clickhouse_common_zookeeper) diff --git a/dbms/src/Storages/tests/CMakeLists.txt b/dbms/src/Storages/tests/CMakeLists.txt index 63ff90d3055..2942ecfe6bc 100644 --- a/dbms/src/Storages/tests/CMakeLists.txt +++ b/dbms/src/Storages/tests/CMakeLists.txt @@ -1,29 +1,29 @@ add_executable (system_numbers system_numbers.cpp) -target_link_libraries (system_numbers dbms clickhouse_storages_system) +target_link_libraries (system_numbers PRIVATE dbms clickhouse_storages_system clickhouse_common_io) add_executable (storage_log storage_log.cpp) -target_link_libraries (storage_log dbms) +target_link_libraries (storage_log PRIVATE dbms) add_executable (seek_speed_test seek_speed_test.cpp) -target_link_libraries (seek_speed_test dbms) +target_link_libraries (seek_speed_test PRIVATE dbms) add_executable (part_checker part_checker.cpp) -target_link_libraries (part_checker dbms) +target_link_libraries (part_checker PRIVATE dbms) add_executable (part_name part_name.cpp) -target_link_libraries (part_name dbms) +target_link_libraries (part_name PRIVATE dbms) add_executable (remove_symlink_directory remove_symlink_directory.cpp) -target_link_libraries (remove_symlink_directory dbms) +target_link_libraries (remove_symlink_directory PRIVATE dbms) add_executable (merge_selector merge_selector.cpp) -target_link_libraries (merge_selector dbms) +target_link_libraries (merge_selector PRIVATE dbms) add_executable (merge_selector2 merge_selector2.cpp) -target_link_libraries (merge_selector2 dbms) +target_link_libraries (merge_selector2 PRIVATE dbms) add_executable (get_current_inserts_in_replicated get_current_inserts_in_replicated.cpp) -target_link_libraries (get_current_inserts_in_replicated dbms) +target_link_libraries (get_current_inserts_in_replicated PRIVATE dbms clickhouse_common_config clickhouse_common_zookeeper string_utils) add_executable (get_abandonable_lock_in_all_partitions get_abandonable_lock_in_all_partitions.cpp) -target_link_libraries (get_abandonable_lock_in_all_partitions dbms) +target_link_libraries (get_abandonable_lock_in_all_partitions PRIVATE dbms clickhouse_common_config clickhouse_common_zookeeper) diff --git a/dbms/src/TableFunctions/CMakeLists.txt b/dbms/src/TableFunctions/CMakeLists.txt index 3eea07fd8e6..0ab5acec19c 100644 --- a/dbms/src/TableFunctions/CMakeLists.txt +++ b/dbms/src/TableFunctions/CMakeLists.txt @@ -5,4 +5,4 @@ list(REMOVE_ITEM clickhouse_table_functions_sources ITableFunction.cpp TableFunc list(REMOVE_ITEM clickhouse_table_functions_headers ITableFunction.h TableFunctionFactory.h) add_library(clickhouse_table_functions ${LINK_MODE} ${clickhouse_table_functions_sources}) -target_link_libraries(clickhouse_table_functions clickhouse_storages_system dbms ${Poco_Foundation_LIBRARY}) +target_link_libraries(clickhouse_table_functions PRIVATE clickhouse_storages_system dbms ${Poco_Foundation_LIBRARY}) diff --git a/libs/libcommon/CMakeLists.txt b/libs/libcommon/CMakeLists.txt index 6d341bee4c2..2754ecb9539 100644 --- a/libs/libcommon/CMakeLists.txt +++ b/libs/libcommon/CMakeLists.txt @@ -93,27 +93,31 @@ endif () find_package (Threads) target_include_directories (common BEFORE PRIVATE ${CCTZ_INCLUDE_DIR}) -target_include_directories (common BEFORE PUBLIC ${CITYHASH_INCLUDE_DIR}) target_include_directories (common PUBLIC ${COMMON_INCLUDE_DIR}) if (NOT USE_INTERNAL_BOOST_LIBRARY) target_include_directories (common BEFORE PUBLIC ${Boost_INCLUDE_DIRS}) endif () -target_link_libraries ( - common +target_link_libraries (common + PRIVATE pocoext + PUBLIC + ${Poco_Foundation_LIBRARY} ${CITYHASH_LIBRARIES} + PRIVATE ${CCTZ_LIBRARY} ${Boost_FILESYSTEM_LIBRARY} + PUBLIC ${Boost_SYSTEM_LIBRARY} + PRIVATE ${MALLOC_LIBRARIES} ${CMAKE_THREAD_LIBS_INIT} ${GLIBC_COMPATIBILITY_LIBRARIES} ${MEMCPY_LIBRARIES}) if (RT_LIBRARY) - target_link_libraries (common ${RT_LIBRARY}) + target_link_libraries (common PRIVATE ${RT_LIBRARY}) endif () if (ENABLE_TESTS) diff --git a/libs/libcommon/cmake/find_jemalloc.cmake b/libs/libcommon/cmake/find_jemalloc.cmake index 35e2a13358b..eb8c9cb6ac7 100644 --- a/libs/libcommon/cmake/find_jemalloc.cmake +++ b/libs/libcommon/cmake/find_jemalloc.cmake @@ -25,7 +25,7 @@ if (ENABLE_JEMALLOC) if ((NOT JEMALLOC_LIBRARIES OR NOT JEMALLOC_INCLUDE_DIR) AND NOT MISSING_INTERNAL_JEMALLOC_LIBRARY) set (JEMALLOC_LIBRARIES "jemalloc") - set (JEMALLOC_INCLUDE_DIR "${ClickHouse_SOURCE_DIR}/contrib/jemalloc/include") + set (JEMALLOC_INCLUDE_DIR "${ClickHouse_SOURCE_DIR}/contrib/jemalloc-cmake/include" "${ClickHouse_SOURCE_DIR}/contrib/jemalloc-cmake/include_linux_x86_64") set (USE_INTERNAL_JEMALLOC_LIBRARY 1) endif () diff --git a/libs/libdaemon/CMakeLists.txt b/libs/libdaemon/CMakeLists.txt index 448c83148b2..b352602f81c 100644 --- a/libs/libdaemon/CMakeLists.txt +++ b/libs/libdaemon/CMakeLists.txt @@ -17,10 +17,10 @@ add_library (daemon ${LINK_MODE} if (USE_UNWIND) target_compile_definitions (daemon PRIVATE USE_UNWIND=1) target_include_directories (daemon BEFORE PRIVATE ${UNWIND_INCLUDE_DIR}) - target_link_libraries (daemon ${UNWIND_LIBRARY}) + target_link_libraries (daemon PRIVATE ${UNWIND_LIBRARY}) endif () target_include_directories (daemon PUBLIC include) target_include_directories (daemon PRIVATE ${ClickHouse_SOURCE_DIR}/libs/libpocoext/include) -target_link_libraries (daemon clickhouse_common_io clickhouse_common_config ${Poco_Util_LIBRARY} ${EXECINFO_LIBRARY} ${ELF_LIBRARY}) +target_link_libraries (daemon PRIVATE clickhouse_common_io clickhouse_common_config common ${Poco_Net_LIBRARY} ${Poco_Util_LIBRARY} ${EXECINFO_LIBRARY} ${ELF_LIBRARY}) diff --git a/libs/libmysqlxx/CMakeLists.txt b/libs/libmysqlxx/CMakeLists.txt index fe7cc52224b..a163d325111 100644 --- a/libs/libmysqlxx/CMakeLists.txt +++ b/libs/libmysqlxx/CMakeLists.txt @@ -42,20 +42,29 @@ else () find_library (ICONV_LIBRARY iconv) set (MYSQLCLIENT_LIBRARIES ${MYSQLCLIENT_LIBRARIES} ${STATIC_MYSQLCLIENT_LIB} ${ICONV_LIBRARY}) elseif (USE_STATIC_LIBRARIES AND STATIC_MYSQLCLIENT_LIB) + if (0) + # old lib patcher (ubuntu trusty?) set (MYSQLCLIENT_LIB ${CMAKE_CURRENT_BINARY_DIR}/libmysqlclient.a) + message(STATUS "will patch mysql lib ${STATIC_MYSQLCLIENT_LIB} => ${MYSQLCLIENT_LIB}") add_custom_command ( OUTPUT ${MYSQLCLIENT_LIB} COMMAND ${CMAKE_CURRENT_SOURCE_DIR}/patch.sh ${STATIC_MYSQLCLIENT_LIB} ${MYSQLCLIENT_LIB} WORKING_DIRECTORY ${CMAKE_CURRENT_BINARY_DIR} + DEPENDS ${STATIC_MYSQLCLIENT_LIB} COMMENT "Patching mysqlclient library.") add_custom_target (our_mysql_client DEPENDS ${MYSQLCLIENT_LIB}) add_dependencies (mysqlxx our_mysql_client) set (MYSQLCLIENT_LIBRARIES ${MYSQLCLIENT_LIB}) + else() + set (MYSQLCLIENT_LIBRARIES ${STATIC_MYSQLCLIENT_LIB}) + endif() endif () endif () -target_link_libraries (mysqlxx common ${Poco_Util_LIBRARY} ${MYSQLCLIENT_LIBRARIES} ${OPENSSL_LIBRARIES} ${ZLIB_LIBRARIES} ${PLATFORM_LIBRARIES}) -target_include_directories (mysqlxx SYSTEM PRIVATE ${OPENSSL_INCLUDE_DIR}) +target_link_libraries (mysqlxx common ${Poco_Util_LIBRARY} ${MYSQLCLIENT_LIBRARIES} ${Boost_SYSTEM_LIBRARY} ${OPENSSL_LIBRARIES} ${ZLIB_LIBRARIES} ${PLATFORM_LIBRARIES}) +if (NOT USE_INTERNAL_MYSQL_LIBRARY) + target_include_directories (mysqlxx SYSTEM PRIVATE ${OPENSSL_INCLUDE_DIR}) +endif () if (ENABLE_TESTS) add_subdirectory (src/tests) diff --git a/utils/check-marks/CMakeLists.txt b/utils/check-marks/CMakeLists.txt index 5b858f929e1..9c534364691 100644 --- a/utils/check-marks/CMakeLists.txt +++ b/utils/check-marks/CMakeLists.txt @@ -1,2 +1,2 @@ add_executable (check-marks main.cpp) -target_link_libraries (check-marks clickhouse_common_io ${Boost_PROGRAM_OPTIONS_LIBRARY}) +target_link_libraries(check-marks PRIVATE clickhouse_common_io ${Boost_PROGRAM_OPTIONS_LIBRARY}) diff --git a/utils/compressor/CMakeLists.txt b/utils/compressor/CMakeLists.txt index 20f7a82aa37..cd140051203 100644 --- a/utils/compressor/CMakeLists.txt +++ b/utils/compressor/CMakeLists.txt @@ -1,19 +1,19 @@ find_package (Threads) add_executable (util-clickhouse-compressor main.cpp) -target_link_libraries (util-clickhouse-compressor clickhouse-compressor-lib) +target_link_libraries (util-clickhouse-compressor PRIVATE clickhouse-compressor-lib) set_target_properties(util-clickhouse-compressor PROPERTIES OUTPUT_NAME "clickhouse-compressor") #install (TARGETS util-clickhouse-compressor RUNTIME DESTINATION ${CMAKE_INSTALL_BINDIR} COMPONENT clickhouse-compressor) add_executable (zstd_test zstd_test.cpp) -target_link_libraries (zstd_test ${ZSTD_LIBRARY} common Threads::Threads) +target_link_libraries (zstd_test PRIVATE ${ZSTD_LIBRARY} common Threads::Threads) add_executable (mutator mutator.cpp) -target_link_libraries (mutator clickhouse_common_io) +target_link_libraries(mutator PRIVATE clickhouse_common_io) add_executable (decompress_perf decompress_perf.cpp) -target_link_libraries (decompress_perf clickhouse_common_io) +target_link_libraries(decompress_perf PRIVATE clickhouse_common_io ${LZ4_LIBRARY}) if (NOT USE_INTERNAL_ZSTD_LIBRARY) target_include_directories (zstd_test BEFORE PRIVATE ${ZSTD_INCLUDE_DIR}) diff --git a/utils/config-processor/CMakeLists.txt b/utils/config-processor/CMakeLists.txt index d25c25d2f02..e7e15d0be53 100644 --- a/utils/config-processor/CMakeLists.txt +++ b/utils/config-processor/CMakeLists.txt @@ -1,4 +1,4 @@ add_executable (config-processor config-processor.cpp) -target_link_libraries (config-processor clickhouse_common_config) +target_link_libraries(config-processor PRIVATE clickhouse_common_config) INSTALL(TARGETS config-processor RUNTIME DESTINATION ${CMAKE_INSTALL_BINDIR} COMPONENT config-processor) diff --git a/utils/corrector_utf8/CMakeLists.txt b/utils/corrector_utf8/CMakeLists.txt index d9c971a6cc5..9114f3f58a0 100644 --- a/utils/corrector_utf8/CMakeLists.txt +++ b/utils/corrector_utf8/CMakeLists.txt @@ -1,7 +1,6 @@ - add_executable(corrector_utf8 corrector_utf8.cpp) # Link the executable to the library. -target_link_libraries (corrector_utf8 clickhouse_common_io) +target_link_libraries(corrector_utf8 PRIVATE clickhouse_common_io) -install( TARGETS corrector_utf8 RUNTIME DESTINATION ${CMAKE_INSTALL_BINDIR} COMPONENT corrector_utf8) +install(TARGETS corrector_utf8 RUNTIME DESTINATION ${CMAKE_INSTALL_BINDIR} COMPONENT corrector_utf8) diff --git a/utils/fill-factor/CMakeLists.txt b/utils/fill-factor/CMakeLists.txt index fad359b63a6..35a6712b115 100644 --- a/utils/fill-factor/CMakeLists.txt +++ b/utils/fill-factor/CMakeLists.txt @@ -1,2 +1,2 @@ add_executable (fill-factor main.cpp) -target_link_libraries (fill-factor clickhouse_common_io) +target_link_libraries(fill-factor PRIVATE clickhouse_common_io) diff --git a/utils/iotest/CMakeLists.txt b/utils/iotest/CMakeLists.txt index f690409a54d..8f141b178f0 100644 --- a/utils/iotest/CMakeLists.txt +++ b/utils/iotest/CMakeLists.txt @@ -1,9 +1,9 @@ add_executable (iotest iotest.cpp ${SRCS}) -target_link_libraries (iotest clickhouse_common_io) +target_link_libraries (iotest PRIVATE clickhouse_common_io) add_executable (iotest_nonblock iotest_nonblock.cpp ${SRCS}) -target_link_libraries (iotest_nonblock clickhouse_common_io) +target_link_libraries (iotest_nonblock PRIVATE clickhouse_common_io) add_executable (iotest_aio iotest_aio.cpp ${SRCS}) -target_link_libraries (iotest_aio clickhouse_common_io) +target_link_libraries (iotest_aio PRIVATE clickhouse_common_io) diff --git a/utils/test-data-generator/CMakeLists.txt b/utils/test-data-generator/CMakeLists.txt index 9f1a0bf5a0d..60f17894704 100644 --- a/utils/test-data-generator/CMakeLists.txt +++ b/utils/test-data-generator/CMakeLists.txt @@ -1,5 +1,5 @@ add_executable (test-data-generator main.cpp) -target_link_libraries (test-data-generator clickhouse_common_io ${Boost_PROGRAM_OPTIONS_LIBRARY}) +target_link_libraries(test-data-generator PRIVATE clickhouse_common_io ${Boost_PROGRAM_OPTIONS_LIBRARY}) add_executable (markov-model markov-model.cpp) -target_link_libraries (markov-model clickhouse_common_io ${Boost_PROGRAM_OPTIONS_LIBRARY}) +target_link_libraries(markov-model PRIVATE clickhouse_common_io ${Boost_PROGRAM_OPTIONS_LIBRARY}) diff --git a/utils/wikistat-loader/CMakeLists.txt b/utils/wikistat-loader/CMakeLists.txt index 84d00e91bd5..7f72cbb9f46 100644 --- a/utils/wikistat-loader/CMakeLists.txt +++ b/utils/wikistat-loader/CMakeLists.txt @@ -1,2 +1,2 @@ add_executable (wikistat-loader main.cpp ${SRCS}) -target_link_libraries (wikistat-loader clickhouse_common_io ${Boost_PROGRAM_OPTIONS_LIBRARY}) +target_link_libraries (wikistat-loader PRIVATE clickhouse_common_io ${Boost_PROGRAM_OPTIONS_LIBRARY}) diff --git a/utils/zookeeper-cli/CMakeLists.txt b/utils/zookeeper-cli/CMakeLists.txt index 27b18bfc1fc..89db7922edd 100644 --- a/utils/zookeeper-cli/CMakeLists.txt +++ b/utils/zookeeper-cli/CMakeLists.txt @@ -1,5 +1,5 @@ add_executable(clickhouse-zookeeper-cli zookeeper-cli.cpp) -target_link_libraries(clickhouse-zookeeper-cli clickhouse_common_zookeeper ${LINE_EDITING_LIBS}) +target_link_libraries(clickhouse-zookeeper-cli PRIVATE clickhouse_common_zookeeper ${LINE_EDITING_LIBS}) if (READLINE_INCLUDE_DIR) target_include_directories (clickhouse-zookeeper-cli SYSTEM PRIVATE ${READLINE_INCLUDE_DIR}) endif () diff --git a/utils/zookeeper-create-entry-to-download-part/CMakeLists.txt b/utils/zookeeper-create-entry-to-download-part/CMakeLists.txt index 4b59b838d66..34f2e608ef9 100644 --- a/utils/zookeeper-create-entry-to-download-part/CMakeLists.txt +++ b/utils/zookeeper-create-entry-to-download-part/CMakeLists.txt @@ -1,2 +1,2 @@ add_executable (zookeeper-create-entry-to-download-part main.cpp ${SRCS}) -target_link_libraries (zookeeper-create-entry-to-download-part dbms ${Boost_PROGRAM_OPTIONS_LIBRARY}) +target_link_libraries (zookeeper-create-entry-to-download-part PRIVATE dbms clickhouse_common_zookeeper ${Boost_PROGRAM_OPTIONS_LIBRARY}) diff --git a/utils/zookeeper-dump-tree/CMakeLists.txt b/utils/zookeeper-dump-tree/CMakeLists.txt index ee28f42143a..d2947fa8932 100644 --- a/utils/zookeeper-dump-tree/CMakeLists.txt +++ b/utils/zookeeper-dump-tree/CMakeLists.txt @@ -1,2 +1,2 @@ add_executable (zookeeper-dump-tree main.cpp ${SRCS}) -target_link_libraries (zookeeper-dump-tree clickhouse_common_zookeeper ${Boost_PROGRAM_OPTIONS_LIBRARY}) +target_link_libraries(zookeeper-dump-tree PRIVATE clickhouse_common_zookeeper clickhouse_common_io ${Boost_PROGRAM_OPTIONS_LIBRARY}) diff --git a/utils/zookeeper-remove-by-list/CMakeLists.txt b/utils/zookeeper-remove-by-list/CMakeLists.txt index 494ef819f44..ba112bab9cf 100644 --- a/utils/zookeeper-remove-by-list/CMakeLists.txt +++ b/utils/zookeeper-remove-by-list/CMakeLists.txt @@ -1,2 +1,2 @@ add_executable (zookeeper-remove-by-list main.cpp ${SRCS}) -target_link_libraries (zookeeper-remove-by-list clickhouse_common_zookeeper ${Boost_PROGRAM_OPTIONS_LIBRARY}) +target_link_libraries(zookeeper-remove-by-list PRIVATE clickhouse_common_zookeeper ${Boost_PROGRAM_OPTIONS_LIBRARY}) From 6f48ebbb06160e2bb804c17e62a9a7b81d2de352 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 21 Nov 2018 23:35:55 +0300 Subject: [PATCH 038/124] Better exception message in case of "unknown identifier" [#CLICKHOUSE-4142] --- dbms/src/Interpreters/ExpressionAnalyzer.cpp | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/dbms/src/Interpreters/ExpressionAnalyzer.cpp b/dbms/src/Interpreters/ExpressionAnalyzer.cpp index 55e04b7a888..79117744f2f 100644 --- a/dbms/src/Interpreters/ExpressionAnalyzer.cpp +++ b/dbms/src/Interpreters/ExpressionAnalyzer.cpp @@ -1087,7 +1087,7 @@ void ExpressionAnalyzer::collectUsedColumns() } joined_block_actions = analyzedJoin().createJoinedBlockActions( - columns_added_by_join, select_query, context, required_columns_from_joined_table); + columns_added_by_join, select_query, context, required_columns_from_joined_table); /// Some columns from right join key may be used in query. This columns will be appended to block during join. for (const auto & right_key_name : analyzedJoin().key_names_right) @@ -1136,7 +1136,9 @@ void ExpressionAnalyzer::collectUsedColumns() } if (!unknown_required_source_columns.empty()) - throw Exception("Unknown identifier: " + *unknown_required_source_columns.begin(), ErrorCodes::UNKNOWN_IDENTIFIER); + throw Exception("Unknown identifier: " + *unknown_required_source_columns.begin() + + (select_query && !select_query->tables ? ". Note that there is no tables (FROM clause) in your query" : ""), + ErrorCodes::UNKNOWN_IDENTIFIER); } From ab5d76fc42e5c560c3d6d327ec03e9d7809455f1 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 21 Nov 2018 23:56:37 +0300 Subject: [PATCH 039/124] Fixed error introduced in #3553 and prevent it happening --- dbms/programs/server/Server.cpp | 3 +- dbms/src/Common/CounterInFile.h | 22 +++++++---- dbms/src/Common/ErrorCodes.cpp | 3 ++ dbms/src/Common/Exception.cpp | 2 +- dbms/src/Common/Exception.h | 10 ++--- dbms/src/Common/StatusFile.cpp | 20 +++++++--- dbms/src/Common/createHardLink.cpp | 14 +++++-- dbms/src/Common/setThreadName.cpp | 4 +- .../Common/tests/arena_with_free_lists.cpp | 10 ++++- .../Common/tests/thread_creation_latency.cpp | 13 ++++++- .../DistributedBlockOutputStream.cpp | 3 +- dbms/src/Storages/MergeTree/MergeTreeData.cpp | 8 +++- .../tests/remove_symlink_directory.cpp | 10 ++++- libs/libdaemon/include/daemon/BaseDaemon.h | 2 +- libs/libdaemon/src/BaseDaemon.cpp | 36 +++++------------ libs/libdaemon/src/ExtendedLogChannel.cpp | 7 +++- utils/iotest/iotest.cpp | 37 ++++++++++++------ utils/iotest/iotest_aio.cpp | 39 ++++++++++++------- utils/iotest/iotest_nonblock.cpp | 37 ++++++++++++------ 19 files changed, 183 insertions(+), 97 deletions(-) diff --git a/dbms/programs/server/Server.cpp b/dbms/programs/server/Server.cpp index 0937cf4b84a..d8d662d8ed0 100644 --- a/dbms/programs/server/Server.cpp +++ b/dbms/programs/server/Server.cpp @@ -66,6 +66,7 @@ namespace ErrorCodes extern const int SUPPORT_IS_DISABLED; extern const int ARGUMENT_OUT_OF_BOUND; extern const int EXCESSIVE_ELEMENT_IN_CONFIG; + extern const int SYSTEM_ERROR; } @@ -141,7 +142,7 @@ int Server::main(const std::vector & /*args*/) { LOG_TRACE(log, "Will mlockall to prevent executable memory from being paged out. It may take a few seconds."); if (0 != mlockall(MCL_CURRENT)) - LOG_WARNING(log, "Failed mlockall: " + errnoToString()); + LOG_WARNING(log, "Failed mlockall: " + errnoToString(ErrorCodes::SYSTEM_ERROR)); else LOG_TRACE(log, "The memory map of clickhouse executable has been mlock'ed"); } diff --git a/dbms/src/Common/CounterInFile.h b/dbms/src/Common/CounterInFile.h index 6ea34362a59..2c831e33302 100644 --- a/dbms/src/Common/CounterInFile.h +++ b/dbms/src/Common/CounterInFile.h @@ -18,7 +18,15 @@ #include -#define SMALL_READ_WRITE_BUFFER_SIZE 16 +namespace DB +{ + namespace ErrorCodes + { + extern const int CANNOT_OPEN_FILE; + extern const int CANNOT_READ_ALL_DATA; + extern const int ATTEMPT_TO_READ_AFTER_EOF; + } +} /** Stores a number in the file. @@ -26,6 +34,9 @@ */ class CounterInFile { +private: + static inline constexpr size_t SMALL_READ_WRITE_BUFFER_SIZE = 16; + public: /// path - the name of the file, including the path CounterInFile(const std::string & path_) : path(path_) {} @@ -56,13 +67,13 @@ public: int fd = ::open(path.c_str(), O_RDWR | O_CREAT, 0666); if (-1 == fd) - DB::throwFromErrno("Cannot open file " + path); + DB::throwFromErrno("Cannot open file " + path, DB::ErrorCodes::CANNOT_OPEN_FILE); try { int flock_ret = flock(fd, LOCK_EX); if (-1 == flock_ret) - DB::throwFromErrno("Cannot lock file " + path); + DB::throwFromErrno("Cannot lock file " + path, DB::ErrorCodes::CANNOT_OPEN_FILE); if (!file_doesnt_exists) { @@ -130,7 +141,7 @@ public: int fd = ::open(path.c_str(), O_RDWR | O_CREAT, 0666); if (-1 == fd) - DB::throwFromErrno("Cannot open file " + path); + DB::throwFromErrno("Cannot open file " + path, DB::ErrorCodes::CANNOT_OPEN_FILE); try { @@ -178,6 +189,3 @@ private: std::string path; std::mutex mutex; }; - - -#undef SMALL_READ_WRITE_BUFFER_SIZE diff --git a/dbms/src/Common/ErrorCodes.cpp b/dbms/src/Common/ErrorCodes.cpp index 3560bcee9af..eb87e4f75cd 100644 --- a/dbms/src/Common/ErrorCodes.cpp +++ b/dbms/src/Common/ErrorCodes.cpp @@ -397,6 +397,9 @@ namespace ErrorCodes extern const int CANNOT_UPDATE_COLUMN = 420; extern const int CANNOT_ADD_DIFFERENT_AGGREGATE_STATES = 421; extern const int UNSUPPORTED_URI_SCHEME = 422; + extern const int CANNOT_GETTIMEOFDAY = 423; + extern const int CANNOT_LINK = 424; + extern const int SYSTEM_ERROR = 425; extern const int KEEPER_EXCEPTION = 999; extern const int POCO_EXCEPTION = 1000; diff --git a/dbms/src/Common/Exception.cpp b/dbms/src/Common/Exception.cpp index 606f180e9d7..a7bfbd64424 100644 --- a/dbms/src/Common/Exception.cpp +++ b/dbms/src/Common/Exception.cpp @@ -52,7 +52,7 @@ std::string errnoToString(int code, int e) void throwFromErrno(const std::string & s, int code, int e) { - throw ErrnoException(s + ", " + errnoToString(code, e)); + throw ErrnoException(s + ", " + errnoToString(code, e), code, e); } void tryLogCurrentException(const char * log_name, const std::string & start_of_message) diff --git a/dbms/src/Common/Exception.h b/dbms/src/Common/Exception.h index 4e3bdc8bafe..37fa92e4515 100644 --- a/dbms/src/Common/Exception.h +++ b/dbms/src/Common/Exception.h @@ -42,11 +42,11 @@ private: class ErrnoException : public Exception { public: - ErrnoException(const std::string & msg, int code = 0, int saved_errno_ = 0) + ErrnoException(const std::string & msg, int code, int saved_errno_) : Exception(msg, code), saved_errno(saved_errno_) {} - ErrnoException(const std::string & msg, const std::string & arg, int code = 0, int saved_errno_ = 0) + ErrnoException(const std::string & msg, const std::string & arg, int code, int saved_errno_) : Exception(msg, arg, code), saved_errno(saved_errno_) {} - ErrnoException(const std::string & msg, const Exception & exc, int code = 0, int saved_errno_ = 0) + ErrnoException(const std::string & msg, const Exception & exc, int code, int saved_errno_) : Exception(msg, exc, code), saved_errno(saved_errno_) {} int getErrno() const { return saved_errno; } @@ -59,8 +59,8 @@ private: using Exceptions = std::vector; -std::string errnoToString(int code = 0, int the_errno = errno); -[[noreturn]] void throwFromErrno(const std::string & s, int code = 0, int the_errno = errno); +std::string errnoToString(int code, int the_errno = errno); +[[noreturn]] void throwFromErrno(const std::string & s, int code, int the_errno = errno); /** Try to write an exception to the log (and forget about it). diff --git a/dbms/src/Common/StatusFile.cpp b/dbms/src/Common/StatusFile.cpp index 84b1edc922b..71cc65fe176 100644 --- a/dbms/src/Common/StatusFile.cpp +++ b/dbms/src/Common/StatusFile.cpp @@ -20,6 +20,14 @@ namespace DB { +namespace ErrorCodes +{ + extern const int CANNOT_OPEN_FILE; + extern const int CANNOT_CLOSE_FILE; + extern const int CANNOT_TRUNCATE_FILE; + extern const int CANNOT_SEEK_THROUGH_FILE; +} + StatusFile::StatusFile(const std::string & path_) : path(path_) @@ -43,7 +51,7 @@ StatusFile::StatusFile(const std::string & path_) fd = ::open(path.c_str(), O_WRONLY | O_CREAT, 0666); if (-1 == fd) - throwFromErrno("Cannot open file " + path); + throwFromErrno("Cannot open file " + path, ErrorCodes::CANNOT_OPEN_FILE); try { @@ -53,14 +61,14 @@ StatusFile::StatusFile(const std::string & path_) if (errno == EWOULDBLOCK) throw Exception("Cannot lock file " + path + ". Another server instance in same directory is already running."); else - throwFromErrno("Cannot lock file " + path); + throwFromErrno("Cannot lock file " + path, ErrorCodes::CANNOT_OPEN_FILE); } if (0 != ftruncate(fd, 0)) - throwFromErrno("Cannot ftruncate " + path); + throwFromErrno("Cannot ftruncate " + path, ErrorCodes::CANNOT_TRUNCATE_FILE); if (0 != lseek(fd, 0, SEEK_SET)) - throwFromErrno("Cannot lseek " + path); + throwFromErrno("Cannot lseek " + path, ErrorCodes::CANNOT_SEEK_THROUGH_FILE); /// Write information about current server instance to the file. { @@ -82,10 +90,10 @@ StatusFile::StatusFile(const std::string & path_) StatusFile::~StatusFile() { if (0 != close(fd)) - LOG_ERROR(&Logger::get("StatusFile"), "Cannot close file " << path << ", " << errnoToString()); + LOG_ERROR(&Logger::get("StatusFile"), "Cannot close file " << path << ", " << errnoToString(ErrorCodes::CANNOT_CLOSE_FILE)); if (0 != unlink(path.c_str())) - LOG_ERROR(&Logger::get("StatusFile"), "Cannot unlink file " << path << ", " << errnoToString()); + LOG_ERROR(&Logger::get("StatusFile"), "Cannot unlink file " << path << ", " << errnoToString(ErrorCodes::CANNOT_CLOSE_FILE)); } } diff --git a/dbms/src/Common/createHardLink.cpp b/dbms/src/Common/createHardLink.cpp index 06647cd437c..824b7e99086 100644 --- a/dbms/src/Common/createHardLink.cpp +++ b/dbms/src/Common/createHardLink.cpp @@ -8,6 +8,12 @@ namespace DB { +namespace ErrorCodes +{ + extern const int CANNOT_STAT; + extern const int CANNOT_LINK; +} + void createHardLink(const String & source_path, const String & destination_path) { if (0 != link(source_path.c_str(), destination_path.c_str())) @@ -20,16 +26,16 @@ void createHardLink(const String & source_path, const String & destination_path) struct stat destination_descr; if (0 != lstat(source_path.c_str(), &source_descr)) - throwFromErrno("Cannot stat " + source_path); + throwFromErrno("Cannot stat " + source_path, ErrorCodes::CANNOT_STAT); if (0 != lstat(destination_path.c_str(), &destination_descr)) - throwFromErrno("Cannot stat " + destination_path); + throwFromErrno("Cannot stat " + destination_path, ErrorCodes::CANNOT_STAT); if (source_descr.st_ino != destination_descr.st_ino) - throwFromErrno("Destination file " + destination_path + " is already exist and have different inode.", 0, link_errno); + throwFromErrno("Destination file " + destination_path + " is already exist and have different inode.", ErrorCodes::CANNOT_LINK, link_errno); } else - throwFromErrno("Cannot link " + source_path + " to " + destination_path); + throwFromErrno("Cannot link " + source_path + " to " + destination_path, ErrorCodes::CANNOT_LINK); } } diff --git a/dbms/src/Common/setThreadName.cpp b/dbms/src/Common/setThreadName.cpp index fc228d78ff4..d035822974a 100644 --- a/dbms/src/Common/setThreadName.cpp +++ b/dbms/src/Common/setThreadName.cpp @@ -32,7 +32,7 @@ void setThreadName(const char * name) #else if (0 != prctl(PR_SET_NAME, name, 0, 0, 0)) #endif - DB::throwFromErrno("Cannot set thread name with prctl(PR_SET_NAME, ...)"); + DB::throwFromErrno("Cannot set thread name with prctl(PR_SET_NAME, ...)", DB::ErrorCodes::PTHREAD_ERROR); } std::string getThreadName() @@ -48,7 +48,7 @@ std::string getThreadName() // throw DB::Exception("Cannot get thread name with pthread_get_name_np()", DB::ErrorCodes::PTHREAD_ERROR); #else if (0 != prctl(PR_GET_NAME, name.data(), 0, 0, 0)) - DB::throwFromErrno("Cannot get thread name with prctl(PR_GET_NAME)"); + DB::throwFromErrno("Cannot get thread name with prctl(PR_GET_NAME)", DB::ErrorCodes::PTHREAD_ERROR); #endif name.resize(std::strlen(name.data())); diff --git a/dbms/src/Common/tests/arena_with_free_lists.cpp b/dbms/src/Common/tests/arena_with_free_lists.cpp index 0a16eff6c17..5091551b550 100644 --- a/dbms/src/Common/tests/arena_with_free_lists.cpp +++ b/dbms/src/Common/tests/arena_with_free_lists.cpp @@ -25,6 +25,14 @@ using namespace DB; +namespace DB +{ + namespace ErrorCodes + { + extern const int SYSTEM_ERROR; + } +} + /// Implementation of ArenaWithFreeLists, which contains a bug. Used to reproduce the bug. #if USE_BAD_ARENA @@ -237,7 +245,7 @@ int main(int argc, char ** argv) rusage resource_usage; if (0 != getrusage(RUSAGE_SELF, &resource_usage)) - throwFromErrno("Cannot getrusage"); + throwFromErrno("Cannot getrusage", ErrorCodes::SYSTEM_ERROR); size_t allocated_bytes = resource_usage.ru_maxrss * 1024; std::cerr << "Current memory usage: " << allocated_bytes << " bytes.\n"; diff --git a/dbms/src/Common/tests/thread_creation_latency.cpp b/dbms/src/Common/tests/thread_creation_latency.cpp index ccc499c9a1c..ef910a3e9f3 100644 --- a/dbms/src/Common/tests/thread_creation_latency.cpp +++ b/dbms/src/Common/tests/thread_creation_latency.cpp @@ -22,6 +22,15 @@ void f() { ++x; } void * g(void *) { f(); return {}; } +namespace DB +{ + namespace ErrorCodes + { + extern const int PTHREAD_ERROR; + } +} + + template void test(size_t n, const char * name, F && kernel) { @@ -80,9 +89,9 @@ int main(int argc, char ** argv) { pthread_t thread; if (pthread_create(&thread, nullptr, g, nullptr)) - DB::throwFromErrno("Cannot create thread."); + DB::throwFromErrno("Cannot create thread.", DB::ErrorCodes::PTHREAD_ERROR); if (pthread_join(thread, nullptr)) - DB::throwFromErrno("Cannot join thread."); + DB::throwFromErrno("Cannot join thread.", DB::ErrorCodes::PTHREAD_ERROR); }); test(n, "Create and destroy std::thread each iteration", [] diff --git a/dbms/src/Storages/Distributed/DistributedBlockOutputStream.cpp b/dbms/src/Storages/Distributed/DistributedBlockOutputStream.cpp index 2464d15e624..319b4c89583 100644 --- a/dbms/src/Storages/Distributed/DistributedBlockOutputStream.cpp +++ b/dbms/src/Storages/Distributed/DistributedBlockOutputStream.cpp @@ -54,6 +54,7 @@ namespace ErrorCodes { extern const int TIMEOUT_EXCEEDED; extern const int TYPE_MISMATCH; + extern const int CANNOT_LINK; } @@ -557,7 +558,7 @@ void DistributedBlockOutputStream::writeToShard(const Block & block, const std:: } if (link(first_file_tmp_path.data(), block_file_path.data())) - throwFromErrno("Could not link " + block_file_path + " to " + first_file_tmp_path); + throwFromErrno("Could not link " + block_file_path + " to " + first_file_tmp_path, ErrorCodes::CANNOT_LINK); } /** remove the temporary file, enabling the OS to reclaim inode after all threads diff --git a/dbms/src/Storages/MergeTree/MergeTreeData.cpp b/dbms/src/Storages/MergeTree/MergeTreeData.cpp index ab2851478c0..ec0e9249bba 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeData.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeData.cpp @@ -79,6 +79,9 @@ namespace ErrorCodes extern const int TOO_MANY_PARTS; extern const int INCOMPATIBLE_COLUMNS; extern const int CANNOT_UPDATE_COLUMN; + extern const int CANNOT_ALLOCATE_MEMORY; + extern const int CANNOT_MUNMAP; + extern const int CANNOT_MREMAP; } @@ -477,7 +480,10 @@ void MergeTreeData::loadDataParts(bool skip_sanity_checks) /// Don't count the part as broken if there is not enough memory to load it. /// In fact, there can be many similar situations. /// But it is OK, because there is a safety guard against deleting too many parts. - if (e.code() == ErrorCodes::MEMORY_LIMIT_EXCEEDED) + if (e.code() == ErrorCodes::MEMORY_LIMIT_EXCEEDED + || e.code() == ErrorCodes::CANNOT_ALLOCATE_MEMORY + || e.code() == ErrorCodes::CANNOT_MUNMAP + || e.code() == ErrorCodes::CANNOT_MREMAP) throw; broken = true; diff --git a/dbms/src/Storages/tests/remove_symlink_directory.cpp b/dbms/src/Storages/tests/remove_symlink_directory.cpp index 59011c557ee..8098ee5dc32 100644 --- a/dbms/src/Storages/tests/remove_symlink_directory.cpp +++ b/dbms/src/Storages/tests/remove_symlink_directory.cpp @@ -5,6 +5,14 @@ #include +namespace DB +{ + namespace ErrorCodes + { + extern const int SYSTEM_ERROR; + } +} + int main(int, char **) try { @@ -14,7 +22,7 @@ try Poco::File("./test_dir/file").createFile(); if (0 != symlink("./test_dir", "./test_link")) - DB::throwFromErrno("Cannot create symlink"); + DB::throwFromErrno("Cannot create symlink", DB::ErrorCodes::SYSTEM_ERROR); Poco::File link("./test_link"); link.renameTo("./test_link2"); diff --git a/libs/libdaemon/include/daemon/BaseDaemon.h b/libs/libdaemon/include/daemon/BaseDaemon.h index 815b03c3b27..65d20927322 100644 --- a/libs/libdaemon/include/daemon/BaseDaemon.h +++ b/libs/libdaemon/include/daemon/BaseDaemon.h @@ -45,7 +45,7 @@ class BaseDaemon : public Poco::Util::ServerApplication friend class SignalListener; public: - static constexpr char DEFAULT_GRAPHITE_CONFIG_NAME[] = "graphite"; + static inline constexpr char DEFAULT_GRAPHITE_CONFIG_NAME[] = "graphite"; BaseDaemon(); ~BaseDaemon() override; diff --git a/libs/libdaemon/src/BaseDaemon.cpp b/libs/libdaemon/src/BaseDaemon.cpp index 34297c937cf..5c1fb503ef2 100644 --- a/libs/libdaemon/src/BaseDaemon.cpp +++ b/libs/libdaemon/src/BaseDaemon.cpp @@ -39,10 +39,8 @@ #include #include #include -#include #include #include -#include #include #include #include @@ -71,20 +69,6 @@ #include -using Poco::Logger; -using Poco::AutoPtr; -using Poco::Observer; -using Poco::FormattingChannel; -using Poco::SplitterChannel; -using Poco::ConsoleChannel; -using Poco::FileChannel; -using Poco::Path; -using Poco::Message; -using Poco::Util::AbstractConfiguration; - - -constexpr char BaseDaemon::DEFAULT_GRAPHITE_CONFIG_NAME[]; - /** For transferring information from signal handler to a separate thread. * If you need to do something serious in case of a signal (example: write a message to the log), * then sending information to a separate thread through pipe and doing all the stuff asynchronously @@ -109,7 +93,7 @@ struct Pipe write_fd = -1; if (0 != pipe(fds)) - DB::throwFromErrno("Cannot create pipe"); + DB::throwFromErrno("Cannot create pipe", 0); } void close() @@ -669,7 +653,7 @@ void BaseDaemon::buildLoggers(Poco::Util::AbstractConfiguration & config) std::cerr << "Logging " << log_level << " to " << log_path << std::endl; // Set up two channel chains. - log_file = new FileChannel; + log_file = new Poco::FileChannel; log_file->setProperty(Poco::FileChannel::PROP_PATH, Poco::Path(log_path).absolute().toString()); log_file->setProperty(Poco::FileChannel::PROP_ROTATION, config.getRawString("logger.size", "100M")); log_file->setProperty(Poco::FileChannel::PROP_ARCHIVE, "number"); @@ -691,7 +675,7 @@ void BaseDaemon::buildLoggers(Poco::Util::AbstractConfiguration & config) createDirectory(errorlog_path); std::cerr << "Logging errors to " << errorlog_path << std::endl; - error_log_file = new FileChannel; + error_log_file = new Poco::FileChannel; error_log_file->setProperty(Poco::FileChannel::PROP_PATH, Poco::Path(errorlog_path).absolute().toString()); error_log_file->setProperty(Poco::FileChannel::PROP_ROTATION, config.getRawString("logger.size", "100M")); error_log_file->setProperty(Poco::FileChannel::PROP_ARCHIVE, "number"); @@ -703,7 +687,7 @@ void BaseDaemon::buildLoggers(Poco::Util::AbstractConfiguration & config) Poco::AutoPtr pf = new OwnPatternFormatter(this); Poco::AutoPtr errorlog = new DB::OwnFormattingChannel(pf, error_log_file); - errorlog->setLevel(Message::PRIO_NOTICE); + errorlog->setLevel(Poco::Message::PRIO_NOTICE); errorlog->open(); split->addChannel(errorlog); } @@ -767,12 +751,12 @@ void BaseDaemon::buildLoggers(Poco::Util::AbstractConfiguration & config) Logger::root().setChannel(logger().getChannel()); // Explicitly specified log levels for specific loggers. - AbstractConfiguration::Keys levels; + Poco::Util::AbstractConfiguration::Keys levels; config.keys("logger.levels", levels); - if(!levels.empty()) - for(AbstractConfiguration::Keys::iterator it = levels.begin(); it != levels.end(); ++it) - Logger::get(*it).setLevel(config.getString("logger.levels." + *it, "trace")); + if (!levels.empty()) + for (const auto & level : levels) + Logger::get(level).setLevel(config.getString("logger.levels." + level, "trace")); } @@ -1077,7 +1061,7 @@ void BaseDaemon::logRevision() const /// Makes server shutdown if at least one Poco::Task have failed. void BaseDaemon::exitOnTaskError() { - Observer obs(*this, &BaseDaemon::handleNotification); + Poco::Observer obs(*this, &BaseDaemon::handleNotification); getTaskManager().addObserver(obs); } @@ -1085,7 +1069,7 @@ void BaseDaemon::exitOnTaskError() void BaseDaemon::handleNotification(Poco::TaskFailedNotification *_tfn) { task_failed = true; - AutoPtr fn(_tfn); + Poco::AutoPtr fn(_tfn); Logger *lg = &(logger()); LOG_ERROR(lg, "Task '" << fn->task()->name() << "' failed. Daemon is shutting down. Reason - " << fn->reason().displayText()); ServerApplication::terminate(); diff --git a/libs/libdaemon/src/ExtendedLogChannel.cpp b/libs/libdaemon/src/ExtendedLogChannel.cpp index 94413f8b8ed..1f517cf5e98 100644 --- a/libs/libdaemon/src/ExtendedLogChannel.cpp +++ b/libs/libdaemon/src/ExtendedLogChannel.cpp @@ -8,13 +8,18 @@ namespace DB { +namespace ErrorCodes +{ + extern const int CANNOT_GETTIMEOFDAY; +} + ExtendedLogMessage ExtendedLogMessage::getFrom(const Poco::Message & base) { ExtendedLogMessage msg_ext(base); ::timeval tv; if (0 != gettimeofday(&tv, nullptr)) - DB::throwFromErrno("Cannot gettimeofday"); + DB::throwFromErrno("Cannot gettimeofday", ErrorCodes::CANNOT_GETTIMEOFDAY); msg_ext.time_seconds = static_cast(tv.tv_sec); msg_ext.time_microseconds = static_cast(tv.tv_usec); diff --git a/utils/iotest/iotest.cpp b/utils/iotest/iotest.cpp index c157e9736d2..3134a49056d 100644 --- a/utils/iotest/iotest.cpp +++ b/utils/iotest/iotest.cpp @@ -17,7 +17,16 @@ #include #include -using DB::throwFromErrno; +namespace DB +{ + namespace ErrorCodes + { + extern const int CANNOT_OPEN_FILE; + extern const int CANNOT_CLOSE_FILE; + extern const int CANNOT_READ_FROM_FILE_DESCRIPTOR; + extern const int CANNOT_WRITE_TO_FILE_DESCRIPTOR; + } +} enum Mode @@ -33,7 +42,9 @@ enum Mode void thread(int fd, int mode, size_t min_offset, size_t max_offset, size_t block_size, size_t count) { - DB::Memory direct_buf(block_size, sysconf(_SC_PAGESIZE)); + using namespace DB; + + Memory direct_buf(block_size, sysconf(_SC_PAGESIZE)); std::vector simple_buf(block_size); char * buf; @@ -60,12 +71,12 @@ void thread(int fd, int mode, size_t min_offset, size_t max_offset, size_t block if (mode & MODE_READ) { if (static_cast(block_size) != pread(fd, buf, block_size, offset)) - throwFromErrno("Cannot read"); + throwFromErrno("Cannot read", ErrorCodes::CANNOT_READ_FROM_FILE_DESCRIPTOR); } else { if (static_cast(block_size) != pwrite(fd, buf, block_size, offset)) - throwFromErrno("Cannot write"); + throwFromErrno("Cannot write", ErrorCodes::CANNOT_WRITE_TO_FILE_DESCRIPTOR); } } } @@ -73,6 +84,8 @@ void thread(int fd, int mode, size_t min_offset, size_t max_offset, size_t block int mainImpl(int argc, char ** argv) { + using namespace DB; + const char * file_name = 0; int mode = MODE_NONE; UInt64 min_offset = 0; @@ -89,11 +102,11 @@ int mainImpl(int argc, char ** argv) } file_name = argv[1]; - min_offset = DB::parse(argv[3]); - max_offset = DB::parse(argv[4]); - block_size = DB::parse(argv[5]); - threads = DB::parse(argv[6]); - count = DB::parse(argv[7]); + min_offset = parse(argv[3]); + max_offset = parse(argv[4]); + block_size = parse(argv[5]); + threads = parse(argv[6]); + count = parse(argv[7]); for (int i = 0; argv[2][i]; ++i) { @@ -128,11 +141,11 @@ int mainImpl(int argc, char ** argv) int fd = open(file_name, ((mode & MODE_READ) ? O_RDONLY : O_WRONLY) | ((mode & MODE_SYNC) ? O_SYNC : 0)); #endif if (-1 == fd) - throwFromErrno("Cannot open file"); + throwFromErrno("Cannot open file", ErrorCodes::CANNOT_OPEN_FILE); #ifdef __APPLE__ if (mode & MODE_DIRECT) if (fcntl(fd, F_NOCACHE, 1) == -1) - throwFromErrno("Cannot open file"); + throwFromErrno("Cannot open file", ErrorCodes::CANNOT_CLOSE_FILE); #endif Stopwatch watch; @@ -145,7 +158,7 @@ int mainImpl(int argc, char ** argv) watch.stop(); if (0 != close(fd)) - throwFromErrno("Cannot close file"); + throwFromErrno("Cannot close file", ErrorCodes::CANNOT_CLOSE_FILE); std::cout << std::fixed << std::setprecision(2) << "Done " << count << " * " << threads << " ops"; diff --git a/utils/iotest/iotest_aio.cpp b/utils/iotest/iotest_aio.cpp index 0274541ec1d..1dcb8ea2ae7 100644 --- a/utils/iotest/iotest_aio.cpp +++ b/utils/iotest/iotest_aio.cpp @@ -29,7 +29,16 @@ int main(int argc, char ** argv) { return 0; } #include -using DB::throwFromErrno; +namespace DB +{ + namespace ErrorCodes + { + extern const int CANNOT_OPEN_FILE; + extern const int CANNOT_CLOSE_FILE; + extern const int CANNOT_IO_SUBMIT; + extern const int CANNOT_IO_GETEVENTS; + } +} enum Mode @@ -41,11 +50,13 @@ enum Mode void thread(int fd, int mode, size_t min_offset, size_t max_offset, size_t block_size, size_t buffers_count, size_t count) { + using namespace DB; + AIOContext ctx; - std::vector buffers(buffers_count); + std::vector buffers(buffers_count); for (size_t i = 0; i < buffers_count; ++i) - buffers[i] = DB::Memory(block_size, sysconf(_SC_PAGESIZE)); + buffers[i] = Memory(block_size, sysconf(_SC_PAGESIZE)); drand48_data rand_data; timespec times; @@ -109,13 +120,13 @@ void thread(int fd, int mode, size_t min_offset, size_t max_offset, size_t block /// Send queries. if (io_submit(ctx.ctx, query_cbs.size(), &query_cbs[0]) < 0) - throwFromErrno("io_submit failed"); + throwFromErrno("io_submit failed", ErrorCodes::CANNOT_IO_SUBMIT); /// Receive answers. If we have something else to send, then receive at least one answer (after that send them), otherwise wait all answers. memset(&events[0], 0, buffers_count * sizeof(events[0])); int evs = io_getevents(ctx.ctx, (blocks_sent < count ? 1 : in_progress), buffers_count, &events[0], nullptr); if (evs < 0) - throwFromErrno("io_getevents failed"); + throwFromErrno("io_getevents failed", ErrorCodes::CANNOT_IO_GETEVENTS); for (int i = 0; i < evs; ++i) { @@ -131,6 +142,8 @@ void thread(int fd, int mode, size_t min_offset, size_t max_offset, size_t block int mainImpl(int argc, char ** argv) { + using namespace DB; + const char * file_name = 0; int mode = MODE_READ; UInt64 min_offset = 0; @@ -149,16 +162,16 @@ int mainImpl(int argc, char ** argv) file_name = argv[1]; if (argv[2][0] == 'w') mode = MODE_WRITE; - min_offset = DB::parse(argv[3]); - max_offset = DB::parse(argv[4]); - block_size = DB::parse(argv[5]); - threads_count = DB::parse(argv[6]); - buffers_count = DB::parse(argv[7]); - count = DB::parse(argv[8]); + min_offset = parse(argv[3]); + max_offset = parse(argv[4]); + block_size = parse(argv[5]); + threads_count = parse(argv[6]); + buffers_count = parse(argv[7]); + count = parse(argv[8]); int fd = open(file_name, ((mode == MODE_READ) ? O_RDONLY : O_WRONLY) | O_DIRECT); if (-1 == fd) - throwFromErrno("Cannot open file"); + throwFromErrno("Cannot open file", ErrorCodes::CANNOT_OPEN_FILE); ThreadPool pool(threads_count); @@ -171,7 +184,7 @@ int mainImpl(int argc, char ** argv) watch.stop(); if (0 != close(fd)) - throwFromErrno("Cannot close file"); + throwFromErrno("Cannot close file", ErrorCodes::CANNOT_CLOSE_FILE); std::cout << std::fixed << std::setprecision(2) << "Done " << count << " * " << threads_count << " ops"; diff --git a/utils/iotest/iotest_nonblock.cpp b/utils/iotest/iotest_nonblock.cpp index 45ab62f60c1..f85e8df91f6 100644 --- a/utils/iotest/iotest_nonblock.cpp +++ b/utils/iotest/iotest_nonblock.cpp @@ -20,7 +20,18 @@ #include #include -using DB::throwFromErrno; +namespace DB +{ + namespace ErrorCodes + { + extern const int CANNOT_OPEN_FILE; + extern const int CANNOT_CLOSE_FILE; + extern const int CANNOT_READ_FROM_FILE_DESCRIPTOR; + extern const int CANNOT_WRITE_TO_FILE_DESCRIPTOR; + extern const int CANNOT_FSYNC; + extern const int SYSTEM_ERROR; + } +} enum Mode @@ -32,6 +43,8 @@ enum Mode int mainImpl(int argc, char ** argv) { + using namespace DB; + const char * file_name = 0; Mode mode = MODE_READ; UInt64 min_offset = 0; @@ -47,11 +60,11 @@ int mainImpl(int argc, char ** argv) } file_name = argv[1]; - min_offset = DB::parse(argv[3]); - max_offset = DB::parse(argv[4]); - block_size = DB::parse(argv[5]); - descriptors = DB::parse(argv[6]); - count = DB::parse(argv[7]); + min_offset = parse(argv[3]); + max_offset = parse(argv[4]); + block_size = parse(argv[5]); + descriptors = parse(argv[6]); + count = parse(argv[7]); if (!strcmp(argv[2], "r")) mode = MODE_READ; @@ -65,7 +78,7 @@ int mainImpl(int argc, char ** argv) { fds[i] = open(file_name, O_SYNC | ((mode == MODE_READ) ? O_RDONLY : O_WRONLY)); if (-1 == fds[i]) - throwFromErrno("Cannot open file"); + throwFromErrno("Cannot open file", ErrorCodes::CANNOT_OPEN_FILE); } std::vector buf(block_size); @@ -87,7 +100,7 @@ int mainImpl(int argc, char ** argv) while (ops < count) { if (poll(&polls[0], descriptors, -1) <= 0) - throwFromErrno("poll failed"); + throwFromErrno("poll failed", ErrorCodes::SYSTEM_ERROR); for (size_t i = 0; i < descriptors; ++i) { if (!polls[i].revents) @@ -109,12 +122,12 @@ int mainImpl(int argc, char ** argv) if (mode == MODE_READ) { if (static_cast(block_size) != pread(fds[i], &buf[0], block_size, offset)) - throwFromErrno("Cannot read"); + throwFromErrno("Cannot read", ErrorCodes::CANNOT_READ_FROM_FILE_DESCRIPTOR); } else { if (static_cast(block_size) != pwrite(fds[i], &buf[0], block_size, offset)) - throwFromErrno("Cannot write"); + throwFromErrno("Cannot write", ErrorCodes::CANNOT_WRITE_TO_FILE_DESCRIPTOR); } } } @@ -122,7 +135,7 @@ int mainImpl(int argc, char ** argv) for (size_t i = 0; i < descriptors; ++i) { if (fsync(fds[i])) - throwFromErrno("Cannot fsync"); + throwFromErrno("Cannot fsync", ErrorCodes::CANNOT_FSYNC); } watch.stop(); @@ -130,7 +143,7 @@ int mainImpl(int argc, char ** argv) for (size_t i = 0; i < descriptors; ++i) { if (0 != close(fds[i])) - throwFromErrno("Cannot close file"); + throwFromErrno("Cannot close file", ErrorCodes::CANNOT_CLOSE_FILE); } std::cout << std::fixed << std::setprecision(2) From ed35e7cdbb56585625082cd442d2e65bb36a2de9 Mon Sep 17 00:00:00 2001 From: proller Date: Thu, 22 Nov 2018 00:11:11 +0300 Subject: [PATCH 040/124] Fix links in CHANGELOG.md --- CHANGELOG.md | 4 ++-- CHANGELOG_RU.md | 4 ++-- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index 33c569d2a88..eba46e0dfd0 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -1,10 +1,10 @@ ## ClickHouse release 18.14.14, 2018-11-20 ### Bug fixes: -* Fixed `ON CLUSTER` queries when cluster configured as secure (flag ``). [#3599] +* Fixed `ON CLUSTER` queries when cluster configured as secure (flag ``). [#3599](https://github.com/yandex/ClickHouse/pull/3599) ### Build changes: -* Fixed problems (llvm-7 from system, macos) [#3582] +* Fixed problems (llvm-7 from system, macos) [#3582](https://github.com/yandex/ClickHouse/pull/3582) ## ClickHouse release 18.14.11, 2018-10-29 diff --git a/CHANGELOG_RU.md b/CHANGELOG_RU.md index 3a4646ff5bc..b946be4b608 100644 --- a/CHANGELOG_RU.md +++ b/CHANGELOG_RU.md @@ -1,10 +1,10 @@ ## ClickHouse release 18.14.14, 2018-11-20 ### Исправления ошибок: -* Исправлена работа запросов `ON CLUSTER` в случае, когда в конфигурации кластера включено шифрование (флаг ``). [#3599] +* Исправлена работа запросов `ON CLUSTER` в случае, когда в конфигурации кластера включено шифрование (флаг ``). [#3599](https://github.com/yandex/ClickHouse/pull/3599) ### Улучшения процесса сборки ClickHouse: -* Испрпавлены проблемы сборки (llvm-7 из системы, macos) [#3582] +* Испрпавлены проблемы сборки (llvm-7 из системы, macos) [#3582](https://github.com/yandex/ClickHouse/pull/3582) ## ClickHouse release 18.14.13, 2018-11-08 From 41951a926e0218f5a6d55bc7e6ac39b2e09494f3 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 22 Nov 2018 12:18:13 +0300 Subject: [PATCH 041/124] Fix test cluster config. --- dbms/programs/server/config.xml | 14 ++++++++++++++ 1 file changed, 14 insertions(+) diff --git a/dbms/programs/server/config.xml b/dbms/programs/server/config.xml index 051f6f7fb2b..fe965902f93 100644 --- a/dbms/programs/server/config.xml +++ b/dbms/programs/server/config.xml @@ -164,6 +164,20 @@ + + + + localhost + 59000 + + + + + localhost + 59000 + + + From b604761a941711c52c474e19383e98c8e8303eda Mon Sep 17 00:00:00 2001 From: alesapin Date: Thu, 22 Nov 2018 15:14:55 +0300 Subject: [PATCH 042/124] CLICKHOUSE-4128: Add unbundled option to packager --- docker/packager/deb/Dockerfile | 1 + docker/packager/packager | 7 +++++-- docker/test/stateless/Dockerfile | 2 +- 3 files changed, 7 insertions(+), 3 deletions(-) diff --git a/docker/packager/deb/Dockerfile b/docker/packager/deb/Dockerfile index 2f7e3d8d348..f7dfcc2513b 100644 --- a/docker/packager/deb/Dockerfile +++ b/docker/packager/deb/Dockerfile @@ -42,6 +42,7 @@ RUN apt-get update -y \ libsparsehash-dev \ libgoogle-perftools-dev \ libzstd-dev \ + libltdl-dev \ libre2-dev \ libjemalloc-dev \ unixodbc-dev \ diff --git a/docker/packager/packager b/docker/packager/packager index c0006c7b0bb..38c4dc107c9 100755 --- a/docker/packager/packager +++ b/docker/packager/packager @@ -52,7 +52,7 @@ def run_image_with_env(image_name, output, env_variables, ch_root): subprocess.check_call(cmd, shell=True) -def parse_env_variables(build_type, compiler, sanitizer, package_type, cache, distcc_hosts): +def parse_env_variables(build_type, compiler, sanitizer, package_type, cache, distcc_hosts, unbundled): result = [] if package_type == "deb": result.append("DEB_CC={}".format(compiler)) @@ -75,6 +75,8 @@ def parse_env_variables(build_type, compiler, sanitizer, package_type, cache, di elif cache == "distcc": result.append('DISTCC_HOSTS="{}"'.format("localhost/`nproc`")) + if unbundled: + result.append('CMAKE_FLAGS="-DUNBUNDLED=1 -DENABLE_MYSQL=0 -DENABLE_POCO_ODBC=0 -DENABLE_ODBC=0 $CMAKE_FLAGS"') return result if __name__ == "__main__": @@ -86,6 +88,7 @@ if __name__ == "__main__": parser.add_argument("--build-type", choices=("debug", ""), default="") parser.add_argument("--compiler", choices=("clang-6.0", "gcc-7", "gcc-8"), default="gcc-7") parser.add_argument("--sanitizer", choices=("address", "thread", "memory", "undefined", ""), default="") + parser.add_argument("--unbundled", action="store_true") parser.add_argument("--cache", choices=("", "ccache", "distcc"), default="") parser.add_argument("--distcc-hosts", nargs="+") parser.add_argument("--force-build-image", action="store_true") @@ -105,6 +108,6 @@ if __name__ == "__main__": if not check_image_exists_locally(image_name) or args.force_build_image: if not pull_image(image_name) or args.force_build_image: build_image(image_name, dockerfile) - env_prepared = parse_env_variables(args.build_type, args.compiler, args.sanitizer, args.package_type, args.cache, args.distcc_hosts) + env_prepared = parse_env_variables(args.build_type, args.compiler, args.sanitizer, args.package_type, args.cache, args.distcc_hosts, args.unbundled) run_image_with_env(image_name, args.output_dir, env_prepared, ch_root) logging.info("Output placed into {}".format(args.output_dir)) diff --git a/docker/test/stateless/Dockerfile b/docker/test/stateless/Dockerfile index 5664a16078e..9cd0a2ba970 100644 --- a/docker/test/stateless/Dockerfile +++ b/docker/test/stateless/Dockerfile @@ -1,4 +1,4 @@ -FROM ubuntu:18.04 +FROM yandex/clickhouse-deb-builder RUN apt-get update -y \ && env DEBIAN_FRONTEND=noninteractive \ From 060fce3899f475a5ae4b99ed07c6dcdde946d495 Mon Sep 17 00:00:00 2001 From: Sabyanin Maxim Date: Thu, 22 Nov 2018 16:51:43 +0300 Subject: [PATCH 043/124] make ReplicatedMergeTree work with comments. ignore empty comments when make column_comments hashmap --- .../Interpreters/InterpreterCreateQuery.cpp | 4 +- dbms/src/Storages/ColumnsDescription.h | 1 + ...cated_merge_tree_alter_zookeeper.reference | 284 +++++++++--------- .../00643_cast_zookeeper.reference | 4 +- .../00753_comment_columns_zookeeper.reference | 4 + .../00753_comment_columns_zookeeper.sql | 19 ++ 6 files changed, 170 insertions(+), 146 deletions(-) create mode 100644 dbms/tests/queries/0_stateless/00753_comment_columns_zookeeper.reference create mode 100644 dbms/tests/queries/0_stateless/00753_comment_columns_zookeeper.sql diff --git a/dbms/src/Interpreters/InterpreterCreateQuery.cpp b/dbms/src/Interpreters/InterpreterCreateQuery.cpp index d7a223194b5..1a1cefd6d16 100644 --- a/dbms/src/Interpreters/InterpreterCreateQuery.cpp +++ b/dbms/src/Interpreters/InterpreterCreateQuery.cpp @@ -223,8 +223,8 @@ static ParsedColumns parseColumns(const ASTExpressionList & column_list_ast, con if (col_decl.comment) { - auto comment_literal = typeid_cast(*col_decl.comment); - comments.emplace(col_decl.name, comment_literal.value.get()); + if (auto comment_str = typeid_cast(*col_decl.comment).value.get(); !comment_str.empty()) + comments.emplace(col_decl.name, comment_str); } } diff --git a/dbms/src/Storages/ColumnsDescription.h b/dbms/src/Storages/ColumnsDescription.h index 89457421364..2d35ac403c6 100644 --- a/dbms/src/Storages/ColumnsDescription.h +++ b/dbms/src/Storages/ColumnsDescription.h @@ -9,6 +9,7 @@ namespace DB { +/// key-values column_name, column_comment. column_comment should be non empty. using ColumnComments = std::unordered_map; struct ColumnsDescription diff --git a/dbms/tests/queries/0_stateless/00062_replicated_merge_tree_alter_zookeeper.reference b/dbms/tests/queries/0_stateless/00062_replicated_merge_tree_alter_zookeeper.reference index d2a95113dd2..cd371656965 100644 --- a/dbms/tests/queries/0_stateless/00062_replicated_merge_tree_alter_zookeeper.reference +++ b/dbms/tests/queries/0_stateless/00062_replicated_merge_tree_alter_zookeeper.reference @@ -1,213 +1,213 @@ -d Date -k UInt64 -i32 Int32 +d Date +k UInt64 +i32 Int32 CREATE TABLE test.replicated_alter1 ( d Date, k UInt64, i32 Int32) ENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test/alter\', \'r1\', d, k, 8192) -d Date -k UInt64 -i32 Int32 +d Date +k UInt64 +i32 Int32 CREATE TABLE test.replicated_alter2 ( d Date, k UInt64, i32 Int32) ENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test/alter\', \'r2\', d, k, 8192) 2015-01-01 10 42 -d Date -k UInt64 -i32 Int32 -dt DateTime +d Date +k UInt64 +i32 Int32 +dt DateTime CREATE TABLE test.replicated_alter1 ( d Date, k UInt64, i32 Int32, dt DateTime) ENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test/alter\', \'r1\', d, k, 8192) -d Date -k UInt64 -i32 Int32 -dt DateTime +d Date +k UInt64 +i32 Int32 +dt DateTime CREATE TABLE test.replicated_alter2 ( d Date, k UInt64, i32 Int32, dt DateTime) ENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test/alter\', \'r2\', d, k, 8192) 2015-01-01 9 41 1992-01-01 08:00:00 2015-01-01 10 42 0000-00-00 00:00:00 -d Date -k UInt64 -i32 Int32 -dt DateTime -n.ui8 Array(UInt8) -n.s Array(String) +d Date +k UInt64 +i32 Int32 +dt DateTime +n.ui8 Array(UInt8) +n.s Array(String) CREATE TABLE test.replicated_alter1 ( d Date, k UInt64, i32 Int32, dt DateTime, `n.ui8` Array(UInt8), `n.s` Array(String)) ENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test/alter\', \'r1\', d, k, 8192) -d Date -k UInt64 -i32 Int32 -dt DateTime -n.ui8 Array(UInt8) -n.s Array(String) +d Date +k UInt64 +i32 Int32 +dt DateTime +n.ui8 Array(UInt8) +n.s Array(String) CREATE TABLE test.replicated_alter2 ( d Date, k UInt64, i32 Int32, dt DateTime, `n.ui8` Array(UInt8), `n.s` Array(String)) ENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test/alter\', \'r2\', d, k, 8192) 2015-01-01 8 40 2012-12-12 12:12:12 [1,2,3] ['12','13','14'] 2015-01-01 9 41 1992-01-01 08:00:00 [] [] 2015-01-01 10 42 0000-00-00 00:00:00 [] [] -d Date -k UInt64 -i32 Int32 -dt DateTime -n.ui8 Array(UInt8) -n.s Array(String) -n.d Array(Date) +d Date +k UInt64 +i32 Int32 +dt DateTime +n.ui8 Array(UInt8) +n.s Array(String) +n.d Array(Date) CREATE TABLE test.replicated_alter1 ( d Date, k UInt64, i32 Int32, dt DateTime, `n.ui8` Array(UInt8), `n.s` Array(String), `n.d` Array(Date)) ENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test/alter\', \'r1\', d, k, 8192) -d Date -k UInt64 -i32 Int32 -dt DateTime -n.ui8 Array(UInt8) -n.s Array(String) -n.d Array(Date) +d Date +k UInt64 +i32 Int32 +dt DateTime +n.ui8 Array(UInt8) +n.s Array(String) +n.d Array(Date) CREATE TABLE test.replicated_alter2 ( d Date, k UInt64, i32 Int32, dt DateTime, `n.ui8` Array(UInt8), `n.s` Array(String), `n.d` Array(Date)) ENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test/alter\', \'r2\', d, k, 8192) 2015-01-01 7 39 2014-07-14 13:26:50 [10,20,30] ['120','130','140'] ['2000-01-01','2000-01-01','2000-01-03'] 2015-01-01 8 40 2012-12-12 12:12:12 [1,2,3] ['12','13','14'] ['0000-00-00','0000-00-00','0000-00-00'] 2015-01-01 9 41 1992-01-01 08:00:00 [] [] [] 2015-01-01 10 42 0000-00-00 00:00:00 [] [] [] -d Date -k UInt64 -i32 Int32 -dt DateTime -n.ui8 Array(UInt8) -n.s Array(String) -n.d Array(Date) -s String DEFAULT \'0\' +d Date +k UInt64 +i32 Int32 +dt DateTime +n.ui8 Array(UInt8) +n.s Array(String) +n.d Array(Date) +s String DEFAULT \'0\' CREATE TABLE test.replicated_alter1 ( d Date, k UInt64, i32 Int32, dt DateTime, `n.ui8` Array(UInt8), `n.s` Array(String), `n.d` Array(Date), s String DEFAULT \'0\') ENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test/alter\', \'r1\', d, k, 8192) -d Date -k UInt64 -i32 Int32 -dt DateTime -n.ui8 Array(UInt8) -n.s Array(String) -n.d Array(Date) -s String DEFAULT \'0\' +d Date +k UInt64 +i32 Int32 +dt DateTime +n.ui8 Array(UInt8) +n.s Array(String) +n.d Array(Date) +s String DEFAULT \'0\' CREATE TABLE test.replicated_alter2 ( d Date, k UInt64, i32 Int32, dt DateTime, `n.ui8` Array(UInt8), `n.s` Array(String), `n.d` Array(Date), s String DEFAULT \'0\') ENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test/alter\', \'r2\', d, k, 8192) 2015-01-01 6 38 2014-07-15 13:26:50 [10,20,30] ['asd','qwe','qwe'] ['2000-01-01','2000-01-01','2000-01-03'] 100500 2015-01-01 7 39 2014-07-14 13:26:50 [10,20,30] ['120','130','140'] ['2000-01-01','2000-01-01','2000-01-03'] 0 2015-01-01 8 40 2012-12-12 12:12:12 [1,2,3] ['12','13','14'] ['0000-00-00','0000-00-00','0000-00-00'] 0 2015-01-01 9 41 1992-01-01 08:00:00 [] [] [] 0 2015-01-01 10 42 0000-00-00 00:00:00 [] [] [] 0 -d Date -k UInt64 -i32 Int32 -dt DateTime -n.ui8 Array(UInt8) -n.s Array(String) -s Int64 +d Date +k UInt64 +i32 Int32 +dt DateTime +n.ui8 Array(UInt8) +n.s Array(String) +s Int64 CREATE TABLE test.replicated_alter1 ( d Date, k UInt64, i32 Int32, dt DateTime, `n.ui8` Array(UInt8), `n.s` Array(String), s Int64) ENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test/alter\', \'r1\', d, k, 8192) -d Date -k UInt64 -i32 Int32 -dt DateTime -n.ui8 Array(UInt8) -n.s Array(String) -s Int64 +d Date +k UInt64 +i32 Int32 +dt DateTime +n.ui8 Array(UInt8) +n.s Array(String) +s Int64 CREATE TABLE test.replicated_alter2 ( d Date, k UInt64, i32 Int32, dt DateTime, `n.ui8` Array(UInt8), `n.s` Array(String), s Int64) ENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test/alter\', \'r2\', d, k, 8192) 2015-01-01 6 38 2014-07-15 13:26:50 [10,20,30] ['asd','qwe','qwe'] 100500 2015-01-01 7 39 2014-07-14 13:26:50 [10,20,30] ['120','130','140'] 0 2015-01-01 8 40 2012-12-12 12:12:12 [1,2,3] ['12','13','14'] 0 2015-01-01 9 41 1992-01-01 08:00:00 [] [] 0 2015-01-01 10 42 0000-00-00 00:00:00 [] [] 0 -d Date -k UInt64 -i32 Int32 -dt DateTime -n.ui8 Array(UInt8) -n.s Array(String) -s UInt32 -n.d Array(Date) +d Date +k UInt64 +i32 Int32 +dt DateTime +n.ui8 Array(UInt8) +n.s Array(String) +s UInt32 +n.d Array(Date) CREATE TABLE test.replicated_alter1 ( d Date, k UInt64, i32 Int32, dt DateTime, `n.ui8` Array(UInt8), `n.s` Array(String), s UInt32, `n.d` Array(Date)) ENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test/alter\', \'r1\', d, k, 8192) -d Date -k UInt64 -i32 Int32 -dt DateTime -n.ui8 Array(UInt8) -n.s Array(String) -s UInt32 -n.d Array(Date) +d Date +k UInt64 +i32 Int32 +dt DateTime +n.ui8 Array(UInt8) +n.s Array(String) +s UInt32 +n.d Array(Date) CREATE TABLE test.replicated_alter2 ( d Date, k UInt64, i32 Int32, dt DateTime, `n.ui8` Array(UInt8), `n.s` Array(String), s UInt32, `n.d` Array(Date)) ENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test/alter\', \'r2\', d, k, 8192) 2015-01-01 6 38 2014-07-15 13:26:50 [10,20,30] ['asd','qwe','qwe'] 100500 ['0000-00-00','0000-00-00','0000-00-00'] 2015-01-01 7 39 2014-07-14 13:26:50 [10,20,30] ['120','130','140'] 0 ['0000-00-00','0000-00-00','0000-00-00'] 2015-01-01 8 40 2012-12-12 12:12:12 [1,2,3] ['12','13','14'] 0 ['0000-00-00','0000-00-00','0000-00-00'] 2015-01-01 9 41 1992-01-01 08:00:00 [] [] 0 [] 2015-01-01 10 42 0000-00-00 00:00:00 [] [] 0 [] -d Date -k UInt64 -i32 Int32 -dt DateTime -n.s Array(String) -s UInt32 +d Date +k UInt64 +i32 Int32 +dt DateTime +n.s Array(String) +s UInt32 CREATE TABLE test.replicated_alter1 ( d Date, k UInt64, i32 Int32, dt DateTime, `n.s` Array(String), s UInt32) ENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test/alter\', \'r1\', d, k, 8192) -d Date -k UInt64 -i32 Int32 -dt DateTime -n.s Array(String) -s UInt32 +d Date +k UInt64 +i32 Int32 +dt DateTime +n.s Array(String) +s UInt32 CREATE TABLE test.replicated_alter2 ( d Date, k UInt64, i32 Int32, dt DateTime, `n.s` Array(String), s UInt32) ENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test/alter\', \'r2\', d, k, 8192) 2015-01-01 6 38 2014-07-15 13:26:50 ['asd','qwe','qwe'] 100500 2015-01-01 7 39 2014-07-14 13:26:50 ['120','130','140'] 0 2015-01-01 8 40 2012-12-12 12:12:12 ['12','13','14'] 0 2015-01-01 9 41 1992-01-01 08:00:00 [] 0 2015-01-01 10 42 0000-00-00 00:00:00 [] 0 -d Date -k UInt64 -i32 Int32 -dt DateTime -s UInt32 +d Date +k UInt64 +i32 Int32 +dt DateTime +s UInt32 CREATE TABLE test.replicated_alter1 ( d Date, k UInt64, i32 Int32, dt DateTime, s UInt32) ENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test/alter\', \'r1\', d, k, 8192) -d Date -k UInt64 -i32 Int32 -dt DateTime -s UInt32 +d Date +k UInt64 +i32 Int32 +dt DateTime +s UInt32 CREATE TABLE test.replicated_alter2 ( d Date, k UInt64, i32 Int32, dt DateTime, s UInt32) ENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test/alter\', \'r2\', d, k, 8192) 2015-01-01 6 38 2014-07-15 13:26:50 100500 2015-01-01 7 39 2014-07-14 13:26:50 0 2015-01-01 8 40 2012-12-12 12:12:12 0 2015-01-01 9 41 1992-01-01 08:00:00 0 2015-01-01 10 42 0000-00-00 00:00:00 0 -d Date -k UInt64 -i32 Int32 -dt DateTime -s UInt32 -n.s Array(String) -n.d Array(Date) +d Date +k UInt64 +i32 Int32 +dt DateTime +s UInt32 +n.s Array(String) +n.d Array(Date) CREATE TABLE test.replicated_alter1 ( d Date, k UInt64, i32 Int32, dt DateTime, s UInt32, `n.s` Array(String), `n.d` Array(Date)) ENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test/alter\', \'r1\', d, k, 8192) -d Date -k UInt64 -i32 Int32 -dt DateTime -s UInt32 -n.s Array(String) -n.d Array(Date) +d Date +k UInt64 +i32 Int32 +dt DateTime +s UInt32 +n.s Array(String) +n.d Array(Date) CREATE TABLE test.replicated_alter2 ( d Date, k UInt64, i32 Int32, dt DateTime, s UInt32, `n.s` Array(String), `n.d` Array(Date)) ENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test/alter\', \'r2\', d, k, 8192) 2015-01-01 6 38 2014-07-15 13:26:50 100500 [] [] 2015-01-01 7 39 2014-07-14 13:26:50 0 [] [] 2015-01-01 8 40 2012-12-12 12:12:12 0 [] [] 2015-01-01 9 41 1992-01-01 08:00:00 0 [] [] 2015-01-01 10 42 0000-00-00 00:00:00 0 [] [] -d Date -k UInt64 -i32 Int32 -dt DateTime -s UInt32 +d Date +k UInt64 +i32 Int32 +dt DateTime +s UInt32 CREATE TABLE test.replicated_alter1 ( d Date, k UInt64, i32 Int32, dt DateTime, s UInt32) ENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test/alter\', \'r1\', d, k, 8192) -d Date -k UInt64 -i32 Int32 -dt DateTime -s UInt32 +d Date +k UInt64 +i32 Int32 +dt DateTime +s UInt32 CREATE TABLE test.replicated_alter2 ( d Date, k UInt64, i32 Int32, dt DateTime, s UInt32) ENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test/alter\', \'r2\', d, k, 8192) 2015-01-01 6 38 2014-07-15 13:26:50 100500 2015-01-01 7 39 2014-07-14 13:26:50 0 2015-01-01 8 40 2012-12-12 12:12:12 0 2015-01-01 9 41 1992-01-01 08:00:00 0 2015-01-01 10 42 0000-00-00 00:00:00 0 -d Date -k UInt64 -i32 Int32 -dt Date -s DateTime +d Date +k UInt64 +i32 Int32 +dt Date +s DateTime CREATE TABLE test.replicated_alter1 ( d Date, k UInt64, i32 Int32, dt Date, s DateTime) ENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test/alter\', \'r1\', d, k, 8192) -d Date -k UInt64 -i32 Int32 -dt Date -s DateTime +d Date +k UInt64 +i32 Int32 +dt Date +s DateTime CREATE TABLE test.replicated_alter2 ( d Date, k UInt64, i32 Int32, dt Date, s DateTime) ENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test/alter\', \'r2\', d, k, 8192) 2015-01-01 6 38 2014-07-15 1970-01-02 06:55:00 2015-01-01 7 39 2014-07-14 0000-00-00 00:00:00 diff --git a/dbms/tests/queries/0_stateless/00643_cast_zookeeper.reference b/dbms/tests/queries/0_stateless/00643_cast_zookeeper.reference index 401a354382d..f3b3ba04d1c 100644 --- a/dbms/tests/queries/0_stateless/00643_cast_zookeeper.reference +++ b/dbms/tests/queries/0_stateless/00643_cast_zookeeper.reference @@ -1,5 +1,5 @@ CREATE TABLE test.cast1 ( x UInt8, e Enum8('hello' = 1, 'world' = 2) DEFAULT CAST(x, 'Enum8(\'hello\' = 1, \'world\' = 2)')) ENGINE = ReplicatedMergeTree('/clickhouse/tables/test_cast', 'r1') ORDER BY e SETTINGS index_granularity = 8192 -x UInt8 -e Enum8(\'hello\' = 1, \'world\' = 2) DEFAULT CAST(x, \'Enum8(\\\'hello\\\' = 1, \\\'world\\\' = 2)\') +x UInt8 +e Enum8(\'hello\' = 1, \'world\' = 2) DEFAULT CAST(x, \'Enum8(\\\'hello\\\' = 1, \\\'world\\\' = 2)\') 1 hello 1 hello diff --git a/dbms/tests/queries/0_stateless/00753_comment_columns_zookeeper.reference b/dbms/tests/queries/0_stateless/00753_comment_columns_zookeeper.reference new file mode 100644 index 00000000000..3c344a523ae --- /dev/null +++ b/dbms/tests/queries/0_stateless/00753_comment_columns_zookeeper.reference @@ -0,0 +1,4 @@ +CREATE TABLE test.check_comments ( column_name1 UInt8 DEFAULT 1 COMMENT \'comment\') ENGINE = ReplicatedMergeTree(\'clickhouse/tables/test_comments\', \'r1\') ORDER BY column_name1 SETTINGS index_granularity = 8192 +column_name1 UInt8 DEFAULT 1 comment +CREATE TABLE test.check_comments ( column_name1 UInt8 DEFAULT 1 COMMENT \'another comment\') ENGINE = ReplicatedMergeTree(\'clickhouse/tables/test_comments\', \'r1\') ORDER BY column_name1 SETTINGS index_granularity = 8192 +column_name1 UInt8 DEFAULT 1 another comment diff --git a/dbms/tests/queries/0_stateless/00753_comment_columns_zookeeper.sql b/dbms/tests/queries/0_stateless/00753_comment_columns_zookeeper.sql new file mode 100644 index 00000000000..e833e79b914 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00753_comment_columns_zookeeper.sql @@ -0,0 +1,19 @@ +DROP TABLE IF EXISTS test.check_comments; + +CREATE TABLE test.check_comments + ( + column_name1 UInt8 DEFAULT 1 COMMENT 'comment' + ) ENGINE = ReplicatedMergeTree('clickhouse/tables/test_comments', 'r1') + ORDER BY column_name1; + +SHOW CREATE test.check_comments; +DESC test.check_comments; + +ALTER TABLE test.check_comments COMMENT COLUMN column_name1 'another comment'; + +SHOW CREATE test.check_comments; +DESC test.check_comments; + +SELECT * FROM system.columns WHERE table = 'check.comments' and database = 'test'; + +DROP TABLE test.check_comments; From 35fce9c3ac7a699305f09faae519614ae9802c48 Mon Sep 17 00:00:00 2001 From: Sabyanin Maxim Date: Thu, 22 Nov 2018 17:32:28 +0300 Subject: [PATCH 044/124] add DROP TABLE IF EXISTS in test --- dbms/tests/queries/0_stateless/00753_quantile_format.sql | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/dbms/tests/queries/0_stateless/00753_quantile_format.sql b/dbms/tests/queries/0_stateless/00753_quantile_format.sql index 9db4d51db1f..d5345c346fe 100644 --- a/dbms/tests/queries/0_stateless/00753_quantile_format.sql +++ b/dbms/tests/queries/0_stateless/00753_quantile_format.sql @@ -1,3 +1,5 @@ +DROP TABLE IF EXISTS test.datetime; + CREATE TABLE test.datetime (d DateTime) ENGINE = Memory; INSERT INTO test.datetime(d) VALUES(toDateTime('2016-06-15 23:00:00')); @@ -24,3 +26,5 @@ SELECT quantilesTDigest(0.2)(d) FROM test.datetime; SELECT quantileTDigestWeighted(0.2)(d, 1) FROM test.datetime; SELECT quantilesTDigestWeighted(0.2)(d, 1) FROM test.datetime; + +DROP TABLE test.datetime; From e53eb702a131a4df187831e96c50d3822f215d54 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 22 Nov 2018 18:07:32 +0300 Subject: [PATCH 045/124] Usability of client error message when it cannot load data for suggestions [#CLICKHOUSE-2] --- dbms/programs/client/Suggest.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/programs/client/Suggest.h b/dbms/programs/client/Suggest.h index b93086d3b00..617e2bb520e 100644 --- a/dbms/programs/client/Suggest.h +++ b/dbms/programs/client/Suggest.h @@ -184,7 +184,7 @@ public: } catch (...) { - std::cerr << "Cannot load data for command line suggestions: " << getCurrentExceptionMessage(false) << "\n"; + std::cerr << "Cannot load data for command line suggestions: " << getCurrentExceptionMessage(false, true) << "\n"; } /// Note that keyword suggestions are available even if we cannot load data from server. From 7b919c9b2bb3353f51f84989864ed536e312e3a7 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 22 Nov 2018 18:09:19 +0300 Subject: [PATCH 046/124] Removed unused constructor #3633 --- dbms/src/Common/Exception.h | 1 - 1 file changed, 1 deletion(-) diff --git a/dbms/src/Common/Exception.h b/dbms/src/Common/Exception.h index 37fa92e4515..5a32f577e1a 100644 --- a/dbms/src/Common/Exception.h +++ b/dbms/src/Common/Exception.h @@ -19,7 +19,6 @@ class Exception : public Poco::Exception public: Exception() {} /// For deferred initialization. Exception(const std::string & msg, int code = 0) : Poco::Exception(msg, code) {} - Exception(const std::string & msg, const std::string & arg, int code = 0) : Poco::Exception(msg, arg, code) {} Exception(const std::string & msg, const Exception & exc, int code = 0) : Poco::Exception(msg, exc, code), trace(exc.trace) {} explicit Exception(const Poco::Exception & exc) : Poco::Exception(exc.displayText()) {} From b8882ddd9000f8dfed973945173855a3357ab343 Mon Sep 17 00:00:00 2001 From: KochetovNicolai Date: Thu, 22 Nov 2018 18:09:55 +0300 Subject: [PATCH 047/124] Update config.xml Fix cluster ports in config.xml --- dbms/programs/server/config.xml | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/dbms/programs/server/config.xml b/dbms/programs/server/config.xml index fe965902f93..514a081eaca 100644 --- a/dbms/programs/server/config.xml +++ b/dbms/programs/server/config.xml @@ -168,13 +168,13 @@ localhost - 59000 + 9000 localhost - 59000 + 9000 From 4721b16b5134e682d2f2f5d47965e37499d338e2 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 22 Nov 2018 14:16:08 +0300 Subject: [PATCH 048/124] Fix array mapped functions with low cardinality lambda argument. --- dbms/src/Columns/ColumnFunction.cpp | 2 +- dbms/src/Functions/FunctionArrayMapped.h | 17 ++++++++++++----- 2 files changed, 13 insertions(+), 6 deletions(-) diff --git a/dbms/src/Columns/ColumnFunction.cpp b/dbms/src/Columns/ColumnFunction.cpp index 852f6f03adf..8bc239eb657 100644 --- a/dbms/src/Columns/ColumnFunction.cpp +++ b/dbms/src/Columns/ColumnFunction.cpp @@ -183,7 +183,7 @@ void ColumnFunction::appendArgument(const ColumnWithTypeAndName & column) auto index = captured_columns.size(); if (!column.type->equals(*argumnet_types[index])) throw Exception("Cannot capture column " + std::to_string(argumnet_types.size()) + - "because it has incompatible type: got " + column.type->getName() + + " because it has incompatible type: got " + column.type->getName() + ", but " + argumnet_types[index]->getName() + " is expected.", ErrorCodes::LOGICAL_ERROR); captured_columns.push_back(column); diff --git a/dbms/src/Functions/FunctionArrayMapped.h b/dbms/src/Functions/FunctionArrayMapped.h index c23e756d4d8..f14ac244a99 100644 --- a/dbms/src/Functions/FunctionArrayMapped.h +++ b/dbms/src/Functions/FunctionArrayMapped.h @@ -9,6 +9,7 @@ #include #include #include +#include namespace DB @@ -70,7 +71,7 @@ public: if (!array_type) throw Exception("Argument " + toString(i + 2) + " of function " + getName() + " must be array. Found " + arguments[i + 1]->getName() + " instead.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); - nested_types[i] = array_type->getNestedType(); + nested_types[i] = removeLowCardinality(array_type->getNestedType()); } const DataTypeFunction * function_type = checkAndGetDataType(arguments[0].get()); @@ -121,7 +122,7 @@ public: /// The types of the remaining arguments are already checked in getLambdaArgumentTypes. - DataTypePtr return_type = data_type_function->getReturnType(); + DataTypePtr return_type = removeLowCardinality(data_type_function->getReturnType()); if (Impl::needBoolean() && !WhichDataType(return_type).isUInt8()) throw Exception("Expression for function " + getName() + " must return UInt8, found " + return_type->getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); @@ -184,6 +185,8 @@ public: if (!column_const_array) throw Exception("Expected array column, found " + column_array_ptr->getName(), ErrorCodes::ILLEGAL_COLUMN); column_array_ptr = column_const_array->convertToFullColumn(); + if (column_array_ptr->lowCardinality()) + column_array_ptr = column_array_ptr->convertToFullColumnIfLowCardinality(); column_array = checkAndGetColumn(column_array_ptr.get()); } @@ -209,7 +212,8 @@ public: } arrays.emplace_back(ColumnWithTypeAndName(column_array->getDataPtr(), - array_type->getNestedType(), array_with_type_and_name.name)); + removeLowCardinality(array_type->getNestedType()), + array_with_type_and_name.name)); } /// Put all the necessary columns multiplied by the sizes of arrays into the block. @@ -217,8 +221,11 @@ public: auto * replicated_column_function = typeid_cast(replicated_column_function_ptr.get()); replicated_column_function->appendArguments(arrays); - block.getByPosition(result).column = Impl::execute(*column_first_array, - replicated_column_function->reduce().column); + auto lambda_result = replicated_column_function->reduce().column; + if (lambda_result->lowCardinality()) + lambda_result = lambda_result->convertToFullColumnIfLowCardinality(); + + block.getByPosition(result).column = Impl::execute(*column_first_array, lambda_result); } } }; From a29fd043a44865b3b537ad64f8b378aa779b0915 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 22 Nov 2018 18:52:26 +0300 Subject: [PATCH 049/124] Added test --- .../00752_low_cardinality_lambda_argument.reference | 11 +++++++++++ .../00752_low_cardinality_lambda_argument.sql | 7 +++++++ 2 files changed, 18 insertions(+) create mode 100644 dbms/tests/queries/0_stateless/00752_low_cardinality_lambda_argument.reference create mode 100644 dbms/tests/queries/0_stateless/00752_low_cardinality_lambda_argument.sql diff --git a/dbms/tests/queries/0_stateless/00752_low_cardinality_lambda_argument.reference b/dbms/tests/queries/0_stateless/00752_low_cardinality_lambda_argument.reference new file mode 100644 index 00000000000..f00e82d5964 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00752_low_cardinality_lambda_argument.reference @@ -0,0 +1,11 @@ +[] +[0] +[0] +[0,2] +[0,2] +[0,2,4] +[0,2,4] +[0,2,4,6] +[0,2,4,6] +[0,2,4,6,8] + diff --git a/dbms/tests/queries/0_stateless/00752_low_cardinality_lambda_argument.sql b/dbms/tests/queries/0_stateless/00752_low_cardinality_lambda_argument.sql new file mode 100644 index 00000000000..eea080d7b9a --- /dev/null +++ b/dbms/tests/queries/0_stateless/00752_low_cardinality_lambda_argument.sql @@ -0,0 +1,7 @@ +set allow_experimental_low_cardinality_type = 1; +drop table if exists test.lc_lambda; +create table test.lc_lambda (arr Array(LowCardinality(UInt64))) engine = Memory; +insert into test.lc_lambda select range(number) from system.numbers limit 10; +select arrayFilter(x -> x % 2 == 0, arr) from test.lc_lambda; +drop table if exists test.lc_lambda; + From 510703fc7c6bec5c9e82eb64d7efc5bb686aacc0 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 22 Nov 2018 18:57:38 +0300 Subject: [PATCH 050/124] Added test --- .../0_stateless/00752_low_cardinality_lambda_argument.reference | 1 - 1 file changed, 1 deletion(-) diff --git a/dbms/tests/queries/0_stateless/00752_low_cardinality_lambda_argument.reference b/dbms/tests/queries/0_stateless/00752_low_cardinality_lambda_argument.reference index f00e82d5964..20076c05d5d 100644 --- a/dbms/tests/queries/0_stateless/00752_low_cardinality_lambda_argument.reference +++ b/dbms/tests/queries/0_stateless/00752_low_cardinality_lambda_argument.reference @@ -8,4 +8,3 @@ [0,2,4,6] [0,2,4,6] [0,2,4,6,8] - From ae0a685bbd4404a1143d825032ee0db17a0f69e6 Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Thu, 22 Nov 2018 18:58:33 +0300 Subject: [PATCH 051/124] Update CHANGELOG.md --- CHANGELOG.md | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/CHANGELOG.md b/CHANGELOG.md index eba46e0dfd0..b5e729386e7 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -1,3 +1,8 @@ +## ClickHouse release 18.14.15, 2018-11-21 + +### Bug fixes: +* Fixed the issue: the size of memory chunk was overestimated while deserializing the column of type `Array(String)` that leads to "Memory limit exceeded" errors. [#3589](https://github.com/yandex/ClickHouse/issues/3589) + ## ClickHouse release 18.14.14, 2018-11-20 ### Bug fixes: From 1e879850bb2c71fc25c3f0a8b5bc5027415595df Mon Sep 17 00:00:00 2001 From: alesapin Date: Thu, 22 Nov 2018 18:59:00 +0300 Subject: [PATCH 052/124] Kill odbc-bridge with clickhouse-server --- dbms/src/Common/ShellCommand.cpp | 17 +++--- dbms/src/Common/ShellCommand.h | 11 ++-- dbms/src/Common/XDBCBridgeHelper.h | 54 ++++++++++--------- .../Dictionaries/DictionarySourceFactory.cpp | 3 +- dbms/src/Interpreters/Context.cpp | 11 ++++ dbms/src/Interpreters/Context.h | 4 ++ dbms/src/Storages/StorageXDBC.cpp | 2 +- .../src/TableFunctions/ITableFunctionXDBC.cpp | 7 ++- dbms/src/TableFunctions/ITableFunctionXDBC.h | 10 ++-- dbms/tests/integration/helpers/cluster.py | 24 +++++---- .../integration/test_odbc_interaction/test.py | 28 +++++++++- 11 files changed, 113 insertions(+), 58 deletions(-) diff --git a/dbms/src/Common/ShellCommand.cpp b/dbms/src/Common/ShellCommand.cpp index c9e625810ca..2b1fbfe9d7b 100644 --- a/dbms/src/Common/ShellCommand.cpp +++ b/dbms/src/Common/ShellCommand.cpp @@ -7,6 +7,7 @@ #include #include #include +#include namespace DB @@ -77,11 +78,13 @@ namespace DB ShellCommand::~ShellCommand() { - if (!wait_called) + if (die_in_destructor) + kill(pid, SIGTERM); + else if (!wait_called) tryWait(); } -std::unique_ptr ShellCommand::executeImpl(const char * filename, char * const argv[], bool pipe_stdin_only) +std::unique_ptr ShellCommand::executeImpl(const char * filename, char * const argv[], bool pipe_stdin_only, bool die_in_destructor) { /** Here it is written that with a normal call `vfork`, there is a chance of deadlock in multithreaded programs, * because of the resolving of characters in the shared library @@ -128,7 +131,7 @@ std::unique_ptr ShellCommand::executeImpl(const char * filename, c _exit(int(ReturnCodes::CANNOT_EXEC)); } - std::unique_ptr res(new ShellCommand(pid, pipe_stdin.write_fd, pipe_stdout.read_fd, pipe_stderr.read_fd)); + std::unique_ptr res(new ShellCommand(pid, pipe_stdin.write_fd, pipe_stdout.read_fd, pipe_stderr.read_fd, die_in_destructor)); /// Now the ownership of the file descriptors is passed to the result. pipe_stdin.write_fd = -1; @@ -139,7 +142,7 @@ std::unique_ptr ShellCommand::executeImpl(const char * filename, c } -std::unique_ptr ShellCommand::execute(const std::string & command, bool pipe_stdin_only) +std::unique_ptr ShellCommand::execute(const std::string & command, bool pipe_stdin_only, bool die_in_destructor) { /// Arguments in non-constant chunks of memory (as required for `execv`). /// Moreover, their copying must be done before calling `vfork`, so after `vfork` do a minimum of things. @@ -149,11 +152,11 @@ std::unique_ptr ShellCommand::execute(const std::string & command, char * const argv[] = { argv0.data(), argv1.data(), argv2.data(), nullptr }; - return executeImpl("/bin/sh", argv, pipe_stdin_only); + return executeImpl("/bin/sh", argv, pipe_stdin_only, die_in_destructor); } -std::unique_ptr ShellCommand::executeDirect(const std::string & path, const std::vector & arguments) +std::unique_ptr ShellCommand::executeDirect(const std::string & path, const std::vector & arguments, bool die_in_destructor) { size_t argv_sum_size = path.size() + 1; for (const auto & arg : arguments) @@ -174,7 +177,7 @@ std::unique_ptr ShellCommand::executeDirect(const std::string & pa argv[arguments.size() + 1] = nullptr; - return executeImpl(path.data(), argv.data(), false); + return executeImpl(path.data(), argv.data(), false, die_in_destructor); } diff --git a/dbms/src/Common/ShellCommand.h b/dbms/src/Common/ShellCommand.h index a439568fe06..a6f5c749adc 100644 --- a/dbms/src/Common/ShellCommand.h +++ b/dbms/src/Common/ShellCommand.h @@ -28,11 +28,12 @@ class ShellCommand private: pid_t pid; bool wait_called = false; + bool die_in_destructor; - ShellCommand(pid_t pid, int in_fd, int out_fd, int err_fd) - : pid(pid), in(in_fd), out(out_fd), err(err_fd) {} + ShellCommand(pid_t pid, int in_fd, int out_fd, int err_fd, bool die_in_destructor_) + : pid(pid), die_in_destructor(die_in_destructor_), in(in_fd), out(out_fd), err(err_fd) {} - static std::unique_ptr executeImpl(const char * filename, char * const argv[], bool pipe_stdin_only); + static std::unique_ptr executeImpl(const char * filename, char * const argv[], bool pipe_stdin_only, bool die_in_destructor); public: WriteBufferFromFile in; /// If the command reads from stdin, do not forget to call in.close() after writing all the data there. @@ -42,10 +43,10 @@ public: ~ShellCommand(); /// Run the command using /bin/sh -c - static std::unique_ptr execute(const std::string & command, bool pipe_stdin_only = false); + static std::unique_ptr execute(const std::string & command, bool pipe_stdin_only = false, bool die_in_destructor = false); /// Run the executable with the specified arguments. `arguments` - without argv[0]. - static std::unique_ptr executeDirect(const std::string & path, const std::vector & arguments); + static std::unique_ptr executeDirect(const std::string & path, const std::vector & arguments, bool die_in_destructor = false); /// Wait for the process to end, throw an exception if the code is not 0 or if the process was not completed by itself. void wait(); diff --git a/dbms/src/Common/XDBCBridgeHelper.h b/dbms/src/Common/XDBCBridgeHelper.h index efdf7b401dd..4e1e9e1082a 100644 --- a/dbms/src/Common/XDBCBridgeHelper.h +++ b/dbms/src/Common/XDBCBridgeHelper.h @@ -68,6 +68,7 @@ protected: public: using Configuration = Poco::Util::AbstractConfiguration; + Context & context; const Configuration & config; static constexpr inline auto DEFAULT_HOST = "localhost"; @@ -78,8 +79,8 @@ public: static constexpr inline auto IDENTIFIER_QUOTE_HANDLER = "/identifier_quote"; static constexpr inline auto PING_OK_ANSWER = "Ok."; - XDBCBridgeHelper(const Configuration & config_, const Poco::Timespan & http_timeout_, const std::string & connection_string_) - : http_timeout(http_timeout_), connection_string(connection_string_), config(config_) + XDBCBridgeHelper(Context & global_context_, const Poco::Timespan & http_timeout_, const std::string & connection_string_) + : http_timeout(http_timeout_), connection_string(connection_string_), context(global_context_), config(context.getConfigRef()) { size_t bridge_port = config.getUInt(BridgeHelperMixin::configPrefix() + ".port", DEFAULT_PORT); std::string bridge_host = config.getString(BridgeHelperMixin::configPrefix() + ".host", DEFAULT_HOST); @@ -210,7 +211,8 @@ private: /* Contains logic for instantiation of the bridge instance */ void startBridge() const { - BridgeHelperMixin::startBridge(config, log, http_timeout); + auto cmd = BridgeHelperMixin::startBridge(config, log, http_timeout); + context.addXDBCBridgeCommand(std::move(cmd)); } }; @@ -230,7 +232,7 @@ struct JDBCBridgeMixin return "JDBC"; } - static void startBridge(const Poco::Util::AbstractConfiguration &, const Poco::Logger *, const Poco::Timespan &) + static std::unique_ptr startBridge(const Poco::Util::AbstractConfiguration &, const Poco::Logger *, const Poco::Timespan &) { throw Exception("jdbc-bridge is not running. Please, start it manually", ErrorCodes::EXTERNAL_SERVER_IS_NOT_RESPONDING); } @@ -253,11 +255,13 @@ struct ODBCBridgeMixin return "ODBC"; } - static void startBridge(const Poco::Util::AbstractConfiguration & config, Poco::Logger * log, const Poco::Timespan & http_timeout) + static std::unique_ptr startBridge(const Poco::Util::AbstractConfiguration & config, Poco::Logger * log, const Poco::Timespan & http_timeout) { /// Path to executable folder Poco::Path path{config.getString("application.dir", "/usr/bin")}; + + std::vector cmd_args; path.setFileName( #if CLICKHOUSE_SPLIT_BINARY "clickhouse-odbc-bridge" @@ -268,34 +272,36 @@ struct ODBCBridgeMixin std::stringstream command; - command << path.toString() << #if CLICKHOUSE_SPLIT_BINARY - " " #else - " odbc-bridge " + cmd_args.push_back("odbc-bridge"); #endif - ; - command << "--http-port " << config.getUInt(configPrefix() + ".port", DEFAULT_PORT) << ' '; - command << "--listen-host " << config.getString(configPrefix() + ".listen_host", XDBCBridgeHelper::DEFAULT_HOST) - << ' '; - command << "--http-timeout " << http_timeout.totalMicroseconds() << ' '; + cmd_args.push_back("--http-port"); + cmd_args.push_back(std::to_string(config.getUInt(configPrefix() + ".port", DEFAULT_PORT))); + cmd_args.push_back("--listen-host"); + cmd_args.push_back(config.getString(configPrefix() + ".listen_host", XDBCBridgeHelper::DEFAULT_HOST)); + cmd_args.push_back("--http-timeout"); + cmd_args.push_back(std::to_string(http_timeout.totalMicroseconds())); if (config.has("logger." + configPrefix() + "_log")) - command << "--log-path " << config.getString("logger." + configPrefix() + "_log") << ' '; + { + cmd_args.push_back("--log-path"); + cmd_args.push_back(config.getString("logger." + configPrefix() + "_log")); + } if (config.has("logger." + configPrefix() + "_errlog")) - command << "--err-log-path " << config.getString("logger." + configPrefix() + "_errlog") << ' '; + { + cmd_args.push_back("--err-log-path"); + cmd_args.push_back(config.getString("logger." + configPrefix() + "_errlog")); + } if (config.has("logger." + configPrefix() + "_level")) - command << "--log-level " << config.getString("logger." + configPrefix() + "_level") << ' '; - command << "&"; /// we don't want to wait this process + { + cmd_args.push_back("--log-level"); + cmd_args.push_back(config.getString("logger." + configPrefix() + "_level")); + } - auto command_str = command.str(); + LOG_TRACE(log, "Starting " + serviceAlias()); - std::cerr << command_str << std::endl; - - LOG_TRACE(log, "Starting " + serviceAlias() + " with command: " << command_str); - - auto cmd = ShellCommand::execute(command_str); - cmd->wait(); + return ShellCommand::executeDirect(path.toString(), cmd_args, true); } }; } diff --git a/dbms/src/Dictionaries/DictionarySourceFactory.cpp b/dbms/src/Dictionaries/DictionarySourceFactory.cpp index 1da2268c1e5..d2deb769839 100644 --- a/dbms/src/Dictionaries/DictionarySourceFactory.cpp +++ b/dbms/src/Dictionaries/DictionarySourceFactory.cpp @@ -155,8 +155,7 @@ DictionarySourcePtr DictionarySourceFactory::create( else if ("odbc" == source_type) { #if USE_POCO_SQLODBC || USE_POCO_DATAODBC - const auto & global_config = context.getConfigRef(); - BridgeHelperPtr bridge = std::make_shared>(global_config, context.getSettings().http_receive_timeout, config.getString(config_prefix + ".odbc.connection_string")); + BridgeHelperPtr bridge = std::make_shared>(context, context.getSettings().http_receive_timeout, config.getString(config_prefix + ".odbc.connection_string")); return std::make_unique(dict_struct, config, config_prefix + ".odbc", sample_block, context, bridge); #else throw Exception{"Dictionary source of type `odbc` is disabled because poco library was built without ODBC support.", diff --git a/dbms/src/Interpreters/Context.cpp b/dbms/src/Interpreters/Context.cpp index 6e7051ba387..505d7b5918e 100644 --- a/dbms/src/Interpreters/Context.cpp +++ b/dbms/src/Interpreters/Context.cpp @@ -50,6 +50,7 @@ #include #include +#include #include @@ -199,6 +200,9 @@ struct ContextShared pcg64 rng{randomSeed()}; + /// vector of xdbc-bridge commands, they will be killed when Context will be destroyed + std::vector> bridge_commands; + Context::ConfigReloadCallback config_reload_callback; ContextShared(std::shared_ptr runtime_components_factory_) @@ -1844,6 +1848,13 @@ void Context::dropCompiledExpressionCache() const #endif + +void Context::addXDBCBridgeCommand(std::unique_ptr cmd) +{ + auto lock = getLock(); + shared->bridge_commands.emplace_back(std::move(cmd)); +} + std::shared_ptr Context::getActionLocksManager() { auto lock = getLock(); diff --git a/dbms/src/Interpreters/Context.h b/dbms/src/Interpreters/Context.h index ad6c12905da..af189b8abed 100644 --- a/dbms/src/Interpreters/Context.h +++ b/dbms/src/Interpreters/Context.h @@ -79,6 +79,7 @@ struct SystemLogs; using SystemLogsPtr = std::shared_ptr; class ActionLocksManager; using ActionLocksManagerPtr = std::shared_ptr; +class ShellCommand; #if USE_EMBEDDED_COMPILER @@ -445,6 +446,9 @@ public: void dropCompiledExpressionCache() const; #endif + /// Add started bridge command. It will be killed after context destruction + void addXDBCBridgeCommand(std::unique_ptr cmd); + private: /** Check if the current client has access to the specified database. * If access is denied, throw an exception. diff --git a/dbms/src/Storages/StorageXDBC.cpp b/dbms/src/Storages/StorageXDBC.cpp index 40ce763f07f..ac8f156305a 100644 --- a/dbms/src/Storages/StorageXDBC.cpp +++ b/dbms/src/Storages/StorageXDBC.cpp @@ -113,7 +113,7 @@ namespace for (size_t i = 0; i < 3; ++i) engine_args[i] = evaluateConstantExpressionOrIdentifierAsLiteral(engine_args[i], args.local_context); - BridgeHelperPtr bridge_helper = std::make_shared>(args.context.getConfigRef(), + BridgeHelperPtr bridge_helper = std::make_shared>(args.context, args.context.getSettingsRef().http_receive_timeout.value, static_cast(*engine_args[0]).value.safeGet()); return std::make_shared(args.table_name, diff --git a/dbms/src/TableFunctions/ITableFunctionXDBC.cpp b/dbms/src/TableFunctions/ITableFunctionXDBC.cpp index c09d346308e..455512310db 100644 --- a/dbms/src/TableFunctions/ITableFunctionXDBC.cpp +++ b/dbms/src/TableFunctions/ITableFunctionXDBC.cpp @@ -59,10 +59,9 @@ StoragePtr ITableFunctionXDBC::executeImpl(const ASTPtr & ast_function, const Co table_name = static_cast(*args[1]).value.safeGet(); } - const auto & config = context.getConfigRef(); - /* Infer external table structure */ - BridgeHelperPtr helper = createBridgeHelper(config, context.getSettingsRef().http_receive_timeout.value, connection_string); + /// Have to const_cast, because bridges store their commands inside context + BridgeHelperPtr helper = createBridgeHelper(const_cast(context), context.getSettingsRef().http_receive_timeout.value, connection_string); helper->startBridgeSync(); Poco::URI columns_info_uri = helper->getColumnsInfoURI(); @@ -95,4 +94,4 @@ void registerTableFunctionODBC(TableFunctionFactory & factory) { factory.registerFunction(); } -} \ No newline at end of file +} diff --git a/dbms/src/TableFunctions/ITableFunctionXDBC.h b/dbms/src/TableFunctions/ITableFunctionXDBC.h index 3a753457f55..8676b85debf 100644 --- a/dbms/src/TableFunctions/ITableFunctionXDBC.h +++ b/dbms/src/TableFunctions/ITableFunctionXDBC.h @@ -18,7 +18,7 @@ private: StoragePtr executeImpl(const ASTPtr & ast_function, const Context & context) const override; /* A factory method to create bridge helper, that will assist in remote interaction */ - virtual BridgeHelperPtr createBridgeHelper(const Poco::Util::AbstractConfiguration & config_, + virtual BridgeHelperPtr createBridgeHelper(Context & context, const Poco::Timespan & http_timeout_, const std::string & connection_string_) const = 0; }; @@ -33,11 +33,11 @@ public: } private: - BridgeHelperPtr createBridgeHelper(const Poco::Util::AbstractConfiguration & config_, + BridgeHelperPtr createBridgeHelper(Context & context, const Poco::Timespan & http_timeout_, const std::string & connection_string_) const override { - return std::make_shared>(config_, http_timeout_, connection_string_); + return std::make_shared>(context, http_timeout_, connection_string_); } }; @@ -51,11 +51,11 @@ public: } private: - BridgeHelperPtr createBridgeHelper(const Poco::Util::AbstractConfiguration & config_, + BridgeHelperPtr createBridgeHelper(Context & context, const Poco::Timespan & http_timeout_, const std::string & connection_string_) const override { - return std::make_shared>(config_, http_timeout_, connection_string_); + return std::make_shared>(context, http_timeout_, connection_string_); } }; } diff --git a/dbms/tests/integration/helpers/cluster.py b/dbms/tests/integration/helpers/cluster.py index 9a30c61734a..39227369c2a 100644 --- a/dbms/tests/integration/helpers/cluster.py +++ b/dbms/tests/integration/helpers/cluster.py @@ -94,7 +94,7 @@ class ClickHouseCluster: cmd += " client" return cmd - def add_instance(self, name, config_dir=None, main_configs=[], user_configs=[], macros={}, with_zookeeper=False, with_mysql=False, with_kafka=False, clickhouse_path_dir=None, with_odbc_drivers=False, with_postgres=False, hostname=None, env_variables={}, image="yandex/clickhouse-integration-test"): + def add_instance(self, name, config_dir=None, main_configs=[], user_configs=[], macros={}, with_zookeeper=False, with_mysql=False, with_kafka=False, clickhouse_path_dir=None, with_odbc_drivers=False, with_postgres=False, hostname=None, env_variables={}, image="yandex/clickhouse-integration-test", stay_alive=False): """Add an instance to the cluster. name - the name of the instance directory and the value of the 'instance' macro in ClickHouse. @@ -113,7 +113,7 @@ class ClickHouseCluster: instance = ClickHouseInstance( self, self.base_dir, name, config_dir, main_configs, user_configs, macros, with_zookeeper, self.zookeeper_config_path, with_mysql, with_kafka, self.base_configs_dir, self.server_bin_path, - clickhouse_path_dir, with_odbc_drivers, hostname=hostname, env_variables=env_variables, image=image) + clickhouse_path_dir, with_odbc_drivers, hostname=hostname, env_variables=env_variables, image=image, stay_alive=stay_alive) self.instances[name] = instance self.base_cmd.extend(['--file', instance.docker_compose_path]) @@ -300,6 +300,10 @@ class ClickHouseCluster: self.pre_zookeeper_commands.append(command) +CLICKHOUSE_START_COMMAND = "clickhouse server --config-file=/etc/clickhouse-server/config.xml --log-file=/var/log/clickhouse-server/clickhouse-server.log --errorlog-file=/var/log/clickhouse-server/clickhouse-server.err.log" + +CLICKHOUSE_STAY_ALIVE_COMMAND = 'bash -c "{} --daemon; tail -f /dev/null"'.format(CLICKHOUSE_START_COMMAND) + DOCKER_COMPOSE_TEMPLATE = ''' version: '2' services: @@ -313,12 +317,7 @@ services: - {db_dir}:/var/lib/clickhouse/ - {logs_dir}:/var/log/clickhouse-server/ {odbc_ini_path} - entrypoint: - - clickhouse - - server - - --config-file=/etc/clickhouse-server/config.xml - - --log-file=/var/log/clickhouse-server/clickhouse-server.log - - --errorlog-file=/var/log/clickhouse-server/clickhouse-server.err.log + entrypoint: {entrypoint_cmd} depends_on: {depends_on} env_file: - {env_file} @@ -329,7 +328,7 @@ class ClickHouseInstance: def __init__( self, cluster, base_path, name, custom_config_dir, custom_main_configs, custom_user_configs, macros, with_zookeeper, zookeeper_config_path, with_mysql, with_kafka, base_configs_dir, server_bin_path, - clickhouse_path_dir, with_odbc_drivers, hostname=None, env_variables={}, image="yandex/clickhouse-integration-test"): + clickhouse_path_dir, with_odbc_drivers, hostname=None, env_variables={}, image="yandex/clickhouse-integration-test", stay_alive=False): self.name = name self.base_cmd = cluster.base_cmd[:] @@ -365,6 +364,7 @@ class ClickHouseInstance: self.client = None self.default_timeout = 20.0 # 20 sec self.image = image + self.stay_alive = stay_alive # Connects to the instance via clickhouse-client, sends a query (1st argument) and returns the answer def query(self, sql, stdin=None, timeout=None, settings=None, user=None, ignore_error=False): @@ -578,6 +578,11 @@ class ClickHouseInstance: self._create_odbc_config_file() odbc_ini_path = '- ' + self.odbc_ini_path + entrypoint_cmd = CLICKHOUSE_START_COMMAND + + if self.stay_alive: + entrypoint_cmd = CLICKHOUSE_STAY_ALIVE_COMMAND + with open(self.docker_compose_path, 'w') as docker_compose: docker_compose.write(DOCKER_COMPOSE_TEMPLATE.format( image=self.image, @@ -592,6 +597,7 @@ class ClickHouseInstance: depends_on=str(depends_on), env_file=env_file, odbc_ini_path=odbc_ini_path, + entrypoint_cmd=entrypoint_cmd, )) diff --git a/dbms/tests/integration/test_odbc_interaction/test.py b/dbms/tests/integration/test_odbc_interaction/test.py index a3af09a3914..8128c548aed 100644 --- a/dbms/tests/integration/test_odbc_interaction/test.py +++ b/dbms/tests/integration/test_odbc_interaction/test.py @@ -10,7 +10,7 @@ from helpers.cluster import ClickHouseCluster SCRIPT_DIR = os.path.dirname(os.path.realpath(__file__)) cluster = ClickHouseCluster(__file__, base_configs_dir=os.path.join(SCRIPT_DIR, 'configs')) -node1 = cluster.add_instance('node1', with_odbc_drivers=True, with_mysql=True, image='alesapin/ubuntu_with_odbc', main_configs=['configs/dictionaries/sqlite3_odbc_hashed_dictionary.xml', 'configs/dictionaries/sqlite3_odbc_cached_dictionary.xml', 'configs/dictionaries/postgres_odbc_hashed_dictionary.xml']) +node1 = cluster.add_instance('node1', with_odbc_drivers=True, with_mysql=True, image='alesapin/ubuntu_with_odbc', main_configs=['configs/dictionaries/sqlite3_odbc_hashed_dictionary.xml', 'configs/dictionaries/sqlite3_odbc_cached_dictionary.xml', 'configs/dictionaries/postgres_odbc_hashed_dictionary.xml'], stay_alive=True) create_table_sql_template = """ CREATE TABLE `clickhouse`.`{}` ( @@ -176,3 +176,29 @@ def test_postgres_odbc_hached_dictionary_with_schema(started_cluster): time.sleep(5) assert node1.query("select dictGetString('postgres_odbc_hashed', 'column2', toUInt64(1))") == "hello\n" assert node1.query("select dictGetString('postgres_odbc_hashed', 'column2', toUInt64(2))") == "world\n" + +def test_bridge_dies_with_parent(started_cluster): + node1.query("select dictGetString('postgres_odbc_hashed', 'column2', toUInt64(1))") + def get_pid(cmd): + output = node1.exec_in_container(["bash", "-c", "ps ax | grep '{}' | grep -v 'grep' | grep -v 'bash -c' | awk '{{print $1}}'".format(cmd)], privileged=True, user='root') + if output: + try: + pid = int(output.split('\n')[0].strip()) + return pid + except: + return None + return None + + clickhouse_pid = get_pid("clickhouse server") + bridge_pid = get_pid("odbc-bridge") + assert clickhouse_pid is not None + assert bridge_pid is not None + + node1.exec_in_container(["bash", "-c", "kill {}".format(clickhouse_pid)], privileged=True, user='root') + node1.exec_in_container(["bash", "-c", "kill {}".format(clickhouse_pid)], privileged=True, user='root') + + clickhouse_pid = get_pid("clickhouse server") + bridge_pid = get_pid("odbc-bridge") + + assert clickhouse_pid is None + assert bridge_pid is None From 659695bbbb0d1a86de633dff41c38901dea687f4 Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Thu, 22 Nov 2018 18:59:53 +0300 Subject: [PATCH 053/124] Update CHANGELOG.md --- CHANGELOG.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index b5e729386e7..4e81432edd0 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -1,7 +1,7 @@ ## ClickHouse release 18.14.15, 2018-11-21 ### Bug fixes: -* Fixed the issue: the size of memory chunk was overestimated while deserializing the column of type `Array(String)` that leads to "Memory limit exceeded" errors. [#3589](https://github.com/yandex/ClickHouse/issues/3589) +* The size of memory chunk was overestimated while deserializing the column of type `Array(String)` that leads to "Memory limit exceeded" errors. The issue appeared in version 18.12.13. [#3589](https://github.com/yandex/ClickHouse/issues/3589) ## ClickHouse release 18.14.14, 2018-11-20 From 2b2195d4dee5562bf55b456707caa9a11300776f Mon Sep 17 00:00:00 2001 From: alesapin Date: Thu, 22 Nov 2018 19:01:17 +0300 Subject: [PATCH 054/124] Fix stupid if --- dbms/src/Common/XDBCBridgeHelper.h | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/dbms/src/Common/XDBCBridgeHelper.h b/dbms/src/Common/XDBCBridgeHelper.h index 4e1e9e1082a..3ff91c902f5 100644 --- a/dbms/src/Common/XDBCBridgeHelper.h +++ b/dbms/src/Common/XDBCBridgeHelper.h @@ -272,8 +272,7 @@ struct ODBCBridgeMixin std::stringstream command; -#if CLICKHOUSE_SPLIT_BINARY -#else +#if !CLICKHOUSE_SPLIT_BINARY cmd_args.push_back("odbc-bridge"); #endif From 2e6b23fb4654c96c84f1d3f8a53a54e106688056 Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Thu, 22 Nov 2018 19:01:52 +0300 Subject: [PATCH 055/124] Update CHANGELOG_RU.md --- CHANGELOG_RU.md | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/CHANGELOG_RU.md b/CHANGELOG_RU.md index b946be4b608..216d80f100f 100644 --- a/CHANGELOG_RU.md +++ b/CHANGELOG_RU.md @@ -1,3 +1,8 @@ +## ClickHouse release 18.14.15, 2018-11-21 + +### Исправления ошибок: +* При чтении столбцов типа `Array(String)`, размер требуемого куска памяти оценивался слишком большим, что приводило к исключению "Memory limit exceeded" при выполнении запроса. Ошибка появилась в версии 18.12.13. [#3589](https://github.com/yandex/ClickHouse/issues/3589) + ## ClickHouse release 18.14.14, 2018-11-20 ### Исправления ошибок: From 8b25084637e1e149d7ff99da71a0fda157c4836a Mon Sep 17 00:00:00 2001 From: Alexey Zatelepin Date: Thu, 22 Nov 2018 20:29:49 +0300 Subject: [PATCH 056/124] fix deadlock in ParallelInputsProcessor in case thread creation fails [#CLICKHOUSE-4155] --- .../src/DataStreams/ParallelInputsProcessor.h | 22 +++++++++++++++++-- 1 file changed, 20 insertions(+), 2 deletions(-) diff --git a/dbms/src/DataStreams/ParallelInputsProcessor.h b/dbms/src/DataStreams/ParallelInputsProcessor.h index a1c4e2ac480..509522de29a 100644 --- a/dbms/src/DataStreams/ParallelInputsProcessor.h +++ b/dbms/src/DataStreams/ParallelInputsProcessor.h @@ -107,8 +107,26 @@ public: active_threads = max_threads; threads.reserve(max_threads); auto thread_group = CurrentThread::getGroup(); - for (size_t i = 0; i < max_threads; ++i) - threads.emplace_back([=] () { thread(thread_group, i); } ); + + try + { + for (size_t i = 0; i < max_threads; ++i) + threads.emplace_back([=] () { thread(thread_group, i); } ); + } + catch (...) + { + cancel(false); + wait(); + if (active_threads) + { + active_threads = 0; + /// handler.onFinish() is supposed to be called from one of the threads when the number of + /// finished threads reaches max_threads. But since we weren't able to launch all threads, + /// we have to call onFinish() manually here. + handler.onFinish(); + } + throw; + } } /// Ask all sources to stop earlier than they run out. From ebfdd0e7f9459f2e186264b50af2204c08966d7d Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Thu, 22 Nov 2018 20:34:16 +0300 Subject: [PATCH 057/124] Update tips.md --- docs/en/operations/tips.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/operations/tips.md b/docs/en/operations/tips.md index a34f79f869f..6e03416e474 100644 --- a/docs/en/operations/tips.md +++ b/docs/en/operations/tips.md @@ -74,7 +74,7 @@ echo 4096 | sudo tee /sys/block/md2/md/stripe_cache_size Calculate the exact number from the number of devices and the block size, using the formula: `2 * num_devices * chunk_size_in_bytes / 4096`. -A block size of 1025 KB is sufficient for all RAID configurations. +A block size of 1024 KB is sufficient for all RAID configurations. Never set the block size too small or too large. You can use RAID-0 on SSD. From a5d1fc0ca7f97130eee26fc42c92ca7377a8d308 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 22 Nov 2018 21:09:17 +0300 Subject: [PATCH 058/124] Removed unused constructor #3633 --- dbms/src/Client/Connection.cpp | 4 ++-- dbms/src/Common/Exception.h | 27 +++++++++++++++-------- dbms/src/Common/NetException.h | 13 +++++------ dbms/src/IO/ReadBufferFromPocoSocket.cpp | 4 ++-- dbms/src/IO/WriteBufferFromPocoSocket.cpp | 4 ++-- 5 files changed, 29 insertions(+), 23 deletions(-) diff --git a/dbms/src/Client/Connection.cpp b/dbms/src/Client/Connection.cpp index ce6246fba3a..07d2ca05f21 100644 --- a/dbms/src/Client/Connection.cpp +++ b/dbms/src/Client/Connection.cpp @@ -108,14 +108,14 @@ void Connection::connect() disconnect(); /// Add server address to exception. Also Exception will remember stack trace. It's a pity that more precise exception type is lost. - throw NetException(e.displayText(), "(" + getDescription() + ")", ErrorCodes::NETWORK_ERROR); + throw NetException(e.displayText() + " (" + getDescription() + ")", ErrorCodes::NETWORK_ERROR); } catch (Poco::TimeoutException & e) { disconnect(); /// Add server address to exception. Also Exception will remember stack trace. It's a pity that more precise exception type is lost. - throw NetException(e.displayText(), "(" + getDescription() + ")", ErrorCodes::SOCKET_TIMEOUT); + throw NetException(e.displayText() + " (" + getDescription() + ")", ErrorCodes::SOCKET_TIMEOUT); } } diff --git a/dbms/src/Common/Exception.h b/dbms/src/Common/Exception.h index 5a32f577e1a..8e9044df769 100644 --- a/dbms/src/Common/Exception.h +++ b/dbms/src/Common/Exception.h @@ -14,18 +14,23 @@ namespace Poco { class Logger; } namespace DB { +namespace ErrorCodes +{ + extern const int POCO_EXCEPTION; +} + class Exception : public Poco::Exception { public: Exception() {} /// For deferred initialization. Exception(const std::string & msg, int code = 0) : Poco::Exception(msg, code) {} - Exception(const std::string & msg, const Exception & exc, int code = 0) : Poco::Exception(msg, exc, code), trace(exc.trace) {} - explicit Exception(const Poco::Exception & exc) : Poco::Exception(exc.displayText()) {} + Exception(const std::string & msg, const Exception & nested_exception, int code = 0) + : Poco::Exception(msg, nested_exception, code), trace(nested_exception.trace) {} + explicit Exception(const Poco::Exception & exc) : Poco::Exception(exc.displayText(), ErrorCodes::POCO_EXCEPTION) {} - const char * name() const throw() override { return "DB::Exception"; } - const char * className() const throw() override { return "DB::Exception"; } - DB::Exception * clone() const override { return new DB::Exception(*this); } + Exception * clone() const override { return new Exception(*this); } void rethrow() const override { throw *this; } + const char * name() const throw() override { return "DB::Exception"; } /// Add something to the existing message. void addMessage(const std::string & arg) { extendedMessage(arg); } @@ -34,6 +39,8 @@ public: private: StackTrace trace; + + const char * className() const throw() override { return "DB::Exception"; } }; @@ -43,15 +50,17 @@ class ErrnoException : public Exception public: ErrnoException(const std::string & msg, int code, int saved_errno_) : Exception(msg, code), saved_errno(saved_errno_) {} - ErrnoException(const std::string & msg, const std::string & arg, int code, int saved_errno_) - : Exception(msg, arg, code), saved_errno(saved_errno_) {} - ErrnoException(const std::string & msg, const Exception & exc, int code, int saved_errno_) - : Exception(msg, exc, code), saved_errno(saved_errno_) {} + + ErrnoException * clone() const override { return new ErrnoException(*this); } + void rethrow() const override { throw *this; } int getErrno() const { return saved_errno; } private: int saved_errno; + + const char * name() const throw() override { return "DB::ErrnoException"; } + const char * className() const throw() override { return "DB::ErrnoException"; } }; diff --git a/dbms/src/Common/NetException.h b/dbms/src/Common/NetException.h index b2b12e85648..dfc3f118cae 100644 --- a/dbms/src/Common/NetException.h +++ b/dbms/src/Common/NetException.h @@ -6,20 +6,17 @@ namespace DB { -class NetException : public DB::Exception +class NetException : public Exception { public: - NetException(const std::string & msg, int code = 0) : DB::Exception(msg, code) {} - NetException(const std::string & msg, const std::string & arg, int code = 0) : DB::Exception(msg, arg, code) {} - NetException(const std::string & msg, const DB::Exception & exc, int code = 0) : DB::Exception(msg, exc, code) {} + NetException(const std::string & msg, int code = 0) : Exception(msg, code) {} - explicit NetException(const DB::Exception & exc) : DB::Exception(exc) {} - explicit NetException(const Poco::Exception & exc) : DB::Exception(exc.displayText()) {} + NetException * clone() const override { return new NetException(*this); } + void rethrow() const override { throw *this; } +private: const char * name() const throw() override { return "DB::NetException"; } const char * className() const throw() override { return "DB::NetException"; } - DB::NetException * clone() const override { return new DB::NetException(*this); } - void rethrow() const override { throw *this; } }; } diff --git a/dbms/src/IO/ReadBufferFromPocoSocket.cpp b/dbms/src/IO/ReadBufferFromPocoSocket.cpp index 9dc21c19714..e44b8c99c59 100644 --- a/dbms/src/IO/ReadBufferFromPocoSocket.cpp +++ b/dbms/src/IO/ReadBufferFromPocoSocket.cpp @@ -36,7 +36,7 @@ bool ReadBufferFromPocoSocket::nextImpl() } catch (const Poco::Net::NetException & e) { - throw NetException(e.displayText(), "while reading from socket (" + peer_address.toString() + ")", ErrorCodes::NETWORK_ERROR); + throw NetException(e.displayText() + ", while reading from socket (" + peer_address.toString() + ")", ErrorCodes::NETWORK_ERROR); } catch (const Poco::TimeoutException &) { @@ -44,7 +44,7 @@ bool ReadBufferFromPocoSocket::nextImpl() } catch (const Poco::IOException & e) { - throw NetException(e.displayText(), "while reading from socket (" + peer_address.toString() + ")", ErrorCodes::NETWORK_ERROR); + throw NetException(e.displayText() + ", while reading from socket (" + peer_address.toString() + ")", ErrorCodes::NETWORK_ERROR); } if (bytes_read < 0) diff --git a/dbms/src/IO/WriteBufferFromPocoSocket.cpp b/dbms/src/IO/WriteBufferFromPocoSocket.cpp index 0712b2c0603..09fca48c6f4 100644 --- a/dbms/src/IO/WriteBufferFromPocoSocket.cpp +++ b/dbms/src/IO/WriteBufferFromPocoSocket.cpp @@ -43,7 +43,7 @@ void WriteBufferFromPocoSocket::nextImpl() } catch (const Poco::Net::NetException & e) { - throw NetException(e.displayText() + " while writing to socket (" + peer_address.toString() + ")", ErrorCodes::NETWORK_ERROR); + throw NetException(e.displayText() + ", while writing to socket (" + peer_address.toString() + ")", ErrorCodes::NETWORK_ERROR); } catch (const Poco::TimeoutException &) { @@ -51,7 +51,7 @@ void WriteBufferFromPocoSocket::nextImpl() } catch (const Poco::IOException & e) { - throw NetException(e.displayText(), " while reading from socket (" + peer_address.toString() + ")", ErrorCodes::NETWORK_ERROR); + throw NetException(e.displayText() + ", while reading from socket (" + peer_address.toString() + ")", ErrorCodes::NETWORK_ERROR); } if (res < 0) From f410c93fb133abb783359e01baab3712a1177392 Mon Sep 17 00:00:00 2001 From: proller Date: Thu, 22 Nov 2018 21:24:38 +0300 Subject: [PATCH 059/124] Add VERSION_INT (ClickHouse version in integer format like 18014009) to system.metrics and system.build_options --- dbms/cmake/version.cmake | 2 ++ dbms/programs/server/Server.cpp | 2 ++ dbms/src/Common/ClickHouseRevision.cpp | 1 + dbms/src/Common/ClickHouseRevision.h | 1 + dbms/src/Common/CurrentMetrics.cpp | 1 + dbms/src/Common/config_version.h.in | 1 + .../Storages/System/StorageSystemBuildOptions.generated.cpp.in | 1 + 7 files changed, 9 insertions(+) diff --git a/dbms/cmake/version.cmake b/dbms/cmake/version.cmake index e62a8e4796c..d3c3eb7ca4e 100644 --- a/dbms/cmake/version.cmake +++ b/dbms/cmake/version.cmake @@ -22,3 +22,5 @@ endif () set (VERSION_NAME "${PROJECT_NAME}" CACHE STRING "") set (VERSION_FULL "${VERSION_NAME} ${VERSION_STRING}" CACHE STRING "") set (VERSION_SO "${VERSION_STRING}" CACHE STRING "") + +math (EXPR VERSION_INT "${VERSION_PATCH} + ${VERSION_MINOR}*1000 + ${VERSION_MAJOR}*1000000") diff --git a/dbms/programs/server/Server.cpp b/dbms/programs/server/Server.cpp index d8d662d8ed0..a4795dec68b 100644 --- a/dbms/programs/server/Server.cpp +++ b/dbms/programs/server/Server.cpp @@ -55,6 +55,7 @@ namespace CurrentMetrics { extern const Metric Revision; + extern const Metric VersionInt; } namespace DB @@ -109,6 +110,7 @@ int Server::main(const std::vector & /*args*/) registerStorages(); CurrentMetrics::set(CurrentMetrics::Revision, ClickHouseRevision::get()); + CurrentMetrics::set(CurrentMetrics::VersionInt, ClickHouseRevision::getVersionInt()); /** Context contains all that query execution is dependent: * settings, available functions, data types, aggregate functions, databases... diff --git a/dbms/src/Common/ClickHouseRevision.cpp b/dbms/src/Common/ClickHouseRevision.cpp index ea48dcdc54f..7a76f7b719b 100644 --- a/dbms/src/Common/ClickHouseRevision.cpp +++ b/dbms/src/Common/ClickHouseRevision.cpp @@ -4,4 +4,5 @@ namespace ClickHouseRevision { unsigned get() { return VERSION_REVISION; } + unsigned getVersionInt() { return VERSION_INT; } } diff --git a/dbms/src/Common/ClickHouseRevision.h b/dbms/src/Common/ClickHouseRevision.h index 7c147fe6f8b..43ecb3618d5 100644 --- a/dbms/src/Common/ClickHouseRevision.h +++ b/dbms/src/Common/ClickHouseRevision.h @@ -3,4 +3,5 @@ namespace ClickHouseRevision { unsigned get(); + unsigned getVersionInt(); } diff --git a/dbms/src/Common/CurrentMetrics.cpp b/dbms/src/Common/CurrentMetrics.cpp index 59b26f4b5e1..cd5e8532c08 100644 --- a/dbms/src/Common/CurrentMetrics.cpp +++ b/dbms/src/Common/CurrentMetrics.cpp @@ -40,6 +40,7 @@ M(StorageBufferBytes, "Number of bytes in buffers of Buffer tables") \ M(DictCacheRequests, "Number of requests in fly to data sources of dictionaries of cache type.") \ M(Revision, "Revision of the server. It is a number incremented for every release or release candidate.") \ + M(VersionInt, "Version of the server. It is a number incremented for every release.") \ M(RWLockWaitingReaders, "Number of threads waiting for read on a table RWLock.") \ M(RWLockWaitingWriters, "Number of threads waiting for write on a table RWLock.") \ M(RWLockActiveReaders, "Number of threads holding read lock in a table RWLock.") \ diff --git a/dbms/src/Common/config_version.h.in b/dbms/src/Common/config_version.h.in index 8f3e2f56c84..a6ec0743744 100644 --- a/dbms/src/Common/config_version.h.in +++ b/dbms/src/Common/config_version.h.in @@ -23,6 +23,7 @@ #cmakedefine VERSION_FULL "@VERSION_FULL@" #cmakedefine VERSION_DESCRIBE "@VERSION_DESCRIBE@" #cmakedefine VERSION_GITHASH "@VERSION_GITHASH@" +#cmakedefine VERSION_INT @VERSION_INT@ #if defined(VERSION_MAJOR) #define DBMS_VERSION_MAJOR VERSION_MAJOR diff --git a/dbms/src/Storages/System/StorageSystemBuildOptions.generated.cpp.in b/dbms/src/Storages/System/StorageSystemBuildOptions.generated.cpp.in index 57c418d7f98..87e29f4e45d 100644 --- a/dbms/src/Storages/System/StorageSystemBuildOptions.generated.cpp.in +++ b/dbms/src/Storages/System/StorageSystemBuildOptions.generated.cpp.in @@ -6,6 +6,7 @@ const char * auto_config_build[] "VERSION_DESCRIBE", "@VERSION_DESCRIBE@", "VERSION_GITHASH", "@VERSION_GITHASH@", "VERSION_REVISION", "@VERSION_REVISION@", + "VERSION_INT", "@VERSION_INT@", "BUILD_DATE", "@BUILD_DATE@", "BUILD_TYPE", "@CMAKE_BUILD_TYPE@", "SYSTEM", "@CMAKE_SYSTEM@", From b6800290306ce7f65df917be152022f2663f52c9 Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Thu, 22 Nov 2018 22:36:56 +0300 Subject: [PATCH 060/124] Update ColumnsDescription.cpp --- dbms/src/Storages/ColumnsDescription.cpp | 2 ++ 1 file changed, 2 insertions(+) diff --git a/dbms/src/Storages/ColumnsDescription.cpp b/dbms/src/Storages/ColumnsDescription.cpp index 087c6dbf605..2432f084441 100644 --- a/dbms/src/Storages/ColumnsDescription.cpp +++ b/dbms/src/Storages/ColumnsDescription.cpp @@ -19,6 +19,8 @@ #include +#include + namespace DB { From 6288f0f07b1ea99ec7087bace0336ba520e68121 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 22 Nov 2018 22:52:26 +0300 Subject: [PATCH 061/124] Updated administration tips [#CLICKHOUSE-3] --- docs/en/operations/tips.md | 5 +++++ docs/ru/operations/tips.md | 5 +++++ 2 files changed, 10 insertions(+) diff --git a/docs/en/operations/tips.md b/docs/en/operations/tips.md index 6e03416e474..bd058afb6dd 100644 --- a/docs/en/operations/tips.md +++ b/docs/en/operations/tips.md @@ -35,6 +35,11 @@ For small amounts of data (up to \~200 GB compressed), it is best to use as much For large amounts of data and when processing interactive (online) queries, you should use a reasonable amount of RAM (128 GB or more) so the hot data subset will fit in the cache of pages. Even for data volumes of \~50 TB per server, using 128 GB of RAM significantly improves query performance compared to 64 GB. +Do not disable overcommit. The value `cat /proc/sys/vm/overcommit_memory` should be 0 or 1. Run +``` +echo 0 | sudo tee /proc/sys/vm/overcommit_memory +``` + ## Swap File Always disable the swap file. The only reason for not doing this is if you are using ClickHouse on your personal laptop. diff --git a/docs/ru/operations/tips.md b/docs/ru/operations/tips.md index 4a6d8787a36..a66a760bc79 100644 --- a/docs/ru/operations/tips.md +++ b/docs/ru/operations/tips.md @@ -35,6 +35,11 @@ sudo echo 'performance' | tee /sys/devices/system/cpu/cpu*/cpufreq/scaling_gover Для больших объемов данных, при выполнении интерактивных (онлайн) запросов, стоит использовать разумный объем оперативной памяти (128 Гб или более) для того, чтобы горячее подмножество данных поместилось в кеше страниц. Даже для объемов данных в \~50 Тб на сервер, использование 128 Гб оперативной памяти намного лучше для производительности выполнения запросов, чем 64 Гб. +Не выключайте overcommit. Значение `cat /proc/sys/vm/overcommit_memory` должно быть 0 или 1. Выполните: +``` +echo 0 | sudo tee /proc/sys/vm/overcommit_memory +``` + ## Файл подкачки Всегда отключайте файл подкачки. Единственной причиной этого не делать может быть только использование ClickHouse на личном ноутбуке. From 2c2d75ccebbb9e8f029b81b8e8d90a4bbbb33cb9 Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Fri, 23 Nov 2018 00:19:58 +0300 Subject: [PATCH 062/124] Don't allow exceptions without code (#3645) * Don't allow exceptions without code #3633 * Don't allow exceptions without code #3633 --- dbms/programs/benchmark/Benchmark.cpp | 6 +-- dbms/programs/client/Client.cpp | 3 -- .../performance-test/PerformanceTest.cpp | 46 ++++++++----------- .../server/InterserverIOHTTPHandler.cpp | 3 -- dbms/programs/server/Server.cpp | 3 +- .../AggregateFunctionWindowFunnel.h | 4 +- dbms/src/Common/ErrorCodes.cpp | 1 + dbms/src/Common/Exception.h | 8 ++-- dbms/src/Common/HashTable/SmallTable.h | 16 +++++-- dbms/src/Common/NetException.h | 2 +- dbms/src/Common/PoolWithFailoverBase.h | 2 +- dbms/src/Common/StatusFile.cpp | 2 +- dbms/src/Common/ZooKeeper/ZooKeeperHolder.cpp | 18 ++++++-- dbms/src/Core/tests/CMakeLists.txt | 3 -- dbms/src/Core/tests/exception.cpp | 25 ---------- .../tests/finish_sorting_stream.cpp | 13 +++++- dbms/src/Databases/DatabaseOrdinary.cpp | 2 +- .../Embedded/RegionsHierarchy.cpp | 12 ++++- .../Dictionaries/Embedded/RegionsNames.cpp | 11 ++++- dbms/src/Formats/CapnProtoRowInputStream.cpp | 20 +++++--- dbms/src/Functions/FunctionsComparison.h | 15 ++++-- dbms/src/Functions/FunctionsGeo.h | 7 ++- dbms/src/Functions/FunctionsStringArray.h | 2 +- dbms/src/Functions/FunctionsStringSearch.cpp | 3 +- dbms/src/Functions/GatherUtils/IArraySink.h | 15 +++++- dbms/src/Functions/GatherUtils/IArraySource.h | 14 +++++- dbms/src/Functions/GatherUtils/IValueSource.h | 15 +++++- dbms/src/Functions/GatherUtils/Sources.h | 4 +- dbms/src/Functions/arrayCumSum.cpp | 7 ++- dbms/src/Functions/arrayCumSumNonNegative.cpp | 7 ++- dbms/src/Functions/arrayDifference.cpp | 7 ++- dbms/src/Functions/arraySum.cpp | 7 ++- dbms/src/Functions/substringUTF8.cpp | 4 +- dbms/src/IO/WriteHelpers.cpp | 2 +- dbms/src/IO/tests/compressed_buffer.cpp | 2 +- dbms/src/IO/tests/limit_read_buffer2.cpp | 29 ++++++++---- dbms/src/IO/tests/zlib_buffers.cpp | 2 +- dbms/src/Interpreters/convertFieldToType.cpp | 5 +- .../tests/expression_analyzer.cpp | 10 +++- dbms/src/Storages/ColumnDefault.cpp | 9 +++- dbms/src/Storages/IStorage.h | 3 +- .../Storages/MergeTree/DataPartsExchange.cpp | 4 +- dbms/src/Storages/MergeTree/MergeTreeData.cpp | 8 ++-- .../MergeTree/MergeTreeDataWriter.cpp | 7 ++- .../ReplicatedMergeTreeBlockOutputStream.cpp | 6 ++- .../ReplicatedMergeTreePartCheckThread.cpp | 7 ++- dbms/src/Storages/MergeTree/checkDataPart.cpp | 5 +- dbms/src/Storages/StorageDictionary.cpp | 3 +- .../Storages/StorageReplicatedMergeTree.cpp | 4 +- .../System/StorageSystemZooKeeper.cpp | 7 ++- libs/libdaemon/src/BaseDaemon.cpp | 2 +- 51 files changed, 274 insertions(+), 148 deletions(-) delete mode 100644 dbms/src/Core/tests/exception.cpp diff --git a/dbms/programs/benchmark/Benchmark.cpp b/dbms/programs/benchmark/Benchmark.cpp index 161321f4d8c..8931774bac1 100644 --- a/dbms/programs/benchmark/Benchmark.cpp +++ b/dbms/programs/benchmark/Benchmark.cpp @@ -42,10 +42,8 @@ namespace DB namespace ErrorCodes { - extern const int POCO_EXCEPTION; - extern const int STD_EXCEPTION; - extern const int UNKNOWN_EXCEPTION; extern const int BAD_ARGUMENTS; + extern const int EMPTY_DATA_PASSED; } class Benchmark @@ -170,7 +168,7 @@ private: } if (queries.empty()) - throw Exception("Empty list of queries."); + throw Exception("Empty list of queries.", ErrorCodes::EMPTY_DATA_PASSED); std::cerr << "Loaded " << queries.size() << " queries.\n"; } diff --git a/dbms/programs/client/Client.cpp b/dbms/programs/client/Client.cpp index 516547de5d1..cba7b23b35f 100644 --- a/dbms/programs/client/Client.cpp +++ b/dbms/programs/client/Client.cpp @@ -86,9 +86,6 @@ namespace DB namespace ErrorCodes { - extern const int POCO_EXCEPTION; - extern const int STD_EXCEPTION; - extern const int UNKNOWN_EXCEPTION; extern const int NETWORK_ERROR; extern const int NO_DATA_TO_INSERT; extern const int BAD_ARGUMENTS; diff --git a/dbms/programs/performance-test/PerformanceTest.cpp b/dbms/programs/performance-test/PerformanceTest.cpp index e63d40e2db0..ebebedd8769 100644 --- a/dbms/programs/performance-test/PerformanceTest.cpp +++ b/dbms/programs/performance-test/PerformanceTest.cpp @@ -49,10 +49,10 @@ namespace DB { namespace ErrorCodes { - extern const int POCO_EXCEPTION; - extern const int STD_EXCEPTION; - extern const int UNKNOWN_EXCEPTION; extern const int NOT_IMPLEMENTED; + extern const int LOGICAL_ERROR; + extern const int BAD_ARGUMENTS; + extern const int FILE_DOESNT_EXIST; } static String pad(size_t padding) @@ -156,7 +156,7 @@ struct StopConditionsSet else if (key == "average_speed_not_changing_for_ms") average_speed_not_changing_for_ms.value = stop_conditions_view->getUInt64(key); else - throw DB::Exception("Met unkown stop condition: " + key); + throw DB::Exception("Met unkown stop condition: " + key, DB::ErrorCodes::LOGICAL_ERROR); ++initialized_count; } @@ -521,7 +521,7 @@ public: { if (input_files.size() < 1) { - throw DB::Exception("No tests were specified", 0); + throw DB::Exception("No tests were specified", DB::ErrorCodes::BAD_ARGUMENTS); } std::string name; @@ -694,7 +694,7 @@ private: size_t ram_size_needed = config->getUInt64("preconditions.ram_size"); size_t actual_ram = getMemoryAmount(); if (!actual_ram) - throw DB::Exception("ram_size precondition not available on this platform", ErrorCodes::NOT_IMPLEMENTED); + throw DB::Exception("ram_size precondition not available on this platform", DB::ErrorCodes::NOT_IMPLEMENTED); if (ram_size_needed > actual_ram) { @@ -868,12 +868,12 @@ private: if (!test_config->has("query") && !test_config->has("query_file")) { - throw DB::Exception("Missing query fields in test's config: " + test_name); + throw DB::Exception("Missing query fields in test's config: " + test_name, DB::ErrorCodes::BAD_ARGUMENTS); } if (test_config->has("query") && test_config->has("query_file")) { - throw DB::Exception("Found both query and query_file fields. Choose only one"); + throw DB::Exception("Found both query and query_file fields. Choose only one", DB::ErrorCodes::BAD_ARGUMENTS); } if (test_config->has("query")) @@ -885,7 +885,7 @@ private: { const String filename = test_config->getString("query_file"); if (filename.empty()) - throw DB::Exception("Empty file name"); + throw DB::Exception("Empty file name", DB::ErrorCodes::BAD_ARGUMENTS); bool tsv = fs::path(filename).extension().string() == ".tsv"; @@ -909,7 +909,7 @@ private: if (queries.empty()) { - throw DB::Exception("Did not find any query to execute: " + test_name); + throw DB::Exception("Did not find any query to execute: " + test_name, DB::ErrorCodes::BAD_ARGUMENTS); } if (test_config->has("substitutions")) @@ -929,7 +929,7 @@ private: if (!test_config->has("type")) { - throw DB::Exception("Missing type property in config: " + test_name); + throw DB::Exception("Missing type property in config: " + test_name, DB::ErrorCodes::BAD_ARGUMENTS); } String config_exec_type = test_config->getString("type"); @@ -938,7 +938,7 @@ private: else if (config_exec_type == "once") exec_type = ExecutionType::Once; else - throw DB::Exception("Unknown type " + config_exec_type + " in :" + test_name); + throw DB::Exception("Unknown type " + config_exec_type + " in :" + test_name, DB::ErrorCodes::BAD_ARGUMENTS); times_to_run = test_config->getUInt("times_to_run", 1); @@ -951,7 +951,7 @@ private: } if (stop_conditions_template.empty()) - throw DB::Exception("No termination conditions were found in config"); + throw DB::Exception("No termination conditions were found in config", DB::ErrorCodes::BAD_ARGUMENTS); for (size_t i = 0; i < times_to_run * queries.size(); ++i) stop_conditions_by_run.push_back(stop_conditions_template); @@ -978,7 +978,7 @@ private: else { if (lite_output) - throw DB::Exception("Specify main_metric for lite output"); + throw DB::Exception("Specify main_metric for lite output", DB::ErrorCodes::BAD_ARGUMENTS); } if (metrics.size() > 0) @@ -1023,22 +1023,14 @@ private: if (exec_type == ExecutionType::Loop) { for (const String & metric : metrics) - { if (std::find(non_loop_metrics.begin(), non_loop_metrics.end(), metric) != non_loop_metrics.end()) - { - throw DB::Exception("Wrong type of metric for loop execution type (" + metric + ")"); - } - } + throw DB::Exception("Wrong type of metric for loop execution type (" + metric + ")", DB::ErrorCodes::BAD_ARGUMENTS); } else { for (const String & metric : metrics) - { if (std::find(loop_metrics.begin(), loop_metrics.end(), metric) != loop_metrics.end()) - { - throw DB::Exception("Wrong type of metric for non-loop execution type (" + metric + ")"); - } - } + throw DB::Exception("Wrong type of metric for non-loop execution type (" + metric + ")", DB::ErrorCodes::BAD_ARGUMENTS); } } @@ -1439,7 +1431,7 @@ try if (input_files.empty()) { std::cerr << std::endl; - throw DB::Exception("Did not find any xml files"); + throw DB::Exception("Did not find any xml files", DB::ErrorCodes::BAD_ARGUMENTS); } else std::cerr << " found " << input_files.size() << " files." << std::endl; @@ -1454,7 +1446,7 @@ try fs::path file(filename); if (!fs::exists(file)) - throw DB::Exception("File '" + filename + "' does not exist"); + throw DB::Exception("File '" + filename + "' does not exist", DB::ErrorCodes::FILE_DOESNT_EXIST); if (fs::is_directory(file)) { @@ -1463,7 +1455,7 @@ try else { if (file.extension().string() != ".xml") - throw DB::Exception("File '" + filename + "' does not have .xml extension"); + throw DB::Exception("File '" + filename + "' does not have .xml extension", DB::ErrorCodes::BAD_ARGUMENTS); collected_files.push_back(filename); } } diff --git a/dbms/programs/server/InterserverIOHTTPHandler.cpp b/dbms/programs/server/InterserverIOHTTPHandler.cpp index 3895b2d8994..3c93ee1989a 100644 --- a/dbms/programs/server/InterserverIOHTTPHandler.cpp +++ b/dbms/programs/server/InterserverIOHTTPHandler.cpp @@ -19,9 +19,6 @@ namespace DB namespace ErrorCodes { extern const int ABORTED; - extern const int POCO_EXCEPTION; - extern const int STD_EXCEPTION; - extern const int UNKNOWN_EXCEPTION; extern const int TOO_MANY_SIMULTANEOUS_QUERIES; } diff --git a/dbms/programs/server/Server.cpp b/dbms/programs/server/Server.cpp index d8d662d8ed0..e82f0d3dd46 100644 --- a/dbms/programs/server/Server.cpp +++ b/dbms/programs/server/Server.cpp @@ -66,6 +66,7 @@ namespace ErrorCodes extern const int SUPPORT_IS_DISABLED; extern const int ARGUMENT_OUT_OF_BOUND; extern const int EXCESSIVE_ELEMENT_IN_CONFIG; + extern const int INVALID_CONFIG_PARAMETER; extern const int SYSTEM_ERROR; } @@ -74,7 +75,7 @@ static std::string getCanonicalPath(std::string && path) { Poco::trimInPlace(path); if (path.empty()) - throw Exception("path configuration parameter is empty"); + throw Exception("path configuration parameter is empty", ErrorCodes::INVALID_CONFIG_PARAMETER); if (path.back() != '/') path += '/'; return std::move(path); diff --git a/dbms/src/AggregateFunctions/AggregateFunctionWindowFunnel.h b/dbms/src/AggregateFunctions/AggregateFunctionWindowFunnel.h index c9a4e6b32a7..317637b1b69 100644 --- a/dbms/src/AggregateFunctions/AggregateFunctionWindowFunnel.h +++ b/dbms/src/AggregateFunctions/AggregateFunctionWindowFunnel.h @@ -16,10 +16,12 @@ namespace DB { + namespace ErrorCodes { extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; extern const int TOO_MANY_ARGUMENTS_FOR_FUNCTION; + extern const int ILLEGAL_TYPE_OF_ARGUMENT; } struct ComparePairFirst final @@ -191,7 +193,7 @@ public: const auto time_arg = arguments.front().get(); if (!WhichDataType(time_arg).isDateTime() && !WhichDataType(time_arg).isUInt32()) throw Exception{"Illegal type " + time_arg->getName() + " of first argument of aggregate function " + getName() - + ", must be DateTime or UInt32"}; + + ", must be DateTime or UInt32", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT}; for (const auto i : ext::range(1, arguments.size())) { diff --git a/dbms/src/Common/ErrorCodes.cpp b/dbms/src/Common/ErrorCodes.cpp index eb87e4f75cd..719e593e3b1 100644 --- a/dbms/src/Common/ErrorCodes.cpp +++ b/dbms/src/Common/ErrorCodes.cpp @@ -400,6 +400,7 @@ namespace ErrorCodes extern const int CANNOT_GETTIMEOFDAY = 423; extern const int CANNOT_LINK = 424; extern const int SYSTEM_ERROR = 425; + extern const int NULL_POINTER_DEREFERENCE = 426; extern const int KEEPER_EXCEPTION = 999; extern const int POCO_EXCEPTION = 1000; diff --git a/dbms/src/Common/Exception.h b/dbms/src/Common/Exception.h index 8e9044df769..d5e9d3112e1 100644 --- a/dbms/src/Common/Exception.h +++ b/dbms/src/Common/Exception.h @@ -23,10 +23,12 @@ class Exception : public Poco::Exception { public: Exception() {} /// For deferred initialization. - Exception(const std::string & msg, int code = 0) : Poco::Exception(msg, code) {} - Exception(const std::string & msg, const Exception & nested_exception, int code = 0) + Exception(const std::string & msg, int code) : Poco::Exception(msg, code) {} + Exception(const std::string & msg, const Exception & nested_exception, int code) : Poco::Exception(msg, nested_exception, code), trace(nested_exception.trace) {} - explicit Exception(const Poco::Exception & exc) : Poco::Exception(exc.displayText(), ErrorCodes::POCO_EXCEPTION) {} + + enum CreateFromPocoTag { CreateFromPoco }; + Exception(CreateFromPocoTag, const Poco::Exception & exc) : Poco::Exception(exc.displayText(), ErrorCodes::POCO_EXCEPTION) {} Exception * clone() const override { return new Exception(*this); } void rethrow() const override { throw *this; } diff --git a/dbms/src/Common/HashTable/SmallTable.h b/dbms/src/Common/HashTable/SmallTable.h index 73c2b358652..27dc8c00332 100644 --- a/dbms/src/Common/HashTable/SmallTable.h +++ b/dbms/src/Common/HashTable/SmallTable.h @@ -3,6 +3,15 @@ #include +namespace DB +{ + namespace ErrorCodes + { + extern const int INCORRECT_DATA; + } +} + + /** Replacement of the hash table for a small number (<10) of keys. * Implemented as an array with linear search. * The array is located inside the object. @@ -13,7 +22,6 @@ * you should check if the table is not full, * and do a `fallback` in this case (for example, use a real hash table). */ - template < typename Key, @@ -86,7 +94,7 @@ public: DB::readVarUInt(size, in); if (size > capacity) - throw DB::Exception("Illegal size"); + throw DB::Exception("Illegal size", DB::ErrorCodes::INCORRECT_DATA); is_initialized = true; } @@ -306,7 +314,7 @@ public: DB::readVarUInt(new_size, rb); if (new_size > capacity) - throw DB::Exception("Illegal size"); + throw DB::Exception("Illegal size", DB::ErrorCodes::INCORRECT_DATA); for (size_t i = 0; i < new_size; ++i) buf[i].read(rb); @@ -324,7 +332,7 @@ public: DB::readText(new_size, rb); if (new_size > capacity) - throw DB::Exception("Illegal size"); + throw DB::Exception("Illegal size", DB::ErrorCodes::INCORRECT_DATA); for (size_t i = 0; i < new_size; ++i) { diff --git a/dbms/src/Common/NetException.h b/dbms/src/Common/NetException.h index dfc3f118cae..ff7d7c5c5fe 100644 --- a/dbms/src/Common/NetException.h +++ b/dbms/src/Common/NetException.h @@ -9,7 +9,7 @@ namespace DB class NetException : public Exception { public: - NetException(const std::string & msg, int code = 0) : Exception(msg, code) {} + NetException(const std::string & msg, int code) : Exception(msg, code) {} NetException * clone() const override { return new NetException(*this); } void rethrow() const override { throw *this; } diff --git a/dbms/src/Common/PoolWithFailoverBase.h b/dbms/src/Common/PoolWithFailoverBase.h index 63420f12ac1..dc986a44ffa 100644 --- a/dbms/src/Common/PoolWithFailoverBase.h +++ b/dbms/src/Common/PoolWithFailoverBase.h @@ -305,7 +305,7 @@ void PoolWithFailoverBase::reportError(const Entry & entry) return; } } - throw DB::Exception("Can't find pool to report error."); + throw DB::Exception("Can't find pool to report error", DB::ErrorCodes::LOGICAL_ERROR); } template diff --git a/dbms/src/Common/StatusFile.cpp b/dbms/src/Common/StatusFile.cpp index 71cc65fe176..afe42262b55 100644 --- a/dbms/src/Common/StatusFile.cpp +++ b/dbms/src/Common/StatusFile.cpp @@ -59,7 +59,7 @@ StatusFile::StatusFile(const std::string & path_) if (-1 == flock_ret) { if (errno == EWOULDBLOCK) - throw Exception("Cannot lock file " + path + ". Another server instance in same directory is already running."); + throw Exception("Cannot lock file " + path + ". Another server instance in same directory is already running.", ErrorCodes::CANNOT_OPEN_FILE); else throwFromErrno("Cannot lock file " + path, ErrorCodes::CANNOT_OPEN_FILE); } diff --git a/dbms/src/Common/ZooKeeper/ZooKeeperHolder.cpp b/dbms/src/Common/ZooKeeper/ZooKeeperHolder.cpp index 1b49c22cf46..9343bb4addc 100644 --- a/dbms/src/Common/ZooKeeper/ZooKeeperHolder.cpp +++ b/dbms/src/Common/ZooKeeper/ZooKeeperHolder.cpp @@ -1,5 +1,15 @@ #include "ZooKeeperHolder.h" + +namespace DB +{ + namespace ErrorCodes + { + extern const int NULL_POINTER_DEREFERENCE; + } +} + + using namespace zkutil; ZooKeeperHolder::UnstorableZookeeperHandler ZooKeeperHolder::getZooKeeper() @@ -47,7 +57,7 @@ ZooKeeperHolder::UnstorableZookeeperHandler::UnstorableZookeeperHandler(ZooKeepe ZooKeeper * ZooKeeperHolder::UnstorableZookeeperHandler::operator->() { if (zk_ptr == nullptr) - throw DB::Exception(nullptr_exception_message); + throw DB::Exception(nullptr_exception_message, DB::ErrorCodes::NULL_POINTER_DEREFERENCE); return zk_ptr.get(); } @@ -55,20 +65,20 @@ ZooKeeper * ZooKeeperHolder::UnstorableZookeeperHandler::operator->() const ZooKeeper * ZooKeeperHolder::UnstorableZookeeperHandler::operator->() const { if (zk_ptr == nullptr) - throw DB::Exception(nullptr_exception_message); + throw DB::Exception(nullptr_exception_message, DB::ErrorCodes::NULL_POINTER_DEREFERENCE); return zk_ptr.get(); } ZooKeeper & ZooKeeperHolder::UnstorableZookeeperHandler::operator*() { if (zk_ptr == nullptr) - throw DB::Exception(nullptr_exception_message); + throw DB::Exception(nullptr_exception_message, DB::ErrorCodes::NULL_POINTER_DEREFERENCE); return *zk_ptr; } const ZooKeeper & ZooKeeperHolder::UnstorableZookeeperHandler::operator*() const { if (zk_ptr == nullptr) - throw DB::Exception(nullptr_exception_message); + throw DB::Exception(nullptr_exception_message, DB::ErrorCodes::NULL_POINTER_DEREFERENCE); return *zk_ptr; } diff --git a/dbms/src/Core/tests/CMakeLists.txt b/dbms/src/Core/tests/CMakeLists.txt index 4748c61e182..b5b08bb161d 100644 --- a/dbms/src/Core/tests/CMakeLists.txt +++ b/dbms/src/Core/tests/CMakeLists.txt @@ -1,6 +1,3 @@ -add_executable (exception exception.cpp) -target_link_libraries (exception PRIVATE clickhouse_common_io) - add_executable (string_pool string_pool.cpp) target_link_libraries (string_pool PRIVATE clickhouse_common_io) target_include_directories (string_pool SYSTEM BEFORE PRIVATE ${SPARCEHASH_INCLUDE_DIR}) diff --git a/dbms/src/Core/tests/exception.cpp b/dbms/src/Core/tests/exception.cpp deleted file mode 100644 index d0e66f75ff7..00000000000 --- a/dbms/src/Core/tests/exception.cpp +++ /dev/null @@ -1,25 +0,0 @@ -#include - -#include - -#include - - -int main(int, char **) -{ - try - { - //throw Poco::Net::ConnectionRefusedException(); - throw DB::Exception(Poco::Net::ConnectionRefusedException()); - } - catch (const DB::Exception & e) - { - std::cerr << e.displayText() << std::endl; - } - catch (const Poco::Exception & e) - { - std::cerr << e.displayText() << std::endl; - } - - return 0; -} diff --git a/dbms/src/DataStreams/tests/finish_sorting_stream.cpp b/dbms/src/DataStreams/tests/finish_sorting_stream.cpp index 7627d1712b2..861965e1e1c 100644 --- a/dbms/src/DataStreams/tests/finish_sorting_stream.cpp +++ b/dbms/src/DataStreams/tests/finish_sorting_stream.cpp @@ -13,6 +13,15 @@ using namespace DB; +namespace DB +{ + namespace ErrorCodes + { + extern const int LOGICAL_ERROR; + } +} + + int main(int argc, char ** argv) { srand(123456); @@ -75,7 +84,7 @@ int main(int argc, char ** argv) } if (res_block.rows() != n * m) - throw Exception("Result block size mismatch"); + throw Exception("Result block size mismatch", ErrorCodes::LOGICAL_ERROR); const auto & columns = res_block.getColumns(); @@ -86,7 +95,7 @@ int main(int argc, char ** argv) if (res < 0) break; else if (res > 0) - throw Exception("Result stream not sorted"); + throw Exception("Result stream not sorted", ErrorCodes::LOGICAL_ERROR); } stopwatch.stop(); diff --git a/dbms/src/Databases/DatabaseOrdinary.cpp b/dbms/src/Databases/DatabaseOrdinary.cpp index 7bb4ae298e6..e0fe4294e7d 100644 --- a/dbms/src/Databases/DatabaseOrdinary.cpp +++ b/dbms/src/Databases/DatabaseOrdinary.cpp @@ -392,7 +392,7 @@ void DatabaseOrdinary::renameTable( catch (const Poco::Exception & e) { /// Better diagnostics. - throw Exception{e}; + throw Exception{Exception::CreateFromPoco, e}; } ASTPtr ast = getQueryFromMetadata(detail::getTableMetadataPath(metadata_path, table_name)); diff --git a/dbms/src/Dictionaries/Embedded/RegionsHierarchy.cpp b/dbms/src/Dictionaries/Embedded/RegionsHierarchy.cpp index 978d7b9e496..71b5adc6afd 100644 --- a/dbms/src/Dictionaries/Embedded/RegionsHierarchy.cpp +++ b/dbms/src/Dictionaries/Embedded/RegionsHierarchy.cpp @@ -10,6 +10,15 @@ #include +namespace DB +{ + namespace ErrorCodes + { + extern const int INCORRECT_DATA; + } +} + + RegionsHierarchy::RegionsHierarchy(IRegionsHierarchyDataSourcePtr data_source_) : data_source(data_source_) { @@ -49,7 +58,8 @@ void RegionsHierarchy::reload() if (region_entry.id > max_region_id) { if (region_entry.id > max_size) - throw DB::Exception("Region id is too large: " + DB::toString(region_entry.id) + ", should be not more than " + DB::toString(max_size)); + throw DB::Exception("Region id is too large: " + DB::toString(region_entry.id) + ", should be not more than " + DB::toString(max_size), + DB::ErrorCodes::INCORRECT_DATA); max_region_id = region_entry.id; diff --git a/dbms/src/Dictionaries/Embedded/RegionsNames.cpp b/dbms/src/Dictionaries/Embedded/RegionsNames.cpp index 4966f30455a..2c48f1f18c2 100644 --- a/dbms/src/Dictionaries/Embedded/RegionsNames.cpp +++ b/dbms/src/Dictionaries/Embedded/RegionsNames.cpp @@ -8,6 +8,14 @@ #include +namespace DB +{ + namespace ErrorCodes + { + extern const int INCORRECT_DATA; + } +} + RegionsNames::RegionsNames(IRegionsNamesDataProviderPtr data_provider) { @@ -78,7 +86,8 @@ void RegionsNames::reload() max_region_id = name_entry.id; if (name_entry.id > max_size) - throw DB::Exception("Region id is too large: " + DB::toString(name_entry.id) + ", should be not more than " + DB::toString(max_size)); + throw DB::Exception("Region id is too large: " + DB::toString(name_entry.id) + ", should be not more than " + DB::toString(max_size), + DB::ErrorCodes::INCORRECT_DATA); } while (name_entry.id >= new_names_refs.size()) diff --git a/dbms/src/Formats/CapnProtoRowInputStream.cpp b/dbms/src/Formats/CapnProtoRowInputStream.cpp index d7cc02a3218..1af37a85cee 100644 --- a/dbms/src/Formats/CapnProtoRowInputStream.cpp +++ b/dbms/src/Formats/CapnProtoRowInputStream.cpp @@ -18,6 +18,13 @@ namespace DB { +namespace ErrorCodes +{ + extern const int BAD_TYPE_OF_FIELD; + extern const int BAD_ARGUMENTS; + extern const int THERE_IS_NO_COLUMN; +} + static String getSchemaPath(const String & schema_dir, const String & schema_file) { return schema_dir + escapeForFileName(schema_file) + ".capnp"; @@ -42,7 +49,7 @@ Field convertNodeToField(capnp::DynamicValue::Reader value) switch (value.getType()) { case capnp::DynamicValue::UNKNOWN: - throw Exception("Unknown field type"); + throw Exception("Unknown field type", ErrorCodes::BAD_TYPE_OF_FIELD); case capnp::DynamicValue::VOID: return Field(); case capnp::DynamicValue::BOOL: @@ -87,9 +94,9 @@ Field convertNodeToField(capnp::DynamicValue::Reader value) return field; } case capnp::DynamicValue::CAPABILITY: - throw Exception("CAPABILITY type not supported"); + throw Exception("CAPABILITY type not supported", ErrorCodes::BAD_TYPE_OF_FIELD); case capnp::DynamicValue::ANY_POINTER: - throw Exception("ANY_POINTER type not supported"); + throw Exception("ANY_POINTER type not supported", ErrorCodes::BAD_TYPE_OF_FIELD); } return Field(); } @@ -99,7 +106,7 @@ capnp::StructSchema::Field getFieldOrThrow(capnp::StructSchema node, const std:: KJ_IF_MAYBE(child, node.findFieldByName(field)) return *child; else - throw Exception("Field " + field + " doesn't exist in schema " + node.getShortDisplayName().cStr()); + throw Exception("Field " + field + " doesn't exist in schema " + node.getShortDisplayName().cStr(), ErrorCodes::THERE_IS_NO_COLUMN); } void CapnProtoRowInputStream::createActions(const NestedFieldList & sortedFields, capnp::StructSchema reader) @@ -135,7 +142,7 @@ void CapnProtoRowInputStream::createActions(const NestedFieldList & sortedFields break; // Collect list } else - throw Exception("Field " + field.tokens[level] + "is neither Struct nor List"); + throw Exception("Field " + field.tokens[level] + "is neither Struct nor List", ErrorCodes::BAD_TYPE_OF_FIELD); } // Read field from the structure @@ -290,7 +297,8 @@ void registerInputFormatCapnProto(FormatFactory & factory) auto schema_and_root = context.getSettingsRef().format_schema.toString(); boost::split(tokens, schema_and_root, boost::is_any_of(":")); if (tokens.size() != 2) - throw Exception("Format CapnProto requires 'format_schema' setting to have a schema_file:root_object format, e.g. 'schema.capnp:Message'"); + throw Exception("Format CapnProto requires 'format_schema' setting to have a schema_file:root_object format, e.g. 'schema.capnp:Message'", + ErrorCodes::BAD_ARGUMENTS); const String & schema_dir = context.getFormatSchemaPath(); diff --git a/dbms/src/Functions/FunctionsComparison.h b/dbms/src/Functions/FunctionsComparison.h index 15d253e7788..d785cf3071c 100644 --- a/dbms/src/Functions/FunctionsComparison.h +++ b/dbms/src/Functions/FunctionsComparison.h @@ -38,6 +38,15 @@ namespace DB { +namespace ErrorCodes +{ + extern const int TOO_LARGE_STRING_SIZE; + extern const int ILLEGAL_COLUMN; + extern const int ILLEGAL_TYPE_OF_ARGUMENT; + extern const int LOGICAL_ERROR; + extern const int NOT_IMPLEMENTED; +} + /** Comparison functions: ==, !=, <, >, <=, >=. * The comparison functions always return 0 or 1 (UInt8). @@ -833,7 +842,7 @@ private: ReadBufferFromMemory in(string_value.data, string_value.size); readDateText(date, in); if (!in.eof()) - throw Exception("String is too long for Date: " + string_value.toString()); + throw Exception("String is too long for Date: " + string_value.toString(), ErrorCodes::TOO_LARGE_STRING_SIZE); ColumnPtr parsed_const_date_holder = DataTypeDate().createColumnConst(input_rows_count, date); const ColumnConst * parsed_const_date = static_cast(parsed_const_date_holder.get()); @@ -847,7 +856,7 @@ private: ReadBufferFromMemory in(string_value.data, string_value.size); readDateTimeText(date_time, in); if (!in.eof()) - throw Exception("String is too long for DateTime: " + string_value.toString()); + throw Exception("String is too long for DateTime: " + string_value.toString(), ErrorCodes::TOO_LARGE_STRING_SIZE); ColumnPtr parsed_const_date_time_holder = DataTypeDateTime().createColumnConst(input_rows_count, UInt64(date_time)); const ColumnConst * parsed_const_date_time = static_cast(parsed_const_date_time_holder.get()); @@ -861,7 +870,7 @@ private: ReadBufferFromMemory in(string_value.data, string_value.size); readText(uuid, in); if (!in.eof()) - throw Exception("String is too long for UUID: " + string_value.toString()); + throw Exception("String is too long for UUID: " + string_value.toString(), ErrorCodes::TOO_LARGE_STRING_SIZE); ColumnPtr parsed_const_uuid_holder = DataTypeUUID().createColumnConst(input_rows_count, uuid); const ColumnConst * parsed_const_uuid = static_cast(parsed_const_uuid_holder.get()); diff --git a/dbms/src/Functions/FunctionsGeo.h b/dbms/src/Functions/FunctionsGeo.h index 456198b9e0d..5ac23138a88 100644 --- a/dbms/src/Functions/FunctionsGeo.h +++ b/dbms/src/Functions/FunctionsGeo.h @@ -20,6 +20,8 @@ namespace DB namespace ErrorCodes { extern const int ARGUMENT_OUT_OF_BOUND; + extern const int TOO_MANY_ARGUMENTS_FOR_FUNCTION; + extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; extern const int ILLEGAL_COLUMN; extern const int LOGICAL_ERROR; } @@ -200,14 +202,15 @@ private: if (arguments.size() < 6 || arguments.size() % 4 != 2) { throw Exception( - "Incorrect number of arguments of function " + getName() + ". Must be 2 for your point plus 4 * N for ellipses (x_i, y_i, a_i, b_i)."); + "Incorrect number of arguments of function " + getName() + ". Must be 2 for your point plus 4 * N for ellipses (x_i, y_i, a_i, b_i).", + ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); } /// For array on stack, see below. if (arguments.size() > 10000) { throw Exception( - "Number of arguments of function " + getName() + " is too large."); + "Number of arguments of function " + getName() + " is too large.", ErrorCodes::TOO_MANY_ARGUMENTS_FOR_FUNCTION); } for (const auto arg_idx : ext::range(0, arguments.size())) diff --git a/dbms/src/Functions/FunctionsStringArray.h b/dbms/src/Functions/FunctionsStringArray.h index 615e9cdb7cc..75b27751fca 100644 --- a/dbms/src/Functions/FunctionsStringArray.h +++ b/dbms/src/Functions/FunctionsStringArray.h @@ -145,7 +145,7 @@ public: String sep_str = col->getValue(); if (sep_str.size() != 1) - throw Exception("Illegal separator for function " + getName() + ". Must be exactly one byte."); + throw Exception("Illegal separator for function " + getName() + ". Must be exactly one byte.", ErrorCodes::BAD_ARGUMENTS); sep = sep_str[0]; } diff --git a/dbms/src/Functions/FunctionsStringSearch.cpp b/dbms/src/Functions/FunctionsStringSearch.cpp index 243120d767f..584a625039f 100644 --- a/dbms/src/Functions/FunctionsStringSearch.cpp +++ b/dbms/src/Functions/FunctionsStringSearch.cpp @@ -26,6 +26,7 @@ namespace DB namespace ErrorCodes { extern const int BAD_ARGUMENTS; + extern const int ILLEGAL_COLUMN; } @@ -976,7 +977,7 @@ public: const ColumnPtr column_replacement = block.getByPosition(arguments[2]).column; if (!column_needle->isColumnConst() || !column_replacement->isColumnConst()) - throw Exception("2nd and 3rd arguments of function " + getName() + " must be constants."); + throw Exception("2nd and 3rd arguments of function " + getName() + " must be constants.", ErrorCodes::ILLEGAL_COLUMN); const IColumn * c1 = block.getByPosition(arguments[1]).column.get(); const IColumn * c2 = block.getByPosition(arguments[2]).column.get(); diff --git a/dbms/src/Functions/GatherUtils/IArraySink.h b/dbms/src/Functions/GatherUtils/IArraySink.h index a3f8433c75a..e7a3da1d5bd 100644 --- a/dbms/src/Functions/GatherUtils/IArraySink.h +++ b/dbms/src/Functions/GatherUtils/IArraySink.h @@ -1,8 +1,17 @@ #pragma once + #include #include -namespace DB::GatherUtils +namespace DB +{ + +namespace ErrorCodes +{ + extern const int NOT_IMPLEMENTED; +} + +namespace GatherUtils { struct IArraySink @@ -11,7 +20,7 @@ struct IArraySink virtual void accept(ArraySinkVisitor &) { - throw Exception("Accept not implemented for " + demangle(typeid(*this).name())); + throw Exception("Accept not implemented for " + demangle(typeid(*this).name()), ErrorCodes::NOT_IMPLEMENTED); } }; @@ -19,3 +28,5 @@ template class ArraySinkImpl : public Visitable {}; } + +} diff --git a/dbms/src/Functions/GatherUtils/IArraySource.h b/dbms/src/Functions/GatherUtils/IArraySource.h index dd623957260..a33f23036cf 100644 --- a/dbms/src/Functions/GatherUtils/IArraySource.h +++ b/dbms/src/Functions/GatherUtils/IArraySource.h @@ -3,7 +3,15 @@ #include #include -namespace DB::GatherUtils +namespace DB +{ + +namespace ErrorCodes +{ + extern const int NOT_IMPLEMENTED; +} + +namespace GatherUtils { struct IArraySource @@ -18,7 +26,7 @@ struct IArraySource virtual void accept(ArraySourceVisitor &) { - throw Exception("Accept not implemented for " + demangle(typeid(*this).name())); + throw Exception("Accept not implemented for " + demangle(typeid(*this).name()), ErrorCodes::NOT_IMPLEMENTED); } }; @@ -26,3 +34,5 @@ template class ArraySourceImpl : public Visitable {}; } + +} diff --git a/dbms/src/Functions/GatherUtils/IValueSource.h b/dbms/src/Functions/GatherUtils/IValueSource.h index 9983a24e9fe..bd26424e1b3 100644 --- a/dbms/src/Functions/GatherUtils/IValueSource.h +++ b/dbms/src/Functions/GatherUtils/IValueSource.h @@ -1,8 +1,17 @@ #pragma once + #include #include -namespace DB::GatherUtils +namespace DB +{ + +namespace ErrorCodes +{ + extern const int NOT_IMPLEMENTED; +} + +namespace GatherUtils { struct IValueSource @@ -11,7 +20,7 @@ struct IValueSource virtual void accept(ValueSourceVisitor &) { - throw Exception("Accept not implemented for " + demangle(typeid(*this).name())); + throw Exception("Accept not implemented for " + demangle(typeid(*this).name()), ErrorCodes::NOT_IMPLEMENTED); } virtual bool isConst() const { return false; } @@ -21,3 +30,5 @@ template class ValueSourceImpl : public Visitable {}; } + +} diff --git a/dbms/src/Functions/GatherUtils/Sources.h b/dbms/src/Functions/GatherUtils/Sources.h index ccfeb25a213..16e78bea26b 100644 --- a/dbms/src/Functions/GatherUtils/Sources.h +++ b/dbms/src/Functions/GatherUtils/Sources.h @@ -142,7 +142,7 @@ struct ConstSource : public Base else throw Exception( "accept(ArraySourceVisitor &) is not implemented for " + demangle(typeid(ConstSource).name()) - + " because " + demangle(typeid(Base).name()) + " is not derived from IArraySource "); + + " because " + demangle(typeid(Base).name()) + " is not derived from IArraySource", ErrorCodes::NOT_IMPLEMENTED); } virtual void accept(ValueSourceVisitor & visitor) // override @@ -152,7 +152,7 @@ struct ConstSource : public Base else throw Exception( "accept(ValueSourceVisitor &) is not implemented for " + demangle(typeid(ConstSource).name()) - + " because " + demangle(typeid(Base).name()) + " is not derived from IValueSource "); + + " because " + demangle(typeid(Base).name()) + " is not derived from IValueSource", ErrorCodes::NOT_IMPLEMENTED); } void next() diff --git a/dbms/src/Functions/arrayCumSum.cpp b/dbms/src/Functions/arrayCumSum.cpp index 756f57b5b73..5351ca745d6 100644 --- a/dbms/src/Functions/arrayCumSum.cpp +++ b/dbms/src/Functions/arrayCumSum.cpp @@ -7,6 +7,11 @@ namespace DB { +namespace ErrorCodes +{ + extern const int ILLEGAL_COLUMN; +} + struct ArrayCumSumImpl { static bool needBoolean() { return false; } @@ -108,7 +113,7 @@ struct ArrayCumSumImpl executeType(mapped, array, res)) return res; else - throw Exception("Unexpected column for arrayCumSum: " + mapped->getName()); + throw Exception("Unexpected column for arrayCumSum: " + mapped->getName(), ErrorCodes::ILLEGAL_COLUMN); } }; diff --git a/dbms/src/Functions/arrayCumSumNonNegative.cpp b/dbms/src/Functions/arrayCumSumNonNegative.cpp index 4dc4c9d0ef1..59c80cef858 100644 --- a/dbms/src/Functions/arrayCumSumNonNegative.cpp +++ b/dbms/src/Functions/arrayCumSumNonNegative.cpp @@ -7,6 +7,11 @@ namespace DB { +namespace ErrorCodes +{ + extern const int ILLEGAL_COLUMN; +} + /** arrayCumSumNonNegative() - returns an array with cumulative sums of the original. (If value < 0 -> 0). */ struct ArrayCumSumNonNegativeImpl @@ -88,7 +93,7 @@ struct ArrayCumSumNonNegativeImpl executeType(mapped, array, res)) return res; else - throw Exception("Unexpected column for arrayCumSumNonNegativeImpl: " + mapped->getName()); + throw Exception("Unexpected column for arrayCumSumNonNegativeImpl: " + mapped->getName(), ErrorCodes::ILLEGAL_COLUMN); } }; diff --git a/dbms/src/Functions/arrayDifference.cpp b/dbms/src/Functions/arrayDifference.cpp index 847d1929db3..1228591aa04 100644 --- a/dbms/src/Functions/arrayDifference.cpp +++ b/dbms/src/Functions/arrayDifference.cpp @@ -7,6 +7,11 @@ namespace DB { +namespace ErrorCodes +{ + extern const int ILLEGAL_COLUMN; +} + /** arrayDifference() - returns an array with the difference between all pairs of neighboring elements. */ struct ArrayDifferenceImpl @@ -85,7 +90,7 @@ struct ArrayDifferenceImpl executeType(mapped, array, res)) return res; else - throw Exception("Unexpected column for arrayDifference: " + mapped->getName()); + throw Exception("Unexpected column for arrayDifference: " + mapped->getName(), ErrorCodes::ILLEGAL_COLUMN); } }; diff --git a/dbms/src/Functions/arraySum.cpp b/dbms/src/Functions/arraySum.cpp index c3d22bf62b6..0f6db12f1e5 100644 --- a/dbms/src/Functions/arraySum.cpp +++ b/dbms/src/Functions/arraySum.cpp @@ -7,6 +7,11 @@ namespace DB { +namespace ErrorCodes +{ + extern const int ILLEGAL_COLUMN; +} + struct ArraySumImpl { static bool needBoolean() { return false; } @@ -93,7 +98,7 @@ struct ArraySumImpl executeType(mapped, offsets, res)) return res; else - throw Exception("Unexpected column for arraySum: " + mapped->getName()); + throw Exception("Unexpected column for arraySum: " + mapped->getName(), ErrorCodes::ILLEGAL_COLUMN); } }; diff --git a/dbms/src/Functions/substringUTF8.cpp b/dbms/src/Functions/substringUTF8.cpp index aa28a541923..3e384bcba04 100644 --- a/dbms/src/Functions/substringUTF8.cpp +++ b/dbms/src/Functions/substringUTF8.cpp @@ -127,13 +127,13 @@ public: const ColumnPtr column_length = block.getByPosition(arguments[2]).column; if (!column_start->isColumnConst() || !column_length->isColumnConst()) - throw Exception("2nd and 3rd arguments of function " + getName() + " must be constants."); + throw Exception("2nd and 3rd arguments of function " + getName() + " must be constants.", ErrorCodes::ILLEGAL_COLUMN); Field start_field = (*block.getByPosition(arguments[1]).column)[0]; Field length_field = (*block.getByPosition(arguments[2]).column)[0]; if (start_field.getType() != Field::Types::UInt64 || length_field.getType() != Field::Types::UInt64) - throw Exception("2nd and 3rd arguments of function " + getName() + " must be non-negative and must have UInt type."); + throw Exception("2nd and 3rd arguments of function " + getName() + " must be non-negative and must have UInt type.", ErrorCodes::ILLEGAL_COLUMN); UInt64 start = start_field.get(); UInt64 length = length_field.get(); diff --git a/dbms/src/IO/WriteHelpers.cpp b/dbms/src/IO/WriteHelpers.cpp index ba0e106b42b..5dc2358c4c0 100644 --- a/dbms/src/IO/WriteHelpers.cpp +++ b/dbms/src/IO/WriteHelpers.cpp @@ -64,7 +64,7 @@ void writeException(const Exception & e, WriteBuffer & buf, bool with_stack_trac writeBinary(has_nested, buf); if (has_nested) - writeException(Exception(*e.nested()), buf, with_stack_trace); + writeException(Exception(Exception::CreateFromPoco, *e.nested()), buf, with_stack_trace); } } diff --git a/dbms/src/IO/tests/compressed_buffer.cpp b/dbms/src/IO/tests/compressed_buffer.cpp index 22e5017f444..bcb8d7ae9ce 100644 --- a/dbms/src/IO/tests/compressed_buffer.cpp +++ b/dbms/src/IO/tests/compressed_buffer.cpp @@ -55,7 +55,7 @@ int main(int, char **) { std::stringstream s; s << "Failed!, read: " << x << ", expected: " << i; - throw DB::Exception(s.str()); + throw DB::Exception(s.str(), 0); } } stopwatch.stop(); diff --git a/dbms/src/IO/tests/limit_read_buffer2.cpp b/dbms/src/IO/tests/limit_read_buffer2.cpp index f9a9218174d..826fb048a0c 100644 --- a/dbms/src/IO/tests/limit_read_buffer2.cpp +++ b/dbms/src/IO/tests/limit_read_buffer2.cpp @@ -6,6 +6,15 @@ #include +namespace DB +{ + namespace ErrorCodes + { + extern const int LOGICAL_ERROR; + } +} + + int main(int, char **) try { @@ -31,18 +40,18 @@ try if (limit_in.count() != 1) { s << "Failed!, incorrect count(): " << limit_in.count(); - throw Exception(s.str()); + throw Exception(s.str(), ErrorCodes::LOGICAL_ERROR); } if (in.count() != limit_in.count()) { s << "Failed!, incorrect underlying buffer's count(): " << in.count(); - throw Exception(s.str()); + throw Exception(s.str(), ErrorCodes::LOGICAL_ERROR); } if (src != dst) { s << "Failed!, incorrect destination value, read: " << dst << ", expected: " << src; - throw Exception(s.str()); + throw Exception(s.str(), ErrorCodes::LOGICAL_ERROR); } } { @@ -65,20 +74,20 @@ try if (in.count() != 2) { s << "Failed!, Incorrect underlying buffer's count: " << in.count() << ", expected: " << 2; - throw Exception(s.str()); + throw Exception(s.str(), ErrorCodes::LOGICAL_ERROR); } if (limit_in.count() != 1) { s << "Failed!, Incorrect count: " << limit_in.count() << ", expected: " << 1; - throw Exception(s.str()); + throw Exception(s.str(), ErrorCodes::LOGICAL_ERROR); } } if (dst != "b") { s << "Failed!, Incorrect destination value: " << dst << ", expected 'b'"; - throw Exception(dst); + throw Exception(dst, ErrorCodes::LOGICAL_ERROR); } char y; @@ -86,14 +95,14 @@ try if (y != 'c') { s << "Failed!, Read incorrect value from underlying buffer: " << y << ", expected 'c'"; - throw Exception(s.str()); + throw Exception(s.str(), ErrorCodes::LOGICAL_ERROR); } while (!in.eof()) in.ignore(); if (in.count() != 3) { s << "Failed!, Incorrect final count from underlying buffer: " << in.count() << ", expected: 3"; - throw Exception(s.str()); + throw Exception(s.str(), ErrorCodes::LOGICAL_ERROR); } } @@ -110,14 +119,14 @@ try if (limit_in.count() != 1) { s << "Failed!, Incorrect count: " << limit_in.count() << ", expected: " << 1; - throw Exception(s.str()); + throw Exception(s.str(), ErrorCodes::LOGICAL_ERROR); } } if (in.count() != 1) { s << "Failed!, Incorrect final count from underlying buffer: " << in.count() << ", expected: 1"; - throw Exception(s.str()); + throw Exception(s.str(), ErrorCodes::LOGICAL_ERROR); } } diff --git a/dbms/src/IO/tests/zlib_buffers.cpp b/dbms/src/IO/tests/zlib_buffers.cpp index af4b003766e..2c55509deb8 100644 --- a/dbms/src/IO/tests/zlib_buffers.cpp +++ b/dbms/src/IO/tests/zlib_buffers.cpp @@ -51,7 +51,7 @@ try inflating_buf.ignore(); if (x != i) - throw DB::Exception("Failed!, read: " + std::to_string(x) + ", expected: " + std::to_string(i)); + throw DB::Exception("Failed!, read: " + std::to_string(x) + ", expected: " + std::to_string(i), 0); } stopwatch.stop(); std::cout << "Reading done. Elapsed: " << stopwatch.elapsedSeconds() << " s." diff --git a/dbms/src/Interpreters/convertFieldToType.cpp b/dbms/src/Interpreters/convertFieldToType.cpp index 7759c2c9b3b..ac9936221d9 100644 --- a/dbms/src/Interpreters/convertFieldToType.cpp +++ b/dbms/src/Interpreters/convertFieldToType.cpp @@ -31,6 +31,7 @@ namespace ErrorCodes { extern const int LOGICAL_ERROR; extern const int TYPE_MISMATCH; + extern const int TOO_LARGE_STRING_SIZE; } @@ -123,7 +124,7 @@ DayNum stringToDate(const String & s) readDateText(date, in); if (!in.eof()) - throw Exception("String is too long for Date: " + s); + throw Exception("String is too long for Date: " + s, ErrorCodes::TOO_LARGE_STRING_SIZE); return date; } @@ -135,7 +136,7 @@ UInt64 stringToDateTime(const String & s) readDateTimeText(date_time, in); if (!in.eof()) - throw Exception("String is too long for DateTime: " + s); + throw Exception("String is too long for DateTime: " + s, ErrorCodes::TOO_LARGE_STRING_SIZE); return UInt64(date_time); } diff --git a/dbms/src/Interpreters/tests/expression_analyzer.cpp b/dbms/src/Interpreters/tests/expression_analyzer.cpp index 8fb5fd88c19..fa9cef39ad0 100644 --- a/dbms/src/Interpreters/tests/expression_analyzer.cpp +++ b/dbms/src/Interpreters/tests/expression_analyzer.cpp @@ -20,6 +20,14 @@ using namespace DB; +namespace DB +{ + namespace ErrorCodes + { + extern const int SYNTAX_ERROR; + } +} + struct TestEntry { String query; @@ -67,7 +75,7 @@ private: auto text = query.data(); if (ASTPtr ast = tryParseQuery(parser, text, text + query.size(), message, false, "", false, 0)) return ast; - throw Exception(message); + throw Exception(message, ErrorCodes::SYNTAX_ERROR); } }; diff --git a/dbms/src/Storages/ColumnDefault.cpp b/dbms/src/Storages/ColumnDefault.cpp index 0ff885ab1e5..fdec46958a1 100644 --- a/dbms/src/Storages/ColumnDefault.cpp +++ b/dbms/src/Storages/ColumnDefault.cpp @@ -5,6 +5,11 @@ namespace DB { +namespace ErrorCodes +{ + extern const int LOGICAL_ERROR; +} + ColumnDefaultKind columnDefaultKindFromString(const std::string & str) { @@ -15,7 +20,7 @@ ColumnDefaultKind columnDefaultKindFromString(const std::string & str) }; const auto it = map.find(str); - return it != std::end(map) ? it->second : throw Exception{"Unknown column default specifier: " + str}; + return it != std::end(map) ? it->second : throw Exception{"Unknown column default specifier: " + str, ErrorCodes::LOGICAL_ERROR}; } @@ -28,7 +33,7 @@ std::string toString(const ColumnDefaultKind kind) }; const auto it = map.find(kind); - return it != std::end(map) ? it->second : throw Exception{"Invalid ColumnDefaultKind"}; + return it != std::end(map) ? it->second : throw Exception{"Invalid ColumnDefaultKind", ErrorCodes::LOGICAL_ERROR}; } diff --git a/dbms/src/Storages/IStorage.h b/dbms/src/Storages/IStorage.h index e343910007b..326cc86f606 100644 --- a/dbms/src/Storages/IStorage.h +++ b/dbms/src/Storages/IStorage.h @@ -19,6 +19,7 @@ namespace DB namespace ErrorCodes { extern const int TABLE_IS_DROPPED; + extern const int NOT_IMPLEMENTED; } class Context; @@ -295,7 +296,7 @@ public: virtual bool mayBenefitFromIndexForIn(const ASTPtr & /* left_in_operand */) const { return false; } /// Checks validity of the data - virtual bool checkData() const { throw DB::Exception("Check query is not supported for " + getName() + " storage"); } + virtual bool checkData() const { throw Exception("Check query is not supported for " + getName() + " storage", ErrorCodes::NOT_IMPLEMENTED); } /// Checks that table could be dropped right now /// Otherwise - throws an exception with detailed information. diff --git a/dbms/src/Storages/MergeTree/DataPartsExchange.cpp b/dbms/src/Storages/MergeTree/DataPartsExchange.cpp index 00e005a3907..ab1d28ee656 100644 --- a/dbms/src/Storages/MergeTree/DataPartsExchange.cpp +++ b/dbms/src/Storages/MergeTree/DataPartsExchange.cpp @@ -25,6 +25,7 @@ namespace ErrorCodes extern const int ABORTED; extern const int BAD_SIZE_OF_FILE_IN_DATA_PART; extern const int CANNOT_WRITE_TO_OSTREAM; + extern const int CHECKSUM_DOESNT_MATCH; extern const int UNKNOWN_TABLE; } @@ -240,7 +241,8 @@ MergeTreeData::MutableDataPartPtr Fetcher::fetchPart( readPODBinary(expected_hash, in); if (expected_hash != hashing_out.getHash()) - throw Exception("Checksum mismatch for file " + absolute_part_path + file_name + " transferred from " + replica_path); + throw Exception("Checksum mismatch for file " + absolute_part_path + file_name + " transferred from " + replica_path, + ErrorCodes::CHECKSUM_DOESNT_MATCH); if (file_name != "checksums.txt" && file_name != "columns.txt") diff --git a/dbms/src/Storages/MergeTree/MergeTreeData.cpp b/dbms/src/Storages/MergeTree/MergeTreeData.cpp index ec0e9249bba..d2a6fb53964 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeData.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeData.cpp @@ -193,7 +193,7 @@ static void checkKeyExpression(const ExpressionActions & expr, const Block & sam for (const ExpressionAction & action : expr.getActions()) { if (action.type == ExpressionAction::ARRAY_JOIN) - throw Exception(key_name + " key cannot contain array joins"); + throw Exception(key_name + " key cannot contain array joins", ErrorCodes::ILLEGAL_COLUMN); if (action.type == ExpressionAction::APPLY_FUNCTION) { @@ -346,7 +346,7 @@ void MergeTreeData::MergingParams::check(const NamesAndTypesList & columns) cons } } if (miss_column) - throw Exception("Sign column " + sign_column + " does not exist in table declaration."); + throw Exception("Sign column " + sign_column + " does not exist in table declaration.", ErrorCodes::NO_SUCH_COLUMN_IN_TABLE); }; /// that if the version_column column is needed, it exists and is of unsigned integer type. @@ -375,7 +375,7 @@ void MergeTreeData::MergingParams::check(const NamesAndTypesList & columns) cons } } if (miss_column) - throw Exception("Version column " + version_column + " does not exist in table declaration."); + throw Exception("Version column " + version_column + " does not exist in table declaration.", ErrorCodes::NO_SUCH_COLUMN_IN_TABLE); }; if (mode == MergingParams::Collapsing) @@ -392,7 +392,7 @@ void MergeTreeData::MergingParams::check(const NamesAndTypesList & columns) cons }; if (columns.end() == std::find_if(columns.begin(), columns.end(), check_column_to_sum_exists)) throw Exception( - "Column " + column_to_sum + " listed in columns to sum does not exist in table declaration."); + "Column " + column_to_sum + " listed in columns to sum does not exist in table declaration.", ErrorCodes::NO_SUCH_COLUMN_IN_TABLE); } } diff --git a/dbms/src/Storages/MergeTree/MergeTreeDataWriter.cpp b/dbms/src/Storages/MergeTree/MergeTreeDataWriter.cpp index 0446506e9d5..af1a6685657 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeDataWriter.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeDataWriter.cpp @@ -19,6 +19,11 @@ namespace ProfileEvents namespace DB { +namespace ErrorCodes +{ + extern const int LOGICAL_ERROR; +} + namespace { @@ -149,7 +154,7 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataWriter::writeTempPart(BlockWithPa DayNum max_month = date_lut.toFirstDayNumOfMonth(DayNum(max_date)); if (min_month != max_month) - throw Exception("Logical error: part spans more than one month."); + throw Exception("Logical error: part spans more than one month.", ErrorCodes::LOGICAL_ERROR); part_name = new_part_info.getPartNameV0(min_date, max_date); } diff --git a/dbms/src/Storages/MergeTree/ReplicatedMergeTreeBlockOutputStream.cpp b/dbms/src/Storages/MergeTree/ReplicatedMergeTreeBlockOutputStream.cpp index 3f242eb9759..cdfb71b5726 100644 --- a/dbms/src/Storages/MergeTree/ReplicatedMergeTreeBlockOutputStream.cpp +++ b/dbms/src/Storages/MergeTree/ReplicatedMergeTreeBlockOutputStream.cpp @@ -27,6 +27,8 @@ namespace ErrorCodes extern const int UNKNOWN_STATUS_OF_INSERT; extern const int INSERT_WAS_DEDUPLICATED; extern const int KEEPER_EXCEPTION; + extern const int TIMEOUT_EXCEEDED; + extern const int NO_ACTIVE_REPLICAS; } @@ -389,14 +391,14 @@ void ReplicatedMergeTreeBlockOutputStream::commitPart(zkutil::ZooKeeperPtr & zoo break; if (!event->tryWait(quorum_timeout_ms)) - throw Exception("Timeout while waiting for quorum"); + throw Exception("Timeout while waiting for quorum", ErrorCodes::TIMEOUT_EXCEEDED); } /// And what if it is possible that the current replica at this time has ceased to be active and the quorum is marked as failed and deleted? String value; if (!zookeeper->tryGet(storage.replica_path + "/is_active", value, nullptr) || value != quorum_info.is_active_node_value) - throw Exception("Replica become inactive while waiting for quorum"); + throw Exception("Replica become inactive while waiting for quorum", ErrorCodes::NO_ACTIVE_REPLICAS); } catch (...) { diff --git a/dbms/src/Storages/MergeTree/ReplicatedMergeTreePartCheckThread.cpp b/dbms/src/Storages/MergeTree/ReplicatedMergeTreePartCheckThread.cpp index b65ba8868e8..fb7a2ad0730 100644 --- a/dbms/src/Storages/MergeTree/ReplicatedMergeTreePartCheckThread.cpp +++ b/dbms/src/Storages/MergeTree/ReplicatedMergeTreePartCheckThread.cpp @@ -14,6 +14,11 @@ namespace ProfileEvents namespace DB { +namespace ErrorCodes +{ + extern const int TABLE_DIFFERS_TOO_MUCH; +} + static const auto PART_CHECK_ERROR_SLEEP_MS = 5 * 1000; @@ -213,7 +218,7 @@ void ReplicatedMergeTreePartCheckThread::checkPart(const String & part_name) auto zk_columns = NamesAndTypesList::parse( zookeeper->get(storage.replica_path + "/parts/" + part_name + "/columns")); if (part->columns != zk_columns) - throw Exception("Columns of local part " + part_name + " are different from ZooKeeper"); + throw Exception("Columns of local part " + part_name + " are different from ZooKeeper", ErrorCodes::TABLE_DIFFERS_TOO_MUCH); checkDataPart( storage.data.getFullPath() + part_name, diff --git a/dbms/src/Storages/MergeTree/checkDataPart.cpp b/dbms/src/Storages/MergeTree/checkDataPart.cpp index a978bdf34f7..eac9145692b 100644 --- a/dbms/src/Storages/MergeTree/checkDataPart.cpp +++ b/dbms/src/Storages/MergeTree/checkDataPart.cpp @@ -21,6 +21,7 @@ namespace DB namespace ErrorCodes { extern const int CORRUPTED_DATA; + extern const int LOGICAL_ERROR; extern const int INCORRECT_MARK; extern const int EMPTY_LIST_OF_COLUMNS_PASSED; } @@ -285,7 +286,7 @@ MergeTreeData::DataPart::Checksums checkDataPart( String file_name = IDataType::getFileNameForStream(name_type.name, substream_path); auto stream_it = streams.find(file_name); if (stream_it == streams.end()) - throw Exception("Logical error: cannot find stream " + file_name); + throw Exception("Logical error: cannot find stream " + file_name, ErrorCodes::LOGICAL_ERROR); return &stream_it->second.uncompressed_hashing_buf; }; @@ -319,7 +320,7 @@ MergeTreeData::DataPart::Checksums checkDataPart( String file_name = IDataType::getFileNameForStream(name_type.name, substream_path); auto stream_it = streams.find(file_name); if (stream_it == streams.end()) - throw Exception("Logical error: cannot find stream " + file_name); + throw Exception("Logical error: cannot find stream " + file_name, ErrorCodes::LOGICAL_ERROR); stream_it->second.assertEnd(); stream_it->second.saveChecksums(checksums_data); diff --git a/dbms/src/Storages/StorageDictionary.cpp b/dbms/src/Storages/StorageDictionary.cpp index 78fac95a907..450a0307e10 100644 --- a/dbms/src/Storages/StorageDictionary.cpp +++ b/dbms/src/Storages/StorageDictionary.cpp @@ -19,6 +19,7 @@ namespace DB namespace ErrorCodes { extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; + extern const int THERE_IS_NO_COLUMN; } @@ -80,7 +81,7 @@ void StorageDictionary::checkNamesAndTypesCompatibleWithDictionary(const Diction message += " in dictionary " + dictionary_name + ". "; message += "There are only columns "; message += generateNamesAndTypesDescription(dictionary_names_and_types.begin(), dictionary_names_and_types.end()); - throw Exception(message); + throw Exception(message, ErrorCodes::THERE_IS_NO_COLUMN); } } } diff --git a/dbms/src/Storages/StorageReplicatedMergeTree.cpp b/dbms/src/Storages/StorageReplicatedMergeTree.cpp index 23a2c2df42e..a89ea555d42 100644 --- a/dbms/src/Storages/StorageReplicatedMergeTree.cpp +++ b/dbms/src/Storages/StorageReplicatedMergeTree.cpp @@ -1014,7 +1014,7 @@ bool StorageReplicatedMergeTree::executeLogEntry(LogEntry & entry) } else { - throw Exception("Unexpected log entry type: " + toString(static_cast(entry.type))); + throw Exception("Unexpected log entry type: " + toString(static_cast(entry.type)), ErrorCodes::LOGICAL_ERROR); } if (do_fetch) @@ -1862,7 +1862,7 @@ bool StorageReplicatedMergeTree::executeReplaceRange(const LogEntry & entry) if (!prev.found_new_part_info.isDisjoint(curr.found_new_part_info)) { throw Exception("Intersected final parts detected: " + prev.found_new_part_name - + " and " + curr.found_new_part_name + ". It should be investigated."); + + " and " + curr.found_new_part_name + ". It should be investigated.", ErrorCodes::INCORRECT_DATA); } } } diff --git a/dbms/src/Storages/System/StorageSystemZooKeeper.cpp b/dbms/src/Storages/System/StorageSystemZooKeeper.cpp index db498d620d3..dd7eb033514 100644 --- a/dbms/src/Storages/System/StorageSystemZooKeeper.cpp +++ b/dbms/src/Storages/System/StorageSystemZooKeeper.cpp @@ -15,6 +15,11 @@ namespace DB { +namespace ErrorCodes +{ + extern const int BAD_ARGUMENTS; +} + NamesAndTypesList StorageSystemZooKeeper::getNamesAndTypes() { @@ -103,7 +108,7 @@ void StorageSystemZooKeeper::fillData(MutableColumns & res_columns, const Contex { String path = extractPath(query_info.query); if (path.empty()) - throw Exception("SELECT from system.zookeeper table must contain condition like path = 'path' in WHERE clause."); + throw Exception("SELECT from system.zookeeper table must contain condition like path = 'path' in WHERE clause.", ErrorCodes::BAD_ARGUMENTS); zkutil::ZooKeeperPtr zookeeper = context.getZooKeeper(); diff --git a/libs/libdaemon/src/BaseDaemon.cpp b/libs/libdaemon/src/BaseDaemon.cpp index 5c1fb503ef2..f65634f1e2f 100644 --- a/libs/libdaemon/src/BaseDaemon.cpp +++ b/libs/libdaemon/src/BaseDaemon.cpp @@ -1189,7 +1189,7 @@ void BaseDaemon::handleSignal(int signal_id) onInterruptSignals(signal_id); } else - throw DB::Exception(std::string("Unsupported signal: ") + strsignal(signal_id)); + throw DB::Exception(std::string("Unsupported signal: ") + strsignal(signal_id), 0); } void BaseDaemon::onInterruptSignals(int signal_id) From d3bb69be747fddbc5f51bc1a94a8bec979a7a081 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 23 Nov 2018 00:27:52 +0300 Subject: [PATCH 063/124] More correct documentation + miscellaneous name change #3644 --- dbms/cmake/version.cmake | 2 +- dbms/programs/server/Server.cpp | 4 ++-- dbms/src/Common/ClickHouseRevision.cpp | 2 +- dbms/src/Common/ClickHouseRevision.h | 2 +- dbms/src/Common/CurrentMetrics.cpp | 2 +- dbms/src/Common/config_version.h.in | 2 +- .../System/StorageSystemBuildOptions.generated.cpp.in | 2 +- 7 files changed, 8 insertions(+), 8 deletions(-) diff --git a/dbms/cmake/version.cmake b/dbms/cmake/version.cmake index d3c3eb7ca4e..9f623279b99 100644 --- a/dbms/cmake/version.cmake +++ b/dbms/cmake/version.cmake @@ -23,4 +23,4 @@ set (VERSION_NAME "${PROJECT_NAME}" CACHE STRING "") set (VERSION_FULL "${VERSION_NAME} ${VERSION_STRING}" CACHE STRING "") set (VERSION_SO "${VERSION_STRING}" CACHE STRING "") -math (EXPR VERSION_INT "${VERSION_PATCH} + ${VERSION_MINOR}*1000 + ${VERSION_MAJOR}*1000000") +math (EXPR VERSION_INTEGER "${VERSION_PATCH} + ${VERSION_MINOR}*1000 + ${VERSION_MAJOR}*1000000") diff --git a/dbms/programs/server/Server.cpp b/dbms/programs/server/Server.cpp index df549920c6c..b80a9c7073c 100644 --- a/dbms/programs/server/Server.cpp +++ b/dbms/programs/server/Server.cpp @@ -55,7 +55,7 @@ namespace CurrentMetrics { extern const Metric Revision; - extern const Metric VersionInt; + extern const Metric VersionInteger; } namespace DB @@ -111,7 +111,7 @@ int Server::main(const std::vector & /*args*/) registerStorages(); CurrentMetrics::set(CurrentMetrics::Revision, ClickHouseRevision::get()); - CurrentMetrics::set(CurrentMetrics::VersionInt, ClickHouseRevision::getVersionInt()); + CurrentMetrics::set(CurrentMetrics::VersionInteger, ClickHouseRevision::getVersionInteger()); /** Context contains all that query execution is dependent: * settings, available functions, data types, aggregate functions, databases... diff --git a/dbms/src/Common/ClickHouseRevision.cpp b/dbms/src/Common/ClickHouseRevision.cpp index 7a76f7b719b..5513922a655 100644 --- a/dbms/src/Common/ClickHouseRevision.cpp +++ b/dbms/src/Common/ClickHouseRevision.cpp @@ -4,5 +4,5 @@ namespace ClickHouseRevision { unsigned get() { return VERSION_REVISION; } - unsigned getVersionInt() { return VERSION_INT; } + unsigned getVersionInteger() { return VERSION_INTEGER; } } diff --git a/dbms/src/Common/ClickHouseRevision.h b/dbms/src/Common/ClickHouseRevision.h index 43ecb3618d5..1d097a5bf89 100644 --- a/dbms/src/Common/ClickHouseRevision.h +++ b/dbms/src/Common/ClickHouseRevision.h @@ -3,5 +3,5 @@ namespace ClickHouseRevision { unsigned get(); - unsigned getVersionInt(); + unsigned getVersionInteger(); } diff --git a/dbms/src/Common/CurrentMetrics.cpp b/dbms/src/Common/CurrentMetrics.cpp index cd5e8532c08..2061791047f 100644 --- a/dbms/src/Common/CurrentMetrics.cpp +++ b/dbms/src/Common/CurrentMetrics.cpp @@ -40,7 +40,7 @@ M(StorageBufferBytes, "Number of bytes in buffers of Buffer tables") \ M(DictCacheRequests, "Number of requests in fly to data sources of dictionaries of cache type.") \ M(Revision, "Revision of the server. It is a number incremented for every release or release candidate.") \ - M(VersionInt, "Version of the server. It is a number incremented for every release.") \ + M(VersionInteger, "Version of the server in a single integer number in base-1000. For example, version 11.22.33 is translated to 11022033.") \ M(RWLockWaitingReaders, "Number of threads waiting for read on a table RWLock.") \ M(RWLockWaitingWriters, "Number of threads waiting for write on a table RWLock.") \ M(RWLockActiveReaders, "Number of threads holding read lock in a table RWLock.") \ diff --git a/dbms/src/Common/config_version.h.in b/dbms/src/Common/config_version.h.in index a6ec0743744..a90fd77b6a8 100644 --- a/dbms/src/Common/config_version.h.in +++ b/dbms/src/Common/config_version.h.in @@ -23,7 +23,7 @@ #cmakedefine VERSION_FULL "@VERSION_FULL@" #cmakedefine VERSION_DESCRIBE "@VERSION_DESCRIBE@" #cmakedefine VERSION_GITHASH "@VERSION_GITHASH@" -#cmakedefine VERSION_INT @VERSION_INT@ +#cmakedefine VERSION_INTEGER @VERSION_INTEGER@ #if defined(VERSION_MAJOR) #define DBMS_VERSION_MAJOR VERSION_MAJOR diff --git a/dbms/src/Storages/System/StorageSystemBuildOptions.generated.cpp.in b/dbms/src/Storages/System/StorageSystemBuildOptions.generated.cpp.in index 87e29f4e45d..ad08e14b00b 100644 --- a/dbms/src/Storages/System/StorageSystemBuildOptions.generated.cpp.in +++ b/dbms/src/Storages/System/StorageSystemBuildOptions.generated.cpp.in @@ -6,7 +6,7 @@ const char * auto_config_build[] "VERSION_DESCRIBE", "@VERSION_DESCRIBE@", "VERSION_GITHASH", "@VERSION_GITHASH@", "VERSION_REVISION", "@VERSION_REVISION@", - "VERSION_INT", "@VERSION_INT@", + "VERSION_INTEGER", "@VERSION_INTEGER@", "BUILD_DATE", "@BUILD_DATE@", "BUILD_TYPE", "@CMAKE_BUILD_TYPE@", "SYSTEM", "@CMAKE_SYSTEM@", From 64c6951b1a99245cf983738c8f3cae6ef8090c22 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 23 Nov 2018 00:28:38 +0300 Subject: [PATCH 064/124] More correct documentation #3644 --- dbms/src/Common/CurrentMetrics.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/src/Common/CurrentMetrics.cpp b/dbms/src/Common/CurrentMetrics.cpp index 2061791047f..2f8346d554e 100644 --- a/dbms/src/Common/CurrentMetrics.cpp +++ b/dbms/src/Common/CurrentMetrics.cpp @@ -39,7 +39,7 @@ M(StorageBufferRows, "Number of rows in buffers of Buffer tables") \ M(StorageBufferBytes, "Number of bytes in buffers of Buffer tables") \ M(DictCacheRequests, "Number of requests in fly to data sources of dictionaries of cache type.") \ - M(Revision, "Revision of the server. It is a number incremented for every release or release candidate.") \ + M(Revision, "Revision of the server. It is a number incremented for every release or release candidate except patch releases.") \ M(VersionInteger, "Version of the server in a single integer number in base-1000. For example, version 11.22.33 is translated to 11022033.") \ M(RWLockWaitingReaders, "Number of threads waiting for read on a table RWLock.") \ M(RWLockWaitingWriters, "Number of threads waiting for write on a table RWLock.") \ From c0e97a97cd256e2e35239e43e437e35da7c9329d Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 23 Nov 2018 01:00:03 +0300 Subject: [PATCH 065/124] Resolved inconsistencies in documentation #3539 --- .../Storages/MergeTree/registerStorageMergeTree.cpp | 10 ++++++---- 1 file changed, 6 insertions(+), 4 deletions(-) diff --git a/dbms/src/Storages/MergeTree/registerStorageMergeTree.cpp b/dbms/src/Storages/MergeTree/registerStorageMergeTree.cpp index a875b743760..ebce8badca6 100644 --- a/dbms/src/Storages/MergeTree/registerStorageMergeTree.cpp +++ b/dbms/src/Storages/MergeTree/registerStorageMergeTree.cpp @@ -205,6 +205,8 @@ static void setGraphitePatternsFromConfig(const Context & context, static String getMergeTreeVerboseHelp(bool is_extended_syntax) { + using namespace std::string_literals; + String help = R"( MergeTree is a family of storage engines. @@ -248,14 +250,14 @@ Next parameter is index (primary key) granularity. Good value is 8192. You have )"; help += R"( -For the Collapsing mode, the last parameter is the name of a sign column - a special column that is used to 'collapse' rows with the same primary key while merging. +For the Collapsing mode, the )" + (is_extended_syntax ? "only"s : "last"s) + R"( parameter is the name of a sign column - a special column that is used to 'collapse' rows with the same primary key while merging. -For the Summing mode, the optional last parameter is a list of columns to sum while merging. This list is passed in round brackets, like (PageViews, Cost). +For the Summing mode, the optional )" + (is_extended_syntax ? ""s : "last "s) + R"(parameter is a list of columns to sum while merging. This list is passed in round brackets, like (PageViews, Cost). If this parameter is omitted, the storage will sum all numeric columns except columns participating in the primary key. -For the Replacing mode, the optional last parameter is the name of a 'version' column. While merging, for all rows with the same primary key, only one row is selected: the last row, if the version column was not specified, or the last row with the maximum version value, if specified. +For the Replacing mode, the optional )" + (is_extended_syntax ? ""s : "last "s) + R"(parameter is the name of a 'version' column. While merging, for all rows with the same primary key, only one row is selected: the last row, if the version column was not specified, or the last row with the maximum version value, if specified. -For VersionedCollapsing mode, the last 2 parameters are the name of a sign column and the name of a 'version' column. Version column must be in primary key. While merging, a pair of rows with the same primary key and different sign may collapse. +For VersionedCollapsing mode, the )" + (is_extended_syntax ? ""s : "last "s) + R"(2 parameters are the name of a sign column and the name of a 'version' column. Version column must be in primary key. While merging, a pair of rows with the same primary key and different sign may collapse. )"; if (is_extended_syntax) From a78984bc3d43365075a94471bc65a9e355be7685 Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Fri, 23 Nov 2018 01:40:41 +0300 Subject: [PATCH 066/124] Update reference.md --- docs/ru/query_language/agg_functions/reference.md | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/docs/ru/query_language/agg_functions/reference.md b/docs/ru/query_language/agg_functions/reference.md index d42f216c6ea..971d006b030 100644 --- a/docs/ru/query_language/agg_functions/reference.md +++ b/docs/ru/query_language/agg_functions/reference.md @@ -160,11 +160,13 @@ GROUP BY timeslot Приближённо вычисляет количество различных значений аргумента. Работает для чисел, строк, дат, дат-с-временем, для нескольких аргументов и аргументов-кортежей. -Используется комбинация трёх алгоритмов: массив, хэш-таблица и [HyperLogLog](https://en.wikipedia.org/wiki/HyperLogLog) с таблицей коррекции погрешности. Расход памяти в несколько раз меньше, чем у функции `uniq`, а точность в несколько раз выше. Скорость работы чуть ниже, чем у функции `uniq`, но иногда может быть даже выше - в случае распределённых запросов, в которых по сети передаётся большое количество состояний агрегации. Максимальный размер состояния составляет 96 KiB (HyperLogLog из 217 6-битовых ячеек). +Используется комбинация трёх алгоритмов: массив, хэш-таблица и [HyperLogLog](https://en.wikipedia.org/wiki/HyperLogLog) с таблицей коррекции погрешности. Для небольшого количества различных значений используется массив; при увеличении количества значений, используется хэш таблица, до тех пор, пока её размер меньше размера HyperLogLog структуры. При дальнейшем увеличении количества значений, используется HyperLogLog структура, имеющая фиксированный размер в памяти. + +Параметр HLL_precision - логарифм по основанию 2 от количества ячеек в HyperLogLog. Параметер можно не указывать (для этого, опустите первую пару скобок). По-умолчанию - 17. При использовании параметра по-умолчанию, расход памяти в несколько раз меньше, чем у функции `uniq`, а точность в несколько раз выше. Скорость работы чуть ниже, чем у функции `uniq`, но иногда может быть даже выше - в случае распределённых запросов, в которых по сети передаётся большое количество состояний агрегации. Каждая ячейка имеет размер 6 бит, что даёт 96 KiB для размера HyperLogLog структуры. Результат детерминирован (не зависит от порядка выполнения запроса). -Функция `uniqCombined` является хорошим выбором по умолчанию для подсчёта количества различных значений, но стоит иметь ввиду что для множеств большой кардинальности (200M+) ошибка оценки будет только расти и для множеств огромной кардинальности (1B+ элементов) функция возвращает результат с очень большой неточностью. +Функция `uniqCombined` является хорошим выбором по умолчанию для подсчёта количества различных значений, но стоит иметь ввиду что для множеств большой кардинальности (200 миллионов различных элементов и больше) ошибка оценки становится существенно больше расчётной из-за недостаточно хорошего выбора хэш-функции. ## uniqHLL12(x) From 9ce56c5ab4e8e1b31d7ed1e43479d87982309438 Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Fri, 23 Nov 2018 01:46:46 +0300 Subject: [PATCH 067/124] Update reference.md --- docs/en/query_language/agg_functions/reference.md | 8 +++++--- 1 file changed, 5 insertions(+), 3 deletions(-) diff --git a/docs/en/query_language/agg_functions/reference.md b/docs/en/query_language/agg_functions/reference.md index 491a710fe5b..2ed568149ee 100644 --- a/docs/en/query_language/agg_functions/reference.md +++ b/docs/en/query_language/agg_functions/reference.md @@ -150,11 +150,13 @@ This function provides excellent accuracy even for data sets with extremely high Calculates the approximate number of different values of the argument. Works for numbers, strings, dates, date-with-time, and for multiple arguments and tuple arguments. -A combination of three algorithms is used: array, hash table and [HyperLogLog](https://en.wikipedia.org/wiki/HyperLogLog) with an error correction table. The memory consumption is several times smaller than for the `uniq` function, and the accuracy is several times higher. Performance is slightly lower than for the `uniq` function, but sometimes it can be even higher than it, such as with distributed queries that transmit a large number of aggregation states over the network. The maximum state size is 96 KiB (HyperLogLog of 217 6-bit cells). +A combination of three algorithms is used: array, hash table and [HyperLogLog](https://en.wikipedia.org/wiki/HyperLogLog) with an error correction table. For small number of distinct elements, the array is used. When the set size becomes larger the hash table is used, while it is smaller than HyperLogLog data structure. For larger number of elements, the HyperLogLog is used, and it will occupy fixed amount of memory. -The result is determinate (it doesn't depend on the order of query processing). +The parameter "HLL_precision" is the base-2 logarithm of the number of cells in HyperLogLog. You can omit the parameter (omit first parens). The default value is 17, that is effectively 96 KiB of space (2^17 cells of 6 bits each). The memory consumption is several times smaller than for the `uniq` function, and the accuracy is several times higher. Performance is slightly lower than for the `uniq` function, but sometimes it can be even higher than it, such as with distributed queries that transmit a large number of aggregation states over the network. -The `uniqCombined` function is a good default choice for calculating the number of different values, but keep in mind that the estimation error will increase for high-cardinality data sets (200M+ elements), and the function will return very inaccurate results for data sets with extremely high cardinality (1B+ elements). +The result is deterministic (it doesn't depend on the order of query processing). + +The `uniqCombined` function is a good default choice for calculating the number of different values, but keep in mind that the estimation error for large sets (200M+ elements) will become larger than theoretical value due to poor choice of hash function. ## uniqHLL12(x) From 709b1177e072de66cc9d1161b6a5a631c940d4d0 Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Fri, 23 Nov 2018 01:47:23 +0300 Subject: [PATCH 068/124] Update reference.md --- docs/en/query_language/agg_functions/reference.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/query_language/agg_functions/reference.md b/docs/en/query_language/agg_functions/reference.md index 2ed568149ee..78a88ca20f6 100644 --- a/docs/en/query_language/agg_functions/reference.md +++ b/docs/en/query_language/agg_functions/reference.md @@ -156,7 +156,7 @@ The parameter "HLL_precision" is the base-2 logarithm of the number of cells in The result is deterministic (it doesn't depend on the order of query processing). -The `uniqCombined` function is a good default choice for calculating the number of different values, but keep in mind that the estimation error for large sets (200M+ elements) will become larger than theoretical value due to poor choice of hash function. +The `uniqCombined` function is a good default choice for calculating the number of different values, but keep in mind that the estimation error for large sets (200 million elements and more) will become larger than theoretical value due to poor choice of hash function. ## uniqHLL12(x) From 2a225174d4e96245f71fc0980c399a684602d024 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 23 Nov 2018 02:45:16 +0300 Subject: [PATCH 069/124] Added performance test #3406 --- dbms/tests/performance/uniq/uniq.xml | 65 ++++++++++++++++++++++++++++ 1 file changed, 65 insertions(+) create mode 100644 dbms/tests/performance/uniq/uniq.xml diff --git a/dbms/tests/performance/uniq/uniq.xml b/dbms/tests/performance/uniq/uniq.xml new file mode 100644 index 00000000000..ba255ac224e --- /dev/null +++ b/dbms/tests/performance/uniq/uniq.xml @@ -0,0 +1,65 @@ + + uniq + once + + + hits_100m_single + 30000000000 + + + + + 10000 + + + 5000 + 20000 + + + + + + + + + 20000000000 + + + + + key + + 1 + SearchEngineID + RegionID + SearchPhrase + ClientIP + + + + func + + sum + uniq + uniqExact + uniqHLL12 + uniqCombined(12) + uniqCombined(13) + uniqCombined(14) + uniqCombined(15) + uniqCombined(16) + uniqCombined(17) + uniqCombined(18) + uniqCombined(19) + uniqCombined(20) + uniqUpTo(3) + uniqUpTo(5) + uniqUpTo(10) + uniqUpTo(25) + uniqUpTo(100) + + + + + SELECT {key} AS k, {func}(UserID) FROM hits_100m_single GROUP BY k + From d31d30c8e1983036b09f52fa6ab6d6d6513fb791 Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 23 Nov 2018 11:08:35 +0300 Subject: [PATCH 070/124] Fix comments, naming, log about kill inability and better test --- dbms/src/Common/ShellCommand.cpp | 29 ++++++++++++++----- dbms/src/Common/ShellCommand.h | 17 ++++++----- .../integration/test_odbc_interaction/test.py | 11 +++++-- 3 files changed, 39 insertions(+), 18 deletions(-) diff --git a/dbms/src/Common/ShellCommand.cpp b/dbms/src/Common/ShellCommand.cpp index 2b1fbfe9d7b..675dd8391f1 100644 --- a/dbms/src/Common/ShellCommand.cpp +++ b/dbms/src/Common/ShellCommand.cpp @@ -4,6 +4,7 @@ #include #include #include +#include #include #include #include @@ -76,15 +77,27 @@ namespace namespace DB { +ShellCommand::ShellCommand(pid_t pid, int in_fd, int out_fd, int err_fd, bool terminate_in_destructor_) + : pid(pid) + , terminate_in_destructor(terminate_in_destructor_) + , log(&Poco::Logger::get("ShellCommand")) + , in(in_fd) + , out(out_fd) + , err(err_fd) {} + ShellCommand::~ShellCommand() { - if (die_in_destructor) - kill(pid, SIGTERM); + if (terminate_in_destructor) + { + int retcode = kill(pid, SIGTERM); + if (retcode != 0) + LOG_WARNING(log, "Cannot kill pid " << pid << " errno '" << errnoToString(retcode) << "'"); + } else if (!wait_called) tryWait(); } -std::unique_ptr ShellCommand::executeImpl(const char * filename, char * const argv[], bool pipe_stdin_only, bool die_in_destructor) +std::unique_ptr ShellCommand::executeImpl(const char * filename, char * const argv[], bool pipe_stdin_only, bool terminate_in_destructor) { /** Here it is written that with a normal call `vfork`, there is a chance of deadlock in multithreaded programs, * because of the resolving of characters in the shared library @@ -131,7 +144,7 @@ std::unique_ptr ShellCommand::executeImpl(const char * filename, c _exit(int(ReturnCodes::CANNOT_EXEC)); } - std::unique_ptr res(new ShellCommand(pid, pipe_stdin.write_fd, pipe_stdout.read_fd, pipe_stderr.read_fd, die_in_destructor)); + std::unique_ptr res(new ShellCommand(pid, pipe_stdin.write_fd, pipe_stdout.read_fd, pipe_stderr.read_fd, terminate_in_destructor)); /// Now the ownership of the file descriptors is passed to the result. pipe_stdin.write_fd = -1; @@ -142,7 +155,7 @@ std::unique_ptr ShellCommand::executeImpl(const char * filename, c } -std::unique_ptr ShellCommand::execute(const std::string & command, bool pipe_stdin_only, bool die_in_destructor) +std::unique_ptr ShellCommand::execute(const std::string & command, bool pipe_stdin_only, bool terminate_in_destructor) { /// Arguments in non-constant chunks of memory (as required for `execv`). /// Moreover, their copying must be done before calling `vfork`, so after `vfork` do a minimum of things. @@ -152,11 +165,11 @@ std::unique_ptr ShellCommand::execute(const std::string & command, char * const argv[] = { argv0.data(), argv1.data(), argv2.data(), nullptr }; - return executeImpl("/bin/sh", argv, pipe_stdin_only, die_in_destructor); + return executeImpl("/bin/sh", argv, pipe_stdin_only, terminate_in_destructor); } -std::unique_ptr ShellCommand::executeDirect(const std::string & path, const std::vector & arguments, bool die_in_destructor) +std::unique_ptr ShellCommand::executeDirect(const std::string & path, const std::vector & arguments, bool terminate_in_destructor) { size_t argv_sum_size = path.size() + 1; for (const auto & arg : arguments) @@ -177,7 +190,7 @@ std::unique_ptr ShellCommand::executeDirect(const std::string & pa argv[arguments.size() + 1] = nullptr; - return executeImpl(path.data(), argv.data(), false, die_in_destructor); + return executeImpl(path.data(), argv.data(), false, terminate_in_destructor); } diff --git a/dbms/src/Common/ShellCommand.h b/dbms/src/Common/ShellCommand.h index a6f5c749adc..3d1308272e9 100644 --- a/dbms/src/Common/ShellCommand.h +++ b/dbms/src/Common/ShellCommand.h @@ -28,12 +28,13 @@ class ShellCommand private: pid_t pid; bool wait_called = false; - bool die_in_destructor; + bool terminate_in_destructor; - ShellCommand(pid_t pid, int in_fd, int out_fd, int err_fd, bool die_in_destructor_) - : pid(pid), die_in_destructor(die_in_destructor_), in(in_fd), out(out_fd), err(err_fd) {} + Poco::Logger * log; - static std::unique_ptr executeImpl(const char * filename, char * const argv[], bool pipe_stdin_only, bool die_in_destructor); + ShellCommand(pid_t pid, int in_fd, int out_fd, int err_fd, bool terminate_in_destructor_); + + static std::unique_ptr executeImpl(const char * filename, char * const argv[], bool pipe_stdin_only, bool terminate_in_destructor); public: WriteBufferFromFile in; /// If the command reads from stdin, do not forget to call in.close() after writing all the data there. @@ -42,11 +43,13 @@ public: ~ShellCommand(); - /// Run the command using /bin/sh -c - static std::unique_ptr execute(const std::string & command, bool pipe_stdin_only = false, bool die_in_destructor = false); + /// Run the command using /bin/sh -c. + /// If terminate_in_destructor is true, send terminate signal in destructor and don't wait process. + static std::unique_ptr execute(const std::string & command, bool pipe_stdin_only = false, bool terminate_in_destructor = false); /// Run the executable with the specified arguments. `arguments` - without argv[0]. - static std::unique_ptr executeDirect(const std::string & path, const std::vector & arguments, bool die_in_destructor = false); + /// If terminate_in_destructor is true, send terminate signal in destructor and don't wait process. + static std::unique_ptr executeDirect(const std::string & path, const std::vector & arguments, bool terminate_in_destructor = false); /// Wait for the process to end, throw an exception if the code is not 0 or if the process was not completed by itself. void wait(); diff --git a/dbms/tests/integration/test_odbc_interaction/test.py b/dbms/tests/integration/test_odbc_interaction/test.py index 8128c548aed..79652e6edb5 100644 --- a/dbms/tests/integration/test_odbc_interaction/test.py +++ b/dbms/tests/integration/test_odbc_interaction/test.py @@ -194,10 +194,15 @@ def test_bridge_dies_with_parent(started_cluster): assert clickhouse_pid is not None assert bridge_pid is not None - node1.exec_in_container(["bash", "-c", "kill {}".format(clickhouse_pid)], privileged=True, user='root') - node1.exec_in_container(["bash", "-c", "kill {}".format(clickhouse_pid)], privileged=True, user='root') + while clickhouse_pid is not None: + try: + node1.exec_in_container(["bash", "-c", "kill {}".format(clickhouse_pid)], privileged=True, user='root') + except: + pass + clickhouse_pid = get_pid("clickhouse server") + time.sleep(1) - clickhouse_pid = get_pid("clickhouse server") + time.sleep(1) # just for sure, that odbc-bridge caught signal bridge_pid = get_pid("odbc-bridge") assert clickhouse_pid is None From 7fea1397d6931832d26fa1a1dc9137ba8ea685b7 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 23 Nov 2018 11:11:54 +0300 Subject: [PATCH 071/124] Remove null map cache from ColumnUnique. --- dbms/src/Columns/ColumnUnique.h | 65 ++++++++++++++++++++++----------- 1 file changed, 43 insertions(+), 22 deletions(-) diff --git a/dbms/src/Columns/ColumnUnique.h b/dbms/src/Columns/ColumnUnique.h index 1f3366fceaa..1632f777e8b 100644 --- a/dbms/src/Columns/ColumnUnique.h +++ b/dbms/src/Columns/ColumnUnique.h @@ -81,7 +81,7 @@ public: { return column_holder->allocatedBytes() + index.allocatedBytes() - + (cached_null_mask ? cached_null_mask->allocatedBytes() : 0); + + (nested_null_mask ? nested_null_mask->allocatedBytes() : 0); } void forEachSubcolumn(IColumn::ColumnCallback callback) override { @@ -100,8 +100,8 @@ private: ReverseIndex index; /// For DataTypeNullable, stores null map. - mutable ColumnPtr cached_null_mask; - mutable ColumnPtr cached_column_nullable; + ColumnPtr nested_null_mask; + ColumnPtr nested_column_nullable; class IncrementalHash { @@ -118,6 +118,8 @@ private: mutable IncrementalHash hash; + void updateNullMask(); + static size_t numSpecialValues(bool is_nullable) { return is_nullable ? 2 : 1; } size_t numSpecialValues() const { return numSpecialValues(is_nullable); } @@ -148,6 +150,7 @@ ColumnUnique::ColumnUnique(const ColumnUnique & other) , index(numSpecialValues(is_nullable), 0) { index.setColumn(getRawColumnPtr()); + updateNullMask(); } template @@ -158,6 +161,7 @@ ColumnUnique::ColumnUnique(const IDataType & type) const auto & holder_type = is_nullable ? *static_cast(type).getNestedType() : type; column_holder = holder_type.createColumn()->cloneResized(numSpecialValues()); index.setColumn(getRawColumnPtr()); + updateNullMask(); } template @@ -172,32 +176,37 @@ ColumnUnique::ColumnUnique(MutableColumnPtr && holder, bool is_nulla throw Exception("Holder column for ColumnUnique can't be nullable.", ErrorCodes::ILLEGAL_COLUMN); index.setColumn(getRawColumnPtr()); + updateNullMask(); +} + +template +void ColumnUnique::updateNullMask() +{ + if (is_nullable) + { + size_t size = getRawColumnPtr()->size(); + if (!nested_null_mask) + { + ColumnUInt8::MutablePtr null_mask = ColumnUInt8::create(size, UInt8(0)); + null_mask->getData()[getNullValueIndex()] = 1; + nested_null_mask = std::move(null_mask); + nested_column_nullable = ColumnNullable::create(column_holder, nested_null_mask); + } + + if (nested_null_mask->size() != size) + { + IColumn & null_mask = nested_null_mask->assumeMutableRef(); + static_cast(null_mask).getData().resize_fill(size); + } + } } template const ColumnPtr & ColumnUnique::getNestedColumn() const { if (is_nullable) - { - size_t size = getRawColumnPtr()->size(); - if (!cached_null_mask) - { - ColumnUInt8::MutablePtr null_mask = ColumnUInt8::create(size, UInt8(0)); - null_mask->getData()[getNullValueIndex()] = 1; - cached_null_mask = std::move(null_mask); - cached_column_nullable = ColumnNullable::create(column_holder, cached_null_mask); - } + return nested_column_nullable; - if (cached_null_mask->size() != size) - { - MutableColumnPtr null_mask = (*std::move(cached_null_mask)).mutate(); - static_cast(*null_mask).getData().resize_fill(size); - cached_null_mask = std::move(null_mask); - cached_column_nullable = ColumnNullable::create(column_holder, cached_null_mask); - } - - return cached_column_nullable; - } return column_holder; } @@ -227,6 +236,8 @@ size_t ColumnUnique::uniqueInsert(const Field & x) if (pos != prev_size) column->popBack(1); + updateNullMask(); + return pos; } @@ -260,6 +271,8 @@ size_t ColumnUnique::uniqueInsertData(const char * pos, size_t lengt index.insertFromLastRow(); } + updateNullMask(); + return insertion_point; } @@ -288,6 +301,8 @@ size_t ColumnUnique::uniqueInsertDataWithTerminatingZero(const char if (position != prev_size) column->popBack(1); + updateNullMask(); + return static_cast(position); } @@ -343,6 +358,8 @@ size_t ColumnUnique::uniqueDeserializeAndInsertFromArena(const char if (index_pos != prev_size) column->popBack(1); + updateNullMask(); + return static_cast(index_pos); } @@ -533,6 +550,8 @@ MutableColumnPtr ColumnUnique::uniqueInsertRangeFrom(const IColumn & if (!positions_column) throw Exception("Can't find index type for ColumnUnique", ErrorCodes::LOGICAL_ERROR); + updateNullMask(); + return positions_column; } @@ -577,6 +596,8 @@ IColumnUnique::IndexesWithOverflow ColumnUnique::uniqueInsertRangeWi if (!positions_column) throw Exception("Can't find index type for ColumnUnique", ErrorCodes::LOGICAL_ERROR); + updateNullMask(); + IColumnUnique::IndexesWithOverflow indexes_with_overflow; indexes_with_overflow.indexes = std::move(positions_column); indexes_with_overflow.overflowed_keys = std::move(overflowed_keys); From 7f40494330bcf09c17a6bd858f604cbb14632edd Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 23 Nov 2018 12:24:59 +0300 Subject: [PATCH 072/124] Enable keep alive only with patched poco --- dbms/src/IO/HTTPCommon.cpp | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/dbms/src/IO/HTTPCommon.cpp b/dbms/src/IO/HTTPCommon.cpp index fb901c4ffaf..c6b7e47225c 100644 --- a/dbms/src/IO/HTTPCommon.cpp +++ b/dbms/src/IO/HTTPCommon.cpp @@ -80,7 +80,11 @@ namespace session->setHost(DNSResolver::instance().resolveHost(host).toString()); session->setPort(port); + + /// doesn't work properly without patch +#if POCO_CLICKHOUSE_PATCH session->setKeepAlive(keep_alive); +#endif return session; } From e74b1edb35565301624bdec083c84932aee3d021 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 23 Nov 2018 13:18:50 +0300 Subject: [PATCH 073/124] Remove null map cache from ColumnUnique. --- dbms/src/Columns/ColumnUnique.h | 23 +++++++++++++++++++---- 1 file changed, 19 insertions(+), 4 deletions(-) diff --git a/dbms/src/Columns/ColumnUnique.h b/dbms/src/Columns/ColumnUnique.h index 1632f777e8b..12763484d7c 100644 --- a/dbms/src/Columns/ColumnUnique.h +++ b/dbms/src/Columns/ColumnUnique.h @@ -118,6 +118,7 @@ private: mutable IncrementalHash hash; + void createNullMask(); void updateNullMask(); static size_t numSpecialValues(bool is_nullable) { return is_nullable ? 2 : 1; } @@ -150,7 +151,7 @@ ColumnUnique::ColumnUnique(const ColumnUnique & other) , index(numSpecialValues(is_nullable), 0) { index.setColumn(getRawColumnPtr()); - updateNullMask(); + createNullMask(); } template @@ -161,7 +162,7 @@ ColumnUnique::ColumnUnique(const IDataType & type) const auto & holder_type = is_nullable ? *static_cast(type).getNestedType() : type; column_holder = holder_type.createColumn()->cloneResized(numSpecialValues()); index.setColumn(getRawColumnPtr()); - updateNullMask(); + createNullMask(); } template @@ -176,11 +177,11 @@ ColumnUnique::ColumnUnique(MutableColumnPtr && holder, bool is_nulla throw Exception("Holder column for ColumnUnique can't be nullable.", ErrorCodes::ILLEGAL_COLUMN); index.setColumn(getRawColumnPtr()); - updateNullMask(); + createNullMask(); } template -void ColumnUnique::updateNullMask() +void ColumnUnique::createNullMask() { if (is_nullable) { @@ -192,6 +193,20 @@ void ColumnUnique::updateNullMask() nested_null_mask = std::move(null_mask); nested_column_nullable = ColumnNullable::create(column_holder, nested_null_mask); } + else + throw Exception("Null mask for ColumnUnique is already created.", ErrorCodes::LOGICAL_ERROR); + } +} + +template +void ColumnUnique::updateNullMask() +{ + if (is_nullable) + { + if (!nested_null_mask) + throw Exception("Null mask for ColumnUnique is was not created.", ErrorCodes::LOGICAL_ERROR); + + size_t size = getRawColumnPtr()->size(); if (nested_null_mask->size() != size) { From d64fe4204abe522534a8ef5d56e30d490a96bb13 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 23 Nov 2018 13:47:01 +0300 Subject: [PATCH 074/124] Remove null map cache from ColumnUnique. --- dbms/src/Columns/ColumnUnique.h | 1 + 1 file changed, 1 insertion(+) diff --git a/dbms/src/Columns/ColumnUnique.h b/dbms/src/Columns/ColumnUnique.h index 12763484d7c..f68a834f729 100644 --- a/dbms/src/Columns/ColumnUnique.h +++ b/dbms/src/Columns/ColumnUnique.h @@ -87,6 +87,7 @@ public: { callback(column_holder); index.setColumn(getRawColumnPtr()); + nested_column_nullable = ColumnNullable::create(column_holder, nested_null_mask); } const UInt64 * tryGetSavedHash() const override { return index.tryGetSavedHash(); } From f3aa1292a9e1e680450b6fc9aae7687ddd4dd168 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 23 Nov 2018 14:55:06 +0300 Subject: [PATCH 075/124] Remove null map cache from ColumnUnique. --- dbms/src/Columns/ColumnUnique.h | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/dbms/src/Columns/ColumnUnique.h b/dbms/src/Columns/ColumnUnique.h index f68a834f729..7fb01620fbd 100644 --- a/dbms/src/Columns/ColumnUnique.h +++ b/dbms/src/Columns/ColumnUnique.h @@ -87,7 +87,8 @@ public: { callback(column_holder); index.setColumn(getRawColumnPtr()); - nested_column_nullable = ColumnNullable::create(column_holder, nested_null_mask); + if (is_nullable) + nested_column_nullable = ColumnNullable::create(column_holder, nested_null_mask); } const UInt64 * tryGetSavedHash() const override { return index.tryGetSavedHash(); } From f32bb684f892ffaf73c1dc521d58f34f3bcf798d Mon Sep 17 00:00:00 2001 From: Ivan Blinkov Date: Fri, 23 Nov 2018 15:59:53 +0300 Subject: [PATCH 076/124] Add blog link to README.md --- README.md | 1 + 1 file changed, 1 insertion(+) diff --git a/README.md b/README.md index b248d5ee906..5ffefd793a9 100644 --- a/README.md +++ b/README.md @@ -7,4 +7,5 @@ ClickHouse is an open-source column-oriented database management system that all * [Official website](https://clickhouse.yandex/) has quick high-level overview of ClickHouse on main page. * [Tutorial](https://clickhouse.yandex/tutorial.html) shows how to set up and query small ClickHouse cluster. * [Documentation](https://clickhouse.yandex/docs/en/) provides more in-depth information. +* [Blog](https://clickhouse.yandex/blog/en/) contains various ClickHouse-related articles, as well as announces and reports about events. * [Contacts](https://clickhouse.yandex/#contacts) can help to get your questions answered if there are any. From 449ab473502bc36583261c36c72d9b055f7ede78 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 23 Nov 2018 16:29:14 +0300 Subject: [PATCH 077/124] Disable low cardinality default implementation for emptyArrayToSingle function. --- dbms/src/Functions/emptyArrayToSingle.cpp | 1 + 1 file changed, 1 insertion(+) diff --git a/dbms/src/Functions/emptyArrayToSingle.cpp b/dbms/src/Functions/emptyArrayToSingle.cpp index 90ed3df529b..4020ab070a5 100644 --- a/dbms/src/Functions/emptyArrayToSingle.cpp +++ b/dbms/src/Functions/emptyArrayToSingle.cpp @@ -32,6 +32,7 @@ public: size_t getNumberOfArguments() const override { return 1; } bool useDefaultImplementationForConstants() const override { return true; } + bool useDefaultImplementationForLowCardinalityColumns() const override { return false; } DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override { From bbc6a11940dbcb8dc578b7962b0ca5a0a175a460 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 23 Nov 2018 16:48:07 +0300 Subject: [PATCH 078/124] Added test for left array join with low cardinality. --- ..._low_cardinality_left_array_join.reference | 39 +++++++++++++++++++ .../00752_low_cardinality_left_array_join.sql | 21 ++++++++++ 2 files changed, 60 insertions(+) create mode 100644 dbms/tests/queries/0_stateless/00752_low_cardinality_left_array_join.reference create mode 100644 dbms/tests/queries/0_stateless/00752_low_cardinality_left_array_join.sql diff --git a/dbms/tests/queries/0_stateless/00752_low_cardinality_left_array_join.reference b/dbms/tests/queries/0_stateless/00752_low_cardinality_left_array_join.reference new file mode 100644 index 00000000000..656b12b511d --- /dev/null +++ b/dbms/tests/queries/0_stateless/00752_low_cardinality_left_array_join.reference @@ -0,0 +1,39 @@ +['a','b'] ['c',NULL] [1,2] [3,NULL] a +['a','b'] ['c',NULL] [1,2] [3,NULL] b +[] ['c',NULL] [1,2] [3,NULL] +['a','b'] [] [1,2] [3,NULL] a +['a','b'] [] [1,2] [3,NULL] b +['a','b'] ['c',NULL] [] [3,NULL] a +['a','b'] ['c',NULL] [] [3,NULL] b +['a','b'] ['c',NULL] [1,2] [] a +['a','b'] ['c',NULL] [1,2] [] b +- +['a','b'] ['c',NULL] [1,2] [3,NULL] c +['a','b'] ['c',NULL] [1,2] [3,NULL] \N +[] ['c',NULL] [1,2] [3,NULL] c +[] ['c',NULL] [1,2] [3,NULL] \N +['a','b'] [] [1,2] [3,NULL] \N +['a','b'] ['c',NULL] [] [3,NULL] c +['a','b'] ['c',NULL] [] [3,NULL] \N +['a','b'] ['c',NULL] [1,2] [] c +['a','b'] ['c',NULL] [1,2] [] \N +- +['a','b'] ['c',NULL] [1,2] [3,NULL] 1 +['a','b'] ['c',NULL] [1,2] [3,NULL] 2 +[] ['c',NULL] [1,2] [3,NULL] 1 +[] ['c',NULL] [1,2] [3,NULL] 2 +['a','b'] [] [1,2] [3,NULL] 1 +['a','b'] [] [1,2] [3,NULL] 2 +['a','b'] ['c',NULL] [] [3,NULL] 0 +['a','b'] ['c',NULL] [1,2] [] 1 +['a','b'] ['c',NULL] [1,2] [] 2 +- +['a','b'] ['c',NULL] [1,2] [3,NULL] 3 +['a','b'] ['c',NULL] [1,2] [3,NULL] \N +[] ['c',NULL] [1,2] [3,NULL] 3 +[] ['c',NULL] [1,2] [3,NULL] \N +['a','b'] [] [1,2] [3,NULL] 3 +['a','b'] [] [1,2] [3,NULL] \N +['a','b'] ['c',NULL] [] [3,NULL] 3 +['a','b'] ['c',NULL] [] [3,NULL] \N +['a','b'] ['c',NULL] [1,2] [] \N diff --git a/dbms/tests/queries/0_stateless/00752_low_cardinality_left_array_join.sql b/dbms/tests/queries/0_stateless/00752_low_cardinality_left_array_join.sql new file mode 100644 index 00000000000..f4fab9728e1 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00752_low_cardinality_left_array_join.sql @@ -0,0 +1,21 @@ +drop table if exists test.lc_left_aj; +CREATE TABLE test.lc_left_aj +( + str Array(LowCardinality(String)), + null_str Array(LowCardinality(Nullable(String))), + val Array(LowCardinality(Float64)), + null_val Array(LowCardinality(Nullable(Float64))) +) +ENGINE = Memory; + +insert into test.lc_left_aj values (['a', 'b'], ['c', Null], [1, 2.0], [3., Null]), ([], ['c', Null], [1, 2.0], [3., Null]), (['a', 'b'], [], [1, 2.0], [3., Null]), (['a', 'b'], ['c', Null], [], [3., Null]), (['a', 'b'], ['c', Null], [1, 2.0], []); + +select *, arr from test.lc_left_aj left array join str as arr; +select '-'; +select *, arr from test.lc_left_aj left array join null_str as arr; +select '-'; +select *, arr from test.lc_left_aj left array join val as arr; +select '-'; +select *, arr from test.lc_left_aj left array join null_val as arr; +drop table if exists test.lc_left_aj; + From fc7007b78e16f5f524e4cb5f2aecc5d32e3b4ed6 Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 23 Nov 2018 18:10:07 +0300 Subject: [PATCH 079/124] Add script and image for running integration tests in docker --- dbms/tests/integration/README.md | 57 ++++++++++++++++- dbms/tests/integration/image/Dockerfile | 61 +++++++++++++++++++ .../integration/image/dockerd-entrypoint.sh | 13 ++++ dbms/tests/integration/image/modprobe.sh | 20 ++++++ dbms/tests/integration/runner | 42 +++++++++++++ 5 files changed, 192 insertions(+), 1 deletion(-) create mode 100644 dbms/tests/integration/image/Dockerfile create mode 100755 dbms/tests/integration/image/dockerd-entrypoint.sh create mode 100755 dbms/tests/integration/image/modprobe.sh create mode 100755 dbms/tests/integration/runner diff --git a/dbms/tests/integration/README.md b/dbms/tests/integration/README.md index 822e37e0aa2..fe9740fd592 100644 --- a/dbms/tests/integration/README.md +++ b/dbms/tests/integration/README.md @@ -2,7 +2,7 @@ This directory contains tests that involve several ClickHouse instances, custom configs, ZooKeeper, etc. -### Running +### Running natively Prerequisites: * Ubuntu 14.04 (Trusty) or higher. @@ -30,6 +30,61 @@ set the following environment variables: * `CLICKHOUSE_TESTS_CLIENT_BIN_PATH` to choose the client binary. * `CLICKHOUSE_TESTS_BASE_CONFIG_DIR` to choose the directory from which base configs (`config.xml` and `users.xml`) are taken. + +### Running with runner script + +The only requirement is fresh docker with access to the internet. You can check it with: +``` +$ docker run ubuntu:14.04 ping github.com +PING github.com (140.82.118.3) 56(84) bytes of data. +64 bytes from 140.82.118.3: icmp_seq=1 ttl=53 time=40.1 ms +64 bytes from 140.82.118.3: icmp_seq=2 ttl=53 time=40.4 ms +64 bytes from 140.82.118.3: icmp_seq=3 ttl=53 time=40.3 ms +64 bytes from 140.82.118.3: icmp_seq=4 ttl=53 time=40.1 ms + +--- github.com ping statistics --- +4 packets transmitted, 4 received, 0% packet loss, time 19823ms +rtt min/avg/max/mdev = 40.157/40.284/40.463/0.278 ms +``` + +You can run tests via `./runner` script and pass pytest arguments as last arg: +``` +$ ./runner.py --binary $HOME/ClickHouse/dbms/programs/clickhouse --configs-dir $HOME/ClickHouse/dbms/programs/server/ 'test_odbc_interaction -ss' +Start tests +============================= test session starts ============================== +platform linux2 -- Python 2.7.15rc1, pytest-4.0.0, py-1.7.0, pluggy-0.8.0 +rootdir: /ClickHouse/dbms/tests/integration, inifile: pytest.ini +collected 6 items + +test_odbc_interaction/test.py Removing network clickhouse_default +... + +Killing roottestodbcinteraction_node1_1 ... done +Killing roottestodbcinteraction_mysql1_1 ... done +Killing roottestodbcinteraction_postgres1_1 ... done +Removing roottestodbcinteraction_node1_1 ... done +Removing roottestodbcinteraction_mysql1_1 ... done +Removing roottestodbcinteraction_postgres1_1 ... done +Removing network roottestodbcinteraction_default + +==================== 6 passed, 1 warnings in 95.21 seconds ===================== + +``` + +Path to binary and configs maybe specified via env variables: +``` +$ export CLICKHOUSE_TESTS_BASE_CONFIG_DIR=$HOME/ClickHouse/dbms/programs/server/ +$ export CLICKHOUSE_TESTS_SERVER_BIN_PATH=$HOME/ClickHouse/dbms/programs/clickhouse +$ ./runner.py 'test_odbc_interaction' +Start tests +============================= test session starts ============================== +platform linux2 -- Python 2.7.15rc1, pytest-4.0.0, py-1.7.0, pluggy-0.8.0 +rootdir: /ClickHouse/dbms/tests/integration, inifile: pytest.ini +collected 6 items + +test_odbc_interaction/test.py ...... [100%] +==================== 6 passed, 1 warnings in 96.33 seconds ===================== +``` ### Adding new tests diff --git a/dbms/tests/integration/image/Dockerfile b/dbms/tests/integration/image/Dockerfile new file mode 100644 index 00000000000..4216f8efffb --- /dev/null +++ b/dbms/tests/integration/image/Dockerfile @@ -0,0 +1,61 @@ +FROM ubuntu + + +RUN apt-get update && env DEBIAN_FRONTEND=noninteractive apt-get install --yes --force-yes \ + ca-certificates \ + bash \ + btrfs-progs \ + e2fsprogs \ + iptables \ + xfsprogs \ + tar \ + pigz \ + wget \ + git \ + iproute2 \ + module-init-tools \ + cgroupfs-mount \ + python-pip \ + tzdata + +ENV TZ=Europe/Moscow +RUN ln -snf /usr/share/zoneinfo/$TZ /etc/localtime && echo $TZ > /etc/timezone + +RUN pip install pytest docker-compose==1.22.0 docker dicttoxml kazoo PyMySQL psycopg2 + +ENV DOCKER_CHANNEL stable +ENV DOCKER_VERSION 18.09.0 + +RUN set -eux; \ + \ +# this "case" statement is generated via "update.sh" + \ + if ! wget -O docker.tgz "https://download.docker.com/linux/static/${DOCKER_CHANNEL}/x86_64/docker-${DOCKER_VERSION}.tgz"; then \ + echo >&2 "error: failed to download 'docker-${DOCKER_VERSION}' from '${DOCKER_CHANNEL}' for '${x86_64}'"; \ + exit 1; \ + fi; \ + \ + tar --extract \ + --file docker.tgz \ + --strip-components 1 \ + --directory /usr/local/bin/ \ + ; \ + rm docker.tgz; \ + \ + dockerd --version; \ + docker --version + +COPY modprobe.sh /usr/local/bin/modprobe +COPY dockerd-entrypoint.sh /usr/local/bin/ + +RUN set -x \ + && addgroup --system dockremap \ + && adduser --system dockremap \ + && adduser dockremap dockremap \ + && echo 'dockremap:165536:65536' >> /etc/subuid \ + && echo 'dockremap:165536:65536' >> /etc/subgid + +VOLUME /var/lib/docker +EXPOSE 2375 +ENTRYPOINT ["dockerd-entrypoint.sh"] +CMD [] \ No newline at end of file diff --git a/dbms/tests/integration/image/dockerd-entrypoint.sh b/dbms/tests/integration/image/dockerd-entrypoint.sh new file mode 100755 index 00000000000..d8bf9511023 --- /dev/null +++ b/dbms/tests/integration/image/dockerd-entrypoint.sh @@ -0,0 +1,13 @@ +#!/bin/bash +set -e + +dockerd --host=unix:///var/run/docker.sock --host=tcp://0.0.0.0:2375 &>/var/log/somefile & + +sleep 3 # to actual start + +echo "Start tests" +export CLICKHOUSE_TESTS_SERVER_BIN_PATH=/clickhouse +export CLICKHOUSE_TESTS_CLIENT_BIN_PATH=/clickhouse +export CLICKHOUSE_TESTS_BASE_CONFIG_DIR=/clickhouse-config + +cd /ClickHouse/dbms/tests/integration && pytest $PYTEST_OPTS diff --git a/dbms/tests/integration/image/modprobe.sh b/dbms/tests/integration/image/modprobe.sh new file mode 100755 index 00000000000..b357d893fda --- /dev/null +++ b/dbms/tests/integration/image/modprobe.sh @@ -0,0 +1,20 @@ +#!/bin/sh +set -eu + +# "modprobe" without modprobe +# https://twitter.com/lucabruno/status/902934379835662336 + +# this isn't 100% fool-proof, but it'll have a much higher success rate than simply using the "real" modprobe + +# Docker often uses "modprobe -va foo bar baz" +# so we ignore modules that start with "-" +for module; do + if [ "${module#-}" = "$module" ]; then + ip link show "$module" || true + lsmod | grep "$module" || true + fi +done + +# remove /usr/local/... from PATH so we can exec the real modprobe as a last resort +export PATH='/usr/sbin:/usr/bin:/sbin:/bin' +exec modprobe "$@" diff --git a/dbms/tests/integration/runner b/dbms/tests/integration/runner new file mode 100755 index 00000000000..542b160fa28 --- /dev/null +++ b/dbms/tests/integration/runner @@ -0,0 +1,42 @@ +#!/usr/bin/env python +#-*- coding: utf-8 -*- +import subprocess +import os +import argparse +import logging + +CUR_FILE_DIR_PATH = os.path.dirname(os.path.realpath(__file__)) +DEFAULT_CLICKHOUSE_ROOT = os.path.abspath(os.path.join(CUR_FILE_DIR_PATH, "../../../")) + +DIND_INTEGRATION_TESTS_IMAGE_NAME = "yandex/clickhouse-integration-tests-runner" + +if __name__ == "__main__": + logging.basicConfig(level=logging.INFO, format='%(asctime)s %(message)s') + parser = argparse.ArgumentParser(description="ClickHouse integration tests runner") + parser.add_argument( + "--binary", + default=os.environ.get("CLICKHOUSE_TESTS_SERVER_BIN_PATH", os.environ.get("CLICKHOUSE_TESTS_CLIENT_BIN_PATH", "/usr/bin/clickhouse")), + help="Path to clickhouse binary") + parser.add_argument( + "--configs-dir", + default=os.environ.get("CLICKHOUSE_TESTS_BASE_CONFIG_DIR", "/etc/clickhouse-server"), + help="Path to clickhouse configs directory" + ) + parser.add_argument( + "--clickhouse-root", + default=DEFAULT_CLICKHOUSE_ROOT, + help="Path to repository root folder" + ) + parser.add_argument('pytest_args', nargs='*', help="args for pytest command") + + args = parser.parse_args() + + cmd = "docker run --privileged --volume={bin}:/clickhouse --volume={cfg}:/clickhouse-config --volume={pth}:/ClickHouse -e PYTEST_OPTS='{opts}' {img}".format( + bin=args.binary, + cfg=args.configs_dir, + pth=args.clickhouse_root, + opts=' '.join(args.pytest_args), + img=DIND_INTEGRATION_TESTS_IMAGE_NAME, + ) + + subprocess.check_call(cmd, shell=True) From 3cfe5283e33abb80aba8022283e8671bccc381be Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 23 Nov 2018 18:51:36 +0300 Subject: [PATCH 080/124] Fix readme and add net=host --- dbms/tests/integration/README.md | 4 ++-- dbms/tests/integration/runner | 3 ++- 2 files changed, 4 insertions(+), 3 deletions(-) diff --git a/dbms/tests/integration/README.md b/dbms/tests/integration/README.md index fe9740fd592..84db3876727 100644 --- a/dbms/tests/integration/README.md +++ b/dbms/tests/integration/README.md @@ -49,7 +49,7 @@ rtt min/avg/max/mdev = 40.157/40.284/40.463/0.278 ms You can run tests via `./runner` script and pass pytest arguments as last arg: ``` -$ ./runner.py --binary $HOME/ClickHouse/dbms/programs/clickhouse --configs-dir $HOME/ClickHouse/dbms/programs/server/ 'test_odbc_interaction -ss' +$ ./runner --binary $HOME/ClickHouse/dbms/programs/clickhouse --configs-dir $HOME/ClickHouse/dbms/programs/server/ 'test_odbc_interaction -ss' Start tests ============================= test session starts ============================== platform linux2 -- Python 2.7.15rc1, pytest-4.0.0, py-1.7.0, pluggy-0.8.0 @@ -75,7 +75,7 @@ Path to binary and configs maybe specified via env variables: ``` $ export CLICKHOUSE_TESTS_BASE_CONFIG_DIR=$HOME/ClickHouse/dbms/programs/server/ $ export CLICKHOUSE_TESTS_SERVER_BIN_PATH=$HOME/ClickHouse/dbms/programs/clickhouse -$ ./runner.py 'test_odbc_interaction' +$ ./runner 'test_odbc_interaction' Start tests ============================= test session starts ============================== platform linux2 -- Python 2.7.15rc1, pytest-4.0.0, py-1.7.0, pluggy-0.8.0 diff --git a/dbms/tests/integration/runner b/dbms/tests/integration/runner index 542b160fa28..ab1ca8003af 100755 --- a/dbms/tests/integration/runner +++ b/dbms/tests/integration/runner @@ -31,7 +31,8 @@ if __name__ == "__main__": args = parser.parse_args() - cmd = "docker run --privileged --volume={bin}:/clickhouse --volume={cfg}:/clickhouse-config --volume={pth}:/ClickHouse -e PYTEST_OPTS='{opts}' {img}".format( + cmd = "docker run --net=host --privileged --volume={bin}:/clickhouse \ + --volume={cfg}:/clickhouse-config --volume={pth}:/ClickHouse -e PYTEST_OPTS='{opts}' {img}".format( bin=args.binary, cfg=args.configs_dir, pth=args.clickhouse_root, From 15ad4b9b1b2ce28a099495d9771c5fb0861201e7 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 23 Nov 2018 18:56:20 +0300 Subject: [PATCH 081/124] Whitespaces [#CLICKHOUSE-2] --- dbms/programs/server/users.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/programs/server/users.xml b/dbms/programs/server/users.xml index 6f746baf2a9..979b2d3fc17 100644 --- a/dbms/programs/server/users.xml +++ b/dbms/programs/server/users.xml @@ -56,7 +56,7 @@ Each element of list has one of the following forms: IP-address or network mask. Examples: 213.180.204.3 or 10.0.0.1/8 or 10.0.0.1/255.255.255.0 - 2a02:6b8::3 or 2a02:6b8::3/64 or 2a02:6b8::3/ffff:ffff:ffff:ffff::. + 2a02:6b8::3 or 2a02:6b8::3/64 or 2a02:6b8::3/ffff:ffff:ffff:ffff::. Hostname. Example: server01.yandex.ru. To check access, DNS query is performed, and all received addresses compared to peer address. Regular expression for host names. Example, ^server\d\d-\d\d-\d\.yandex\.ru$ From de16f3625638b8b0e7378b59f38a1e3cce2b0c61 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 23 Nov 2018 18:59:45 +0300 Subject: [PATCH 082/124] Fix functions with low cardinality argumant and array result. --- dbms/src/DataTypes/DataTypeLowCardinality.cpp | 4 +--- dbms/src/DataTypes/DataTypeNullable.h | 1 + dbms/src/DataTypes/IDataType.h | 2 ++ dbms/src/Functions/IFunction.cpp | 9 ++++++--- 4 files changed, 10 insertions(+), 6 deletions(-) diff --git a/dbms/src/DataTypes/DataTypeLowCardinality.cpp b/dbms/src/DataTypes/DataTypeLowCardinality.cpp index 1e973bc9c9b..5e2c58c31ae 100644 --- a/dbms/src/DataTypes/DataTypeLowCardinality.cpp +++ b/dbms/src/DataTypes/DataTypeLowCardinality.cpp @@ -42,9 +42,7 @@ DataTypeLowCardinality::DataTypeLowCardinality(DataTypePtr dictionary_type_) if (dictionary_type->isNullable()) inner_type = static_cast(*dictionary_type).getNestedType(); - if (!isStringOrFixedString(inner_type) - && !isDateOrDateTime(inner_type) - && !isNumber(inner_type)) + if (!inner_type->canBeInsideLowCardinality()) throw Exception("DataTypeLowCardinality is supported only for numbers, strings, Date or DateTime, but got " + dictionary_type->getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); } diff --git a/dbms/src/DataTypes/DataTypeNullable.h b/dbms/src/DataTypes/DataTypeNullable.h index a8bb00862aa..c6c228d441d 100644 --- a/dbms/src/DataTypes/DataTypeNullable.h +++ b/dbms/src/DataTypes/DataTypeNullable.h @@ -91,6 +91,7 @@ public: bool isNullable() const override { return true; } size_t getSizeOfValueInMemory() const override; bool onlyNull() const override; + bool canBeInsideLowCardinality() const override { return nested_data_type->canBeInsideLowCardinality(); } const DataTypePtr & getNestedType() const { return nested_data_type; } diff --git a/dbms/src/DataTypes/IDataType.h b/dbms/src/DataTypes/IDataType.h index 727d80540ce..7abac15e294 100644 --- a/dbms/src/DataTypes/IDataType.h +++ b/dbms/src/DataTypes/IDataType.h @@ -398,6 +398,8 @@ public: virtual bool lowCardinality() const { return false; } + /// Strings, Numbers, Date, DateTime, Nullable + virtual bool canBeInsideLowCardinality() const { return isStringOrFixedString() || isDateOrDateTime() || isNumber(); } /// Updates avg_value_size_hint for newly read column. Uses to optimize deserialization. Zero expected for first column. static void updateAvgValueSizeHint(const IColumn & column, double & avg_value_size_hint); diff --git a/dbms/src/Functions/IFunction.cpp b/dbms/src/Functions/IFunction.cpp index e0d1081246a..7d6d09624d2 100644 --- a/dbms/src/Functions/IFunction.cpp +++ b/dbms/src/Functions/IFunction.cpp @@ -656,11 +656,14 @@ DataTypePtr FunctionBuilderImpl::getReturnType(const ColumnsWithTypeAndName & ar arg.type = recursiveRemoveLowCardinality(arg.type); } + auto type_without_low_cardinality = getReturnTypeWithoutLowCardinality(args_without_low_cardinality); + if (canBeExecutedOnLowCardinalityDictionary() && has_low_cardinality - && num_full_low_cardinality_columns <= 1 && num_full_ordinary_columns == 0) - return std::make_shared(getReturnTypeWithoutLowCardinality(args_without_low_cardinality)); + && num_full_low_cardinality_columns <= 1 && num_full_ordinary_columns == 0 + && type_without_low_cardinality->canBeInsideLowCardinality()) + return std::make_shared(type_without_low_cardinality); else - return getReturnTypeWithoutLowCardinality(args_without_low_cardinality); + return type_without_low_cardinality; } return getReturnTypeWithoutLowCardinality(arguments); From 560e0a04d0f3b3cd5dc45ff2ea395d11ede46d4c Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 23 Nov 2018 19:09:07 +0300 Subject: [PATCH 083/124] Fix functions with low cardinality argumant and array result. --- dbms/src/DataTypes/DataTypeFixedString.h | 1 + dbms/src/DataTypes/DataTypeNumberBase.h | 1 + dbms/src/DataTypes/DataTypeString.h | 1 + dbms/src/DataTypes/IDataType.h | 2 +- 4 files changed, 4 insertions(+), 1 deletion(-) diff --git a/dbms/src/DataTypes/DataTypeFixedString.h b/dbms/src/DataTypes/DataTypeFixedString.h index 34a3f02c3c1..abea529ad42 100644 --- a/dbms/src/DataTypes/DataTypeFixedString.h +++ b/dbms/src/DataTypes/DataTypeFixedString.h @@ -82,6 +82,7 @@ public: size_t getSizeOfValueInMemory() const override { return n; } bool isCategorial() const override { return true; } bool canBeInsideNullable() const override { return true; } + bool canBeInsideLowCardinality() const override { return true; } }; } diff --git a/dbms/src/DataTypes/DataTypeNumberBase.h b/dbms/src/DataTypes/DataTypeNumberBase.h index 2aac709090c..67da1c7602e 100644 --- a/dbms/src/DataTypes/DataTypeNumberBase.h +++ b/dbms/src/DataTypes/DataTypeNumberBase.h @@ -51,6 +51,7 @@ public: bool haveMaximumSizeOfValue() const override { return true; } size_t getSizeOfValueInMemory() const override { return sizeof(T); } bool isCategorial() const override { return isValueRepresentedByInteger(); } + bool canBeInsideLowCardinality() const override { return true; } }; } diff --git a/dbms/src/DataTypes/DataTypeString.h b/dbms/src/DataTypes/DataTypeString.h index b2e36e30cff..d0a210dcbf7 100644 --- a/dbms/src/DataTypes/DataTypeString.h +++ b/dbms/src/DataTypes/DataTypeString.h @@ -61,6 +61,7 @@ public: bool isValueUnambiguouslyRepresentedInContiguousMemoryRegion() const override { return true; } bool isCategorial() const override { return true; } bool canBeInsideNullable() const override { return true; } + bool canBeInsideLowCardinality() const override { return true; } }; } diff --git a/dbms/src/DataTypes/IDataType.h b/dbms/src/DataTypes/IDataType.h index 7abac15e294..f88fdfc22a5 100644 --- a/dbms/src/DataTypes/IDataType.h +++ b/dbms/src/DataTypes/IDataType.h @@ -399,7 +399,7 @@ public: virtual bool lowCardinality() const { return false; } /// Strings, Numbers, Date, DateTime, Nullable - virtual bool canBeInsideLowCardinality() const { return isStringOrFixedString() || isDateOrDateTime() || isNumber(); } + virtual bool canBeInsideLowCardinality() const { return false; } /// Updates avg_value_size_hint for newly read column. Uses to optimize deserialization. Zero expected for first column. static void updateAvgValueSizeHint(const IColumn & column, double & avg_value_size_hint); From ced3937e8701bdf9a56624e379968b9b12eb335e Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 23 Nov 2018 20:00:45 +0300 Subject: [PATCH 084/124] Added test. --- .../0_stateless/00752_low_cardinality_array_result.reference | 3 +++ .../queries/0_stateless/00752_low_cardinality_array_result.sql | 2 ++ 2 files changed, 5 insertions(+) create mode 100644 dbms/tests/queries/0_stateless/00752_low_cardinality_array_result.reference create mode 100644 dbms/tests/queries/0_stateless/00752_low_cardinality_array_result.sql diff --git a/dbms/tests/queries/0_stateless/00752_low_cardinality_array_result.reference b/dbms/tests/queries/0_stateless/00752_low_cardinality_array_result.reference new file mode 100644 index 00000000000..de980441c3a --- /dev/null +++ b/dbms/tests/queries/0_stateless/00752_low_cardinality_array_result.reference @@ -0,0 +1,3 @@ +a +b +c diff --git a/dbms/tests/queries/0_stateless/00752_low_cardinality_array_result.sql b/dbms/tests/queries/0_stateless/00752_low_cardinality_array_result.sql new file mode 100644 index 00000000000..a14138f19f9 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00752_low_cardinality_array_result.sql @@ -0,0 +1,2 @@ +SELECT arrayJoin(splitByChar(',', toLowCardinality('a,b,c'))); + From 2bd5e2b1ebb4e13d4ba544d67ebbc6c4ee8958bd Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 23 Nov 2018 20:56:50 +0300 Subject: [PATCH 085/124] https://stackoverflow.com/questions/44067609/getting-error-importmismatcherror-while-running-py-test --- dbms/tests/integration/.dockerignore | 2 ++ 1 file changed, 2 insertions(+) create mode 100644 dbms/tests/integration/.dockerignore diff --git a/dbms/tests/integration/.dockerignore b/dbms/tests/integration/.dockerignore new file mode 100644 index 00000000000..76683618065 --- /dev/null +++ b/dbms/tests/integration/.dockerignore @@ -0,0 +1,2 @@ +**/__pycache__ +**/*.pyc From dbbacf4f87522e5852182b6358181d22d161149b Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 23 Nov 2018 21:52:00 +0300 Subject: [PATCH 086/124] Added more style checks [#CLICKHOUSE-3] --- contrib/base64-cmake/.gitignore | 2 +- dbms/programs/odbc-bridge/ColumnInfoHandler.cpp | 4 ++-- dbms/programs/odbc-bridge/HandlerFactory.cpp | 2 +- dbms/src/Columns/ReverseIndex.h | 2 +- dbms/src/Core/BackgroundSchedulePool.cpp | 2 +- dbms/src/Core/ExternalTable.cpp | 2 +- dbms/src/DataStreams/DistinctSortedBlockInputStream.cpp | 2 +- dbms/src/Databases/DatabasesCommon.cpp | 2 +- dbms/src/Dictionaries/ExecutableDictionarySource.cpp | 2 +- dbms/src/Functions/CMakeLists.txt | 2 +- dbms/src/Functions/FunctionsArithmetic.h | 2 +- dbms/src/Functions/FunctionsVisitParam.h | 2 +- dbms/src/Functions/GatherUtils/Selectors.h | 2 +- dbms/src/Functions/arrayIntersect.cpp | 4 ++-- dbms/src/Functions/formatDateTime.cpp | 2 +- dbms/src/IO/ReadHelpers.h | 2 +- dbms/src/Interpreters/ExpressionActions.cpp | 2 +- dbms/src/Interpreters/InterpreterSelectQuery.cpp | 2 +- dbms/src/Interpreters/PredicateExpressionsOptimizer.cpp | 2 +- dbms/src/Storages/MergeTree/MergeTreeData.h | 2 +- dbms/src/Storages/StorageMySQL.cpp | 2 +- dbms/src/Storages/StorageReplicatedMergeTree.cpp | 2 +- utils/check-style/check-style | 4 ++-- 23 files changed, 26 insertions(+), 26 deletions(-) diff --git a/contrib/base64-cmake/.gitignore b/contrib/base64-cmake/.gitignore index 299bb98e503..0e56cf2f8c1 100644 --- a/contrib/base64-cmake/.gitignore +++ b/contrib/base64-cmake/.gitignore @@ -1 +1 @@ -config.h \ No newline at end of file +config.h diff --git a/dbms/programs/odbc-bridge/ColumnInfoHandler.cpp b/dbms/programs/odbc-bridge/ColumnInfoHandler.cpp index f59abd5f587..997ef9cf216 100644 --- a/dbms/programs/odbc-bridge/ColumnInfoHandler.cpp +++ b/dbms/programs/odbc-bridge/ColumnInfoHandler.cpp @@ -124,9 +124,9 @@ void ODBCColumnsInfoHandler::handleRequest(Poco::Net::HTTPServerRequest & reques auto identifier_quote = getIdentifierQuote(hdbc); if (identifier_quote.length() == 0) settings.identifier_quoting_style = IdentifierQuotingStyle::None; - else if(identifier_quote[0] == '`') + else if (identifier_quote[0] == '`') settings.identifier_quoting_style = IdentifierQuotingStyle::Backticks; - else if(identifier_quote[0] == '"') + else if (identifier_quote[0] == '"') settings.identifier_quoting_style = IdentifierQuotingStyle::DoubleQuotes; else throw Exception("Can not map quote identifier '" + identifier_quote + "' to IdentifierQuotingStyle value", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); diff --git a/dbms/programs/odbc-bridge/HandlerFactory.cpp b/dbms/programs/odbc-bridge/HandlerFactory.cpp index 8a0ff06268f..a6422db268c 100644 --- a/dbms/programs/odbc-bridge/HandlerFactory.cpp +++ b/dbms/programs/odbc-bridge/HandlerFactory.cpp @@ -25,7 +25,7 @@ Poco::Net::HTTPRequestHandler * HandlerFactory::createRequestHandler(const Poco: #else return nullptr; #endif - else if(uri.getPath() == "/identifier_quote") + else if (uri.getPath() == "/identifier_quote") #if USE_POCO_SQLODBC || USE_POCO_DATAODBC return new IdentifierQuoteHandler(keep_alive_timeout, context); #else diff --git a/dbms/src/Columns/ReverseIndex.h b/dbms/src/Columns/ReverseIndex.h index cf735343167..a003e8282dc 100644 --- a/dbms/src/Columns/ReverseIndex.h +++ b/dbms/src/Columns/ReverseIndex.h @@ -272,7 +272,7 @@ public: auto hash = calcHashes(); ptr = &hash->getData()[0]; UInt64 * expected = nullptr; - if(saved_hash_ptr.compare_exchange_strong(expected, ptr)) + if (saved_hash_ptr.compare_exchange_strong(expected, ptr)) saved_hash = std::move(hash); else ptr = expected; diff --git a/dbms/src/Core/BackgroundSchedulePool.cpp b/dbms/src/Core/BackgroundSchedulePool.cpp index 3e24cfca922..5da499e5ae9 100644 --- a/dbms/src/Core/BackgroundSchedulePool.cpp +++ b/dbms/src/Core/BackgroundSchedulePool.cpp @@ -278,7 +278,7 @@ void BackgroundSchedulePool::delayExecutionThreadFunction() { std::unique_lock lock(delayed_tasks_mutex); - while(!shutdown) + while (!shutdown) { Poco::Timestamp min_time; diff --git a/dbms/src/Core/ExternalTable.cpp b/dbms/src/Core/ExternalTable.cpp index 9ab3a8e8d35..5bfdbb12e95 100644 --- a/dbms/src/Core/ExternalTable.cpp +++ b/dbms/src/Core/ExternalTable.cpp @@ -168,7 +168,7 @@ void ExternalTablesHandler::handlePart(const Poco::Net::MessageHeader & header, /// Write data data.first->readPrefix(); output->writePrefix(); - while(Block block = data.first->read()) + while (Block block = data.first->read()) output->write(block); data.first->readSuffix(); output->writeSuffix(); diff --git a/dbms/src/DataStreams/DistinctSortedBlockInputStream.cpp b/dbms/src/DataStreams/DistinctSortedBlockInputStream.cpp index a14d32794f3..651b9738f73 100644 --- a/dbms/src/DataStreams/DistinctSortedBlockInputStream.cpp +++ b/dbms/src/DataStreams/DistinctSortedBlockInputStream.cpp @@ -150,7 +150,7 @@ ColumnRawPtrs DistinctSortedBlockInputStream::getClearingColumns(const Block & b { ColumnRawPtrs clearing_hint_columns; clearing_hint_columns.reserve(description.size()); - for(const auto & sort_column_description : description) + for (const auto & sort_column_description : description) { const auto sort_column_ptr = block.safeGetByPosition(sort_column_description.column_number).column.get(); const auto it = std::find(key_columns.cbegin(), key_columns.cend(), sort_column_ptr); diff --git a/dbms/src/Databases/DatabasesCommon.cpp b/dbms/src/Databases/DatabasesCommon.cpp index 2617390fd6d..2d8dbbc2392 100644 --- a/dbms/src/Databases/DatabasesCommon.cpp +++ b/dbms/src/Databases/DatabasesCommon.cpp @@ -163,7 +163,7 @@ DatabaseWithOwnTablesBase::~DatabaseWithOwnTablesBase() { shutdown(); } - catch(...) + catch (...) { tryLogCurrentException(__PRETTY_FUNCTION__); } diff --git a/dbms/src/Dictionaries/ExecutableDictionarySource.cpp b/dbms/src/Dictionaries/ExecutableDictionarySource.cpp index 8dae4cfce2a..c17ce3e2461 100644 --- a/dbms/src/Dictionaries/ExecutableDictionarySource.cpp +++ b/dbms/src/Dictionaries/ExecutableDictionarySource.cpp @@ -213,7 +213,7 @@ bool ExecutableDictionarySource::supportsSelectiveLoad() const bool ExecutableDictionarySource::hasUpdateField() const { - if(update_field.empty()) + if (update_field.empty()) return false; else return true; diff --git a/dbms/src/Functions/CMakeLists.txt b/dbms/src/Functions/CMakeLists.txt index 4203939a289..95268580dda 100644 --- a/dbms/src/Functions/CMakeLists.txt +++ b/dbms/src/Functions/CMakeLists.txt @@ -84,4 +84,4 @@ endif () if (USE_BASE64) target_include_directories (clickhouse_functions SYSTEM PRIVATE ${BASE64_INCLUDE_DIR}) -endif() \ No newline at end of file +endif() diff --git a/dbms/src/Functions/FunctionsArithmetic.h b/dbms/src/Functions/FunctionsArithmetic.h index ab0b5a1ec00..7589d7b912d 100644 --- a/dbms/src/Functions/FunctionsArithmetic.h +++ b/dbms/src/Functions/FunctionsArithmetic.h @@ -1238,7 +1238,7 @@ class FunctionBinaryArithmetic : public IFunction auto column_to = ColumnAggregateFunction::create(columns[0]->getAggregateFunction()); column_to->reserve(input_rows_count); - for(size_t i = 0; i < input_rows_count; ++i) + for (size_t i = 0; i < input_rows_count; ++i) { column_to->insertFrom(columns[0]->getData()[i]); column_to->insertMergeFrom(columns[1]->getData()[i]); diff --git a/dbms/src/Functions/FunctionsVisitParam.h b/dbms/src/Functions/FunctionsVisitParam.h index 82bf8946e7c..13ae4002236 100644 --- a/dbms/src/Functions/FunctionsVisitParam.h +++ b/dbms/src/Functions/FunctionsVisitParam.h @@ -104,7 +104,7 @@ struct ExtractRaw } else { - switch(*pos) + switch (*pos) { case '[': expects_end.push_back((current_expect_end = ']')); diff --git a/dbms/src/Functions/GatherUtils/Selectors.h b/dbms/src/Functions/GatherUtils/Selectors.h index 8b81d2f1db6..a09fdb4105a 100644 --- a/dbms/src/Functions/GatherUtils/Selectors.h +++ b/dbms/src/Functions/GatherUtils/Selectors.h @@ -179,7 +179,7 @@ struct ArrayAndValueSourceSelectorBySink : public ArraySinkSelector *>(&array_source), typeid_cast *>(&value_source)); - else if(array_source.isConst()) + else if (array_source.isConst()) checkTypeAndCallConcat(typeid_cast *>(&array_source), typeid_cast(&value_source)); else if (value_source.isConst()) diff --git a/dbms/src/Functions/arrayIntersect.cpp b/dbms/src/Functions/arrayIntersect.cpp index b646d5d89b2..e5f1d4f0dac 100644 --- a/dbms/src/Functions/arrayIntersect.cpp +++ b/dbms/src/Functions/arrayIntersect.cpp @@ -324,9 +324,9 @@ void FunctionArrayIntersect::executeImpl(Block & block, const ColumnNumbers & ar result_column = execute, true>(arrays, std::move(column)); else if (which.isDateTime()) result_column = execute, true>(arrays, std::move(column)); - else if(which.isString()) + else if (which.isString()) result_column = execute(arrays, std::move(column)); - else if(which.isFixedString()) + else if (which.isFixedString()) result_column = execute(arrays, std::move(column)); else { diff --git a/dbms/src/Functions/formatDateTime.cpp b/dbms/src/Functions/formatDateTime.cpp index 874cfcfcd5e..d9d76ab936c 100644 --- a/dbms/src/Functions/formatDateTime.cpp +++ b/dbms/src/Functions/formatDateTime.cpp @@ -350,7 +350,7 @@ public: for (size_t i = 0; i < vec.size(); ++i) { - for(auto & instruction : instructions) + for (auto & instruction : instructions) instruction.perform(pos, vec[i], time_zone); dst_offsets[i] = pos - begin; diff --git a/dbms/src/IO/ReadHelpers.h b/dbms/src/IO/ReadHelpers.h index bb012585a7c..dcf866848c3 100644 --- a/dbms/src/IO/ReadHelpers.h +++ b/dbms/src/IO/ReadHelpers.h @@ -58,7 +58,7 @@ namespace ErrorCodes inline char parseEscapeSequence(char c) { - switch(c) + switch (c) { case 'a': return '\a'; diff --git a/dbms/src/Interpreters/ExpressionActions.cpp b/dbms/src/Interpreters/ExpressionActions.cpp index a2c0a9ceb46..11a7ecfb1a4 100644 --- a/dbms/src/Interpreters/ExpressionActions.cpp +++ b/dbms/src/Interpreters/ExpressionActions.cpp @@ -1150,7 +1150,7 @@ UInt128 ExpressionAction::ActionHash::operator()(const ExpressionAction & action SipHash hash; hash.update(action.type); hash.update(action.is_function_compiled); - switch(action.type) + switch (action.type) { case ADD_COLUMN: hash.update(action.result_name); diff --git a/dbms/src/Interpreters/InterpreterSelectQuery.cpp b/dbms/src/Interpreters/InterpreterSelectQuery.cpp index 5ee3b1cc5bf..2a139edce9f 100644 --- a/dbms/src/Interpreters/InterpreterSelectQuery.cpp +++ b/dbms/src/Interpreters/InterpreterSelectQuery.cpp @@ -590,7 +590,7 @@ void InterpreterSelectQuery::executeImpl(Pipeline & pipeline, const BlockInputSt if (query.group_by_with_rollup) executeRollupOrCube(pipeline, Modificator::ROLLUP); - else if(query.group_by_with_cube) + else if (query.group_by_with_cube) executeRollupOrCube(pipeline, Modificator::CUBE); } else if (expressions.has_having) diff --git a/dbms/src/Interpreters/PredicateExpressionsOptimizer.cpp b/dbms/src/Interpreters/PredicateExpressionsOptimizer.cpp index bbb89e2aead..a29e161126d 100644 --- a/dbms/src/Interpreters/PredicateExpressionsOptimizer.cpp +++ b/dbms/src/Interpreters/PredicateExpressionsOptimizer.cpp @@ -72,7 +72,7 @@ bool PredicateExpressionsOptimizer::optimizeImpl( ASTPtr inner_predicate; cloneOuterPredicateForInnerPredicate(outer_predicate, projection_columns, database_and_table_with_aliases, inner_predicate); - switch(optimize_kind) + switch (optimize_kind) { case OptimizeKind::NONE: continue; case OptimizeKind::PUSH_TO_WHERE: is_rewrite_subquery |= optimizeExpression(inner_predicate, subquery->where_expression, subquery); continue; diff --git a/dbms/src/Storages/MergeTree/MergeTreeData.h b/dbms/src/Storages/MergeTree/MergeTreeData.h index 875e70fe3a7..034719828bd 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeData.h +++ b/dbms/src/Storages/MergeTree/MergeTreeData.h @@ -192,7 +192,7 @@ public: { rollback(); } - catch(...) + catch (...) { tryLogCurrentException("~MergeTreeData::Transaction"); } diff --git a/dbms/src/Storages/StorageMySQL.cpp b/dbms/src/Storages/StorageMySQL.cpp index 609c53486ee..0d25e82dcf3 100644 --- a/dbms/src/Storages/StorageMySQL.cpp +++ b/dbms/src/Storages/StorageMySQL.cpp @@ -100,7 +100,7 @@ public: } trans.commit(); } - catch(...) + catch (...) { trans.rollback(); throw; diff --git a/dbms/src/Storages/StorageReplicatedMergeTree.cpp b/dbms/src/Storages/StorageReplicatedMergeTree.cpp index a89ea555d42..022b6302779 100644 --- a/dbms/src/Storages/StorageReplicatedMergeTree.cpp +++ b/dbms/src/Storages/StorageReplicatedMergeTree.cpp @@ -2943,7 +2943,7 @@ StorageReplicatedMergeTree::~StorageReplicatedMergeTree() { shutdown(); } - catch(...) + catch (...) { tryLogCurrentException(__PRETTY_FUNCTION__); } diff --git a/utils/check-style/check-style b/utils/check-style/check-style index 53acaee55ae..6248f9f4adb 100755 --- a/utils/check-style/check-style +++ b/utils/check-style/check-style @@ -16,5 +16,5 @@ ROOT_PATH=$(git rev-parse --show-toplevel) find $ROOT_PATH/dbms -name '*.h' -or -name '*.cpp' | grep -vP 'Compiler|build' | - xargs grep $@ -P '((class|struct|namespace|enum|if|for|while|else|throw|switch).*|\)(\s*const)?(\s*override)?\s*)\{$|\s$|\t|^ {1,3}[^\* ]\S|\t' -# a curly brace not in a new line, but not for the case of C++11 init or agg. initialization | trailing whitespace | number of ws not a multiple of 4, but not in the case of comment continuation | a tab character + xargs grep $@ -P '((class|struct|namespace|enum|if|for|while|else|throw|switch).*|\)(\s*const)?(\s*override)?\s*)\{$|\s$|\t|^ {1,3}[^\* ]\S|\t|^\s*(if|else if|for|while|catch|switch)\(' +# a curly brace not in a new line, but not for the case of C++11 init or agg. initialization | trailing whitespace | number of ws not a multiple of 4, but not in the case of comment continuation | a tab character | missing whitespace after for/if/while... before opening brace From 62e30de7158ba40fb6aa523278a0e9958ce99088 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 23 Nov 2018 21:54:23 +0300 Subject: [PATCH 087/124] Whitespaces #3350 --- dbms/src/Functions/FunctionBase64Conversion.h | 2 +- dbms/src/Functions/base64Decode.cpp | 3 +-- dbms/src/Functions/base64Encode.cpp | 3 +-- dbms/src/Functions/tryBase64Decode.cpp | 3 +-- 4 files changed, 4 insertions(+), 7 deletions(-) diff --git a/dbms/src/Functions/FunctionBase64Conversion.h b/dbms/src/Functions/FunctionBase64Conversion.h index 418de6e59c8..33e52654530 100644 --- a/dbms/src/Functions/FunctionBase64Conversion.h +++ b/dbms/src/Functions/FunctionBase64Conversion.h @@ -163,4 +163,4 @@ private: } }; } -#endif \ No newline at end of file +#endif diff --git a/dbms/src/Functions/base64Decode.cpp b/dbms/src/Functions/base64Decode.cpp index 2adfc04463a..e79e6bcb18c 100644 --- a/dbms/src/Functions/base64Decode.cpp +++ b/dbms/src/Functions/base64Decode.cpp @@ -5,10 +5,9 @@ namespace DB { - void registerFunctionBase64Decode(FunctionFactory & factory) { factory.registerFunction>(); } } -#endif \ No newline at end of file +#endif diff --git a/dbms/src/Functions/base64Encode.cpp b/dbms/src/Functions/base64Encode.cpp index 38b9cdcddf7..9aeb3df40ac 100644 --- a/dbms/src/Functions/base64Encode.cpp +++ b/dbms/src/Functions/base64Encode.cpp @@ -5,10 +5,9 @@ namespace DB { - void registerFunctionBase64Encode(FunctionFactory & factory) { factory.registerFunction>(); } } -#endif \ No newline at end of file +#endif diff --git a/dbms/src/Functions/tryBase64Decode.cpp b/dbms/src/Functions/tryBase64Decode.cpp index 17b09d67eda..a475745265c 100644 --- a/dbms/src/Functions/tryBase64Decode.cpp +++ b/dbms/src/Functions/tryBase64Decode.cpp @@ -5,10 +5,9 @@ namespace DB { - void registerFunctionTryBase64Decode(FunctionFactory & factory) { factory.registerFunction>(); } } -#endif \ No newline at end of file +#endif From 5ec3db20cd2d037dc1834d7ca786e0bb0754f19d Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 23 Nov 2018 21:55:43 +0300 Subject: [PATCH 088/124] More style checks [#CLICKHOUSE-2] --- utils/check-style/check-style | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/utils/check-style/check-style b/utils/check-style/check-style index 6248f9f4adb..a79d24568d7 100755 --- a/utils/check-style/check-style +++ b/utils/check-style/check-style @@ -16,5 +16,5 @@ ROOT_PATH=$(git rev-parse --show-toplevel) find $ROOT_PATH/dbms -name '*.h' -or -name '*.cpp' | grep -vP 'Compiler|build' | - xargs grep $@ -P '((class|struct|namespace|enum|if|for|while|else|throw|switch).*|\)(\s*const)?(\s*override)?\s*)\{$|\s$|\t|^ {1,3}[^\* ]\S|\t|^\s*(if|else if|for|while|catch|switch)\(' + xargs grep $@ -P '((class|struct|namespace|enum|if|for|while|else|throw|switch).*|\)(\s*const)?(\s*override)?\s*)\{$|\s$|\t|^ {1,3}[^\* ]\S|\t|^\s*(if|else if|if constexpr|else if constexpr|for|while|catch|switch)\(' # a curly brace not in a new line, but not for the case of C++11 init or agg. initialization | trailing whitespace | number of ws not a multiple of 4, but not in the case of comment continuation | a tab character | missing whitespace after for/if/while... before opening brace From af6f91e762ef3448a45cd46167d60fed29f4abba Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 23 Nov 2018 22:47:15 +0300 Subject: [PATCH 089/124] Added comment #3350 --- dbms/src/Functions/FunctionBase64Conversion.h | 3 +++ 1 file changed, 3 insertions(+) diff --git a/dbms/src/Functions/FunctionBase64Conversion.h b/dbms/src/Functions/FunctionBase64Conversion.h index 33e52654530..3afa65c8aca 100644 --- a/dbms/src/Functions/FunctionBase64Conversion.h +++ b/dbms/src/Functions/FunctionBase64Conversion.h @@ -159,6 +159,9 @@ public: private: static int getCodec() { + /// You can provide different value if you want to test specific codecs. + /// Due to poor implementation of "base64" library (it will write to a global variable), + /// it doesn't scale for multiple threads. Never use non-zero values in production. return 0; } }; From 910d57c1b5bb779c7b029d5848d70751eafd2cde Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 23 Nov 2018 22:52:24 +0300 Subject: [PATCH 090/124] Better test #3406 --- dbms/tests/performance/uniq/uniq.xml | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/dbms/tests/performance/uniq/uniq.xml b/dbms/tests/performance/uniq/uniq.xml index ba255ac224e..ba9e347ac52 100644 --- a/dbms/tests/performance/uniq/uniq.xml +++ b/dbms/tests/performance/uniq/uniq.xml @@ -1,6 +1,6 @@ uniq - once + loop hits_100m_single @@ -18,7 +18,7 @@ - + From 6e2a6d989407e2aa934f7bb20f2462b068c640ee Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 24 Nov 2018 00:04:14 +0300 Subject: [PATCH 091/124] Added performance test #3350 --- dbms/tests/performance/base64/base64.xml | 40 +++++++++++++++++++ dbms/tests/performance/base64/base64_hits.xml | 38 ++++++++++++++++++ 2 files changed, 78 insertions(+) create mode 100644 dbms/tests/performance/base64/base64.xml create mode 100644 dbms/tests/performance/base64/base64_hits.xml diff --git a/dbms/tests/performance/base64/base64.xml b/dbms/tests/performance/base64/base64.xml new file mode 100644 index 00000000000..b8155529f28 --- /dev/null +++ b/dbms/tests/performance/base64/base64.xml @@ -0,0 +1,40 @@ + + base64 + once + + + + 10000 + + + 5000 + 20000 + + + + + + + + + + string + + materialize('Hello, world!') + materialize('This is a long string to test ClickHouse base64 functions impl..') + toString(number) + + + + table + + numbers + numbers_mt + + + + + SELECT count() FROM system.{table} WHERE NOT ignore(base64Encode({string})) + SELECT count() FROM system.{table} WHERE base64Decode(base64Encode({string})) != {string} + SELECT count() FROM system.{table} WHERE tryBase64Decode(base64Encode({string})) != {string} + diff --git a/dbms/tests/performance/base64/base64_hits.xml b/dbms/tests/performance/base64/base64_hits.xml new file mode 100644 index 00000000000..82def78f390 --- /dev/null +++ b/dbms/tests/performance/base64/base64_hits.xml @@ -0,0 +1,38 @@ + + base64 + loop + + + hits_100m_single + + + + + 10000 + + + 5000 + 20000 + + + + + + + + + + string + + SearchPhrase + MobilePhoneModel + URL + Title + + + + + SELECT count() FROM hits_100m_single WHERE NOT ignore(base64Encode({string})) + SELECT count() FROM hits_100m_single WHERE base64Decode(base64Encode({string})) != {string} + SELECT count() FROM hits_100m_single WHERE tryBase64Decode(base64Encode({string})) != {string} + From 954c9745079166ef76cd0df5863946d44b65b063 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 24 Nov 2018 00:14:28 +0300 Subject: [PATCH 092/124] Added "POWER" alias to "pow" function; made some math functions case insensitive for compatibility #3638 --- dbms/src/Functions/FunctionsMath.cpp | 28 ++++++++++--------- .../0_stateless/00756_power_alias.reference | 1 + .../queries/0_stateless/00756_power_alias.sql | 1 + 3 files changed, 17 insertions(+), 13 deletions(-) create mode 100644 dbms/tests/queries/0_stateless/00756_power_alias.reference create mode 100644 dbms/tests/queries/0_stateless/00756_power_alias.sql diff --git a/dbms/src/Functions/FunctionsMath.cpp b/dbms/src/Functions/FunctionsMath.cpp index 49036dbe247..eb549e425a2 100644 --- a/dbms/src/Functions/FunctionsMath.cpp +++ b/dbms/src/Functions/FunctionsMath.cpp @@ -11,26 +11,28 @@ const double PiImpl::value = 3.1415926535897932384626433832795028841971693; void registerFunctionsMath(FunctionFactory & factory) { factory.registerFunction(); - factory.registerFunction(); - factory.registerFunction(); - factory.registerFunction(); + factory.registerFunction(FunctionFactory::CaseInsensitive); + factory.registerFunction(FunctionFactory::CaseInsensitive); + factory.registerFunction(FunctionFactory::CaseInsensitive); factory.registerFunction(); - factory.registerFunction(); + factory.registerFunction(FunctionFactory::CaseInsensitive); factory.registerFunction(); - factory.registerFunction(); - factory.registerFunction(); + factory.registerFunction(FunctionFactory::CaseInsensitive); + factory.registerFunction(FunctionFactory::CaseInsensitive); factory.registerFunction(); factory.registerFunction(); factory.registerFunction(); factory.registerFunction(); factory.registerFunction(); - factory.registerFunction(); - factory.registerFunction(); - factory.registerFunction(); - factory.registerFunction(); - factory.registerFunction(); - factory.registerFunction(); - factory.registerFunction(); + factory.registerFunction(FunctionFactory::CaseInsensitive); + factory.registerFunction(FunctionFactory::CaseInsensitive); + factory.registerFunction(FunctionFactory::CaseInsensitive); + factory.registerFunction(FunctionFactory::CaseInsensitive); + factory.registerFunction(FunctionFactory::CaseInsensitive); + factory.registerFunction(FunctionFactory::CaseInsensitive); + factory.registerFunction(FunctionFactory::CaseInsensitive); + + factory.registerAlias("power", "pow", FunctionFactory::CaseInsensitive); } } diff --git a/dbms/tests/queries/0_stateless/00756_power_alias.reference b/dbms/tests/queries/0_stateless/00756_power_alias.reference new file mode 100644 index 00000000000..dcc7e7cf0a2 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00756_power_alias.reference @@ -0,0 +1 @@ +9 9 9 9 diff --git a/dbms/tests/queries/0_stateless/00756_power_alias.sql b/dbms/tests/queries/0_stateless/00756_power_alias.sql new file mode 100644 index 00000000000..e8f543ffb3a --- /dev/null +++ b/dbms/tests/queries/0_stateless/00756_power_alias.sql @@ -0,0 +1 @@ +SELECT pow(3, 2), POW(3, 2), power(3, 2), POWER(3, 2); From fac15418aafc059ab58a7b14b739b86d5c2b5494 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 24 Nov 2018 02:10:38 +0300 Subject: [PATCH 093/124] Added a test just in case #3584 --- .../tests/queries/0_stateless/00757_enum_defaults.reference | 6 ++++++ dbms/tests/queries/0_stateless/00757_enum_defaults.sql | 2 ++ 2 files changed, 8 insertions(+) create mode 100644 dbms/tests/queries/0_stateless/00757_enum_defaults.reference create mode 100644 dbms/tests/queries/0_stateless/00757_enum_defaults.sql diff --git a/dbms/tests/queries/0_stateless/00757_enum_defaults.reference b/dbms/tests/queries/0_stateless/00757_enum_defaults.reference new file mode 100644 index 00000000000..56ead34ad3b --- /dev/null +++ b/dbms/tests/queries/0_stateless/00757_enum_defaults.reference @@ -0,0 +1,6 @@ +iphone 1 + +iphone 1 +iphone 1 + +\N 1 diff --git a/dbms/tests/queries/0_stateless/00757_enum_defaults.sql b/dbms/tests/queries/0_stateless/00757_enum_defaults.sql new file mode 100644 index 00000000000..58f54a98b70 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00757_enum_defaults.sql @@ -0,0 +1,2 @@ +select os_name, count() from (SELECT CAST('iphone' AS Enum8('iphone' = 1, 'android' = 2)) AS os_name) group by os_name WITH TOTALS; +select toNullable(os_name) AS os_name, count() from (SELECT CAST('iphone' AS Enum8('iphone' = 1, 'android' = 2)) AS os_name) group by os_name WITH TOTALS; From 1e449b2e16658aa41183d79dfa00e09dd3235084 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 24 Nov 2018 02:14:18 +0300 Subject: [PATCH 094/124] Added another compatibility alias #3638 --- dbms/src/Functions/FunctionsMath.cpp | 1 + 1 file changed, 1 insertion(+) diff --git a/dbms/src/Functions/FunctionsMath.cpp b/dbms/src/Functions/FunctionsMath.cpp index eb549e425a2..0252361083f 100644 --- a/dbms/src/Functions/FunctionsMath.cpp +++ b/dbms/src/Functions/FunctionsMath.cpp @@ -33,6 +33,7 @@ void registerFunctionsMath(FunctionFactory & factory) factory.registerFunction(FunctionFactory::CaseInsensitive); factory.registerAlias("power", "pow", FunctionFactory::CaseInsensitive); + factory.registerAlias("ln", "log", FunctionFactory::CaseInsensitive); } } From 980e07784214111be2d98d15a0776fbca4512496 Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Sat, 24 Nov 2018 02:29:10 +0300 Subject: [PATCH 095/124] Update settings.md --- docs/ru/operations/server_settings/settings.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/ru/operations/server_settings/settings.md b/docs/ru/operations/server_settings/settings.md index 567551cc394..3b6360ac9f8 100644 --- a/docs/ru/operations/server_settings/settings.md +++ b/docs/ru/operations/server_settings/settings.md @@ -542,7 +542,7 @@ ClickHouse проверит условия `min_part_size` и `min_part_size_rat - database - Имя базы данных. - table - Имя таблицы. - partition_by - Устанавливает [произвольный ключ партиционирования](../../operations/table_engines/custom_partitioning_key.md#custom-partitioning-key). -- flush_interval_milliseconds - Период сброса данных из оперативной памяти на диск. +- flush_interval_milliseconds - Период сброса данных из буфера в памяти в таблицу. **Пример** @@ -584,7 +584,7 @@ ClickHouse проверит условия `min_part_size` и `min_part_size_rat - database - Имя базы данных. - table - Имя таблицы. - partition_by - Устанавливает [произвольный ключ партиционирования](../../operations/table_engines/custom_partitioning_key.md#custom-partitioning-key). -- flush_interval_milliseconds - Период сброса данных из оперативной памяти на диск. +- flush_interval_milliseconds - Период сброса данных из буфера в памяти в таблицу. Если таблица не существует, то ClickHouse создаст её. Если структура журнала запросов изменилась при обновлении сервера ClickHouse, то таблица со старой структурой переименовывается, а новая таблица создается автоматически. From 843d91965f173f7eb255f33fee970330f701efa9 Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Sat, 24 Nov 2018 02:30:02 +0300 Subject: [PATCH 096/124] Update settings.md --- docs/en/operations/server_settings/settings.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/en/operations/server_settings/settings.md b/docs/en/operations/server_settings/settings.md index 42dce70b948..f1546fbe62d 100644 --- a/docs/en/operations/server_settings/settings.md +++ b/docs/en/operations/server_settings/settings.md @@ -542,7 +542,7 @@ Use the following parameters to configure logging: - database – Name of the database. - table – Name of the table. - partition_by – Sets a [custom partitioning key](../../operations/table_engines/custom_partitioning_key.md#custom-partitioning-key). -- flush_interval_milliseconds – Interval for flushing data from memory to the disk. +- flush_interval_milliseconds – Interval for flushing data from the buffer in memory to the table. **Example** @@ -583,7 +583,7 @@ Use the following parameters to configure logging: - database – Name of the database. - table – Name of the table. - partition_by – Sets a [custom partitioning key](../../operations/table_engines/custom_partitioning_key.md#custom-partitioning-key). -- flush_interval_milliseconds – Interval for flushing data from memory to the disk. +- flush_interval_milliseconds – Interval for flushing data from the buffer in memory to the table. If the table doesn't exist, ClickHouse will create it. If the structure of the query log changed when the ClickHouse server was updated, the table with the old structure is renamed, and a new table is created automatically. From cc06488ddc0a0d80f41672bb81880c6799eb33ec Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 24 Nov 2018 03:10:51 +0300 Subject: [PATCH 097/124] Whitespace #2601 --- dbms/src/Storages/MergeTree/MergeTreeThreadBlockInputStream.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/dbms/src/Storages/MergeTree/MergeTreeThreadBlockInputStream.cpp b/dbms/src/Storages/MergeTree/MergeTreeThreadBlockInputStream.cpp index 3e4a96dc2e4..0fa4d4c7019 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeThreadBlockInputStream.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeThreadBlockInputStream.cpp @@ -81,7 +81,6 @@ bool MergeTreeThreadBlockInputStream::getNewTask() path, task->data_part, task->columns, owned_uncompressed_cache.get(), owned_mark_cache.get(), save_marks_in_cache, storage, rest_mark_ranges, min_bytes_to_use_direct_io, max_read_buffer_size, MergeTreeReader::ValueSizeMap{}, profile_callback); - if (prewhere_info) pre_reader = std::make_unique( path, task->data_part, task->pre_columns, owned_uncompressed_cache.get(), owned_mark_cache.get(), save_marks_in_cache, From ad6ae4fdbea33b5c56ed7674611c7794974c71b6 Mon Sep 17 00:00:00 2001 From: proller Date: Sat, 24 Nov 2018 04:15:00 +0300 Subject: [PATCH 098/124] Fix build on mac (caused by #3631) (#3654) * Fix build on mac (caused by #3631) * fix --- dbms/src/Common/tests/CMakeLists.txt | 2 +- dbms/src/Functions/CMakeLists.txt | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/dbms/src/Common/tests/CMakeLists.txt b/dbms/src/Common/tests/CMakeLists.txt index 802963bf404..05984c9d42e 100644 --- a/dbms/src/Common/tests/CMakeLists.txt +++ b/dbms/src/Common/tests/CMakeLists.txt @@ -1,5 +1,5 @@ add_executable (hashes_test hashes_test.cpp) -target_link_libraries (hashes_test PRIVATE dbms) +target_link_libraries (hashes_test PRIVATE dbms ${OPENSSL_CRYPTO_LIBRARY}) add_executable (sip_hash sip_hash.cpp) target_link_libraries (sip_hash PRIVATE clickhouse_common_io) diff --git a/dbms/src/Functions/CMakeLists.txt b/dbms/src/Functions/CMakeLists.txt index 95268580dda..47144516fb0 100644 --- a/dbms/src/Functions/CMakeLists.txt +++ b/dbms/src/Functions/CMakeLists.txt @@ -52,7 +52,7 @@ list(REMOVE_ITEM clickhouse_functions_headers IFunction.h FunctionFactory.h Func add_library(clickhouse_functions ${LINK_MODE} ${clickhouse_functions_sources}) -target_link_libraries(clickhouse_functions PUBLIC dbms PRIVATE ${CONSISTENT_HASHING_LIBRARY} consistent-hashing-sumbur ${FARMHASH_LIBRARIES} ${METROHASH_LIBRARIES} murmurhash ${BASE64_LIBRARY}) +target_link_libraries(clickhouse_functions PUBLIC dbms PRIVATE ${CONSISTENT_HASHING_LIBRARY} consistent-hashing-sumbur ${FARMHASH_LIBRARIES} ${METROHASH_LIBRARIES} murmurhash ${BASE64_LIBRARY} ${OPENSSL_CRYPTO_LIBRARY}) target_include_directories (clickhouse_functions SYSTEM BEFORE PUBLIC ${DIVIDE_INCLUDE_DIR}) From bbed8a8ca87efa97997bce53b5ba623c9688c98e Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 24 Nov 2018 04:20:27 +0300 Subject: [PATCH 099/124] Fixed bad code [#CLICKHOUSE-3] --- dbms/src/Functions/GatherUtils/Algorithms.h | 6 +++--- dbms/src/Functions/arrayDistinct.cpp | 6 +++--- dbms/src/Functions/arrayReverse.cpp | 14 +++++++------- 3 files changed, 13 insertions(+), 13 deletions(-) diff --git a/dbms/src/Functions/GatherUtils/Algorithms.h b/dbms/src/Functions/GatherUtils/Algorithms.h index 67cfa132e74..7ae4ef703a2 100644 --- a/dbms/src/Functions/GatherUtils/Algorithms.h +++ b/dbms/src/Functions/GatherUtils/Algorithms.h @@ -294,7 +294,7 @@ void NO_INLINE sliceDynamicOffsetUnbounded(Source && src, Sink && sink, const IC { const bool is_null = offset_column.onlyNull(); const auto * nullable = typeid_cast(&offset_column); - const ColumnUInt8::Container * null_map = nullable ? &nullable->getNullMapColumn().getData() : nullptr; + const ColumnUInt8::Container * null_map = nullable ? &nullable->getNullMapData() : nullptr; const IColumn * nested_column = nullable ? &nullable->getNestedColumn() : &offset_column; while (!src.isEnd()) @@ -325,12 +325,12 @@ void NO_INLINE sliceDynamicOffsetBounded(Source && src, Sink && sink, const ICol { const bool is_offset_null = offset_column.onlyNull(); const auto * offset_nullable = typeid_cast(&offset_column); - const ColumnUInt8::Container * offset_null_map = offset_nullable ? &offset_nullable->getNullMapColumn().getData() : nullptr; + const ColumnUInt8::Container * offset_null_map = offset_nullable ? &offset_nullable->getNullMapData() : nullptr; const IColumn * offset_nested_column = offset_nullable ? &offset_nullable->getNestedColumn() : &offset_column; const bool is_length_null = length_column.onlyNull(); const auto * length_nullable = typeid_cast(&length_column); - const ColumnUInt8::Container * length_null_map = length_nullable ? &length_nullable->getNullMapColumn().getData() : nullptr; + const ColumnUInt8::Container * length_null_map = length_nullable ? &length_nullable->getNullMapData() : nullptr; const IColumn * length_nested_column = length_nullable ? &length_nullable->getNestedColumn() : &length_column; while (!src.isEnd()) diff --git a/dbms/src/Functions/arrayDistinct.cpp b/dbms/src/Functions/arrayDistinct.cpp index 4a0f6fe1d6b..18dd7be850a 100644 --- a/dbms/src/Functions/arrayDistinct.cpp +++ b/dbms/src/Functions/arrayDistinct.cpp @@ -151,7 +151,7 @@ bool FunctionArrayDistinct::executeNumber( const PaddedPODArray * src_null_map = nullptr; if (nullable_col) - src_null_map = &static_cast(&nullable_col->getNullMapColumn())->getData(); + src_null_map = &nullable_col->getNullMapData(); using Set = ClearableHashSet, @@ -210,7 +210,7 @@ bool FunctionArrayDistinct::executeString( const PaddedPODArray * src_null_map = nullptr; if (nullable_col) - src_null_map = &static_cast(&nullable_col->getNullMapColumn())->getData(); + src_null_map = &nullable_col->getNullMapData(); Set set; @@ -257,7 +257,7 @@ void FunctionArrayDistinct::executeHashed( const PaddedPODArray * src_null_map = nullptr; if (nullable_col) - src_null_map = &static_cast(&nullable_col->getNullMapColumn())->getData(); + src_null_map = &nullable_col->getNullMapData(); Set set; diff --git a/dbms/src/Functions/arrayReverse.cpp b/dbms/src/Functions/arrayReverse.cpp index 1d4ca49d61a..cc823669ed4 100644 --- a/dbms/src/Functions/arrayReverse.cpp +++ b/dbms/src/Functions/arrayReverse.cpp @@ -190,8 +190,8 @@ bool FunctionArrayReverse::executeNumber( if ((nullable_col) && (nullable_res_col)) { /// Make a reverted null map. - const auto & src_null_map = static_cast(nullable_col->getNullMapColumn()).getData(); - auto & res_null_map = static_cast(nullable_res_col->getNullMapColumn()).getData(); + const auto & src_null_map = nullable_col->getNullMapData(); + auto & res_null_map = nullable_res_col->getNullMapData(); res_null_map.resize(src_data.size()); do_reverse(src_null_map, src_offsets, res_null_map); } @@ -242,8 +242,8 @@ bool FunctionArrayReverse::executeFixedString( if ((nullable_col) && (nullable_res_col)) { /// Make a reverted null map. - const auto & src_null_map = static_cast(nullable_col->getNullMapColumn()).getData(); - auto & res_null_map = static_cast(nullable_res_col->getNullMapColumn()).getData(); + const auto & src_null_map = nullable_col->getNullMapData(); + auto & res_null_map = nullable_res_col->getNullMapData(); res_null_map.resize(src_null_map.size()); ColumnArray::Offset src_prev_offset = 0; @@ -319,11 +319,11 @@ bool FunctionArrayReverse::executeString( src_array_prev_offset = src_array_offsets[i]; } - if ((nullable_col) && (nullable_res_col)) + if (nullable_col && nullable_res_col) { /// Make a reverted null map. - const auto & src_null_map = static_cast(nullable_col->getNullMapColumn()).getData(); - auto & res_null_map = static_cast(nullable_res_col->getNullMapColumn()).getData(); + const auto & src_null_map = nullable_col->getNullMapData(); + auto & res_null_map = nullable_res_col->getNullMapData(); res_null_map.resize(src_string_offsets.size()); size_t size = src_string_offsets.size(); From 7d6fffff488205b55f184ea28bf22d145964ebd1 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 24 Nov 2018 04:48:06 +0300 Subject: [PATCH 100/124] More style checks [#CLICKHOUSE-3] --- dbms/programs/copier/ClusterCopier.cpp | 3 +- dbms/programs/server/Server.cpp | 8 ++--- dbms/src/Client/Connection.cpp | 2 +- dbms/src/Columns/ColumnDecimal.cpp | 2 +- dbms/src/Common/Config/ConfigProcessor.cpp | 8 ++--- dbms/src/Common/FieldVisitors.h | 2 +- dbms/src/Common/RWLockFIFO.cpp | 2 +- ...regatedMemoryEfficientBlockInputStream.cpp | 2 +- .../src/DataStreams/ParallelInputsProcessor.h | 2 +- dbms/src/DataTypes/DataTypeNullable.cpp | 10 +++--- dbms/src/DataTypes/DataTypesDecimal.cpp | 2 +- dbms/src/DataTypes/DataTypesDecimal.h | 2 +- dbms/src/Formats/XMLRowOutputStream.cpp | 2 +- dbms/src/Functions/FunctionBase64Conversion.h | 4 +-- dbms/src/Functions/FunctionsComparison.h | 2 +- dbms/src/Functions/FunctionsConditional.h | 2 +- dbms/src/Functions/FunctionsConversion.h | 2 +- .../Functions/FunctionsExternalDictionaries.h | 4 +-- dbms/src/Functions/FunctionsFormatting.h | 36 +++++++++---------- dbms/src/Functions/FunctionsLogical.h | 2 +- dbms/src/Functions/FunctionsStringArray.h | 2 +- dbms/src/Functions/arrayElement.cpp | 4 +-- dbms/src/Functions/arrayEnumerateExtended.h | 2 +- dbms/src/Functions/arrayIndex.h | 2 +- dbms/src/Functions/arrayReverse.cpp | 2 +- dbms/src/Functions/arrayUniq.cpp | 2 +- dbms/src/Functions/emptyArrayToSingle.cpp | 2 +- .../gtest_cascade_and_memory_write_buffer.cpp | 24 +++++-------- dbms/src/IO/tests/parse_int_perf.cpp | 2 +- dbms/src/Interpreters/ActionLocksManager.cpp | 4 +-- dbms/src/Interpreters/DDLWorker.cpp | 2 +- .../Interpreters/InterpreterExplainQuery.h | 2 +- .../LogicalExpressionsOptimizer.cpp | 4 +-- dbms/src/Interpreters/QueryAliasesVisitor.h | 4 +-- dbms/src/Storages/AlterCommands.cpp | 2 +- .../MergeTree/MergeTreeDataMergerMutator.cpp | 2 +- .../MergeTree/ReplicatedMergeTreeQueue.cpp | 2 +- dbms/src/Storages/StorageMySQL.cpp | 2 +- .../Storages/System/StorageSystemTables.cpp | 2 +- utils/check-style/check-style | 6 ++-- 40 files changed, 83 insertions(+), 90 deletions(-) diff --git a/dbms/programs/copier/ClusterCopier.cpp b/dbms/programs/copier/ClusterCopier.cpp index 06f826d58d5..edbe49e36be 100644 --- a/dbms/programs/copier/ClusterCopier.cpp +++ b/dbms/programs/copier/ClusterCopier.cpp @@ -410,8 +410,7 @@ BlockInputStreamPtr squashStreamIntoOneBlock(const BlockInputStreamPtr & stream) return std::make_shared( stream, std::numeric_limits::max(), - std::numeric_limits::max() - ); + std::numeric_limits::max()); } Block getBlockWithAllStreamData(const BlockInputStreamPtr & stream) diff --git a/dbms/programs/server/Server.cpp b/dbms/programs/server/Server.cpp index b80a9c7073c..832f02d16da 100644 --- a/dbms/programs/server/Server.cpp +++ b/dbms/programs/server/Server.cpp @@ -559,10 +559,10 @@ int Server::main(const std::vector & /*args*/) socket.setReceiveTimeout(settings.receive_timeout); socket.setSendTimeout(settings.send_timeout); servers.emplace_back(new Poco::Net::TCPServer( - new TCPHandlerFactory(*this, /* secure= */ true ), - server_pool, - socket, - new Poco::Net::TCPServerParams)); + new TCPHandlerFactory(*this, /* secure= */ true), + server_pool, + socket, + new Poco::Net::TCPServerParams)); LOG_INFO(log, "Listening tcp_secure: " + address.toString()); #else throw Exception{"SSL support for TCP protocol is disabled because Poco library was built without NetSSL support.", diff --git a/dbms/src/Client/Connection.cpp b/dbms/src/Client/Connection.cpp index 07d2ca05f21..8915504d133 100644 --- a/dbms/src/Client/Connection.cpp +++ b/dbms/src/Client/Connection.cpp @@ -56,7 +56,7 @@ void Connection::connect() disconnect(); LOG_TRACE(log_wrapper.get(), "Connecting. Database: " << (default_database.empty() ? "(not specified)" : default_database) << ". User: " << user - << (static_cast(secure) ? ". Secure" : "") << (static_cast(compression) ? "" : ". Uncompressed") ); + << (static_cast(secure) ? ". Secure" : "") << (static_cast(compression) ? "" : ". Uncompressed")); if (static_cast(secure)) { diff --git a/dbms/src/Columns/ColumnDecimal.cpp b/dbms/src/Columns/ColumnDecimal.cpp index e2cb798360e..e6124f6f6d3 100644 --- a/dbms/src/Columns/ColumnDecimal.cpp +++ b/dbms/src/Columns/ColumnDecimal.cpp @@ -21,7 +21,7 @@ namespace ErrorCodes } template -int ColumnDecimal::compareAt(size_t n, size_t m, const IColumn & rhs_, int ) const +int ColumnDecimal::compareAt(size_t n, size_t m, const IColumn & rhs_, int) const { auto other = static_cast(rhs_); const T & a = data[n]; diff --git a/dbms/src/Common/Config/ConfigProcessor.cpp b/dbms/src/Common/Config/ConfigProcessor.cpp index eebf9b75a35..cc7660f9641 100644 --- a/dbms/src/Common/Config/ConfigProcessor.cpp +++ b/dbms/src/Common/Config/ConfigProcessor.cpp @@ -260,10 +260,10 @@ void ConfigProcessor::doIncludesRecursive( /// Substitute for the number extracted from the hostname only if there is an /// empty tag without attributes in the original file. - if ( node->nodeName() == "layer" && - !node->hasAttributes() && - !node->hasChildNodes() && - node->nodeValue().empty()) + if (node->nodeName() == "layer" + && !node->hasAttributes() + && !node->hasChildNodes() + && node->nodeValue().empty()) { NodePtr new_node = config->createTextNode(layerFromHost()); node->appendChild(new_node); diff --git a/dbms/src/Common/FieldVisitors.h b/dbms/src/Common/FieldVisitors.h index 6b5c0730f93..ecfd1add179 100644 --- a/dbms/src/Common/FieldVisitors.h +++ b/dbms/src/Common/FieldVisitors.h @@ -18,7 +18,7 @@ namespace ErrorCodes extern const int LOGICAL_ERROR; } -UInt128 stringToUUID(const String & ); +UInt128 stringToUUID(const String &); /** StaticVisitor (and its descendants) - class with overloaded operator() for all types of fields. diff --git a/dbms/src/Common/RWLockFIFO.cpp b/dbms/src/Common/RWLockFIFO.cpp index 2d293164f3f..d511685111d 100644 --- a/dbms/src/Common/RWLockFIFO.cpp +++ b/dbms/src/Common/RWLockFIFO.cpp @@ -134,7 +134,7 @@ RWLockFIFO::LockHandler RWLockFIFO::getLock(RWLockFIFO::Type type, RWLockFIFO::C } /// Wait a notification - it_group->cv.wait(lock, [&] () { return it_group == queue.begin(); } ); + it_group->cv.wait(lock, [&] () { return it_group == queue.begin(); }); it_client->start_time = time(nullptr); finalize_metrics(); diff --git a/dbms/src/DataStreams/MergingAggregatedMemoryEfficientBlockInputStream.cpp b/dbms/src/DataStreams/MergingAggregatedMemoryEfficientBlockInputStream.cpp index 4cdc2655933..d85992f8e5d 100644 --- a/dbms/src/DataStreams/MergingAggregatedMemoryEfficientBlockInputStream.cpp +++ b/dbms/src/DataStreams/MergingAggregatedMemoryEfficientBlockInputStream.cpp @@ -198,7 +198,7 @@ void MergingAggregatedMemoryEfficientBlockInputStream::start() */ for (size_t i = 0; i < merging_threads; ++i) - pool.schedule([this, thread_group=CurrentThread::getGroup()] () { mergeThread(thread_group); } ); + pool.schedule([this, thread_group=CurrentThread::getGroup()] () { mergeThread(thread_group); }); } } diff --git a/dbms/src/DataStreams/ParallelInputsProcessor.h b/dbms/src/DataStreams/ParallelInputsProcessor.h index 509522de29a..500561e0282 100644 --- a/dbms/src/DataStreams/ParallelInputsProcessor.h +++ b/dbms/src/DataStreams/ParallelInputsProcessor.h @@ -111,7 +111,7 @@ public: try { for (size_t i = 0; i < max_threads; ++i) - threads.emplace_back([=] () { thread(thread_group, i); } ); + threads.emplace_back([=] () { thread(thread_group, i); }); } catch (...) { diff --git a/dbms/src/DataTypes/DataTypeNullable.cpp b/dbms/src/DataTypes/DataTypeNullable.cpp index ebd10fe9735..6322f5b2b17 100644 --- a/dbms/src/DataTypes/DataTypeNullable.cpp +++ b/dbms/src/DataTypes/DataTypeNullable.cpp @@ -161,7 +161,7 @@ void DataTypeNullable::deserializeBinary(IColumn & column, ReadBuffer & istr) co { safeDeserialize(column, [&istr] { bool is_null = 0; readBinary(is_null, istr); return is_null; }, - [this, &istr] (IColumn & nested) { nested_data_type->deserializeBinary(nested, istr); } ); + [this, &istr] (IColumn & nested) { nested_data_type->deserializeBinary(nested, istr); }); } @@ -188,7 +188,7 @@ void DataTypeNullable::deserializeTextEscaped(IColumn & column, ReadBuffer & ist { safeDeserialize(column, [] { return false; }, - [this, &istr, &settings] (IColumn & nested) { nested_data_type->deserializeTextEscaped(nested, istr, settings); } ); + [this, &istr, &settings] (IColumn & nested) { nested_data_type->deserializeTextEscaped(nested, istr, settings); }); } else { @@ -248,7 +248,7 @@ void DataTypeNullable::deserializeTextQuoted(IColumn & column, ReadBuffer & istr { safeDeserialize(column, [&istr] { return checkStringByFirstCharacterAndAssertTheRestCaseInsensitive("NULL", istr); }, - [this, &istr, &settings] (IColumn & nested) { nested_data_type->deserializeTextQuoted(nested, istr, settings); } ); + [this, &istr, &settings] (IColumn & nested) { nested_data_type->deserializeTextQuoted(nested, istr, settings); }); } void DataTypeNullable::serializeTextCSV(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const @@ -265,7 +265,7 @@ void DataTypeNullable::deserializeTextCSV(IColumn & column, ReadBuffer & istr, c { safeDeserialize(column, [&istr] { return checkStringByFirstCharacterAndAssertTheRest("\\N", istr); }, - [this, &settings, &istr] (IColumn & nested) { nested_data_type->deserializeTextCSV(nested, istr, settings); } ); + [this, &settings, &istr] (IColumn & nested) { nested_data_type->deserializeTextCSV(nested, istr, settings); }); } void DataTypeNullable::serializeText(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const @@ -298,7 +298,7 @@ void DataTypeNullable::deserializeTextJSON(IColumn & column, ReadBuffer & istr, { safeDeserialize(column, [&istr] { return checkStringByFirstCharacterAndAssertTheRest("null", istr); }, - [this, &istr, &settings] (IColumn & nested) { nested_data_type->deserializeTextJSON(nested, istr, settings); } ); + [this, &istr, &settings] (IColumn & nested) { nested_data_type->deserializeTextJSON(nested, istr, settings); }); } void DataTypeNullable::serializeTextXML(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const diff --git a/dbms/src/DataTypes/DataTypesDecimal.cpp b/dbms/src/DataTypes/DataTypesDecimal.cpp index 9f81107eb68..0c5bd6c6559 100644 --- a/dbms/src/DataTypes/DataTypesDecimal.cpp +++ b/dbms/src/DataTypes/DataTypesDecimal.cpp @@ -123,7 +123,7 @@ void DataTypeDecimal::deserializeBinary(IColumn & column, ReadBuffer & istr) } template -void DataTypeDecimal::deserializeBinaryBulk(IColumn & column, ReadBuffer & istr, size_t limit, double ) const +void DataTypeDecimal::deserializeBinaryBulk(IColumn & column, ReadBuffer & istr, size_t limit, double) const { typename ColumnType::Container & x = typeid_cast(column).getData(); size_t initial_size = x.size(); diff --git a/dbms/src/DataTypes/DataTypesDecimal.h b/dbms/src/DataTypes/DataTypesDecimal.h index 863ba7bcae1..96319dccfa4 100644 --- a/dbms/src/DataTypes/DataTypesDecimal.h +++ b/dbms/src/DataTypes/DataTypesDecimal.h @@ -152,7 +152,7 @@ public: /// @returns multiplier for U to become T with correct scale template - T scaleFactorFor(const DataTypeDecimal & x, bool ) const + T scaleFactorFor(const DataTypeDecimal & x, bool) const { if (getScale() < x.getScale()) throw Exception("Decimal result's scale is less then argiment's one", ErrorCodes::ARGUMENT_OUT_OF_BOUND); diff --git a/dbms/src/Formats/XMLRowOutputStream.cpp b/dbms/src/Formats/XMLRowOutputStream.cpp index f06fc9cc89d..7bd5a5d4c84 100644 --- a/dbms/src/Formats/XMLRowOutputStream.cpp +++ b/dbms/src/Formats/XMLRowOutputStream.cpp @@ -29,7 +29,7 @@ XMLRowOutputStream::XMLRowOutputStream(WriteBuffer & ostr_, const Block & sample for (const char * pos = begin; pos != end; ++pos) { char c = *pos; - if (!( isAlphaASCII(c) + if (!(isAlphaASCII(c) || (pos != begin && isNumericASCII(c)) || c == '_' || c == '-' diff --git a/dbms/src/Functions/FunctionBase64Conversion.h b/dbms/src/Functions/FunctionBase64Conversion.h index 3afa65c8aca..5ee6ff8f257 100644 --- a/dbms/src/Functions/FunctionBase64Conversion.h +++ b/dbms/src/Functions/FunctionBase64Conversion.h @@ -27,7 +27,7 @@ struct Base64Encode static constexpr auto name = "base64Encode"; static size_t getBufferSize(size_t string_length, size_t string_count) { - return ( ( string_length - string_count ) / 3 + string_count ) * 4 + string_count ; + return ((string_length - string_count) / 3 + string_count) * 4 + string_count; } }; @@ -37,7 +37,7 @@ struct Base64Decode static size_t getBufferSize(size_t string_length, size_t string_count) { - return ( ( string_length - string_count) / 4 + string_count) * 3 + string_count; + return ((string_length - string_count) / 4 + string_count) * 3 + string_count; } }; diff --git a/dbms/src/Functions/FunctionsComparison.h b/dbms/src/Functions/FunctionsComparison.h index d785cf3071c..358d52cc58e 100644 --- a/dbms/src/Functions/FunctionsComparison.h +++ b/dbms/src/Functions/FunctionsComparison.h @@ -1148,7 +1148,7 @@ public: if (left_is_num && right_is_num) { - if (!( executeNumLeftType(block, result, col_left_untyped, col_right_untyped) + if (!(executeNumLeftType(block, result, col_left_untyped, col_right_untyped) || executeNumLeftType(block, result, col_left_untyped, col_right_untyped) || executeNumLeftType(block, result, col_left_untyped, col_right_untyped) || executeNumLeftType(block, result, col_left_untyped, col_right_untyped) diff --git a/dbms/src/Functions/FunctionsConditional.h b/dbms/src/Functions/FunctionsConditional.h index a469a6bb7b2..11364e6ed12 100644 --- a/dbms/src/Functions/FunctionsConditional.h +++ b/dbms/src/Functions/FunctionsConditional.h @@ -982,7 +982,7 @@ public: bool executed_with_nums = callOnBasicTypes(left_id, right_id, call); - if (!( executed_with_nums + if (!(executed_with_nums || executeTyped(cond_col, block, arguments, result, input_rows_count) || executeString(cond_col, block, arguments, result) || executeGenericArray(cond_col, block, arguments, result) diff --git a/dbms/src/Functions/FunctionsConversion.h b/dbms/src/Functions/FunctionsConversion.h index f9802cd5178..9fbb1828827 100644 --- a/dbms/src/Functions/FunctionsConversion.h +++ b/dbms/src/Functions/FunctionsConversion.h @@ -793,7 +793,7 @@ public: return createDecimal(9, scale); else if constexpr (std::is_same_v) return createDecimal(18, scale); - else if constexpr ( std::is_same_v) + else if constexpr (std::is_same_v) return createDecimal(38, scale); throw Exception("Someting wrong with toDecimalNN()", ErrorCodes::LOGICAL_ERROR); diff --git a/dbms/src/Functions/FunctionsExternalDictionaries.h b/dbms/src/Functions/FunctionsExternalDictionaries.h index 86f463ac0b1..b99ec11bf6b 100644 --- a/dbms/src/Functions/FunctionsExternalDictionaries.h +++ b/dbms/src/Functions/FunctionsExternalDictionaries.h @@ -713,7 +713,7 @@ private: throw Exception{"Illegal type " + arguments[2]->getName() + " of third argument of function " + getName() + ", must be UInt64 or tuple(...).", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT}; - if (arguments.size() == 4 ) + if (arguments.size() == 4) { const auto range_argument = arguments[3].get(); if (!(range_argument->isValueRepresentedByInteger() && @@ -1215,7 +1215,7 @@ private: throw Exception{"Illegal type " + arguments[2].type->getName() + " of third argument of function " + getName() + ", must be UInt64 or tuple(...).", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT}; - if (arguments.size() == 4 ) + if (arguments.size() == 4) { const auto range_argument = arguments[3].type.get(); if (!(range_argument->isValueRepresentedByInteger() && diff --git a/dbms/src/Functions/FunctionsFormatting.h b/dbms/src/Functions/FunctionsFormatting.h index d474078ac55..8fca3ee890e 100644 --- a/dbms/src/Functions/FunctionsFormatting.h +++ b/dbms/src/Functions/FunctionsFormatting.h @@ -56,14 +56,14 @@ public: void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) override { - if (!( executeType(block, arguments, result) - || executeType(block, arguments, result) - || executeType(block, arguments, result) - || executeType(block, arguments, result) - || executeType(block, arguments, result) - || executeType(block, arguments, result) - || executeType(block, arguments, result) - || executeType(block, arguments, result))) + if (!(executeType(block, arguments, result) + || executeType(block, arguments, result) + || executeType(block, arguments, result) + || executeType(block, arguments, result) + || executeType(block, arguments, result) + || executeType(block, arguments, result) + || executeType(block, arguments, result) + || executeType(block, arguments, result))) throw Exception("Illegal column " + block.getByPosition(arguments[0]).column->getName() + " of argument of function " + getName(), ErrorCodes::ILLEGAL_COLUMN); @@ -149,16 +149,16 @@ public: void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) override { - if (!( executeType(block, arguments, result) - || executeType(block, arguments, result) - || executeType(block, arguments, result) - || executeType(block, arguments, result) - || executeType(block, arguments, result) - || executeType(block, arguments, result) - || executeType(block, arguments, result) - || executeType(block, arguments, result) - || executeType(block, arguments, result) - || executeType(block, arguments, result))) + if (!(executeType(block, arguments, result) + || executeType(block, arguments, result) + || executeType(block, arguments, result) + || executeType(block, arguments, result) + || executeType(block, arguments, result) + || executeType(block, arguments, result) + || executeType(block, arguments, result) + || executeType(block, arguments, result) + || executeType(block, arguments, result) + || executeType(block, arguments, result))) throw Exception("Illegal column " + block.getByPosition(arguments[0]).column->getName() + " of argument of function " + getName(), ErrorCodes::ILLEGAL_COLUMN); diff --git a/dbms/src/Functions/FunctionsLogical.h b/dbms/src/Functions/FunctionsLogical.h index 6f2bb62f9cf..cf34b9bf669 100644 --- a/dbms/src/Functions/FunctionsLogical.h +++ b/dbms/src/Functions/FunctionsLogical.h @@ -501,7 +501,7 @@ public: void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) override { - if (!( executeType(block, arguments, result) + if (!(executeType(block, arguments, result) || executeType(block, arguments, result) || executeType(block, arguments, result) || executeType(block, arguments, result) diff --git a/dbms/src/Functions/FunctionsStringArray.h b/dbms/src/Functions/FunctionsStringArray.h index 75b27751fca..82c022d5e2f 100644 --- a/dbms/src/Functions/FunctionsStringArray.h +++ b/dbms/src/Functions/FunctionsStringArray.h @@ -265,7 +265,7 @@ public: static size_t getNumberOfArguments() { return 2; } /// Check the type of function arguments. - static void checkArguments( const DataTypes & arguments ) + static void checkArguments(const DataTypes & arguments) { SplitByStringImpl::checkArguments(arguments); } diff --git a/dbms/src/Functions/arrayElement.cpp b/dbms/src/Functions/arrayElement.cpp index c6ba66cba11..b887e163f2a 100644 --- a/dbms/src/Functions/arrayElement.cpp +++ b/dbms/src/Functions/arrayElement.cpp @@ -647,7 +647,7 @@ bool FunctionArrayElement::executeArgument(Block & block, const ColumnNumbers & if (builder) builder.initSink(index_data.size()); - if (!( executeNumber(block, arguments, result, index_data, builder) + if (!(executeNumber(block, arguments, result, index_data, builder) || executeNumber(block, arguments, result, index_data, builder) || executeNumber(block, arguments, result, index_data, builder) || executeNumber(block, arguments, result, index_data, builder) @@ -859,7 +859,7 @@ void FunctionArrayElement::perform(Block & block, const ColumnNumbers & argument if (index == 0u) throw Exception("Array indices is 1-based", ErrorCodes::ZERO_ARRAY_OR_TUPLE_INDEX); - if (!( executeNumberConst(block, arguments, result, index, builder) + if (!(executeNumberConst(block, arguments, result, index, builder) || executeNumberConst(block, arguments, result, index, builder) || executeNumberConst(block, arguments, result, index, builder) || executeNumberConst(block, arguments, result, index, builder) diff --git a/dbms/src/Functions/arrayEnumerateExtended.h b/dbms/src/Functions/arrayEnumerateExtended.h index a88231d63ec..86eba2ded7c 100644 --- a/dbms/src/Functions/arrayEnumerateExtended.h +++ b/dbms/src/Functions/arrayEnumerateExtended.h @@ -144,7 +144,7 @@ void FunctionArrayEnumerateExtended::executeImpl(Block & block, const C if (num_columns == 1) { - if (!( executeNumber(first_array, first_null_map, res_values) + if (!(executeNumber(first_array, first_null_map, res_values) || executeNumber(first_array, first_null_map, res_values) || executeNumber(first_array, first_null_map, res_values) || executeNumber(first_array, first_null_map, res_values) diff --git a/dbms/src/Functions/arrayIndex.h b/dbms/src/Functions/arrayIndex.h index c4861184699..f3aa4b47022 100644 --- a/dbms/src/Functions/arrayIndex.h +++ b/dbms/src/Functions/arrayIndex.h @@ -53,7 +53,7 @@ private: #pragma GCC diagnostic ignored "-Wsign-compare" /// compares `lhs` against `i`-th element of `rhs` - static bool compare(const T & lhs, const PaddedPODArray & rhs, const size_t i ) { return lhs == rhs[i]; } + static bool compare(const T & lhs, const PaddedPODArray & rhs, const size_t i) { return lhs == rhs[i]; } /// compares `lhs against `rhs`, third argument unused static bool compare(const T & lhs, const U & rhs, size_t) { return lhs == rhs; } diff --git a/dbms/src/Functions/arrayReverse.cpp b/dbms/src/Functions/arrayReverse.cpp index cc823669ed4..cedf4dfc7f3 100644 --- a/dbms/src/Functions/arrayReverse.cpp +++ b/dbms/src/Functions/arrayReverse.cpp @@ -107,7 +107,7 @@ void FunctionArrayReverse::executeImpl(Block & block, const ColumnNumbers & argu inner_res_col = &res_data; } - if (!( executeNumber(*inner_col, offsets, *inner_res_col, nullable_col, nullable_res_col) + if (!(executeNumber(*inner_col, offsets, *inner_res_col, nullable_col, nullable_res_col) || executeNumber(*inner_col, offsets, *inner_res_col, nullable_col, nullable_res_col) || executeNumber(*inner_col, offsets, *inner_res_col, nullable_col, nullable_res_col) || executeNumber(*inner_col, offsets, *inner_res_col, nullable_col, nullable_res_col) diff --git a/dbms/src/Functions/arrayUniq.cpp b/dbms/src/Functions/arrayUniq.cpp index 4c567c3a9d1..9dbf0e39886 100644 --- a/dbms/src/Functions/arrayUniq.cpp +++ b/dbms/src/Functions/arrayUniq.cpp @@ -139,7 +139,7 @@ void FunctionArrayUniq::executeImpl(Block & block, const ColumnNumbers & argumen if (arguments.size() == 1) { - if (!( executeNumber(first_array, first_null_map, res_values) + if (!(executeNumber(first_array, first_null_map, res_values) || executeNumber(first_array, first_null_map, res_values) || executeNumber(first_array, first_null_map, res_values) || executeNumber(first_array, first_null_map, res_values) diff --git a/dbms/src/Functions/emptyArrayToSingle.cpp b/dbms/src/Functions/emptyArrayToSingle.cpp index 4020ab070a5..8ca18049c87 100644 --- a/dbms/src/Functions/emptyArrayToSingle.cpp +++ b/dbms/src/Functions/emptyArrayToSingle.cpp @@ -350,7 +350,7 @@ namespace const NullMap * src_null_map, NullMap * res_null_map) { - if (!( executeNumber (src_data, src_array_offsets, res_data_col, res_array_offsets, src_null_map, res_null_map) + if (!(executeNumber (src_data, src_array_offsets, res_data_col, res_array_offsets, src_null_map, res_null_map) || executeNumber (src_data, src_array_offsets, res_data_col, res_array_offsets, src_null_map, res_null_map) || executeNumber (src_data, src_array_offsets, res_data_col, res_array_offsets, src_null_map, res_null_map) || executeNumber (src_data, src_array_offsets, res_data_col, res_array_offsets, src_null_map, res_null_map) diff --git a/dbms/src/IO/tests/gtest_cascade_and_memory_write_buffer.cpp b/dbms/src/IO/tests/gtest_cascade_and_memory_write_buffer.cpp index f0f40e25c01..603c7f3f92c 100644 --- a/dbms/src/IO/tests/gtest_cascade_and_memory_write_buffer.cpp +++ b/dbms/src/IO/tests/gtest_cascade_and_memory_write_buffer.cpp @@ -78,30 +78,26 @@ try std::make_shared(s/2, 1, 2.0), std::make_shared(s - s/2, 1, 2.0) }, - {} - ); + {}); testCascadeBufferRedability(makeTestArray(s), { std::make_shared(s, 2, 1.5), }, - {} - ); + {}); testCascadeBufferRedability(makeTestArray(s), { std::make_shared(0, 1, 1.0), }, - {} - ); + {}); testCascadeBufferRedability(makeTestArray(s), { std::make_shared(std::max(1ul, s/2), std::max(2ul, s/4), 0.5), std::make_shared(0, 4, 1.0), }, - {} - ); + {}); testCascadeBufferRedability(makeTestArray(max_s), { @@ -109,16 +105,14 @@ try }, { [=] (auto) { return std::make_shared(max_s - s, 1, 2.0); } - } - ); + }); testCascadeBufferRedability(makeTestArray(max_s), {}, { [=] (auto) { return std::make_shared(max_s - s, 1, 2.0); }, [=] (auto) { return std::make_shared(s, 1, 2.0); } - } - ); + }); } } catch (...) @@ -261,8 +255,7 @@ try {}, { [=] (auto) { return WriteBufferFromTemporaryFile::create(tmp_template); } - } - ); + }); testCascadeBufferRedability(makeTestArray(s), { @@ -270,8 +263,7 @@ try }, { [=] (auto) { return WriteBufferFromTemporaryFile::create(tmp_template); } - } - ); + }); } } catch (...) diff --git a/dbms/src/IO/tests/parse_int_perf.cpp b/dbms/src/IO/tests/parse_int_perf.cpp index 71cba3152c5..353d29c204f 100644 --- a/dbms/src/IO/tests/parse_int_perf.cpp +++ b/dbms/src/IO/tests/parse_int_perf.cpp @@ -19,7 +19,7 @@ UInt64 rdtsc() { #if __x86_64__ UInt64 val; - __asm__ __volatile__("rdtsc" : "=A" (val) : ); + __asm__ __volatile__("rdtsc" : "=A" (val) :); return val; #else // TODO: make for arm diff --git a/dbms/src/Interpreters/ActionLocksManager.cpp b/dbms/src/Interpreters/ActionLocksManager.cpp index ddcdd3e6a4d..ee64eb1e0e9 100644 --- a/dbms/src/Interpreters/ActionLocksManager.cpp +++ b/dbms/src/Interpreters/ActionLocksManager.cpp @@ -76,10 +76,10 @@ void ActionLocksManager::cleanExpired() { std::lock_guard lock(mutex); - for (auto it_storage = storage_locks.begin(); it_storage != storage_locks.end(); ) + for (auto it_storage = storage_locks.begin(); it_storage != storage_locks.end();) { auto & locks = it_storage->second; - for (auto it_lock = locks.begin(); it_lock != locks.end(); ) + for (auto it_lock = locks.begin(); it_lock != locks.end();) { if (it_lock->second.expired()) it_lock = locks.erase(it_lock); diff --git a/dbms/src/Interpreters/DDLWorker.cpp b/dbms/src/Interpreters/DDLWorker.cpp index 0b42d7cbace..60183cd0c3a 100644 --- a/dbms/src/Interpreters/DDLWorker.cpp +++ b/dbms/src/Interpreters/DDLWorker.cpp @@ -1200,7 +1200,7 @@ BlockIO executeDDLQueryOnCluster(const ASTPtr & query_ptr_, const Context & cont 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 ); + databases_to_access.emplace(has_shard_default_db ? addr.default_database : current_database); } else databases_to_access.emplace(database); diff --git a/dbms/src/Interpreters/InterpreterExplainQuery.h b/dbms/src/Interpreters/InterpreterExplainQuery.h index 828e10ecd0b..4d366a9d56e 100644 --- a/dbms/src/Interpreters/InterpreterExplainQuery.h +++ b/dbms/src/Interpreters/InterpreterExplainQuery.h @@ -14,7 +14,7 @@ using ASTPtr = std::shared_ptr; class InterpreterExplainQuery : public IInterpreter { public: - InterpreterExplainQuery(const ASTPtr & query_, const Context & ) + InterpreterExplainQuery(const ASTPtr & query_, const Context &) : query(query_) {} diff --git a/dbms/src/Interpreters/LogicalExpressionsOptimizer.cpp b/dbms/src/Interpreters/LogicalExpressionsOptimizer.cpp index 42176788cbd..b2c2fcace64 100644 --- a/dbms/src/Interpreters/LogicalExpressionsOptimizer.cpp +++ b/dbms/src/Interpreters/LogicalExpressionsOptimizer.cpp @@ -367,8 +367,8 @@ void LogicalExpressionsOptimizer::fixBrokenOrExpressions() { // The order of children matters if or is children of some function, e.g. minus std::replace_if(parent->children.begin(), parent->children.end(), - [or_function](const ASTPtr & ptr) { return ptr.get() == or_function; }, - operands[0] ); + [or_function](const ASTPtr & ptr) { return ptr.get() == or_function; }, + operands[0]); } /// If the OR node was the root of the WHERE, PREWHERE, or HAVING expression, then update this root. diff --git a/dbms/src/Interpreters/QueryAliasesVisitor.h b/dbms/src/Interpreters/QueryAliasesVisitor.h index 18d507821d2..fd385e8b774 100644 --- a/dbms/src/Interpreters/QueryAliasesVisitor.h +++ b/dbms/src/Interpreters/QueryAliasesVisitor.h @@ -35,8 +35,8 @@ private: mutable size_t visit_depth; std::ostream * ostr; - void visit(const ASTTableExpression &, const ASTPtr & ) const {} - void visit(const ASTSelectWithUnionQuery &, const ASTPtr & ) const {} + void visit(const ASTTableExpression &, const ASTPtr &) const {} + void visit(const ASTSelectWithUnionQuery &, const ASTPtr &) const {} void visit(ASTSubquery & subquery, const ASTPtr & ast) const; void visit(const ASTArrayJoin &, const ASTPtr & ast) const; diff --git a/dbms/src/Storages/AlterCommands.cpp b/dbms/src/Storages/AlterCommands.cpp index abbc277e710..e976b07becb 100644 --- a/dbms/src/Storages/AlterCommands.cpp +++ b/dbms/src/Storages/AlterCommands.cpp @@ -223,7 +223,7 @@ void AlterCommand::apply(ColumnsDescription & columns_description) const const auto find_column = [this] (NamesAndTypesList & columns) { const auto it = std::find_if(columns.begin(), columns.end(), - std::bind(namesEqual, std::cref(column_name), std::placeholders::_1) ); + std::bind(namesEqual, std::cref(column_name), std::placeholders::_1)); if (it == columns.end()) throw Exception("Wrong column name. Cannot find column " + column_name + " to modify", ErrorCodes::ILLEGAL_COLUMN); diff --git a/dbms/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp b/dbms/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp index 2244fb28af2..e413cf25e70 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp @@ -990,7 +990,7 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mutatePartToTempor new_data_part->columns = all_columns; Names source_column_names = source_part->columns.getNames(); NameSet source_columns_name_set(source_column_names.begin(), source_column_names.end()); - for (auto it = new_data_part->columns.begin(); it != new_data_part->columns.end(); ) + for (auto it = new_data_part->columns.begin(); it != new_data_part->columns.end();) { if (source_columns_name_set.count(it->name) || in_header.has(it->name)) ++it; diff --git a/dbms/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp b/dbms/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp index 369b6bf9f1e..7fda70af4dc 100644 --- a/dbms/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp +++ b/dbms/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp @@ -554,7 +554,7 @@ void ReplicatedMergeTreeQueue::updateMutations(zkutil::ZooKeeperPtr zookeeper, C { std::lock_guard lock(state_mutex); - for (auto it = mutations_by_znode.begin(); it != mutations_by_znode.end(); ) + for (auto it = mutations_by_znode.begin(); it != mutations_by_znode.end();) { const ReplicatedMergeTreeMutationEntry & entry = *it->second.entry; if (!entries_in_zk_set.count(entry.znode_name)) diff --git a/dbms/src/Storages/StorageMySQL.cpp b/dbms/src/Storages/StorageMySQL.cpp index 0d25e82dcf3..bad1fed4a4c 100644 --- a/dbms/src/Storages/StorageMySQL.cpp +++ b/dbms/src/Storages/StorageMySQL.cpp @@ -112,7 +112,7 @@ public: WriteBufferFromOwnString sqlbuf; sqlbuf << (storage.replace_query ? "REPLACE" : "INSERT") << " INTO "; sqlbuf << backQuoteIfNeed(remote_database_name) << "." << backQuoteIfNeed(remote_table_name); - sqlbuf << " ( " << dumpNamesWithBackQuote(block) << " ) VALUES "; + sqlbuf << " (" << dumpNamesWithBackQuote(block) << ") VALUES "; auto writer = FormatFactory::instance().getOutput("Values", sqlbuf, storage.getSampleBlock(), storage.context); writer->write(block); diff --git a/dbms/src/Storages/System/StorageSystemTables.cpp b/dbms/src/Storages/System/StorageSystemTables.cpp index 1f2c24e1fb0..adbf1aeed06 100644 --- a/dbms/src/Storages/System/StorageSystemTables.cpp +++ b/dbms/src/Storages/System/StorageSystemTables.cpp @@ -54,7 +54,7 @@ static ColumnPtr getFilteredDatabases(const ASTPtr & query, const Context & cont for (const auto & db : context.getDatabases()) column->insert(db.first); - Block block { ColumnWithTypeAndName( std::move(column), std::make_shared(), "database" ) }; + Block block { ColumnWithTypeAndName(std::move(column), std::make_shared(), "database") }; VirtualColumnUtils::filterBlockWithQuery(query, block, context); return block.getByPosition(0).column; } diff --git a/utils/check-style/check-style b/utils/check-style/check-style index a79d24568d7..46dfbafe7e7 100755 --- a/utils/check-style/check-style +++ b/utils/check-style/check-style @@ -16,5 +16,7 @@ ROOT_PATH=$(git rev-parse --show-toplevel) find $ROOT_PATH/dbms -name '*.h' -or -name '*.cpp' | grep -vP 'Compiler|build' | - xargs grep $@ -P '((class|struct|namespace|enum|if|for|while|else|throw|switch).*|\)(\s*const)?(\s*override)?\s*)\{$|\s$|\t|^ {1,3}[^\* ]\S|\t|^\s*(if|else if|if constexpr|else if constexpr|for|while|catch|switch)\(' -# a curly brace not in a new line, but not for the case of C++11 init or agg. initialization | trailing whitespace | number of ws not a multiple of 4, but not in the case of comment continuation | a tab character | missing whitespace after for/if/while... before opening brace + xargs grep $@ -P '((class|struct|namespace|enum|if|for|while|else|throw|switch).*|\)(\s*const)?(\s*override)?\s*)\{$|\s$|\t|^ {1,3}[^\* ]\S|\t|^\s*(if|else if|if constexpr|else if constexpr|for|while|catch|switch)\(|\( [^\s\\]|\S \)' | +# a curly brace not in a new line, but not for the case of C++11 init or agg. initialization | trailing whitespace | number of ws not a multiple of 4, but not in the case of comment continuation | a tab character | missing whitespace after for/if/while... before opening brace | whitespaces inside braces + grep -v -P '(//|:\s+\*|\$\(\()| \)"' +# single-line comment | continuation of a multiline comment | a typical piece of embedded shell code | something like ending of raw string literal \ No newline at end of file From 73e3a7b662161d6005e7727d8a711b930386b871 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 24 Nov 2018 06:14:54 +0300 Subject: [PATCH 101/124] Fixed bad code; added generic implementation for "arrayReverse" function; added test [#CLICKHOUSE-3] --- dbms/src/Functions/arrayReverse.cpp | 242 +++++------------- .../0_stateless/00758_array_reverse.reference | 12 + .../0_stateless/00758_array_reverse.sql | 12 + 3 files changed, 89 insertions(+), 177 deletions(-) create mode 100644 dbms/tests/queries/0_stateless/00758_array_reverse.reference create mode 100644 dbms/tests/queries/0_stateless/00758_array_reverse.sql diff --git a/dbms/src/Functions/arrayReverse.cpp b/dbms/src/Functions/arrayReverse.cpp index cedf4dfc7f3..83f4cedbe49 100644 --- a/dbms/src/Functions/arrayReverse.cpp +++ b/dbms/src/Functions/arrayReverse.cpp @@ -16,11 +16,10 @@ namespace ErrorCodes { extern const int ILLEGAL_COLUMN; extern const int ILLEGAL_TYPE_OF_ARGUMENT; + extern const int LOGICAL_ERROR; } -/// TODO Add generic implementation. - class FunctionArrayReverse : public IFunction { public: @@ -42,38 +41,20 @@ public: return arguments[0]; } - void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override; + void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t) override; private: - bool executeConst(Block & block, const ColumnNumbers & arguments, size_t result, - size_t input_rows_count); - template - bool executeNumber( - const IColumn & src_data, const ColumnArray::Offsets & src_offsets, - IColumn & res_data_col, - const ColumnNullable * nullable_col, - ColumnNullable * nullable_res_col); + bool executeNumber(const IColumn & src_data, const ColumnArray::Offsets & src_offsets, IColumn & res_data); - bool executeFixedString( - const IColumn & src_data, const ColumnArray::Offsets & src_offsets, - IColumn & res_data_col, - const ColumnNullable * nullable_col, - ColumnNullable * nullable_res_col); - - bool executeString( - const IColumn & src_data, const ColumnArray::Offsets & src_array_offsets, - IColumn & res_data_col, - const ColumnNullable * nullable_col, - ColumnNullable * nullable_res_col); + bool executeFixedString(const IColumn & src_data, const ColumnArray::Offsets & src_offsets, IColumn & res_data); + bool executeString(const IColumn & src_data, const ColumnArray::Offsets & src_array_offsets, IColumn & res_data); + bool executeGeneric(const IColumn & src_data, const ColumnArray::Offsets & src_array_offsets, IColumn & res_data); }; -void FunctionArrayReverse::executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) +void FunctionArrayReverse::executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t) { - if (executeConst(block, arguments, result, input_rows_count)) - return; - const ColumnArray * array = checkAndGetColumn(block.getByPosition(arguments[0]).column.get()); if (!array) throw Exception("Illegal column " + block.getByPosition(arguments[0]).column->getName() + " of first argument of function " + getName(), @@ -85,89 +66,83 @@ void FunctionArrayReverse::executeImpl(Block & block, const ColumnNumbers & argu const IColumn & src_data = array->getData(); const ColumnArray::Offsets & offsets = array->getOffsets(); + IColumn & res_data = res.getData(); - const ColumnNullable * nullable_col = nullptr; - ColumnNullable * nullable_res_col = nullptr; + const ColumnNullable * src_nullable_col = typeid_cast(&src_data); + ColumnNullable * res_nullable_col = typeid_cast(&res_data); - const IColumn * inner_col; - IColumn * inner_res_col; + const IColumn * src_inner_col = src_nullable_col ? &src_nullable_col->getNestedColumn() : &src_data; + IColumn * res_inner_col = res_nullable_col ? &res_nullable_col->getNestedColumn() : &res_data; - if (src_data.isColumnNullable()) - { - nullable_col = static_cast(&src_data); - inner_col = &nullable_col->getNestedColumn(); + false + || executeNumber(*src_inner_col, offsets, *res_inner_col) + || executeNumber(*src_inner_col, offsets, *res_inner_col) + || executeNumber(*src_inner_col, offsets, *res_inner_col) + || executeNumber(*src_inner_col, offsets, *res_inner_col) + || executeNumber(*src_inner_col, offsets, *res_inner_col) + || executeNumber(*src_inner_col, offsets, *res_inner_col) + || executeNumber(*src_inner_col, offsets, *res_inner_col) + || executeNumber(*src_inner_col, offsets, *res_inner_col) + || executeNumber(*src_inner_col, offsets, *res_inner_col) + || executeNumber(*src_inner_col, offsets, *res_inner_col) + || executeString(*src_inner_col, offsets, *res_inner_col) + || executeFixedString(*src_inner_col, offsets, *res_inner_col) + || executeGeneric(*src_inner_col, offsets, *res_inner_col); - nullable_res_col = static_cast(&res_data); - inner_res_col = &nullable_res_col->getNestedColumn(); - } - else - { - inner_col = &src_data; - inner_res_col = &res_data; - } - - if (!(executeNumber(*inner_col, offsets, *inner_res_col, nullable_col, nullable_res_col) - || executeNumber(*inner_col, offsets, *inner_res_col, nullable_col, nullable_res_col) - || executeNumber(*inner_col, offsets, *inner_res_col, nullable_col, nullable_res_col) - || executeNumber(*inner_col, offsets, *inner_res_col, nullable_col, nullable_res_col) - || executeNumber(*inner_col, offsets, *inner_res_col, nullable_col, nullable_res_col) - || executeNumber(*inner_col, offsets, *inner_res_col, nullable_col, nullable_res_col) - || executeNumber(*inner_col, offsets, *inner_res_col, nullable_col, nullable_res_col) - || executeNumber(*inner_col, offsets, *inner_res_col, nullable_col, nullable_res_col) - || executeNumber(*inner_col, offsets, *inner_res_col, nullable_col, nullable_res_col) - || executeNumber(*inner_col, offsets, *inner_res_col, nullable_col, nullable_res_col) - || executeString(*inner_col, offsets, *inner_res_col, nullable_col, nullable_res_col) - || executeFixedString(*inner_col, offsets, *inner_res_col, nullable_col, nullable_res_col))) - throw Exception("Illegal column " + block.getByPosition(arguments[0]).column->getName() - + " of first argument of function " + getName(), - ErrorCodes::ILLEGAL_COLUMN); + if (src_nullable_col) + if (!executeNumber(src_nullable_col->getNullMapColumn(), offsets, res_nullable_col->getNullMapColumn())) + throw Exception("Illegal column " + src_nullable_col->getNullMapColumn().getName() + + " of null map of the first argument of function " + getName(), + ErrorCodes::ILLEGAL_COLUMN); block.getByPosition(result).column = std::move(res_ptr); } -bool FunctionArrayReverse::executeConst(Block & block, const ColumnNumbers & arguments, size_t result, - size_t input_rows_count) + +bool FunctionArrayReverse::executeGeneric(const IColumn & src_data, const ColumnArray::Offsets & src_offsets, IColumn & res_data) { - if (const ColumnConst * const_array = checkAndGetColumnConst(block.getByPosition(arguments[0]).column.get())) + size_t size = src_offsets.size(); + res_data.reserve(size); + + ColumnArray::Offset src_prev_offset = 0; + for (size_t i = 0; i < size; ++i) { - Array arr = const_array->getValue(); + ssize_t src_index = src_offsets[i] - 1; - size_t size = arr.size(); - Array res(size); + while (src_index >= ssize_t(src_prev_offset)) + { + res_data.insertFrom(src_data, src_index); + --src_index; + } - for (size_t i = 0; i < size; ++i) - res[i] = arr[size - i - 1]; - - block.getByPosition(result).column = block.getByPosition(result).type->createColumnConst(input_rows_count, res); - - return true; + src_prev_offset = src_offsets[i]; } - else - return false; + + return true; } template -bool FunctionArrayReverse::executeNumber( - const IColumn & src_data, const ColumnArray::Offsets & src_offsets, - IColumn & res_data_col, - const ColumnNullable * nullable_col, - ColumnNullable * nullable_res_col) +bool FunctionArrayReverse::executeNumber(const IColumn & src_data, const ColumnArray::Offsets & src_offsets, IColumn & res_data) { - auto do_reverse = [](const auto & src_data, const auto & src_offsets, auto & res_data) + if (const ColumnVector * src_data_concrete = checkAndGetColumn>(&src_data)) { + const PaddedPODArray & src_vec = src_data_concrete->getData(); + PaddedPODArray & res_vec = typeid_cast &>(res_data).getData(); + res_vec.resize(src_data.size()); + size_t size = src_offsets.size(); ColumnArray::Offset src_prev_offset = 0; for (size_t i = 0; i < size; ++i) { - const auto * src = &src_data[src_prev_offset]; - const auto * src_end = &src_data[src_offsets[i]]; + const auto * src = &src_vec[src_prev_offset]; + const auto * src_end = &src_vec[src_offsets[i]]; if (src == src_end) continue; - auto dst = &res_data[src_offsets[i] - 1]; + auto dst = &res_vec[src_offsets[i] - 1]; while (src < src_end) { @@ -178,23 +153,6 @@ bool FunctionArrayReverse::executeNumber( src_prev_offset = src_offsets[i]; } - }; - - if (const ColumnVector * src_data_concrete = checkAndGetColumn>(&src_data)) - { - const PaddedPODArray & src_data = src_data_concrete->getData(); - PaddedPODArray & res_data = typeid_cast &>(res_data_col).getData(); - res_data.resize(src_data.size()); - do_reverse(src_data, src_offsets, res_data); - - if ((nullable_col) && (nullable_res_col)) - { - /// Make a reverted null map. - const auto & src_null_map = nullable_col->getNullMapData(); - auto & res_null_map = nullable_res_col->getNullMapData(); - res_null_map.resize(src_data.size()); - do_reverse(src_null_map, src_offsets, res_null_map); - } return true; } @@ -202,19 +160,15 @@ bool FunctionArrayReverse::executeNumber( return false; } -bool FunctionArrayReverse::executeFixedString( - const IColumn & src_data, const ColumnArray::Offsets & src_offsets, - IColumn & res_data_col, - const ColumnNullable * nullable_col, - ColumnNullable * nullable_res_col) +bool FunctionArrayReverse::executeFixedString(const IColumn & src_data, const ColumnArray::Offsets & src_offsets, IColumn & res_data) { if (const ColumnFixedString * src_data_concrete = checkAndGetColumn(&src_data)) { const size_t n = src_data_concrete->getN(); const ColumnFixedString::Chars_t & src_data = src_data_concrete->getChars(); - ColumnFixedString::Chars_t & res_data = typeid_cast(res_data_col).getChars(); + ColumnFixedString::Chars_t & res_chars = typeid_cast(res_data).getChars(); size_t size = src_offsets.size(); - res_data.resize(src_data.size()); + res_chars.resize(src_data.size()); ColumnArray::Offset src_prev_offset = 0; @@ -226,7 +180,7 @@ bool FunctionArrayReverse::executeFixedString( if (src == src_end) continue; - UInt8 * dst = &res_data[src_offsets[i] * n - n]; + UInt8 * dst = &res_chars[src_offsets[i] * n - n]; while (src < src_end) { @@ -238,60 +192,25 @@ bool FunctionArrayReverse::executeFixedString( src_prev_offset = src_offsets[i]; } - - if ((nullable_col) && (nullable_res_col)) - { - /// Make a reverted null map. - const auto & src_null_map = nullable_col->getNullMapData(); - auto & res_null_map = nullable_res_col->getNullMapData(); - res_null_map.resize(src_null_map.size()); - - ColumnArray::Offset src_prev_offset = 0; - - for (size_t i = 0; i < size; ++i) - { - const UInt8 * src = &src_null_map[src_prev_offset]; - const UInt8 * src_end = &src_null_map[src_offsets[i]]; - - if (src == src_end) - continue; - - UInt8 * dst = &res_null_map[src_offsets[i] - 1]; - - while (src < src_end) - { - *dst = *src; - ++src; - --dst; - } - - src_prev_offset = src_offsets[i]; - } - } - return true; } else return false; } -bool FunctionArrayReverse::executeString( - const IColumn & src_data, const ColumnArray::Offsets & src_array_offsets, - IColumn & res_data_col, - const ColumnNullable * nullable_col, - ColumnNullable * nullable_res_col) +bool FunctionArrayReverse::executeString(const IColumn & src_data, const ColumnArray::Offsets & src_array_offsets, IColumn & res_data) { if (const ColumnString * src_data_concrete = checkAndGetColumn(&src_data)) { const ColumnString::Offsets & src_string_offsets = src_data_concrete->getOffsets(); - ColumnString::Offsets & res_string_offsets = typeid_cast(res_data_col).getOffsets(); + ColumnString::Offsets & res_string_offsets = typeid_cast(res_data).getOffsets(); const ColumnString::Chars_t & src_data = src_data_concrete->getChars(); - ColumnString::Chars_t & res_data = typeid_cast(res_data_col).getChars(); + ColumnString::Chars_t & res_chars = typeid_cast(res_data).getChars(); size_t size = src_array_offsets.size(); res_string_offsets.resize(src_string_offsets.size()); - res_data.resize(src_data.size()); + res_chars.resize(src_data.size()); ColumnArray::Offset src_array_prev_offset = 0; ColumnString::Offset res_string_prev_offset = 0; @@ -309,7 +228,7 @@ bool FunctionArrayReverse::executeString( auto src_pos = src_array_prev_offset + j_reversed == 0 ? 0 : src_string_offsets[src_array_prev_offset + j_reversed - 1]; size_t string_size = src_string_offsets[src_array_prev_offset + j_reversed] - src_pos; - memcpySmallAllowReadWriteOverflow15(&res_data[res_string_prev_offset], &src_data[src_pos], string_size); + memcpySmallAllowReadWriteOverflow15(&res_chars[res_string_prev_offset], &src_data[src_pos], string_size); res_string_prev_offset += string_size; res_string_offsets[src_array_prev_offset + j] = res_string_prev_offset; @@ -319,37 +238,6 @@ bool FunctionArrayReverse::executeString( src_array_prev_offset = src_array_offsets[i]; } - if (nullable_col && nullable_res_col) - { - /// Make a reverted null map. - const auto & src_null_map = nullable_col->getNullMapData(); - auto & res_null_map = nullable_res_col->getNullMapData(); - res_null_map.resize(src_string_offsets.size()); - - size_t size = src_string_offsets.size(); - ColumnArray::Offset src_prev_offset = 0; - - for (size_t i = 0; i < size; ++i) - { - const auto * src = &src_null_map[src_prev_offset]; - const auto * src_end = &src_null_map[src_array_offsets[i]]; - - if (src == src_end) - continue; - - auto dst = &res_null_map[src_array_offsets[i] - 1]; - - while (src < src_end) - { - *dst = *src; - ++src; - --dst; - } - - src_prev_offset = src_array_offsets[i]; - } - } - return true; } else diff --git a/dbms/tests/queries/0_stateless/00758_array_reverse.reference b/dbms/tests/queries/0_stateless/00758_array_reverse.reference new file mode 100644 index 00000000000..f3e6bb6d467 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00758_array_reverse.reference @@ -0,0 +1,12 @@ +['\0',NULL] +[NULL,123,NULL] +[NULL,'Hello'] +['world','Hello'] +['world',NULL,'Hello'] +[NULL,NULL,NULL] +[[' '],[''],[]] +[[NULL],[''],[]] +[(nan,'World',[NULL]),(1,'Hello',[])] +\N +[] +[[[[]]]] diff --git a/dbms/tests/queries/0_stateless/00758_array_reverse.sql b/dbms/tests/queries/0_stateless/00758_array_reverse.sql new file mode 100644 index 00000000000..c453a33ccfd --- /dev/null +++ b/dbms/tests/queries/0_stateless/00758_array_reverse.sql @@ -0,0 +1,12 @@ +SELECT reverse([NULL, '\0']); +SELECT reverse([NULL, 123, NULL]); +SELECT reverse([toFixedString('Hello', 5), NULL]); +SELECT reverse(['Hello', 'world']); +SELECT reverse(['Hello', NULL, 'world']); +SELECT reverse([NULL, NULL, NULL]); +SELECT reverse([[], [''], [' ']]); +SELECT reverse([[], [''], [NULL]]); +SELECT reverse([(1, 'Hello', []), (nan, 'World', [NULL])]); +SELECT reverse(NULL); +SELECT reverse([]); +SELECT reverse([[[[]]]]); From fc9c7680b003d9285620b08ca403852fcc45aff1 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 24 Nov 2018 06:20:05 +0300 Subject: [PATCH 102/124] Added a test from Kodieg #2680 --- dbms/tests/queries/0_stateless/00759_kodieg.reference | 1 + dbms/tests/queries/0_stateless/00759_kodieg.sql | 4 ++++ 2 files changed, 5 insertions(+) create mode 100644 dbms/tests/queries/0_stateless/00759_kodieg.reference create mode 100644 dbms/tests/queries/0_stateless/00759_kodieg.sql diff --git a/dbms/tests/queries/0_stateless/00759_kodieg.reference b/dbms/tests/queries/0_stateless/00759_kodieg.reference new file mode 100644 index 00000000000..40727bcdda7 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00759_kodieg.reference @@ -0,0 +1 @@ +[1,2,3,1,3] 2 [1,3] diff --git a/dbms/tests/queries/0_stateless/00759_kodieg.sql b/dbms/tests/queries/0_stateless/00759_kodieg.sql new file mode 100644 index 00000000000..2037f210dea --- /dev/null +++ b/dbms/tests/queries/0_stateless/00759_kodieg.sql @@ -0,0 +1,4 @@ +SELECT + [1, 2, 3, 1, 3] AS a, + indexOf(arrayReverse(arraySlice(a, 1, -1)), 3) AS offset_from_right, + arraySlice(a, multiIf(offset_from_right = 0, 1, (length(a) - offset_from_right) + 1)); From 244e64a757c306d636b4de7127e704c24dde1297 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 24 Nov 2018 06:22:38 +0300 Subject: [PATCH 103/124] Removed bug [#CLICKHOUSE-3] --- dbms/tests/queries/0_stateless/00758_array_reverse.sql | 3 +++ dbms/tests/queries/bugs/fuzzy.sql | 1 - 2 files changed, 3 insertions(+), 1 deletion(-) diff --git a/dbms/tests/queries/0_stateless/00758_array_reverse.sql b/dbms/tests/queries/0_stateless/00758_array_reverse.sql index c453a33ccfd..1c6afe0995b 100644 --- a/dbms/tests/queries/0_stateless/00758_array_reverse.sql +++ b/dbms/tests/queries/0_stateless/00758_array_reverse.sql @@ -10,3 +10,6 @@ SELECT reverse([(1, 'Hello', []), (nan, 'World', [NULL])]); SELECT reverse(NULL); SELECT reverse([]); SELECT reverse([[[[]]]]); + +SET send_logs_level = 'none'; +SELECT '[RE7', ( SELECT '\0' ) AS riwwq, ( SELECT reverse([( SELECT bitTestAll(NULL) ) , ( SELECT '\0' ) AS ddfweeuy]) ) AS xuvv, '', ( SELECT * FROM mysql() ) AS wqgdswyc, ( SELECT * FROM mysql() ); -- { serverError 42 } diff --git a/dbms/tests/queries/bugs/fuzzy.sql b/dbms/tests/queries/bugs/fuzzy.sql index 80153803cfe..e90b0592e79 100644 --- a/dbms/tests/queries/bugs/fuzzy.sql +++ b/dbms/tests/queries/bugs/fuzzy.sql @@ -18,4 +18,3 @@ SELECT subtractDays((CAST((-5263074.47) AS DateTime)), -737895); SELECT quantileDeterministic([], findClusterIndex(( SELECT subtractDays((CAST((566450.398706) AS DateTime)), 54) ) )), '\0', []; SELECT addDays((CAST((96.338) AS DateTime)), -3); SELECT arrayEnumerateDense([], [sequenceCount(NULL)]); -SELECT '[RE7', ( SELECT '\0' ) AS riwwq, ( SELECT reverse([( SELECT bitTestAll(NULL) ) , ( SELECT '\0' ) AS ddfweeuy]) ) AS xuvv, '', ( SELECT * FROM mysql() ) AS wqgdswyc, ( SELECT * FROM mysql() ); From 141e9799e49201d84ea8e951d1bed4fb6d3dacb5 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 25 Nov 2018 03:08:50 +0300 Subject: [PATCH 104/124] Fixed error with "extractURLParameter" function (read after buffer); improved performance; added support for zero bytes in URLs; renamed Chars_t type [#CLICKHOUSE-2] --- dbms/programs/client/Client.cpp | 2 +- .../validateODBCConnectionString.cpp | 2 +- dbms/src/Columns/ColumnArray.cpp | 8 +- dbms/src/Columns/ColumnFixedString.cpp | 6 +- dbms/src/Columns/ColumnFixedString.h | 8 +- dbms/src/Columns/ColumnString.cpp | 8 +- dbms/src/Columns/ColumnString.h | 8 +- dbms/src/Common/parseAddress.cpp | 2 +- dbms/src/DataTypes/DataTypeFixedString.cpp | 16 +- dbms/src/DataTypes/DataTypeString.cpp | 18 +-- dbms/src/DataTypes/tests/data_type_string.cpp | 2 +- dbms/src/Functions/EmptyImpl.h | 6 +- dbms/src/Functions/FunctionsCoding.h | 42 ++--- dbms/src/Functions/FunctionsComparison.h | 60 ++++---- dbms/src/Functions/FunctionsConversion.h | 16 +- dbms/src/Functions/FunctionsFormatting.h | 8 +- dbms/src/Functions/FunctionsHashing.h | 6 +- dbms/src/Functions/FunctionsReinterpret.h | 8 +- dbms/src/Functions/FunctionsStringArray.h | 8 +- dbms/src/Functions/FunctionsStringSearch.cpp | 32 ++-- dbms/src/Functions/FunctionsStringSearch.h | 2 +- dbms/src/Functions/FunctionsTransform.h | 28 ++-- dbms/src/Functions/FunctionsURL.cpp | 8 +- dbms/src/Functions/FunctionsURL.h | 145 ++++++++++-------- dbms/src/Functions/FunctionsVisitParam.h | 10 +- dbms/src/Functions/GatherUtils/Sinks.h | 4 +- dbms/src/Functions/GatherUtils/Sources.h | 2 +- dbms/src/Functions/LowerUpperImpl.h | 6 +- dbms/src/Functions/LowerUpperUTF8Impl.h | 6 +- dbms/src/Functions/arrayElement.cpp | 8 +- dbms/src/Functions/arrayIndex.h | 8 +- dbms/src/Functions/arrayReverse.cpp | 8 +- dbms/src/Functions/bar.cpp | 2 +- dbms/src/Functions/convertCharset.cpp | 4 +- dbms/src/Functions/dateDiff.cpp | 2 +- dbms/src/Functions/emptyArrayToSingle.cpp | 8 +- dbms/src/Functions/formatDateTime.cpp | 2 +- dbms/src/Functions/length.cpp | 6 +- dbms/src/Functions/lengthUTF8.cpp | 6 +- dbms/src/Functions/reverse.cpp | 6 +- dbms/src/Functions/reverseUTF8.cpp | 6 +- dbms/src/Functions/substringUTF8.cpp | 4 +- dbms/src/IO/ReadHelpers.cpp | 2 +- dbms/src/IO/WriteHelpers.h | 2 +- dbms/src/IO/tests/mempbrk.cpp | 2 +- dbms/src/Interpreters/Aggregator.h | 4 +- dbms/src/Interpreters/Join.h | 4 +- dbms/src/Interpreters/SetVariants.h | 4 +- dbms/src/Parsers/CommonParsers.cpp | 2 +- dbms/src/Parsers/Lexer.cpp | 2 +- dbms/src/Parsers/parseQuery.cpp | 2 +- libs/libcommon/CMakeLists.txt | 1 + .../{find_first_symbols.h => find_symbols.h} | 49 +++++- libs/libcommon/src/JSON.cpp | 2 +- libs/libcommon/src/tests/CMakeLists.txt | 2 +- .../src/tests/gtest_find_symbols.cpp | 25 +++ 56 files changed, 368 insertions(+), 282 deletions(-) rename libs/libcommon/include/common/{find_first_symbols.h => find_symbols.h} (76%) create mode 100644 libs/libcommon/src/tests/gtest_find_symbols.cpp diff --git a/dbms/programs/client/Client.cpp b/dbms/programs/client/Client.cpp index cba7b23b35f..ee2a940e24f 100644 --- a/dbms/programs/client/Client.cpp +++ b/dbms/programs/client/Client.cpp @@ -18,7 +18,7 @@ #include #include #include -#include +#include #include #include #include diff --git a/dbms/programs/odbc-bridge/validateODBCConnectionString.cpp b/dbms/programs/odbc-bridge/validateODBCConnectionString.cpp index a817a01c288..a64d7bb0b7c 100644 --- a/dbms/programs/odbc-bridge/validateODBCConnectionString.cpp +++ b/dbms/programs/odbc-bridge/validateODBCConnectionString.cpp @@ -2,7 +2,7 @@ #include #include #include -#include +#include #include #include #include "validateODBCConnectionString.h" diff --git a/dbms/src/Columns/ColumnArray.cpp b/dbms/src/Columns/ColumnArray.cpp index b556e608a86..f9e0ff3219b 100644 --- a/dbms/src/Columns/ColumnArray.cpp +++ b/dbms/src/Columns/ColumnArray.cpp @@ -437,11 +437,11 @@ ColumnPtr ColumnArray::filterString(const Filter & filt, ssize_t result_size_hin auto res = ColumnArray::create(data->cloneEmpty()); const ColumnString & src_string = typeid_cast(*data); - const ColumnString::Chars_t & src_chars = src_string.getChars(); + const ColumnString::Chars & src_chars = src_string.getChars(); const Offsets & src_string_offsets = src_string.getOffsets(); const Offsets & src_offsets = getOffsets(); - ColumnString::Chars_t & res_chars = typeid_cast(res->getData()).getChars(); + ColumnString::Chars & res_chars = typeid_cast(res->getData()).getChars(); Offsets & res_string_offsets = typeid_cast(res->getData()).getOffsets(); Offsets & res_offsets = res->getOffsets(); @@ -781,11 +781,11 @@ ColumnPtr ColumnArray::replicateString(const Offsets & replicate_offsets) const ColumnArray & res_ = static_cast(*res); const ColumnString & src_string = typeid_cast(*data); - const ColumnString::Chars_t & src_chars = src_string.getChars(); + const ColumnString::Chars & src_chars = src_string.getChars(); const Offsets & src_string_offsets = src_string.getOffsets(); const Offsets & src_offsets = getOffsets(); - ColumnString::Chars_t & res_chars = typeid_cast(res_.getData()).getChars(); + ColumnString::Chars & res_chars = typeid_cast(res_.getData()).getChars(); Offsets & res_string_offsets = typeid_cast(res_.getData()).getOffsets(); Offsets & res_offsets = res_.getOffsets(); diff --git a/dbms/src/Columns/ColumnFixedString.cpp b/dbms/src/Columns/ColumnFixedString.cpp index 130eb155337..b91c27becd7 100644 --- a/dbms/src/Columns/ColumnFixedString.cpp +++ b/dbms/src/Columns/ColumnFixedString.cpp @@ -248,7 +248,7 @@ ColumnPtr ColumnFixedString::permute(const Permutation & perm, size_t limit) con auto res = ColumnFixedString::create(n); - Chars_t & res_chars = res->chars; + Chars & res_chars = res->chars; res_chars.resize(n * limit); @@ -274,7 +274,7 @@ ColumnPtr ColumnFixedString::indexImpl(const PaddedPODArray & indexes, siz auto res = ColumnFixedString::create(n); - Chars_t & res_chars = res->chars; + Chars & res_chars = res->chars; res_chars.resize(n * limit); @@ -296,7 +296,7 @@ ColumnPtr ColumnFixedString::replicate(const Offsets & offsets) const if (0 == col_size) return res; - Chars_t & res_chars = res->chars; + Chars & res_chars = res->chars; res_chars.resize(n * offsets.back()); Offset curr_offset = 0; diff --git a/dbms/src/Columns/ColumnFixedString.h b/dbms/src/Columns/ColumnFixedString.h index e8ce153e1a1..fcfed7481e8 100644 --- a/dbms/src/Columns/ColumnFixedString.h +++ b/dbms/src/Columns/ColumnFixedString.h @@ -17,14 +17,14 @@ class ColumnFixedString final : public COWPtrHelper public: friend class COWPtrHelper; - using Chars_t = PaddedPODArray; + using Chars = PaddedPODArray; private: /// Bytes of rows, laid in succession. The strings are stored without a trailing zero byte. /** NOTE It is required that the offset and type of chars in the object be the same as that of `data in ColumnUInt8`. * Used in `packFixed` function (AggregationCommon.h) */ - Chars_t chars; + Chars chars; /// The size of the rows. const size_t n; @@ -138,8 +138,8 @@ public: /// Specialized part of interface, not from IColumn. - Chars_t & getChars() { return chars; } - const Chars_t & getChars() const { return chars; } + Chars & getChars() { return chars; } + const Chars & getChars() const { return chars; } size_t getN() const { return n; } }; diff --git a/dbms/src/Columns/ColumnString.cpp b/dbms/src/Columns/ColumnString.cpp index 64806d56d72..a7e1cb1576d 100644 --- a/dbms/src/Columns/ColumnString.cpp +++ b/dbms/src/Columns/ColumnString.cpp @@ -101,7 +101,7 @@ ColumnPtr ColumnString::filter(const Filter & filt, ssize_t result_size_hint) co auto res = ColumnString::create(); - Chars_t & res_chars = res->chars; + Chars & res_chars = res->chars; Offsets & res_offsets = res->offsets; filterArraysImpl(chars, offsets, res_chars, res_offsets, filt, result_size_hint); @@ -126,7 +126,7 @@ ColumnPtr ColumnString::permute(const Permutation & perm, size_t limit) const auto res = ColumnString::create(); - Chars_t & res_chars = res->chars; + Chars & res_chars = res->chars; Offsets & res_offsets = res->offsets; if (limit == size) @@ -202,7 +202,7 @@ ColumnPtr ColumnString::indexImpl(const PaddedPODArray & indexes, size_t l auto res = ColumnString::create(); - Chars_t & res_chars = res->chars; + Chars & res_chars = res->chars; Offsets & res_offsets = res->offsets; size_t new_chars_size = 0; @@ -287,7 +287,7 @@ ColumnPtr ColumnString::replicate(const Offsets & replicate_offsets) const if (0 == col_size) return res; - Chars_t & res_chars = res->chars; + Chars & res_chars = res->chars; Offsets & res_offsets = res->offsets; res_chars.reserve(chars.size() / col_size * replicate_offsets.back()); res_offsets.reserve(replicate_offsets.back()); diff --git a/dbms/src/Columns/ColumnString.h b/dbms/src/Columns/ColumnString.h index 767d67a7385..bdf9be13773 100644 --- a/dbms/src/Columns/ColumnString.h +++ b/dbms/src/Columns/ColumnString.h @@ -19,7 +19,7 @@ namespace DB class ColumnString final : public COWPtrHelper { public: - using Chars_t = PaddedPODArray; + using Chars = PaddedPODArray; private: friend class COWPtrHelper; @@ -29,7 +29,7 @@ private: /// Bytes of strings, placed contiguously. /// For convenience, every string ends with terminating zero byte. Note that strings could contain zero bytes in the middle. - Chars_t chars; + Chars chars; size_t ALWAYS_INLINE offsetAt(size_t i) const { return i == 0 ? 0 : offsets[i - 1]; } @@ -245,8 +245,8 @@ public: bool canBeInsideNullable() const override { return true; } - Chars_t & getChars() { return chars; } - const Chars_t & getChars() const { return chars; } + Chars & getChars() { return chars; } + const Chars & getChars() const { return chars; } Offsets & getOffsets() { return offsets; } const Offsets & getOffsets() const { return offsets; } diff --git a/dbms/src/Common/parseAddress.cpp b/dbms/src/Common/parseAddress.cpp index 5cd6d353e42..1e266ad503f 100644 --- a/dbms/src/Common/parseAddress.cpp +++ b/dbms/src/Common/parseAddress.cpp @@ -1,7 +1,7 @@ #include #include #include -#include +#include namespace DB diff --git a/dbms/src/DataTypes/DataTypeFixedString.cpp b/dbms/src/DataTypes/DataTypeFixedString.cpp index 82c0bec489f..f43de616e8d 100644 --- a/dbms/src/DataTypes/DataTypeFixedString.cpp +++ b/dbms/src/DataTypes/DataTypeFixedString.cpp @@ -64,7 +64,7 @@ void DataTypeFixedString::serializeBinary(const IColumn & column, size_t row_num void DataTypeFixedString::deserializeBinary(IColumn & column, ReadBuffer & istr) const { - ColumnFixedString::Chars_t & data = static_cast(column).getChars(); + ColumnFixedString::Chars & data = static_cast(column).getChars(); size_t old_size = data.size(); data.resize(old_size + n); try @@ -81,7 +81,7 @@ void DataTypeFixedString::deserializeBinary(IColumn & column, ReadBuffer & istr) void DataTypeFixedString::serializeBinaryBulk(const IColumn & column, WriteBuffer & ostr, size_t offset, size_t limit) const { - const ColumnFixedString::Chars_t & data = typeid_cast(column).getChars(); + const ColumnFixedString::Chars & data = typeid_cast(column).getChars(); size_t size = data.size() / n; @@ -95,7 +95,7 @@ void DataTypeFixedString::serializeBinaryBulk(const IColumn & column, WriteBuffe void DataTypeFixedString::deserializeBinaryBulk(IColumn & column, ReadBuffer & istr, size_t limit, double /*avg_value_size_hint*/) const { - ColumnFixedString::Chars_t & data = typeid_cast(column).getChars(); + ColumnFixedString::Chars & data = typeid_cast(column).getChars(); size_t initial_size = data.size(); size_t max_bytes = limit * n; @@ -126,7 +126,7 @@ void DataTypeFixedString::serializeTextEscaped(const IColumn & column, size_t ro template static inline void read(const DataTypeFixedString & self, IColumn & column, Reader && reader) { - ColumnFixedString::Chars_t & data = typeid_cast(column).getChars(); + ColumnFixedString::Chars & data = typeid_cast(column).getChars(); size_t prev_size = data.size(); try @@ -152,7 +152,7 @@ static inline void read(const DataTypeFixedString & self, IColumn & column, Read void DataTypeFixedString::deserializeTextEscaped(IColumn & column, ReadBuffer & istr, const FormatSettings &) const { - read(*this, column, [&istr](ColumnFixedString::Chars_t & data) { readEscapedStringInto(data, istr); }); + read(*this, column, [&istr](ColumnFixedString::Chars & data) { readEscapedStringInto(data, istr); }); } @@ -165,7 +165,7 @@ void DataTypeFixedString::serializeTextQuoted(const IColumn & column, size_t row void DataTypeFixedString::deserializeTextQuoted(IColumn & column, ReadBuffer & istr, const FormatSettings &) const { - read(*this, column, [&istr](ColumnFixedString::Chars_t & data) { readQuotedStringInto(data, istr); }); + read(*this, column, [&istr](ColumnFixedString::Chars & data) { readQuotedStringInto(data, istr); }); } @@ -178,7 +178,7 @@ void DataTypeFixedString::serializeTextJSON(const IColumn & column, size_t row_n void DataTypeFixedString::deserializeTextJSON(IColumn & column, ReadBuffer & istr, const FormatSettings &) const { - read(*this, column, [&istr](ColumnFixedString::Chars_t & data) { readJSONStringInto(data, istr); }); + read(*this, column, [&istr](ColumnFixedString::Chars & data) { readJSONStringInto(data, istr); }); } @@ -198,7 +198,7 @@ void DataTypeFixedString::serializeTextCSV(const IColumn & column, size_t row_nu void DataTypeFixedString::deserializeTextCSV(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const { - read(*this, column, [&istr, &csv = settings.csv](ColumnFixedString::Chars_t & data) { readCSVStringInto(data, istr, csv); }); + read(*this, column, [&istr, &csv = settings.csv](ColumnFixedString::Chars & data) { readCSVStringInto(data, istr, csv); }); } diff --git a/dbms/src/DataTypes/DataTypeString.cpp b/dbms/src/DataTypes/DataTypeString.cpp index 2023523aa41..e33cf5d064e 100644 --- a/dbms/src/DataTypes/DataTypeString.cpp +++ b/dbms/src/DataTypes/DataTypeString.cpp @@ -53,7 +53,7 @@ void DataTypeString::serializeBinary(const IColumn & column, size_t row_num, Wri void DataTypeString::deserializeBinary(IColumn & column, ReadBuffer & istr) const { ColumnString & column_string = static_cast(column); - ColumnString::Chars_t & data = column_string.getChars(); + ColumnString::Chars & data = column_string.getChars(); ColumnString::Offsets & offsets = column_string.getOffsets(); UInt64 size; @@ -81,7 +81,7 @@ void DataTypeString::deserializeBinary(IColumn & column, ReadBuffer & istr) cons void DataTypeString::serializeBinaryBulk(const IColumn & column, WriteBuffer & ostr, size_t offset, size_t limit) const { const ColumnString & column_string = typeid_cast(column); - const ColumnString::Chars_t & data = column_string.getChars(); + const ColumnString::Chars & data = column_string.getChars(); const ColumnString::Offsets & offsets = column_string.getOffsets(); size_t size = column.size(); @@ -111,7 +111,7 @@ void DataTypeString::serializeBinaryBulk(const IColumn & column, WriteBuffer & o template -static NO_INLINE void deserializeBinarySSE2(ColumnString::Chars_t & data, ColumnString::Offsets & offsets, ReadBuffer & istr, size_t limit) +static NO_INLINE void deserializeBinarySSE2(ColumnString::Chars & data, ColumnString::Offsets & offsets, ReadBuffer & istr, size_t limit) { size_t offset = data.size(); for (size_t i = 0; i < limit; ++i) @@ -174,7 +174,7 @@ static NO_INLINE void deserializeBinarySSE2(ColumnString::Chars_t & data, Column void DataTypeString::deserializeBinaryBulk(IColumn & column, ReadBuffer & istr, size_t limit, double avg_value_size_hint) const { ColumnString & column_string = typeid_cast(column); - ColumnString::Chars_t & data = column_string.getChars(); + ColumnString::Chars & data = column_string.getChars(); ColumnString::Offsets & offsets = column_string.getOffsets(); double avg_chars_size = 1; /// By default reserve only for empty strings. @@ -235,7 +235,7 @@ template static inline void read(IColumn & column, Reader && reader) { ColumnString & column_string = static_cast(column); - ColumnString::Chars_t & data = column_string.getChars(); + ColumnString::Chars & data = column_string.getChars(); ColumnString::Offsets & offsets = column_string.getOffsets(); size_t old_chars_size = data.size(); @@ -258,7 +258,7 @@ static inline void read(IColumn & column, Reader && reader) void DataTypeString::deserializeTextEscaped(IColumn & column, ReadBuffer & istr, const FormatSettings &) const { - read(column, [&](ColumnString::Chars_t & data) { readEscapedStringInto(data, istr); }); + read(column, [&](ColumnString::Chars & data) { readEscapedStringInto(data, istr); }); } @@ -270,7 +270,7 @@ void DataTypeString::serializeTextQuoted(const IColumn & column, size_t row_num, void DataTypeString::deserializeTextQuoted(IColumn & column, ReadBuffer & istr, const FormatSettings &) const { - read(column, [&](ColumnString::Chars_t & data) { readQuotedStringInto(data, istr); }); + read(column, [&](ColumnString::Chars & data) { readQuotedStringInto(data, istr); }); } @@ -282,7 +282,7 @@ void DataTypeString::serializeTextJSON(const IColumn & column, size_t row_num, W void DataTypeString::deserializeTextJSON(IColumn & column, ReadBuffer & istr, const FormatSettings &) const { - read(column, [&](ColumnString::Chars_t & data) { readJSONStringInto(data, istr); }); + read(column, [&](ColumnString::Chars & data) { readJSONStringInto(data, istr); }); } @@ -300,7 +300,7 @@ void DataTypeString::serializeTextCSV(const IColumn & column, size_t row_num, Wr void DataTypeString::deserializeTextCSV(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const { - read(column, [&](ColumnString::Chars_t & data) { readCSVStringInto(data, istr, settings.csv); }); + read(column, [&](ColumnString::Chars & data) { readCSVStringInto(data, istr, settings.csv); }); } diff --git a/dbms/src/DataTypes/tests/data_type_string.cpp b/dbms/src/DataTypes/tests/data_type_string.cpp index 65a15d059b4..bb394b1784c 100644 --- a/dbms/src/DataTypes/tests/data_type_string.cpp +++ b/dbms/src/DataTypes/tests/data_type_string.cpp @@ -25,7 +25,7 @@ try { auto column = ColumnString::create(); - ColumnString::Chars_t & data = column->getChars(); + ColumnString::Chars & data = column->getChars(); ColumnString::Offsets & offsets = column->getOffsets(); data.resize(n * size); diff --git a/dbms/src/Functions/EmptyImpl.h b/dbms/src/Functions/EmptyImpl.h index e66edf5f4ba..a9702fe8869 100644 --- a/dbms/src/Functions/EmptyImpl.h +++ b/dbms/src/Functions/EmptyImpl.h @@ -19,7 +19,7 @@ struct EmptyImpl /// If the function will return constant value for FixedString data type. static constexpr auto is_fixed_to_constant = false; - static void vector(const ColumnString::Chars_t & /*data*/, const ColumnString::Offsets & offsets, PaddedPODArray & res) + static void vector(const ColumnString::Chars & /*data*/, const ColumnString::Offsets & offsets, PaddedPODArray & res) { size_t size = offsets.size(); ColumnString::Offset prev_offset = 1; @@ -31,12 +31,12 @@ struct EmptyImpl } /// Only make sense if is_fixed_to_constant. - static void vector_fixed_to_constant(const ColumnString::Chars_t & /*data*/, size_t /*n*/, UInt8 & /*res*/) + static void vector_fixed_to_constant(const ColumnString::Chars & /*data*/, size_t /*n*/, UInt8 & /*res*/) { throw Exception("Logical error: 'vector_fixed_to_constant method' is called", ErrorCodes::LOGICAL_ERROR); } - static void vector_fixed_to_vector(const ColumnString::Chars_t & data, size_t n, PaddedPODArray & res) + static void vector_fixed_to_vector(const ColumnString::Chars & data, size_t n, PaddedPODArray & res) { std::vector empty_chars(n); size_t size = data.size() / n; diff --git a/dbms/src/Functions/FunctionsCoding.h b/dbms/src/Functions/FunctionsCoding.h index 27424f7355b..f2378e68faf 100644 --- a/dbms/src/Functions/FunctionsCoding.h +++ b/dbms/src/Functions/FunctionsCoding.h @@ -105,7 +105,7 @@ public: auto col_res = ColumnString::create(); - ColumnString::Chars_t & vec_res = col_res->getChars(); + ColumnString::Chars & vec_res = col_res->getChars(); ColumnString::Offsets & offsets_res = col_res->getOffsets(); vec_res.resize(size * (IPV6_MAX_TEXT_LENGTH + 1)); offsets_res.resize(size); @@ -214,7 +214,7 @@ public: auto col_res = ColumnString::create(); - ColumnString::Chars_t & vec_res = col_res->getChars(); + ColumnString::Chars & vec_res = col_res->getChars(); ColumnString::Offsets & offsets_res = col_res->getOffsets(); vec_res.resize(size * (IPV6_MAX_TEXT_LENGTH + 1)); offsets_res.resize(size); @@ -425,7 +425,7 @@ public: auto & vec_res = col_res->getChars(); vec_res.resize(col_in->size() * ipv6_bytes_length); - const ColumnString::Chars_t & vec_src = col_in->getChars(); + const ColumnString::Chars & vec_src = col_in->getChars(); const ColumnString::Offsets & offsets_src = col_in->getOffsets(); size_t src_offset = 0; @@ -538,7 +538,7 @@ public: auto col_res = ColumnString::create(); - ColumnString::Chars_t & vec_res = col_res->getChars(); + ColumnString::Chars & vec_res = col_res->getChars(); ColumnString::Offsets & offsets_res = col_res->getOffsets(); vec_res.resize(vec_in.size() * (IPV4_MAX_TEXT_LENGTH + 1)); /// the longest value is: 255.255.255.255\0 @@ -622,7 +622,7 @@ public: ColumnUInt32::Container & vec_res = col_res->getData(); vec_res.resize(col->size()); - const ColumnString::Chars_t & vec_src = col->getChars(); + const ColumnString::Chars & vec_src = col->getChars(); const ColumnString::Offsets & offsets_src = col->getOffsets(); size_t prev_offset = 0; @@ -752,7 +752,7 @@ public: auto col_res = ColumnString::create(); - ColumnString::Chars_t & vec_res = col_res->getChars(); + ColumnString::Chars & vec_res = col_res->getChars(); ColumnString::Offsets & offsets_res = col_res->getOffsets(); vec_res.resize(vec_in.size() * 18); /// the value is: xx:xx:xx:xx:xx:xx\0 @@ -863,7 +863,7 @@ public: ColumnUInt64::Container & vec_res = col_res->getData(); vec_res.resize(col->size()); - const ColumnString::Chars_t & vec_src = col->getChars(); + const ColumnString::Chars & vec_src = col->getChars(); const ColumnString::Offsets & offsets_src = col->getOffsets(); size_t prev_offset = 0; @@ -938,7 +938,7 @@ public: auto col_res = ColumnString::create(); - ColumnString::Chars_t & vec_res = col_res->getChars(); + ColumnString::Chars & vec_res = col_res->getChars(); ColumnString::Offsets & offsets_res = col_res->getOffsets(); vec_res.resize(size * (uuid_text_length + 1)); offsets_res.resize(size); @@ -1034,7 +1034,7 @@ public: auto col_res = ColumnFixedString::create(uuid_bytes_length); - ColumnString::Chars_t & vec_res = col_res->getChars(); + ColumnString::Chars & vec_res = col_res->getChars(); vec_res.resize(size * uuid_bytes_length); size_t src_offset = 0; @@ -1071,7 +1071,7 @@ public: auto col_res = ColumnFixedString::create(uuid_bytes_length); - ColumnString::Chars_t & vec_res = col_res->getChars(); + ColumnString::Chars & vec_res = col_res->getChars(); vec_res.resize(size * uuid_bytes_length); size_t src_offset = 0; @@ -1193,7 +1193,7 @@ public: if (col_vec) { auto col_str = ColumnString::create(); - ColumnString::Chars_t & out_vec = col_str->getChars(); + ColumnString::Chars & out_vec = col_str->getChars(); ColumnString::Offsets & out_offsets = col_str->getOffsets(); const typename ColumnVector::Container & in_vec = col_vec->getData(); @@ -1247,10 +1247,10 @@ public: if (col_str_in) { auto col_str = ColumnString::create(); - ColumnString::Chars_t & out_vec = col_str->getChars(); + ColumnString::Chars & out_vec = col_str->getChars(); ColumnString::Offsets & out_offsets = col_str->getOffsets(); - const ColumnString::Chars_t & in_vec = col_str_in->getChars(); + const ColumnString::Chars & in_vec = col_str_in->getChars(); const ColumnString::Offsets & in_offsets = col_str_in->getOffsets(); size_t size = in_offsets.size(); @@ -1291,10 +1291,10 @@ public: if (col_fstr_in) { auto col_str = ColumnString::create(); - ColumnString::Chars_t & out_vec = col_str->getChars(); + ColumnString::Chars & out_vec = col_str->getChars(); ColumnString::Offsets & out_offsets = col_str->getOffsets(); - const ColumnString::Chars_t & in_vec = col_fstr_in->getChars(); + const ColumnString::Chars & in_vec = col_fstr_in->getChars(); size_t size = col_fstr_in->size(); @@ -1403,10 +1403,10 @@ public: { auto col_res = ColumnString::create(); - ColumnString::Chars_t & out_vec = col_res->getChars(); + ColumnString::Chars & out_vec = col_res->getChars(); ColumnString::Offsets & out_offsets = col_res->getOffsets(); - const ColumnString::Chars_t & in_vec = col->getChars(); + const ColumnString::Chars & in_vec = col->getChars(); const ColumnString::Offsets & in_offsets = col->getOffsets(); size_t size = in_offsets.size(); @@ -1557,10 +1557,10 @@ public: if (col_str_in) { auto col_str = ColumnString::create(); - ColumnString::Chars_t & out_vec = col_str->getChars(); + ColumnString::Chars & out_vec = col_str->getChars(); ColumnString::Offsets & out_offsets = col_str->getOffsets(); - const ColumnString::Chars_t & in_vec = col_str_in->getChars(); + const ColumnString::Chars & in_vec = col_str_in->getChars(); const ColumnString::Offsets & in_offsets = col_str_in->getOffsets(); size_t size = in_offsets.size(); @@ -1604,10 +1604,10 @@ public: if (col_fstr_in) { auto col_str = ColumnString::create(); - ColumnString::Chars_t & out_vec = col_str->getChars(); + ColumnString::Chars & out_vec = col_str->getChars(); ColumnString::Offsets & out_offsets = col_str->getOffsets(); - const ColumnString::Chars_t & in_vec = col_fstr_in->getChars(); + const ColumnString::Chars & in_vec = col_fstr_in->getChars(); size_t size = col_fstr_in->size(); diff --git a/dbms/src/Functions/FunctionsComparison.h b/dbms/src/Functions/FunctionsComparison.h index 358d52cc58e..65d6d1cbad5 100644 --- a/dbms/src/Functions/FunctionsComparison.h +++ b/dbms/src/Functions/FunctionsComparison.h @@ -144,8 +144,8 @@ template struct StringComparisonImpl { static void NO_INLINE string_vector_string_vector( - const ColumnString::Chars_t & a_data, const ColumnString::Offsets & a_offsets, - const ColumnString::Chars_t & b_data, const ColumnString::Offsets & b_offsets, + const ColumnString::Chars & a_data, const ColumnString::Offsets & a_offsets, + const ColumnString::Chars & b_data, const ColumnString::Offsets & b_offsets, PaddedPODArray & c) { size_t size = a_offsets.size(); @@ -174,8 +174,8 @@ struct StringComparisonImpl } static void NO_INLINE string_vector_fixed_string_vector( - const ColumnString::Chars_t & a_data, const ColumnString::Offsets & a_offsets, - const ColumnString::Chars_t & b_data, ColumnString::Offset b_n, + const ColumnString::Chars & a_data, const ColumnString::Offsets & a_offsets, + const ColumnString::Chars & b_data, ColumnString::Offset b_n, PaddedPODArray & c) { size_t size = a_offsets.size(); @@ -196,7 +196,7 @@ struct StringComparisonImpl } static void NO_INLINE string_vector_constant( - const ColumnString::Chars_t & a_data, const ColumnString::Offsets & a_offsets, + const ColumnString::Chars & a_data, const ColumnString::Offsets & a_offsets, const std::string & b, PaddedPODArray & c) { @@ -220,16 +220,16 @@ struct StringComparisonImpl } static void fixed_string_vector_string_vector( - const ColumnString::Chars_t & a_data, ColumnString::Offset a_n, - const ColumnString::Chars_t & b_data, const ColumnString::Offsets & b_offsets, + const ColumnString::Chars & a_data, ColumnString::Offset a_n, + const ColumnString::Chars & b_data, const ColumnString::Offsets & b_offsets, PaddedPODArray & c) { StringComparisonImpl::string_vector_fixed_string_vector(b_data, b_offsets, a_data, a_n, c); } static void NO_INLINE fixed_string_vector_fixed_string_vector_16( - const ColumnString::Chars_t & a_data, - const ColumnString::Chars_t & b_data, + const ColumnString::Chars & a_data, + const ColumnString::Chars & b_data, PaddedPODArray & c) { size_t size = a_data.size(); @@ -239,7 +239,7 @@ struct StringComparisonImpl } static void NO_INLINE fixed_string_vector_constant_16( - const ColumnString::Chars_t & a_data, + const ColumnString::Chars & a_data, const std::string & b, PaddedPODArray & c) { @@ -250,8 +250,8 @@ struct StringComparisonImpl } static void NO_INLINE fixed_string_vector_fixed_string_vector( - const ColumnString::Chars_t & a_data, ColumnString::Offset a_n, - const ColumnString::Chars_t & b_data, ColumnString::Offset b_n, + const ColumnString::Chars & a_data, ColumnString::Offset a_n, + const ColumnString::Chars & b_data, ColumnString::Offset b_n, PaddedPODArray & c) { /** Specialization if both sizes are 16. @@ -275,7 +275,7 @@ struct StringComparisonImpl } static void NO_INLINE fixed_string_vector_constant( - const ColumnString::Chars_t & a_data, ColumnString::Offset a_n, + const ColumnString::Chars & a_data, ColumnString::Offset a_n, const std::string & b, PaddedPODArray & c) { @@ -298,7 +298,7 @@ struct StringComparisonImpl static void constant_string_vector( const std::string & a, - const ColumnString::Chars_t & b_data, const ColumnString::Offsets & b_offsets, + const ColumnString::Chars & b_data, const ColumnString::Offsets & b_offsets, PaddedPODArray & c) { StringComparisonImpl::string_vector_constant(b_data, b_offsets, a, c); @@ -306,7 +306,7 @@ struct StringComparisonImpl static void constant_fixed_string_vector( const std::string & a, - const ColumnString::Chars_t & b_data, ColumnString::Offset b_n, + const ColumnString::Chars & b_data, ColumnString::Offset b_n, PaddedPODArray & c) { StringComparisonImpl::fixed_string_vector_constant(b_data, b_n, a, c); @@ -331,8 +331,8 @@ template struct StringEqualsImpl { static void NO_INLINE string_vector_string_vector( - const ColumnString::Chars_t & a_data, const ColumnString::Offsets & a_offsets, - const ColumnString::Chars_t & b_data, const ColumnString::Offsets & b_offsets, + const ColumnString::Chars & a_data, const ColumnString::Offsets & a_offsets, + const ColumnString::Chars & b_data, const ColumnString::Offsets & b_offsets, PaddedPODArray & c) { size_t size = a_offsets.size(); @@ -344,8 +344,8 @@ struct StringEqualsImpl } static void NO_INLINE string_vector_fixed_string_vector( - const ColumnString::Chars_t & a_data, const ColumnString::Offsets & a_offsets, - const ColumnString::Chars_t & b_data, ColumnString::Offset b_n, + const ColumnString::Chars & a_data, const ColumnString::Offsets & a_offsets, + const ColumnString::Chars & b_data, ColumnString::Offset b_n, PaddedPODArray & c) { size_t size = a_offsets.size(); @@ -357,7 +357,7 @@ struct StringEqualsImpl } static void NO_INLINE string_vector_constant( - const ColumnString::Chars_t & a_data, const ColumnString::Offsets & a_offsets, + const ColumnString::Chars & a_data, const ColumnString::Offsets & a_offsets, const std::string & b, PaddedPODArray & c) { @@ -373,8 +373,8 @@ struct StringEqualsImpl #if __SSE2__ static void NO_INLINE fixed_string_vector_fixed_string_vector_16( - const ColumnString::Chars_t & a_data, - const ColumnString::Chars_t & b_data, + const ColumnString::Chars & a_data, + const ColumnString::Chars & b_data, PaddedPODArray & c) { size_t size = c.size(); @@ -397,7 +397,7 @@ struct StringEqualsImpl } static void NO_INLINE fixed_string_vector_constant_16( - const ColumnString::Chars_t & a_data, + const ColumnString::Chars & a_data, const std::string & b, PaddedPODArray & c) { @@ -421,8 +421,8 @@ struct StringEqualsImpl #endif static void NO_INLINE fixed_string_vector_fixed_string_vector( - const ColumnString::Chars_t & a_data, ColumnString::Offset a_n, - const ColumnString::Chars_t & b_data, ColumnString::Offset b_n, + const ColumnString::Chars & a_data, ColumnString::Offset a_n, + const ColumnString::Chars & b_data, ColumnString::Offset b_n, PaddedPODArray & c) { /** Specialization if both sizes are 16. @@ -443,7 +443,7 @@ struct StringEqualsImpl } static void NO_INLINE fixed_string_vector_constant( - const ColumnString::Chars_t & a_data, ColumnString::Offset a_n, + const ColumnString::Chars & a_data, ColumnString::Offset a_n, const std::string & b, PaddedPODArray & c) { @@ -464,8 +464,8 @@ struct StringEqualsImpl } static void fixed_string_vector_string_vector( - const ColumnString::Chars_t & a_data, ColumnString::Offset a_n, - const ColumnString::Chars_t & b_data, const ColumnString::Offsets & b_offsets, + const ColumnString::Chars & a_data, ColumnString::Offset a_n, + const ColumnString::Chars & b_data, const ColumnString::Offsets & b_offsets, PaddedPODArray & c) { string_vector_fixed_string_vector(b_data, b_offsets, a_data, a_n, c); @@ -473,7 +473,7 @@ struct StringEqualsImpl static void constant_string_vector( const std::string & a, - const ColumnString::Chars_t & b_data, const ColumnString::Offsets & b_offsets, + const ColumnString::Chars & b_data, const ColumnString::Offsets & b_offsets, PaddedPODArray & c) { string_vector_constant(b_data, b_offsets, a, c); @@ -481,7 +481,7 @@ struct StringEqualsImpl static void constant_fixed_string_vector( const std::string & a, - const ColumnString::Chars_t & b_data, ColumnString::Offset b_n, + const ColumnString::Chars & b_data, ColumnString::Offset b_n, PaddedPODArray & c) { fixed_string_vector_constant(b_data, b_n, a, c); diff --git a/dbms/src/Functions/FunctionsConversion.h b/dbms/src/Functions/FunctionsConversion.h index 9fbb1828827..d23a8104efa 100644 --- a/dbms/src/Functions/FunctionsConversion.h +++ b/dbms/src/Functions/FunctionsConversion.h @@ -290,7 +290,7 @@ struct ConvertImplgetData(); - ColumnString::Chars_t & data_to = col_to->getChars(); + ColumnString::Chars & data_to = col_to->getChars(); ColumnString::Offsets & offsets_to = col_to->getOffsets(); size_t size = vec_from.size(); @@ -303,7 +303,7 @@ struct ConvertImpl write_buffer(data_to); + WriteBufferFromVector write_buffer(data_to); for (size_t i = 0; i < size; ++i) { @@ -337,13 +337,13 @@ struct ConvertImplGenericToString auto col_to = ColumnString::create(); - ColumnString::Chars_t & data_to = col_to->getChars(); + ColumnString::Chars & data_to = col_to->getChars(); ColumnString::Offsets & offsets_to = col_to->getOffsets(); data_to.resize(size * 2); /// Using coefficient 2 for initial size is arbitrary. offsets_to.resize(size); - WriteBufferFromVector write_buffer(data_to); + WriteBufferFromVector write_buffer(data_to); FormatSettings format_settings; for (size_t i = 0; i < size; ++i) @@ -520,7 +520,7 @@ struct ConvertThroughParsing vec_null_map_to = &col_null_map_to->getData(); } - const ColumnString::Chars_t * chars = nullptr; + const ColumnString::Chars * chars = nullptr; const IColumn::Offsets * offsets = nullptr; size_t fixed_string_size = 0; @@ -622,7 +622,7 @@ struct ConvertImplGenericFromString IColumn & column_to = *res; column_to.reserve(size); - const ColumnString::Chars_t & chars = col_from_string->getChars(); + const ColumnString::Chars & chars = col_from_string->getChars(); const IColumn::Offsets & offsets = col_from_string->getOffsets(); size_t current_offset = 0; @@ -682,8 +682,8 @@ struct ConvertImpl { auto col_to = ColumnString::create(); - const ColumnFixedString::Chars_t & data_from = col_from->getChars(); - ColumnString::Chars_t & data_to = col_to->getChars(); + const ColumnFixedString::Chars & data_from = col_from->getChars(); + ColumnString::Chars & data_to = col_to->getChars(); ColumnString::Offsets & offsets_to = col_to->getOffsets(); size_t size = col_from->size(); size_t n = col_from->getN(); diff --git a/dbms/src/Functions/FunctionsFormatting.h b/dbms/src/Functions/FunctionsFormatting.h index 8fca3ee890e..d44e2de1ab8 100644 --- a/dbms/src/Functions/FunctionsFormatting.h +++ b/dbms/src/Functions/FunctionsFormatting.h @@ -94,13 +94,13 @@ private: auto col_to = ColumnString::create(); const typename ColumnVector::Container & vec_from = col_from->getData(); - ColumnString::Chars_t & data_to = col_to->getChars(); + ColumnString::Chars & data_to = col_to->getChars(); ColumnString::Offsets & offsets_to = col_to->getOffsets(); size_t size = vec_from.size(); data_to.resize(size * 2); offsets_to.resize(size); - WriteBufferFromVector buf_to(data_to); + WriteBufferFromVector buf_to(data_to); for (size_t i = 0; i < size; ++i) { @@ -173,13 +173,13 @@ private: auto col_to = ColumnString::create(); const typename ColumnVector::Container & vec_from = col_from->getData(); - ColumnString::Chars_t & data_to = col_to->getChars(); + ColumnString::Chars & data_to = col_to->getChars(); ColumnString::Offsets & offsets_to = col_to->getOffsets(); size_t size = vec_from.size(); data_to.resize(size * 2); offsets_to.resize(size); - WriteBufferFromVector buf_to(data_to); + WriteBufferFromVector buf_to(data_to); for (size_t i = 0; i < size; ++i) { diff --git a/dbms/src/Functions/FunctionsHashing.h b/dbms/src/Functions/FunctionsHashing.h index bb6c2668f21..22c664d433e 100644 --- a/dbms/src/Functions/FunctionsHashing.h +++ b/dbms/src/Functions/FunctionsHashing.h @@ -387,7 +387,7 @@ public: { auto col_to = ColumnFixedString::create(Impl::length); - const typename ColumnString::Chars_t & data = col_from->getChars(); + const typename ColumnString::Chars & data = col_from->getChars(); const typename ColumnString::Offsets & offsets = col_from->getOffsets(); auto & chars_to = col_to->getChars(); const auto size = offsets.size(); @@ -558,7 +558,7 @@ private: { if (const ColumnString * col_from = checkAndGetColumn(column)) { - const typename ColumnString::Chars_t & data = col_from->getChars(); + const typename ColumnString::Chars & data = col_from->getChars(); const typename ColumnString::Offsets & offsets = col_from->getOffsets(); size_t size = offsets.size(); @@ -579,7 +579,7 @@ private: } else if (const ColumnFixedString * col_from = checkAndGetColumn(column)) { - const typename ColumnString::Chars_t & data = col_from->getChars(); + const typename ColumnString::Chars & data = col_from->getChars(); size_t n = col_from->getN(); size_t size = data.size() / n; diff --git a/dbms/src/Functions/FunctionsReinterpret.h b/dbms/src/Functions/FunctionsReinterpret.h index 547e565ea6a..c6e0475d4a4 100644 --- a/dbms/src/Functions/FunctionsReinterpret.h +++ b/dbms/src/Functions/FunctionsReinterpret.h @@ -54,7 +54,7 @@ public: void executeToString(const IColumn & src, ColumnString & dst) { size_t rows = src.size(); - ColumnString::Chars_t & data_to = dst.getChars(); + ColumnString::Chars & data_to = dst.getChars(); ColumnString::Offsets & offsets_to = dst.getOffsets(); offsets_to.resize(rows); @@ -119,7 +119,7 @@ public: void executeToFixedString(const IColumn & src, ColumnFixedString & dst, size_t n) { size_t rows = src.size(); - ColumnFixedString::Chars_t & data_to = dst.getChars(); + ColumnFixedString::Chars & data_to = dst.getChars(); data_to.resize(n * rows); ColumnFixedString::Offset offset = 0; @@ -181,7 +181,7 @@ public: { auto col_res = ColumnVector::create(); - const ColumnString::Chars_t & data_from = col_from->getChars(); + const ColumnString::Chars & data_from = col_from->getChars(); const ColumnString::Offsets & offsets_from = col_from->getOffsets(); size_t size = offsets_from.size(); typename ColumnVector::Container & vec_res = col_res->getData(); @@ -202,7 +202,7 @@ public: { auto col_res = ColumnVector::create(); - const ColumnString::Chars_t & data_from = col_from->getChars(); + const ColumnString::Chars & data_from = col_from->getChars(); size_t step = col_from->getN(); size_t size = data_from.size() / step; typename ColumnVector::Container & vec_res = col_res->getData(); diff --git a/dbms/src/Functions/FunctionsStringArray.h b/dbms/src/Functions/FunctionsStringArray.h index 82c022d5e2f..19838a56081 100644 --- a/dbms/src/Functions/FunctionsStringArray.h +++ b/dbms/src/Functions/FunctionsStringArray.h @@ -352,12 +352,12 @@ public: auto col_res = ColumnArray::create(ColumnString::create()); ColumnString & res_strings = typeid_cast(col_res->getData()); ColumnArray::Offsets & res_offsets = col_res->getOffsets(); - ColumnString::Chars_t & res_strings_chars = res_strings.getChars(); + ColumnString::Chars & res_strings_chars = res_strings.getChars(); ColumnString::Offsets & res_strings_offsets = res_strings.getOffsets(); if (col_str) { - const ColumnString::Chars_t & src_chars = col_str->getChars(); + const ColumnString::Chars & src_chars = col_str->getChars(); const ColumnString::Offsets & src_offsets = col_str->getOffsets(); res_offsets.reserve(src_offsets.size()); @@ -427,11 +427,11 @@ class FunctionArrayStringConcat : public IFunction { private: void executeInternal( - const ColumnString::Chars_t & src_chars, + const ColumnString::Chars & src_chars, const ColumnString::Offsets & src_string_offsets, const ColumnArray::Offsets & src_array_offsets, const char * delimiter, const size_t delimiter_size, - ColumnString::Chars_t & dst_chars, + ColumnString::Chars & dst_chars, ColumnString::Offsets & dst_string_offsets) { size_t size = src_array_offsets.size(); diff --git a/dbms/src/Functions/FunctionsStringSearch.cpp b/dbms/src/Functions/FunctionsStringSearch.cpp index 584a625039f..6b9136a6e77 100644 --- a/dbms/src/Functions/FunctionsStringSearch.cpp +++ b/dbms/src/Functions/FunctionsStringSearch.cpp @@ -155,7 +155,7 @@ struct PositionImpl using ResultType = UInt64; /// Find one substring in many strings. - static void vector_constant(const ColumnString::Chars_t & data, + static void vector_constant(const ColumnString::Chars & data, const ColumnString::Offsets & offsets, const std::string & needle, PaddedPODArray & res) @@ -210,9 +210,9 @@ struct PositionImpl } /// Search each time for a different single substring inside each time different string. - static void vector_vector(const ColumnString::Chars_t & haystack_data, + static void vector_vector(const ColumnString::Chars & haystack_data, const ColumnString::Offsets & haystack_offsets, - const ColumnString::Chars_t & needle_data, + const ColumnString::Chars & needle_data, const ColumnString::Offsets & needle_offsets, PaddedPODArray & res) { @@ -258,7 +258,7 @@ struct PositionImpl /// Find many substrings in one line. static void constant_vector(const String & haystack, - const ColumnString::Chars_t & needle_data, + const ColumnString::Chars & needle_data, const ColumnString::Offsets & needle_offsets, PaddedPODArray & res) { @@ -348,7 +348,7 @@ struct MatchImpl { using ResultType = UInt8; - static void vector_constant(const ColumnString::Chars_t & data, + static void vector_constant(const ColumnString::Chars & data, const ColumnString::Offsets & offsets, const std::string & pattern, PaddedPODArray & res) @@ -519,10 +519,10 @@ struct MatchImpl struct ExtractImpl { - static void vector(const ColumnString::Chars_t & data, + static void vector(const ColumnString::Chars & data, const ColumnString::Offsets & offsets, const std::string & pattern, - ColumnString::Chars_t & res_data, + ColumnString::Chars & res_data, ColumnString::Offsets & res_offsets) { res_data.reserve(data.size() / 5); @@ -623,7 +623,7 @@ struct ReplaceRegexpImpl static void processString(const re2_st::StringPiece & input, - ColumnString::Chars_t & res_data, + ColumnString::Chars & res_data, ColumnString::Offset & res_offset, re2_st::RE2 & searcher, int num_captures, @@ -687,11 +687,11 @@ struct ReplaceRegexpImpl } - static void vector(const ColumnString::Chars_t & data, + static void vector(const ColumnString::Chars & data, const ColumnString::Offsets & offsets, const std::string & needle, const std::string & replacement, - ColumnString::Chars_t & res_data, + ColumnString::Chars & res_data, ColumnString::Offsets & res_offsets) { ColumnString::Offset res_offset = 0; @@ -715,11 +715,11 @@ struct ReplaceRegexpImpl } } - static void vector_fixed(const ColumnString::Chars_t & data, + static void vector_fixed(const ColumnString::Chars & data, size_t n, const std::string & needle, const std::string & replacement, - ColumnString::Chars_t & res_data, + ColumnString::Chars & res_data, ColumnString::Offsets & res_offsets) { ColumnString::Offset res_offset = 0; @@ -749,11 +749,11 @@ struct ReplaceRegexpImpl template struct ReplaceStringImpl { - static void vector(const ColumnString::Chars_t & data, + static void vector(const ColumnString::Chars & data, const ColumnString::Offsets & offsets, const std::string & needle, const std::string & replacement, - ColumnString::Chars_t & res_data, + ColumnString::Chars & res_data, ColumnString::Offsets & res_offsets) { const UInt8 * begin = data.data(); @@ -824,11 +824,11 @@ struct ReplaceStringImpl /// Note: this function converts fixed-length strings to variable-length strings /// and each variable-length string should ends with zero byte. - static void vector_fixed(const ColumnString::Chars_t & data, + static void vector_fixed(const ColumnString::Chars & data, size_t n, const std::string & needle, const std::string & replacement, - ColumnString::Chars_t & res_data, + ColumnString::Chars & res_data, ColumnString::Offsets & res_offsets) { const UInt8 * begin = data.data(); diff --git a/dbms/src/Functions/FunctionsStringSearch.h b/dbms/src/Functions/FunctionsStringSearch.h index e679a811027..7eb7f06b1e6 100644 --- a/dbms/src/Functions/FunctionsStringSearch.h +++ b/dbms/src/Functions/FunctionsStringSearch.h @@ -173,7 +173,7 @@ public: { auto col_res = ColumnString::create(); - ColumnString::Chars_t & vec_res = col_res->getChars(); + ColumnString::Chars & vec_res = col_res->getChars(); ColumnString::Offsets & offsets_res = col_res->getOffsets(); Impl::vector(col->getChars(), col->getOffsets(), col_needle->getValue(), vec_res, offsets_res); diff --git a/dbms/src/Functions/FunctionsTransform.h b/dbms/src/Functions/FunctionsTransform.h index b3d14a927f4..f7096ef38dc 100644 --- a/dbms/src/Functions/FunctionsTransform.h +++ b/dbms/src/Functions/FunctionsTransform.h @@ -549,7 +549,7 @@ private: template void executeImplNumToStringWithConstDefault(const PaddedPODArray & src, - ColumnString::Chars_t & dst_data, ColumnString::Offsets & dst_offsets, StringRef dst_default) + ColumnString::Chars & dst_data, ColumnString::Offsets & dst_offsets, StringRef dst_default) { const auto & table = *table_num_to_string; size_t size = src.size(); @@ -568,8 +568,8 @@ private: template void executeImplNumToStringWithNonConstDefault(const PaddedPODArray & src, - ColumnString::Chars_t & dst_data, ColumnString::Offsets & dst_offsets, - const ColumnString::Chars_t & dst_default_data, const ColumnString::Offsets & dst_default_offsets) + ColumnString::Chars & dst_data, ColumnString::Offsets & dst_offsets, + const ColumnString::Chars & dst_default_data, const ColumnString::Offsets & dst_default_offsets) { const auto & table = *table_num_to_string; size_t size = src.size(); @@ -599,7 +599,7 @@ private: template void executeImplStringToNumWithConstDefault( - const ColumnString::Chars_t & src_data, const ColumnString::Offsets & src_offsets, + const ColumnString::Chars & src_data, const ColumnString::Offsets & src_offsets, PaddedPODArray & dst, U dst_default) { const auto & table = *table_string_to_num; @@ -620,7 +620,7 @@ private: template void executeImplStringToNumWithNonConstDefault( - const ColumnString::Chars_t & src_data, const ColumnString::Offsets & src_offsets, + const ColumnString::Chars & src_data, const ColumnString::Offsets & src_offsets, PaddedPODArray & dst, const PaddedPODArray & dst_default) { const auto & table = *table_string_to_num; @@ -641,8 +641,8 @@ private: template void executeImplStringToStringWithOrWithoutConstDefault( - const ColumnString::Chars_t & src_data, const ColumnString::Offsets & src_offsets, - ColumnString::Chars_t & dst_data, ColumnString::Offsets & dst_offsets, StringRef dst_default) + const ColumnString::Chars & src_data, const ColumnString::Offsets & src_offsets, + ColumnString::Chars & dst_data, ColumnString::Offsets & dst_offsets, StringRef dst_default) { const auto & table = *table_string_to_string; size_t size = src_offsets.size(); @@ -665,23 +665,23 @@ private: } void executeImplStringToString( - const ColumnString::Chars_t & src_data, const ColumnString::Offsets & src_offsets, - ColumnString::Chars_t & dst_data, ColumnString::Offsets & dst_offsets) + const ColumnString::Chars & src_data, const ColumnString::Offsets & src_offsets, + ColumnString::Chars & dst_data, ColumnString::Offsets & dst_offsets) { executeImplStringToStringWithOrWithoutConstDefault(src_data, src_offsets, dst_data, dst_offsets, {}); } void executeImplStringToStringWithConstDefault( - const ColumnString::Chars_t & src_data, const ColumnString::Offsets & src_offsets, - ColumnString::Chars_t & dst_data, ColumnString::Offsets & dst_offsets, StringRef dst_default) + const ColumnString::Chars & src_data, const ColumnString::Offsets & src_offsets, + ColumnString::Chars & dst_data, ColumnString::Offsets & dst_offsets, StringRef dst_default) { executeImplStringToStringWithOrWithoutConstDefault(src_data, src_offsets, dst_data, dst_offsets, dst_default); } void executeImplStringToStringWithNonConstDefault( - const ColumnString::Chars_t & src_data, const ColumnString::Offsets & src_offsets, - ColumnString::Chars_t & dst_data, ColumnString::Offsets & dst_offsets, - const ColumnString::Chars_t & dst_default_data, const ColumnString::Offsets & dst_default_offsets) + const ColumnString::Chars & src_data, const ColumnString::Offsets & src_offsets, + ColumnString::Chars & dst_data, ColumnString::Offsets & dst_offsets, + const ColumnString::Chars & dst_default_data, const ColumnString::Offsets & dst_default_offsets) { const auto & table = *table_string_to_string; size_t size = src_offsets.size(); diff --git a/dbms/src/Functions/FunctionsURL.cpp b/dbms/src/Functions/FunctionsURL.cpp index e0f747b4061..34b05460320 100644 --- a/dbms/src/Functions/FunctionsURL.cpp +++ b/dbms/src/Functions/FunctionsURL.cpp @@ -2,7 +2,7 @@ #include #include #include -#include +#include namespace DB { @@ -84,8 +84,8 @@ void ExtractProtocol::execute(Pos data, size_t size, Pos & res_data, size_t & re } -void DecodeURLComponentImpl::vector(const ColumnString::Chars_t & data, const ColumnString::Offsets & offsets, - ColumnString::Chars_t & res_data, ColumnString::Offsets & res_offsets) +void DecodeURLComponentImpl::vector(const ColumnString::Chars & data, const ColumnString::Offsets & offsets, + ColumnString::Chars & res_data, ColumnString::Offsets & res_offsets) { res_data.resize(data.size()); size_t size = offsets.size(); @@ -109,7 +109,7 @@ void DecodeURLComponentImpl::vector(const ColumnString::Chars_t & data, const Co } -void DecodeURLComponentImpl::vector_fixed(const ColumnString::Chars_t &, size_t, ColumnString::Chars_t &) +void DecodeURLComponentImpl::vector_fixed(const ColumnString::Chars &, size_t, ColumnString::Chars &) { throw Exception("Column of type FixedString is not supported by URL functions", ErrorCodes::ILLEGAL_COLUMN); } diff --git a/dbms/src/Functions/FunctionsURL.h b/dbms/src/Functions/FunctionsURL.h index f66b428e4a7..cfd5f11406d 100644 --- a/dbms/src/Functions/FunctionsURL.h +++ b/dbms/src/Functions/FunctionsURL.h @@ -4,12 +4,11 @@ #include #include #include -#include +#include #include #include #include #include -#include namespace DB @@ -89,7 +88,7 @@ inline StringRef getURLHost(const char * data, size_t size) Pos pos = data; Pos end = data + size; - if (nullptr == (pos = strchr(pos, '/'))) + if (end == (pos = find_first_symbols<'/'>(pos, end))) return {}; if (pos != data) @@ -269,16 +268,18 @@ struct ExtractTopLevelDomain if (host.data[host.size - 1] == '.') host.size -= 1; - Pos last_dot = reinterpret_cast(memrchr(host.data, '.', host.size)); + auto host_end = host.data + host.size; + Pos last_dot = find_last_symbols_or_null<'.'>(host.data, host_end); if (!last_dot) return; + /// For IPv4 addresses select nothing. if (last_dot[1] <= '9') return; res_data = last_dot + 1; - res_size = (host.data + host.size) - res_data; + res_size = host_end - res_data; } } }; @@ -295,12 +296,12 @@ struct ExtractPath Pos pos = data; Pos end = pos + size; - if (nullptr != (pos = strchr(data, '/')) && pos[1] == '/' && nullptr != (pos = strchr(pos + 2, '/'))) + if (end != (pos = find_first_symbols<'/'>(pos, end)) && pos[1] == '/' && end != (pos = find_first_symbols<'/'>(pos + 2, end))) { - Pos query_string_or_fragment = strpbrk(pos, "?#"); + Pos query_string_or_fragment = find_first_symbols<'?', '#'>(pos, end); res_data = pos; - res_size = (query_string_or_fragment ? query_string_or_fragment : end) - res_data; + res_size = query_string_or_fragment - res_data; } } }; @@ -317,7 +318,7 @@ struct ExtractPathFull Pos pos = data; Pos end = pos + size; - if (nullptr != (pos = strchr(data, '/')) && pos[1] == '/' && nullptr != (pos = strchr(pos + 2, '/'))) + if (end != (pos = find_first_symbols<'/'>(pos, end)) && pos[1] == '/' && end != (pos = find_first_symbols<'/'>(pos + 2, end))) { res_data = pos; res_size = end - res_data; @@ -338,12 +339,12 @@ struct ExtractQueryString Pos pos = data; Pos end = pos + size; - if (nullptr != (pos = strchr(data, '?'))) + if (end != (pos = find_first_symbols<'?'>(pos, end))) { - Pos fragment = strchr(pos, '#'); + Pos fragment = find_first_symbols<'#'>(pos, end); res_data = pos + (without_leading_char ? 1 : 0); - res_size = (fragment ? fragment : end) - res_data; + res_size = fragment - res_data; } } }; @@ -361,7 +362,7 @@ struct ExtractFragment Pos pos = data; Pos end = pos + size; - if (nullptr != (pos = strchr(data, '#'))) + if (end != (pos = find_first_symbols<'#'>(pos, end))) { res_data = pos + (without_leading_char ? 1 : 0); res_size = end - res_data; @@ -382,12 +383,12 @@ struct ExtractQueryStringAndFragment Pos pos = data; Pos end = pos + size; - if (nullptr != (pos = strchr(data, '?'))) + if (end != (pos = find_first_symbols<'?'>(pos, end))) { res_data = pos + (without_leading_char ? 1 : 0); res_size = end - res_data; } - else if (nullptr != (pos = strchr(data, '#'))) + else if (end != (pos = find_first_symbols<'#'>(pos, end))) { res_data = pos; res_size = end - res_data; @@ -406,7 +407,7 @@ struct ExtractWWW Pos pos = data; Pos end = pos + size; - if (nullptr != (pos = strchr(pos, '/'))) + if (end != (pos = find_first_symbols<'/'>(pos, end))) { if (pos != data) { @@ -442,64 +443,76 @@ struct ExtractWWW struct ExtractURLParameterImpl { - static void vector(const ColumnString::Chars_t & data, - const ColumnString::Offsets & offsets, - std::string pattern, - ColumnString::Chars_t & res_data, ColumnString::Offsets & res_offsets) + static void vector(const ColumnString::Chars & data, + const ColumnString::Offsets & offsets, + std::string pattern, + ColumnString::Chars & res_data, ColumnString::Offsets & res_offsets) { - res_data.reserve(data.size() / 5); + res_data.reserve(data.size() / 5); res_offsets.resize(offsets.size()); pattern += '='; const char * param_str = pattern.c_str(); size_t param_len = pattern.size(); - size_t prev_offset = 0; - size_t res_offset = 0; + ColumnString::Offset prev_offset = 0; + ColumnString::Offset res_offset = 0; for (size_t i = 0; i < offsets.size(); ++i) { - size_t cur_offset = offsets[i]; + ColumnString::Offset cur_offset = offsets[i]; const char * str = reinterpret_cast(&data[prev_offset]); + const char * end = reinterpret_cast(&data[cur_offset]); - const char * pos = nullptr; - const char * begin = strpbrk(str, "?#"); - if (begin) + /// Find query string or fragment identifier. + /// Note that we support parameters in fragment identifier in the same way as in query string. + + const char * const query_string_begin = find_first_symbols<'?', '#'>(str, end); + + /// Will point to the beginning of "name=value" pair. Then it will be reassigned to the beginning of "value". + const char * param_begin = nullptr; + + if (query_string_begin + 1 < end) { - pos = begin + 1; + param_begin = query_string_begin + 1; + while (true) { - pos = strstr(pos, param_str); + param_begin = strstr(param_begin, param_str); - if (pos == nullptr) + if (!param_begin) break; - if (pos[-1] != '?' && pos[-1] != '#' && pos[-1] != '&') + if (param_begin[-1] != '?' && param_begin[-1] != '#' && param_begin[-1] != '&') { - pos += param_len; + /// Parameter name is different but has the same suffix. + param_begin += param_len; continue; } else { - pos += param_len; + param_begin += param_len; break; } } } - if (pos) + if (param_begin) { - const char * end = strpbrk(pos, "&#"); - if (end == nullptr) - end = pos + strlen(pos); + const char * param_end = find_first_symbols<'&', '#'>(param_begin, end); + if (param_end == end) + param_end = param_begin + strlen(param_begin); - res_data.resize(res_offset + (end - pos) + 1); - memcpySmallAllowReadWriteOverflow15(&res_data[res_offset], pos, end - pos); - res_offset += end - pos; + size_t param_size = param_end - param_begin; + + res_data.resize(res_offset + param_size + 1); + memcpySmallAllowReadWriteOverflow15(&res_data[res_offset], param_begin, param_size); + res_offset += param_size; } else { + /// No parameter found, put empty string in result. res_data.resize(res_offset + 1); } @@ -515,10 +528,10 @@ struct ExtractURLParameterImpl struct CutURLParameterImpl { - static void vector(const ColumnString::Chars_t & data, + static void vector(const ColumnString::Chars & data, const ColumnString::Offsets & offsets, std::string pattern, - ColumnString::Chars_t & res_data, ColumnString::Offsets & res_offsets) + ColumnString::Chars & res_data, ColumnString::Offsets & res_offsets) { res_data.reserve(data.size()); res_offsets.resize(offsets.size()); @@ -541,11 +554,11 @@ struct CutURLParameterImpl do { - const char * begin = strpbrk(url_begin, "?#"); - if (begin == nullptr) + const char * query_string_begin = find_first_symbols<'?', '#'>(url_begin, url_end); + if (query_string_begin == url_end) break; - const char * pos = strstr(begin + 1, param_str); + const char * pos = strstr(query_string_begin + 1, param_str); if (pos == nullptr) break; @@ -628,8 +641,8 @@ public: if (first) { first = false; - pos = strpbrk(pos, "?#"); - if (pos == nullptr) + pos = find_first_symbols<'?', '#'>(pos, end); + if (pos == end) return false; ++pos; } @@ -637,8 +650,8 @@ public: while (true) { token_begin = pos; - pos = strpbrk(pos, "=&#?"); - if (pos == nullptr) + pos = find_first_symbols<'=', '&', '#', '?'>(pos, end); + if (pos == end) return false; if (*pos == '?') @@ -657,8 +670,8 @@ public: else { ++pos; - pos = strpbrk(pos, "&#"); - if (pos == nullptr) + pos = find_first_symbols<'&', '#'>(pos, end); + if (pos == end) token_end = end; else token_end = pos++; @@ -713,12 +726,12 @@ public: if (first) { first = false; - pos = strpbrk(pos, "?#"); + pos = find_first_symbols<'?', '#'>(pos, end); } else - pos = strpbrk(pos, "&#"); + pos = find_first_symbols<'&', '#'>(pos, end); - if (pos == nullptr) + if (pos == end) return false; ++pos; @@ -726,8 +739,8 @@ public: { token_begin = pos; - pos = strpbrk(pos, "=&#?"); - if (pos == nullptr) + pos = find_first_symbols<'=', '&', '#', '?'>(pos, end); + if (pos == end) return false; else token_end = pos; @@ -936,8 +949,8 @@ public: template struct ExtractSubstringImpl { - static void vector(const ColumnString::Chars_t & data, const ColumnString::Offsets & offsets, - ColumnString::Chars_t & res_data, ColumnString::Offsets & res_offsets) + static void vector(const ColumnString::Chars & data, const ColumnString::Offsets & offsets, + ColumnString::Chars & res_data, ColumnString::Offsets & res_offsets) { size_t size = offsets.size(); res_offsets.resize(size); @@ -973,7 +986,7 @@ struct ExtractSubstringImpl res_data.assign(start, length); } - static void vector_fixed(const ColumnString::Chars_t &, size_t, ColumnString::Chars_t &) + static void vector_fixed(const ColumnString::Chars &, size_t, ColumnString::Chars &) { throw Exception("Column of type FixedString is not supported by URL functions", ErrorCodes::ILLEGAL_COLUMN); } @@ -985,8 +998,8 @@ struct ExtractSubstringImpl template struct CutSubstringImpl { - static void vector(const ColumnString::Chars_t & data, const ColumnString::Offsets & offsets, - ColumnString::Chars_t & res_data, ColumnString::Offsets & res_offsets) + static void vector(const ColumnString::Chars & data, const ColumnString::Offsets & offsets, + ColumnString::Chars & res_data, ColumnString::Offsets & res_offsets) { res_data.reserve(data.size()); size_t size = offsets.size(); @@ -1028,7 +1041,7 @@ struct CutSubstringImpl res_data.append(start + length, data.data() + data.size()); } - static void vector_fixed(const ColumnString::Chars_t &, size_t, ColumnString::Chars_t &) + static void vector_fixed(const ColumnString::Chars &, size_t, ColumnString::Chars &) { throw Exception("Column of type FixedString is not supported by URL functions", ErrorCodes::ILLEGAL_COLUMN); } @@ -1038,14 +1051,14 @@ struct CutSubstringImpl /// Percent decode of url data. struct DecodeURLComponentImpl { - static void vector(const ColumnString::Chars_t & data, const ColumnString::Offsets & offsets, - ColumnString::Chars_t & res_data, ColumnString::Offsets & res_offsets); + static void vector(const ColumnString::Chars & data, const ColumnString::Offsets & offsets, + ColumnString::Chars & res_data, ColumnString::Offsets & res_offsets); static void constant(const std::string & data, std::string & res_data); - static void vector_fixed(const ColumnString::Chars_t & data, size_t n, - ColumnString::Chars_t & res_data); + static void vector_fixed(const ColumnString::Chars & data, size_t n, + ColumnString::Chars & res_data); }; } diff --git a/dbms/src/Functions/FunctionsVisitParam.h b/dbms/src/Functions/FunctionsVisitParam.h index 13ae4002236..940ef877476 100644 --- a/dbms/src/Functions/FunctionsVisitParam.h +++ b/dbms/src/Functions/FunctionsVisitParam.h @@ -90,7 +90,7 @@ struct ExtractRaw static constexpr size_t bytes_on_stack = 64; using ExpectChars = PODArray, bytes_on_stack>>; - static void extract(const UInt8 * pos, const UInt8 * end, ColumnString::Chars_t & res_data) + static void extract(const UInt8 * pos, const UInt8 * end, ColumnString::Chars & res_data) { ExpectChars expects_end; UInt8 current_expect_end = 0; @@ -134,7 +134,7 @@ struct ExtractRaw struct ExtractString { - static void extract(const UInt8 * pos, const UInt8 * end, ColumnString::Chars_t & res_data) + static void extract(const UInt8 * pos, const UInt8 * end, ColumnString::Chars & res_data) { size_t old_size = res_data.size(); ReadBufferFromMemory in(pos, end - pos); @@ -158,7 +158,7 @@ struct ExtractParamImpl using ResultType = typename ParamExtractor::ResultType; /// It is assumed that `res` is the correct size and initialized with zeros. - static void vector_constant(const ColumnString::Chars_t & data, const ColumnString::Offsets & offsets, + static void vector_constant(const ColumnString::Chars & data, const ColumnString::Offsets & offsets, std::string needle, PaddedPODArray & res) { @@ -228,9 +228,9 @@ struct ExtractParamImpl template struct ExtractParamToStringImpl { - static void vector(const ColumnString::Chars_t & data, const ColumnString::Offsets & offsets, + static void vector(const ColumnString::Chars & data, const ColumnString::Offsets & offsets, std::string needle, - ColumnString::Chars_t & res_data, ColumnString::Offsets & res_offsets) + ColumnString::Chars & res_data, ColumnString::Offsets & res_offsets) { /// Constant 5 is taken from a function that performs a similar task FunctionsStringSearch.h::ExtractImpl res_data.reserve(data.size() / 5); diff --git a/dbms/src/Functions/GatherUtils/Sinks.h b/dbms/src/Functions/GatherUtils/Sinks.h index 0b92237c8bd..9d5467646e6 100644 --- a/dbms/src/Functions/GatherUtils/Sinks.h +++ b/dbms/src/Functions/GatherUtils/Sinks.h @@ -73,7 +73,7 @@ struct NumericArraySink : public ArraySinkImpl> struct StringSink { - typename ColumnString::Chars_t & elements; + typename ColumnString::Chars & elements; typename ColumnString::Offsets & offsets; size_t row_num = 0; @@ -112,7 +112,7 @@ struct StringSink struct FixedStringSink { - typename ColumnString::Chars_t & elements; + typename ColumnString::Chars & elements; size_t string_size; size_t row_num = 0; diff --git a/dbms/src/Functions/GatherUtils/Sources.h b/dbms/src/Functions/GatherUtils/Sources.h index 16e78bea26b..e076b9989d8 100644 --- a/dbms/src/Functions/GatherUtils/Sources.h +++ b/dbms/src/Functions/GatherUtils/Sources.h @@ -191,7 +191,7 @@ struct StringSource using Slice = NumericArraySlice; using Column = ColumnString; - const typename ColumnString::Chars_t & elements; + const typename ColumnString::Chars & elements; const typename ColumnString::Offsets & offsets; size_t row_num = 0; diff --git a/dbms/src/Functions/LowerUpperImpl.h b/dbms/src/Functions/LowerUpperImpl.h index 50806372188..5fb58587321 100644 --- a/dbms/src/Functions/LowerUpperImpl.h +++ b/dbms/src/Functions/LowerUpperImpl.h @@ -7,9 +7,9 @@ namespace DB template struct LowerUpperImpl { - static void vector(const ColumnString::Chars_t & data, + static void vector(const ColumnString::Chars & data, const ColumnString::Offsets & offsets, - ColumnString::Chars_t & res_data, + ColumnString::Chars & res_data, ColumnString::Offsets & res_offsets) { res_data.resize(data.size()); @@ -17,7 +17,7 @@ struct LowerUpperImpl array(data.data(), data.data() + data.size(), res_data.data()); } - static void vector_fixed(const ColumnString::Chars_t & data, size_t /*n*/, ColumnString::Chars_t & res_data) + static void vector_fixed(const ColumnString::Chars & data, size_t /*n*/, ColumnString::Chars & res_data) { res_data.resize(data.size()); array(data.data(), data.data() + data.size(), res_data.data()); diff --git a/dbms/src/Functions/LowerUpperUTF8Impl.h b/dbms/src/Functions/LowerUpperUTF8Impl.h index 8e67a7976b2..b6e9fab40cf 100644 --- a/dbms/src/Functions/LowerUpperUTF8Impl.h +++ b/dbms/src/Functions/LowerUpperUTF8Impl.h @@ -80,9 +80,9 @@ template static void vectorConst( - const ColumnString::Chars_t & data, const ColumnArray::Offsets & offsets, const ColumnString::Offsets & string_offsets, + const ColumnString::Chars & data, const ColumnArray::Offsets & offsets, const ColumnString::Offsets & string_offsets, const ColumnArray::Offset index, - ColumnString::Chars_t & result_data, ColumnArray::Offsets & result_offsets, + ColumnString::Chars & result_data, ColumnArray::Offsets & result_offsets, ArrayImpl::NullMapBuilder & builder) { size_t size = offsets.size(); @@ -279,9 +279,9 @@ struct ArrayElementStringImpl */ template static void vector( - const ColumnString::Chars_t & data, const ColumnArray::Offsets & offsets, const ColumnString::Offsets & string_offsets, + const ColumnString::Chars & data, const ColumnArray::Offsets & offsets, const ColumnString::Offsets & string_offsets, const PaddedPODArray & indices, - ColumnString::Chars_t & result_data, ColumnArray::Offsets & result_offsets, + ColumnString::Chars & result_data, ColumnArray::Offsets & result_offsets, ArrayImpl::NullMapBuilder & builder) { size_t size = offsets.size(); diff --git a/dbms/src/Functions/arrayIndex.h b/dbms/src/Functions/arrayIndex.h index f3aa4b47022..14429a1a4a5 100644 --- a/dbms/src/Functions/arrayIndex.h +++ b/dbms/src/Functions/arrayIndex.h @@ -278,7 +278,7 @@ template struct ArrayIndexStringNullImpl { static void vector_const( - const ColumnString::Chars_t & /*data*/, const ColumnArray::Offsets & offsets, const ColumnString::Offsets & /*string_offsets*/, + const ColumnString::Chars & /*data*/, const ColumnArray::Offsets & offsets, const ColumnString::Offsets & /*string_offsets*/, PaddedPODArray & result, const PaddedPODArray * null_map_data) { @@ -310,7 +310,7 @@ template struct ArrayIndexStringImpl { static void vector_const( - const ColumnString::Chars_t & data, const ColumnArray::Offsets & offsets, const ColumnString::Offsets & string_offsets, + const ColumnString::Chars & data, const ColumnArray::Offsets & offsets, const ColumnString::Offsets & string_offsets, const String & value, PaddedPODArray & result, const PaddedPODArray * null_map_data) @@ -349,8 +349,8 @@ struct ArrayIndexStringImpl } static void vector_vector( - const ColumnString::Chars_t & data, const ColumnArray::Offsets & offsets, const ColumnString::Offsets & string_offsets, - const ColumnString::Chars_t & item_values, const ColumnString::Offsets & item_offsets, + const ColumnString::Chars & data, const ColumnArray::Offsets & offsets, const ColumnString::Offsets & string_offsets, + const ColumnString::Chars & item_values, const ColumnString::Offsets & item_offsets, PaddedPODArray & result, const PaddedPODArray * null_map_data, const PaddedPODArray * null_map_item) diff --git a/dbms/src/Functions/arrayReverse.cpp b/dbms/src/Functions/arrayReverse.cpp index 83f4cedbe49..19d294b97cd 100644 --- a/dbms/src/Functions/arrayReverse.cpp +++ b/dbms/src/Functions/arrayReverse.cpp @@ -165,8 +165,8 @@ bool FunctionArrayReverse::executeFixedString(const IColumn & src_data, const Co if (const ColumnFixedString * src_data_concrete = checkAndGetColumn(&src_data)) { const size_t n = src_data_concrete->getN(); - const ColumnFixedString::Chars_t & src_data = src_data_concrete->getChars(); - ColumnFixedString::Chars_t & res_chars = typeid_cast(res_data).getChars(); + const ColumnFixedString::Chars & src_data = src_data_concrete->getChars(); + ColumnFixedString::Chars & res_chars = typeid_cast(res_data).getChars(); size_t size = src_offsets.size(); res_chars.resize(src_data.size()); @@ -205,8 +205,8 @@ bool FunctionArrayReverse::executeString(const IColumn & src_data, const ColumnA const ColumnString::Offsets & src_string_offsets = src_data_concrete->getOffsets(); ColumnString::Offsets & res_string_offsets = typeid_cast(res_data).getOffsets(); - const ColumnString::Chars_t & src_data = src_data_concrete->getChars(); - ColumnString::Chars_t & res_chars = typeid_cast(res_data).getChars(); + const ColumnString::Chars & src_data = src_data_concrete->getChars(); + ColumnString::Chars & res_chars = typeid_cast(res_data).getChars(); size_t size = src_array_offsets.size(); res_string_offsets.resize(src_string_offsets.size()); diff --git a/dbms/src/Functions/bar.cpp b/dbms/src/Functions/bar.cpp index 6cb40154ad7..4d164ab5099 100644 --- a/dbms/src/Functions/bar.cpp +++ b/dbms/src/Functions/bar.cpp @@ -117,7 +117,7 @@ private: template static void fill(const PaddedPODArray & src, - ColumnString::Chars_t & dst_chars, + ColumnString::Chars & dst_chars, ColumnString::Offsets & dst_offsets, Int64 min, Int64 max, diff --git a/dbms/src/Functions/convertCharset.cpp b/dbms/src/Functions/convertCharset.cpp index e4097b8a6f3..a824369b455 100644 --- a/dbms/src/Functions/convertCharset.cpp +++ b/dbms/src/Functions/convertCharset.cpp @@ -86,8 +86,8 @@ private: } void convert(const String & from_charset, const String & to_charset, - const ColumnString::Chars_t & from_chars, const ColumnString::Offsets & from_offsets, - ColumnString::Chars_t & to_chars, ColumnString::Offsets & to_offsets) + const ColumnString::Chars & from_chars, const ColumnString::Offsets & from_offsets, + ColumnString::Chars & to_chars, ColumnString::Offsets & to_offsets) { auto converter_from = getConverter(from_charset); auto converter_to = getConverter(to_charset); diff --git a/dbms/src/Functions/dateDiff.cpp b/dbms/src/Functions/dateDiff.cpp index 859896ad863..bb309f0d8b9 100644 --- a/dbms/src/Functions/dateDiff.cpp +++ b/dbms/src/Functions/dateDiff.cpp @@ -11,7 +11,7 @@ #include -#include +#include #include diff --git a/dbms/src/Functions/emptyArrayToSingle.cpp b/dbms/src/Functions/emptyArrayToSingle.cpp index 8ca18049c87..4ef54f6665a 100644 --- a/dbms/src/Functions/emptyArrayToSingle.cpp +++ b/dbms/src/Functions/emptyArrayToSingle.cpp @@ -142,13 +142,13 @@ namespace if (const ColumnFixedString * src_data_concrete = checkAndGetColumn(&src_data)) { const size_t n = src_data_concrete->getN(); - const ColumnFixedString::Chars_t & src_data = src_data_concrete->getChars(); + const ColumnFixedString::Chars & src_data = src_data_concrete->getChars(); auto concrete_res_data = typeid_cast(&res_data_col); if (!concrete_res_data) throw Exception{"Internal error", ErrorCodes::LOGICAL_ERROR}; - ColumnFixedString::Chars_t & res_data = concrete_res_data->getChars(); + ColumnFixedString::Chars & res_data = concrete_res_data->getChars(); size_t size = src_offsets.size(); res_offsets.resize(size); res_data.reserve(src_data.size()); @@ -215,12 +215,12 @@ namespace throw Exception{"Internal error", ErrorCodes::LOGICAL_ERROR}; ColumnString::Offsets & res_string_offsets = concrete_res_string_offsets->getOffsets(); - const ColumnString::Chars_t & src_data = src_data_concrete->getChars(); + const ColumnString::Chars & src_data = src_data_concrete->getChars(); auto concrete_res_data = typeid_cast(&res_data_col); if (!concrete_res_data) throw Exception{"Internal error", ErrorCodes::LOGICAL_ERROR}; - ColumnString::Chars_t & res_data = concrete_res_data->getChars(); + ColumnString::Chars & res_data = concrete_res_data->getChars(); size_t size = src_array_offsets.size(); res_array_offsets.resize(size); diff --git a/dbms/src/Functions/formatDateTime.cpp b/dbms/src/Functions/formatDateTime.cpp index d9d76ab936c..e242f70bd58 100644 --- a/dbms/src/Functions/formatDateTime.cpp +++ b/dbms/src/Functions/formatDateTime.cpp @@ -10,7 +10,7 @@ #include #include -#include +#include #include diff --git a/dbms/src/Functions/length.cpp b/dbms/src/Functions/length.cpp index 4d3a0dd597c..24a2984abe2 100644 --- a/dbms/src/Functions/length.cpp +++ b/dbms/src/Functions/length.cpp @@ -13,19 +13,19 @@ struct LengthImpl { static constexpr auto is_fixed_to_constant = true; - static void vector(const ColumnString::Chars_t & /*data*/, const ColumnString::Offsets & offsets, PaddedPODArray & res) + static void vector(const ColumnString::Chars & /*data*/, const ColumnString::Offsets & offsets, PaddedPODArray & res) { size_t size = offsets.size(); for (size_t i = 0; i < size; ++i) res[i] = i == 0 ? (offsets[i] - 1) : (offsets[i] - 1 - offsets[i - 1]); } - static void vector_fixed_to_constant(const ColumnString::Chars_t & /*data*/, size_t n, UInt64 & res) + static void vector_fixed_to_constant(const ColumnString::Chars & /*data*/, size_t n, UInt64 & res) { res = n; } - static void vector_fixed_to_vector(const ColumnString::Chars_t & /*data*/, size_t /*n*/, PaddedPODArray & /*res*/) + static void vector_fixed_to_vector(const ColumnString::Chars & /*data*/, size_t /*n*/, PaddedPODArray & /*res*/) { } diff --git a/dbms/src/Functions/lengthUTF8.cpp b/dbms/src/Functions/lengthUTF8.cpp index 5f3b2c9969e..ba2214fc9d7 100644 --- a/dbms/src/Functions/lengthUTF8.cpp +++ b/dbms/src/Functions/lengthUTF8.cpp @@ -22,7 +22,7 @@ struct LengthUTF8Impl { static constexpr auto is_fixed_to_constant = false; - static void vector(const ColumnString::Chars_t & data, const ColumnString::Offsets & offsets, PaddedPODArray & res) + static void vector(const ColumnString::Chars & data, const ColumnString::Offsets & offsets, PaddedPODArray & res) { size_t size = offsets.size(); @@ -34,11 +34,11 @@ struct LengthUTF8Impl } } - static void vector_fixed_to_constant(const ColumnString::Chars_t & /*data*/, size_t /*n*/, UInt64 & /*res*/) + static void vector_fixed_to_constant(const ColumnString::Chars & /*data*/, size_t /*n*/, UInt64 & /*res*/) { } - static void vector_fixed_to_vector(const ColumnString::Chars_t & data, size_t n, PaddedPODArray & res) + static void vector_fixed_to_vector(const ColumnString::Chars & data, size_t n, PaddedPODArray & res) { size_t size = data.size() / n; diff --git a/dbms/src/Functions/reverse.cpp b/dbms/src/Functions/reverse.cpp index e9327f58f61..065e1d28073 100644 --- a/dbms/src/Functions/reverse.cpp +++ b/dbms/src/Functions/reverse.cpp @@ -21,9 +21,9 @@ namespace ErrorCodes */ struct ReverseImpl { - static void vector(const ColumnString::Chars_t & data, + static void vector(const ColumnString::Chars & data, const ColumnString::Offsets & offsets, - ColumnString::Chars_t & res_data, + ColumnString::Chars & res_data, ColumnString::Offsets & res_offsets) { res_data.resize(data.size()); @@ -40,7 +40,7 @@ struct ReverseImpl } } - static void vector_fixed(const ColumnString::Chars_t & data, size_t n, ColumnString::Chars_t & res_data) + static void vector_fixed(const ColumnString::Chars & data, size_t n, ColumnString::Chars & res_data) { res_data.resize(data.size()); size_t size = data.size() / n; diff --git a/dbms/src/Functions/reverseUTF8.cpp b/dbms/src/Functions/reverseUTF8.cpp index 756a959141c..188b48bf7dc 100644 --- a/dbms/src/Functions/reverseUTF8.cpp +++ b/dbms/src/Functions/reverseUTF8.cpp @@ -19,9 +19,9 @@ namespace ErrorCodes */ struct ReverseUTF8Impl { - static void vector(const ColumnString::Chars_t & data, + static void vector(const ColumnString::Chars & data, const ColumnString::Offsets & offsets, - ColumnString::Chars_t & res_data, + ColumnString::Chars & res_data, ColumnString::Offsets & res_offsets) { res_data.resize(data.size()); @@ -61,7 +61,7 @@ struct ReverseUTF8Impl } } - static void vector_fixed(const ColumnString::Chars_t &, size_t, ColumnString::Chars_t &) + static void vector_fixed(const ColumnString::Chars &, size_t, ColumnString::Chars &) { throw Exception("Cannot apply function reverseUTF8 to fixed string.", ErrorCodes::ILLEGAL_COLUMN); } diff --git a/dbms/src/Functions/substringUTF8.cpp b/dbms/src/Functions/substringUTF8.cpp index 3e384bcba04..c66ef79bae3 100644 --- a/dbms/src/Functions/substringUTF8.cpp +++ b/dbms/src/Functions/substringUTF8.cpp @@ -21,11 +21,11 @@ namespace ErrorCodes */ struct SubstringUTF8Impl { - static void vector(const ColumnString::Chars_t & data, + static void vector(const ColumnString::Chars & data, const ColumnString::Offsets & offsets, size_t start, size_t length, - ColumnString::Chars_t & res_data, + ColumnString::Chars & res_data, ColumnString::Offsets & res_offsets) { res_data.reserve(data.size()); diff --git a/dbms/src/IO/ReadHelpers.cpp b/dbms/src/IO/ReadHelpers.cpp index 786d566a815..45c4185eff0 100644 --- a/dbms/src/IO/ReadHelpers.cpp +++ b/dbms/src/IO/ReadHelpers.cpp @@ -7,7 +7,7 @@ #include #include #include -#include +#include #include #include diff --git a/dbms/src/IO/WriteHelpers.h b/dbms/src/IO/WriteHelpers.h index 67f80819127..68a26791dca 100644 --- a/dbms/src/IO/WriteHelpers.h +++ b/dbms/src/IO/WriteHelpers.h @@ -9,7 +9,7 @@ #include #include #include -#include +#include #include #include diff --git a/dbms/src/IO/tests/mempbrk.cpp b/dbms/src/IO/tests/mempbrk.cpp index a871a521108..85b3e2d89ee 100644 --- a/dbms/src/IO/tests/mempbrk.cpp +++ b/dbms/src/IO/tests/mempbrk.cpp @@ -10,7 +10,7 @@ #include #include -#include +#include namespace DB { diff --git a/dbms/src/Interpreters/Aggregator.h b/dbms/src/Interpreters/Aggregator.h index 5e0972dd3f1..0dadf08b8a2 100644 --- a/dbms/src/Interpreters/Aggregator.h +++ b/dbms/src/Interpreters/Aggregator.h @@ -201,7 +201,7 @@ struct AggregationMethodString struct State { const ColumnString::Offsets * offsets; - const ColumnString::Chars_t * chars; + const ColumnString::Chars * chars; void init(ColumnRawPtrs & key_columns) { @@ -272,7 +272,7 @@ struct AggregationMethodFixedString struct State { size_t n; - const ColumnFixedString::Chars_t * chars; + const ColumnFixedString::Chars * chars; void init(ColumnRawPtrs & key_columns) { diff --git a/dbms/src/Interpreters/Join.h b/dbms/src/Interpreters/Join.h index 38527aa3cec..7c288e0a008 100644 --- a/dbms/src/Interpreters/Join.h +++ b/dbms/src/Interpreters/Join.h @@ -57,7 +57,7 @@ struct JoinKeyGetterString using Key = StringRef; const ColumnString::Offsets * offsets; - const ColumnString::Chars_t * chars; + const ColumnString::Chars * chars; JoinKeyGetterString(const ColumnRawPtrs & key_columns) { @@ -90,7 +90,7 @@ struct JoinKeyGetterFixedString using Key = StringRef; size_t n; - const ColumnFixedString::Chars_t * chars; + const ColumnFixedString::Chars * chars; JoinKeyGetterFixedString(const ColumnRawPtrs & key_columns) { diff --git a/dbms/src/Interpreters/SetVariants.h b/dbms/src/Interpreters/SetVariants.h index 7d5c963b728..e7f731a12e6 100644 --- a/dbms/src/Interpreters/SetVariants.h +++ b/dbms/src/Interpreters/SetVariants.h @@ -68,7 +68,7 @@ struct SetMethodString struct State { const ColumnString::Offsets * offsets; - const ColumnString::Chars_t * chars; + const ColumnString::Chars * chars; void init(const ColumnRawPtrs & key_columns) { @@ -108,7 +108,7 @@ struct SetMethodFixedString struct State { size_t n; - const ColumnFixedString::Chars_t * chars; + const ColumnFixedString::Chars * chars; void init(const ColumnRawPtrs & key_columns) { diff --git a/dbms/src/Parsers/CommonParsers.cpp b/dbms/src/Parsers/CommonParsers.cpp index 2eee0f276ee..ddbf1b17966 100644 --- a/dbms/src/Parsers/CommonParsers.cpp +++ b/dbms/src/Parsers/CommonParsers.cpp @@ -1,6 +1,6 @@ #include #include -#include +#include #include /// strncmp, strncasecmp diff --git a/dbms/src/Parsers/Lexer.cpp b/dbms/src/Parsers/Lexer.cpp index d5f1de558de..b3779512ef7 100644 --- a/dbms/src/Parsers/Lexer.cpp +++ b/dbms/src/Parsers/Lexer.cpp @@ -1,6 +1,6 @@ #include #include -#include +#include namespace DB diff --git a/dbms/src/Parsers/parseQuery.cpp b/dbms/src/Parsers/parseQuery.cpp index 7449d7e8511..f40d156f17d 100644 --- a/dbms/src/Parsers/parseQuery.cpp +++ b/dbms/src/Parsers/parseQuery.cpp @@ -6,7 +6,7 @@ #include #include #include -#include +#include #include #include #include diff --git a/libs/libcommon/CMakeLists.txt b/libs/libcommon/CMakeLists.txt index 2754ecb9539..7d9da65aa72 100644 --- a/libs/libcommon/CMakeLists.txt +++ b/libs/libcommon/CMakeLists.txt @@ -42,6 +42,7 @@ add_library (common ${LINK_MODE} include/common/ThreadPool.h include/common/demangle.h include/common/SetTerminalEcho.h + include/common/find_symbols.h include/ext/bit_cast.h include/ext/collection_cast.h diff --git a/libs/libcommon/include/common/find_first_symbols.h b/libs/libcommon/include/common/find_symbols.h similarity index 76% rename from libs/libcommon/include/common/find_first_symbols.h rename to libs/libcommon/include/common/find_symbols.h index a96d5763a42..8ea09eb37df 100644 --- a/libs/libcommon/include/common/find_first_symbols.h +++ b/libs/libcommon/include/common/find_symbols.h @@ -10,7 +10,9 @@ #endif -/** Allow to search for next character from the set of 'symbols...' in a string. +/** find_first_symbols(begin, end): + * + * Allow to search for next character from the set of 'symbols...' in a string. * It is similar to 'strpbrk', 'strcspn' (and 'strchr', 'memchr' in the case of one symbol and '\0'), * but with the following differencies: * - works with any memory ranges, including containing zero bytes; @@ -24,6 +26,11 @@ * In the case of parsing tab separated dump with short strings, there is no performance degradation over trivial loop. * * Note: the optimal threshold to choose between SSE 2 and SSE 4.2 may depend on CPU model. + * + * find_last_symbols_or_null(begin, end): + * + * Allow to search for the last matching character in a string. + * If no such characters, returns nullptr. */ namespace detail @@ -82,6 +89,31 @@ inline const char * find_first_symbols_sse2(const char * begin, const char * end } +template +inline const char * find_last_symbols_or_null_sse2(const char * begin, const char * end) +{ +#if __SSE2__ + for (; end - 16 >= begin; end -= 16) /// Assuming the pointer cannot overflow. Assuming we can compare these pointers. + { + __m128i bytes = _mm_loadu_si128(reinterpret_cast(end - 16)); + + __m128i eq = mm_is_in(bytes); + + uint16_t bit_mask = _mm_movemask_epi8(eq); + if (bit_mask) + return end - 1 - (__builtin_clz(bit_mask) - 16); /// because __builtin_clz works with mask as uint32. + } +#endif + + --end; + for (; end >= begin; --end) + if (is_in(*end)) + return end; + + return nullptr; +} + + template (begin, end); } +/// NOTE No SSE 4.2 implementation for find_last_symbols_or_null. Not worth to do. + template inline const char * find_first_symbols_dispatch(const char * begin, const char * end) { @@ -158,3 +192,16 @@ inline char * find_first_symbols(char * begin, char * end) { return const_cast(detail::find_first_symbols_dispatch(begin, end)); } + + +template +inline const char * find_last_symbols_or_null(const char * begin, const char * end) +{ + return detail::find_last_symbols_or_null_sse2(begin, end); +} + +template +inline char * find_last_symbols_or_null(char * begin, char * end) +{ + return const_cast(detail::find_last_symbols_or_null_sse2(begin, end)); +} diff --git a/libs/libcommon/src/JSON.cpp b/libs/libcommon/src/JSON.cpp index 8a27499a7a9..fcc23bd9b7b 100644 --- a/libs/libcommon/src/JSON.cpp +++ b/libs/libcommon/src/JSON.cpp @@ -4,7 +4,7 @@ #include #include #include -#include +#include #include #include diff --git a/libs/libcommon/src/tests/CMakeLists.txt b/libs/libcommon/src/tests/CMakeLists.txt index 8e4f52d135a..86f15cd7a53 100644 --- a/libs/libcommon/src/tests/CMakeLists.txt +++ b/libs/libcommon/src/tests/CMakeLists.txt @@ -20,7 +20,7 @@ target_link_libraries (local_date_time_comparison common) add_check(multi_version) add_check(local_date_time_comparison) -add_executable (unit_tests_libcommon gtest_json_test.cpp gtest_strong_typedef.cpp) +add_executable (unit_tests_libcommon gtest_json_test.cpp gtest_strong_typedef.cpp gtest_find_symbols.cpp) target_link_libraries (unit_tests_libcommon common ${GTEST_MAIN_LIBRARIES}) add_check(unit_tests_libcommon) diff --git a/libs/libcommon/src/tests/gtest_find_symbols.cpp b/libs/libcommon/src/tests/gtest_find_symbols.cpp new file mode 100644 index 00000000000..118a1f5c178 --- /dev/null +++ b/libs/libcommon/src/tests/gtest_find_symbols.cpp @@ -0,0 +1,25 @@ +#include +#include +#include + + +TEST(find_symbols, SimpleTest) +{ + std::string s = "Hello, world! Goodbye..."; + const char * begin = s.data(); + const char * end = s.data() + s.size(); + + ASSERT_EQ(find_first_symbols<'a'>(begin, end), end); + ASSERT_EQ(find_first_symbols<'e'>(begin, end), begin + 1); + ASSERT_EQ(find_first_symbols<'.'>(begin, end), begin + 21); + ASSERT_EQ(find_first_symbols<' '>(begin, end), begin + 6); + ASSERT_EQ(find_first_symbols<'H'>(begin, end), begin); + ASSERT_EQ((find_first_symbols<'a', 'e'>(begin, end)), begin + 1); + + ASSERT_EQ(find_last_symbols_or_null<'a'>(begin, end), nullptr); + ASSERT_EQ(find_last_symbols_or_null<'e'>(begin, end), end - 4); + ASSERT_EQ(find_last_symbols_or_null<'.'>(begin, end), end - 1); + ASSERT_EQ(find_last_symbols_or_null<' '>(begin, end), end - 11); + ASSERT_EQ(find_last_symbols_or_null<'H'>(begin, end), begin); + ASSERT_EQ((find_last_symbols_or_null<'a', 'e'>(begin, end)), end - 4); +} From af9ad0f8ae6bb968838e86830f6bd50f0e234d97 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 25 Nov 2018 03:15:43 +0300 Subject: [PATCH 105/124] Fixed bad code [#CLICKHOUSE-2] --- dbms/src/Functions/FunctionsURL.h | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/dbms/src/Functions/FunctionsURL.h b/dbms/src/Functions/FunctionsURL.h index cfd5f11406d..639f4d850e3 100644 --- a/dbms/src/Functions/FunctionsURL.h +++ b/dbms/src/Functions/FunctionsURL.h @@ -555,7 +555,7 @@ struct CutURLParameterImpl do { const char * query_string_begin = find_first_symbols<'?', '#'>(url_begin, url_end); - if (query_string_begin == url_end) + if (query_string_begin + 1 >= url_end) break; const char * pos = strstr(query_string_begin + 1, param_str); @@ -642,7 +642,7 @@ public: { first = false; pos = find_first_symbols<'?', '#'>(pos, end); - if (pos == end) + if (pos + 1 >= end) return false; ++pos; } @@ -731,7 +731,7 @@ public: else pos = find_first_symbols<'&', '#'>(pos, end); - if (pos == end) + if (pos + 1 >= end) return false; ++pos; From b53c1ea913d7e2ea70817a1451f7d00b3b96f31e Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 25 Nov 2018 03:16:14 +0300 Subject: [PATCH 106/124] Removed useless include [#CLICKHOUSE-2] --- dbms/src/Functions/FunctionsVisitParam.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/dbms/src/Functions/FunctionsVisitParam.cpp b/dbms/src/Functions/FunctionsVisitParam.cpp index 650b569bc2a..8d82f93a3db 100644 --- a/dbms/src/Functions/FunctionsVisitParam.cpp +++ b/dbms/src/Functions/FunctionsVisitParam.cpp @@ -1,7 +1,6 @@ #include #include #include -#include namespace DB From 5a045ff11c32567545be05c8da3498e86ae2a492 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 25 Nov 2018 03:31:24 +0300 Subject: [PATCH 107/124] Fixed error in URL functions [#CLICKHOUSE-2] --- dbms/src/Functions/FunctionsURL.h | 5 +++-- .../0_stateless/00760_url_functions_overflow.reference | 6 ++++++ .../queries/0_stateless/00760_url_functions_overflow.sql | 6 ++++++ dbms/tests/queries/bugs/fuzzy.sql | 3 --- 4 files changed, 15 insertions(+), 5 deletions(-) create mode 100644 dbms/tests/queries/0_stateless/00760_url_functions_overflow.reference create mode 100644 dbms/tests/queries/0_stateless/00760_url_functions_overflow.sql diff --git a/dbms/src/Functions/FunctionsURL.h b/dbms/src/Functions/FunctionsURL.h index 639f4d850e3..634f6db5d56 100644 --- a/dbms/src/Functions/FunctionsURL.h +++ b/dbms/src/Functions/FunctionsURL.h @@ -9,6 +9,7 @@ #include #include #include +#include namespace DB @@ -479,7 +480,7 @@ struct ExtractURLParameterImpl while (true) { - param_begin = strstr(param_begin, param_str); + param_begin = static_cast(memmem(param_begin, end - param_begin, param_str, param_len)); if (!param_begin) break; @@ -558,7 +559,7 @@ struct CutURLParameterImpl if (query_string_begin + 1 >= url_end) break; - const char * pos = strstr(query_string_begin + 1, param_str); + const char * pos = static_cast(memmem(query_string_begin + 1, url_end - query_string_begin - 1, param_str, param_len)); if (pos == nullptr) break; diff --git a/dbms/tests/queries/0_stateless/00760_url_functions_overflow.reference b/dbms/tests/queries/0_stateless/00760_url_functions_overflow.reference new file mode 100644 index 00000000000..4c02c52230c --- /dev/null +++ b/dbms/tests/queries/0_stateless/00760_url_functions_overflow.reference @@ -0,0 +1,6 @@ + + +Hello + + +Xx|sfF 1 [] C,Ai?X \0 diff --git a/dbms/tests/queries/0_stateless/00760_url_functions_overflow.sql b/dbms/tests/queries/0_stateless/00760_url_functions_overflow.sql new file mode 100644 index 00000000000..f4ab4b55520 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00760_url_functions_overflow.sql @@ -0,0 +1,6 @@ +SELECT extractURLParameter('?_', '\0_________________________________'); +SELECT extractURLParameter('?abc=def', 'abc\0def'); +SELECT extractURLParameter('?abc\0def=Hello', 'abc\0def'); +SELECT extractURLParameter('?_', '\0'); +SELECT extractURLParameter('ZiqSZeh?', '\0'); +SELECT 'Xx|sfF', match('', '\0'), [], ( SELECT cutURLParameter('C,Ai?X', '\0') ), '\0'; diff --git a/dbms/tests/queries/bugs/fuzzy.sql b/dbms/tests/queries/bugs/fuzzy.sql index e90b0592e79..9216d91219a 100644 --- a/dbms/tests/queries/bugs/fuzzy.sql +++ b/dbms/tests/queries/bugs/fuzzy.sql @@ -2,9 +2,6 @@ SELECT __inner_restore_projection__(2.0885, -66.72488); SELECT __inner_restore_projection__(-4, ''); SELECT __inner_restore_projection__(067274, 'vb\s'); SELECT sequenceCount((CAST((( SELECT NULL ) AS rg, ( SELECT ( SELECT [], 'A') AS String))]]); From eac2d032bb58c9633d355a7723f7beaa7d11e53c Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 25 Nov 2018 04:16:48 +0300 Subject: [PATCH 108/124] Better performance test [#CLICKHOUSE-2] --- dbms/tests/performance/url/url_hits.xml | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/dbms/tests/performance/url/url_hits.xml b/dbms/tests/performance/url/url_hits.xml index ccfd094ad5b..db7d4b3e830 100644 --- a/dbms/tests/performance/url/url_hits.xml +++ b/dbms/tests/performance/url/url_hits.xml @@ -1,6 +1,6 @@ url_hits - once + loop hits_100m_single @@ -17,7 +17,7 @@ - + From 68e0a687c850be02cd668a4f4fde7014afb304bb Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 25 Nov 2018 04:24:03 +0300 Subject: [PATCH 109/124] Removed unused method [#CLICKHOUSE-2] --- dbms/src/Functions/LowerUpperUTF8Impl.h | 8 -------- 1 file changed, 8 deletions(-) diff --git a/dbms/src/Functions/LowerUpperUTF8Impl.h b/dbms/src/Functions/LowerUpperUTF8Impl.h index b6e9fab40cf..53b2b9506c8 100644 --- a/dbms/src/Functions/LowerUpperUTF8Impl.h +++ b/dbms/src/Functions/LowerUpperUTF8Impl.h @@ -96,14 +96,6 @@ struct LowerUpperUTF8Impl array(data.data(), data.data() + data.size(), res_data.data()); } - static void constant(const std::string & data, std::string & res_data) - { - res_data.resize(data.size()); - array(reinterpret_cast(data.data()), - reinterpret_cast(data.data() + data.size()), - reinterpret_cast(res_data.data())); - } - /** Converts a single code point starting at `src` to desired case, storing result starting at `dst`. * `src` and `dst` are incremented by corresponding sequence lengths. */ static void toCase(const UInt8 *& src, const UInt8 * src_end, UInt8 *& dst) From 4cb7f2896c7867a0244bca0034a6b73339cf6d88 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 26 Nov 2018 01:26:36 +0300 Subject: [PATCH 110/124] Fixed error in lowerUTF8 and upperUTF8 functions [#CLICKHOUSE-2] --- dbms/src/Functions/LowerUpperUTF8Impl.h | 41 +++++++++++++------ .../00761_lower_utf8_bug.reference | 1 + .../0_stateless/00761_lower_utf8_bug.sql | 1 + dbms/tests/queries/bugs/fuzzy.sql | 3 -- 4 files changed, 31 insertions(+), 15 deletions(-) create mode 100644 dbms/tests/queries/0_stateless/00761_lower_utf8_bug.reference create mode 100644 dbms/tests/queries/0_stateless/00761_lower_utf8_bug.sql diff --git a/dbms/src/Functions/LowerUpperUTF8Impl.h b/dbms/src/Functions/LowerUpperUTF8Impl.h index 53b2b9506c8..dd1e87efead 100644 --- a/dbms/src/Functions/LowerUpperUTF8Impl.h +++ b/dbms/src/Functions/LowerUpperUTF8Impl.h @@ -1,5 +1,6 @@ #include #include +#include #if __SSE2__ #include @@ -9,6 +10,11 @@ namespace DB { +namespace ErrorCodes +{ + extern const int BAD_ARGUMENTS; +} + namespace { /// xor or do nothing @@ -90,10 +96,9 @@ struct LowerUpperUTF8Impl array(data.data(), data.data() + data.size(), res_data.data()); } - static void vector_fixed(const ColumnString::Chars & data, size_t /*n*/, ColumnString::Chars & res_data) + static void vector_fixed(const ColumnString::Chars &, size_t, ColumnString::Chars &) { - res_data.resize(data.size()); - array(data.data(), data.data() + data.size(), res_data.data()); + throw Exception("Functions lowerUTF8 and upperUTF8 cannot work with FixedString argument", ErrorCodes::BAD_ARGUMENTS); } /** Converts a single code point starting at `src` to desired case, storing result starting at `dst`. @@ -129,16 +134,28 @@ struct LowerUpperUTF8Impl { static const Poco::UTF8Encoding utf8; - if (const auto chars = utf8.convert(to_case(utf8.convert(src)), dst, src_end - src)) + int src_sequence_length = UTF8::seqLength(*src); + + int src_code_point = utf8.queryConvert(src, src_end - src); + if (src_code_point > 0) { - src += chars; - dst += chars; - } - else - { - ++src; - ++dst; + int dst_code_point = to_case(src_code_point); + if (dst_code_point > 0) + { + int dst_sequence_length = utf8.convert(dst_code_point, dst, src_end - src); + + /// We don't support cases when lowercase and uppercase characters occupy different number of bytes in UTF-8. + /// As an example, this happens for ß and ẞ. + if (dst_sequence_length == src_sequence_length) + { + src += dst_sequence_length; + dst += dst_sequence_length; + return; + } + } } + + *dst++ = *src++; } } @@ -149,7 +166,7 @@ private: static void array(const UInt8 * src, const UInt8 * src_end, UInt8 * dst) { #if __SSE2__ - const auto bytes_sse = sizeof(__m128i); + static constexpr auto bytes_sse = sizeof(__m128i); auto src_end_sse = src + (src_end - src) / bytes_sse * bytes_sse; /// SSE2 packed comparison operate on signed types, hence compare (c < 0) instead of (c > 0x7f) diff --git a/dbms/tests/queries/0_stateless/00761_lower_utf8_bug.reference b/dbms/tests/queries/0_stateless/00761_lower_utf8_bug.reference new file mode 100644 index 00000000000..d00491fd7e5 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00761_lower_utf8_bug.reference @@ -0,0 +1 @@ +1 diff --git a/dbms/tests/queries/0_stateless/00761_lower_utf8_bug.sql b/dbms/tests/queries/0_stateless/00761_lower_utf8_bug.sql new file mode 100644 index 00000000000..de20b894331 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00761_lower_utf8_bug.sql @@ -0,0 +1 @@ +SELECT lowerUTF8('\xF0') = lowerUTF8('\xF0'); diff --git a/dbms/tests/queries/bugs/fuzzy.sql b/dbms/tests/queries/bugs/fuzzy.sql index 9216d91219a..5712498a555 100644 --- a/dbms/tests/queries/bugs/fuzzy.sql +++ b/dbms/tests/queries/bugs/fuzzy.sql @@ -6,9 +6,6 @@ SELECT globalNotIn(['"wh'], [NULL]); SELECT globalIn([''], [NULL]) SELECT ( SELECT toDecimal128([], rowNumberInBlock()) ) , lcm('', [[(CAST(('>A') AS String))]]); SELECT truncate(895, -16); -SELECT (CAST((lowerUTF8('a7\xwK>-')) AS String)), [6935]; -SELECT upperUTF8(sipHash128('\0')), [], ['xD2jG']; -SELECT upperUTF8(SHA256('')); SELECT arrayEnumerateUniq(anyHeavy([]), []); SELECT notIn([['']], [[NULL]]); SELECT subtractDays((CAST((-5263074.47) AS DateTime)), -737895); From ed48b3845472257bb3148329d305741aeb1f3832 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 26 Nov 2018 02:23:01 +0300 Subject: [PATCH 111/124] Added two more bugs [#CLICKHOUSE-2] --- dbms/tests/queries/bugs/position_case_insensitive_utf8.sql | 2 ++ 1 file changed, 2 insertions(+) create mode 100644 dbms/tests/queries/bugs/position_case_insensitive_utf8.sql diff --git a/dbms/tests/queries/bugs/position_case_insensitive_utf8.sql b/dbms/tests/queries/bugs/position_case_insensitive_utf8.sql new file mode 100644 index 00000000000..00ddd1b498d --- /dev/null +++ b/dbms/tests/queries/bugs/position_case_insensitive_utf8.sql @@ -0,0 +1,2 @@ +SELECT positionCaseInsensitiveUTF8('Hello', materialize('%\xF0%')); +SELECT positionCaseInsensitiveUTF8(materialize('Hello'), '%\xF0%') FROM numbers(1000); From 5b1eaba276131eb2e648baf7d838eb65068f281a Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Mon, 26 Nov 2018 02:24:26 +0300 Subject: [PATCH 112/124] Every function in its own file, part 6 (#3661) * Every function in its own file, part 6 [#CLICKHOUSE-2] * Every function in its own file, part 6 [#CLICKHOUSE-2] * Every function in its own file, part 6 [#CLICKHOUSE-2] --- dbms/src/Functions/FunctionsURL.cpp | 192 ---- dbms/src/Functions/FunctionsURL.h | 917 +----------------- dbms/src/Functions/URLHierarchy.cpp | 108 +++ dbms/src/Functions/URLPathHierarchy.cpp | 110 +++ dbms/src/Functions/cutFragment.cpp | 16 + dbms/src/Functions/cutQueryString.cpp | 16 + .../Functions/cutQueryStringAndFragment.cpp | 16 + .../cutToFirstSignificantSubdomain.cpp | 39 + dbms/src/Functions/cutURLParameter.cpp | 86 ++ dbms/src/Functions/cutWWW.cpp | 63 ++ dbms/src/Functions/decodeURLComponent.cpp | 108 +++ dbms/src/Functions/domain.cpp | 17 + dbms/src/Functions/domain.h | 71 ++ dbms/src/Functions/domainWithoutWWW.cpp | 17 + dbms/src/Functions/extractURLParameter.cpp | 101 ++ .../Functions/extractURLParameterNames.cpp | 93 ++ dbms/src/Functions/extractURLParameters.cpp | 101 ++ .../Functions/firstSignificantSubdomain.cpp | 17 + .../src/Functions/firstSignificantSubdomain.h | 91 ++ dbms/src/Functions/fragment.cpp | 16 + dbms/src/Functions/fragment.h | 30 + dbms/src/Functions/path.cpp | 39 + dbms/src/Functions/pathFull.cpp | 37 + dbms/src/Functions/protocol.cpp | 17 + dbms/src/Functions/protocol.h | 57 ++ dbms/src/Functions/queryString.cpp | 16 + dbms/src/Functions/queryString.h | 33 + dbms/src/Functions/queryStringAndFragment.cpp | 16 + dbms/src/Functions/queryStringAndFragment.h | 37 + dbms/src/Functions/registerFunctionsArray.cpp | 4 +- .../Functions/registerFunctionsComparison.cpp | 4 +- .../Functions/registerFunctionsDateTime.cpp | 5 +- .../registerFunctionsHigherOrder.cpp | 4 +- .../registerFunctionsMiscellaneous.cpp | 4 +- dbms/src/Functions/registerFunctionsNull.cpp | 4 +- .../src/Functions/registerFunctionsString.cpp | 5 +- dbms/src/Functions/registerFunctionsTuple.cpp | 4 +- dbms/src/Functions/registerFunctionsURL.cpp | 56 ++ dbms/src/Functions/topLevelDomain.cpp | 48 + 39 files changed, 1497 insertions(+), 1118 deletions(-) delete mode 100644 dbms/src/Functions/FunctionsURL.cpp create mode 100644 dbms/src/Functions/URLHierarchy.cpp create mode 100644 dbms/src/Functions/URLPathHierarchy.cpp create mode 100644 dbms/src/Functions/cutFragment.cpp create mode 100644 dbms/src/Functions/cutQueryString.cpp create mode 100644 dbms/src/Functions/cutQueryStringAndFragment.cpp create mode 100644 dbms/src/Functions/cutToFirstSignificantSubdomain.cpp create mode 100644 dbms/src/Functions/cutURLParameter.cpp create mode 100644 dbms/src/Functions/cutWWW.cpp create mode 100644 dbms/src/Functions/decodeURLComponent.cpp create mode 100644 dbms/src/Functions/domain.cpp create mode 100644 dbms/src/Functions/domain.h create mode 100644 dbms/src/Functions/domainWithoutWWW.cpp create mode 100644 dbms/src/Functions/extractURLParameter.cpp create mode 100644 dbms/src/Functions/extractURLParameterNames.cpp create mode 100644 dbms/src/Functions/extractURLParameters.cpp create mode 100644 dbms/src/Functions/firstSignificantSubdomain.cpp create mode 100644 dbms/src/Functions/firstSignificantSubdomain.h create mode 100644 dbms/src/Functions/fragment.cpp create mode 100644 dbms/src/Functions/fragment.h create mode 100644 dbms/src/Functions/path.cpp create mode 100644 dbms/src/Functions/pathFull.cpp create mode 100644 dbms/src/Functions/protocol.cpp create mode 100644 dbms/src/Functions/protocol.h create mode 100644 dbms/src/Functions/queryString.cpp create mode 100644 dbms/src/Functions/queryString.h create mode 100644 dbms/src/Functions/queryStringAndFragment.cpp create mode 100644 dbms/src/Functions/queryStringAndFragment.h create mode 100644 dbms/src/Functions/registerFunctionsURL.cpp create mode 100644 dbms/src/Functions/topLevelDomain.cpp diff --git a/dbms/src/Functions/FunctionsURL.cpp b/dbms/src/Functions/FunctionsURL.cpp deleted file mode 100644 index 34b05460320..00000000000 --- a/dbms/src/Functions/FunctionsURL.cpp +++ /dev/null @@ -1,192 +0,0 @@ -#include -#include -#include -#include -#include - -namespace DB -{ - -/// We assume that size of the dst buf isn't less than src_size. -static size_t decodeURL(const char * src, size_t src_size, char * dst) -{ - const char * src_prev_pos = src; - const char * src_curr_pos = src; - const char * src_end = src + src_size; - char * dst_pos = dst; - - while (true) - { - src_curr_pos = find_first_symbols<'%'>(src_curr_pos, src_end); - - if (src_curr_pos == src_end) - { - break; - } - else if (src_end - src_curr_pos < 3) - { - src_curr_pos = src_end; - break; - } - else - { - unsigned char high = unhex(src_curr_pos[1]); - unsigned char low = unhex(src_curr_pos[2]); - - if (high != 0xFF && low != 0xFF) - { - unsigned char octet = (high << 4) + low; - - size_t bytes_to_copy = src_curr_pos - src_prev_pos; - memcpySmallAllowReadWriteOverflow15(dst_pos, src_prev_pos, bytes_to_copy); - dst_pos += bytes_to_copy; - - *dst_pos = octet; - ++dst_pos; - - src_prev_pos = src_curr_pos + 3; - } - - src_curr_pos += 3; - } - } - - if (src_prev_pos < src_curr_pos) - { - size_t bytes_to_copy = src_curr_pos - src_prev_pos; - memcpySmallAllowReadWriteOverflow15(dst_pos, src_prev_pos, bytes_to_copy); - dst_pos += bytes_to_copy; - } - - return dst_pos - dst; -} - - -size_t ExtractProtocol::getReserveLengthForElement() -{ - return strlen("https") + 1; -} - - -void ExtractProtocol::execute(Pos data, size_t size, Pos & res_data, size_t & res_size) -{ - res_data = data; - res_size = 0; - - StringRef scheme = getURLScheme(data, size); - Pos pos = data + scheme.size; - - if (scheme.size == 0 || (data + size) - pos < 4) - return; - - if (pos[0] == ':') - res_size = pos - data; -} - - -void DecodeURLComponentImpl::vector(const ColumnString::Chars & data, const ColumnString::Offsets & offsets, - ColumnString::Chars & res_data, ColumnString::Offsets & res_offsets) -{ - res_data.resize(data.size()); - size_t size = offsets.size(); - res_offsets.resize(size); - - size_t prev_offset = 0; - size_t res_offset = 0; - - for (size_t i = 0; i < size; ++i) - { - const char * src_data = reinterpret_cast(&data[prev_offset]); - size_t src_size = offsets[i] - prev_offset; - size_t dst_size = decodeURL(src_data, src_size, reinterpret_cast(res_data.data() + res_offset)); - - res_offset += dst_size; - res_offsets[i] = res_offset; - prev_offset = offsets[i]; - } - - res_data.resize(res_offset); -} - - -void DecodeURLComponentImpl::vector_fixed(const ColumnString::Chars &, size_t, ColumnString::Chars &) -{ - throw Exception("Column of type FixedString is not supported by URL functions", ErrorCodes::ILLEGAL_COLUMN); -} - -struct NameProtocol { static constexpr auto name = "protocol"; }; -struct NameDomain { static constexpr auto name = "domain"; }; -struct NameDomainWithoutWWW { static constexpr auto name = "domainWithoutWWW"; }; -struct NameFirstSignificantSubdomain { static constexpr auto name = "firstSignificantSubdomain"; }; -struct NameTopLevelDomain { static constexpr auto name = "topLevelDomain"; }; -struct NamePath { static constexpr auto name = "path"; }; -struct NamePathFull { static constexpr auto name = "pathFull"; }; -struct NameQueryString { static constexpr auto name = "queryString"; }; -struct NameFragment { static constexpr auto name = "fragment"; }; -struct NameQueryStringAndFragment { static constexpr auto name = "queryStringAndFragment"; }; -struct NameDecodeURLComponent { static constexpr auto name = "decodeURLComponent"; }; - -struct NameCutToFirstSignificantSubdomain { static constexpr auto name = "cutToFirstSignificantSubdomain"; }; - -struct NameCutWWW { static constexpr auto name = "cutWWW"; }; -struct NameCutQueryString { static constexpr auto name = "cutQueryString"; }; -struct NameCutFragment { static constexpr auto name = "cutFragment"; }; -struct NameCutQueryStringAndFragment { static constexpr auto name = "cutQueryStringAndFragment"; }; - -struct NameExtractURLParameter { static constexpr auto name = "extractURLParameter"; }; -struct NameCutURLParameter { static constexpr auto name = "cutURLParameter"; }; - -using FunctionProtocol = FunctionStringToString, NameProtocol>; -using FunctionDomain = FunctionStringToString>, NameDomain>; -using FunctionDomainWithoutWWW = FunctionStringToString>, NameDomainWithoutWWW>; -using FunctionFirstSignificantSubdomain = FunctionStringToString, NameFirstSignificantSubdomain>; -using FunctionTopLevelDomain = FunctionStringToString, NameTopLevelDomain>; -using FunctionPath = FunctionStringToString, NamePath>; -using FunctionPathFull = FunctionStringToString, NamePathFull>; -using FunctionQueryString = FunctionStringToString>, NameQueryString>; -using FunctionFragment = FunctionStringToString>, NameFragment>; -using FunctionQueryStringAndFragment = FunctionStringToString>, NameQueryStringAndFragment>; -using FunctionDecodeURLComponent = FunctionStringToString; - -using FunctionCutToFirstSignificantSubdomain = FunctionStringToString, NameCutToFirstSignificantSubdomain>; - -using FunctionCutWWW = FunctionStringToString, NameCutWWW>; -using FunctionCutQueryString = FunctionStringToString>, NameCutQueryString>; -using FunctionCutFragment = FunctionStringToString>, NameCutFragment>; -using FunctionCutQueryStringAndFragment = FunctionStringToString>, NameCutQueryStringAndFragment>; - -using FunctionExtractURLParameter = FunctionsStringSearchToString; -using FunctionCutURLParameter = FunctionsStringSearchToString; -using FunctionExtractURLParameters = FunctionTokens; -using FunctionURLHierarchy = FunctionTokens; -using FunctionURLPathHierarchy = FunctionTokens; -using FunctionExtractURLParameterNames = FunctionTokens; - - -void registerFunctionsURL(FunctionFactory & factory) -{ - factory.registerFunction(); - factory.registerFunction(); - factory.registerFunction(); - factory.registerFunction(); - factory.registerFunction(); - factory.registerFunction(); - factory.registerFunction(); - factory.registerFunction(); - factory.registerFunction(); - factory.registerFunction(); - factory.registerFunction(); - factory.registerFunction(); - factory.registerFunction(); - factory.registerFunction(); - factory.registerFunction(); - factory.registerFunction(); - factory.registerFunction(); - factory.registerFunction(); - factory.registerFunction(); - factory.registerFunction(); - factory.registerFunction(); - factory.registerFunction(); -} - -} diff --git a/dbms/src/Functions/FunctionsURL.h b/dbms/src/Functions/FunctionsURL.h index 634f6db5d56..69f0b363112 100644 --- a/dbms/src/Functions/FunctionsURL.h +++ b/dbms/src/Functions/FunctionsURL.h @@ -1,21 +1,13 @@ #pragma once -#include #include -#include -#include -#include -#include -#include -#include -#include -#include +#include namespace DB { -/** URL processing functions. +/** URL processing functions. See implementation in separate .cpp files. * All functions are not strictly follow RFC, instead they are maximally simplified for performance reasons. * * Functions for extraction parts of URL. @@ -56,895 +48,14 @@ namespace DB * URLHierarchy(URL) */ +namespace ErrorCodes +{ + extern const int ILLEGAL_COLUMN; +} + using Pos = const char *; -/// Extracts scheme from given url. -inline StringRef getURLScheme(const char * data, size_t size) -{ - // scheme = ALPHA *( ALPHA / DIGIT / "+" / "-" / "." ) - const char * pos = data; - const char * end = data + size; - - if (isAlphaASCII(*pos)) - { - for (++pos; pos < end; ++pos) - { - if (!(isAlphaNumericASCII(*pos) || *pos == '+' || *pos == '-' || *pos == '.')) - { - break; - } - } - - return StringRef(data, pos - data); - } - - return {}; -} - - -/// Extracts host from given url. -inline StringRef getURLHost(const char * data, size_t size) -{ - Pos pos = data; - Pos end = data + size; - - if (end == (pos = find_first_symbols<'/'>(pos, end))) - return {}; - - if (pos != data) - { - StringRef scheme = getURLScheme(data, size); - Pos scheme_end = data + scheme.size; - - // Colon must follows after scheme. - if (pos - scheme_end != 1 || *scheme_end != ':') - return {}; - } - - if (end - pos < 2 || *(pos) != '/' || *(pos + 1) != '/') - return {}; - pos += 2; - - const char * start_of_host = pos; - for (; pos < end; ++pos) - { - if (*pos == '@') - start_of_host = pos + 1; - else if (*pos == ':' || *pos == '/' || *pos == '?' || *pos == '#') - break; - } - - return (pos == start_of_host) ? StringRef{} : StringRef(start_of_host, pos - start_of_host); -} - - -struct ExtractProtocol -{ - static size_t getReserveLengthForElement(); - - static void execute(Pos data, size_t size, Pos & res_data, size_t & res_size); -}; - -template -struct ExtractDomain -{ - static size_t getReserveLengthForElement() { return 15; } - - static void execute(Pos data, size_t size, Pos & res_data, size_t & res_size) - { - StringRef host = getURLHost(data, size); - - if (host.size == 0) - { - res_data = data; - res_size = 0; - } - else - { - if (without_www && host.size > 4 && !strncmp(host.data, "www.", 4)) - host = { host.data + 4, host.size - 4 }; - - res_data = host.data; - res_size = host.size; - } - } -}; - -struct ExtractFirstSignificantSubdomain -{ - static size_t getReserveLengthForElement() { return 10; } - - static void execute(const Pos data, const size_t size, Pos & res_data, size_t & res_size, Pos * out_domain_end = nullptr) - { - res_data = data; - res_size = 0; - - Pos tmp; - size_t domain_length; - ExtractDomain::execute(data, size, tmp, domain_length); - - if (domain_length == 0) - return; - - if (out_domain_end) - *out_domain_end = tmp + domain_length; - - /// cut useless dot - if (tmp[domain_length - 1] == '.') - --domain_length; - - res_data = tmp; - res_size = domain_length; - - auto begin = tmp; - auto end = begin + domain_length; - const char * last_3_periods[3]{}; - - auto pos = find_first_symbols<'.'>(begin, end); - while (pos < end) - { - last_3_periods[2] = last_3_periods[1]; - last_3_periods[1] = last_3_periods[0]; - last_3_periods[0] = pos; - pos = find_first_symbols<'.'>(pos + 1, end); - } - - if (!last_3_periods[0]) - return; - - if (!last_3_periods[1]) - { - res_size = last_3_periods[0] - begin; - return; - } - - if (!last_3_periods[2]) - last_3_periods[2] = begin - 1; - - size_t size_of_second_subdomain_plus_period = last_3_periods[0] - last_3_periods[1]; - if (size_of_second_subdomain_plus_period == 4 || size_of_second_subdomain_plus_period == 3) - { - /// We will key by four bytes that are either ".xyz" or ".xy.". - UInt32 key = unalignedLoad(last_3_periods[1]); - - /// NOTE: assuming little endian. - /// NOTE: does the compiler generate SIMD code? - /// NOTE: for larger amount of cases we can use a perfect hash table (see 'gperf' as an example). - if ( key == '.' + 'c' * 0x100U + 'o' * 0x10000U + 'm' * 0x1000000U - || key == '.' + 'n' * 0x100U + 'e' * 0x10000U + 't' * 0x1000000U - || key == '.' + 'o' * 0x100U + 'r' * 0x10000U + 'g' * 0x1000000U - || key == '.' + 'b' * 0x100U + 'i' * 0x10000U + 'z' * 0x1000000U - || key == '.' + 'g' * 0x100U + 'o' * 0x10000U + 'v' * 0x1000000U - || key == '.' + 'm' * 0x100U + 'i' * 0x10000U + 'l' * 0x1000000U - || key == '.' + 'e' * 0x100U + 'd' * 0x10000U + 'u' * 0x1000000U - || key == '.' + 'c' * 0x100U + 'o' * 0x10000U + '.' * 0x1000000U) - { - res_data += last_3_periods[2] + 1 - begin; - res_size = last_3_periods[1] - last_3_periods[2] - 1; - return; - } - } - - res_data += last_3_periods[1] + 1 - begin; - res_size = last_3_periods[0] - last_3_periods[1] - 1; - } -}; - -struct CutToFirstSignificantSubdomain -{ - static size_t getReserveLengthForElement() { return 15; } - - static void execute(const Pos data, const size_t size, Pos & res_data, size_t & res_size) - { - res_data = data; - res_size = 0; - - Pos tmp_data; - size_t tmp_length; - Pos domain_end; - ExtractFirstSignificantSubdomain::execute(data, size, tmp_data, tmp_length, &domain_end); - - if (tmp_length == 0) - return; - - res_data = tmp_data; - res_size = domain_end - tmp_data; - } -}; - -struct ExtractTopLevelDomain -{ - static size_t getReserveLengthForElement() { return 5; } - - static void execute(Pos data, size_t size, Pos & res_data, size_t & res_size) - { - StringRef host = getURLHost(data, size); - - res_data = data; - res_size = 0; - - if (host.size != 0) - { - if (host.data[host.size - 1] == '.') - host.size -= 1; - - auto host_end = host.data + host.size; - - Pos last_dot = find_last_symbols_or_null<'.'>(host.data, host_end); - if (!last_dot) - return; - - /// For IPv4 addresses select nothing. - if (last_dot[1] <= '9') - return; - - res_data = last_dot + 1; - res_size = host_end - res_data; - } - } -}; - -struct ExtractPath -{ - static size_t getReserveLengthForElement() { return 25; } - - static void execute(Pos data, size_t size, Pos & res_data, size_t & res_size) - { - res_data = data; - res_size = 0; - - Pos pos = data; - Pos end = pos + size; - - if (end != (pos = find_first_symbols<'/'>(pos, end)) && pos[1] == '/' && end != (pos = find_first_symbols<'/'>(pos + 2, end))) - { - Pos query_string_or_fragment = find_first_symbols<'?', '#'>(pos, end); - - res_data = pos; - res_size = query_string_or_fragment - res_data; - } - } -}; - -struct ExtractPathFull -{ - static size_t getReserveLengthForElement() { return 30; } - - static void execute(const Pos data, const size_t size, Pos & res_data, size_t & res_size) - { - res_data = data; - res_size = 0; - - Pos pos = data; - Pos end = pos + size; - - if (end != (pos = find_first_symbols<'/'>(pos, end)) && pos[1] == '/' && end != (pos = find_first_symbols<'/'>(pos + 2, end))) - { - res_data = pos; - res_size = end - res_data; - } - } -}; - -template -struct ExtractQueryString -{ - static size_t getReserveLengthForElement() { return 10; } - - static void execute(Pos data, size_t size, Pos & res_data, size_t & res_size) - { - res_data = data; - res_size = 0; - - Pos pos = data; - Pos end = pos + size; - - if (end != (pos = find_first_symbols<'?'>(pos, end))) - { - Pos fragment = find_first_symbols<'#'>(pos, end); - - res_data = pos + (without_leading_char ? 1 : 0); - res_size = fragment - res_data; - } - } -}; - -template -struct ExtractFragment -{ - static size_t getReserveLengthForElement() { return 10; } - - static void execute(Pos data, size_t size, Pos & res_data, size_t & res_size) - { - res_data = data; - res_size = 0; - - Pos pos = data; - Pos end = pos + size; - - if (end != (pos = find_first_symbols<'#'>(pos, end))) - { - res_data = pos + (without_leading_char ? 1 : 0); - res_size = end - res_data; - } - } -}; - -template -struct ExtractQueryStringAndFragment -{ - static size_t getReserveLengthForElement() { return 20; } - - static void execute(Pos data, size_t size, Pos & res_data, size_t & res_size) - { - res_data = data; - res_size = 0; - - Pos pos = data; - Pos end = pos + size; - - if (end != (pos = find_first_symbols<'?'>(pos, end))) - { - res_data = pos + (without_leading_char ? 1 : 0); - res_size = end - res_data; - } - else if (end != (pos = find_first_symbols<'#'>(pos, end))) - { - res_data = pos; - res_size = end - res_data; - } - } -}; - -/// With dot at the end. -struct ExtractWWW -{ - static void execute(Pos data, size_t size, Pos & res_data, size_t & res_size) - { - res_data = data; - res_size = 0; - - Pos pos = data; - Pos end = pos + size; - - if (end != (pos = find_first_symbols<'/'>(pos, end))) - { - if (pos != data) - { - Pos tmp; - size_t protocol_length; - ExtractProtocol::execute(data, size, tmp, protocol_length); - - if (pos != data + protocol_length + 1) - return; - } - - if (end - pos < 2 || *(pos) != '/' || *(pos + 1) != '/') - return; - - const char *start_of_host = (pos += 2); - for (; pos < end; ++pos) - { - if (*pos == '@') - start_of_host = pos + 1; - else if (*pos == ':' || *pos == '/' || *pos == '?' || *pos == '#') - break; - } - - if (start_of_host + 4 < end && !strncmp(start_of_host, "www.", 4)) - { - res_data = start_of_host; - res_size = 4; - } - } - } -}; - - -struct ExtractURLParameterImpl -{ - static void vector(const ColumnString::Chars & data, - const ColumnString::Offsets & offsets, - std::string pattern, - ColumnString::Chars & res_data, ColumnString::Offsets & res_offsets) - { - res_data.reserve(data.size() / 5); - res_offsets.resize(offsets.size()); - - pattern += '='; - const char * param_str = pattern.c_str(); - size_t param_len = pattern.size(); - - ColumnString::Offset prev_offset = 0; - ColumnString::Offset res_offset = 0; - - for (size_t i = 0; i < offsets.size(); ++i) - { - ColumnString::Offset cur_offset = offsets[i]; - - const char * str = reinterpret_cast(&data[prev_offset]); - const char * end = reinterpret_cast(&data[cur_offset]); - - /// Find query string or fragment identifier. - /// Note that we support parameters in fragment identifier in the same way as in query string. - - const char * const query_string_begin = find_first_symbols<'?', '#'>(str, end); - - /// Will point to the beginning of "name=value" pair. Then it will be reassigned to the beginning of "value". - const char * param_begin = nullptr; - - if (query_string_begin + 1 < end) - { - param_begin = query_string_begin + 1; - - while (true) - { - param_begin = static_cast(memmem(param_begin, end - param_begin, param_str, param_len)); - - if (!param_begin) - break; - - if (param_begin[-1] != '?' && param_begin[-1] != '#' && param_begin[-1] != '&') - { - /// Parameter name is different but has the same suffix. - param_begin += param_len; - continue; - } - else - { - param_begin += param_len; - break; - } - } - } - - if (param_begin) - { - const char * param_end = find_first_symbols<'&', '#'>(param_begin, end); - if (param_end == end) - param_end = param_begin + strlen(param_begin); - - size_t param_size = param_end - param_begin; - - res_data.resize(res_offset + param_size + 1); - memcpySmallAllowReadWriteOverflow15(&res_data[res_offset], param_begin, param_size); - res_offset += param_size; - } - else - { - /// No parameter found, put empty string in result. - res_data.resize(res_offset + 1); - } - - res_data[res_offset] = 0; - ++res_offset; - res_offsets[i] = res_offset; - - prev_offset = cur_offset; - } - } -}; - - -struct CutURLParameterImpl -{ - static void vector(const ColumnString::Chars & data, - const ColumnString::Offsets & offsets, - std::string pattern, - ColumnString::Chars & res_data, ColumnString::Offsets & res_offsets) - { - res_data.reserve(data.size()); - res_offsets.resize(offsets.size()); - - pattern += '='; - const char * param_str = pattern.c_str(); - size_t param_len = pattern.size(); - - size_t prev_offset = 0; - size_t res_offset = 0; - - for (size_t i = 0; i < offsets.size(); ++i) - { - size_t cur_offset = offsets[i]; - - const char * url_begin = reinterpret_cast(&data[prev_offset]); - const char * url_end = reinterpret_cast(&data[cur_offset]) - 1; - const char * begin_pos = url_begin; - const char * end_pos = begin_pos; - - do - { - const char * query_string_begin = find_first_symbols<'?', '#'>(url_begin, url_end); - if (query_string_begin + 1 >= url_end) - break; - - const char * pos = static_cast(memmem(query_string_begin + 1, url_end - query_string_begin - 1, param_str, param_len)); - if (pos == nullptr) - break; - - if (pos[-1] != '?' && pos[-1] != '#' && pos[-1] != '&') - { - pos = nullptr; - break; - } - - begin_pos = pos; - end_pos = begin_pos + param_len; - - /// Skip the value. - while (*end_pos && *end_pos != '&' && *end_pos != '#') - ++end_pos; - - /// Capture '&' before or after the parameter. - if (*end_pos == '&') - ++end_pos; - else if (begin_pos[-1] == '&') - --begin_pos; - } while (false); - - size_t cut_length = (url_end - url_begin) - (end_pos - begin_pos); - res_data.resize(res_offset + cut_length + 1); - memcpySmallAllowReadWriteOverflow15(&res_data[res_offset], url_begin, begin_pos - url_begin); - memcpySmallAllowReadWriteOverflow15(&res_data[res_offset] + (begin_pos - url_begin), end_pos, url_end - end_pos); - res_offset += cut_length + 1; - res_data[res_offset - 1] = 0; - res_offsets[i] = res_offset; - - prev_offset = cur_offset; - } - } -}; - - -class ExtractURLParametersImpl -{ -private: - Pos pos; - Pos end; - bool first; - -public: - static constexpr auto name = "extractURLParameters"; - static String getName() { return name; } - - static size_t getNumberOfArguments() { return 1; } - - static void checkArguments(const DataTypes & arguments) - { - if (!isString(arguments[0])) - throw Exception("Illegal type " + arguments[0]->getName() + " of first argument of function " + getName() + ". Must be String.", - ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); - } - - void init(Block & /*block*/, const ColumnNumbers & /*arguments*/) {} - - /// Returns the position of the argument that is the column of rows - size_t getStringsArgumentPosition() - { - return 0; - } - - /// Called for each next string. - void set(Pos pos_, Pos end_) - { - pos = pos_; - end = end_; - first = true; - } - - /// Get the next token, if any, or return false. - bool get(Pos & token_begin, Pos & token_end) - { - if (pos == nullptr) - return false; - - if (first) - { - first = false; - pos = find_first_symbols<'?', '#'>(pos, end); - if (pos + 1 >= end) - return false; - ++pos; - } - - while (true) - { - token_begin = pos; - pos = find_first_symbols<'=', '&', '#', '?'>(pos, end); - if (pos == end) - return false; - - if (*pos == '?') - { - ++pos; - continue; - } - - break; - } - - if (*pos == '&' || *pos == '#') - { - token_end = pos++; - } - else - { - ++pos; - pos = find_first_symbols<'&', '#'>(pos, end); - if (pos == end) - token_end = end; - else - token_end = pos++; - } - - return true; - } -}; - -class ExtractURLParameterNamesImpl -{ -private: - Pos pos; - Pos end; - bool first; - -public: - static constexpr auto name = "extractURLParameterNames"; - static String getName() { return name; } - - static size_t getNumberOfArguments() { return 1; } - - static void checkArguments(const DataTypes & arguments) - { - if (!isString(arguments[0])) - throw Exception("Illegal type " + arguments[0]->getName() + " of first argument of function " + getName() + ". Must be String.", - ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); - } - - /// Returns the position of the argument that is the column of rows - size_t getStringsArgumentPosition() - { - return 0; - } - - void init(Block & /*block*/, const ColumnNumbers & /*arguments*/) {} - - /// Called for each next string. - void set(Pos pos_, Pos end_) - { - pos = pos_; - end = end_; - first = true; - } - - /// Get the next token, if any, or return false. - bool get(Pos & token_begin, Pos & token_end) - { - if (pos == nullptr) - return false; - - if (first) - { - first = false; - pos = find_first_symbols<'?', '#'>(pos, end); - } - else - pos = find_first_symbols<'&', '#'>(pos, end); - - if (pos + 1 >= end) - return false; - ++pos; - - while (true) - { - token_begin = pos; - - pos = find_first_symbols<'=', '&', '#', '?'>(pos, end); - if (pos == end) - return false; - else - token_end = pos; - - if (*pos == '?') - { - ++pos; - continue; - } - - break; - } - - return true; - } -}; - -class URLHierarchyImpl -{ -private: - Pos begin; - Pos pos; - Pos end; - -public: - static constexpr auto name = "URLHierarchy"; - static String getName() { return name; } - - static size_t getNumberOfArguments() { return 1; } - - static void checkArguments(const DataTypes & arguments) - { - if (!isString(arguments[0])) - throw Exception("Illegal type " + arguments[0]->getName() + " of first argument of function " + getName() + ". Must be String.", - ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); - } - - void init(Block & /*block*/, const ColumnNumbers & /*arguments*/) {} - - /// Returns the position of the argument that is the column of rows - size_t getStringsArgumentPosition() - { - return 0; - } - - /// Called for each next string. - void set(Pos pos_, Pos end_) - { - begin = pos = pos_; - end = end_; - } - - /// Get the next token, if any, or return false. - bool get(Pos & token_begin, Pos & token_end) - { - /// Code from URLParser. - if (pos == end) - return false; - - if (pos == begin) - { - /// Let's parse everything that goes before the path - - /// Assume that the protocol has already been changed to lowercase. - while (pos < end && ((*pos > 'a' && *pos < 'z') || (*pos > '0' && *pos < '9'))) - ++pos; - - /** We will calculate the hierarchy only for URLs in which there is a protocol, and after it there are two slashes. - * (http, file - fit, mailto, magnet - do not fit), and after two slashes still at least something is there - * For the rest, simply return the full URL as the only element of the hierarchy. - */ - if (pos == begin || pos == end || !(*pos++ == ':' && pos < end && *pos++ == '/' && pos < end && *pos++ == '/' && pos < end)) - { - pos = end; - token_begin = begin; - token_end = end; - return true; - } - - /// The domain for simplicity is everything that after the protocol and two slashes, until the next slash or `?` or `#` - while (pos < end && !(*pos == '/' || *pos == '?' || *pos == '#')) - ++pos; - - if (pos != end) - ++pos; - - token_begin = begin; - token_end = pos; - - return true; - } - - /// We go to the next `/` or `?` or `#`, skipping all those at the beginning. - while (pos < end && (*pos == '/' || *pos == '?' || *pos == '#')) - ++pos; - if (pos == end) - return false; - while (pos < end && !(*pos == '/' || *pos == '?' || *pos == '#')) - ++pos; - - if (pos != end) - ++pos; - - token_begin = begin; - token_end = pos; - - return true; - } -}; - - -class URLPathHierarchyImpl -{ -private: - Pos begin; - Pos pos; - Pos end; - Pos start; - -public: - static constexpr auto name = "URLPathHierarchy"; - static String getName() { return name; } - - static size_t getNumberOfArguments() { return 1; } - - static void checkArguments(const DataTypes & arguments) - { - if (!isString(arguments[0])) - throw Exception("Illegal type " + arguments[0]->getName() + " of first argument of function " + getName() + ". Must be String.", - ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); - } - - void init(Block & /*block*/, const ColumnNumbers & /*arguments*/) {} - - /// Returns the position of the argument that is the column of rows - size_t getStringsArgumentPosition() - { - return 0; - } - - /// Called for each next string. - void set(Pos pos_, Pos end_) - { - begin = pos = pos_; - start = begin; - end = end_; - } - - /// Get the next token, if any, or return false. - bool get(Pos & token_begin, Pos & token_end) - { - /// Code from URLParser. - - if (pos == end) - return false; - - if (pos == begin) - { - /// Let's parse everything that goes before the path - - /// Assume that the protocol has already been changed to lowercase. - while (pos < end && ((*pos > 'a' && *pos < 'z') || (*pos > '0' && *pos < '9'))) - ++pos; - - /** We will calculate the hierarchy only for URLs in which there is a protocol, and after it there are two slashes. - * (http, file - fit, mailto, magnet - do not fit), and after two slashes still at least something is there. - * For the rest, just return an empty array. - */ - if (pos == begin || pos == end || !(*pos++ == ':' && pos < end && *pos++ == '/' && pos < end && *pos++ == '/' && pos < end)) - { - pos = end; - return false; - } - - /// The domain for simplicity is everything that after the protocol and the two slashes, until the next slash or `?` or `#` - while (pos < end && !(*pos == '/' || *pos == '?' || *pos == '#')) - ++pos; - - start = pos; - - if (pos != end) - ++pos; - } - - /// We go to the next `/` or `?` or `#`, skipping all those at the beginning. - while (pos < end && (*pos == '/' || *pos == '?' || *pos == '#')) - ++pos; - if (pos == end) - return false; - while (pos < end && !(*pos == '/' || *pos == '?' || *pos == '#')) - ++pos; - - if (pos != end) - ++pos; - - token_begin = start; - token_end = pos; - - return true; - } -}; - - /** Select part of string using the Extractor. */ template @@ -1048,18 +159,4 @@ struct CutSubstringImpl } }; - -/// Percent decode of url data. -struct DecodeURLComponentImpl -{ - static void vector(const ColumnString::Chars & data, const ColumnString::Offsets & offsets, - ColumnString::Chars & res_data, ColumnString::Offsets & res_offsets); - - static void constant(const std::string & data, - std::string & res_data); - - static void vector_fixed(const ColumnString::Chars & data, size_t n, - ColumnString::Chars & res_data); -}; - } diff --git a/dbms/src/Functions/URLHierarchy.cpp b/dbms/src/Functions/URLHierarchy.cpp new file mode 100644 index 00000000000..eea0c7f437b --- /dev/null +++ b/dbms/src/Functions/URLHierarchy.cpp @@ -0,0 +1,108 @@ +#include +#include +#include + +namespace DB +{ + +class URLPathHierarchyImpl +{ +private: + Pos begin; + Pos pos; + Pos end; + Pos start; + +public: + static constexpr auto name = "URLPathHierarchy"; + static String getName() { return name; } + + static size_t getNumberOfArguments() { return 1; } + + static void checkArguments(const DataTypes & arguments) + { + if (!isString(arguments[0])) + throw Exception("Illegal type " + arguments[0]->getName() + " of first argument of function " + getName() + ". Must be String.", + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + } + + void init(Block & /*block*/, const ColumnNumbers & /*arguments*/) {} + + /// Returns the position of the argument that is the column of rows + size_t getStringsArgumentPosition() + { + return 0; + } + + /// Called for each next string. + void set(Pos pos_, Pos end_) + { + begin = pos = pos_; + start = begin; + end = end_; + } + + /// Get the next token, if any, or return false. + bool get(Pos & token_begin, Pos & token_end) + { + /// Code from URLParser. + + if (pos == end) + return false; + + if (pos == begin) + { + /// Let's parse everything that goes before the path + + /// Assume that the protocol has already been changed to lowercase. + while (pos < end && ((*pos > 'a' && *pos < 'z') || (*pos > '0' && *pos < '9'))) + ++pos; + + /** We will calculate the hierarchy only for URLs in which there is a protocol, and after it there are two slashes. + * (http, file - fit, mailto, magnet - do not fit), and after two slashes still at least something is there. + * For the rest, just return an empty array. + */ + if (pos == begin || pos == end || !(*pos++ == ':' && pos < end && *pos++ == '/' && pos < end && *pos++ == '/' && pos < end)) + { + pos = end; + return false; + } + + /// The domain for simplicity is everything that after the protocol and the two slashes, until the next slash or `?` or `#` + while (pos < end && !(*pos == '/' || *pos == '?' || *pos == '#')) + ++pos; + + start = pos; + + if (pos != end) + ++pos; + } + + /// We go to the next `/` or `?` or `#`, skipping all those at the beginning. + while (pos < end && (*pos == '/' || *pos == '?' || *pos == '#')) + ++pos; + if (pos == end) + return false; + while (pos < end && !(*pos == '/' || *pos == '?' || *pos == '#')) + ++pos; + + if (pos != end) + ++pos; + + token_begin = start; + token_end = pos; + + return true; + } +}; + + +struct NameURLPathHierarchy { static constexpr auto name = "URLPathHierarchy"; }; +using FunctionURLPathHierarchy = FunctionTokens; + +void registerFunctionURLPathHierarchy(FunctionFactory & factory) +{ + factory.registerFunction(); +} + +} diff --git a/dbms/src/Functions/URLPathHierarchy.cpp b/dbms/src/Functions/URLPathHierarchy.cpp new file mode 100644 index 00000000000..39ffb2824c4 --- /dev/null +++ b/dbms/src/Functions/URLPathHierarchy.cpp @@ -0,0 +1,110 @@ +#include +#include +#include + +namespace DB +{ + +class URLHierarchyImpl +{ +private: + Pos begin; + Pos pos; + Pos end; + +public: + static constexpr auto name = "URLHierarchy"; + static String getName() { return name; } + + static size_t getNumberOfArguments() { return 1; } + + static void checkArguments(const DataTypes & arguments) + { + if (!isString(arguments[0])) + throw Exception("Illegal type " + arguments[0]->getName() + " of first argument of function " + getName() + ". Must be String.", + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + } + + void init(Block & /*block*/, const ColumnNumbers & /*arguments*/) {} + + /// Returns the position of the argument that is the column of rows + size_t getStringsArgumentPosition() + { + return 0; + } + + /// Called for each next string. + void set(Pos pos_, Pos end_) + { + begin = pos = pos_; + end = end_; + } + + /// Get the next token, if any, or return false. + bool get(Pos & token_begin, Pos & token_end) + { + /// Code from URLParser. + if (pos == end) + return false; + + if (pos == begin) + { + /// Let's parse everything that goes before the path + + /// Assume that the protocol has already been changed to lowercase. + while (pos < end && ((*pos > 'a' && *pos < 'z') || (*pos > '0' && *pos < '9'))) + ++pos; + + /** We will calculate the hierarchy only for URLs in which there is a protocol, and after it there are two slashes. + * (http, file - fit, mailto, magnet - do not fit), and after two slashes still at least something is there + * For the rest, simply return the full URL as the only element of the hierarchy. + */ + if (pos == begin || pos == end || !(*pos++ == ':' && pos < end && *pos++ == '/' && pos < end && *pos++ == '/' && pos < end)) + { + pos = end; + token_begin = begin; + token_end = end; + return true; + } + + /// The domain for simplicity is everything that after the protocol and two slashes, until the next slash or `?` or `#` + while (pos < end && !(*pos == '/' || *pos == '?' || *pos == '#')) + ++pos; + + if (pos != end) + ++pos; + + token_begin = begin; + token_end = pos; + + return true; + } + + /// We go to the next `/` or `?` or `#`, skipping all those at the beginning. + while (pos < end && (*pos == '/' || *pos == '?' || *pos == '#')) + ++pos; + if (pos == end) + return false; + while (pos < end && !(*pos == '/' || *pos == '?' || *pos == '#')) + ++pos; + + if (pos != end) + ++pos; + + token_begin = begin; + token_end = pos; + + return true; + } +}; + + +struct NameURLHierarchy { static constexpr auto name = "URLHierarchy"; }; +using FunctionURLHierarchy = FunctionTokens; + +void registerFunctionURLHierarchy(FunctionFactory & factory) +{ + factory.registerFunction(); +} + +} diff --git a/dbms/src/Functions/cutFragment.cpp b/dbms/src/Functions/cutFragment.cpp new file mode 100644 index 00000000000..36980b18d61 --- /dev/null +++ b/dbms/src/Functions/cutFragment.cpp @@ -0,0 +1,16 @@ +#include +#include +#include + +namespace DB +{ + +struct NameCutFragment { static constexpr auto name = "cutFragment"; }; +using FunctionCutFragment = FunctionStringToString>, NameCutFragment>; + +void registerFunctionCutFragment(FunctionFactory & factory) +{ + factory.registerFunction(); +} + +} diff --git a/dbms/src/Functions/cutQueryString.cpp b/dbms/src/Functions/cutQueryString.cpp new file mode 100644 index 00000000000..4afa3e6f618 --- /dev/null +++ b/dbms/src/Functions/cutQueryString.cpp @@ -0,0 +1,16 @@ +#include +#include +#include + +namespace DB +{ + +struct NameCutQueryString { static constexpr auto name = "cutQueryString"; }; +using FunctionCutQueryString = FunctionStringToString>, NameCutQueryString>; + +void registerFunctionCutQueryString(FunctionFactory & factory) +{ + factory.registerFunction(); +} + +} diff --git a/dbms/src/Functions/cutQueryStringAndFragment.cpp b/dbms/src/Functions/cutQueryStringAndFragment.cpp new file mode 100644 index 00000000000..910386790d9 --- /dev/null +++ b/dbms/src/Functions/cutQueryStringAndFragment.cpp @@ -0,0 +1,16 @@ +#include +#include +#include + +namespace DB +{ + +struct NameCutQueryStringAndFragment { static constexpr auto name = "cutQueryStringAndFragment"; }; +using FunctionCutQueryStringAndFragment = FunctionStringToString>, NameCutQueryStringAndFragment>; + +void registerFunctionCutQueryStringAndFragment(FunctionFactory & factory) +{ + factory.registerFunction(); +} + +} diff --git a/dbms/src/Functions/cutToFirstSignificantSubdomain.cpp b/dbms/src/Functions/cutToFirstSignificantSubdomain.cpp new file mode 100644 index 00000000000..f28000d22ad --- /dev/null +++ b/dbms/src/Functions/cutToFirstSignificantSubdomain.cpp @@ -0,0 +1,39 @@ +#include +#include +#include + + +namespace DB +{ + +struct CutToFirstSignificantSubdomain +{ + static size_t getReserveLengthForElement() { return 15; } + + static void execute(const Pos data, const size_t size, Pos & res_data, size_t & res_size) + { + res_data = data; + res_size = 0; + + Pos tmp_data; + size_t tmp_length; + Pos domain_end; + ExtractFirstSignificantSubdomain::execute(data, size, tmp_data, tmp_length, &domain_end); + + if (tmp_length == 0) + return; + + res_data = tmp_data; + res_size = domain_end - tmp_data; + } +}; + +struct NameCutToFirstSignificantSubdomain { static constexpr auto name = "cutToFirstSignificantSubdomain"; }; +using FunctionCutToFirstSignificantSubdomain = FunctionStringToString, NameCutToFirstSignificantSubdomain>; + +void registerFunctionCutToFirstSignificantSubdomain(FunctionFactory & factory) +{ + factory.registerFunction(); +} + +} diff --git a/dbms/src/Functions/cutURLParameter.cpp b/dbms/src/Functions/cutURLParameter.cpp new file mode 100644 index 00000000000..a7a6b51e2f3 --- /dev/null +++ b/dbms/src/Functions/cutURLParameter.cpp @@ -0,0 +1,86 @@ +#include +#include +#include +#include + +namespace DB +{ + +struct CutURLParameterImpl +{ + static void vector(const ColumnString::Chars & data, + const ColumnString::Offsets & offsets, + std::string pattern, + ColumnString::Chars & res_data, ColumnString::Offsets & res_offsets) + { + res_data.reserve(data.size()); + res_offsets.resize(offsets.size()); + + pattern += '='; + const char * param_str = pattern.c_str(); + size_t param_len = pattern.size(); + + size_t prev_offset = 0; + size_t res_offset = 0; + + for (size_t i = 0; i < offsets.size(); ++i) + { + size_t cur_offset = offsets[i]; + + const char * url_begin = reinterpret_cast(&data[prev_offset]); + const char * url_end = reinterpret_cast(&data[cur_offset]) - 1; + const char * begin_pos = url_begin; + const char * end_pos = begin_pos; + + do + { + const char * query_string_begin = find_first_symbols<'?', '#'>(url_begin, url_end); + if (query_string_begin + 1 >= url_end) + break; + + const char * pos = static_cast(memmem(query_string_begin + 1, url_end - query_string_begin - 1, param_str, param_len)); + if (pos == nullptr) + break; + + if (pos[-1] != '?' && pos[-1] != '#' && pos[-1] != '&') + { + pos = nullptr; + break; + } + + begin_pos = pos; + end_pos = begin_pos + param_len; + + /// Skip the value. + while (*end_pos && *end_pos != '&' && *end_pos != '#') + ++end_pos; + + /// Capture '&' before or after the parameter. + if (*end_pos == '&') + ++end_pos; + else if (begin_pos[-1] == '&') + --begin_pos; + } while (false); + + size_t cut_length = (url_end - url_begin) - (end_pos - begin_pos); + res_data.resize(res_offset + cut_length + 1); + memcpySmallAllowReadWriteOverflow15(&res_data[res_offset], url_begin, begin_pos - url_begin); + memcpySmallAllowReadWriteOverflow15(&res_data[res_offset] + (begin_pos - url_begin), end_pos, url_end - end_pos); + res_offset += cut_length + 1; + res_data[res_offset - 1] = 0; + res_offsets[i] = res_offset; + + prev_offset = cur_offset; + } + } +}; + +struct NameCutURLParameter { static constexpr auto name = "cutURLParameter"; }; +using FunctionCutURLParameter = FunctionsStringSearchToString; + +void registerFunctionCutURLParameter(FunctionFactory & factory) +{ + factory.registerFunction(); +} + +} diff --git a/dbms/src/Functions/cutWWW.cpp b/dbms/src/Functions/cutWWW.cpp new file mode 100644 index 00000000000..e8bf88e8d8b --- /dev/null +++ b/dbms/src/Functions/cutWWW.cpp @@ -0,0 +1,63 @@ +#include +#include +#include +#include +#include + + +namespace DB +{ + +/// With dot at the end. +struct ExtractWWW +{ + static void execute(Pos data, size_t size, Pos & res_data, size_t & res_size) + { + res_data = data; + res_size = 0; + + Pos pos = data; + Pos end = pos + size; + + if (end != (pos = find_first_symbols<'/'>(pos, end))) + { + if (pos != data) + { + Pos tmp; + size_t protocol_length; + ExtractProtocol::execute(data, size, tmp, protocol_length); + + if (pos != data + protocol_length + 1) + return; + } + + if (end - pos < 2 || *(pos) != '/' || *(pos + 1) != '/') + return; + + const char *start_of_host = (pos += 2); + for (; pos < end; ++pos) + { + if (*pos == '@') + start_of_host = pos + 1; + else if (*pos == ':' || *pos == '/' || *pos == '?' || *pos == '#') + break; + } + + if (start_of_host + 4 < end && !strncmp(start_of_host, "www.", 4)) + { + res_data = start_of_host; + res_size = 4; + } + } + } +}; + +struct NameCutWWW { static constexpr auto name = "cutWWW"; }; +using FunctionCutWWW = FunctionStringToString, NameCutWWW>; + +void registerFunctionCutWWW(FunctionFactory & factory) +{ + factory.registerFunction(); +} + +} diff --git a/dbms/src/Functions/decodeURLComponent.cpp b/dbms/src/Functions/decodeURLComponent.cpp new file mode 100644 index 00000000000..11f49836688 --- /dev/null +++ b/dbms/src/Functions/decodeURLComponent.cpp @@ -0,0 +1,108 @@ +#include +#include +#include +#include +#include + + +namespace DB +{ + +/// We assume that size of the dst buf isn't less than src_size. +static size_t decodeURL(const char * src, size_t src_size, char * dst) +{ + const char * src_prev_pos = src; + const char * src_curr_pos = src; + const char * src_end = src + src_size; + char * dst_pos = dst; + + while (true) + { + src_curr_pos = find_first_symbols<'%'>(src_curr_pos, src_end); + + if (src_curr_pos == src_end) + { + break; + } + else if (src_end - src_curr_pos < 3) + { + src_curr_pos = src_end; + break; + } + else + { + unsigned char high = unhex(src_curr_pos[1]); + unsigned char low = unhex(src_curr_pos[2]); + + if (high != 0xFF && low != 0xFF) + { + unsigned char octet = (high << 4) + low; + + size_t bytes_to_copy = src_curr_pos - src_prev_pos; + memcpySmallAllowReadWriteOverflow15(dst_pos, src_prev_pos, bytes_to_copy); + dst_pos += bytes_to_copy; + + *dst_pos = octet; + ++dst_pos; + + src_prev_pos = src_curr_pos + 3; + } + + src_curr_pos += 3; + } + } + + if (src_prev_pos < src_curr_pos) + { + size_t bytes_to_copy = src_curr_pos - src_prev_pos; + memcpySmallAllowReadWriteOverflow15(dst_pos, src_prev_pos, bytes_to_copy); + dst_pos += bytes_to_copy; + } + + return dst_pos - dst; +} + + +/// Percent decode of URL data. +struct DecodeURLComponentImpl +{ + static void vector(const ColumnString::Chars & data, const ColumnString::Offsets & offsets, + ColumnString::Chars & res_data, ColumnString::Offsets & res_offsets) + { + res_data.resize(data.size()); + size_t size = offsets.size(); + res_offsets.resize(size); + + size_t prev_offset = 0; + size_t res_offset = 0; + + for (size_t i = 0; i < size; ++i) + { + const char * src_data = reinterpret_cast(&data[prev_offset]); + size_t src_size = offsets[i] - prev_offset; + size_t dst_size = decodeURL(src_data, src_size, reinterpret_cast(res_data.data() + res_offset)); + + res_offset += dst_size; + res_offsets[i] = res_offset; + prev_offset = offsets[i]; + } + + res_data.resize(res_offset); + } + + static void vector_fixed(const ColumnString::Chars &, size_t, ColumnString::Chars &) + { + throw Exception("Column of type FixedString is not supported by URL functions", ErrorCodes::ILLEGAL_COLUMN); + } +}; + + +struct NameDecodeURLComponent { static constexpr auto name = "decodeURLComponent"; }; +using FunctionDecodeURLComponent = FunctionStringToString; + +void registerFunctionDecodeURLComponent(FunctionFactory & factory) +{ + factory.registerFunction(); +} + +} diff --git a/dbms/src/Functions/domain.cpp b/dbms/src/Functions/domain.cpp new file mode 100644 index 00000000000..1d6bd307f59 --- /dev/null +++ b/dbms/src/Functions/domain.cpp @@ -0,0 +1,17 @@ +#include +#include +#include + +namespace DB +{ + +struct NameDomain { static constexpr auto name = "domain"; }; +using FunctionDomain = FunctionStringToString>, NameDomain>; + + +void registerFunctionDomain(FunctionFactory & factory) +{ + factory.registerFunction(); +} + +} diff --git a/dbms/src/Functions/domain.h b/dbms/src/Functions/domain.h new file mode 100644 index 00000000000..53a6d7ed4da --- /dev/null +++ b/dbms/src/Functions/domain.h @@ -0,0 +1,71 @@ +#pragma once + +#include +#include +#include + + +namespace DB +{ + +/// Extracts host from given url. +inline StringRef getURLHost(const char * data, size_t size) +{ + Pos pos = data; + Pos end = data + size; + + if (end == (pos = find_first_symbols<'/'>(pos, end))) + return {}; + + if (pos != data) + { + StringRef scheme = getURLScheme(data, size); + Pos scheme_end = data + scheme.size; + + // Colon must follows after scheme. + if (pos - scheme_end != 1 || *scheme_end != ':') + return {}; + } + + if (end - pos < 2 || *(pos) != '/' || *(pos + 1) != '/') + return {}; + pos += 2; + + const char * start_of_host = pos; + for (; pos < end; ++pos) + { + if (*pos == '@') + start_of_host = pos + 1; + else if (*pos == ':' || *pos == '/' || *pos == '?' || *pos == '#') + break; + } + + return (pos == start_of_host) ? StringRef{} : StringRef(start_of_host, pos - start_of_host); +} + +template +struct ExtractDomain +{ + static size_t getReserveLengthForElement() { return 15; } + + static void execute(Pos data, size_t size, Pos & res_data, size_t & res_size) + { + StringRef host = getURLHost(data, size); + + if (host.size == 0) + { + res_data = data; + res_size = 0; + } + else + { + if (without_www && host.size > 4 && !strncmp(host.data, "www.", 4)) + host = { host.data + 4, host.size - 4 }; + + res_data = host.data; + res_size = host.size; + } + } +}; + +} diff --git a/dbms/src/Functions/domainWithoutWWW.cpp b/dbms/src/Functions/domainWithoutWWW.cpp new file mode 100644 index 00000000000..9d54d997042 --- /dev/null +++ b/dbms/src/Functions/domainWithoutWWW.cpp @@ -0,0 +1,17 @@ +#include +#include +#include + +namespace DB +{ + +struct NameDomainWithoutWWW { static constexpr auto name = "domainWithoutWWW"; }; +using FunctionDomainWithoutWWW = FunctionStringToString>, NameDomainWithoutWWW>; + + +void registerFunctionDomainWithoutWWW(FunctionFactory & factory) +{ + factory.registerFunction(); +} + +} diff --git a/dbms/src/Functions/extractURLParameter.cpp b/dbms/src/Functions/extractURLParameter.cpp new file mode 100644 index 00000000000..475bbd76da1 --- /dev/null +++ b/dbms/src/Functions/extractURLParameter.cpp @@ -0,0 +1,101 @@ +#include +#include +#include +#include + +namespace DB +{ + +struct ExtractURLParameterImpl +{ + static void vector(const ColumnString::Chars & data, + const ColumnString::Offsets & offsets, + std::string pattern, + ColumnString::Chars & res_data, ColumnString::Offsets & res_offsets) + { + res_data.reserve(data.size() / 5); + res_offsets.resize(offsets.size()); + + pattern += '='; + const char * param_str = pattern.c_str(); + size_t param_len = pattern.size(); + + ColumnString::Offset prev_offset = 0; + ColumnString::Offset res_offset = 0; + + for (size_t i = 0; i < offsets.size(); ++i) + { + ColumnString::Offset cur_offset = offsets[i]; + + const char * str = reinterpret_cast(&data[prev_offset]); + const char * end = reinterpret_cast(&data[cur_offset]); + + /// Find query string or fragment identifier. + /// Note that we support parameters in fragment identifier in the same way as in query string. + + const char * const query_string_begin = find_first_symbols<'?', '#'>(str, end); + + /// Will point to the beginning of "name=value" pair. Then it will be reassigned to the beginning of "value". + const char * param_begin = nullptr; + + if (query_string_begin + 1 < end) + { + param_begin = query_string_begin + 1; + + while (true) + { + param_begin = static_cast(memmem(param_begin, end - param_begin, param_str, param_len)); + + if (!param_begin) + break; + + if (param_begin[-1] != '?' && param_begin[-1] != '#' && param_begin[-1] != '&') + { + /// Parameter name is different but has the same suffix. + param_begin += param_len; + continue; + } + else + { + param_begin += param_len; + break; + } + } + } + + if (param_begin) + { + const char * param_end = find_first_symbols<'&', '#'>(param_begin, end); + if (param_end == end) + param_end = param_begin + strlen(param_begin); + + size_t param_size = param_end - param_begin; + + res_data.resize(res_offset + param_size + 1); + memcpySmallAllowReadWriteOverflow15(&res_data[res_offset], param_begin, param_size); + res_offset += param_size; + } + else + { + /// No parameter found, put empty string in result. + res_data.resize(res_offset + 1); + } + + res_data[res_offset] = 0; + ++res_offset; + res_offsets[i] = res_offset; + + prev_offset = cur_offset; + } + } +}; + +struct NameExtractURLParameter { static constexpr auto name = "extractURLParameter"; }; +using FunctionExtractURLParameter = FunctionsStringSearchToString; + +void registerFunctionExtractURLParameter(FunctionFactory & factory) +{ + factory.registerFunction(); +} + +} diff --git a/dbms/src/Functions/extractURLParameterNames.cpp b/dbms/src/Functions/extractURLParameterNames.cpp new file mode 100644 index 00000000000..a97581f3bc3 --- /dev/null +++ b/dbms/src/Functions/extractURLParameterNames.cpp @@ -0,0 +1,93 @@ +#include +#include +#include + +namespace DB +{ + +class ExtractURLParameterNamesImpl +{ +private: + Pos pos; + Pos end; + bool first; + +public: + static constexpr auto name = "extractURLParameterNames"; + static String getName() { return name; } + + static size_t getNumberOfArguments() { return 1; } + + static void checkArguments(const DataTypes & arguments) + { + if (!isString(arguments[0])) + throw Exception("Illegal type " + arguments[0]->getName() + " of first argument of function " + getName() + ". Must be String.", + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + } + + /// Returns the position of the argument that is the column of rows + size_t getStringsArgumentPosition() + { + return 0; + } + + void init(Block & /*block*/, const ColumnNumbers & /*arguments*/) {} + + /// Called for each next string. + void set(Pos pos_, Pos end_) + { + pos = pos_; + end = end_; + first = true; + } + + /// Get the next token, if any, or return false. + bool get(Pos & token_begin, Pos & token_end) + { + if (pos == nullptr) + return false; + + if (first) + { + first = false; + pos = find_first_symbols<'?', '#'>(pos, end); + } + else + pos = find_first_symbols<'&', '#'>(pos, end); + + if (pos + 1 >= end) + return false; + ++pos; + + while (true) + { + token_begin = pos; + + pos = find_first_symbols<'=', '&', '#', '?'>(pos, end); + if (pos == end) + return false; + else + token_end = pos; + + if (*pos == '?') + { + ++pos; + continue; + } + + break; + } + + return true; + } +}; + +struct NameExtractURLParameterNames { static constexpr auto name = "extractURLParameterNames"; }; +using FunctionExtractURLParameterNames = FunctionTokens; + +void registerFunctionExtractURLParameterNames(FunctionFactory & factory) +{ + factory.registerFunction(); +} + +} diff --git a/dbms/src/Functions/extractURLParameters.cpp b/dbms/src/Functions/extractURLParameters.cpp new file mode 100644 index 00000000000..15b2ccd222d --- /dev/null +++ b/dbms/src/Functions/extractURLParameters.cpp @@ -0,0 +1,101 @@ +#include +#include +#include + +namespace DB +{ + +class ExtractURLParametersImpl +{ +private: + Pos pos; + Pos end; + bool first; + +public: + static constexpr auto name = "extractURLParameters"; + static String getName() { return name; } + + static size_t getNumberOfArguments() { return 1; } + + static void checkArguments(const DataTypes & arguments) + { + if (!isString(arguments[0])) + throw Exception("Illegal type " + arguments[0]->getName() + " of first argument of function " + getName() + ". Must be String.", + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + } + + void init(Block & /*block*/, const ColumnNumbers & /*arguments*/) {} + + /// Returns the position of the argument that is the column of rows + size_t getStringsArgumentPosition() + { + return 0; + } + + /// Called for each next string. + void set(Pos pos_, Pos end_) + { + pos = pos_; + end = end_; + first = true; + } + + /// Get the next token, if any, or return false. + bool get(Pos & token_begin, Pos & token_end) + { + if (pos == nullptr) + return false; + + if (first) + { + first = false; + pos = find_first_symbols<'?', '#'>(pos, end); + if (pos + 1 >= end) + return false; + ++pos; + } + + while (true) + { + token_begin = pos; + pos = find_first_symbols<'=', '&', '#', '?'>(pos, end); + if (pos == end) + return false; + + if (*pos == '?') + { + ++pos; + continue; + } + + break; + } + + if (*pos == '&' || *pos == '#') + { + token_end = pos++; + } + else + { + ++pos; + pos = find_first_symbols<'&', '#'>(pos, end); + if (pos == end) + token_end = end; + else + token_end = pos++; + } + + return true; + } +}; + +struct NameExtractURLParameters { static constexpr auto name = "extractURLParameters"; }; +using FunctionExtractURLParameters = FunctionTokens; + +void registerFunctionExtractURLParameters(FunctionFactory & factory) +{ + factory.registerFunction(); +} + +} diff --git a/dbms/src/Functions/firstSignificantSubdomain.cpp b/dbms/src/Functions/firstSignificantSubdomain.cpp new file mode 100644 index 00000000000..0b271277f07 --- /dev/null +++ b/dbms/src/Functions/firstSignificantSubdomain.cpp @@ -0,0 +1,17 @@ +#include +#include +#include + + +namespace DB +{ + +struct NameFirstSignificantSubdomain { static constexpr auto name = "firstSignificantSubdomain"; }; +using FunctionFirstSignificantSubdomain = FunctionStringToString, NameFirstSignificantSubdomain>; + +void registerFunctionFirstSignificantSubdomain(FunctionFactory & factory) +{ + factory.registerFunction(); +} + +} diff --git a/dbms/src/Functions/firstSignificantSubdomain.h b/dbms/src/Functions/firstSignificantSubdomain.h new file mode 100644 index 00000000000..f464c3fdead --- /dev/null +++ b/dbms/src/Functions/firstSignificantSubdomain.h @@ -0,0 +1,91 @@ +#pragma once + +#include +#include + + +namespace DB +{ + +struct ExtractFirstSignificantSubdomain +{ + static size_t getReserveLengthForElement() { return 10; } + + static void execute(const Pos data, const size_t size, Pos & res_data, size_t & res_size, Pos * out_domain_end = nullptr) + { + res_data = data; + res_size = 0; + + Pos tmp; + size_t domain_length; + ExtractDomain::execute(data, size, tmp, domain_length); + + if (domain_length == 0) + return; + + if (out_domain_end) + *out_domain_end = tmp + domain_length; + + /// cut useless dot + if (tmp[domain_length - 1] == '.') + --domain_length; + + res_data = tmp; + res_size = domain_length; + + auto begin = tmp; + auto end = begin + domain_length; + const char * last_3_periods[3]{}; + + auto pos = find_first_symbols<'.'>(begin, end); + while (pos < end) + { + last_3_periods[2] = last_3_periods[1]; + last_3_periods[1] = last_3_periods[0]; + last_3_periods[0] = pos; + pos = find_first_symbols<'.'>(pos + 1, end); + } + + if (!last_3_periods[0]) + return; + + if (!last_3_periods[1]) + { + res_size = last_3_periods[0] - begin; + return; + } + + if (!last_3_periods[2]) + last_3_periods[2] = begin - 1; + + size_t size_of_second_subdomain_plus_period = last_3_periods[0] - last_3_periods[1]; + if (size_of_second_subdomain_plus_period == 4 || size_of_second_subdomain_plus_period == 3) + { + /// We will key by four bytes that are either ".xyz" or ".xy.". + UInt32 key = unalignedLoad(last_3_periods[1]); + + /// NOTE: assuming little endian. + /// NOTE: does the compiler generate SIMD code? + /// NOTE: for larger amount of cases we can use a perfect hash table (see 'gperf' as an example). + if ( key == '.' + 'c' * 0x100U + 'o' * 0x10000U + 'm' * 0x1000000U + || key == '.' + 'n' * 0x100U + 'e' * 0x10000U + 't' * 0x1000000U + || key == '.' + 'o' * 0x100U + 'r' * 0x10000U + 'g' * 0x1000000U + || key == '.' + 'b' * 0x100U + 'i' * 0x10000U + 'z' * 0x1000000U + || key == '.' + 'g' * 0x100U + 'o' * 0x10000U + 'v' * 0x1000000U + || key == '.' + 'm' * 0x100U + 'i' * 0x10000U + 'l' * 0x1000000U + || key == '.' + 'e' * 0x100U + 'd' * 0x10000U + 'u' * 0x1000000U + || key == '.' + 'c' * 0x100U + 'o' * 0x10000U + '.' * 0x1000000U) + { + res_data += last_3_periods[2] + 1 - begin; + res_size = last_3_periods[1] - last_3_periods[2] - 1; + return; + } + } + + res_data += last_3_periods[1] + 1 - begin; + res_size = last_3_periods[0] - last_3_periods[1] - 1; + } +}; + +} + diff --git a/dbms/src/Functions/fragment.cpp b/dbms/src/Functions/fragment.cpp new file mode 100644 index 00000000000..bc4ab6bb6de --- /dev/null +++ b/dbms/src/Functions/fragment.cpp @@ -0,0 +1,16 @@ +#include +#include +#include + +namespace DB +{ + +struct NameFragment { static constexpr auto name = "fragment"; }; +using FunctionFragment = FunctionStringToString>, NameFragment>; + +void registerFunctionFragment(FunctionFactory & factory) +{ + factory.registerFunction(); +} + +} diff --git a/dbms/src/Functions/fragment.h b/dbms/src/Functions/fragment.h new file mode 100644 index 00000000000..ffe507d8e7a --- /dev/null +++ b/dbms/src/Functions/fragment.h @@ -0,0 +1,30 @@ +#pragma once + +#include +#include + +namespace DB +{ + +template +struct ExtractFragment +{ + static size_t getReserveLengthForElement() { return 10; } + + static void execute(Pos data, size_t size, Pos & res_data, size_t & res_size) + { + res_data = data; + res_size = 0; + + Pos pos = data; + Pos end = pos + size; + + if (end != (pos = find_first_symbols<'#'>(pos, end))) + { + res_data = pos + (without_leading_char ? 1 : 0); + res_size = end - res_data; + } + } +}; + +} diff --git a/dbms/src/Functions/path.cpp b/dbms/src/Functions/path.cpp new file mode 100644 index 00000000000..e24137e20b4 --- /dev/null +++ b/dbms/src/Functions/path.cpp @@ -0,0 +1,39 @@ +#include +#include +#include +#include + +namespace DB +{ + +struct ExtractPath +{ + static size_t getReserveLengthForElement() { return 25; } + + static void execute(Pos data, size_t size, Pos & res_data, size_t & res_size) + { + res_data = data; + res_size = 0; + + Pos pos = data; + Pos end = pos + size; + + if (end != (pos = find_first_symbols<'/'>(pos, end)) && pos[1] == '/' && end != (pos = find_first_symbols<'/'>(pos + 2, end))) + { + Pos query_string_or_fragment = find_first_symbols<'?', '#'>(pos, end); + + res_data = pos; + res_size = query_string_or_fragment - res_data; + } + } +}; + +struct NamePath { static constexpr auto name = "path"; }; +using FunctionPath = FunctionStringToString, NamePath>; + +void registerFunctionPath(FunctionFactory & factory) +{ + factory.registerFunction(); +} + +} diff --git a/dbms/src/Functions/pathFull.cpp b/dbms/src/Functions/pathFull.cpp new file mode 100644 index 00000000000..2f0772eb867 --- /dev/null +++ b/dbms/src/Functions/pathFull.cpp @@ -0,0 +1,37 @@ +#include +#include +#include +#include + +namespace DB +{ + +struct ExtractPathFull +{ + static size_t getReserveLengthForElement() { return 30; } + + static void execute(const Pos data, const size_t size, Pos & res_data, size_t & res_size) + { + res_data = data; + res_size = 0; + + Pos pos = data; + Pos end = pos + size; + + if (end != (pos = find_first_symbols<'/'>(pos, end)) && pos[1] == '/' && end != (pos = find_first_symbols<'/'>(pos + 2, end))) + { + res_data = pos; + res_size = end - res_data; + } + } +}; + +struct NamePathFull { static constexpr auto name = "pathFull"; }; +using FunctionPathFull = FunctionStringToString, NamePathFull>; + +void registerFunctionPathFull(FunctionFactory & factory) +{ + factory.registerFunction(); +} + +} diff --git a/dbms/src/Functions/protocol.cpp b/dbms/src/Functions/protocol.cpp new file mode 100644 index 00000000000..70615e121e3 --- /dev/null +++ b/dbms/src/Functions/protocol.cpp @@ -0,0 +1,17 @@ +#include +#include +#include + + +namespace DB +{ + +struct NameProtocol { static constexpr auto name = "protocol"; }; +using FunctionProtocol = FunctionStringToString, NameProtocol>; + +void registerFunctionProtocol(FunctionFactory & factory) +{ + factory.registerFunction(); +} + +} diff --git a/dbms/src/Functions/protocol.h b/dbms/src/Functions/protocol.h new file mode 100644 index 00000000000..6d188687a45 --- /dev/null +++ b/dbms/src/Functions/protocol.h @@ -0,0 +1,57 @@ +#pragma once + +#include +#include + + +namespace DB +{ + +/// Extracts scheme from given url. +inline StringRef getURLScheme(const char * data, size_t size) +{ + // scheme = ALPHA *( ALPHA / DIGIT / "+" / "-" / "." ) + const char * pos = data; + const char * end = data + size; + + if (isAlphaASCII(*pos)) + { + for (++pos; pos < end; ++pos) + { + if (!(isAlphaNumericASCII(*pos) || *pos == '+' || *pos == '-' || *pos == '.')) + { + break; + } + } + + return StringRef(data, pos - data); + } + + return {}; +} + +struct ExtractProtocol +{ + static size_t getReserveLengthForElement() + { + return strlen("https") + 1; + } + + static void execute(Pos data, size_t size, Pos & res_data, size_t & res_size) + { + res_data = data; + res_size = 0; + + StringRef scheme = getURLScheme(data, size); + Pos pos = data + scheme.size; + + if (scheme.size == 0 || (data + size) - pos < 4) + return; + + if (pos[0] == ':') + res_size = pos - data; + } +}; + +} + diff --git a/dbms/src/Functions/queryString.cpp b/dbms/src/Functions/queryString.cpp new file mode 100644 index 00000000000..40a55b0d0bd --- /dev/null +++ b/dbms/src/Functions/queryString.cpp @@ -0,0 +1,16 @@ +#include +#include +#include + +namespace DB +{ + +struct NameQueryString { static constexpr auto name = "queryString"; }; +using FunctionQueryString = FunctionStringToString>, NameQueryString>; + +void registerFunctionQueryString(FunctionFactory & factory) +{ + factory.registerFunction(); +} + +} diff --git a/dbms/src/Functions/queryString.h b/dbms/src/Functions/queryString.h new file mode 100644 index 00000000000..e422a8b1419 --- /dev/null +++ b/dbms/src/Functions/queryString.h @@ -0,0 +1,33 @@ +#pragma once + +#include +#include + + +namespace DB +{ + +template +struct ExtractQueryString +{ + static size_t getReserveLengthForElement() { return 10; } + + static void execute(Pos data, size_t size, Pos & res_data, size_t & res_size) + { + res_data = data; + res_size = 0; + + Pos pos = data; + Pos end = pos + size; + + if (end != (pos = find_first_symbols<'?'>(pos, end))) + { + Pos fragment = find_first_symbols<'#'>(pos, end); + + res_data = pos + (without_leading_char ? 1 : 0); + res_size = fragment - res_data; + } + } +}; + +} diff --git a/dbms/src/Functions/queryStringAndFragment.cpp b/dbms/src/Functions/queryStringAndFragment.cpp new file mode 100644 index 00000000000..cc0fd55eb17 --- /dev/null +++ b/dbms/src/Functions/queryStringAndFragment.cpp @@ -0,0 +1,16 @@ +#include +#include +#include + +namespace DB +{ + +struct NameQueryStringAndFragment { static constexpr auto name = "queryStringAndFragment"; }; +using FunctionQueryStringAndFragment = FunctionStringToString>, NameQueryStringAndFragment>; + +void registerFunctionQueryStringAndFragment(FunctionFactory & factory) +{ + factory.registerFunction(); +} + +} diff --git a/dbms/src/Functions/queryStringAndFragment.h b/dbms/src/Functions/queryStringAndFragment.h new file mode 100644 index 00000000000..33f1a198182 --- /dev/null +++ b/dbms/src/Functions/queryStringAndFragment.h @@ -0,0 +1,37 @@ +#pragma once + +#include +#include + + +namespace DB +{ + +template +struct ExtractQueryStringAndFragment +{ + static size_t getReserveLengthForElement() { return 20; } + + static void execute(Pos data, size_t size, Pos & res_data, size_t & res_size) + { + res_data = data; + res_size = 0; + + Pos pos = data; + Pos end = pos + size; + + if (end != (pos = find_first_symbols<'?'>(pos, end))) + { + res_data = pos + (without_leading_char ? 1 : 0); + res_size = end - res_data; + } + else if (end != (pos = find_first_symbols<'#'>(pos, end))) + { + res_data = pos; + res_size = end - res_data; + } + } +}; + +} + diff --git a/dbms/src/Functions/registerFunctionsArray.cpp b/dbms/src/Functions/registerFunctionsArray.cpp index 0cf80bc4f73..ac7a5b38cad 100644 --- a/dbms/src/Functions/registerFunctionsArray.cpp +++ b/dbms/src/Functions/registerFunctionsArray.cpp @@ -1,8 +1,8 @@ -#include - namespace DB { +class FunctionFactory; + void registerFunctionArray(FunctionFactory &); void registerFunctionArrayElement(FunctionFactory &); void registerFunctionArrayResize(FunctionFactory &); diff --git a/dbms/src/Functions/registerFunctionsComparison.cpp b/dbms/src/Functions/registerFunctionsComparison.cpp index 15f5e33d0f6..f9ceb6e4821 100644 --- a/dbms/src/Functions/registerFunctionsComparison.cpp +++ b/dbms/src/Functions/registerFunctionsComparison.cpp @@ -1,8 +1,8 @@ -#include - namespace DB { +class FunctionFactory; + void registerFunctionEquals(FunctionFactory & factory); void registerFunctionNotEquals(FunctionFactory & factory); void registerFunctionLess(FunctionFactory & factory); diff --git a/dbms/src/Functions/registerFunctionsDateTime.cpp b/dbms/src/Functions/registerFunctionsDateTime.cpp index a48c1e445cf..3e7f2a6affd 100644 --- a/dbms/src/Functions/registerFunctionsDateTime.cpp +++ b/dbms/src/Functions/registerFunctionsDateTime.cpp @@ -1,9 +1,8 @@ -#include - - namespace DB { +class FunctionFactory; + void registerFunctionToYear(FunctionFactory &); void registerFunctionToQuarter(FunctionFactory &); void registerFunctionToMonth(FunctionFactory &); diff --git a/dbms/src/Functions/registerFunctionsHigherOrder.cpp b/dbms/src/Functions/registerFunctionsHigherOrder.cpp index 6df3e335c08..e0948ebc913 100644 --- a/dbms/src/Functions/registerFunctionsHigherOrder.cpp +++ b/dbms/src/Functions/registerFunctionsHigherOrder.cpp @@ -1,8 +1,8 @@ -#include - namespace DB { +class FunctionFactory; + void registerFunctionArrayMap(FunctionFactory &); void registerFunctionArrayFilter(FunctionFactory &); void registerFunctionArrayCount(FunctionFactory &); diff --git a/dbms/src/Functions/registerFunctionsMiscellaneous.cpp b/dbms/src/Functions/registerFunctionsMiscellaneous.cpp index e6fe5ca1524..e483deee961 100644 --- a/dbms/src/Functions/registerFunctionsMiscellaneous.cpp +++ b/dbms/src/Functions/registerFunctionsMiscellaneous.cpp @@ -1,8 +1,8 @@ -#include - namespace DB { +class FunctionFactory; + void registerFunctionCurrentDatabase(FunctionFactory &); void registerFunctionHostName(FunctionFactory &); void registerFunctionVisibleWidth(FunctionFactory &); diff --git a/dbms/src/Functions/registerFunctionsNull.cpp b/dbms/src/Functions/registerFunctionsNull.cpp index e97c128018e..0cc447c3db3 100644 --- a/dbms/src/Functions/registerFunctionsNull.cpp +++ b/dbms/src/Functions/registerFunctionsNull.cpp @@ -1,8 +1,8 @@ -#include - namespace DB { +class FunctionFactory; + void registerFunctionIsNull(FunctionFactory & factory); void registerFunctionIsNotNull(FunctionFactory & factory); void registerFunctionCoalesce(FunctionFactory & factory); diff --git a/dbms/src/Functions/registerFunctionsString.cpp b/dbms/src/Functions/registerFunctionsString.cpp index 5d4b05acdf6..3a07d8bbd65 100644 --- a/dbms/src/Functions/registerFunctionsString.cpp +++ b/dbms/src/Functions/registerFunctionsString.cpp @@ -1,8 +1,10 @@ -#include +#include namespace DB { +class FunctionFactory; + void registerFunctionEmpty(FunctionFactory &); void registerFunctionNotEmpty(FunctionFactory &); void registerFunctionLength(FunctionFactory &); @@ -19,6 +21,7 @@ void registerFunctionSubstringUTF8(FunctionFactory &); void registerFunctionAppendTrailingCharIfAbsent(FunctionFactory &); void registerFunctionStartsWith(FunctionFactory &); void registerFunctionEndsWith(FunctionFactory &); + #if USE_BASE64 void registerFunctionBase64Encode(FunctionFactory &); void registerFunctionBase64Decode(FunctionFactory &); diff --git a/dbms/src/Functions/registerFunctionsTuple.cpp b/dbms/src/Functions/registerFunctionsTuple.cpp index 051ac9a6384..12092e1e7e0 100644 --- a/dbms/src/Functions/registerFunctionsTuple.cpp +++ b/dbms/src/Functions/registerFunctionsTuple.cpp @@ -1,8 +1,8 @@ -#include - namespace DB { +class FunctionFactory; + void registerFunctionTuple(FunctionFactory &); void registerFunctionTupleElement(FunctionFactory &); diff --git a/dbms/src/Functions/registerFunctionsURL.cpp b/dbms/src/Functions/registerFunctionsURL.cpp new file mode 100644 index 00000000000..906b2adfb3f --- /dev/null +++ b/dbms/src/Functions/registerFunctionsURL.cpp @@ -0,0 +1,56 @@ +namespace DB +{ + +class FunctionFactory; + +void registerFunctionProtocol(FunctionFactory &); +void registerFunctionDomain(FunctionFactory &); +void registerFunctionDomainWithoutWWW(FunctionFactory &); +void registerFunctionFirstSignificantSubdomain(FunctionFactory &); +void registerFunctionTopLevelDomain(FunctionFactory &); +void registerFunctionPath(FunctionFactory &); +void registerFunctionPathFull(FunctionFactory &); +void registerFunctionQueryString(FunctionFactory &); +void registerFunctionFragment(FunctionFactory &); +void registerFunctionQueryStringAndFragment(FunctionFactory &); +void registerFunctionExtractURLParameter(FunctionFactory &); +void registerFunctionExtractURLParameters(FunctionFactory &); +void registerFunctionExtractURLParameterNames(FunctionFactory &); +void registerFunctionURLHierarchy(FunctionFactory &); +void registerFunctionURLPathHierarchy(FunctionFactory &); +void registerFunctionCutToFirstSignificantSubdomain(FunctionFactory &); +void registerFunctionCutWWW(FunctionFactory &); +void registerFunctionCutQueryString(FunctionFactory &); +void registerFunctionCutFragment(FunctionFactory &); +void registerFunctionCutQueryStringAndFragment(FunctionFactory &); +void registerFunctionCutURLParameter(FunctionFactory &); +void registerFunctionDecodeURLComponent(FunctionFactory &); + +void registerFunctionsURL(FunctionFactory & factory) +{ + registerFunctionProtocol(factory); + registerFunctionDomain(factory); + registerFunctionDomainWithoutWWW(factory); + registerFunctionFirstSignificantSubdomain(factory); + registerFunctionTopLevelDomain(factory); + registerFunctionPath(factory); + registerFunctionPathFull(factory); + registerFunctionQueryString(factory); + registerFunctionFragment(factory); + registerFunctionQueryStringAndFragment(factory); + registerFunctionExtractURLParameter(factory); + registerFunctionExtractURLParameters(factory); + registerFunctionExtractURLParameterNames(factory); + registerFunctionURLHierarchy(factory); + registerFunctionURLPathHierarchy(factory); + registerFunctionCutToFirstSignificantSubdomain(factory); + registerFunctionCutWWW(factory); + registerFunctionCutQueryString(factory); + registerFunctionCutFragment(factory); + registerFunctionCutQueryStringAndFragment(factory); + registerFunctionCutURLParameter(factory); + registerFunctionDecodeURLComponent(factory); +} + +} + diff --git a/dbms/src/Functions/topLevelDomain.cpp b/dbms/src/Functions/topLevelDomain.cpp new file mode 100644 index 00000000000..abee54c596a --- /dev/null +++ b/dbms/src/Functions/topLevelDomain.cpp @@ -0,0 +1,48 @@ +#include +#include +#include + +namespace DB +{ + +struct ExtractTopLevelDomain +{ + static size_t getReserveLengthForElement() { return 5; } + + static void execute(Pos data, size_t size, Pos & res_data, size_t & res_size) + { + StringRef host = getURLHost(data, size); + + res_data = data; + res_size = 0; + + if (host.size != 0) + { + if (host.data[host.size - 1] == '.') + host.size -= 1; + + auto host_end = host.data + host.size; + + Pos last_dot = find_last_symbols_or_null<'.'>(host.data, host_end); + if (!last_dot) + return; + + /// For IPv4 addresses select nothing. + if (last_dot[1] <= '9') + return; + + res_data = last_dot + 1; + res_size = host_end - res_data; + } + } +}; + +struct NameTopLevelDomain { static constexpr auto name = "topLevelDomain"; }; +using FunctionTopLevelDomain = FunctionStringToString, NameTopLevelDomain>; + +void registerFunctionTopLevelDomain(FunctionFactory & factory) +{ + factory.registerFunction(); +} + +} From c77930fe30824ff09a005b11ebd23d4d74c6d6ad Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 26 Nov 2018 03:56:50 +0300 Subject: [PATCH 113/124] Whitespaces [#CLICKHOUSE-2] --- dbms/programs/client/Client.cpp | 2 +- dbms/programs/copier/ClusterCopier.cpp | 2 +- dbms/programs/main.cpp | 2 +- dbms/programs/server/Server.cpp | 2 +- dbms/programs/server/TCPHandler.cpp | 2 +- dbms/programs/server/TCPHandler.h | 2 +- .../AggregateFunctionGroupArray.cpp | 4 +- .../AggregateFunctionStatistics.h | 2 +- dbms/src/Client/Connection.cpp | 2 +- .../src/Client/ConnectionPoolWithFailover.cpp | 2 +- dbms/src/Columns/ColumnArray.cpp | 28 +++---- dbms/src/Columns/ColumnLowCardinality.cpp | 2 +- dbms/src/Columns/FilterDescription.cpp | 2 +- dbms/src/Columns/IColumn.h | 2 +- dbms/src/Columns/ReverseIndex.h | 2 +- dbms/src/Common/DNSResolver.cpp | 2 +- dbms/src/Common/FieldVisitors.h | 16 ++-- dbms/src/Common/HashTable/HashTable.h | 8 +- dbms/src/Common/HyperLogLogCounter.h | 6 +- dbms/src/Common/ProfileEvents.h | 2 +- dbms/src/Common/Stopwatch.h | 34 ++++---- dbms/src/Common/ThreadProfileEvents.h | 4 +- dbms/src/Common/ZooKeeper/IKeeper.cpp | 48 +++++------ dbms/src/Common/formatIPv6.cpp | 2 +- dbms/src/Common/isLocalAddress.h | 2 +- dbms/src/Core/QueryProcessingStage.h | 6 +- dbms/src/DataTypes/DataTypeLowCardinality.cpp | 2 +- dbms/src/DataTypes/DataTypeLowCardinality.h | 2 +- dbms/src/DataTypes/FieldToDataType.cpp | 4 +- dbms/src/DataTypes/NumberTraits.h | 24 +++--- .../Embedded/RegionsHierarchy.cpp | 2 +- .../Dictionaries/MongoDBDictionarySource.cpp | 2 +- .../src/Dictionaries/XDBCDictionarySource.cpp | 2 +- dbms/src/Functions/FunctionsCoding.h | 2 +- dbms/src/Functions/FunctionsConditional.cpp | 4 +- dbms/src/Functions/FunctionsExternalModels.h | 2 +- dbms/src/Functions/FunctionsMiscellaneous.h | 2 +- dbms/src/Functions/FunctionsProjection.cpp | 2 +- dbms/src/Functions/FunctionsVisitParam.h | 2 +- dbms/src/Functions/GatherUtils/Algorithms.h | 2 +- dbms/src/Functions/arrayFirstIndex.cpp | 2 +- dbms/src/Functions/arrayUniq.cpp | 4 +- dbms/src/Functions/formatDateTime.cpp | 2 +- dbms/src/IO/DoubleConverter.h | 2 +- dbms/src/IO/Operators.h | 8 +- dbms/src/IO/ReadHelpers.cpp | 2 +- dbms/src/IO/WriteBufferAIO.cpp | 2 +- dbms/src/IO/ZlibDeflatingWriteBuffer.cpp | 12 +-- dbms/src/IO/ZlibInflatingReadBuffer.cpp | 12 +-- dbms/src/IO/tests/hashing_read_buffer.cpp | 2 +- dbms/src/IO/tests/parse_int_perf.cpp | 4 +- dbms/src/IO/tests/read_buffer_aio.cpp | 4 +- .../ClusterProxy/SelectStreamFactory.cpp | 2 +- dbms/src/Interpreters/Compiler.cpp | 6 +- dbms/src/Interpreters/ExpressionAnalyzer.cpp | 2 +- .../LogicalExpressionsOptimizer.cpp | 2 +- dbms/src/Interpreters/PartLog.cpp | 30 +++---- dbms/src/Interpreters/SettingsCommon.cpp | 10 +-- dbms/src/Interpreters/convertFieldToType.cpp | 2 +- .../tests/expression_analyzer.cpp | 2 +- .../Interpreters/tests/hash_map_string_2.cpp | 80 +++++++++---------- .../Interpreters/tests/hash_map_string_3.cpp | 2 +- dbms/src/Parsers/ASTInsertQuery.h | 2 +- dbms/src/Parsers/ParserDropQuery.h | 2 +- dbms/src/Storages/MergeTree/KeyCondition.cpp | 2 +- dbms/src/Storages/MergeTree/MergeTreeData.cpp | 2 +- .../Storages/MergeTree/MergeTreeDataPart.cpp | 2 +- .../Storages/MergeTree/MergeTreeReader.cpp | 2 +- .../Storages/MergeTree/MergeTreeSettings.h | 2 +- .../MergeTree/ReplicatedMergeTreeQueue.cpp | 2 +- .../ReplicatedMergeTreeRestartingThread.cpp | 2 +- .../MergeTree/registerStorageMergeTree.cpp | 4 +- dbms/src/Storages/StorageCatBoostPool.cpp | 2 +- dbms/src/Storages/StorageLog.cpp | 2 +- dbms/src/Storages/StorageMerge.cpp | 2 +- dbms/src/Storages/StorageMergeTree.h | 2 +- .../Storages/StorageReplicatedMergeTree.cpp | 10 +-- .../System/StorageSystemMutations.cpp | 20 +++-- .../Storages/System/StorageSystemTables.cpp | 2 +- .../get_current_inserts_in_replicated.cpp | 8 +- .../TableFunctions/TableFunctionRemote.cpp | 2 +- 81 files changed, 253 insertions(+), 255 deletions(-) diff --git a/dbms/programs/client/Client.cpp b/dbms/programs/client/Client.cpp index ee2a940e24f..ee57b322b94 100644 --- a/dbms/programs/client/Client.cpp +++ b/dbms/programs/client/Client.cpp @@ -526,7 +526,7 @@ private: if (max_client_network_bandwidth) { - ThrottlerPtr throttler = std::make_shared(max_client_network_bandwidth, 0, ""); + ThrottlerPtr throttler = std::make_shared(max_client_network_bandwidth, 0, ""); connection->setThrottler(throttler); } diff --git a/dbms/programs/copier/ClusterCopier.cpp b/dbms/programs/copier/ClusterCopier.cpp index edbe49e36be..02d371957ef 100644 --- a/dbms/programs/copier/ClusterCopier.cpp +++ b/dbms/programs/copier/ClusterCopier.cpp @@ -323,7 +323,7 @@ struct TaskTable struct TaskCluster { TaskCluster(const String & task_zookeeper_path_, const String & default_local_database_) - : task_zookeeper_path(task_zookeeper_path_), default_local_database(default_local_database_) {} + : task_zookeeper_path(task_zookeeper_path_), default_local_database(default_local_database_) {} void loadTasks(const Poco::Util::AbstractConfiguration & config, const String & base_key = ""); diff --git a/dbms/programs/main.cpp b/dbms/programs/main.cpp index adffde3979e..29d64213d9c 100644 --- a/dbms/programs/main.cpp +++ b/dbms/programs/main.cpp @@ -1,7 +1,7 @@ #include #include #include -#include /// pair +#include /// pair #if __has_include("config_tools.h") #include "config_tools.h" diff --git a/dbms/programs/server/Server.cpp b/dbms/programs/server/Server.cpp index 832f02d16da..03d9a8268c0 100644 --- a/dbms/programs/server/Server.cpp +++ b/dbms/programs/server/Server.cpp @@ -637,7 +637,7 @@ int Server::main(const std::vector & /*args*/) message << "Available RAM = " << formatReadableSizeWithBinarySuffix(memory_amount) << ";" << " physical cores = " << getNumberOfPhysicalCPUCores() << ";" // on ARM processors it can show only enabled at current moment cores - << " threads = " << std::thread::hardware_concurrency() << "."; + << " threads = " << std::thread::hardware_concurrency() << "."; LOG_INFO(log, message.str()); } diff --git a/dbms/programs/server/TCPHandler.cpp b/dbms/programs/server/TCPHandler.cpp index 63e60d22c0f..834369582af 100644 --- a/dbms/programs/server/TCPHandler.cpp +++ b/dbms/programs/server/TCPHandler.cpp @@ -181,7 +181,7 @@ void TCPHandler::runImpl() /// Reset the input stream, as we received an empty block while receiving external table data. /// So, the stream has been marked as cancelled and we can't read from it anymore. state.block_in.reset(); - state.maybe_compressed_in.reset(); /// For more accurate accounting by MemoryTracker. + state.maybe_compressed_in.reset(); /// For more accurate accounting by MemoryTracker. }); /// Processing Query diff --git a/dbms/programs/server/TCPHandler.h b/dbms/programs/server/TCPHandler.h index af422921f07..098019e86da 100644 --- a/dbms/programs/server/TCPHandler.h +++ b/dbms/programs/server/TCPHandler.h @@ -89,7 +89,7 @@ public: , connection_context(server.context()) , query_context(server.context()) { - server_display_name = server.config().getString("display_name", getFQDNOrHostName()); + server_display_name = server.config().getString("display_name", getFQDNOrHostName()); } void run(); diff --git a/dbms/src/AggregateFunctions/AggregateFunctionGroupArray.cpp b/dbms/src/AggregateFunctions/AggregateFunctionGroupArray.cpp index 842fe5c3e14..51bc04ea39c 100644 --- a/dbms/src/AggregateFunctions/AggregateFunctionGroupArray.cpp +++ b/dbms/src/AggregateFunctions/AggregateFunctionGroupArray.cpp @@ -55,10 +55,10 @@ static AggregateFunctionPtr createAggregateFunctionGroupArray(const std::string else if (parameters.size() == 1) { auto type = parameters[0].getType(); - if (type != Field::Types::Int64 && type != Field::Types::UInt64) + if (type != Field::Types::Int64 && type != Field::Types::UInt64) throw Exception("Parameter for aggregate function " + name + " should be positive number", ErrorCodes::BAD_ARGUMENTS); - if ((type == Field::Types::Int64 && parameters[0].get() < 0) || + if ((type == Field::Types::Int64 && parameters[0].get() < 0) || (type == Field::Types::UInt64 && parameters[0].get() == 0)) throw Exception("Parameter for aggregate function " + name + " should be positive number", ErrorCodes::BAD_ARGUMENTS); diff --git a/dbms/src/AggregateFunctions/AggregateFunctionStatistics.h b/dbms/src/AggregateFunctions/AggregateFunctionStatistics.h index 75ccf50e44b..82d34fc2954 100644 --- a/dbms/src/AggregateFunctions/AggregateFunctionStatistics.h +++ b/dbms/src/AggregateFunctions/AggregateFunctionStatistics.h @@ -274,7 +274,7 @@ public: ++count; - left_mean += left_delta / count; + left_mean += left_delta / count; right_mean += right_delta / count; co_moment += (left_val - left_mean) * (right_val - old_right_mean); diff --git a/dbms/src/Client/Connection.cpp b/dbms/src/Client/Connection.cpp index 8915504d133..60fd070064b 100644 --- a/dbms/src/Client/Connection.cpp +++ b/dbms/src/Client/Connection.cpp @@ -696,7 +696,7 @@ void Connection::setDescription() { auto resolved_address = getResolvedAddress(); description = host + ":" + toString(resolved_address.port()); - auto ip_address = resolved_address.host().toString(); + auto ip_address = resolved_address.host().toString(); if (host != ip_address) description += ", " + ip_address; diff --git a/dbms/src/Client/ConnectionPoolWithFailover.cpp b/dbms/src/Client/ConnectionPoolWithFailover.cpp index d71d1f08339..7687742169f 100644 --- a/dbms/src/Client/ConnectionPoolWithFailover.cpp +++ b/dbms/src/Client/ConnectionPoolWithFailover.cpp @@ -203,7 +203,7 @@ ConnectionPoolWithFailover::tryGetEntry( LOG_TRACE( log, "Server " << result.entry->getDescription() << " has unacceptable replica delay " << "for table " << table_to_check->database << "." << table_to_check->table - << ": " << delay); + << ": " << delay); ProfileEvents::increment(ProfileEvents::DistributedConnectionStaleReplica); } } diff --git a/dbms/src/Columns/ColumnArray.cpp b/dbms/src/Columns/ColumnArray.cpp index f9e0ff3219b..4a9b54b08d5 100644 --- a/dbms/src/Columns/ColumnArray.cpp +++ b/dbms/src/Columns/ColumnArray.cpp @@ -697,20 +697,20 @@ ColumnPtr ColumnArray::replicate(const Offsets & replicate_offsets) const if (replicate_offsets.empty()) return cloneEmpty(); - if (typeid_cast(data.get())) return replicateNumber(replicate_offsets); - if (typeid_cast(data.get())) return replicateNumber(replicate_offsets); - if (typeid_cast(data.get())) return replicateNumber(replicate_offsets); - if (typeid_cast(data.get())) return replicateNumber(replicate_offsets); - if (typeid_cast(data.get())) return replicateNumber(replicate_offsets); - if (typeid_cast(data.get())) return replicateNumber(replicate_offsets); - if (typeid_cast(data.get())) return replicateNumber(replicate_offsets); - if (typeid_cast(data.get())) return replicateNumber(replicate_offsets); - if (typeid_cast(data.get())) return replicateNumber(replicate_offsets); - if (typeid_cast(data.get())) return replicateNumber(replicate_offsets); - if (typeid_cast(data.get())) return replicateString(replicate_offsets); - if (typeid_cast(data.get())) return replicateConst(replicate_offsets); - if (typeid_cast(data.get())) return replicateNullable(replicate_offsets); - if (typeid_cast(data.get())) return replicateTuple(replicate_offsets); + if (typeid_cast(data.get())) return replicateNumber(replicate_offsets); + if (typeid_cast(data.get())) return replicateNumber(replicate_offsets); + if (typeid_cast(data.get())) return replicateNumber(replicate_offsets); + if (typeid_cast(data.get())) return replicateNumber(replicate_offsets); + if (typeid_cast(data.get())) return replicateNumber(replicate_offsets); + if (typeid_cast(data.get())) return replicateNumber(replicate_offsets); + if (typeid_cast(data.get())) return replicateNumber(replicate_offsets); + if (typeid_cast(data.get())) return replicateNumber(replicate_offsets); + if (typeid_cast(data.get())) return replicateNumber(replicate_offsets); + if (typeid_cast(data.get())) return replicateNumber(replicate_offsets); + if (typeid_cast(data.get())) return replicateString(replicate_offsets); + if (typeid_cast(data.get())) return replicateConst(replicate_offsets); + if (typeid_cast(data.get())) return replicateNullable(replicate_offsets); + if (typeid_cast(data.get())) return replicateTuple(replicate_offsets); return replicateGeneric(replicate_offsets); } diff --git a/dbms/src/Columns/ColumnLowCardinality.cpp b/dbms/src/Columns/ColumnLowCardinality.cpp index 873ecfa0613..da6bcfa6597 100644 --- a/dbms/src/Columns/ColumnLowCardinality.cpp +++ b/dbms/src/Columns/ColumnLowCardinality.cpp @@ -604,7 +604,7 @@ void ColumnLowCardinality::Index::check(size_t /*max_dictionary_size*/) void ColumnLowCardinality::Index::checkSizeOfType() { if (size_of_type != getSizeOfIndexType(*positions, size_of_type)) - throw Exception("Invalid size of type. Expected " + toString(8 * size_of_type) + + throw Exception("Invalid size of type. Expected " + toString(8 * size_of_type) + ", but positions are " + positions->getName(), ErrorCodes::LOGICAL_ERROR); } diff --git a/dbms/src/Columns/FilterDescription.cpp b/dbms/src/Columns/FilterDescription.cpp index 104599ba974..f9fb39a9f47 100644 --- a/dbms/src/Columns/FilterDescription.cpp +++ b/dbms/src/Columns/FilterDescription.cpp @@ -72,7 +72,7 @@ FilterDescription::FilterDescription(const IColumn & column_) ErrorCodes::ILLEGAL_TYPE_OF_COLUMN_FOR_FILTER); const NullMap & null_map = nullable_column->getNullMapData(); - IColumn::Filter & res = concrete_column->getData(); + IColumn::Filter & res = concrete_column->getData(); size_t size = res.size(); for (size_t i = 0; i < size; ++i) diff --git a/dbms/src/Columns/IColumn.h b/dbms/src/Columns/IColumn.h index 82a68662588..088be128e8d 100644 --- a/dbms/src/Columns/IColumn.h +++ b/dbms/src/Columns/IColumn.h @@ -377,7 +377,7 @@ protected: }; using ColumnPtr = IColumn::Ptr; -using MutableColumnPtr = IColumn::MutablePtr; +using MutableColumnPtr = IColumn::MutablePtr; using Columns = std::vector; using MutableColumns = std::vector; diff --git a/dbms/src/Columns/ReverseIndex.h b/dbms/src/Columns/ReverseIndex.h index a003e8282dc..a2b791fed6e 100644 --- a/dbms/src/Columns/ReverseIndex.h +++ b/dbms/src/Columns/ReverseIndex.h @@ -83,7 +83,7 @@ namespace template - struct ReverseIndexHashTableCell + struct ReverseIndexHashTableCell : public HashTableCell> { using Base = HashTableCell>; diff --git a/dbms/src/Common/DNSResolver.cpp b/dbms/src/Common/DNSResolver.cpp index c82f302d7b2..bba3305684c 100644 --- a/dbms/src/Common/DNSResolver.cpp +++ b/dbms/src/Common/DNSResolver.cpp @@ -107,7 +107,7 @@ Poco::Net::SocketAddress DNSResolver::resolveAddress(const std::string & host_an Poco::Net::SocketAddress DNSResolver::resolveAddress(const std::string & host, UInt16 port) { - return !impl->disable_cache ? Poco::Net::SocketAddress(impl->cache_host(host), port) : Poco::Net::SocketAddress(host, port); + return !impl->disable_cache ? Poco::Net::SocketAddress(impl->cache_host(host), port) : Poco::Net::SocketAddress(host, port); } void DNSResolver::dropCache() diff --git a/dbms/src/Common/FieldVisitors.h b/dbms/src/Common/FieldVisitors.h index ecfd1add179..19b4e583338 100644 --- a/dbms/src/Common/FieldVisitors.h +++ b/dbms/src/Common/FieldVisitors.h @@ -38,16 +38,16 @@ typename std::decay_t::ResultType applyVisitor(Visitor && visitor, F && { switch (field.getType()) { - case Field::Types::Null: return visitor(field.template get()); - case Field::Types::UInt64: return visitor(field.template get()); + case Field::Types::Null: return visitor(field.template get()); + case Field::Types::UInt64: return visitor(field.template get()); case Field::Types::UInt128: return visitor(field.template get()); - case Field::Types::Int64: return visitor(field.template get()); + case Field::Types::Int64: return visitor(field.template get()); case Field::Types::Float64: return visitor(field.template get()); - case Field::Types::String: return visitor(field.template get()); - case Field::Types::Array: return visitor(field.template get()); - case Field::Types::Tuple: return visitor(field.template get()); - case Field::Types::Decimal32: return visitor(field.template get>()); - case Field::Types::Decimal64: return visitor(field.template get>()); + case Field::Types::String: return visitor(field.template get()); + case Field::Types::Array: return visitor(field.template get()); + case Field::Types::Tuple: return visitor(field.template get()); + case Field::Types::Decimal32: return visitor(field.template get>()); + case Field::Types::Decimal64: return visitor(field.template get>()); case Field::Types::Decimal128: return visitor(field.template get>()); default: diff --git a/dbms/src/Common/HashTable/HashTable.h b/dbms/src/Common/HashTable/HashTable.h index b35d3b41acc..b78ce798ba2 100644 --- a/dbms/src/Common/HashTable/HashTable.h +++ b/dbms/src/Common/HashTable/HashTable.h @@ -229,8 +229,8 @@ public: void setHasZero() { has_zero = true; } void clearHasZero() { has_zero = false; } - Cell * zeroValue() { return reinterpret_cast(&zero_value_storage); } - const Cell * zeroValue() const { return reinterpret_cast(&zero_value_storage); } + Cell * zeroValue() { return reinterpret_cast(&zero_value_storage); } + const Cell * zeroValue() const { return reinterpret_cast(&zero_value_storage); } }; template @@ -240,8 +240,8 @@ struct ZeroValueStorage void setHasZero() { throw DB::Exception("HashTable: logical error", DB::ErrorCodes::LOGICAL_ERROR); } void clearHasZero() {} - Cell * zeroValue() { return nullptr; } - const Cell * zeroValue() const { return nullptr; } + Cell * zeroValue() { return nullptr; } + const Cell * zeroValue() const { return nullptr; } }; diff --git a/dbms/src/Common/HyperLogLogCounter.h b/dbms/src/Common/HyperLogLogCounter.h index d457a9342b9..d85da31e7fd 100644 --- a/dbms/src/Common/HyperLogLogCounter.h +++ b/dbms/src/Common/HyperLogLogCounter.h @@ -310,9 +310,9 @@ public: { /// Normalizing factor for harmonic mean. static constexpr double alpha_m = - bucket_count == 2 ? 0.351 : - bucket_count == 4 ? 0.532 : - bucket_count == 8 ? 0.626 : + bucket_count == 2 ? 0.351 : + bucket_count == 4 ? 0.532 : + bucket_count == 8 ? 0.626 : bucket_count == 16 ? 0.673 : bucket_count == 32 ? 0.697 : bucket_count == 64 ? 0.709 : 0.7213 / (1 + 1.079 / bucket_count); diff --git a/dbms/src/Common/ProfileEvents.h b/dbms/src/Common/ProfileEvents.h index ac4ae504f50..78b4ebbf42e 100644 --- a/dbms/src/Common/ProfileEvents.h +++ b/dbms/src/Common/ProfileEvents.h @@ -37,7 +37,7 @@ namespace ProfileEvents /// Global level static initializer Counters(Counter * allocated_counters) - : counters(allocated_counters), parent(nullptr), level(VariableContext::Global) {} + : counters(allocated_counters), parent(nullptr), level(VariableContext::Global) {} Counter & operator[] (Event event) { diff --git a/dbms/src/Common/Stopwatch.h b/dbms/src/Common/Stopwatch.h index c1ec623e100..d6508b23b44 100644 --- a/dbms/src/Common/Stopwatch.h +++ b/dbms/src/Common/Stopwatch.h @@ -28,15 +28,15 @@ public: */ Stopwatch(clockid_t clock_type_ = CLOCK_MONOTONIC) : clock_type(clock_type_) { start(); } - void start() { start_ns = nanoseconds(); is_running = true; } - void stop() { stop_ns = nanoseconds(); is_running = false; } - void reset() { start_ns = 0; stop_ns = 0; is_running = false; } - void restart() { start(); } - UInt64 elapsed() const { return elapsedNanoseconds(); } - UInt64 elapsedNanoseconds() const { return is_running ? nanoseconds() - start_ns : stop_ns - start_ns; } - UInt64 elapsedMicroseconds() const { return elapsedNanoseconds() / 1000U; } - UInt64 elapsedMilliseconds() const { return elapsedNanoseconds() / 1000000UL; } - double elapsedSeconds() const { return static_cast(elapsedNanoseconds()) / 1000000000ULL; } + void start() { start_ns = nanoseconds(); is_running = true; } + void stop() { stop_ns = nanoseconds(); is_running = false; } + void reset() { start_ns = 0; stop_ns = 0; is_running = false; } + void restart() { start(); } + UInt64 elapsed() const { return elapsedNanoseconds(); } + UInt64 elapsedNanoseconds() const { return is_running ? nanoseconds() - start_ns : stop_ns - start_ns; } + UInt64 elapsedMicroseconds() const { return elapsedNanoseconds() / 1000U; } + UInt64 elapsedMilliseconds() const { return elapsedNanoseconds() / 1000000UL; } + double elapsedSeconds() const { return static_cast(elapsedNanoseconds()) / 1000000000ULL; } private: UInt64 start_ns = 0; @@ -53,10 +53,10 @@ class AtomicStopwatch public: AtomicStopwatch(clockid_t clock_type_ = CLOCK_MONOTONIC) : clock_type(clock_type_) { restart(); } - void restart() { start_ns = nanoseconds(); } - UInt64 elapsed() const { return nanoseconds() - start_ns; } - UInt64 elapsedMilliseconds() const { return elapsed() / 1000000UL; } - double elapsedSeconds() const { return static_cast(elapsed()) / 1000000000ULL; } + void restart() { start_ns = nanoseconds(); } + UInt64 elapsed() const { return nanoseconds() - start_ns; } + UInt64 elapsedMilliseconds() const { return elapsed() / 1000000UL; } + double elapsedSeconds() const { return static_cast(elapsed()) / 1000000000ULL; } /** If specified amount of time has passed, then restarts timer and returns true. * Otherwise returns false. @@ -140,10 +140,10 @@ struct StopwatchRUsage { StopwatchRUsage() = default; - void start() { start_ts = Timestamp::current(); is_running = true; } - void stop() { stop_ts = Timestamp::current(); is_running = false; } - void reset() { start_ts = Timestamp(); stop_ts = Timestamp(); is_running = false; } - void restart() { start(); } + void start() { start_ts = Timestamp::current(); is_running = true; } + void stop() { stop_ts = Timestamp::current(); is_running = false; } + void reset() { start_ts = Timestamp(); stop_ts = Timestamp(); is_running = false; } + void restart() { start(); } UInt64 elapsed(bool count_user = true, bool count_sys = true) const { diff --git a/dbms/src/Common/ThreadProfileEvents.h b/dbms/src/Common/ThreadProfileEvents.h index 233d4eec047..3be784f135e 100644 --- a/dbms/src/Common/ThreadProfileEvents.h +++ b/dbms/src/Common/ThreadProfileEvents.h @@ -141,9 +141,9 @@ struct TasksStatsCounters if (curr.stat.version < TASKSTATS_VERSION) return; - profile_events.increment(ProfileEvents::OSReadChars, safeDiff(prev.stat.read_char, curr.stat.read_char)); + profile_events.increment(ProfileEvents::OSReadChars, safeDiff(prev.stat.read_char, curr.stat.read_char)); profile_events.increment(ProfileEvents::OSWriteChars, safeDiff(prev.stat.write_char, curr.stat.write_char)); - profile_events.increment(ProfileEvents::OSReadBytes, safeDiff(prev.stat.read_bytes, curr.stat.read_bytes)); + profile_events.increment(ProfileEvents::OSReadBytes, safeDiff(prev.stat.read_bytes, curr.stat.read_bytes)); profile_events.increment(ProfileEvents::OSWriteBytes, safeDiff(prev.stat.write_bytes, curr.stat.write_bytes)); } diff --git a/dbms/src/Common/ZooKeeper/IKeeper.cpp b/dbms/src/Common/ZooKeeper/IKeeper.cpp index e6713cca3de..114b14d0118 100644 --- a/dbms/src/Common/ZooKeeper/IKeeper.cpp +++ b/dbms/src/Common/ZooKeeper/IKeeper.cpp @@ -91,30 +91,30 @@ const char * errorMessage(int32_t code) { switch (code) { - case ZOK: return "Ok"; - case ZSYSTEMERROR: return "System error"; - case ZRUNTIMEINCONSISTENCY: return "Run time inconsistency"; - case ZDATAINCONSISTENCY: return "Data inconsistency"; - case ZCONNECTIONLOSS: return "Connection loss"; - case ZMARSHALLINGERROR: return "Marshalling error"; - case ZUNIMPLEMENTED: return "Unimplemented"; - case ZOPERATIONTIMEOUT: return "Operation timeout"; - case ZBADARGUMENTS: return "Bad arguments"; - case ZINVALIDSTATE: return "Invalid zhandle state"; - case ZAPIERROR: return "API error"; - case ZNONODE: return "No node"; - case ZNOAUTH: return "Not authenticated"; - case ZBADVERSION: return "Bad version"; - case ZNOCHILDRENFOREPHEMERALS: return "No children for ephemerals"; - case ZNODEEXISTS: return "Node exists"; - case ZNOTEMPTY: return "Not empty"; - case ZSESSIONEXPIRED: return "Session expired"; - case ZINVALIDCALLBACK: return "Invalid callback"; - case ZINVALIDACL: return "Invalid ACL"; - case ZAUTHFAILED: return "Authentication failed"; - case ZCLOSING: return "ZooKeeper is closing"; - case ZNOTHING: return "(not error) no server responses to process"; - case ZSESSIONMOVED: return "Session moved to another server, so operation is ignored"; + case ZOK: return "Ok"; + case ZSYSTEMERROR: return "System error"; + case ZRUNTIMEINCONSISTENCY: return "Run time inconsistency"; + case ZDATAINCONSISTENCY: return "Data inconsistency"; + case ZCONNECTIONLOSS: return "Connection loss"; + case ZMARSHALLINGERROR: return "Marshalling error"; + case ZUNIMPLEMENTED: return "Unimplemented"; + case ZOPERATIONTIMEOUT: return "Operation timeout"; + case ZBADARGUMENTS: return "Bad arguments"; + case ZINVALIDSTATE: return "Invalid zhandle state"; + case ZAPIERROR: return "API error"; + case ZNONODE: return "No node"; + case ZNOAUTH: return "Not authenticated"; + case ZBADVERSION: return "Bad version"; + case ZNOCHILDRENFOREPHEMERALS: return "No children for ephemerals"; + case ZNODEEXISTS: return "Node exists"; + case ZNOTEMPTY: return "Not empty"; + case ZSESSIONEXPIRED: return "Session expired"; + case ZINVALIDCALLBACK: return "Invalid callback"; + case ZINVALIDACL: return "Invalid ACL"; + case ZAUTHFAILED: return "Authentication failed"; + case ZCLOSING: return "ZooKeeper is closing"; + case ZNOTHING: return "(not error) no server responses to process"; + case ZSESSIONMOVED: return "Session moved to another server, so operation is ignored"; } if (code > 0) return strerror(code); diff --git a/dbms/src/Common/formatIPv6.cpp b/dbms/src/Common/formatIPv6.cpp index e2a295cb923..1b09a7e0e1d 100644 --- a/dbms/src/Common/formatIPv6.cpp +++ b/dbms/src/Common/formatIPv6.cpp @@ -7,7 +7,7 @@ namespace DB { -/// integer logarithm, return ceil(log(value, base)) (the smallest integer greater or equal than log(value, base) +/// integer logarithm, return ceil(log(value, base)) (the smallest integer greater or equal than log(value, base) static constexpr UInt32 intLog(const UInt32 value, const UInt32 base, const bool carry) { return value >= base ? 1 + intLog(value / base, base, value % base || carry) : value % base > 1 || carry; diff --git a/dbms/src/Common/isLocalAddress.h b/dbms/src/Common/isLocalAddress.h index ffa03977a3f..81039dff68e 100644 --- a/dbms/src/Common/isLocalAddress.h +++ b/dbms/src/Common/isLocalAddress.h @@ -22,7 +22,7 @@ namespace DB * - only the first address is taken for each network interface; * - the routing rules that affect which network interface we go to the specified address are not checked. */ - bool isLocalAddress(const Poco::Net::SocketAddress & address, UInt16 clickhouse_port); + bool isLocalAddress(const Poco::Net::SocketAddress & address, UInt16 clickhouse_port); bool isLocalAddress(const Poco::Net::SocketAddress & address); diff --git a/dbms/src/Core/QueryProcessingStage.h b/dbms/src/Core/QueryProcessingStage.h index a16b041ae35..6de65d32f93 100644 --- a/dbms/src/Core/QueryProcessingStage.h +++ b/dbms/src/Core/QueryProcessingStage.h @@ -12,9 +12,9 @@ namespace QueryProcessingStage /// Numbers matter - the later stage has a larger number. enum Enum { - FetchColumns = 0, /// Only read/have been read the columns specified in the query. - WithMergeableState = 1, /// Until the stage where the results of processing on different servers can be combined. - Complete = 2, /// Completely. + FetchColumns = 0, /// Only read/have been read the columns specified in the query. + WithMergeableState = 1, /// Until the stage where the results of processing on different servers can be combined. + Complete = 2, /// Completely. }; inline const char * toString(UInt64 stage) diff --git a/dbms/src/DataTypes/DataTypeLowCardinality.cpp b/dbms/src/DataTypes/DataTypeLowCardinality.cpp index 5e2c58c31ae..cf38941b743 100644 --- a/dbms/src/DataTypes/DataTypeLowCardinality.cpp +++ b/dbms/src/DataTypes/DataTypeLowCardinality.cpp @@ -56,7 +56,7 @@ void DataTypeLowCardinality::enumerateStreams(const StreamCallback & callback, S path.pop_back(); } -struct KeysSerializationVersion +struct KeysSerializationVersion { enum Value { diff --git a/dbms/src/DataTypes/DataTypeLowCardinality.h b/dbms/src/DataTypes/DataTypeLowCardinality.h index a51e009534b..86698ded893 100644 --- a/dbms/src/DataTypes/DataTypeLowCardinality.h +++ b/dbms/src/DataTypes/DataTypeLowCardinality.h @@ -96,7 +96,7 @@ public: serializeImpl(column, row_num, ostr, &IDataType::serializeText, settings); } - void serializeTextJSON(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const override + void serializeTextJSON(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const override { serializeImpl(column, row_num, ostr, &IDataType::serializeTextJSON, settings); } diff --git a/dbms/src/DataTypes/FieldToDataType.cpp b/dbms/src/DataTypes/FieldToDataType.cpp index 10f083173e4..3086c9167c2 100644 --- a/dbms/src/DataTypes/FieldToDataType.cpp +++ b/dbms/src/DataTypes/FieldToDataType.cpp @@ -30,7 +30,7 @@ DataTypePtr FieldToDataType::operator() (const Null &) const DataTypePtr FieldToDataType::operator() (const UInt64 & x) const { - if (x <= std::numeric_limits::max()) return std::make_shared(); + if (x <= std::numeric_limits::max()) return std::make_shared(); if (x <= std::numeric_limits::max()) return std::make_shared(); if (x <= std::numeric_limits::max()) return std::make_shared(); return std::make_shared(); @@ -43,7 +43,7 @@ DataTypePtr FieldToDataType::operator() (const UInt128 &) const DataTypePtr FieldToDataType::operator() (const Int64 & x) const { - if (x <= std::numeric_limits::max() && x >= std::numeric_limits::min()) return std::make_shared(); + if (x <= std::numeric_limits::max() && x >= std::numeric_limits::min()) return std::make_shared(); if (x <= std::numeric_limits::max() && x >= std::numeric_limits::min()) return std::make_shared(); if (x <= std::numeric_limits::max() && x >= std::numeric_limits::min()) return std::make_shared(); return std::make_shared(); diff --git a/dbms/src/DataTypes/NumberTraits.h b/dbms/src/DataTypes/NumberTraits.h index 3ebe9420052..c3642f20c3b 100644 --- a/dbms/src/DataTypes/NumberTraits.h +++ b/dbms/src/DataTypes/NumberTraits.h @@ -43,18 +43,18 @@ template <> struct Construct { using Type = UInt8; }; template <> struct Construct { using Type = UInt16; }; template <> struct Construct { using Type = UInt32; }; template <> struct Construct { using Type = UInt64; }; -template <> struct Construct { using Type = Float32; }; -template <> struct Construct { using Type = Float32; }; -template <> struct Construct { using Type = Float32; }; -template <> struct Construct { using Type = Float64; }; -template <> struct Construct { using Type = Int8; }; -template <> struct Construct { using Type = Int16; }; -template <> struct Construct { using Type = Int32; }; -template <> struct Construct { using Type = Int64; }; -template <> struct Construct { using Type = Float32; }; -template <> struct Construct { using Type = Float32; }; -template <> struct Construct { using Type = Float32; }; -template <> struct Construct { using Type = Float64; }; +template <> struct Construct { using Type = Float32; }; +template <> struct Construct { using Type = Float32; }; +template <> struct Construct { using Type = Float32; }; +template <> struct Construct { using Type = Float64; }; +template <> struct Construct { using Type = Int8; }; +template <> struct Construct { using Type = Int16; }; +template <> struct Construct { using Type = Int32; }; +template <> struct Construct { using Type = Int64; }; +template <> struct Construct { using Type = Float32; }; +template <> struct Construct { using Type = Float32; }; +template <> struct Construct { using Type = Float32; }; +template <> struct Construct { using Type = Float64; }; /** The result of addition or multiplication is calculated according to the following rules: diff --git a/dbms/src/Dictionaries/Embedded/RegionsHierarchy.cpp b/dbms/src/Dictionaries/Embedded/RegionsHierarchy.cpp index 71b5adc6afd..2a277e10fe2 100644 --- a/dbms/src/Dictionaries/Embedded/RegionsHierarchy.cpp +++ b/dbms/src/Dictionaries/Embedded/RegionsHierarchy.cpp @@ -45,7 +45,7 @@ void RegionsHierarchy::reload() RegionParents new_continent(initial_size); RegionParents new_top_continent(initial_size); RegionPopulations new_populations(initial_size); - RegionDepths new_depths(initial_size); + RegionDepths new_depths(initial_size); RegionTypes types(initial_size); RegionID max_region_id = 0; diff --git a/dbms/src/Dictionaries/MongoDBDictionarySource.cpp b/dbms/src/Dictionaries/MongoDBDictionarySource.cpp index ee3c493d3cd..e8fe62bd8b6 100644 --- a/dbms/src/Dictionaries/MongoDBDictionarySource.cpp +++ b/dbms/src/Dictionaries/MongoDBDictionarySource.cpp @@ -86,7 +86,7 @@ static void authenticate(Poco::MongoDB::Connection & connection, Poco::MD5Engine md5; md5.update(first); std::string digest_first(Poco::DigestEngine::digestToHex(md5.digest())); - std::string second = nonce + user + digest_first; + std::string second = nonce + user + digest_first; md5.reset(); md5.update(second); std::string digest_second(Poco::DigestEngine::digestToHex(md5.digest())); diff --git a/dbms/src/Dictionaries/XDBCDictionarySource.cpp b/dbms/src/Dictionaries/XDBCDictionarySource.cpp index 4e9637502e7..e7285f17025 100644 --- a/dbms/src/Dictionaries/XDBCDictionarySource.cpp +++ b/dbms/src/Dictionaries/XDBCDictionarySource.cpp @@ -168,7 +168,7 @@ DictionarySourcePtr XDBCDictionarySource::clone() const std::string XDBCDictionarySource::toString() const { - return bridge_helper->getName() + ": " + db + '.' + table + (where.empty() ? "" : ", where: " + where); + return bridge_helper->getName() + ": " + db + '.' + table + (where.empty() ? "" : ", where: " + where); } bool XDBCDictionarySource::isModified() const diff --git a/dbms/src/Functions/FunctionsCoding.h b/dbms/src/Functions/FunctionsCoding.h index f2378e68faf..5270cea0ac6 100644 --- a/dbms/src/Functions/FunctionsCoding.h +++ b/dbms/src/Functions/FunctionsCoding.h @@ -310,7 +310,7 @@ public: } /// slightly altered implementation from http://svn.apache.org/repos/asf/apr/apr/trunk/network_io/unix/inet_pton.c - static void ipv6_scan(const char * src, unsigned char * dst) + static void ipv6_scan(const char * src, unsigned char * dst) { const auto clear_dst = [dst] { diff --git a/dbms/src/Functions/FunctionsConditional.cpp b/dbms/src/Functions/FunctionsConditional.cpp index 92ae0a5b0fb..2b39a772f14 100644 --- a/dbms/src/Functions/FunctionsConditional.cpp +++ b/dbms/src/Functions/FunctionsConditional.cpp @@ -279,8 +279,8 @@ void FunctionCaseWithExpression::executeImpl(Block & block, const ColumnNumbers /// CASE expr WHEN val[0] THEN branch[0] ... WHEN val[N-1] then branch[N-1] ELSE branchN /// into the construction transform(expr, src, dest, branchN) /// where: - /// src = [val[0], val[1], ..., val[N-1]] - /// dest = [branch[0], ..., branch[N-1]] + /// src = [val[0], val[1], ..., val[N-1]] + /// dst = [branch[0], ..., branch[N-1]] /// then we perform it. /// Create the arrays required by the transform function. diff --git a/dbms/src/Functions/FunctionsExternalModels.h b/dbms/src/Functions/FunctionsExternalModels.h index 6d0b3c7a43c..e32fe7f066f 100644 --- a/dbms/src/Functions/FunctionsExternalModels.h +++ b/dbms/src/Functions/FunctionsExternalModels.h @@ -1,7 +1,7 @@ #pragma once #include -namespace DB +namespace DB { class ExternalModels; diff --git a/dbms/src/Functions/FunctionsMiscellaneous.h b/dbms/src/Functions/FunctionsMiscellaneous.h index 4451696478c..ee3a92bb6b6 100644 --- a/dbms/src/Functions/FunctionsMiscellaneous.h +++ b/dbms/src/Functions/FunctionsMiscellaneous.h @@ -104,7 +104,7 @@ public: return_type = std::make_shared(argument_types, function_return_type); - name = "Capture[" + toString(captured_types) + "](" + toString(argument_types) + ") -> " + name = "Capture[" + toString(captured_types) + "](" + toString(argument_types) + ") -> " + function_return_type->getName(); } diff --git a/dbms/src/Functions/FunctionsProjection.cpp b/dbms/src/Functions/FunctionsProjection.cpp index 3b67fbc0fc2..c526d20b558 100644 --- a/dbms/src/Functions/FunctionsProjection.cpp +++ b/dbms/src/Functions/FunctionsProjection.cpp @@ -147,7 +147,7 @@ void FunctionBuildProjectionComposition::executeImpl( } if (current_reserve_index != second_projection_column->size()) { - throw Exception("Second argument size is not appropriate: " + std::to_string(second_projection_column->size()) + " instead of " + throw Exception("Second argument size is not appropriate: " + std::to_string(second_projection_column->size()) + " instead of " + std::to_string(current_reserve_index), ErrorCodes::BAD_ARGUMENTS); } diff --git a/dbms/src/Functions/FunctionsVisitParam.h b/dbms/src/Functions/FunctionsVisitParam.h index 940ef877476..a746c7b3979 100644 --- a/dbms/src/Functions/FunctionsVisitParam.h +++ b/dbms/src/Functions/FunctionsVisitParam.h @@ -233,7 +233,7 @@ struct ExtractParamToStringImpl ColumnString::Chars & res_data, ColumnString::Offsets & res_offsets) { /// Constant 5 is taken from a function that performs a similar task FunctionsStringSearch.h::ExtractImpl - res_data.reserve(data.size() / 5); + res_data.reserve(data.size() / 5); res_offsets.resize(offsets.size()); /// We are looking for a parameter simply as a substring of the form "name" diff --git a/dbms/src/Functions/GatherUtils/Algorithms.h b/dbms/src/Functions/GatherUtils/Algorithms.h index 7ae4ef703a2..34cef4939e5 100644 --- a/dbms/src/Functions/GatherUtils/Algorithms.h +++ b/dbms/src/Functions/GatherUtils/Algorithms.h @@ -12,7 +12,7 @@ namespace DB::ErrorCodes extern const int LOGICAL_ERROR; } -namespace DB::GatherUtils +namespace DB::GatherUtils { /// Methods to copy Slice to Sink, overloaded for various combinations of types. diff --git a/dbms/src/Functions/arrayFirstIndex.cpp b/dbms/src/Functions/arrayFirstIndex.cpp index 5a0e291f5a1..1c4f130d6c1 100644 --- a/dbms/src/Functions/arrayFirstIndex.cpp +++ b/dbms/src/Functions/arrayFirstIndex.cpp @@ -74,7 +74,7 @@ struct ArrayFirstIndexImpl } }; -struct NameArrayFirstIndex { static constexpr auto name = "arrayFirstIndex"; }; +struct NameArrayFirstIndex { static constexpr auto name = "arrayFirstIndex"; }; using FunctionArrayFirstIndex = FunctionArrayMapped; void registerFunctionArrayFirstIndex(FunctionFactory & factory) diff --git a/dbms/src/Functions/arrayUniq.cpp b/dbms/src/Functions/arrayUniq.cpp index 9dbf0e39886..3a90d76aae6 100644 --- a/dbms/src/Functions/arrayUniq.cpp +++ b/dbms/src/Functions/arrayUniq.cpp @@ -63,9 +63,9 @@ private: static constexpr size_t INITIAL_SIZE_DEGREE = 9; template - bool executeNumber(const ColumnArray * array, const IColumn * null_map, ColumnUInt32::Container & res_values); + bool executeNumber(const ColumnArray * array, const IColumn * null_map, ColumnUInt32::Container & res_values); - bool executeString(const ColumnArray * array, const IColumn * null_map, ColumnUInt32::Container & res_values); + bool executeString(const ColumnArray * array, const IColumn * null_map, ColumnUInt32::Container & res_values); bool execute128bit( const ColumnArray::Offsets & offsets, diff --git a/dbms/src/Functions/formatDateTime.cpp b/dbms/src/Functions/formatDateTime.cpp index e242f70bd58..b9f75d1e6bd 100644 --- a/dbms/src/Functions/formatDateTime.cpp +++ b/dbms/src/Functions/formatDateTime.cpp @@ -285,7 +285,7 @@ public: if (!executeType(block, arguments, result) && !executeType(block, arguments, result)) throw Exception("Illegal column " + block.getByPosition(arguments[0]).column->getName() - + " of function " + getName() + ", must be Date or DateTime", + + " of function " + getName() + ", must be Date or DateTime", ErrorCodes::ILLEGAL_COLUMN); } diff --git a/dbms/src/IO/DoubleConverter.h b/dbms/src/IO/DoubleConverter.h index f924cfea91f..e05226333dd 100644 --- a/dbms/src/IO/DoubleConverter.h +++ b/dbms/src/IO/DoubleConverter.h @@ -29,7 +29,7 @@ template class DoubleConverter { DoubleConverter(const DoubleConverter &) = delete; - DoubleConverter & operator=(const DoubleConverter &) = delete; + DoubleConverter & operator=(const DoubleConverter &) = delete; DoubleConverter() = default; diff --git a/dbms/src/IO/Operators.h b/dbms/src/IO/Operators.h index e5f0de9df42..82c74fc0b16 100644 --- a/dbms/src/IO/Operators.h +++ b/dbms/src/IO/Operators.h @@ -75,10 +75,10 @@ template <> inline ReadBuffer & operator>> (ReadBuffer & buf, char & x) /// If you specify a string literal for reading, this will mean - make sure there is a sequence of bytes and skip it. inline ReadBuffer & operator>> (ReadBuffer & buf, const char * x) { assertString(x, buf); return buf; } -inline EscapeManipReadBuffer & operator>> (ReadBuffer & buf, EscapeManip) { return static_cast(buf); } -inline QuoteManipReadBuffer & operator>> (ReadBuffer & buf, QuoteManip) { return static_cast(buf); } -inline DoubleQuoteManipReadBuffer & operator>> (ReadBuffer & buf, DoubleQuoteManip) { return static_cast(buf); } -inline BinaryManipReadBuffer & operator>> (ReadBuffer & buf, BinaryManip) { return static_cast(buf); } +inline EscapeManipReadBuffer & operator>> (ReadBuffer & buf, EscapeManip) { return static_cast(buf); } +inline QuoteManipReadBuffer & operator>> (ReadBuffer & buf, QuoteManip) { return static_cast(buf); } +inline DoubleQuoteManipReadBuffer & operator>> (ReadBuffer & buf, DoubleQuoteManip) { return static_cast(buf); } +inline BinaryManipReadBuffer & operator>> (ReadBuffer & buf, BinaryManip) { return static_cast(buf); } template ReadBuffer & operator>> (EscapeManipReadBuffer & buf, T & x) { readText(x, buf); return buf; } template ReadBuffer & operator>> (QuoteManipReadBuffer & buf, T & x) { readQuoted(x, buf); return buf; } diff --git a/dbms/src/IO/ReadHelpers.cpp b/dbms/src/IO/ReadHelpers.cpp index 45c4185eff0..c1d8754c1b1 100644 --- a/dbms/src/IO/ReadHelpers.cpp +++ b/dbms/src/IO/ReadHelpers.cpp @@ -74,7 +74,7 @@ UInt128 stringToUUID(const String & str) void NO_INLINE throwAtAssertionFailed(const char * s, ReadBuffer & buf) { WriteBufferFromOwnString out; - out << "Cannot parse input: expected " << escape << s; + out << "Cannot parse input: expected " << escape << s; if (buf.eof()) out << " at end of stream."; diff --git a/dbms/src/IO/WriteBufferAIO.cpp b/dbms/src/IO/WriteBufferAIO.cpp index 9d46567ffd3..fdd6a61fb7b 100644 --- a/dbms/src/IO/WriteBufferAIO.cpp +++ b/dbms/src/IO/WriteBufferAIO.cpp @@ -121,7 +121,7 @@ void WriteBufferAIO::nextImpl() { if (errno != EINTR) { - aio_failed = true; + aio_failed = true; throw Exception("Cannot submit request for asynchronous IO on file " + filename, ErrorCodes::CANNOT_IO_SUBMIT); } } diff --git a/dbms/src/IO/ZlibDeflatingWriteBuffer.cpp b/dbms/src/IO/ZlibDeflatingWriteBuffer.cpp index 41864ccc590..086c9e53a8d 100644 --- a/dbms/src/IO/ZlibDeflatingWriteBuffer.cpp +++ b/dbms/src/IO/ZlibDeflatingWriteBuffer.cpp @@ -14,12 +14,12 @@ ZlibDeflatingWriteBuffer::ZlibDeflatingWriteBuffer( : BufferWithOwnMemory(buf_size, existing_memory, alignment) , out(out_) { - zstr.zalloc = nullptr; - zstr.zfree = nullptr; - zstr.opaque = nullptr; - zstr.next_in = nullptr; - zstr.avail_in = 0; - zstr.next_out = nullptr; + zstr.zalloc = nullptr; + zstr.zfree = nullptr; + zstr.opaque = nullptr; + zstr.next_in = nullptr; + zstr.avail_in = 0; + zstr.next_out = nullptr; zstr.avail_out = 0; int window_bits = 15; diff --git a/dbms/src/IO/ZlibInflatingReadBuffer.cpp b/dbms/src/IO/ZlibInflatingReadBuffer.cpp index aed017b3288..5b0796134ac 100644 --- a/dbms/src/IO/ZlibInflatingReadBuffer.cpp +++ b/dbms/src/IO/ZlibInflatingReadBuffer.cpp @@ -14,12 +14,12 @@ ZlibInflatingReadBuffer::ZlibInflatingReadBuffer( , in(in_) , eof(false) { - zstr.zalloc = nullptr; - zstr.zfree = nullptr; - zstr.opaque = nullptr; - zstr.next_in = nullptr; - zstr.avail_in = 0; - zstr.next_out = nullptr; + zstr.zalloc = nullptr; + zstr.zfree = nullptr; + zstr.opaque = nullptr; + zstr.next_in = nullptr; + zstr.avail_in = 0; + zstr.next_out = nullptr; zstr.avail_out = 0; int window_bits = 15; diff --git a/dbms/src/IO/tests/hashing_read_buffer.cpp b/dbms/src/IO/tests/hashing_read_buffer.cpp index 8148ba98fe4..a9332707c4d 100644 --- a/dbms/src/IO/tests/hashing_read_buffer.cpp +++ b/dbms/src/IO/tests/hashing_read_buffer.cpp @@ -17,7 +17,7 @@ void test(size_t data_size) std::vector block_sizes = {56, 128, 513, 2048, 3055, 4097, 4096}; for (size_t read_buffer_block_size : block_sizes) { - std::cout << "block size " << read_buffer_block_size << std::endl; + std::cout << "block size " << read_buffer_block_size << std::endl; std::stringstream io; DB::WriteBufferFromOStream out_(io); DB::HashingWriteBuffer out(out_); diff --git a/dbms/src/IO/tests/parse_int_perf.cpp b/dbms/src/IO/tests/parse_int_perf.cpp index 353d29c204f..ae6762d39c9 100644 --- a/dbms/src/IO/tests/parse_int_perf.cpp +++ b/dbms/src/IO/tests/parse_int_perf.cpp @@ -84,9 +84,9 @@ int main(int argc, char ** argv) << n / watch.elapsedSeconds() << " num/s., " << wb.count() / watch.elapsedSeconds() / 1000000 << " MB/s., " << watch.elapsed() / n << " ns/num., " - << tsc / n << " ticks/num., " + << tsc / n << " ticks/num., " << watch.elapsed() / wb.count() << " ns/byte., " - << tsc / wb.count() << " ticks/byte." + << tsc / wb.count() << " ticks/byte." << std::endl; } diff --git a/dbms/src/IO/tests/read_buffer_aio.cpp b/dbms/src/IO/tests/read_buffer_aio.cpp index 782b3ccf301..7ed07b0930c 100644 --- a/dbms/src/IO/tests/read_buffer_aio.cpp +++ b/dbms/src/IO/tests/read_buffer_aio.cpp @@ -338,7 +338,7 @@ bool test9(const std::string & filename, const std::string & buf) newbuf.resize(buf.length()); DB::ReadBufferAIO in(filename, 3 * DEFAULT_AIO_FILE_BLOCK_SIZE); - size_t count = in.read(newbuf.data(), newbuf.length()); + size_t count = in.read(newbuf.data(), newbuf.length()); if (count != newbuf.length()) return false; in.setMaxBytes(9 * DEFAULT_AIO_FILE_BLOCK_SIZE); @@ -411,7 +411,7 @@ bool test12(const std::string & filename, const std::string &) newbuf.resize(4 * DEFAULT_AIO_FILE_BLOCK_SIZE); DB::ReadBufferAIO in(filename, 3 * DEFAULT_AIO_FILE_BLOCK_SIZE); - size_t count = in.read(newbuf.data(), newbuf.length()); + size_t count = in.read(newbuf.data(), newbuf.length()); if (count != newbuf.length()) return false; diff --git a/dbms/src/Interpreters/ClusterProxy/SelectStreamFactory.cpp b/dbms/src/Interpreters/ClusterProxy/SelectStreamFactory.cpp index 8a304f7701e..b34b3d34633 100644 --- a/dbms/src/Interpreters/ClusterProxy/SelectStreamFactory.cpp +++ b/dbms/src/Interpreters/ClusterProxy/SelectStreamFactory.cpp @@ -188,7 +188,7 @@ void SelectStreamFactory::createForShard( try { if (table_func_ptr) - try_results = pool->getManyForTableFunction(&context.getSettingsRef(), PoolMode::GET_MANY); + try_results = pool->getManyForTableFunction(&context.getSettingsRef(), PoolMode::GET_MANY); else try_results = pool->getManyChecked(&context.getSettingsRef(), PoolMode::GET_MANY, main_table); } diff --git a/dbms/src/Interpreters/Compiler.cpp b/dbms/src/Interpreters/Compiler.cpp index 1bf03f00ffd..e3150082834 100644 --- a/dbms/src/Interpreters/Compiler.cpp +++ b/dbms/src/Interpreters/Compiler.cpp @@ -222,9 +222,9 @@ void Compiler::compile( std::stringstream command; - auto compiler_executable_root = Poco::Util::Application::instance().config().getString("compiler_executable_root", INTERNAL_COMPILER_BIN_ROOT); - auto compiler_headers = Poco::Util::Application::instance().config().getString("compiler_headers", INTERNAL_COMPILER_HEADERS); - auto compiler_headers_root = Poco::Util::Application::instance().config().getString("compiler_headers_root", INTERNAL_COMPILER_HEADERS_ROOT); + auto compiler_executable_root = Poco::Util::Application::instance().config().getString("compiler_executable_root", INTERNAL_COMPILER_BIN_ROOT); + auto compiler_headers = Poco::Util::Application::instance().config().getString("compiler_headers", INTERNAL_COMPILER_HEADERS); + auto compiler_headers_root = Poco::Util::Application::instance().config().getString("compiler_headers_root", INTERNAL_COMPILER_HEADERS_ROOT); LOG_DEBUG(log, "Using internal compiler: compiler_executable_root=" << compiler_executable_root << "; compiler_headers_root=" << compiler_headers_root << "; compiler_headers=" << compiler_headers); /// Slightly unconvenient. diff --git a/dbms/src/Interpreters/ExpressionAnalyzer.cpp b/dbms/src/Interpreters/ExpressionAnalyzer.cpp index 79117744f2f..8e00a77e109 100644 --- a/dbms/src/Interpreters/ExpressionAnalyzer.cpp +++ b/dbms/src/Interpreters/ExpressionAnalyzer.cpp @@ -925,7 +925,7 @@ void ExpressionAnalyzer::appendProjectResult(ExpressionActionsChain & chain) con { String result_name = asts[i]->getAliasOrColumnName(); if (required_result_columns.empty() - || std::find(required_result_columns.begin(), required_result_columns.end(), result_name) != required_result_columns.end()) + || std::find(required_result_columns.begin(), required_result_columns.end(), result_name) != required_result_columns.end()) { result_columns.emplace_back(asts[i]->getColumnName(), result_name); step.required_output.push_back(result_columns.back().second); diff --git a/dbms/src/Interpreters/LogicalExpressionsOptimizer.cpp b/dbms/src/Interpreters/LogicalExpressionsOptimizer.cpp index b2c2fcace64..5226c96dce6 100644 --- a/dbms/src/Interpreters/LogicalExpressionsOptimizer.cpp +++ b/dbms/src/Interpreters/LogicalExpressionsOptimizer.cpp @@ -197,7 +197,7 @@ inline ASTs & getFunctionOperands(ASTFunction * or_function) bool LogicalExpressionsOptimizer::mayOptimizeDisjunctiveEqualityChain(const DisjunctiveEqualityChain & chain) const { - const auto & equalities = chain.second; + const auto & equalities = chain.second; const auto & equality_functions = equalities.functions; /// We eliminate too short chains. diff --git a/dbms/src/Interpreters/PartLog.cpp b/dbms/src/Interpreters/PartLog.cpp index 31bb5b01b12..d5ae95b5ea3 100644 --- a/dbms/src/Interpreters/PartLog.cpp +++ b/dbms/src/Interpreters/PartLog.cpp @@ -31,28 +31,28 @@ Block PartLogElement::createBlock() return { - {ColumnInt8::create(), std::move(event_type_datatype), "event_type"}, - {ColumnUInt16::create(), std::make_shared(), "event_date"}, - {ColumnUInt32::create(), std::make_shared(), "event_time"}, - {ColumnUInt64::create(), std::make_shared(), "duration_ms"}, + {ColumnInt8::create(), std::move(event_type_datatype), "event_type"}, + {ColumnUInt16::create(), std::make_shared(), "event_date"}, + {ColumnUInt32::create(), std::make_shared(), "event_time"}, + {ColumnUInt64::create(), std::make_shared(), "duration_ms"}, - {ColumnString::create(), std::make_shared(), "database"}, - {ColumnString::create(), std::make_shared(), "table"}, - {ColumnString::create(), std::make_shared(), "part_name"}, - {ColumnString::create(), std::make_shared(), "partition_id"}, + {ColumnString::create(), std::make_shared(), "database"}, + {ColumnString::create(), std::make_shared(), "table"}, + {ColumnString::create(), std::make_shared(), "part_name"}, + {ColumnString::create(), std::make_shared(), "partition_id"}, - {ColumnUInt64::create(), std::make_shared(), "rows"}, - {ColumnUInt64::create(), std::make_shared(), "size_in_bytes"}, // On disk + {ColumnUInt64::create(), std::make_shared(), "rows"}, + {ColumnUInt64::create(), std::make_shared(), "size_in_bytes"}, // On disk /// Merge-specific info {ColumnArray::create(ColumnString::create()), std::make_shared(std::make_shared()), "merged_from"}, - {ColumnUInt64::create(), std::make_shared(), "bytes_uncompressed"}, // Result bytes - {ColumnUInt64::create(), std::make_shared(), "read_rows"}, - {ColumnUInt64::create(), std::make_shared(), "read_bytes"}, + {ColumnUInt64::create(), std::make_shared(), "bytes_uncompressed"}, // Result bytes + {ColumnUInt64::create(), std::make_shared(), "read_rows"}, + {ColumnUInt64::create(), std::make_shared(), "read_bytes"}, /// Is there an error during the execution or commit - {ColumnUInt16::create(), std::make_shared(), "error"}, - {ColumnString::create(), std::make_shared(), "exception"}, + {ColumnUInt16::create(), std::make_shared(), "error"}, + {ColumnString::create(), std::make_shared(), "exception"}, }; } diff --git a/dbms/src/Interpreters/SettingsCommon.cpp b/dbms/src/Interpreters/SettingsCommon.cpp index ccf4cade6a1..08e5d1b1781 100644 --- a/dbms/src/Interpreters/SettingsCommon.cpp +++ b/dbms/src/Interpreters/SettingsCommon.cpp @@ -511,10 +511,10 @@ void SettingCompressionMethod::write(WriteBuffer & buf) const DistributedProductMode SettingDistributedProductMode::getDistributedProductMode(const String & s) { - if (s == "deny") return DistributedProductMode::DENY; - if (s == "local") return DistributedProductMode::LOCAL; + if (s == "deny") return DistributedProductMode::DENY; + if (s == "local") return DistributedProductMode::LOCAL; if (s == "global") return DistributedProductMode::GLOBAL; - if (s == "allow") return DistributedProductMode::ALLOW; + if (s == "allow") return DistributedProductMode::ALLOW; throw Exception("Unknown distributed product mode: '" + s + "', must be one of 'deny', 'local', 'global', 'allow'", ErrorCodes::UNKNOWN_DISTRIBUTED_PRODUCT_MODE); @@ -632,8 +632,8 @@ void SettingChar::write(WriteBuffer & buf) const SettingDateTimeInputFormat::Value SettingDateTimeInputFormat::getValue(const String & s) { - if (s == "basic") return Value::Basic; - if (s == "best_effort") return Value::BestEffort; + if (s == "basic") return Value::Basic; + if (s == "best_effort") return Value::BestEffort; throw Exception("Unknown DateTime input format: '" + s + "', must be one of 'basic', 'best_effort'", ErrorCodes::BAD_ARGUMENTS); } diff --git a/dbms/src/Interpreters/convertFieldToType.cpp b/dbms/src/Interpreters/convertFieldToType.cpp index ac9936221d9..0a46078a602 100644 --- a/dbms/src/Interpreters/convertFieldToType.cpp +++ b/dbms/src/Interpreters/convertFieldToType.cpp @@ -163,7 +163,7 @@ Field convertFieldToTypeImpl(const Field & src, const IDataType & type, const ID if (which_type.isUInt16()) return convertNumericType(src, type); if (which_type.isUInt32()) return convertNumericType(src, type); if (which_type.isUInt64()) return convertNumericType(src, type); - if (which_type.isInt8()) return convertNumericType(src, type); + if (which_type.isInt8()) return convertNumericType(src, type); if (which_type.isInt16()) return convertNumericType(src, type); if (which_type.isInt32()) return convertNumericType(src, type); if (which_type.isInt64()) return convertNumericType(src, type); diff --git a/dbms/src/Interpreters/tests/expression_analyzer.cpp b/dbms/src/Interpreters/tests/expression_analyzer.cpp index fa9cef39ad0..3732e7dce72 100644 --- a/dbms/src/Interpreters/tests/expression_analyzer.cpp +++ b/dbms/src/Interpreters/tests/expression_analyzer.cpp @@ -39,7 +39,7 @@ struct TestEntry { ASTPtr ast = parse(query); - auto res = SyntaxAnalyzer(context, {}).analyze(ast, source_columns, required_result_columns); + auto res = SyntaxAnalyzer(context, {}).analyze(ast, source_columns, required_result_columns); return checkAliases(*res); } diff --git a/dbms/src/Interpreters/tests/hash_map_string_2.cpp b/dbms/src/Interpreters/tests/hash_map_string_2.cpp index d6b9bb9f246..aba8d502270 100644 --- a/dbms/src/Interpreters/tests/hash_map_string_2.cpp +++ b/dbms/src/Interpreters/tests/hash_map_string_2.cpp @@ -247,17 +247,17 @@ inline bool memequal(const char * p1, const char * p2, size_t size) case 12: if (reinterpret_cast(p1)[2] == reinterpret_cast(p2)[2]) goto l8; else return false; case 11: if (p1[10] != p2[10]) return false; [[fallthrough]]; case 10: if (p1[9] != p2[9]) return false; [[fallthrough]]; - case 9: if (p1[8] != p2[8]) return false; + case 9: if (p1[8] != p2[8]) return false; l8: [[fallthrough]]; - case 8: return reinterpret_cast(p1)[0] == reinterpret_cast(p2)[0]; - case 7: if (p1[6] != p2[6]) return false; [[fallthrough]]; - case 6: if (p1[5] != p2[5]) return false; [[fallthrough]]; - case 5: if (p1[4] != p2[4]) return false; [[fallthrough]]; - case 4: return reinterpret_cast(p1)[0] == reinterpret_cast(p2)[0]; - case 3: if (p1[2] != p2[2]) return false; [[fallthrough]]; - case 2: return reinterpret_cast(p1)[0] == reinterpret_cast(p2)[0]; - case 1: if (p1[0] != p2[0]) return false; [[fallthrough]]; - case 0: break; + case 8: return reinterpret_cast(p1)[0] == reinterpret_cast(p2)[0]; + case 7: if (p1[6] != p2[6]) return false; [[fallthrough]]; + case 6: if (p1[5] != p2[5]) return false; [[fallthrough]]; + case 5: if (p1[4] != p2[4]) return false; [[fallthrough]]; + case 4: return reinterpret_cast(p1)[0] == reinterpret_cast(p2)[0]; + case 3: if (p1[2] != p2[2]) return false; [[fallthrough]]; + case 2: return reinterpret_cast(p1)[0] == reinterpret_cast(p2)[0]; + case 1: if (p1[0] != p2[0]) return false; [[fallthrough]]; + case 0: break; } return true; @@ -303,17 +303,17 @@ inline bool memequal_sse41(const char * p1, const char * p2, size_t size) case 12: if (reinterpret_cast(p1)[2] == reinterpret_cast(p2)[2]) goto l8; else return false; case 11: if (p1[10] != p2[10]) return false; [[fallthrough]]; case 10: if (p1[9] != p2[9]) return false; [[fallthrough]]; - case 9: if (p1[8] != p2[8]) return false; + case 9: if (p1[8] != p2[8]) return false; l8: [[fallthrough]]; - case 8: return reinterpret_cast(p1)[0] == reinterpret_cast(p2)[0]; - case 7: if (p1[6] != p2[6]) return false; [[fallthrough]]; - case 6: if (p1[5] != p2[5]) return false; [[fallthrough]]; - case 5: if (p1[4] != p2[4]) return false; [[fallthrough]]; - case 4: return reinterpret_cast(p1)[0] == reinterpret_cast(p2)[0]; - case 3: if (p1[2] != p2[2]) return false; [[fallthrough]]; - case 2: return reinterpret_cast(p1)[0] == reinterpret_cast(p2)[0]; - case 1: if (p1[0] != p2[0]) return false; [[fallthrough]]; - case 0: break; + case 8: return reinterpret_cast(p1)[0] == reinterpret_cast(p2)[0]; + case 7: if (p1[6] != p2[6]) return false; [[fallthrough]]; + case 6: if (p1[5] != p2[5]) return false; [[fallthrough]]; + case 5: if (p1[4] != p2[4]) return false; [[fallthrough]]; + case 4: return reinterpret_cast(p1)[0] == reinterpret_cast(p2)[0]; + case 3: if (p1[2] != p2[2]) return false; [[fallthrough]]; + case 2: return reinterpret_cast(p1)[0] == reinterpret_cast(p2)[0]; + case 1: if (p1[0] != p2[0]) return false; [[fallthrough]]; + case 0: break; } return true; @@ -433,17 +433,17 @@ inline bool memequal_sse41_wide(const char * p1, const char * p2, size_t size) case 12: if (reinterpret_cast(p1)[2] == reinterpret_cast(p2)[2]) goto l8; else return false; case 11: if (p1[10] != p2[10]) return false; [[fallthrough]]; case 10: if (p1[9] != p2[9]) return false; [[fallthrough]]; - case 9: if (p1[8] != p2[8]) return false; + case 9: if (p1[8] != p2[8]) return false; l8: [[fallthrough]]; - case 8: return reinterpret_cast(p1)[0] == reinterpret_cast(p2)[0]; - case 7: if (p1[6] != p2[6]) return false; [[fallthrough]]; - case 6: if (p1[5] != p2[5]) return false; [[fallthrough]]; - case 5: if (p1[4] != p2[4]) return false; [[fallthrough]]; - case 4: return reinterpret_cast(p1)[0] == reinterpret_cast(p2)[0]; - case 3: if (p1[2] != p2[2]) return false; [[fallthrough]]; - case 2: return reinterpret_cast(p1)[0] == reinterpret_cast(p2)[0]; - case 1: if (p1[0] != p2[0]) return false; [[fallthrough]]; - case 0: break; + case 8: return reinterpret_cast(p1)[0] == reinterpret_cast(p2)[0]; + case 7: if (p1[6] != p2[6]) return false; [[fallthrough]]; + case 6: if (p1[5] != p2[5]) return false; [[fallthrough]]; + case 5: if (p1[4] != p2[4]) return false; [[fallthrough]]; + case 4: return reinterpret_cast(p1)[0] == reinterpret_cast(p2)[0]; + case 3: if (p1[2] != p2[2]) return false; [[fallthrough]]; + case 2: return reinterpret_cast(p1)[0] == reinterpret_cast(p2)[0]; + case 1: if (p1[0] != p2[0]) return false; [[fallthrough]]; + case 0: break; } return true; @@ -485,17 +485,17 @@ inline bool memequal_sse_wide(const char * p1, const char * p2, size_t size) case 12: if (reinterpret_cast(p1)[2] == reinterpret_cast(p2)[2]) goto l8; else return false; case 11: if (p1[10] != p2[10]) return false; [[fallthrough]]; case 10: if (p1[9] != p2[9]) return false; [[fallthrough]]; - case 9: if (p1[8] != p2[8]) return false; + case 9: if (p1[8] != p2[8]) return false; l8: [[fallthrough]]; - case 8: return reinterpret_cast(p1)[0] == reinterpret_cast(p2)[0]; - case 7: if (p1[6] != p2[6]) return false; [[fallthrough]]; - case 6: if (p1[5] != p2[5]) return false; [[fallthrough]]; - case 5: if (p1[4] != p2[4]) return false; [[fallthrough]]; - case 4: return reinterpret_cast(p1)[0] == reinterpret_cast(p2)[0]; - case 3: if (p1[2] != p2[2]) return false; [[fallthrough]]; - case 2: return reinterpret_cast(p1)[0] == reinterpret_cast(p2)[0]; - case 1: if (p1[0] != p2[0]) return false; [[fallthrough]]; - case 0: break; + case 8: return reinterpret_cast(p1)[0] == reinterpret_cast(p2)[0]; + case 7: if (p1[6] != p2[6]) return false; [[fallthrough]]; + case 6: if (p1[5] != p2[5]) return false; [[fallthrough]]; + case 5: if (p1[4] != p2[4]) return false; [[fallthrough]]; + case 4: return reinterpret_cast(p1)[0] == reinterpret_cast(p2)[0]; + case 3: if (p1[2] != p2[2]) return false; [[fallthrough]]; + case 2: return reinterpret_cast(p1)[0] == reinterpret_cast(p2)[0]; + case 1: if (p1[0] != p2[0]) return false; [[fallthrough]]; + case 0: break; } return true; diff --git a/dbms/src/Interpreters/tests/hash_map_string_3.cpp b/dbms/src/Interpreters/tests/hash_map_string_3.cpp index 5f72004db07..24c923db8ad 100644 --- a/dbms/src/Interpreters/tests/hash_map_string_3.cpp +++ b/dbms/src/Interpreters/tests/hash_map_string_3.cpp @@ -68,7 +68,7 @@ struct DefaultHash \ { \ size_t operator() (STRUCT x) const \ { \ - return CityHash_v1_0_2::CityHash64(x.data, x.size); \ + return CityHash_v1_0_2::CityHash64(x.data, x.size); \ } \ }; diff --git a/dbms/src/Parsers/ASTInsertQuery.h b/dbms/src/Parsers/ASTInsertQuery.h index 9e72174470c..9da68ca21c8 100644 --- a/dbms/src/Parsers/ASTInsertQuery.h +++ b/dbms/src/Parsers/ASTInsertQuery.h @@ -35,7 +35,7 @@ public: res->children.clear(); if (columns) { res->columns = columns->clone(); res->children.push_back(res->columns); } - if (select) { res->select = select->clone(); res->children.push_back(res->select); } + if (select) { res->select = select->clone(); res->children.push_back(res->select); } if (table_function) { res->table_function = table_function->clone(); res->children.push_back(res->table_function); diff --git a/dbms/src/Parsers/ParserDropQuery.h b/dbms/src/Parsers/ParserDropQuery.h index e686e3dd1a2..fa0b1616463 100644 --- a/dbms/src/Parsers/ParserDropQuery.h +++ b/dbms/src/Parsers/ParserDropQuery.h @@ -20,7 +20,7 @@ protected: bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected); bool parseDropQuery(Pos & pos, ASTPtr & node, Expected & expected); - bool parseDetachQuery(Pos & pos, ASTPtr & node, Expected & expected); + bool parseDetachQuery(Pos & pos, ASTPtr & node, Expected & expected); bool parseTruncateQuery(Pos & pos, ASTPtr & node, Expected & expected); }; diff --git a/dbms/src/Storages/MergeTree/KeyCondition.cpp b/dbms/src/Storages/MergeTree/KeyCondition.cpp index 37610f59f1f..9484bd8c3cc 100644 --- a/dbms/src/Storages/MergeTree/KeyCondition.cpp +++ b/dbms/src/Storages/MergeTree/KeyCondition.cpp @@ -663,7 +663,7 @@ bool KeyCondition::atomFromAST(const ASTPtr & node, const Context & context, Blo key_arg_pos = 1; } else if (getConstant(args[0], block_with_constants, const_value, const_type) - && canConstantBeWrappedByMonotonicFunctions(args[1], key_column_num, key_expr_type, const_value, const_type)) + && canConstantBeWrappedByMonotonicFunctions(args[1], key_column_num, key_expr_type, const_value, const_type)) { key_arg_pos = 1; is_constant_transformed = true; diff --git a/dbms/src/Storages/MergeTree/MergeTreeData.cpp b/dbms/src/Storages/MergeTree/MergeTreeData.cpp index 4905c135f44..9a87728047e 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeData.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeData.cpp @@ -419,7 +419,7 @@ String MergeTreeData::MergingParams::getModeName() const case Aggregating: return "Aggregating"; case Replacing: return "Replacing"; case Graphite: return "Graphite"; - case VersionedCollapsing: return "VersionedCollapsing"; + case VersionedCollapsing: return "VersionedCollapsing"; default: throw Exception("Unknown mode of operation for MergeTreeData: " + toString(mode), ErrorCodes::LOGICAL_ERROR); diff --git a/dbms/src/Storages/MergeTree/MergeTreeDataPart.cpp b/dbms/src/Storages/MergeTree/MergeTreeDataPart.cpp index 1c3d21d4653..3aabc840ff9 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeDataPart.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeDataPart.cpp @@ -506,7 +506,7 @@ void MergeTreeDataPart::loadPartitionAndMinMaxIndex() String calculated_partition_id = partition.getID(storage); if (calculated_partition_id != info.partition_id) throw Exception( - "While loading part " + getFullPath() + ": calculated partition ID: " + calculated_partition_id + "While loading part " + getFullPath() + ": calculated partition ID: " + calculated_partition_id + " differs from partition ID in part name: " + info.partition_id, ErrorCodes::CORRUPTED_DATA); } diff --git a/dbms/src/Storages/MergeTree/MergeTreeReader.cpp b/dbms/src/Storages/MergeTree/MergeTreeReader.cpp index c9818c4ce7f..05c272f09f4 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeReader.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeReader.cpp @@ -273,7 +273,7 @@ void MergeTreeReader::Stream::loadMarks() size_t expected_file_size = sizeof(MarkInCompressedFile) * marks_count; if (expected_file_size != file_size) throw Exception( - "bad size of marks file `" + path + "':" + std::to_string(file_size) + ", must be: " + std::to_string(expected_file_size), + "bad size of marks file `" + path + "':" + std::to_string(file_size) + ", must be: " + std::to_string(expected_file_size), ErrorCodes::CORRUPTED_DATA); auto res = std::make_shared(marks_count); diff --git a/dbms/src/Storages/MergeTree/MergeTreeSettings.h b/dbms/src/Storages/MergeTree/MergeTreeSettings.h index e135cf24a1f..5378a3ec876 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeSettings.h +++ b/dbms/src/Storages/MergeTree/MergeTreeSettings.h @@ -156,7 +156,7 @@ struct MergeTreeSettings M(SettingUInt64, min_merge_bytes_to_use_direct_io, 10ULL * 1024 * 1024 * 1024) /// Settings that should not change after the creation of a table. -#define APPLY_FOR_IMMUTABLE_MERGE_TREE_SETTINGS(M) \ +#define APPLY_FOR_IMMUTABLE_MERGE_TREE_SETTINGS(M) \ M(index_granularity) #define DECLARE(TYPE, NAME, DEFAULT) \ diff --git a/dbms/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp b/dbms/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp index 7fda70af4dc..41a8f8ee224 100644 --- a/dbms/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp +++ b/dbms/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp @@ -1609,7 +1609,7 @@ bool ReplicatedMergeTreeMergePredicate::isMutationFinished(const ReplicatedMerge if (blocks_count) { LOG_TRACE(queue.log, "Mutation " << mutation.znode_name << " is not done yet because " - << "in partition ID " << partition_id << " there are still " + << "in partition ID " << partition_id << " there are still " << blocks_count << " uncommitted blocks."); return false; } diff --git a/dbms/src/Storages/MergeTree/ReplicatedMergeTreeRestartingThread.cpp b/dbms/src/Storages/MergeTree/ReplicatedMergeTreeRestartingThread.cpp index 4f48fc38a4a..1243094641e 100644 --- a/dbms/src/Storages/MergeTree/ReplicatedMergeTreeRestartingThread.cpp +++ b/dbms/src/Storages/MergeTree/ReplicatedMergeTreeRestartingThread.cpp @@ -202,7 +202,7 @@ bool ReplicatedMergeTreeRestartingThread::tryStartup() } catch (...) { - storage.replica_is_active_node = nullptr; + storage.replica_is_active_node = nullptr; try { diff --git a/dbms/src/Storages/MergeTree/registerStorageMergeTree.cpp b/dbms/src/Storages/MergeTree/registerStorageMergeTree.cpp index ebce8badca6..0b3efc2116e 100644 --- a/dbms/src/Storages/MergeTree/registerStorageMergeTree.cpp +++ b/dbms/src/Storages/MergeTree/registerStorageMergeTree.cpp @@ -138,8 +138,8 @@ static void appendGraphitePattern( { pattern.retentions.emplace_back( Graphite::Retention{ - .age = config.getUInt(config_element + "." + key + ".age"), - .precision = config.getUInt(config_element + "." + key + ".precision")}); + .age = config.getUInt(config_element + "." + key + ".age"), + .precision = config.getUInt(config_element + "." + key + ".precision")}); } else throw Exception("Unknown element in config: " + key, ErrorCodes::UNKNOWN_ELEMENT_IN_CONFIG); diff --git a/dbms/src/Storages/StorageCatBoostPool.cpp b/dbms/src/Storages/StorageCatBoostPool.cpp index 955ee4c1f1f..3a9dbc258d6 100644 --- a/dbms/src/Storages/StorageCatBoostPool.cpp +++ b/dbms/src/Storages/StorageCatBoostPool.cpp @@ -198,7 +198,7 @@ void StorageCatBoostPool::parseColumnDescription() } if (num_id >= columns_description.size()) - throw Exception("Invalid index at row " + str_line_num + ": " + str_id + throw Exception("Invalid index at row " + str_line_num + ": " + str_id + ", expected in range [0, " + std::to_string(columns_description.size()) + ")", ErrorCodes::CANNOT_PARSE_TEXT); diff --git a/dbms/src/Storages/StorageLog.cpp b/dbms/src/Storages/StorageLog.cpp index 9192dabeac5..0229a8ff2bb 100644 --- a/dbms/src/Storages/StorageLog.cpp +++ b/dbms/src/Storages/StorageLog.cpp @@ -536,7 +536,7 @@ void StorageLog::truncate(const ASTPtr &) for (auto & file : data_files) file.remove(false); - for (const auto & column : getColumns().getAllPhysical()) + for (const auto & column : getColumns().getAllPhysical()) addFiles(column.name, *column.type); file_checker = FileChecker{table_dir + "/" + "sizes.json"}; diff --git a/dbms/src/Storages/StorageMerge.cpp b/dbms/src/Storages/StorageMerge.cpp index e8684478065..6fcdb86e861 100644 --- a/dbms/src/Storages/StorageMerge.cpp +++ b/dbms/src/Storages/StorageMerge.cpp @@ -217,7 +217,7 @@ BlockInputStreams StorageMerge::read( for (auto it = selected_tables.begin(); it != selected_tables.end(); ++it) { - size_t current_need_streams = tables_count >= num_streams ? 1 : (num_streams / tables_count); + size_t current_need_streams = tables_count >= num_streams ? 1 : (num_streams / tables_count); size_t current_streams = std::min(current_need_streams, remaining_streams); remaining_streams -= current_streams; current_streams = std::max(1, current_streams); diff --git a/dbms/src/Storages/StorageMergeTree.h b/dbms/src/Storages/StorageMergeTree.h index 1dfbb617ec4..04856835fcf 100644 --- a/dbms/src/Storages/StorageMergeTree.h +++ b/dbms/src/Storages/StorageMergeTree.h @@ -160,7 +160,7 @@ private: friend struct CurrentlyMergingPartsTagger; protected: - /** Attach the table with the appropriate name, along the appropriate path (with / at the end), + /** Attach the table with the appropriate name, along the appropriate path (with / at the end), * (correctness of names and paths are not checked) * consisting of the specified columns. * diff --git a/dbms/src/Storages/StorageReplicatedMergeTree.cpp b/dbms/src/Storages/StorageReplicatedMergeTree.cpp index 022b6302779..abb2e46e584 100644 --- a/dbms/src/Storages/StorageReplicatedMergeTree.cpp +++ b/dbms/src/Storages/StorageReplicatedMergeTree.cpp @@ -749,7 +749,7 @@ void StorageReplicatedMergeTree::checkParts(bool skip_sanity_checks) if (insane && !skip_sanity_checks) { std::stringstream why; - why << "The local set of parts of table " << database_name << "." << table_name << " doesn't look like the set of parts " + why << "The local set of parts of table " << database_name << "." << table_name << " doesn't look like the set of parts " << "in ZooKeeper: " << formatReadableQuantity(total_suspicious_rows) << " rows of " << formatReadableQuantity(total_rows_on_filesystem) << " total rows in filesystem are suspicious."; @@ -1153,7 +1153,7 @@ bool StorageReplicatedMergeTree::tryExecuteMerge(const LogEntry & entry) MergeTreeDataMergerMutator::FuturePart future_merged_part(parts); if (future_merged_part.name != entry.new_part_name) { - throw Exception("Future merged part name `" + future_merged_part.name + "` differs from part name in log entry: `" + throw Exception("Future merged part name `" + future_merged_part.name + "` differs from part name in log entry: `" + entry.new_part_name + "`", ErrorCodes::BAD_DATA_PART_NAME); } @@ -1806,7 +1806,7 @@ bool StorageReplicatedMergeTree::executeReplaceRange(const LogEntry & entry) if (replica.empty()) { - LOG_DEBUG(log, "Part " << part_desc->new_part_name << " is not found on remote replicas"); + LOG_DEBUG(log, "Part " << part_desc->new_part_name << " is not found on remote replicas"); /// Fallback to covering part replica = findReplicaHavingCoveringPart(part_desc->new_part_name, true, found_part_name); @@ -1814,7 +1814,7 @@ bool StorageReplicatedMergeTree::executeReplaceRange(const LogEntry & entry) if (replica.empty()) { /// It is not fail, since adjacent parts could cover current part - LOG_DEBUG(log, "Parts covering " << part_desc->new_part_name << " are not found on remote replicas"); + LOG_DEBUG(log, "Parts covering " << part_desc->new_part_name << " are not found on remote replicas"); continue; } } @@ -2033,7 +2033,7 @@ void StorageReplicatedMergeTree::cloneReplica(const String & source_replica, Coo { LogEntry log_entry; log_entry.type = LogEntry::GET_PART; - log_entry.source_replica = ""; + log_entry.source_replica = ""; log_entry.new_part_name = name; log_entry.create_time = tryGetPartCreateTime(zookeeper, source_path, name); diff --git a/dbms/src/Storages/System/StorageSystemMutations.cpp b/dbms/src/Storages/System/StorageSystemMutations.cpp index 51bdc94720f..752b6a44160 100644 --- a/dbms/src/Storages/System/StorageSystemMutations.cpp +++ b/dbms/src/Storages/System/StorageSystemMutations.cpp @@ -17,17 +17,15 @@ namespace DB NamesAndTypesList StorageSystemMutations::getNamesAndTypes() { return { - { "database", std::make_shared() }, - { "table", std::make_shared() }, - { "mutation_id", std::make_shared() }, - { "command", std::make_shared() }, - { "create_time", std::make_shared() }, - { "block_numbers.partition_id", std::make_shared( - std::make_shared()) }, - { "block_numbers.number", std::make_shared( - std::make_shared()) }, - { "parts_to_do", std::make_shared() }, - { "is_done", std::make_shared() }, + { "database", std::make_shared() }, + { "table", std::make_shared() }, + { "mutation_id", std::make_shared() }, + { "command", std::make_shared() }, + { "create_time", std::make_shared() }, + { "block_numbers.partition_id", std::make_shared(std::make_shared()) }, + { "block_numbers.number", std::make_shared(std::make_shared()) }, + { "parts_to_do", std::make_shared() }, + { "is_done", std::make_shared() }, }; } diff --git a/dbms/src/Storages/System/StorageSystemTables.cpp b/dbms/src/Storages/System/StorageSystemTables.cpp index adbf1aeed06..7a09f0aa30c 100644 --- a/dbms/src/Storages/System/StorageSystemTables.cpp +++ b/dbms/src/Storages/System/StorageSystemTables.cpp @@ -104,7 +104,7 @@ protected: break; } - /// This is for temporary tables. They are output in single block regardless to max_block_size. + /// This is for temporary tables. They are output in single block regardless to max_block_size. if (database_idx >= databases->size()) { if (context.hasSessionContext()) diff --git a/dbms/src/Storages/tests/get_current_inserts_in_replicated.cpp b/dbms/src/Storages/tests/get_current_inserts_in_replicated.cpp index f5e69e59300..9012ccb6eb8 100644 --- a/dbms/src/Storages/tests/get_current_inserts_in_replicated.cpp +++ b/dbms/src/Storages/tests/get_current_inserts_in_replicated.cpp @@ -44,14 +44,14 @@ try lock_holder_paths.insert(zookeeper_path + "/temp/" + entry); } std::cerr << "Stage 1 (get lock holders): " << lock_holder_paths.size() - << " lock holders, elapsed: " << stage.elapsedSeconds() << "s." << std::endl; + << " lock holders, elapsed: " << stage.elapsedSeconds() << "s." << std::endl; stage.restart(); if (!lock_holder_paths.empty()) { Strings partitions = zookeeper->getChildren(zookeeper_path + "/block_numbers"); std::cerr << "Stage 2 (get partitions): " << partitions.size() - << " partitions, elapsed: " << stage.elapsedSeconds() << "s." << std::endl; + << " partitions, elapsed: " << stage.elapsedSeconds() << "s." << std::endl; stage.restart(); std::vector> lock_futures; @@ -79,7 +79,7 @@ try } } std::cerr << "Stage 3 (get block numbers): " << block_infos.size() - << " block numbers, elapsed: " << stage.elapsedSeconds() << "s." << std::endl; + << " block numbers, elapsed: " << stage.elapsedSeconds() << "s." << std::endl; stage.restart(); size_t total_count = 0; @@ -93,7 +93,7 @@ try } } std::cerr << "Stage 4 (get block number contents): " << total_count - << " current_inserts, elapsed: " << stage.elapsedSeconds() << "s." << std::endl; + << " current_inserts, elapsed: " << stage.elapsedSeconds() << "s." << std::endl; stage.restart(); } diff --git a/dbms/src/TableFunctions/TableFunctionRemote.cpp b/dbms/src/TableFunctions/TableFunctionRemote.cpp index acc094408f8..1478f8960a8 100644 --- a/dbms/src/TableFunctions/TableFunctionRemote.cpp +++ b/dbms/src/TableFunctions/TableFunctionRemote.cpp @@ -130,7 +130,7 @@ static std::vector parseDescription(const String & description, size_t l throw Exception("Table function 'remote': incorrect argument in braces (left number is greater then right): " + description.substr(i, m - i + 1), ErrorCodes::BAD_ARGUMENTS); - if (right - left + 1 > max_addresses) + if (right - left + 1 > max_addresses) throw Exception("Table function 'remote': first argument generates too many result addresses", ErrorCodes::BAD_ARGUMENTS); bool add_leading_zeroes = false; From 5a0feee45eddc8c6530815db0140d69f3038e3be Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 26 Nov 2018 04:39:04 +0300 Subject: [PATCH 114/124] Removed "conditional computations" feature, because it is not ready #2272 --- dbms/src/Functions/CMakeLists.txt | 7 - dbms/src/Functions/FunctionsProjection.cpp | 203 ----------- dbms/src/Functions/FunctionsProjection.h | 71 ---- dbms/src/Functions/registerFunctions.cpp | 2 - dbms/src/Interpreters/ActionsVisitor.cpp | 56 +-- dbms/src/Interpreters/ActionsVisitor.h | 4 +- dbms/src/Interpreters/ExpressionActions.cpp | 69 +--- dbms/src/Interpreters/ExpressionActions.h | 14 +- dbms/src/Interpreters/ExpressionAnalyzer.cpp | 7 +- dbms/src/Interpreters/ExpressionAnalyzer.h | 2 - .../Interpreters/ProjectionManipulation.cpp | 341 ------------------ .../src/Interpreters/ProjectionManipulation.h | 256 ------------- dbms/src/Interpreters/Settings.h | 1 - dbms/src/Storages/MergeTree/MergeTreeData.cpp | 2 +- .../00631_conditional_computation.reference | 4 - .../00631_conditional_computation.sql | 23 -- dbms/tests/queries/bugs/fuzzy.sql | 3 - 17 files changed, 29 insertions(+), 1036 deletions(-) delete mode 100644 dbms/src/Functions/FunctionsProjection.cpp delete mode 100644 dbms/src/Functions/FunctionsProjection.h delete mode 100644 dbms/src/Interpreters/ProjectionManipulation.cpp delete mode 100644 dbms/src/Interpreters/ProjectionManipulation.h delete mode 100644 dbms/tests/queries/0_stateless/00631_conditional_computation.reference delete mode 100644 dbms/tests/queries/0_stateless/00631_conditional_computation.sql diff --git a/dbms/src/Functions/CMakeLists.txt b/dbms/src/Functions/CMakeLists.txt index 47144516fb0..a6109c70934 100644 --- a/dbms/src/Functions/CMakeLists.txt +++ b/dbms/src/Functions/CMakeLists.txt @@ -32,13 +32,6 @@ generate_function_register(Arithmetic FunctionIntExp10 ) -generate_function_register(Projection - FunctionOneOrZero - FunctionProject - FunctionBuildProjectionComposition - FunctionRestoreProjection -) - add_headers_and_sources(clickhouse_functions .) add_headers_and_sources(clickhouse_functions ./GatherUtils) diff --git a/dbms/src/Functions/FunctionsProjection.cpp b/dbms/src/Functions/FunctionsProjection.cpp deleted file mode 100644 index c526d20b558..00000000000 --- a/dbms/src/Functions/FunctionsProjection.cpp +++ /dev/null @@ -1,203 +0,0 @@ -#include -#include -#include -#include - -namespace DB -{ -FunctionPtr FunctionOneOrZero::create(const Context &) -{ - return std::make_shared(); -} - -String FunctionOneOrZero::getName() const -{ - return name; -} - -size_t FunctionOneOrZero::getNumberOfArguments() const -{ - return 1; -} - -DataTypePtr FunctionOneOrZero::getReturnTypeImpl(const DataTypes & /*arguments*/) const -{ - return std::make_shared(); -} - -void FunctionOneOrZero::executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) -{ - const auto & data_column = block.getByPosition(arguments[0]).column; - auto col_res = ColumnUInt8::create(); - auto & vec_res = col_res->getData(); - vec_res.resize(data_column->size()); - for (size_t i = 0; i < data_column->size(); ++i) - { - if (data_column->getBool(i)) - { - vec_res[i] = 1; - } - else - { - vec_res[i] = 0; - } - } - block.getByPosition(result).column = std::move(col_res); -} - -FunctionPtr FunctionProject::create(const Context &) -{ - return std::make_shared(); -} - -String FunctionProject::getName() const -{ - return name; -} - -size_t FunctionProject::getNumberOfArguments() const -{ - return 2; -} - -DataTypePtr FunctionProject::getReturnTypeImpl(const DataTypes & arguments) const -{ - if (!checkAndGetDataType(arguments[1].get())) - { - throw Exception( - "Illegal type " + arguments[1]->getName() + " of 2nd argument of function " + getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); - } - return arguments[0]; -} - -void FunctionProject::executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) -{ - const auto & data_column = block.getByPosition(arguments[0]).column; - const auto & projection_column = block.getByPosition(arguments[1]).column; - if (const auto projection_column_uint8 = checkAndGetColumn(projection_column.get())) - { - block.getByPosition(result).column = data_column->filter(projection_column_uint8->getData(), -1); - } - else if (const auto projection_column_uint8_const = checkAndGetColumnConst(projection_column.get())) - { - if (projection_column_uint8_const->getBool(0)) - { - block.getByPosition(result).column = data_column->cloneResized(data_column->size()); - } - else - { - block.getByPosition(result).column = data_column->cloneEmpty(); - } - } - else - { - throw Exception("Unexpected column: " + projection_column->getName(), ErrorCodes::ILLEGAL_COLUMN); - } -} - -FunctionPtr FunctionBuildProjectionComposition::create(const Context &) -{ - return std::make_shared(); -} - -String FunctionBuildProjectionComposition::getName() const -{ - return name; -} - -size_t FunctionBuildProjectionComposition::getNumberOfArguments() const -{ - return 2; -} - -DataTypePtr FunctionBuildProjectionComposition::getReturnTypeImpl(const DataTypes & arguments) const -{ - for (size_t i = 0; i < 2; ++i) - { - if (!checkAndGetDataType(arguments[i].get())) - { - throw Exception( - "Illegal type " + arguments[i]->getName() + " of " + std::to_string(i + 1) + " argument of function " + getName(), - ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); - } - } - return std::make_shared(); -} - -void FunctionBuildProjectionComposition::executeImpl( - Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) -{ - const auto & first_projection_column = block.getByPosition(arguments[0]).column; - const auto & second_projection_column = block.getByPosition(arguments[1]).column; - auto col_res = ColumnUInt8::create(); - auto & vec_res = col_res->getData(); - vec_res.resize(first_projection_column->size()); - size_t current_reserve_index = 0; - for (size_t i = 0; i < first_projection_column->size(); ++i) - { - if (!first_projection_column->getBool(i)) - { - vec_res[i] = 0; - } - else - { - vec_res[i] = second_projection_column->getBool(current_reserve_index); - ++current_reserve_index; - } - } - if (current_reserve_index != second_projection_column->size()) - { - throw Exception("Second argument size is not appropriate: " + std::to_string(second_projection_column->size()) + " instead of " - + std::to_string(current_reserve_index), - ErrorCodes::BAD_ARGUMENTS); - } - block.getByPosition(result).column = std::move(col_res); -} - -FunctionPtr FunctionRestoreProjection::create(const Context &) -{ - return std::make_shared(); -} - -String FunctionRestoreProjection::getName() const -{ - return name; -} - -bool FunctionRestoreProjection::isVariadic() const -{ - return true; -} - -size_t FunctionRestoreProjection::getNumberOfArguments() const -{ - return 0; -} - -DataTypePtr FunctionRestoreProjection::getReturnTypeImpl(const DataTypes & arguments) const -{ - if (arguments.size() < 2) - { - throw Exception("Wrong argument count: " + std::to_string(arguments.size()), ErrorCodes::BAD_ARGUMENTS); - } - return arguments[1]; -} - -void FunctionRestoreProjection::executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) -{ - if (arguments.size() < 2) - { - throw Exception("Wrong argument count: " + std::to_string(arguments.size()), ErrorCodes::BAD_ARGUMENTS); - } - const auto & projection_column = block.getByPosition(arguments[0]).column; - auto col_res = block.getByPosition(arguments[1]).column->cloneEmpty(); - std::vector override_indices(arguments.size() - 1, 0); - for (size_t i = 0; i < projection_column->size(); ++i) - { - size_t argument_index = projection_column->getBool(i); - col_res->insertFrom(*block.getByPosition(arguments[argument_index + 1]).column, override_indices[argument_index]++); - } - block.getByPosition(result).column = std::move(col_res); -} - -} diff --git a/dbms/src/Functions/FunctionsProjection.h b/dbms/src/Functions/FunctionsProjection.h deleted file mode 100644 index bbb1951fc5b..00000000000 --- a/dbms/src/Functions/FunctionsProjection.h +++ /dev/null @@ -1,71 +0,0 @@ -#pragma once - -#include -#include "FunctionsConversion.h" - -namespace DB -{ -/* - * This function accepts one column and converts it to UInt8, replacing values, which evaluate to true, with 1, and values, - * which evaluate to false with 0 - */ -class FunctionOneOrZero final : public IFunction -{ -public: - static constexpr auto name = "one_or_zero"; - static FunctionPtr create(const Context &); - String getName() const override; - size_t getNumberOfArguments() const override; - DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override; - void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override; -}; - -/* - * FunctionProject accepts two columns: data column and projection column. - * Projection column is a column of UInt8 values 0 and 1, which indicate the binary mask of rows, where to project. - * This function builds a column of a smaller, which contains values of the data column at the positions where - * the projection column contained 1. The size of result column equals the count of ones in the projection column. - */ -class FunctionProject final : public IFunction -{ -public: - static constexpr auto name = "__inner_project__"; - static FunctionPtr create(const Context &); - String getName() const override; - size_t getNumberOfArguments() const override; - DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override; - void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override; -}; - -/* - * FunctionBuildProjectionComposition constructs the composition of two projection columns. The size of - * second projection column should equal the count of ones in the first input projection column. - */ -class FunctionBuildProjectionComposition final : public IFunction -{ -public: - static constexpr auto name = "__inner_build_projection_composition__"; - static FunctionPtr create(const Context &); - String getName() const override; - size_t getNumberOfArguments() const override; - DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override; - void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override; -}; - -/* - * Accepts mapping column with values from range [0, N) and N more columns as arguments. - * Forms a column by taking value from column, which number is in the mapping column. - */ -class FunctionRestoreProjection final : public IFunction -{ -public: - static constexpr auto name = "__inner_restore_projection__"; - static FunctionPtr create(const Context &); - String getName() const override; - bool isVariadic() const override; - size_t getNumberOfArguments() const override; - DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override; - void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override; -}; - -} diff --git a/dbms/src/Functions/registerFunctions.cpp b/dbms/src/Functions/registerFunctions.cpp index c7f1ca82cab..292f6f608df 100644 --- a/dbms/src/Functions/registerFunctions.cpp +++ b/dbms/src/Functions/registerFunctions.cpp @@ -40,7 +40,6 @@ void registerFunctionsTransform(FunctionFactory &); void registerFunctionsGeo(FunctionFactory &); void registerFunctionsNull(FunctionFactory &); void registerFunctionsFindCluster(FunctionFactory &); -void registerFunctionsProjection(FunctionFactory &); #if USE_ICU void registerFunctionConvertCharset(FunctionFactory &); @@ -80,7 +79,6 @@ void registerFunctions() registerFunctionsGeo(factory); registerFunctionsNull(factory); registerFunctionsFindCluster(factory); - registerFunctionsProjection(factory); #if USE_ICU registerFunctionConvertCharset(factory); diff --git a/dbms/src/Interpreters/ActionsVisitor.cpp b/dbms/src/Interpreters/ActionsVisitor.cpp index c781da56bcc..ef9f73ac194 100644 --- a/dbms/src/Interpreters/ActionsVisitor.cpp +++ b/dbms/src/Interpreters/ActionsVisitor.cpp @@ -27,7 +27,6 @@ #include #include -#include #include #include #include @@ -223,7 +222,7 @@ const Block & ScopeStack::getSampleBlock() const ActionsVisitor::ActionsVisitor( - const Context & context_, SizeLimits set_size_limit_, bool is_conditional_tree, size_t subquery_depth_, + const Context & context_, SizeLimits set_size_limit_, size_t subquery_depth_, const NamesAndTypesList & source_columns_, const ExpressionActionsPtr & actions, PreparedSets & prepared_sets_, SubqueriesForSets & subqueries_for_sets_, bool no_subqueries_, bool only_consts_, bool no_storage_or_local_, std::ostream * ostr_) @@ -240,10 +239,6 @@ ActionsVisitor::ActionsVisitor( ostr(ostr_), actions_stack(actions, context) { - if (is_conditional_tree) - projection_manipulator = std::make_shared(actions_stack, context); - else - projection_manipulator = std::make_shared(actions_stack); } void ActionsVisitor::visit(const ASTPtr & ast) @@ -261,12 +256,12 @@ void ActionsVisitor::visit(const ASTPtr & ast) /// If the result of the calculation already exists in the block. if ((typeid_cast(ast.get()) || typeid_cast(ast.get())) - && projection_manipulator->tryToGetFromUpperProjection(getColumnName())) + && actions_stack.getSampleBlock().has(getColumnName())) return; if (auto * identifier = typeid_cast(ast.get())) { - if (!only_consts && !projection_manipulator->tryToGetFromUpperProjection(getColumnName())) + if (!only_consts && !actions_stack.getSampleBlock().has(getColumnName())) { /// The requested column is not in the block. /// If such a column exists in the table, then the user probably forgot to surround it with an aggregate function or add it to GROUP BY. @@ -300,8 +295,8 @@ void ActionsVisitor::visit(const ASTPtr & ast) visit(arg); if (!only_consts) { - String result_name = projection_manipulator->getColumnName(getColumnName()); - actions_stack.addAction(ExpressionAction::copyColumn(projection_manipulator->getColumnName(arg->getColumnName()), result_name)); + String result_name = getColumnName(); + actions_stack.addAction(ExpressionAction::copyColumn(arg->getColumnName(), result_name)); NameSet joined_columns; joined_columns.insert(result_name); actions_stack.addAction(ExpressionAction::arrayJoin(joined_columns, false, context)); @@ -330,8 +325,7 @@ void ActionsVisitor::visit(const ASTPtr & ast) actions_stack.addAction(ExpressionAction::applyFunction( FunctionFactory::instance().get("ignore", context), { node->arguments->children.at(0)->getColumnName() }, - projection_manipulator->getColumnName(getColumnName()), - projection_manipulator->getProjectionSourceColumn())); + getColumnName())); } return; } @@ -343,7 +337,7 @@ void ActionsVisitor::visit(const ASTPtr & ast) { actions_stack.addAction(ExpressionAction::addColumn(ColumnWithTypeAndName( ColumnConst::create(ColumnUInt8::create(1, 1), 1), std::make_shared(), - projection_manipulator->getColumnName(getColumnName())), projection_manipulator->getProjectionSourceColumn(), false)); + getColumnName()))); return; } @@ -356,7 +350,6 @@ void ActionsVisitor::visit(const ASTPtr & ast) : context; const FunctionBuilderPtr & function_builder = FunctionFactory::instance().get(node->name, function_context); - auto projection_action = getProjectionAction(node->name, actions_stack, projection_manipulator, getColumnName(), function_context); Names argument_names; DataTypes argument_types; @@ -401,13 +394,11 @@ void ActionsVisitor::visit(const ASTPtr & ast) else column.name = child_column_name; - column.name = projection_manipulator->getColumnName(column.name); - if (!actions_stack.getSampleBlock().has(column.name)) { column.column = ColumnSet::create(1, set); - actions_stack.addAction(ExpressionAction::addColumn(column, projection_manipulator->getProjectionSourceColumn(), false)); + actions_stack.addAction(ExpressionAction::addColumn(column)); } argument_types.push_back(column.type); @@ -416,10 +407,8 @@ void ActionsVisitor::visit(const ASTPtr & ast) else { /// If the argument is not a lambda expression, call it recursively and find out its type. - projection_action->preArgumentAction(); visit(child); - std::string name = projection_manipulator->getColumnName(child_column_name); - projection_action->postArgumentAction(child_column_name); + std::string name = child_column_name; if (actions_stack.getSampleBlock().has(name)) { argument_types.push_back(actions_stack.getSampleBlock().getByName(name).type); @@ -428,13 +417,9 @@ void ActionsVisitor::visit(const ASTPtr & ast) else { if (only_consts) - { arguments_present = false; - } else - { - throw Exception("Unknown identifier: " + name + ", projection layer " + projection_manipulator->getProjectionExpression() , ErrorCodes::UNKNOWN_IDENTIFIER); - } + throw Exception("Unknown identifier: " + name, ErrorCodes::UNKNOWN_IDENTIFIER); } } } @@ -470,12 +455,11 @@ void ActionsVisitor::visit(const ASTPtr & ast) lambda_arguments.emplace_back(arg_name, lambda_type->getArgumentTypes()[j]); } - projection_action->preArgumentAction(); actions_stack.pushLevel(lambda_arguments); visit(lambda->arguments->children.at(1)); ExpressionActionsPtr lambda_actions = actions_stack.popLevel(); - String result_name = projection_manipulator->getColumnName(lambda->arguments->children.at(1)->getColumnName()); + String result_name = lambda->arguments->children.at(1)->getColumnName(); lambda_actions->finalize(Names(1, result_name)); DataTypePtr result_type = lambda_actions->getSampleBlock().getByName(result_name).type; @@ -491,12 +475,10 @@ void ActionsVisitor::visit(const ASTPtr & ast) auto function_capture = std::make_shared( lambda_actions, captured, lambda_arguments, result_type, result_name); - actions_stack.addAction(ExpressionAction::applyFunction(function_capture, captured, lambda_name, - projection_manipulator->getProjectionSourceColumn())); + actions_stack.addAction(ExpressionAction::applyFunction(function_capture, captured, lambda_name)); argument_types[i] = std::make_shared(lambda_type->getArgumentTypes(), result_type); argument_names[i] = lambda_name; - projection_action->postArgumentAction(lambda_name); } } } @@ -515,15 +497,8 @@ void ActionsVisitor::visit(const ASTPtr & ast) if (arguments_present) { - projection_action->preCalculation(); - if (projection_action->isCalculationRequired()) - { - actions_stack.addAction( - ExpressionAction::applyFunction(function_builder, - argument_names, - projection_manipulator->getColumnName(getColumnName()), - projection_manipulator->getProjectionSourceColumn())); - } + actions_stack.addAction( + ExpressionAction::applyFunction(function_builder, argument_names, getColumnName())); } } else if (ASTLiteral * literal = typeid_cast(ast.get())) @@ -535,8 +510,7 @@ void ActionsVisitor::visit(const ASTPtr & ast) column.type = type; column.name = getColumnName(); - actions_stack.addAction(ExpressionAction::addColumn(column, "", false)); - projection_manipulator->tryToGetFromUpperProjection(column.name); + actions_stack.addAction(ExpressionAction::addColumn(column)); } else { diff --git a/dbms/src/Interpreters/ActionsVisitor.h b/dbms/src/Interpreters/ActionsVisitor.h index 0344934455d..805b0ec02c4 100644 --- a/dbms/src/Interpreters/ActionsVisitor.h +++ b/dbms/src/Interpreters/ActionsVisitor.h @@ -9,7 +9,6 @@ namespace DB class Context; class ASTFunction; -struct ProjectionManipulatorBase; class Set; @@ -89,7 +88,7 @@ struct ScopeStack class ActionsVisitor { public: - ActionsVisitor(const Context & context_, SizeLimits set_size_limit_, bool is_conditional_tree, size_t subquery_depth_, + ActionsVisitor(const Context & context_, SizeLimits set_size_limit_, size_t subquery_depth_, const NamesAndTypesList & source_columns_, const ExpressionActionsPtr & actions, PreparedSets & prepared_sets_, SubqueriesForSets & subqueries_for_sets_, bool no_subqueries_, bool only_consts_, bool no_storage_or_local_, std::ostream * ostr_ = nullptr); @@ -111,7 +110,6 @@ private: mutable size_t visit_depth; std::ostream * ostr; ScopeStack actions_stack; - std::shared_ptr projection_manipulator; void makeSet(const ASTFunction * node, const Block & sample_block); }; diff --git a/dbms/src/Interpreters/ExpressionActions.cpp b/dbms/src/Interpreters/ExpressionActions.cpp index 11a7ecfb1a4..b1fab40a654 100644 --- a/dbms/src/Interpreters/ExpressionActions.cpp +++ b/dbms/src/Interpreters/ExpressionActions.cpp @@ -51,11 +51,6 @@ Names ExpressionAction::getNeededColumns() const if (!source_name.empty()) res.push_back(source_name); - if (!row_projection_column.empty()) - { - res.push_back(row_projection_column); - } - return res; } @@ -63,8 +58,7 @@ Names ExpressionAction::getNeededColumns() const ExpressionAction ExpressionAction::applyFunction( const FunctionBuilderPtr & function_, const std::vector & argument_names_, - std::string result_name_, - const std::string & row_projection_column) + std::string result_name_) { if (result_name_ == "") { @@ -83,22 +77,17 @@ ExpressionAction ExpressionAction::applyFunction( a.result_name = result_name_; a.function_builder = function_; a.argument_names = argument_names_; - a.row_projection_column = row_projection_column; return a; } ExpressionAction ExpressionAction::addColumn( - const ColumnWithTypeAndName & added_column_, - const std::string & row_projection_column, - bool is_row_projection_complementary) + const ColumnWithTypeAndName & added_column_) { ExpressionAction a; a.type = ADD_COLUMN; a.result_name = added_column_.name; a.result_type = added_column_.type; a.added_column = added_column_.column; - a.row_projection_column = row_projection_column; - a.is_row_projection_complementary = is_row_projection_complementary; return a; } @@ -335,40 +324,10 @@ void ExpressionAction::prepare(Block & sample_block, const Settings & settings) } } -size_t ExpressionAction::getInputRowsCount(Block & block, std::unordered_map & input_rows_counts) const + +void ExpressionAction::execute(Block & block) const { - auto it = input_rows_counts.find(row_projection_column); - size_t projection_space_dimension; - if (it == input_rows_counts.end()) - { - const auto & projection_column = block.getByName(row_projection_column).column; - projection_space_dimension = 0; - for (size_t i = 0; i < projection_column->size(); ++i) - if (projection_column->getBool(i)) - ++projection_space_dimension; - - input_rows_counts[row_projection_column] = projection_space_dimension; - } - else - { - projection_space_dimension = it->second; - } - size_t parent_space_dimension; - if (row_projection_column.empty()) - { - parent_space_dimension = input_rows_counts[""]; - } - else - { - parent_space_dimension = block.getByName(row_projection_column).column->size(); - } - - return is_row_projection_complementary ? parent_space_dimension - projection_space_dimension : projection_space_dimension; -} - -void ExpressionAction::execute(Block & block, std::unordered_map & input_rows_counts) const -{ - size_t input_rows_count = getInputRowsCount(block, input_rows_counts); + size_t input_rows_count = block.rows(); if (type == REMOVE_COLUMN || type == COPY_COLUMN) if (!block.has(source_name)) @@ -463,8 +422,6 @@ void ExpressionAction::execute(Block & block, std::unordered_map input_rows_counts; - input_rows_counts[""] = block.rows(); if (type != JOIN) - execute(block, input_rows_counts); + execute(block); else join->joinTotals(block); } @@ -751,11 +706,9 @@ bool ExpressionActions::popUnusedArrayJoin(const Names & required_columns, Expre void ExpressionActions::execute(Block & block) const { - std::unordered_map input_rows_counts; - input_rows_counts[""] = block.rows(); for (const auto & action : actions) { - action.execute(block, input_rows_counts); + action.execute(block); checkLimits(block); } } @@ -980,9 +933,6 @@ void ExpressionActions::finalize(const Names & output_columns) if (!action.source_name.empty()) ++columns_refcount[action.source_name]; - if (!action.row_projection_column.empty()) - ++columns_refcount[action.row_projection_column]; - for (const auto & name : action.argument_names) ++columns_refcount[name]; @@ -1011,9 +961,6 @@ void ExpressionActions::finalize(const Names & output_columns) if (!action.source_name.empty()) process(action.source_name); - if (!action.row_projection_column.empty()) - process(action.row_projection_column); - for (const auto & name : action.argument_names) process(name); @@ -1240,8 +1187,6 @@ bool ExpressionAction::operator==(const ExpressionAction & other) const return source_name == other.source_name && result_name == other.result_name - && row_projection_column == other.row_projection_column - && is_row_projection_complementary == other.is_row_projection_complementary && argument_names == other.argument_names && array_joined_columns == other.array_joined_columns && array_join_is_left == other.array_join_is_left diff --git a/dbms/src/Interpreters/ExpressionActions.h b/dbms/src/Interpreters/ExpressionActions.h index 4dc81c7d938..781134dbeb2 100644 --- a/dbms/src/Interpreters/ExpressionActions.h +++ b/dbms/src/Interpreters/ExpressionActions.h @@ -81,10 +81,6 @@ public: /// If COPY_COLUMN can replace the result column. bool can_replace = false; - /// For conditional projections (projections on subset of rows) - std::string row_projection_column; - bool is_row_projection_complementary = false; - /// For ADD_COLUMN. ColumnPtr added_column; @@ -112,12 +108,9 @@ public: /// If result_name_ == "", as name "function_name(arguments separated by commas) is used". static ExpressionAction applyFunction( - const FunctionBuilderPtr & function_, const std::vector & argument_names_, std::string result_name_ = "", - const std::string & row_projection_column = ""); + const FunctionBuilderPtr & function_, const std::vector & argument_names_, std::string result_name_ = ""); - static ExpressionAction addColumn(const ColumnWithTypeAndName & added_column_, - const std::string & row_projection_column, - bool is_row_projection_complementary); + static ExpressionAction addColumn(const ColumnWithTypeAndName & added_column_); static ExpressionAction removeColumn(const std::string & removed_name); static ExpressionAction copyColumn(const std::string & from_name, const std::string & to_name, bool can_replace = false); static ExpressionAction project(const NamesWithAliases & projected_columns_); @@ -143,8 +136,7 @@ private: friend class ExpressionActions; void prepare(Block & sample_block, const Settings & settings); - size_t getInputRowsCount(Block & block, std::unordered_map & input_rows_counts) const; - void execute(Block & block, std::unordered_map & input_rows_counts) const; + void execute(Block & block) const; void executeOnTotals(Block & block) const; }; diff --git a/dbms/src/Interpreters/ExpressionAnalyzer.cpp b/dbms/src/Interpreters/ExpressionAnalyzer.cpp index 8e00a77e109..d9f2406ef4b 100644 --- a/dbms/src/Interpreters/ExpressionAnalyzer.cpp +++ b/dbms/src/Interpreters/ExpressionAnalyzer.cpp @@ -392,10 +392,8 @@ bool ExpressionAnalyzer::isThereArrayJoin(const ASTPtr & ast) void ExpressionAnalyzer::getRootActions(const ASTPtr & ast, bool no_subqueries, ExpressionActionsPtr & actions, bool only_consts) { - bool is_conditional_tree = !isThereArrayJoin(ast) && settings.enable_conditional_computation && !only_consts; - LogAST log; - ActionsVisitor actions_visitor(context, settings.size_limits_for_set, is_conditional_tree, subquery_depth, + ActionsVisitor actions_visitor(context, settings.size_limits_for_set, subquery_depth, source_columns, actions, prepared_sets, subqueries_for_sets, no_subqueries, only_consts, !isRemoteStorage(), log.stream()); actions_visitor.visit(ast); @@ -406,10 +404,9 @@ void ExpressionAnalyzer::getRootActions(const ASTPtr & ast, bool no_subqueries, void ExpressionAnalyzer::getActionsFromJoinKeys(const ASTTableJoin & table_join, bool no_subqueries, ExpressionActionsPtr & actions) { bool only_consts = false; - bool is_conditional_tree = !isThereArrayJoin(query) && settings.enable_conditional_computation && !only_consts; LogAST log; - ActionsVisitor actions_visitor(context, settings.size_limits_for_set, is_conditional_tree, subquery_depth, + ActionsVisitor actions_visitor(context, settings.size_limits_for_set, subquery_depth, source_columns, actions, prepared_sets, subqueries_for_sets, no_subqueries, only_consts, !isRemoteStorage(), log.stream()); diff --git a/dbms/src/Interpreters/ExpressionAnalyzer.h b/dbms/src/Interpreters/ExpressionAnalyzer.h index 0c714e78af0..ba895b41650 100644 --- a/dbms/src/Interpreters/ExpressionAnalyzer.h +++ b/dbms/src/Interpreters/ExpressionAnalyzer.h @@ -113,7 +113,6 @@ private: /// for ExpressionAnalyzer const bool asterisk_left_columns_only; const bool use_index_for_in_with_subqueries; - const bool enable_conditional_computation; const bool join_use_nulls; const SizeLimits size_limits_for_set; const SizeLimits size_limits_for_join; @@ -127,7 +126,6 @@ private: enable_optimize_predicate_expression(settings.enable_optimize_predicate_expression), asterisk_left_columns_only(settings.asterisk_left_columns_only), use_index_for_in_with_subqueries(settings.use_index_for_in_with_subqueries), - enable_conditional_computation(settings.enable_conditional_computation), join_use_nulls(settings.join_use_nulls), size_limits_for_set(settings.max_rows_in_set, settings.max_bytes_in_set, settings.set_overflow_mode), size_limits_for_join(settings.max_rows_in_join, settings.max_bytes_in_join, settings.join_overflow_mode), diff --git a/dbms/src/Interpreters/ProjectionManipulation.cpp b/dbms/src/Interpreters/ProjectionManipulation.cpp deleted file mode 100644 index afdc289d2fa..00000000000 --- a/dbms/src/Interpreters/ProjectionManipulation.cpp +++ /dev/null @@ -1,341 +0,0 @@ -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include - -namespace DB -{ -ProjectionManipulatorBase::~ProjectionManipulatorBase() {} - -DefaultProjectionManipulator::DefaultProjectionManipulator(ScopeStack & scopes) : scopes(scopes) {} - -bool DefaultProjectionManipulator::tryToGetFromUpperProjection(const std::string & column_name) -{ - return scopes.getSampleBlock().has(column_name); -} - -std::string DefaultProjectionManipulator::getColumnName(const std::string & column_name) const -{ - return column_name; -} - -std::string DefaultProjectionManipulator::getProjectionExpression() -{ - return ""; -} - -std::string DefaultProjectionManipulator::getProjectionSourceColumn() const -{ - return ""; -} - -ConditionalTree::Node::Node() : projection_expression_string(), parent_node(0), is_root(false) {} - -size_t ConditionalTree::Node::getParentNode() const -{ - if (is_root) - { - throw Exception( - "Failed to get parent projection node of node " + projection_expression_string, ErrorCodes::CONDITIONAL_TREE_PARENT_NOT_FOUND); - } - else - { - return parent_node; - } -} - -std::string ConditionalTree::getColumnNameByIndex(const std::string & col_name, const size_t node) const -{ - std::string projection_name = nodes[node].projection_expression_string; - if (projection_name.empty()) - { - return col_name; - } - else - { - return col_name + '<' + projection_name + '>'; - } -} - -std::string ConditionalTree::getColumnName(const std::string & col_name) const -{ - return getColumnNameByIndex(col_name, current_node); -} - -std::string ConditionalTree::getProjectionColumnName( - const std::string & first_projection_expr, const std::string & second_projection_expr) const -{ - return std::string("P<") + first_projection_expr + "><" + second_projection_expr + ">"; -} - -std::string ConditionalTree::getProjectionColumnName(const size_t first_index, const size_t second_index) const -{ - return getProjectionColumnName(nodes[first_index].projection_expression_string, nodes[second_index].projection_expression_string); -} - -void ConditionalTree::buildProjectionCompositionRecursive( - const std::vector & path, const size_t child_index, const size_t parent_index) -{ - std::string projection_name = getProjectionColumnName(path[parent_index], path[child_index]); - if (parent_index - child_index >= 2 && !scopes.getSampleBlock().has(projection_name)) - { - size_t middle_index = (child_index + parent_index) / 2; - buildProjectionCompositionRecursive(path, child_index, middle_index); - buildProjectionCompositionRecursive(path, middle_index, parent_index); - const FunctionBuilderPtr & function_builder = FunctionFactory::instance().get("__inner_build_projection_composition__", context); - scopes.addAction(ExpressionAction::applyFunction(function_builder, - {getProjectionColumnName(path[parent_index], path[middle_index]), - getProjectionColumnName(path[middle_index], path[child_index])}, - projection_name, - getProjectionSourceColumn())); - } -} - -void ConditionalTree::buildProjectionComposition(const size_t child_node, const size_t parent_node) -{ - std::vector path; - size_t node = child_node; - while (true) - { - path.push_back(node); - if (node == parent_node) - { - break; - } - node = nodes[node].getParentNode(); - } - buildProjectionCompositionRecursive(path, 0, path.size() - 1); -} - -std::string ConditionalTree::getProjectionSourceColumn(size_t node) const -{ - if (nodes[node].is_root) - { - return ""; - } - else - { - return ConditionalTree::getProjectionColumnName(nodes[node].getParentNode(), node); - } -} - -ConditionalTree::ConditionalTree(ScopeStack & scopes, const Context & context) - : current_node(0), nodes(1), scopes(scopes), context(context), projection_expression_index() -{ - nodes[0].is_root = true; -} - -void ConditionalTree::goToProjection(const std::string & field_name) -{ - std::string current_projection_name = nodes[current_node].projection_expression_string; - std::string new_projection_name = current_projection_name.empty() ? field_name : current_projection_name + ";" + field_name; - std::string projection_column_name = getProjectionColumnName(current_projection_name, new_projection_name); - if (!scopes.getSampleBlock().has(projection_column_name)) - { - const FunctionBuilderPtr & function_builder = FunctionFactory::instance().get("one_or_zero", context); - scopes.addAction(ExpressionAction::applyFunction( - function_builder, {getColumnName(field_name)}, projection_column_name, getProjectionSourceColumn())); - nodes.emplace_back(Node()); - nodes.back().projection_expression_string = new_projection_name; - nodes.back().parent_node = current_node; - current_node = nodes.size() - 1; - projection_expression_index[projection_column_name] = current_node; - } - else - { - current_node = projection_expression_index[projection_column_name]; - } -} - -std::string ConditionalTree::buildRestoreProjectionAndGetName(const size_t levels_up) -{ - size_t target_node = current_node; - for (size_t i = 0; i < levels_up; ++i) - { - target_node = nodes[target_node].getParentNode(); - } - buildProjectionComposition(current_node, target_node); - return getProjectionColumnName(target_node, current_node); -} - -void ConditionalTree::restoreColumn( - const std::string & default_values_name, const std::string & new_values_name, const size_t levels_up, const std::string & result_name) -{ - size_t target_node = current_node; - for (size_t i = 0; i < levels_up; ++i) - { - target_node = nodes[target_node].getParentNode(); - } - const FunctionBuilderPtr & function_builder = FunctionFactory::instance().get("__inner_restore_projection__", context); - scopes.addAction(ExpressionAction::applyFunction(function_builder, - {getProjectionColumnName(target_node, current_node), - getColumnNameByIndex(default_values_name, current_node), - getColumnNameByIndex(new_values_name, current_node)}, - getColumnNameByIndex(result_name, target_node), - getProjectionSourceColumn())); -} - -void ConditionalTree::goUp(const size_t levels_up) -{ - for (size_t i = 0; i < levels_up; ++i) - { - current_node = nodes[current_node].getParentNode(); - } -} - -bool ConditionalTree::tryToGetFromUpperProjection(const std::string & column_name) -{ - size_t node = current_node; - while (true) - { - if (scopes.getSampleBlock().has(getColumnNameByIndex(column_name, node))) - { - if (node != current_node) - { - buildProjectionComposition(current_node, node); - const FunctionBuilderPtr & function_builder = FunctionFactory::instance().get("__inner_project__", context); - scopes.addAction(ExpressionAction::applyFunction(function_builder, - {getColumnNameByIndex(column_name, node), getProjectionColumnName(node, current_node)}, - getColumnName(column_name), - getProjectionSourceColumn(node))); - } - return true; - } - if (nodes[node].is_root) - { - break; - } - node = nodes[node].getParentNode(); - } - return false; -} - -std::string ConditionalTree::getProjectionExpression() -{ - return nodes[current_node].projection_expression_string; -} - -std::string ConditionalTree::getProjectionSourceColumn() const -{ - return getProjectionSourceColumn(current_node); -} - -void DefaultProjectionAction::preArgumentAction() {} - -void DefaultProjectionAction::postArgumentAction(const std::string & /*argument_name*/) {} - -void DefaultProjectionAction::preCalculation() {} - -bool DefaultProjectionAction::isCalculationRequired() -{ - return true; -} - -AndOperatorProjectionAction::AndOperatorProjectionAction( - ScopeStack & scopes, ProjectionManipulatorPtr projection_manipulator, const std::string & expression_name, const Context & context) - : scopes(scopes) - , projection_manipulator(projection_manipulator) - , previous_argument_name() - , projection_levels_count(0) - , expression_name(expression_name) - , context(context) -{ -} - -std::string AndOperatorProjectionAction::getZerosColumnName() -{ - return "__inner_zeroes_column__" + expression_name; -} - -std::string AndOperatorProjectionAction::getFinalColumnName() -{ - return "__inner_final_column__" + expression_name; -} - -void AndOperatorProjectionAction::createZerosColumn(const std::string & restore_projection_name) -{ - auto zeros_column_name = projection_manipulator->getColumnName(getZerosColumnName()); - if (!scopes.getSampleBlock().has(zeros_column_name)) - { - scopes.addAction(ExpressionAction::addColumn( - ColumnWithTypeAndName(ColumnUInt8::create(0, 1), std::make_shared(), zeros_column_name), - restore_projection_name, - true)); - } -} - -void AndOperatorProjectionAction::preArgumentAction() -{ - if (!previous_argument_name.empty()) - { - // Before processing arguments starting from second to last - if (auto * conditional_tree = typeid_cast(projection_manipulator.get())) - { - conditional_tree->goToProjection(previous_argument_name); - } - else - { - throw Exception( - "Illegal projection manipulator used in AndOperatorProjectionAction", ErrorCodes::ILLEGAL_PROJECTION_MANIPULATOR); - } - ++projection_levels_count; - } -} - -void AndOperatorProjectionAction::postArgumentAction(const std::string & argument_name) -{ - previous_argument_name = argument_name; -} - -void AndOperatorProjectionAction::preCalculation() -{ - if (auto * conditional_tree = typeid_cast(projection_manipulator.get())) - { - auto final_column = getFinalColumnName(); - const FunctionBuilderPtr & function_builder = FunctionFactory::instance().get("one_or_zero", context); - scopes.addAction(ExpressionAction::applyFunction(function_builder, - {projection_manipulator->getColumnName(previous_argument_name)}, - projection_manipulator->getColumnName(final_column), - projection_manipulator->getProjectionSourceColumn())); - std::string restore_projection_name = conditional_tree->buildRestoreProjectionAndGetName(projection_levels_count); - createZerosColumn(restore_projection_name); - conditional_tree->restoreColumn(getZerosColumnName(), final_column, projection_levels_count, expression_name); - conditional_tree->goUp(projection_levels_count); - } - else - { - throw Exception("Illegal projection manipulator used in AndOperatorProjectionAction", ErrorCodes::ILLEGAL_PROJECTION_MANIPULATOR); - } -} - -bool AndOperatorProjectionAction::isCalculationRequired() -{ - return false; -} - -ProjectionActionBase::~ProjectionActionBase() {} - -ProjectionActionPtr getProjectionAction(const std::string & node_name, - ScopeStack & scopes, - ProjectionManipulatorPtr projection_manipulator, - const std::string & expression_name, - const Context & context) -{ - if (typeid_cast(projection_manipulator.get()) && node_name == "and") - { - return std::make_shared(scopes, projection_manipulator, expression_name, context); - } - else - { - return std::make_shared(); - } -} - -} diff --git a/dbms/src/Interpreters/ProjectionManipulation.h b/dbms/src/Interpreters/ProjectionManipulation.h deleted file mode 100644 index b5529b5ee5c..00000000000 --- a/dbms/src/Interpreters/ProjectionManipulation.h +++ /dev/null @@ -1,256 +0,0 @@ -#pragma once - -#include -#include -#include -#include - -namespace DB -{ - -class ExpressionAnalyzer; -class Context; -struct ScopeStack; - - -namespace ErrorCodes -{ - extern const int CONDITIONAL_TREE_PARENT_NOT_FOUND; - extern const int ILLEGAL_PROJECTION_MANIPULATOR; -} - -/* - * This is a base class for the ConditionalTree. Look at the description of ConditionalTree. - */ -struct ProjectionManipulatorBase -{ -public: - virtual bool tryToGetFromUpperProjection(const std::string & column_name) = 0; - - virtual std::string getColumnName(const std::string & col_name) const = 0; - - virtual std::string getProjectionExpression() = 0; - - virtual std::string getProjectionSourceColumn() const = 0; - - virtual ~ProjectionManipulatorBase(); -}; - -using ProjectionManipulatorPtr = std::shared_ptr; - -/* - * This is the default ProjectionManipulator. It is needed for backwards compatibility. - * For the better understanding of what ProjectionManipulator does, - * look at the description of ConditionalTree. - */ -struct DefaultProjectionManipulator : public ProjectionManipulatorBase -{ -private: - ScopeStack & scopes; - -public: - explicit DefaultProjectionManipulator(ScopeStack & scopes); - - bool tryToGetFromUpperProjection(const std::string & column_name) final; - - std::string getColumnName(const std::string & col_name) const final; - - std::string getProjectionExpression() final; - - std::string getProjectionSourceColumn() const final; -}; - -/* - * ConditionalTree is a projection manipulator. It is used in ExpressionAnalyzer::getActionsImpl. - * It is a helper class, which helps to build sequence of ExpressionAction instances -- actions, needed for - * computation of expression. It represents the current state of a projection layer. That is, if we have an expression - * f and g, we need to calculate f, afterwards we need to calculate g on the projection layer . - * This projection layer is stored in the ConditionalTree. Also, it stores the tree of all projection layers, which - * was seen before. If we have seen the projection layer and , conditional tree will put - * the second layer as a child to the first one. - * - * The description of what methods do: - * 1) getColumnName -- constructs the name of expression. which contains the information of the projection layer. - * It is needed to make computed column name unique. That is, if we have an expression g and conditional layer - * , it forms the name g - * - * 2) goToProjection -- accepts field name f and builds child projection layer with the additional condition - * . For instance, if we are on the projection layer a != 0 and the function accepts the expression b != 0, - * it will build a projection layer , and remember that this layer is a child to a previous one. - * Moreover, the function will store the actions to build projection between this two layers in the corresponding - * ScopeStack - * - * 3) restoreColumn(default_values_name, new_values_name, levels, result_name) -- stores action to restore calculated - * 'new_values_name' column, to insert its values to the projection layer, which is 'levels' number of levels higher. - * - * 4) goUp -- goes several levels up in the conditional tree, raises the exception if we hit the root of the tree and - * there are still remained some levels up to go. - * - * 5) tryToGetFromUpperProjection -- goes up to the root projection level and checks whether the expression is - * already calculated somewhere in the higher projection level. If it is, we may just project it to the current - * layer to have it computed in the current layer. In this case, the function stores all actions needed to compute - * the projection: computes composition of projections and uses it to project the column. In the other case, if - * the column is not computed on the higher level, the function returns false. It is used in getActinosImpl to - * understand whether we need to scan the expression deeply, or can it be easily computed just with the projection - * from one of the higher projection layers. - */ -struct ConditionalTree : public ProjectionManipulatorBase -{ -private: - struct Node - { - Node(); - - size_t getParentNode() const; - - std::string projection_expression_string; - size_t parent_node; - bool is_root; - }; - - size_t current_node; - std::vector nodes; - ScopeStack & scopes; - const Context & context; - std::unordered_map projection_expression_index; - -private: - std::string getColumnNameByIndex(const std::string & col_name, size_t node) const; - - std::string getProjectionColumnName(const std::string & first_projection_expr, const std::string & second_projection_expr) const; - - std::string getProjectionColumnName(size_t first_index, size_t second_index) const; - - void buildProjectionCompositionRecursive(const std::vector & path, size_t child_index, size_t parent_index); - - void buildProjectionComposition(size_t child_node, size_t parent_node); - - std::string getProjectionSourceColumn(size_t node) const; - -public: - ConditionalTree(ScopeStack & scopes, const Context & context); - - std::string getColumnName(const std::string & col_name) const final; - - void goToProjection(const std::string & field_name); - - std::string buildRestoreProjectionAndGetName(size_t levels_up); - - void restoreColumn( - const std::string & default_values_name, const std::string & new_values_name, size_t levels_up, const std::string & result_name); - - void goUp(size_t levels_up); - - bool tryToGetFromUpperProjection(const std::string & column_name) final; - - std::string getProjectionExpression() final; - - std::string getProjectionSourceColumn() const final; -}; - -using ConditionalTreePtr = std::shared_ptr; - -/* - * ProjectionAction describes in what way should some specific function use the projection manipulator. - * This class has two inherited classes: DefaultProjectionAction, which does nothing, and AndOperatorProjectionAction, - * which represents how function "and" uses projection manipulator. - */ -class ProjectionActionBase -{ -public: - /* - * What to do before scanning the function argument (each of it) - */ - virtual void preArgumentAction() = 0; - - /* - * What to do after scanning each argument - */ - virtual void postArgumentAction(const std::string & argument_name) = 0; - - /* - * What to do after scanning all the arguments, before the computation - */ - virtual void preCalculation() = 0; - - /* - * Should default computation procedure be run or not - */ - virtual bool isCalculationRequired() = 0; - - virtual ~ProjectionActionBase(); -}; - -using ProjectionActionPtr = std::shared_ptr; - -class DefaultProjectionAction : public ProjectionActionBase -{ -public: - void preArgumentAction() final; - - void postArgumentAction(const std::string & argument_name) final; - - void preCalculation() final; - - bool isCalculationRequired() final; -}; - -/* - * This is a specification of ProjectionAction specifically for the 'and' operation - */ -class AndOperatorProjectionAction : public ProjectionActionBase -{ -private: - ScopeStack & scopes; - ProjectionManipulatorPtr projection_manipulator; - std::string previous_argument_name; - size_t projection_levels_count; - std::string expression_name; - const Context & context; - - std::string getZerosColumnName(); - - std::string getFinalColumnName(); - - void createZerosColumn(const std::string & restore_projection_name); - -public: - AndOperatorProjectionAction( - ScopeStack & scopes, ProjectionManipulatorPtr projection_manipulator, const std::string & expression_name, const Context & context); - - /* - * Before scanning each argument, we should go to the next projection layer. For example, if the expression is - * f and g and h, then before computing g we should project to and before computing h we should project to - * - */ - void preArgumentAction() final; - - /* - * Stores the previous argument name - */ - void postArgumentAction(const std::string & argument_name) final; - - /* - * Restores the result column to the uppermost projection level. For example, if the expression is f and g and h, - * we should restore h to the main projection layer - */ - void preCalculation() final; - - /* - * After what is done in preCalculation, we do not need to run default calculation of 'and' operator. So, the - * function returns false. - */ - bool isCalculationRequired() final; -}; - -/* - * This function accepts the operator name and returns its projection action. For example, for 'and' operator, - * it returns the pointer to AndOperatorProjectionAction. - */ -ProjectionActionPtr getProjectionAction(const std::string & node_name, - ScopeStack & scopes, - ProjectionManipulatorPtr projection_manipulator, - const std::string & expression_name, - const Context & context); - -} diff --git a/dbms/src/Interpreters/Settings.h b/dbms/src/Interpreters/Settings.h index c1436267ffc..77ba93ec174 100644 --- a/dbms/src/Interpreters/Settings.h +++ b/dbms/src/Interpreters/Settings.h @@ -272,7 +272,6 @@ struct Settings M(SettingBool, format_csv_allow_single_quotes, 1, "If it is set to true, allow strings in single quotes.") \ M(SettingBool, format_csv_allow_double_quotes, 1, "If it is set to true, allow strings in double quotes.") \ \ - M(SettingUInt64, enable_conditional_computation, 0, "Enable conditional computations") \ M(SettingDateTimeInputFormat, date_time_input_format, FormatSettings::DateTimeInputFormat::Basic, "Method to read DateTime from text input formats. Possible values: 'basic' and 'best_effort'.") \ M(SettingBool, log_profile_events, true, "Log query performance statistics into the query_log and query_thread_log.") \ M(SettingBool, log_query_settings, true, "Log query settings into the query_log.") \ diff --git a/dbms/src/Storages/MergeTree/MergeTreeData.cpp b/dbms/src/Storages/MergeTree/MergeTreeData.cpp index 9a87728047e..b60e4bed26f 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeData.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeData.cpp @@ -1021,7 +1021,7 @@ void MergeTreeData::createConvertExpression(const DataPartPtr & part, const Name /// This is temporary name for expression. TODO Invent the name more safely. const String new_type_name_column = '#' + new_type_name + "_column"; out_expression->add(ExpressionAction::addColumn( - { DataTypeString().createColumnConst(1, new_type_name), std::make_shared(), new_type_name_column }, "", false)); + { DataTypeString().createColumnConst(1, new_type_name), std::make_shared(), new_type_name_column })); const auto & function = FunctionFactory::instance().get("CAST", context); out_expression->add(ExpressionAction::applyFunction( diff --git a/dbms/tests/queries/0_stateless/00631_conditional_computation.reference b/dbms/tests/queries/0_stateless/00631_conditional_computation.reference deleted file mode 100644 index ea763551e39..00000000000 --- a/dbms/tests/queries/0_stateless/00631_conditional_computation.reference +++ /dev/null @@ -1,4 +0,0 @@ -0 0 0 0 0 0 0 -0 5 0 0 0 0 0 -6 3 1 1 1 1 0 -7 0 0 0 0 0 0 diff --git a/dbms/tests/queries/0_stateless/00631_conditional_computation.sql b/dbms/tests/queries/0_stateless/00631_conditional_computation.sql deleted file mode 100644 index 70438c766ed..00000000000 --- a/dbms/tests/queries/0_stateless/00631_conditional_computation.sql +++ /dev/null @@ -1,23 +0,0 @@ -USE test; - -DROP TABLE IF EXISTS test; -CREATE TABLE test (d Date DEFAULT '2000-01-01', x UInt64, y UInt64) ENGINE = MergeTree(d, x, 1); -INSERT INTO test(x,y) VALUES (6, 3); -INSERT INTO test(x,y) VALUES (0, 5); -INSERT INTO test(x,y) VALUES (7, 0); -INSERT INTO test(x,y) VALUES (0, 0); -SET enable_conditional_computation=1; -SELECT - x, - y, - x and y, - y and x, - x and 1 and x and y, - x and modulo(y, x), - y and modulo(x,y) -FROM - test -ORDER BY - x, y -; -SET enable_conditional_computation=0; diff --git a/dbms/tests/queries/bugs/fuzzy.sql b/dbms/tests/queries/bugs/fuzzy.sql index 5712498a555..f81140ba8c9 100644 --- a/dbms/tests/queries/bugs/fuzzy.sql +++ b/dbms/tests/queries/bugs/fuzzy.sql @@ -1,6 +1,3 @@ -SELECT __inner_restore_projection__(2.0885, -66.72488); -SELECT __inner_restore_projection__(-4, ''); -SELECT __inner_restore_projection__(067274, 'vb\s'); SELECT sequenceCount((CAST((( SELECT NULL ) AS rg, ( SELECT ( SELECT [], ' Date: Mon, 26 Nov 2018 13:23:03 +0300 Subject: [PATCH 115/124] Add option to disable host net --- dbms/tests/integration/runner | 14 +++++++++++++- 1 file changed, 13 insertions(+), 1 deletion(-) diff --git a/dbms/tests/integration/runner b/dbms/tests/integration/runner index ab1ca8003af..99d90102868 100755 --- a/dbms/tests/integration/runner +++ b/dbms/tests/integration/runner @@ -27,12 +27,24 @@ if __name__ == "__main__": default=DEFAULT_CLICKHOUSE_ROOT, help="Path to repository root folder" ) + parser.add_argument( + "--disable-net-host", + action='store_true', + default=False, + help="Don't use net host in parent docker container" + ) + parser.add_argument('pytest_args', nargs='*', help="args for pytest command") args = parser.parse_args() - cmd = "docker run --net=host --privileged --volume={bin}:/clickhouse \ + net = "" + if not args.disable_net_host: + net = "--net=host" + + cmd = "docker run {net} --privileged --volume={bin}:/clickhouse \ --volume={cfg}:/clickhouse-config --volume={pth}:/ClickHouse -e PYTEST_OPTS='{opts}' {img}".format( + net=net, bin=args.binary, cfg=args.configs_dir, pth=args.clickhouse_root, From 757d808a57d6725fc391288c5b128dc82df4c66f Mon Sep 17 00:00:00 2001 From: alesapin Date: Mon, 26 Nov 2018 14:04:42 +0300 Subject: [PATCH 116/124] Add env variable for tests skip --- docker/test/stateless/Dockerfile | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docker/test/stateless/Dockerfile b/docker/test/stateless/Dockerfile index 9cd0a2ba970..4bdad6aa02c 100644 --- a/docker/test/stateless/Dockerfile +++ b/docker/test/stateless/Dockerfile @@ -31,4 +31,4 @@ CMD dpkg -i package_folder/clickhouse-common-static_*.deb; \ dpkg -i package_folder/clickhouse-client_*.deb; \ dpkg -i package_folder/clickhouse-test_*.deb; \ service zookeeper start; sleep 5; \ - service clickhouse-server start && sleep 5 && clickhouse-test --shard --zookeeper 2>&1 | tee test_output/test_result.txt + service clickhouse-server start && sleep 5 && clickhouse-test --shard --zookeeper $SKIP_TESTS_OPTION 2>&1 | tee test_output/test_result.txt From 3a3dda5ec508af5127c6686a8812acc778ccd5ae Mon Sep 17 00:00:00 2001 From: Alexey Zatelepin Date: Mon, 26 Nov 2018 15:41:17 +0300 Subject: [PATCH 117/124] use standard Tuple serialization [#CLICKHOUSE-4054] --- .../Storages/MergeTree/MergeTreePartition.cpp | 18 +++++++++++------- .../00502_custom_partitioning_local.reference | 16 ++++++++-------- ...partitioning_replicated_zookeeper.reference | 16 ++++++++-------- 3 files changed, 27 insertions(+), 23 deletions(-) diff --git a/dbms/src/Storages/MergeTree/MergeTreePartition.cpp b/dbms/src/Storages/MergeTree/MergeTreePartition.cpp index d4aad4230b0..b13c9fb6740 100644 --- a/dbms/src/Storages/MergeTree/MergeTreePartition.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreePartition.cpp @@ -5,6 +5,8 @@ #include #include #include +#include +#include #include #include #include @@ -94,18 +96,20 @@ void MergeTreePartition::serializeText(const MergeTreeData & storage, WriteBuffe } else { - writeChar('(', out); + DataTypes types; + Columns columns; for (size_t i = 0; i < key_size; ++i) { - if (i > 0) - writeCString(", ", out); - - const DataTypePtr & type = storage.partition_key_sample.getByPosition(i).type; + const auto & type = storage.partition_key_sample.getByPosition(i).type; + types.push_back(type); auto column = type->createColumn(); column->insert(value[i]); - type->serializeTextQuoted(*column, 0, out, format_settings); + columns.push_back(std::move(column)); } - writeChar(')', out); + + DataTypeTuple tuple_type(types); + auto tuple_column = ColumnTuple::create(columns); + tuple_type.serializeText(*tuple_column, 0, out, format_settings); } } diff --git a/dbms/tests/queries/0_stateless/00502_custom_partitioning_local.reference b/dbms/tests/queries/0_stateless/00502_custom_partitioning_local.reference index b8370d9e27a..71f0102c20b 100644 --- a/dbms/tests/queries/0_stateless/00502_custom_partitioning_local.reference +++ b/dbms/tests/queries/0_stateless/00502_custom_partitioning_local.reference @@ -22,15 +22,15 @@ Sum after DROP PARTITION: 12 *** Partitioned by a (Date, UInt8) tuple *** Parts before OPTIMIZE: -(\'2000-01-01\', 1) 20000101-1_1_1_0 -(\'2000-01-01\', 1) 20000101-1_5_5_0 -(\'2000-01-01\', 2) 20000101-2_2_2_0 -(\'2000-01-02\', 1) 20000102-1_3_3_0 -(\'2000-01-02\', 1) 20000102-1_4_4_0 +(\'2000-01-01\',1) 20000101-1_1_1_0 +(\'2000-01-01\',1) 20000101-1_5_5_0 +(\'2000-01-01\',2) 20000101-2_2_2_0 +(\'2000-01-02\',1) 20000102-1_3_3_0 +(\'2000-01-02\',1) 20000102-1_4_4_0 Parts after OPTIMIZE: -(\'2000-01-01\', 1) 20000101-1_1_5_1 -(\'2000-01-01\', 2) 20000101-2_2_2_0 -(\'2000-01-02\', 1) 20000102-1_3_4_1 +(\'2000-01-01\',1) 20000101-1_1_5_1 +(\'2000-01-01\',2) 20000101-2_2_2_0 +(\'2000-01-02\',1) 20000102-1_3_4_1 Sum before DETACH PARTITION: 15 Sum after DETACH PARTITION: diff --git a/dbms/tests/queries/0_stateless/00502_custom_partitioning_replicated_zookeeper.reference b/dbms/tests/queries/0_stateless/00502_custom_partitioning_replicated_zookeeper.reference index 8b6d0ed1384..689fd252a21 100644 --- a/dbms/tests/queries/0_stateless/00502_custom_partitioning_replicated_zookeeper.reference +++ b/dbms/tests/queries/0_stateless/00502_custom_partitioning_replicated_zookeeper.reference @@ -22,15 +22,15 @@ Sum after DROP PARTITION: 12 *** Partitioned by a (Date, UInt8) tuple *** Parts before OPTIMIZE: -(\'2000-01-01\', 1) 20000101-1_0_0_0 -(\'2000-01-01\', 1) 20000101-1_1_1_0 -(\'2000-01-01\', 2) 20000101-2_0_0_0 -(\'2000-01-02\', 1) 20000102-1_0_0_0 -(\'2000-01-02\', 1) 20000102-1_1_1_0 +(\'2000-01-01\',1) 20000101-1_0_0_0 +(\'2000-01-01\',1) 20000101-1_1_1_0 +(\'2000-01-01\',2) 20000101-2_0_0_0 +(\'2000-01-02\',1) 20000102-1_0_0_0 +(\'2000-01-02\',1) 20000102-1_1_1_0 Parts after OPTIMIZE: -(\'2000-01-01\', 1) 20000101-1_0_1_1 -(\'2000-01-01\', 2) 20000101-2_0_0_0 -(\'2000-01-02\', 1) 20000102-1_0_1_1 +(\'2000-01-01\',1) 20000101-1_0_1_1 +(\'2000-01-01\',2) 20000101-2_0_0_0 +(\'2000-01-02\',1) 20000102-1_0_1_1 Sum before DETACH PARTITION: 15 Sum after DETACH PARTITION: From e793a27cc341f30678c135b8c3cc71306564f1fe Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 26 Nov 2018 16:57:54 +0300 Subject: [PATCH 118/124] Restored old test util #3665 --- dbms/src/Client/CMakeLists.txt | 2 + dbms/src/Client/tests/CMakeLists.txt | 2 + dbms/src/Client/tests/test_connect.cpp | 59 ++++++++++++++++++++++++++ 3 files changed, 63 insertions(+) create mode 100644 dbms/src/Client/tests/CMakeLists.txt create mode 100644 dbms/src/Client/tests/test_connect.cpp diff --git a/dbms/src/Client/CMakeLists.txt b/dbms/src/Client/CMakeLists.txt index 907d9787bf8..cc8b3f8eda8 100644 --- a/dbms/src/Client/CMakeLists.txt +++ b/dbms/src/Client/CMakeLists.txt @@ -4,3 +4,5 @@ #add_library(clickhouse_client ${LINK_MODE} ${clickhouse_client_headers} ${clickhouse_client_sources}) #target_link_libraries (clickhouse_client clickhouse_common_io ${Poco_Net_LIBRARY}) #target_include_directories (clickhouse_client PRIVATE ${DBMS_INCLUDE_DIR}) + +add_subdirectory(tests) diff --git a/dbms/src/Client/tests/CMakeLists.txt b/dbms/src/Client/tests/CMakeLists.txt new file mode 100644 index 00000000000..f4471136a8a --- /dev/null +++ b/dbms/src/Client/tests/CMakeLists.txt @@ -0,0 +1,2 @@ +add_executable(test-connect test_connect.cpp) +target_link_libraries (test-connect dbms) diff --git a/dbms/src/Client/tests/test_connect.cpp b/dbms/src/Client/tests/test_connect.cpp new file mode 100644 index 00000000000..75eb606cc97 --- /dev/null +++ b/dbms/src/Client/tests/test_connect.cpp @@ -0,0 +1,59 @@ +#include +#include +#include + +#include +#include +#include +#include + + +/** In a loop it connects to the server and immediately breaks the connection. +  * Using the SO_LINGER option, we ensure that the connection is terminated by sending a RST packet (not FIN). +  * This behavior causes a bug in the TCPServer implementation in the Poco library. +  */ +int main(int argc, char ** argv) +try +{ + for (size_t i = 0, num_iters = argc >= 2 ? DB::parse(argv[1]) : 1; i < num_iters; ++i) + { + std::cerr << "."; + + Poco::Net::SocketAddress address("localhost", 9000); + + int fd = socket(PF_INET, SOCK_STREAM, IPPROTO_IP); + + if (fd < 0) + DB::throwFromErrno("Cannot create socket", 0); + + linger linger_value; + linger_value.l_onoff = 1; + linger_value.l_linger = 0; + + if (0 != setsockopt(fd, SOL_SOCKET, SO_LINGER, &linger_value, sizeof(linger_value))) + DB::throwFromErrno("Cannot set linger", 0); + + try + { + int res = connect(fd, address.addr(), address.length()); + + if (res != 0 && errno != EINPROGRESS && errno != EWOULDBLOCK) + { + close(fd); + DB::throwFromErrno("Cannot connect", 0); + } + + close(fd); + } + catch (const Poco::Exception & e) + { + std::cerr << e.displayText() << "\n"; + } + } + + std::cerr << "\n"; +} +catch (const Poco::Exception & e) +{ + std::cerr << e.displayText() << "\n"; +} From 73aa7f2027fe9a41101efc315224052d9d33e521 Mon Sep 17 00:00:00 2001 From: Ivan Lezhankin Date: Mon, 26 Nov 2018 17:43:40 +0300 Subject: [PATCH 119/124] =?UTF-8?q?Rename=20method=20partition()=20?= =?UTF-8?q?=E2=86=92=20alterPartition()?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Also fix build with internal shared libraries. --- dbms/programs/CMakeLists.txt | 2 +- dbms/src/Interpreters/InterpreterAlterQuery.cpp | 2 +- dbms/src/Storages/IStorage.h | 2 +- dbms/src/Storages/StorageMaterializedView.cpp | 4 ++-- dbms/src/Storages/StorageMaterializedView.h | 2 +- dbms/src/Storages/StorageMergeTree.cpp | 4 ++-- dbms/src/Storages/StorageMergeTree.h | 2 +- dbms/src/Storages/StorageReplicatedMergeTree.cpp | 4 ++-- dbms/src/Storages/StorageReplicatedMergeTree.h | 2 +- 9 files changed, 12 insertions(+), 12 deletions(-) diff --git a/dbms/programs/CMakeLists.txt b/dbms/programs/CMakeLists.txt index 441b39d9966..9d7c6f2cda1 100644 --- a/dbms/programs/CMakeLists.txt +++ b/dbms/programs/CMakeLists.txt @@ -48,7 +48,7 @@ else () link_directories (${LLVM_LIBRARY_DIRS}) endif () add_executable (clickhouse main.cpp) - target_link_libraries (clickhouse PRIVATE clickhouse_common_io) + target_link_libraries (clickhouse PRIVATE clickhouse_common_io string_utils) target_include_directories (clickhouse BEFORE PRIVATE ${COMMON_INCLUDE_DIR}) target_include_directories (clickhouse PRIVATE ${CMAKE_CURRENT_BINARY_DIR}) diff --git a/dbms/src/Interpreters/InterpreterAlterQuery.cpp b/dbms/src/Interpreters/InterpreterAlterQuery.cpp index c3bab56edcf..ab24d7e1164 100644 --- a/dbms/src/Interpreters/InterpreterAlterQuery.cpp +++ b/dbms/src/Interpreters/InterpreterAlterQuery.cpp @@ -57,7 +57,7 @@ BlockIO InterpreterAlterQuery::execute() if (!partition_commands.empty()) { partition_commands.validate(*table); - table->partition(query_ptr, partition_commands, context); + table->alterPartition(query_ptr, partition_commands, context); } if (!alter_commands.empty()) diff --git a/dbms/src/Storages/IStorage.h b/dbms/src/Storages/IStorage.h index 65d40d82c9a..81249446746 100644 --- a/dbms/src/Storages/IStorage.h +++ b/dbms/src/Storages/IStorage.h @@ -256,7 +256,7 @@ public: /** ALTER tables with regard to its partitions. * Should handle locks for each command on its own. */ - virtual void partition(const ASTPtr & /* query */, const PartitionCommands & /* commands */, const Context & /* context */) + virtual void alterPartition(const ASTPtr & /* query */, const PartitionCommands & /* commands */, const Context & /* context */) { throw Exception("Partition operations are not supported by storage " + getName(), ErrorCodes::NOT_IMPLEMENTED); } diff --git a/dbms/src/Storages/StorageMaterializedView.cpp b/dbms/src/Storages/StorageMaterializedView.cpp index e1af2576873..f69efad04ae 100644 --- a/dbms/src/Storages/StorageMaterializedView.cpp +++ b/dbms/src/Storages/StorageMaterializedView.cpp @@ -240,10 +240,10 @@ bool StorageMaterializedView::optimize(const ASTPtr & query, const ASTPtr & part return getTargetTable()->optimize(query, partition, final, deduplicate, context); } -void StorageMaterializedView::partition(const ASTPtr & query, const PartitionCommands &commands, const Context &context) +void StorageMaterializedView::alterPartition(const ASTPtr & query, const PartitionCommands &commands, const Context &context) { checkStatementCanBeForwarded(); - getTargetTable()->partition(query, commands, context); + getTargetTable()->alterPartition(query, commands, context); } void StorageMaterializedView::mutate(const MutationCommands & commands, const Context & context) diff --git a/dbms/src/Storages/StorageMaterializedView.h b/dbms/src/Storages/StorageMaterializedView.h index d1f463151e6..176a95c2d03 100644 --- a/dbms/src/Storages/StorageMaterializedView.h +++ b/dbms/src/Storages/StorageMaterializedView.h @@ -35,7 +35,7 @@ public: bool optimize(const ASTPtr & query, const ASTPtr & partition, bool final, bool deduplicate, const Context & context) override; - void partition(const ASTPtr & query, const PartitionCommands & commands, const Context & context) override; + void alterPartition(const ASTPtr & query, const PartitionCommands & commands, const Context & context) override; void mutate(const MutationCommands & commands, const Context & context) override; diff --git a/dbms/src/Storages/StorageMergeTree.cpp b/dbms/src/Storages/StorageMergeTree.cpp index 7ed871e23c7..50d7af076d2 100644 --- a/dbms/src/Storages/StorageMergeTree.cpp +++ b/dbms/src/Storages/StorageMergeTree.cpp @@ -781,7 +781,7 @@ bool StorageMergeTree::optimize( return true; } -void StorageMergeTree::partition(const ASTPtr & query, const PartitionCommands & commands, const Context & context) +void StorageMergeTree::alterPartition(const ASTPtr & query, const PartitionCommands & commands, const Context & context) { for (const PartitionCommand & command : commands) { @@ -824,7 +824,7 @@ void StorageMergeTree::partition(const ASTPtr & query, const PartitionCommands & break; default: - IStorage::partition(query, commands, context); // should throw an exception. + IStorage::alterPartition(query, commands, context); // should throw an exception. } } } diff --git a/dbms/src/Storages/StorageMergeTree.h b/dbms/src/Storages/StorageMergeTree.h index 04856835fcf..d5e4616d920 100644 --- a/dbms/src/Storages/StorageMergeTree.h +++ b/dbms/src/Storages/StorageMergeTree.h @@ -60,7 +60,7 @@ public: */ bool optimize(const ASTPtr & query, const ASTPtr & partition, bool final, bool deduplicate, const Context & context) override; - void partition(const ASTPtr & query, const PartitionCommands & commands, const Context & context) override; + void alterPartition(const ASTPtr & query, const PartitionCommands & commands, const Context & context) override; void mutate(const MutationCommands & commands, const Context & context) override; diff --git a/dbms/src/Storages/StorageReplicatedMergeTree.cpp b/dbms/src/Storages/StorageReplicatedMergeTree.cpp index abb2e46e584..7fe4a72c0eb 100644 --- a/dbms/src/Storages/StorageReplicatedMergeTree.cpp +++ b/dbms/src/Storages/StorageReplicatedMergeTree.cpp @@ -3282,7 +3282,7 @@ void StorageReplicatedMergeTree::alter(const AlterCommands & params, LOG_DEBUG(log, "ALTER finished"); } -void StorageReplicatedMergeTree::partition(const ASTPtr & query, const PartitionCommands & commands, const Context & context) +void StorageReplicatedMergeTree::alterPartition(const ASTPtr & query, const PartitionCommands & commands, const Context & context) { for (const PartitionCommand & command : commands) { @@ -3329,7 +3329,7 @@ void StorageReplicatedMergeTree::partition(const ASTPtr & query, const Partition break; default: - IStorage::partition(query, commands, context); // should throw an exception. + IStorage::alterPartition(query, commands, context); // should throw an exception. } } } diff --git a/dbms/src/Storages/StorageReplicatedMergeTree.h b/dbms/src/Storages/StorageReplicatedMergeTree.h index 9bc6d4f08f1..236ca204daf 100644 --- a/dbms/src/Storages/StorageReplicatedMergeTree.h +++ b/dbms/src/Storages/StorageReplicatedMergeTree.h @@ -116,7 +116,7 @@ public: void alter(const AlterCommands & params, const String & database_name, const String & table_name, const Context & context) override; - void partition(const ASTPtr & query, const PartitionCommands & commands, const Context & context) override; + void alterPartition(const ASTPtr & query, const PartitionCommands & commands, const Context & context) override; void mutate(const MutationCommands & commands, const Context & context) override; From 5a6f9a5ff948f9d033247682ee1d358b627e4db3 Mon Sep 17 00:00:00 2001 From: alesapin Date: Mon, 26 Nov 2018 18:18:45 +0300 Subject: [PATCH 120/124] Replace function mysql with function file --- dbms/tests/queries/0_stateless/00758_array_reverse.sql | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/tests/queries/0_stateless/00758_array_reverse.sql b/dbms/tests/queries/0_stateless/00758_array_reverse.sql index 1c6afe0995b..8a2417e68b0 100644 --- a/dbms/tests/queries/0_stateless/00758_array_reverse.sql +++ b/dbms/tests/queries/0_stateless/00758_array_reverse.sql @@ -12,4 +12,4 @@ SELECT reverse([]); SELECT reverse([[[[]]]]); SET send_logs_level = 'none'; -SELECT '[RE7', ( SELECT '\0' ) AS riwwq, ( SELECT reverse([( SELECT bitTestAll(NULL) ) , ( SELECT '\0' ) AS ddfweeuy]) ) AS xuvv, '', ( SELECT * FROM mysql() ) AS wqgdswyc, ( SELECT * FROM mysql() ); -- { serverError 42 } +SELECT '[RE7', ( SELECT '\0' ) AS riwwq, ( SELECT reverse([( SELECT bitTestAll(NULL) ) , ( SELECT '\0' ) AS ddfweeuy]) ) AS xuvv, '', ( SELECT * FROM file() ) AS wqgdswyc, ( SELECT * FROM file() ); -- { serverError 42 } From e00ce0bb57d1cd2ba70e86ebd0dbae77d09dec80 Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Mon, 26 Nov 2018 19:20:40 +0300 Subject: [PATCH 121/124] Every function in its own file, part 7 (#3666) * Every function in its own file, part 7 [#CLICKHOUSE-2] * Every function in its own file, part 7 [#CLICKHOUSE-2] * Every function in its own file, part 7 [#CLICKHOUSE-2] * Every function in its own file, part 7 [#CLICKHOUSE-2] * Every function in its own file, part 7 [#CLICKHOUSE-2] * Fixed build #3666 --- cmake/dbms_generate_function.cmake | 10 - dbms/src/Functions/CMakeLists.txt | 35 - dbms/src/Functions/FunctionBinaryArithmetic.h | 921 +++++++ dbms/src/Functions/FunctionBitTestMany.h | 204 ++ dbms/src/Functions/FunctionUnaryArithmetic.h | 220 ++ dbms/src/Functions/FunctionsArithmetic.h | 2176 ----------------- dbms/src/Functions/FunctionsRound.cpp | 4 - dbms/src/Functions/FunctionsRound.h | 139 +- dbms/src/Functions/abs.cpp | 54 + dbms/src/Functions/bitAnd.cpp | 38 + dbms/src/Functions/bitNot.cpp | 47 + dbms/src/Functions/bitOr.cpp | 38 + dbms/src/Functions/bitRotateLeft.cpp | 41 + dbms/src/Functions/bitRotateRight.cpp | 40 + dbms/src/Functions/bitShiftLeft.cpp | 38 + dbms/src/Functions/bitShiftRight.cpp | 38 + dbms/src/Functions/bitTest.cpp | 31 + dbms/src/Functions/bitTestAll.cpp | 21 + dbms/src/Functions/bitTestAny.cpp | 21 + dbms/src/Functions/bitXor.cpp | 38 + dbms/src/Functions/castTypeToEither.h | 17 + dbms/src/Functions/divide.cpp | 39 + dbms/src/Functions/gcd.cpp | 36 + dbms/src/Functions/greatest.cpp | 61 + dbms/src/Functions/intDiv.cpp | 104 + dbms/src/Functions/intDiv.h | 66 + dbms/src/Functions/intDivOrZero.cpp | 31 + dbms/src/Functions/intExp10.cpp | 48 + dbms/src/Functions/intExp2.cpp | 55 + dbms/src/Functions/lcm.cpp | 36 + dbms/src/Functions/least.cpp | 61 + dbms/src/Functions/minus.cpp | 45 + dbms/src/Functions/modulo.cpp | 102 + dbms/src/Functions/multiply.cpp | 45 + dbms/src/Functions/negate.cpp | 45 + dbms/src/Functions/plus.cpp | 46 + .../Functions/registerFunctionsArithmetic.cpp | 69 + dbms/src/Functions/roundAge.cpp | 38 + dbms/src/Functions/roundDuration.cpp | 47 + dbms/src/Functions/roundToExp2.cpp | 73 + 40 files changed, 2858 insertions(+), 2360 deletions(-) delete mode 100644 cmake/dbms_generate_function.cmake create mode 100644 dbms/src/Functions/FunctionBinaryArithmetic.h create mode 100644 dbms/src/Functions/FunctionBitTestMany.h create mode 100644 dbms/src/Functions/FunctionUnaryArithmetic.h delete mode 100644 dbms/src/Functions/FunctionsArithmetic.h create mode 100644 dbms/src/Functions/abs.cpp create mode 100644 dbms/src/Functions/bitAnd.cpp create mode 100644 dbms/src/Functions/bitNot.cpp create mode 100644 dbms/src/Functions/bitOr.cpp create mode 100644 dbms/src/Functions/bitRotateLeft.cpp create mode 100644 dbms/src/Functions/bitRotateRight.cpp create mode 100644 dbms/src/Functions/bitShiftLeft.cpp create mode 100644 dbms/src/Functions/bitShiftRight.cpp create mode 100644 dbms/src/Functions/bitTest.cpp create mode 100644 dbms/src/Functions/bitTestAll.cpp create mode 100644 dbms/src/Functions/bitTestAny.cpp create mode 100644 dbms/src/Functions/bitXor.cpp create mode 100644 dbms/src/Functions/castTypeToEither.h create mode 100644 dbms/src/Functions/divide.cpp create mode 100644 dbms/src/Functions/gcd.cpp create mode 100644 dbms/src/Functions/greatest.cpp create mode 100644 dbms/src/Functions/intDiv.cpp create mode 100644 dbms/src/Functions/intDiv.h create mode 100644 dbms/src/Functions/intDivOrZero.cpp create mode 100644 dbms/src/Functions/intExp10.cpp create mode 100644 dbms/src/Functions/intExp2.cpp create mode 100644 dbms/src/Functions/lcm.cpp create mode 100644 dbms/src/Functions/least.cpp create mode 100644 dbms/src/Functions/minus.cpp create mode 100644 dbms/src/Functions/modulo.cpp create mode 100644 dbms/src/Functions/multiply.cpp create mode 100644 dbms/src/Functions/negate.cpp create mode 100644 dbms/src/Functions/plus.cpp create mode 100644 dbms/src/Functions/registerFunctionsArithmetic.cpp create mode 100644 dbms/src/Functions/roundAge.cpp create mode 100644 dbms/src/Functions/roundDuration.cpp create mode 100644 dbms/src/Functions/roundToExp2.cpp diff --git a/cmake/dbms_generate_function.cmake b/cmake/dbms_generate_function.cmake deleted file mode 100644 index ae701f798fe..00000000000 --- a/cmake/dbms_generate_function.cmake +++ /dev/null @@ -1,10 +0,0 @@ -function(generate_function_register FUNCTION_AREA) - foreach(FUNCTION IN LISTS ARGN) - configure_file (registerFunction.h.in ${FUNCTIONS_GENERATED_DIR}register${FUNCTION}.h) - configure_file (registerFunction.cpp.in ${FUNCTIONS_GENERATED_DIR}register${FUNCTION}.cpp) - set(REGISTER_HEADERS "${REGISTER_HEADERS}#include \"register${FUNCTION}.h\"\n") - set(REGISTER_FUNCTIONS "${REGISTER_FUNCTIONS} register${FUNCTION}(factory);\n") - endforeach() - - configure_file (registerFunctions_area.cpp.in ${FUNCTIONS_GENERATED_DIR}registerFunctions${FUNCTION_AREA}.cpp) -endfunction() diff --git a/dbms/src/Functions/CMakeLists.txt b/dbms/src/Functions/CMakeLists.txt index a6109c70934..1a1d6f06f64 100644 --- a/dbms/src/Functions/CMakeLists.txt +++ b/dbms/src/Functions/CMakeLists.txt @@ -1,43 +1,8 @@ include(${ClickHouse_SOURCE_DIR}/cmake/dbms_glob_sources.cmake) -include(${ClickHouse_SOURCE_DIR}/cmake/dbms_generate_function.cmake) - -set (FUNCTIONS_GENERATED_DIR ${CMAKE_CURRENT_BINARY_DIR}/generated/) - -generate_function_register(Arithmetic - FunctionPlus - FunctionMinus - FunctionMultiply - FunctionDivideFloating - FunctionDivideIntegral - FunctionDivideIntegralOrZero - FunctionModulo - FunctionNegate - FunctionAbs - FunctionBitAnd - FunctionBitOr - FunctionBitXor - FunctionBitNot - FunctionBitShiftLeft - FunctionBitShiftRight - FunctionBitRotateLeft - FunctionBitRotateRight - FunctionLeast - FunctionGreatest - FunctionBitTest - FunctionBitTestAny - FunctionBitTestAll - FunctionGCD - FunctionLCM - FunctionIntExp2 - FunctionIntExp10 -) - add_headers_and_sources(clickhouse_functions .) add_headers_and_sources(clickhouse_functions ./GatherUtils) add_headers_and_sources(clickhouse_functions ./Conditional) -#add_headers_and_sources(clickhouse_functions ${ClickHouse_BINARY_DIR}/dbms/src/Functions) -add_headers_and_sources(clickhouse_functions ${FUNCTIONS_GENERATED_DIR}) list(REMOVE_ITEM clickhouse_functions_sources IFunction.cpp FunctionFactory.cpp FunctionHelpers.cpp) diff --git a/dbms/src/Functions/FunctionBinaryArithmetic.h b/dbms/src/Functions/FunctionBinaryArithmetic.h new file mode 100644 index 00000000000..a668c43b122 --- /dev/null +++ b/dbms/src/Functions/FunctionBinaryArithmetic.h @@ -0,0 +1,921 @@ +#pragma once + +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + +#if USE_EMBEDDED_COMPILER +#pragma GCC diagnostic push +#pragma GCC diagnostic ignored "-Wunused-parameter" +#include // Y_IGNORE +#pragma GCC diagnostic pop +#endif + + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int ILLEGAL_COLUMN; + extern const int ILLEGAL_TYPE_OF_ARGUMENT; + extern const int LOGICAL_ERROR; + extern const int DECIMAL_OVERFLOW; + extern const int CANNOT_ADD_DIFFERENT_AGGREGATE_STATES; + extern const int ILLEGAL_DIVISION; +} + + +/** Arithmetic operations: +, -, *, /, %, + * intDiv (integer division) + * Bitwise operations: |, &, ^, ~. + * Etc. + */ + +template +struct BinaryOperationImplBase +{ + using ResultType = ResultType_; + + static void NO_INLINE vector_vector(const PaddedPODArray & a, const PaddedPODArray & b, PaddedPODArray & c) + { + size_t size = a.size(); + for (size_t i = 0; i < size; ++i) + c[i] = Op::template apply(a[i], b[i]); + } + + static void NO_INLINE vector_constant(const PaddedPODArray & a, B b, PaddedPODArray & c) + { + size_t size = a.size(); + for (size_t i = 0; i < size; ++i) + c[i] = Op::template apply(a[i], b); + } + + static void NO_INLINE constant_vector(A a, const PaddedPODArray & b, PaddedPODArray & c) + { + size_t size = b.size(); + for (size_t i = 0; i < size; ++i) + c[i] = Op::template apply(a, b[i]); + } + + static ResultType constant_constant(A a, B b) + { + return Op::template apply(a, b); + } +}; + +template +struct BinaryOperationImpl : BinaryOperationImplBase +{ +}; + + +template struct PlusImpl; +template struct MinusImpl; +template struct MultiplyImpl; +template struct DivideFloatingImpl; +template struct DivideIntegralImpl; +template struct DivideIntegralOrZeroImpl; +template struct LeastBaseImpl; +template struct GreatestBaseImpl; +template struct ModuloImpl; + + +template struct NativeType { using Type = T; }; +template <> struct NativeType { using Type = Int32; }; +template <> struct NativeType { using Type = Int64; }; +template <> struct NativeType { using Type = Int128; }; + +/// Binary operations for Decimals need scale args +/// +|- scale one of args (which scale factor is not 1). ScaleR = oneof(Scale1, Scale2); +/// * no agrs scale. ScaleR = Scale1 + Scale2; +/// / first arg scale. ScaleR = Scale1 (scale_a = DecimalType::getScale()). +template typename Operation, typename ResultType_, bool _check_overflow = true> +struct DecimalBinaryOperation +{ + static constexpr bool is_plus_minus = std::is_same_v, PlusImpl> || + std::is_same_v, MinusImpl>; + static constexpr bool is_multiply = std::is_same_v, MultiplyImpl>; + static constexpr bool is_float_division = std::is_same_v, DivideFloatingImpl>; + static constexpr bool is_int_division = std::is_same_v, DivideIntegralImpl> || + std::is_same_v, DivideIntegralOrZeroImpl>; + static constexpr bool is_division = is_float_division || is_int_division; + static constexpr bool is_compare = std::is_same_v, LeastBaseImpl> || + std::is_same_v, GreatestBaseImpl>; + static constexpr bool is_plus_minus_compare = is_plus_minus || is_compare; + static constexpr bool can_overflow = is_plus_minus || is_multiply; + + using ResultType = ResultType_; + using NativeResultType = typename NativeType::Type; + using Op = std::conditional_t, /// substitute divide by intDiv (throw on division by zero) + Operation>; + using ColVecA = std::conditional_t, ColumnDecimal, ColumnVector>; + using ColVecB = std::conditional_t, ColumnDecimal, ColumnVector>; + using ArrayA = typename ColVecA::Container; + using ArrayB = typename ColVecB::Container; + using ArrayC = typename ColumnDecimal::Container; + using SelfNoOverflow = DecimalBinaryOperation; + + static void vector_vector(const ArrayA & a, const ArrayB & b, ArrayC & c, ResultType scale_a, ResultType scale_b, bool check_overflow) + { + if (check_overflow) + vector_vector(a, b, c, scale_a, scale_b); + else + SelfNoOverflow::vector_vector(a, b, c, scale_a, scale_b); + } + + static void vector_constant(const ArrayA & a, B b, ArrayC & c, ResultType scale_a, ResultType scale_b, bool check_overflow) + { + if (check_overflow) + vector_constant(a, b, c, scale_a, scale_b); + else + SelfNoOverflow::vector_constant(a, b, c, scale_a, scale_b); + } + + static void constant_vector(A a, const ArrayB & b, ArrayC & c, ResultType scale_a, ResultType scale_b, bool check_overflow) + { + if (check_overflow) + constant_vector(a, b, c, scale_a, scale_b); + else + SelfNoOverflow::constant_vector(a, b, c, scale_a, scale_b); + } + + static ResultType constant_constant(A a, B b, ResultType scale_a, ResultType scale_b, bool check_overflow) + { + if (check_overflow) + return constant_constant(a, b, scale_a, scale_b); + else + return SelfNoOverflow::constant_constant(a, b, scale_a, scale_b); + } + + static void NO_INLINE vector_vector(const ArrayA & a, const ArrayB & b, ArrayC & c, + ResultType scale_a [[maybe_unused]], ResultType scale_b [[maybe_unused]]) + { + size_t size = a.size(); + if constexpr (is_plus_minus_compare) + { + if (scale_a != 1) + { + for (size_t i = 0; i < size; ++i) + c[i] = applyScaled(a[i], b[i], scale_a); + return; + } + else if (scale_b != 1) + { + for (size_t i = 0; i < size; ++i) + c[i] = applyScaled(a[i], b[i], scale_b); + return; + } + } + else if constexpr (is_division && IsDecimalNumber) + { + for (size_t i = 0; i < size; ++i) + c[i] = applyScaledDiv(a[i], b[i], scale_a); + return; + } + + /// default: use it if no return before + for (size_t i = 0; i < size; ++i) + c[i] = apply(a[i], b[i]); + } + + static void NO_INLINE vector_constant(const ArrayA & a, B b, ArrayC & c, + ResultType scale_a [[maybe_unused]], ResultType scale_b [[maybe_unused]]) + { + size_t size = a.size(); + if constexpr (is_plus_minus_compare) + { + if (scale_a != 1) + { + for (size_t i = 0; i < size; ++i) + c[i] = applyScaled(a[i], b, scale_a); + return; + } + else if (scale_b != 1) + { + for (size_t i = 0; i < size; ++i) + c[i] = applyScaled(a[i], b, scale_b); + return; + } + } + else if constexpr (is_division && IsDecimalNumber) + { + for (size_t i = 0; i < size; ++i) + c[i] = applyScaledDiv(a[i], b, scale_a); + return; + } + + /// default: use it if no return before + for (size_t i = 0; i < size; ++i) + c[i] = apply(a[i], b); + } + + static void NO_INLINE constant_vector(A a, const ArrayB & b, ArrayC & c, + ResultType scale_a [[maybe_unused]], ResultType scale_b [[maybe_unused]]) + { + size_t size = b.size(); + if constexpr (is_plus_minus_compare) + { + if (scale_a != 1) + { + for (size_t i = 0; i < size; ++i) + c[i] = applyScaled(a, b[i], scale_a); + return; + } + else if (scale_b != 1) + { + for (size_t i = 0; i < size; ++i) + c[i] = applyScaled(a, b[i], scale_b); + return; + } + } + else if constexpr (is_division && IsDecimalNumber) + { + for (size_t i = 0; i < size; ++i) + c[i] = applyScaledDiv(a, b[i], scale_a); + return; + } + + /// default: use it if no return before + for (size_t i = 0; i < size; ++i) + c[i] = apply(a, b[i]); + } + + static ResultType constant_constant(A a, B b, ResultType scale_a [[maybe_unused]], ResultType scale_b [[maybe_unused]]) + { + if constexpr (is_plus_minus_compare) + { + if (scale_a != 1) + return applyScaled(a, b, scale_a); + else if (scale_b != 1) + return applyScaled(a, b, scale_b); + } + else if constexpr (is_division && IsDecimalNumber) + return applyScaledDiv(a, b, scale_a); + return apply(a, b); + } + +private: + /// there's implicit type convertion here + static NativeResultType apply(NativeResultType a, NativeResultType b) + { + if constexpr (can_overflow && _check_overflow) + { + NativeResultType res; + if (Op::template apply(a, b, res)) + throw Exception("Decimal math overflow", ErrorCodes::DECIMAL_OVERFLOW); + return res; + } + else + return Op::template apply(a, b); + } + + template + static NativeResultType applyScaled(NativeResultType a, NativeResultType b, NativeResultType scale) + { + if constexpr (is_plus_minus_compare) + { + NativeResultType res; + + if constexpr (_check_overflow) + { + bool overflow = false; + if constexpr (scale_left) + overflow |= common::mulOverflow(a, scale, a); + else + overflow |= common::mulOverflow(b, scale, b); + + if constexpr (can_overflow) + overflow |= Op::template apply(a, b, res); + else + res = Op::template apply(a, b); + + if (overflow) + throw Exception("Decimal math overflow", ErrorCodes::DECIMAL_OVERFLOW); + } + else + { + if constexpr (scale_left) + a *= scale; + else + b *= scale; + res = Op::template apply(a, b); + } + + return res; + } + } + + static NativeResultType applyScaledDiv(NativeResultType a, NativeResultType b, NativeResultType scale) + { + if constexpr (is_division) + { + if constexpr (_check_overflow) + { + bool overflow = false; + if constexpr (!IsDecimalNumber) + overflow |= common::mulOverflow(scale, scale, scale); + overflow |= common::mulOverflow(a, scale, a); + if (overflow) + throw Exception("Decimal math overflow", ErrorCodes::DECIMAL_OVERFLOW); + } + else + { + if constexpr (!IsDecimalNumber) + scale *= scale; + a *= scale; + } + + return Op::template apply(a, b); + } + } +}; + + +/// Used to indicate undefined operation +struct InvalidType; + +template struct Case : std::bool_constant { using type = T; }; + +/// Switch, ...> -- select the first Ti for which Ci is true; InvalidType if none. +template using Switch = typename std::disjunction>::type; + +template constexpr bool IsIntegral = false; +template <> constexpr bool IsIntegral = true; +template <> constexpr bool IsIntegral = true; +template <> constexpr bool IsIntegral = true; +template <> constexpr bool IsIntegral = true; +template <> constexpr bool IsIntegral = true; +template <> constexpr bool IsIntegral = true; +template <> constexpr bool IsIntegral = true; +template <> constexpr bool IsIntegral = true; + +template constexpr bool IsFloatingPoint = false; +template <> constexpr bool IsFloatingPoint = true; +template <> constexpr bool IsFloatingPoint = true; + +template constexpr bool IsDateOrDateTime = false; +template <> constexpr bool IsDateOrDateTime = true; +template <> constexpr bool IsDateOrDateTime = true; + +template constexpr bool UseLeftDecimal = false; +template <> constexpr bool UseLeftDecimal, DataTypeDecimal> = true; +template <> constexpr bool UseLeftDecimal, DataTypeDecimal> = true; +template <> constexpr bool UseLeftDecimal, DataTypeDecimal> = true; + +template using DataTypeFromFieldType = std::conditional_t, InvalidType, DataTypeNumber>; + +template