From 8ea4d7987be4ed700d667571854e6d2460680c9f Mon Sep 17 00:00:00 2001 From: Amos Bird Date: Sun, 30 Dec 2018 23:54:45 +0800 Subject: [PATCH 01/86] Add join_overwrite settings for StorageJoin. This makes using StorageJoin as dictionary more flexible. --- dbms/src/Interpreters/Join.cpp | 121 ++++++---------- dbms/src/Interpreters/Join.h | 131 +++++++++++++++--- dbms/src/Interpreters/Settings.h | 1 + dbms/src/Storages/StorageJoin.cpp | 19 ++- dbms/src/Storages/StorageJoin.h | 3 +- .../00830_join_overwrite.reference | 2 + .../0_stateless/00830_join_overwrite.sql | 15 ++ libs/libcommon/CMakeLists.txt | 1 + .../include/common/constexpr_helpers.h | 31 +++++ 9 files changed, 219 insertions(+), 105 deletions(-) create mode 100644 dbms/tests/queries/0_stateless/00830_join_overwrite.reference create mode 100644 dbms/tests/queries/0_stateless/00830_join_overwrite.sql create mode 100644 libs/libcommon/include/common/constexpr_helpers.h diff --git a/dbms/src/Interpreters/Join.cpp b/dbms/src/Interpreters/Join.cpp index 8783d16c3c1..da8209a09d3 100644 --- a/dbms/src/Interpreters/Join.cpp +++ b/dbms/src/Interpreters/Join.cpp @@ -1,3 +1,5 @@ +#include +#include #include #include @@ -32,10 +34,11 @@ namespace ErrorCodes Join::Join(const Names & key_names_right_, bool use_nulls_, const SizeLimits & limits, - ASTTableJoin::Kind kind_, ASTTableJoin::Strictness strictness_) + ASTTableJoin::Kind kind_, ASTTableJoin::Strictness strictness_, bool overwrite_) : kind(kind_), strictness(strictness_), key_names_right(key_names_right_), use_nulls(use_nulls_), + overwrite(overwrite_), log(&Logger::get("Join")), limits(limits) { @@ -177,21 +180,8 @@ void Join::init(Type type_) if (kind == ASTTableJoin::Kind::Cross) return; - - if (!getFullness(kind)) - { - if (strictness == ASTTableJoin::Strictness::Any) - initImpl(maps_any, type); - else - initImpl(maps_all, type); - } - else - { - if (strictness == ASTTableJoin::Strictness::Any) - initImpl(maps_any_full, type); - else - initImpl(maps_all_full, type); - } + dispatch(MapInitTag()); + dispatch([&](auto, auto, auto & map) { initImpl(map, type); }); } size_t Join::getTotalRowCount() const @@ -205,10 +195,7 @@ size_t Join::getTotalRowCount() const } else { - res += getTotalRowCountImpl(maps_any, type); - res += getTotalRowCountImpl(maps_all, type); - res += getTotalRowCountImpl(maps_any_full, type); - res += getTotalRowCountImpl(maps_all_full, type); + dispatch([&](auto, auto, auto & map) { res += getTotalRowCountImpl(map, type); }); } return res; @@ -225,10 +212,7 @@ size_t Join::getTotalByteCount() const } else { - res += getTotalByteCountImpl(maps_any, type); - res += getTotalByteCountImpl(maps_all, type); - res += getTotalByteCountImpl(maps_any_full, type); - res += getTotalByteCountImpl(maps_all_full, type); + dispatch([&](auto, auto, auto & map) { res += getTotalByteCountImpl(map, type); }); res += pool.size(); } @@ -326,6 +310,8 @@ namespace KeyGetter::onNewKey(it->first, pool); new (&it->second) typename Map::mapped_type(stored_block, i); } + else if (it->second.overwrite) + new (&it->second) typename Map::mapped_type(stored_block, i); } }; @@ -482,21 +468,9 @@ bool Join::insertFromBlock(const Block & block) if (kind != ASTTableJoin::Kind::Cross) { - /// Fill the hash table. - if (!getFullness(kind)) - { - if (strictness == ASTTableJoin::Strictness::Any) - insertFromBlockImpl(type, maps_any, rows, key_columns, keys_size, key_sizes, stored_block, null_map, pool); - else - insertFromBlockImpl(type, maps_all, rows, key_columns, keys_size, key_sizes, stored_block, null_map, pool); - } - else - { - if (strictness == ASTTableJoin::Strictness::Any) - insertFromBlockImpl(type, maps_any_full, rows, key_columns, keys_size, key_sizes, stored_block, null_map, pool); - else - insertFromBlockImpl(type, maps_all_full, rows, key_columns, keys_size, key_sizes, stored_block, null_map, pool); - } + dispatch([&](auto, auto strictness_, auto & map) { + insertFromBlockImpl(type, map, rows, key_columns, keys_size, key_sizes, stored_block, null_map, pool); + }); } return limits.check(getTotalRowCount(), getTotalByteCount(), "JOIN", ErrorCodes::SET_SIZE_LIMIT_EXCEEDED); @@ -505,11 +479,11 @@ bool Join::insertFromBlock(const Block & block) namespace { - template + template struct Adder; template - struct Adder + struct Adder { static void addFound(const typename Map::const_iterator & it, size_t num_columns_to_add, MutableColumns & added_columns, size_t i, IColumn::Filter * filter, IColumn::Offset & /*current_offset*/, IColumn::Offsets * /*offsets*/, @@ -532,7 +506,7 @@ namespace }; template - struct Adder + struct Adder { static void addFound(const typename Map::const_iterator & it, size_t num_columns_to_add, MutableColumns & added_columns, size_t i, IColumn::Filter * filter, IColumn::Offset & /*current_offset*/, IColumn::Offsets * /*offsets*/, @@ -551,8 +525,8 @@ namespace } }; - template - struct Adder + template + struct Adder { static void addFound(const typename Map::const_iterator & it, size_t num_columns_to_add, MutableColumns & added_columns, size_t i, IColumn::Filter * filter, IColumn::Offset & current_offset, IColumn::Offsets * offsets, @@ -578,7 +552,7 @@ namespace { (*filter)[i] = 0; - if (KIND == ASTTableJoin::Kind::Inner) + if (!pad_left) { (*offsets)[i] = current_offset; } @@ -609,7 +583,7 @@ namespace { if (has_null_map && (*null_map)[i]) { - Adder::addNotFound( + Adder::pad_left, STRICTNESS, Map>::addNotFound( num_columns_to_add, added_columns, i, filter.get(), current_offset, offsets_to_replicate.get()); } else @@ -620,11 +594,11 @@ namespace if (it != map.end()) { it->second.setUsed(); - Adder::addFound( + Adder::pad_left, STRICTNESS, Map>::addFound( it, num_columns_to_add, added_columns, i, filter.get(), current_offset, offsets_to_replicate.get(), right_indexes); } else - Adder::addNotFound( + Adder::pad_left, STRICTNESS, Map>::addNotFound( num_columns_to_add, added_columns, i, filter.get(), current_offset, offsets_to_replicate.get()); } } @@ -655,7 +629,7 @@ void Join::joinBlockImpl( const Names & key_names_left, const NameSet & needed_key_names_right, const Block & block_with_columns_to_add, - const Maps & maps) const + const Maps & maps_) const { size_t keys_size = key_names_left.size(); ColumnRawPtrs key_columns(keys_size); @@ -749,7 +723,7 @@ void Join::joinBlockImpl( #define M(TYPE) \ case Join::Type::TYPE: \ joinBlockImplType::Type>(\ - *maps.TYPE, rows, key_columns, key_sizes, added_columns, null_map, \ + *maps_.TYPE, rows, key_columns, key_sizes, added_columns, null_map, \ filter, current_offset, offsets_to_replicate, right_indexes); \ break; APPLY_FOR_JOIN_VARIANTS(M) @@ -906,10 +880,10 @@ DataTypePtr Join::joinGetReturnType(const String & column_name) const template -void Join::joinGetImpl(Block & block, const String & column_name, const Maps & maps) const +void Join::joinGetImpl(Block & block, const String & column_name, const Maps & maps_) const { joinBlockImpl( - block, {block.getByPosition(0).name}, {}, {sample_block_with_columns_to_add.getByName(column_name)}, maps); + block, {block.getByPosition(0).name}, {}, {sample_block_with_columns_to_add.getByName(column_name)}, maps_); } @@ -926,7 +900,12 @@ void Join::joinGet(Block & block, const String & column_name) const checkTypeOfKey(block, sample_block_with_keys); if (kind == ASTTableJoin::Kind::Left && strictness == ASTTableJoin::Strictness::Any) - joinGetImpl(block, column_name, maps_any); + { + if (overwrite) + joinGetImpl(block, column_name, std::get(maps)); + else + joinGetImpl(block, column_name, std::get(maps)); + } else throw Exception("joinGet only supports StorageJoin of type Left Any", ErrorCodes::LOGICAL_ERROR); } @@ -940,30 +919,10 @@ void Join::joinBlock(Block & block, const Names & key_names_left, const NameSet checkTypesOfKeys(block, key_names_left, sample_block_with_keys); - if (kind == ASTTableJoin::Kind::Left && strictness == ASTTableJoin::Strictness::Any) - joinBlockImpl( - block, key_names_left, needed_key_names_right, sample_block_with_columns_to_add, maps_any); - else if (kind == ASTTableJoin::Kind::Inner && strictness == ASTTableJoin::Strictness::Any) - joinBlockImpl( - block, key_names_left, needed_key_names_right, sample_block_with_columns_to_add, maps_any); - else if (kind == ASTTableJoin::Kind::Left && strictness == ASTTableJoin::Strictness::All) - joinBlockImpl( - block, key_names_left, needed_key_names_right, sample_block_with_columns_to_add, maps_all); - else if (kind == ASTTableJoin::Kind::Inner && strictness == ASTTableJoin::Strictness::All) - joinBlockImpl( - block, key_names_left, needed_key_names_right, sample_block_with_columns_to_add, maps_all); - else if (kind == ASTTableJoin::Kind::Full && strictness == ASTTableJoin::Strictness::Any) - joinBlockImpl( - block, key_names_left, needed_key_names_right, sample_block_with_columns_to_add, maps_any_full); - else if (kind == ASTTableJoin::Kind::Right && strictness == ASTTableJoin::Strictness::Any) - joinBlockImpl( - block, key_names_left, needed_key_names_right, sample_block_with_columns_to_add, maps_any_full); - else if (kind == ASTTableJoin::Kind::Full && strictness == ASTTableJoin::Strictness::All) - joinBlockImpl( - block, key_names_left, needed_key_names_right, sample_block_with_columns_to_add, maps_all_full); - else if (kind == ASTTableJoin::Kind::Right && strictness == ASTTableJoin::Strictness::All) - joinBlockImpl( - block, key_names_left, needed_key_names_right, sample_block_with_columns_to_add, maps_all_full); + if (dispatch([&](auto kind_, auto strictness_, auto & map) { + joinBlockImpl(block, key_names_left, needed_key_names_right, sample_block_with_columns_to_add, map); + })) + ; else if (kind == ASTTableJoin::Kind::Cross) joinBlockImplCross(block); else @@ -1112,12 +1071,12 @@ protected: if (parent.blocks.empty()) return Block(); - if (parent.strictness == ASTTableJoin::Strictness::Any) - return createBlock(parent.maps_any_full); - else if (parent.strictness == ASTTableJoin::Strictness::All) - return createBlock(parent.maps_all_full); + Block block; + if (parent.dispatch([&](auto, auto strictness, auto & map) { block = createBlock(map); })) + ; else throw Exception("Logical error: unknown JOIN strictness (must be ANY or ALL)", ErrorCodes::LOGICAL_ERROR); + return block; } private: diff --git a/dbms/src/Interpreters/Join.h b/dbms/src/Interpreters/Join.h index 65371239851..c1f665262a8 100644 --- a/dbms/src/Interpreters/Join.h +++ b/dbms/src/Interpreters/Join.h @@ -15,6 +15,8 @@ #include #include +#include +#include namespace DB @@ -221,7 +223,7 @@ class Join { public: Join(const Names & key_names_right_, bool use_nulls_, const SizeLimits & limits, - ASTTableJoin::Kind kind_, ASTTableJoin::Strictness strictness_); + ASTTableJoin::Kind kind_, ASTTableJoin::Strictness strictness_, bool overwrite_ = false); bool empty() { return type == Type::EMPTY; } @@ -289,15 +291,18 @@ public: /** Depending on template parameter, adds or doesn't add a flag, that element was used (row was joined). - * For implementation of RIGHT and FULL JOINs. + * Depending on template parameter, decide whether to overwrite existing values when encountering the same key again + * with_used is for implementation of RIGHT and FULL JOINs. + * overwrite is for implementation of StorageJoin with overwrite setting enabled * NOTE: It is possible to store the flag in one bit of pointer to block or row_num. It seems not reasonable, because memory saving is minimal. */ - template - struct WithUsedFlag; + template + struct WithFlags; - template - struct WithUsedFlag : Base + template + struct WithFlags : Base { + static constexpr bool overwrite = overwrite_; mutable std::atomic used {}; using Base::Base; using Base_t = Base; @@ -305,9 +310,10 @@ public: bool getUsed() const { return used; } }; - template - struct WithUsedFlag : Base + template + struct WithFlags : Base { + static constexpr bool overwrite = overwrite_; using Base::Base; using Base_t = Base; void setUsed() const {} @@ -363,10 +369,80 @@ public: std::unique_ptr> hashed; }; - using MapsAny = MapsTemplate>; - using MapsAll = MapsTemplate>; - using MapsAnyFull = MapsTemplate>; - using MapsAllFull = MapsTemplate>; + using MapsAny = MapsTemplate>; + using MapsAnyOverwrite = MapsTemplate>; + using MapsAll = MapsTemplate>; + using MapsAnyFull = MapsTemplate>; + using MapsAnyFullOverwrite = MapsTemplate>; + using MapsAllFull = MapsTemplate>; + + template + struct KindTrait + { + // Affects the Adder trait so that when the right part is empty, adding a default value on the left + static constexpr bool pad_left = static_in_v; + + // Affects the Map trait so that a `used` flag is attached to map slots in order to + // generate default values on the right when the left part is empty + static constexpr bool pad_right = static_in_v; + }; + + template + struct MapGetterImpl; + + template + using Map = typename MapGetterImpl::pad_right, strictness, overwrite>::Map; + + static constexpr std::array STRICTNESSES = {ASTTableJoin::Strictness::Any, ASTTableJoin::Strictness::All}; + static constexpr std::array KINDS + = {ASTTableJoin::Kind::Left, ASTTableJoin::Kind::Inner, ASTTableJoin::Kind::Full, ASTTableJoin::Kind::Right}; + + struct MapInitTag {}; + + template + bool dispatch(Func && func) + { + if (overwrite) + return static_for<0, KINDS.size()>([&](auto i) { + if (kind == KINDS[i] && strictness == ASTTableJoin::Strictness::Any) + { + if constexpr (std::is_same_v) + maps = Map(); + else + func( + std::integral_constant(), + std::integral_constant(), + std::get>(maps)); + return true; + } + return false; + }); + else + return static_for<0, KINDS.size() * STRICTNESSES.size()>([&](auto ij) { + // NOTE: Avoid using nested static loop as GCC and CLANG have bugs in different ways + // See https://stackoverflow.com/questions/44386415/gcc-and-clang-disagree-about-c17-constexpr-lambda-captures + constexpr auto i = ij / STRICTNESSES.size(); + constexpr auto j = ij % STRICTNESSES.size(); + if (kind == KINDS[i] && strictness == STRICTNESSES[j]) + { + if constexpr (std::is_same_v) + maps = Map(); + else + func( + std::integral_constant(), + std::integral_constant(), + std::get>(maps)); + return true; + } + return false; + }); + } + + template + bool dispatch(Func && func) const + { + return const_cast(*this).dispatch(std::forward(func)); + } private: friend class NonJoinedBlockInputStream; @@ -381,14 +457,14 @@ private: /// Substitute NULLs for non-JOINed rows. bool use_nulls; + /// Overwrite existing values when encountering the same key again + bool overwrite; + /** Blocks of "right" table. */ BlocksList blocks; - MapsAny maps_any; /// For ANY LEFT|INNER JOIN - MapsAll maps_all; /// For ALL LEFT|INNER JOIN - MapsAnyFull maps_any_full; /// For ANY RIGHT|FULL JOIN - MapsAllFull maps_all_full; /// For ALL RIGHT|FULL JOIN + std::variant maps; /// Additional data - strings for string keys and continuation elements of single-linked lists of references to rows. Arena pool; @@ -441,5 +517,28 @@ private: using JoinPtr = std::shared_ptr; using Joins = std::vector; +template +struct Join::MapGetterImpl +{ + using Map = std::conditional_t; +}; + +template +struct Join::MapGetterImpl +{ + using Map = std::conditional_t; +}; + +template <> +struct Join::MapGetterImpl +{ + using Map = MapsAll; +}; + +template <> +struct Join::MapGetterImpl +{ + using Map = MapsAllFull; +}; } diff --git a/dbms/src/Interpreters/Settings.h b/dbms/src/Interpreters/Settings.h index 646064edc86..f43776a8cf2 100644 --- a/dbms/src/Interpreters/Settings.h +++ b/dbms/src/Interpreters/Settings.h @@ -253,6 +253,7 @@ struct Settings M(SettingUInt64, max_rows_in_join, 0, "Maximum size of the hash table for JOIN (in number of rows).") \ M(SettingUInt64, max_bytes_in_join, 0, "Maximum size of the hash table for JOIN (in number of bytes in memory).") \ M(SettingOverflowMode, join_overflow_mode, OverflowMode::THROW, "What to do when the limit is exceeded.") \ + M(SettingBool, join_overwrite, false, "Whether to overwrite existing values when encountering the same key again.") \ \ M(SettingUInt64, max_rows_to_transfer, 0, "Maximum size (in rows) of the transmitted external table obtained when the GLOBAL IN/JOIN section is executed.") \ M(SettingUInt64, max_bytes_to_transfer, 0, "Maximum size (in uncompressed bytes) of the transmitted external table obtained when the GLOBAL IN/JOIN section is executed.") \ diff --git a/dbms/src/Storages/StorageJoin.cpp b/dbms/src/Storages/StorageJoin.cpp index 8d1d6d52fbf..4140570d2e6 100644 --- a/dbms/src/Storages/StorageJoin.cpp +++ b/dbms/src/Storages/StorageJoin.cpp @@ -33,7 +33,8 @@ StorageJoin::StorageJoin( SizeLimits limits_, ASTTableJoin::Kind kind_, ASTTableJoin::Strictness strictness_, - const ColumnsDescription & columns_) + const ColumnsDescription & columns_, + bool overwrite) : StorageSetOrJoinBase{path_, name_, columns_} , key_names(key_names_) , use_nulls(use_nulls_) @@ -45,7 +46,7 @@ StorageJoin::StorageJoin( if (!getColumns().hasPhysical(key)) throw Exception{"Key column (" + key + ") does not exist in table declaration.", ErrorCodes::NO_SUCH_COLUMN_IN_TABLE}; - join = std::make_shared(key_names, use_nulls, limits, kind, strictness); + join = std::make_shared(key_names, use_nulls, limits, kind, strictness, overwrite); join->setSampleBlock(getSampleBlock().sortColumns()); restore(); } @@ -134,6 +135,7 @@ void registerStorageJoin(StorageFactory & factory) auto max_rows_in_join = settings.max_rows_in_join; auto max_bytes_in_join = settings.max_bytes_in_join; auto join_overflow_mode = settings.join_overflow_mode; + auto join_overwrite = settings.join_overwrite; if (args.storage_def && args.storage_def->settings) { @@ -147,6 +149,8 @@ void registerStorageJoin(StorageFactory & factory) max_bytes_in_join.set(setting.value); else if (setting.name == "join_overflow_mode") join_overflow_mode.set(setting.value); + else if (setting.name == "join_overwrite") + join_overwrite.set(setting.value); else throw Exception( "Unknown setting " + setting.name + " for storage " + args.engine_name, @@ -162,7 +166,8 @@ void registerStorageJoin(StorageFactory & factory) SizeLimits{max_rows_in_join.value, max_bytes_in_join.value, join_overflow_mode.value}, kind, strictness, - args.columns); + args.columns, + join_overwrite); }); } @@ -224,12 +229,12 @@ protected: if (parent.blocks.empty()) return Block(); - if (parent.strictness == ASTTableJoin::Strictness::Any) - return createBlock(parent.maps_any); - else if (parent.strictness == ASTTableJoin::Strictness::All) - return createBlock(parent.maps_all); + Block block; + if (parent.dispatch([&](auto, auto strictness, auto & map) { block = createBlock(map); })) + ; else throw Exception("Logical error: unknown JOIN strictness (must be ANY or ALL)", ErrorCodes::LOGICAL_ERROR); + return block; } private: diff --git a/dbms/src/Storages/StorageJoin.h b/dbms/src/Storages/StorageJoin.h index ebe9dc68ce4..177cdee9834 100644 --- a/dbms/src/Storages/StorageJoin.h +++ b/dbms/src/Storages/StorageJoin.h @@ -62,7 +62,8 @@ protected: bool use_nulls_, SizeLimits limits_, ASTTableJoin::Kind kind_, ASTTableJoin::Strictness strictness_, - const ColumnsDescription & columns_); + const ColumnsDescription & columns_, + bool overwrite); }; } diff --git a/dbms/tests/queries/0_stateless/00830_join_overwrite.reference b/dbms/tests/queries/0_stateless/00830_join_overwrite.reference new file mode 100644 index 00000000000..4792e70f333 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00830_join_overwrite.reference @@ -0,0 +1,2 @@ +2 +3 diff --git a/dbms/tests/queries/0_stateless/00830_join_overwrite.sql b/dbms/tests/queries/0_stateless/00830_join_overwrite.sql new file mode 100644 index 00000000000..1f981eb00bd --- /dev/null +++ b/dbms/tests/queries/0_stateless/00830_join_overwrite.sql @@ -0,0 +1,15 @@ +USE test; + +DROP TABLE IF EXISTS kv; + +CREATE TABLE kv (k UInt32, v UInt32) ENGINE Join(Any, Left, k); +INSERT INTO kv VALUES (1, 2); +INSERT INTO kv VALUES (1, 3); +SELECT joinGet('kv', 'v', toUInt32(1)); +CREATE TABLE kv_overwrite (k UInt32, v UInt32) ENGINE Join(Any, Left, k) SETTINGS join_overwrite = 1; +INSERT INTO kv_overwrite VALUES (1, 2); +INSERT INTO kv_overwrite VALUES (1, 3); +SELECT joinGet('kv_overwrite', 'v', toUInt32(1)); + +DROP TABLE kv; +DROP TABLE kv_overwrite; diff --git a/libs/libcommon/CMakeLists.txt b/libs/libcommon/CMakeLists.txt index 5c6c242407f..0caa57453c9 100644 --- a/libs/libcommon/CMakeLists.txt +++ b/libs/libcommon/CMakeLists.txt @@ -42,6 +42,7 @@ add_library (common ${LINK_MODE} include/common/demangle.h include/common/SetTerminalEcho.h include/common/find_symbols.h + include/common/constexpr_helpers.h include/ext/bit_cast.h include/ext/collection_cast.h diff --git a/libs/libcommon/include/common/constexpr_helpers.h b/libs/libcommon/include/common/constexpr_helpers.h new file mode 100644 index 00000000000..c6fcb0bb4db --- /dev/null +++ b/libs/libcommon/include/common/constexpr_helpers.h @@ -0,0 +1,31 @@ +#pragma once + +#include + +template +inline constexpr bool static_in_v = std::disjunction_v...>; + +template +bool func_wrapper(Func && func, Arg && arg) +{ + if constexpr (std::is_void_v>) + { + func(arg); + return false; + } + else + return func(arg); +} + +template +constexpr bool static_for_impl(Func && f, std::integer_sequence) +{ + return (func_wrapper(std::forward(f), std::integral_constant{}) || ...); +} + +template +constexpr bool static_for(Func && f) +{ + using T = decltype(Begin); + return static_for_impl(std::forward(f), std::make_integer_sequence{}); +} From 8a401e491a5806c5ff3ba9049ad299b4d0bdcb49 Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Tue, 15 Jan 2019 00:40:02 +0300 Subject: [PATCH 02/86] Update Join.cpp --- dbms/src/Interpreters/Join.cpp | 10 +++++++--- 1 file changed, 7 insertions(+), 3 deletions(-) diff --git a/dbms/src/Interpreters/Join.cpp b/dbms/src/Interpreters/Join.cpp index da8209a09d3..0571723646f 100644 --- a/dbms/src/Interpreters/Join.cpp +++ b/dbms/src/Interpreters/Join.cpp @@ -468,7 +468,8 @@ bool Join::insertFromBlock(const Block & block) if (kind != ASTTableJoin::Kind::Cross) { - dispatch([&](auto, auto strictness_, auto & map) { + dispatch([&](auto, auto strictness_, auto & map) + { insertFromBlockImpl(type, map, rows, key_columns, keys_size, key_sizes, stored_block, null_map, pool); }); } @@ -919,10 +920,13 @@ void Join::joinBlock(Block & block, const Names & key_names_left, const NameSet checkTypesOfKeys(block, key_names_left, sample_block_with_keys); - if (dispatch([&](auto kind_, auto strictness_, auto & map) { + if (dispatch([&](auto kind_, auto strictness_, auto & map) + { joinBlockImpl(block, key_names_left, needed_key_names_right, sample_block_with_columns_to_add, map); })) - ; + { + /// Joined + } else if (kind == ASTTableJoin::Kind::Cross) joinBlockImplCross(block); else From 64c31a6ad7fed521089da9c234180b602c8f1c4c Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Tue, 15 Jan 2019 00:40:42 +0300 Subject: [PATCH 03/86] Update Join.h --- dbms/src/Interpreters/Join.h | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/dbms/src/Interpreters/Join.h b/dbms/src/Interpreters/Join.h index c1f665262a8..38b05fb2de1 100644 --- a/dbms/src/Interpreters/Join.h +++ b/dbms/src/Interpreters/Join.h @@ -403,7 +403,8 @@ public: bool dispatch(Func && func) { if (overwrite) - return static_for<0, KINDS.size()>([&](auto i) { + return static_for<0, KINDS.size()>([&](auto i) + { if (kind == KINDS[i] && strictness == ASTTableJoin::Strictness::Any) { if constexpr (std::is_same_v) @@ -418,7 +419,8 @@ public: return false; }); else - return static_for<0, KINDS.size() * STRICTNESSES.size()>([&](auto ij) { + return static_for<0, KINDS.size() * STRICTNESSES.size()>([&](auto ij) + { // NOTE: Avoid using nested static loop as GCC and CLANG have bugs in different ways // See https://stackoverflow.com/questions/44386415/gcc-and-clang-disagree-about-c17-constexpr-lambda-captures constexpr auto i = ij / STRICTNESSES.size(); From 27a0c37d7773d4392065cb45a72c8681d65847b9 Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Tue, 15 Jan 2019 00:41:35 +0300 Subject: [PATCH 04/86] Update Join.h --- dbms/src/Interpreters/Join.h | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/dbms/src/Interpreters/Join.h b/dbms/src/Interpreters/Join.h index 38b05fb2de1..43de3092409 100644 --- a/dbms/src/Interpreters/Join.h +++ b/dbms/src/Interpreters/Join.h @@ -403,6 +403,7 @@ public: bool dispatch(Func && func) { if (overwrite) + { return static_for<0, KINDS.size()>([&](auto i) { if (kind == KINDS[i] && strictness == ASTTableJoin::Strictness::Any) @@ -418,7 +419,9 @@ public: } return false; }); + } else + { return static_for<0, KINDS.size() * STRICTNESSES.size()>([&](auto ij) { // NOTE: Avoid using nested static loop as GCC and CLANG have bugs in different ways @@ -438,6 +441,7 @@ public: } return false; }); + } } template From b2690d8eed51bf172e3a7fffe1108186d86930b4 Mon Sep 17 00:00:00 2001 From: Odin Hultgren Van Der Horst Date: Wed, 16 Jan 2019 14:49:41 +0100 Subject: [PATCH 05/86] Added zstd support for kafka --- contrib/librdkafka-cmake/CMakeLists.txt | 1 + contrib/librdkafka-cmake/config.h | 3 +++ 2 files changed, 4 insertions(+) diff --git a/contrib/librdkafka-cmake/CMakeLists.txt b/contrib/librdkafka-cmake/CMakeLists.txt index 115c916e9f4..fb27895c3da 100644 --- a/contrib/librdkafka-cmake/CMakeLists.txt +++ b/contrib/librdkafka-cmake/CMakeLists.txt @@ -2,6 +2,7 @@ set(RDKAFKA_SOURCE_DIR ${CMAKE_SOURCE_DIR}/contrib/librdkafka/src) set(SRCS ${RDKAFKA_SOURCE_DIR}/crc32c.c + ${RDKAFKA_SOURCE_DIR}/rdkafka_zstd.c ${RDKAFKA_SOURCE_DIR}/rdaddr.c ${RDKAFKA_SOURCE_DIR}/rdavl.c ${RDKAFKA_SOURCE_DIR}/rdbuf.c diff --git a/contrib/librdkafka-cmake/config.h b/contrib/librdkafka-cmake/config.h index 2ffc5a497ae..0de2ca1a383 100644 --- a/contrib/librdkafka-cmake/config.h +++ b/contrib/librdkafka-cmake/config.h @@ -10,6 +10,7 @@ #define ENABLE_SHAREDPTR_DEBUG 0 #define ENABLE_LZ4_EXT 1 #define ENABLE_SSL 1 +//#define WITH_HDRHISTOGRAM 0 //#define ENABLE_SASL 1 #define MKL_APP_NAME "librdkafka" #define MKL_APP_DESC_ONELINE "The Apache Kafka C/C++ library" @@ -51,6 +52,8 @@ //#define WITH_PLUGINS 1 // zlib #define WITH_ZLIB 1 +// zstd +#define WITH_ZSTD 1 // WITH_SNAPPY #define WITH_SNAPPY 1 // WITH_SOCKEM From addc9a43b014702c6dd8f51ccb6e55293988cd2a Mon Sep 17 00:00:00 2001 From: Alexey Zatelepin Date: Thu, 17 Jan 2019 18:52:29 +0300 Subject: [PATCH 06/86] remove unused IAST field --- dbms/src/Interpreters/InterpreterCreateQuery.cpp | 2 -- dbms/src/Parsers/IAST.h | 2 -- 2 files changed, 4 deletions(-) diff --git a/dbms/src/Interpreters/InterpreterCreateQuery.cpp b/dbms/src/Interpreters/InterpreterCreateQuery.cpp index 7fa74884d5a..429c257b8d5 100644 --- a/dbms/src/Interpreters/InterpreterCreateQuery.cpp +++ b/dbms/src/Interpreters/InterpreterCreateQuery.cpp @@ -345,7 +345,6 @@ ASTPtr InterpreterCreateQuery::formatColumns(const NamesAndTypesList & columns) ParserIdentifierWithOptionalParameters storage_p; column_declaration->type = parseQuery(storage_p, pos, end, "data type", 0); - column_declaration->type->owned_string = type_name; columns_list->children.emplace_back(column_declaration); } @@ -369,7 +368,6 @@ ASTPtr InterpreterCreateQuery::formatColumns(const ColumnsDescription & columns) ParserIdentifierWithOptionalParameters storage_p; column_declaration->type = parseQuery(storage_p, type_name_pos, type_name_end, "data type", 0); - column_declaration->type->owned_string = type_name; const auto defaults_it = columns.defaults.find(column.name); if (defaults_it != std::end(columns.defaults)) diff --git a/dbms/src/Parsers/IAST.h b/dbms/src/Parsers/IAST.h index 703fdfc2765..a597d40080f 100644 --- a/dbms/src/Parsers/IAST.h +++ b/dbms/src/Parsers/IAST.h @@ -56,8 +56,6 @@ public: ASTs children; StringRange range; - /// This pointer does not allow it to be deleted while the range refers to it. - StringPtr owned_string; SemanticPtr semantic; virtual ~IAST() = default; From 8bc6eef97c6a2704fcd88c24002448cef291ba5e Mon Sep 17 00:00:00 2001 From: Danila Kutenin Date: Wed, 23 Jan 2019 11:32:43 +0300 Subject: [PATCH 07/86] Docs for multi string search --- .../functions/string_search_functions.md | 18 ++++++++++++++++++ .../functions/string_search_functions.md | 15 +++++++++++++++ 2 files changed, 33 insertions(+) diff --git a/docs/en/query_language/functions/string_search_functions.md b/docs/en/query_language/functions/string_search_functions.md index ea5b1ecf8ff..ced657da2ed 100644 --- a/docs/en/query_language/functions/string_search_functions.md +++ b/docs/en/query_language/functions/string_search_functions.md @@ -16,6 +16,24 @@ The same as `position`, but the position is returned in Unicode code points. Wor For a case-insensitive search, use the function `positionCaseInsensitiveUTF8`. +## multiPosition(haystack, [needle_1, needle_2, ..., needle_n]) + +The same as `position`, but returns `Array` of the `position`s for all `needle_i`. + +For a case-insensitive search or/and in UTF-8 format use functions `multiPositionCaseInsensitive, multiPositionUTF8, multiPositionCaseInsensitiveUTF8`. + +## firstMatch(haystack, [needle_1, needle_2, ..., needle_n]) + +Returns the index `i` (starting from 1) of the first found `needle_i` in the string `haystack` and 0 otherwise. + +For a case-insensitive search or/and in UTF-8 format use functions `firstMatchCaseInsensitive, firstMatchUTF8, firstMatchCaseInsensitiveUTF8`. + +## multiSearch(haystack, [needle_1, needle_2, ..., needle_n]) + +Returns 1, if at least one string `needle_i` matches the string `haystack` and 0 otherwise. + +For a case-insensitive search or/and in UTF-8 format use functions `multiSearchCaseInsensitive, multiSearchUTF8, multiSearchCaseInsensitiveUTF8`. + ## match(haystack, pattern) Checks whether the string matches the `pattern` regular expression. A `re2` regular expression. The [syntax](https://github.com/google/re2/wiki/Syntax) of the `re2` regular expressions is more limited than the syntax of the Perl regular expressions. diff --git a/docs/ru/query_language/functions/string_search_functions.md b/docs/ru/query_language/functions/string_search_functions.md index 031082c6659..3bd5e77fac2 100644 --- a/docs/ru/query_language/functions/string_search_functions.md +++ b/docs/ru/query_language/functions/string_search_functions.md @@ -14,6 +14,21 @@ Для поиска без учета регистра используйте функцию `positionCaseInsensitiveUTF8`. +## multiPosition(haystack, [needle_1, needle_2, ..., needle_n]) +Так же, как и `position`, только возвращает `Array` первых вхождений. + +Для поиска без учета регистра и/или в кодировке UTF-8 используйте функции `multiPositionCaseInsensitive, multiPositionUTF8, multiPositionCaseInsensitiveUTF8`. + +## firstMatch(haystack, [needle_1, needle_2, ..., needle_n]) +Возвращает индекс `i` (нумерация с единицы) первой найденной строки `needle_i` в строке `haystack` и 0 иначе. + +Для поиска без учета регистра и/или в кодировке UTF-8 используйте функции `firstMatchCaseInsensitive, firstMatchUTF8, firstMatchCaseInsensitiveUTF8`. + +## multiSearch(haystack, [needle_1, needle_2, ..., needle_n]) +Возвращает 1, если хотя бы одна подстрока `needle_i` нашлась в строке `haystack` и 0 иначе. + +Для поиска без учета регистра и/или в кодировке UTF-8 используйте функции `multiSearchCaseInsensitive, multiSearchUTF8, multiSearchCaseInsensitiveUTF8`. + ## match(haystack, pattern) Проверка строки на соответствие регулярному выражению pattern. Регулярное выражение **re2**. Синтаксис регулярных выражений **re2** является более ограниченным по сравнению с регулярными выражениями **Perl** ([подробнее](https://github.com/google/re2/wiki/Syntax)). Возвращает 0 (если не соответствует) или 1 (если соответствует). From f6225b9c321f37bff9f0a128efd7267c81750339 Mon Sep 17 00:00:00 2001 From: Ivan Blinkov Date: Wed, 23 Jan 2019 11:57:42 +0300 Subject: [PATCH 08/86] Restore Upcoming Events README.md section [C++ ClickHouse and CatBoost Sprints](https://events.yandex.ru/events/ClickHouse/2-feb-2019/) in Moscow on February 2. --- README.md | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/README.md b/README.md index 0e9974f763f..f496e32b905 100644 --- a/README.md +++ b/README.md @@ -10,3 +10,7 @@ ClickHouse is an open-source column-oriented database management system that all * [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. * You can also [fill this form](https://forms.yandex.com/surveys/meet-yandex-clickhouse-team/) to meet Yandex ClickHouse team in person. + +## Upcoming Events + +* [C++ ClickHouse and CatBoost Sprints](https://events.yandex.ru/events/ClickHouse/2-feb-2019/) in Moscow on February 2. From 0ed41dc4c011734b69f924d52c1cd520696019e4 Mon Sep 17 00:00:00 2001 From: Ivan Blinkov Date: Wed, 23 Jan 2019 12:12:29 +0300 Subject: [PATCH 09/86] Add C++ ClickHouse and CatBoost Sprints link to website front page (#4124) --- website/index.html | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/website/index.html b/website/index.html index 8e2f2427057..9961d229320 100644 --- a/website/index.html +++ b/website/index.html @@ -92,6 +92,11 @@
+
+
+ C++ ClickHouse and CatBoost Sprints in Moscow on February 2 +
+

ClickHouse. Just makes you think faster.

From 95f2f3faebbee6e1c5fbed940efb246114330419 Mon Sep 17 00:00:00 2001 From: Odin Hultgren Van Der Horst Date: Wed, 23 Jan 2019 11:48:30 +0100 Subject: [PATCH 10/86] Removed old comment --- contrib/librdkafka-cmake/config.h | 1 - 1 file changed, 1 deletion(-) diff --git a/contrib/librdkafka-cmake/config.h b/contrib/librdkafka-cmake/config.h index 0de2ca1a383..979d66cdca5 100644 --- a/contrib/librdkafka-cmake/config.h +++ b/contrib/librdkafka-cmake/config.h @@ -10,7 +10,6 @@ #define ENABLE_SHAREDPTR_DEBUG 0 #define ENABLE_LZ4_EXT 1 #define ENABLE_SSL 1 -//#define WITH_HDRHISTOGRAM 0 //#define ENABLE_SASL 1 #define MKL_APP_NAME "librdkafka" #define MKL_APP_DESC_ONELINE "The Apache Kafka C/C++ library" From a96f638cb091bc66e0cb3d18e5a2ee319708be93 Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 23 Jan 2019 14:00:06 +0300 Subject: [PATCH 11/86] Fixes in performance-test: more correct json, choose main_metric by default, fix params accumulation bug --- .../performance-test/PerformanceTest.cpp | 22 ++++++++++++------- 1 file changed, 14 insertions(+), 8 deletions(-) diff --git a/dbms/programs/performance-test/PerformanceTest.cpp b/dbms/programs/performance-test/PerformanceTest.cpp index 27bf986fc1b..e91365aeade 100644 --- a/dbms/programs/performance-test/PerformanceTest.cpp +++ b/dbms/programs/performance-test/PerformanceTest.cpp @@ -46,6 +46,8 @@ namespace fs = boost::filesystem; using String = std::string; const String FOUR_SPACES = " "; +const std::regex QUOTE_REGEX{"\""}; +const std::regex NEW_LINE{"\n"}; namespace DB { @@ -80,7 +82,7 @@ public: bool reserved = (value[0] == '[' || value[0] == '{' || value == "null"); if (!reserved && wrap) - value = '"' + value + '"'; + value = '"' + std::regex_replace(value, NEW_LINE, "\\n") + '"'; content[key] = value; } @@ -579,7 +581,8 @@ private: using Paths = std::vector; using StringToVector = std::map>; - StringToVector substitutions; + using StringToMap = std::map; + StringToMap substitutions; using StringKeyValue = std::map; std::vector substitutions_maps; @@ -933,13 +936,13 @@ private: { /// Make "subconfig" of inner xml block ConfigurationPtr substitutions_view(test_config->createView("substitutions")); - constructSubstitutions(substitutions_view, substitutions); + constructSubstitutions(substitutions_view, substitutions[test_name]); auto queries_pre_format = queries; queries.clear(); for (const auto & query : queries_pre_format) { - auto formatted = formatQueries(query, substitutions); + auto formatted = formatQueries(query, substitutions[test_name]); queries.insert(queries.end(), formatted.begin(), formatted.end()); } } @@ -994,6 +997,9 @@ private: } else { + if (metrics.empty()) + throw DB::Exception("You shoud specify at least one metric", DB::ErrorCodes::BAD_ARGUMENTS); + main_metric = metrics[0]; if (lite_output) throw DB::Exception("Specify main_metric for lite output", DB::ErrorCodes::BAD_ARGUMENTS); } @@ -1219,11 +1225,11 @@ public: json_output.set("test_name", test_name); json_output.set("main_metric", main_metric); - if (substitutions.size()) + if (substitutions[test_name].size()) { JSONString json_parameters(2); /// here, 2 is the size of \t padding - for (auto it = substitutions.begin(); it != substitutions.end(); ++it) + for (auto it = substitutions[test_name].begin(); it != substitutions[test_name].end(); ++it) { String parameter = it->first; std::vector values = it->second; @@ -1231,7 +1237,7 @@ public: String array_string = "["; for (size_t i = 0; i != values.size(); ++i) { - array_string += '"' + values[i] + '"'; + array_string += '"' + std::regex_replace(values[i], QUOTE_REGEX, "\\\"") + '"'; if (i != values.size() - 1) { array_string += ", "; @@ -1257,7 +1263,7 @@ public: JSONString runJSON; - runJSON.set("query", queries[query_index]); + runJSON.set("query", std::regex_replace(queries[query_index], QUOTE_REGEX, "\\\"")); if (!statistics.exception.empty()) runJSON.set("exception", statistics.exception); From 5154814bf941eb5b9b3b6262136d6f6fc27294fd Mon Sep 17 00:00:00 2001 From: Alexey Zatelepin Date: Fri, 18 Jan 2019 19:30:35 +0300 Subject: [PATCH 12/86] try to get rid of IAST::range #4058 --- dbms/src/Interpreters/ActionsVisitor.cpp | 15 +++++----- dbms/src/Interpreters/ActionsVisitor.h | 10 ++----- dbms/src/Interpreters/ExpressionAnalyzer.cpp | 4 +-- .../Interpreters/InterpreterCreateQuery.cpp | 14 ++++----- .../LogicalExpressionsOptimizer.cpp | 8 ----- dbms/src/Interpreters/PreparedSets.h | 20 +++++++++++++ dbms/src/Interpreters/QueryNormalizer.h | 1 + dbms/src/Interpreters/executeQuery.cpp | 14 +++++---- dbms/src/Interpreters/getClusterName.cpp | 14 +++++++-- dbms/src/Parsers/ASTIdentifier.h | 1 - dbms/src/Parsers/ExpressionElementParsers.cpp | 6 ---- dbms/src/Parsers/ExpressionListParsers.cpp | 30 ------------------- dbms/src/Parsers/IAST.h | 8 +---- dbms/src/Parsers/IParserBase.cpp | 2 -- dbms/src/Storages/MergeTree/KeyCondition.cpp | 21 +++++++------ .../MergeTree/MergeTreeWhereOptimizer.cpp | 2 +- .../ReplicatedMergeTreeMutationEntry.h | 1 + dbms/src/Storages/SelectQueryInfo.h | 12 +------- 18 files changed, 75 insertions(+), 108 deletions(-) create mode 100644 dbms/src/Interpreters/PreparedSets.h diff --git a/dbms/src/Interpreters/ActionsVisitor.cpp b/dbms/src/Interpreters/ActionsVisitor.cpp index 8d679a4cb58..b53d166ddb4 100644 --- a/dbms/src/Interpreters/ActionsVisitor.cpp +++ b/dbms/src/Interpreters/ActionsVisitor.cpp @@ -131,7 +131,7 @@ void makeExplicitSet(const ASTFunction * node, const Block & sample_block, bool SetPtr set = std::make_shared(size_limits, create_ordered_set); set->createFromAST(set_element_types, elements_ast, context); - prepared_sets[right_arg->range] = std::move(set); + prepared_sets[right_arg->getTreeHash()] = std::move(set); } static String getUniqueName(const Block & block, const String & prefix) @@ -380,12 +380,12 @@ void ActionsVisitor::visit(const ASTPtr & ast) /// Select the name in the next cycle. argument_names.emplace_back(); } - else if (prepared_sets.count(child->range) && functionIsInOrGlobalInOperator(node->name) && arg == 1) + else if (functionIsInOrGlobalInOperator(node->name) && arg == 1 && prepared_sets.count(child->getTreeHash())) { ColumnWithTypeAndName column; column.type = std::make_shared(); - const SetPtr & set = prepared_sets[child->range]; + const SetPtr & set = prepared_sets[child->getTreeHash()]; /// If the argument is a set given by an enumeration of values (so, the set was already built), give it a unique name, /// so that sets with the same literal representation do not fuse together (they can have different types). @@ -530,9 +530,10 @@ void ActionsVisitor::makeSet(const ASTFunction * node, const Block & sample_bloc */ const IAST & args = *node->arguments; const ASTPtr & arg = args.children.at(1); + IAST::Hash tree_hash = arg->getTreeHash(); /// Already converted. - if (prepared_sets.count(arg->range)) + if (prepared_sets.count(tree_hash)) return; /// If the subquery or table name for SELECT. @@ -552,7 +553,7 @@ void ActionsVisitor::makeSet(const ASTFunction * node, const Block & sample_bloc if (storage_set) { - prepared_sets[arg->range] = storage_set->getSet(); + prepared_sets[tree_hash] = storage_set->getSet(); return; } } @@ -566,7 +567,7 @@ void ActionsVisitor::makeSet(const ASTFunction * node, const Block & sample_bloc /// If you already created a Set with the same subquery / table. if (subquery_for_set.set) { - prepared_sets[arg->range] = subquery_for_set.set; + prepared_sets[tree_hash] = subquery_for_set.set; return; } @@ -612,7 +613,7 @@ void ActionsVisitor::makeSet(const ASTFunction * node, const Block & sample_bloc } subquery_for_set.set = set; - prepared_sets[arg->range] = set; + prepared_sets[tree_hash] = set; } else { diff --git a/dbms/src/Interpreters/ActionsVisitor.h b/dbms/src/Interpreters/ActionsVisitor.h index 805b0ec02c4..d4a6d388fbc 100644 --- a/dbms/src/Interpreters/ActionsVisitor.h +++ b/dbms/src/Interpreters/ActionsVisitor.h @@ -1,6 +1,7 @@ #pragma once -#include +#include +#include #include @@ -10,13 +11,6 @@ namespace DB class Context; class ASTFunction; - -class Set; -using SetPtr = std::shared_ptr; -/// Will compare sets by their position in query string. It's possible because IAST::clone() doesn't chane IAST::range. -/// It should be taken into account when we want to change AST part which contains sets. -using PreparedSets = std::unordered_map; - class Join; using JoinPtr = std::shared_ptr; diff --git a/dbms/src/Interpreters/ExpressionAnalyzer.cpp b/dbms/src/Interpreters/ExpressionAnalyzer.cpp index 8c1e13815ec..6d39a1303a4 100644 --- a/dbms/src/Interpreters/ExpressionAnalyzer.cpp +++ b/dbms/src/Interpreters/ExpressionAnalyzer.cpp @@ -279,7 +279,7 @@ void ExpressionAnalyzer::tryMakeSetForIndexFromSubquery(const ASTPtr & subquery_ return; } - prepared_sets[subquery_or_table_name->range] = std::move(set); + prepared_sets[subquery_or_table_name->getTreeHash()] = std::move(set); } @@ -308,7 +308,7 @@ void ExpressionAnalyzer::makeSetsForIndexImpl(const ASTPtr & node, const Block & { const ASTPtr & arg = args.children.at(1); - if (!prepared_sets.count(arg->range)) /// Not already prepared. + if (!prepared_sets.count(arg->getTreeHash())) /// Not already prepared. { if (typeid_cast(arg.get()) || isIdentifier(arg)) { diff --git a/dbms/src/Interpreters/InterpreterCreateQuery.cpp b/dbms/src/Interpreters/InterpreterCreateQuery.cpp index 429c257b8d5..53bf85d2099 100644 --- a/dbms/src/Interpreters/InterpreterCreateQuery.cpp +++ b/dbms/src/Interpreters/InterpreterCreateQuery.cpp @@ -339,11 +339,10 @@ ASTPtr InterpreterCreateQuery::formatColumns(const NamesAndTypesList & columns) const auto column_declaration = std::make_shared(); column_declaration->name = column.name; - StringPtr type_name = std::make_shared(column.type->getName()); - auto pos = type_name->data(); - const auto end = pos + type_name->size(); - ParserIdentifierWithOptionalParameters storage_p; + String type_name = column.type->getName(); + auto pos = type_name.data(); + const auto end = pos + type_name.size(); column_declaration->type = parseQuery(storage_p, pos, end, "data type", 0); columns_list->children.emplace_back(column_declaration); } @@ -362,11 +361,10 @@ ASTPtr InterpreterCreateQuery::formatColumns(const ColumnsDescription & columns) column_declaration->name = column.name; - StringPtr type_name = std::make_shared(column.type->getName()); - auto type_name_pos = type_name->data(); - const auto type_name_end = type_name_pos + type_name->size(); - ParserIdentifierWithOptionalParameters storage_p; + String type_name = column.type->getName(); + auto type_name_pos = type_name.data(); + const auto type_name_end = type_name_pos + type_name.size(); column_declaration->type = parseQuery(storage_p, type_name_pos, type_name_end, "data type", 0); const auto defaults_it = columns.defaults.find(column.name); diff --git a/dbms/src/Interpreters/LogicalExpressionsOptimizer.cpp b/dbms/src/Interpreters/LogicalExpressionsOptimizer.cpp index 5226c96dce6..d7f04ff2a25 100644 --- a/dbms/src/Interpreters/LogicalExpressionsOptimizer.cpp +++ b/dbms/src/Interpreters/LogicalExpressionsOptimizer.cpp @@ -228,17 +228,10 @@ void LogicalExpressionsOptimizer::addInExpression(const DisjunctiveEqualityChain /// Construct a list of literals `x1, ..., xN` from the string `expr = x1 OR ... OR expr = xN` ASTPtr value_list = std::make_shared(); - const char * min_range_first = nullptr; - const char * max_range_second = nullptr; for (const auto function : equality_functions) { const auto & operands = getFunctionOperands(function); value_list->children.push_back(operands[1]); - /// Get range min/max from all literals x1...xN, which will be used as tuple_functions' range - if (min_range_first == nullptr || min_range_first > operands[1]->range.first) - min_range_first = operands[1]->range.first; - if (max_range_second == nullptr || max_range_second < operands[1]->range.second) - max_range_second = operands[1]->range.second; } /// Sort the literals so that they are specified in the same order in the IN expression. @@ -260,7 +253,6 @@ void LogicalExpressionsOptimizer::addInExpression(const DisjunctiveEqualityChain auto tuple_function = std::make_shared(); tuple_function->name = "tuple"; - tuple_function->range = StringRange(min_range_first, max_range_second); tuple_function->arguments = value_list; tuple_function->children.push_back(tuple_function->arguments); diff --git a/dbms/src/Interpreters/PreparedSets.h b/dbms/src/Interpreters/PreparedSets.h new file mode 100644 index 00000000000..4276e62a239 --- /dev/null +++ b/dbms/src/Interpreters/PreparedSets.h @@ -0,0 +1,20 @@ +#pragma once + +#include +#include +#include + +namespace DB +{ + +struct ASTHalfHash +{ + UInt64 operator()(const IAST::Hash & ast_hash) const { return ast_hash.first; } +}; + +class Set; +using SetPtr = std::shared_ptr; + +using PreparedSets = std::unordered_map; + +} diff --git a/dbms/src/Interpreters/QueryNormalizer.h b/dbms/src/Interpreters/QueryNormalizer.h index 3e55e0253e6..57f4645569a 100644 --- a/dbms/src/Interpreters/QueryNormalizer.h +++ b/dbms/src/Interpreters/QueryNormalizer.h @@ -3,6 +3,7 @@ #include #include #include +#include namespace DB { diff --git a/dbms/src/Interpreters/executeQuery.cpp b/dbms/src/Interpreters/executeQuery.cpp index 3911e437fa6..728225fdb80 100644 --- a/dbms/src/Interpreters/executeQuery.cpp +++ b/dbms/src/Interpreters/executeQuery.cpp @@ -150,7 +150,7 @@ static std::tuple executeQueryImpl( ParserQuery parser(end, settings.enable_debug_queries); ASTPtr ast; - size_t query_size; + const char * query_end; /// Don't limit the size of internal queries. size_t max_query_size = 0; @@ -162,10 +162,11 @@ static std::tuple executeQueryImpl( /// TODO Parser should fail early when max_query_size limit is reached. ast = parseQuery(parser, begin, end, "", max_query_size); - /// Copy query into string. It will be written to log and presented in processlist. If an INSERT query, string will not include data to insertion. - if (!(begin <= ast->range.first && ast->range.second <= end)) - throw Exception("Unexpected behavior: AST chars range is not inside source range", ErrorCodes::LOGICAL_ERROR); - query_size = ast->range.second - begin; + const auto * insert_query = dynamic_cast(ast.get()); + if (insert_query && insert_query->data) + query_end = insert_query->data; + else + query_end = end; } catch (...) { @@ -180,7 +181,8 @@ static std::tuple executeQueryImpl( throw; } - String query(begin, query_size); + /// Copy query into string. It will be written to log and presented in processlist. If an INSERT query, string will not include data to insertion. + String query(begin, query_end); BlockIO res; try diff --git a/dbms/src/Interpreters/getClusterName.cpp b/dbms/src/Interpreters/getClusterName.cpp index 4312da3e1a6..bc32e3dbea7 100644 --- a/dbms/src/Interpreters/getClusterName.cpp +++ b/dbms/src/Interpreters/getClusterName.cpp @@ -24,12 +24,22 @@ std::string getClusterName(const IAST & node) if (const ASTLiteral * ast_lit = typeid_cast(&node)) return ast_lit->value.safeGet(); + /// A hack to support hyphens in cluster names. if (const ASTFunction * ast_func = typeid_cast(&node)) { - if (!ast_func->range.first || !ast_func->range.second) + if (ast_func->name != "minus" || !ast_func->arguments || ast_func->arguments->children.size() < 2) throw Exception("Illegal expression instead of cluster name.", ErrorCodes::BAD_ARGUMENTS); - return String(ast_func->range.first, ast_func->range.second); + String name; + for (const auto & arg : ast_func->arguments->children) + { + if (name.empty()) + name += getClusterName(*arg); + else + name += "-" + getClusterName(*arg); + } + + return name; } throw Exception("Illegal expression instead of cluster name.", ErrorCodes::BAD_ARGUMENTS); diff --git a/dbms/src/Parsers/ASTIdentifier.h b/dbms/src/Parsers/ASTIdentifier.h index 4805a1f4fd5..a2b7f3ea555 100644 --- a/dbms/src/Parsers/ASTIdentifier.h +++ b/dbms/src/Parsers/ASTIdentifier.h @@ -19,7 +19,6 @@ public: : name(name_) , special(false) { - range = StringRange(name.data(), name.data() + name.size()); } /** Get the text that identifies this element. */ diff --git a/dbms/src/Parsers/ExpressionElementParsers.cpp b/dbms/src/Parsers/ExpressionElementParsers.cpp index 9ba848dbe1a..a1fab382c36 100644 --- a/dbms/src/Parsers/ExpressionElementParsers.cpp +++ b/dbms/src/Parsers/ExpressionElementParsers.cpp @@ -672,8 +672,6 @@ bool ParserRightExpression::parseImpl(Pos & pos, ASTPtr & node, Expected & expec bool ParserExtractExpression::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) { - auto begin = pos; - if (!ParserKeyword("EXTRACT").ignore(pos, expected)) return false; @@ -734,14 +732,10 @@ bool ParserExtractExpression::parseImpl(Pos & pos, ASTPtr & node, Expected & exp auto function = std::make_shared(); auto exp_list = std::make_shared(); - function->range.first = begin->begin; - function->range.second = pos->begin; function->name = function_name; //"toYear"; function->arguments = exp_list; function->children.push_back(exp_list); exp_list->children.push_back(expr); - exp_list->range.first = begin->begin; - exp_list->range.second = pos->begin; node = function; return true; diff --git a/dbms/src/Parsers/ExpressionListParsers.cpp b/dbms/src/Parsers/ExpressionListParsers.cpp index 9f17b3935f5..c732ce4f38e 100644 --- a/dbms/src/Parsers/ExpressionListParsers.cpp +++ b/dbms/src/Parsers/ExpressionListParsers.cpp @@ -138,7 +138,6 @@ static bool parseOperator(IParser::Pos & pos, const char * op, Expected & expect bool ParserLeftAssociativeBinaryOperatorList::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) { bool first = true; - Pos begin = pos; while (1) { @@ -174,16 +173,12 @@ bool ParserLeftAssociativeBinaryOperatorList::parseImpl(Pos & pos, ASTPtr & node return false; /// the first argument of the function is the previous element, the second is the next one - function->range.first = begin->begin; - function->range.second = pos->begin; function->name = it[1]; function->arguments = exp_list; function->children.push_back(exp_list); exp_list->children.push_back(node); exp_list->children.push_back(elem); - exp_list->range.first = begin->begin; - exp_list->range.second = pos->begin; /** special exception for the access operator to the element of the array `x[y]`, which * contains the infix part '[' and the suffix ''] '(specified as' [') @@ -243,8 +238,6 @@ bool ParserBetweenExpression::parseImpl(Pos & pos, ASTPtr & node, Expected & exp ASTPtr left; ASTPtr right; - Pos begin = pos; - if (!elem_parser.parse(pos, subject, expected)) return false; @@ -279,14 +272,10 @@ bool ParserBetweenExpression::parseImpl(Pos & pos, ASTPtr & node, Expected & exp args_le->children.emplace_back(subject); args_le->children.emplace_back(right); - f_ge->range.first = begin->begin; - f_ge->range.second = pos->begin; f_ge->name = "greaterOrEquals"; f_ge->arguments = args_ge; f_ge->children.emplace_back(f_ge->arguments); - f_le->range.first = begin->begin; - f_le->range.second = pos->begin; f_le->name = "lessOrEquals"; f_le->arguments = args_le; f_le->children.emplace_back(f_le->arguments); @@ -294,8 +283,6 @@ bool ParserBetweenExpression::parseImpl(Pos & pos, ASTPtr & node, Expected & exp args_and->children.emplace_back(f_ge); args_and->children.emplace_back(f_le); - f_and->range.first = begin->begin; - f_and->range.second = pos->begin; f_and->name = "and"; f_and->arguments = args_and; f_and->children.emplace_back(f_and->arguments); @@ -315,8 +302,6 @@ bool ParserTernaryOperatorExpression::parseImpl(Pos & pos, ASTPtr & node, Expect ASTPtr elem_then; ASTPtr elem_else; - Pos begin = pos; - if (!elem_parser.parse(pos, elem_cond, expected)) return false; @@ -339,8 +324,6 @@ bool ParserTernaryOperatorExpression::parseImpl(Pos & pos, ASTPtr & node, Expect /// function arguments auto exp_list = std::make_shared(); - function->range.first = begin->begin; - function->range.second = pos->begin; function->name = "if"; function->arguments = exp_list; function->children.push_back(exp_list); @@ -348,8 +331,6 @@ bool ParserTernaryOperatorExpression::parseImpl(Pos & pos, ASTPtr & node, Expect exp_list->children.push_back(elem_cond); exp_list->children.push_back(elem_then); exp_list->children.push_back(elem_else); - exp_list->range.first = begin->begin; - exp_list->range.second = pos->begin; node = function; } @@ -423,7 +404,6 @@ bool ParserLambdaExpression::parseImpl(Pos & pos, ASTPtr & node, Expected & expe bool ParserPrefixUnaryOperatorExpression::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) { /// try to find any of the valid operators - Pos begin = pos; const char ** it; for (it = operators; *it; it += 2) { @@ -471,15 +451,11 @@ bool ParserPrefixUnaryOperatorExpression::parseImpl(Pos & pos, ASTPtr & node, Ex /// function arguments auto exp_list = std::make_shared(); - function->range.first = begin->begin; - function->range.second = pos->begin; function->name = it[1]; function->arguments = exp_list; function->children.push_back(exp_list); exp_list->children.push_back(elem); - exp_list->range.first = begin->begin; - exp_list->range.second = pos->begin; node = function; } @@ -595,8 +571,6 @@ bool ParserNullityChecking::parseImpl(Pos & pos, ASTPtr & node, Expected & expec bool ParserIntervalOperatorExpression::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) { - Pos begin = pos; - /// If no INTERVAL keyword, go to nested parser. if (!ParserKeyword("INTERVAL").ignore(pos, expected)) return next_parser.parse(pos, node, expected); @@ -620,15 +594,11 @@ bool ParserIntervalOperatorExpression::parseImpl(Pos & pos, ASTPtr & node, Expec auto exp_list = std::make_shared(); /// the first argument of the function is the previous element, the second is the next one - function->range.first = begin->begin; - function->range.second = pos->begin; function->name = function_name; function->arguments = exp_list; function->children.push_back(exp_list); exp_list->children.push_back(expr); - exp_list->range.first = begin->begin; - exp_list->range.second = pos->begin; node = function; return true; diff --git a/dbms/src/Parsers/IAST.h b/dbms/src/Parsers/IAST.h index a597d40080f..3c4b9dec007 100644 --- a/dbms/src/Parsers/IAST.h +++ b/dbms/src/Parsers/IAST.h @@ -7,7 +7,6 @@ #include #include -#include #include @@ -54,7 +53,6 @@ class IAST : public std::enable_shared_from_this { public: ASTs children; - StringRange range; SemanticPtr semantic; @@ -209,11 +207,7 @@ public: virtual void formatImpl(const FormatSettings & /*settings*/, FormatState & /*state*/, FormatStateStacked /*frame*/) const { - throw Exception("Unknown element in AST: " + getID() - + ((range.first && (range.second > range.first)) - ? " '" + std::string(range.first, range.second - range.first) + "'" - : ""), - ErrorCodes::UNKNOWN_ELEMENT_IN_AST); + throw Exception("Unknown element in AST: " + getID(), ErrorCodes::UNKNOWN_ELEMENT_IN_AST); } void cloneChildren(); diff --git a/dbms/src/Parsers/IParserBase.cpp b/dbms/src/Parsers/IParserBase.cpp index dffb1d7597d..ddbac8e92ee 100644 --- a/dbms/src/Parsers/IParserBase.cpp +++ b/dbms/src/Parsers/IParserBase.cpp @@ -22,8 +22,6 @@ bool IParserBase::parse(Pos & pos, ASTPtr & node, Expected & expected) node = nullptr; pos = begin; } - else if (node) - node->range = StringRange(begin, pos); return res; } diff --git a/dbms/src/Storages/MergeTree/KeyCondition.cpp b/dbms/src/Storages/MergeTree/KeyCondition.cpp index d386b865231..87d9f8def3d 100644 --- a/dbms/src/Storages/MergeTree/KeyCondition.cpp +++ b/dbms/src/Storages/MergeTree/KeyCondition.cpp @@ -635,18 +635,18 @@ bool KeyCondition::atomFromAST(const ASTPtr & node, const Context & context, Blo DataTypePtr key_expr_type; /// Type of expression containing key column size_t key_arg_pos; /// Position of argument with key column (non-const argument) - size_t key_column_num; /// Number of a key column (inside key_column_names array) + size_t key_column_num = -1; /// Number of a key column (inside key_column_names array) MonotonicFunctionsChain chain; bool is_set_const = false; bool is_constant_transformed = false; - if (prepared_sets.count(args[1]->range) - && tryPrepareSetIndex(args[0], context, out, prepared_sets[args[1]->range], key_column_num)) - { - key_arg_pos = 0; - is_set_const = true; - } - else if (getConstant(args[1], block_with_constants, const_value, const_type) + if (prepared_sets.count(args[1]->getTreeHash()) + && tryPrepareSetIndex(args[0], context, out, prepared_sets[args[1]->getTreeHash()], key_column_num)) + { + key_arg_pos = 0; + is_set_const = true; + } + else if (getConstant(args[1], block_with_constants, const_value, const_type) && isKeyPossiblyWrappedByMonotonicFunctions(args[0], context, key_column_num, key_expr_type, chain)) { key_arg_pos = 0; @@ -671,6 +671,9 @@ bool KeyCondition::atomFromAST(const ASTPtr & node, const Context & context, Blo else return false; + if (key_column_num == static_cast(-1)) + throw Exception("`key_column_num` wasn't initialized. It is a bug.", ErrorCodes::LOGICAL_ERROR); + std::string func_name = func->name; /// Transformed constant must weaken the condition, for example "x > 5" must weaken to "round(x) >= 5" @@ -1015,7 +1018,7 @@ bool KeyCondition::mayBeTrueInParallelogram(const std::vector & parallelo { auto in_func = typeid_cast(element.in_function.get()); const ASTs & args = typeid_cast(*in_func->arguments).children; - PreparedSets::const_iterator it = prepared_sets.find(args[1]->range); + PreparedSets::const_iterator it = prepared_sets.find(args[1]->getTreeHash()); if (in_func && it != prepared_sets.end()) { rpn_stack.emplace_back(element.set_index->mayBeTrueInRange(parallelogram, data_types)); diff --git a/dbms/src/Storages/MergeTree/MergeTreeWhereOptimizer.cpp b/dbms/src/Storages/MergeTree/MergeTreeWhereOptimizer.cpp index 850c696b266..3eb9576b1c0 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeWhereOptimizer.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeWhereOptimizer.cpp @@ -334,7 +334,7 @@ bool MergeTreeWhereOptimizer::isPrimaryKeyAtom(const IAST * const ast) const if ((primary_key_columns.count(first_arg_name) && isConstant(args[1])) || (primary_key_columns.count(second_arg_name) && isConstant(args[0])) || (primary_key_columns.count(first_arg_name) - && (prepared_sets.count(args[1]->range) || typeid_cast(args[1].get())))) + && (typeid_cast(args[1].get()) || prepared_sets.count(args[1]->getTreeHash())))) return true; } diff --git a/dbms/src/Storages/MergeTree/ReplicatedMergeTreeMutationEntry.h b/dbms/src/Storages/MergeTree/ReplicatedMergeTreeMutationEntry.h index 68aab6fa021..8f253df3f10 100644 --- a/dbms/src/Storages/MergeTree/ReplicatedMergeTreeMutationEntry.h +++ b/dbms/src/Storages/MergeTree/ReplicatedMergeTreeMutationEntry.h @@ -4,6 +4,7 @@ #include #include #include +#include namespace DB diff --git a/dbms/src/Storages/SelectQueryInfo.h b/dbms/src/Storages/SelectQueryInfo.h index 1aed0a98f8f..e68078a6712 100644 --- a/dbms/src/Storages/SelectQueryInfo.h +++ b/dbms/src/Storages/SelectQueryInfo.h @@ -1,24 +1,14 @@ #pragma once +#include #include -#include -#include namespace DB { -class IAST; -using ASTPtr = std::shared_ptr; - class ExpressionActions; using ExpressionActionsPtr = std::shared_ptr; -class Set; -using SetPtr = std::shared_ptr; - -/// Information about calculated sets in right hand side of IN. -using PreparedSets = std::unordered_map; - struct PrewhereInfo { /// Actions which are executed in order to alias columns are used for prewhere actions. From f3b4292fc691634473e5d4f6161c869db1813a8f Mon Sep 17 00:00:00 2001 From: Alexey Zatelepin Date: Tue, 22 Jan 2019 15:33:56 +0300 Subject: [PATCH 13/86] add data types to the prepared_sets key for literals #4058 --- dbms/src/Core/Block.cpp | 12 ++ dbms/src/Core/Block.h | 1 + dbms/src/Interpreters/ActionsVisitor.cpp | 69 +++++---- dbms/src/Interpreters/ActionsVisitor.h | 9 +- dbms/src/Interpreters/ExpressionAnalyzer.cpp | 59 ++++---- dbms/src/Interpreters/ExpressionAnalyzer.h | 2 +- dbms/src/Interpreters/PreparedSets.h | 50 ++++++- dbms/src/Storages/MergeTree/KeyCondition.cpp | 139 ++++++++---------- dbms/src/Storages/MergeTree/KeyCondition.h | 13 +- .../MergeTree/MergeTreeWhereOptimizer.cpp | 5 +- .../MergeTree/MergeTreeWhereOptimizer.h | 1 - 11 files changed, 201 insertions(+), 159 deletions(-) diff --git a/dbms/src/Core/Block.cpp b/dbms/src/Core/Block.cpp index de0e831f37c..27b2cb81b09 100644 --- a/dbms/src/Core/Block.cpp +++ b/dbms/src/Core/Block.cpp @@ -427,6 +427,18 @@ Names Block::getNames() const } +DataTypes Block::getDataTypes() const +{ + DataTypes res; + res.reserve(columns()); + + for (const auto & elem : data) + res.push_back(elem.type); + + return res; +} + + template static ReturnType checkBlockStructure(const Block & lhs, const Block & rhs, const std::string & context_description) { diff --git a/dbms/src/Core/Block.h b/dbms/src/Core/Block.h index d8efc939ecd..d3ce0c67b79 100644 --- a/dbms/src/Core/Block.h +++ b/dbms/src/Core/Block.h @@ -82,6 +82,7 @@ public: const ColumnsWithTypeAndName & getColumnsWithTypeAndName() const; NamesAndTypesList getNamesAndTypesList() const; Names getNames() const; + DataTypes getDataTypes() const; /// Returns number of rows from first column in block, not equal to nullptr. If no columns, returns 0. size_t rows() const; diff --git a/dbms/src/Interpreters/ActionsVisitor.cpp b/dbms/src/Interpreters/ActionsVisitor.cpp index b53d166ddb4..1e8a17adecd 100644 --- a/dbms/src/Interpreters/ActionsVisitor.cpp +++ b/dbms/src/Interpreters/ActionsVisitor.cpp @@ -54,8 +54,9 @@ NamesAndTypesList::iterator findColumn(const String & name, NamesAndTypesList & [&](const NamesAndTypesList::value_type & val) { return val.name == name; }); } -void makeExplicitSet(const ASTFunction * node, const Block & sample_block, bool create_ordered_set, - const Context & context, const SizeLimits & size_limits, PreparedSets & prepared_sets) +SetPtr makeExplicitSet( + const ASTFunction * node, const Block & sample_block, bool create_ordered_set, + const Context & context, const SizeLimits & size_limits, PreparedSets & prepared_sets) { const IAST & args = *node->arguments; @@ -65,6 +66,21 @@ void makeExplicitSet(const ASTFunction * node, const Block & sample_block, bool const ASTPtr & left_arg = args.children.at(0); const ASTPtr & right_arg = args.children.at(1); + const DataTypePtr & left_arg_type = sample_block.getByName(left_arg->getColumnName()).type; + + DataTypes set_element_types = {left_arg_type}; + auto left_tuple_type = typeid_cast(left_arg_type.get()); + if (left_tuple_type && left_tuple_type->getElements().size() != 1) + set_element_types = left_tuple_type->getElements(); + + for (auto & element_type : set_element_types) + if (const auto * low_cardinality_type = typeid_cast(element_type.get())) + element_type = low_cardinality_type->getDictionaryType(); + + auto set_key = PreparedSetKey::forLiteral(*right_arg, set_element_types); + if (prepared_sets.count(set_key)) + return prepared_sets.at(set_key); /// Already prepared. + auto getTupleTypeFromAst = [&context](const ASTPtr & tuple_ast) -> DataTypePtr { auto ast_function = typeid_cast(tuple_ast.get()); @@ -79,7 +95,6 @@ void makeExplicitSet(const ASTFunction * node, const Block & sample_block, bool return evaluateConstantExpression(tuple_ast, context).second; }; - const DataTypePtr & left_arg_type = sample_block.getByName(left_arg->getColumnName()).type; const DataTypePtr & right_arg_type = getTupleTypeFromAst(right_arg); std::function getTupleDepth; @@ -94,15 +109,6 @@ void makeExplicitSet(const ASTFunction * node, const Block & sample_block, bool size_t left_tuple_depth = getTupleDepth(left_arg_type); size_t right_tuple_depth = getTupleDepth(right_arg_type); - DataTypes set_element_types = {left_arg_type}; - auto left_tuple_type = typeid_cast(left_arg_type.get()); - if (left_tuple_type && left_tuple_type->getElements().size() != 1) - set_element_types = left_tuple_type->getElements(); - - for (auto & element_type : set_element_types) - if (const auto * low_cardinality_type = typeid_cast(element_type.get())) - element_type = low_cardinality_type->getDictionaryType(); - ASTPtr elements_ast = nullptr; /// 1 in 1; (1, 2) in (1, 2); identity(tuple(tuple(tuple(1)))) in tuple(tuple(tuple(1))); etc. @@ -131,7 +137,8 @@ void makeExplicitSet(const ASTFunction * node, const Block & sample_block, bool SetPtr set = std::make_shared(size_limits, create_ordered_set); set->createFromAST(set_element_types, elements_ast, context); - prepared_sets[right_arg->getTreeHash()] = std::move(set); + prepared_sets[set_key] = set; + return set; } static String getUniqueName(const Block & block, const String & prefix) @@ -305,6 +312,7 @@ void ActionsVisitor::visit(const ASTPtr & ast) return; } + SetPtr prepared_set; if (functionIsInOrGlobalInOperator(node->name)) { /// Let's find the type of the first argument (then getActionsImpl will be called again and will not affect anything). @@ -313,7 +321,7 @@ void ActionsVisitor::visit(const ASTPtr & ast) if (!no_subqueries) { /// Transform tuple or subquery into a set. - makeSet(node, actions_stack.getSampleBlock()); + prepared_set = makeSet(node, actions_stack.getSampleBlock()); } else { @@ -380,23 +388,21 @@ void ActionsVisitor::visit(const ASTPtr & ast) /// Select the name in the next cycle. argument_names.emplace_back(); } - else if (functionIsInOrGlobalInOperator(node->name) && arg == 1 && prepared_sets.count(child->getTreeHash())) + else if (functionIsInOrGlobalInOperator(node->name) && arg == 1 && prepared_set) { ColumnWithTypeAndName column; column.type = std::make_shared(); - const SetPtr & set = prepared_sets[child->getTreeHash()]; - /// If the argument is a set given by an enumeration of values (so, the set was already built), give it a unique name, /// so that sets with the same literal representation do not fuse together (they can have different types). - if (!set->empty()) + if (!prepared_set->empty()) column.name = getUniqueName(actions_stack.getSampleBlock(), "__set"); else column.name = child_column_name; if (!actions_stack.getSampleBlock().has(column.name)) { - column.column = ColumnSet::create(1, set); + column.column = ColumnSet::create(1, prepared_set); actions_stack.addAction(ExpressionAction::addColumn(column)); } @@ -522,7 +528,7 @@ void ActionsVisitor::visit(const ASTPtr & ast) } } -void ActionsVisitor::makeSet(const ASTFunction * node, const Block & sample_block) +SetPtr ActionsVisitor::makeSet(const ASTFunction * node, const Block & sample_block) { /** You need to convert the right argument to a set. * This can be a table name, a value, a value enumeration, or a subquery. @@ -530,16 +536,15 @@ void ActionsVisitor::makeSet(const ASTFunction * node, const Block & sample_bloc */ const IAST & args = *node->arguments; const ASTPtr & arg = args.children.at(1); - IAST::Hash tree_hash = arg->getTreeHash(); - - /// Already converted. - if (prepared_sets.count(tree_hash)) - return; /// If the subquery or table name for SELECT. const ASTIdentifier * identifier = typeid_cast(arg.get()); if (typeid_cast(arg.get()) || identifier) { + auto set_key = PreparedSetKey::forSubquery(*arg); + if (prepared_sets.count(set_key)) + return prepared_sets.at(set_key); + /// A special case is if the name of the table is specified on the right side of the IN statement, /// and the table has the type Set (a previously prepared set). if (identifier) @@ -550,11 +555,10 @@ void ActionsVisitor::makeSet(const ASTFunction * node, const Block & sample_bloc if (table) { StorageSet * storage_set = dynamic_cast(table.get()); - if (storage_set) { - prepared_sets[tree_hash] = storage_set->getSet(); - return; + prepared_sets[set_key] = storage_set->getSet(); + return storage_set->getSet(); } } } @@ -567,8 +571,8 @@ void ActionsVisitor::makeSet(const ASTFunction * node, const Block & sample_bloc /// If you already created a Set with the same subquery / table. if (subquery_for_set.set) { - prepared_sets[tree_hash] = subquery_for_set.set; - return; + prepared_sets[set_key] = subquery_for_set.set; + return subquery_for_set.set; } SetPtr set = std::make_shared(set_size_limit, false); @@ -613,12 +617,13 @@ void ActionsVisitor::makeSet(const ASTFunction * node, const Block & sample_bloc } subquery_for_set.set = set; - prepared_sets[tree_hash] = set; + prepared_sets[set_key] = set; + return set; } else { /// An explicit enumeration of values in parentheses. - makeExplicitSet(node, sample_block, false, context, set_size_limit, prepared_sets); + return makeExplicitSet(node, sample_block, false, context, set_size_limit, prepared_sets); } } diff --git a/dbms/src/Interpreters/ActionsVisitor.h b/dbms/src/Interpreters/ActionsVisitor.h index d4a6d388fbc..12f9e1116c0 100644 --- a/dbms/src/Interpreters/ActionsVisitor.h +++ b/dbms/src/Interpreters/ActionsVisitor.h @@ -37,9 +37,10 @@ struct SubqueryForSet using SubqueriesForSets = std::unordered_map; -/// The case of an explicit enumeration of values. -void makeExplicitSet(const ASTFunction * node, const Block & sample_block, bool create_ordered_set, - const Context & context, const SizeLimits & limits, PreparedSets & prepared_sets); + /// The case of an explicit enumeration of values. +SetPtr makeExplicitSet( + const ASTFunction * node, const Block & sample_block, bool create_ordered_set, + const Context & context, const SizeLimits & limits, PreparedSets & prepared_sets); /** For ActionsVisitor @@ -105,7 +106,7 @@ private: std::ostream * ostr; ScopeStack actions_stack; - void makeSet(const ASTFunction * node, const Block & sample_block); + SetPtr makeSet(const ASTFunction * node, const Block & sample_block); }; } diff --git a/dbms/src/Interpreters/ExpressionAnalyzer.cpp b/dbms/src/Interpreters/ExpressionAnalyzer.cpp index 6d39a1303a4..fd56c55e05f 100644 --- a/dbms/src/Interpreters/ExpressionAnalyzer.cpp +++ b/dbms/src/Interpreters/ExpressionAnalyzer.cpp @@ -258,20 +258,25 @@ void ExpressionAnalyzer::makeSetsForIndex() if (storage && select_query && storage->supportsIndexForIn()) { if (select_query->where_expression) - makeSetsForIndexImpl(select_query->where_expression, storage->getSampleBlock()); + makeSetsForIndexImpl(select_query->where_expression); if (select_query->prewhere_expression) - makeSetsForIndexImpl(select_query->prewhere_expression, storage->getSampleBlock()); + makeSetsForIndexImpl(select_query->prewhere_expression); } } void ExpressionAnalyzer::tryMakeSetForIndexFromSubquery(const ASTPtr & subquery_or_table_name) { - BlockIO res = interpretSubquery(subquery_or_table_name, context, subquery_depth + 1, {})->execute(); + auto set_key = PreparedSetKey::forSubquery(*subquery_or_table_name); + if (prepared_sets.count(set_key)) + return; /// Already prepared. + + auto interpreter_subquery = interpretSubquery(subquery_or_table_name, context, subquery_depth + 1, {}); + BlockIO res = interpreter_subquery->execute(); SetPtr set = std::make_shared(settings.size_limits_for_set, true); - set->setHeader(res.in->getHeader()); + while (Block block = res.in->read()) { /// If the limits have been exceeded, give up and let the default subquery processing actions take place. @@ -279,24 +284,24 @@ void ExpressionAnalyzer::tryMakeSetForIndexFromSubquery(const ASTPtr & subquery_ return; } - prepared_sets[subquery_or_table_name->getTreeHash()] = std::move(set); + prepared_sets[set_key] = std::move(set); } -void ExpressionAnalyzer::makeSetsForIndexImpl(const ASTPtr & node, const Block & sample_block) +void ExpressionAnalyzer::makeSetsForIndexImpl(const ASTPtr & node) { for (auto & child : node->children) { - /// Don't descent into subqueries. + /// Don't descend into subqueries. if (typeid_cast(child.get())) continue; - /// Don't dive into lambda functions + /// Don't descend into lambda functions const ASTFunction * func = typeid_cast(child.get()); if (func && func->name == "lambda") continue; - makeSetsForIndexImpl(child, sample_block); + makeSetsForIndexImpl(child); } const ASTFunction * func = typeid_cast(node.get()); @@ -307,28 +312,24 @@ void ExpressionAnalyzer::makeSetsForIndexImpl(const ASTPtr & node, const Block & if (storage && storage->mayBenefitFromIndexForIn(args.children.at(0))) { const ASTPtr & arg = args.children.at(1); - - if (!prepared_sets.count(arg->getTreeHash())) /// Not already prepared. + if (typeid_cast(arg.get()) || isIdentifier(arg)) { - if (typeid_cast(arg.get()) || isIdentifier(arg)) - { - if (settings.use_index_for_in_with_subqueries) - tryMakeSetForIndexFromSubquery(arg); - } - else - { - NamesAndTypesList temp_columns = source_columns; - temp_columns.insert(temp_columns.end(), array_join_columns.begin(), array_join_columns.end()); - for (const auto & joined_column : columns_added_by_join) - temp_columns.push_back(joined_column.name_and_type); - ExpressionActionsPtr temp_actions = std::make_shared(temp_columns, context); - getRootActions(func->arguments->children.at(0), true, temp_actions); + if (settings.use_index_for_in_with_subqueries) + tryMakeSetForIndexFromSubquery(arg); + } + else + { + NamesAndTypesList temp_columns = source_columns; + temp_columns.insert(temp_columns.end(), array_join_columns.begin(), array_join_columns.end()); + for (const auto & joined_column : columns_added_by_join) + temp_columns.push_back(joined_column.name_and_type); + ExpressionActionsPtr temp_actions = std::make_shared(temp_columns, context); + getRootActions(func->arguments->children.at(0), true, temp_actions); - Block sample_block_with_calculated_columns = temp_actions->getSampleBlock(); - if (sample_block_with_calculated_columns.has(args.children.at(0)->getColumnName())) - makeExplicitSet(func, sample_block_with_calculated_columns, true, context, - settings.size_limits_for_set, prepared_sets); - } + Block sample_block_with_calculated_columns = temp_actions->getSampleBlock(); + if (sample_block_with_calculated_columns.has(args.children.at(0)->getColumnName())) + makeExplicitSet(func, sample_block_with_calculated_columns, true, context, + settings.size_limits_for_set, prepared_sets); } } } diff --git a/dbms/src/Interpreters/ExpressionAnalyzer.h b/dbms/src/Interpreters/ExpressionAnalyzer.h index feb8009e92e..ae698f81282 100644 --- a/dbms/src/Interpreters/ExpressionAnalyzer.h +++ b/dbms/src/Interpreters/ExpressionAnalyzer.h @@ -271,7 +271,7 @@ private: */ void tryMakeSetForIndexFromSubquery(const ASTPtr & subquery_or_table_name); - void makeSetsForIndexImpl(const ASTPtr & node, const Block & sample_block); + void makeSetsForIndexImpl(const ASTPtr & node); bool isRemoteStorage() const; }; diff --git a/dbms/src/Interpreters/PreparedSets.h b/dbms/src/Interpreters/PreparedSets.h index 4276e62a239..a2c6f6de40a 100644 --- a/dbms/src/Interpreters/PreparedSets.h +++ b/dbms/src/Interpreters/PreparedSets.h @@ -1,20 +1,64 @@ #pragma once #include +#include #include #include namespace DB { -struct ASTHalfHash +struct PreparedSetKey { - UInt64 operator()(const IAST::Hash & ast_hash) const { return ast_hash.first; } + /// Prepared sets for tuple literals are indexed by the hash of the tree contents and by the desired + /// data types of set elements (two different Sets can be required for two tuples with the same contents + /// if left hand sides of the IN operators have different types). + static PreparedSetKey forLiteral(const IAST & ast, DataTypes types_) + { + PreparedSetKey key; + key.ast_hash = ast.getTreeHash(); + key.types = std::move(types_); + return key; + } + + /// Prepared sets for subqueries are indexed only by the AST contents because the type of the resulting + /// set is fully determined by the subquery. + static PreparedSetKey forSubquery(const IAST & ast) + { + PreparedSetKey key; + key.ast_hash = ast.getTreeHash(); + return key; + } + + IAST::Hash ast_hash; + DataTypes types; /// Empty for subqueries. + + bool operator==(const PreparedSetKey & other) const + { + if (ast_hash != other.ast_hash) + return false; + + if (types.size() != other.types.size()) + return false; + + for (size_t i = 0; i < types.size(); ++i) + { + if (!types[i]->equals(*other.types[i])) + return false; + } + + return true; + } + + struct Hash + { + UInt64 operator()(const PreparedSetKey & key) const { return key.ast_hash.first; } + }; }; class Set; using SetPtr = std::shared_ptr; -using PreparedSets = std::unordered_map; +using PreparedSets = std::unordered_map; } diff --git a/dbms/src/Storages/MergeTree/KeyCondition.cpp b/dbms/src/Storages/MergeTree/KeyCondition.cpp index 87d9f8def3d..b64920d6233 100644 --- a/dbms/src/Storages/MergeTree/KeyCondition.cpp +++ b/dbms/src/Storages/MergeTree/KeyCondition.cpp @@ -4,6 +4,7 @@ #include #include #include +#include #include #include #include @@ -12,6 +13,8 @@ #include #include #include +#include +#include namespace DB @@ -107,7 +110,7 @@ const KeyCondition::AtomMap KeyCondition::atom_map { { "notEquals", - [] (RPNElement & out, const Field & value, const ASTPtr &) + [] (RPNElement & out, const Field & value) { out.function = RPNElement::FUNCTION_NOT_IN_RANGE; out.range = Range(value); @@ -116,7 +119,7 @@ const KeyCondition::AtomMap KeyCondition::atom_map }, { "equals", - [] (RPNElement & out, const Field & value, const ASTPtr &) + [] (RPNElement & out, const Field & value) { out.function = RPNElement::FUNCTION_IN_RANGE; out.range = Range(value); @@ -125,7 +128,7 @@ const KeyCondition::AtomMap KeyCondition::atom_map }, { "less", - [] (RPNElement & out, const Field & value, const ASTPtr &) + [] (RPNElement & out, const Field & value) { out.function = RPNElement::FUNCTION_IN_RANGE; out.range = Range::createRightBounded(value, false); @@ -134,7 +137,7 @@ const KeyCondition::AtomMap KeyCondition::atom_map }, { "greater", - [] (RPNElement & out, const Field & value, const ASTPtr &) + [] (RPNElement & out, const Field & value) { out.function = RPNElement::FUNCTION_IN_RANGE; out.range = Range::createLeftBounded(value, false); @@ -143,7 +146,7 @@ const KeyCondition::AtomMap KeyCondition::atom_map }, { "lessOrEquals", - [] (RPNElement & out, const Field & value, const ASTPtr &) + [] (RPNElement & out, const Field & value) { out.function = RPNElement::FUNCTION_IN_RANGE; out.range = Range::createRightBounded(value, true); @@ -152,7 +155,7 @@ const KeyCondition::AtomMap KeyCondition::atom_map }, { "greaterOrEquals", - [] (RPNElement & out, const Field & value, const ASTPtr &) + [] (RPNElement & out, const Field & value) { out.function = RPNElement::FUNCTION_IN_RANGE; out.range = Range::createLeftBounded(value, true); @@ -161,25 +164,23 @@ const KeyCondition::AtomMap KeyCondition::atom_map }, { "in", - [] (RPNElement & out, const Field &, const ASTPtr & node) + [] (RPNElement & out, const Field &) { out.function = RPNElement::FUNCTION_IN_SET; - out.in_function = node; return true; } }, { "notIn", - [] (RPNElement & out, const Field &, const ASTPtr & node) + [] (RPNElement & out, const Field &) { out.function = RPNElement::FUNCTION_NOT_IN_SET; - out.in_function = node; return true; } }, { "like", - [] (RPNElement & out, const Field & value, const ASTPtr &) + [] (RPNElement & out, const Field & value) { if (value.getType() != Field::Types::String) return false; @@ -458,71 +459,64 @@ bool KeyCondition::canConstantBeWrappedByMonotonicFunctions( return found_transformation; } -void KeyCondition::getKeyTuplePositionMapping( - const ASTPtr & node, - const Context & context, - std::vector & indexes_mapping, - const size_t tuple_index, - size_t & out_key_column_num) -{ - MergeTreeSetIndex::KeyTuplePositionMapping index_mapping; - index_mapping.tuple_index = tuple_index; - DataTypePtr data_type; - if (isKeyPossiblyWrappedByMonotonicFunctions( - node, context, index_mapping.key_index, - data_type, index_mapping.functions)) - { - indexes_mapping.push_back(index_mapping); - if (out_key_column_num < index_mapping.key_index) - { - out_key_column_num = index_mapping.key_index; - } - } -} - - bool KeyCondition::tryPrepareSetIndex( - const ASTPtr & node, + const ASTs & args, const Context & context, RPNElement & out, - const SetPtr & prepared_set, size_t & out_key_column_num) { - /// The index can be prepared if the elements of the set were saved in advance. - if (!prepared_set->hasExplicitSetElements()) - return false; + const ASTPtr & left_arg = args[0]; out_key_column_num = 0; std::vector indexes_mapping; + DataTypes data_types; - size_t num_key_columns = prepared_set->getDataTypes().size(); - - const ASTFunction * node_tuple = typeid_cast(node.get()); - if (node_tuple && node_tuple->name == "tuple") + auto get_key_tuple_position_mapping = [&](const ASTPtr & node, size_t tuple_index) { - if (num_key_columns != node_tuple->arguments->children.size()) + MergeTreeSetIndex::KeyTuplePositionMapping index_mapping; + index_mapping.tuple_index = tuple_index; + DataTypePtr data_type; + if (isKeyPossiblyWrappedByMonotonicFunctions( + node, context, index_mapping.key_index, data_type, index_mapping.functions)) { - std::stringstream message; - message << "Number of columns in section IN doesn't match. " - << node_tuple->arguments->children.size() << " at left, " << num_key_columns << " at right."; - throw Exception(message.str(), ErrorCodes::NUMBER_OF_COLUMNS_DOESNT_MATCH); + indexes_mapping.push_back(index_mapping); + data_types.push_back(data_type); + if (out_key_column_num < index_mapping.key_index) + out_key_column_num = index_mapping.key_index; } + }; - size_t current_tuple_index = 0; - for (const auto & arg : node_tuple->arguments->children) - { - getKeyTuplePositionMapping(arg, context, indexes_mapping, current_tuple_index, out_key_column_num); - ++current_tuple_index; - } + const ASTFunction * left_arg_tuple = typeid_cast(left_arg.get()); + if (left_arg_tuple && left_arg_tuple->name == "tuple") + { + const auto & tuple_elements = left_arg_tuple->arguments->children; + for (size_t i = 0; i < tuple_elements.size(); ++i) + get_key_tuple_position_mapping(tuple_elements[i], i); } else - { - getKeyTuplePositionMapping(node, context, indexes_mapping, 0, out_key_column_num); - } + get_key_tuple_position_mapping(left_arg, 0); if (indexes_mapping.empty()) return false; + const ASTPtr & right_arg = args[1]; + + PreparedSetKey set_key; + if (typeid_cast(right_arg.get()) || typeid_cast(right_arg.get())) + set_key = PreparedSetKey::forSubquery(*right_arg); + else + set_key = PreparedSetKey::forLiteral(*right_arg, data_types); + + auto set_it = prepared_sets.find(set_key); + if (set_it == prepared_sets.end()) + return false; + + const SetPtr & prepared_set = set_it->second; + + /// The index can be prepared if the elements of the set were saved in advance. + if (!prepared_set->hasExplicitSetElements()) + return false; + out.set_index = std::make_shared(prepared_set->getSetElements(), std::move(indexes_mapping)); return true; @@ -640,13 +634,13 @@ bool KeyCondition::atomFromAST(const ASTPtr & node, const Context & context, Blo bool is_set_const = false; bool is_constant_transformed = false; - if (prepared_sets.count(args[1]->getTreeHash()) - && tryPrepareSetIndex(args[0], context, out, prepared_sets[args[1]->getTreeHash()], key_column_num)) - { - key_arg_pos = 0; - is_set_const = true; - } - else if (getConstant(args[1], block_with_constants, const_value, const_type) + if (functionIsInOrGlobalInOperator(func->name) + && tryPrepareSetIndex(args, context, out, key_column_num)) + { + key_arg_pos = 0; + is_set_const = true; + } + else if (getConstant(args[1], block_with_constants, const_value, const_type) && isKeyPossiblyWrappedByMonotonicFunctions(args[0], context, key_column_num, key_expr_type, chain)) { key_arg_pos = 0; @@ -717,7 +711,7 @@ bool KeyCondition::atomFromAST(const ASTPtr & node, const Context & context, Blo if (!cast_not_needed) castValueToType(key_expr_type, const_value, const_type, node); - return atom_it->second(out, const_value, node); + return atom_it->second(out, const_value); } else if (getConstant(node, block_with_constants, const_value, const_type)) /// For cases where it says, for example, `WHERE 0 AND something` { @@ -1016,17 +1010,12 @@ bool KeyCondition::mayBeTrueInParallelogram(const std::vector & parallelo element.function == RPNElement::FUNCTION_IN_SET || element.function == RPNElement::FUNCTION_NOT_IN_SET) { - auto in_func = typeid_cast(element.in_function.get()); - const ASTs & args = typeid_cast(*in_func->arguments).children; - PreparedSets::const_iterator it = prepared_sets.find(args[1]->getTreeHash()); - if (in_func && it != prepared_sets.end()) - { - rpn_stack.emplace_back(element.set_index->mayBeTrueInRange(parallelogram, data_types)); - if (element.function == RPNElement::FUNCTION_NOT_IN_SET) - rpn_stack.back() = !rpn_stack.back(); - } - else + if (!element.set_index) throw Exception("Set for IN is not created yet", ErrorCodes::LOGICAL_ERROR); + + rpn_stack.emplace_back(element.set_index->mayBeTrueInRange(parallelogram, data_types)); + if (element.function == RPNElement::FUNCTION_NOT_IN_SET) + rpn_stack.back() = !rpn_stack.back(); } else if (element.function == RPNElement::FUNCTION_NOT) { diff --git a/dbms/src/Storages/MergeTree/KeyCondition.h b/dbms/src/Storages/MergeTree/KeyCondition.h index 1d700ad80d9..20009b50f34 100644 --- a/dbms/src/Storages/MergeTree/KeyCondition.h +++ b/dbms/src/Storages/MergeTree/KeyCondition.h @@ -310,7 +310,6 @@ private: Range range; size_t key_column = 0; /// For FUNCTION_IN_SET, FUNCTION_NOT_IN_SET - ASTPtr in_function; using MergeTreeSetIndexPtr = std::shared_ptr; MergeTreeSetIndexPtr set_index; @@ -320,7 +319,7 @@ private: using RPN = std::vector; using ColumnIndices = std::map; - using AtomMap = std::unordered_map; + using AtomMap = std::unordered_map; public: static const AtomMap atom_map; @@ -363,21 +362,13 @@ private: Field & out_value, DataTypePtr & out_type); - void getKeyTuplePositionMapping( - const ASTPtr & node, - const Context & context, - std::vector & indexes_mapping, - const size_t tuple_index, - size_t & out_key_column_num); - /// If it's possible to make an RPNElement /// that will filter values (possibly tuples) by the content of 'prepared_set', /// do it and return true. bool tryPrepareSetIndex( - const ASTPtr & node, + const ASTs & args, const Context & context, RPNElement & out, - const SetPtr & prepared_set, size_t & out_key_column_num); RPN rpn; diff --git a/dbms/src/Storages/MergeTree/MergeTreeWhereOptimizer.cpp b/dbms/src/Storages/MergeTree/MergeTreeWhereOptimizer.cpp index 3eb9576b1c0..66a9a6f90d2 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeWhereOptimizer.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeWhereOptimizer.cpp @@ -8,6 +8,7 @@ #include #include #include +#include #include #include #include @@ -44,7 +45,6 @@ MergeTreeWhereOptimizer::MergeTreeWhereOptimizer( table_columns{ext::map(data.getColumns().getAllPhysical(), [] (const NameAndTypePair & col) { return col.name; })}, block_with_constants{KeyCondition::getBlockWithConstants(query_info.query, query_info.syntax_analyzer_result, context)}, - prepared_sets(query_info.sets), log{log} { calculateColumnSizes(data, column_names); @@ -333,8 +333,7 @@ bool MergeTreeWhereOptimizer::isPrimaryKeyAtom(const IAST * const ast) const if ((primary_key_columns.count(first_arg_name) && isConstant(args[1])) || (primary_key_columns.count(second_arg_name) && isConstant(args[0])) || - (primary_key_columns.count(first_arg_name) - && (typeid_cast(args[1].get()) || prepared_sets.count(args[1]->getTreeHash())))) + (primary_key_columns.count(first_arg_name) && functionIsInOrGlobalInOperator(func->name))) return true; } diff --git a/dbms/src/Storages/MergeTree/MergeTreeWhereOptimizer.h b/dbms/src/Storages/MergeTree/MergeTreeWhereOptimizer.h index ccf4970d300..5f8dd587a92 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeWhereOptimizer.h +++ b/dbms/src/Storages/MergeTree/MergeTreeWhereOptimizer.h @@ -77,7 +77,6 @@ private: const string_set_t primary_key_columns; const string_set_t table_columns; const Block block_with_constants; - const PreparedSets & prepared_sets; Poco::Logger * log; std::unordered_map column_sizes{}; size_t total_column_size{}; From 38945cfca41f6c256c79d6b5e9d951d8cf2bb473 Mon Sep 17 00:00:00 2001 From: BayoNet Date: Wed, 23 Jan 2019 14:55:07 +0300 Subject: [PATCH 14/86] Fixed anchors in docs (#4127) --- docs/en/operations/settings/index.md | 2 +- docs/en/operations/table_engines/replication.md | 2 +- docs/ru/operations/settings/settings.md | 2 +- docs/ru/operations/table_engines/collapsingmergetree.md | 2 +- docs/ru/operations/table_engines/replication.md | 4 ++-- 5 files changed, 6 insertions(+), 6 deletions(-) diff --git a/docs/en/operations/settings/index.md b/docs/en/operations/settings/index.md index 73152183079..16487c2a558 100644 --- a/docs/en/operations/settings/index.md +++ b/docs/en/operations/settings/index.md @@ -11,7 +11,7 @@ Ways to configure settings, in order of priority: - Session settings. - Send ` SET setting=value` from the ClickHouse console client in interactive mode. + Send `SET setting=value` from the ClickHouse console client in interactive mode. Similarly, you can use ClickHouse sessions in the HTTP protocol. To do this, you need to specify the `session_id` HTTP parameter. - Query settings. diff --git a/docs/en/operations/table_engines/replication.md b/docs/en/operations/table_engines/replication.md index 75b1866e01e..bb2e1eb62b6 100644 --- a/docs/en/operations/table_engines/replication.md +++ b/docs/en/operations/table_engines/replication.md @@ -68,7 +68,7 @@ You can have any number of replicas of the same data. Yandex.Metrica uses double The system monitors data synchronicity on replicas and is able to recover after a failure. Failover is automatic (for small differences in data) or semi-automatic (when data differs too much, which may indicate a configuration error). -## Creating Replicated Tables +## Creating Replicated Tables {#creating-replicated-tables} The `Replicated` prefix is added to the table engine name. For example:`ReplicatedMergeTree`. diff --git a/docs/ru/operations/settings/settings.md b/docs/ru/operations/settings/settings.md index ed1395a1aaf..c174507859b 100644 --- a/docs/ru/operations/settings/settings.md +++ b/docs/ru/operations/settings/settings.md @@ -118,7 +118,7 @@ ClickHouse применяет настройку в тех случаях, ко Это намного больше, чем `max_block_size`. Это сделано, потому что некоторые движки таблиц (`*MergeTree`) будут на каждый вставляемый блок формировать кусок данных на диске, что является довольно большой сущностью. Также, в таблицах типа `*MergeTree`, данные сортируются при вставке, и достаточно большой размер блока позволяет отсортировать больше данных в оперативке. -## max_replica_delay_for_distributed_queries {#settings_settings_max_replica_delay_for_distributed_queries} +## max_replica_delay_for_distributed_queries {#settings-max_replica_delay_for_distributed_queries} Отключает отстающие реплики при распределенных запросах. Смотрите "[Репликация](../../operations/table_engines/replication.md)". diff --git a/docs/ru/operations/table_engines/collapsingmergetree.md b/docs/ru/operations/table_engines/collapsingmergetree.md index cc938de2af3..f09d84cf8e8 100644 --- a/docs/ru/operations/table_engines/collapsingmergetree.md +++ b/docs/ru/operations/table_engines/collapsingmergetree.md @@ -54,7 +54,7 @@ CREATE TABLE [IF NOT EXISTS] [db.]table_name [ON CLUSTER cluster] -## Сворачивание (удаление) строк {table_engine-collapsingmergetree-collapsing} +## Сворачивание (удаление) строк {#table_engine-collapsingmergetree-collapsing} ### Данные diff --git a/docs/ru/operations/table_engines/replication.md b/docs/ru/operations/table_engines/replication.md index fe61e3a232c..8f94015b37a 100644 --- a/docs/ru/operations/table_engines/replication.md +++ b/docs/ru/operations/table_engines/replication.md @@ -1,4 +1,4 @@ -# Репликация данных {#table_engines-репликация} +# Репликация данных {#table_engines-replication} Репликация поддерживается только для таблиц семейства MergeTree: @@ -67,7 +67,7 @@ Система следит за синхронностью данных на репликах и умеет восстанавливаться после сбоя. Восстановление после сбоя автоматическое (в случае небольших различий в данных) или полуавтоматическое (когда данные отличаются слишком сильно, что может свидетельствовать об ошибке конфигурации). -## Создание реплицируемых таблиц +## Создание реплицируемых таблиц {#creating-replicated-tables} В начало имени движка таблицы добавляется `Replicated`. Например, `ReplicatedMergeTree`. From eb207b2c5ccd96d815c2bc3bd99790bcdc57d05e Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 23 Jan 2019 15:48:15 +0300 Subject: [PATCH 15/86] Replace untrusted hits_1000m with hits_1000m_single in performance test --- .../test_hits_agg_functions_min_max_any.xml | 186 +++++++++--------- 1 file changed, 93 insertions(+), 93 deletions(-) diff --git a/dbms/tests/performance/test_hits/test_hits_agg_functions_min_max_any.xml b/dbms/tests/performance/test_hits/test_hits_agg_functions_min_max_any.xml index 50ccb6542ab..3c6eb79a359 100644 --- a/dbms/tests/performance/test_hits/test_hits_agg_functions_min_max_any.xml +++ b/dbms/tests/performance/test_hits/test_hits_agg_functions_min_max_any.xml @@ -18,102 +18,102 @@ - default.hits_1000m + default.hits_1000m_single -select min(Title) from hits_1000m where Title != '' group by intHash32(UserID) % 1000000 -select max(Title) from hits_1000m where Title != '' group by intHash32(UserID) % 1000000 -select any(Title) from hits_1000m where Title != '' group by intHash32(UserID) % 1000000 -select anyHeavy(Title) from hits_1000m where Title != '' group by intHash32(UserID) % 1000000 -select min(URL) from hits_1000m where URL != '' group by intHash32(UserID) % 1000000 -select max(URL) from hits_1000m where URL != '' group by intHash32(UserID) % 1000000 -select any(URL) from hits_1000m where URL != '' group by intHash32(UserID) % 1000000 -select anyHeavy(URL) from hits_1000m where URL != '' group by intHash32(UserID) % 1000000 -select min(Referer) from hits_1000m where Referer != '' group by intHash32(UserID) % 1000000 -select max(Referer) from hits_1000m where Referer != '' group by intHash32(UserID) % 1000000 -select any(Referer) from hits_1000m where Referer != '' group by intHash32(UserID) % 1000000 -select anyHeavy(Referer) from hits_1000m where Referer != '' group by intHash32(UserID) % 1000000 -select min(FlashMinor2) from hits_1000m where FlashMinor2 != '' group by intHash32(UserID) % 1000000 -select max(FlashMinor2) from hits_1000m where FlashMinor2 != '' group by intHash32(UserID) % 1000000 -select any(FlashMinor2) from hits_1000m where FlashMinor2 != '' group by intHash32(UserID) % 1000000 -select anyHeavy(FlashMinor2) from hits_1000m where FlashMinor2 != '' group by intHash32(UserID) % 1000000 -select min(MobilePhoneModel) from hits_1000m where MobilePhoneModel != '' group by intHash32(UserID) % 1000000 -select max(MobilePhoneModel) from hits_1000m where MobilePhoneModel != '' group by intHash32(UserID) % 1000000 -select any(MobilePhoneModel) from hits_1000m where MobilePhoneModel != '' group by intHash32(UserID) % 1000000 -select anyHeavy(MobilePhoneModel) from hits_1000m where MobilePhoneModel != '' group by intHash32(UserID) % 1000000 -select min(Params) from hits_1000m where Params != '' group by intHash32(UserID) % 1000000 -select max(Params) from hits_1000m where Params != '' group by intHash32(UserID) % 1000000 -select any(Params) from hits_1000m where Params != '' group by intHash32(UserID) % 1000000 -select anyHeavy(Params) from hits_1000m where Params != '' group by intHash32(UserID) % 1000000 -select min(SearchPhrase) from hits_1000m where SearchPhrase != '' group by intHash32(UserID) % 1000000 -select max(SearchPhrase) from hits_1000m where SearchPhrase != '' group by intHash32(UserID) % 1000000 -select any(SearchPhrase) from hits_1000m where SearchPhrase != '' group by intHash32(UserID) % 1000000 -select anyHeavy(SearchPhrase) from hits_1000m where SearchPhrase != '' group by intHash32(UserID) % 1000000 -select min(PageCharset) from hits_1000m where PageCharset != '' group by intHash32(UserID) % 1000000 -select max(PageCharset) from hits_1000m where PageCharset != '' group by intHash32(UserID) % 1000000 -select any(PageCharset) from hits_1000m where PageCharset != '' group by intHash32(UserID) % 1000000 -select anyHeavy(PageCharset) from hits_1000m where PageCharset != '' group by intHash32(UserID) % 1000000 -select min(OriginalURL) from hits_1000m where OriginalURL != '' group by intHash32(UserID) % 1000000 -select max(OriginalURL) from hits_1000m where OriginalURL != '' group by intHash32(UserID) % 1000000 -select any(OriginalURL) from hits_1000m where OriginalURL != '' group by intHash32(UserID) % 1000000 -select anyHeavy(OriginalURL) from hits_1000m where OriginalURL != '' group by intHash32(UserID) % 1000000 -select min(SocialNetwork) from hits_1000m where SocialNetwork != '' group by intHash32(UserID) % 1000000 -select max(SocialNetwork) from hits_1000m where SocialNetwork != '' group by intHash32(UserID) % 1000000 -select any(SocialNetwork) from hits_1000m where SocialNetwork != '' group by intHash32(UserID) % 1000000 -select anyHeavy(SocialNetwork) from hits_1000m where SocialNetwork != '' group by intHash32(UserID) % 1000000 -select min(SocialAction) from hits_1000m where SocialAction != '' group by intHash32(UserID) % 1000000 -select max(SocialAction) from hits_1000m where SocialAction != '' group by intHash32(UserID) % 1000000 -select any(SocialAction) from hits_1000m where SocialAction != '' group by intHash32(UserID) % 1000000 -select anyHeavy(SocialAction) from hits_1000m where SocialAction != '' group by intHash32(UserID) % 1000000 -select min(SocialSourcePage) from hits_1000m where SocialSourcePage != '' group by intHash32(UserID) % 1000000 -select max(SocialSourcePage) from hits_1000m where SocialSourcePage != '' group by intHash32(UserID) % 1000000 -select any(SocialSourcePage) from hits_1000m where SocialSourcePage != '' group by intHash32(UserID) % 1000000 -select anyHeavy(SocialSourcePage) from hits_1000m where SocialSourcePage != '' group by intHash32(UserID) % 1000000 -select min(ParamOrderID) from hits_1000m where ParamOrderID != '' group by intHash32(UserID) % 1000000 -select max(ParamOrderID) from hits_1000m where ParamOrderID != '' group by intHash32(UserID) % 1000000 -select any(ParamOrderID) from hits_1000m where ParamOrderID != '' group by intHash32(UserID) % 1000000 -select anyHeavy(ParamOrderID) from hits_1000m where ParamOrderID != '' group by intHash32(UserID) % 1000000 -select min(OpenstatServiceName) from hits_1000m where OpenstatServiceName != '' group by intHash32(UserID) % 1000000 -select max(OpenstatServiceName) from hits_1000m where OpenstatServiceName != '' group by intHash32(UserID) % 1000000 -select any(OpenstatServiceName) from hits_1000m where OpenstatServiceName != '' group by intHash32(UserID) % 1000000 -select anyHeavy(OpenstatServiceName) from hits_1000m where OpenstatServiceName != '' group by intHash32(UserID) % 1000000 -select min(OpenstatCampaignID) from hits_1000m where OpenstatCampaignID != '' group by intHash32(UserID) % 1000000 -select max(OpenstatCampaignID) from hits_1000m where OpenstatCampaignID != '' group by intHash32(UserID) % 1000000 -select any(OpenstatCampaignID) from hits_1000m where OpenstatCampaignID != '' group by intHash32(UserID) % 1000000 -select anyHeavy(OpenstatCampaignID) from hits_1000m where OpenstatCampaignID != '' group by intHash32(UserID) % 1000000 -select min(OpenstatAdID) from hits_1000m where OpenstatAdID != '' group by intHash32(UserID) % 1000000 -select max(OpenstatAdID) from hits_1000m where OpenstatAdID != '' group by intHash32(UserID) % 1000000 -select any(OpenstatAdID) from hits_1000m where OpenstatAdID != '' group by intHash32(UserID) % 1000000 -select anyHeavy(OpenstatAdID) from hits_1000m where OpenstatAdID != '' group by intHash32(UserID) % 1000000 -select min(OpenstatSourceID) from hits_1000m where OpenstatSourceID != '' group by intHash32(UserID) % 1000000 -select max(OpenstatSourceID) from hits_1000m where OpenstatSourceID != '' group by intHash32(UserID) % 1000000 -select any(OpenstatSourceID) from hits_1000m where OpenstatSourceID != '' group by intHash32(UserID) % 1000000 -select anyHeavy(OpenstatSourceID) from hits_1000m where OpenstatSourceID != '' group by intHash32(UserID) % 1000000 -select min(UTMSource) from hits_1000m where UTMSource != '' group by intHash32(UserID) % 1000000 -select max(UTMSource) from hits_1000m where UTMSource != '' group by intHash32(UserID) % 1000000 -select any(UTMSource) from hits_1000m where UTMSource != '' group by intHash32(UserID) % 1000000 -select anyHeavy(UTMSource) from hits_1000m where UTMSource != '' group by intHash32(UserID) % 1000000 -select min(UTMMedium) from hits_1000m where UTMMedium != '' group by intHash32(UserID) % 1000000 -select max(UTMMedium) from hits_1000m where UTMMedium != '' group by intHash32(UserID) % 1000000 -select any(UTMMedium) from hits_1000m where UTMMedium != '' group by intHash32(UserID) % 1000000 -select anyHeavy(UTMMedium) from hits_1000m where UTMMedium != '' group by intHash32(UserID) % 1000000 -select min(UTMCampaign) from hits_1000m where UTMCampaign != '' group by intHash32(UserID) % 1000000 -select max(UTMCampaign) from hits_1000m where UTMCampaign != '' group by intHash32(UserID) % 1000000 -select any(UTMCampaign) from hits_1000m where UTMCampaign != '' group by intHash32(UserID) % 1000000 -select anyHeavy(UTMCampaign) from hits_1000m where UTMCampaign != '' group by intHash32(UserID) % 1000000 -select min(UTMContent) from hits_1000m where UTMContent != '' group by intHash32(UserID) % 1000000 -select max(UTMContent) from hits_1000m where UTMContent != '' group by intHash32(UserID) % 1000000 -select any(UTMContent) from hits_1000m where UTMContent != '' group by intHash32(UserID) % 1000000 -select anyHeavy(UTMContent) from hits_1000m where UTMContent != '' group by intHash32(UserID) % 1000000 -select min(UTMTerm) from hits_1000m where UTMTerm != '' group by intHash32(UserID) % 1000000 -select max(UTMTerm) from hits_1000m where UTMTerm != '' group by intHash32(UserID) % 1000000 -select any(UTMTerm) from hits_1000m where UTMTerm != '' group by intHash32(UserID) % 1000000 -select anyHeavy(UTMTerm) from hits_1000m where UTMTerm != '' group by intHash32(UserID) % 1000000 -select min(FromTag) from hits_1000m where FromTag != '' group by intHash32(UserID) % 1000000 -select max(FromTag) from hits_1000m where FromTag != '' group by intHash32(UserID) % 1000000 -select any(FromTag) from hits_1000m where FromTag != '' group by intHash32(UserID) % 1000000 -select anyHeavy(FromTag) from hits_1000m where FromTag != '' group by intHash32(UserID) % 1000000 +select min(Title) from hits_1000m_single where Title != '' group by intHash32(UserID) % 1000000 +select max(Title) from hits_1000m_single where Title != '' group by intHash32(UserID) % 1000000 +select any(Title) from hits_1000m_single where Title != '' group by intHash32(UserID) % 1000000 +select anyHeavy(Title) from hits_1000m_single where Title != '' group by intHash32(UserID) % 1000000 +select min(URL) from hits_1000m_single where URL != '' group by intHash32(UserID) % 1000000 +select max(URL) from hits_1000m_single where URL != '' group by intHash32(UserID) % 1000000 +select any(URL) from hits_1000m_single where URL != '' group by intHash32(UserID) % 1000000 +select anyHeavy(URL) from hits_1000m_single where URL != '' group by intHash32(UserID) % 1000000 +select min(Referer) from hits_1000m_single where Referer != '' group by intHash32(UserID) % 1000000 +select max(Referer) from hits_1000m_single where Referer != '' group by intHash32(UserID) % 1000000 +select any(Referer) from hits_1000m_single where Referer != '' group by intHash32(UserID) % 1000000 +select anyHeavy(Referer) from hits_1000m_single where Referer != '' group by intHash32(UserID) % 1000000 +select min(FlashMinor2) from hits_1000m_single where FlashMinor2 != '' group by intHash32(UserID) % 1000000 +select max(FlashMinor2) from hits_1000m_single where FlashMinor2 != '' group by intHash32(UserID) % 1000000 +select any(FlashMinor2) from hits_1000m_single where FlashMinor2 != '' group by intHash32(UserID) % 1000000 +select anyHeavy(FlashMinor2) from hits_1000m_single where FlashMinor2 != '' group by intHash32(UserID) % 1000000 +select min(MobilePhoneModel) from hits_1000m_single where MobilePhoneModel != '' group by intHash32(UserID) % 1000000 +select max(MobilePhoneModel) from hits_1000m_single where MobilePhoneModel != '' group by intHash32(UserID) % 1000000 +select any(MobilePhoneModel) from hits_1000m_single where MobilePhoneModel != '' group by intHash32(UserID) % 1000000 +select anyHeavy(MobilePhoneModel) from hits_1000m_single where MobilePhoneModel != '' group by intHash32(UserID) % 1000000 +select min(Params) from hits_1000m_single where Params != '' group by intHash32(UserID) % 1000000 +select max(Params) from hits_1000m_single where Params != '' group by intHash32(UserID) % 1000000 +select any(Params) from hits_1000m_single where Params != '' group by intHash32(UserID) % 1000000 +select anyHeavy(Params) from hits_1000m_single where Params != '' group by intHash32(UserID) % 1000000 +select min(SearchPhrase) from hits_1000m_single where SearchPhrase != '' group by intHash32(UserID) % 1000000 +select max(SearchPhrase) from hits_1000m_single where SearchPhrase != '' group by intHash32(UserID) % 1000000 +select any(SearchPhrase) from hits_1000m_single where SearchPhrase != '' group by intHash32(UserID) % 1000000 +select anyHeavy(SearchPhrase) from hits_1000m_single where SearchPhrase != '' group by intHash32(UserID) % 1000000 +select min(PageCharset) from hits_1000m_single where PageCharset != '' group by intHash32(UserID) % 1000000 +select max(PageCharset) from hits_1000m_single where PageCharset != '' group by intHash32(UserID) % 1000000 +select any(PageCharset) from hits_1000m_single where PageCharset != '' group by intHash32(UserID) % 1000000 +select anyHeavy(PageCharset) from hits_1000m_single where PageCharset != '' group by intHash32(UserID) % 1000000 +select min(OriginalURL) from hits_1000m_single where OriginalURL != '' group by intHash32(UserID) % 1000000 +select max(OriginalURL) from hits_1000m_single where OriginalURL != '' group by intHash32(UserID) % 1000000 +select any(OriginalURL) from hits_1000m_single where OriginalURL != '' group by intHash32(UserID) % 1000000 +select anyHeavy(OriginalURL) from hits_1000m_single where OriginalURL != '' group by intHash32(UserID) % 1000000 +select min(SocialNetwork) from hits_1000m_single where SocialNetwork != '' group by intHash32(UserID) % 1000000 +select max(SocialNetwork) from hits_1000m_single where SocialNetwork != '' group by intHash32(UserID) % 1000000 +select any(SocialNetwork) from hits_1000m_single where SocialNetwork != '' group by intHash32(UserID) % 1000000 +select anyHeavy(SocialNetwork) from hits_1000m_single where SocialNetwork != '' group by intHash32(UserID) % 1000000 +select min(SocialAction) from hits_1000m_single where SocialAction != '' group by intHash32(UserID) % 1000000 +select max(SocialAction) from hits_1000m_single where SocialAction != '' group by intHash32(UserID) % 1000000 +select any(SocialAction) from hits_1000m_single where SocialAction != '' group by intHash32(UserID) % 1000000 +select anyHeavy(SocialAction) from hits_1000m_single where SocialAction != '' group by intHash32(UserID) % 1000000 +select min(SocialSourcePage) from hits_1000m_single where SocialSourcePage != '' group by intHash32(UserID) % 1000000 +select max(SocialSourcePage) from hits_1000m_single where SocialSourcePage != '' group by intHash32(UserID) % 1000000 +select any(SocialSourcePage) from hits_1000m_single where SocialSourcePage != '' group by intHash32(UserID) % 1000000 +select anyHeavy(SocialSourcePage) from hits_1000m_single where SocialSourcePage != '' group by intHash32(UserID) % 1000000 +select min(ParamOrderID) from hits_1000m_single where ParamOrderID != '' group by intHash32(UserID) % 1000000 +select max(ParamOrderID) from hits_1000m_single where ParamOrderID != '' group by intHash32(UserID) % 1000000 +select any(ParamOrderID) from hits_1000m_single where ParamOrderID != '' group by intHash32(UserID) % 1000000 +select anyHeavy(ParamOrderID) from hits_1000m_single where ParamOrderID != '' group by intHash32(UserID) % 1000000 +select min(OpenstatServiceName) from hits_1000m_single where OpenstatServiceName != '' group by intHash32(UserID) % 1000000 +select max(OpenstatServiceName) from hits_1000m_single where OpenstatServiceName != '' group by intHash32(UserID) % 1000000 +select any(OpenstatServiceName) from hits_1000m_single where OpenstatServiceName != '' group by intHash32(UserID) % 1000000 +select anyHeavy(OpenstatServiceName) from hits_1000m_single where OpenstatServiceName != '' group by intHash32(UserID) % 1000000 +select min(OpenstatCampaignID) from hits_1000m_single where OpenstatCampaignID != '' group by intHash32(UserID) % 1000000 +select max(OpenstatCampaignID) from hits_1000m_single where OpenstatCampaignID != '' group by intHash32(UserID) % 1000000 +select any(OpenstatCampaignID) from hits_1000m_single where OpenstatCampaignID != '' group by intHash32(UserID) % 1000000 +select anyHeavy(OpenstatCampaignID) from hits_1000m_single where OpenstatCampaignID != '' group by intHash32(UserID) % 1000000 +select min(OpenstatAdID) from hits_1000m_single where OpenstatAdID != '' group by intHash32(UserID) % 1000000 +select max(OpenstatAdID) from hits_1000m_single where OpenstatAdID != '' group by intHash32(UserID) % 1000000 +select any(OpenstatAdID) from hits_1000m_single where OpenstatAdID != '' group by intHash32(UserID) % 1000000 +select anyHeavy(OpenstatAdID) from hits_1000m_single where OpenstatAdID != '' group by intHash32(UserID) % 1000000 +select min(OpenstatSourceID) from hits_1000m_single where OpenstatSourceID != '' group by intHash32(UserID) % 1000000 +select max(OpenstatSourceID) from hits_1000m_single where OpenstatSourceID != '' group by intHash32(UserID) % 1000000 +select any(OpenstatSourceID) from hits_1000m_single where OpenstatSourceID != '' group by intHash32(UserID) % 1000000 +select anyHeavy(OpenstatSourceID) from hits_1000m_single where OpenstatSourceID != '' group by intHash32(UserID) % 1000000 +select min(UTMSource) from hits_1000m_single where UTMSource != '' group by intHash32(UserID) % 1000000 +select max(UTMSource) from hits_1000m_single where UTMSource != '' group by intHash32(UserID) % 1000000 +select any(UTMSource) from hits_1000m_single where UTMSource != '' group by intHash32(UserID) % 1000000 +select anyHeavy(UTMSource) from hits_1000m_single where UTMSource != '' group by intHash32(UserID) % 1000000 +select min(UTMMedium) from hits_1000m_single where UTMMedium != '' group by intHash32(UserID) % 1000000 +select max(UTMMedium) from hits_1000m_single where UTMMedium != '' group by intHash32(UserID) % 1000000 +select any(UTMMedium) from hits_1000m_single where UTMMedium != '' group by intHash32(UserID) % 1000000 +select anyHeavy(UTMMedium) from hits_1000m_single where UTMMedium != '' group by intHash32(UserID) % 1000000 +select min(UTMCampaign) from hits_1000m_single where UTMCampaign != '' group by intHash32(UserID) % 1000000 +select max(UTMCampaign) from hits_1000m_single where UTMCampaign != '' group by intHash32(UserID) % 1000000 +select any(UTMCampaign) from hits_1000m_single where UTMCampaign != '' group by intHash32(UserID) % 1000000 +select anyHeavy(UTMCampaign) from hits_1000m_single where UTMCampaign != '' group by intHash32(UserID) % 1000000 +select min(UTMContent) from hits_1000m_single where UTMContent != '' group by intHash32(UserID) % 1000000 +select max(UTMContent) from hits_1000m_single where UTMContent != '' group by intHash32(UserID) % 1000000 +select any(UTMContent) from hits_1000m_single where UTMContent != '' group by intHash32(UserID) % 1000000 +select anyHeavy(UTMContent) from hits_1000m_single where UTMContent != '' group by intHash32(UserID) % 1000000 +select min(UTMTerm) from hits_1000m_single where UTMTerm != '' group by intHash32(UserID) % 1000000 +select max(UTMTerm) from hits_1000m_single where UTMTerm != '' group by intHash32(UserID) % 1000000 +select any(UTMTerm) from hits_1000m_single where UTMTerm != '' group by intHash32(UserID) % 1000000 +select anyHeavy(UTMTerm) from hits_1000m_single where UTMTerm != '' group by intHash32(UserID) % 1000000 +select min(FromTag) from hits_1000m_single where FromTag != '' group by intHash32(UserID) % 1000000 +select max(FromTag) from hits_1000m_single where FromTag != '' group by intHash32(UserID) % 1000000 +select any(FromTag) from hits_1000m_single where FromTag != '' group by intHash32(UserID) % 1000000 +select anyHeavy(FromTag) from hits_1000m_single where FromTag != '' group by intHash32(UserID) % 1000000 From da0f5554849f7b5e9707225f5dc46aca08c6a15b Mon Sep 17 00:00:00 2001 From: Alexey Zatelepin Date: Wed, 23 Jan 2019 16:20:03 +0300 Subject: [PATCH 16/86] speedup calculation of ASTLiteral hash a bit #4058 --- dbms/src/Parsers/ASTLiteral.cpp | 7 +++++++ dbms/src/Parsers/ASTLiteral.h | 2 ++ dbms/src/Parsers/IAST.cpp | 16 +++++++++++----- dbms/src/Parsers/IAST.h | 3 ++- 4 files changed, 22 insertions(+), 6 deletions(-) diff --git a/dbms/src/Parsers/ASTLiteral.cpp b/dbms/src/Parsers/ASTLiteral.cpp index 44c7662ba43..04504fb3594 100644 --- a/dbms/src/Parsers/ASTLiteral.cpp +++ b/dbms/src/Parsers/ASTLiteral.cpp @@ -7,6 +7,13 @@ namespace DB { +void ASTLiteral::updateTreeHashImpl(SipHash & hash_state) const +{ + const char * prefix = "Literal_"; + hash_state.update(prefix, strlen(prefix)); + applyVisitor(FieldVisitorHash(hash_state), value); +} + void ASTLiteral::appendColumnNameImpl(WriteBuffer & ostr) const { /// Special case for very large arrays. Instead of listing all elements, will use hash of them. diff --git a/dbms/src/Parsers/ASTLiteral.h b/dbms/src/Parsers/ASTLiteral.h index d6b8b4efc3b..dd5bb572e7d 100644 --- a/dbms/src/Parsers/ASTLiteral.h +++ b/dbms/src/Parsers/ASTLiteral.h @@ -22,6 +22,8 @@ public: ASTPtr clone() const override { return std::make_shared(*this); } + void updateTreeHashImpl(SipHash & hash_state) const override; + protected: void formatImplWithoutAlias(const FormatSettings & settings, FormatState &, FormatStateStacked) const override { diff --git a/dbms/src/Parsers/IAST.cpp b/dbms/src/Parsers/IAST.cpp index a3244e1abeb..74956c43166 100644 --- a/dbms/src/Parsers/IAST.cpp +++ b/dbms/src/Parsers/IAST.cpp @@ -53,20 +53,26 @@ size_t IAST::checkSize(size_t max_size) const IAST::Hash IAST::getTreeHash() const { SipHash hash_state; - getTreeHashImpl(hash_state); + updateTreeHash(hash_state); IAST::Hash res; hash_state.get128(res.first, res.second); return res; } -void IAST::getTreeHashImpl(SipHash & hash_state) const +void IAST::updateTreeHash(SipHash & hash_state) const +{ + updateTreeHashImpl(hash_state); + hash_state.update(children.size()); + for (const auto & child : children) + child->updateTreeHash(hash_state); +} + + +void IAST::updateTreeHashImpl(SipHash & hash_state) const { auto id = getID(); hash_state.update(id.data(), id.size()); - hash_state.update(children.size()); - for (const auto & child : children) - child->getTreeHashImpl(hash_state); } diff --git a/dbms/src/Parsers/IAST.h b/dbms/src/Parsers/IAST.h index 563eeb3c125..7692691073d 100644 --- a/dbms/src/Parsers/IAST.h +++ b/dbms/src/Parsers/IAST.h @@ -76,7 +76,8 @@ public: */ using Hash = std::pair; Hash getTreeHash() const; - void getTreeHashImpl(SipHash & hash_state) const; + void updateTreeHash(SipHash & hash_state) const; + virtual void updateTreeHashImpl(SipHash & hash_state) const; void dumpTree(std::ostream & ostr, size_t indent = 0) const { From b623d4302c9d86d21ecc4388d3114f1e8040fab9 Mon Sep 17 00:00:00 2001 From: Alexey Zatelepin Date: Wed, 23 Jan 2019 16:27:01 +0300 Subject: [PATCH 17/86] add clone() calls when copying ASTs from stored mutation commands (just in case) #4058 --- dbms/src/Interpreters/MutationsInterpreter.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/dbms/src/Interpreters/MutationsInterpreter.cpp b/dbms/src/Interpreters/MutationsInterpreter.cpp index 64f30bdbc8a..01db6679e6b 100644 --- a/dbms/src/Interpreters/MutationsInterpreter.cpp +++ b/dbms/src/Interpreters/MutationsInterpreter.cpp @@ -53,7 +53,7 @@ bool MutationsInterpreter::isStorageTouchedByMutations() const select->select_expression_list->children.push_back(count_func); if (commands.size() == 1) - select->where_expression = commands[0].predicate; + select->where_expression = commands[0].predicate->clone(); else { auto coalesced_predicates = std::make_shared(); @@ -62,7 +62,7 @@ bool MutationsInterpreter::isStorageTouchedByMutations() const coalesced_predicates->children.push_back(coalesced_predicates->arguments); for (const MutationCommand & command : commands) - coalesced_predicates->arguments->children.push_back(command.predicate); + coalesced_predicates->arguments->children.push_back(command.predicate->clone()); select->where_expression = std::move(coalesced_predicates); } From e3d5b21d6a0c4a591d1d7852af79bad238f8cc01 Mon Sep 17 00:00:00 2001 From: Odin Hultgren Van Der Horst Date: Wed, 23 Jan 2019 15:05:11 +0100 Subject: [PATCH 18/86] Fixes to clickhouse-test script --- dbms/tests/clickhouse-test | 7 +++++-- 1 file changed, 5 insertions(+), 2 deletions(-) diff --git a/dbms/tests/clickhouse-test b/dbms/tests/clickhouse-test index 32eec7271eb..ac745c894c2 100755 --- a/dbms/tests/clickhouse-test +++ b/dbms/tests/clickhouse-test @@ -369,12 +369,15 @@ if __name__ == '__main__': args.tmp = '/tmp/clickhouse-test' if args.tmp is None: args.tmp = args.queries - if args.client is None: if os.access(args.binary + '-client', os.X_OK): args.client = args.binary + '-client' - else: + elif os.access(args.binary,os.X_OK): args.client = args.binary + ' client' + else: + print("No clickhouse binary found") + parser.print_help(); + exit(0); if args.configclient: args.client += ' --config-file=' + args.configclient if os.getenv("CLICKHOUSE_HOST"): From 7f4b0f3c042b8c3e3d4421fa44eef97fcaf797f2 Mon Sep 17 00:00:00 2001 From: Danila Kutenin Date: Wed, 23 Jan 2019 17:18:19 +0300 Subject: [PATCH 19/86] added powerpc build support --- ci/README.md | 13 +- cmake/arch.cmake | 7 + cmake/test_cpu.cmake | 3 + contrib/libdivide/libdivide.h | 7 + contrib/libhdfs3-cmake/CMake/Options.cmake | 2 +- contrib/librdkafka-cmake/config.h | 2 + contrib/libunwind/CMakeLists.txt | 154 ++++++++++++------ .../include/tdep-ppc64/libunwind_i.h | 1 + .../libunwind/src/ppc/Greg_states_iterate.c | 2 +- .../src/Compression/LZ4_decompress_faster.cpp | 4 +- dbms/src/Core/Defines.h | 4 +- dbms/src/DataTypes/DataTypeString.cpp | 2 +- dbms/src/IO/parseDateTimeBestEffort.cpp | 9 + docs/en/development/tests.md | 3 +- libs/libcommon/cmake/find_jemalloc.cmake | 2 +- libs/libcommon/include/common/DateLUTImpl.h | 9 + 16 files changed, 159 insertions(+), 65 deletions(-) diff --git a/ci/README.md b/ci/README.md index 733cbce80c9..003f547fb79 100644 --- a/ci/README.md +++ b/ci/README.md @@ -25,16 +25,17 @@ Various possible options. We are not going to automate testing all of them. #### CPU architectures: - x86_64; -- AArch64. +- AArch64; +- PowerPC64LE. -x86_64 is the main CPU architecture. We also have minimal support for AArch64. +x86_64 is the main CPU architecture. We also have minimal support for AArch64 and PowerPC64LE. #### Operating systems: - Linux; - FreeBSD. -We also target Mac OS X, but it's more difficult to test. -Linux is the main. FreeBSD is also supported as production OS. +We also target Mac OS X, but it's more difficult to test. +Linux is the main. FreeBSD is also supported as production OS. Mac OS is intended only for development and have minimal support: client should work, server should just start. #### Linux distributions: @@ -98,14 +99,14 @@ We also have intent to build RPM and simple tgz packages. - from contrib directory (submodules); - from OS packages. -The only production option is to use libraries from contrib directory. +The only production option is to use libraries from contrib directory. Using libraries from OS packages is discouraged, but we also support this option. #### Linkage types: - static; - shared; -Static linking is the only option for production usage. +Static linking is the only option for production usage. We also have support for shared linking, but it is indended only for developers. #### Make tools: diff --git a/cmake/arch.cmake b/cmake/arch.cmake index abc30d99e32..8f772d3cdcf 100644 --- a/cmake/arch.cmake +++ b/cmake/arch.cmake @@ -24,3 +24,10 @@ if (CMAKE_CXX_COMPILER_ID STREQUAL "GNU") elseif (CMAKE_CXX_COMPILER_ID STREQUAL "Clang") set (COMPILER_CLANG 1) endif () + +if (CMAKE_SYSTEM_PROCESSOR MATCHES "^(ppc64le.*|PPC64LE.*)") + set (PPC64LE 1) + if (COMPILER_CLANG OR (COMPILER_GCC AND CMAKE_CXX_COMPILER_VERSION VERSION_LESS 8)) + message(FATAL_ERROR "Only gcc-8 is supported for powerpc architecture") + endif () +endif () diff --git a/cmake/test_cpu.cmake b/cmake/test_cpu.cmake index c360de5b962..5b95ac5d25a 100644 --- a/cmake/test_cpu.cmake +++ b/cmake/test_cpu.cmake @@ -27,6 +27,9 @@ if (HAVE_SSE41) set (COMPILER_FLAGS "${COMPILER_FLAGS} ${TEST_FLAG}") endif () +if (PPC64LE) + set (COMPILER_FLAGS "${COMPILER_FLAGS} -maltivec -D__SSE2__=1 -DNO_WARN_X86_INTRINSICS") +endif () # gcc -dM -E -msse4.2 - < /dev/null | sort > gcc-dump-sse42 #define __SSE4_2__ 1 diff --git a/contrib/libdivide/libdivide.h b/contrib/libdivide/libdivide.h index dd3b85008c5..1404295f814 100644 --- a/contrib/libdivide/libdivide.h +++ b/contrib/libdivide/libdivide.h @@ -341,6 +341,13 @@ static inline __m128i libdivide_get_0000FFFF(void) { #pragma clang diagnostic pop #endif +/// This is a bug in gcc-8, _MM_SHUFFLE was forgotten, though in trunk it is ok https://github.com/gcc-mirror/gcc/blob/master/gcc/config/rs6000/xmmintrin.h#L61 +#if __PPC__ +#ifndef _MM_SHUFFLE +#define _MM_SHUFFLE(w,x,y,z) (((w) << 6) | ((x) << 4) | ((y) << 2) | (z)) +#endif +#endif + static inline __m128i libdivide_s64_signbits(__m128i v) { //we want to compute v >> 63, that is, _mm_srai_epi64(v, 63). But there is no 64 bit shift right arithmetic instruction in SSE2. So we have to fake it by first duplicating the high 32 bit values, and then using a 32 bit shift. Another option would be to use _mm_srli_epi64(v, 63) and then subtract that from 0, but that approach appears to be substantially slower for unknown reasons __m128i hiBitsDuped = _mm_shuffle_epi32(v, _MM_SHUFFLE(3, 3, 1, 1)); diff --git a/contrib/libhdfs3-cmake/CMake/Options.cmake b/contrib/libhdfs3-cmake/CMake/Options.cmake index 728aef60e17..c35eac3c08d 100644 --- a/contrib/libhdfs3-cmake/CMake/Options.cmake +++ b/contrib/libhdfs3-cmake/CMake/Options.cmake @@ -7,7 +7,7 @@ CHECK_FUNCTION_EXISTS(nanosleep HAVE_NANOSLEEP) SET(CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} -fno-strict-aliasing") SET(CMAKE_C_FLAGS "${CMAKE_C_FLAGS} -fno-strict-aliasing") -IF(ENABLE_SSE STREQUAL ON) +IF(ENABLE_SSE STREQUAL ON AND NOT PPC64LE AND NOT ARCH_AARCH64 AND NOT ARCH_ARM) SET(CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} -msse4.2") ENDIF(ENABLE_SSE STREQUAL ON) diff --git a/contrib/librdkafka-cmake/config.h b/contrib/librdkafka-cmake/config.h index 2ffc5a497ae..ac732dd0f58 100644 --- a/contrib/librdkafka-cmake/config.h +++ b/contrib/librdkafka-cmake/config.h @@ -60,7 +60,9 @@ // WITH_SASL_SCRAM //#define WITH_SASL_SCRAM 1 // crc32chw +#if !defined(__PPC__) #define WITH_CRC32C_HW 1 +#endif // regex #define HAVE_REGEX 1 // strndup diff --git a/contrib/libunwind/CMakeLists.txt b/contrib/libunwind/CMakeLists.txt index 47032be92bb..22470069938 100644 --- a/contrib/libunwind/CMakeLists.txt +++ b/contrib/libunwind/CMakeLists.txt @@ -1,56 +1,110 @@ - enable_language(ASM) -add_library(unwind -src/mi/init.c -src/mi/flush_cache.c -src/mi/mempool.c -src/mi/strerror.c -src/x86_64/is_fpreg.c -src/x86_64/regname.c -src/mi/_ReadULEB.c -src/mi/_ReadSLEB.c -src/mi/backtrace.c -src/mi/dyn-cancel.c -src/mi/dyn-info-list.c -src/mi/dyn-register.c -src/mi/Ldyn-extract.c -src/mi/Lfind_dynamic_proc_info.c -src/mi/Lget_accessors.c -src/mi/Lget_proc_info_by_ip.c -src/mi/Lget_proc_name.c -src/mi/Lput_dynamic_unwind_info.c -src/mi/Ldestroy_addr_space.c -src/mi/Lget_reg.c -src/mi/Lset_reg.c -src/mi/Lget_fpreg.c -src/mi/Lset_fpreg.c -src/mi/Lset_caching_policy.c -src/x86_64/setcontext.S -src/x86_64/Lcreate_addr_space.c -src/x86_64/Lget_save_loc.c -src/x86_64/Lglobal.c -src/x86_64/Linit.c -src/x86_64/Linit_local.c -src/x86_64/Linit_remote.c -src/x86_64/Lget_proc_info.c -src/x86_64/Lregs.c -src/x86_64/Lresume.c -src/x86_64/Lstash_frame.c -src/x86_64/Lstep.c -src/x86_64/Ltrace.c -src/x86_64/getcontext.S -src/dwarf/Lexpr.c -src/dwarf/Lfde.c -src/dwarf/Lfind_proc_info-lsb.c -src/dwarf/Lparser.c -src/dwarf/Lpe.c -src/dwarf/global.c -src/elf64.c +if (PPC64LE) + add_library(unwind + src/mi/init.c + src/mi/flush_cache.c + src/mi/mempool.c + src/mi/strerror.c + src/mi/_ReadULEB.c + src/mi/_ReadSLEB.c + src/mi/backtrace.c + src/mi/dyn-cancel.c + src/mi/dyn-info-list.c + src/mi/dyn-register.c + src/mi/Ldyn-extract.c + src/mi/Lfind_dynamic_proc_info.c + src/mi/Lget_accessors.c + src/mi/Lget_proc_info_by_ip.c + src/mi/Lget_proc_name.c + src/mi/Lput_dynamic_unwind_info.c + src/mi/Ldestroy_addr_space.c + src/mi/Lget_reg.c + src/mi/Lset_reg.c + src/mi/Lget_fpreg.c + src/mi/Lset_fpreg.c + src/mi/Lset_caching_policy.c + src/dwarf/Lexpr.c + src/dwarf/Lfde.c + src/dwarf/Lfind_proc_info-lsb.c + src/dwarf/Lparser.c + src/dwarf/Lpe.c + src/dwarf/global.c + src/elf64.c + src/os-linux.c -src/os-linux.c -src/x86_64/Los-linux.c -) + src/ppc64/is_fpreg.c + src/ppc64/regname.c + src/ppc64/get_func_addr.c + src/ppc/Linit_local.c + src/ppc/Linit_remote.c + src/ppc/Lis_signal_frame.c + src/ppc/longjmp.S + src/ppc/Lreg_states_iterate.c + src/ppc/siglongjmp.S + src/ppc64/setcontext.S + src/ppc64/Lcreate_addr_space.c + src/ppc64/Lglobal.c + src/ppc64/Linit.c + src/ppc64/Lreg_states_iterate.c + src/ppc64/Lregs.c + src/ppc64/Lresume.c + src/ppc64/Lstep.c + src/ppc64/regname.c + src/ppc64/setcontext.S + ) +else () + add_library(unwind + src/mi/init.c + src/mi/flush_cache.c + src/mi/mempool.c + src/mi/strerror.c + src/mi/_ReadULEB.c + src/mi/_ReadSLEB.c + src/mi/backtrace.c + src/mi/dyn-cancel.c + src/mi/dyn-info-list.c + src/mi/dyn-register.c + src/mi/Ldyn-extract.c + src/mi/Lfind_dynamic_proc_info.c + src/mi/Lget_accessors.c + src/mi/Lget_proc_info_by_ip.c + src/mi/Lget_proc_name.c + src/mi/Lput_dynamic_unwind_info.c + src/mi/Ldestroy_addr_space.c + src/mi/Lget_reg.c + src/mi/Lset_reg.c + src/mi/Lget_fpreg.c + src/mi/Lset_fpreg.c + src/mi/Lset_caching_policy.c + src/dwarf/Lexpr.c + src/dwarf/Lfde.c + src/dwarf/Lfind_proc_info-lsb.c + src/dwarf/Lparser.c + src/dwarf/Lpe.c + src/dwarf/global.c + src/elf64.c + src/os-linux.c + + src/x86_64/is_fpreg.c + src/x86_64/regname.c + src/x86_64/setcontext.S + src/x86_64/Lcreate_addr_space.c + src/x86_64/Lget_save_loc.c + src/x86_64/Lglobal.c + src/x86_64/Linit.c + src/x86_64/Linit_local.c + src/x86_64/Linit_remote.c + src/x86_64/Lget_proc_info.c + src/x86_64/Lregs.c + src/x86_64/Lresume.c + src/x86_64/Lstash_frame.c + src/x86_64/Lstep.c + src/x86_64/Ltrace.c + src/x86_64/getcontext.S + src/x86_64/Los-linux.c + ) +endif() find_file (HAVE_ATOMIC_OPS_H "atomic_ops.h") configure_file (config/config.h.in ${CMAKE_CURRENT_BINARY_DIR}/config/config.h) diff --git a/contrib/libunwind/include/tdep-ppc64/libunwind_i.h b/contrib/libunwind/include/tdep-ppc64/libunwind_i.h index 975f3bb3662..5c50ad63942 100644 --- a/contrib/libunwind/include/tdep-ppc64/libunwind_i.h +++ b/contrib/libunwind/include/tdep-ppc64/libunwind_i.h @@ -37,6 +37,7 @@ WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE SOFTWARE. */ #include #include +#include #include "elf64.h" #include "mempool.h" diff --git a/contrib/libunwind/src/ppc/Greg_states_iterate.c b/contrib/libunwind/src/ppc/Greg_states_iterate.c index a39837a1781..99d967daebc 100644 --- a/contrib/libunwind/src/ppc/Greg_states_iterate.c +++ b/contrib/libunwind/src/ppc/Greg_states_iterate.c @@ -25,7 +25,7 @@ LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE SOFTWARE. */ -#include "unwind_i.h" +#include "../ppc64/unwind_i.h" PROTECTED int unw_reg_states_iterate (unw_cursor_t *cursor, diff --git a/dbms/src/Compression/LZ4_decompress_faster.cpp b/dbms/src/Compression/LZ4_decompress_faster.cpp index 11e222c757a..6c2f072649c 100644 --- a/dbms/src/Compression/LZ4_decompress_faster.cpp +++ b/dbms/src/Compression/LZ4_decompress_faster.cpp @@ -70,7 +70,7 @@ inline void copyOverlap8(UInt8 * op, const UInt8 *& match, const size_t offset) } -#ifdef __x86_64__ +#if defined(__x86_64__) || defined(__PPC__) /** We use 'xmm' (128bit SSE) registers here to shuffle 16 bytes. * @@ -260,7 +260,7 @@ inline void copyOverlap16(UInt8 * op, const UInt8 *& match, const size_t offset) } -#ifdef __x86_64__ +#if defined(__x86_64__) || defined(__PPC__) inline void copyOverlap16Shuffle(UInt8 * op, const UInt8 *& match, const size_t offset) { diff --git a/dbms/src/Core/Defines.h b/dbms/src/Core/Defines.h index ee9ff1cbf79..2333fad774f 100644 --- a/dbms/src/Core/Defines.h +++ b/dbms/src/Core/Defines.h @@ -82,9 +82,9 @@ #endif -#define PLATFORM_NOT_SUPPORTED "The only supported platforms are x86_64 and AArch64 (work in progress)" +#define PLATFORM_NOT_SUPPORTED "The only supported platforms are x86_64 and AArch64, PowerPC (work in progress)" -#if !defined(__x86_64__) && !defined(__aarch64__) +#if !defined(__x86_64__) && !defined(__aarch64__) && !defined(__PPC__) // #error PLATFORM_NOT_SUPPORTED #endif diff --git a/dbms/src/DataTypes/DataTypeString.cpp b/dbms/src/DataTypes/DataTypeString.cpp index 0563a2e01d1..af5b463eff5 100644 --- a/dbms/src/DataTypes/DataTypeString.cpp +++ b/dbms/src/DataTypes/DataTypeString.cpp @@ -129,7 +129,7 @@ static NO_INLINE void deserializeBinarySSE2(ColumnString::Chars & data, ColumnSt if (size) { -#ifdef __SSE2__ +#ifdef __x86_64__ /// An optimistic branch in which more efficient copying is possible. if (offset + 16 * UNROLL_TIMES <= data.allocated_bytes() && istr.position() + size + 16 * UNROLL_TIMES <= istr.buffer().end()) { diff --git a/dbms/src/IO/parseDateTimeBestEffort.cpp b/dbms/src/IO/parseDateTimeBestEffort.cpp index 4aea4d621c2..8c923947a90 100644 --- a/dbms/src/IO/parseDateTimeBestEffort.cpp +++ b/dbms/src/IO/parseDateTimeBestEffort.cpp @@ -44,6 +44,12 @@ inline size_t readAlpha(char * res, size_t max_chars, ReadBuffer & in) return num_chars; } +#if __PPC__ +#if !__clang__ +#pragma GCC diagnostic ignored "-Wmaybe-uninitialized" +#endif +#endif + template inline void readDecimalNumberImpl(T & res, const char * src) { @@ -513,6 +519,9 @@ ReturnType parseDateTimeBestEffortImpl(time_t & res, ReadBuffer & in, const Date } +#if __PPC__ +#pragma GCC diagnostic pop +#endif void parseDateTimeBestEffort(time_t & res, ReadBuffer & in, const DateLUTImpl & local_time_zone, const DateLUTImpl & utc_time_zone) { diff --git a/docs/en/development/tests.md b/docs/en/development/tests.md index 2e2e1660270..bef50139083 100644 --- a/docs/en/development/tests.md +++ b/docs/en/development/tests.md @@ -154,7 +154,8 @@ Normally we release and run all tests on a single variant of ClickHouse build. B - build on FreeBSD; - build on Debian with libraries from system packages; - build with shared linking of libraries; -- build on AArch64 platform. +- build on AArch64 platform; +- build on PowerPc platform. For example, build with system packages is bad practice, because we cannot guarantee what exact version of packages a system will have. But this is really needed by Debian maintainers. For this reason we at least have to support this variant of build. Another example: shared linking is a common source of trouble, but it is needed for some enthusiasts. diff --git a/libs/libcommon/cmake/find_jemalloc.cmake b/libs/libcommon/cmake/find_jemalloc.cmake index d9bc37f9d6c..c611dfc541a 100644 --- a/libs/libcommon/cmake/find_jemalloc.cmake +++ b/libs/libcommon/cmake/find_jemalloc.cmake @@ -1,4 +1,4 @@ -if (OS_LINUX AND NOT SANITIZE AND NOT ARCH_ARM AND NOT ARCH_32) +if (OS_LINUX AND NOT SANITIZE AND NOT ARCH_ARM AND NOT ARCH_32 AND NOT PPC64LE) set(ENABLE_JEMALLOC_DEFAULT 1) else () set(ENABLE_JEMALLOC_DEFAULT 0) diff --git a/libs/libcommon/include/common/DateLUTImpl.h b/libs/libcommon/include/common/DateLUTImpl.h index 8fd015afa15..843237f840f 100644 --- a/libs/libcommon/include/common/DateLUTImpl.h +++ b/libs/libcommon/include/common/DateLUTImpl.h @@ -14,6 +14,11 @@ #define DATE_LUT_MAX_YEAR 2105 /// Last supported year #define DATE_LUT_YEARS (1 + DATE_LUT_MAX_YEAR - DATE_LUT_MIN_YEAR) /// Number of years in lookup table +#if __PPC__ +#if !__clang__ +#pragma GCC diagnostic ignored "-Wmaybe-uninitialized" +#endif +#endif /** Lookup table to conversion of time to date, and to month / year / day of week / day of month and so on. * First time was implemented for OLAPServer, that needed to do billions of such transformations. @@ -684,3 +689,7 @@ public: return s; } }; + +#if __PPC__ +#pragma GCC diagnostic pop +#endif \ No newline at end of file From f5f5674cd6471030580cf55b00343741151605d3 Mon Sep 17 00:00:00 2001 From: Danila Kutenin Date: Wed, 23 Jan 2019 17:21:42 +0300 Subject: [PATCH 20/86] fix --- libs/libcommon/include/common/DateLUTImpl.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/libs/libcommon/include/common/DateLUTImpl.h b/libs/libcommon/include/common/DateLUTImpl.h index 843237f840f..aa9cd34c525 100644 --- a/libs/libcommon/include/common/DateLUTImpl.h +++ b/libs/libcommon/include/common/DateLUTImpl.h @@ -692,4 +692,4 @@ public: #if __PPC__ #pragma GCC diagnostic pop -#endif \ No newline at end of file +#endif From 8ad1a55f3bfe1015f41fb76aecc98df221d08b08 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?L=C3=A9o=20Ercolanelli?= Date: Tue, 22 Jan 2019 17:47:43 +0100 Subject: [PATCH 21/86] implement sumMapFiltered --- .../AggregateFunctionSumMap.cpp | 35 ++++++++++-- .../AggregateFunctionSumMap.h | 54 +++++++++++++++++-- .../0_stateless/00502_sum_map.reference | 2 + .../queries/0_stateless/00502_sum_map.sql | 3 ++ .../agg_functions/parametric_functions.md | 5 ++ 5 files changed, 92 insertions(+), 7 deletions(-) diff --git a/dbms/src/AggregateFunctions/AggregateFunctionSumMap.cpp b/dbms/src/AggregateFunctions/AggregateFunctionSumMap.cpp index 571d6f5c0a1..5138d8f1f02 100644 --- a/dbms/src/AggregateFunctions/AggregateFunctionSumMap.cpp +++ b/dbms/src/AggregateFunctions/AggregateFunctionSumMap.cpp @@ -12,10 +12,10 @@ namespace DB namespace { -AggregateFunctionPtr createAggregateFunctionSumMap(const std::string & name, const DataTypes & arguments, const Array & params) -{ - assertNoParameters(name, params); +using SumMapArgs = std::pair; +SumMapArgs parseArguments(const std::string & name, const DataTypes & arguments) +{ if (arguments.size() < 2) throw Exception("Aggregate function " + name + " requires at least two arguments of Array type.", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); @@ -25,6 +25,7 @@ AggregateFunctionPtr createAggregateFunctionSumMap(const std::string & name, con throw Exception("First argument for function " + name + " must be an array.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + const DataTypePtr & keys_type = array_type->getNestedType(); DataTypes values_types; @@ -37,6 +38,15 @@ AggregateFunctionPtr createAggregateFunctionSumMap(const std::string & name, con values_types.push_back(array_type->getNestedType()); } + return {keys_type, std::move(values_types)}; +} + +AggregateFunctionPtr createAggregateFunctionSumMap(const std::string & name, const DataTypes & arguments, const Array & params) +{ + assertNoParameters(name, params); + + auto [keys_type, values_types] = parseArguments(name, arguments); + AggregateFunctionPtr res(createWithNumericBasedType(*keys_type, keys_type, values_types)); if (!res) res.reset(createWithDecimalType(*keys_type, keys_type, values_types)); @@ -46,11 +56,30 @@ AggregateFunctionPtr createAggregateFunctionSumMap(const std::string & name, con return res; } +AggregateFunctionPtr createAggregateFunctionSumMapFiltered(const std::string & name, const DataTypes & arguments, const Array & params) +{ + if (params.size() != 1) + throw Exception("Aggregate function " + name + "requires exactly one parameter of Array type.", + ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); + + Array keys_to_keep = params.front().safeGet(); + + auto [keys_type, values_types] = parseArguments(name, arguments); + + AggregateFunctionPtr res(createWithNumericBasedType(*keys_type, keys_type, values_types, keys_to_keep)); + if (!res) + res.reset(createWithDecimalType(*keys_type, keys_type, values_types, keys_to_keep)); + if (!res) + throw Exception("Illegal type of argument for aggregate function " + name, ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + + return res; +} } void registerAggregateFunctionSumMap(AggregateFunctionFactory & factory) { factory.registerFunction("sumMap", createAggregateFunctionSumMap); + factory.registerFunction("sumMapFiltered", createAggregateFunctionSumMapFiltered); } } diff --git a/dbms/src/AggregateFunctions/AggregateFunctionSumMap.h b/dbms/src/AggregateFunctions/AggregateFunctionSumMap.h index 4a20a314789..e9c70eaa5f1 100644 --- a/dbms/src/AggregateFunctions/AggregateFunctionSumMap.h +++ b/dbms/src/AggregateFunctions/AggregateFunctionSumMap.h @@ -50,9 +50,9 @@ struct AggregateFunctionSumMapData * ([1,2,3,4,5,6,7,8,9,10],[10,10,45,20,35,20,15,30,20,20]) */ -template -class AggregateFunctionSumMap final : public IAggregateFunctionDataHelper< - AggregateFunctionSumMapData>, AggregateFunctionSumMap> +template +class AggregateFunctionSumMapBase : public IAggregateFunctionDataHelper< + AggregateFunctionSumMapData>, Derived> { private: using ColVecType = std::conditional_t, ColumnDecimal, ColumnVector>; @@ -61,7 +61,7 @@ private: DataTypes values_types; public: - AggregateFunctionSumMap(const DataTypePtr & keys_type, const DataTypes & values_types) + AggregateFunctionSumMapBase(const DataTypePtr & keys_type, const DataTypes & values_types) : keys_type(keys_type), values_types(values_types) {} String getName() const override { return "sumMap"; } @@ -109,6 +109,11 @@ public: array_column.getData().get(values_vec_offset + i, value); const auto & key = keys_vec.getData()[keys_vec_offset + i]; + if (!keepKey(key)) + { + continue; + } + IteratorType it; if constexpr (IsDecimalNumber) { @@ -253,6 +258,47 @@ public: } const char * getHeaderFilePath() const override { return __FILE__; } + + virtual bool keepKey(const T & key) const = 0; +}; + +template +class AggregateFunctionSumMap final : public AggregateFunctionSumMapBase> +{ +public: + AggregateFunctionSumMap(const DataTypePtr & keys_type, DataTypes & values_types) + : AggregateFunctionSumMapBase>{keys_type, values_types} + {} + + String getName() const override { return "sumMap"; } + + bool keepKey(const T &) const override { return true; } +}; + +template +class AggregateFunctionSumMapFiltered final : public AggregateFunctionSumMapBase> +{ +private: + std::vector keys_to_keep; + +public: + AggregateFunctionSumMapFiltered(const DataTypePtr & keys_type, const DataTypes & values_types, const Array & keys_to_keep) + : AggregateFunctionSumMapBase>{keys_type, values_types} + { + this->keys_to_keep.reserve(keys_to_keep.size()); + for (const Field & f : keys_to_keep) + { + this->keys_to_keep.emplace_back(f.safeGet>()); + } + std::sort(begin(this->keys_to_keep), end(this->keys_to_keep)); + } + + String getName() const override { return "sumMapFiltered"; } + + bool keepKey(const T & key) const override + { + return std::binary_search(begin(keys_to_keep), end(keys_to_keep), key); + } }; } diff --git a/dbms/tests/queries/0_stateless/00502_sum_map.reference b/dbms/tests/queries/0_stateless/00502_sum_map.reference index 6da96805974..ac5678ebeab 100644 --- a/dbms/tests/queries/0_stateless/00502_sum_map.reference +++ b/dbms/tests/queries/0_stateless/00502_sum_map.reference @@ -8,6 +8,8 @@ 2000-01-01 00:01:00 ([4,5,6,7,8],[10,10,20,10,10]) 2000-01-01 00:00:00 [1,2,3,4,5] [10,10,20,10,10] 2000-01-01 00:01:00 [4,5,6,7,8] [10,10,20,10,10] +([1],[10]) +([1, 4, 8], [10, 20, 10]) ([1],[1]) ([1],[1]) (['a'],[1]) diff --git a/dbms/tests/queries/0_stateless/00502_sum_map.sql b/dbms/tests/queries/0_stateless/00502_sum_map.sql index e6377155dac..9cf941dd908 100644 --- a/dbms/tests/queries/0_stateless/00502_sum_map.sql +++ b/dbms/tests/queries/0_stateless/00502_sum_map.sql @@ -12,6 +12,9 @@ SELECT sumMapMerge(s) FROM (SELECT sumMapState(statusMap.status, statusMap.reque SELECT timeslot, sumMap(statusMap.status, statusMap.requests) FROM test.sum_map GROUP BY timeslot ORDER BY timeslot; SELECT timeslot, sumMap(statusMap.status, statusMap.requests).1, sumMap(statusMap.status, statusMap.requests).2 FROM test.sum_map GROUP BY timeslot ORDER BY timeslot; +SELECT sumMapFiltered([1])(statusMap.status, statusMap.requests) FROM test.sum_map; +SELECT sumMapFiltered([1, 4, 8])(statusMap.status, statusMap.requests) FROM test.sum_map; + DROP TABLE test.sum_map; select sumMap(val, cnt) from ( SELECT [ CAST(1, 'UInt64') ] as val, [1] as cnt ); diff --git a/docs/en/query_language/agg_functions/parametric_functions.md b/docs/en/query_language/agg_functions/parametric_functions.md index 1cbe784e621..15b9c3360fa 100644 --- a/docs/en/query_language/agg_functions/parametric_functions.md +++ b/docs/en/query_language/agg_functions/parametric_functions.md @@ -155,3 +155,8 @@ Solution: Write in the GROUP BY query SearchPhrase HAVING uniqUpTo(4)(UserID) >= ``` [Original article](https://clickhouse.yandex/docs/en/query_language/agg_functions/parametric_functions/) + + +## sumMapFiltered(keys_to_keep)(keys, values) + +Same behavior as [sumMap](reference.md#sumMap) except that an array of keys is passed as a parameter. This can be especially useful when working with a high cardinality of keys. From f6e6f997377ae5a886df0696046f5633a3263077 Mon Sep 17 00:00:00 2001 From: Danila Kutenin Date: Wed, 23 Jan 2019 17:26:05 +0300 Subject: [PATCH 22/86] fix --- dbms/src/IO/parseDateTimeBestEffort.cpp | 2 ++ libs/libcommon/include/common/DateLUTImpl.h | 2 ++ 2 files changed, 4 insertions(+) diff --git a/dbms/src/IO/parseDateTimeBestEffort.cpp b/dbms/src/IO/parseDateTimeBestEffort.cpp index 8c923947a90..f17c3fc61a8 100644 --- a/dbms/src/IO/parseDateTimeBestEffort.cpp +++ b/dbms/src/IO/parseDateTimeBestEffort.cpp @@ -520,8 +520,10 @@ ReturnType parseDateTimeBestEffortImpl(time_t & res, ReadBuffer & in, const Date } #if __PPC__ +#if !__clang__ #pragma GCC diagnostic pop #endif +#endif void parseDateTimeBestEffort(time_t & res, ReadBuffer & in, const DateLUTImpl & local_time_zone, const DateLUTImpl & utc_time_zone) { diff --git a/libs/libcommon/include/common/DateLUTImpl.h b/libs/libcommon/include/common/DateLUTImpl.h index aa9cd34c525..c529d0938b5 100644 --- a/libs/libcommon/include/common/DateLUTImpl.h +++ b/libs/libcommon/include/common/DateLUTImpl.h @@ -691,5 +691,7 @@ public: }; #if __PPC__ +#if !__clang__ #pragma GCC diagnostic pop #endif +#endif From 05ab15cc84f491e91bfda413f0f72d233bbbb777 Mon Sep 17 00:00:00 2001 From: Ivan Lezhankin Date: Wed, 23 Jan 2019 17:48:50 +0300 Subject: [PATCH 23/86] Merge `IBlockInputStream` and `IProfilingBlockInputStream` into one --- dbms/programs/server/HTTPHandler.cpp | 2 +- dbms/programs/server/TCPHandler.cpp | 51 +-- .../AddingConstColumnBlockInputStream.h | 4 +- .../AddingDefaultsBlockInputStream.h | 4 +- .../AddingMissedBlockInputStream.h | 4 +- .../DataStreams/AggregatingBlockInputStream.h | 4 +- .../AsynchronousBlockInputStream.h | 4 +- .../DataStreams/BlockStreamProfileInfo.cpp | 6 +- dbms/src/DataStreams/BlockStreamProfileInfo.h | 6 +- .../DataStreams/BlocksListBlockInputStream.h | 4 +- .../CollapsingFinalBlockInputStream.h | 4 +- dbms/src/DataStreams/ColumnGathererStream.h | 4 +- dbms/src/DataStreams/ConcatBlockInputStream.h | 4 +- ...lumnLowCardinalityToFullBlockInputStream.h | 4 +- .../DataStreams/ConvertingBlockInputStream.h | 4 +- .../DataStreams/CountingBlockOutputStream.h | 2 +- .../CreatingSetsBlockInputStream.cpp | 22 +- .../CreatingSetsBlockInputStream.h | 4 +- dbms/src/DataStreams/CubeBlockInputStream.h | 4 +- .../DataStreams/DistinctBlockInputStream.h | 4 +- .../DistinctSortedBlockInputStream.h | 4 +- .../ExpressionBlockInputStream.cpp | 7 +- .../DataStreams/ExpressionBlockInputStream.h | 4 +- .../DataStreams/FilterBlockInputStream.cpp | 7 +- dbms/src/DataStreams/FilterBlockInputStream.h | 4 +- .../FilterColumnsBlockInputStream.h | 4 +- .../FinishSortingBlockInputStream.h | 4 +- dbms/src/DataStreams/IBlockInputStream.cpp | 425 ++++++++++++++++- dbms/src/DataStreams/IBlockInputStream.h | 309 ++++++++++--- .../IProfilingBlockInputStream.cpp | 427 ------------------ .../DataStreams/IProfilingBlockInputStream.h | 247 ---------- .../InputStreamFromASTInsertQuery.h | 4 +- dbms/src/DataStreams/LazyBlockInputStream.h | 6 +- .../src/DataStreams/LimitBlockInputStream.cpp | 4 +- dbms/src/DataStreams/LimitBlockInputStream.h | 4 +- .../src/DataStreams/LimitByBlockInputStream.h | 4 +- .../MaterializingBlockInputStream.h | 4 +- .../MergeSortingBlockInputStream.h | 6 +- .../MergingAggregatedBlockInputStream.h | 4 +- ...regatedMemoryEfficientBlockInputStream.cpp | 23 +- ...ggregatedMemoryEfficientBlockInputStream.h | 4 +- .../MergingSortedBlockInputStream.h | 4 +- dbms/src/DataStreams/NativeBlockInputStream.h | 4 +- .../NullAndDoCopyBlockInputStream.h | 4 +- dbms/src/DataStreams/NullBlockInputStream.h | 10 +- dbms/src/DataStreams/OneBlockInputStream.h | 4 +- dbms/src/DataStreams/OwningBlockInputStream.h | 4 +- .../ParallelAggregatingBlockInputStream.h | 4 +- .../src/DataStreams/ParallelInputsProcessor.h | 25 +- .../PartialSortingBlockInputStream.h | 4 +- .../DataStreams/RemoteBlockInputStream.cpp | 3 +- dbms/src/DataStreams/RemoteBlockInputStream.h | 4 +- dbms/src/DataStreams/RollupBlockInputStream.h | 4 +- .../DataStreams/SquashingBlockInputStream.h | 4 +- .../TotalsHavingBlockInputStream.h | 6 +- dbms/src/DataStreams/UnionBlockInputStream.h | 4 +- dbms/src/DataStreams/copyData.cpp | 13 +- .../ClickHouseDictionarySource.cpp | 2 +- .../Dictionaries/DictionaryBlockInputStream.h | 2 +- .../DictionaryBlockInputStreamBase.h | 4 +- .../ExecutableDictionarySource.cpp | 4 +- .../Dictionaries/MongoDBBlockInputStream.h | 4 +- dbms/src/Dictionaries/MySQLBlockInputStream.h | 4 +- dbms/src/Dictionaries/ODBCBlockInputStream.h | 4 +- .../RangeDictionaryBlockInputStream.h | 2 +- .../src/Dictionaries/XDBCDictionarySource.cpp | 4 +- dbms/src/Dictionaries/readInvalidateQuery.cpp | 4 +- dbms/src/Dictionaries/readInvalidateQuery.h | 4 +- .../BlockInputStreamFromRowInputStream.h | 4 +- dbms/src/Interpreters/Aggregator.cpp | 4 +- dbms/src/Interpreters/Context.h | 4 +- dbms/src/Interpreters/DDLWorker.cpp | 4 +- .../InterpreterKillQueryQuery.cpp | 6 +- .../Interpreters/InterpreterSelectQuery.cpp | 21 +- dbms/src/Interpreters/Join.cpp | 4 +- dbms/src/Interpreters/ProcessList.cpp | 7 +- dbms/src/Interpreters/Set.cpp | 2 +- dbms/src/Interpreters/executeQuery.cpp | 59 +-- dbms/src/Storages/Kafka/StorageKafka.cpp | 16 +- .../MergeTreeBaseSelectBlockInputStream.h | 4 +- .../MergeTree/MergeTreeDataMergerMutator.cpp | 2 +- .../MergeTree/MergeTreeDataSelectExecutor.cpp | 2 +- .../MergeTreeSelectBlockInputStream.h | 2 +- .../MergeTreeSequentialBlockInputStream.h | 4 +- dbms/src/Storages/StorageBuffer.cpp | 4 +- dbms/src/Storages/StorageCatBoostPool.cpp | 4 +- dbms/src/Storages/StorageFile.cpp | 4 +- dbms/src/Storages/StorageHDFS.cpp | 4 +- dbms/src/Storages/StorageJoin.cpp | 4 +- dbms/src/Storages/StorageLog.cpp | 4 +- dbms/src/Storages/StorageMemory.cpp | 4 +- dbms/src/Storages/StorageStripeLog.cpp | 4 +- dbms/src/Storages/StorageTinyLog.cpp | 4 +- dbms/src/Storages/StorageURL.cpp | 4 +- .../Storages/System/StorageSystemColumns.cpp | 2 +- .../Storages/System/StorageSystemNumbers.cpp | 4 +- .../Storages/System/StorageSystemTables.cpp | 2 +- dbms/src/Storages/tests/CMakeLists.txt | 2 +- 98 files changed, 923 insertions(+), 1061 deletions(-) delete mode 100644 dbms/src/DataStreams/IProfilingBlockInputStream.cpp delete mode 100644 dbms/src/DataStreams/IProfilingBlockInputStream.h diff --git a/dbms/programs/server/HTTPHandler.cpp b/dbms/programs/server/HTTPHandler.cpp index d86c526784b..01968e19224 100644 --- a/dbms/programs/server/HTTPHandler.cpp +++ b/dbms/programs/server/HTTPHandler.cpp @@ -31,7 +31,7 @@ #include #include -#include +#include #include #include diff --git a/dbms/programs/server/TCPHandler.cpp b/dbms/programs/server/TCPHandler.cpp index c3dff11146e..fbf4e22120a 100644 --- a/dbms/programs/server/TCPHandler.cpp +++ b/dbms/programs/server/TCPHandler.cpp @@ -485,53 +485,44 @@ void TCPHandler::processTablesStatusRequest() void TCPHandler::sendProfileInfo() { - if (const IProfilingBlockInputStream * input = dynamic_cast(state.io.in.get())) - { - writeVarUInt(Protocol::Server::ProfileInfo, *out); - input->getProfileInfo().write(*out); - out->next(); - } + writeVarUInt(Protocol::Server::ProfileInfo, *out); + state.io.in->getProfileInfo().write(*out); + out->next(); } void TCPHandler::sendTotals() { - if (IProfilingBlockInputStream * input = dynamic_cast(state.io.in.get())) + const Block & totals = state.io.in->getTotals(); + + if (totals) { - const Block & totals = input->getTotals(); + initBlockOutput(totals); - if (totals) - { - initBlockOutput(totals); + writeVarUInt(Protocol::Server::Totals, *out); + writeStringBinary("", *out); - writeVarUInt(Protocol::Server::Totals, *out); - writeStringBinary("", *out); - - state.block_out->write(totals); - state.maybe_compressed_out->next(); - out->next(); - } + state.block_out->write(totals); + state.maybe_compressed_out->next(); + out->next(); } } void TCPHandler::sendExtremes() { - if (IProfilingBlockInputStream * input = dynamic_cast(state.io.in.get())) + Block extremes = state.io.in->getExtremes(); + + if (extremes) { - Block extremes = input->getExtremes(); + initBlockOutput(extremes); - if (extremes) - { - initBlockOutput(extremes); + writeVarUInt(Protocol::Server::Extremes, *out); + writeStringBinary("", *out); - writeVarUInt(Protocol::Server::Extremes, *out); - writeStringBinary("", *out); - - state.block_out->write(extremes); - state.maybe_compressed_out->next(); - out->next(); - } + state.block_out->write(extremes); + state.maybe_compressed_out->next(); + out->next(); } } diff --git a/dbms/src/DataStreams/AddingConstColumnBlockInputStream.h b/dbms/src/DataStreams/AddingConstColumnBlockInputStream.h index e22dd7f4310..de51317211d 100644 --- a/dbms/src/DataStreams/AddingConstColumnBlockInputStream.h +++ b/dbms/src/DataStreams/AddingConstColumnBlockInputStream.h @@ -1,6 +1,6 @@ #pragma once -#include +#include #include namespace DB @@ -9,7 +9,7 @@ namespace DB /** Adds a materialized const column to the block with a specified value. */ template -class AddingConstColumnBlockInputStream : public IProfilingBlockInputStream +class AddingConstColumnBlockInputStream : public IBlockInputStream { public: AddingConstColumnBlockInputStream( diff --git a/dbms/src/DataStreams/AddingDefaultsBlockInputStream.h b/dbms/src/DataStreams/AddingDefaultsBlockInputStream.h index 6711a3daee9..436beb9f032 100644 --- a/dbms/src/DataStreams/AddingDefaultsBlockInputStream.h +++ b/dbms/src/DataStreams/AddingDefaultsBlockInputStream.h @@ -1,6 +1,6 @@ #pragma once -#include +#include #include #include @@ -9,7 +9,7 @@ namespace DB { /// Adds defaults to columns using BlockDelayedDefaults bitmask attached to Block by child InputStream. -class AddingDefaultsBlockInputStream : public IProfilingBlockInputStream +class AddingDefaultsBlockInputStream : public IBlockInputStream { public: AddingDefaultsBlockInputStream( diff --git a/dbms/src/DataStreams/AddingMissedBlockInputStream.h b/dbms/src/DataStreams/AddingMissedBlockInputStream.h index b3b98509645..a1314638573 100644 --- a/dbms/src/DataStreams/AddingMissedBlockInputStream.h +++ b/dbms/src/DataStreams/AddingMissedBlockInputStream.h @@ -1,6 +1,6 @@ #pragma once -#include +#include #include @@ -14,7 +14,7 @@ namespace DB * 3. Columns that materialized from other columns (materialized columns) * All three types of columns are materialized (not constants). */ -class AddingMissedBlockInputStream : public IProfilingBlockInputStream +class AddingMissedBlockInputStream : public IBlockInputStream { public: AddingMissedBlockInputStream( diff --git a/dbms/src/DataStreams/AggregatingBlockInputStream.h b/dbms/src/DataStreams/AggregatingBlockInputStream.h index 19483455742..53469744d95 100644 --- a/dbms/src/DataStreams/AggregatingBlockInputStream.h +++ b/dbms/src/DataStreams/AggregatingBlockInputStream.h @@ -3,7 +3,7 @@ #include #include #include -#include +#include namespace DB @@ -15,7 +15,7 @@ namespace DB * If final = false, the aggregate functions are not finalized, that is, they are not replaced by their value, but contain an intermediate state of calculations. * This is necessary so that aggregation can continue (for example, by combining streams of partially aggregated data). */ -class AggregatingBlockInputStream : public IProfilingBlockInputStream +class AggregatingBlockInputStream : public IBlockInputStream { public: /** keys are taken from the GROUP BY part of the query diff --git a/dbms/src/DataStreams/AsynchronousBlockInputStream.h b/dbms/src/DataStreams/AsynchronousBlockInputStream.h index c790deb49c2..20b710b25c8 100644 --- a/dbms/src/DataStreams/AsynchronousBlockInputStream.h +++ b/dbms/src/DataStreams/AsynchronousBlockInputStream.h @@ -2,7 +2,7 @@ #include -#include +#include #include #include #include @@ -26,7 +26,7 @@ namespace DB * has come over the network with a request to interrupt the execution of the query. * It also allows you to execute multiple queries at the same time. */ -class AsynchronousBlockInputStream : public IProfilingBlockInputStream +class AsynchronousBlockInputStream : public IBlockInputStream { public: AsynchronousBlockInputStream(const BlockInputStreamPtr & in) diff --git a/dbms/src/DataStreams/BlockStreamProfileInfo.cpp b/dbms/src/DataStreams/BlockStreamProfileInfo.cpp index 5a31f97a748..0277c19ae42 100644 --- a/dbms/src/DataStreams/BlockStreamProfileInfo.cpp +++ b/dbms/src/DataStreams/BlockStreamProfileInfo.cpp @@ -1,5 +1,5 @@ #include -#include +#include #include #include @@ -77,7 +77,7 @@ void BlockStreamProfileInfo::collectInfosForStreamsWithName(const char * name, B return; } - parent->forEachProfilingChild([&] (IProfilingBlockInputStream & child) + parent->forEachChild([&] (IBlockInputStream & child) { child.getProfileInfo().collectInfosForStreamsWithName(name, res); return false; @@ -107,7 +107,7 @@ void BlockStreamProfileInfo::calculateRowsBeforeLimit() const for (const BlockStreamProfileInfo * info_limit_or_sort : limits_or_sortings) { - info_limit_or_sort->parent->forEachProfilingChild([&] (IProfilingBlockInputStream & child) + info_limit_or_sort->parent->forEachChild([&] (IBlockInputStream & child) { rows_before_limit += child.getProfileInfo().rows; return false; diff --git a/dbms/src/DataStreams/BlockStreamProfileInfo.h b/dbms/src/DataStreams/BlockStreamProfileInfo.h index f1adb5f9ebc..1f6047ab706 100644 --- a/dbms/src/DataStreams/BlockStreamProfileInfo.h +++ b/dbms/src/DataStreams/BlockStreamProfileInfo.h @@ -10,13 +10,13 @@ namespace DB class Block; class ReadBuffer; class WriteBuffer; -class IProfilingBlockInputStream; +class IBlockInputStream; -/// Information for profiling. See IProfilingBlockInputStream.h +/// Information for profiling. See IBlockInputStream.h struct BlockStreamProfileInfo { /// Info about stream object this profile info refers to. - IProfilingBlockInputStream * parent = nullptr; + IBlockInputStream * parent = nullptr; bool started = false; Stopwatch total_stopwatch {CLOCK_MONOTONIC_COARSE}; /// Time with waiting time diff --git a/dbms/src/DataStreams/BlocksListBlockInputStream.h b/dbms/src/DataStreams/BlocksListBlockInputStream.h index 6c8852de4a3..0947fbb4406 100644 --- a/dbms/src/DataStreams/BlocksListBlockInputStream.h +++ b/dbms/src/DataStreams/BlocksListBlockInputStream.h @@ -1,6 +1,6 @@ #pragma once -#include +#include namespace DB @@ -9,7 +9,7 @@ namespace DB /** A stream of blocks from which you can read the next block from an explicitly provided list. * Also see OneBlockInputStream. */ -class BlocksListBlockInputStream : public IProfilingBlockInputStream +class BlocksListBlockInputStream : public IBlockInputStream { public: /// Acquires the ownership of the block list. diff --git a/dbms/src/DataStreams/CollapsingFinalBlockInputStream.h b/dbms/src/DataStreams/CollapsingFinalBlockInputStream.h index b08f31996f0..769c8de7e2b 100644 --- a/dbms/src/DataStreams/CollapsingFinalBlockInputStream.h +++ b/dbms/src/DataStreams/CollapsingFinalBlockInputStream.h @@ -1,6 +1,6 @@ #pragma once #include -#include +#include #include #include #include @@ -12,7 +12,7 @@ namespace DB /// Collapses the same rows with the opposite sign roughly like CollapsingSortedBlockInputStream. /// Outputs the rows in random order (the input streams must still be ordered). /// Outputs only rows with a positive sign. -class CollapsingFinalBlockInputStream : public IProfilingBlockInputStream +class CollapsingFinalBlockInputStream : public IBlockInputStream { public: CollapsingFinalBlockInputStream( diff --git a/dbms/src/DataStreams/ColumnGathererStream.h b/dbms/src/DataStreams/ColumnGathererStream.h index 10b71ca3321..05665ab3f42 100644 --- a/dbms/src/DataStreams/ColumnGathererStream.h +++ b/dbms/src/DataStreams/ColumnGathererStream.h @@ -1,6 +1,6 @@ #pragma once -#include +#include #include #include @@ -53,7 +53,7 @@ using MergedRowSources = PODArray; * Stream mask maps row number to index of source stream. * Streams should contain exactly one column. */ -class ColumnGathererStream : public IProfilingBlockInputStream +class ColumnGathererStream : public IBlockInputStream { public: ColumnGathererStream( diff --git a/dbms/src/DataStreams/ConcatBlockInputStream.h b/dbms/src/DataStreams/ConcatBlockInputStream.h index fafcbc6950c..baaa876e96d 100644 --- a/dbms/src/DataStreams/ConcatBlockInputStream.h +++ b/dbms/src/DataStreams/ConcatBlockInputStream.h @@ -1,6 +1,6 @@ #pragma once -#include +#include namespace DB @@ -11,7 +11,7 @@ namespace DB * Unlike UnionBlockInputStream, it does this sequentially. * Blocks of different sources are not interleaved with each other. */ -class ConcatBlockInputStream : public IProfilingBlockInputStream +class ConcatBlockInputStream : public IBlockInputStream { public: ConcatBlockInputStream(BlockInputStreams inputs_) diff --git a/dbms/src/DataStreams/ConvertColumnLowCardinalityToFullBlockInputStream.h b/dbms/src/DataStreams/ConvertColumnLowCardinalityToFullBlockInputStream.h index 2ced0adbfcf..4e0eb22ff80 100644 --- a/dbms/src/DataStreams/ConvertColumnLowCardinalityToFullBlockInputStream.h +++ b/dbms/src/DataStreams/ConvertColumnLowCardinalityToFullBlockInputStream.h @@ -1,6 +1,6 @@ #pragma once -#include +#include #include #include #include @@ -13,7 +13,7 @@ namespace DB * Unlike UnionBlockInputStream, it does this sequentially. * Blocks of different sources are not interleaved with each other. */ -class ConvertColumnLowCardinalityToFullBlockInputStream : public IProfilingBlockInputStream +class ConvertColumnLowCardinalityToFullBlockInputStream : public IBlockInputStream { public: explicit ConvertColumnLowCardinalityToFullBlockInputStream(const BlockInputStreamPtr & input) diff --git a/dbms/src/DataStreams/ConvertingBlockInputStream.h b/dbms/src/DataStreams/ConvertingBlockInputStream.h index e4511477a72..553d9221dd6 100644 --- a/dbms/src/DataStreams/ConvertingBlockInputStream.h +++ b/dbms/src/DataStreams/ConvertingBlockInputStream.h @@ -1,7 +1,7 @@ #pragma once #include -#include +#include namespace DB @@ -20,7 +20,7 @@ namespace DB * throw if they are const in result and non const in source, * or if they are const and have different values. */ -class ConvertingBlockInputStream : public IProfilingBlockInputStream +class ConvertingBlockInputStream : public IBlockInputStream { public: enum class MatchColumnsMode diff --git a/dbms/src/DataStreams/CountingBlockOutputStream.h b/dbms/src/DataStreams/CountingBlockOutputStream.h index ea1b5ec037d..5c36c40c1ad 100644 --- a/dbms/src/DataStreams/CountingBlockOutputStream.h +++ b/dbms/src/DataStreams/CountingBlockOutputStream.h @@ -1,6 +1,6 @@ #pragma once #include -#include +#include #include diff --git a/dbms/src/DataStreams/CreatingSetsBlockInputStream.cpp b/dbms/src/DataStreams/CreatingSetsBlockInputStream.cpp index f03ba2f66ff..57f8a2e0423 100644 --- a/dbms/src/DataStreams/CreatingSetsBlockInputStream.cpp +++ b/dbms/src/DataStreams/CreatingSetsBlockInputStream.cpp @@ -59,12 +59,7 @@ void CreatingSetsBlockInputStream::readPrefixImpl() Block CreatingSetsBlockInputStream::getTotals() { - auto input = dynamic_cast(children.back().get()); - - if (input) - return input->getTotals(); - else - return totals; + return children.back()->getTotals(); } @@ -158,9 +153,7 @@ void CreatingSetsBlockInputStream::createOne(SubqueryForSet & subquery) if (done_with_set && done_with_join && done_with_table) { - if (IProfilingBlockInputStream * profiling_in = dynamic_cast(&*subquery.source)) - profiling_in->cancel(false); - + subquery.source->cancel(false); break; } } @@ -171,15 +164,12 @@ void CreatingSetsBlockInputStream::createOne(SubqueryForSet & subquery) watch.stop(); size_t head_rows = 0; - if (IProfilingBlockInputStream * profiling_in = dynamic_cast(&*subquery.source)) - { - const BlockStreamProfileInfo & profile_info = profiling_in->getProfileInfo(); + const BlockStreamProfileInfo & profile_info = subquery.source->getProfileInfo(); - head_rows = profile_info.rows; + head_rows = profile_info.rows; - if (subquery.join) - subquery.join->setTotals(profiling_in->getTotals()); - } + if (subquery.join) + subquery.join->setTotals(subquery.source->getTotals()); if (head_rows != 0) { diff --git a/dbms/src/DataStreams/CreatingSetsBlockInputStream.h b/dbms/src/DataStreams/CreatingSetsBlockInputStream.h index ff8fe5683c7..241f43c9a06 100644 --- a/dbms/src/DataStreams/CreatingSetsBlockInputStream.h +++ b/dbms/src/DataStreams/CreatingSetsBlockInputStream.h @@ -1,7 +1,7 @@ #pragma once #include -#include +#include #include /// SubqueriesForSets @@ -14,7 +14,7 @@ namespace DB * in the `readPrefix` function or before reading the first block * initializes all the passed sets. */ -class CreatingSetsBlockInputStream : public IProfilingBlockInputStream +class CreatingSetsBlockInputStream : public IBlockInputStream { public: CreatingSetsBlockInputStream( diff --git a/dbms/src/DataStreams/CubeBlockInputStream.h b/dbms/src/DataStreams/CubeBlockInputStream.h index e7f7bc91592..2f435a6031c 100644 --- a/dbms/src/DataStreams/CubeBlockInputStream.h +++ b/dbms/src/DataStreams/CubeBlockInputStream.h @@ -1,6 +1,6 @@ #pragma once -#include +#include #include #include @@ -14,7 +14,7 @@ class ExpressionActions; /** Takes blocks after grouping, with non-finalized aggregate functions. * Calculates all subsets of columns and aggreagetes over them. */ -class CubeBlockInputStream : public IProfilingBlockInputStream +class CubeBlockInputStream : public IBlockInputStream { private: using ExpressionActionsPtr = std::shared_ptr; diff --git a/dbms/src/DataStreams/DistinctBlockInputStream.h b/dbms/src/DataStreams/DistinctBlockInputStream.h index 3ab915cb964..ca55c7e7ac4 100644 --- a/dbms/src/DataStreams/DistinctBlockInputStream.h +++ b/dbms/src/DataStreams/DistinctBlockInputStream.h @@ -1,6 +1,6 @@ #pragma once -#include +#include #include namespace DB @@ -13,7 +13,7 @@ namespace DB * set limit_hint to non zero value. So we stop emitting new rows after * count of already emitted rows will reach the limit_hint. */ -class DistinctBlockInputStream : public IProfilingBlockInputStream +class DistinctBlockInputStream : public IBlockInputStream { public: /// Empty columns_ means all collumns. diff --git a/dbms/src/DataStreams/DistinctSortedBlockInputStream.h b/dbms/src/DataStreams/DistinctSortedBlockInputStream.h index 5673e1376b0..661a2897ff5 100644 --- a/dbms/src/DataStreams/DistinctSortedBlockInputStream.h +++ b/dbms/src/DataStreams/DistinctSortedBlockInputStream.h @@ -1,6 +1,6 @@ #pragma once -#include +#include #include @@ -17,7 +17,7 @@ namespace DB * set limit_hint to non zero value. So we stop emitting new rows after * count of already emitted rows will reach the limit_hint. */ -class DistinctSortedBlockInputStream : public IProfilingBlockInputStream +class DistinctSortedBlockInputStream : public IBlockInputStream { public: /// Empty columns_ means all collumns. diff --git a/dbms/src/DataStreams/ExpressionBlockInputStream.cpp b/dbms/src/DataStreams/ExpressionBlockInputStream.cpp index 3eb559abaad..1aed849a42a 100644 --- a/dbms/src/DataStreams/ExpressionBlockInputStream.cpp +++ b/dbms/src/DataStreams/ExpressionBlockInputStream.cpp @@ -15,11 +15,8 @@ String ExpressionBlockInputStream::getName() const { return "Expression"; } Block ExpressionBlockInputStream::getTotals() { - if (IProfilingBlockInputStream * child = dynamic_cast(&*children.back())) - { - totals = child->getTotals(); - expression->executeOnTotals(totals); - } + totals = children.back()->getTotals(); + expression->executeOnTotals(totals); return totals; } diff --git a/dbms/src/DataStreams/ExpressionBlockInputStream.h b/dbms/src/DataStreams/ExpressionBlockInputStream.h index 022a573f858..9a3452ce1f3 100644 --- a/dbms/src/DataStreams/ExpressionBlockInputStream.h +++ b/dbms/src/DataStreams/ExpressionBlockInputStream.h @@ -1,6 +1,6 @@ #pragma once -#include +#include namespace DB @@ -13,7 +13,7 @@ class ExpressionActions; * For example: hits * 2 + 3, url LIKE '%yandex%' * The expression processes each row independently of the others. */ -class ExpressionBlockInputStream : public IProfilingBlockInputStream +class ExpressionBlockInputStream : public IBlockInputStream { private: using ExpressionActionsPtr = std::shared_ptr; diff --git a/dbms/src/DataStreams/FilterBlockInputStream.cpp b/dbms/src/DataStreams/FilterBlockInputStream.cpp index 24a429aaf3a..53410c024a3 100644 --- a/dbms/src/DataStreams/FilterBlockInputStream.cpp +++ b/dbms/src/DataStreams/FilterBlockInputStream.cpp @@ -52,11 +52,8 @@ String FilterBlockInputStream::getName() const { return "Filter"; } Block FilterBlockInputStream::getTotals() { - if (IProfilingBlockInputStream * child = dynamic_cast(&*children.back())) - { - totals = child->getTotals(); - expression->executeOnTotals(totals); - } + totals = children.back()->getTotals(); + expression->executeOnTotals(totals); return totals; } diff --git a/dbms/src/DataStreams/FilterBlockInputStream.h b/dbms/src/DataStreams/FilterBlockInputStream.h index ca63b34f45c..9bee8a50c8a 100644 --- a/dbms/src/DataStreams/FilterBlockInputStream.h +++ b/dbms/src/DataStreams/FilterBlockInputStream.h @@ -1,6 +1,6 @@ #pragma once -#include +#include #include @@ -14,7 +14,7 @@ class ExpressionActions; * A stream of blocks and an expression, which adds to the block one ColumnUInt8 column containing the filtering conditions, are passed as input. * The expression is evaluated and a stream of blocks is returned, which contains only the filtered rows. */ -class FilterBlockInputStream : public IProfilingBlockInputStream +class FilterBlockInputStream : public IBlockInputStream { private: using ExpressionActionsPtr = std::shared_ptr; diff --git a/dbms/src/DataStreams/FilterColumnsBlockInputStream.h b/dbms/src/DataStreams/FilterColumnsBlockInputStream.h index dc63ec2823f..4416287195d 100644 --- a/dbms/src/DataStreams/FilterColumnsBlockInputStream.h +++ b/dbms/src/DataStreams/FilterColumnsBlockInputStream.h @@ -1,6 +1,6 @@ #pragma once -#include +#include #include namespace DB @@ -9,7 +9,7 @@ namespace DB /// Removes columns other than columns_to_save_ from block, /// and reorders columns as in columns_to_save_. /// Functionality is similar to ExpressionBlockInputStream with ExpressionActions containing PROJECT action. -class FilterColumnsBlockInputStream : public IProfilingBlockInputStream +class FilterColumnsBlockInputStream : public IBlockInputStream { public: FilterColumnsBlockInputStream( diff --git a/dbms/src/DataStreams/FinishSortingBlockInputStream.h b/dbms/src/DataStreams/FinishSortingBlockInputStream.h index 6688eefbca7..b03eb24028b 100644 --- a/dbms/src/DataStreams/FinishSortingBlockInputStream.h +++ b/dbms/src/DataStreams/FinishSortingBlockInputStream.h @@ -2,7 +2,7 @@ #include #include -#include +#include namespace DB @@ -11,7 +11,7 @@ namespace DB /** Takes stream already sorted by `x` and finishes sorting it by (`x`, `y`). * During sorting only blocks with rows that equal by `x` saved in RAM. * */ -class FinishSortingBlockInputStream : public IProfilingBlockInputStream +class FinishSortingBlockInputStream : public IBlockInputStream { public: /// limit - if not 0, allowed to return just first 'limit' rows in sorted order. diff --git a/dbms/src/DataStreams/IBlockInputStream.cpp b/dbms/src/DataStreams/IBlockInputStream.cpp index 9242c94173f..31e6f9e1de8 100644 --- a/dbms/src/DataStreams/IBlockInputStream.cpp +++ b/dbms/src/DataStreams/IBlockInputStream.cpp @@ -1,9 +1,14 @@ -#include - -#include #include -#include +#include +#include +#include + + +namespace ProfileEvents +{ + extern const Event ThrottlerSleepMicroseconds; +} namespace DB @@ -11,12 +16,414 @@ namespace DB namespace ErrorCodes { + extern const int TOO_MANY_ROWS; + extern const int TOO_MANY_BYTES; + extern const int TOO_MANY_ROWS_OR_BYTES; + extern const int TIMEOUT_EXCEEDED; + extern const int TOO_SLOW; + extern const int LOGICAL_ERROR; + extern const int BLOCKS_HAVE_DIFFERENT_STRUCTURE; extern const int TOO_DEEP_PIPELINE; } -/** It's safe to access children without mutex as long as these methods are called before first call to read, readPrefix. - */ +/// It's safe to access children without mutex as long as these methods are called before first call to `read()` or `readPrefix()`. + + +Block IBlockInputStream::read() +{ + if (total_rows_approx) + { + progressImpl(Progress(0, 0, total_rows_approx)); + total_rows_approx = 0; + } + + if (!info.started) + { + info.total_stopwatch.start(); + info.started = true; + } + + Block res; + + if (isCancelledOrThrowIfKilled()) + return res; + + if (!checkTimeLimit()) + limit_exceeded_need_break = true; + + if (!limit_exceeded_need_break) + res = readImpl(); + + if (res) + { + info.update(res); + + if (enabled_extremes) + updateExtremes(res); + + if (limits.mode == LIMITS_CURRENT && !limits.size_limits.check(info.rows, info.bytes, "result", ErrorCodes::TOO_MANY_ROWS_OR_BYTES)) + limit_exceeded_need_break = true; + + if (quota != nullptr) + checkQuota(res); + } + else + { + /** If the thread is over, then we will ask all children to abort the execution. + * This makes sense when running a query with LIMIT + * - there is a situation when all the necessary data has already been read, + * but children sources are still working, + * herewith they can work in separate threads or even remotely. + */ + cancel(false); + } + + progress(Progress(res.rows(), res.bytes())); + +#ifndef NDEBUG + if (res) + { + Block header = getHeader(); + if (header) + assertBlocksHaveEqualStructure(res, header, getName()); + } +#endif + + return res; +} + + +void IBlockInputStream::readPrefix() +{ + readPrefixImpl(); + + forEachChild([&] (IBlockInputStream & child) + { + child.readPrefix(); + return false; + }); +} + + +void IBlockInputStream::readSuffix() +{ + forEachChild([&] (IBlockInputStream & child) + { + child.readSuffix(); + return false; + }); + + readSuffixImpl(); +} + + +void IBlockInputStream::updateExtremes(Block & block) +{ + size_t num_columns = block.columns(); + + if (!extremes) + { + MutableColumns extremes_columns(num_columns); + + for (size_t i = 0; i < num_columns; ++i) + { + const ColumnPtr & src = block.safeGetByPosition(i).column; + + if (src->isColumnConst()) + { + /// Equal min and max. + extremes_columns[i] = src->cloneResized(2); + } + else + { + Field min_value; + Field max_value; + + src->getExtremes(min_value, max_value); + + extremes_columns[i] = src->cloneEmpty(); + + extremes_columns[i]->insert(min_value); + extremes_columns[i]->insert(max_value); + } + } + + extremes = block.cloneWithColumns(std::move(extremes_columns)); + } + else + { + for (size_t i = 0; i < num_columns; ++i) + { + ColumnPtr & old_extremes = extremes.safeGetByPosition(i).column; + + if (old_extremes->isColumnConst()) + continue; + + Field min_value = (*old_extremes)[0]; + Field max_value = (*old_extremes)[1]; + + Field cur_min_value; + Field cur_max_value; + + block.safeGetByPosition(i).column->getExtremes(cur_min_value, cur_max_value); + + if (cur_min_value < min_value) + min_value = cur_min_value; + if (cur_max_value > max_value) + max_value = cur_max_value; + + MutableColumnPtr new_extremes = old_extremes->cloneEmpty(); + + new_extremes->insert(min_value); + new_extremes->insert(max_value); + + old_extremes = std::move(new_extremes); + } + } +} + + +static bool handleOverflowMode(OverflowMode mode, const String & message, int code) +{ + switch (mode) + { + case OverflowMode::THROW: + throw Exception(message, code); + case OverflowMode::BREAK: + return false; + default: + throw Exception("Logical error: unknown overflow mode", ErrorCodes::LOGICAL_ERROR); + } +} + + +bool IBlockInputStream::checkTimeLimit() +{ + if (limits.max_execution_time != 0 + && info.total_stopwatch.elapsed() > static_cast(limits.max_execution_time.totalMicroseconds()) * 1000) + return handleOverflowMode(limits.timeout_overflow_mode, + "Timeout exceeded: elapsed " + toString(info.total_stopwatch.elapsedSeconds()) + + " seconds, maximum: " + toString(limits.max_execution_time.totalMicroseconds() / 1000000.0), + ErrorCodes::TIMEOUT_EXCEEDED); + + return true; +} + + +void IBlockInputStream::checkQuota(Block & block) +{ + switch (limits.mode) + { + case LIMITS_TOTAL: + /// Checked in `progress` method. + break; + + case LIMITS_CURRENT: + { + time_t current_time = time(nullptr); + double total_elapsed = info.total_stopwatch.elapsedSeconds(); + + quota->checkAndAddResultRowsBytes(current_time, block.rows(), block.bytes()); + quota->checkAndAddExecutionTime(current_time, Poco::Timespan((total_elapsed - prev_elapsed) * 1000000.0)); + + prev_elapsed = total_elapsed; + break; + } + } +} + + +void IBlockInputStream::progressImpl(const Progress & value) +{ + if (progress_callback) + progress_callback(value); + + if (process_list_elem) + { + if (!process_list_elem->updateProgressIn(value)) + cancel(/* kill */ true); + + /// The total amount of data processed or intended for processing in all leaf sources, possibly on remote servers. + + ProgressValues progress = process_list_elem->getProgressIn(); + size_t total_rows_estimate = std::max(progress.rows, progress.total_rows); + + /** Check the restrictions on the amount of data to read, the speed of the query, the quota on the amount of data to read. + * NOTE: Maybe it makes sense to have them checked directly in ProcessList? + */ + + if (limits.mode == LIMITS_TOTAL + && ((limits.size_limits.max_rows && total_rows_estimate > limits.size_limits.max_rows) + || (limits.size_limits.max_bytes && progress.bytes > limits.size_limits.max_bytes))) + { + switch (limits.size_limits.overflow_mode) + { + case OverflowMode::THROW: + { + if (limits.size_limits.max_rows && total_rows_estimate > limits.size_limits.max_rows) + throw Exception("Limit for rows to read exceeded: " + toString(total_rows_estimate) + + " rows read (or to read), maximum: " + toString(limits.size_limits.max_rows), + ErrorCodes::TOO_MANY_ROWS); + else + throw Exception("Limit for (uncompressed) bytes to read exceeded: " + toString(progress.bytes) + + " bytes read, maximum: " + toString(limits.size_limits.max_bytes), + ErrorCodes::TOO_MANY_BYTES); + } + + case OverflowMode::BREAK: + { + /// For `break`, we will stop only if so many rows were actually read, and not just supposed to be read. + if ((limits.size_limits.max_rows && progress.rows > limits.size_limits.max_rows) + || (limits.size_limits.max_bytes && progress.bytes > limits.size_limits.max_bytes)) + { + cancel(false); + } + + break; + } + + default: + throw Exception("Logical error: unknown overflow mode", ErrorCodes::LOGICAL_ERROR); + } + } + + size_t total_rows = progress.total_rows; + + constexpr UInt64 profile_events_update_period_microseconds = 10 * 1000; // 10 milliseconds + UInt64 total_elapsed_microseconds = info.total_stopwatch.elapsedMicroseconds(); + + if (last_profile_events_update_time + profile_events_update_period_microseconds < total_elapsed_microseconds) + { + CurrentThread::updatePerformanceCounters(); + last_profile_events_update_time = total_elapsed_microseconds; + } + + if ((limits.min_execution_speed || (total_rows && limits.timeout_before_checking_execution_speed != 0)) + && (static_cast(total_elapsed_microseconds) > limits.timeout_before_checking_execution_speed.totalMicroseconds())) + { + /// Do not count sleeps in throttlers + UInt64 throttler_sleep_microseconds = CurrentThread::getProfileEvents()[ProfileEvents::ThrottlerSleepMicroseconds]; + double elapsed_seconds = (throttler_sleep_microseconds > total_elapsed_microseconds) + ? 0.0 : (total_elapsed_microseconds - throttler_sleep_microseconds) / 1000000.0; + + if (elapsed_seconds > 0) + { + if (limits.min_execution_speed && progress.rows / elapsed_seconds < limits.min_execution_speed) + throw Exception("Query is executing too slow: " + toString(progress.rows / elapsed_seconds) + + " rows/sec., minimum: " + toString(limits.min_execution_speed), + ErrorCodes::TOO_SLOW); + + /// If the predicted execution time is longer than `max_execution_time`. + if (limits.max_execution_time != 0 && total_rows) + { + double estimated_execution_time_seconds = elapsed_seconds * (static_cast(total_rows) / progress.rows); + + if (estimated_execution_time_seconds > limits.max_execution_time.totalSeconds()) + throw Exception("Estimated query execution time (" + toString(estimated_execution_time_seconds) + " seconds)" + + " is too long. Maximum: " + toString(limits.max_execution_time.totalSeconds()) + + ". Estimated rows to process: " + toString(total_rows), + ErrorCodes::TOO_SLOW); + } + } + } + + if (quota != nullptr && limits.mode == LIMITS_TOTAL) + { + quota->checkAndAddReadRowsBytes(time(nullptr), value.rows, value.bytes); + } + } +} + + +void IBlockInputStream::cancel(bool kill) +{ + if (kill) + is_killed = true; + + bool old_val = false; + if (!is_cancelled.compare_exchange_strong(old_val, true, std::memory_order_seq_cst, std::memory_order_relaxed)) + return; + + forEachChild([&] (IBlockInputStream & child) + { + child.cancel(kill); + return false; + }); +} + + +bool IBlockInputStream::isCancelled() const +{ + return is_cancelled; +} + +bool IBlockInputStream::isCancelledOrThrowIfKilled() const +{ + if (!is_cancelled) + return false; + if (is_killed) + throw Exception("Query was cancelled", ErrorCodes::QUERY_WAS_CANCELLED); + return true; +} + + +void IBlockInputStream::setProgressCallback(const ProgressCallback & callback) +{ + progress_callback = callback; + + forEachChild([&] (IBlockInputStream & child) + { + child.setProgressCallback(callback); + return false; + }); +} + + +void IBlockInputStream::setProcessListElement(QueryStatus * elem) +{ + process_list_elem = elem; + + forEachChild([&] (IBlockInputStream & child) + { + child.setProcessListElement(elem); + return false; + }); +} + + +Block IBlockInputStream::getTotals() +{ + if (totals) + return totals; + + Block res; + forEachChild([&] (IBlockInputStream & child) + { + res = child.getTotals(); + if (res) + return true; + return false; + }); + return res; +} + + +Block IBlockInputStream::getExtremes() +{ + if (extremes) + return extremes; + + Block res; + forEachChild([&] (IBlockInputStream & child) + { + res = child.getExtremes(); + if (res) + return true; + return false; + }); + return res; +} String IBlockInputStream::getTreeID() const @@ -40,11 +447,6 @@ String IBlockInputStream::getTreeID() const } -size_t IBlockInputStream::checkDepth(size_t max_depth) const -{ - return checkDepthImpl(max_depth, max_depth); -} - size_t IBlockInputStream::checkDepthImpl(size_t max_depth, size_t level) const { if (children.empty()) @@ -94,4 +496,3 @@ void IBlockInputStream::dumpTree(std::ostream & ostr, size_t indent, size_t mult } } - diff --git a/dbms/src/DataStreams/IBlockInputStream.h b/dbms/src/DataStreams/IBlockInputStream.h index c82761dc02b..6d2ddbfdf2c 100644 --- a/dbms/src/DataStreams/IBlockInputStream.h +++ b/dbms/src/DataStreams/IBlockInputStream.h @@ -1,41 +1,38 @@ #pragma once -#include -#include -#include -#include -#include -#include #include #include +#include +#include +#include +#include + +#include +#include namespace DB { - -class IBlockInputStream; - -using BlockInputStreamPtr = std::shared_ptr; -using BlockInputStreams = std::vector; - -class TableStructureReadLock; - -using TableStructureReadLockPtr = std::shared_ptr; -using TableStructureReadLocks = std::vector; -using TableStructureReadLocksList = std::list; - -struct Progress; - namespace ErrorCodes { extern const int OUTPUT_IS_NOT_SORTED; - extern const int NOT_IMPLEMENTED; + extern const int QUERY_WAS_CANCELLED; } +class IBlockInputStream; +class ProcessListElement; +class QuotaForIntervals; +class QueryStatus; +class TableStructureReadLock; + +using BlockInputStreamPtr = std::shared_ptr; +using BlockInputStreams = std::vector; +using TableStructureReadLockPtr = std::shared_ptr; +using TableStructureReadLocks = std::vector; /** Callback to track the progress of the query. - * Used in IProfilingBlockInputStream and Context. + * Used in IBlockInputStream and Context. * The function takes the number of rows in the last block, the number of bytes in the last block. * Note that the callback can be called from different threads. */ @@ -44,11 +41,23 @@ using ProgressCallback = std::function; /** The stream interface for reading data by blocks from the database. * Relational operations are supposed to be done also as implementations of this interface. + * Watches out at how the source of the blocks works. + * Lets you get information for profiling: rows per second, blocks per second, megabytes per second, etc. + * Allows you to stop reading data (in nested sources). */ -class IBlockInputStream : private boost::noncopyable +class IBlockInputStream { + friend struct BlockStreamProfileInfo; + public: - IBlockInputStream() {} + IBlockInputStream() { info.parent = this; } + virtual ~IBlockInputStream() {} + + IBlockInputStream(const IBlockInputStream &) = delete; + IBlockInputStream & operator=(const IBlockInputStream &) = delete; + + /// To output the data stream transformation tree (query execution plan). + virtual String getName() const = 0; /** Get data structure of the stream in a form of "header" block (it is also called "sample block"). * Header block contains column names, data types, columns of size 0. Constant columns must have corresponding values. @@ -56,52 +65,244 @@ public: */ virtual Block getHeader() const = 0; - /** Read next block. - * If there are no more blocks, return an empty block (for which operator `bool` returns false). - * NOTE: Only one thread can read from one instance of IBlockInputStream simultaneously. - * This also applies for readPrefix, readSuffix. - */ - virtual Block read() = 0; - virtual const BlockMissingValues & getMissingValues() const { static const BlockMissingValues none; return none; } + /// If this stream generates data in order by some keys, return true. + virtual bool isSortedOutput() const { return false; } + + /// In case of isSortedOutput, return corresponding SortDescription + virtual const SortDescription & getSortDescription() const + { + throw Exception("Output of " + getName() + " is not sorted", ErrorCodes::OUTPUT_IS_NOT_SORTED); + } + + /** Read next block. + * If there are no more blocks, return an empty block (for which operator `bool` returns false). + * NOTE: Only one thread can read from one instance of IBlockInputStream simultaneously. + * This also applies for readPrefix, readSuffix. + */ + Block read(); + /** Read something before starting all data or after the end of all data. * In the `readSuffix` function, you can implement a finalization that can lead to an exception. * readPrefix() must be called before the first call to read(). * readSuffix() should be called after read() returns an empty block, or after a call to cancel(), but not during read() execution. */ - virtual void readPrefix() {} - virtual void readSuffix() {} - virtual ~IBlockInputStream() {} - - /** To output the data stream transformation tree (query execution plan). + /** The default implementation calls readPrefixImpl() on itself, and then readPrefix() recursively for all children. + * There are cases when you do not want `readPrefix` of children to be called synchronously, in this function, + * but you want them to be called, for example, in separate threads (for parallel initialization of children). + * Then overload `readPrefix` function. */ - virtual String getName() const = 0; + virtual void readPrefix(); - /// If this stream generates data in order by some keys, return true. - virtual bool isSortedOutput() const { return false; } - /// In case of isSortedOutput, return corresponding SortDescription - virtual const SortDescription & getSortDescription() const { throw Exception("Output of " + getName() + " is not sorted", ErrorCodes::OUTPUT_IS_NOT_SORTED); } - - /** Must be called before read, readPrefix. + /** The default implementation calls recursively readSuffix() on all children, and then readSuffixImpl() on itself. + * If this stream calls read() in children in a separate thread, this behavior is usually incorrect: + * readSuffix() of the child can not be called at the moment when the same child's read() is executed in another thread. + * In this case, you need to override this method so that readSuffix() in children is called, for example, after connecting streams. */ + virtual void readSuffix(); + + /// Must be called before `read()` and `readPrefix()`. void dumpTree(std::ostream & ostr, size_t indent = 0, size_t multiplier = 1) const; /** Check the depth of the pipeline. * If max_depth is specified and the `depth` is greater - throw an exception. - * Must be called before read, readPrefix. + * Must be called before `read()` and `readPrefix()`. */ - size_t checkDepth(size_t max_depth) const; + size_t checkDepth(size_t max_depth) const { return checkDepthImpl(max_depth, max_depth); } - /** Do not allow to change the table while the blocks stream is alive. - */ + /// Do not allow to change the table while the blocks stream is alive. void addTableLock(const TableStructureReadLockPtr & lock) { table_locks.push_back(lock); } + /// Get information about execution speed. + const BlockStreamProfileInfo & getProfileInfo() const { return info; } + + /** Get "total" values. + * The default implementation takes them from itself or from the first child source in which they are. + * The overridden method can perform some calculations. For example, apply an expression to the `totals` of the child source. + * There can be no total values - then an empty block is returned. + * + * Call this method only after all the data has been retrieved with `read`, + * otherwise there will be problems if any data at the same time is computed in another thread. + */ + virtual Block getTotals(); + + /// The same for minimums and maximums. + Block getExtremes(); + + + /** Set the execution progress bar callback. + * The callback is passed to all child sources. + * By default, it is called for leaf sources, after each block. + * (But this can be overridden in the progress() method) + * The function takes the number of rows in the last block, the number of bytes in the last block. + * Note that the callback can be called from different threads. + */ + void setProgressCallback(const ProgressCallback & callback); + + + /** In this method: + * - the progress callback is called; + * - the status of the query execution in ProcessList is updated; + * - checks restrictions and quotas that should be checked not within the same source, + * but over the total amount of resources spent in all sources at once (information in the ProcessList). + */ + virtual void progress(const Progress & value) + { + /// The data for progress is taken from leaf sources. + if (children.empty()) + progressImpl(value); + } + + void progressImpl(const Progress & value); + + + /** Set the pointer to the process list item. + * It is passed to all child sources. + * General information about the resources spent on the request will be written into it. + * Based on this information, the quota and some restrictions will be checked. + * This information will also be available in the SHOW PROCESSLIST request. + */ + void setProcessListElement(QueryStatus * elem); + + /** Set the approximate total number of rows to read. + */ + void addTotalRowsApprox(size_t value) { total_rows_approx += value; } + + + /** Ask to abort the receipt of data as soon as possible. + * By default - just sets the flag is_cancelled and asks that all children be interrupted. + * This function can be called several times, including simultaneously from different threads. + * Have two modes: + * with kill = false only is_cancelled is set - streams will stop silently with returning some processed data. + * with kill = true also is_killed set - queries will stop with exception. + */ + virtual void cancel(bool kill); + + bool isCancelled() const; + bool isCancelledOrThrowIfKilled() const; + + /** What limitations and quotas should be checked. + * LIMITS_CURRENT - checks amount of data read by current stream only (BlockStreamProfileInfo is used for check). + * Currently it is used in root streams to check max_result_{rows,bytes} limits. + * LIMITS_TOTAL - checks total amount of read data from leaf streams (i.e. data read from disk and remote servers). + * It is checks max_{rows,bytes}_to_read in progress handler and use info from ProcessListElement::progress_in for this. + * Currently this check is performed only in leaf streams. + */ + enum LimitsMode + { + LIMITS_CURRENT, + LIMITS_TOTAL, + }; + + /// It is a subset of limitations from Limits. + struct LocalLimits + { + LimitsMode mode = LIMITS_CURRENT; + + SizeLimits size_limits; + + Poco::Timespan max_execution_time = 0; + OverflowMode timeout_overflow_mode = OverflowMode::THROW; + + /// in rows per second + size_t min_execution_speed = 0; + /// Verify that the speed is not too low after the specified time has elapsed. + Poco::Timespan timeout_before_checking_execution_speed = 0; + }; + + /** Set limitations that checked on each block. */ + void setLimits(const LocalLimits & limits_) + { + limits = limits_; + } + + const LocalLimits & getLimits() const + { + return limits; + } + + /** Set the quota. If you set a quota on the amount of raw data, + * then you should also set mode = LIMITS_TOTAL to LocalLimits with setLimits. + */ + void setQuota(QuotaForIntervals & quota_) + { + quota = "a_; + } + + /// Enable calculation of minimums and maximums by the result columns. + void enableExtremes() { enabled_extremes = true; } + +protected: + BlockInputStreams children; + std::shared_mutex children_mutex; + + BlockStreamProfileInfo info; + std::atomic is_cancelled{false}; + std::atomic is_killed{false}; + ProgressCallback progress_callback; + QueryStatus * process_list_elem = nullptr; + /// According to total_stopwatch in microseconds + UInt64 last_profile_events_update_time = 0; + + /// Additional information that can be generated during the work process. + + /// Total values during aggregation. + Block totals; + /// Minimums and maximums. The first row of the block - minimums, the second - the maximums. + Block extremes; + + + void addChild(BlockInputStreamPtr & child) + { + std::unique_lock lock(children_mutex); + children.push_back(child); + } + +private: + TableStructureReadLocks table_locks; + + bool enabled_extremes = false; + + /// The limit on the number of rows/bytes has been exceeded, and you need to stop execution on the next `read` call, as if the thread has run out. + bool limit_exceeded_need_break = false; + + /// Limitations and quotas. + + LocalLimits limits; + + QuotaForIntervals * quota = nullptr; /// If nullptr - the quota is not used. + double prev_elapsed = 0; + + /// The approximate total number of rows to read. For progress bar. + size_t total_rows_approx = 0; + + /// The successors must implement this function. + virtual Block readImpl() = 0; + + /// Here you can do a preliminary initialization. + virtual void readPrefixImpl() {} + + /// Here you need to do a finalization, which can lead to an exception. + virtual void readSuffixImpl() {} + + void updateExtremes(Block & block); + + /** Check limits and quotas. + * But only those that can be checked within each separate stream. + */ + bool checkTimeLimit(); + void checkQuota(Block & block); + + size_t checkDepthImpl(size_t max_depth, size_t level) const; + + /// Get text with names of this source and the entire subtree. + String getTreeID() const; template void forEachChild(F && f) @@ -113,20 +314,6 @@ public: if (f(*child)) return; } - -protected: - BlockInputStreams children; - std::shared_mutex children_mutex; - -private: - TableStructureReadLocks table_locks; - - size_t checkDepthImpl(size_t max_depth, size_t level) const; - - /// Get text with names of this source and the entire subtree. - String getTreeID() const; }; - } - diff --git a/dbms/src/DataStreams/IProfilingBlockInputStream.cpp b/dbms/src/DataStreams/IProfilingBlockInputStream.cpp deleted file mode 100644 index f7b9a02ab9e..00000000000 --- a/dbms/src/DataStreams/IProfilingBlockInputStream.cpp +++ /dev/null @@ -1,427 +0,0 @@ -#include -#include -#include -#include - - -namespace ProfileEvents -{ - extern const Event ThrottlerSleepMicroseconds; -} - - -namespace DB -{ - -namespace ErrorCodes -{ - extern const int TOO_MANY_ROWS; - extern const int TOO_MANY_BYTES; - extern const int TOO_MANY_ROWS_OR_BYTES; - extern const int TIMEOUT_EXCEEDED; - extern const int TOO_SLOW; - extern const int LOGICAL_ERROR; - extern const int BLOCKS_HAVE_DIFFERENT_STRUCTURE; -} - - -IProfilingBlockInputStream::IProfilingBlockInputStream() -{ - info.parent = this; -} - -Block IProfilingBlockInputStream::read() -{ - if (total_rows_approx) - { - progressImpl(Progress(0, 0, total_rows_approx)); - total_rows_approx = 0; - } - - if (!info.started) - { - info.total_stopwatch.start(); - info.started = true; - } - - Block res; - - if (isCancelledOrThrowIfKilled()) - return res; - - if (!checkTimeLimit()) - limit_exceeded_need_break = true; - - if (!limit_exceeded_need_break) - res = readImpl(); - - if (res) - { - info.update(res); - - if (enabled_extremes) - updateExtremes(res); - - if (limits.mode == LIMITS_CURRENT && !limits.size_limits.check(info.rows, info.bytes, "result", ErrorCodes::TOO_MANY_ROWS_OR_BYTES)) - limit_exceeded_need_break = true; - - if (quota != nullptr) - checkQuota(res); - } - else - { - /** If the thread is over, then we will ask all children to abort the execution. - * This makes sense when running a query with LIMIT - * - there is a situation when all the necessary data has already been read, - * but children sources are still working, - * herewith they can work in separate threads or even remotely. - */ - cancel(false); - } - - progress(Progress(res.rows(), res.bytes())); - -#ifndef NDEBUG - if (res) - { - Block header = getHeader(); - if (header) - assertBlocksHaveEqualStructure(res, header, getName()); - } -#endif - - return res; -} - - -void IProfilingBlockInputStream::readPrefix() -{ - readPrefixImpl(); - - forEachChild([&] (IBlockInputStream & child) - { - child.readPrefix(); - return false; - }); -} - - -void IProfilingBlockInputStream::readSuffix() -{ - forEachChild([&] (IBlockInputStream & child) - { - child.readSuffix(); - return false; - }); - - readSuffixImpl(); -} - - -void IProfilingBlockInputStream::updateExtremes(Block & block) -{ - size_t num_columns = block.columns(); - - if (!extremes) - { - MutableColumns extremes_columns(num_columns); - - for (size_t i = 0; i < num_columns; ++i) - { - const ColumnPtr & src = block.safeGetByPosition(i).column; - - if (src->isColumnConst()) - { - /// Equal min and max. - extremes_columns[i] = src->cloneResized(2); - } - else - { - Field min_value; - Field max_value; - - src->getExtremes(min_value, max_value); - - extremes_columns[i] = src->cloneEmpty(); - - extremes_columns[i]->insert(min_value); - extremes_columns[i]->insert(max_value); - } - } - - extremes = block.cloneWithColumns(std::move(extremes_columns)); - } - else - { - for (size_t i = 0; i < num_columns; ++i) - { - ColumnPtr & old_extremes = extremes.safeGetByPosition(i).column; - - if (old_extremes->isColumnConst()) - continue; - - Field min_value = (*old_extremes)[0]; - Field max_value = (*old_extremes)[1]; - - Field cur_min_value; - Field cur_max_value; - - block.safeGetByPosition(i).column->getExtremes(cur_min_value, cur_max_value); - - if (cur_min_value < min_value) - min_value = cur_min_value; - if (cur_max_value > max_value) - max_value = cur_max_value; - - MutableColumnPtr new_extremes = old_extremes->cloneEmpty(); - - new_extremes->insert(min_value); - new_extremes->insert(max_value); - - old_extremes = std::move(new_extremes); - } - } -} - - -static bool handleOverflowMode(OverflowMode mode, const String & message, int code) -{ - switch (mode) - { - case OverflowMode::THROW: - throw Exception(message, code); - case OverflowMode::BREAK: - return false; - default: - throw Exception("Logical error: unknown overflow mode", ErrorCodes::LOGICAL_ERROR); - } -} - - -bool IProfilingBlockInputStream::checkTimeLimit() -{ - if (limits.max_execution_time != 0 - && info.total_stopwatch.elapsed() > static_cast(limits.max_execution_time.totalMicroseconds()) * 1000) - return handleOverflowMode(limits.timeout_overflow_mode, - "Timeout exceeded: elapsed " + toString(info.total_stopwatch.elapsedSeconds()) - + " seconds, maximum: " + toString(limits.max_execution_time.totalMicroseconds() / 1000000.0), - ErrorCodes::TIMEOUT_EXCEEDED); - - return true; -} - - -void IProfilingBlockInputStream::checkQuota(Block & block) -{ - switch (limits.mode) - { - case LIMITS_TOTAL: - /// Checked in `progress` method. - break; - - case LIMITS_CURRENT: - { - time_t current_time = time(nullptr); - double total_elapsed = info.total_stopwatch.elapsedSeconds(); - - quota->checkAndAddResultRowsBytes(current_time, block.rows(), block.bytes()); - quota->checkAndAddExecutionTime(current_time, Poco::Timespan((total_elapsed - prev_elapsed) * 1000000.0)); - - prev_elapsed = total_elapsed; - break; - } - } -} - - -void IProfilingBlockInputStream::progressImpl(const Progress & value) -{ - if (progress_callback) - progress_callback(value); - - if (process_list_elem) - { - if (!process_list_elem->updateProgressIn(value)) - cancel(/* kill */ true); - - /// The total amount of data processed or intended for processing in all leaf sources, possibly on remote servers. - - ProgressValues progress = process_list_elem->getProgressIn(); - size_t total_rows_estimate = std::max(progress.rows, progress.total_rows); - - /** Check the restrictions on the amount of data to read, the speed of the query, the quota on the amount of data to read. - * NOTE: Maybe it makes sense to have them checked directly in ProcessList? - */ - - if (limits.mode == LIMITS_TOTAL - && ((limits.size_limits.max_rows && total_rows_estimate > limits.size_limits.max_rows) - || (limits.size_limits.max_bytes && progress.bytes > limits.size_limits.max_bytes))) - { - switch (limits.size_limits.overflow_mode) - { - case OverflowMode::THROW: - { - if (limits.size_limits.max_rows && total_rows_estimate > limits.size_limits.max_rows) - throw Exception("Limit for rows to read exceeded: " + toString(total_rows_estimate) - + " rows read (or to read), maximum: " + toString(limits.size_limits.max_rows), - ErrorCodes::TOO_MANY_ROWS); - else - throw Exception("Limit for (uncompressed) bytes to read exceeded: " + toString(progress.bytes) - + " bytes read, maximum: " + toString(limits.size_limits.max_bytes), - ErrorCodes::TOO_MANY_BYTES); - } - - case OverflowMode::BREAK: - { - /// For `break`, we will stop only if so many rows were actually read, and not just supposed to be read. - if ((limits.size_limits.max_rows && progress.rows > limits.size_limits.max_rows) - || (limits.size_limits.max_bytes && progress.bytes > limits.size_limits.max_bytes)) - { - cancel(false); - } - - break; - } - - default: - throw Exception("Logical error: unknown overflow mode", ErrorCodes::LOGICAL_ERROR); - } - } - - size_t total_rows = progress.total_rows; - - constexpr UInt64 profile_events_update_period_microseconds = 10 * 1000; // 10 milliseconds - UInt64 total_elapsed_microseconds = info.total_stopwatch.elapsedMicroseconds(); - - if (last_profile_events_update_time + profile_events_update_period_microseconds < total_elapsed_microseconds) - { - CurrentThread::updatePerformanceCounters(); - last_profile_events_update_time = total_elapsed_microseconds; - } - - if ((limits.min_execution_speed || (total_rows && limits.timeout_before_checking_execution_speed != 0)) - && (static_cast(total_elapsed_microseconds) > limits.timeout_before_checking_execution_speed.totalMicroseconds())) - { - /// Do not count sleeps in throttlers - UInt64 throttler_sleep_microseconds = CurrentThread::getProfileEvents()[ProfileEvents::ThrottlerSleepMicroseconds]; - double elapsed_seconds = (throttler_sleep_microseconds > total_elapsed_microseconds) - ? 0.0 : (total_elapsed_microseconds - throttler_sleep_microseconds) / 1000000.0; - - if (elapsed_seconds > 0) - { - if (limits.min_execution_speed && progress.rows / elapsed_seconds < limits.min_execution_speed) - throw Exception("Query is executing too slow: " + toString(progress.rows / elapsed_seconds) - + " rows/sec., minimum: " + toString(limits.min_execution_speed), - ErrorCodes::TOO_SLOW); - - /// If the predicted execution time is longer than `max_execution_time`. - if (limits.max_execution_time != 0 && total_rows) - { - double estimated_execution_time_seconds = elapsed_seconds * (static_cast(total_rows) / progress.rows); - - if (estimated_execution_time_seconds > limits.max_execution_time.totalSeconds()) - throw Exception("Estimated query execution time (" + toString(estimated_execution_time_seconds) + " seconds)" - + " is too long. Maximum: " + toString(limits.max_execution_time.totalSeconds()) - + ". Estimated rows to process: " + toString(total_rows), - ErrorCodes::TOO_SLOW); - } - } - } - - if (quota != nullptr && limits.mode == LIMITS_TOTAL) - { - quota->checkAndAddReadRowsBytes(time(nullptr), value.rows, value.bytes); - } - } -} - - -void IProfilingBlockInputStream::cancel(bool kill) -{ - if (kill) - is_killed = true; - - bool old_val = false; - if (!is_cancelled.compare_exchange_strong(old_val, true, std::memory_order_seq_cst, std::memory_order_relaxed)) - return; - - forEachProfilingChild([&] (IProfilingBlockInputStream & child) - { - child.cancel(kill); - return false; - }); -} - - -bool IProfilingBlockInputStream::isCancelled() const -{ - return is_cancelled; -} - -bool IProfilingBlockInputStream::isCancelledOrThrowIfKilled() const -{ - if (!is_cancelled) - return false; - if (is_killed) - throw Exception("Query was cancelled", ErrorCodes::QUERY_WAS_CANCELLED); - return true; -} - - -void IProfilingBlockInputStream::setProgressCallback(const ProgressCallback & callback) -{ - progress_callback = callback; - - forEachProfilingChild([&] (IProfilingBlockInputStream & child) - { - child.setProgressCallback(callback); - return false; - }); -} - - -void IProfilingBlockInputStream::setProcessListElement(QueryStatus * elem) -{ - process_list_elem = elem; - - forEachProfilingChild([&] (IProfilingBlockInputStream & child) - { - child.setProcessListElement(elem); - return false; - }); -} - - -Block IProfilingBlockInputStream::getTotals() -{ - if (totals) - return totals; - - Block res; - forEachProfilingChild([&] (IProfilingBlockInputStream & child) - { - res = child.getTotals(); - if (res) - return true; - return false; - }); - return res; -} - -Block IProfilingBlockInputStream::getExtremes() -{ - if (extremes) - return extremes; - - Block res; - forEachProfilingChild([&] (IProfilingBlockInputStream & child) - { - res = child.getExtremes(); - if (res) - return true; - return false; - }); - return res; -} - -} diff --git a/dbms/src/DataStreams/IProfilingBlockInputStream.h b/dbms/src/DataStreams/IProfilingBlockInputStream.h deleted file mode 100644 index ab0db8dd99d..00000000000 --- a/dbms/src/DataStreams/IProfilingBlockInputStream.h +++ /dev/null @@ -1,247 +0,0 @@ -#pragma once - -#include - -#include -#include -#include - -#include - -#include - - -namespace DB -{ - -namespace ErrorCodes -{ - extern const int QUERY_WAS_CANCELLED; -} - -class QuotaForIntervals; -class QueryStatus; -class ProcessListElement; -class IProfilingBlockInputStream; - -using ProfilingBlockInputStreamPtr = std::shared_ptr; - - -/** Watches out at how the source of the blocks works. - * Lets you get information for profiling: - * rows per second, blocks per second, megabytes per second, etc. - * Allows you to stop reading data (in nested sources). - */ -class IProfilingBlockInputStream : public IBlockInputStream -{ - friend struct BlockStreamProfileInfo; - -public: - IProfilingBlockInputStream(); - - Block read() override final; - - /** The default implementation calls readPrefixImpl() on itself, and then readPrefix() recursively for all children. - * There are cases when you do not want `readPrefix` of children to be called synchronously, in this function, - * but you want them to be called, for example, in separate threads (for parallel initialization of children). - * Then overload `readPrefix` function. - */ - void readPrefix() override; - - /** The default implementation calls recursively readSuffix() on all children, and then readSuffixImpl() on itself. - * If this stream calls read() in children in a separate thread, this behavior is usually incorrect: - * readSuffix() of the child can not be called at the moment when the same child's read() is executed in another thread. - * In this case, you need to override this method so that readSuffix() in children is called, for example, after connecting streams. - */ - void readSuffix() override; - - /// Get information about execution speed. - const BlockStreamProfileInfo & getProfileInfo() const { return info; } - - /** Get "total" values. - * The default implementation takes them from itself or from the first child source in which they are. - * The overridden method can perform some calculations. For example, apply an expression to the `totals` of the child source. - * There can be no total values - then an empty block is returned. - * - * Call this method only after all the data has been retrieved with `read`, - * otherwise there will be problems if any data at the same time is computed in another thread. - */ - virtual Block getTotals(); - - /// The same for minimums and maximums. - Block getExtremes(); - - - /** Set the execution progress bar callback. - * The callback is passed to all child sources. - * By default, it is called for leaf sources, after each block. - * (But this can be overridden in the progress() method) - * The function takes the number of rows in the last block, the number of bytes in the last block. - * Note that the callback can be called from different threads. - */ - void setProgressCallback(const ProgressCallback & callback); - - - /** In this method: - * - the progress callback is called; - * - the status of the query execution in ProcessList is updated; - * - checks restrictions and quotas that should be checked not within the same source, - * but over the total amount of resources spent in all sources at once (information in the ProcessList). - */ - virtual void progress(const Progress & value) - { - /// The data for progress is taken from leaf sources. - if (children.empty()) - progressImpl(value); - } - - void progressImpl(const Progress & value); - - - /** Set the pointer to the process list item. - * It is passed to all child sources. - * General information about the resources spent on the request will be written into it. - * Based on this information, the quota and some restrictions will be checked. - * This information will also be available in the SHOW PROCESSLIST request. - */ - void setProcessListElement(QueryStatus * elem); - - /** Set the approximate total number of rows to read. - */ - void addTotalRowsApprox(size_t value) { total_rows_approx += value; } - - - /** Ask to abort the receipt of data as soon as possible. - * By default - just sets the flag is_cancelled and asks that all children be interrupted. - * This function can be called several times, including simultaneously from different threads. - * Have two modes: - * with kill = false only is_cancelled is set - streams will stop silently with returning some processed data. - * with kill = true also is_killed set - queries will stop with exception. - */ - virtual void cancel(bool kill); - - bool isCancelled() const; - bool isCancelledOrThrowIfKilled() const; - - /** What limitations and quotas should be checked. - * LIMITS_CURRENT - checks amount of data read by current stream only (BlockStreamProfileInfo is used for check). - * Currently it is used in root streams to check max_result_{rows,bytes} limits. - * LIMITS_TOTAL - checks total amount of read data from leaf streams (i.e. data read from disk and remote servers). - * It is checks max_{rows,bytes}_to_read in progress handler and use info from ProcessListElement::progress_in for this. - * Currently this check is performed only in leaf streams. - */ - enum LimitsMode - { - LIMITS_CURRENT, - LIMITS_TOTAL, - }; - - /// It is a subset of limitations from Limits. - struct LocalLimits - { - LimitsMode mode = LIMITS_CURRENT; - - SizeLimits size_limits; - - Poco::Timespan max_execution_time = 0; - OverflowMode timeout_overflow_mode = OverflowMode::THROW; - - /// in rows per second - size_t min_execution_speed = 0; - /// Verify that the speed is not too low after the specified time has elapsed. - Poco::Timespan timeout_before_checking_execution_speed = 0; - }; - - /** Set limitations that checked on each block. */ - void setLimits(const LocalLimits & limits_) - { - limits = limits_; - } - - const LocalLimits & getLimits() const - { - return limits; - } - - /** Set the quota. If you set a quota on the amount of raw data, - * then you should also set mode = LIMITS_TOTAL to LocalLimits with setLimits. - */ - void setQuota(QuotaForIntervals & quota_) - { - quota = "a_; - } - - /// Enable calculation of minimums and maximums by the result columns. - void enableExtremes() { enabled_extremes = true; } - -protected: - BlockStreamProfileInfo info; - std::atomic is_cancelled{false}; - std::atomic is_killed{false}; - ProgressCallback progress_callback; - QueryStatus * process_list_elem = nullptr; - /// According to total_stopwatch in microseconds - UInt64 last_profile_events_update_time = 0; - - /// Additional information that can be generated during the work process. - - /// Total values during aggregation. - Block totals; - /// Minimums and maximums. The first row of the block - minimums, the second - the maximums. - Block extremes; - - - void addChild(BlockInputStreamPtr & child) - { - std::unique_lock lock(children_mutex); - children.push_back(child); - } - -private: - bool enabled_extremes = false; - - /// The limit on the number of rows/bytes has been exceeded, and you need to stop execution on the next `read` call, as if the thread has run out. - bool limit_exceeded_need_break = false; - - /// Limitations and quotas. - - LocalLimits limits; - - QuotaForIntervals * quota = nullptr; /// If nullptr - the quota is not used. - double prev_elapsed = 0; - - /// The approximate total number of rows to read. For progress bar. - size_t total_rows_approx = 0; - - /// The successors must implement this function. - virtual Block readImpl() = 0; - - /// Here you can do a preliminary initialization. - virtual void readPrefixImpl() {} - - /// Here you need to do a finalization, which can lead to an exception. - virtual void readSuffixImpl() {} - - void updateExtremes(Block & block); - - /** Check limits and quotas. - * But only those that can be checked within each separate stream. - */ - bool checkTimeLimit(); - void checkQuota(Block & block); - - - template - void forEachProfilingChild(F && f) - { - /// NOTE: Acquire a read lock, therefore f() should be thread safe - std::shared_lock lock(children_mutex); - - for (auto & child : children) - if (IProfilingBlockInputStream * p_child = dynamic_cast(child.get())) - if (f(*p_child)) - return; - } -}; - -} diff --git a/dbms/src/DataStreams/InputStreamFromASTInsertQuery.h b/dbms/src/DataStreams/InputStreamFromASTInsertQuery.h index dbcf3799e3e..876a10e563d 100644 --- a/dbms/src/DataStreams/InputStreamFromASTInsertQuery.h +++ b/dbms/src/DataStreams/InputStreamFromASTInsertQuery.h @@ -1,7 +1,7 @@ #pragma once #include -#include +#include #include #include @@ -16,7 +16,7 @@ class Context; * Head of inserting data could be stored in INSERT ast directly * Remaining (tail) data could be stored in input_buffer_tail_part */ -class InputStreamFromASTInsertQuery : public IProfilingBlockInputStream +class InputStreamFromASTInsertQuery : public IBlockInputStream { public: InputStreamFromASTInsertQuery(const ASTPtr & ast, ReadBuffer & input_buffer_tail_part, const BlockIO & streams, Context & context); diff --git a/dbms/src/DataStreams/LazyBlockInputStream.h b/dbms/src/DataStreams/LazyBlockInputStream.h index 3bb6e67c3a3..321778d30b3 100644 --- a/dbms/src/DataStreams/LazyBlockInputStream.h +++ b/dbms/src/DataStreams/LazyBlockInputStream.h @@ -1,6 +1,6 @@ #pragma once -#include +#include namespace DB @@ -10,7 +10,7 @@ namespace DB * This is needed, for example, to read from a table that will be populated * after creation of LazyBlockInputStream object, but before the first `read` call. */ -class LazyBlockInputStream : public IProfilingBlockInputStream +class LazyBlockInputStream : public IBlockInputStream { public: using Generator = std::function; @@ -42,7 +42,7 @@ protected: if (!input) return Block(); - auto * p_input = dynamic_cast(input.get()); + auto * p_input = dynamic_cast(input.get()); if (p_input) { diff --git a/dbms/src/DataStreams/LimitBlockInputStream.cpp b/dbms/src/DataStreams/LimitBlockInputStream.cpp index 7939d1f73ca..298ea28e883 100644 --- a/dbms/src/DataStreams/LimitBlockInputStream.cpp +++ b/dbms/src/DataStreams/LimitBlockInputStream.cpp @@ -58,8 +58,10 @@ Block LimitBlockInputStream::readImpl() for (size_t i = 0; i < res.columns(); ++i) res.safeGetByPosition(i).column = res.safeGetByPosition(i).column->cut(start, length); + // TODO: we should provide feedback to child-block, so it will know how many rows are actually consumed. + // It's crucial for streaming engines like Kafka. + return res; } } - diff --git a/dbms/src/DataStreams/LimitBlockInputStream.h b/dbms/src/DataStreams/LimitBlockInputStream.h index 9739665243f..a9dea35dd21 100644 --- a/dbms/src/DataStreams/LimitBlockInputStream.h +++ b/dbms/src/DataStreams/LimitBlockInputStream.h @@ -1,6 +1,6 @@ #pragma once -#include +#include namespace DB @@ -9,7 +9,7 @@ namespace DB /** Implements the LIMIT relational operation. */ -class LimitBlockInputStream : public IProfilingBlockInputStream +class LimitBlockInputStream : public IBlockInputStream { public: /** If always_read_till_end = false (by default), then after reading enough data, diff --git a/dbms/src/DataStreams/LimitByBlockInputStream.h b/dbms/src/DataStreams/LimitByBlockInputStream.h index 88cf2ff1670..4244908ba1c 100644 --- a/dbms/src/DataStreams/LimitByBlockInputStream.h +++ b/dbms/src/DataStreams/LimitByBlockInputStream.h @@ -1,6 +1,6 @@ #pragma once -#include +#include #include #include @@ -15,7 +15,7 @@ namespace DB * the query SELECT Num FROM T LIMIT 2 BY Num * will give you the following result: (Num: 1 1 3 3 4 4 5 7 7). */ -class LimitByBlockInputStream : public IProfilingBlockInputStream +class LimitByBlockInputStream : public IBlockInputStream { public: LimitByBlockInputStream(const BlockInputStreamPtr & input, size_t group_size_, const Names & columns); diff --git a/dbms/src/DataStreams/MaterializingBlockInputStream.h b/dbms/src/DataStreams/MaterializingBlockInputStream.h index dfbad3b14b2..90fd91bd550 100644 --- a/dbms/src/DataStreams/MaterializingBlockInputStream.h +++ b/dbms/src/DataStreams/MaterializingBlockInputStream.h @@ -1,13 +1,13 @@ #pragma once -#include +#include namespace DB { /** Converts columns-constants to full columns ("materializes" them). */ -class MaterializingBlockInputStream : public IProfilingBlockInputStream +class MaterializingBlockInputStream : public IBlockInputStream { public: MaterializingBlockInputStream(const BlockInputStreamPtr & input); diff --git a/dbms/src/DataStreams/MergeSortingBlockInputStream.h b/dbms/src/DataStreams/MergeSortingBlockInputStream.h index 7ff6ea6cfc2..c9335208cb4 100644 --- a/dbms/src/DataStreams/MergeSortingBlockInputStream.h +++ b/dbms/src/DataStreams/MergeSortingBlockInputStream.h @@ -8,7 +8,7 @@ #include #include -#include +#include #include #include @@ -25,7 +25,7 @@ namespace DB /** Part of implementation. Merging array of ready (already read from somewhere) blocks. * Returns result of merge as stream of blocks, not more than 'max_merged_block_size' rows in each. */ -class MergeSortingBlocksBlockInputStream : public IProfilingBlockInputStream +class MergeSortingBlocksBlockInputStream : public IBlockInputStream { public: /// limit - if not 0, allowed to return just first 'limit' rows in sorted order. @@ -66,7 +66,7 @@ private: }; -class MergeSortingBlockInputStream : public IProfilingBlockInputStream +class MergeSortingBlockInputStream : public IBlockInputStream { public: /// limit - if not 0, allowed to return just first 'limit' rows in sorted order. diff --git a/dbms/src/DataStreams/MergingAggregatedBlockInputStream.h b/dbms/src/DataStreams/MergingAggregatedBlockInputStream.h index b299aff0d1f..e717d2b335d 100644 --- a/dbms/src/DataStreams/MergingAggregatedBlockInputStream.h +++ b/dbms/src/DataStreams/MergingAggregatedBlockInputStream.h @@ -1,7 +1,7 @@ #pragma once #include -#include +#include namespace DB @@ -11,7 +11,7 @@ namespace DB /** A pre-aggregate stream of blocks in which each block is already aggregated. * Aggregate functions in blocks should not be finalized so that their states can be merged. */ -class MergingAggregatedBlockInputStream : public IProfilingBlockInputStream +class MergingAggregatedBlockInputStream : public IBlockInputStream { public: MergingAggregatedBlockInputStream(const BlockInputStreamPtr & input, const Aggregator::Params & params, bool final_, size_t max_threads_) diff --git a/dbms/src/DataStreams/MergingAggregatedMemoryEfficientBlockInputStream.cpp b/dbms/src/DataStreams/MergingAggregatedMemoryEfficientBlockInputStream.cpp index 7773da79008..f226da9e442 100644 --- a/dbms/src/DataStreams/MergingAggregatedMemoryEfficientBlockInputStream.cpp +++ b/dbms/src/DataStreams/MergingAggregatedMemoryEfficientBlockInputStream.cpp @@ -136,20 +136,17 @@ void MergingAggregatedMemoryEfficientBlockInputStream::cancel(bool kill) for (auto & input : inputs) { - if (IProfilingBlockInputStream * child = dynamic_cast(input.stream.get())) + try { - try - { - child->cancel(kill); - } - catch (...) - { - /** If failed to ask to stop processing one or more sources. - * (example: connection reset during distributed query execution) - * - then don't care. - */ - LOG_ERROR(log, "Exception while cancelling " << child->getName()); - } + input.stream->cancel(kill); + } + catch (...) + { + /** If failed to ask to stop processing one or more sources. + * (example: connection reset during distributed query execution) + * - then don't care. + */ + LOG_ERROR(log, "Exception while cancelling " << input.stream->getName()); } } } diff --git a/dbms/src/DataStreams/MergingAggregatedMemoryEfficientBlockInputStream.h b/dbms/src/DataStreams/MergingAggregatedMemoryEfficientBlockInputStream.h index bdabd8cc1f8..62f8ed4d74b 100644 --- a/dbms/src/DataStreams/MergingAggregatedMemoryEfficientBlockInputStream.h +++ b/dbms/src/DataStreams/MergingAggregatedMemoryEfficientBlockInputStream.h @@ -1,7 +1,7 @@ #pragma once #include -#include +#include #include #include #include @@ -57,7 +57,7 @@ namespace DB * data from sources can also be read in several threads (reading_threads) * for optimal performance in the presence of a fast network or disks (from where these blocks are read). */ -class MergingAggregatedMemoryEfficientBlockInputStream final : public IProfilingBlockInputStream +class MergingAggregatedMemoryEfficientBlockInputStream final : public IBlockInputStream { public: MergingAggregatedMemoryEfficientBlockInputStream( diff --git a/dbms/src/DataStreams/MergingSortedBlockInputStream.h b/dbms/src/DataStreams/MergingSortedBlockInputStream.h index 2a3fb7f5133..6521ef383ed 100644 --- a/dbms/src/DataStreams/MergingSortedBlockInputStream.h +++ b/dbms/src/DataStreams/MergingSortedBlockInputStream.h @@ -12,7 +12,7 @@ #include -#include +#include #include @@ -60,7 +60,7 @@ inline void intrusive_ptr_release(detail::SharedBlock * ptr) /** Merges several sorted streams into one sorted stream. */ -class MergingSortedBlockInputStream : public IProfilingBlockInputStream +class MergingSortedBlockInputStream : public IBlockInputStream { public: /** limit - if isn't 0, then we can produce only first limit rows in sorted order. diff --git a/dbms/src/DataStreams/NativeBlockInputStream.h b/dbms/src/DataStreams/NativeBlockInputStream.h index c9b565cd9d7..72f6e0d211b 100644 --- a/dbms/src/DataStreams/NativeBlockInputStream.h +++ b/dbms/src/DataStreams/NativeBlockInputStream.h @@ -1,6 +1,6 @@ #pragma once -#include +#include #include #include @@ -57,7 +57,7 @@ struct IndexForNativeFormat * Can also be used to store data on disk. * In this case, can use the index. */ -class NativeBlockInputStream : public IProfilingBlockInputStream +class NativeBlockInputStream : public IBlockInputStream { public: /// If a non-zero server_revision is specified, additional block information may be expected and read. diff --git a/dbms/src/DataStreams/NullAndDoCopyBlockInputStream.h b/dbms/src/DataStreams/NullAndDoCopyBlockInputStream.h index d03996be53f..43584797898 100644 --- a/dbms/src/DataStreams/NullAndDoCopyBlockInputStream.h +++ b/dbms/src/DataStreams/NullAndDoCopyBlockInputStream.h @@ -1,6 +1,6 @@ #pragma once -#include +#include #include @@ -17,7 +17,7 @@ using BlockOutputStreamPtr = std::shared_ptr; * The query could be executed without wrapping it in an empty BlockInputStream, * but the progress of query execution and the ability to cancel the query would not work. */ -class NullAndDoCopyBlockInputStream : public IProfilingBlockInputStream +class NullAndDoCopyBlockInputStream : public IBlockInputStream { public: NullAndDoCopyBlockInputStream(const BlockInputStreamPtr & input_, BlockOutputStreamPtr output_) diff --git a/dbms/src/DataStreams/NullBlockInputStream.h b/dbms/src/DataStreams/NullBlockInputStream.h index 9272dddd15a..7598f8a5811 100644 --- a/dbms/src/DataStreams/NullBlockInputStream.h +++ b/dbms/src/DataStreams/NullBlockInputStream.h @@ -6,19 +6,19 @@ namespace DB { -/** Empty stream of blocks of specified structure. - */ +/// Empty stream of blocks of specified structure. class NullBlockInputStream : public IBlockInputStream { public: - NullBlockInputStream(const Block & header) : header(header) {} + NullBlockInputStream(const Block & header_) : header(header_) {} - Block read() override { return {}; } Block getHeader() const override { return header; } String getName() const override { return "Null"; } private: Block header; + + Block readImpl() override { return {}; } }; -} +} /// namespace DB diff --git a/dbms/src/DataStreams/OneBlockInputStream.h b/dbms/src/DataStreams/OneBlockInputStream.h index 22480089f86..3f1da34fcd8 100644 --- a/dbms/src/DataStreams/OneBlockInputStream.h +++ b/dbms/src/DataStreams/OneBlockInputStream.h @@ -1,6 +1,6 @@ #pragma once -#include +#include namespace DB @@ -9,7 +9,7 @@ namespace DB /** A stream of blocks from which you can read one block. * Also see BlocksListBlockInputStream. */ -class OneBlockInputStream : public IProfilingBlockInputStream +class OneBlockInputStream : public IBlockInputStream { public: OneBlockInputStream(const Block & block_) : block(block_) {} diff --git a/dbms/src/DataStreams/OwningBlockInputStream.h b/dbms/src/DataStreams/OwningBlockInputStream.h index 5bed4a9f9c3..b7ea121814c 100644 --- a/dbms/src/DataStreams/OwningBlockInputStream.h +++ b/dbms/src/DataStreams/OwningBlockInputStream.h @@ -2,7 +2,7 @@ #include -#include +#include namespace DB { @@ -11,7 +11,7 @@ namespace DB * simplifies usage of ReadBufferFromFile (no need to manage buffer lifetime) etc. */ template -class OwningBlockInputStream : public IProfilingBlockInputStream +class OwningBlockInputStream : public IBlockInputStream { public: OwningBlockInputStream(const BlockInputStreamPtr & stream, std::unique_ptr own) diff --git a/dbms/src/DataStreams/ParallelAggregatingBlockInputStream.h b/dbms/src/DataStreams/ParallelAggregatingBlockInputStream.h index 5719dc68e84..52852526935 100644 --- a/dbms/src/DataStreams/ParallelAggregatingBlockInputStream.h +++ b/dbms/src/DataStreams/ParallelAggregatingBlockInputStream.h @@ -3,7 +3,7 @@ #include #include #include -#include +#include #include @@ -16,7 +16,7 @@ namespace DB * If final == false, aggregate functions are not finalized, that is, they are not replaced by their value, but contain an intermediate state of calculations. * This is necessary so that aggregation can continue (for example, by combining streams of partially aggregated data). */ -class ParallelAggregatingBlockInputStream : public IProfilingBlockInputStream +class ParallelAggregatingBlockInputStream : public IBlockInputStream { public: /** Columns from key_names and arguments of aggregate functions must already be computed. diff --git a/dbms/src/DataStreams/ParallelInputsProcessor.h b/dbms/src/DataStreams/ParallelInputsProcessor.h index ba086b98939..92faf6e46f3 100644 --- a/dbms/src/DataStreams/ParallelInputsProcessor.h +++ b/dbms/src/DataStreams/ParallelInputsProcessor.h @@ -8,7 +8,7 @@ #include -#include +#include #include #include #include @@ -125,20 +125,17 @@ public: for (auto & input : inputs) { - if (IProfilingBlockInputStream * child = dynamic_cast(&*input)) + try { - try - { - child->cancel(kill); - } - catch (...) - { - /** If you can not ask one or more sources to stop. - * (for example, the connection is broken for distributed query processing) - * - then do not care. - */ - LOG_ERROR(log, "Exception while cancelling " << child->getName()); - } + input->cancel(kill); + } + catch (...) + { + /** If you can not ask one or more sources to stop. + * (for example, the connection is broken for distributed query processing) + * - then do not care. + */ + LOG_ERROR(log, "Exception while cancelling " << input->getName()); } } } diff --git a/dbms/src/DataStreams/PartialSortingBlockInputStream.h b/dbms/src/DataStreams/PartialSortingBlockInputStream.h index b35b1c5d6bc..b0f57f7564d 100644 --- a/dbms/src/DataStreams/PartialSortingBlockInputStream.h +++ b/dbms/src/DataStreams/PartialSortingBlockInputStream.h @@ -2,7 +2,7 @@ #include -#include +#include namespace DB @@ -11,7 +11,7 @@ namespace DB /** Sorts each block individually by the values of the specified columns. * At the moment, not very optimal algorithm is used. */ -class PartialSortingBlockInputStream : public IProfilingBlockInputStream +class PartialSortingBlockInputStream : public IBlockInputStream { public: /// limit - if not 0, then you can sort each block not completely, but only `limit` first rows by order. diff --git a/dbms/src/DataStreams/RemoteBlockInputStream.cpp b/dbms/src/DataStreams/RemoteBlockInputStream.cpp index 21c7645ec45..e8a29880aad 100644 --- a/dbms/src/DataStreams/RemoteBlockInputStream.cpp +++ b/dbms/src/DataStreams/RemoteBlockInputStream.cpp @@ -109,8 +109,7 @@ void RemoteBlockInputStream::cancel(bool kill) /// Stop sending external data. for (auto & vec : external_tables_data) for (auto & elem : vec) - if (IProfilingBlockInputStream * stream = dynamic_cast(elem.first.get())) - stream->cancel(kill); + elem.first->cancel(kill); } if (!isQueryPending() || hasThrownException()) diff --git a/dbms/src/DataStreams/RemoteBlockInputStream.h b/dbms/src/DataStreams/RemoteBlockInputStream.h index 9eb59ea60a9..7f8626398dc 100644 --- a/dbms/src/DataStreams/RemoteBlockInputStream.h +++ b/dbms/src/DataStreams/RemoteBlockInputStream.h @@ -4,7 +4,7 @@ #include -#include +#include #include #include #include @@ -17,7 +17,7 @@ namespace DB /** This class allows one to launch queries on remote replicas of one shard and get results */ -class RemoteBlockInputStream : public IProfilingBlockInputStream +class RemoteBlockInputStream : public IBlockInputStream { public: /// Takes already set connection. diff --git a/dbms/src/DataStreams/RollupBlockInputStream.h b/dbms/src/DataStreams/RollupBlockInputStream.h index 0e4b9b80900..1c1e29e7a13 100644 --- a/dbms/src/DataStreams/RollupBlockInputStream.h +++ b/dbms/src/DataStreams/RollupBlockInputStream.h @@ -1,6 +1,6 @@ #pragma once -#include +#include #include #include @@ -14,7 +14,7 @@ class ExpressionActions; /** Takes blocks after grouping, with non-finalized aggregate functions. * Calculates subtotals and grand totals values for a set of columns. */ -class RollupBlockInputStream : public IProfilingBlockInputStream +class RollupBlockInputStream : public IBlockInputStream { private: using ExpressionActionsPtr = std::shared_ptr; diff --git a/dbms/src/DataStreams/SquashingBlockInputStream.h b/dbms/src/DataStreams/SquashingBlockInputStream.h index feac5e2cdcb..062696ca40b 100644 --- a/dbms/src/DataStreams/SquashingBlockInputStream.h +++ b/dbms/src/DataStreams/SquashingBlockInputStream.h @@ -1,6 +1,6 @@ #pragma once -#include +#include #include @@ -9,7 +9,7 @@ namespace DB /** Merging consecutive blocks of stream to specified minimum size. */ -class SquashingBlockInputStream : public IProfilingBlockInputStream +class SquashingBlockInputStream : public IBlockInputStream { public: SquashingBlockInputStream(const BlockInputStreamPtr & src, size_t min_block_size_rows, size_t min_block_size_bytes); diff --git a/dbms/src/DataStreams/TotalsHavingBlockInputStream.h b/dbms/src/DataStreams/TotalsHavingBlockInputStream.h index 31e72c6f4d5..73ecdf62134 100644 --- a/dbms/src/DataStreams/TotalsHavingBlockInputStream.h +++ b/dbms/src/DataStreams/TotalsHavingBlockInputStream.h @@ -1,6 +1,6 @@ #pragma once -#include +#include namespace DB @@ -16,7 +16,7 @@ class ExpressionActions; * Calculates total values according to totals_mode. * If necessary, evaluates the expression from HAVING and filters rows. Returns the finalized and filtered blocks. */ -class TotalsHavingBlockInputStream : public IProfilingBlockInputStream +class TotalsHavingBlockInputStream : public IBlockInputStream { private: using ExpressionActionsPtr = std::shared_ptr; @@ -52,7 +52,7 @@ private: */ Block overflow_aggregates; - /// Here, total values are accumulated. After the work is finished, they will be placed in IProfilingBlockInputStream::totals. + /// Here, total values are accumulated. After the work is finished, they will be placed in IBlockInputStream::totals. MutableColumns current_totals; /// Arena for aggregate function states in totals. ArenaPtr arena; diff --git a/dbms/src/DataStreams/UnionBlockInputStream.h b/dbms/src/DataStreams/UnionBlockInputStream.h index da24c9b131a..4eea1817cfb 100644 --- a/dbms/src/DataStreams/UnionBlockInputStream.h +++ b/dbms/src/DataStreams/UnionBlockInputStream.h @@ -3,7 +3,7 @@ #include #include -#include +#include #include @@ -26,7 +26,7 @@ namespace ErrorCodes * - the completed blocks are added to a limited queue of finished blocks; * - the main thread takes out completed blocks from the queue of finished blocks; */ -class UnionBlockInputStream final : public IProfilingBlockInputStream +class UnionBlockInputStream final : public IBlockInputStream { private: /// A block or an exception. diff --git a/dbms/src/DataStreams/copyData.cpp b/dbms/src/DataStreams/copyData.cpp index 106f2f5033e..9d17596fc8d 100644 --- a/dbms/src/DataStreams/copyData.cpp +++ b/dbms/src/DataStreams/copyData.cpp @@ -1,4 +1,4 @@ -#include +#include #include #include @@ -35,14 +35,11 @@ void copyDataImpl(IBlockInputStream & from, IBlockOutputStream & to, TCancelCall return; /// For outputting additional information in some formats. - if (IProfilingBlockInputStream * input = dynamic_cast(&from)) - { - if (input->getProfileInfo().hasAppliedLimit()) - to.setRowsBeforeLimit(input->getProfileInfo().getRowsBeforeLimit()); + if (from.getProfileInfo().hasAppliedLimit()) + to.setRowsBeforeLimit(from.getProfileInfo().getRowsBeforeLimit()); - to.setTotals(input->getTotals()); - to.setExtremes(input->getExtremes()); - } + to.setTotals(from.getTotals()); + to.setExtremes(from.getExtremes()); if (is_cancelled()) return; diff --git a/dbms/src/Dictionaries/ClickHouseDictionarySource.cpp b/dbms/src/Dictionaries/ClickHouseDictionarySource.cpp index faa93055073..390f63ff7bf 100644 --- a/dbms/src/Dictionaries/ClickHouseDictionarySource.cpp +++ b/dbms/src/Dictionaries/ClickHouseDictionarySource.cpp @@ -183,7 +183,7 @@ std::string ClickHouseDictionarySource::doInvalidateQuery(const std::string & re if (is_local) { auto input_block = executeQuery(request, context, true).in; - return readInvalidateQuery(dynamic_cast((*input_block))); + return readInvalidateQuery(*input_block); } else { diff --git a/dbms/src/Dictionaries/DictionaryBlockInputStream.h b/dbms/src/Dictionaries/DictionaryBlockInputStream.h index afdc26cdba3..c05ebc3b935 100644 --- a/dbms/src/Dictionaries/DictionaryBlockInputStream.h +++ b/dbms/src/Dictionaries/DictionaryBlockInputStream.h @@ -6,7 +6,7 @@ #include #include #include -#include +#include #include #include #include diff --git a/dbms/src/Dictionaries/DictionaryBlockInputStreamBase.h b/dbms/src/Dictionaries/DictionaryBlockInputStreamBase.h index a94311726fe..571a0da3b50 100644 --- a/dbms/src/Dictionaries/DictionaryBlockInputStreamBase.h +++ b/dbms/src/Dictionaries/DictionaryBlockInputStreamBase.h @@ -1,10 +1,10 @@ #pragma once -#include +#include namespace DB { -class DictionaryBlockInputStreamBase : public IProfilingBlockInputStream +class DictionaryBlockInputStreamBase : public IBlockInputStream { protected: DictionaryBlockInputStreamBase(size_t rows_count, size_t max_block_size); diff --git a/dbms/src/Dictionaries/ExecutableDictionarySource.cpp b/dbms/src/Dictionaries/ExecutableDictionarySource.cpp index 376153bd0e9..900668d4f78 100644 --- a/dbms/src/Dictionaries/ExecutableDictionarySource.cpp +++ b/dbms/src/Dictionaries/ExecutableDictionarySource.cpp @@ -114,7 +114,7 @@ namespace /** A stream, that also runs and waits for background thread * (that will feed data into pipe to be read from the other side of the pipe). */ - class BlockInputStreamWithBackgroundThread final : public IProfilingBlockInputStream + class BlockInputStreamWithBackgroundThread final : public IBlockInputStream { public: BlockInputStreamWithBackgroundThread( @@ -149,7 +149,7 @@ namespace void readSuffix() override { - IProfilingBlockInputStream::readSuffix(); + IBlockInputStream::readSuffix(); if (!wait_called) { wait_called = true; diff --git a/dbms/src/Dictionaries/MongoDBBlockInputStream.h b/dbms/src/Dictionaries/MongoDBBlockInputStream.h index 94923a96b57..f042b7a2dff 100644 --- a/dbms/src/Dictionaries/MongoDBBlockInputStream.h +++ b/dbms/src/Dictionaries/MongoDBBlockInputStream.h @@ -1,7 +1,7 @@ #pragma once #include -#include +#include #include "ExternalResultDescription.h" @@ -18,7 +18,7 @@ namespace MongoDB namespace DB { /// Converts MongoDB Cursor to a stream of Blocks -class MongoDBBlockInputStream final : public IProfilingBlockInputStream +class MongoDBBlockInputStream final : public IBlockInputStream { public: MongoDBBlockInputStream( diff --git a/dbms/src/Dictionaries/MySQLBlockInputStream.h b/dbms/src/Dictionaries/MySQLBlockInputStream.h index c262f64259c..01aff1732a2 100644 --- a/dbms/src/Dictionaries/MySQLBlockInputStream.h +++ b/dbms/src/Dictionaries/MySQLBlockInputStream.h @@ -2,7 +2,7 @@ #include #include -#include +#include #include #include #include "ExternalResultDescription.h" @@ -11,7 +11,7 @@ namespace DB { /// Allows processing results of a MySQL query as a sequence of Blocks, simplifies chaining -class MySQLBlockInputStream final : public IProfilingBlockInputStream +class MySQLBlockInputStream final : public IBlockInputStream { public: MySQLBlockInputStream( diff --git a/dbms/src/Dictionaries/ODBCBlockInputStream.h b/dbms/src/Dictionaries/ODBCBlockInputStream.h index 42230fc46c4..0cf19616b51 100644 --- a/dbms/src/Dictionaries/ODBCBlockInputStream.h +++ b/dbms/src/Dictionaries/ODBCBlockInputStream.h @@ -2,7 +2,7 @@ #include #include -#include +#include #include #include #include @@ -12,7 +12,7 @@ namespace DB { /// Allows processing results of a query to ODBC source as a sequence of Blocks, simplifies chaining -class ODBCBlockInputStream final : public IProfilingBlockInputStream +class ODBCBlockInputStream final : public IBlockInputStream { public: ODBCBlockInputStream( diff --git a/dbms/src/Dictionaries/RangeDictionaryBlockInputStream.h b/dbms/src/Dictionaries/RangeDictionaryBlockInputStream.h index 3cf6b8157fc..f7911365506 100644 --- a/dbms/src/Dictionaries/RangeDictionaryBlockInputStream.h +++ b/dbms/src/Dictionaries/RangeDictionaryBlockInputStream.h @@ -2,7 +2,7 @@ #include #include #include -#include +#include #include #include #include diff --git a/dbms/src/Dictionaries/XDBCDictionarySource.cpp b/dbms/src/Dictionaries/XDBCDictionarySource.cpp index b91ce104383..f2d41ec75c8 100644 --- a/dbms/src/Dictionaries/XDBCDictionarySource.cpp +++ b/dbms/src/Dictionaries/XDBCDictionarySource.cpp @@ -1,7 +1,7 @@ #include "XDBCDictionarySource.h" #include -#include +#include #include #include #include @@ -31,7 +31,7 @@ namespace ErrorCodes namespace { - class XDBCBridgeBlockInputStream : public IProfilingBlockInputStream + class XDBCBridgeBlockInputStream : public IBlockInputStream { public: XDBCBridgeBlockInputStream( diff --git a/dbms/src/Dictionaries/readInvalidateQuery.cpp b/dbms/src/Dictionaries/readInvalidateQuery.cpp index 67766f23548..5b9433d9a3b 100644 --- a/dbms/src/Dictionaries/readInvalidateQuery.cpp +++ b/dbms/src/Dictionaries/readInvalidateQuery.cpp @@ -1,5 +1,5 @@ #include "readInvalidateQuery.h" -#include +#include namespace DB { @@ -10,7 +10,7 @@ namespace ErrorCodes extern const int RECEIVED_EMPTY_DATA; } -std::string readInvalidateQuery(IProfilingBlockInputStream & block_input_stream) +std::string readInvalidateQuery(IBlockInputStream & block_input_stream) { block_input_stream.readPrefix(); std::string response; diff --git a/dbms/src/Dictionaries/readInvalidateQuery.h b/dbms/src/Dictionaries/readInvalidateQuery.h index a8bcde166ae..8810c291391 100644 --- a/dbms/src/Dictionaries/readInvalidateQuery.h +++ b/dbms/src/Dictionaries/readInvalidateQuery.h @@ -1,12 +1,12 @@ #pragma once #include -class IProfilingBlockInputStream; +class IBlockInputStream; namespace DB { // Using in MySQLDictionarySource and XDBCDictionarySource after processing invalidate_query -std::string readInvalidateQuery(IProfilingBlockInputStream & block_input_stream); +std::string readInvalidateQuery(IBlockInputStream & block_input_stream); } diff --git a/dbms/src/Formats/BlockInputStreamFromRowInputStream.h b/dbms/src/Formats/BlockInputStreamFromRowInputStream.h index c993f62adfe..b17e80e0dd3 100644 --- a/dbms/src/Formats/BlockInputStreamFromRowInputStream.h +++ b/dbms/src/Formats/BlockInputStreamFromRowInputStream.h @@ -1,7 +1,7 @@ #pragma once #include -#include +#include #include #include @@ -14,7 +14,7 @@ namespace DB * * Also controls over parsing errors and prints diagnostic information about them. */ -class BlockInputStreamFromRowInputStream : public IProfilingBlockInputStream +class BlockInputStreamFromRowInputStream : public IBlockInputStream { public: /** sample_ - block with zero rows, that structure describes how to interpret values */ diff --git a/dbms/src/Interpreters/Aggregator.cpp b/dbms/src/Interpreters/Aggregator.cpp index 145ce98dbbc..517f882f103 100644 --- a/dbms/src/Interpreters/Aggregator.cpp +++ b/dbms/src/Interpreters/Aggregator.cpp @@ -10,7 +10,7 @@ #include #include #include -#include +#include #include #include #include @@ -1723,7 +1723,7 @@ void NO_INLINE Aggregator::mergeBucketImpl( * (This is important for distributed processing.) * In doing so, it can handle different buckets in parallel, using up to `threads` threads. */ -class MergingAndConvertingBlockInputStream : public IProfilingBlockInputStream +class MergingAndConvertingBlockInputStream : public IBlockInputStream { public: /** The input is a set of non-empty sets of partially aggregated data, diff --git a/dbms/src/Interpreters/Context.h b/dbms/src/Interpreters/Context.h index 72354753e36..e25d08ce25a 100644 --- a/dbms/src/Interpreters/Context.h +++ b/dbms/src/Interpreters/Context.h @@ -329,10 +329,10 @@ public: void setProgressCallback(ProgressCallback callback); - /// Used in InterpreterSelectQuery to pass it to the IProfilingBlockInputStream. + /// Used in InterpreterSelectQuery to pass it to the IBlockInputStream. ProgressCallback getProgressCallback() const; - /** Set in executeQuery and InterpreterSelectQuery. Then it is used in IProfilingBlockInputStream, + /** Set in executeQuery and InterpreterSelectQuery. Then it is used in IBlockInputStream, * to update and monitor information about the total number of resources spent for the query. */ void setProcessListElement(QueryStatus * elem); diff --git a/dbms/src/Interpreters/DDLWorker.cpp b/dbms/src/Interpreters/DDLWorker.cpp index c7e1fa3c178..f17cb9be733 100644 --- a/dbms/src/Interpreters/DDLWorker.cpp +++ b/dbms/src/Interpreters/DDLWorker.cpp @@ -9,7 +9,7 @@ #include #include #include -#include +#include #include #include #include @@ -975,7 +975,7 @@ void DDLWorker::run() } -class DDLQueryStatusInputSream : public IProfilingBlockInputStream +class DDLQueryStatusInputSream : public IBlockInputStream { public: diff --git a/dbms/src/Interpreters/InterpreterKillQueryQuery.cpp b/dbms/src/Interpreters/InterpreterKillQueryQuery.cpp index fcb7b74bdbd..43f4e55297f 100644 --- a/dbms/src/Interpreters/InterpreterKillQueryQuery.cpp +++ b/dbms/src/Interpreters/InterpreterKillQueryQuery.cpp @@ -101,7 +101,7 @@ static QueryDescriptors extractQueriesExceptMeAndCheckAccess(const Block & proce -class SyncKillQueryInputStream : public IProfilingBlockInputStream +class SyncKillQueryInputStream : public IBlockInputStream { public: SyncKillQueryInputStream(ProcessList & process_list_, QueryDescriptors && processes_to_stop_, Block && processes_block_, @@ -140,7 +140,7 @@ public: auto code = process_list.sendCancelToQuery(curr_process.query_id, curr_process.user, true); /// Raise exception if this query is immortal, user have to know - /// This could happen only if query generate streams that don't implement IProfilingBlockInputStream + /// This could happen only if query generate streams that don't implement IBlockInputStream if (code == CancellationCode::CancelCannotBeSent) throw Exception("Can't kill query '" + curr_process.query_id + "' it consits of unkillable stages", ErrorCodes::CANNOT_KILL); else if (code != CancellationCode::QueryIsNotInitializedYet && code != CancellationCode::CancelSent) @@ -201,7 +201,7 @@ BlockIO InterpreterKillQueryQuery::execute() auto code = (query.test) ? CancellationCode::Unknown : process_list.sendCancelToQuery(query_desc.query_id, query_desc.user, true); /// Raise exception if this query is immortal, user have to know - /// This could happen only if query generate streams that don't implement IProfilingBlockInputStream + /// This could happen only if query generate streams that don't implement IBlockInputStream if (code == CancellationCode::CancelCannotBeSent) throw Exception("Can't kill query '" + query_desc.query_id + "' it consits of unkillable stages", ErrorCodes::CANNOT_KILL); diff --git a/dbms/src/Interpreters/InterpreterSelectQuery.cpp b/dbms/src/Interpreters/InterpreterSelectQuery.cpp index c490f280cc3..ed73e2d09ae 100644 --- a/dbms/src/Interpreters/InterpreterSelectQuery.cpp +++ b/dbms/src/Interpreters/InterpreterSelectQuery.cpp @@ -978,8 +978,8 @@ void InterpreterSelectQuery::executeFetchColumns( /// Set the limits and quota for reading data, the speed and time of the query. { - IProfilingBlockInputStream::LocalLimits limits; - limits.mode = IProfilingBlockInputStream::LIMITS_TOTAL; + IBlockInputStream::LocalLimits limits; + limits.mode = IBlockInputStream::LIMITS_TOTAL; limits.size_limits = SizeLimits(settings.max_rows_to_read, settings.max_bytes_to_read, settings.read_overflow_mode); limits.max_execution_time = settings.max_execution_time; limits.timeout_overflow_mode = settings.timeout_overflow_mode; @@ -1001,13 +1001,10 @@ void InterpreterSelectQuery::executeFetchColumns( pipeline.transform([&](auto & stream) { - if (IProfilingBlockInputStream * p_stream = dynamic_cast(stream.get())) - { - p_stream->setLimits(limits); + stream->setLimits(limits); - if (to_stage == QueryProcessingStage::Complete) - p_stream->setQuota(quota); - } + if (to_stage == QueryProcessingStage::Complete) + stream->setQuota(quota); }); } } @@ -1260,8 +1257,8 @@ void InterpreterSelectQuery::executeOrder(Pipeline & pipeline) auto sorting_stream = std::make_shared(stream, order_descr, limit); /// Limits on sorting - IProfilingBlockInputStream::LocalLimits limits; - limits.mode = IProfilingBlockInputStream::LIMITS_TOTAL; + IBlockInputStream::LocalLimits limits; + limits.mode = IBlockInputStream::LIMITS_TOTAL; limits.size_limits = SizeLimits(settings.max_rows_to_sort, settings.max_bytes_to_sort, settings.sort_overflow_mode); sorting_stream->setLimits(limits); @@ -1459,8 +1456,7 @@ void InterpreterSelectQuery::executeExtremes(Pipeline & pipeline) pipeline.transform([&](auto & stream) { - if (IProfilingBlockInputStream * p_stream = dynamic_cast(stream.get())) - p_stream->enableExtremes(); + stream->enableExtremes(); }); } @@ -1506,4 +1502,3 @@ void InterpreterSelectQuery::initSettings() } } - diff --git a/dbms/src/Interpreters/Join.cpp b/dbms/src/Interpreters/Join.cpp index 8783d16c3c1..4f093912f37 100644 --- a/dbms/src/Interpreters/Join.cpp +++ b/dbms/src/Interpreters/Join.cpp @@ -10,7 +10,7 @@ #include #include -#include +#include #include #include @@ -1041,7 +1041,7 @@ struct AdderNonJoined /// Stream from not joined earlier rows of the right table. -class NonJoinedBlockInputStream : public IProfilingBlockInputStream +class NonJoinedBlockInputStream : public IBlockInputStream { public: NonJoinedBlockInputStream(const Join & parent_, const Block & left_sample_block, const Names & key_names_left, size_t max_block_size_) diff --git a/dbms/src/Interpreters/ProcessList.cpp b/dbms/src/Interpreters/ProcessList.cpp index 4c254731277..0ba14316a3e 100644 --- a/dbms/src/Interpreters/ProcessList.cpp +++ b/dbms/src/Interpreters/ProcessList.cpp @@ -9,7 +9,7 @@ #include #include #include -#include +#include #include #include @@ -374,10 +374,9 @@ ProcessList::CancellationCode ProcessList::sendCancelToQuery(const String & curr if (elem->tryGetQueryStreams(input_stream, output_stream)) { - IProfilingBlockInputStream * input_stream_casted; - if (input_stream && (input_stream_casted = dynamic_cast(input_stream.get()))) + if (input_stream) { - input_stream_casted->cancel(kill); + input_stream->cancel(kill); return CancellationCode::CancelSent; } return CancellationCode::CancelCannotBeSent; diff --git a/dbms/src/Interpreters/Set.cpp b/dbms/src/Interpreters/Set.cpp index b41593acb4d..f61dd6d4570 100644 --- a/dbms/src/Interpreters/Set.cpp +++ b/dbms/src/Interpreters/Set.cpp @@ -9,7 +9,7 @@ #include -#include +#include #include #include diff --git a/dbms/src/Interpreters/executeQuery.cpp b/dbms/src/Interpreters/executeQuery.cpp index 0a2a3960ab7..1acf1e3a8c5 100644 --- a/dbms/src/Interpreters/executeQuery.cpp +++ b/dbms/src/Interpreters/executeQuery.cpp @@ -9,7 +9,7 @@ #include #include -#include +#include #include #include @@ -235,22 +235,19 @@ static std::tuple executeQueryImpl( if (res.in) { - if (auto stream = dynamic_cast(res.in.get())) + res.in->setProgressCallback(context.getProgressCallback()); + res.in->setProcessListElement(context.getProcessListElement()); + + /// Limits on the result, the quota on the result, and also callback for progress. + /// Limits apply only to the final result. + if (stage == QueryProcessingStage::Complete) { - stream->setProgressCallback(context.getProgressCallback()); - stream->setProcessListElement(context.getProcessListElement()); + IBlockInputStream::LocalLimits limits; + limits.mode = IBlockInputStream::LIMITS_CURRENT; + limits.size_limits = SizeLimits(settings.max_result_rows, settings.max_result_bytes, settings.result_overflow_mode); - /// Limits on the result, the quota on the result, and also callback for progress. - /// Limits apply only to the final result. - if (stage == QueryProcessingStage::Complete) - { - IProfilingBlockInputStream::LocalLimits limits; - limits.mode = IProfilingBlockInputStream::LIMITS_CURRENT; - limits.size_limits = SizeLimits(settings.max_result_rows, settings.max_result_bytes, settings.result_overflow_mode); - - stream->setLimits(limits); - stream->setQuota(quota); - } + res.in->setLimits(limits); + res.in->setQuota(quota); } } @@ -317,14 +314,11 @@ static std::tuple executeQueryImpl( if (stream_in) { - if (auto profiling_stream = dynamic_cast(stream_in)) - { - const BlockStreamProfileInfo & stream_in_info = profiling_stream->getProfileInfo(); + const BlockStreamProfileInfo & stream_in_info = stream_in->getProfileInfo(); - /// NOTE: INSERT SELECT query contains zero metrics - elem.result_rows = stream_in_info.rows; - elem.result_bytes = stream_in_info.bytes; - } + /// NOTE: INSERT SELECT query contains zero metrics + elem.result_rows = stream_in_info.rows; + elem.result_bytes = stream_in_info.bytes; } else if (stream_out) /// will be used only for ordinary INSERT queries { @@ -504,19 +498,16 @@ void executeQuery( BlockOutputStreamPtr out = context.getOutputFormat(format_name, *out_buf, streams.in->getHeader()); - if (auto stream = dynamic_cast(streams.in.get())) - { - /// Save previous progress callback if any. TODO Do it more conveniently. - auto previous_progress_callback = context.getProgressCallback(); + /// Save previous progress callback if any. TODO Do it more conveniently. + auto previous_progress_callback = context.getProgressCallback(); - /// NOTE Progress callback takes shared ownership of 'out'. - stream->setProgressCallback([out, previous_progress_callback] (const Progress & progress) - { - if (previous_progress_callback) - previous_progress_callback(progress); - out->onProgress(progress); - }); - } + /// NOTE Progress callback takes shared ownership of 'out'. + streams.in->setProgressCallback([out, previous_progress_callback] (const Progress & progress) + { + if (previous_progress_callback) + previous_progress_callback(progress); + out->onProgress(progress); + }); if (set_content_type) set_content_type(out->getContentType()); diff --git a/dbms/src/Storages/Kafka/StorageKafka.cpp b/dbms/src/Storages/Kafka/StorageKafka.cpp index 94ed3497f25..2d8b85ad639 100644 --- a/dbms/src/Storages/Kafka/StorageKafka.cpp +++ b/dbms/src/Storages/Kafka/StorageKafka.cpp @@ -2,7 +2,7 @@ #if USE_RDKAFKA -#include +#include #include #include #include @@ -131,7 +131,7 @@ public: } }; -class KafkaBlockInputStream : public IProfilingBlockInputStream +class KafkaBlockInputStream : public IBlockInputStream { public: KafkaBlockInputStream(StorageKafka & storage_, const Context & context_, const String & schema, size_t max_block_size_) @@ -492,11 +492,10 @@ bool StorageKafka::streamToViews() streams.emplace_back(stream); // Limit read batch to maximum block size to allow DDL - IProfilingBlockInputStream::LocalLimits limits; + IBlockInputStream::LocalLimits limits; limits.max_execution_time = settings.stream_flush_interval_ms; limits.timeout_overflow_mode = OverflowMode::BREAK; - if (IProfilingBlockInputStream * p_stream = dynamic_cast(stream.get())) - p_stream->setLimits(limits); + stream->setLimits(limits); } // Join multiple streams if necessary @@ -513,11 +512,8 @@ bool StorageKafka::streamToViews() // Check whether the limits were applied during query execution bool limits_applied = false; - if (IProfilingBlockInputStream * p_stream = dynamic_cast(in.get())) - { - const BlockStreamProfileInfo & info = p_stream->getProfileInfo(); - limits_applied = info.hasAppliedLimit(); - } + const BlockStreamProfileInfo & info = in->getProfileInfo(); + limits_applied = info.hasAppliedLimit(); return limits_applied; } diff --git a/dbms/src/Storages/MergeTree/MergeTreeBaseSelectBlockInputStream.h b/dbms/src/Storages/MergeTree/MergeTreeBaseSelectBlockInputStream.h index c9e7831fc78..5580bf1e6d9 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeBaseSelectBlockInputStream.h +++ b/dbms/src/Storages/MergeTree/MergeTreeBaseSelectBlockInputStream.h @@ -1,6 +1,6 @@ #pragma once -#include +#include #include #include #include @@ -14,7 +14,7 @@ class MarkCache; /// Base class for MergeTreeThreadSelectBlockInputStream and MergeTreeSelectBlockInputStream -class MergeTreeBaseSelectBlockInputStream : public IProfilingBlockInputStream +class MergeTreeBaseSelectBlockInputStream : public IBlockInputStream { public: MergeTreeBaseSelectBlockInputStream( diff --git a/dbms/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp b/dbms/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp index e8932397e51..acc94cd016a 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp @@ -648,7 +648,7 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mergePartsToTempor /// The order of the streams is important: when the key is matched, the elements go in the order of the source stream number. /// In the merged part, the lines with the same key must be in the ascending order of the identifier of original part, /// that is going in insertion order. - std::shared_ptr merged_stream; + std::shared_ptr merged_stream; switch (data.merging_params.mode) { diff --git a/dbms/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp b/dbms/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp index 01fb3169013..897083bee2f 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp @@ -662,7 +662,7 @@ BlockInputStreams MergeTreeDataSelectExecutor::spreadMarkRangesAmongStreams( if (i == 0) { /// Set the approximate number of rows for the first source only - static_cast(*res.front()).addTotalRowsApprox(total_rows); + res.front()->addTotalRowsApprox(total_rows); } } } diff --git a/dbms/src/Storages/MergeTree/MergeTreeSelectBlockInputStream.h b/dbms/src/Storages/MergeTree/MergeTreeSelectBlockInputStream.h index 7ff54838ffb..2bf6443ee8a 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeSelectBlockInputStream.h +++ b/dbms/src/Storages/MergeTree/MergeTreeSelectBlockInputStream.h @@ -1,5 +1,5 @@ #pragma once -#include +#include #include #include #include diff --git a/dbms/src/Storages/MergeTree/MergeTreeSequentialBlockInputStream.h b/dbms/src/Storages/MergeTree/MergeTreeSequentialBlockInputStream.h index 7c0d2cc469c..aab1b89d31f 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeSequentialBlockInputStream.h +++ b/dbms/src/Storages/MergeTree/MergeTreeSequentialBlockInputStream.h @@ -1,5 +1,5 @@ #pragma once -#include +#include #include #include #include @@ -9,7 +9,7 @@ namespace DB { /// Lightweight (in terms of logic) stream for reading single part from MergeTree -class MergeTreeSequentialBlockInputStream : public IProfilingBlockInputStream +class MergeTreeSequentialBlockInputStream : public IBlockInputStream { public: MergeTreeSequentialBlockInputStream( diff --git a/dbms/src/Storages/StorageBuffer.cpp b/dbms/src/Storages/StorageBuffer.cpp index 5d76279c95f..9af0742e210 100644 --- a/dbms/src/Storages/StorageBuffer.cpp +++ b/dbms/src/Storages/StorageBuffer.cpp @@ -6,7 +6,7 @@ #include #include #include -#include +#include #include #include #include @@ -80,7 +80,7 @@ StorageBuffer::~StorageBuffer() /// Reads from one buffer (from one block) under its mutex. -class BufferBlockInputStream : public IProfilingBlockInputStream +class BufferBlockInputStream : public IBlockInputStream { public: BufferBlockInputStream(const Names & column_names_, StorageBuffer::Buffer & buffer_, const StorageBuffer & storage_) diff --git a/dbms/src/Storages/StorageCatBoostPool.cpp b/dbms/src/Storages/StorageCatBoostPool.cpp index 3a9dbc258d6..011b5a7d80c 100644 --- a/dbms/src/Storages/StorageCatBoostPool.cpp +++ b/dbms/src/Storages/StorageCatBoostPool.cpp @@ -3,7 +3,7 @@ #include #include -#include +#include #include #include #include @@ -25,7 +25,7 @@ namespace ErrorCodes namespace { -class CatBoostDatasetBlockInputStream : public IProfilingBlockInputStream +class CatBoostDatasetBlockInputStream : public IBlockInputStream { public: diff --git a/dbms/src/Storages/StorageFile.cpp b/dbms/src/Storages/StorageFile.cpp index 341160f69f1..95f38862d9b 100644 --- a/dbms/src/Storages/StorageFile.cpp +++ b/dbms/src/Storages/StorageFile.cpp @@ -12,7 +12,7 @@ #include #include -#include +#include #include #include @@ -113,7 +113,7 @@ StorageFile::StorageFile( } -class StorageFileBlockInputStream : public IProfilingBlockInputStream +class StorageFileBlockInputStream : public IBlockInputStream { public: StorageFileBlockInputStream(StorageFile & storage_, const Context & context, size_t max_block_size) diff --git a/dbms/src/Storages/StorageHDFS.cpp b/dbms/src/Storages/StorageHDFS.cpp index 5384cf6bafd..7b553fc1ab8 100644 --- a/dbms/src/Storages/StorageHDFS.cpp +++ b/dbms/src/Storages/StorageHDFS.cpp @@ -12,7 +12,7 @@ #include #include #include -#include +#include #include @@ -41,7 +41,7 @@ StorageHDFS::StorageHDFS(const String & uri_, namespace { -class HDFSBlockInputStream : public IProfilingBlockInputStream +class HDFSBlockInputStream : public IBlockInputStream { public: HDFSBlockInputStream(const String & uri, diff --git a/dbms/src/Storages/StorageJoin.cpp b/dbms/src/Storages/StorageJoin.cpp index e188afc1dab..fd6cb4cbf2c 100644 --- a/dbms/src/Storages/StorageJoin.cpp +++ b/dbms/src/Storages/StorageJoin.cpp @@ -4,7 +4,7 @@ #include #include #include -#include +#include #include #include /// toLower @@ -186,7 +186,7 @@ size_t rawSize(const StringRef & t) return t.size; } -class JoinBlockInputStream : public IProfilingBlockInputStream +class JoinBlockInputStream : public IBlockInputStream { public: JoinBlockInputStream(const Join & parent_, size_t max_block_size_, Block && sample_block_) diff --git a/dbms/src/Storages/StorageLog.cpp b/dbms/src/Storages/StorageLog.cpp index 1f2fa87da53..76329da7827 100644 --- a/dbms/src/Storages/StorageLog.cpp +++ b/dbms/src/Storages/StorageLog.cpp @@ -13,7 +13,7 @@ #include -#include +#include #include #include @@ -45,7 +45,7 @@ namespace ErrorCodes } -class LogBlockInputStream final : public IProfilingBlockInputStream +class LogBlockInputStream final : public IBlockInputStream { public: LogBlockInputStream( diff --git a/dbms/src/Storages/StorageMemory.cpp b/dbms/src/Storages/StorageMemory.cpp index 29d084e7480..3e2e779244f 100644 --- a/dbms/src/Storages/StorageMemory.cpp +++ b/dbms/src/Storages/StorageMemory.cpp @@ -1,6 +1,6 @@ #include -#include +#include #include #include @@ -17,7 +17,7 @@ namespace ErrorCodes } -class MemoryBlockInputStream : public IProfilingBlockInputStream +class MemoryBlockInputStream : public IBlockInputStream { public: MemoryBlockInputStream(const Names & column_names_, BlocksList::iterator begin_, BlocksList::iterator end_, const StorageMemory & storage_) diff --git a/dbms/src/Storages/StorageStripeLog.cpp b/dbms/src/Storages/StorageStripeLog.cpp index 0512682f9de..3519dfbc37c 100644 --- a/dbms/src/Storages/StorageStripeLog.cpp +++ b/dbms/src/Storages/StorageStripeLog.cpp @@ -16,7 +16,7 @@ #include #include -#include +#include #include #include #include @@ -48,7 +48,7 @@ namespace ErrorCodes } -class StripeLogBlockInputStream final : public IProfilingBlockInputStream +class StripeLogBlockInputStream final : public IBlockInputStream { public: StripeLogBlockInputStream(StorageStripeLog & storage_, size_t max_read_buffer_size_, diff --git a/dbms/src/Storages/StorageTinyLog.cpp b/dbms/src/Storages/StorageTinyLog.cpp index c7546742202..4bca40520b4 100644 --- a/dbms/src/Storages/StorageTinyLog.cpp +++ b/dbms/src/Storages/StorageTinyLog.cpp @@ -20,7 +20,7 @@ #include -#include +#include #include #include @@ -53,7 +53,7 @@ namespace ErrorCodes } -class TinyLogBlockInputStream final : public IProfilingBlockInputStream +class TinyLogBlockInputStream final : public IBlockInputStream { public: TinyLogBlockInputStream(size_t block_size_, const NamesAndTypesList & columns_, StorageTinyLog & storage_, size_t max_read_buffer_size_) diff --git a/dbms/src/Storages/StorageURL.cpp b/dbms/src/Storages/StorageURL.cpp index af8eb71c654..cc2dda4a67a 100644 --- a/dbms/src/Storages/StorageURL.cpp +++ b/dbms/src/Storages/StorageURL.cpp @@ -11,7 +11,7 @@ #include #include -#include +#include #include #include @@ -35,7 +35,7 @@ IStorageURLBase::IStorageURLBase(const Poco::URI & uri_, namespace { - class StorageURLBlockInputStream : public IProfilingBlockInputStream + class StorageURLBlockInputStream : public IBlockInputStream { public: StorageURLBlockInputStream(const Poco::URI & uri, diff --git a/dbms/src/Storages/System/StorageSystemColumns.cpp b/dbms/src/Storages/System/StorageSystemColumns.cpp index 81e9a4cba6e..8229b4f0d7c 100644 --- a/dbms/src/Storages/System/StorageSystemColumns.cpp +++ b/dbms/src/Storages/System/StorageSystemColumns.cpp @@ -52,7 +52,7 @@ namespace } -class ColumnsBlockInputStream : public IProfilingBlockInputStream +class ColumnsBlockInputStream : public IBlockInputStream { public: ColumnsBlockInputStream( diff --git a/dbms/src/Storages/System/StorageSystemNumbers.cpp b/dbms/src/Storages/System/StorageSystemNumbers.cpp index 9132f78281b..dbf5bb54783 100644 --- a/dbms/src/Storages/System/StorageSystemNumbers.cpp +++ b/dbms/src/Storages/System/StorageSystemNumbers.cpp @@ -1,14 +1,14 @@ #include #include #include -#include +#include #include #include namespace DB { -class NumbersBlockInputStream : public IProfilingBlockInputStream +class NumbersBlockInputStream : public IBlockInputStream { public: NumbersBlockInputStream(size_t block_size_, size_t offset_, size_t step_) diff --git a/dbms/src/Storages/System/StorageSystemTables.cpp b/dbms/src/Storages/System/StorageSystemTables.cpp index 3fb22f8db66..f7de3713bc0 100644 --- a/dbms/src/Storages/System/StorageSystemTables.cpp +++ b/dbms/src/Storages/System/StorageSystemTables.cpp @@ -60,7 +60,7 @@ static ColumnPtr getFilteredDatabases(const ASTPtr & query, const Context & cont } -class TablesBlockInputStream : public IProfilingBlockInputStream +class TablesBlockInputStream : public IBlockInputStream { public: TablesBlockInputStream( diff --git a/dbms/src/Storages/tests/CMakeLists.txt b/dbms/src/Storages/tests/CMakeLists.txt index b43f03feb52..c6704628620 100644 --- a/dbms/src/Storages/tests/CMakeLists.txt +++ b/dbms/src/Storages/tests/CMakeLists.txt @@ -26,4 +26,4 @@ add_executable (get_abandonable_lock_in_all_partitions get_abandonable_lock_in_a target_link_libraries (get_abandonable_lock_in_all_partitions PRIVATE dbms clickhouse_common_config clickhouse_common_zookeeper) add_executable (transform_part_zk_nodes transform_part_zk_nodes.cpp) -target_link_libraries (transform_part_zk_nodes PRIVATE dbms clickhouse_common_config clickhouse_common_zookeeper) +target_link_libraries (transform_part_zk_nodes PRIVATE dbms clickhouse_common_config clickhouse_common_zookeeper string_utils ${Boost_PROGRAM_OPTIONS_LIBRARY}) From 784c3e5fdacd9f8de28ab818cc103d4d991a9abb Mon Sep 17 00:00:00 2001 From: Danila Kutenin Date: Wed, 23 Jan 2019 18:24:14 +0300 Subject: [PATCH 24/86] fix of ppc64le definitions --- cmake/arch.cmake | 2 +- cmake/test_cpu.cmake | 2 +- contrib/libdivide/libdivide.h | 2 +- contrib/libhdfs3-cmake/CMake/Options.cmake | 4 ++-- contrib/libunwind/CMakeLists.txt | 2 +- dbms/src/IO/parseDateTimeBestEffort.cpp | 4 ++-- libs/libcommon/cmake/find_jemalloc.cmake | 2 +- libs/libcommon/include/common/DateLUTImpl.h | 4 ++-- 8 files changed, 11 insertions(+), 11 deletions(-) diff --git a/cmake/arch.cmake b/cmake/arch.cmake index 8f772d3cdcf..f8e18629b09 100644 --- a/cmake/arch.cmake +++ b/cmake/arch.cmake @@ -26,7 +26,7 @@ elseif (CMAKE_CXX_COMPILER_ID STREQUAL "Clang") endif () if (CMAKE_SYSTEM_PROCESSOR MATCHES "^(ppc64le.*|PPC64LE.*)") - set (PPC64LE 1) + set (ARCH_PPC64LE 1) if (COMPILER_CLANG OR (COMPILER_GCC AND CMAKE_CXX_COMPILER_VERSION VERSION_LESS 8)) message(FATAL_ERROR "Only gcc-8 is supported for powerpc architecture") endif () diff --git a/cmake/test_cpu.cmake b/cmake/test_cpu.cmake index 5b95ac5d25a..42f9bac936e 100644 --- a/cmake/test_cpu.cmake +++ b/cmake/test_cpu.cmake @@ -27,7 +27,7 @@ if (HAVE_SSE41) set (COMPILER_FLAGS "${COMPILER_FLAGS} ${TEST_FLAG}") endif () -if (PPC64LE) +if (ARCH_PPC64LE) set (COMPILER_FLAGS "${COMPILER_FLAGS} -maltivec -D__SSE2__=1 -DNO_WARN_X86_INTRINSICS") endif () diff --git a/contrib/libdivide/libdivide.h b/contrib/libdivide/libdivide.h index 1404295f814..eaeaec7db6b 100644 --- a/contrib/libdivide/libdivide.h +++ b/contrib/libdivide/libdivide.h @@ -342,7 +342,7 @@ static inline __m128i libdivide_get_0000FFFF(void) { #endif /// This is a bug in gcc-8, _MM_SHUFFLE was forgotten, though in trunk it is ok https://github.com/gcc-mirror/gcc/blob/master/gcc/config/rs6000/xmmintrin.h#L61 -#if __PPC__ +#if defined(__PPC__) #ifndef _MM_SHUFFLE #define _MM_SHUFFLE(w,x,y,z) (((w) << 6) | ((x) << 4) | ((y) << 2) | (z)) #endif diff --git a/contrib/libhdfs3-cmake/CMake/Options.cmake b/contrib/libhdfs3-cmake/CMake/Options.cmake index c35eac3c08d..d7ccc8b6475 100644 --- a/contrib/libhdfs3-cmake/CMake/Options.cmake +++ b/contrib/libhdfs3-cmake/CMake/Options.cmake @@ -7,9 +7,9 @@ CHECK_FUNCTION_EXISTS(nanosleep HAVE_NANOSLEEP) SET(CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} -fno-strict-aliasing") SET(CMAKE_C_FLAGS "${CMAKE_C_FLAGS} -fno-strict-aliasing") -IF(ENABLE_SSE STREQUAL ON AND NOT PPC64LE AND NOT ARCH_AARCH64 AND NOT ARCH_ARM) +IF(ENABLE_SSE STREQUAL ON AND NOT ARCH_PPC64LE AND NOT ARCH_AARCH64 AND NOT ARCH_ARM) SET(CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} -msse4.2") -ENDIF(ENABLE_SSE STREQUAL ON) +ENDIF(ENABLE_SSE STREQUAL ON AND NOT ARCH_PPC64LE AND NOT ARCH_AARCH64 AND NOT ARCH_ARM) IF(NOT TEST_HDFS_PREFIX) SET(TEST_HDFS_PREFIX "./" CACHE STRING "default directory prefix used for test." FORCE) diff --git a/contrib/libunwind/CMakeLists.txt b/contrib/libunwind/CMakeLists.txt index 22470069938..80bd8a301f0 100644 --- a/contrib/libunwind/CMakeLists.txt +++ b/contrib/libunwind/CMakeLists.txt @@ -1,6 +1,6 @@ enable_language(ASM) -if (PPC64LE) +if (ARCH_PPC64LE) add_library(unwind src/mi/init.c src/mi/flush_cache.c diff --git a/dbms/src/IO/parseDateTimeBestEffort.cpp b/dbms/src/IO/parseDateTimeBestEffort.cpp index f17c3fc61a8..11208424c62 100644 --- a/dbms/src/IO/parseDateTimeBestEffort.cpp +++ b/dbms/src/IO/parseDateTimeBestEffort.cpp @@ -44,7 +44,7 @@ inline size_t readAlpha(char * res, size_t max_chars, ReadBuffer & in) return num_chars; } -#if __PPC__ +#if defined(__PPC__) #if !__clang__ #pragma GCC diagnostic ignored "-Wmaybe-uninitialized" #endif @@ -519,7 +519,7 @@ ReturnType parseDateTimeBestEffortImpl(time_t & res, ReadBuffer & in, const Date } -#if __PPC__ +#if defined(__PPC__) #if !__clang__ #pragma GCC diagnostic pop #endif diff --git a/libs/libcommon/cmake/find_jemalloc.cmake b/libs/libcommon/cmake/find_jemalloc.cmake index c611dfc541a..3a1b14d9c33 100644 --- a/libs/libcommon/cmake/find_jemalloc.cmake +++ b/libs/libcommon/cmake/find_jemalloc.cmake @@ -1,4 +1,4 @@ -if (OS_LINUX AND NOT SANITIZE AND NOT ARCH_ARM AND NOT ARCH_32 AND NOT PPC64LE) +if (OS_LINUX AND NOT SANITIZE AND NOT ARCH_ARM AND NOT ARCH_32 AND NOT ARCH_PPC64LE) set(ENABLE_JEMALLOC_DEFAULT 1) else () set(ENABLE_JEMALLOC_DEFAULT 0) diff --git a/libs/libcommon/include/common/DateLUTImpl.h b/libs/libcommon/include/common/DateLUTImpl.h index c529d0938b5..936efa31ae4 100644 --- a/libs/libcommon/include/common/DateLUTImpl.h +++ b/libs/libcommon/include/common/DateLUTImpl.h @@ -14,7 +14,7 @@ #define DATE_LUT_MAX_YEAR 2105 /// Last supported year #define DATE_LUT_YEARS (1 + DATE_LUT_MAX_YEAR - DATE_LUT_MIN_YEAR) /// Number of years in lookup table -#if __PPC__ +#if defined(__PPC__) #if !__clang__ #pragma GCC diagnostic ignored "-Wmaybe-uninitialized" #endif @@ -690,7 +690,7 @@ public: } }; -#if __PPC__ +#if defined(__PPC__) #if !__clang__ #pragma GCC diagnostic pop #endif From 0ac0557e0f13c1049be42916da190d4eea0b513e Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Wed, 23 Jan 2019 22:19:23 +0300 Subject: [PATCH 25/86] Make DBMS dependent on the protobuf library. --- CMakeLists.txt | 1 + contrib/CMakeLists.txt | 14 +++++++------- dbms/CMakeLists.txt | 3 +++ 3 files changed, 11 insertions(+), 7 deletions(-) diff --git a/CMakeLists.txt b/CMakeLists.txt index 4265cc126f1..6bd0aae5fd1 100644 --- a/CMakeLists.txt +++ b/CMakeLists.txt @@ -232,6 +232,7 @@ include (cmake/find_llvm.cmake) include (cmake/find_cpuid.cmake) include (cmake/find_libgsasl.cmake) include (cmake/find_libxml2.cmake) +include (cmake/find_protobuf.cmake) include (cmake/find_hdfs3.cmake) include (cmake/find_consistent-hashing.cmake) include (cmake/find_base64.cmake) diff --git a/contrib/CMakeLists.txt b/contrib/CMakeLists.txt index 7b66bfce063..0c4b6c15287 100644 --- a/contrib/CMakeLists.txt +++ b/contrib/CMakeLists.txt @@ -207,14 +207,14 @@ if (USE_INTERNAL_LIBXML2_LIBRARY) add_subdirectory(libxml2-cmake) endif () +if (USE_INTERNAL_PROTOBUF_LIBRARY) + set(protobuf_BUILD_TESTS OFF CACHE INTERNAL "" FORCE) + set(protobuf_BUILD_SHARED_LIBS OFF CACHE INTERNAL "" FORCE) + set(protobuf_WITH_ZLIB 0 CACHE INTERNAL "" FORCE) # actually will use zlib, but skip find + add_subdirectory(protobuf/cmake) +endif () + if (USE_INTERNAL_HDFS3_LIBRARY) - include(${ClickHouse_SOURCE_DIR}/cmake/find_protobuf.cmake) - if (USE_INTERNAL_PROTOBUF_LIBRARY) - set(protobuf_BUILD_TESTS OFF CACHE INTERNAL "" FORCE) - set(protobuf_BUILD_SHARED_LIBS OFF CACHE INTERNAL "" FORCE) - set(protobuf_WITH_ZLIB 0 CACHE INTERNAL "" FORCE) # actually will use zlib, but skip find - add_subdirectory(protobuf/cmake) - endif () add_subdirectory(libhdfs3-cmake) endif () diff --git a/dbms/CMakeLists.txt b/dbms/CMakeLists.txt index 3ce29e9d65e..cd539444c97 100644 --- a/dbms/CMakeLists.txt +++ b/dbms/CMakeLists.txt @@ -298,6 +298,9 @@ 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}) +target_link_libraries (dbms PRIVATE ${Protobuf_LIBRARY}) +target_include_directories (dbms SYSTEM BEFORE PRIVATE ${Protobuf_INCLUDE_DIR}) + if (USE_HDFS) target_link_libraries (clickhouse_common_io PRIVATE ${HDFS3_LIBRARY}) target_include_directories (clickhouse_common_io SYSTEM BEFORE PRIVATE ${HDFS3_INCLUDE_DIR}) From 67fca7e02929ba9f4939657179beea5bae8237c4 Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Wed, 23 Jan 2019 22:23:37 +0300 Subject: [PATCH 26/86] Add support for settings 'format_schema' in an output query. This fixes syntax error when executing 'SELECT ... FORMAT ... SETTINGS format_schema=...' --- dbms/programs/client/Client.cpp | 5 +++++ dbms/src/Interpreters/executeQuery.cpp | 4 ++++ dbms/src/Parsers/ASTQueryWithOutput.cpp | 12 ++++++++++++ dbms/src/Parsers/ASTQueryWithOutput.h | 4 +++- dbms/src/Parsers/ParserQueryWithOutput.cpp | 11 +++++++++++ dbms/src/Parsers/ParserQueryWithOutput.h | 2 +- 6 files changed, 36 insertions(+), 2 deletions(-) diff --git a/dbms/programs/client/Client.cpp b/dbms/programs/client/Client.cpp index 0edffbcf4de..66acc7fcea5 100644 --- a/dbms/programs/client/Client.cpp +++ b/dbms/programs/client/Client.cpp @@ -56,6 +56,7 @@ #include #include #include +#include #include #include #include @@ -1206,6 +1207,10 @@ private: const auto & id = typeid_cast(*query_with_output->format); current_format = id.name; } + if (query_with_output->settings) + { + InterpreterSetQuery(query_with_output->settings, context).executeForCurrentContext(); + } } if (has_vertical_output_suffix) diff --git a/dbms/src/Interpreters/executeQuery.cpp b/dbms/src/Interpreters/executeQuery.cpp index 3911e437fa6..435de7c6f1f 100644 --- a/dbms/src/Interpreters/executeQuery.cpp +++ b/dbms/src/Interpreters/executeQuery.cpp @@ -26,6 +26,7 @@ #include #include #include +#include #include #include "DNSCacheUpdater.h" @@ -502,6 +503,9 @@ void executeQuery( ? *getIdentifierName(ast_query_with_output->format) : context.getDefaultFormat(); + if (ast_query_with_output && ast_query_with_output->settings) + InterpreterSetQuery(ast_query_with_output->settings, context).executeForCurrentContext(); + BlockOutputStreamPtr out = context.getOutputFormat(format_name, *out_buf, streams.in->getHeader()); if (auto stream = dynamic_cast(streams.in.get())) diff --git a/dbms/src/Parsers/ASTQueryWithOutput.cpp b/dbms/src/Parsers/ASTQueryWithOutput.cpp index 62abe5de9e1..fd6957c436b 100644 --- a/dbms/src/Parsers/ASTQueryWithOutput.cpp +++ b/dbms/src/Parsers/ASTQueryWithOutput.cpp @@ -15,6 +15,11 @@ void ASTQueryWithOutput::cloneOutputOptions(ASTQueryWithOutput & cloned) const cloned.format = format->clone(); cloned.children.push_back(cloned.format); } + if (settings) + { + cloned.settings = settings->clone(); + cloned.children.push_back(cloned.settings); + } } void ASTQueryWithOutput::formatImpl(const FormatSettings & s, FormatState & state, FormatStateStacked frame) const @@ -34,6 +39,12 @@ void ASTQueryWithOutput::formatImpl(const FormatSettings & s, FormatState & stat s.ostr << (s.hilite ? hilite_keyword : "") << s.nl_or_ws << indent_str << "FORMAT " << (s.hilite ? hilite_none : ""); format->formatImpl(s, state, frame); } + + if (settings) + { + s.ostr << (s.hilite ? hilite_keyword : "") << s.nl_or_ws << indent_str << "SETTINGS " << (s.hilite ? hilite_none : ""); + settings->formatImpl(s, state, frame); + } } bool ASTQueryWithOutput::resetOutputASTIfExist(IAST & ast) @@ -42,6 +53,7 @@ bool ASTQueryWithOutput::resetOutputASTIfExist(IAST & ast) { ast_with_output->format.reset(); ast_with_output->out_file.reset(); + ast_with_output->settings.reset(); return true; } diff --git a/dbms/src/Parsers/ASTQueryWithOutput.h b/dbms/src/Parsers/ASTQueryWithOutput.h index 0660b1bec63..ec525b3aca9 100644 --- a/dbms/src/Parsers/ASTQueryWithOutput.h +++ b/dbms/src/Parsers/ASTQueryWithOutput.h @@ -6,13 +6,15 @@ namespace DB { -/** Query with output options (supporting [INTO OUTFILE 'file_name'] [FORMAT format_name] suffix). +/** Query with output options + * (supporting [INTO OUTFILE 'file_name'] [FORMAT format_name] [SETTINGS key1 = value1, key2 = value2, ...] suffix). */ class ASTQueryWithOutput : public IAST { public: ASTPtr out_file; ASTPtr format; + ASTPtr settings; void formatImpl(const FormatSettings & s, FormatState & state, FormatStateStacked frame) const final; diff --git a/dbms/src/Parsers/ParserQueryWithOutput.cpp b/dbms/src/Parsers/ParserQueryWithOutput.cpp index 2e1a5ff529a..830060cb953 100644 --- a/dbms/src/Parsers/ParserQueryWithOutput.cpp +++ b/dbms/src/Parsers/ParserQueryWithOutput.cpp @@ -11,6 +11,7 @@ #include #include #include +#include #include @@ -81,6 +82,16 @@ bool ParserQueryWithOutput::parseImpl(Pos & pos, ASTPtr & node, Expected & expec query_with_output.children.push_back(query_with_output.format); } + // SETTINGS key1 = value1, key2 = value2, ... + ParserKeyword s_settings("SETTINGS"); + if (s_settings.ignore(pos, expected)) + { + ParserSetQuery parser_settings(true); + if (!parser_settings.parse(pos, query_with_output.settings, expected)) + return false; + query_with_output.children.push_back(query_with_output.settings); + } + if (explain_ast) { node = std::make_shared(); diff --git a/dbms/src/Parsers/ParserQueryWithOutput.h b/dbms/src/Parsers/ParserQueryWithOutput.h index b0d08465354..3163bc38437 100644 --- a/dbms/src/Parsers/ParserQueryWithOutput.h +++ b/dbms/src/Parsers/ParserQueryWithOutput.h @@ -7,7 +7,7 @@ namespace DB { -/// Parse queries supporting [INTO OUTFILE 'file_name'] [FORMAT format_name] suffix. +/// Parse queries supporting [INTO OUTFILE 'file_name'] [FORMAT format_name] [SETTINGS key1 = value1, key2 = value2, ...] suffix. class ParserQueryWithOutput : public IParserBase { public: From 135cfc42fadb11f7af987faf3a7efc62f5a517ee Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 23 Jan 2019 22:37:07 +0300 Subject: [PATCH 27/86] Fixed error when program cannot start within container due to capabilities. --- debian/clickhouse-server.postinst | 13 ++++++++++++- 1 file changed, 12 insertions(+), 1 deletion(-) diff --git a/debian/clickhouse-server.postinst b/debian/clickhouse-server.postinst index 694507b9366..875432b5471 100644 --- a/debian/clickhouse-server.postinst +++ b/debian/clickhouse-server.postinst @@ -88,9 +88,20 @@ Please fix this and reinstall this package." >&2 chmod 775 ${CLICKHOUSE_LOGDIR} fi - # Set net_admin capabilities to support introspection of "taskstats" performance metrics from the kernel. + # Set net_admin capabilities to support introspection of "taskstats" performance metrics from the kernel + # and ipc_lock capabilities to allow mlock of clickhouse binary. + + # 1. Check that "setcap" tool exists. + # 2. Check that an arbitrary program with installed capabilities can run. + # 3. Set the capabilities. + + # The second is important for Docker and systemd-nspawn. + # When the container has no capabilities, + # but the executable file inside the container has capabilities, + # then attempt to run this file will end up with a cryptic "Operation not permitted" message. command -v setcap >/dev/null \ + && echo > /tmp/test.sh && chmod a+x /tmp/test.sh && /tmp/test.sh && setcap "cap_net_admin,cap_ipc_lock+ep" /tmp/test.sh && /tmp/test.sh && rm /tmp/test.sh && setcap "cap_net_admin=+ep cap_ipc_lock=+ep" "${CLICKHOUSE_BINDIR}/${CLICKHOUSE_GENERIC_PROGRAM}" \ || echo "Cannot set 'net_admin' or 'ipc_lock' capability for clickhouse binary. This is optional. Taskstats accounting will be disabled. To enable taskstats accounting you may add the required capability later manually." From 5394ef08af69ad0df4ff8830616b9e5511f960d9 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 23 Jan 2019 22:43:17 +0300 Subject: [PATCH 28/86] Fixed error when program cannot start within container due to capabilities. --- debian/clickhouse-server.postinst | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/debian/clickhouse-server.postinst b/debian/clickhouse-server.postinst index 875432b5471..9313f4b2fe0 100644 --- a/debian/clickhouse-server.postinst +++ b/debian/clickhouse-server.postinst @@ -101,8 +101,8 @@ Please fix this and reinstall this package." >&2 # then attempt to run this file will end up with a cryptic "Operation not permitted" message. command -v setcap >/dev/null \ - && echo > /tmp/test.sh && chmod a+x /tmp/test.sh && /tmp/test.sh && setcap "cap_net_admin,cap_ipc_lock+ep" /tmp/test.sh && /tmp/test.sh && rm /tmp/test.sh - && setcap "cap_net_admin=+ep cap_ipc_lock=+ep" "${CLICKHOUSE_BINDIR}/${CLICKHOUSE_GENERIC_PROGRAM}" \ + && touch /tmp/test.sh && chmod a+x /tmp/test.sh && /tmp/test.sh && setcap "cap_net_admin,cap_ipc_lock+ep" /tmp/test.sh && /tmp/test.sh && rm /tmp/test.sh \ + && setcap "cap_net_admin,cap_ipc_lock+ep" "${CLICKHOUSE_BINDIR}/${CLICKHOUSE_GENERIC_PROGRAM}" \ || echo "Cannot set 'net_admin' or 'ipc_lock' capability for clickhouse binary. This is optional. Taskstats accounting will be disabled. To enable taskstats accounting you may add the required capability later manually." # Clean old dynamic compilation results From ae2e64f753bcf967380b81187d0406b76c15e09b Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 23 Jan 2019 22:45:31 +0300 Subject: [PATCH 29/86] Fixed error when program cannot start within container due to capabilities. --- debian/clickhouse-server.postinst | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/debian/clickhouse-server.postinst b/debian/clickhouse-server.postinst index 9313f4b2fe0..b8f2c8542ea 100644 --- a/debian/clickhouse-server.postinst +++ b/debian/clickhouse-server.postinst @@ -100,8 +100,10 @@ Please fix this and reinstall this package." >&2 # but the executable file inside the container has capabilities, # then attempt to run this file will end up with a cryptic "Operation not permitted" message. + TMPFILE=/tmp/test_setcap.sh + command -v setcap >/dev/null \ - && touch /tmp/test.sh && chmod a+x /tmp/test.sh && /tmp/test.sh && setcap "cap_net_admin,cap_ipc_lock+ep" /tmp/test.sh && /tmp/test.sh && rm /tmp/test.sh \ + && echo > $TMPFILE && chmod a+x $TMPFILE && $TMPFILE && setcap "cap_net_admin,cap_ipc_lock+ep" $TMPFILE && $TMPFILE && rm $TMPFILE \ && setcap "cap_net_admin,cap_ipc_lock+ep" "${CLICKHOUSE_BINDIR}/${CLICKHOUSE_GENERIC_PROGRAM}" \ || echo "Cannot set 'net_admin' or 'ipc_lock' capability for clickhouse binary. This is optional. Taskstats accounting will be disabled. To enable taskstats accounting you may add the required capability later manually." From 51dd2198f03efbf77630582e0d5f0ef2d4a2617b Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Thu, 24 Jan 2019 00:17:07 +0300 Subject: [PATCH 30/86] Auto version update to [19.1.6] [54413] --- dbms/cmake/version.cmake | 8 ++++---- debian/changelog | 4 ++-- docker/client/Dockerfile | 2 +- docker/server/Dockerfile | 2 +- docker/test/Dockerfile | 2 +- 5 files changed, 9 insertions(+), 9 deletions(-) diff --git a/dbms/cmake/version.cmake b/dbms/cmake/version.cmake index c684c9a4f7c..b39f7d14b2b 100644 --- a/dbms/cmake/version.cmake +++ b/dbms/cmake/version.cmake @@ -2,10 +2,10 @@ set(VERSION_REVISION 54413) set(VERSION_MAJOR 19) set(VERSION_MINOR 1) -set(VERSION_PATCH 5) -set(VERSION_GITHASH 2a7e7364c139b3c97f54f38ca6ea76ab4fa61e4b) -set(VERSION_DESCRIBE v19.1.5-testing) -set(VERSION_STRING 19.1.5) +set(VERSION_PATCH 6) +set(VERSION_GITHASH f73b337a93d534671b2187660398b8573fc1d464) +set(VERSION_DESCRIBE v19.1.6-testing) +set(VERSION_STRING 19.1.6) # end of autochange set(VERSION_EXTRA "" CACHE STRING "") diff --git a/debian/changelog b/debian/changelog index 33331e2f495..d21b8036d6f 100644 --- a/debian/changelog +++ b/debian/changelog @@ -1,5 +1,5 @@ -clickhouse (19.1.5) unstable; urgency=low +clickhouse (19.1.6) unstable; urgency=low * Modified source code - -- Tue, 22 Jan 2019 22:18:30 +0300 + -- Thu, 24 Jan 2019 00:17:07 +0300 diff --git a/docker/client/Dockerfile b/docker/client/Dockerfile index 688c52c372b..cb24d70efa8 100644 --- a/docker/client/Dockerfile +++ b/docker/client/Dockerfile @@ -1,7 +1,7 @@ FROM ubuntu:18.04 ARG repository="deb http://repo.yandex.ru/clickhouse/deb/stable/ main/" -ARG version=19.1.5 +ARG version=19.1.6 RUN apt-get update \ && apt-get install --yes --no-install-recommends \ diff --git a/docker/server/Dockerfile b/docker/server/Dockerfile index ef272d1447e..3a0f1d1cce9 100644 --- a/docker/server/Dockerfile +++ b/docker/server/Dockerfile @@ -1,7 +1,7 @@ FROM ubuntu:18.04 ARG repository="deb http://repo.yandex.ru/clickhouse/deb/stable/ main/" -ARG version=19.1.5 +ARG version=19.1.6 ARG gosu_ver=1.10 RUN apt-get update \ diff --git a/docker/test/Dockerfile b/docker/test/Dockerfile index 5965bc4e9a6..23c2a67d19d 100644 --- a/docker/test/Dockerfile +++ b/docker/test/Dockerfile @@ -1,7 +1,7 @@ FROM ubuntu:18.04 ARG repository="deb http://repo.yandex.ru/clickhouse/deb/stable/ main/" -ARG version=19.1.5 +ARG version=19.1.6 RUN apt-get update && \ apt-get install -y apt-transport-https dirmngr && \ From efb1cb0528601369f3ca89a95fff8ff999809588 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?L=C3=A9o=20Ercolanelli?= Date: Thu, 24 Jan 2019 08:58:57 +0100 Subject: [PATCH 31/86] sumMapFiltered: remove `this->` in constructor --- dbms/src/AggregateFunctions/AggregateFunctionSumMap.h | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/dbms/src/AggregateFunctions/AggregateFunctionSumMap.h b/dbms/src/AggregateFunctions/AggregateFunctionSumMap.h index e9c70eaa5f1..1e39db916f0 100644 --- a/dbms/src/AggregateFunctions/AggregateFunctionSumMap.h +++ b/dbms/src/AggregateFunctions/AggregateFunctionSumMap.h @@ -282,15 +282,15 @@ private: std::vector keys_to_keep; public: - AggregateFunctionSumMapFiltered(const DataTypePtr & keys_type, const DataTypes & values_types, const Array & keys_to_keep) + AggregateFunctionSumMapFiltered(const DataTypePtr & keys_type, const DataTypes & values_types, const Array & keys_to_keep_) : AggregateFunctionSumMapBase>{keys_type, values_types} { - this->keys_to_keep.reserve(keys_to_keep.size()); - for (const Field & f : keys_to_keep) + keys_to_keep.reserve(keys_to_keep_.size()); + for (const Field & f : keys_to_keep_) { - this->keys_to_keep.emplace_back(f.safeGet>()); + keys_to_keep.emplace_back(f.safeGet>()); } - std::sort(begin(this->keys_to_keep), end(this->keys_to_keep)); + std::sort(begin(keys_to_keep), end(keys_to_keep)); } String getName() const override { return "sumMapFiltered"; } From 22f7bd03b9bb03a6d83317c16cff4d83d67a0ea7 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?L=C3=A9o=20Ercolanelli?= Date: Thu, 24 Jan 2019 09:02:17 +0100 Subject: [PATCH 32/86] sumMapFiltered: use CRTP instead of virtual for keepKey --- dbms/src/AggregateFunctions/AggregateFunctionSumMap.h | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/dbms/src/AggregateFunctions/AggregateFunctionSumMap.h b/dbms/src/AggregateFunctions/AggregateFunctionSumMap.h index 1e39db916f0..17dcf91d5e1 100644 --- a/dbms/src/AggregateFunctions/AggregateFunctionSumMap.h +++ b/dbms/src/AggregateFunctions/AggregateFunctionSumMap.h @@ -259,7 +259,7 @@ public: const char * getHeaderFilePath() const override { return __FILE__; } - virtual bool keepKey(const T & key) const = 0; + bool keepKey(const T & key) const { return static_cast(*this).keepKey(key); } }; template @@ -272,7 +272,7 @@ public: String getName() const override { return "sumMap"; } - bool keepKey(const T &) const override { return true; } + bool keepKey(const T &) const { return true; } }; template @@ -295,7 +295,7 @@ public: String getName() const override { return "sumMapFiltered"; } - bool keepKey(const T & key) const override + bool keepKey(const T & key) const { return std::binary_search(begin(keys_to_keep), end(keys_to_keep), key); } From a8524b6e363e719691df4d7b692705e85bfd4560 Mon Sep 17 00:00:00 2001 From: alesapin Date: Thu, 24 Jan 2019 14:02:55 +0300 Subject: [PATCH 33/86] Fix wrong code in clickhouse-test --- dbms/tests/clickhouse-test | 19 ++++++++++++++----- 1 file changed, 14 insertions(+), 5 deletions(-) diff --git a/dbms/tests/clickhouse-test b/dbms/tests/clickhouse-test index ac745c894c2..c66eb51317c 100755 --- a/dbms/tests/clickhouse-test +++ b/dbms/tests/clickhouse-test @@ -328,6 +328,14 @@ def main(args): print(colored("\n{passed_total} tests passed. {skipped_total} tests skipped.".format(passed_total = passed_total, skipped_total = skipped_total), "green", attrs=["bold"])) sys.exit(0) +def find_binary(name): + paths = os.environ.get("PATH").split(':') + for path in paths: + if os.access(os.path.join(path, name), os.X_OK): + return True + + # maybe it wasn't in PATH + return os.access(os.path.join('/usr/bin', name), os.X_OK) if __name__ == '__main__': parser = ArgumentParser(description = 'ClickHouse functional tests') @@ -370,14 +378,15 @@ if __name__ == '__main__': if args.tmp is None: args.tmp = args.queries if args.client is None: - if os.access(args.binary + '-client', os.X_OK): + if find_binary(args.binary + '-client'): args.client = args.binary + '-client' - elif os.access(args.binary,os.X_OK): + elif find_binary(args.binary): args.client = args.binary + ' client' else: - print("No clickhouse binary found") - parser.print_help(); - exit(0); + print("No 'clickhouse' binary found in PATH") + parser.print_help() + exit(1) + if args.configclient: args.client += ' --config-file=' + args.configclient if os.getenv("CLICKHOUSE_HOST"): From a8647761ce462a54fb007377834e5f579986825b Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?L=C3=A9o=20Ercolanelli?= Date: Thu, 24 Jan 2019 09:10:08 +0100 Subject: [PATCH 34/86] sumMapFiltered: fix dangling reference --- dbms/src/AggregateFunctions/AggregateFunctionSumMap.cpp | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/dbms/src/AggregateFunctions/AggregateFunctionSumMap.cpp b/dbms/src/AggregateFunctions/AggregateFunctionSumMap.cpp index 5138d8f1f02..151e64640eb 100644 --- a/dbms/src/AggregateFunctions/AggregateFunctionSumMap.cpp +++ b/dbms/src/AggregateFunctions/AggregateFunctionSumMap.cpp @@ -12,7 +12,7 @@ namespace DB namespace { -using SumMapArgs = std::pair; +using SumMapArgs = std::pair; SumMapArgs parseArguments(const std::string & name, const DataTypes & arguments) { @@ -26,9 +26,10 @@ SumMapArgs parseArguments(const std::string & name, const DataTypes & arguments) ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); - const DataTypePtr & keys_type = array_type->getNestedType(); + DataTypePtr keys_type = array_type->getNestedType(); DataTypes values_types; + values_types.reserve(arguments.size() - 1); for (size_t i = 1; i < arguments.size(); ++i) { array_type = checkAndGetDataType(arguments[i].get()); @@ -38,7 +39,7 @@ SumMapArgs parseArguments(const std::string & name, const DataTypes & arguments) values_types.push_back(array_type->getNestedType()); } - return {keys_type, std::move(values_types)}; + return {std::move(keys_type), std::move(values_types)}; } AggregateFunctionPtr createAggregateFunctionSumMap(const std::string & name, const DataTypes & arguments, const Array & params) From 90afad8c00a523267271c6fa77cc599f91522bef Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?L=C3=A9o=20Ercolanelli?= Date: Thu, 24 Jan 2019 13:30:31 +0100 Subject: [PATCH 35/86] sumMapFiltered: improve error reporting on parameter type mismatch --- dbms/src/AggregateFunctions/AggregateFunctionSumMap.cpp | 7 +++++-- 1 file changed, 5 insertions(+), 2 deletions(-) diff --git a/dbms/src/AggregateFunctions/AggregateFunctionSumMap.cpp b/dbms/src/AggregateFunctions/AggregateFunctionSumMap.cpp index 151e64640eb..02303b953d9 100644 --- a/dbms/src/AggregateFunctions/AggregateFunctionSumMap.cpp +++ b/dbms/src/AggregateFunctions/AggregateFunctionSumMap.cpp @@ -60,10 +60,13 @@ AggregateFunctionPtr createAggregateFunctionSumMap(const std::string & name, con AggregateFunctionPtr createAggregateFunctionSumMapFiltered(const std::string & name, const DataTypes & arguments, const Array & params) { if (params.size() != 1) - throw Exception("Aggregate function " + name + "requires exactly one parameter of Array type.", + throw Exception("Aggregate function " + name + " requires exactly one parameter of Array type.", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); - Array keys_to_keep = params.front().safeGet(); + Array keys_to_keep; + if (!params.front().tryGet(keys_to_keep)) + throw Exception("Aggregate function " + name + " requires an Array as parameter.", + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); auto [keys_type, values_types] = parseArguments(name, arguments); From ba1f1c74608f766d60493b8e928930ad43d0f965 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?L=C3=A9o=20Ercolanelli?= Date: Thu, 24 Jan 2019 13:31:33 +0100 Subject: [PATCH 36/86] sumMapFiltered: use an unordered_set --- .../AggregateFunctionSumMap.h | 10 +++------- dbms/src/Core/Types.h | 17 +++++++++++++++++ 2 files changed, 20 insertions(+), 7 deletions(-) diff --git a/dbms/src/AggregateFunctions/AggregateFunctionSumMap.h b/dbms/src/AggregateFunctions/AggregateFunctionSumMap.h index 17dcf91d5e1..1e5f3e38cd2 100644 --- a/dbms/src/AggregateFunctions/AggregateFunctionSumMap.h +++ b/dbms/src/AggregateFunctions/AggregateFunctionSumMap.h @@ -279,7 +279,7 @@ template class AggregateFunctionSumMapFiltered final : public AggregateFunctionSumMapBase> { private: - std::vector keys_to_keep; + std::unordered_set keys_to_keep; public: AggregateFunctionSumMapFiltered(const DataTypePtr & keys_type, const DataTypes & values_types, const Array & keys_to_keep_) @@ -288,17 +288,13 @@ public: keys_to_keep.reserve(keys_to_keep_.size()); for (const Field & f : keys_to_keep_) { - keys_to_keep.emplace_back(f.safeGet>()); + keys_to_keep.emplace(f.safeGet>()); } - std::sort(begin(keys_to_keep), end(keys_to_keep)); } String getName() const override { return "sumMapFiltered"; } - bool keepKey(const T & key) const - { - return std::binary_search(begin(keys_to_keep), end(keys_to_keep), key); - } + bool keepKey(const T & key) const { return keys_to_keep.count(key); } }; } diff --git a/dbms/src/Core/Types.h b/dbms/src/Core/Types.h index 5e2cd47f440..e4882cd64f7 100644 --- a/dbms/src/Core/Types.h +++ b/dbms/src/Core/Types.h @@ -166,3 +166,20 @@ template <> constexpr bool IsDecimalNumber = true; template <> constexpr bool IsDecimalNumber = true; } + +/// Specialization of `std::hash` for the Decimal types. +namespace std +{ + template + struct hash> { size_t operator()(const DB::Decimal & x) const { return hash()(x.value); } }; + + template <> + struct hash + { + size_t operator()(const DB::Decimal128 & x) const + { + return std::hash()(x.value >> 64) + ^ std::hash()(x.value & std::numeric_limits::max()); + } + }; +} From e64dc4798e23a3d335ad07f69bbe9b2c7ae18e9b Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?L=C3=A9o=20Ercolanelli?= Date: Thu, 24 Jan 2019 14:54:29 +0100 Subject: [PATCH 37/86] sumMapFiltered: fix test output --- dbms/tests/queries/0_stateless/00502_sum_map.reference | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/tests/queries/0_stateless/00502_sum_map.reference b/dbms/tests/queries/0_stateless/00502_sum_map.reference index ac5678ebeab..67639ad230f 100644 --- a/dbms/tests/queries/0_stateless/00502_sum_map.reference +++ b/dbms/tests/queries/0_stateless/00502_sum_map.reference @@ -9,7 +9,7 @@ 2000-01-01 00:00:00 [1,2,3,4,5] [10,10,20,10,10] 2000-01-01 00:01:00 [4,5,6,7,8] [10,10,20,10,10] ([1],[10]) -([1, 4, 8], [10, 20, 10]) +([1,4,8], [10,20,10]) ([1],[1]) ([1],[1]) (['a'],[1]) From cbbf04204b22c93231fe0fbb8c920b2644ee628b Mon Sep 17 00:00:00 2001 From: chertus Date: Thu, 24 Jan 2019 17:22:58 +0300 Subject: [PATCH 38/86] JOIN with USING: do not duplicate columns --- dbms/src/Interpreters/QueryNormalizer.cpp | 88 +++++++++++++++++-- dbms/src/Interpreters/QueryNormalizer.h | 28 +++++- dbms/src/Interpreters/SyntaxAnalyzer.cpp | 41 ++------- .../00051_any_inner_join.reference | 10 +-- .../0_stateless/00052_all_left_join.reference | 30 +++---- .../00053_all_inner_join.reference | 20 ++--- .../0_stateless/00054_join_string.reference | 30 +++---- .../00555_right_join_excessive_rows.sql | 2 +- ...query_aggregation_column_removal.reference | 8 +- .../00597_push_down_predicate.reference | 8 +- .../00679_replace_asterisk.reference | 6 +- .../00702_join_with_using.reference | 42 ++++++++- .../0_stateless/00702_join_with_using.sql | 26 ++++++ .../0_stateless/00725_join_on_bug_2.sql | 2 +- .../00740_database_in_nested_view.sql | 2 +- .../00800_low_cardinality_join.reference | 18 ++-- 16 files changed, 245 insertions(+), 116 deletions(-) diff --git a/dbms/src/Interpreters/QueryNormalizer.cpp b/dbms/src/Interpreters/QueryNormalizer.cpp index b8f11ddbf79..aa00e6e8b5c 100644 --- a/dbms/src/Interpreters/QueryNormalizer.cpp +++ b/dbms/src/Interpreters/QueryNormalizer.cpp @@ -1,13 +1,16 @@ +#include #include #include +#include +#include #include #include #include +#include #include #include #include #include -#include #include #include @@ -16,10 +19,14 @@ namespace DB namespace ErrorCodes { + extern const int LOGICAL_ERROR; extern const int TOO_DEEP_AST; extern const int CYCLIC_ALIASES; } +NameSet removeDuplicateColumns(NamesAndTypesList & columns); + + class CheckASTDepth { public: @@ -135,7 +142,8 @@ void QueryNormalizer::visit(ASTIdentifier & node, ASTPtr & ast, Data & data) /// Replace *, alias.*, database.table.* with a list of columns. void QueryNormalizer::visit(ASTExpressionList & node, const ASTPtr &, Data & data) { - auto & tables_with_columns = data.tables_with_columns; + const auto & tables_with_columns = data.tables_with_columns; + const auto & source_columns_set = data.source_columns_set; ASTs old_children; if (data.processAsterisks()) @@ -162,22 +170,43 @@ void QueryNormalizer::visit(ASTExpressionList & node, const ASTPtr &, Data & dat { if (typeid_cast(child.get())) { - for (const auto & pr : tables_with_columns) - for (const auto & column_name : pr.second) - node.children.emplace_back(std::make_shared(column_name)); + bool first_table = true; + for (const auto & [table_name, table_columns] : tables_with_columns) + { + for (const auto & column_name : table_columns) + if (first_table || !data.join_using_columns.count(column_name)) + { + /// qualifed names for duplicates + if (!first_table && source_columns_set && source_columns_set->count(column_name)) + node.children.emplace_back(std::make_shared(table_name.getQualifiedNamePrefix() + column_name)); + else + node.children.emplace_back(std::make_shared(column_name)); + } + + first_table = false; + } } else if (const auto * qualified_asterisk = typeid_cast(child.get())) { DatabaseAndTableWithAlias ident_db_and_name(qualified_asterisk->children[0]); + bool first_table = true; for (const auto & [table_name, table_columns] : tables_with_columns) { if (ident_db_and_name.satisfies(table_name, true)) { for (const auto & column_name : table_columns) - node.children.emplace_back(std::make_shared(column_name)); + { + /// qualifed names for duplicates + if (!first_table && source_columns_set && source_columns_set->count(column_name)) + node.children.emplace_back(std::make_shared(table_name.getQualifiedNamePrefix() + column_name)); + else + node.children.emplace_back(std::make_shared(column_name)); + } break; } + + first_table = false; } } else @@ -198,6 +227,11 @@ void QueryNormalizer::visit(ASTTablesInSelectQueryElement & node, const ASTPtr & /// special visitChildren() for ASTSelectQuery void QueryNormalizer::visit(ASTSelectQuery & select, const ASTPtr & ast, Data & data) { + extractTablesWithColumns(select, data); + + if (auto join = select.join()) + extractJoinUsingColumns(join->table_join, data); + for (auto & child : ast->children) { if (typeid_cast(child.get()) || @@ -312,4 +346,46 @@ void QueryNormalizer::visit(ASTPtr & ast, Data & data) } } +void QueryNormalizer::extractTablesWithColumns(const ASTSelectQuery & select_query, Data & data) +{ + if (data.context && select_query.tables && !select_query.tables->children.empty()) + { + data.tables_with_columns.clear(); + String current_database = data.context->getCurrentDatabase(); + + for (const ASTTableExpression * table_expression : getSelectTablesExpression(select_query)) + { + DatabaseAndTableWithAlias table_name(*table_expression, current_database); + + NamesAndTypesList names_and_types = getNamesAndTypeListFromTableExpression(*table_expression, *data.context); + removeDuplicateColumns(names_and_types); + + data.tables_with_columns.emplace_back(std::move(table_name), names_and_types.getNames()); + } + } +} + +/// 'select * from a join b using id' should result one 'id' column +void QueryNormalizer::extractJoinUsingColumns(const ASTPtr ast, Data & data) +{ + const auto & table_join = typeid_cast(*ast); + + if (table_join.using_expression_list) + { + auto & keys = typeid_cast(*table_join.using_expression_list); + for (const auto & key : keys.children) + if (auto opt_column = getIdentifierName(key)) + data.join_using_columns.insert(*opt_column); + else if (auto * literal = typeid_cast(key.get())) + data.join_using_columns.insert(key->getColumnName()); + else + { + String alias = key->tryGetAlias(); + if (alias.empty()) + throw Exception("Logical error: expected identifier or alias, got: " + key->getID(), ErrorCodes::LOGICAL_ERROR); + data.join_using_columns.insert(alias); + } + } +} + } diff --git a/dbms/src/Interpreters/QueryNormalizer.h b/dbms/src/Interpreters/QueryNormalizer.h index 3e55e0253e6..4c72328a92d 100644 --- a/dbms/src/Interpreters/QueryNormalizer.h +++ b/dbms/src/Interpreters/QueryNormalizer.h @@ -1,5 +1,7 @@ #pragma once +#include + #include #include #include @@ -17,11 +19,11 @@ inline bool functionIsInOrGlobalInOperator(const String & name) return functionIsInOperator(name) || name == "globalIn" || name == "globalNotIn"; } - class ASTFunction; class ASTIdentifier; class ASTExpressionList; struct ASTTablesInSelectQueryElement; +class Context; class QueryNormalizer @@ -52,7 +54,10 @@ public: const Aliases & aliases; const ExtractedSettings settings; - const std::vector tables_with_columns; + const Context * context; + const NameSet * source_columns_set; + std::vector tables_with_columns; + std::unordered_set join_using_columns; /// tmp data size_t level; @@ -60,10 +65,22 @@ public: SetOfASTs current_asts; /// vertices in the current call stack of this method std::string current_alias; /// the alias referencing to the ancestor of ast (the deepest ancestor with aliases) - Data(const Aliases & aliases_, ExtractedSettings && settings_, std::vector && tables_with_columns_ = {}) + Data(const Aliases & aliases_, ExtractedSettings && settings_, const Context & context_, + const NameSet & source_columns_set, Names && all_columns) : aliases(aliases_) , settings(settings_) - , tables_with_columns(tables_with_columns_) + , context(&context_) + , source_columns_set(&source_columns_set) + , level(0) + { + tables_with_columns.emplace_back(DatabaseAndTableWithAlias{}, std::move(all_columns)); + } + + Data(const Aliases & aliases_, ExtractedSettings && settings_) + : aliases(aliases_) + , settings(settings_) + , context(nullptr) + , source_columns_set(nullptr) , level(0) {} @@ -91,6 +108,9 @@ private: static void visit(ASTSelectQuery &, const ASTPtr &, Data &); static void visitChildren(const ASTPtr &, Data & data); + + static void extractTablesWithColumns(const ASTSelectQuery & select_query, Data & data); + static void extractJoinUsingColumns(const ASTPtr ast, Data & data); }; } diff --git a/dbms/src/Interpreters/SyntaxAnalyzer.cpp b/dbms/src/Interpreters/SyntaxAnalyzer.cpp index e09f1b3a95b..fb6a34d37b9 100644 --- a/dbms/src/Interpreters/SyntaxAnalyzer.cpp +++ b/dbms/src/Interpreters/SyntaxAnalyzer.cpp @@ -100,12 +100,13 @@ void normalizeTree( const Names & source_columns, const NameSet & source_columns_set, const Context & context, - const ASTSelectQuery * select_query, - bool asterisk_left_columns_only) + const ASTSelectQuery * select_query) { + const auto & settings = context.getSettingsRef(); + Names all_columns_name = source_columns; - if (!asterisk_left_columns_only) + if (!settings.asterisk_left_columns_only) { auto columns_from_joined_table = result.analyzed_join.getColumnsFromJoinedTable(source_columns_set, context, select_query); for (auto & column : columns_from_joined_table) @@ -115,37 +116,7 @@ void normalizeTree( if (all_columns_name.empty()) throw Exception("An asterisk cannot be replaced with empty columns.", ErrorCodes::LOGICAL_ERROR); - std::vector table_with_columns; - if (select_query && select_query->tables && !select_query->tables->children.empty()) - { - std::vector tables_expression = getSelectTablesExpression(*select_query); - - bool first = true; - String current_database = context.getCurrentDatabase(); - for (const auto * table_expression : tables_expression) - { - DatabaseAndTableWithAlias table_name(*table_expression, current_database); - NamesAndTypesList names_and_types = getNamesAndTypeListFromTableExpression(*table_expression, context); - - removeDuplicateColumns(names_and_types); - - if (!first) - { - /// For joined tables qualify duplicating names. - for (auto & name_and_type : names_and_types) - if (source_columns_set.count(name_and_type.name)) - name_and_type.name = table_name.getQualifiedNamePrefix() + name_and_type.name; - } - - first = false; - - table_with_columns.emplace_back(std::move(table_name), names_and_types.getNames()); - } - } - else - table_with_columns.emplace_back(DatabaseAndTableWithAlias{}, std::move(all_columns_name)); - - QueryNormalizer::Data normalizer_data(result.aliases, context.getSettingsRef(), std::move(table_with_columns)); + QueryNormalizer::Data normalizer_data(result.aliases, settings, context, source_columns_set, std::move(all_columns_name)); QueryNormalizer(normalizer_data).visit(query); } @@ -754,7 +725,7 @@ SyntaxAnalyzerResultPtr SyntaxAnalyzer::analyze( /// Common subexpression elimination. Rewrite rules. normalizeTree(query, result, (storage ? storage->getColumns().ordinary.getNames() : source_columns_list), source_columns_set, - context, select_query, settings.asterisk_left_columns_only != 0); + context, select_query); /// Remove unneeded columns according to 'required_result_columns'. /// Leave all selected columns in case of DISTINCT; columns that contain arrayJoin function inside. diff --git a/dbms/tests/queries/0_stateless/00051_any_inner_join.reference b/dbms/tests/queries/0_stateless/00051_any_inner_join.reference index a7949c6a350..2a61b78f6ea 100644 --- a/dbms/tests/queries/0_stateless/00051_any_inner_join.reference +++ b/dbms/tests/queries/0_stateless/00051_any_inner_join.reference @@ -1,5 +1,5 @@ -0 0 0 -2 2 1 -4 4 2 -6 6 3 -8 8 4 +0 0 +2 1 +4 2 +6 3 +8 4 diff --git a/dbms/tests/queries/0_stateless/00052_all_left_join.reference b/dbms/tests/queries/0_stateless/00052_all_left_join.reference index 7d48d304488..a33465fcd1d 100644 --- a/dbms/tests/queries/0_stateless/00052_all_left_join.reference +++ b/dbms/tests/queries/0_stateless/00052_all_left_join.reference @@ -1,15 +1,15 @@ -0 0 0 -0 0 1 -1 1 2 -1 1 3 -2 2 4 -2 2 5 -3 3 6 -3 3 7 -4 4 8 -4 4 9 -5 5 0 -6 6 0 -7 7 0 -8 8 0 -9 9 0 +0 0 +0 1 +1 2 +1 3 +2 4 +2 5 +3 6 +3 7 +4 8 +4 9 +5 0 +6 0 +7 0 +8 0 +9 0 diff --git a/dbms/tests/queries/0_stateless/00053_all_inner_join.reference b/dbms/tests/queries/0_stateless/00053_all_inner_join.reference index 24857668974..41707378e8d 100644 --- a/dbms/tests/queries/0_stateless/00053_all_inner_join.reference +++ b/dbms/tests/queries/0_stateless/00053_all_inner_join.reference @@ -1,10 +1,10 @@ -0 0 0 -0 0 1 -1 1 2 -1 1 3 -2 2 4 -2 2 5 -3 3 6 -3 3 7 -4 4 8 -4 4 9 +0 0 +0 1 +1 2 +1 3 +2 4 +2 5 +3 6 +3 7 +4 8 +4 9 diff --git a/dbms/tests/queries/0_stateless/00054_join_string.reference b/dbms/tests/queries/0_stateless/00054_join_string.reference index 4c35b240b32..75a0a5bc5fc 100644 --- a/dbms/tests/queries/0_stateless/00054_join_string.reference +++ b/dbms/tests/queries/0_stateless/00054_join_string.reference @@ -1,15 +1,15 @@ -A A 0 -A A 1 -B B 2 -B B 3 -C C 4 -C C 5 -D D 6 -D D 7 -E E 8 -E E 9 -F F 0 -G G 0 -H H 0 -I I 0 -J J 0 +A 0 +A 1 +B 2 +B 3 +C 4 +C 5 +D 6 +D 7 +E 8 +E 9 +F 0 +G 0 +H 0 +I 0 +J 0 diff --git a/dbms/tests/queries/0_stateless/00555_right_join_excessive_rows.sql b/dbms/tests/queries/0_stateless/00555_right_join_excessive_rows.sql index 668ad5e1bc9..1ec82d5f1b4 100644 --- a/dbms/tests/queries/0_stateless/00555_right_join_excessive_rows.sql +++ b/dbms/tests/queries/0_stateless/00555_right_join_excessive_rows.sql @@ -1,2 +1,2 @@ SET max_block_size = 10; -SELECT * FROM (select toUInt64(1) s limit 1) any right join (select number s from numbers(11)) using (s) ORDER BY s; +SELECT * FROM (select toUInt64(1) s limit 1) any right join (select number s, s as x from numbers(11)) using (s) ORDER BY s; diff --git a/dbms/tests/queries/0_stateless/00585_union_all_subquery_aggregation_column_removal.reference b/dbms/tests/queries/0_stateless/00585_union_all_subquery_aggregation_column_removal.reference index 936106d559e..a0265bdb7ed 100644 --- a/dbms/tests/queries/0_stateless/00585_union_all_subquery_aggregation_column_removal.reference +++ b/dbms/tests/queries/0_stateless/00585_union_all_subquery_aggregation_column_removal.reference @@ -14,10 +14,10 @@ 2 facebook.com 1 google.com 2 yandex.ru -1 baidu.com 1 baidu.com -1 google.com 1 google.com -2 facebook.com 2 facebook.com -2 yandex.ru 2 yandex.ru +1 baidu.com +1 google.com +2 facebook.com +2 yandex.ru 1 1 2 diff --git a/dbms/tests/queries/0_stateless/00597_push_down_predicate.reference b/dbms/tests/queries/0_stateless/00597_push_down_predicate.reference index 3dd49a0bede..df5aebabc89 100644 --- a/dbms/tests/queries/0_stateless/00597_push_down_predicate.reference +++ b/dbms/tests/queries/0_stateless/00597_push_down_predicate.reference @@ -3,7 +3,7 @@ 1 1 -------Need push down------- -0 0 +0 1 1 1 @@ -23,13 +23,13 @@ 1 2000-01-01 1 2000-01-01 1 test string 1 1 2000-01-01 1 test string 1 1 -2000-01-01 1 test string 1 1 2000-01-01 1 test string 1 1 +2000-01-01 1 test string 1 1 2000-01-01 test string 1 1 2000-01-01 1 test string 1 1 -1 2000-01-01 2000-01-01 1 test string 1 1 +1 2000-01-01 1 test string 1 1 2000-01-01 1 test string 1 1 2000-01-01 2 test string 2 2 1 1 -------Push to having expression, need check.------- -------Compatibility test------- -1 2000-01-01 2000-01-01 1 test string 1 1 +1 2000-01-01 test string 1 1 diff --git a/dbms/tests/queries/0_stateless/00679_replace_asterisk.reference b/dbms/tests/queries/0_stateless/00679_replace_asterisk.reference index ab22461f6f2..d05e3183147 100644 --- a/dbms/tests/queries/0_stateless/00679_replace_asterisk.reference +++ b/dbms/tests/queries/0_stateless/00679_replace_asterisk.reference @@ -1,4 +1,4 @@ 1 2 -1 2 3 1 4 5 -1 2 1 3 1 3 -1 2 1 3 1 3 3 +1 2 3 4 5 +1 2 3 1 3 +1 2 3 1 3 3 diff --git a/dbms/tests/queries/0_stateless/00702_join_with_using.reference b/dbms/tests/queries/0_stateless/00702_join_with_using.reference index 53594f8d906..c9035acc656 100644 --- a/dbms/tests/queries/0_stateless/00702_join_with_using.reference +++ b/dbms/tests/queries/0_stateless/00702_join_with_using.reference @@ -1,3 +1,39 @@ -1 1 1 1 -2 2 2 2 -3 3 3 3 +1 1 +2 2 +3 3 +1 John Robert +1 John Susan +3 Daniel Sarah +4 James David +4 James Joseph +5 Amanda Robert +1 John Robert +1 John Susan +3 Daniel Sarah +4 James David +4 James Joseph +5 Amanda Robert +1 John Robert +1 John Susan +3 Daniel Sarah +4 James David +4 James Joseph +5 Amanda Robert +1 John Robert +1 John Susan +3 Daniel Sarah +4 James David +4 James Joseph +5 Amanda Robert +1 John Robert +1 John Susan +3 Daniel Sarah +4 James David +4 James Joseph +5 Amanda Robert +1 John Robert +1 John Susan +3 Daniel Sarah +4 James David +4 James Joseph +5 Amanda Robert diff --git a/dbms/tests/queries/0_stateless/00702_join_with_using.sql b/dbms/tests/queries/0_stateless/00702_join_with_using.sql index 4dcb4461018..6956730eabf 100644 --- a/dbms/tests/queries/0_stateless/00702_join_with_using.sql +++ b/dbms/tests/queries/0_stateless/00702_join_with_using.sql @@ -11,3 +11,29 @@ SELECT * FROM test.using1 ALL LEFT JOIN (SELECT * FROM test.using2) USING (a, a, DROP TABLE test.using1; DROP TABLE test.using2; + +-- + +use test; +drop table if exists persons; +drop table if exists children; + +create table persons (id String, name String) engine MergeTree order by id; +create table children (id String, childName String) engine MergeTree order by id; + +insert into persons (id, name) +values ('1', 'John'), ('2', 'Jack'), ('3', 'Daniel'), ('4', 'James'), ('5', 'Amanda'); + +insert into children (id, childName) +values ('1', 'Robert'), ('1', 'Susan'), ('3', 'Sarah'), ('4', 'David'), ('4', 'Joseph'), ('5', 'Robert'); + +select * from persons all inner join children using id; +select * from persons all inner join (select * from children) as j using id; +select * from (select * from persons) as s all inner join (select * from children ) as j using id; +-- +select * from persons all inner join (select * from children) using id; +select * from (select * from persons) all inner join (select * from children) using id; +select * from (select * from persons) as s all inner join (select * from children) using id; + +drop table persons; +drop table children; diff --git a/dbms/tests/queries/0_stateless/00725_join_on_bug_2.sql b/dbms/tests/queries/0_stateless/00725_join_on_bug_2.sql index 116295d967f..7e95aa4a1d3 100644 --- a/dbms/tests/queries/0_stateless/00725_join_on_bug_2.sql +++ b/dbms/tests/queries/0_stateless/00725_join_on_bug_2.sql @@ -9,7 +9,7 @@ insert into test.s values(1,1); select a, b, s_a, s_b from test.t all left join (select a,b,a s_a, b s_b from test.s) using (a,b); select '-'; -select * from test.t all left join test.s using (a,b); +select t.*, s.* from test.t all left join test.s using (a,b); select '-'; select a,b,s_a,s_b from test.t all left join (select a, b, a s_a, b s_b from test.s) s on (s.a = t.a and s.b = t.b); select '-'; diff --git a/dbms/tests/queries/0_stateless/00740_database_in_nested_view.sql b/dbms/tests/queries/0_stateless/00740_database_in_nested_view.sql index e4dabc3a5a6..afaf23b4950 100644 --- a/dbms/tests/queries/0_stateless/00740_database_in_nested_view.sql +++ b/dbms/tests/queries/0_stateless/00740_database_in_nested_view.sql @@ -7,7 +7,7 @@ USE test; CREATE VIEW test AS SELECT 1 AS N; CREATE VIEW test_view AS SELECT * FROM test; CREATE VIEW test_nested_view AS SELECT * FROM (SELECT * FROM test); -CREATE VIEW test_joined_view AS SELECT * FROM test ANY LEFT JOIN test USING N; +CREATE VIEW test_joined_view AS SELECT *, N AS x FROM test ANY LEFT JOIN test USING N; SELECT * FROM test_view; SELECT * FROM test_nested_view; diff --git a/dbms/tests/queries/0_stateless/00800_low_cardinality_join.reference b/dbms/tests/queries/0_stateless/00800_low_cardinality_join.reference index 31d1de2d8c7..8e032c0a542 100644 --- a/dbms/tests/queries/0_stateless/00800_low_cardinality_join.reference +++ b/dbms/tests/queries/0_stateless/00800_low_cardinality_join.reference @@ -1,12 +1,12 @@ -0 0 -0 0 -0 0 -0 0 -0 0 -0 0 -0 0 -0 0 -0 0 +0 +0 +0 +0 +0 +0 +0 +0 +0 - 0 0 0 0 From ec37ef5a89f549ec55d9edc4c90c6bfb97717fd0 Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Wed, 23 Jan 2019 22:28:13 +0300 Subject: [PATCH 39/86] No escaping for the format schema file's name anymore, added an utility class for parsing and checking the format schema. --- dbms/src/Formats/CapnProtoRowInputStream.cpp | 45 +++++---------- dbms/src/Formats/CapnProtoRowInputStream.h | 3 +- dbms/src/Formats/parseFormatSchema.cpp | 60 ++++++++++++++++++++ dbms/src/Formats/parseFormatSchema.h | 35 ++++++++++++ 4 files changed, 112 insertions(+), 31 deletions(-) create mode 100644 dbms/src/Formats/parseFormatSchema.cpp create mode 100644 dbms/src/Formats/parseFormatSchema.h diff --git a/dbms/src/Formats/CapnProtoRowInputStream.cpp b/dbms/src/Formats/CapnProtoRowInputStream.cpp index 5a2832af9a0..bc3b1bc3926 100644 --- a/dbms/src/Formats/CapnProtoRowInputStream.cpp +++ b/dbms/src/Formats/CapnProtoRowInputStream.cpp @@ -1,12 +1,12 @@ #include #if USE_CAPNP -#include #include #include #include // Y_IGNORE #include #include +#include #include // Y_IGNORE #include // Y_IGNORE #include // Y_IGNORE @@ -26,11 +26,6 @@ namespace ErrorCodes extern const int LOGICAL_ERROR; } -static String getSchemaPath(const String & schema_dir, const String & schema_file) -{ - return schema_dir + escapeForFileName(schema_file) + ".capnp"; -} - CapnProtoRowInputStream::NestedField split(const Block & header, size_t i) { CapnProtoRowInputStream::NestedField field = {{}, i}; @@ -168,7 +163,7 @@ void CapnProtoRowInputStream::createActions(const NestedFieldList & sorted_field auto node = getFieldOrThrow(cur_reader, field.tokens[parents.size()]); if (node.getType().isList() && actions.size() > 0 && actions.back().field == node) { - // The field list here flattens Nested elements into multiple arrays + // The field list here flattens Nested elements into multiple arrays // In order to map Nested types in Cap'nProto back, they need to be collected // Since the field names are sorted, the order of field positions must be preserved // For example, if the fields are { b @0 :Text, a @1 :Text }, the `a` would come first @@ -184,17 +179,17 @@ void CapnProtoRowInputStream::createActions(const NestedFieldList & sorted_field } } -CapnProtoRowInputStream::CapnProtoRowInputStream(ReadBuffer & istr_, const Block & header_, const String & schema_dir, const String & schema_file, const String & root_object) +CapnProtoRowInputStream::CapnProtoRowInputStream(ReadBuffer & istr_, const Block & header_, const FormatSchemaInfo& info) : istr(istr_), header(header_), parser(std::make_shared()) { // Parse the schema and fetch the root object #pragma GCC diagnostic push #pragma GCC diagnostic ignored "-Wdeprecated-declarations" - auto schema = parser->impl.parseDiskFile(schema_file, getSchemaPath(schema_dir, schema_file), {}); + auto schema = parser->impl.parseDiskFile(info.schemaPath(), info.absoluteSchemaPath(), {}); #pragma GCC diagnostic pop - root = schema.getNested(root_object).asStruct(); + root = schema.getNested(info.messageName()).asStruct(); /** * The schema typically consists of fields in various nested structures. @@ -298,26 +293,16 @@ bool CapnProtoRowInputStream::read(MutableColumns & columns, RowReadExtension &) void registerInputFormatCapnProto(FormatFactory & factory) { - factory.registerInputFormat("CapnProto", []( - ReadBuffer & buf, - const Block & sample, - const Context & context, - size_t max_block_size, - const FormatSettings & settings) - { - std::vector tokens; - 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'", - ErrorCodes::BAD_ARGUMENTS); - - const String & schema_dir = context.getFormatSchemaPath(); - - return std::make_shared( - std::make_shared(buf, sample, schema_dir, tokens[0], tokens[1]), - sample, max_block_size, settings); - }); + factory.registerInputFormat( + "CapnProto", + [](ReadBuffer & buf, const Block & sample, const Context & context, size_t max_block_size, const FormatSettings & settings) + { + return std::make_shared( + std::make_shared(buf, sample, FormatSchemaInfo(context, "capnp")), + sample, + max_block_size, + settings); + }); } } diff --git a/dbms/src/Formats/CapnProtoRowInputStream.h b/dbms/src/Formats/CapnProtoRowInputStream.h index a6186d7488d..c40827522aa 100644 --- a/dbms/src/Formats/CapnProtoRowInputStream.h +++ b/dbms/src/Formats/CapnProtoRowInputStream.h @@ -10,6 +10,7 @@ namespace DB { +class FormatSchemaInfo; class ReadBuffer; /** A stream for reading messages in Cap'n Proto format in given schema. @@ -32,7 +33,7 @@ public: * schema_file - location of the capnproto schema, e.g. "schema.capnp" * root_object - name to the root object, e.g. "Message" */ - CapnProtoRowInputStream(ReadBuffer & istr_, const Block & header_, const String & schema_dir, const String & schema_file, const String & root_object); + CapnProtoRowInputStream(ReadBuffer & istr_, const Block & header_, const FormatSchemaInfo & info); bool read(MutableColumns & columns, RowReadExtension &) override; diff --git a/dbms/src/Formats/parseFormatSchema.cpp b/dbms/src/Formats/parseFormatSchema.cpp new file mode 100644 index 00000000000..54c42f8f6ab --- /dev/null +++ b/dbms/src/Formats/parseFormatSchema.cpp @@ -0,0 +1,60 @@ +#include +#include +#include +#include + + +namespace DB +{ +namespace ErrorCodes +{ + extern const int BAD_ARGUMENTS; +} + +FormatSchemaInfo::FormatSchemaInfo(const Context & context, const String & schema_file_extension, bool schema_required) +{ + String format_schema = context.getSettingsRef().format_schema.toString(); + if (format_schema.empty()) + { + if (schema_required) + { + throw Exception( + "Format schema requires the 'format_schema' setting to have the 'schema_file:message_name' format" + + (schema_file_extension.empty() ? "" : "e.g. 'schema." + schema_file_extension + ":Message'"), + ErrorCodes::BAD_ARGUMENTS); + } + return; + } + + size_t colon_pos = format_schema.find(':'); + if ((colon_pos == String::npos) || (colon_pos == 0) || (colon_pos == format_schema.length() - 1)) + { + throw Exception( + "Format schema requires the 'format_schema' setting to have the 'schema_file:message_name' format" + + (schema_file_extension.empty() ? "" : "e.g. 'schema." + schema_file_extension + ":Message'") + ". Got '" + format_schema + + "'", + ErrorCodes::BAD_ARGUMENTS); + } + + Poco::Path path(format_schema.substr(0, colon_pos)); + if (context.hasGlobalContext() && (context.getGlobalContext().getApplicationType() == Context::ApplicationType::SERVER)) + { + if (path.isAbsolute()) + throw Exception("Absolute path in the 'format_schema' setting is prohibited: " + path.toString(), ErrorCodes::BAD_ARGUMENTS); + + if (path.depth() >= 1 && path.directory(0) == "..") + throw Exception( + "Path in the 'format_schema' setting shouldn't go outside the 'format_schema_path' directory: " + path.toString(), + ErrorCodes::BAD_ARGUMENTS); + } + + if (path.getExtension().empty() && !schema_file_extension.empty()) + path.setExtension(schema_file_extension); + + schema_path = path.toString(); + schema_directory = context.getFormatSchemaPath(); + message_name = format_schema.substr(colon_pos + 1); + is_null = false; +} + +} diff --git a/dbms/src/Formats/parseFormatSchema.h b/dbms/src/Formats/parseFormatSchema.h new file mode 100644 index 00000000000..6ad9e6fb2b0 --- /dev/null +++ b/dbms/src/Formats/parseFormatSchema.h @@ -0,0 +1,35 @@ +#pragma once + +#include + +namespace DB +{ +class Context; + +/// Extracts information about where the format schema file is from passed context and keep it. +class FormatSchemaInfo +{ +public: + FormatSchemaInfo() = default; + FormatSchemaInfo(const Context & context, const String & schema_file_extension = String(), bool schema_required = true); + + bool isNull() const { return is_null; } + + /// Returns path to the schema file. + const String & schemaPath() const { return schema_path; } + String absoluteSchemaPath() const { return schema_directory + schema_path; } + + /// Returns directory containing the schema file. + const String & schemaDirectory() const { return schema_directory; } + + /// Returns name of the message type. + const String & messageName() const { return message_name; } + +private: + bool is_null = true; + String schema_path; + String schema_directory; + String message_name; +}; + +} From b7ee466cd77611fd9dba8d2e92afae67a12f5817 Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Wed, 23 Jan 2019 22:31:08 +0300 Subject: [PATCH 40/86] Added possibility to set 'format_schema_path' parameter in the client config too. --- dbms/programs/client/Client.cpp | 3 +++ 1 file changed, 3 insertions(+) diff --git a/dbms/programs/client/Client.cpp b/dbms/programs/client/Client.cpp index 66acc7fcea5..5da18f3c45d 100644 --- a/dbms/programs/client/Client.cpp +++ b/dbms/programs/client/Client.cpp @@ -220,6 +220,9 @@ private: APPLY_FOR_SETTINGS(EXTRACT_SETTING) #undef EXTRACT_SETTING + /// Set path for format schema files + if (config().has("format_schema_path")) + context.setFormatSchemaPath(Poco::Path(config().getString("format_schema_path")).toString() + "/"); } From b1aba679d062e87073f3ad88937a749b41c5139f Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Wed, 23 Jan 2019 22:32:59 +0300 Subject: [PATCH 41/86] Class WriteBufferFromOwnString now can return StringRef too. --- dbms/src/IO/WriteBufferFromString.h | 3 +++ dbms/src/IO/WriteBufferFromVector.h | 8 ++++++++ 2 files changed, 11 insertions(+) diff --git a/dbms/src/IO/WriteBufferFromString.h b/dbms/src/IO/WriteBufferFromString.h index 0611d284102..c833c7e3f52 100644 --- a/dbms/src/IO/WriteBufferFromString.h +++ b/dbms/src/IO/WriteBufferFromString.h @@ -2,6 +2,7 @@ #include #include +#include namespace DB @@ -29,6 +30,8 @@ class WriteBufferFromOwnString : public detail::StringHolder, public WriteBuffer public: WriteBufferFromOwnString() : WriteBufferFromString(value) {} + StringRef stringRef() const { return isFinished() ? StringRef(value) : StringRef(value.data(), pos - value.data()); } + std::string & str() { finish(); diff --git a/dbms/src/IO/WriteBufferFromVector.h b/dbms/src/IO/WriteBufferFromVector.h index 70e6ef5e36e..7cc7eaf25cb 100644 --- a/dbms/src/IO/WriteBufferFromVector.h +++ b/dbms/src/IO/WriteBufferFromVector.h @@ -64,6 +64,14 @@ public: set(nullptr, 0); } + bool isFinished() const { return is_finished; } + + void restart() + { + set(reinterpret_cast(vector.data()), vector.size()); + is_finished = false; + } + ~WriteBufferFromVector() override { if (!is_finished) From 28e983781991559bd28e6c753ef298040e5a05e6 Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Wed, 23 Jan 2019 22:34:38 +0300 Subject: [PATCH 42/86] Implemented utility class for writing protobufs. --- dbms/src/Formats/ProtobufSimpleWriter.cpp | 412 ++++++++++++++++++++++ dbms/src/Formats/ProtobufSimpleWriter.h | 88 +++++ 2 files changed, 500 insertions(+) create mode 100644 dbms/src/Formats/ProtobufSimpleWriter.cpp create mode 100644 dbms/src/Formats/ProtobufSimpleWriter.h diff --git a/dbms/src/Formats/ProtobufSimpleWriter.cpp b/dbms/src/Formats/ProtobufSimpleWriter.cpp new file mode 100644 index 00000000000..dce58f889f1 --- /dev/null +++ b/dbms/src/Formats/ProtobufSimpleWriter.cpp @@ -0,0 +1,412 @@ +#include +#include +#include + + +namespace DB +{ +namespace +{ + void writeBytes(WriteBuffer & buf, const void * data, size_t size) { buf.write(reinterpret_cast(data), size); } + + void writeVariant(WriteBuffer & buf, UInt32 value) + { + while (value >= 0x80) + { + buf.write(static_cast(value | 0x80)); + value >>= 7; + } + buf.write(static_cast(value)); + } + + void writeVariant(WriteBuffer & buf, Int32 value) { writeVariant(buf, static_cast(value)); } + + void writeVariant(WriteBuffer & buf, UInt64 value) + { + while (value >= 0x80) + { + buf.write(static_cast(value | 0x80)); + value >>= 7; + } + buf.write(static_cast(value)); + } + + void writeVariant(WriteBuffer & buf, Int64 value) { writeVariant(buf, static_cast(value)); } + + void writeLittleEndian(WriteBuffer & buf, UInt32 value) + { + value = Poco::ByteOrder::toLittleEndian(value); + writeBytes(buf, &value, sizeof(value)); + } + + void writeLittleEndian(WriteBuffer & buf, Int32 value) { writeLittleEndian(buf, static_cast(value)); } + + void writeLittleEndian(WriteBuffer & buf, float value) + { + union + { + Float32 f; + UInt32 i; + }; + f = value; + writeLittleEndian(buf, i); + } + + void writeLittleEndian(WriteBuffer & buf, UInt64 value) + { + value = Poco::ByteOrder::toLittleEndian(value); + writeBytes(buf, &value, sizeof(value)); + } + + void writeLittleEndian(WriteBuffer & buf, Int64 value) { writeLittleEndian(buf, static_cast(value)); } + + void writeLittleEndian(WriteBuffer & buf, double value) + { + union + { + Float64 f; + UInt64 i; + }; + f = value; + writeLittleEndian(buf, i); + } + + UInt32 zigZag(Int32 value) { return (static_cast(value) << 1) ^ static_cast(value >> 31); } + UInt64 zigZag(Int64 value) { return (static_cast(value) << 1) ^ static_cast(value >> 63); } + +} + + +enum ProtobufSimpleWriter::WireType : UInt32 +{ + VARIANT = 0, + BITS64 = 1, + LENGTH_DELIMITED = 2, + BITS32 = 5 +}; + +ProtobufSimpleWriter::ProtobufSimpleWriter(WriteBuffer & out_) : out(out_) +{ +} + +ProtobufSimpleWriter::~ProtobufSimpleWriter() +{ + finishCurrentMessage(); +} + +void ProtobufSimpleWriter::newMessage() +{ + finishCurrentMessage(); + were_messages = true; +} + +void ProtobufSimpleWriter::finishCurrentMessage() +{ + if (!were_messages) + return; + finishCurrentField(); + current_field_number = 0; + StringRef str = message_buffer.stringRef(); + writeVariant(out, str.size); + writeBytes(out, str.data, str.size); + message_buffer.restart(); +} + +void ProtobufSimpleWriter::setCurrentField(UInt32 field_number) +{ + finishCurrentField(); + assert(current_field_number < field_number); + current_field_number = field_number; + num_normal_values = 0; + num_packed_values = 0; +} + +void ProtobufSimpleWriter::finishCurrentField() +{ + if (num_packed_values) + { + assert(!num_normal_values); + StringRef str = repeated_packing_buffer.stringRef(); + if (str.size) + { + writeKey(message_buffer, LENGTH_DELIMITED); + writeVariant(message_buffer, str.size); + writeBytes(message_buffer, str.data, str.size); + repeated_packing_buffer.restart(); + } + } +} + +void ProtobufSimpleWriter::writeKey(WriteBuffer & buf, WireType wire_type) +{ + writeVariant(buf, (current_field_number << 3) | wire_type); +} + +void ProtobufSimpleWriter::writeInt32(Int32 value) +{ + assert(current_field_number); + writeKey(message_buffer, VARIANT); + writeVariant(message_buffer, value); + ++num_normal_values; +} + +void ProtobufSimpleWriter::writeUInt32(UInt32 value) +{ + assert(current_field_number); + writeKey(message_buffer, VARIANT); + writeVariant(message_buffer, value); + ++num_normal_values; +} + +void ProtobufSimpleWriter::writeSInt32(Int32 value) +{ + assert(current_field_number); + writeKey(message_buffer, VARIANT); + writeVariant(message_buffer, zigZag(value)); + ++num_normal_values; +} + +void ProtobufSimpleWriter::writeInt64(Int64 value) +{ + assert(current_field_number); + writeKey(message_buffer, VARIANT); + writeVariant(message_buffer, value); + ++num_normal_values; +} + +void ProtobufSimpleWriter::writeUInt64(UInt64 value) +{ + assert(current_field_number); + writeKey(message_buffer, VARIANT); + writeVariant(message_buffer, value); + ++num_normal_values; +} + +void ProtobufSimpleWriter::writeSInt64(Int64 value) +{ + assert(current_field_number); + writeKey(message_buffer, VARIANT); + writeVariant(message_buffer, zigZag(value)); + ++num_normal_values; +} + +void ProtobufSimpleWriter::writeFixed32(UInt32 value) +{ + assert(current_field_number); + writeKey(message_buffer, BITS32); + writeLittleEndian(message_buffer, value); + ++num_normal_values; +} + +void ProtobufSimpleWriter::writeSFixed32(Int32 value) +{ + assert(current_field_number); + writeKey(message_buffer, BITS32); + writeLittleEndian(message_buffer, value); + ++num_normal_values; +} + +void ProtobufSimpleWriter::writeFloat(float value) +{ + assert(current_field_number); + writeKey(message_buffer, BITS32); + writeLittleEndian(message_buffer, value); + ++num_normal_values; +} + +void ProtobufSimpleWriter::writeFixed64(UInt64 value) +{ + assert(current_field_number); + writeKey(message_buffer, BITS64); + writeLittleEndian(message_buffer, value); + ++num_normal_values; +} + +void ProtobufSimpleWriter::writeSFixed64(Int64 value) +{ + assert(current_field_number); + writeKey(message_buffer, BITS64); + writeLittleEndian(message_buffer, value); + ++num_normal_values; +} + +void ProtobufSimpleWriter::writeDouble(double value) +{ + assert(current_field_number); + writeKey(message_buffer, BITS64); + writeLittleEndian(message_buffer, value); + ++num_normal_values; +} + +void ProtobufSimpleWriter::writeString(const StringRef & str) +{ + assert(current_field_number); + ++num_normal_values; + writeKey(message_buffer, LENGTH_DELIMITED); + writeVariant(message_buffer, str.size); + writeBytes(message_buffer, str.data, str.size); +} + +void ProtobufSimpleWriter::writeInt32IfNonZero(Int32 value) +{ + if (value) + writeInt32(value); +} + +void ProtobufSimpleWriter::writeUInt32IfNonZero(UInt32 value) +{ + if (value) + writeUInt32(value); +} + +void ProtobufSimpleWriter::writeSInt32IfNonZero(Int32 value) +{ + if (value) + writeSInt32(value); +} + +void ProtobufSimpleWriter::writeInt64IfNonZero(Int64 value) +{ + if (value) + writeInt64(value); +} + +void ProtobufSimpleWriter::writeUInt64IfNonZero(UInt64 value) +{ + if (value) + writeUInt64(value); +} + +void ProtobufSimpleWriter::writeSInt64IfNonZero(Int64 value) +{ + if (value) + writeSInt64(value); +} + +void ProtobufSimpleWriter::writeFixed32IfNonZero(UInt32 value) +{ + if (value) + writeFixed32(value); +} + +void ProtobufSimpleWriter::writeSFixed32IfNonZero(Int32 value) +{ + if (value) + writeSFixed32(value); +} + +void ProtobufSimpleWriter::writeFloatIfNonZero(float value) +{ + if (value != 0) + writeFloat(value); +} + +void ProtobufSimpleWriter::writeFixed64IfNonZero(UInt64 value) +{ + if (value) + writeFixed64(value); +} + +void ProtobufSimpleWriter::writeSFixed64IfNonZero(Int64 value) +{ + if (value) + writeSFixed64(value); +} + +void ProtobufSimpleWriter::writeDoubleIfNonZero(double value) +{ + if (value != 0) + writeDouble(value); +} + +void ProtobufSimpleWriter::writeStringIfNotEmpty(const StringRef & str) +{ + if (str.size) + writeString(str); +} + +void ProtobufSimpleWriter::packRepeatedInt32(Int32 value) +{ + assert(current_field_number); + writeVariant(repeated_packing_buffer, value); + ++num_packed_values; +} + +void ProtobufSimpleWriter::packRepeatedUInt32(UInt32 value) +{ + assert(current_field_number); + writeVariant(repeated_packing_buffer, value); + ++num_packed_values; +} + +void ProtobufSimpleWriter::packRepeatedSInt32(Int32 value) +{ + assert(current_field_number); + writeVariant(repeated_packing_buffer, zigZag(value)); + ++num_packed_values; +} + +void ProtobufSimpleWriter::packRepeatedInt64(Int64 value) +{ + assert(current_field_number); + writeVariant(repeated_packing_buffer, value); + ++num_packed_values; +} + +void ProtobufSimpleWriter::packRepeatedUInt64(UInt64 value) +{ + assert(current_field_number); + writeVariant(repeated_packing_buffer, value); + ++num_packed_values; +} + +void ProtobufSimpleWriter::packRepeatedSInt64(Int64 value) +{ + assert(current_field_number); + writeVariant(repeated_packing_buffer, zigZag(value)); + ++num_packed_values; +} + +void ProtobufSimpleWriter::packRepeatedFixed32(UInt32 value) +{ + assert(current_field_number); + writeLittleEndian(repeated_packing_buffer, value); + ++num_packed_values; +} + +void ProtobufSimpleWriter::packRepeatedSFixed32(Int32 value) +{ + assert(current_field_number); + writeLittleEndian(repeated_packing_buffer, value); + ++num_packed_values; +} + +void ProtobufSimpleWriter::packRepeatedFloat(float value) +{ + assert(current_field_number); + writeLittleEndian(repeated_packing_buffer, value); + ++num_packed_values; +} + +void ProtobufSimpleWriter::packRepeatedFixed64(UInt64 value) +{ + assert(current_field_number); + writeLittleEndian(repeated_packing_buffer, value); + ++num_packed_values; +} + +void ProtobufSimpleWriter::packRepeatedSFixed64(Int64 value) +{ + assert(current_field_number); + writeLittleEndian(repeated_packing_buffer, value); + ++num_packed_values; +} + +void ProtobufSimpleWriter::packRepeatedDouble(double value) +{ + assert(current_field_number); + writeLittleEndian(repeated_packing_buffer, value); + ++num_packed_values; +} + +} diff --git a/dbms/src/Formats/ProtobufSimpleWriter.h b/dbms/src/Formats/ProtobufSimpleWriter.h new file mode 100644 index 00000000000..4880cfb4e21 --- /dev/null +++ b/dbms/src/Formats/ProtobufSimpleWriter.h @@ -0,0 +1,88 @@ +#pragma once + +#include +#include +#include "IO/WriteBufferFromString.h" + + +namespace DB +{ +/** Utility class to serialize protobufs. + * Knows nothing about protobuf schemas, just provides useful functions to serialize data. + * This class is written following the documentation: https://developers.google.com/protocol-buffers/docs/encoding + */ +class ProtobufSimpleWriter : private boost::noncopyable +{ +public: + ProtobufSimpleWriter(WriteBuffer & out_); + ~ProtobufSimpleWriter(); + + /// Should be called when we start writing a new message. + void newMessage(); + + /// Should be called when we start writing a new field. + /// A passed 'field_number' should be positive and greater than any previous 'field_number'. + void setCurrentField(UInt32 field_number); + UInt32 currentFieldNumber() const { return current_field_number; } + + /// Returns number of values added to the current field. + size_t numValues() const { return num_normal_values + num_packed_values; } + + void writeInt32(Int32 value); + void writeUInt32(UInt32 value); + void writeSInt32(Int32 value); + void writeInt64(Int64 value); + void writeUInt64(UInt64 value); + void writeSInt64(Int64 value); + void writeFixed32(UInt32 value); + void writeSFixed32(Int32 value); + void writeFloat(float value); + void writeFixed64(UInt64 value); + void writeSFixed64(Int64 value); + void writeDouble(double value); + void writeString(const StringRef & str); + + void writeInt32IfNonZero(Int32 value); + void writeUInt32IfNonZero(UInt32 value); + void writeSInt32IfNonZero(Int32 value); + void writeInt64IfNonZero(Int64 value); + void writeUInt64IfNonZero(UInt64 value); + void writeSInt64IfNonZero(Int64 value); + void writeFixed32IfNonZero(UInt32 value); + void writeSFixed32IfNonZero(Int32 value); + void writeFloatIfNonZero(float value); + void writeFixed64IfNonZero(UInt64 value); + void writeSFixed64IfNonZero(Int64 value); + void writeDoubleIfNonZero(double value); + void writeStringIfNotEmpty(const StringRef & str); + + void packRepeatedInt32(Int32 value); + void packRepeatedUInt32(UInt32 value); + void packRepeatedSInt32(Int32 value); + void packRepeatedInt64(Int64 value); + void packRepeatedUInt64(UInt64 value); + void packRepeatedSInt64(Int64 value); + void packRepeatedFixed32(UInt32 value); + void packRepeatedSFixed32(Int32 value); + void packRepeatedFloat(float value); + void packRepeatedFixed64(UInt64 value); + void packRepeatedSFixed64(Int64 value); + void packRepeatedDouble(double value); + +private: + void finishCurrentMessage(); + void finishCurrentField(); + + enum WireType : UInt32; + void writeKey(WriteBuffer & buf, WireType wire_type); + + WriteBuffer & out; + bool were_messages = false; + WriteBufferFromOwnString message_buffer; + UInt32 current_field_number = 0; + size_t num_normal_values = 0; + size_t num_packed_values = 0; + WriteBufferFromOwnString repeated_packing_buffer; +}; + +} From 42d9d4e81d482b61e5c9959685ad3bf8c2e1af88 Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Wed, 23 Jan 2019 22:36:57 +0300 Subject: [PATCH 43/86] Implemented storage for parsed protobuf schemas. --- dbms/src/Common/ErrorCodes.cpp | 1 + dbms/src/Formats/ProtobufSchemas.cpp | 70 ++++++++++++++++++++++++++++ dbms/src/Formats/ProtobufSchemas.h | 49 +++++++++++++++++++ 3 files changed, 120 insertions(+) create mode 100644 dbms/src/Formats/ProtobufSchemas.cpp create mode 100644 dbms/src/Formats/ProtobufSchemas.h diff --git a/dbms/src/Common/ErrorCodes.cpp b/dbms/src/Common/ErrorCodes.cpp index 13ea9e4744a..1e26a2dd7ed 100644 --- a/dbms/src/Common/ErrorCodes.cpp +++ b/dbms/src/Common/ErrorCodes.cpp @@ -408,6 +408,7 @@ namespace ErrorCodes extern const int ILLEGAL_SYNTAX_FOR_CODEC_TYPE = 431; extern const int UNKNOWN_CODEC = 432; extern const int ILLEGAL_CODEC_PARAMETER = 433; + extern const int CANNOT_PARSE_PROTOBUF_SCHEMA = 434; extern const int KEEPER_EXCEPTION = 999; extern const int POCO_EXCEPTION = 1000; diff --git a/dbms/src/Formats/ProtobufSchemas.cpp b/dbms/src/Formats/ProtobufSchemas.cpp new file mode 100644 index 00000000000..3b60b862319 --- /dev/null +++ b/dbms/src/Formats/ProtobufSchemas.cpp @@ -0,0 +1,70 @@ +#include +#include +#include +#include +#include + + +namespace DB +{ +namespace ErrorCodes +{ + extern const int BAD_ARGUMENTS; + extern const int CANNOT_PARSE_PROTOBUF_SCHEMA; +} + +ProtobufSchemas::ProtobufSchemas() + : disk_source_tree(new google::protobuf::compiler::DiskSourceTree()) + , importer(new google::protobuf::compiler::Importer(disk_source_tree.get(), this)) +{ +} + +ProtobufSchemas::~ProtobufSchemas() = default; + +const google::protobuf::Descriptor * +ProtobufSchemas::getMessageTypeForFormatSchema(const FormatSchemaInfo& info) +{ + // Search the message type among already imported ones. + const auto * descriptor = importer->pool()->FindMessageTypeByName(info.messageName()); + if (descriptor) + return descriptor; + + // Initialize mapping in protobuf's DiskSourceTree. + if (proto_directory.has_value()) + { + assert(*proto_directory == info.schemaDirectory()); // format_schema_path should not be changed! + } + else + { + proto_directory = info.schemaDirectory(); + disk_source_tree->MapPath("", *proto_directory); + } + + const auto * file_descriptor = importer->Import(info.schemaPath()); + + // If there parsing errors AddError() throws an exception and in this case the following line + // isn't executed. + assert(file_descriptor); + + descriptor = file_descriptor->FindMessageTypeByName(info.messageName()); + if (!descriptor) + throw Exception( + "Not found a message named '" + info.messageName() + "' in the schema file '" + info.schemaPath() + "'", + ErrorCodes::BAD_ARGUMENTS); + return descriptor; +} + +const google::protobuf::Descriptor * ProtobufSchemas::getMessageTypeForColumns(const std::vector & /*columns*/) +{ + throw Exception("Using the 'Protobuf' format without schema is not implemented", ErrorCodes::NOT_IMPLEMENTED); +} + +void ProtobufSchemas::AddError(const String & filename, int line, int column, const String & message) +{ + throw Exception( + "Cannot parse '" + filename + "' file, found an error at line " + std::to_string(line) + ", column " + std::to_string(column) + ", " + + message, + ErrorCodes::CANNOT_PARSE_PROTOBUF_SCHEMA); +} + +} diff --git a/dbms/src/Formats/ProtobufSchemas.h b/dbms/src/Formats/ProtobufSchemas.h new file mode 100644 index 00000000000..b67f03f5270 --- /dev/null +++ b/dbms/src/Formats/ProtobufSchemas.h @@ -0,0 +1,49 @@ +#pragma once + +#include +#include +#include +#include + + +namespace google +{ +namespace protobuf +{ + class Descriptor; +} +} + +namespace DB +{ +class Block; +class FormatSchemaInfo; +struct ColumnWithTypeAndName; + +/** Keeps parsed google protobuf schemas either parsed from files or generated from DB columns. + * This class is used to handle the "Protobuf" input/output formats. + */ +class ProtobufSchemas : public ext::singleton, public google::protobuf::compiler::MultiFileErrorCollector +{ +public: + ProtobufSchemas(); + ~ProtobufSchemas() override; + + /// Parses the format schema, then parses the corresponding proto file, and returns the descriptor of the message type. + /// The function never returns nullptr, it throws an exception if it cannot load or parse the file. + const google::protobuf::Descriptor * getMessageTypeForFormatSchema(const FormatSchemaInfo& info); + + /// Generates a message type with suitable types of fields to store a block with |header|, then returns the descriptor + /// of the generated message type. + const google::protobuf::Descriptor * getMessageTypeForColumns(const std::vector & columns); + +private: + // Overrides google::protobuf::compiler::MultiFileErrorCollector: + void AddError(const String & filename, int line, int column, const String & message) override; + + std::optional proto_directory; + std::unique_ptr disk_source_tree; + std::unique_ptr importer; +}; + +} From 1e7a231c56d7fc1e4a3ea84eacda7755f988c954 Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Wed, 23 Jan 2019 22:39:19 +0300 Subject: [PATCH 44/86] Implemented main class for writing protobufs. --- dbms/src/Common/ErrorCodes.cpp | 3 + dbms/src/Formats/ProtobufWriter.cpp | 809 ++++++++++++++++++++++++++++ dbms/src/Formats/ProtobufWriter.h | 102 ++++ 3 files changed, 914 insertions(+) create mode 100644 dbms/src/Formats/ProtobufWriter.cpp create mode 100644 dbms/src/Formats/ProtobufWriter.h diff --git a/dbms/src/Common/ErrorCodes.cpp b/dbms/src/Common/ErrorCodes.cpp index 1e26a2dd7ed..f06a88c96da 100644 --- a/dbms/src/Common/ErrorCodes.cpp +++ b/dbms/src/Common/ErrorCodes.cpp @@ -409,6 +409,9 @@ namespace ErrorCodes extern const int UNKNOWN_CODEC = 432; extern const int ILLEGAL_CODEC_PARAMETER = 433; extern const int CANNOT_PARSE_PROTOBUF_SCHEMA = 434; + extern const int NO_DATA_FOR_REQUIRED_PROTOBUF_FIELD = 435; + extern const int CANNOT_CONVERT_TO_PROTOBUF_TYPE = 436; + extern const int PROTOBUF_FIELD_NOT_REPEATED = 437; extern const int KEEPER_EXCEPTION = 999; extern const int POCO_EXCEPTION = 1000; diff --git a/dbms/src/Formats/ProtobufWriter.cpp b/dbms/src/Formats/ProtobufWriter.cpp new file mode 100644 index 00000000000..d17754e3478 --- /dev/null +++ b/dbms/src/Formats/ProtobufWriter.cpp @@ -0,0 +1,809 @@ +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + + +namespace DB +{ +namespace ErrorCodes +{ + extern const int NOT_IMPLEMENTED; + extern const int NO_DATA_FOR_REQUIRED_PROTOBUF_FIELD; + extern const int CANNOT_CONVERT_TO_PROTOBUF_TYPE; + extern const int PROTOBUF_FIELD_NOT_REPEATED; +} + + +// Converter ----------------------------------------------------------------------------------------------------------- + +class ProtobufWriter::Converter : private boost::noncopyable +{ +public: + Converter(ProtobufSimpleWriter & simple_writer_, const google::protobuf::FieldDescriptor * field_) + : simple_writer(simple_writer_), field(field_) + { + } + + virtual ~Converter() = default; + + virtual void writeString(const StringRef &) { cannotConvertType("String"); } + + virtual void writeInt8(Int8) { cannotConvertType("Int8"); } + virtual void writeUInt8(UInt8) { cannotConvertType("UInt8"); } + virtual void writeInt16(Int16) { cannotConvertType("Int16"); } + virtual void writeUInt16(UInt16) { cannotConvertType("UInt16"); } + virtual void writeInt32(Int32) { cannotConvertType("Int32"); } + virtual void writeUInt32(UInt32) { cannotConvertType("UInt32"); } + virtual void writeInt64(Int64) { cannotConvertType("Int64"); } + virtual void writeUInt64(UInt64) { cannotConvertType("UInt64"); } + virtual void writeUInt128(const UInt128 &) { cannotConvertType("UInt128"); } + virtual void writeFloat32(Float32) { cannotConvertType("Float32"); } + virtual void writeFloat64(Float64) { cannotConvertType("Float64"); } + + virtual void prepareEnumMappingInt8(const std::vector> &) {} + virtual void prepareEnumMappingInt16(const std::vector> &) {} + virtual void writeEnumInt8(Int8) { cannotConvertType("Enum"); } + virtual void writeEnumInt16(Int16) { cannotConvertType("Enum"); } + + virtual void writeUUID(const UUID &) { cannotConvertType("UUID"); } + virtual void writeDate(DayNum) { cannotConvertType("Date"); } + virtual void writeDateTime(time_t) { cannotConvertType("DateTime"); } + + virtual void writeDecimal32(Decimal32, UInt32) { cannotConvertType("Decimal32"); } + virtual void writeDecimal64(Decimal64, UInt32) { cannotConvertType("Decimal64"); } + virtual void writeDecimal128(const Decimal128 &, UInt32) { cannotConvertType("Decimal128"); } + + virtual void writeAggregateFunction(const AggregateFunctionPtr &, ConstAggregateDataPtr) { cannotConvertType("AggregateFunction"); } + +protected: + void cannotConvertType(const String & type_name) + { + throw Exception( + "Could not convert data type '" + type_name + "' to protobuf type '" + field->type_name() + "' (field: " + field->name() + ")", + ErrorCodes::CANNOT_CONVERT_TO_PROTOBUF_TYPE); + } + + void cannotConvertValue(const String & value) + { + throw Exception( + "Could not convert value '" + value + "' to protobuf type '" + field->type_name() + "' (field: " + field->name() + ")", + ErrorCodes::CANNOT_CONVERT_TO_PROTOBUF_TYPE); + } + + template + To numericCast(From value) + { + if constexpr (std::is_same_v) + return value; + To result; + try + { + result = boost::numeric_cast(value); + } + catch (boost::numeric::bad_numeric_cast &) + { + cannotConvertValue(toString(value)); + } + return result; + } + + template + To parseFromString(const StringRef & str) + { + To result; + try + { + result = ::DB::parse(str.data, str.size); + } + catch (...) + { + cannotConvertValue(str.toString()); + } + return result; + } + + bool packRepeated() const + { + if (!field->is_repeated()) + return false; + if (field->options().has_packed()) + return field->options().packed(); + return field->file()->syntax() == google::protobuf::FileDescriptor::SYNTAX_PROTO3; + } + + bool skipNullValue() const + { + return field->is_optional() && (field->file()->syntax() == google::protobuf::FileDescriptor::SYNTAX_PROTO3); + } + + ProtobufSimpleWriter & simple_writer; + const google::protobuf::FieldDescriptor * field; +}; + + +// ToStringConverter --------------------------------------------------------------------------------------------------- + +class ProtobufWriter::ToStringConverter : public Converter +{ +public: + ToStringConverter(ProtobufSimpleWriter & simple_writer_, const google::protobuf::FieldDescriptor * field_) + : Converter(simple_writer_, field_) + { + initWriteFieldFunction(); + } + + void writeString(const StringRef & str) override { writeField(str); } + + void writeInt8(Int8 value) override { convertToStringAndWriteField(value); } + void writeUInt8(UInt8 value) override { convertToStringAndWriteField(value); } + void writeInt16(Int16 value) override { convertToStringAndWriteField(value); } + void writeUInt16(UInt16 value) override { convertToStringAndWriteField(value); } + void writeInt32(Int32 value) override { convertToStringAndWriteField(value); } + void writeUInt32(UInt32 value) override { convertToStringAndWriteField(value); } + void writeInt64(Int64 value) override { convertToStringAndWriteField(value); } + void writeUInt64(UInt64 value) override { convertToStringAndWriteField(value); } + void writeFloat32(Float32 value) override { convertToStringAndWriteField(value); } + void writeFloat64(Float64 value) override { convertToStringAndWriteField(value); } + + void prepareEnumMappingInt8(const std::vector> & name_value_pairs) override + { + prepareEnumValueToNameMap(name_value_pairs); + } + void prepareEnumMappingInt16(const std::vector> & name_value_pairs) override + { + prepareEnumValueToNameMap(name_value_pairs); + } + + void writeEnumInt8(Int8 value) override { writeEnumInt16(value); } + + void writeEnumInt16(Int16 value) override + { + auto it = enum_value_to_name_map->find(value); + if (it == enum_value_to_name_map->end()) + cannotConvertValue(toString(value)); + writeField(it->second); + } + + void writeUUID(const UUID & uuid) override { convertToStringAndWriteField(uuid); } + void writeDate(DayNum date) override { convertToStringAndWriteField(date); } + + void writeDateTime(time_t tm) override + { + writeDateTimeText(tm, text_buffer); + writeField(text_buffer.stringRef()); + text_buffer.restart(); + } + + void writeDecimal32(Decimal32 decimal, UInt32 scale) override { writeDecimal(decimal, scale); } + void writeDecimal64(Decimal64 decimal, UInt32 scale) override { writeDecimal(decimal, scale); } + void writeDecimal128(const Decimal128 & decimal, UInt32 scale) override { writeDecimal(decimal, scale); } + + void writeAggregateFunction(const AggregateFunctionPtr & function, ConstAggregateDataPtr place) override + { + function->serialize(place, text_buffer); + writeField(text_buffer.stringRef()); + text_buffer.restart(); + } + +private: + template + void convertToStringAndWriteField(T value) + { + writeText(value, text_buffer); + writeField(text_buffer.stringRef()); + text_buffer.restart(); + } + + template + void writeDecimal(const Decimal & decimal, UInt32 scale) + { + writeText(decimal, scale, text_buffer); + writeField(text_buffer.stringRef()); + text_buffer.restart(); + } + + template + void prepareEnumValueToNameMap(const std::vector> & name_value_pairs) + { + if (enum_value_to_name_map.has_value()) + return; + enum_value_to_name_map.emplace(); + for (const auto & name_value_pair : name_value_pairs) + enum_value_to_name_map->emplace(name_value_pair.second, name_value_pair.first); + } + + void writeField(const StringRef & str) { (simple_writer.*write_field_function)(str); } + + void initWriteFieldFunction() + { + write_field_function = skipNullValue() ? &ProtobufSimpleWriter::writeStringIfNotEmpty : &ProtobufSimpleWriter::writeString; + } + + void (ProtobufSimpleWriter::*write_field_function)(const StringRef & str); + WriteBufferFromOwnString text_buffer; + std::optional> enum_value_to_name_map; +}; + + +// ToNumberConverter --------------------------------------------------------------------------------------------------- + +template +class ProtobufWriter::ToNumberConverter : public Converter +{ +public: + ToNumberConverter(ProtobufSimpleWriter & simple_writer_, const google::protobuf::FieldDescriptor * field_) + : Converter(simple_writer_, field_) + { + initWriteFieldFunction(); + } + + void writeString(const StringRef & str) override { writeField(parseFromString(str)); } + + void writeInt8(Int8 value) override { castNumericAndWriteField(value); } + void writeUInt8(UInt8 value) override { castNumericAndWriteField(value); } + void writeInt16(Int16 value) override { castNumericAndWriteField(value); } + void writeUInt16(UInt16 value) override { castNumericAndWriteField(value); } + void writeInt32(Int32 value) override { castNumericAndWriteField(value); } + void writeUInt32(UInt32 value) override { castNumericAndWriteField(value); } + void writeInt64(Int64 value) override { castNumericAndWriteField(value); } + void writeUInt64(UInt64 value) override { castNumericAndWriteField(value); } + void writeFloat32(Float32 value) override { castNumericAndWriteField(value); } + void writeFloat64(Float64 value) override { castNumericAndWriteField(value); } + + void writeEnumInt8(Int8 value) override { writeEnumInt16(value); } + + void writeEnumInt16(Int16 value) override + { + if constexpr (!std::is_integral_v) + cannotConvertType("Enum"); // It's not correct to convert enum to floating point. + castNumericAndWriteField(value); + } + + void writeDate(DayNum date) override { castNumericAndWriteField(static_cast(date)); } + void writeDateTime(time_t tm) override { castNumericAndWriteField(tm); } + + void writeDecimal32(Decimal32 decimal, UInt32 scale) override { writeDecimal(decimal, scale); } + void writeDecimal64(Decimal64 decimal, UInt32 scale) override { writeDecimal(decimal, scale); } + +private: + template + void castNumericAndWriteField(From value) + { + writeField(numericCast(value)); + } + + template + void writeDecimal(const Decimal & decimal, UInt32 scale) + { + if constexpr (std::is_integral_v) + castNumericAndWriteField(decimal.value / decimalScaleMultiplier(scale)); + else + castNumericAndWriteField(double(decimal.value) * pow(10., -double(scale))); + } + + void writeField(T value) { (simple_writer.*write_field_function)(value); } + + void initWriteFieldFunction() + { + if constexpr (std::is_same_v) + { + switch (field->type()) + { + case google::protobuf::FieldDescriptor::TYPE_INT32: + write_field_function = packRepeated() + ? &ProtobufSimpleWriter::packRepeatedInt32 + : (skipNullValue() ? &ProtobufSimpleWriter::writeInt32IfNonZero : &ProtobufSimpleWriter::writeInt32); + break; + case google::protobuf::FieldDescriptor::TYPE_SINT32: + write_field_function = packRepeated() + ? &ProtobufSimpleWriter::packRepeatedSInt32 + : (skipNullValue() ? &ProtobufSimpleWriter::writeSInt32IfNonZero : &ProtobufSimpleWriter::writeSInt32); + break; + case google::protobuf::FieldDescriptor::TYPE_SFIXED32: + write_field_function = packRepeated() + ? &ProtobufSimpleWriter::packRepeatedSFixed32 + : (skipNullValue() ? &ProtobufSimpleWriter::writeSFixed32IfNonZero : &ProtobufSimpleWriter::writeSFixed32); + break; + default: + assert(false); + } + } + else if constexpr (std::is_same_v) + { + switch (field->type()) + { + case google::protobuf::FieldDescriptor::TYPE_UINT32: + write_field_function = packRepeated() + ? &ProtobufSimpleWriter::packRepeatedUInt32 + : (skipNullValue() ? &ProtobufSimpleWriter::writeUInt32IfNonZero : &ProtobufSimpleWriter::writeUInt32); + break; + case google::protobuf::FieldDescriptor::TYPE_FIXED32: + write_field_function = packRepeated() + ? &ProtobufSimpleWriter::packRepeatedFixed32 + : (skipNullValue() ? &ProtobufSimpleWriter::writeFixed32IfNonZero : &ProtobufSimpleWriter::writeFixed32); + break; + default: + assert(false); + } + } + else if constexpr (std::is_same_v) + { + switch (field->type()) + { + case google::protobuf::FieldDescriptor::TYPE_INT64: + write_field_function = packRepeated() + ? &ProtobufSimpleWriter::packRepeatedInt64 + : (skipNullValue() ? &ProtobufSimpleWriter::writeInt64IfNonZero : &ProtobufSimpleWriter::writeInt64); + break; + case google::protobuf::FieldDescriptor::TYPE_SINT64: + write_field_function = packRepeated() + ? &ProtobufSimpleWriter::packRepeatedSInt64 + : (skipNullValue() ? &ProtobufSimpleWriter::writeSInt64IfNonZero : &ProtobufSimpleWriter::writeSInt64); + break; + case google::protobuf::FieldDescriptor::TYPE_SFIXED64: + write_field_function = packRepeated() + ? &ProtobufSimpleWriter::packRepeatedSFixed64 + : (skipNullValue() ? &ProtobufSimpleWriter::writeSFixed64IfNonZero : &ProtobufSimpleWriter::writeSFixed64); + break; + default: + assert(false); + } + } + else if constexpr (std::is_same_v) + { + switch (field->type()) + { + case google::protobuf::FieldDescriptor::TYPE_UINT64: + write_field_function = packRepeated() + ? &ProtobufSimpleWriter::packRepeatedUInt64 + : (skipNullValue() ? &ProtobufSimpleWriter::writeUInt64IfNonZero : &ProtobufSimpleWriter::writeUInt64); + break; + case google::protobuf::FieldDescriptor::TYPE_FIXED64: + write_field_function = packRepeated() + ? &ProtobufSimpleWriter::packRepeatedFixed64 + : (skipNullValue() ? &ProtobufSimpleWriter::writeFixed64IfNonZero : &ProtobufSimpleWriter::writeFixed64); + break; + default: + assert(false); + } + } + else if constexpr (std::is_same_v) + { + write_field_function = packRepeated() + ? &ProtobufSimpleWriter::packRepeatedFloat + : (skipNullValue() ? &ProtobufSimpleWriter::writeFloatIfNonZero : &ProtobufSimpleWriter::writeFloat); + } + else if constexpr (std::is_same_v) + { + write_field_function = packRepeated() + ? &ProtobufSimpleWriter::packRepeatedDouble + : (skipNullValue() ? &ProtobufSimpleWriter::writeDoubleIfNonZero : &ProtobufSimpleWriter::writeDouble); + } + else + { + assert(false); + } + } + + void (ProtobufSimpleWriter::*write_field_function)(T value); +}; + + +// ToBoolConverter ----------------------------------------------------------------------------------------------------- + +class ProtobufWriter::ToBoolConverter : public Converter +{ +public: + ToBoolConverter(ProtobufSimpleWriter & simple_writer_, const google::protobuf::FieldDescriptor * field_) + : Converter(simple_writer_, field_) + { + initWriteFieldFunction(); + } + + void writeString(const StringRef & str) override + { + if (str == "true") + writeField(true); + else if (str == "false") + writeField(false); + else + cannotConvertValue(str.toString()); + } + + void writeInt8(Int8 value) override { convertToBoolAndWriteField(value); } + void writeUInt8(UInt8 value) override { convertToBoolAndWriteField(value); } + void writeInt16(Int16 value) override { convertToBoolAndWriteField(value); } + void writeUInt16(UInt16 value) override { convertToBoolAndWriteField(value); } + void writeInt32(Int32 value) override { convertToBoolAndWriteField(value); } + void writeUInt32(UInt32 value) override { convertToBoolAndWriteField(value); } + void writeInt64(Int64 value) override { convertToBoolAndWriteField(value); } + void writeUInt64(UInt64 value) override { convertToBoolAndWriteField(value); } + void writeFloat32(Float32 value) override { convertToBoolAndWriteField(value); } + void writeFloat64(Float64 value) override { convertToBoolAndWriteField(value); } + void writeDecimal32(Decimal32 decimal, UInt32) override { convertToBoolAndWriteField(decimal.value); } + void writeDecimal64(Decimal64 decimal, UInt32) override { convertToBoolAndWriteField(decimal.value); } + void writeDecimal128(const Decimal128 & decimal, UInt32) override { convertToBoolAndWriteField(decimal.value); } + +private: + template + void convertToBoolAndWriteField(T value) + { + writeField(static_cast(value)); + } + + void writeField(bool b) { (simple_writer.*write_field_function)(b); } + + void initWriteFieldFunction() + { + write_field_function = packRepeated() + ? &ProtobufSimpleWriter::packRepeatedUInt32 + : (skipNullValue() ? &ProtobufSimpleWriter::writeUInt32IfNonZero : &ProtobufSimpleWriter::writeUInt32); + } + + void (ProtobufSimpleWriter::*write_field_function)(UInt32 b); +}; + + +// ToEnumConverter ----------------------------------------------------------------------------------------------------- + +class ProtobufWriter::ToEnumConverter : public Converter +{ +public: + ToEnumConverter(ProtobufSimpleWriter & simple_writer_, const google::protobuf::FieldDescriptor * field_) + : Converter(simple_writer_, field_) + { + initWriteFieldFunction(); + } + + void writeString(const StringRef & str) override + { + prepareEnumNameToPbNumberMap(); + auto it = enum_name_to_pbnumber_map->find(str); + if (it == enum_name_to_pbnumber_map->end()) + cannotConvertValue(str.toString()); + writeField(it->second); + } + + void writeInt8(Int8 value) override { convertToEnumAndWriteField(value); } + void writeUInt8(UInt8 value) override { convertToEnumAndWriteField(value); } + void writeInt16(Int16 value) override { convertToEnumAndWriteField(value); } + void writeUInt16(UInt16 value) override { convertToEnumAndWriteField(value); } + void writeInt32(Int32 value) override { convertToEnumAndWriteField(value); } + void writeUInt32(UInt32 value) override { convertToEnumAndWriteField(value); } + void writeInt64(Int64 value) override { convertToEnumAndWriteField(value); } + void writeUInt64(UInt64 value) override { convertToEnumAndWriteField(value); } + + void prepareEnumMappingInt8(const std::vector> & name_value_pairs) override + { + prepareEnumValueToPbNumberMap(name_value_pairs); + } + void prepareEnumMappingInt16(const std::vector> & name_value_pairs) override + { + prepareEnumValueToPbNumberMap(name_value_pairs); + } + + void writeEnumInt8(Int8 value) override { writeEnumInt16(value); } + + void writeEnumInt16(Int16 value) override + { + auto it = enum_value_to_pbnumber_map->find(value); + if (it == enum_value_to_pbnumber_map->end()) + cannotConvertValue(toString(value)); + writeField(it->second); + } + +private: + template + void convertToEnumAndWriteField(T value) + { + const auto * enum_descriptor = field->enum_type()->FindValueByNumber(numericCast(value)); + if (!enum_descriptor) + cannotConvertValue(toString(value)); + writeField(enum_descriptor->number()); + } + + void prepareEnumNameToPbNumberMap() + { + if (enum_name_to_pbnumber_map.has_value()) + return; + enum_name_to_pbnumber_map.emplace(); + const auto * enum_type = field->enum_type(); + for (int i = 0; i != enum_type->value_count(); ++i) + { + const auto * enum_value = enum_type->value(i); + enum_name_to_pbnumber_map->emplace(enum_value->name(), enum_value->number()); + } + } + + template + void prepareEnumValueToPbNumberMap(const std::vector> & name_value_pairs) + { + if (enum_value_to_pbnumber_map.has_value()) + return; + enum_value_to_pbnumber_map.emplace(); + for (const auto & name_value_pair : name_value_pairs) + { + Int16 value = name_value_pair.second; + const auto * enum_descriptor = field->enum_type()->FindValueByName(name_value_pair.first); + if (enum_descriptor) + enum_value_to_pbnumber_map->emplace(value, enum_descriptor->number()); + } + } + + void writeField(int enum_number) { (simple_writer.*write_field_function)(enum_number); } + + void initWriteFieldFunction() + { + write_field_function = packRepeated() + ? &ProtobufSimpleWriter::packRepeatedUInt32 + : (skipNullValue() ? &ProtobufSimpleWriter::writeUInt32IfNonZero : &ProtobufSimpleWriter::writeUInt32); + } + + void (ProtobufSimpleWriter::*write_field_function)(UInt32 enum_number); + std::optional> enum_name_to_pbnumber_map; + std::optional> enum_value_to_pbnumber_map; +}; + + +// ProtobufWriter ------------------------------------------------------------------------------------------------------ + +ProtobufWriter::ProtobufWriter(WriteBuffer & out, const google::protobuf::Descriptor * message_type) : simple_writer(out) +{ + enumerateFieldsInWriteOrder(message_type); + createConverters(); +} + +ProtobufWriter::~ProtobufWriter() +{ + finishCurrentMessage(); +} + +void ProtobufWriter::enumerateFieldsInWriteOrder(const google::protobuf::Descriptor * message_type) +{ + assert(fields_in_write_order.empty()); + fields_in_write_order.reserve(message_type->field_count()); + for (int i = 0; i < message_type->field_count(); ++i) + fields_in_write_order.emplace_back(message_type->field(i)); + + std::sort( + fields_in_write_order.begin(), + fields_in_write_order.end(), + [](const google::protobuf::FieldDescriptor * left, const google::protobuf::FieldDescriptor * right) + { + return left->number() < right->number(); + }); +} + +void ProtobufWriter::createConverters() +{ + assert(converters.empty()); + converters.reserve(fields_in_write_order.size()); + for (size_t i = 0; i != fields_in_write_order.size(); ++i) + { + const auto * field = fields_in_write_order[i]; + std::unique_ptr converter; + switch (field->type()) + { + case google::protobuf::FieldDescriptor::TYPE_INT32: + case google::protobuf::FieldDescriptor::TYPE_SINT32: + case google::protobuf::FieldDescriptor::TYPE_SFIXED32: + converter = std::make_unique>(simple_writer, field); + break; + case google::protobuf::FieldDescriptor::TYPE_UINT32: + case google::protobuf::FieldDescriptor::TYPE_FIXED32: + converter = std::make_unique>(simple_writer, field); + break; + case google::protobuf::FieldDescriptor::TYPE_INT64: + case google::protobuf::FieldDescriptor::TYPE_SFIXED64: + case google::protobuf::FieldDescriptor::TYPE_SINT64: + converter = std::make_unique>(simple_writer, field); + break; + case google::protobuf::FieldDescriptor::TYPE_UINT64: + case google::protobuf::FieldDescriptor::TYPE_FIXED64: + converter = std::make_unique>(simple_writer, field); + break; + case google::protobuf::FieldDescriptor::TYPE_FLOAT: + converter = std::make_unique>(simple_writer, field); + break; + case google::protobuf::FieldDescriptor::TYPE_DOUBLE: + converter = std::make_unique>(simple_writer, field); + break; + case google::protobuf::FieldDescriptor::TYPE_BOOL: + converter = std::make_unique(simple_writer, field); + break; + case google::protobuf::FieldDescriptor::TYPE_ENUM: + converter = std::make_unique(simple_writer, field); + break; + case google::protobuf::FieldDescriptor::TYPE_STRING: + case google::protobuf::FieldDescriptor::TYPE_BYTES: + converter = std::make_unique(simple_writer, field); + break; + default: + throw Exception(String("Protobuf type '") + field->type_name() + "' isn't supported", ErrorCodes::NOT_IMPLEMENTED); + } + converters.emplace_back(std::move(converter)); + } +} + +const std::vector & ProtobufWriter::fieldsInWriteOrder() const +{ + return fields_in_write_order; +} + +void ProtobufWriter::newMessage() +{ + finishCurrentMessage(); + simple_writer.newMessage(); + if (fields_in_write_order.empty()) + return; + + current_field_index = 0; + current_field = fields_in_write_order[current_field_index]; + current_converter = converters[current_field_index].get(); + simple_writer.setCurrentField(current_field->number()); +} + +void ProtobufWriter::finishCurrentMessage() +{ + if (current_field) + { + assert(current_field_index == fields_in_write_order.size() - 1); + finishCurrentField(); + } +} + +bool ProtobufWriter::nextField() +{ + if (current_field_index == fields_in_write_order.size() - 1) + return false; + + finishCurrentField(); + + ++current_field_index; + current_field = fields_in_write_order[current_field_index]; + current_converter = converters[current_field_index].get(); + simple_writer.setCurrentField(current_field->number()); + return true; +} + +void ProtobufWriter::finishCurrentField() +{ + assert(current_field); + size_t num_values = simple_writer.numValues(); + if (num_values == 0) + { + if (current_field->is_required()) + throw Exception( + "No data for the required field '" + current_field->name() + "'", ErrorCodes::NO_DATA_FOR_REQUIRED_PROTOBUF_FIELD); + } + else if (num_values > 1 && !current_field->is_repeated()) + { + throw Exception( + "Cannot write more than single value to the non-repeated field '" + current_field->name() + "'", + ErrorCodes::PROTOBUF_FIELD_NOT_REPEATED); + } +} + +void ProtobufWriter::writeNumber(Int8 value) +{ + current_converter->writeInt8(value); +} + +void ProtobufWriter::writeNumber(UInt8 value) +{ + current_converter->writeUInt8(value); +} + +void ProtobufWriter::writeNumber(Int16 value) +{ + current_converter->writeInt16(value); +} + +void ProtobufWriter::writeNumber(UInt16 value) +{ + current_converter->writeUInt16(value); +} + +void ProtobufWriter::writeNumber(Int32 value) +{ + current_converter->writeInt32(value); +} + +void ProtobufWriter::writeNumber(UInt32 value) +{ + current_converter->writeUInt32(value); +} + +void ProtobufWriter::writeNumber(Int64 value) +{ + current_converter->writeInt64(value); +} + +void ProtobufWriter::writeNumber(UInt64 value) +{ + current_converter->writeUInt64(value); +} + +void ProtobufWriter::writeNumber(UInt128 value) +{ + current_converter->writeUInt128(value); +} + +void ProtobufWriter::writeNumber(Float32 value) +{ + current_converter->writeFloat32(value); +} + +void ProtobufWriter::writeNumber(Float64 value) +{ + current_converter->writeFloat64(value); +} + +void ProtobufWriter::writeString(const StringRef & str) +{ + current_converter->writeString(str); +} + +void ProtobufWriter::prepareEnumMapping(const std::vector> & enum_values) +{ + current_converter->prepareEnumMappingInt8(enum_values); +} + +void ProtobufWriter::prepareEnumMapping(const std::vector> & enum_values) +{ + current_converter->prepareEnumMappingInt16(enum_values); +} + +void ProtobufWriter::writeEnum(Int8 value) +{ + current_converter->writeEnumInt8(value); +} + +void ProtobufWriter::writeEnum(Int16 value) +{ + current_converter->writeEnumInt16(value); +} + +void ProtobufWriter::writeUUID(const UUID & uuid) +{ + current_converter->writeUUID(uuid); +} + +void ProtobufWriter::writeDate(DayNum date) +{ + current_converter->writeDate(date); +} + +void ProtobufWriter::writeDateTime(time_t tm) +{ + current_converter->writeDateTime(tm); +} + +void ProtobufWriter::writeDecimal(Decimal32 decimal, UInt32 scale) +{ + current_converter->writeDecimal32(decimal, scale); +} + +void ProtobufWriter::writeDecimal(Decimal64 decimal, UInt32 scale) +{ + current_converter->writeDecimal64(decimal, scale); +} + +void ProtobufWriter::writeDecimal(const Decimal128 & decimal, UInt32 scale) +{ + current_converter->writeDecimal128(decimal, scale); +} + +void ProtobufWriter::writeAggregateFunction(const AggregateFunctionPtr & function, ConstAggregateDataPtr place) +{ + current_converter->writeAggregateFunction(function, place); +} + +} diff --git a/dbms/src/Formats/ProtobufWriter.h b/dbms/src/Formats/ProtobufWriter.h new file mode 100644 index 00000000000..a6be952dc2b --- /dev/null +++ b/dbms/src/Formats/ProtobufWriter.h @@ -0,0 +1,102 @@ +#pragma once + +#include +#include +#include +#include +#include +#include + + +namespace google +{ +namespace protobuf +{ + class Descriptor; + class FieldDescriptor; +} +} + + +namespace DB +{ +class IAggregateFunction; +using AggregateFunctionPtr = std::shared_ptr; +using ConstAggregateDataPtr = const char *; + + +/** Serializes a protobuf, tries to cast types if necessarily. + */ +class ProtobufWriter : private boost::noncopyable +{ +public: + ProtobufWriter(WriteBuffer & out, const google::protobuf::Descriptor * message_type); + ~ProtobufWriter(); + + /// Returns fields of the protobuf schema sorted by their numbers. + const std::vector & fieldsInWriteOrder() const; + + /// Should be called when we start writing a new message. + void newMessage(); + + /// Should be called when we start writing a new field. + /// Returns false if there is no more fields in the message type. + bool nextField(); + + /// Returns the current field of the message type. + /// The value returned by this function changes after calling nextField() or newMessage(). + const google::protobuf::FieldDescriptor * currentField() const { return current_field; } + + void writeNumber(Int8 value); + void writeNumber(UInt8 value); + void writeNumber(Int16 value); + void writeNumber(UInt16 value); + void writeNumber(Int32 value); + void writeNumber(UInt32 value); + void writeNumber(Int64 value); + void writeNumber(UInt64 value); + void writeNumber(UInt128 value); + void writeNumber(Float32 value); + void writeNumber(Float64 value); + + void writeString(const StringRef & value); + + void prepareEnumMapping(const std::vector> & name_value_pairs); + void prepareEnumMapping(const std::vector> & name_value_pairs); + void writeEnum(Int8 value); + void writeEnum(Int16 value); + + void writeUUID(const UUID & value); + void writeDate(DayNum date); + void writeDateTime(time_t tm); + + void writeDecimal(Decimal32 decimal, UInt32 scale); + void writeDecimal(Decimal64 decimal, UInt32 scale); + void writeDecimal(const Decimal128 & decimal, UInt32 scale); + + void writeAggregateFunction(const AggregateFunctionPtr & function, ConstAggregateDataPtr place); + +private: + void enumerateFieldsInWriteOrder(const google::protobuf::Descriptor * message_type); + void createConverters(); + + void finishCurrentMessage(); + void finishCurrentField(); + + ProtobufSimpleWriter simple_writer; + std::vector fields_in_write_order; + size_t current_field_index = -1; + const google::protobuf::FieldDescriptor * current_field = nullptr; + + class Converter; + class ToStringConverter; + template + class ToNumberConverter; + class ToBoolConverter; + class ToEnumConverter; + + std::vector> converters; + Converter * current_converter = nullptr; +}; + +} From c241a4ea6d31167dc7b4c7b0197b40d969fd2ea5 Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Wed, 23 Jan 2019 22:41:18 +0300 Subject: [PATCH 45/86] Added function serializeProtobuf() to each data type. --- .../DataTypes/DataTypeAggregateFunction.cpp | 7 +++ .../src/DataTypes/DataTypeAggregateFunction.h | 1 + dbms/src/DataTypes/DataTypeArray.cpp | 12 +++++ dbms/src/DataTypes/DataTypeArray.h | 4 ++ dbms/src/DataTypes/DataTypeDate.cpp | 6 +++ dbms/src/DataTypes/DataTypeDate.h | 1 + dbms/src/DataTypes/DataTypeDateTime.cpp | 6 +++ dbms/src/DataTypes/DataTypeDateTime.h | 1 + dbms/src/DataTypes/DataTypeEnum.cpp | 8 ++++ dbms/src/DataTypes/DataTypeEnum.h | 2 + dbms/src/DataTypes/DataTypeFixedString.cpp | 8 ++++ dbms/src/DataTypes/DataTypeFixedString.h | 2 + dbms/src/DataTypes/DataTypeLowCardinality.cpp | 12 +++-- dbms/src/DataTypes/DataTypeLowCardinality.h | 47 ++++++++++--------- dbms/src/DataTypes/DataTypeNullable.cpp | 7 +++ dbms/src/DataTypes/DataTypeNullable.h | 2 + dbms/src/DataTypes/DataTypeNumberBase.cpp | 11 ++++- dbms/src/DataTypes/DataTypeNumberBase.h | 3 +- dbms/src/DataTypes/DataTypeString.cpp | 8 +++- dbms/src/DataTypes/DataTypeString.h | 2 + dbms/src/DataTypes/DataTypeTuple.cpp | 6 +++ dbms/src/DataTypes/DataTypeTuple.h | 2 + dbms/src/DataTypes/DataTypeUUID.cpp | 6 +++ dbms/src/DataTypes/DataTypeUUID.h | 1 + dbms/src/DataTypes/DataTypesDecimal.cpp | 8 ++++ dbms/src/DataTypes/DataTypesDecimal.h | 2 + dbms/src/DataTypes/IDataType.h | 5 ++ dbms/src/DataTypes/IDataTypeDummy.h | 1 + ...eFormatSchema.cpp => FormatSchemaInfo.cpp} | 0 ...parseFormatSchema.h => FormatSchemaInfo.h} | 0 30 files changed, 152 insertions(+), 29 deletions(-) rename dbms/src/Formats/{parseFormatSchema.cpp => FormatSchemaInfo.cpp} (100%) rename dbms/src/Formats/{parseFormatSchema.h => FormatSchemaInfo.h} (100%) diff --git a/dbms/src/DataTypes/DataTypeAggregateFunction.cpp b/dbms/src/DataTypes/DataTypeAggregateFunction.cpp index 7a1b163f3b6..19800db3b05 100644 --- a/dbms/src/DataTypes/DataTypeAggregateFunction.cpp +++ b/dbms/src/DataTypes/DataTypeAggregateFunction.cpp @@ -9,6 +9,7 @@ #include #include +#include #include #include @@ -248,6 +249,12 @@ void DataTypeAggregateFunction::deserializeTextCSV(IColumn & column, ReadBuffer } +void DataTypeAggregateFunction::serializeProtobuf(const IColumn & column, size_t row_num, ProtobufWriter & protobuf) const +{ + protobuf.writeAggregateFunction(function, static_cast(column).getData()[row_num]); +} + + MutableColumnPtr DataTypeAggregateFunction::createColumn() const { return ColumnAggregateFunction::create(function); diff --git a/dbms/src/DataTypes/DataTypeAggregateFunction.h b/dbms/src/DataTypes/DataTypeAggregateFunction.h index d68f460dea6..a21f5d65125 100644 --- a/dbms/src/DataTypes/DataTypeAggregateFunction.h +++ b/dbms/src/DataTypes/DataTypeAggregateFunction.h @@ -56,6 +56,7 @@ public: void serializeTextXML(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override; void serializeTextCSV(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override; void deserializeTextCSV(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const override; + void serializeProtobuf(const IColumn & column, size_t row_num, ProtobufWriter & protobuf) const override; MutableColumnPtr createColumn() const override; diff --git a/dbms/src/DataTypes/DataTypeArray.cpp b/dbms/src/DataTypes/DataTypeArray.cpp index 0b985039ccf..a1677d9bdbd 100644 --- a/dbms/src/DataTypes/DataTypeArray.cpp +++ b/dbms/src/DataTypes/DataTypeArray.cpp @@ -430,6 +430,18 @@ void DataTypeArray::deserializeTextCSV(IColumn & column, ReadBuffer & istr, cons } +void DataTypeArray::serializeProtobuf(const IColumn & column, size_t row_num, ProtobufWriter & protobuf) const +{ + const ColumnArray & column_array = static_cast(column); + const ColumnArray::Offsets & offsets = column_array.getOffsets(); + size_t offset = offsets[row_num - 1]; + size_t next_offset = offsets[row_num]; + const IColumn & nested_column = column_array.getData(); + for (size_t i = offset; i < next_offset; ++i) + nested->serializeProtobuf(nested_column, i, protobuf); +} + + MutableColumnPtr DataTypeArray::createColumn() const { return ColumnArray::create(nested->createColumn(), ColumnArray::ColumnOffsets::create()); diff --git a/dbms/src/DataTypes/DataTypeArray.h b/dbms/src/DataTypes/DataTypeArray.h index 5a89a5d8e5a..4c04a2624f5 100644 --- a/dbms/src/DataTypes/DataTypeArray.h +++ b/dbms/src/DataTypes/DataTypeArray.h @@ -84,6 +84,10 @@ public: DeserializeBinaryBulkSettings & settings, DeserializeBinaryBulkStatePtr & state) const override; + void serializeProtobuf(const IColumn & column, + size_t row_num, + ProtobufWriter & protobuf) const override; + MutableColumnPtr createColumn() const override; Field getDefault() const override; diff --git a/dbms/src/DataTypes/DataTypeDate.cpp b/dbms/src/DataTypes/DataTypeDate.cpp index 688e3c91db3..d5556cb3378 100644 --- a/dbms/src/DataTypes/DataTypeDate.cpp +++ b/dbms/src/DataTypes/DataTypeDate.cpp @@ -4,6 +4,7 @@ #include #include #include +#include namespace DB @@ -72,6 +73,11 @@ void DataTypeDate::deserializeTextCSV(IColumn & column, ReadBuffer & istr, const static_cast(column).getData().push_back(value.getDayNum()); } +void DataTypeDate::serializeProtobuf(const IColumn & column, size_t row_num, ProtobufWriter & protobuf) const +{ + protobuf.writeDate(DayNum(static_cast(column).getData()[row_num])); +} + bool DataTypeDate::equals(const IDataType & rhs) const { return typeid(rhs) == typeid(*this); diff --git a/dbms/src/DataTypes/DataTypeDate.h b/dbms/src/DataTypes/DataTypeDate.h index 89d773149c5..9bc56cc3762 100644 --- a/dbms/src/DataTypes/DataTypeDate.h +++ b/dbms/src/DataTypes/DataTypeDate.h @@ -21,6 +21,7 @@ public: void deserializeTextJSON(IColumn & column, ReadBuffer & istr, const FormatSettings &) const override; void serializeTextCSV(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override; void deserializeTextCSV(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const override; + void serializeProtobuf(const IColumn & column, size_t row_num, ProtobufWriter & protobuf) const override; bool canBeUsedAsVersion() const override { return true; } bool canBeInsideNullable() const override { return true; } diff --git a/dbms/src/DataTypes/DataTypeDateTime.cpp b/dbms/src/DataTypes/DataTypeDateTime.cpp index 6ea042454e5..e894eb79252 100644 --- a/dbms/src/DataTypes/DataTypeDateTime.cpp +++ b/dbms/src/DataTypes/DataTypeDateTime.cpp @@ -6,6 +6,7 @@ #include #include #include +#include #include #include @@ -138,6 +139,11 @@ void DataTypeDateTime::deserializeTextCSV(IColumn & column, ReadBuffer & istr, c static_cast(column).getData().push_back(x); } +void DataTypeDateTime::serializeProtobuf(const IColumn & column, size_t row_num, ProtobufWriter & protobuf) const +{ + protobuf.writeDateTime(static_cast(column).getData()[row_num]); +} + bool DataTypeDateTime::equals(const IDataType & rhs) const { /// DateTime with different timezones are equal, because: diff --git a/dbms/src/DataTypes/DataTypeDateTime.h b/dbms/src/DataTypes/DataTypeDateTime.h index 95b120c6c9d..10fe80a8213 100644 --- a/dbms/src/DataTypes/DataTypeDateTime.h +++ b/dbms/src/DataTypes/DataTypeDateTime.h @@ -46,6 +46,7 @@ public: void deserializeTextJSON(IColumn & column, ReadBuffer & istr, const FormatSettings &) const override; void serializeTextCSV(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override; void deserializeTextCSV(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const override; + void serializeProtobuf(const IColumn & column, size_t row_num, ProtobufWriter & protobuf) const override; bool canBeUsedAsVersion() const override { return true; } bool canBeInsideNullable() const override { return true; } diff --git a/dbms/src/DataTypes/DataTypeEnum.cpp b/dbms/src/DataTypes/DataTypeEnum.cpp index 3cd6af46241..f6566ad9040 100644 --- a/dbms/src/DataTypes/DataTypeEnum.cpp +++ b/dbms/src/DataTypes/DataTypeEnum.cpp @@ -1,5 +1,6 @@ #include #include +#include #include #include #include @@ -222,6 +223,13 @@ void DataTypeEnum::deserializeBinaryBulk( x.resize(initial_size + size / sizeof(FieldType)); } +template +void DataTypeEnum::serializeProtobuf(const IColumn & column, size_t row_num, ProtobufWriter & protobuf) const +{ + protobuf.prepareEnumMapping(values); + protobuf.writeEnum(static_cast(column).getData()[row_num]); +} + template Field DataTypeEnum::getDefault() const { diff --git a/dbms/src/DataTypes/DataTypeEnum.h b/dbms/src/DataTypes/DataTypeEnum.h index 43e4c578f76..f25d387c818 100644 --- a/dbms/src/DataTypes/DataTypeEnum.h +++ b/dbms/src/DataTypes/DataTypeEnum.h @@ -105,6 +105,8 @@ public: void serializeBinaryBulk(const IColumn & column, WriteBuffer & ostr, const size_t offset, size_t limit) const override; void deserializeBinaryBulk(IColumn & column, ReadBuffer & istr, const size_t limit, const double avg_value_size_hint) const override; + void serializeProtobuf(const IColumn & column, size_t row_num, ProtobufWriter & protobuf) const override; + MutableColumnPtr createColumn() const override { return ColumnType::create(); } Field getDefault() const override; diff --git a/dbms/src/DataTypes/DataTypeFixedString.cpp b/dbms/src/DataTypes/DataTypeFixedString.cpp index f43de616e8d..6fbea93efbd 100644 --- a/dbms/src/DataTypes/DataTypeFixedString.cpp +++ b/dbms/src/DataTypes/DataTypeFixedString.cpp @@ -6,6 +6,7 @@ #include #include +#include #include #include @@ -202,6 +203,13 @@ void DataTypeFixedString::deserializeTextCSV(IColumn & column, ReadBuffer & istr } +void DataTypeFixedString::serializeProtobuf(const IColumn & column, size_t row_num, ProtobufWriter & protobuf) const +{ + const char * pos = reinterpret_cast(&static_cast(column).getChars()[n * row_num]); + protobuf.writeString(StringRef(pos, n)); +} + + MutableColumnPtr DataTypeFixedString::createColumn() const { return ColumnFixedString::create(n); diff --git a/dbms/src/DataTypes/DataTypeFixedString.h b/dbms/src/DataTypes/DataTypeFixedString.h index abea529ad42..c69d4859584 100644 --- a/dbms/src/DataTypes/DataTypeFixedString.h +++ b/dbms/src/DataTypes/DataTypeFixedString.h @@ -64,6 +64,8 @@ public: void serializeTextCSV(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override; void deserializeTextCSV(IColumn & column, ReadBuffer & istr, const FormatSettings &) const override; + void serializeProtobuf(const IColumn & column, size_t row_num, ProtobufWriter & protobuf) const override; + MutableColumnPtr createColumn() const override; Field getDefault() const override diff --git a/dbms/src/DataTypes/DataTypeLowCardinality.cpp b/dbms/src/DataTypes/DataTypeLowCardinality.cpp index e73deaae2ca..8d6c457646d 100644 --- a/dbms/src/DataTypes/DataTypeLowCardinality.cpp +++ b/dbms/src/DataTypes/DataTypeLowCardinality.cpp @@ -729,10 +729,11 @@ void DataTypeLowCardinality::deserializeBinary(Field & field, ReadBuffer & istr) dictionary_type->deserializeBinary(field, istr); } -template +template void DataTypeLowCardinality::serializeImpl( - const IColumn & column, size_t row_num, WriteBuffer & ostr, - DataTypeLowCardinality::SerealizeFunctionPtr func, Args & ... args) const + const IColumn & column, size_t row_num, + DataTypeLowCardinality::SerializeFunctionPtr func, + OutputStream & ostr, Args & ... args) const { auto & low_cardinality_column = getColumnLowCardinality(column); size_t unique_row_number = low_cardinality_column.getIndexes().getUInt(row_num); @@ -741,8 +742,9 @@ void DataTypeLowCardinality::serializeImpl( template void DataTypeLowCardinality::deserializeImpl( - IColumn & column, ReadBuffer & istr, - DataTypeLowCardinality::DeserealizeFunctionPtr func, Args & ... args) const + IColumn & column, + DataTypeLowCardinality::DeserializeFunctionPtr func, + ReadBuffer & istr, Args & ... args) const { auto & low_cardinality_column= getColumnLowCardinality(column); auto temp_column = low_cardinality_column.getDictionary().getNestedColumn()->cloneEmpty(); diff --git a/dbms/src/DataTypes/DataTypeLowCardinality.h b/dbms/src/DataTypes/DataTypeLowCardinality.h index 5744419bf01..a099ce72fa8 100644 --- a/dbms/src/DataTypes/DataTypeLowCardinality.h +++ b/dbms/src/DataTypes/DataTypeLowCardinality.h @@ -54,60 +54,65 @@ public: void serializeBinary(const IColumn & column, size_t row_num, WriteBuffer & ostr) const override { - serializeImpl(column, row_num, ostr, &IDataType::serializeBinary); + serializeImpl(column, row_num, &IDataType::serializeBinary, ostr); } void deserializeBinary(IColumn & column, ReadBuffer & istr) const override { - deserializeImpl(column, istr, &IDataType::deserializeBinary); + deserializeImpl(column, &IDataType::deserializeBinary, istr); } void serializeTextEscaped(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const override { - serializeImpl(column, row_num, ostr, &IDataType::serializeTextEscaped, settings); + serializeImpl(column, row_num, &IDataType::serializeTextEscaped, ostr, settings); } void deserializeTextEscaped(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const override { - deserializeImpl(column, istr, &IDataType::deserializeTextEscaped, settings); + deserializeImpl(column, &IDataType::deserializeTextEscaped, istr, settings); } void serializeTextQuoted(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const override { - serializeImpl(column, row_num, ostr, &IDataType::serializeTextQuoted, settings); + serializeImpl(column, row_num, &IDataType::serializeTextQuoted, ostr, settings); } void deserializeTextQuoted(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const override { - deserializeImpl(column, istr, &IDataType::deserializeTextQuoted, settings); + deserializeImpl(column, &IDataType::deserializeTextQuoted, istr, settings); } void serializeTextCSV(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const override { - serializeImpl(column, row_num, ostr, &IDataType::serializeTextCSV, settings); + serializeImpl(column, row_num, &IDataType::serializeTextCSV, ostr, settings); } void deserializeTextCSV(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const override { - deserializeImpl(column, istr, &IDataType::deserializeTextCSV, settings); + deserializeImpl(column, &IDataType::deserializeTextCSV, istr, settings); } void serializeText(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const override { - serializeImpl(column, row_num, ostr, &IDataType::serializeText, settings); + serializeImpl(column, row_num, &IDataType::serializeText, ostr, settings); } void serializeTextJSON(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const override { - serializeImpl(column, row_num, ostr, &IDataType::serializeTextJSON, settings); + serializeImpl(column, row_num, &IDataType::serializeTextJSON, ostr, settings); } void deserializeTextJSON(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const override { - deserializeImpl(column, istr, &IDataType::deserializeTextJSON, settings); + deserializeImpl(column, &IDataType::deserializeTextJSON, istr, settings); } void serializeTextXML(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const override { - serializeImpl(column, row_num, ostr, &IDataType::serializeTextXML, settings); + serializeImpl(column, row_num, &IDataType::serializeTextXML, ostr, settings); + } + + void serializeProtobuf(const IColumn & column, size_t row_num, ProtobufWriter & protobuf) const override + { + serializeImpl(column, row_num, &IDataType::serializeProtobuf, protobuf); } MutableColumnPtr createColumn() const override; @@ -143,19 +148,19 @@ public: private: - template - using SerealizeFunctionPtr = void (IDataType::*)(const IColumn &, size_t, WriteBuffer &, Args & ...) const; + template + using SerializeFunctionPtr = void (IDataType::*)(const IColumn &, size_t, OutputStream &, Args & ...) const; + + template + void serializeImpl(const IColumn & column, size_t row_num, SerializeFunctionPtr func, + OutputStream & ostr, Args & ... args) const; template - void serializeImpl(const IColumn & column, size_t row_num, WriteBuffer & ostr, - SerealizeFunctionPtr func, Args & ... args) const; + using DeserializeFunctionPtr = void (IDataType::*)(IColumn &, ReadBuffer &, Args & ...) const; template - using DeserealizeFunctionPtr = void (IDataType::*)(IColumn &, ReadBuffer &, Args & ...) const; - - template - void deserializeImpl(IColumn & column, ReadBuffer & istr, - DeserealizeFunctionPtr func, Args & ... args) const; + void deserializeImpl(IColumn & column, DeserializeFunctionPtr func, + ReadBuffer & istr, Args & ... args) const; template static MutableColumnUniquePtr createColumnUniqueImpl(const IDataType & keys_type, const Creator & creator); diff --git a/dbms/src/DataTypes/DataTypeNullable.cpp b/dbms/src/DataTypes/DataTypeNullable.cpp index 6322f5b2b17..f8c19b73bbe 100644 --- a/dbms/src/DataTypes/DataTypeNullable.cpp +++ b/dbms/src/DataTypes/DataTypeNullable.cpp @@ -311,6 +311,13 @@ void DataTypeNullable::serializeTextXML(const IColumn & column, size_t row_num, nested_data_type->serializeTextXML(col.getNestedColumn(), row_num, ostr, settings); } +void DataTypeNullable::serializeProtobuf(const IColumn & column, size_t row_num, ProtobufWriter & protobuf) const +{ + const ColumnNullable & col = static_cast(column); + if (!col.isNullAt(row_num)) + nested_data_type->serializeProtobuf(col.getNestedColumn(), row_num, protobuf); +} + MutableColumnPtr DataTypeNullable::createColumn() const { return ColumnNullable::create(nested_data_type->createColumn(), ColumnUInt8::create()); diff --git a/dbms/src/DataTypes/DataTypeNullable.h b/dbms/src/DataTypes/DataTypeNullable.h index c6c228d441d..c45736d06a4 100644 --- a/dbms/src/DataTypes/DataTypeNullable.h +++ b/dbms/src/DataTypes/DataTypeNullable.h @@ -70,6 +70,8 @@ public: void serializeText(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override; void serializeTextXML(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override; + void serializeProtobuf(const IColumn & column, size_t row_num, ProtobufWriter & protobuf) const override; + MutableColumnPtr createColumn() const override; Field getDefault() const override { return Null(); } diff --git a/dbms/src/DataTypes/DataTypeNumberBase.cpp b/dbms/src/DataTypes/DataTypeNumberBase.cpp index 12904b34c17..94eb5618f3f 100644 --- a/dbms/src/DataTypes/DataTypeNumberBase.cpp +++ b/dbms/src/DataTypes/DataTypeNumberBase.cpp @@ -1,4 +1,4 @@ -#include +#include #include #include #include @@ -7,6 +7,7 @@ #include #include #include +#include namespace DB @@ -201,6 +202,14 @@ void DataTypeNumberBase::deserializeBinaryBulk(IColumn & column, ReadBuffer & x.resize(initial_size + size / sizeof(typename ColumnVector::value_type)); } + +template +void DataTypeNumberBase::serializeProtobuf(const IColumn & column, size_t row_num, ProtobufWriter & protobuf) const +{ + protobuf.writeNumber(static_cast &>(column).getData()[row_num]); +} + + template MutableColumnPtr DataTypeNumberBase::createColumn() const { diff --git a/dbms/src/DataTypes/DataTypeNumberBase.h b/dbms/src/DataTypes/DataTypeNumberBase.h index 67da1c7602e..2728d32a6a9 100644 --- a/dbms/src/DataTypes/DataTypeNumberBase.h +++ b/dbms/src/DataTypes/DataTypeNumberBase.h @@ -29,7 +29,6 @@ public: Field getDefault() const override; /** Format is platform-dependent. */ - void serializeBinary(const Field & field, WriteBuffer & ostr) const override; void deserializeBinary(Field & field, ReadBuffer & istr) const override; void serializeBinary(const IColumn & column, size_t row_num, WriteBuffer & ostr) const override; @@ -37,6 +36,8 @@ public: void serializeBinaryBulk(const IColumn & column, WriteBuffer & ostr, size_t offset, size_t limit) const override; void deserializeBinaryBulk(IColumn & column, ReadBuffer & istr, size_t limit, double avg_value_size_hint) const override; + void serializeProtobuf(const IColumn & column, size_t row_num, ProtobufWriter & protobuf) const override; + MutableColumnPtr createColumn() const override; bool isParametric() const override { return false; } diff --git a/dbms/src/DataTypes/DataTypeString.cpp b/dbms/src/DataTypes/DataTypeString.cpp index 55a4ac920b6..ed04b7600be 100644 --- a/dbms/src/DataTypes/DataTypeString.cpp +++ b/dbms/src/DataTypes/DataTypeString.cpp @@ -7,6 +7,7 @@ #include #include +#include #include #include @@ -22,7 +23,6 @@ namespace DB { - void DataTypeString::serializeBinary(const Field & field, WriteBuffer & ostr) const { const String & s = get(field); @@ -304,6 +304,12 @@ void DataTypeString::deserializeTextCSV(IColumn & column, ReadBuffer & istr, con } +void DataTypeString::serializeProtobuf(const IColumn & column, size_t row_num, ProtobufWriter & protobuf) const +{ + protobuf.writeString(static_cast(column).getDataAt(row_num)); +} + + MutableColumnPtr DataTypeString::createColumn() const { return ColumnString::create(); diff --git a/dbms/src/DataTypes/DataTypeString.h b/dbms/src/DataTypes/DataTypeString.h index d0a210dcbf7..202c8374c27 100644 --- a/dbms/src/DataTypes/DataTypeString.h +++ b/dbms/src/DataTypes/DataTypeString.h @@ -45,6 +45,8 @@ public: void serializeTextCSV(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override; void deserializeTextCSV(IColumn & column, ReadBuffer & istr, const FormatSettings &) const override; + void serializeProtobuf(const IColumn & column, size_t row_num, ProtobufWriter & protobuf) const override; + MutableColumnPtr createColumn() const override; Field getDefault() const override diff --git a/dbms/src/DataTypes/DataTypeTuple.cpp b/dbms/src/DataTypes/DataTypeTuple.cpp index 1f0b0f6ca3e..18ae891d576 100644 --- a/dbms/src/DataTypes/DataTypeTuple.cpp +++ b/dbms/src/DataTypes/DataTypeTuple.cpp @@ -407,6 +407,12 @@ void DataTypeTuple::deserializeBinaryBulkWithMultipleStreams( settings.path.pop_back(); } +void DataTypeTuple::serializeProtobuf(const IColumn & column, size_t row_num, ProtobufWriter & protobuf) const +{ + for (const auto i : ext::range(0, ext::size(elems))) + elems[i]->serializeProtobuf(extractElementColumn(column, i), row_num, protobuf); +} + MutableColumnPtr DataTypeTuple::createColumn() const { size_t size = elems.size(); diff --git a/dbms/src/DataTypes/DataTypeTuple.h b/dbms/src/DataTypes/DataTypeTuple.h index 146efee936c..c3422be70db 100644 --- a/dbms/src/DataTypes/DataTypeTuple.h +++ b/dbms/src/DataTypes/DataTypeTuple.h @@ -77,6 +77,8 @@ public: DeserializeBinaryBulkSettings & settings, DeserializeBinaryBulkStatePtr & state) const override; + void serializeProtobuf(const IColumn & column, size_t row_num, ProtobufWriter & protobuf) const override; + MutableColumnPtr createColumn() const override; Field getDefault() const override; diff --git a/dbms/src/DataTypes/DataTypeUUID.cpp b/dbms/src/DataTypes/DataTypeUUID.cpp index 8e022aaa6a0..9f913b5bf80 100644 --- a/dbms/src/DataTypes/DataTypeUUID.cpp +++ b/dbms/src/DataTypes/DataTypeUUID.cpp @@ -1,6 +1,7 @@ #include #include #include +#include #include #include @@ -71,6 +72,11 @@ void DataTypeUUID::deserializeTextCSV(IColumn & column, ReadBuffer & istr, const static_cast(column).getData().push_back(value); } +void DataTypeUUID::serializeProtobuf(const IColumn & column, size_t row_num, ProtobufWriter & protobuf) const +{ + protobuf.writeUUID(UUID(static_cast(column).getData()[row_num])); +} + bool DataTypeUUID::equals(const IDataType & rhs) const { diff --git a/dbms/src/DataTypes/DataTypeUUID.h b/dbms/src/DataTypes/DataTypeUUID.h index 3d43b947753..0a0ce6ad035 100644 --- a/dbms/src/DataTypes/DataTypeUUID.h +++ b/dbms/src/DataTypes/DataTypeUUID.h @@ -24,6 +24,7 @@ public: void deserializeTextJSON(IColumn & column, ReadBuffer & istr, const FormatSettings &) const override; void serializeTextCSV(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override; void deserializeTextCSV(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const override; + void serializeProtobuf(const IColumn & column, size_t row_num, ProtobufWriter & protobuf) const override; bool canBeUsedInBitOperations() const override { return true; } bool canBeInsideNullable() const override { return true; } diff --git a/dbms/src/DataTypes/DataTypesDecimal.cpp b/dbms/src/DataTypes/DataTypesDecimal.cpp index 0c5bd6c6559..439a98928ea 100644 --- a/dbms/src/DataTypes/DataTypesDecimal.cpp +++ b/dbms/src/DataTypes/DataTypesDecimal.cpp @@ -2,6 +2,7 @@ #include #include #include +#include #include #include #include @@ -133,6 +134,13 @@ void DataTypeDecimal::deserializeBinaryBulk(IColumn & column, ReadBuffer & is } +template +void DataTypeDecimal::serializeProtobuf(const IColumn & column, size_t row_num, ProtobufWriter & protobuf) const +{ + protobuf.writeDecimal(static_cast(column).getData()[row_num], scale); +} + + template Field DataTypeDecimal::getDefault() const { diff --git a/dbms/src/DataTypes/DataTypesDecimal.h b/dbms/src/DataTypes/DataTypesDecimal.h index f192dfe1a75..3f93f5aaae1 100644 --- a/dbms/src/DataTypes/DataTypesDecimal.h +++ b/dbms/src/DataTypes/DataTypesDecimal.h @@ -100,6 +100,8 @@ public: void deserializeBinary(IColumn & column, ReadBuffer & istr) const override; void deserializeBinaryBulk(IColumn & column, ReadBuffer & istr, size_t limit, double avg_value_size_hint) const override; + void serializeProtobuf(const IColumn & column, size_t row_num, ProtobufWriter & protobuf) const override; + Field getDefault() const override; MutableColumnPtr createColumn() const override; bool equals(const IDataType & rhs) const override; diff --git a/dbms/src/DataTypes/IDataType.h b/dbms/src/DataTypes/IDataType.h index 8f38e334024..fdac4e454bc 100644 --- a/dbms/src/DataTypes/IDataType.h +++ b/dbms/src/DataTypes/IDataType.h @@ -22,6 +22,8 @@ using MutableColumnPtr = COWPtr::MutablePtr; using DataTypePtr = std::shared_ptr; using DataTypes = std::vector; +class ProtobufWriter; + /** Properties of data type. * Contains methods for serialization/deserialization. @@ -254,6 +256,9 @@ public: serializeText(column, row_num, ostr, settings); } + /** Serialize to a protobuf. */ + virtual void serializeProtobuf(const IColumn & column, size_t row_num, ProtobufWriter & protobuf) const = 0; + /** Create empty column for corresponding type. */ virtual MutableColumnPtr createColumn() const = 0; diff --git a/dbms/src/DataTypes/IDataTypeDummy.h b/dbms/src/DataTypes/IDataTypeDummy.h index 8f204b0e67c..bb122126577 100644 --- a/dbms/src/DataTypes/IDataTypeDummy.h +++ b/dbms/src/DataTypes/IDataTypeDummy.h @@ -28,6 +28,7 @@ public: void deserializeBinaryBulk(IColumn &, ReadBuffer &, size_t, double) const override { throwNoSerialization(); } void serializeText(const IColumn &, size_t, WriteBuffer &, const FormatSettings &) const override { throwNoSerialization(); } void deserializeText(IColumn &, ReadBuffer &, const FormatSettings &) const override { throwNoSerialization(); } + void serializeProtobuf(const IColumn &, size_t, ProtobufWriter &) const override { throwNoSerialization(); } MutableColumnPtr createColumn() const override { diff --git a/dbms/src/Formats/parseFormatSchema.cpp b/dbms/src/Formats/FormatSchemaInfo.cpp similarity index 100% rename from dbms/src/Formats/parseFormatSchema.cpp rename to dbms/src/Formats/FormatSchemaInfo.cpp diff --git a/dbms/src/Formats/parseFormatSchema.h b/dbms/src/Formats/FormatSchemaInfo.h similarity index 100% rename from dbms/src/Formats/parseFormatSchema.h rename to dbms/src/Formats/FormatSchemaInfo.h From 2886f42c0ed8df90832410c0022f5cd800f2d6a8 Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Wed, 23 Jan 2019 22:42:20 +0300 Subject: [PATCH 46/86] Implemented a new block output stream using protobuf output format. --- dbms/src/Formats/FormatFactory.cpp | 2 + .../src/Formats/ProtobufBlockOutputStream.cpp | 75 +++++++++++++++++++ dbms/src/Formats/ProtobufBlockOutputStream.h | 47 ++++++++++++ 3 files changed, 124 insertions(+) create mode 100644 dbms/src/Formats/ProtobufBlockOutputStream.cpp create mode 100644 dbms/src/Formats/ProtobufBlockOutputStream.h diff --git a/dbms/src/Formats/FormatFactory.cpp b/dbms/src/Formats/FormatFactory.cpp index 60672817db5..9674c376c47 100644 --- a/dbms/src/Formats/FormatFactory.cpp +++ b/dbms/src/Formats/FormatFactory.cpp @@ -111,6 +111,7 @@ void registerInputFormatTSKV(FormatFactory & factory); void registerOutputFormatTSKV(FormatFactory & factory); void registerInputFormatJSONEachRow(FormatFactory & factory); void registerOutputFormatJSONEachRow(FormatFactory & factory); +void registerOutputFormatProtobuf(FormatFactory & factory); /// Output only (presentational) formats. @@ -146,6 +147,7 @@ FormatFactory::FormatFactory() registerOutputFormatTSKV(*this); registerInputFormatJSONEachRow(*this); registerOutputFormatJSONEachRow(*this); + registerOutputFormatProtobuf(*this); registerInputFormatCapnProto(*this); registerOutputFormatPretty(*this); diff --git a/dbms/src/Formats/ProtobufBlockOutputStream.cpp b/dbms/src/Formats/ProtobufBlockOutputStream.cpp new file mode 100644 index 00000000000..985a92faaca --- /dev/null +++ b/dbms/src/Formats/ProtobufBlockOutputStream.cpp @@ -0,0 +1,75 @@ +#include +#include +#include +#include +#include +#include + + +namespace DB +{ +namespace ErrorCodes +{ + extern const int NOT_IMPLEMENTED; + extern const int NO_DATA_FOR_REQUIRED_PROTOBUF_FIELD; +} + + +ProtobufBlockOutputStream::ProtobufBlockOutputStream( + WriteBuffer & buffer_, + const Block & header_, + const google::protobuf::Descriptor * message_type, + const FormatSettings & format_settings_) + : writer(buffer_, message_type), header(header_), format_settings(format_settings_) +{ +} + +void ProtobufBlockOutputStream::write(const Block & block) +{ + std::vector columns_in_write_order; + const auto & fields_in_write_order = writer.fieldsInWriteOrder(); + columns_in_write_order.reserve(fields_in_write_order.size()); + for (size_t i = 0; i != fields_in_write_order.size(); ++i) + { + const auto * field = fields_in_write_order[i]; + const ColumnWithTypeAndName * column = nullptr; + if (block.has(field->name())) + { + column = &block.getByName(field->name()); + } + else if (field->is_required()) + { + throw Exception( + "Output doesn't have a column named '" + field->name() + "' which is required to write the output in the protobuf format.", + ErrorCodes::NO_DATA_FOR_REQUIRED_PROTOBUF_FIELD); + } + columns_in_write_order.emplace_back(column); + } + + for (size_t row_num = 0; row_num != block.rows(); ++row_num) + { + writer.newMessage(); + for (const auto * column : columns_in_write_order) + { + if (column) + { + assert(column->name == writer.currentField()->name()); + column->type->serializeProtobuf(*(column->column), row_num, writer); + } + writer.nextField(); + } + } +} + + +void registerOutputFormatProtobuf(FormatFactory & factory) +{ + factory.registerOutputFormat( + "Protobuf", [](WriteBuffer & buf, const Block & header, const Context & context, const FormatSettings & format_settings) + { + const auto * message_type = ProtobufSchemas::instance().getMessageTypeForFormatSchema(FormatSchemaInfo(context, "proto")); + return std::make_shared(buf, header, message_type, format_settings); + }); +} + +} diff --git a/dbms/src/Formats/ProtobufBlockOutputStream.h b/dbms/src/Formats/ProtobufBlockOutputStream.h new file mode 100644 index 00000000000..c4b5236a5ab --- /dev/null +++ b/dbms/src/Formats/ProtobufBlockOutputStream.h @@ -0,0 +1,47 @@ +#pragma once + +#include +#include +#include +#include + + +namespace google +{ +namespace protobuf +{ + class Message; +} +} + + +namespace DB +{ +/** Stream designed to serialize data in the google protobuf format. + * Each row is written as a separated message. + * These messages are delimited according to documentation + * https://github.com/protocolbuffers/protobuf/blob/master/src/google/protobuf/util/delimited_message_util.h + * Serializing in the protobuf format requires the 'format_schema' setting to be set, e.g. + * SELECT * from table FORMAT Protobuf SETTINGS format_schema = 'schema:Message' + * where schema is the name of "schema.proto" file specifying protobuf schema. + */ +class ProtobufBlockOutputStream : public IBlockOutputStream +{ +public: + ProtobufBlockOutputStream( + WriteBuffer & buffer_, + const Block & header_, + const google::protobuf::Descriptor * message_prototype_, + const FormatSettings & format_settings_); + + Block getHeader() const override { return header; } + void write(const Block & block) override; + std::string getContentType() const override { return "application/octet-stream"; } + +private: + ProtobufWriter writer; + const Block header; + const FormatSettings format_settings; +}; + +} From 2e383bfdebdc873942bc11684310fa2e9871aa9c Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Wed, 23 Jan 2019 22:43:23 +0300 Subject: [PATCH 47/86] Added tests for writing protobufs. --- .../0_stateless/00825_protobuf_format.proto | 98 ++++++ .../00825_protobuf_format_output.reference | Bin 0 -> 2019 bytes .../00825_protobuf_format_output.sh | 60 ++++ .../00825_protobuf_format_syntax2.proto | 47 +++ utils/test-data-generator/CMakeLists.txt | 6 + .../ProtobufDelimitedMessagesSerializer.cpp | 332 ++++++++++++++++++ 6 files changed, 543 insertions(+) create mode 100644 dbms/tests/queries/0_stateless/00825_protobuf_format.proto create mode 100644 dbms/tests/queries/0_stateless/00825_protobuf_format_output.reference create mode 100755 dbms/tests/queries/0_stateless/00825_protobuf_format_output.sh create mode 100644 dbms/tests/queries/0_stateless/00825_protobuf_format_syntax2.proto create mode 100644 utils/test-data-generator/ProtobufDelimitedMessagesSerializer.cpp diff --git a/dbms/tests/queries/0_stateless/00825_protobuf_format.proto b/dbms/tests/queries/0_stateless/00825_protobuf_format.proto new file mode 100644 index 00000000000..47c4e74ff23 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00825_protobuf_format.proto @@ -0,0 +1,98 @@ +syntax = "proto3"; + +enum Gender { + female = 0; + male = 1; +}; + +enum ZodiacSign { + aries = 0; + taurus = 1; + gemini = 2; + cancer = 3; + leo = 4; + virgo = 5; + libra = 6; + scorpius = 7; + sagittarius = 8; + capricorn = 9; + aquarius = 10; + pisces = 11; +}; + +message Person { + string uuid = 1; + string name = 2; + string surname = 3; + Gender gender = 4; + uint32 birthDate = 5; + bytes photo = 6; + string phoneNumber = 7; + bool isOnline = 8; + fixed32 visitTime = 9; + uint32 age = 10; + ZodiacSign zodiacSign = 11; + repeated string songs = 12; + repeated uint32 color = 13; + string hometown = 14; + repeated float location = 15; + double pi = 16; + double lotteryWin = 17; + float someRatio = 18; + float temperature = 19; + sint64 randomBigNumber = 20; +}; + +enum OnlineStatus { + offline = 0; + online = 1; +}; + +message AltPerson { + enum Gender { + male = 0; + female = 1; + }; + repeated int32 location = 101 [packed=false]; + float pi = 103; + bytes uuid = 300; + bool newFieldBool = 299; + string name = 2; + Gender gender = 102; + int32 zodiacSign = 130; + int64 birthDate = 150; + bytes age = 111; + OnlineStatus isOnline = 1; + double someRatio = 100; + fixed64 visitTime = 15; + sfixed64 randomBigNumber = 140; + repeated int32 newFieldInt = 104; + repeated float color = 14; + uint64 lotteryWin = 202; + bytes surname = 10; + uint64 phoneNumber = 5; + sint32 temperature = 41; + string newFieldStr = 21; +}; + +message StrPerson { + string uuid = 1; + string name = 2; + string surname = 3; + string gender = 4; + string birthDate = 5; + string phoneNumber = 7; + string isOnline = 8; + string visitTime = 9; + string age = 10; + string zodiacSign = 11; + repeated string songs = 12; + repeated string color = 13; + string hometown = 14; + repeated string location = 15; + string pi = 16; + string lotteryWin = 17; + string someRatio = 18; + string temperature = 19; + string randomBigNumber = 20; +}; diff --git a/dbms/tests/queries/0_stateless/00825_protobuf_format_output.reference b/dbms/tests/queries/0_stateless/00825_protobuf_format_output.reference new file mode 100644 index 0000000000000000000000000000000000000000..ed3e22f41e33b807ae354cd7045e96b86eae973f GIT binary patch literal 2019 zcmbtVe`p(39KV-KuXkCcT0xFPi)x$Dw2P0FR~ zXjD1}Djf=CV@NAwv*?IeSpRf?6b2nqWNrmT!T#e91i>Fn=s;T~zDtvC48a<|B$pTN z-F-jb&*$@f-;Y4xfTdH66>31$CJH1;U5A#fnNUkn({LP0Q^aiX_fA>4Rw3$4y7{Rf z=(yO1`H9?-&buDdiAfc#5~}H32=xECjt)m39+HyMA*V3uxVAN&Ys2U-&i9_WDoHfBc{nngyyCtC`Rl@Y&U?Pn-n{^Or*B!SO3! zy**P597t&kLd z>9?f!D?HhjVp$p-A(%p<8zwYNqQfLo*&ofZRh_g5gH~ZQJKfqCAGPy4RBcBvpB$fH zR1tzvl`4Z#75(Os!_oVP{K<|GF_7UhDHSboaoX z#5O}W%LwFtoQmk-Gi(*<@o@HCW(0k4?lkD`OB{?1bSHWT55gw{vp@tpx^=u+y7KNj zzF1X0w~6Cso?`E*#I&cFw_5WB-V@mG`T6-vT_r&;2@kCZt2ZN|Ey1T6bK)n`w<7o3 z*T3^@&$4u}ZN2GB@LkKE?DO)jCb3;jJQQnTRs%y43D=T`#HCkQa;+o;mj$pQtT*?1 z`LQ-9ZUC15a%<}PFXi&GbfJso-isaPJQ}L9bX@y%^ZKt%v6?y_h9b*9$-cqvX-|+H zCK@VCu=Q-HB=nSo_7!2HibS0gKW|`2ZmN>EcUjtwtT!EB-&o!BRoM7-qD6(pgemovziH!0uc==Mkk@22*sF=!&1tca7R;l zH}{V%%41410I|<*pGuU(;pV1>vEIt6$nD+_?9vatyE zDLT^7PAsz`_afA;M{f1)eqqF^)oNmhREUf)V0B<^SeO_sq?|&AR}jkZm|c7nRohyk zMAi+gsUBi1BSJA%PJl)au|myLpaIDU=@qnO5i~>g_-*lXz1@CG8IwY!Lu}m9^)+P- zPd60>5n{4bl)>sSpLO!Jvw+FqRYgd~r=Z)81W7YwlW}WcR+%azcKRBuvb(F5VU4QP zFm>}JXlA_Z2CJ3{WFWKv{7{wgB9IhHNacR~&_JSl2s1k_)n&(er$M?5(p dbms/tests/queries/0_stateless/00825_protobuf_format_output.reference + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +. $CURDIR/../shell_config.sh + +set -e -o pipefail + +# Copy schema files to the current directory because the client can open schemas from the current directory only. +cp "$CURDIR/00825_protobuf_format.proto" 00825_protobuf_format_copy.proto +cp "$CURDIR/00825_protobuf_format_syntax2.proto" 00825_protobuf_format_syntax2_copy.proto + +# Run the client. +$CLICKHOUSE_CLIENT --multiquery <<'EOF' +SET allow_experimental_low_cardinality_type = 1; +CREATE DATABASE IF NOT EXISTS test; +DROP TABLE IF EXISTS test.table; + +CREATE TABLE test.table (uuid UUID, + name String, + surname String, + gender Enum8('male'=1, 'female'=0), + birthDate Date, + photo Nullable(String), + phoneNumber Nullable(FixedString(13)), + isOnline UInt8, + visitTime Nullable(DateTime), + age UInt8, + zodiacSign Enum16('aries'=321, 'taurus'=420, 'gemini'=521, 'cancer'=621, 'leo'=723, 'virgo'=823, + 'libra'=923, 'scorpius'=1023, 'sagittarius'=1122, 'capricorn'=1222, 'aquarius'=120, + 'pisces'=219), + songs Array(String), + color Array(UInt8), + hometown LowCardinality(String), + location Array(Decimal32(6)), + pi Nullable(Float64), + lotteryWin Nullable(Decimal64(2)), + someRatio Float32, + temperature Decimal32(1), + randomBigNumber Int64 + ) ENGINE = MergeTree ORDER BY tuple(); + +INSERT INTO test.table VALUES (toUUID('a7522158-3d41-4b77-ad69-6c598ee55c49'), 'Ivan', 'Petrov', 'male', toDate('1980-12-29'), 'png', '+74951234567', 1, toDateTime('2019-01-05 18:45:00'), 38, 'capricorn', ['Yesterday', 'Flowers'], [255, 0, 0], 'Moscow', [55.753215, 37.622504], 3.14, 214.10, 0.1, 5.8, 17060000000); +INSERT INTO test.table VALUES (toUUID('c694ad8a-f714-4ea3-907d-fd54fb25d9b5'), 'Natalia', 'Sokolova', 'female', toDate('1992-03-08'), 'jpg', NULL, 0, NULL, 26, 'pisces', [], [100, 200, 50], 'Tver', [54.782635, 32.045251], 3.14159, NULL, 0.007, 5.4, -20000000000000); +INSERT INTO test.table VALUES (toUUID('a7da1aa6-f425-4789-8947-b034786ed374'), 'Vasily', 'Sidorov', 'male', toDate('1995-07-28'), 'bmp', '+442012345678', 1, toDateTime('2018-12-30 00:00:00'), 23, 'leo', ['Sunny'], [250, 244, 10], 'Murmansk', [68.970682, 33.074981], 3.14159265358979, 100000000000, 800, -3.2, 154400000); + +SELECT * FROM test.table ORDER BY name FORMAT Protobuf SETTINGS format_schema = '00825_protobuf_format_copy:Person'; +SELECT 'ALTERNATIVE->'; +SELECT * FROM test.table ORDER BY name FORMAT Protobuf SETTINGS format_schema = '00825_protobuf_format_copy:AltPerson'; +SELECT 'STRINGS->'; +SELECT * FROM test.table ORDER BY name FORMAT Protobuf SETTINGS format_schema = '00825_protobuf_format_copy:StrPerson'; +SELECT 'SYNTAX2->'; +SELECT * FROM test.table ORDER BY name FORMAT Protobuf SETTINGS format_schema = '00825_protobuf_format_syntax2_copy:Syntax2Person'; +EOF + +# Remove copies of the schema files. +rm "00825_protobuf_format_copy.proto" "00825_protobuf_format_syntax2_copy.proto" diff --git a/dbms/tests/queries/0_stateless/00825_protobuf_format_syntax2.proto b/dbms/tests/queries/0_stateless/00825_protobuf_format_syntax2.proto new file mode 100644 index 00000000000..dad1b596824 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00825_protobuf_format_syntax2.proto @@ -0,0 +1,47 @@ +syntax = "proto2"; + +message Syntax2Person { + enum Gender { + female = 0; + male = 1; + }; + + enum ZodiacSign { + aries = 0; + taurus = 1; + gemini = 2; + cancer = 3; + leo = 4; + virgo = 5; + libra = 6; + scorpius = 7; + sagittarius = 8; + capricorn = 9; + aquarius = 10; + pisces = 11; + }; + + required string uuid = 1; + required string name = 2; + required string surname = 3; + required Gender gender = 4; + required uint32 birthDate = 5; + optional bytes photo = 6; + optional string phoneNumber = 7; + optional bool isOnline = 8; + optional fixed32 visitTime = 9; + optional uint32 age = 10; + optional ZodiacSign zodiacSign = 11; + repeated string songs = 12; + repeated uint32 color = 13; + optional string hometown = 14 [default='Moscow']; + repeated float location = 15 [packed=true]; + optional double pi = 16; + optional double lotteryWin = 17; + optional float someRatio = 18; + optional float temperature = 19; + optional sint64 randomBigNumber = 20; + optional string newFieldStr = 21 [default='abc']; + optional int32 newFieldInt = 22 [default=-11]; + optional bool newBool = 23 [default=true]; +}; diff --git a/utils/test-data-generator/CMakeLists.txt b/utils/test-data-generator/CMakeLists.txt index 60f17894704..bf3cfa1bf7f 100644 --- a/utils/test-data-generator/CMakeLists.txt +++ b/utils/test-data-generator/CMakeLists.txt @@ -3,3 +3,9 @@ target_link_libraries(test-data-generator PRIVATE clickhouse_common_io ${Boost_P add_executable (markov-model markov-model.cpp) target_link_libraries(markov-model PRIVATE clickhouse_common_io ${Boost_PROGRAM_OPTIONS_LIBRARY}) + +protobuf_generate_cpp(ProtobufDelimitedMessagesSerializer_Srcs ProtobufDelimitedMessagesSerializer_Hdrs ${CMAKE_CURRENT_SOURCE_DIR}/../../dbms/tests/queries/0_stateless/00825_protobuf_format.proto) +protobuf_generate_cpp(ProtobufDelimitedMessagesSerializer_Srcs2 ProtobufDelimitedMessagesSerializer_Hdrs2 ${CMAKE_CURRENT_SOURCE_DIR}/../../dbms/tests/queries/0_stateless/00825_protobuf_format_syntax2.proto) +add_executable (ProtobufDelimitedMessagesSerializer ProtobufDelimitedMessagesSerializer.cpp ${ProtobufDelimitedMessagesSerializer_Srcs} ${ProtobufDelimitedMessagesSerializer_Hdrs} ${ProtobufDelimitedMessagesSerializer_Srcs2} ${ProtobufDelimitedMessagesSerializer_Hdrs2}) +target_include_directories (ProtobufDelimitedMessagesSerializer SYSTEM BEFORE PRIVATE ${Protobuf_INCLUDE_DIR} ${CMAKE_CURRENT_BINARY_DIR}) +target_link_libraries (ProtobufDelimitedMessagesSerializer PRIVATE ${Protobuf_LIBRARY}) \ No newline at end of file diff --git a/utils/test-data-generator/ProtobufDelimitedMessagesSerializer.cpp b/utils/test-data-generator/ProtobufDelimitedMessagesSerializer.cpp new file mode 100644 index 00000000000..4a5596cb7f2 --- /dev/null +++ b/utils/test-data-generator/ProtobufDelimitedMessagesSerializer.cpp @@ -0,0 +1,332 @@ +// Reference file generator for the test dbms/tests/queries/0_stateless/00825_protobuf_format_output.sh + +#include +#include +#include "00825_protobuf_format.pb.h" +#include "00825_protobuf_format_syntax2.pb.h" + + +int main(int, char **) +{ + std::ostream* out = &std::cout; + + { + Person person; + person.set_uuid("a7522158-3d41-4b77-ad69-6c598ee55c49"); + person.set_name("Ivan"); + person.set_surname("Petrov"); + person.set_gender(Gender::male); + person.set_birthdate(4015); // 1980-12-29 + person.set_photo("png"); + person.set_phonenumber(std::string("+74951234567\0", 13)); // Converted from FixedString(13) + person.set_isonline(true); + person.set_visittime(1546703100); // 2019-01-05 18:45:00 + person.set_age(38); + person.set_zodiacsign(ZodiacSign::capricorn); + person.add_songs("Yesterday"); + person.add_songs("Flowers"); + person.add_color(255); + person.add_color(0); + person.add_color(0); + person.set_hometown("Moscow"); + person.add_location(55.753215); + person.add_location(37.622504); + person.set_pi(3.14); + person.set_lotterywin(214.10); + person.set_someratio(0.1); + person.set_temperature(5.8); + person.set_randombignumber(17060000000); + google::protobuf::util::SerializeDelimitedToOstream(person, out); + } + + { + Person person; + person.set_uuid("c694ad8a-f714-4ea3-907d-fd54fb25d9b5"); + person.set_name("Natalia"); + person.set_surname("Sokolova"); + person.set_gender(Gender::female); + person.set_birthdate(8102); // 1992-03-08 + person.set_photo("jpg"); + person.set_isonline(false); + person.set_age(26); + person.set_zodiacsign(ZodiacSign::pisces); + person.add_color(100); + person.add_color(200); + person.add_color(50); + person.set_hometown("Tver"); + person.add_location(54.782635); + person.add_location(32.045251); + person.set_pi(3.14159); + person.set_someratio(0.007); + person.set_temperature(5.4); + person.set_randombignumber(-20000000000000); + google::protobuf::util::SerializeDelimitedToOstream(person, out); + } + + { + Person person; + person.set_uuid("a7da1aa6-f425-4789-8947-b034786ed374"); + person.set_name("Vasily"); + person.set_surname("Sidorov"); + person.set_gender(Gender::male); + person.set_birthdate(9339); // 1995-07-28 + person.set_photo("bmp"); + person.set_phonenumber("+442012345678"); + person.set_isonline(true); + person.set_visittime(1546117200); // 2018-12-30 00:00:00 + person.set_age(23); + person.set_zodiacsign(ZodiacSign::leo); + person.add_songs("Sunny"); + person.add_color(250); + person.add_color(244); + person.add_color(10); + person.set_hometown("Murmansk"); + person.add_location(68.970682); + person.add_location(33.074981); + person.set_pi(3.14159265358979); + person.set_lotterywin(100000000000); + person.set_someratio(800); + person.set_temperature(-3.2); + person.set_randombignumber(154400000); + google::protobuf::util::SerializeDelimitedToOstream(person, out); + } + + *out << "ALTERNATIVE->" << std::endl; + + { + AltPerson person; + person.add_location(55); + person.add_location(37); + person.set_pi(3.14); + person.set_uuid("a7522158-3d41-4b77-ad69-6c598ee55c49"); + person.set_name("Ivan"); + person.set_gender(AltPerson::male); + person.set_zodiacsign(1222); // capricorn + person.set_birthdate(4015); // 1980-12-29 + person.set_age("38"); + person.set_isonline(OnlineStatus::online); + person.set_someratio(0.100000001490116119384765625); // 0.1 converted from float to double + person.set_visittime(1546703100); // 2019-01-05 18:45:00 + person.set_randombignumber(17060000000); + person.add_color(255); + person.add_color(0); + person.add_color(0); + person.set_lotterywin(214); + person.set_surname("Petrov"); + person.set_phonenumber(+74951234567); + person.set_temperature(5); + google::protobuf::util::SerializeDelimitedToOstream(person, out); + } + + { + AltPerson person; + person.add_location(54); + person.add_location(32); + person.set_pi(3.14159); + person.set_uuid("c694ad8a-f714-4ea3-907d-fd54fb25d9b5"); + person.set_name("Natalia"); + person.set_gender(AltPerson::female); + person.set_zodiacsign(219); // pisces + person.set_birthdate(8102); // 1992-03-08 + person.set_age("26"); + person.set_isonline(OnlineStatus::offline); + person.set_someratio(0.007000000216066837310791015625); // 0.007 converted from float to double + person.set_randombignumber(-20000000000000); + person.add_color(100); + person.add_color(200); + person.add_color(50); + person.set_surname("Sokolova"); + person.set_temperature(5); + google::protobuf::util::SerializeDelimitedToOstream(person, out); + } + + { + AltPerson person; + person.add_location(68); + person.add_location(33); + person.set_pi(3.1415926535897); + person.set_uuid("a7da1aa6-f425-4789-8947-b034786ed374"); + person.set_name("Vasily"); + person.set_gender(AltPerson::male); + person.set_zodiacsign(723); // leo + person.set_birthdate(9339); // 1995-07-28 + person.set_age("23"); + person.set_isonline(OnlineStatus::online); + person.set_someratio(800); + person.set_visittime(1546117200); // 2018-12-30 00:00:00 + person.set_randombignumber(154400000); + person.add_color(250); + person.add_color(244); + person.add_color(10); + person.set_lotterywin(100000000000); + person.set_surname("Sidorov"); + person.set_phonenumber(+442012345678); + person.set_temperature(-3); + google::protobuf::util::SerializeDelimitedToOstream(person, out); + } + + *out << "STRINGS->" << std::endl; + + { + StrPerson person; + person.set_uuid("a7522158-3d41-4b77-ad69-6c598ee55c49"); + person.set_name("Ivan"); + person.set_surname("Petrov"); + person.set_gender("male"); + person.set_birthdate("1980-12-29"); + person.set_phonenumber(std::string("+74951234567\0", 13)); // Converted from FixedString(13) + person.set_isonline("1"); + person.set_visittime("2019-01-05 18:45:00"); + person.set_age("38"); + person.set_zodiacsign("capricorn"); + person.add_songs("Yesterday"); + person.add_songs("Flowers"); + person.add_color("255"); + person.add_color("0"); + person.add_color("0"); + person.set_hometown("Moscow"); + person.add_location("55.753215"); + person.add_location("37.622504"); + person.set_pi("3.14"); + person.set_lotterywin("214.10"); + person.set_someratio("0.1"); + person.set_temperature("5.8"); + person.set_randombignumber("17060000000"); + google::protobuf::util::SerializeDelimitedToOstream(person, out); + } + + { + StrPerson person; + person.set_uuid("c694ad8a-f714-4ea3-907d-fd54fb25d9b5"); + person.set_name("Natalia"); + person.set_surname("Sokolova"); + person.set_gender("female"); + person.set_birthdate("1992-03-08"); + person.set_isonline("0"); + person.set_age("26"); + person.set_zodiacsign("pisces"); + person.add_color("100"); + person.add_color("200"); + person.add_color("50"); + person.set_hometown("Tver"); + person.add_location("54.782635"); + person.add_location("32.045251"); + person.set_pi("3.14159"); + person.set_someratio("0.007"); + person.set_temperature("5.4"); + person.set_randombignumber("-20000000000000"); + google::protobuf::util::SerializeDelimitedToOstream(person, out); + } + + { + StrPerson person; + person.set_uuid("a7da1aa6-f425-4789-8947-b034786ed374"); + person.set_name("Vasily"); + person.set_surname("Sidorov"); + person.set_gender("male"); + person.set_birthdate("1995-07-28"); + person.set_phonenumber("+442012345678"); + person.set_isonline("1"); + person.set_visittime("2018-12-30 00:00:00"); + person.set_age("23"); + person.set_zodiacsign("leo"); + person.add_songs("Sunny"); + person.add_color("250"); + person.add_color("244"); + person.add_color("10"); + person.set_hometown("Murmansk"); + person.add_location("68.970682"); + person.add_location("33.074981"); + person.set_pi("3.14159265358979"); + person.set_lotterywin("100000000000.00"); + person.set_someratio("800"); + person.set_temperature("-3.2"); + person.set_randombignumber("154400000"); + google::protobuf::util::SerializeDelimitedToOstream(person, out); + } + + *out << "SYNTAX2->" << std::endl; + + { + Syntax2Person person; + person.set_uuid("a7522158-3d41-4b77-ad69-6c598ee55c49"); + person.set_name("Ivan"); + person.set_surname("Petrov"); + person.set_gender(Syntax2Person::male); + person.set_birthdate(4015); // 1980-12-29 + person.set_photo("png"); + person.set_phonenumber(std::string("+74951234567\0", 13)); // Converted from FixedString(13) + person.set_isonline(true); + person.set_visittime(1546703100); // 2019-01-05 18:45:00 + person.set_age(38); + person.set_zodiacsign(Syntax2Person::capricorn); + person.add_songs("Yesterday"); + person.add_songs("Flowers"); + person.add_color(255); + person.add_color(0); + person.add_color(0); + person.set_hometown("Moscow"); + person.add_location(55.753215); + person.add_location(37.622504); + person.set_pi(3.14); + person.set_lotterywin(214.10); + person.set_someratio(0.1); + person.set_temperature(5.8); + person.set_randombignumber(17060000000); + google::protobuf::util::SerializeDelimitedToOstream(person, out); + } + + { + Syntax2Person person; + person.set_uuid("c694ad8a-f714-4ea3-907d-fd54fb25d9b5"); + person.set_name("Natalia"); + person.set_surname("Sokolova"); + person.set_gender(Syntax2Person::female); + person.set_birthdate(8102); // 1992-03-08 + person.set_photo("jpg"); + person.set_isonline(false); + person.set_age(26); + person.set_zodiacsign(Syntax2Person::pisces); + person.add_color(100); + person.add_color(200); + person.add_color(50); + person.set_hometown("Tver"); + person.add_location(54.782635); + person.add_location(32.045251); + person.set_pi(3.14159); + person.set_someratio(0.007); + person.set_temperature(5.4); + person.set_randombignumber(-20000000000000); + google::protobuf::util::SerializeDelimitedToOstream(person, out); + } + + { + Syntax2Person person; + person.set_uuid("a7da1aa6-f425-4789-8947-b034786ed374"); + person.set_name("Vasily"); + person.set_surname("Sidorov"); + person.set_gender(Syntax2Person::male); + person.set_birthdate(9339); // 1995-07-28 + person.set_photo("bmp"); + person.set_phonenumber("+442012345678"); + person.set_isonline(true); + person.set_visittime(1546117200); // 2018-12-30 00:00:00 + person.set_age(23); + person.set_zodiacsign(Syntax2Person::leo); + person.add_songs("Sunny"); + person.add_color(250); + person.add_color(244); + person.add_color(10); + person.set_hometown("Murmansk"); + person.add_location(68.970682); + person.add_location(33.074981); + person.set_pi(3.14159265358979); + person.set_lotterywin(100000000000); + person.set_someratio(800); + person.set_temperature(-3.2); + person.set_randombignumber(154400000); + google::protobuf::util::SerializeDelimitedToOstream(person, out); + } + + return 0; +} From af81d3cf0dc212911348ee3eff583ed6f881ec36 Mon Sep 17 00:00:00 2001 From: George G Date: Thu, 24 Jan 2019 14:55:59 +0000 Subject: [PATCH 48/86] docs: Fix `sudo` position in CPU scaling governor command (#4142) --- docs/en/operations/tips.md | 2 +- docs/ru/operations/tips.md | 2 +- docs/zh/operations/tips.md | 2 +- 3 files changed, 3 insertions(+), 3 deletions(-) diff --git a/docs/en/operations/tips.md b/docs/en/operations/tips.md index e3b66a639ba..3508c66f1af 100644 --- a/docs/en/operations/tips.md +++ b/docs/en/operations/tips.md @@ -21,7 +21,7 @@ You can use `turbostat` to view the CPU's actual clock rate under a load. Always use the `performance` scaling governor. The `on-demand` scaling governor works much worse with constantly high demand. ```bash -sudo echo 'performance' | tee /sys/devices/system/cpu/cpu*/cpufreq/scaling_governor +echo 'performance' | sudo tee /sys/devices/system/cpu/cpu*/cpufreq/scaling_governor ``` ## CPU Limitations diff --git a/docs/ru/operations/tips.md b/docs/ru/operations/tips.md index a66a760bc79..e9bbf77d041 100644 --- a/docs/ru/operations/tips.md +++ b/docs/ru/operations/tips.md @@ -21,7 +21,7 @@ Turbo-Boost крайне не рекомендуется отключать. П Нужно всегда использовать `performance` scaling governor. `ondemand` scaling governor работает намного хуже при постоянно высоком спросе. ```bash -sudo echo 'performance' | tee /sys/devices/system/cpu/cpu*/cpufreq/scaling_governor +echo 'performance' | sudo tee /sys/devices/system/cpu/cpu*/cpufreq/scaling_governor ``` ## Ограничение CPU diff --git a/docs/zh/operations/tips.md b/docs/zh/operations/tips.md index 11b7a782deb..1dcbb74a2cc 100644 --- a/docs/zh/operations/tips.md +++ b/docs/zh/operations/tips.md @@ -21,7 +21,7 @@ You can use `turbostat` to view the CPU's actual clock rate under a load. Always use the `performance` scaling governor. The `on-demand` scaling governor works much worse with constantly high demand. ```bash -sudo echo 'performance' | tee /sys/devices/system/cpu/cpu*/cpufreq/scaling_governor +echo 'performance' | sudo tee /sys/devices/system/cpu/cpu*/cpufreq/scaling_governor ``` ## CPU Limitations From ccd9bd0877ecdab2a23f3578f2cfce7006aec14e Mon Sep 17 00:00:00 2001 From: chertus Date: Thu, 24 Jan 2019 18:06:15 +0300 Subject: [PATCH 49/86] fix build --- dbms/src/Interpreters/QueryNormalizer.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/src/Interpreters/QueryNormalizer.cpp b/dbms/src/Interpreters/QueryNormalizer.cpp index aa00e6e8b5c..a5151be7c64 100644 --- a/dbms/src/Interpreters/QueryNormalizer.cpp +++ b/dbms/src/Interpreters/QueryNormalizer.cpp @@ -376,7 +376,7 @@ void QueryNormalizer::extractJoinUsingColumns(const ASTPtr ast, Data & data) for (const auto & key : keys.children) if (auto opt_column = getIdentifierName(key)) data.join_using_columns.insert(*opt_column); - else if (auto * literal = typeid_cast(key.get())) + else if (typeid_cast(key.get())) data.join_using_columns.insert(key->getColumnName()); else { From b8d7922e42f0f82f348a2c113671ea2c19e4a93f Mon Sep 17 00:00:00 2001 From: alesapin Date: Thu, 24 Jan 2019 19:10:05 +0300 Subject: [PATCH 50/86] CLICKHOUSE-4170: Add docs for datasets downloading (#4144) * CLICKHOUSE-4170: Add docs for datasets downloading * CLICKHOUSE-4170: Fix Review remarks --- .../example_datasets/nyc_taxi.md | 20 ++++++++++++++++ .../example_datasets/ontime.md | 23 ++++++++++++++++++- .../example_datasets/nyc_taxi.md | 21 ++++++++++++++++- .../example_datasets/ontime.md | 23 ++++++++++++++++++- 4 files changed, 84 insertions(+), 3 deletions(-) diff --git a/docs/en/getting_started/example_datasets/nyc_taxi.md b/docs/en/getting_started/example_datasets/nyc_taxi.md index 0f5f1cb3cb4..ed073fc85a6 100644 --- a/docs/en/getting_started/example_datasets/nyc_taxi.md +++ b/docs/en/getting_started/example_datasets/nyc_taxi.md @@ -1,5 +1,10 @@ # New York Taxi Data +This dataset can be obtained in two ways: + +- import from raw data +- download of prepared partitions + ## How to Import The Raw Data See and for the description of a dataset and instructions for downloading. @@ -272,6 +277,21 @@ WHERE (table = 'trips_mergetree') AND active Among other things, you can run the OPTIMIZE query on MergeTree. But it's not required, since everything will be fine without it. +## Dowload of Prepared Partitions + +```bash +curl -O https://clickhouse-datasets.s3.yandex.net/trips_mergetree/partitions/trips_mergetree.tar +tar xvf trips_mergetree.tar -C /var/lib/clickhouse # path to ClickHouse data directory +# check permissions of unpacked data, fix if required +sudo service clickhouse-server restart +clickhouse-client --query "select count(*) from datasets.trips_mergetree" +``` + +!!!info + If you will run queries described below, you have to use full table name, + `datasets.trips_mergetree`. + + ## Results on Single Server Q1: diff --git a/docs/en/getting_started/example_datasets/ontime.md b/docs/en/getting_started/example_datasets/ontime.md index c4dcaf9575e..6cf2e1ecb52 100644 --- a/docs/en/getting_started/example_datasets/ontime.md +++ b/docs/en/getting_started/example_datasets/ontime.md @@ -1,6 +1,13 @@ # OnTime +This dataset can be obtained in two ways: + +- import from raw data +- download of prepared partitions + +## Import From Raw Data + Downloading data: ```bash @@ -137,7 +144,21 @@ Loading data: for i in *.zip; do echo $i; unzip -cq $i '*.csv' | sed 's/\.00//g' | clickhouse-client --host=example-perftest01j --query="INSERT INTO ontime FORMAT CSVWithNames"; done ``` -Queries: +## Dowload of Prepared Partitions + +```bash +curl -O https://clickhouse-datasets.s3.yandex.net/ontime/partitions/ontime.tar +tar xvf ontime.tar -C /var/lib/clickhouse # path to ClickHouse data directory +# check permissions of unpacked data, fix if required +sudo service clickhouse-server restart +clickhouse-client --query "select count(*) from datasets.ontime" +``` + +!!!info + If you will run queries described below, you have to use full table name, + `datasets.ontime`. + +## Queries Q0. diff --git a/docs/ru/getting_started/example_datasets/nyc_taxi.md b/docs/ru/getting_started/example_datasets/nyc_taxi.md index cd86ff05b76..1050a589669 100644 --- a/docs/ru/getting_started/example_datasets/nyc_taxi.md +++ b/docs/ru/getting_started/example_datasets/nyc_taxi.md @@ -1,5 +1,10 @@ # Данные о такси в Нью-Йорке +Этот датасет может быть получен двумя способами: + +- импорт из сырых данных; +- скачивание готовых партиций. + ## Как импортировать сырые данные См. и для описания набора данных и инструкций по загрузке. @@ -272,7 +277,21 @@ WHERE (table = 'trips_mergetree') AND active Между прочим, на MergeTree можно запустить запрос OPTIMIZE. Но это не обязательно, всё будет в порядке и без этого. -## Results on single server +## Скачивание готовых партиций + +```bash +curl -O https://clickhouse-datasets.s3.yandex.net/trips_mergetree/partitions/trips_mergetree.tar +tar xvf trips_mergetree.tar -C /var/lib/clickhouse # путь к папке с данными ClickHouse +# убедитесь, что установлены корректные права доступа на файлы +sudo service clickhouse-server restart +clickhouse-client --query "select count(*) from datasets.trips_mergetree" +``` + +!!!info + Если вы собираетесь выполнять запросы, приведенные ниже, то к имени таблицы + нужно добавить имя базы, `datasets.trips_mergetree`. + +## Результаты на одном сервере Q1: diff --git a/docs/ru/getting_started/example_datasets/ontime.md b/docs/ru/getting_started/example_datasets/ontime.md index f138817b400..89a3fa7ebe6 100644 --- a/docs/ru/getting_started/example_datasets/ontime.md +++ b/docs/ru/getting_started/example_datasets/ontime.md @@ -1,6 +1,13 @@ # OnTime +Этот датасет может быть получен двумя способами: + +- импорт из сырых данных; +- скачивание готовых партиций. + +## Импорт из сырых данных + Скачивание данных: ```bash @@ -137,7 +144,21 @@ CREATE TABLE `ontime` ( for i in *.zip; do echo $i; unzip -cq $i '*.csv' | sed 's/\.00//g' | clickhouse-client --host=example-perftest01j --query="INSERT INTO ontime FORMAT CSVWithNames"; done ``` -Запросы: +## Скачивание готовых партиций + +```bash +curl -O https://clickhouse-datasets.s3.yandex.net/ontime/partitions/ontime.tar +tar xvf ontime.tar -C /var/lib/clickhouse # путь к папке с данными ClickHouse +# убедитесь, что установлены корректные права доступа на файлы +sudo service clickhouse-server restart +clickhouse-client --query "select count(*) from datasets.ontime" +``` + +!!!info + Если вы собираетесь выполнять запросы, приведенные ниже, то к имени таблицы + нужно добавить имя базы, `datasets.ontime`. + +## Запросы: Q0. From d79ed9fde68827d216b6bbabf738bbcc640e6412 Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Thu, 24 Jan 2019 19:34:27 +0300 Subject: [PATCH 51/86] Update 00502_sum_map.reference --- dbms/tests/queries/0_stateless/00502_sum_map.reference | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/tests/queries/0_stateless/00502_sum_map.reference b/dbms/tests/queries/0_stateless/00502_sum_map.reference index 67639ad230f..7bb325be814 100644 --- a/dbms/tests/queries/0_stateless/00502_sum_map.reference +++ b/dbms/tests/queries/0_stateless/00502_sum_map.reference @@ -9,7 +9,7 @@ 2000-01-01 00:00:00 [1,2,3,4,5] [10,10,20,10,10] 2000-01-01 00:01:00 [4,5,6,7,8] [10,10,20,10,10] ([1],[10]) -([1,4,8], [10,20,10]) +([1,4,8],[10,20,10]) ([1],[1]) ([1],[1]) (['a'],[1]) From 0e59fe0b46e0e12b9e652e995a7939ad75d624bc Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 24 Jan 2019 20:12:05 +0300 Subject: [PATCH 52/86] Renamed setting and variables #3973 --- dbms/src/Interpreters/Join.cpp | 22 +++++++++---------- dbms/src/Interpreters/Join.h | 14 ++++++------ dbms/src/Interpreters/Settings.h | 2 +- dbms/src/Storages/StorageJoin.cpp | 8 +++---- .../0_stateless/00830_join_overwrite.sql | 2 +- 5 files changed, 24 insertions(+), 24 deletions(-) diff --git a/dbms/src/Interpreters/Join.cpp b/dbms/src/Interpreters/Join.cpp index 553e5e8810a..6ef873fb6c7 100644 --- a/dbms/src/Interpreters/Join.cpp +++ b/dbms/src/Interpreters/Join.cpp @@ -34,11 +34,11 @@ namespace ErrorCodes Join::Join(const Names & key_names_right_, bool use_nulls_, const SizeLimits & limits, - ASTTableJoin::Kind kind_, ASTTableJoin::Strictness strictness_, bool overwrite_) + ASTTableJoin::Kind kind_, ASTTableJoin::Strictness strictness_, bool any_take_last_row_) : kind(kind_), strictness(strictness_), key_names_right(key_names_right_), use_nulls(use_nulls_), - overwrite(overwrite_), + any_take_last_row(any_take_last_row_), log(&Logger::get("Join")), limits(limits) { @@ -480,7 +480,7 @@ bool Join::insertFromBlock(const Block & block) namespace { - template + template struct Adder; template @@ -526,8 +526,8 @@ namespace } }; - template - struct Adder + template + struct Adder { static void addFound(const typename Map::const_iterator & it, size_t num_columns_to_add, MutableColumns & added_columns, size_t i, IColumn::Filter * filter, IColumn::Offset & current_offset, IColumn::Offsets * offsets, @@ -553,7 +553,7 @@ namespace { (*filter)[i] = 0; - if (!pad_left) + if (!fill_left) { (*offsets)[i] = current_offset; } @@ -584,7 +584,7 @@ namespace { if (has_null_map && (*null_map)[i]) { - Adder::pad_left, STRICTNESS, Map>::addNotFound( + Adder::fill_left, STRICTNESS, Map>::addNotFound( num_columns_to_add, added_columns, i, filter.get(), current_offset, offsets_to_replicate.get()); } else @@ -595,11 +595,11 @@ namespace if (it != map.end()) { it->second.setUsed(); - Adder::pad_left, STRICTNESS, Map>::addFound( + Adder::fill_left, STRICTNESS, Map>::addFound( it, num_columns_to_add, added_columns, i, filter.get(), current_offset, offsets_to_replicate.get(), right_indexes); } else - Adder::pad_left, STRICTNESS, Map>::addNotFound( + Adder::fill_left, STRICTNESS, Map>::addNotFound( num_columns_to_add, added_columns, i, filter.get(), current_offset, offsets_to_replicate.get()); } } @@ -889,7 +889,7 @@ void Join::joinGetImpl(Block & block, const String & column_name, const Maps & m // TODO: support composite key -// TODO: return multible columns as named tuple +// TODO: return multiple columns as named tuple // TODO: return array of values when strictness == ASTTableJoin::Strictness::All void Join::joinGet(Block & block, const String & column_name) const { @@ -902,7 +902,7 @@ void Join::joinGet(Block & block, const String & column_name) const if (kind == ASTTableJoin::Kind::Left && strictness == ASTTableJoin::Strictness::Any) { - if (overwrite) + if (any_take_last_row) joinGetImpl(block, column_name, std::get(maps)); else joinGetImpl(block, column_name, std::get(maps)); diff --git a/dbms/src/Interpreters/Join.h b/dbms/src/Interpreters/Join.h index 43de3092409..3a70f1d07ac 100644 --- a/dbms/src/Interpreters/Join.h +++ b/dbms/src/Interpreters/Join.h @@ -223,7 +223,7 @@ class Join { public: Join(const Names & key_names_right_, bool use_nulls_, const SizeLimits & limits, - ASTTableJoin::Kind kind_, ASTTableJoin::Strictness strictness_, bool overwrite_ = false); + ASTTableJoin::Kind kind_, ASTTableJoin::Strictness strictness_, bool any_take_last_row_ = false); bool empty() { return type == Type::EMPTY; } @@ -380,18 +380,18 @@ public: struct KindTrait { // Affects the Adder trait so that when the right part is empty, adding a default value on the left - static constexpr bool pad_left = static_in_v; + static constexpr bool fill_left = static_in_v; // Affects the Map trait so that a `used` flag is attached to map slots in order to // generate default values on the right when the left part is empty - static constexpr bool pad_right = static_in_v; + static constexpr bool fill_right = static_in_v; }; - template + template struct MapGetterImpl; template - using Map = typename MapGetterImpl::pad_right, strictness, overwrite>::Map; + using Map = typename MapGetterImpl::fill_right, strictness, overwrite>::Map; static constexpr std::array STRICTNESSES = {ASTTableJoin::Strictness::Any, ASTTableJoin::Strictness::All}; static constexpr std::array KINDS @@ -402,7 +402,7 @@ public: template bool dispatch(Func && func) { - if (overwrite) + if (any_take_last_row) { return static_for<0, KINDS.size()>([&](auto i) { @@ -464,7 +464,7 @@ private: bool use_nulls; /// Overwrite existing values when encountering the same key again - bool overwrite; + bool any_take_last_row; /** Blocks of "right" table. */ diff --git a/dbms/src/Interpreters/Settings.h b/dbms/src/Interpreters/Settings.h index 464cbad8a8e..b719a11b1a6 100644 --- a/dbms/src/Interpreters/Settings.h +++ b/dbms/src/Interpreters/Settings.h @@ -253,7 +253,7 @@ struct Settings M(SettingUInt64, max_rows_in_join, 0, "Maximum size of the hash table for JOIN (in number of rows).") \ M(SettingUInt64, max_bytes_in_join, 0, "Maximum size of the hash table for JOIN (in number of bytes in memory).") \ M(SettingOverflowMode, join_overflow_mode, OverflowMode::THROW, "What to do when the limit is exceeded.") \ - M(SettingBool, join_overwrite, false, "Whether to overwrite existing values when encountering the same key again.") \ + M(SettingBool, join_any_take_last_row, false, "When disabled (default) ANY JOIN will take the first found row for a key. When enabled, it will take the last row seen if there are multiple rows for the same key.") \ \ M(SettingUInt64, max_rows_to_transfer, 0, "Maximum size (in rows) of the transmitted external table obtained when the GLOBAL IN/JOIN section is executed.") \ M(SettingUInt64, max_bytes_to_transfer, 0, "Maximum size (in uncompressed bytes) of the transmitted external table obtained when the GLOBAL IN/JOIN section is executed.") \ diff --git a/dbms/src/Storages/StorageJoin.cpp b/dbms/src/Storages/StorageJoin.cpp index dfaea34905c..78ef15a4fc8 100644 --- a/dbms/src/Storages/StorageJoin.cpp +++ b/dbms/src/Storages/StorageJoin.cpp @@ -134,7 +134,7 @@ void registerStorageJoin(StorageFactory & factory) auto max_rows_in_join = settings.max_rows_in_join; auto max_bytes_in_join = settings.max_bytes_in_join; auto join_overflow_mode = settings.join_overflow_mode; - auto join_overwrite = settings.join_overwrite; + auto join_any_take_last_row = settings.join_any_take_last_row; if (args.storage_def && args.storage_def->settings) { @@ -148,8 +148,8 @@ void registerStorageJoin(StorageFactory & factory) max_bytes_in_join.set(setting.value); else if (setting.name == "join_overflow_mode") join_overflow_mode.set(setting.value); - else if (setting.name == "join_overwrite") - join_overwrite.set(setting.value); + else if (setting.name == "join_any_take_last_row") + join_any_take_last_row.set(setting.value); else throw Exception( "Unknown setting " + setting.name + " for storage " + args.engine_name, @@ -166,7 +166,7 @@ void registerStorageJoin(StorageFactory & factory) kind, strictness, args.columns, - join_overwrite); + join_any_take_last_row); }); } diff --git a/dbms/tests/queries/0_stateless/00830_join_overwrite.sql b/dbms/tests/queries/0_stateless/00830_join_overwrite.sql index 1f981eb00bd..d6fca471746 100644 --- a/dbms/tests/queries/0_stateless/00830_join_overwrite.sql +++ b/dbms/tests/queries/0_stateless/00830_join_overwrite.sql @@ -6,7 +6,7 @@ CREATE TABLE kv (k UInt32, v UInt32) ENGINE Join(Any, Left, k); INSERT INTO kv VALUES (1, 2); INSERT INTO kv VALUES (1, 3); SELECT joinGet('kv', 'v', toUInt32(1)); -CREATE TABLE kv_overwrite (k UInt32, v UInt32) ENGINE Join(Any, Left, k) SETTINGS join_overwrite = 1; +CREATE TABLE kv_overwrite (k UInt32, v UInt32) ENGINE Join(Any, Left, k) SETTINGS join_any_take_last_row = 1; INSERT INTO kv_overwrite VALUES (1, 2); INSERT INTO kv_overwrite VALUES (1, 3); SELECT joinGet('kv_overwrite', 'v', toUInt32(1)); From ffd18163de014126b04a06f97c8c214172b1c45a Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 24 Jan 2019 21:33:57 +0300 Subject: [PATCH 53/86] Fixed fuzz test in UBSan: added parameter check for quantile*Weighted function --- dbms/src/AggregateFunctions/AggregateFunctionQuantile.h | 7 +++++-- 1 file changed, 5 insertions(+), 2 deletions(-) diff --git a/dbms/src/AggregateFunctions/AggregateFunctionQuantile.h b/dbms/src/AggregateFunctions/AggregateFunctionQuantile.h index ad25ff95af3..cee2b6fe0c0 100644 --- a/dbms/src/AggregateFunctions/AggregateFunctionQuantile.h +++ b/dbms/src/AggregateFunctions/AggregateFunctionQuantile.h @@ -100,7 +100,7 @@ public: return res; } - void NO_SANITIZE_UNDEFINED add(AggregateDataPtr place, const IColumn ** columns, size_t row_num, Arena *) const override + void add(AggregateDataPtr place, const IColumn ** columns, size_t row_num, Arena *) const override { /// Out of range conversion may occur. This is Ok. @@ -177,8 +177,11 @@ public: static void assertSecondArg(const DataTypes & argument_types) { if constexpr (has_second_arg) - /// TODO: check that second argument is of numerical type. + { assertBinary(Name::name, argument_types); + if (!isUnsignedInteger(argument_types[1])) + throw Exception("Second argument (weight) for function " + std::string(Name::name) + " must be unsigned integer, but it has type " + argument_types[1]->getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + } else assertUnary(Name::name, argument_types); } From 8a11f753bef9841ff1418e759c96100def3c727b Mon Sep 17 00:00:00 2001 From: chertus Date: Thu, 24 Jan 2019 21:34:31 +0300 Subject: [PATCH 54/86] fix test results --- .../0_stateless/00050_any_left_join.reference | 10 +++++----- .../0_stateless/00050_any_left_join.sql | 6 +++--- .../00051_any_inner_join.reference | 10 +++++----- .../0_stateless/00051_any_inner_join.sql | 6 +++--- .../00053_all_inner_join.reference | 20 +++++++++---------- .../0_stateless/00053_all_inner_join.sql | 6 +++--- .../queries/0_stateless/00054_join_string.sql | 6 +++--- 7 files changed, 32 insertions(+), 32 deletions(-) diff --git a/dbms/tests/queries/0_stateless/00050_any_left_join.reference b/dbms/tests/queries/0_stateless/00050_any_left_join.reference index dc46f7e8a75..98dc95d69e1 100644 --- a/dbms/tests/queries/0_stateless/00050_any_left_join.reference +++ b/dbms/tests/queries/0_stateless/00050_any_left_join.reference @@ -1,10 +1,10 @@ 0 0 0 -1 1 0 +1 0 0 2 2 1 -3 3 0 +3 0 0 4 4 2 -5 5 0 +5 0 0 6 6 3 -7 7 0 +7 0 0 8 8 4 -9 9 0 +9 0 0 diff --git a/dbms/tests/queries/0_stateless/00050_any_left_join.sql b/dbms/tests/queries/0_stateless/00050_any_left_join.sql index 2440a9f93ab..61ccb200307 100644 --- a/dbms/tests/queries/0_stateless/00050_any_left_join.sql +++ b/dbms/tests/queries/0_stateless/00050_any_left_join.sql @@ -1,9 +1,9 @@ -SELECT * FROM +SELECT a.*, b.* FROM ( SELECT number AS k FROM system.numbers LIMIT 10 -) +) AS a ANY LEFT JOIN ( SELECT number * 2 AS k, number AS joined FROM system.numbers LIMIT 10 -) +) AS b USING k; diff --git a/dbms/tests/queries/0_stateless/00051_any_inner_join.reference b/dbms/tests/queries/0_stateless/00051_any_inner_join.reference index 2a61b78f6ea..a7949c6a350 100644 --- a/dbms/tests/queries/0_stateless/00051_any_inner_join.reference +++ b/dbms/tests/queries/0_stateless/00051_any_inner_join.reference @@ -1,5 +1,5 @@ -0 0 -2 1 -4 2 -6 3 -8 4 +0 0 0 +2 2 1 +4 4 2 +6 6 3 +8 8 4 diff --git a/dbms/tests/queries/0_stateless/00051_any_inner_join.sql b/dbms/tests/queries/0_stateless/00051_any_inner_join.sql index a3ff3c437cf..986c798d763 100644 --- a/dbms/tests/queries/0_stateless/00051_any_inner_join.sql +++ b/dbms/tests/queries/0_stateless/00051_any_inner_join.sql @@ -1,9 +1,9 @@ -SELECT * FROM +SELECT a.*, b.* FROM ( SELECT number AS k FROM system.numbers LIMIT 10 -) +) AS a ANY INNER JOIN ( SELECT number * 2 AS k, number AS joined FROM system.numbers LIMIT 10 -) +) AS b USING k; diff --git a/dbms/tests/queries/0_stateless/00053_all_inner_join.reference b/dbms/tests/queries/0_stateless/00053_all_inner_join.reference index 41707378e8d..15bed0fbe0c 100644 --- a/dbms/tests/queries/0_stateless/00053_all_inner_join.reference +++ b/dbms/tests/queries/0_stateless/00053_all_inner_join.reference @@ -1,10 +1,10 @@ -0 0 -0 1 -1 2 -1 3 -2 4 -2 5 -3 6 -3 7 -4 8 -4 9 +0 0 0 +0 1 1 +1 2 2 +1 3 3 +2 4 4 +2 0 5 +3 0 6 +3 0 7 +4 0 8 +4 0 9 diff --git a/dbms/tests/queries/0_stateless/00053_all_inner_join.sql b/dbms/tests/queries/0_stateless/00053_all_inner_join.sql index 18a94588033..b2e65c80574 100644 --- a/dbms/tests/queries/0_stateless/00053_all_inner_join.sql +++ b/dbms/tests/queries/0_stateless/00053_all_inner_join.sql @@ -1,9 +1,9 @@ -SELECT * FROM +SELECT a.*, b.* FROM ( SELECT number AS k FROM system.numbers LIMIT 10 -) +) AS a ALL INNER JOIN ( SELECT intDiv(number, 2) AS k, number AS joined FROM system.numbers LIMIT 10 -) +) AS b USING k; diff --git a/dbms/tests/queries/0_stateless/00054_join_string.sql b/dbms/tests/queries/0_stateless/00054_join_string.sql index 7912a08c1d6..00e1278d330 100644 --- a/dbms/tests/queries/0_stateless/00054_join_string.sql +++ b/dbms/tests/queries/0_stateless/00054_join_string.sql @@ -1,9 +1,9 @@ -SELECT * FROM +SELECT a.*, b.* FROM ( SELECT reinterpretAsString(number + reinterpretAsUInt8('A')) AS k FROM system.numbers LIMIT 10 -) +) AS a ALL LEFT JOIN ( SELECT reinterpretAsString(intDiv(number, 2) + reinterpretAsUInt8('A')) AS k, number AS joined FROM system.numbers LIMIT 10 -) +) AS b USING k; From 2d490acc3e8a5f70bbdbe3be524bb3388d25a887 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 24 Jan 2019 21:36:45 +0300 Subject: [PATCH 55/86] Added test --- .../00831_quantile_weighted_parameter_check.reference | 1 + .../0_stateless/00831_quantile_weighted_parameter_check.sql | 2 ++ 2 files changed, 3 insertions(+) create mode 100644 dbms/tests/queries/0_stateless/00831_quantile_weighted_parameter_check.reference create mode 100644 dbms/tests/queries/0_stateless/00831_quantile_weighted_parameter_check.sql diff --git a/dbms/tests/queries/0_stateless/00831_quantile_weighted_parameter_check.reference b/dbms/tests/queries/0_stateless/00831_quantile_weighted_parameter_check.reference new file mode 100644 index 00000000000..7f8f011eb73 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00831_quantile_weighted_parameter_check.reference @@ -0,0 +1 @@ +7 diff --git a/dbms/tests/queries/0_stateless/00831_quantile_weighted_parameter_check.sql b/dbms/tests/queries/0_stateless/00831_quantile_weighted_parameter_check.sql new file mode 100644 index 00000000000..1d31b80f193 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00831_quantile_weighted_parameter_check.sql @@ -0,0 +1,2 @@ +SELECT quantileExactWeighted(0.5)(number, number) FROM numbers(10); +SELECT quantileExactWeighted(0.5)(number, 0.1) FROM numbers(10); -- { serverError 43 } From 1e04ef16a07f00f32fffd96a475c1ef4e6668511 Mon Sep 17 00:00:00 2001 From: chertus Date: Thu, 24 Jan 2019 21:55:18 +0300 Subject: [PATCH 56/86] one more test fix --- dbms/tests/queries/0_stateless/00054_join_string.sql | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/dbms/tests/queries/0_stateless/00054_join_string.sql b/dbms/tests/queries/0_stateless/00054_join_string.sql index 00e1278d330..7912a08c1d6 100644 --- a/dbms/tests/queries/0_stateless/00054_join_string.sql +++ b/dbms/tests/queries/0_stateless/00054_join_string.sql @@ -1,9 +1,9 @@ -SELECT a.*, b.* FROM +SELECT * FROM ( SELECT reinterpretAsString(number + reinterpretAsUInt8('A')) AS k FROM system.numbers LIMIT 10 -) AS a +) ALL LEFT JOIN ( SELECT reinterpretAsString(intDiv(number, 2) + reinterpretAsUInt8('A')) AS k, number AS joined FROM system.numbers LIMIT 10 -) AS b +) USING k; From 04fa827e4b26f872e482a25bb273dfda816ca606 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 24 Jan 2019 22:21:47 +0300 Subject: [PATCH 57/86] Style --- dbms/src/Common/SpaceSaving.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/src/Common/SpaceSaving.h b/dbms/src/Common/SpaceSaving.h index feba4414a54..6e446d1c938 100644 --- a/dbms/src/Common/SpaceSaving.h +++ b/dbms/src/Common/SpaceSaving.h @@ -79,7 +79,7 @@ private: constexpr uint64_t nextAlphaSize(uint64_t x) { constexpr uint64_t ALPHA_MAP_ELEMENTS_PER_COUNTER = 6; - return 1ULL<<(sizeof(uint64_t) * 8 - __builtin_clzll(x * ALPHA_MAP_ELEMENTS_PER_COUNTER)); + return 1ULL << (sizeof(uint64_t) * 8 - __builtin_clzll(x * ALPHA_MAP_ELEMENTS_PER_COUNTER)); } public: From 854817c232ddf60f6f22eb87420e3b5a82705465 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 24 Jan 2019 22:22:26 +0300 Subject: [PATCH 58/86] Fixed -Wshadow-field #4005 --- dbms/programs/client/Client.cpp | 4 ++-- dbms/src/Interpreters/executeQuery.cpp | 4 ++-- dbms/src/Parsers/ASTQueryWithOutput.cpp | 12 ++++++------ dbms/src/Parsers/ASTQueryWithOutput.h | 2 +- dbms/src/Parsers/ParserQueryWithOutput.cpp | 4 ++-- 5 files changed, 13 insertions(+), 13 deletions(-) diff --git a/dbms/programs/client/Client.cpp b/dbms/programs/client/Client.cpp index 4c69d5d3e6a..74b1db69896 100644 --- a/dbms/programs/client/Client.cpp +++ b/dbms/programs/client/Client.cpp @@ -1210,9 +1210,9 @@ private: const auto & id = typeid_cast(*query_with_output->format); current_format = id.name; } - if (query_with_output->settings) + if (query_with_output->settings_ast) { - InterpreterSetQuery(query_with_output->settings, context).executeForCurrentContext(); + InterpreterSetQuery(query_with_output->settings_ast, context).executeForCurrentContext(); } } diff --git a/dbms/src/Interpreters/executeQuery.cpp b/dbms/src/Interpreters/executeQuery.cpp index a2a6bb46035..6a21437399b 100644 --- a/dbms/src/Interpreters/executeQuery.cpp +++ b/dbms/src/Interpreters/executeQuery.cpp @@ -499,8 +499,8 @@ void executeQuery( ? *getIdentifierName(ast_query_with_output->format) : context.getDefaultFormat(); - if (ast_query_with_output && ast_query_with_output->settings) - InterpreterSetQuery(ast_query_with_output->settings, context).executeForCurrentContext(); + if (ast_query_with_output && ast_query_with_output->settings_ast) + InterpreterSetQuery(ast_query_with_output->settings_ast, context).executeForCurrentContext(); BlockOutputStreamPtr out = context.getOutputFormat(format_name, *out_buf, streams.in->getHeader()); diff --git a/dbms/src/Parsers/ASTQueryWithOutput.cpp b/dbms/src/Parsers/ASTQueryWithOutput.cpp index fd6957c436b..c28b15cb8c1 100644 --- a/dbms/src/Parsers/ASTQueryWithOutput.cpp +++ b/dbms/src/Parsers/ASTQueryWithOutput.cpp @@ -15,10 +15,10 @@ void ASTQueryWithOutput::cloneOutputOptions(ASTQueryWithOutput & cloned) const cloned.format = format->clone(); cloned.children.push_back(cloned.format); } - if (settings) + if (settings_ast) { - cloned.settings = settings->clone(); - cloned.children.push_back(cloned.settings); + cloned.settings_ast = settings_ast->clone(); + cloned.children.push_back(cloned.settings_ast); } } @@ -40,10 +40,10 @@ void ASTQueryWithOutput::formatImpl(const FormatSettings & s, FormatState & stat format->formatImpl(s, state, frame); } - if (settings) + if (settings_ast) { s.ostr << (s.hilite ? hilite_keyword : "") << s.nl_or_ws << indent_str << "SETTINGS " << (s.hilite ? hilite_none : ""); - settings->formatImpl(s, state, frame); + settings_ast->formatImpl(s, state, frame); } } @@ -53,7 +53,7 @@ bool ASTQueryWithOutput::resetOutputASTIfExist(IAST & ast) { ast_with_output->format.reset(); ast_with_output->out_file.reset(); - ast_with_output->settings.reset(); + ast_with_output->settings_ast.reset(); return true; } diff --git a/dbms/src/Parsers/ASTQueryWithOutput.h b/dbms/src/Parsers/ASTQueryWithOutput.h index ec525b3aca9..9018d5661d9 100644 --- a/dbms/src/Parsers/ASTQueryWithOutput.h +++ b/dbms/src/Parsers/ASTQueryWithOutput.h @@ -14,7 +14,7 @@ class ASTQueryWithOutput : public IAST public: ASTPtr out_file; ASTPtr format; - ASTPtr settings; + ASTPtr settings_ast; void formatImpl(const FormatSettings & s, FormatState & state, FormatStateStacked frame) const final; diff --git a/dbms/src/Parsers/ParserQueryWithOutput.cpp b/dbms/src/Parsers/ParserQueryWithOutput.cpp index 830060cb953..91e320deeeb 100644 --- a/dbms/src/Parsers/ParserQueryWithOutput.cpp +++ b/dbms/src/Parsers/ParserQueryWithOutput.cpp @@ -87,9 +87,9 @@ bool ParserQueryWithOutput::parseImpl(Pos & pos, ASTPtr & node, Expected & expec if (s_settings.ignore(pos, expected)) { ParserSetQuery parser_settings(true); - if (!parser_settings.parse(pos, query_with_output.settings, expected)) + if (!parser_settings.parse(pos, query_with_output.settings_ast, expected)) return false; - query_with_output.children.push_back(query_with_output.settings); + query_with_output.children.push_back(query_with_output.settings_ast); } if (explain_ast) From a28f0acf09c448b2fc2f531170cf0ec0ef309895 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 24 Jan 2019 22:49:36 +0300 Subject: [PATCH 59/86] Removed non-breaking spaces: find dbms -name '*.h' -or -name '*.cpp' | xargs grep -l $'\xc2\xa0' | xargs sed -i -r -e 's/\xc2\xa0/ /' --- .../AggregateFunctions/UniqCombinedBiasData.h | 38 ++++++------- dbms/src/AggregateFunctions/UniquesHashSet.h | 54 +++++++++---------- dbms/src/Client/tests/test_connect.cpp | 6 +-- dbms/src/Common/InterruptListener.h | 8 +-- 4 files changed, 53 insertions(+), 53 deletions(-) diff --git a/dbms/src/AggregateFunctions/UniqCombinedBiasData.h b/dbms/src/AggregateFunctions/UniqCombinedBiasData.h index 7206aabcf7e..36d4c5fb99e 100644 --- a/dbms/src/AggregateFunctions/UniqCombinedBiasData.h +++ b/dbms/src/AggregateFunctions/UniqCombinedBiasData.h @@ -6,25 +6,25 @@ namespace DB { /** Data for HyperLogLogBiasEstimator in the uniqCombined function. -  * The development plan is as follows: -  * 1. Assemble ClickHouse. -  * 2. Run the script src/dbms/scripts/gen-bias-data.py, which returns one array for getRawEstimates() -  * and another array for getBiases(). -  * 3. Update `raw_estimates` and `biases` arrays. Also update the size of arrays in InterpolatedData. -  * 4. Assemble ClickHouse. -  * 5. Run the script src/dbms/scripts/linear-counting-threshold.py, which creates 3 files: -  * - raw_graph.txt (1st column: the present number of unique values; -  * 2nd column: relative error in the case of HyperLogLog without applying any corrections) -  * - linear_counting_graph.txt (1st column: the present number of unique values; -  * 2nd column: relative error in the case of HyperLogLog using LinearCounting) -  * - bias_corrected_graph.txt (1st column: the present number of unique values; -  * 2nd column: relative error in the case of HyperLogLog with the use of corrections from the algorithm HyperLogLog++) -  * 6. Generate a graph with gnuplot based on this data. -  * 7. Determine the minimum number of unique values at which it is better to correct the error -  * using its evaluation (ie, using the HyperLogLog++ algorithm) than applying the LinearCounting algorithm. -  * 7. Accordingly, update the constant in the function getThreshold() -  * 8. Assemble ClickHouse. -  */ +  * The development plan is as follows: +  * 1. Assemble ClickHouse. +  * 2. Run the script src/dbms/scripts/gen-bias-data.py, which returns one array for getRawEstimates() +  * and another array for getBiases(). +  * 3. Update `raw_estimates` and `biases` arrays. Also update the size of arrays in InterpolatedData. +  * 4. Assemble ClickHouse. +  * 5. Run the script src/dbms/scripts/linear-counting-threshold.py, which creates 3 files: +  * - raw_graph.txt (1st column: the present number of unique values; +  * 2nd column: relative error in the case of HyperLogLog without applying any corrections) +  * - linear_counting_graph.txt (1st column: the present number of unique values; +  * 2nd column: relative error in the case of HyperLogLog using LinearCounting) +  * - bias_corrected_graph.txt (1st column: the present number of unique values; +  * 2nd column: relative error in the case of HyperLogLog with the use of corrections from the algorithm HyperLogLog++) +  * 6. Generate a graph with gnuplot based on this data. +  * 7. Determine the minimum number of unique values at which it is better to correct the error +  * using its evaluation (ie, using the HyperLogLog++ algorithm) than applying the LinearCounting algorithm. +  * 7. Accordingly, update the constant in the function getThreshold() +  * 8. Assemble ClickHouse. +  */ struct UniqCombinedBiasData { using InterpolatedData = std::array; diff --git a/dbms/src/AggregateFunctions/UniquesHashSet.h b/dbms/src/AggregateFunctions/UniquesHashSet.h index b0036d4832d..6f3bdc4e7f4 100644 --- a/dbms/src/AggregateFunctions/UniquesHashSet.h +++ b/dbms/src/AggregateFunctions/UniquesHashSet.h @@ -15,33 +15,33 @@ /** Approximate calculation of anything, as usual, is constructed according to the following scheme: -  * - some data structure is used to calculate the value of X; -  * - Not all values are added to the data structure, but only selected ones (according to some selectivity criteria); -  * - after processing all elements, the data structure is in some state S; -  * - as an approximate value of X, the value calculated according to the maximum likelihood principle is returned: -  * at what real value X, the probability of finding the data structure in the obtained state S is maximal. -  */ +  * - some data structure is used to calculate the value of X; +  * - Not all values are added to the data structure, but only selected ones (according to some selectivity criteria); +  * - after processing all elements, the data structure is in some state S; +  * - as an approximate value of X, the value calculated according to the maximum likelihood principle is returned: +  * at what real value X, the probability of finding the data structure in the obtained state S is maximal. +  */ /** In particular, what is described below can be found by the name of the BJKST algorithm. -  */ +  */ /** Very simple hash-set for approximate number of unique values. -  * Works like this: -  * - you can insert UInt64; -  * - before insertion, first the hash function UInt64 -> UInt32 is calculated; -  * - the original value is not saved (lost); -  * - further all operations are made with these hashes; -  * - hash table is constructed according to the scheme: -  * - open addressing (one buffer, position in buffer is calculated by taking remainder of division by its size); -  * - linear probing (if the cell already has a value, then the cell following it is taken, etc.); -  * - the missing value is zero-encoded; to remember presence of zero in set, separate variable of type bool is used; -  * - buffer growth by 2 times when filling more than 50%; -  * - if the set has more UNIQUES_HASH_MAX_SIZE elements, then all the elements are removed from the set, -  * not divisible by 2, and then all elements that do not divide by 2 are not inserted into the set; -  * - if the situation repeats, then only elements dividing by 4, etc., are taken. -  * - the size() method returns an approximate number of elements that have been inserted into the set; -  * - there are methods for quick reading and writing in binary and text form. -  */ +  * Works like this: +  * - you can insert UInt64; +  * - before insertion, first the hash function UInt64 -> UInt32 is calculated; +  * - the original value is not saved (lost); +  * - further all operations are made with these hashes; +  * - hash table is constructed according to the scheme: +  * - open addressing (one buffer, position in buffer is calculated by taking remainder of division by its size); +  * - linear probing (if the cell already has a value, then the cell following it is taken, etc.); +  * - the missing value is zero-encoded; to remember presence of zero in set, separate variable of type bool is used; +  * - buffer growth by 2 times when filling more than 50%; +  * - if the set has more UNIQUES_HASH_MAX_SIZE elements, then all the elements are removed from the set, +  * not divisible by 2, and then all elements that do not divide by 2 are not inserted into the set; +  * - if the situation repeats, then only elements dividing by 4, etc., are taken. +  * - the size() method returns an approximate number of elements that have been inserted into the set; +  * - there are methods for quick reading and writing in binary and text form. +  */ /// The maximum degree of buffer size before the values are discarded #define UNIQUES_HASH_MAX_SIZE_DEGREE 17 @@ -50,8 +50,8 @@ #define UNIQUES_HASH_MAX_SIZE (1ULL << (UNIQUES_HASH_MAX_SIZE_DEGREE - 1)) /** The number of least significant bits used for thinning. The remaining high-order bits are used to determine the position in the hash table. -  * (high-order bits are taken because the younger bits will be constant after dropping some of the values) -  */ +  * (high-order bits are taken because the younger bits will be constant after dropping some of the values) +  */ #define UNIQUES_HASH_BITS_FOR_SKIP (32 - UNIQUES_HASH_MAX_SIZE_DEGREE) /// Initial buffer size degree @@ -59,8 +59,8 @@ /** This hash function is not the most optimal, but UniquesHashSet states counted with it, -  * stored in many places on disks (in the Yandex.Metrika), so it continues to be used. -  */ +  * stored in many places on disks (in the Yandex.Metrika), so it continues to be used. +  */ struct UniquesHashSetDefaultHash { size_t operator() (UInt64 x) const diff --git a/dbms/src/Client/tests/test_connect.cpp b/dbms/src/Client/tests/test_connect.cpp index 75eb606cc97..50075cc24a6 100644 --- a/dbms/src/Client/tests/test_connect.cpp +++ b/dbms/src/Client/tests/test_connect.cpp @@ -9,9 +9,9 @@ /** 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. -  */ + * 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 { diff --git a/dbms/src/Common/InterruptListener.h b/dbms/src/Common/InterruptListener.h index eb818671037..bcb7c18c6fb 100644 --- a/dbms/src/Common/InterruptListener.h +++ b/dbms/src/Common/InterruptListener.h @@ -45,10 +45,10 @@ static int sigtimedwait(const sigset_t *set, siginfo_t *info, const struct times /** As long as there exists an object of this class - it blocks the INT signal, at the same time it lets you know if it came. -  * This is necessary so that you can interrupt the execution of the request with Ctrl+C. -  * Use only one instance of this class at a time. -  * If `check` method returns true (the signal has arrived), the next call will wait for the next signal. -  */ +  * This is necessary so that you can interrupt the execution of the request with Ctrl+C. +  * Use only one instance of this class at a time. +  * If `check` method returns true (the signal has arrived), the next call will wait for the next signal. +  */ class InterruptListener { private: From 304c7b03861cca72d9a0cee56468c8f54e5bb414 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 24 Jan 2019 22:50:16 +0300 Subject: [PATCH 60/86] Style #4005 --- dbms/src/Formats/ProtobufWriter.cpp | 12 ------------ 1 file changed, 12 deletions(-) diff --git a/dbms/src/Formats/ProtobufWriter.cpp b/dbms/src/Formats/ProtobufWriter.cpp index d17754e3478..1f2d200a7ca 100644 --- a/dbms/src/Formats/ProtobufWriter.cpp +++ b/dbms/src/Formats/ProtobufWriter.cpp @@ -21,8 +21,6 @@ namespace ErrorCodes } -// Converter ----------------------------------------------------------------------------------------------------------- - class ProtobufWriter::Converter : private boost::noncopyable { public: @@ -128,8 +126,6 @@ protected: }; -// ToStringConverter --------------------------------------------------------------------------------------------------- - class ProtobufWriter::ToStringConverter : public Converter { public: @@ -232,8 +228,6 @@ private: }; -// ToNumberConverter --------------------------------------------------------------------------------------------------- - template class ProtobufWriter::ToNumberConverter : public Converter { @@ -396,8 +390,6 @@ private: }; -// ToBoolConverter ----------------------------------------------------------------------------------------------------- - class ProtobufWriter::ToBoolConverter : public Converter { public: @@ -451,8 +443,6 @@ private: }; -// ToEnumConverter ----------------------------------------------------------------------------------------------------- - class ProtobufWriter::ToEnumConverter : public Converter { public: @@ -552,8 +542,6 @@ private: }; -// ProtobufWriter ------------------------------------------------------------------------------------------------------ - ProtobufWriter::ProtobufWriter(WriteBuffer & out, const google::protobuf::Descriptor * message_type) : simple_writer(out) { enumerateFieldsInWriteOrder(message_type); From 5ccc42fd96fcf177ebb2c81596216e6e7f46a1ae Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 25 Jan 2019 01:59:45 +0300 Subject: [PATCH 61/86] Revert "Removed non-breaking spaces: find dbms -name '*.h' -or -name '*.cpp' | xargs grep -l $'\xc2\xa0' | xargs sed -i -r -e 's/\xc2\xa0/ /'" This reverts commit a28f0acf09c448b2fc2f531170cf0ec0ef309895. --- .../AggregateFunctions/UniqCombinedBiasData.h | 38 ++++++------- dbms/src/AggregateFunctions/UniquesHashSet.h | 54 +++++++++---------- dbms/src/Client/tests/test_connect.cpp | 6 +-- dbms/src/Common/InterruptListener.h | 8 +-- 4 files changed, 53 insertions(+), 53 deletions(-) diff --git a/dbms/src/AggregateFunctions/UniqCombinedBiasData.h b/dbms/src/AggregateFunctions/UniqCombinedBiasData.h index 36d4c5fb99e..7206aabcf7e 100644 --- a/dbms/src/AggregateFunctions/UniqCombinedBiasData.h +++ b/dbms/src/AggregateFunctions/UniqCombinedBiasData.h @@ -6,25 +6,25 @@ namespace DB { /** Data for HyperLogLogBiasEstimator in the uniqCombined function. -  * The development plan is as follows: -  * 1. Assemble ClickHouse. -  * 2. Run the script src/dbms/scripts/gen-bias-data.py, which returns one array for getRawEstimates() -  * and another array for getBiases(). -  * 3. Update `raw_estimates` and `biases` arrays. Also update the size of arrays in InterpolatedData. -  * 4. Assemble ClickHouse. -  * 5. Run the script src/dbms/scripts/linear-counting-threshold.py, which creates 3 files: -  * - raw_graph.txt (1st column: the present number of unique values; -  * 2nd column: relative error in the case of HyperLogLog without applying any corrections) -  * - linear_counting_graph.txt (1st column: the present number of unique values; -  * 2nd column: relative error in the case of HyperLogLog using LinearCounting) -  * - bias_corrected_graph.txt (1st column: the present number of unique values; -  * 2nd column: relative error in the case of HyperLogLog with the use of corrections from the algorithm HyperLogLog++) -  * 6. Generate a graph with gnuplot based on this data. -  * 7. Determine the minimum number of unique values at which it is better to correct the error -  * using its evaluation (ie, using the HyperLogLog++ algorithm) than applying the LinearCounting algorithm. -  * 7. Accordingly, update the constant in the function getThreshold() -  * 8. Assemble ClickHouse. -  */ +  * The development plan is as follows: +  * 1. Assemble ClickHouse. +  * 2. Run the script src/dbms/scripts/gen-bias-data.py, which returns one array for getRawEstimates() +  * and another array for getBiases(). +  * 3. Update `raw_estimates` and `biases` arrays. Also update the size of arrays in InterpolatedData. +  * 4. Assemble ClickHouse. +  * 5. Run the script src/dbms/scripts/linear-counting-threshold.py, which creates 3 files: +  * - raw_graph.txt (1st column: the present number of unique values; +  * 2nd column: relative error in the case of HyperLogLog without applying any corrections) +  * - linear_counting_graph.txt (1st column: the present number of unique values; +  * 2nd column: relative error in the case of HyperLogLog using LinearCounting) +  * - bias_corrected_graph.txt (1st column: the present number of unique values; +  * 2nd column: relative error in the case of HyperLogLog with the use of corrections from the algorithm HyperLogLog++) +  * 6. Generate a graph with gnuplot based on this data. +  * 7. Determine the minimum number of unique values at which it is better to correct the error +  * using its evaluation (ie, using the HyperLogLog++ algorithm) than applying the LinearCounting algorithm. +  * 7. Accordingly, update the constant in the function getThreshold() +  * 8. Assemble ClickHouse. +  */ struct UniqCombinedBiasData { using InterpolatedData = std::array; diff --git a/dbms/src/AggregateFunctions/UniquesHashSet.h b/dbms/src/AggregateFunctions/UniquesHashSet.h index 6f3bdc4e7f4..b0036d4832d 100644 --- a/dbms/src/AggregateFunctions/UniquesHashSet.h +++ b/dbms/src/AggregateFunctions/UniquesHashSet.h @@ -15,33 +15,33 @@ /** Approximate calculation of anything, as usual, is constructed according to the following scheme: -  * - some data structure is used to calculate the value of X; -  * - Not all values are added to the data structure, but only selected ones (according to some selectivity criteria); -  * - after processing all elements, the data structure is in some state S; -  * - as an approximate value of X, the value calculated according to the maximum likelihood principle is returned: -  * at what real value X, the probability of finding the data structure in the obtained state S is maximal. -  */ +  * - some data structure is used to calculate the value of X; +  * - Not all values are added to the data structure, but only selected ones (according to some selectivity criteria); +  * - after processing all elements, the data structure is in some state S; +  * - as an approximate value of X, the value calculated according to the maximum likelihood principle is returned: +  * at what real value X, the probability of finding the data structure in the obtained state S is maximal. +  */ /** In particular, what is described below can be found by the name of the BJKST algorithm. -  */ +  */ /** Very simple hash-set for approximate number of unique values. -  * Works like this: -  * - you can insert UInt64; -  * - before insertion, first the hash function UInt64 -> UInt32 is calculated; -  * - the original value is not saved (lost); -  * - further all operations are made with these hashes; -  * - hash table is constructed according to the scheme: -  * - open addressing (one buffer, position in buffer is calculated by taking remainder of division by its size); -  * - linear probing (if the cell already has a value, then the cell following it is taken, etc.); -  * - the missing value is zero-encoded; to remember presence of zero in set, separate variable of type bool is used; -  * - buffer growth by 2 times when filling more than 50%; -  * - if the set has more UNIQUES_HASH_MAX_SIZE elements, then all the elements are removed from the set, -  * not divisible by 2, and then all elements that do not divide by 2 are not inserted into the set; -  * - if the situation repeats, then only elements dividing by 4, etc., are taken. -  * - the size() method returns an approximate number of elements that have been inserted into the set; -  * - there are methods for quick reading and writing in binary and text form. -  */ +  * Works like this: +  * - you can insert UInt64; +  * - before insertion, first the hash function UInt64 -> UInt32 is calculated; +  * - the original value is not saved (lost); +  * - further all operations are made with these hashes; +  * - hash table is constructed according to the scheme: +  * - open addressing (one buffer, position in buffer is calculated by taking remainder of division by its size); +  * - linear probing (if the cell already has a value, then the cell following it is taken, etc.); +  * - the missing value is zero-encoded; to remember presence of zero in set, separate variable of type bool is used; +  * - buffer growth by 2 times when filling more than 50%; +  * - if the set has more UNIQUES_HASH_MAX_SIZE elements, then all the elements are removed from the set, +  * not divisible by 2, and then all elements that do not divide by 2 are not inserted into the set; +  * - if the situation repeats, then only elements dividing by 4, etc., are taken. +  * - the size() method returns an approximate number of elements that have been inserted into the set; +  * - there are methods for quick reading and writing in binary and text form. +  */ /// The maximum degree of buffer size before the values are discarded #define UNIQUES_HASH_MAX_SIZE_DEGREE 17 @@ -50,8 +50,8 @@ #define UNIQUES_HASH_MAX_SIZE (1ULL << (UNIQUES_HASH_MAX_SIZE_DEGREE - 1)) /** The number of least significant bits used for thinning. The remaining high-order bits are used to determine the position in the hash table. -  * (high-order bits are taken because the younger bits will be constant after dropping some of the values) -  */ +  * (high-order bits are taken because the younger bits will be constant after dropping some of the values) +  */ #define UNIQUES_HASH_BITS_FOR_SKIP (32 - UNIQUES_HASH_MAX_SIZE_DEGREE) /// Initial buffer size degree @@ -59,8 +59,8 @@ /** This hash function is not the most optimal, but UniquesHashSet states counted with it, -  * stored in many places on disks (in the Yandex.Metrika), so it continues to be used. -  */ +  * stored in many places on disks (in the Yandex.Metrika), so it continues to be used. +  */ struct UniquesHashSetDefaultHash { size_t operator() (UInt64 x) const diff --git a/dbms/src/Client/tests/test_connect.cpp b/dbms/src/Client/tests/test_connect.cpp index 50075cc24a6..75eb606cc97 100644 --- a/dbms/src/Client/tests/test_connect.cpp +++ b/dbms/src/Client/tests/test_connect.cpp @@ -9,9 +9,9 @@ /** 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. - */ +  * 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 { diff --git a/dbms/src/Common/InterruptListener.h b/dbms/src/Common/InterruptListener.h index bcb7c18c6fb..eb818671037 100644 --- a/dbms/src/Common/InterruptListener.h +++ b/dbms/src/Common/InterruptListener.h @@ -45,10 +45,10 @@ static int sigtimedwait(const sigset_t *set, siginfo_t *info, const struct times /** As long as there exists an object of this class - it blocks the INT signal, at the same time it lets you know if it came. -  * This is necessary so that you can interrupt the execution of the request with Ctrl+C. -  * Use only one instance of this class at a time. -  * If `check` method returns true (the signal has arrived), the next call will wait for the next signal. -  */ +  * This is necessary so that you can interrupt the execution of the request with Ctrl+C. +  * Use only one instance of this class at a time. +  * If `check` method returns true (the signal has arrived), the next call will wait for the next signal. +  */ class InterruptListener { private: From ef50601b5ceeeaf5763eab6c0013954c12eb00b1 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 25 Jan 2019 02:01:56 +0300 Subject: [PATCH 62/86] Removed non-breaking spaces: find dbms -name '*.h' -or -name '*.cpp' | xargs grep -l $'\xc2\xa0' | xargs sed -i -r -e 's/\xc2\xa0/ /g' --- .../AggregateFunctions/UniqCombinedBiasData.h | 38 ++++++------- dbms/src/AggregateFunctions/UniquesHashSet.h | 54 +++++++++---------- dbms/src/Client/tests/test_connect.cpp | 6 +-- dbms/src/Common/InterruptListener.h | 8 +-- 4 files changed, 53 insertions(+), 53 deletions(-) diff --git a/dbms/src/AggregateFunctions/UniqCombinedBiasData.h b/dbms/src/AggregateFunctions/UniqCombinedBiasData.h index 7206aabcf7e..0a69a211206 100644 --- a/dbms/src/AggregateFunctions/UniqCombinedBiasData.h +++ b/dbms/src/AggregateFunctions/UniqCombinedBiasData.h @@ -6,25 +6,25 @@ namespace DB { /** Data for HyperLogLogBiasEstimator in the uniqCombined function. -  * The development plan is as follows: -  * 1. Assemble ClickHouse. -  * 2. Run the script src/dbms/scripts/gen-bias-data.py, which returns one array for getRawEstimates() -  * and another array for getBiases(). -  * 3. Update `raw_estimates` and `biases` arrays. Also update the size of arrays in InterpolatedData. -  * 4. Assemble ClickHouse. -  * 5. Run the script src/dbms/scripts/linear-counting-threshold.py, which creates 3 files: -  * - raw_graph.txt (1st column: the present number of unique values; -  * 2nd column: relative error in the case of HyperLogLog without applying any corrections) -  * - linear_counting_graph.txt (1st column: the present number of unique values; -  * 2nd column: relative error in the case of HyperLogLog using LinearCounting) -  * - bias_corrected_graph.txt (1st column: the present number of unique values; -  * 2nd column: relative error in the case of HyperLogLog with the use of corrections from the algorithm HyperLogLog++) -  * 6. Generate a graph with gnuplot based on this data. -  * 7. Determine the minimum number of unique values at which it is better to correct the error -  * using its evaluation (ie, using the HyperLogLog++ algorithm) than applying the LinearCounting algorithm. -  * 7. Accordingly, update the constant in the function getThreshold() -  * 8. Assemble ClickHouse. -  */ + * The development plan is as follows: + * 1. Assemble ClickHouse. + * 2. Run the script src/dbms/scripts/gen-bias-data.py, which returns one array for getRawEstimates() + * and another array for getBiases(). + * 3. Update `raw_estimates` and `biases` arrays. Also update the size of arrays in InterpolatedData. + * 4. Assemble ClickHouse. + * 5. Run the script src/dbms/scripts/linear-counting-threshold.py, which creates 3 files: + * - raw_graph.txt (1st column: the present number of unique values; + * 2nd column: relative error in the case of HyperLogLog without applying any corrections) + * - linear_counting_graph.txt (1st column: the present number of unique values; + * 2nd column: relative error in the case of HyperLogLog using LinearCounting) + * - bias_corrected_graph.txt (1st column: the present number of unique values; + * 2nd column: relative error in the case of HyperLogLog with the use of corrections from the algorithm HyperLogLog++) + * 6. Generate a graph with gnuplot based on this data. + * 7. Determine the minimum number of unique values at which it is better to correct the error + * using its evaluation (ie, using the HyperLogLog++ algorithm) than applying the LinearCounting algorithm. + * 7. Accordingly, update the constant in the function getThreshold() + * 8. Assemble ClickHouse. + */ struct UniqCombinedBiasData { using InterpolatedData = std::array; diff --git a/dbms/src/AggregateFunctions/UniquesHashSet.h b/dbms/src/AggregateFunctions/UniquesHashSet.h index b0036d4832d..bc9a65c1bb6 100644 --- a/dbms/src/AggregateFunctions/UniquesHashSet.h +++ b/dbms/src/AggregateFunctions/UniquesHashSet.h @@ -15,33 +15,33 @@ /** Approximate calculation of anything, as usual, is constructed according to the following scheme: -  * - some data structure is used to calculate the value of X; -  * - Not all values are added to the data structure, but only selected ones (according to some selectivity criteria); -  * - after processing all elements, the data structure is in some state S; -  * - as an approximate value of X, the value calculated according to the maximum likelihood principle is returned: -  * at what real value X, the probability of finding the data structure in the obtained state S is maximal. -  */ + * - some data structure is used to calculate the value of X; + * - Not all values are added to the data structure, but only selected ones (according to some selectivity criteria); + * - after processing all elements, the data structure is in some state S; + * - as an approximate value of X, the value calculated according to the maximum likelihood principle is returned: + * at what real value X, the probability of finding the data structure in the obtained state S is maximal. + */ /** In particular, what is described below can be found by the name of the BJKST algorithm. -  */ + */ /** Very simple hash-set for approximate number of unique values. -  * Works like this: -  * - you can insert UInt64; -  * - before insertion, first the hash function UInt64 -> UInt32 is calculated; -  * - the original value is not saved (lost); -  * - further all operations are made with these hashes; -  * - hash table is constructed according to the scheme: -  * - open addressing (one buffer, position in buffer is calculated by taking remainder of division by its size); -  * - linear probing (if the cell already has a value, then the cell following it is taken, etc.); -  * - the missing value is zero-encoded; to remember presence of zero in set, separate variable of type bool is used; -  * - buffer growth by 2 times when filling more than 50%; -  * - if the set has more UNIQUES_HASH_MAX_SIZE elements, then all the elements are removed from the set, -  * not divisible by 2, and then all elements that do not divide by 2 are not inserted into the set; -  * - if the situation repeats, then only elements dividing by 4, etc., are taken. -  * - the size() method returns an approximate number of elements that have been inserted into the set; -  * - there are methods for quick reading and writing in binary and text form. -  */ + * Works like this: + * - you can insert UInt64; + * - before insertion, first the hash function UInt64 -> UInt32 is calculated; + * - the original value is not saved (lost); + * - further all operations are made with these hashes; + * - hash table is constructed according to the scheme: + * - open addressing (one buffer, position in buffer is calculated by taking remainder of division by its size); + * - linear probing (if the cell already has a value, then the cell following it is taken, etc.); + * - the missing value is zero-encoded; to remember presence of zero in set, separate variable of type bool is used; + * - buffer growth by 2 times when filling more than 50%; + * - if the set has more UNIQUES_HASH_MAX_SIZE elements, then all the elements are removed from the set, + * not divisible by 2, and then all elements that do not divide by 2 are not inserted into the set; + * - if the situation repeats, then only elements dividing by 4, etc., are taken. + * - the size() method returns an approximate number of elements that have been inserted into the set; + * - there are methods for quick reading and writing in binary and text form. + */ /// The maximum degree of buffer size before the values are discarded #define UNIQUES_HASH_MAX_SIZE_DEGREE 17 @@ -50,8 +50,8 @@ #define UNIQUES_HASH_MAX_SIZE (1ULL << (UNIQUES_HASH_MAX_SIZE_DEGREE - 1)) /** The number of least significant bits used for thinning. The remaining high-order bits are used to determine the position in the hash table. -  * (high-order bits are taken because the younger bits will be constant after dropping some of the values) -  */ + * (high-order bits are taken because the younger bits will be constant after dropping some of the values) + */ #define UNIQUES_HASH_BITS_FOR_SKIP (32 - UNIQUES_HASH_MAX_SIZE_DEGREE) /// Initial buffer size degree @@ -59,8 +59,8 @@ /** This hash function is not the most optimal, but UniquesHashSet states counted with it, -  * stored in many places on disks (in the Yandex.Metrika), so it continues to be used. -  */ + * stored in many places on disks (in the Yandex.Metrika), so it continues to be used. + */ struct UniquesHashSetDefaultHash { size_t operator() (UInt64 x) const diff --git a/dbms/src/Client/tests/test_connect.cpp b/dbms/src/Client/tests/test_connect.cpp index 75eb606cc97..50075cc24a6 100644 --- a/dbms/src/Client/tests/test_connect.cpp +++ b/dbms/src/Client/tests/test_connect.cpp @@ -9,9 +9,9 @@ /** 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. -  */ + * 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 { diff --git a/dbms/src/Common/InterruptListener.h b/dbms/src/Common/InterruptListener.h index eb818671037..37ec0533dd6 100644 --- a/dbms/src/Common/InterruptListener.h +++ b/dbms/src/Common/InterruptListener.h @@ -45,10 +45,10 @@ static int sigtimedwait(const sigset_t *set, siginfo_t *info, const struct times /** As long as there exists an object of this class - it blocks the INT signal, at the same time it lets you know if it came. -  * This is necessary so that you can interrupt the execution of the request with Ctrl+C. -  * Use only one instance of this class at a time. -  * If `check` method returns true (the signal has arrived), the next call will wait for the next signal. -  */ + * This is necessary so that you can interrupt the execution of the request with Ctrl+C. + * Use only one instance of this class at a time. + * If `check` method returns true (the signal has arrived), the next call will wait for the next signal. + */ class InterruptListener { private: From a0fafefc96bc6b569f0f55d0f97ed20c7e65bb85 Mon Sep 17 00:00:00 2001 From: chertus Date: Fri, 25 Jan 2019 14:43:19 +0300 Subject: [PATCH 63/86] minor change - add Aliases.h --- dbms/src/Interpreters/Aliases.h | 15 +++++++++++++++ dbms/src/Interpreters/ArrayJoinedColumnsVisitor.h | 3 +-- .../OptimizeIfWithConstantConditionVisitor.h | 6 +----- dbms/src/Interpreters/QueryAliasesVisitor.h | 4 +--- dbms/src/Interpreters/QueryNormalizer.h | 4 ++-- dbms/src/Interpreters/SyntaxAnalyzer.h | 2 +- 6 files changed, 21 insertions(+), 13 deletions(-) create mode 100644 dbms/src/Interpreters/Aliases.h diff --git a/dbms/src/Interpreters/Aliases.h b/dbms/src/Interpreters/Aliases.h new file mode 100644 index 00000000000..80976c7551f --- /dev/null +++ b/dbms/src/Interpreters/Aliases.h @@ -0,0 +1,15 @@ +#pragma once + +#include + +#include + +namespace DB +{ + +class IAST; +using ASTPtr = std::shared_ptr; + +using Aliases = std::unordered_map; + +} diff --git a/dbms/src/Interpreters/ArrayJoinedColumnsVisitor.h b/dbms/src/Interpreters/ArrayJoinedColumnsVisitor.h index d345ee42dbe..0c19ffe7387 100644 --- a/dbms/src/Interpreters/ArrayJoinedColumnsVisitor.h +++ b/dbms/src/Interpreters/ArrayJoinedColumnsVisitor.h @@ -10,6 +10,7 @@ #include #include +#include namespace DB @@ -28,8 +29,6 @@ class ArrayJoinedColumnsMatcher public: struct Data { - using Aliases = std::unordered_map; - const Aliases & aliases; NameToNameMap & array_join_name_to_alias; NameToNameMap & array_join_alias_to_name; diff --git a/dbms/src/Interpreters/OptimizeIfWithConstantConditionVisitor.h b/dbms/src/Interpreters/OptimizeIfWithConstantConditionVisitor.h index ee738ec05e2..05d0330196b 100644 --- a/dbms/src/Interpreters/OptimizeIfWithConstantConditionVisitor.h +++ b/dbms/src/Interpreters/OptimizeIfWithConstantConditionVisitor.h @@ -1,8 +1,6 @@ #pragma once -#include - -#include +#include namespace DB { @@ -12,8 +10,6 @@ namespace DB class OptimizeIfWithConstantConditionVisitor { public: - using Aliases = std::unordered_map; - OptimizeIfWithConstantConditionVisitor(Aliases & aliases_) : aliases(aliases_) {} diff --git a/dbms/src/Interpreters/QueryAliasesVisitor.h b/dbms/src/Interpreters/QueryAliasesVisitor.h index de6080f2609..5ca712a1007 100644 --- a/dbms/src/Interpreters/QueryAliasesVisitor.h +++ b/dbms/src/Interpreters/QueryAliasesVisitor.h @@ -1,6 +1,6 @@ #pragma once -#include +#include #include namespace DB @@ -11,8 +11,6 @@ class ASTSubquery; struct ASTTableExpression; struct ASTArrayJoin; -using Aliases = std::unordered_map; - /// Visits AST node to collect aliases. class QueryAliasesMatcher { diff --git a/dbms/src/Interpreters/QueryNormalizer.h b/dbms/src/Interpreters/QueryNormalizer.h index 671ddf1b106..62aaa09bb34 100644 --- a/dbms/src/Interpreters/QueryNormalizer.h +++ b/dbms/src/Interpreters/QueryNormalizer.h @@ -1,11 +1,12 @@ #pragma once #include +#include #include #include #include -#include +#include namespace DB { @@ -45,7 +46,6 @@ class QueryNormalizer }; public: - using Aliases = std::unordered_map; using TableWithColumnNames = std::pair; struct Data diff --git a/dbms/src/Interpreters/SyntaxAnalyzer.h b/dbms/src/Interpreters/SyntaxAnalyzer.h index d34b16c1a40..4450881ee68 100644 --- a/dbms/src/Interpreters/SyntaxAnalyzer.h +++ b/dbms/src/Interpreters/SyntaxAnalyzer.h @@ -1,6 +1,7 @@ #pragma once #include +#include namespace DB { @@ -16,7 +17,6 @@ struct SyntaxAnalyzerResult NamesAndTypesList source_columns; - using Aliases = std::unordered_map; Aliases aliases; /// Which column is needed to be ARRAY-JOIN'ed to get the specified. From 4bb487763aeaf59e7a9c9ec4caf870d60f32fa73 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 25 Jan 2019 15:45:35 +0300 Subject: [PATCH 64/86] Make it work #4126 --- cmake/find_zstd.cmake | 1 + contrib/librdkafka-cmake/CMakeLists.txt | 5 +++-- 2 files changed, 4 insertions(+), 2 deletions(-) diff --git a/cmake/find_zstd.cmake b/cmake/find_zstd.cmake index 0e6db94a7c0..24bc851ed57 100644 --- a/cmake/find_zstd.cmake +++ b/cmake/find_zstd.cmake @@ -14,6 +14,7 @@ if (ZSTD_LIBRARY AND ZSTD_INCLUDE_DIR) else () set (USE_INTERNAL_ZSTD_LIBRARY 1) set (ZSTD_LIBRARY zstd) + set (ZSTD_INCLUDE_DIR ${ClickHouse_SOURCE_DIR}/contrib/zstd/lib) endif () message (STATUS "Using zstd: ${ZSTD_INCLUDE_DIR} : ${ZSTD_LIBRARY}") diff --git a/contrib/librdkafka-cmake/CMakeLists.txt b/contrib/librdkafka-cmake/CMakeLists.txt index fb27895c3da..62337f60fb5 100644 --- a/contrib/librdkafka-cmake/CMakeLists.txt +++ b/contrib/librdkafka-cmake/CMakeLists.txt @@ -60,5 +60,6 @@ set(SRCS add_library(rdkafka ${LINK_MODE} ${SRCS}) target_include_directories(rdkafka SYSTEM PUBLIC include) -target_include_directories(rdkafka SYSTEM PUBLIC ${RDKAFKA_SOURCE_DIR}) -target_link_libraries(rdkafka PUBLIC ${ZLIB_LIBRARIES} ${OPENSSL_SSL_LIBRARY} ${OPENSSL_CRYPTO_LIBRARY}) +target_include_directories(rdkafka SYSTEM PUBLIC ${RDKAFKA_SOURCE_DIR}) # Because weird logic with "include_next" is used. +target_include_directories(rdkafka SYSTEM PRIVATE ${ZSTD_INCLUDE_DIR}/common) # Because wrong path to "zstd_errors.h" is used. +target_link_libraries(rdkafka PUBLIC ${ZLIB_LIBRARIES} ${ZSTD_LIBRARY} ${OPENSSL_SSL_LIBRARY} ${OPENSSL_CRYPTO_LIBRARY}) From 02f236e45709fca2e8d82e5caf644b1c27487ff0 Mon Sep 17 00:00:00 2001 From: chertus Date: Fri, 25 Jan 2019 16:15:29 +0300 Subject: [PATCH 65/86] one more test --- .../0_stateless/00819_ast_refactoring_bugs.reference | 1 + .../queries/0_stateless/00819_ast_refactoring_bugs.sql | 8 ++++++++ 2 files changed, 9 insertions(+) diff --git a/dbms/tests/queries/0_stateless/00819_ast_refactoring_bugs.reference b/dbms/tests/queries/0_stateless/00819_ast_refactoring_bugs.reference index e69de29bb2d..26c4a78a1a9 100644 --- a/dbms/tests/queries/0_stateless/00819_ast_refactoring_bugs.reference +++ b/dbms/tests/queries/0_stateless/00819_ast_refactoring_bugs.reference @@ -0,0 +1 @@ +1 [1] diff --git a/dbms/tests/queries/0_stateless/00819_ast_refactoring_bugs.sql b/dbms/tests/queries/0_stateless/00819_ast_refactoring_bugs.sql index 27171f2e0de..010d03920c7 100644 --- a/dbms/tests/queries/0_stateless/00819_ast_refactoring_bugs.sql +++ b/dbms/tests/queries/0_stateless/00819_ast_refactoring_bugs.sql @@ -20,3 +20,11 @@ ARRAY JOIN WHERE CounterID = 100500; DROP TABLE test.visits; + +select u, cumSum from ( + select u, min(d) mn, max(d) mx, groupArray(d) dg, groupArray(v) vg, + arrayMap(x -> x + mn, range(toUInt32(mx - mn + 1))) days, + toString(arrayCumSum(arrayMap( x -> vg[indexOf(dg, x)] , days))) cumSum + from (select 1 u, today()-1 d, 1 v) + group by u +); From d9195cda99fae43857a0e5bde0958b044dbc9d02 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?L=C3=A9o=20Ercolanelli?= Date: Fri, 25 Jan 2019 14:06:21 +0100 Subject: [PATCH 66/86] implement `getWidenType()` on IDataType This is essentially to help implementing the `getReturnType` of aggregation function susceptible to provoque overflow (such as `sumMap`). --- dbms/src/Common/ErrorCodes.cpp | 1 + dbms/src/DataTypes/DataTypesDecimal.cpp | 8 ++++++++ dbms/src/DataTypes/DataTypesDecimal.h | 2 ++ dbms/src/DataTypes/DataTypesNumber.h | 7 +++++++ dbms/src/DataTypes/IDataType.cpp | 5 +++++ dbms/src/DataTypes/IDataType.h | 9 +++++++++ 6 files changed, 32 insertions(+) diff --git a/dbms/src/Common/ErrorCodes.cpp b/dbms/src/Common/ErrorCodes.cpp index f06a88c96da..d8749c1b1e7 100644 --- a/dbms/src/Common/ErrorCodes.cpp +++ b/dbms/src/Common/ErrorCodes.cpp @@ -412,6 +412,7 @@ namespace ErrorCodes extern const int NO_DATA_FOR_REQUIRED_PROTOBUF_FIELD = 435; extern const int CANNOT_CONVERT_TO_PROTOBUF_TYPE = 436; extern const int PROTOBUF_FIELD_NOT_REPEATED = 437; + extern const int DATA_TYPE_CANNOT_BE_WIDEN = 438; extern const int KEEPER_EXCEPTION = 999; extern const int POCO_EXCEPTION = 1000; diff --git a/dbms/src/DataTypes/DataTypesDecimal.cpp b/dbms/src/DataTypes/DataTypesDecimal.cpp index 439a98928ea..f8f3084b254 100644 --- a/dbms/src/DataTypes/DataTypesDecimal.cpp +++ b/dbms/src/DataTypes/DataTypesDecimal.cpp @@ -148,6 +148,14 @@ Field DataTypeDecimal::getDefault() const } +template +DataTypePtr DataTypeDecimal::getWidenDataType() const +{ + using WidenDataType = DataTypeDecimal; + return std::make_shared(WidenDataType::maxPrecision(), scale); +} + + template MutableColumnPtr DataTypeDecimal::createColumn() const { diff --git a/dbms/src/DataTypes/DataTypesDecimal.h b/dbms/src/DataTypes/DataTypesDecimal.h index 3f93f5aaae1..125f14cbf0a 100644 --- a/dbms/src/DataTypes/DataTypesDecimal.h +++ b/dbms/src/DataTypes/DataTypesDecimal.h @@ -103,6 +103,8 @@ public: void serializeProtobuf(const IColumn & column, size_t row_num, ProtobufWriter & protobuf) const override; Field getDefault() const override; + bool canBeWiden() const override { return true; } + DataTypePtr getWidenDataType() const override; MutableColumnPtr createColumn() const override; bool equals(const IDataType & rhs) const override; diff --git a/dbms/src/DataTypes/DataTypesNumber.h b/dbms/src/DataTypes/DataTypesNumber.h index 6ce1cbc0d24..0794af41f36 100644 --- a/dbms/src/DataTypes/DataTypesNumber.h +++ b/dbms/src/DataTypes/DataTypesNumber.h @@ -17,6 +17,13 @@ class DataTypeNumber final : public DataTypeNumberBase bool canBeUsedInBitOperations() const override { return true; } bool canBeUsedInBooleanContext() const override { return true; } bool canBeInsideNullable() const override { return true; } + + bool canBeWiden() const override { return true; } + DataTypePtr getWidenDataType() const override + { + using WidenDataType = DataTypeNumber>; + return std::make_shared(); + } }; using DataTypeUInt8 = DataTypeNumber; diff --git a/dbms/src/DataTypes/IDataType.cpp b/dbms/src/DataTypes/IDataType.cpp index e8c6a3cc63d..4ebe1eb3e48 100644 --- a/dbms/src/DataTypes/IDataType.cpp +++ b/dbms/src/DataTypes/IDataType.cpp @@ -19,6 +19,7 @@ namespace ErrorCodes { extern const int MULTIPLE_STREAMS_REQUIRED; extern const int LOGICAL_ERROR; + extern const int DATA_TYPE_CANNOT_BE_WIDEN; } @@ -51,6 +52,10 @@ ColumnPtr IDataType::createColumnConstWithDefaultValue(size_t size) const return createColumnConst(size, getDefault()); } +DataTypePtr IDataType::getWidenDataType() const +{ + throw Exception("Data type " + getName() + " can't be widen.", ErrorCodes::DATA_TYPE_CANNOT_BE_WIDEN); +} void IDataType::serializeBinaryBulk(const IColumn &, WriteBuffer &, size_t, size_t) const { diff --git a/dbms/src/DataTypes/IDataType.h b/dbms/src/DataTypes/IDataType.h index fdac4e454bc..3c20b258d09 100644 --- a/dbms/src/DataTypes/IDataType.h +++ b/dbms/src/DataTypes/IDataType.h @@ -273,6 +273,15 @@ public: */ virtual Field getDefault() const = 0; + /** The data type can be widen in order to try to avoid overflows. + * Widenable data types are typically Number or Decimal data types. + */ + virtual bool canBeWiden() const { return false; } + + /** Return the widen data type of the current data type. Throw an exception if `canBeWiden() == false`. + */ + virtual DataTypePtr getWidenDataType() const; + /** Directly insert default value into a column. Default implementation use method IColumn::insertDefault. * This should be overriden if data type default value differs from column default value (example: Enum data types). */ From 0d6094a3eae8c0268fa66064bf92088dfe2db322 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?L=C3=A9o=20Ercolanelli?= Date: Fri, 25 Jan 2019 14:08:16 +0100 Subject: [PATCH 67/86] sumMap: return types less prone to oveflows It used to be that sumMap would return the same type as the values columns. If columns of Array(UInt8) were to be given, that would really easily cause oveflow. It now uses `getWidenDataType` (and ultimately `NearestFieldType`) in order to define the result type. --- .../src/AggregateFunctions/AggregateFunctionSumMap.h | 12 +++++++++++- .../queries/0_stateless/00502_sum_map.reference | 1 + dbms/tests/queries/0_stateless/00502_sum_map.sql | 9 +++++++++ 3 files changed, 21 insertions(+), 1 deletion(-) diff --git a/dbms/src/AggregateFunctions/AggregateFunctionSumMap.h b/dbms/src/AggregateFunctions/AggregateFunctionSumMap.h index 1e5f3e38cd2..8c7c24faed5 100644 --- a/dbms/src/AggregateFunctions/AggregateFunctionSumMap.h +++ b/dbms/src/AggregateFunctions/AggregateFunctionSumMap.h @@ -72,7 +72,7 @@ public: types.emplace_back(std::make_shared(keys_type)); for (const auto & value_type : values_types) - types.emplace_back(std::make_shared(value_type)); + types.emplace_back(std::make_shared(widenDataType(value_type))); return std::make_shared(types); } @@ -260,6 +260,16 @@ public: const char * getHeaderFilePath() const override { return __FILE__; } bool keepKey(const T & key) const { return static_cast(*this).keepKey(key); } + +private: + static DataTypePtr widenDataType(const DataTypePtr & data_type) + { + if (!data_type->canBeWiden()) + throw new Exception{"Values to be summed are expected to be Numeric, Float or Decimal.", + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT}; + + return data_type->getWidenDataType(); + } }; template diff --git a/dbms/tests/queries/0_stateless/00502_sum_map.reference b/dbms/tests/queries/0_stateless/00502_sum_map.reference index 7bb325be814..a8d9fe95af3 100644 --- a/dbms/tests/queries/0_stateless/00502_sum_map.reference +++ b/dbms/tests/queries/0_stateless/00502_sum_map.reference @@ -10,6 +10,7 @@ 2000-01-01 00:01:00 [4,5,6,7,8] [10,10,20,10,10] ([1],[10]) ([1,4,8],[10,20,10]) +([1],[257]) ([1],[1]) ([1],[1]) (['a'],[1]) diff --git a/dbms/tests/queries/0_stateless/00502_sum_map.sql b/dbms/tests/queries/0_stateless/00502_sum_map.sql index 9cf941dd908..24eab44d3d0 100644 --- a/dbms/tests/queries/0_stateless/00502_sum_map.sql +++ b/dbms/tests/queries/0_stateless/00502_sum_map.sql @@ -17,6 +17,15 @@ SELECT sumMapFiltered([1, 4, 8])(statusMap.status, statusMap.requests) FROM test DROP TABLE test.sum_map; +DROP TABLE IF EXISTS test.sum_map_overflow; +CREATE TABLE test.sum_map_overflow(events Array(UInt8), counts Array(UInt8)) ENGINE = Log; + +INSERT INTO test.sum_map_overflow VALUES ([1], [255]), ([1], [2]); + +SELECT sumMap(events, counts) FROM test.sum_map_overflow; + +DROP TABLE test.sum_map_overflow; + select sumMap(val, cnt) from ( SELECT [ CAST(1, 'UInt64') ] as val, [1] as cnt ); select sumMap(val, cnt) from ( SELECT [ CAST(1, 'Float64') ] as val, [1] as cnt ); select sumMap(val, cnt) from ( SELECT [ CAST('a', 'Enum16(\'a\'=1)') ] as val, [1] as cnt ); From ed1136e49e21740002ef16bfe9476e3816761915 Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Fri, 25 Jan 2019 17:15:44 +0300 Subject: [PATCH 68/86] Use more IO/WriteHelpers.h functions in the class ProtobufSimpleWriter, rename variant=>varint. --- dbms/src/Formats/ProtobufSimpleWriter.cpp | 159 ++++++------------ dbms/src/Formats/ProtobufSimpleWriter.h | 2 +- .../00825_protobuf_format_output.reference | Bin 2019 -> 2042 bytes .../00825_protobuf_format_output.sh | 2 +- .../ProtobufDelimitedMessagesSerializer.cpp | 22 +-- 5 files changed, 68 insertions(+), 117 deletions(-) diff --git a/dbms/src/Formats/ProtobufSimpleWriter.cpp b/dbms/src/Formats/ProtobufSimpleWriter.cpp index dce58f889f1..3959f0e09b5 100644 --- a/dbms/src/Formats/ProtobufSimpleWriter.cpp +++ b/dbms/src/Formats/ProtobufSimpleWriter.cpp @@ -1,15 +1,15 @@ #include #include -#include - +#include namespace DB { namespace { - void writeBytes(WriteBuffer & buf, const void * data, size_t size) { buf.write(reinterpret_cast(data), size); } - - void writeVariant(WriteBuffer & buf, UInt32 value) + // Note: We cannot simply use writeVarUInt() from IO/VarInt.h here because there is one small difference: + // Google protobuf's representation of 64-bit integer contains from 1 to 10 bytes, whileas writeVarUInt() writes from 1 to 9 bytes + // because it omits the tenth byte (which is not necessary to decode actually). + void writePbVarUInt(UInt64 value, WriteBuffer & buf) { while (value >= 0x80) { @@ -19,67 +19,18 @@ namespace buf.write(static_cast(value)); } - void writeVariant(WriteBuffer & buf, Int32 value) { writeVariant(buf, static_cast(value)); } - - void writeVariant(WriteBuffer & buf, UInt64 value) + void writePbVarInt(Int64 value, WriteBuffer & buf) { - while (value >= 0x80) - { - buf.write(static_cast(value | 0x80)); - value >>= 7; - } - buf.write(static_cast(value)); + writePbVarUInt((static_cast(value) << 1) ^ static_cast(value >> 63), buf); } - void writeVariant(WriteBuffer & buf, Int64 value) { writeVariant(buf, static_cast(value)); } - - void writeLittleEndian(WriteBuffer & buf, UInt32 value) - { - value = Poco::ByteOrder::toLittleEndian(value); - writeBytes(buf, &value, sizeof(value)); - } - - void writeLittleEndian(WriteBuffer & buf, Int32 value) { writeLittleEndian(buf, static_cast(value)); } - - void writeLittleEndian(WriteBuffer & buf, float value) - { - union - { - Float32 f; - UInt32 i; - }; - f = value; - writeLittleEndian(buf, i); - } - - void writeLittleEndian(WriteBuffer & buf, UInt64 value) - { - value = Poco::ByteOrder::toLittleEndian(value); - writeBytes(buf, &value, sizeof(value)); - } - - void writeLittleEndian(WriteBuffer & buf, Int64 value) { writeLittleEndian(buf, static_cast(value)); } - - void writeLittleEndian(WriteBuffer & buf, double value) - { - union - { - Float64 f; - UInt64 i; - }; - f = value; - writeLittleEndian(buf, i); - } - - UInt32 zigZag(Int32 value) { return (static_cast(value) << 1) ^ static_cast(value >> 31); } - UInt64 zigZag(Int64 value) { return (static_cast(value) << 1) ^ static_cast(value >> 63); } - + void writePbVarIntNoZigZagEncoding(Int64 value, WriteBuffer & buf) { writePbVarUInt(static_cast(value), buf); } } enum ProtobufSimpleWriter::WireType : UInt32 { - VARIANT = 0, + VARINT = 0, BITS64 = 1, LENGTH_DELIMITED = 2, BITS32 = 5 @@ -107,8 +58,8 @@ void ProtobufSimpleWriter::finishCurrentMessage() finishCurrentField(); current_field_number = 0; StringRef str = message_buffer.stringRef(); - writeVariant(out, str.size); - writeBytes(out, str.data, str.size); + writePbVarUInt(str.size, out); + out.write(str.data, str.size); message_buffer.restart(); } @@ -129,112 +80,112 @@ void ProtobufSimpleWriter::finishCurrentField() StringRef str = repeated_packing_buffer.stringRef(); if (str.size) { - writeKey(message_buffer, LENGTH_DELIMITED); - writeVariant(message_buffer, str.size); - writeBytes(message_buffer, str.data, str.size); + writeKey(LENGTH_DELIMITED, message_buffer); + writePbVarUInt(str.size, message_buffer); + message_buffer.write(str.data, str.size); repeated_packing_buffer.restart(); } } } -void ProtobufSimpleWriter::writeKey(WriteBuffer & buf, WireType wire_type) +void ProtobufSimpleWriter::writeKey(WireType wire_type, WriteBuffer & buf) { - writeVariant(buf, (current_field_number << 3) | wire_type); + writePbVarUInt((current_field_number << 3) | wire_type, buf); } void ProtobufSimpleWriter::writeInt32(Int32 value) { assert(current_field_number); - writeKey(message_buffer, VARIANT); - writeVariant(message_buffer, value); + writeKey(VARINT, message_buffer); + writePbVarIntNoZigZagEncoding(value, message_buffer); ++num_normal_values; } void ProtobufSimpleWriter::writeUInt32(UInt32 value) { assert(current_field_number); - writeKey(message_buffer, VARIANT); - writeVariant(message_buffer, value); + writeKey(VARINT, message_buffer); + writePbVarUInt(value, message_buffer); ++num_normal_values; } void ProtobufSimpleWriter::writeSInt32(Int32 value) { assert(current_field_number); - writeKey(message_buffer, VARIANT); - writeVariant(message_buffer, zigZag(value)); + writeKey(VARINT, message_buffer); + writePbVarInt(value, message_buffer); ++num_normal_values; } void ProtobufSimpleWriter::writeInt64(Int64 value) { assert(current_field_number); - writeKey(message_buffer, VARIANT); - writeVariant(message_buffer, value); + writeKey(VARINT, message_buffer); + writePbVarIntNoZigZagEncoding(value, message_buffer); ++num_normal_values; } void ProtobufSimpleWriter::writeUInt64(UInt64 value) { assert(current_field_number); - writeKey(message_buffer, VARIANT); - writeVariant(message_buffer, value); + writeKey(VARINT, message_buffer); + writePbVarUInt(value, message_buffer); ++num_normal_values; } void ProtobufSimpleWriter::writeSInt64(Int64 value) { assert(current_field_number); - writeKey(message_buffer, VARIANT); - writeVariant(message_buffer, zigZag(value)); + writeKey(VARINT, message_buffer); + writePbVarInt(value, message_buffer); ++num_normal_values; } void ProtobufSimpleWriter::writeFixed32(UInt32 value) { assert(current_field_number); - writeKey(message_buffer, BITS32); - writeLittleEndian(message_buffer, value); + writeKey(BITS32, message_buffer); + writePODBinary(value, message_buffer); ++num_normal_values; } void ProtobufSimpleWriter::writeSFixed32(Int32 value) { assert(current_field_number); - writeKey(message_buffer, BITS32); - writeLittleEndian(message_buffer, value); + writeKey(BITS32, message_buffer); + writePODBinary(value, message_buffer); ++num_normal_values; } void ProtobufSimpleWriter::writeFloat(float value) { assert(current_field_number); - writeKey(message_buffer, BITS32); - writeLittleEndian(message_buffer, value); + writeKey(BITS32, message_buffer); + writePODBinary(value, message_buffer); ++num_normal_values; } void ProtobufSimpleWriter::writeFixed64(UInt64 value) { assert(current_field_number); - writeKey(message_buffer, BITS64); - writeLittleEndian(message_buffer, value); + writeKey(BITS64, message_buffer); + writePODBinary(value, message_buffer); ++num_normal_values; } void ProtobufSimpleWriter::writeSFixed64(Int64 value) { assert(current_field_number); - writeKey(message_buffer, BITS64); - writeLittleEndian(message_buffer, value); + writeKey(BITS64, message_buffer); + writePODBinary(value, message_buffer); ++num_normal_values; } void ProtobufSimpleWriter::writeDouble(double value) { assert(current_field_number); - writeKey(message_buffer, BITS64); - writeLittleEndian(message_buffer, value); + writeKey(BITS64, message_buffer); + writePODBinary(value, message_buffer); ++num_normal_values; } @@ -242,9 +193,9 @@ void ProtobufSimpleWriter::writeString(const StringRef & str) { assert(current_field_number); ++num_normal_values; - writeKey(message_buffer, LENGTH_DELIMITED); - writeVariant(message_buffer, str.size); - writeBytes(message_buffer, str.data, str.size); + writeKey(LENGTH_DELIMITED, message_buffer); + writePbVarUInt(str.size, message_buffer); + message_buffer.write(str.data, str.size); } void ProtobufSimpleWriter::writeInt32IfNonZero(Int32 value) @@ -328,84 +279,84 @@ void ProtobufSimpleWriter::writeStringIfNotEmpty(const StringRef & str) void ProtobufSimpleWriter::packRepeatedInt32(Int32 value) { assert(current_field_number); - writeVariant(repeated_packing_buffer, value); + writePbVarIntNoZigZagEncoding(value, repeated_packing_buffer); ++num_packed_values; } void ProtobufSimpleWriter::packRepeatedUInt32(UInt32 value) { assert(current_field_number); - writeVariant(repeated_packing_buffer, value); + writePbVarUInt(value, repeated_packing_buffer); ++num_packed_values; } void ProtobufSimpleWriter::packRepeatedSInt32(Int32 value) { assert(current_field_number); - writeVariant(repeated_packing_buffer, zigZag(value)); + writePbVarInt(value, repeated_packing_buffer); ++num_packed_values; } void ProtobufSimpleWriter::packRepeatedInt64(Int64 value) { assert(current_field_number); - writeVariant(repeated_packing_buffer, value); + writePbVarIntNoZigZagEncoding(value, repeated_packing_buffer); ++num_packed_values; } void ProtobufSimpleWriter::packRepeatedUInt64(UInt64 value) { assert(current_field_number); - writeVariant(repeated_packing_buffer, value); + writePbVarUInt(value, repeated_packing_buffer); ++num_packed_values; } void ProtobufSimpleWriter::packRepeatedSInt64(Int64 value) { assert(current_field_number); - writeVariant(repeated_packing_buffer, zigZag(value)); + writePbVarInt(value, repeated_packing_buffer); ++num_packed_values; } void ProtobufSimpleWriter::packRepeatedFixed32(UInt32 value) { assert(current_field_number); - writeLittleEndian(repeated_packing_buffer, value); + writePODBinary(value, repeated_packing_buffer); ++num_packed_values; } void ProtobufSimpleWriter::packRepeatedSFixed32(Int32 value) { assert(current_field_number); - writeLittleEndian(repeated_packing_buffer, value); + writePODBinary(value, repeated_packing_buffer); ++num_packed_values; } void ProtobufSimpleWriter::packRepeatedFloat(float value) { assert(current_field_number); - writeLittleEndian(repeated_packing_buffer, value); + writePODBinary(value, repeated_packing_buffer); ++num_packed_values; } void ProtobufSimpleWriter::packRepeatedFixed64(UInt64 value) { assert(current_field_number); - writeLittleEndian(repeated_packing_buffer, value); + writePODBinary(value, repeated_packing_buffer); ++num_packed_values; } void ProtobufSimpleWriter::packRepeatedSFixed64(Int64 value) { assert(current_field_number); - writeLittleEndian(repeated_packing_buffer, value); + writePODBinary(value, repeated_packing_buffer); ++num_packed_values; } void ProtobufSimpleWriter::packRepeatedDouble(double value) { assert(current_field_number); - writeLittleEndian(repeated_packing_buffer, value); + writePODBinary(value, repeated_packing_buffer); ++num_packed_values; } diff --git a/dbms/src/Formats/ProtobufSimpleWriter.h b/dbms/src/Formats/ProtobufSimpleWriter.h index 4880cfb4e21..185fc1c2289 100644 --- a/dbms/src/Formats/ProtobufSimpleWriter.h +++ b/dbms/src/Formats/ProtobufSimpleWriter.h @@ -74,7 +74,7 @@ private: void finishCurrentField(); enum WireType : UInt32; - void writeKey(WriteBuffer & buf, WireType wire_type); + void writeKey(WireType wire_type, WriteBuffer & buf); WriteBuffer & out; bool were_messages = false; diff --git a/dbms/tests/queries/0_stateless/00825_protobuf_format_output.reference b/dbms/tests/queries/0_stateless/00825_protobuf_format_output.reference index ed3e22f41e33b807ae354cd7045e96b86eae973f..d66a039f65c2aab60fa84126a7d0559dd4f4d800 100644 GIT binary patch delta 147 zcmaFN|BHXZTE@DG>wJVc0&*&I^Gi!IsyMRddOErFv>cc$%PzlJgVCOmsfBTJ29p7= z(F(Rd|Dk|!^BkrVj7(b?C$6)cJfFo#9$^5dsezt}fw8%fNfoEAiJqZ}k)e_C<~J-C Q7@3;E%D=Hm;IQu)0FNj)asU7T delta 125 zcmeyx|CoQmTE?o$7uYN(J{RU^2`Ni0s^Z92jCKmsV{qEc%NWPVSUowH$$-Ud1)IX= zRZJ%s88=Km&t^4wEsK#fL^-FaiJrNIk(se+6{oS0o`H#}k*VS4e=HXm8S8-t2cnxg MnU_(1axnXH0P?sdW&i*H diff --git a/dbms/tests/queries/0_stateless/00825_protobuf_format_output.sh b/dbms/tests/queries/0_stateless/00825_protobuf_format_output.sh index a89126a3af8..ab021017e2b 100755 --- a/dbms/tests/queries/0_stateless/00825_protobuf_format_output.sh +++ b/dbms/tests/queries/0_stateless/00825_protobuf_format_output.sh @@ -44,7 +44,7 @@ CREATE TABLE test.table (uuid UUID, ) ENGINE = MergeTree ORDER BY tuple(); INSERT INTO test.table VALUES (toUUID('a7522158-3d41-4b77-ad69-6c598ee55c49'), 'Ivan', 'Petrov', 'male', toDate('1980-12-29'), 'png', '+74951234567', 1, toDateTime('2019-01-05 18:45:00'), 38, 'capricorn', ['Yesterday', 'Flowers'], [255, 0, 0], 'Moscow', [55.753215, 37.622504], 3.14, 214.10, 0.1, 5.8, 17060000000); -INSERT INTO test.table VALUES (toUUID('c694ad8a-f714-4ea3-907d-fd54fb25d9b5'), 'Natalia', 'Sokolova', 'female', toDate('1992-03-08'), 'jpg', NULL, 0, NULL, 26, 'pisces', [], [100, 200, 50], 'Tver', [54.782635, 32.045251], 3.14159, NULL, 0.007, 5.4, -20000000000000); +INSERT INTO test.table VALUES (toUUID('c694ad8a-f714-4ea3-907d-fd54fb25d9b5'), 'Natalia', 'Sokolova', 'female', toDate('1992-03-08'), 'jpg', NULL, 0, NULL, 26, 'pisces', [], [100, 200, 50], 'Plymouth', [50.403724, -4.142123], 3.14159, NULL, 0.007, 5.4, -20000000000000); INSERT INTO test.table VALUES (toUUID('a7da1aa6-f425-4789-8947-b034786ed374'), 'Vasily', 'Sidorov', 'male', toDate('1995-07-28'), 'bmp', '+442012345678', 1, toDateTime('2018-12-30 00:00:00'), 23, 'leo', ['Sunny'], [250, 244, 10], 'Murmansk', [68.970682, 33.074981], 3.14159265358979, 100000000000, 800, -3.2, 154400000); SELECT * FROM test.table ORDER BY name FORMAT Protobuf SETTINGS format_schema = '00825_protobuf_format_copy:Person'; diff --git a/utils/test-data-generator/ProtobufDelimitedMessagesSerializer.cpp b/utils/test-data-generator/ProtobufDelimitedMessagesSerializer.cpp index 4a5596cb7f2..39877956686 100644 --- a/utils/test-data-generator/ProtobufDelimitedMessagesSerializer.cpp +++ b/utils/test-data-generator/ProtobufDelimitedMessagesSerializer.cpp @@ -53,9 +53,9 @@ int main(int, char **) person.add_color(100); person.add_color(200); person.add_color(50); - person.set_hometown("Tver"); - person.add_location(54.782635); - person.add_location(32.045251); + person.set_hometown("Plymouth"); + person.add_location(50.403724); + person.add_location(-4.142123); person.set_pi(3.14159); person.set_someratio(0.007); person.set_temperature(5.4); @@ -120,8 +120,8 @@ int main(int, char **) { AltPerson person; - person.add_location(54); - person.add_location(32); + person.add_location(50); + person.add_location(-4); person.set_pi(3.14159); person.set_uuid("c694ad8a-f714-4ea3-907d-fd54fb25d9b5"); person.set_name("Natalia"); @@ -208,9 +208,9 @@ int main(int, char **) person.add_color("100"); person.add_color("200"); person.add_color("50"); - person.set_hometown("Tver"); - person.add_location("54.782635"); - person.add_location("32.045251"); + person.set_hometown("Plymouth"); + person.add_location("50.403724"); + person.add_location("-4.142123"); person.set_pi("3.14159"); person.set_someratio("0.007"); person.set_temperature("5.4"); @@ -290,9 +290,9 @@ int main(int, char **) person.add_color(100); person.add_color(200); person.add_color(50); - person.set_hometown("Tver"); - person.add_location(54.782635); - person.add_location(32.045251); + person.set_hometown("Plymouth"); + person.add_location(50.403724); + person.add_location(-4.142123); person.set_pi(3.14159); person.set_someratio(0.007); person.set_temperature(5.4); From 57104f81da213e95afb4623e5d416b0f8e1f3bb6 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?L=C3=A9o=20Ercolanelli?= Date: Fri, 25 Jan 2019 15:16:23 +0100 Subject: [PATCH 69/86] rename `widen` to `promote` for IDataType --- dbms/src/AggregateFunctions/AggregateFunctionSumMap.h | 8 ++++---- dbms/src/Common/ErrorCodes.cpp | 2 +- dbms/src/DataTypes/DataTypesDecimal.cpp | 6 +++--- dbms/src/DataTypes/DataTypesDecimal.h | 4 ++-- dbms/src/DataTypes/DataTypesNumber.h | 8 ++++---- dbms/src/DataTypes/IDataType.cpp | 6 +++--- dbms/src/DataTypes/IDataType.h | 10 +++++----- 7 files changed, 22 insertions(+), 22 deletions(-) diff --git a/dbms/src/AggregateFunctions/AggregateFunctionSumMap.h b/dbms/src/AggregateFunctions/AggregateFunctionSumMap.h index 8c7c24faed5..a1be9fa2a86 100644 --- a/dbms/src/AggregateFunctions/AggregateFunctionSumMap.h +++ b/dbms/src/AggregateFunctions/AggregateFunctionSumMap.h @@ -72,7 +72,7 @@ public: types.emplace_back(std::make_shared(keys_type)); for (const auto & value_type : values_types) - types.emplace_back(std::make_shared(widenDataType(value_type))); + types.emplace_back(std::make_shared(promoteNumericType(value_type))); return std::make_shared(types); } @@ -262,13 +262,13 @@ public: bool keepKey(const T & key) const { return static_cast(*this).keepKey(key); } private: - static DataTypePtr widenDataType(const DataTypePtr & data_type) + static DataTypePtr promoteNumericType(const DataTypePtr & data_type) { - if (!data_type->canBeWiden()) + if (!data_type->canBePromoted()) throw new Exception{"Values to be summed are expected to be Numeric, Float or Decimal.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT}; - return data_type->getWidenDataType(); + return data_type->promoteNumericType(); } }; diff --git a/dbms/src/Common/ErrorCodes.cpp b/dbms/src/Common/ErrorCodes.cpp index d8749c1b1e7..574360d6979 100644 --- a/dbms/src/Common/ErrorCodes.cpp +++ b/dbms/src/Common/ErrorCodes.cpp @@ -412,7 +412,7 @@ namespace ErrorCodes extern const int NO_DATA_FOR_REQUIRED_PROTOBUF_FIELD = 435; extern const int CANNOT_CONVERT_TO_PROTOBUF_TYPE = 436; extern const int PROTOBUF_FIELD_NOT_REPEATED = 437; - extern const int DATA_TYPE_CANNOT_BE_WIDEN = 438; + extern const int DATA_TYPE_CANNOT_BE_PROMOTED = 438; extern const int KEEPER_EXCEPTION = 999; extern const int POCO_EXCEPTION = 1000; diff --git a/dbms/src/DataTypes/DataTypesDecimal.cpp b/dbms/src/DataTypes/DataTypesDecimal.cpp index f8f3084b254..e6fe1bbd7f2 100644 --- a/dbms/src/DataTypes/DataTypesDecimal.cpp +++ b/dbms/src/DataTypes/DataTypesDecimal.cpp @@ -149,10 +149,10 @@ Field DataTypeDecimal::getDefault() const template -DataTypePtr DataTypeDecimal::getWidenDataType() const +DataTypePtr DataTypeDecimal::promoteNumericType() const { - using WidenDataType = DataTypeDecimal; - return std::make_shared(WidenDataType::maxPrecision(), scale); + using PromotedType = DataTypeDecimal; + return std::make_shared(PromotedType::maxPrecision(), scale); } diff --git a/dbms/src/DataTypes/DataTypesDecimal.h b/dbms/src/DataTypes/DataTypesDecimal.h index 125f14cbf0a..c287742928a 100644 --- a/dbms/src/DataTypes/DataTypesDecimal.h +++ b/dbms/src/DataTypes/DataTypesDecimal.h @@ -103,8 +103,8 @@ public: void serializeProtobuf(const IColumn & column, size_t row_num, ProtobufWriter & protobuf) const override; Field getDefault() const override; - bool canBeWiden() const override { return true; } - DataTypePtr getWidenDataType() const override; + bool canBePromoted() const override { return true; } + DataTypePtr promoteNumericType() const override; MutableColumnPtr createColumn() const override; bool equals(const IDataType & rhs) const override; diff --git a/dbms/src/DataTypes/DataTypesNumber.h b/dbms/src/DataTypes/DataTypesNumber.h index 0794af41f36..f54e014dfc5 100644 --- a/dbms/src/DataTypes/DataTypesNumber.h +++ b/dbms/src/DataTypes/DataTypesNumber.h @@ -18,11 +18,11 @@ class DataTypeNumber final : public DataTypeNumberBase bool canBeUsedInBooleanContext() const override { return true; } bool canBeInsideNullable() const override { return true; } - bool canBeWiden() const override { return true; } - DataTypePtr getWidenDataType() const override + bool canBePromoted() const override { return true; } + DataTypePtr promoteNumericType() const override { - using WidenDataType = DataTypeNumber>; - return std::make_shared(); + using PromotedType = DataTypeNumber>; + return std::make_shared(); } }; diff --git a/dbms/src/DataTypes/IDataType.cpp b/dbms/src/DataTypes/IDataType.cpp index 4ebe1eb3e48..a8eec08a7c8 100644 --- a/dbms/src/DataTypes/IDataType.cpp +++ b/dbms/src/DataTypes/IDataType.cpp @@ -19,7 +19,7 @@ namespace ErrorCodes { extern const int MULTIPLE_STREAMS_REQUIRED; extern const int LOGICAL_ERROR; - extern const int DATA_TYPE_CANNOT_BE_WIDEN; + extern const int DATA_TYPE_CANNOT_BE_PROMOTED; } @@ -52,9 +52,9 @@ ColumnPtr IDataType::createColumnConstWithDefaultValue(size_t size) const return createColumnConst(size, getDefault()); } -DataTypePtr IDataType::getWidenDataType() const +DataTypePtr IDataType::promoteNumericType() const { - throw Exception("Data type " + getName() + " can't be widen.", ErrorCodes::DATA_TYPE_CANNOT_BE_WIDEN); + throw Exception("Data type " + getName() + " can't be promoted.", ErrorCodes::DATA_TYPE_CANNOT_BE_PROMOTED); } void IDataType::serializeBinaryBulk(const IColumn &, WriteBuffer &, size_t, size_t) const diff --git a/dbms/src/DataTypes/IDataType.h b/dbms/src/DataTypes/IDataType.h index 3c20b258d09..1fce04639bf 100644 --- a/dbms/src/DataTypes/IDataType.h +++ b/dbms/src/DataTypes/IDataType.h @@ -273,14 +273,14 @@ public: */ virtual Field getDefault() const = 0; - /** The data type can be widen in order to try to avoid overflows. - * Widenable data types are typically Number or Decimal data types. + /** The data type can be promoted in order to try to avoid overflows. + * Data types which can be promoted are typically Number or Decimal data types. */ - virtual bool canBeWiden() const { return false; } + virtual bool canBePromoted() const { return false; } - /** Return the widen data type of the current data type. Throw an exception if `canBeWiden() == false`. + /** Return the promoted numeric data type of the current data type. Throw an exception if `canBePromoted() == false`. */ - virtual DataTypePtr getWidenDataType() const; + virtual DataTypePtr promoteNumericType() const; /** Directly insert default value into a column. Default implementation use method IColumn::insertDefault. * This should be overriden if data type default value differs from column default value (example: Enum data types). From a78282b8347704d1456f9b1e66383347def5b094 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?L=C3=A9o=20Ercolanelli?= Date: Fri, 25 Jan 2019 15:42:44 +0100 Subject: [PATCH 70/86] IDataType.h: remove trailing whitespace --- dbms/src/DataTypes/IDataType.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/src/DataTypes/IDataType.h b/dbms/src/DataTypes/IDataType.h index 1fce04639bf..ae14fe1aa36 100644 --- a/dbms/src/DataTypes/IDataType.h +++ b/dbms/src/DataTypes/IDataType.h @@ -280,7 +280,7 @@ public: /** Return the promoted numeric data type of the current data type. Throw an exception if `canBePromoted() == false`. */ - virtual DataTypePtr promoteNumericType() const; + virtual DataTypePtr promoteNumericType() const; /** Directly insert default value into a column. Default implementation use method IColumn::insertDefault. * This should be overriden if data type default value differs from column default value (example: Enum data types). From d8eba7e71b95d617ae5df1865d2c397c6e69a635 Mon Sep 17 00:00:00 2001 From: Amos Bird Date: Fri, 25 Jan 2019 22:50:31 +0800 Subject: [PATCH 71/86] Better joinGet. --- dbms/src/Functions/FunctionJoinGet.cpp | 51 ++++++++++++++----- dbms/src/Functions/FunctionJoinGet.h | 21 ++++++-- .../00800_versatile_storage_join.reference | 2 + .../00800_versatile_storage_join.sql | 13 +++++ 4 files changed, 71 insertions(+), 16 deletions(-) diff --git a/dbms/src/Functions/FunctionJoinGet.cpp b/dbms/src/Functions/FunctionJoinGet.cpp index d2f65148b23..1c0cbff7506 100644 --- a/dbms/src/Functions/FunctionJoinGet.cpp +++ b/dbms/src/Functions/FunctionJoinGet.cpp @@ -1,10 +1,10 @@ #include +#include #include #include #include #include -#include namespace DB @@ -15,10 +15,10 @@ namespace ErrorCodes extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; } -FunctionBasePtr FunctionBuilderJoinGet::buildImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &) const +static auto getJoin(const ColumnsWithTypeAndName & arguments, const Context & context) { if (arguments.size() != 3) - throw Exception{"Function " + getName() + " takes 3 arguments", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH}; + throw Exception{"Function joinGet takes 3 arguments", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH}; String join_name; if (auto name_col = checkAndGetColumnConst(arguments[0].column.get())) @@ -26,37 +26,62 @@ FunctionBasePtr FunctionBuilderJoinGet::buildImpl(const ColumnsWithTypeAndName & join_name = name_col->getValue(); } else - throw Exception{"Illegal type " + arguments[0].type->getName() + " of first argument of function " + getName() - + ", expected a const string.", + throw Exception{"Illegal type " + arguments[0].type->getName() + " of first argument of function joinGet, expected a const string.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT}; - auto table = context.getTable("", join_name); - - StorageJoin * storage_join = dynamic_cast(table.get()); - + size_t dot = join_name.find('.'); + String database_name; + if (dot == String::npos) + { + database_name = context.getCurrentDatabase(); + dot = 0; + } + else + { + database_name = join_name.substr(0, dot); + ++dot; + } + String table_name = join_name.substr(dot); + auto table = context.getTable(database_name, table_name); + auto storage_join = std::dynamic_pointer_cast(table); if (!storage_join) throw Exception{"Table " + join_name + " should have engine StorageJoin", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT}; - auto join = storage_join->getJoin(); String attr_name; if (auto name_col = checkAndGetColumnConst(arguments[1].column.get())) { attr_name = name_col->getValue(); } else - throw Exception{"Illegal type " + arguments[1].type->getName() + " of second argument of function " + getName() - + ", expected a const string.", + throw Exception{"Illegal type " + arguments[1].type->getName() + + " of second argument of function joinGet, expected a const string.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT}; + return std::make_pair(storage_join, attr_name); +} +FunctionBasePtr FunctionBuilderJoinGet::buildImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &) const +{ + auto [storage_join, attr_name] = getJoin(arguments, context); + auto join = storage_join->getJoin(); DataTypes data_types(arguments.size()); + auto table_lock = storage_join->lockStructure(false); for (size_t i = 0; i < arguments.size(); ++i) data_types[i] = arguments[i].type; + auto return_type = join->joinGetReturnType(attr_name); return std::make_shared( - std::make_shared(join, attr_name), data_types, join->joinGetReturnType(attr_name)); + std::make_shared(table_lock, storage_join, join, attr_name, return_type), data_types, return_type); } +DataTypePtr FunctionBuilderJoinGet::getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const +{ + auto [storage_join, attr_name] = getJoin(arguments, context); + auto join = storage_join->getJoin(); + return join->joinGetReturnType(attr_name); +} + + void FunctionJoinGet::executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) { auto & ctn = block.getByPosition(arguments[2]); diff --git a/dbms/src/Functions/FunctionJoinGet.h b/dbms/src/Functions/FunctionJoinGet.h index 6573dcf9405..edf45adab6a 100644 --- a/dbms/src/Functions/FunctionJoinGet.h +++ b/dbms/src/Functions/FunctionJoinGet.h @@ -3,20 +3,32 @@ namespace DB { class Context; +class IStorage; +using StoragePtr = std::shared_ptr; class Join; using JoinPtr = std::shared_ptr; +class TableStructureReadLock; +using TableStructureReadLockPtr = std::shared_ptr; class FunctionJoinGet final : public IFunction, public std::enable_shared_from_this { public: static constexpr auto name = "joinGet"; - FunctionJoinGet(JoinPtr join, const String & attr_name) : join(std::move(join)), attr_name(attr_name) {} + FunctionJoinGet( + TableStructureReadLockPtr table_lock, StoragePtr storage_join, JoinPtr join, const String & attr_name, DataTypePtr return_type) + : table_lock(std::move(table_lock)) + , storage_join(std::move(storage_join)) + , join(std::move(join)) + , attr_name(attr_name) + , return_type(std::move(return_type)) + { + } String getName() const override { return name; } protected: - DataTypePtr getReturnTypeImpl(const DataTypes & /*arguments*/) const override { return nullptr; } + DataTypePtr getReturnTypeImpl(const DataTypes & /*arguments*/) const override { return return_type; } void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override; private: @@ -24,8 +36,11 @@ private: size_t getNumberOfArguments() const override { return 0; } private: + TableStructureReadLockPtr table_lock; + StoragePtr storage_join; JoinPtr join; const String attr_name; + DataTypePtr return_type; }; class FunctionBuilderJoinGet final : public FunctionBuilderImpl @@ -40,7 +55,7 @@ public: protected: FunctionBasePtr buildImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &) const override; - DataTypePtr getReturnTypeImpl(const DataTypes & /*arguments*/) const override { return nullptr; } + DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override; private: bool isVariadic() const override { return true; } diff --git a/dbms/tests/queries/0_stateless/00800_versatile_storage_join.reference b/dbms/tests/queries/0_stateless/00800_versatile_storage_join.reference index 1fa9ac74e57..9b706623c8c 100644 --- a/dbms/tests/queries/0_stateless/00800_versatile_storage_join.reference +++ b/dbms/tests/queries/0_stateless/00800_versatile_storage_join.reference @@ -17,3 +17,5 @@ abc def [0] 1 +0 +0 diff --git a/dbms/tests/queries/0_stateless/00800_versatile_storage_join.sql b/dbms/tests/queries/0_stateless/00800_versatile_storage_join.sql index 80f7616766c..47c14765ba9 100644 --- a/dbms/tests/queries/0_stateless/00800_versatile_storage_join.sql +++ b/dbms/tests/queries/0_stateless/00800_versatile_storage_join.sql @@ -49,3 +49,16 @@ DROP TABLE test.join_any_left_null; DROP TABLE test.join_all_inner; DROP TABLE test.join_all_left; DROP TABLE test.join_string_key; + +-- test provided by Alexander Zaitsev +DROP TABLE IF EXISTS test.join_test; +CREATE TABLE test.join_test (a UInt8, b UInt8) Engine = Join(ANY, LEFT, a); + +USE test; +select joinGet('join_test', 'b', 1); + +USE system; +SELECT joinGet('test.join_test', 'b', 1); + +USE default; +DROP TABLE test.join_test; From 091efcda9553610818c00088b93e903ed1ed67cf Mon Sep 17 00:00:00 2001 From: chertus Date: Fri, 25 Jan 2019 18:42:24 +0300 Subject: [PATCH 72/86] add IdentifierSemantic and PredicateExpressionsOptimizer refactoring --- .../Interpreters/AddDefaultDatabaseVisitor.h | 3 +- .../Interpreters/ArrayJoinedColumnsVisitor.h | 3 +- dbms/src/Interpreters/ColumnNamesContext.cpp | 3 +- .../DatabaseAndTableWithAlias.cpp | 68 +---- .../Interpreters/DatabaseAndTableWithAlias.h | 10 +- dbms/src/Interpreters/ExternalTablesVisitor.h | 3 +- .../Interpreters/GlobalSubqueriesVisitor.h | 1 + dbms/src/Interpreters/IdentifierSemantic.cpp | 106 +++++++ dbms/src/Interpreters/IdentifierSemantic.h | 35 +++ .../InJoinSubqueriesPreprocessor.cpp | 1 + .../Interpreters/InterpreterDescribeQuery.cpp | 16 +- .../PredicateExpressionsOptimizer.cpp | 276 ++++++++++-------- .../PredicateExpressionsOptimizer.h | 48 ++- dbms/src/Interpreters/QueryNormalizer.cpp | 8 +- .../RequiredSourceColumnsVisitor.cpp | 3 +- dbms/src/Interpreters/SyntaxAnalyzer.cpp | 35 +-- .../TranslateQualifiedNamesVisitor.cpp | 45 +-- .../TranslateQualifiedNamesVisitor.h | 2 +- dbms/src/Parsers/ASTIdentifier.cpp | 82 ++---- dbms/src/Parsers/ASTIdentifier.h | 54 ++-- .../MergeTree/MergeTreeWhereOptimizer.cpp | 3 +- dbms/src/Storages/StorageView.cpp | 1 + dbms/src/Storages/VirtualColumnUtils.cpp | 3 +- .../TableFunctions/TableFunctionRemote.cpp | 1 + .../00597_push_down_predicate.reference | 1 + .../0_stateless/00597_push_down_predicate.sql | 3 +- .../00674_join_on_syntax.reference | 4 +- .../0_stateless/00674_join_on_syntax.sql | 12 +- .../queries/0_stateless/00703_join_crash.sql | 2 +- 29 files changed, 433 insertions(+), 399 deletions(-) create mode 100644 dbms/src/Interpreters/IdentifierSemantic.cpp create mode 100644 dbms/src/Interpreters/IdentifierSemantic.h diff --git a/dbms/src/Interpreters/AddDefaultDatabaseVisitor.h b/dbms/src/Interpreters/AddDefaultDatabaseVisitor.h index 2b032ff8a88..ce542a63503 100644 --- a/dbms/src/Interpreters/AddDefaultDatabaseVisitor.h +++ b/dbms/src/Interpreters/AddDefaultDatabaseVisitor.h @@ -11,6 +11,7 @@ #include #include #include +#include namespace DB { @@ -99,7 +100,7 @@ private: /// @note It expects that only table (not column) identifiers are visited. void visit(const ASTIdentifier & identifier, ASTPtr & ast) const { - if (identifier.name_parts.empty()) + if (!identifier.compound()) ast = createTableIdentifier(database_name, identifier.name); } diff --git a/dbms/src/Interpreters/ArrayJoinedColumnsVisitor.h b/dbms/src/Interpreters/ArrayJoinedColumnsVisitor.h index 0c19ffe7387..6aed7436572 100644 --- a/dbms/src/Interpreters/ArrayJoinedColumnsVisitor.h +++ b/dbms/src/Interpreters/ArrayJoinedColumnsVisitor.h @@ -10,6 +10,7 @@ #include #include +#include #include @@ -95,7 +96,7 @@ private: NameToNameMap & array_join_alias_to_name = data.array_join_alias_to_name; NameToNameMap & array_join_result_to_source = data.array_join_result_to_source; - if (!getColumnIdentifierName(node)) + if (!IdentifierSemantic::getColumnName(node)) return; auto splitted = Nested::splitName(node.name); /// ParsedParams, Key1 diff --git a/dbms/src/Interpreters/ColumnNamesContext.cpp b/dbms/src/Interpreters/ColumnNamesContext.cpp index 246b5f5306e..3240ca7fea1 100644 --- a/dbms/src/Interpreters/ColumnNamesContext.cpp +++ b/dbms/src/Interpreters/ColumnNamesContext.cpp @@ -1,4 +1,5 @@ #include +#include #include namespace DB @@ -31,7 +32,7 @@ bool ColumnNamesContext::addColumnAliasIfAny(const IAST & ast, bool is_public) void ColumnNamesContext::addColumnIdentifier(const ASTIdentifier & node, bool is_public) { - if (!getColumnIdentifierName(node)) + if (!IdentifierSemantic::getColumnName(node)) return; required_names.insert(node.name); diff --git a/dbms/src/Interpreters/DatabaseAndTableWithAlias.cpp b/dbms/src/Interpreters/DatabaseAndTableWithAlias.cpp index efd29f962d0..c9afb5da722 100644 --- a/dbms/src/Interpreters/DatabaseAndTableWithAlias.cpp +++ b/dbms/src/Interpreters/DatabaseAndTableWithAlias.cpp @@ -1,4 +1,5 @@ #include +#include #include #include @@ -11,60 +12,13 @@ namespace DB { -/// Checks that ast is ASTIdentifier and remove num_qualifiers_to_strip components from left. -/// Example: 'database.table.name' -> (num_qualifiers_to_strip = 2) -> 'name'. -void stripIdentifier(const DB::ASTPtr & ast, size_t num_qualifiers_to_strip) -{ - ASTIdentifier * identifier = typeid_cast(ast.get()); - - if (!identifier) - throw DB::Exception("ASTIdentifier expected for stripIdentifier", DB::ErrorCodes::LOGICAL_ERROR); - - if (num_qualifiers_to_strip) - { - identifier->name_parts.erase(identifier->name_parts.begin(), identifier->name_parts.begin() + num_qualifiers_to_strip); - DB::String new_name; - for (const auto & part : identifier->name_parts) - { - if (!new_name.empty()) - new_name += '.'; - new_name += part; - } - identifier->name.swap(new_name); - } -} - -/// Get the number of components of identifier which are correspond to 'alias.', 'table.' or 'databas.table.' from names. -size_t getNumComponentsToStripInOrderToTranslateQualifiedName(const ASTIdentifier & identifier, - const DatabaseAndTableWithAlias & names) -{ - /// database.table.column - if (doesIdentifierBelongTo(identifier, names.database, names.table)) - return 2; - - /// table.column or alias.column. - if (doesIdentifierBelongTo(identifier, names.table) || - doesIdentifierBelongTo(identifier, names.alias)) - return 1; - - return 0; -} - - DatabaseAndTableWithAlias::DatabaseAndTableWithAlias(const ASTIdentifier & identifier, const String & current_database) { - database = current_database; - table = identifier.name; alias = identifier.tryGetAlias(); - if (!identifier.name_parts.empty()) - { - if (identifier.name_parts.size() != 2) - throw Exception("Logical error: 2 components expected in table expression '" + identifier.name + "'", ErrorCodes::LOGICAL_ERROR); - - database = identifier.name_parts[0]; - table = identifier.name_parts[1]; - } + std::tie(database, table) = IdentifierSemantic::extractDatabaseAndTable(identifier); + if (database.empty()) + database = current_database; } DatabaseAndTableWithAlias::DatabaseAndTableWithAlias(const ASTPtr & node, const String & current_database) @@ -108,19 +62,7 @@ String DatabaseAndTableWithAlias::getQualifiedNamePrefix() const { if (alias.empty() && table.empty()) return ""; - - return (!alias.empty() ? alias : (database + '.' + table)) + '.'; -} - -void DatabaseAndTableWithAlias::makeQualifiedName(const ASTPtr & ast) const -{ - if (auto identifier = typeid_cast(ast.get())) - { - String prefix = getQualifiedNamePrefix(); - identifier->name.insert(identifier->name.begin(), prefix.begin(), prefix.end()); - - addIdentifierQualifier(*identifier, database, table, alias); - } + return (!alias.empty() ? alias : table) + '.'; } std::vector getSelectTablesExpression(const ASTSelectQuery & select_query) diff --git a/dbms/src/Interpreters/DatabaseAndTableWithAlias.h b/dbms/src/Interpreters/DatabaseAndTableWithAlias.h index 87884e26273..bb4f7ca92ef 100644 --- a/dbms/src/Interpreters/DatabaseAndTableWithAlias.h +++ b/dbms/src/Interpreters/DatabaseAndTableWithAlias.h @@ -29,21 +29,13 @@ struct DatabaseAndTableWithAlias DatabaseAndTableWithAlias(const ASTIdentifier & identifier, const String & current_database = ""); DatabaseAndTableWithAlias(const ASTTableExpression & table_expression, const String & current_database); - /// "alias." or "database.table." if alias is empty + /// "alias." or "table." if alias is empty String getQualifiedNamePrefix() const; - /// If ast is ASTIdentifier, prepend getQualifiedNamePrefix() to it's name. - void makeQualifiedName(const ASTPtr & ast) const; - /// Check if it satisfies another db_table name. @note opterion is not symmetric. bool satisfies(const DatabaseAndTableWithAlias & table, bool table_may_be_an_alias); }; -void stripIdentifier(const DB::ASTPtr & ast, size_t num_qualifiers_to_strip); - -size_t getNumComponentsToStripInOrderToTranslateQualifiedName(const ASTIdentifier & identifier, - const DatabaseAndTableWithAlias & names); - std::vector getDatabaseAndTables(const ASTSelectQuery & select_query, const String & current_database); std::optional getDatabaseAndTable(const ASTSelectQuery & select, size_t table_number); diff --git a/dbms/src/Interpreters/ExternalTablesVisitor.h b/dbms/src/Interpreters/ExternalTablesVisitor.h index d8b177b1ed3..2f3eecd2828 100644 --- a/dbms/src/Interpreters/ExternalTablesVisitor.h +++ b/dbms/src/Interpreters/ExternalTablesVisitor.h @@ -5,6 +5,7 @@ #include #include #include +#include namespace DB { @@ -33,7 +34,7 @@ public: private: static std::vector visit(const ASTIdentifier & node, ASTPtr &, Data & data) { - if (auto opt_name = getTableIdentifierName(node)) + if (auto opt_name = IdentifierSemantic::getTableName(node)) if (StoragePtr external_storage = data.context.tryGetExternalTable(*opt_name)) data.external_tables[*opt_name] = external_storage; return {}; diff --git a/dbms/src/Interpreters/GlobalSubqueriesVisitor.h b/dbms/src/Interpreters/GlobalSubqueriesVisitor.h index 3e4108464f9..81e45d2abea 100644 --- a/dbms/src/Interpreters/GlobalSubqueriesVisitor.h +++ b/dbms/src/Interpreters/GlobalSubqueriesVisitor.h @@ -16,6 +16,7 @@ #include #include #include +#include namespace DB { diff --git a/dbms/src/Interpreters/IdentifierSemantic.cpp b/dbms/src/Interpreters/IdentifierSemantic.cpp new file mode 100644 index 00000000000..e6fe2257d20 --- /dev/null +++ b/dbms/src/Interpreters/IdentifierSemantic.cpp @@ -0,0 +1,106 @@ +#include + +#include + +namespace DB +{ + +std::optional IdentifierSemantic::getColumnName(const ASTIdentifier & node) +{ + if (!node.semantic->special) + return node.name; + return {}; +} + +std::optional IdentifierSemantic::getColumnName(const ASTPtr & ast) +{ + if (ast) + if (auto id = typeid_cast(ast.get())) + if (!id->semantic->special) + return id->name; + return {}; +} + +std::optional IdentifierSemantic::getTableName(const ASTIdentifier & node) +{ + if (node.semantic->special) + return node.name; + return {}; +} + +std::optional IdentifierSemantic::getTableName(const ASTPtr & ast) +{ + if (ast) + if (auto id = typeid_cast(ast.get())) + if (id->semantic->special) + return id->name; + return {}; +} + +std::pair IdentifierSemantic::extractDatabaseAndTable(const ASTIdentifier & identifier) +{ + if (identifier.name_parts.size() > 2) + throw Exception("Logical error: more than two components in table expression", ErrorCodes::LOGICAL_ERROR); + + if (identifier.name_parts.size() == 2) + return { identifier.name_parts[0], identifier.name_parts[1] }; + return { "", identifier.name }; +} + +bool IdentifierSemantic::doesIdentifierBelongTo(const ASTIdentifier & identifier, const String & database, const String & table) +{ + size_t num_components = identifier.name_parts.size(); + if (num_components >= 3) + return identifier.name_parts[0] == database && + identifier.name_parts[1] == table; + return false; +} + +bool IdentifierSemantic::doesIdentifierBelongTo(const ASTIdentifier & identifier, const String & table) +{ + size_t num_components = identifier.name_parts.size(); + if (num_components >= 2) + return identifier.name_parts[0] == table; + return false; +} + +size_t IdentifierSemantic::canReferColumnToTable(const ASTIdentifier & identifier, const DatabaseAndTableWithAlias & db_and_table) +{ + /// database.table.column + if (doesIdentifierBelongTo(identifier, db_and_table.database, db_and_table.table)) + return 2; + + /// table.column or alias.column. + if (doesIdentifierBelongTo(identifier, db_and_table.table) || + doesIdentifierBelongTo(identifier, db_and_table.alias)) + return 1; + + return 0; +} + +/// Checks that ast is ASTIdentifier and remove num_qualifiers_to_strip components from left. +/// Example: 'database.table.name' -> (num_qualifiers_to_strip = 2) -> 'name'. +void IdentifierSemantic::setColumnShortName(ASTIdentifier & identifier, size_t to_strip) +{ + if (!to_strip) + return; + + std::vector stripped(identifier.name_parts.begin() + to_strip, identifier.name_parts.end()); + + DB::String new_name; + for (const auto & part : stripped) + { + if (!new_name.empty()) + new_name += '.'; + new_name += part; + } + identifier.name.swap(new_name); +} + +void IdentifierSemantic::setColumnQualifiedName(ASTIdentifier & identifier, const DatabaseAndTableWithAlias & db_and_table) +{ + String prefix = db_and_table.getQualifiedNamePrefix(); + identifier.name.insert(identifier.name.begin(), prefix.begin(), prefix.end()); +} + +} diff --git a/dbms/src/Interpreters/IdentifierSemantic.h b/dbms/src/Interpreters/IdentifierSemantic.h new file mode 100644 index 00000000000..895a51899fe --- /dev/null +++ b/dbms/src/Interpreters/IdentifierSemantic.h @@ -0,0 +1,35 @@ +#pragma once + +#include +#include + +namespace DB +{ + +struct IdentifierSemanticImpl +{ + bool special = false; +}; + +/// Static calss to manipulate IdentifierSemanticImpl via ASTIdentifier +struct IdentifierSemantic +{ + /// @returns name for column identifiers + static std::optional getColumnName(const ASTIdentifier & node); + static std::optional getColumnName(const ASTPtr & ast); + + /// @returns name for 'not a column' identifiers + static std::optional getTableName(const ASTIdentifier & node); + static std::optional getTableName(const ASTPtr & ast); + static std::pair extractDatabaseAndTable(const ASTIdentifier & identifier); + + static size_t canReferColumnToTable(const ASTIdentifier & identifier, const DatabaseAndTableWithAlias & db_and_table); + static void setColumnShortName(ASTIdentifier & identifier, size_t match); + static void setColumnQualifiedName(ASTIdentifier & identifier, const DatabaseAndTableWithAlias & db_and_table); + +private: + static bool doesIdentifierBelongTo(const ASTIdentifier & identifier, const String & database, const String & table); + static bool doesIdentifierBelongTo(const ASTIdentifier & identifier, const String & table); +}; + +} diff --git a/dbms/src/Interpreters/InJoinSubqueriesPreprocessor.cpp b/dbms/src/Interpreters/InJoinSubqueriesPreprocessor.cpp index e8e697cfe4f..e0be0d068e0 100644 --- a/dbms/src/Interpreters/InJoinSubqueriesPreprocessor.cpp +++ b/dbms/src/Interpreters/InJoinSubqueriesPreprocessor.cpp @@ -1,6 +1,7 @@ #include #include #include +#include #include #include #include diff --git a/dbms/src/Interpreters/InterpreterDescribeQuery.cpp b/dbms/src/Interpreters/InterpreterDescribeQuery.cpp index 7c2812eb428..b80426fcdd9 100644 --- a/dbms/src/Interpreters/InterpreterDescribeQuery.cpp +++ b/dbms/src/Interpreters/InterpreterDescribeQuery.cpp @@ -9,6 +9,7 @@ #include #include #include +#include #include #include #include @@ -83,20 +84,11 @@ BlockInputStreamPtr InterpreterDescribeQuery::executeImpl() } else { + auto identifier = typeid_cast(table_expression->database_and_table_name.get()); + String database_name; String table_name; - - auto identifier = typeid_cast(table_expression->database_and_table_name.get()); - if (identifier->name_parts.size() > 2) - throw Exception("Logical error: more than two components in table expression", ErrorCodes::LOGICAL_ERROR); - - if (identifier->name_parts.size() > 1) - { - database_name = identifier->name_parts[0]; - table_name = identifier->name_parts[1]; - } - else - table_name = identifier->name; + std::tie(database_name, table_name) = IdentifierSemantic::extractDatabaseAndTable(*identifier); table = context.getTable(database_name, table_name); } diff --git a/dbms/src/Interpreters/PredicateExpressionsOptimizer.cpp b/dbms/src/Interpreters/PredicateExpressionsOptimizer.cpp index 99ea0479009..cd4c33ce558 100644 --- a/dbms/src/Interpreters/PredicateExpressionsOptimizer.cpp +++ b/dbms/src/Interpreters/PredicateExpressionsOptimizer.cpp @@ -1,12 +1,23 @@ +#include + #include #include #include #include +#include #include -#include +#include +#include +#include +#include +#include +#include +#include #include #include #include +#include +#include #include #include #include "TranslateQualifiedNamesVisitor.h" @@ -22,6 +33,59 @@ namespace ErrorCodes static constexpr auto and_function_name = "and"; + +struct FindIdentifierBestTableData +{ + using TypeToVisit = ASTIdentifier; + + const std::vector & tables; + std::vector> identifier_table; + + FindIdentifierBestTableData(const std::vector & tables_) + : tables(tables_) + {} + + void visit(ASTIdentifier & identifier, ASTPtr &) + { + const DatabaseAndTableWithAlias * best_table = nullptr; + + if (!identifier.compound()) + { + if (!tables.empty()) + best_table = &tables[0]; + } + else + { + size_t best_match = 0; + for (const DatabaseAndTableWithAlias & table : tables) + { + if (size_t match = IdentifierSemantic::canReferColumnToTable(identifier, table)) + if (match > best_match) + { + best_match = match; + best_table = &table; + } + } + } + + identifier_table.emplace_back(&identifier, best_table); + } +}; + +using FindIdentifierBestTableMatcher = OneTypeMatcher; +using FindIdentifierBestTableVisitor = InDepthNodeVisitor; + + +static bool allowPushDown(const ASTSelectQuery * subquery) +{ + return subquery && + !subquery->final() && + !subquery->limit_by_expression_list && + !subquery->limit_length && + !subquery->with_expression_list; +} + + PredicateExpressionsOptimizer::PredicateExpressionsOptimizer( ASTSelectQuery * ast_select_, ExtractedSettings && settings_, const Context & context_) : ast_select(ast_select_), settings(settings_), context(context_) @@ -36,47 +100,50 @@ bool PredicateExpressionsOptimizer::optimize() if (!ast_select->where_expression && !ast_select->prewhere_expression) return false; - SubqueriesProjectionColumns all_subquery_projection_columns; - getAllSubqueryProjectionColumns(all_subquery_projection_columns); + SubqueriesProjectionColumns all_subquery_projection_columns = getAllSubqueryProjectionColumns(); bool is_rewrite_subqueries = false; if (!all_subquery_projection_columns.empty()) { - is_rewrite_subqueries |= optimizeImpl(ast_select->where_expression, all_subquery_projection_columns, false); - is_rewrite_subqueries |= optimizeImpl(ast_select->prewhere_expression, all_subquery_projection_columns, true); + is_rewrite_subqueries |= optimizeImpl(ast_select->where_expression, all_subquery_projection_columns, OptimizeKind::PUSH_TO_WHERE); + is_rewrite_subqueries |= optimizeImpl(ast_select->prewhere_expression, all_subquery_projection_columns, OptimizeKind::PUSH_TO_PREWHERE); } return is_rewrite_subqueries; } bool PredicateExpressionsOptimizer::optimizeImpl( - ASTPtr & outer_expression, SubqueriesProjectionColumns & subqueries_projection_columns, bool is_prewhere) + ASTPtr & outer_expression, SubqueriesProjectionColumns & subqueries_projection_columns, OptimizeKind expression_kind) { /// split predicate with `and` - PredicateExpressions outer_predicate_expressions = splitConjunctionPredicate(outer_expression); + std::vector outer_predicate_expressions = splitConjunctionPredicate(outer_expression); std::vector database_and_table_with_aliases = getDatabaseAndTables(*ast_select, context.getCurrentDatabase()); bool is_rewrite_subquery = false; - for (const auto & outer_predicate : outer_predicate_expressions) + for (auto & outer_predicate : outer_predicate_expressions) { if (isArrayJoinFunction(outer_predicate)) continue; - IdentifiersWithQualifiedNameSet outer_predicate_dependencies; - getDependenciesAndQualifiedOfExpression(outer_predicate, outer_predicate_dependencies, database_and_table_with_aliases); + auto outer_predicate_dependencies = getDependenciesAndQualifiers(outer_predicate, database_and_table_with_aliases); /// TODO: remove origin expression - for (const auto & subquery_projection_columns : subqueries_projection_columns) + for (const auto & [subquery, projection_columns] : subqueries_projection_columns) { - auto subquery = static_cast(subquery_projection_columns.first); - const ProjectionsWithAliases projection_columns = subquery_projection_columns.second; - OptimizeKind optimize_kind = OptimizeKind::NONE; - if (!cannotPushDownOuterPredicate(projection_columns, subquery, outer_predicate_dependencies, is_prewhere, optimize_kind)) + if (allowPushDown(subquery) && canPushDownOuterPredicate(projection_columns, outer_predicate_dependencies, optimize_kind)) { - ASTPtr inner_predicate; - cloneOuterPredicateForInnerPredicate(outer_predicate, projection_columns, database_and_table_with_aliases, inner_predicate); + if (optimize_kind == OptimizeKind::NONE) + optimize_kind = expression_kind; + + ASTPtr inner_predicate = outer_predicate->clone(); + cleanExpressionAlias(inner_predicate); /// clears the alias name contained in the outer predicate + + std::vector inner_predicate_dependencies = + getDependenciesAndQualifiers(inner_predicate, database_and_table_with_aliases); + + setNewAliasesForInnerPredicate(projection_columns, inner_predicate_dependencies); switch (optimize_kind) { @@ -91,9 +158,9 @@ bool PredicateExpressionsOptimizer::optimizeImpl( return is_rewrite_subquery; } -PredicateExpressions PredicateExpressionsOptimizer::splitConjunctionPredicate(ASTPtr & predicate_expression) +std::vector PredicateExpressionsOptimizer::splitConjunctionPredicate(ASTPtr & predicate_expression) { - PredicateExpressions predicate_expressions; + std::vector predicate_expressions; if (predicate_expression) { @@ -127,77 +194,79 @@ PredicateExpressions PredicateExpressionsOptimizer::splitConjunctionPredicate(AS return predicate_expressions; } -void PredicateExpressionsOptimizer::getDependenciesAndQualifiedOfExpression(const ASTPtr & expression, - IdentifiersWithQualifiedNameSet & dependencies_and_qualified, - std::vector & tables_with_aliases) +std::vector +PredicateExpressionsOptimizer::getDependenciesAndQualifiers(ASTPtr & expression, std::vector & tables) { - if (const auto identifier = typeid_cast(expression.get())) + FindIdentifierBestTableVisitor::Data find_data(tables); + FindIdentifierBestTableVisitor(find_data).visit(expression); + + std::vector dependencies; + + for (const auto & [identifier, table] : find_data.identifier_table) { String table_alias; - if (!identifier->name_parts.empty()) - { - if (!tables_with_aliases.empty()) - table_alias = tables_with_aliases[0].getQualifiedNamePrefix(); - } - else - { - size_t best_table_pos = 0; - size_t max_num_qualifiers_to_strip = 0; + if (table) + table_alias = table->getQualifiedNamePrefix(); - /// translate qualifiers for dependent columns - for (size_t table_pos = 0; table_pos < tables_with_aliases.size(); ++table_pos) - { - const auto & table = tables_with_aliases[table_pos]; - auto num_qualifiers_to_strip = getNumComponentsToStripInOrderToTranslateQualifiedName(*identifier, table); - - if (num_qualifiers_to_strip > max_num_qualifiers_to_strip) - { - max_num_qualifiers_to_strip = num_qualifiers_to_strip; - best_table_pos = table_pos; - } - } - - table_alias = tables_with_aliases[best_table_pos].getQualifiedNamePrefix(); - } - - String qualified_name = table_alias + expression->getAliasOrColumnName(); - dependencies_and_qualified.emplace_back(std::pair(identifier, qualified_name)); - } - else - { - for (const auto & child : expression->children) - getDependenciesAndQualifiedOfExpression(child, dependencies_and_qualified, tables_with_aliases); + dependencies.emplace_back(identifier, table_alias); } + + return dependencies; } -bool PredicateExpressionsOptimizer::cannotPushDownOuterPredicate( - const ProjectionsWithAliases & subquery_projection_columns, ASTSelectQuery * subquery, - IdentifiersWithQualifiedNameSet & outer_predicate_dependencies, bool & is_prewhere, OptimizeKind & optimize_kind) +static String qualifiedName(ASTIdentifier * identifier, const String & prefix) { - if (subquery->final() || subquery->limit_by_expression_list || subquery->limit_length || subquery->with_expression_list) - return true; + if (identifier->isShort()) + return prefix + identifier->getAliasOrColumnName(); + return identifier->getAliasOrColumnName(); +} - for (auto & predicate_dependency : outer_predicate_dependencies) +bool PredicateExpressionsOptimizer::canPushDownOuterPredicate( + const std::vector & projection_columns, + const std::vector & dependencies, + OptimizeKind & optimize_kind) +{ + for (const auto & [identifier, prefix] : dependencies) { bool is_found = false; + String qualified_name = qualifiedName(identifier, prefix); - for (auto projection_column : subquery_projection_columns) + for (const auto & [ast, alias] : projection_columns) { - if (projection_column.second == predicate_dependency.second) + if (alias == qualified_name) { is_found = true; - optimize_kind = isAggregateFunction(projection_column.first) ? OptimizeKind::PUSH_TO_HAVING : optimize_kind; + if (isAggregateFunction(ast)) + optimize_kind = OptimizeKind::PUSH_TO_HAVING; } } if (!is_found) - return true; + return false; } - if (optimize_kind == OptimizeKind::NONE) - optimize_kind = is_prewhere ? OptimizeKind::PUSH_TO_PREWHERE : OptimizeKind::PUSH_TO_WHERE; + return true; +} - return false; +void PredicateExpressionsOptimizer::setNewAliasesForInnerPredicate( + const std::vector & projection_columns, + const std::vector & dependencies) +{ + for (auto & [identifier, prefix] : dependencies) + { + String qualified_name = qualifiedName(identifier, prefix); + + for (auto & [ast, alias] : projection_columns) + { + if (alias == qualified_name) + { + if (!isIdentifier(ast) && ast->tryGetAlias().empty()) + ast->setAlias(ast->getColumnName()); + + identifier->resetWithAlias(ast->getAliasOrColumnName()); + } + } + } } bool PredicateExpressionsOptimizer::isArrayJoinFunction(const ASTPtr & node) @@ -215,47 +284,21 @@ bool PredicateExpressionsOptimizer::isArrayJoinFunction(const ASTPtr & node) return false; } -bool PredicateExpressionsOptimizer::isAggregateFunction(ASTPtr & node) +bool PredicateExpressionsOptimizer::isAggregateFunction(const ASTPtr & node) { - if (auto function = typeid_cast(node.get())) + if (auto function = typeid_cast(node.get())) { if (AggregateFunctionFactory::instance().isAggregateFunctionName(function->name)) return true; } - for (auto & child : node->children) + for (const auto & child : node->children) if (isAggregateFunction(child)) return true; return false; } -void PredicateExpressionsOptimizer::cloneOuterPredicateForInnerPredicate( - const ASTPtr & outer_predicate, const ProjectionsWithAliases & projection_columns, - std::vector & tables, ASTPtr & inner_predicate) -{ - inner_predicate = outer_predicate->clone(); - - /// clears the alias name contained in the outer predicate - cleanExpressionAlias(inner_predicate); - IdentifiersWithQualifiedNameSet new_expression_requires; - getDependenciesAndQualifiedOfExpression(inner_predicate, new_expression_requires, tables); - - for (auto & require : new_expression_requires) - { - for (auto projection : projection_columns) - { - if (require.second == projection.second) - { - ASTPtr & ast = projection.first; - if (!typeid_cast(ast.get()) && ast->tryGetAlias().empty()) - ast->setAlias(ast->getColumnName()); - require.first->name = ast->getAliasOrColumnName(); - } - } - } -} - bool PredicateExpressionsOptimizer::optimizeExpression(const ASTPtr & outer_expression, ASTPtr & subquery_expression, ASTSelectQuery * subquery) { ASTPtr new_subquery_expression = subquery_expression; @@ -272,33 +315,32 @@ bool PredicateExpressionsOptimizer::optimizeExpression(const ASTPtr & outer_expr return true; } -void PredicateExpressionsOptimizer::getAllSubqueryProjectionColumns(SubqueriesProjectionColumns & all_subquery_projection_columns) +PredicateExpressionsOptimizer::SubqueriesProjectionColumns PredicateExpressionsOptimizer::getAllSubqueryProjectionColumns() { - const auto tables_expression = getSelectTablesExpression(*ast_select); + SubqueriesProjectionColumns projection_columns; - for (const auto & table_expression : tables_expression) - { + for (const auto & table_expression : getSelectTablesExpression(*ast_select)) if (table_expression->subquery) - { - /// Use qualifiers to translate the columns of subqueries - DatabaseAndTableWithAlias database_and_table_with_alias(*table_expression, context.getCurrentDatabase()); - String qualified_name_prefix = database_and_table_with_alias.getQualifiedNamePrefix(); - getSubqueryProjectionColumns(all_subquery_projection_columns, qualified_name_prefix, - static_cast(table_expression->subquery.get())->children[0]); - } - } + getSubqueryProjectionColumns(table_expression->subquery, projection_columns); + + return projection_columns; } -void PredicateExpressionsOptimizer::getSubqueryProjectionColumns(SubqueriesProjectionColumns & all_subquery_projection_columns, - String & qualified_name_prefix, const ASTPtr & subquery) +void PredicateExpressionsOptimizer::getSubqueryProjectionColumns(const ASTPtr & subquery, SubqueriesProjectionColumns & projection_columns) { - ASTs select_with_union_projections; - auto select_with_union_query = static_cast(subquery.get()); + String qualified_name_prefix = subquery->tryGetAlias(); + if (!qualified_name_prefix.empty()) + qualified_name_prefix += '.'; - for (auto & select_without_union_query : select_with_union_query->list_of_selects->children) + const ASTPtr & subselect = subquery->children[0]; + + ASTs select_with_union_projections; + auto select_with_union_query = static_cast(subselect.get()); + + for (auto & select : select_with_union_query->list_of_selects->children) { - ProjectionsWithAliases subquery_projections; - auto select_projection_columns = getSelectQueryProjectionColumns(select_without_union_query); + std::vector subquery_projections; + auto select_projection_columns = getSelectQueryProjectionColumns(select); if (!select_projection_columns.empty()) { @@ -309,7 +351,7 @@ void PredicateExpressionsOptimizer::getSubqueryProjectionColumns(SubqueriesProje subquery_projections.emplace_back(std::pair(select_projection_columns[i], qualified_name_prefix + select_with_union_projections[i]->getAliasOrColumnName())); - all_subquery_projection_columns.insert(std::pair(select_without_union_query.get(), subquery_projections)); + projection_columns.insert(std::pair(static_cast(select.get()), subquery_projections)); } } } diff --git a/dbms/src/Interpreters/PredicateExpressionsOptimizer.h b/dbms/src/Interpreters/PredicateExpressionsOptimizer.h index 65148e0682a..93e666dde32 100644 --- a/dbms/src/Interpreters/PredicateExpressionsOptimizer.h +++ b/dbms/src/Interpreters/PredicateExpressionsOptimizer.h @@ -1,26 +1,14 @@ #pragma once -#include -#include -#include -#include -#include -#include -#include -#include -#include #include namespace DB { -using PredicateExpressions = std::vector; -using ProjectionWithAlias = std::pair; -using ProjectionsWithAliases = std::vector; -using SubqueriesProjectionColumns = std::map; -using IdentifierWithQualifiedName = std::pair; -using IdentifiersWithQualifiedNameSet = std::vector; - +class ASTIdentifier; +class ASTSelectQuery; +class ASTSubquery; +class Context; /** This class provides functions for Push-Down predicate expressions * @@ -35,6 +23,10 @@ using IdentifiersWithQualifiedNameSet = std::vector */ class PredicateExpressionsOptimizer { + using ProjectionWithAlias = std::pair; + using SubqueriesProjectionColumns = std::map>; + using IdentifierWithQualifier = std::pair; + /// Extracts settings, mostly to show which are used and which are not. struct ExtractedSettings { @@ -73,29 +65,29 @@ private: PUSH_TO_HAVING, }; - bool isAggregateFunction(ASTPtr & node); + bool isAggregateFunction(const ASTPtr & node); bool isArrayJoinFunction(const ASTPtr & node); - PredicateExpressions splitConjunctionPredicate(ASTPtr & predicate_expression); + std::vector splitConjunctionPredicate(ASTPtr & predicate_expression); - void getDependenciesAndQualifiedOfExpression(const ASTPtr & expression, IdentifiersWithQualifiedNameSet & dependencies_and_qualified, - std::vector & tables_with_aliases); + std::vector getDependenciesAndQualifiers(ASTPtr & expression, + std::vector & tables_with_aliases); bool optimizeExpression(const ASTPtr & outer_expression, ASTPtr & subquery_expression, ASTSelectQuery * subquery); - bool optimizeImpl(ASTPtr & outer_expression, SubqueriesProjectionColumns & subqueries_projection_columns, bool is_prewhere); + bool optimizeImpl(ASTPtr & outer_expression, SubqueriesProjectionColumns & subqueries_projection_columns, OptimizeKind optimize_kind); - bool cannotPushDownOuterPredicate(const ProjectionsWithAliases & subquery_projection_columns, ASTSelectQuery * subquery, - IdentifiersWithQualifiedNameSet & outer_predicate_dependencies, bool & is_prewhere, OptimizeKind & optimize_kind); + bool canPushDownOuterPredicate(const std::vector & subquery_projection_columns, + const std::vector & outer_predicate_dependencies, + OptimizeKind & optimize_kind); - void cloneOuterPredicateForInnerPredicate(const ASTPtr & outer_predicate, const ProjectionsWithAliases & projection_columns, - std::vector & tables, ASTPtr & inner_predicate); + void setNewAliasesForInnerPredicate(const std::vector & projection_columns, + const std::vector & inner_predicate_dependencies); - void getAllSubqueryProjectionColumns(SubqueriesProjectionColumns & all_subquery_projection_columns); + SubqueriesProjectionColumns getAllSubqueryProjectionColumns(); - void getSubqueryProjectionColumns(SubqueriesProjectionColumns & all_subquery_projection_columns, - String & qualified_name_prefix, const ASTPtr & subquery); + void getSubqueryProjectionColumns(const ASTPtr & subquery, SubqueriesProjectionColumns & all_subquery_projection_columns); ASTs getSelectQueryProjectionColumns(ASTPtr & ast); diff --git a/dbms/src/Interpreters/QueryNormalizer.cpp b/dbms/src/Interpreters/QueryNormalizer.cpp index a5151be7c64..56529ae595c 100644 --- a/dbms/src/Interpreters/QueryNormalizer.cpp +++ b/dbms/src/Interpreters/QueryNormalizer.cpp @@ -1,6 +1,7 @@ #include #include #include +#include #include #include #include @@ -107,7 +108,7 @@ void QueryNormalizer::visit(ASTIdentifier & node, ASTPtr & ast, Data & data) auto & current_asts = data.current_asts; String & current_alias = data.current_alias; - if (!getColumnIdentifierName(node)) + if (!IdentifierSemantic::getColumnName(node)) return; /// If it is an alias, but not a parent alias (for constructs like "SELECT column + 1 AS column"). @@ -124,7 +125,7 @@ void QueryNormalizer::visit(ASTIdentifier & node, ASTPtr & ast, Data & data) if (!my_alias.empty() && my_alias != alias_node->getAliasOrColumnName()) { /// Avoid infinite recursion here - auto opt_name = getColumnIdentifierName(alias_node); + auto opt_name = IdentifierSemantic::getColumnName(alias_node); bool is_cycle = opt_name && *opt_name == node.name; if (!is_cycle) @@ -273,8 +274,7 @@ void QueryNormalizer::visitChildren(const ASTPtr & node, Data & data) visit(child, data); } } - else if (!typeid_cast(node.get()) && - !typeid_cast(node.get())) + else if (!typeid_cast(node.get())) { for (auto & child : node->children) { diff --git a/dbms/src/Interpreters/RequiredSourceColumnsVisitor.cpp b/dbms/src/Interpreters/RequiredSourceColumnsVisitor.cpp index 5dc479fee5e..436e636bec8 100644 --- a/dbms/src/Interpreters/RequiredSourceColumnsVisitor.cpp +++ b/dbms/src/Interpreters/RequiredSourceColumnsVisitor.cpp @@ -46,8 +46,7 @@ bool RequiredSourceColumnsMatcher::needChildVisit(ASTPtr & node, const ASTPtr & return false; /// Processed. Do not need children. - if (typeid_cast(node.get()) || - typeid_cast(node.get()) || + if (typeid_cast(node.get()) || typeid_cast(node.get()) || typeid_cast(node.get())) return false; diff --git a/dbms/src/Interpreters/SyntaxAnalyzer.cpp b/dbms/src/Interpreters/SyntaxAnalyzer.cpp index fb6a34d37b9..cc9fe20a69b 100644 --- a/dbms/src/Interpreters/SyntaxAnalyzer.cpp +++ b/dbms/src/Interpreters/SyntaxAnalyzer.cpp @@ -18,6 +18,7 @@ #include #include #include +#include #include #include @@ -477,20 +478,18 @@ void collectJoinedColumnsFromJoinOnExpr(AnalyzedJoin & analyzed_join, const ASTS std::function get_table_belonging; get_table_belonging = [&](const ASTPtr & ast) -> TableBelonging { - if (getColumnIdentifierName(ast)) + if (IdentifierSemantic::getColumnName(ast)) { auto * identifier = typeid_cast(ast.get()); - { - auto left_num_components = getNumComponentsToStripInOrderToTranslateQualifiedName(*identifier, left_source_names); - auto right_num_components = getNumComponentsToStripInOrderToTranslateQualifiedName(*identifier, right_source_names); + size_t left_match_degree = IdentifierSemantic::canReferColumnToTable(*identifier, left_source_names); + size_t right_match_degree = IdentifierSemantic::canReferColumnToTable(*identifier, right_source_names); + + if (left_match_degree > right_match_degree) + return {identifier, nullptr}; + if (left_match_degree < right_match_degree) + return {nullptr, identifier}; - /// Assume that component from definite table if num_components is greater than for the other table. - if (left_num_components > right_num_components) - return {identifier, nullptr}; - if (left_num_components < right_num_components) - return {nullptr, identifier}; - } return {}; } @@ -516,19 +515,15 @@ void collectJoinedColumnsFromJoinOnExpr(AnalyzedJoin & analyzed_join, const ASTS std::function translate_qualified_names; translate_qualified_names = [&](ASTPtr & ast, const DatabaseAndTableWithAlias & source_names, bool right_table) { - if (getColumnIdentifierName(ast)) + if (IdentifierSemantic::getColumnName(ast)) { - auto * identifier = typeid_cast(ast.get()); + auto * identifier = typeid_cast(ast.get()); - { - auto num_components = getNumComponentsToStripInOrderToTranslateQualifiedName(*identifier, source_names); - stripIdentifier(ast, num_components); + size_t match = IdentifierSemantic::canReferColumnToTable(*identifier, source_names); + IdentifierSemantic::setColumnShortName(*identifier, match); - if (right_table && source_columns.count(ast->getColumnName())) - source_names.makeQualifiedName(ast); - - } - return; + if (right_table && source_columns.count(ast->getColumnName())) + IdentifierSemantic::setColumnQualifiedName(*identifier, source_names); } for (auto & child : ast->children) diff --git a/dbms/src/Interpreters/TranslateQualifiedNamesVisitor.cpp b/dbms/src/Interpreters/TranslateQualifiedNamesVisitor.cpp index f884028dd1c..382c8043f2d 100644 --- a/dbms/src/Interpreters/TranslateQualifiedNamesVisitor.cpp +++ b/dbms/src/Interpreters/TranslateQualifiedNamesVisitor.cpp @@ -1,4 +1,5 @@ #include +#include #include #include @@ -28,8 +29,7 @@ bool TranslateQualifiedNamesMatcher::needChildVisit(ASTPtr & node, const ASTPtr return false; /// Processed nodes. Do not go into children. - if (typeid_cast(node.get()) || - typeid_cast(node.get()) || + if (typeid_cast(node.get()) || typeid_cast(node.get())) return false; @@ -50,38 +50,25 @@ std::vector TranslateQualifiedNamesMatcher::visit(ASTPtr & ast, Data & return {}; } -std::vector TranslateQualifiedNamesMatcher::visit(const ASTIdentifier & identifier, ASTPtr & ast, Data & data) +std::vector TranslateQualifiedNamesMatcher::visit(ASTIdentifier & identifier, ASTPtr & ast, Data & data) { - const NameSet & source_columns = data.source_columns; - const std::vector & tables = data.tables; - - if (getColumnIdentifierName(identifier)) + if (IdentifierSemantic::getColumnName(identifier)) { - /// Select first table name with max number of qualifiers which can be stripped. - size_t max_num_qualifiers_to_strip = 0; - size_t best_table_pos = 0; + bool best_table_pos = 0; + size_t best_match = 0; + for (size_t i = 0; i < data.tables.size(); ++i) + if (size_t match = IdentifierSemantic::canReferColumnToTable(identifier, data.tables[i])) + if (match > best_match) + { + best_match = match; + best_table_pos = i; + } - for (size_t table_pos = 0; table_pos < tables.size(); ++table_pos) - { - const auto & table = tables[table_pos]; - auto num_qualifiers_to_strip = getNumComponentsToStripInOrderToTranslateQualifiedName(identifier, table); - - if (num_qualifiers_to_strip > max_num_qualifiers_to_strip) - { - max_num_qualifiers_to_strip = num_qualifiers_to_strip; - best_table_pos = table_pos; - } - } - - if (max_num_qualifiers_to_strip) - stripIdentifier(ast, max_num_qualifiers_to_strip); + IdentifierSemantic::setColumnShortName(identifier, best_match); /// In case if column from the joined table are in source columns, change it's name to qualified. - if (best_table_pos && source_columns.count(ast->getColumnName())) - { - const DatabaseAndTableWithAlias & table = tables[best_table_pos]; - table.makeQualifiedName(ast); - } + if (best_table_pos && data.source_columns.count(ast->getColumnName())) + IdentifierSemantic::setColumnQualifiedName(identifier, data.tables[best_table_pos]); } return {}; diff --git a/dbms/src/Interpreters/TranslateQualifiedNamesVisitor.h b/dbms/src/Interpreters/TranslateQualifiedNamesVisitor.h index 2b996bd0370..48d41213cb8 100644 --- a/dbms/src/Interpreters/TranslateQualifiedNamesVisitor.h +++ b/dbms/src/Interpreters/TranslateQualifiedNamesVisitor.h @@ -30,7 +30,7 @@ public: static bool needChildVisit(ASTPtr & node, const ASTPtr & child); private: - static std::vector visit(const ASTIdentifier & node, ASTPtr & ast, Data &); + static std::vector visit(ASTIdentifier & node, ASTPtr & ast, Data &); static std::vector visit(const ASTQualifiedAsterisk & node, const ASTPtr & ast, Data &); static std::vector visit(ASTTableJoin & node, const ASTPtr & ast, Data &); static std::vector visit(ASTSelectQuery & node, const ASTPtr & ast, Data &); diff --git a/dbms/src/Parsers/ASTIdentifier.cpp b/dbms/src/Parsers/ASTIdentifier.cpp index 81e6bda9741..e5500a89bd0 100644 --- a/dbms/src/Parsers/ASTIdentifier.cpp +++ b/dbms/src/Parsers/ASTIdentifier.cpp @@ -1,12 +1,27 @@ -#include #include +#include #include #include +#include namespace DB { +std::shared_ptr ASTIdentifier::createSpecial(const String & name, std::vector && name_parts) +{ + auto ret = std::make_shared(name, std::move(name_parts)); + ret->semantic->special = true; + return ret; +} + +ASTIdentifier::ASTIdentifier(const String & name_, std::vector && name_parts_) + : name(name_) + , name_parts(name_parts_) + , semantic(std::make_shared()) +{ +} + void ASTIdentifier::formatImplWithoutAlias(const FormatSettings & settings, FormatState &, FormatStateStacked) const { auto format_element = [&](const String & elem_name) @@ -74,74 +89,11 @@ bool getIdentifierName(const ASTPtr & ast, String & name) return false; } -std::optional getColumnIdentifierName(const ASTIdentifier & node) -{ - if (!node.special) - return node.name; - return {}; -} - -std::optional getColumnIdentifierName(const ASTPtr & ast) -{ - if (ast) - if (auto id = typeid_cast(ast.get())) - if (!id->special) - return id->name; - return {}; -} - -std::optional getTableIdentifierName(const ASTIdentifier & node) -{ - if (node.special) - return node.name; - return {}; -} - -std::optional getTableIdentifierName(const ASTPtr & ast) -{ - if (ast) - if (auto id = typeid_cast(ast.get())) - if (id->special) - return id->name; - return {}; -} - void setIdentifierSpecial(ASTPtr & ast) { if (ast) if (ASTIdentifier * id = typeid_cast(ast.get())) - id->setSpecial(); -} - -void addIdentifierQualifier(ASTIdentifier & identifier, const String & database, const String & table, const String & alias) -{ - if (!alias.empty()) - { - identifier.name_parts.emplace_back(alias); - } - else - { - if (!database.empty()) - identifier.name_parts.emplace_back(database); - identifier.name_parts.emplace_back(table); - } -} - -bool doesIdentifierBelongTo(const ASTIdentifier & identifier, const String & database, const String & table) -{ - size_t num_components = identifier.name_parts.size(); - if (num_components >= 3) - return identifier.name_parts[0] == database && - identifier.name_parts[1] == table; - return false; -} - -bool doesIdentifierBelongTo(const ASTIdentifier & identifier, const String & table) -{ - size_t num_components = identifier.name_parts.size(); - if (num_components >= 2) - return identifier.name_parts[0] == table; - return false; + id->semantic->special = true; } } diff --git a/dbms/src/Parsers/ASTIdentifier.h b/dbms/src/Parsers/ASTIdentifier.h index 3b7550f8d5c..5c287eb9da4 100644 --- a/dbms/src/Parsers/ASTIdentifier.h +++ b/dbms/src/Parsers/ASTIdentifier.h @@ -8,6 +8,11 @@ namespace DB { +struct IdentifierSemantic; +struct IdentifierSemanticImpl; +struct DatabaseAndTableWithAlias; + + /// Identifier (column, table or alias) class ASTIdentifier : public ASTWithAlias { @@ -15,14 +20,8 @@ public: /// The composite identifier will have a concatenated name (of the form a.b.c), /// and individual components will be available inside the name_parts. String name; - std::vector name_parts; - ASTIdentifier(const String & name_, std::vector && name_parts_ = {}) - : name(name_) - , name_parts(name_parts_) - , special(false) - { - } + ASTIdentifier(const String & name_, std::vector && name_parts_ = {}); /** Get the text that identifies this element. */ String getID(char delim) const override { return "Identifier" + (delim + name); } @@ -34,6 +33,15 @@ public: set.insert(name); } + bool compound() const { return !name_parts.empty(); } + bool isShort() const { return name_parts.empty() || name == name_parts.back(); } + + void resetWithAlias(const String & new_name) + { + name = new_name; + name_parts.clear(); + } + protected: void formatImplWithoutAlias(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override; void appendColumnNameImpl(WriteBuffer & ostr) const override; @@ -41,29 +49,21 @@ protected: private: using ASTWithAlias::children; /// ASTIdentifier is child free - bool special; /// TODO: it would be ptr to semantic here + std::vector name_parts; + std::shared_ptr semantic; /// pimpl - static std::shared_ptr createSpecial(const String & name, std::vector && name_parts = {}) - { - auto ret = std::make_shared(name, std::move(name_parts)); - ret->special = true; - return ret; - } + static std::shared_ptr createSpecial(const String & name, std::vector && name_parts = {}); - void setSpecial() { special = true; } - - friend void setIdentifierSpecial(ASTPtr &); - friend std::optional getColumnIdentifierName(const ASTIdentifier & node); - friend std::optional getColumnIdentifierName(const ASTPtr & ast); - friend std::optional getTableIdentifierName(const ASTIdentifier & node); - friend std::optional getTableIdentifierName(const ASTPtr & ast); + friend struct IdentifierSemantic; friend ASTPtr createTableIdentifier(const String & database_name, const String & table_name); + friend void setIdentifierSpecial(ASTPtr & ast); }; /// ASTIdentifier Helpers: hide casts and semantic. ASTPtr createTableIdentifier(const String & database_name, const String & table_name); +void setIdentifierSpecial(ASTPtr & ast); bool isIdentifier(const IAST * const ast); inline bool isIdentifier(const ASTPtr & ast) { return isIdentifier(ast.get()); } @@ -72,17 +72,5 @@ std::optional getIdentifierName(const IAST * const ast); inline std::optional getIdentifierName(const ASTPtr & ast) { return getIdentifierName(ast.get()); } bool getIdentifierName(const ASTPtr & ast, String & name); -/// @returns name for column identifiers -std::optional getColumnIdentifierName(const ASTIdentifier & node); -std::optional getColumnIdentifierName(const ASTPtr & ast); - -/// @returns name for 'not a column' identifiers -std::optional getTableIdentifierName(const ASTIdentifier & node); -std::optional getTableIdentifierName(const ASTPtr & ast); - -void setIdentifierSpecial(ASTPtr & ast); -void addIdentifierQualifier(ASTIdentifier & identifier, const String & database, const String & table, const String & alias); -bool doesIdentifierBelongTo(const ASTIdentifier & identifier, const String & table_or_alias); -bool doesIdentifierBelongTo(const ASTIdentifier & identifier, const String & database, const String & table); } diff --git a/dbms/src/Storages/MergeTree/MergeTreeWhereOptimizer.cpp b/dbms/src/Storages/MergeTree/MergeTreeWhereOptimizer.cpp index 66a9a6f90d2..d52a855e3f3 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeWhereOptimizer.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeWhereOptimizer.cpp @@ -1,6 +1,7 @@ #include #include #include +#include #include #include #include @@ -380,7 +381,7 @@ bool MergeTreeWhereOptimizer::cannotBeMoved(const ASTPtr & ptr) const if ("indexHint" == function_ptr->name) return true; } - else if (auto opt_name = getColumnIdentifierName(ptr)) + else if (auto opt_name = IdentifierSemantic::getColumnName(ptr)) { /// disallow moving result of ARRAY JOIN to PREWHERE if (array_joined_names.count(*opt_name) || diff --git a/dbms/src/Storages/StorageView.cpp b/dbms/src/Storages/StorageView.cpp index bb31bd81e53..97c085d16e3 100644 --- a/dbms/src/Storages/StorageView.cpp +++ b/dbms/src/Storages/StorageView.cpp @@ -2,6 +2,7 @@ #include #include #include +#include #include #include diff --git a/dbms/src/Storages/VirtualColumnUtils.cpp b/dbms/src/Storages/VirtualColumnUtils.cpp index 990a587445c..b7ea5e66c37 100644 --- a/dbms/src/Storages/VirtualColumnUtils.cpp +++ b/dbms/src/Storages/VirtualColumnUtils.cpp @@ -4,6 +4,7 @@ #include #include #include +#include #include #include @@ -96,7 +97,7 @@ static bool isValidFunction(const ASTPtr & expression, const NameSet & columns) if (!isValidFunction(expression->children[i], columns)) return false; - if (auto opt_name = getColumnIdentifierName(expression)) + if (auto opt_name = IdentifierSemantic::getColumnName(expression)) return columns.count(*opt_name); return true; diff --git a/dbms/src/TableFunctions/TableFunctionRemote.cpp b/dbms/src/TableFunctions/TableFunctionRemote.cpp index a41f1c89c49..0d457a01be3 100644 --- a/dbms/src/TableFunctions/TableFunctionRemote.cpp +++ b/dbms/src/TableFunctions/TableFunctionRemote.cpp @@ -9,6 +9,7 @@ #include #include #include +#include #include #include #include diff --git a/dbms/tests/queries/0_stateless/00597_push_down_predicate.reference b/dbms/tests/queries/0_stateless/00597_push_down_predicate.reference index df5aebabc89..bc28d4efb7e 100644 --- a/dbms/tests/queries/0_stateless/00597_push_down_predicate.reference +++ b/dbms/tests/queries/0_stateless/00597_push_down_predicate.reference @@ -13,6 +13,7 @@ 3 3 2000-01-01 1 test string 1 1 3 3 +-------Force push down------- 2000-01-01 1 test string 1 1 2000-01-01 1 test string 1 1 2000-01-01 1 test string 1 1 diff --git a/dbms/tests/queries/0_stateless/00597_push_down_predicate.sql b/dbms/tests/queries/0_stateless/00597_push_down_predicate.sql index 2f1af7fa1fa..0180fcdeb1e 100644 --- a/dbms/tests/queries/0_stateless/00597_push_down_predicate.sql +++ b/dbms/tests/queries/0_stateless/00597_push_down_predicate.sql @@ -33,6 +33,7 @@ SELECT * FROM (SELECT toUInt64(b), sum(id) AS b FROM test.test) WHERE `toUInt64( SELECT date, id, name, value FROM (SELECT date, name, value, min(id) AS id FROM test.test GROUP BY date, name, value) WHERE id = 1; SELECT * FROM (SELECT toUInt64(table_alias.b) AS a, sum(id) AS b FROM test.test AS table_alias) AS outer_table_alias WHERE outer_table_alias.b = 3; +SELECT '-------Force push down-------'; SET force_primary_key = 1; -- Optimize predicate expression with asterisk @@ -65,7 +66,7 @@ SELECT * FROM (SELECT 1 AS id, toDate('2000-01-01') AS date FROM system.numbers SELECT * FROM test.test_view WHERE id = 1; SELECT * FROM test.test_view WHERE id = 2; SELECT id FROM test.test_view WHERE id = 1; -SELECT s.id FROM test.test_view AS s WHERE id = 1; +SELECT s.id FROM test.test_view AS s WHERE s.id = 1; SELECT '-------Push to having expression, need check.-------'; SELECT id FROM (SELECT min(id) AS id FROM test.test) WHERE id = 1; -- { serverError 277 } diff --git a/dbms/tests/queries/0_stateless/00674_join_on_syntax.reference b/dbms/tests/queries/0_stateless/00674_join_on_syntax.reference index ae0944580e5..10c31e56fe8 100644 --- a/dbms/tests/queries/0_stateless/00674_join_on_syntax.reference +++ b/dbms/tests/queries/0_stateless/00674_join_on_syntax.reference @@ -46,8 +46,8 @@ join on complex expression 2 3 2 3 duplicate column names -{"a1":1,"test.tab1_copy.a1":2} -{"a1":1,"test.tab1_copy.a1":2} +{"a1":1,"tab1_copy.a1":2} +{"a1":1,"tab1_copy.a1":2} {"a1":1,"copy.a1":2} {"a1":1,"copy.a1":2} {"a1":1,"copy.a1":2} diff --git a/dbms/tests/queries/0_stateless/00674_join_on_syntax.sql b/dbms/tests/queries/0_stateless/00674_join_on_syntax.sql index 06d930d8c6d..27d83e4684c 100644 --- a/dbms/tests/queries/0_stateless/00674_join_on_syntax.sql +++ b/dbms/tests/queries/0_stateless/00674_join_on_syntax.sql @@ -76,11 +76,11 @@ select a2, b2 from test.tab2 second any left join test.tab3 third on third.a3 + select a2, b2 from test.tab2 second any left join test.tab3 third on third.a3 + test.tab3.b3 = test.tab2.a2 + second.b2; select 'duplicate column names'; -select a1, tab1_copy.a1 from test.tab1 any left join test.tab1_copy on tab1.b1 + 3 = b1 + 2 FORMAT JSONEachRow; -select a1, test.tab1_copy.a1 from test.tab1 any left join test.tab1_copy on tab1.b1 + 3 = b1 + 2 FORMAT JSONEachRow; -select a1, copy.a1 from test.tab1 any left join test.tab1_copy copy on tab1.b1 + 3 = b1 + 2 FORMAT JSONEachRow; -select a1, tab1_copy.a1 from test.tab1 any left join test.tab1_copy copy on tab1.b1 + 3 = b1 + 2 FORMAT JSONEachRow; -select a1, test.tab1_copy.a1 from test.tab1 any left join test.tab1_copy copy on tab1.b1 + 3 = b1 + 2 FORMAT JSONEachRow; +select a1, tab1_copy.a1 from test.tab1 any left join test.tab1_copy on tab1.b1 + 3 = tab1_copy.b1 + 2 FORMAT JSONEachRow; +select a1, test.tab1_copy.a1 from test.tab1 any left join test.tab1_copy on tab1.b1 + 3 = tab1_copy.b1 + 2 FORMAT JSONEachRow; +select a1, copy.a1 from test.tab1 any left join test.tab1_copy copy on tab1.b1 + 3 = tab1_copy.b1 + 2 FORMAT JSONEachRow; +select a1, tab1_copy.a1 from test.tab1 any left join test.tab1_copy copy on tab1.b1 + 3 = tab1_copy.b1 + 2 FORMAT JSONEachRow; +select a1, test.tab1_copy.a1 from test.tab1 any left join test.tab1_copy copy on tab1.b1 + 3 = tab1_copy.b1 + 2 FORMAT JSONEachRow; select 'subquery'; select a1 from test.tab1 any left join (select * from test.tab2) on b1 = a2; @@ -104,4 +104,4 @@ select a1, a2, b1, b2 from test.tab1 first any left join (select * from test.tab select a1, a2, b1, b2 from test.tab1 first any left join (select *, a2 as z from test.tab2) second on first.b1 = second.z; select a1, a2, b1, b2 from test.tab1 first any left join (select *, a2 + 1 as z from test.tab2) second on first.b1 + 1 = second.z; select tab1.a1, a2, test.tab1.b1, second.b2 from test.tab1 first any left join (select * from test.tab2) second on first.b1 = second.a2; -select a1, s.a1 from test.tab1 any left join (select * from test.tab1_copy) s on tab1.b1 + 3 = b1 + 2 FORMAT JSONEachRow; +select a1, s.a1 from test.tab1 any left join (select * from test.tab1_copy) s on tab1.b1 + 3 = s.b1 + 2 FORMAT JSONEachRow; diff --git a/dbms/tests/queries/0_stateless/00703_join_crash.sql b/dbms/tests/queries/0_stateless/00703_join_crash.sql index 32c0668b3bf..29b86e055fb 100644 --- a/dbms/tests/queries/0_stateless/00703_join_crash.sql +++ b/dbms/tests/queries/0_stateless/00703_join_crash.sql @@ -7,7 +7,7 @@ create table test.tab1_copy (a1 Int32, b1 Int32) engine = MergeTree order by a1; insert into test.tab1 values (1, 2); insert into test.tab1_copy values (2, 3); -select tab1.a1, tab1_copy.a1, tab1.b1 from test.tab1 any left join test.tab1_copy on tab1.b1 + 3 = b1 + 2; +select tab1.a1, tab1_copy.a1, tab1.b1 from test.tab1 any left join test.tab1_copy on tab1.b1 + 3 = tab1_copy.b1 + 2; drop table test.tab1; drop table test.tab1_copy; From 9539467b59ea344971a0bbe2d9ac7d4f108c94ae Mon Sep 17 00:00:00 2001 From: chertus Date: Fri, 25 Jan 2019 18:48:53 +0300 Subject: [PATCH 73/86] hotfix --- dbms/src/Interpreters/TranslateQualifiedNamesVisitor.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/src/Interpreters/TranslateQualifiedNamesVisitor.cpp b/dbms/src/Interpreters/TranslateQualifiedNamesVisitor.cpp index 382c8043f2d..016d176caba 100644 --- a/dbms/src/Interpreters/TranslateQualifiedNamesVisitor.cpp +++ b/dbms/src/Interpreters/TranslateQualifiedNamesVisitor.cpp @@ -54,7 +54,7 @@ std::vector TranslateQualifiedNamesMatcher::visit(ASTIdentifier & iden { if (IdentifierSemantic::getColumnName(identifier)) { - bool best_table_pos = 0; + size_t best_table_pos = 0; size_t best_match = 0; for (size_t i = 0; i < data.tables.size(); ++i) if (size_t match = IdentifierSemantic::canReferColumnToTable(identifier, data.tables[i])) From 038a48bb3833b2241c7afe9cab6d5b8bdc48c132 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?L=C3=A9o=20Ercolanelli?= Date: Fri, 25 Jan 2019 20:35:53 +0100 Subject: [PATCH 74/86] sumMap: implement sumMapWithOverflow --- .../AggregateFunctionSumMap.cpp | 47 ++++++++++++++++--- .../AggregateFunctionSumMap.h | 35 +++++++------- 2 files changed, 58 insertions(+), 24 deletions(-) diff --git a/dbms/src/AggregateFunctions/AggregateFunctionSumMap.cpp b/dbms/src/AggregateFunctions/AggregateFunctionSumMap.cpp index 02303b953d9..75cd62c00f1 100644 --- a/dbms/src/AggregateFunctions/AggregateFunctionSumMap.cpp +++ b/dbms/src/AggregateFunctions/AggregateFunctionSumMap.cpp @@ -12,6 +12,37 @@ namespace DB namespace { +struct WithOverflowPolicy +{ + /// Overflow, meaning that the returned type is the same as the input type. + static DataTypePtr promoteType(const DataTypePtr & data_type) { return data_type; } +}; + +struct WithoutOverflowPolicy +{ + /// No overflow, meaning we promote the types if necessary. + static DataTypePtr promoteType(const DataTypePtr & data_type) + { + if (!data_type->canBePromoted()) + throw new Exception{"Values to be summed are expected to be Numeric, Float or Decimal.", + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT}; + + return data_type->promoteNumericType(); + } +}; + +template +using SumMapWithOverflow = AggregateFunctionSumMap; + +template +using SumMapWithoutOverflow = AggregateFunctionSumMap; + +template +using SumMapFilteredWithOverflow = AggregateFunctionSumMapFiltered; + +template +using SumMapFilteredWithoutOverflow = AggregateFunctionSumMapFiltered; + using SumMapArgs = std::pair; SumMapArgs parseArguments(const std::string & name, const DataTypes & arguments) @@ -42,21 +73,23 @@ SumMapArgs parseArguments(const std::string & name, const DataTypes & arguments) return {std::move(keys_type), std::move(values_types)}; } +template