diff --git a/.clang-tidy b/.clang-tidy index de19059d09e..bb63bf2eea6 100644 --- a/.clang-tidy +++ b/.clang-tidy @@ -27,6 +27,8 @@ Checks: [ '-bugprone-not-null-terminated-result', '-bugprone-reserved-identifier', # useful but too slow, TODO retry when https://reviews.llvm.org/rG1c282052624f9d0bd273bde0b47b30c96699c6c7 is merged '-bugprone-unchecked-optional-access', + '-bugprone-crtp-constructor-accessibility', + '-bugprone-suspicious-stringview-data-usage', '-cert-dcl16-c', '-cert-dcl37-c', @@ -36,6 +38,7 @@ Checks: [ '-cert-msc51-cpp', '-cert-oop54-cpp', '-cert-oop57-cpp', + '-cert-err33-c', # Misreports on clang-19: it warns about all functions containing 'remove' in the name, not only about the standard library. '-clang-analyzer-optin.performance.Padding', @@ -99,6 +102,7 @@ Checks: [ '-modernize-use-emplace', '-modernize-use-nodiscard', '-modernize-use-trailing-return-type', + '-modernize-use-designated-initializers', '-performance-enum-size', '-performance-inefficient-string-concatenation', diff --git a/contrib/libunwind b/contrib/libunwind index d6a01c46327..8f28e64d158 160000 --- a/contrib/libunwind +++ b/contrib/libunwind @@ -1 +1 @@ -Subproject commit d6a01c46327e56fd86beb8aaa31591fcd9a6b7df +Subproject commit 8f28e64d15819d2d096badd598c7d85bebddb1f2 diff --git a/contrib/libunwind-cmake/CMakeLists.txt b/contrib/libunwind-cmake/CMakeLists.txt index 37a2f29afcf..b566e8cb9b3 100644 --- a/contrib/libunwind-cmake/CMakeLists.txt +++ b/contrib/libunwind-cmake/CMakeLists.txt @@ -4,9 +4,6 @@ set(LIBUNWIND_CXX_SOURCES "${LIBUNWIND_SOURCE_DIR}/src/libunwind.cpp" "${LIBUNWIND_SOURCE_DIR}/src/Unwind-EHABI.cpp" "${LIBUNWIND_SOURCE_DIR}/src/Unwind-seh.cpp") -if (APPLE) - set(LIBUNWIND_CXX_SOURCES ${LIBUNWIND_CXX_SOURCES} "${LIBUNWIND_SOURCE_DIR}/src/Unwind_AppleExtras.cpp") -endif () set(LIBUNWIND_C_SOURCES "${LIBUNWIND_SOURCE_DIR}/src/UnwindLevel1.c" @@ -32,6 +29,7 @@ set_target_properties(unwind PROPERTIES FOLDER "contrib/libunwind-cmake") target_include_directories(unwind SYSTEM BEFORE PUBLIC $) target_compile_definitions(unwind PRIVATE -D_LIBUNWIND_NO_HEAP=1) +target_compile_definitions(unwind PRIVATE -D_LIBUNWIND_REMEMBER_STACK_ALLOC=1) # NOTE: from this macros sizeof(unw_context_t)/sizeof(unw_cursor_t) is depends, so it should be set always target_compile_definitions(unwind PUBLIC -D_LIBUNWIND_IS_NATIVE_ONLY) diff --git a/docs/en/development/build-cross-riscv.md b/docs/en/development/build-cross-riscv.md index 759d97823e2..dd97b6081e8 100644 --- a/docs/en/development/build-cross-riscv.md +++ b/docs/en/development/build-cross-riscv.md @@ -11,7 +11,7 @@ This is for the case when you have Linux machine and want to use it to build `cl The cross-build for RISC-V 64 is based on the [Build instructions](../development/build.md), follow them first. -## Install Clang-16 +## Install Clang-18 Follow the instructions from https://apt.llvm.org/ for your Ubuntu or Debian setup or do ``` diff --git a/docs/en/sql-reference/functions/uuid-functions.md b/docs/en/sql-reference/functions/uuid-functions.md index e990023efbc..0bde207dcc9 100644 --- a/docs/en/sql-reference/functions/uuid-functions.md +++ b/docs/en/sql-reference/functions/uuid-functions.md @@ -567,12 +567,13 @@ While no standard or recommendation exists for the epoch of Snowflake IDs, imple **Syntax** ``` sql -generateSnowflakeID([expr]) +generateSnowflakeID([expr, [machine_id]]) ``` **Arguments** - `expr` — An arbitrary [expression](../../sql-reference/syntax.md#syntax-expressions) used to bypass [common subexpression elimination](../../sql-reference/functions/index.md#common-subexpression-elimination) if the function is called multiple times in a query. The value of the expression has no effect on the returned Snowflake ID. Optional. +- `machine_id` — A machine ID, the lowest 10 bits are used. [Int64](../data-types/int-uint.md). Optional. **Returned value** @@ -608,6 +609,16 @@ SELECT generateSnowflakeID(1), generateSnowflakeID(2); └────────────────────────┴────────────────────────┘ ``` +**Example with expression and a machine ID** + +``` +SELECT generateSnowflakeID('expr', 1); + +┌─generateSnowflakeID('expr', 1)─┐ +│ 7201148511606784002 │ +└────────────────────────────────┘ +``` + ## snowflakeToDateTime :::warning diff --git a/docs/zh/sql-reference/functions/conditional-functions.md b/docs/zh/sql-reference/functions/conditional-functions.md index e0f8f7a4777..378e4d897c0 100644 --- a/docs/zh/sql-reference/functions/conditional-functions.md +++ b/docs/zh/sql-reference/functions/conditional-functions.md @@ -82,14 +82,14 @@ FROM LEFT_RIGHT SELECT left, right, - if(left < right, 'left is smaller than right', 'right is greater or equal than left') AS is_smaller + if(left < right, 'left is smaller than right', 'right is smaller or equal than left') AS is_smaller FROM LEFT_RIGHT WHERE isNotNull(left) AND isNotNull(right) ┌─left─┬─right─┬─is_smaller──────────────────────────┐ │ 1 │ 3 │ left is smaller than right │ -│ 2 │ 2 │ right is greater or equal than left │ -│ 3 │ 1 │ right is greater or equal than left │ +│ 2 │ 2 │ right is smaller or equal than left │ +│ 3 │ 1 │ right is smaller or equal than left │ └──────┴───────┴─────────────────────────────────────┘ ``` diff --git a/programs/CMakeLists.txt b/programs/CMakeLists.txt index 6b3a0b16624..ce3a4659e0e 100644 --- a/programs/CMakeLists.txt +++ b/programs/CMakeLists.txt @@ -4,6 +4,9 @@ if (USE_CLANG_TIDY) set (CMAKE_CXX_CLANG_TIDY "${CLANG_TIDY_PATH}") endif () +set(MAX_LINKER_MEMORY 3500) +include(../cmake/limit_jobs.cmake) + include(${ClickHouse_SOURCE_DIR}/cmake/split_debug_symbols.cmake) # The `clickhouse` binary is a multi purpose tool that contains multiple execution modes (client, server, etc.), diff --git a/programs/client/Client.cpp b/programs/client/Client.cpp index e4b40d98819..887c5cb86bc 100644 --- a/programs/client/Client.cpp +++ b/programs/client/Client.cpp @@ -186,6 +186,8 @@ void Client::parseConnectionsCredentials(Poco::Util::AbstractConfiguration & con history_file = home_path + "/" + history_file.substr(1); config.setString("history_file", history_file); } + if (config.has(prefix + ".accept-invalid-certificate")) + config.setBool("accept-invalid-certificate", config.getBool(prefix + ".accept-invalid-certificate")); } if (!connection_name.empty() && !connection_found) @@ -277,6 +279,12 @@ void Client::initialize(Poco::Util::Application & self) else if (config().has("connection")) throw Exception(ErrorCodes::BAD_ARGUMENTS, "--connection was specified, but config does not exist"); + if (config().has("accept-invalid-certificate")) + { + config().setString("openSSL.client.invalidCertificateHandler.name", "AcceptCertificateHandler"); + config().setString("openSSL.client.verificationMode", "none"); + } + /** getenv is thread-safe in Linux glibc and in all sane libc implementations. * But the standard does not guarantee that subsequent calls will not rewrite the value by returned pointer. * @@ -731,7 +739,7 @@ bool Client::processWithFuzzing(const String & full_query) } if (auto *q = orig_ast->as()) { - if (auto *setDialect = q->changes.tryGet("dialect"); setDialect && setDialect->safeGet() == "kusto") + if (auto *set_dialect = q->changes.tryGet("dialect"); set_dialect && set_dialect->safeGet() == "kusto") return true; } diff --git a/programs/client/clickhouse-client.xml b/programs/client/clickhouse-client.xml index d0deb818c1e..c32b63413e9 100644 --- a/programs/client/clickhouse-client.xml +++ b/programs/client/clickhouse-client.xml @@ -1,5 +1,6 @@ + true @@ -72,6 +73,7 @@ Default: "hostname" will be used. --> default + 127.0.0.1 9000 diff --git a/src/Analyzer/Passes/LogicalExpressionOptimizerPass.cpp b/src/Analyzer/Passes/LogicalExpressionOptimizerPass.cpp index 854697bca9f..4a11eae15ea 100644 --- a/src/Analyzer/Passes/LogicalExpressionOptimizerPass.cpp +++ b/src/Analyzer/Passes/LogicalExpressionOptimizerPass.cpp @@ -12,6 +12,7 @@ #include #include +#include namespace DB { @@ -616,6 +617,7 @@ private: bool is_any_nullable = false; Tuple args; args.reserve(equals_functions.size()); + DataTypes tuple_element_types; /// first we create tuple from RHS of equals functions for (const auto & equals : equals_functions) { @@ -628,16 +630,18 @@ private: if (const auto * rhs_literal = equals_arguments[1]->as()) { args.push_back(rhs_literal->getValue()); + tuple_element_types.push_back(rhs_literal->getResultType()); } else { const auto * lhs_literal = equals_arguments[0]->as(); assert(lhs_literal); args.push_back(lhs_literal->getValue()); + tuple_element_types.push_back(lhs_literal->getResultType()); } } - auto rhs_node = std::make_shared(std::move(args)); + auto rhs_node = std::make_shared(std::move(args), std::make_shared(std::move(tuple_element_types))); auto in_function = std::make_shared("in"); diff --git a/src/Columns/ColumnObject.cpp b/src/Columns/ColumnObject.cpp index adcd42b16e9..9c9dade3dd8 100644 --- a/src/Columns/ColumnObject.cpp +++ b/src/Columns/ColumnObject.cpp @@ -1101,4 +1101,10 @@ void ColumnObject::finalize() checkObjectHasNoAmbiguosPaths(getKeys()); } +void ColumnObject::updateHashFast(SipHash & hash) const +{ + for (const auto & entry : subcolumns) + for (auto & part : entry->data.data) + part->updateHashFast(hash); +} } diff --git a/src/Columns/ColumnObject.h b/src/Columns/ColumnObject.h index fadf2e18779..7470dfa6302 100644 --- a/src/Columns/ColumnObject.h +++ b/src/Columns/ColumnObject.h @@ -253,7 +253,7 @@ public: const char * skipSerializedInArena(const char *) const override { throwMustBeConcrete(); } void updateHashWithValue(size_t, SipHash &) const override { throwMustBeConcrete(); } void updateWeakHash32(WeakHash32 &) const override { throwMustBeConcrete(); } - void updateHashFast(SipHash &) const override { throwMustBeConcrete(); } + void updateHashFast(SipHash & hash) const override; void expand(const Filter &, bool) override { throwMustBeConcrete(); } bool hasEqualValues() const override { throwMustBeConcrete(); } size_t byteSizeAt(size_t) const override { throwMustBeConcrete(); } diff --git a/src/Common/CollectionOfDerived.h b/src/Common/CollectionOfDerived.h new file mode 100644 index 00000000000..97c0c3fbc06 --- /dev/null +++ b/src/Common/CollectionOfDerived.h @@ -0,0 +1,184 @@ +#pragma once + +#include + +#include + +#include +#include +#include +#include +#include + + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int LOGICAL_ERROR; +} + +/* This is a collections of objects derived from ItemBase. +* Collection contains no more than one instance for each derived type. +* The derived type is used to access the instance. +*/ + +template +class CollectionOfDerivedItems +{ +public: + using Self = CollectionOfDerivedItems; + using ItemPtr = std::shared_ptr; + +private: + struct Rec + { + std::type_index type_idx; + ItemPtr ptr; + + bool operator<(const Rec & other) const + { + return type_idx < other.type_idx; + } + + bool operator<(const std::type_index & value) const + { + return type_idx < value; + } + + bool operator==(const Rec & other) const + { + return type_idx == other.type_idx; + } + }; + using Records = std::vector; + +public: + void swap(Self & other) noexcept + { + records.swap(other.records); + } + + void clear() + { + records.clear(); + } + + bool empty() const + { + return records.empty(); + } + + size_t size() const + { + return records.size(); + } + + Self clone() const + { + Self result; + result.records.reserve(records.size()); + for (const auto & rec : records) + result.records.emplace_back(rec.type_idx, rec.ptr->clone()); + return result; + } + + void append(Self && other) + { + auto middle_idx = records.size(); + std::move(other.records.begin(), other.records.end(), std::back_inserter(records)); + std::inplace_merge(records.begin(), records.begin() + middle_idx, records.end()); + chassert(isUniqTypes()); + } + + template + void add(std::shared_ptr info) + { + static_assert(std::is_base_of_v, "Template parameter must inherit items base class"); + return addImpl(std::type_index(typeid(T)), std::move(info)); + } + + template + std::shared_ptr get() const + { + static_assert(std::is_base_of_v, "Template parameter must inherit items base class"); + auto it = getImpl(std::type_index(typeid(T))); + if (it == records.cend()) + return nullptr; + auto cast = std::dynamic_pointer_cast(it->ptr); + chassert(cast); + return cast; + } + + template + std::shared_ptr extract() + { + static_assert(std::is_base_of_v, "Template parameter must inherit items base class"); + auto it = getImpl(std::type_index(typeid(T))); + if (it == records.cend()) + return nullptr; + auto cast = std::dynamic_pointer_cast(it->ptr); + chassert(cast); + + records.erase(it); + return cast; + } + + std::string debug() const + { + std::string result; + + for (auto & rec : records) + { + result.append(rec.type_idx.name()); + result.append(" "); + } + + return result; + } + +private: + bool isUniqTypes() const + { + auto uniq_it = std::adjacent_find(records.begin(), records.end()); + + return uniq_it == records.end(); + } + + void addImpl(std::type_index type_idx, ItemPtr item) + { + auto it = std::lower_bound(records.begin(), records.end(), type_idx); + + if (it == records.end()) + { + records.emplace_back(type_idx, item); + return; + } + + if (it->type_idx == type_idx) + throw Exception(ErrorCodes::LOGICAL_ERROR, "inserted items must be unique by their type, type {} is inserted twice", type_idx.name()); + + + records.emplace(it, type_idx, item); + + chassert(isUniqTypes()); + } + + Records::const_iterator getImpl(std::type_index type_idx) const + { + auto it = std::lower_bound(records.cbegin(), records.cend(), type_idx); + + if (it == records.cend()) + return records.cend(); + + if (it->type_idx != type_idx) + return records.cend(); + + return it; + } + + Records records; +}; + +} diff --git a/src/Common/memory.h b/src/Common/memory.h index caa0418fa56..dbef069b408 100644 --- a/src/Common/memory.h +++ b/src/Common/memory.h @@ -80,7 +80,7 @@ inline ALWAYS_INLINE void * newImpl(std::size_t size, TAlign... align) throw std::bad_alloc{}; } -inline ALWAYS_INLINE void * newNoExept(std::size_t size) noexcept +inline ALWAYS_INLINE void * newNoExcept(std::size_t size) noexcept { #if USE_GWP_ASAN if (unlikely(GWPAsan::GuardedAlloc.shouldSample())) @@ -99,7 +99,7 @@ inline ALWAYS_INLINE void * newNoExept(std::size_t size) noexcept return malloc(size); } -inline ALWAYS_INLINE void * newNoExept(std::size_t size, std::align_val_t align) noexcept +inline ALWAYS_INLINE void * newNoExcept(std::size_t size, std::align_val_t align) noexcept { #if USE_GWP_ASAN if (unlikely(GWPAsan::GuardedAlloc.shouldSample())) diff --git a/src/Common/new_delete.cpp b/src/Common/new_delete.cpp index e8151fbe201..80e05fc4ea0 100644 --- a/src/Common/new_delete.cpp +++ b/src/Common/new_delete.cpp @@ -87,7 +87,7 @@ void * operator new(std::size_t size, const std::nothrow_t &) noexcept { AllocationTrace trace; std::size_t actual_size = Memory::trackMemory(size, trace); - void * ptr = Memory::newNoExept(size); + void * ptr = Memory::newNoExcept(size); trace.onAlloc(ptr, actual_size); return ptr; } @@ -96,7 +96,7 @@ void * operator new[](std::size_t size, const std::nothrow_t &) noexcept { AllocationTrace trace; std::size_t actual_size = Memory::trackMemory(size, trace); - void * ptr = Memory::newNoExept(size); + void * ptr = Memory::newNoExcept(size); trace.onAlloc(ptr, actual_size); return ptr; } @@ -105,7 +105,7 @@ void * operator new(std::size_t size, std::align_val_t align, const std::nothrow { AllocationTrace trace; std::size_t actual_size = Memory::trackMemory(size, trace, align); - void * ptr = Memory::newNoExept(size, align); + void * ptr = Memory::newNoExcept(size, align); trace.onAlloc(ptr, actual_size); return ptr; } @@ -114,7 +114,7 @@ void * operator new[](std::size_t size, std::align_val_t align, const std::nothr { AllocationTrace trace; std::size_t actual_size = Memory::trackMemory(size, trace, align); - void * ptr = Memory::newNoExept(size, align); + void * ptr = Memory::newNoExcept(size, align); trace.onAlloc(ptr, actual_size); return ptr; } diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 59b4d211b27..4ea575e5375 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -36,7 +36,7 @@ class IColumn; M(Dialect, dialect, Dialect::clickhouse, "Which dialect will be used to parse query", 0)\ M(UInt64, min_compress_block_size, 65536, "The actual size of the block to compress, if the uncompressed data less than max_compress_block_size is no less than this value and no less than the volume of data for one mark.", 0) \ M(UInt64, max_compress_block_size, 1048576, "The maximum size of blocks of uncompressed data before compressing for writing to a table.", 0) \ - M(UInt64, max_block_size, DEFAULT_BLOCK_SIZE, "Maximum block size for reading", 0) \ + M(UInt64, max_block_size, DEFAULT_BLOCK_SIZE, "Maximum block size in rows for reading", 0) \ M(UInt64, max_insert_block_size, DEFAULT_INSERT_BLOCK_SIZE, "The maximum block size for insertion, if we control the creation of blocks for insertion.", 0) \ M(UInt64, min_insert_block_size_rows, DEFAULT_INSERT_BLOCK_SIZE, "Squash blocks passed to INSERT query to specified size in rows, if blocks are not big enough.", 0) \ M(UInt64, min_insert_block_size_bytes, (DEFAULT_INSERT_BLOCK_SIZE * 256), "Squash blocks passed to INSERT query to specified size in bytes, if blocks are not big enough.", 0) \ @@ -609,9 +609,8 @@ class IColumn; M(Bool, optimize_time_filter_with_preimage, true, "Optimize Date and DateTime predicates by converting functions into equivalent comparisons without conversions (e.g. toYear(col) = 2023 -> col >= '2023-01-01' AND col <= '2023-12-31')", 0) \ M(Bool, normalize_function_names, true, "Normalize function names to their canonical names", 0) \ M(Bool, enable_early_constant_folding, true, "Enable query optimization where we analyze function and subqueries results and rewrite query if there are constants there", 0) \ - M(Bool, deduplicate_blocks_in_dependent_materialized_views, false, "Should deduplicate blocks for materialized views if the block is not a duplicate for the table. Use true to always deduplicate in dependent tables.", 0) \ + M(Bool, deduplicate_blocks_in_dependent_materialized_views, false, "Should deduplicate blocks for materialized views. Use true to always deduplicate in dependent tables.", 0) \ M(Bool, throw_if_deduplication_in_dependent_materialized_views_enabled_with_async_insert, true, "Throw exception on INSERT query when the setting `deduplicate_blocks_in_dependent_materialized_views` is enabled along with `async_insert`. It guarantees correctness, because these features can't work together.", 0) \ - M(Bool, update_insert_deduplication_token_in_dependent_materialized_views, false, "Should update insert deduplication token with table identifier during insert in dependent materialized views.", 0) \ M(Bool, materialized_views_ignore_errors, false, "Allows to ignore errors for MATERIALIZED VIEW, and deliver original block to the table regardless of MVs", 0) \ M(Bool, ignore_materialized_views_with_dropped_target_table, false, "Ignore MVs with dropped target table during pushing to views", 0) \ M(Bool, use_compact_format_in_distributed_parts_names, true, "Changes format of directories names for distributed table insert parts.", 0) \ @@ -978,6 +977,7 @@ class IColumn; #define OBSOLETE_SETTINGS(M, ALIAS) \ /** Obsolete settings that do nothing but left for compatibility reasons. Remove each one after half a year of obsolescence. */ \ + MAKE_OBSOLETE(M, Bool, update_insert_deduplication_token_in_dependent_materialized_views, 0) \ MAKE_OBSOLETE(M, UInt64, max_memory_usage_for_all_queries, 0) \ MAKE_OBSOLETE(M, UInt64, multiple_joins_rewriter_version, 0) \ MAKE_OBSOLETE(M, Bool, enable_debug_queries, false) \ diff --git a/src/Functions/generateSnowflakeID.cpp b/src/Functions/generateSnowflakeID.cpp index a171b6bf86e..c95e3edf4ca 100644 --- a/src/Functions/generateSnowflakeID.cpp +++ b/src/Functions/generateSnowflakeID.cpp @@ -4,10 +4,10 @@ #include #include #include +#include #include #include "base/types.h" - namespace DB { @@ -96,10 +96,11 @@ struct SnowflakeIdRange /// 1. calculate Snowflake ID by current timestamp (`now`) /// 2. `begin = max(available, now)` /// 3. Calculate `end = begin + input_rows_count` handling `machine_seq_num` overflow -SnowflakeIdRange getRangeOfAvailableIds(const SnowflakeId & available, size_t input_rows_count) +SnowflakeIdRange getRangeOfAvailableIds(const SnowflakeId & available, uint64_t machine_id, size_t input_rows_count) + { /// 1. `now` - SnowflakeId begin = {.timestamp = getTimestamp(), .machine_id = getMachineId(), .machine_seq_num = 0}; + SnowflakeId begin = {.timestamp = getTimestamp(), .machine_id = machine_id, .machine_seq_num = 0}; /// 2. `begin` if (begin.timestamp <= available.timestamp) @@ -128,13 +129,13 @@ struct Data /// Guarantee counter monotonicity within one timestamp across all threads generating Snowflake IDs simultaneously. static inline std::atomic lowest_available_snowflake_id = 0; - SnowflakeId reserveRange(size_t input_rows_count) + SnowflakeId reserveRange(uint64_t machine_id, size_t input_rows_count) { uint64_t available_snowflake_id = lowest_available_snowflake_id.load(); SnowflakeIdRange range; do { - range = getRangeOfAvailableIds(toSnowflakeId(available_snowflake_id), input_rows_count); + range = getRangeOfAvailableIds(toSnowflakeId(available_snowflake_id), machine_id, input_rows_count); } while (!lowest_available_snowflake_id.compare_exchange_weak(available_snowflake_id, fromSnowflakeId(range.end))); /// CAS failed --> another thread updated `lowest_available_snowflake_id` and we re-try @@ -165,24 +166,32 @@ public: { FunctionArgumentDescriptors mandatory_args; FunctionArgumentDescriptors optional_args{ - {"expr", nullptr, nullptr, "Arbitrary expression"} + {"expr", nullptr, nullptr, "Arbitrary expression"}, + {"machine_id", static_cast(&isNativeUInt), static_cast(&isColumnConst), "const UInt*"} }; validateFunctionArguments(*this, arguments, mandatory_args, optional_args); return std::make_shared(); } - ColumnPtr executeImpl(const ColumnsWithTypeAndName & /*arguments*/, const DataTypePtr &, size_t input_rows_count) const override + ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t input_rows_count) const override { auto col_res = ColumnVector::create(); typename ColumnVector::Container & vec_to = col_res->getData(); - if (input_rows_count != 0) + if (input_rows_count > 0) { vec_to.resize(input_rows_count); + uint64_t machine_id = getMachineId(); + if (arguments.size() == 2) + { + machine_id = arguments[1].column->getUInt(0); + machine_id &= (1ull << machine_id_bits_count) - 1; + } + Data data; - SnowflakeId snowflake_id = data.reserveRange(input_rows_count); /// returns begin of available snowflake ids range + SnowflakeId snowflake_id = data.reserveRange(machine_id, input_rows_count); for (UInt64 & to_row : vec_to) { @@ -208,10 +217,13 @@ public: REGISTER_FUNCTION(GenerateSnowflakeID) { FunctionDocumentation::Description description = R"(Generates a Snowflake ID. The generated Snowflake ID contains the current Unix timestamp in milliseconds (41 + 1 top zero bits), followed by a machine id (10 bits), and a counter (12 bits) to distinguish IDs within a millisecond. For any given timestamp (unix_ts_ms), the counter starts at 0 and is incremented by 1 for each new Snowflake ID until the timestamp changes. In case the counter overflows, the timestamp field is incremented by 1 and the counter is reset to 0. Function generateSnowflakeID guarantees that the counter field within a timestamp increments monotonically across all function invocations in concurrently running threads and queries.)"; - FunctionDocumentation::Syntax syntax = "generateSnowflakeID([expression])"; - FunctionDocumentation::Arguments arguments = {{"expression", "The expression is used to bypass common subexpression elimination if the function is called multiple times in a query but otherwise ignored. Optional."}}; + FunctionDocumentation::Syntax syntax = "generateSnowflakeID([expression, [machine_id]])"; + FunctionDocumentation::Arguments arguments = { + {"expression", "The expression is used to bypass common subexpression elimination if the function is called multiple times in a query but otherwise ignored. Optional."}, + {"machine_id", "A machine ID, the lowest 10 bits are used. Optional."} + }; FunctionDocumentation::ReturnedValue returned_value = "A value of type UInt64"; - FunctionDocumentation::Examples examples = {{"single", "SELECT generateSnowflakeID()", "7201148511606784000"}, {"multiple", "SELECT generateSnowflakeID(1), generateSnowflakeID(2)", ""}}; + FunctionDocumentation::Examples examples = {{"no_arguments", "SELECT generateSnowflakeID()", "7201148511606784000"}, {"with_machine_id", "SELECT generateSnowflakeID(1)", "7201148511606784001"}, {"with_expression_and_machine_id", "SELECT generateSnowflakeID('some_expression', 1)", "7201148511606784002"}}; FunctionDocumentation::Categories categories = {"Snowflake ID"}; factory.registerFunction({description, syntax, arguments, returned_value, examples, categories}); diff --git a/src/Interpreters/AsynchronousInsertQueue.cpp b/src/Interpreters/AsynchronousInsertQueue.cpp index 56055e7044c..62777524c2a 100644 --- a/src/Interpreters/AsynchronousInsertQueue.cpp +++ b/src/Interpreters/AsynchronousInsertQueue.cpp @@ -301,7 +301,13 @@ void AsynchronousInsertQueue::preprocessInsertQuery(const ASTPtr & query, const auto & insert_query = query->as(); insert_query.async_insert_flush = true; - InterpreterInsertQuery interpreter(query, query_context, query_context->getSettingsRef().insert_allow_materialized_columns); + InterpreterInsertQuery interpreter( + query, + query_context, + query_context->getSettingsRef().insert_allow_materialized_columns, + /* no_squash */ false, + /* no_destination */ false, + /* async_insert */ false); auto table = interpreter.getTable(insert_query); auto sample_block = InterpreterInsertQuery::getSampleBlock(insert_query, table, table->getInMemoryMetadataPtr(), query_context); @@ -784,7 +790,12 @@ try try { interpreter = std::make_unique( - key.query, insert_context, key.settings.insert_allow_materialized_columns, false, false, true); + key.query, + insert_context, + key.settings.insert_allow_materialized_columns, + false, + false, + true); pipeline = interpreter->execute().pipeline; chassert(pipeline.pushing()); @@ -1003,7 +1014,7 @@ Chunk AsynchronousInsertQueue::processEntriesWithParsing( } Chunk chunk(executor.getResultColumns(), total_rows); - chunk.setChunkInfo(std::move(chunk_info)); + chunk.getChunkInfos().add(std::move(chunk_info)); return chunk; } @@ -1055,7 +1066,7 @@ Chunk AsynchronousInsertQueue::processPreprocessedEntries( } Chunk chunk(std::move(result_columns), total_rows); - chunk.setChunkInfo(std::move(chunk_info)); + chunk.getChunkInfos().add(std::move(chunk_info)); return chunk; } diff --git a/src/Interpreters/InterpreterCheckQuery.cpp b/src/Interpreters/InterpreterCheckQuery.cpp index 5c4fb7a1443..2b022a8cee0 100644 --- a/src/Interpreters/InterpreterCheckQuery.cpp +++ b/src/Interpreters/InterpreterCheckQuery.cpp @@ -2,6 +2,7 @@ #include #include +#include #include @@ -24,6 +25,7 @@ #include #include +#include #include #include #include @@ -93,7 +95,7 @@ Chunk getChunkFromCheckResult(const String & database, const String & table, con return Chunk(std::move(columns), 1); } -class TableCheckTask : public ChunkInfo +class TableCheckTask : public ChunkInfoCloneable { public: TableCheckTask(StorageID table_id, const std::variant & partition_or_part, ContextPtr context) @@ -112,6 +114,12 @@ public: context->checkAccess(AccessType::SHOW_TABLES, table_->getStorageID()); } + TableCheckTask(const TableCheckTask & other) + : table(other.table) + , check_data_tasks(other.check_data_tasks) + , is_finished(other.is_finished.load()) + {} + std::optional checkNext() const { if (isFinished()) @@ -123,8 +131,8 @@ public: std::this_thread::sleep_for(sleep_time); }); - IStorage::DataValidationTasksPtr check_data_tasks_ = check_data_tasks; - auto result = table->checkDataNext(check_data_tasks_); + IStorage::DataValidationTasksPtr tmp = check_data_tasks; + auto result = table->checkDataNext(tmp); is_finished = !result.has_value(); return result; } @@ -182,7 +190,7 @@ protected: /// source should return at least one row to start pipeline result.addColumn(ColumnUInt8::create(1, 1)); /// actual data stored in chunk info - result.setChunkInfo(std::move(current_check_task)); + result.getChunkInfos().add(std::move(current_check_task)); return result; } @@ -282,7 +290,7 @@ public: protected: void transform(Chunk & chunk) override { - auto table_check_task = std::dynamic_pointer_cast(chunk.getChunkInfo()); + auto table_check_task = chunk.getChunkInfos().get(); auto check_result = table_check_task->checkNext(); if (!check_result) { diff --git a/src/Interpreters/InterpreterCreateQuery.cpp b/src/Interpreters/InterpreterCreateQuery.cpp index 84d7f0a587c..ea10ad59db4 100644 --- a/src/Interpreters/InterpreterCreateQuery.cpp +++ b/src/Interpreters/InterpreterCreateQuery.cpp @@ -1777,8 +1777,13 @@ BlockIO InterpreterCreateQuery::fillTableIfNeeded(const ASTCreateQuery & create) else insert->select = create.select->clone(); - return InterpreterInsertQuery(insert, getContext(), - getContext()->getSettingsRef().insert_allow_materialized_columns).execute(); + return InterpreterInsertQuery( + insert, + getContext(), + getContext()->getSettingsRef().insert_allow_materialized_columns, + /* no_squash */ false, + /* no_destination */ false, + /* async_isnert */ false).execute(); } return {}; diff --git a/src/Interpreters/InterpreterExplainQuery.cpp b/src/Interpreters/InterpreterExplainQuery.cpp index d7addcd6e34..bedd9cb4a80 100644 --- a/src/Interpreters/InterpreterExplainQuery.cpp +++ b/src/Interpreters/InterpreterExplainQuery.cpp @@ -535,7 +535,13 @@ QueryPipeline InterpreterExplainQuery::executeImpl() } else if (dynamic_cast(ast.getExplainedQuery().get())) { - InterpreterInsertQuery insert(ast.getExplainedQuery(), getContext()); + InterpreterInsertQuery insert( + ast.getExplainedQuery(), + getContext(), + /* allow_materialized */ false, + /* no_squash */ false, + /* no_destination */ false, + /* async_isnert */ false); auto io = insert.execute(); printPipeline(io.pipeline.getProcessors(), buf); } diff --git a/src/Interpreters/InterpreterInsertQuery.cpp b/src/Interpreters/InterpreterInsertQuery.cpp index ef298d4d45a..3906cef1995 100644 --- a/src/Interpreters/InterpreterInsertQuery.cpp +++ b/src/Interpreters/InterpreterInsertQuery.cpp @@ -17,6 +17,7 @@ #include #include #include +#include #include #include #include @@ -27,6 +28,7 @@ #include #include #include +#include #include #include #include @@ -36,9 +38,11 @@ #include #include #include +#include #include #include #include +#include "base/defines.h" namespace ProfileEvents @@ -395,28 +399,349 @@ Chain InterpreterInsertQuery::buildPreSinkChain( return out; } +std::pair, std::vector> InterpreterInsertQuery::buildPreAndSinkChains(size_t presink_streams, size_t sink_streams, StoragePtr table, const StorageMetadataPtr & metadata_snapshot, const Block & query_sample_block) +{ + chassert(presink_streams > 0); + chassert(sink_streams > 0); + + ThreadGroupPtr running_group; + if (current_thread) + running_group = current_thread->getThreadGroup(); + if (!running_group) + running_group = std::make_shared(getContext()); + + std::vector sink_chains; + std::vector presink_chains; + + for (size_t i = 0; i < sink_streams; ++i) + { + auto out = buildSink(table, metadata_snapshot, /* thread_status_holder= */ nullptr, + running_group, /* elapsed_counter_ms= */ nullptr); + + sink_chains.emplace_back(std::move(out)); + } + + for (size_t i = 0; i < presink_streams; ++i) + { + auto out = buildPreSinkChain(sink_chains[0].getInputHeader(), table, metadata_snapshot, query_sample_block); + presink_chains.emplace_back(std::move(out)); + } + + return {std::move(presink_chains), std::move(sink_chains)}; +} + + +QueryPipeline InterpreterInsertQuery::buildInsertSelectPipeline(ASTInsertQuery & query, StoragePtr table) +{ + const Settings & settings = getContext()->getSettingsRef(); + + auto metadata_snapshot = table->getInMemoryMetadataPtr(); + auto query_sample_block = getSampleBlock(query, table, metadata_snapshot, getContext(), no_destination, allow_materialized); + + bool is_trivial_insert_select = false; + + if (settings.optimize_trivial_insert_select) + { + const auto & select_query = query.select->as(); + const auto & selects = select_query.list_of_selects->children; + const auto & union_modes = select_query.list_of_modes; + + /// ASTSelectWithUnionQuery is not normalized now, so it may pass some queries which can be Trivial select queries + const auto mode_is_all = [](const auto & mode) { return mode == SelectUnionMode::UNION_ALL; }; + + is_trivial_insert_select = + std::all_of(union_modes.begin(), union_modes.end(), std::move(mode_is_all)) + && std::all_of(selects.begin(), selects.end(), isTrivialSelect); + } + + ContextPtr select_context = getContext(); + + if (is_trivial_insert_select) + { + /** When doing trivial INSERT INTO ... SELECT ... FROM table, + * don't need to process SELECT with more than max_insert_threads + * and it's reasonable to set block size for SELECT to the desired block size for INSERT + * to avoid unnecessary squashing. + */ + + Settings new_settings = select_context->getSettings(); + + new_settings.max_threads = std::max(1, settings.max_insert_threads); + + if (table->prefersLargeBlocks()) + { + if (settings.min_insert_block_size_rows) + new_settings.max_block_size = settings.min_insert_block_size_rows; + if (settings.min_insert_block_size_bytes) + new_settings.preferred_block_size_bytes = settings.min_insert_block_size_bytes; + } + + auto context_for_trivial_select = Context::createCopy(context); + context_for_trivial_select->setSettings(new_settings); + context_for_trivial_select->setInsertionTable(getContext()->getInsertionTable(), getContext()->getInsertionTableColumnNames()); + + select_context = context_for_trivial_select; + } + + QueryPipelineBuilder pipeline; + + { + auto select_query_options = SelectQueryOptions(QueryProcessingStage::Complete, 1); + + if (settings.allow_experimental_analyzer) + { + InterpreterSelectQueryAnalyzer interpreter_select_analyzer(query.select, select_context, select_query_options); + pipeline = interpreter_select_analyzer.buildQueryPipeline(); + } + else + { + InterpreterSelectWithUnionQuery interpreter_select(query.select, select_context, select_query_options); + pipeline = interpreter_select.buildQueryPipeline(); + } + } + + pipeline.dropTotalsAndExtremes(); + + /// Allow to insert Nullable into non-Nullable columns, NULL values will be added as defaults values. + if (getContext()->getSettingsRef().insert_null_as_default) + { + const auto & input_columns = pipeline.getHeader().getColumnsWithTypeAndName(); + const auto & query_columns = query_sample_block.getColumnsWithTypeAndName(); + const auto & output_columns = metadata_snapshot->getColumns(); + + if (input_columns.size() == query_columns.size()) + { + for (size_t col_idx = 0; col_idx < query_columns.size(); ++col_idx) + { + /// Change query sample block columns to Nullable to allow inserting nullable columns, where NULL values will be substituted with + /// default column values (in AddingDefaultsTransform), so all values will be cast correctly. + if (isNullableOrLowCardinalityNullable(input_columns[col_idx].type) + && !isNullableOrLowCardinalityNullable(query_columns[col_idx].type) + && !isVariant(query_columns[col_idx].type) + && !isDynamic(query_columns[col_idx].type) + && output_columns.has(query_columns[col_idx].name)) + { + query_sample_block.setColumn( + col_idx, + ColumnWithTypeAndName( + makeNullableOrLowCardinalityNullable(query_columns[col_idx].column), + makeNullableOrLowCardinalityNullable(query_columns[col_idx].type), + query_columns[col_idx].name)); + } + } + } + } + + auto actions_dag = ActionsDAG::makeConvertingActions( + pipeline.getHeader().getColumnsWithTypeAndName(), + query_sample_block.getColumnsWithTypeAndName(), + ActionsDAG::MatchColumnsMode::Position); + auto actions = std::make_shared(actions_dag, ExpressionActionsSettings::fromContext(getContext(), CompileExpressions::yes)); + + pipeline.addSimpleTransform([&](const Block & in_header) -> ProcessorPtr + { + return std::make_shared(in_header, actions); + }); + + /// We need to convert Sparse columns to full, because it's destination storage + /// may not support it or may have different settings for applying Sparse serialization. + pipeline.addSimpleTransform([&](const Block & in_header) -> ProcessorPtr + { + return std::make_shared(in_header); + }); + + pipeline.addSimpleTransform([&](const Block & in_header) -> ProcessorPtr + { + auto context_ptr = getContext(); + auto counting = std::make_shared(in_header, nullptr, context_ptr->getQuota()); + counting->setProcessListElement(context_ptr->getProcessListElement()); + counting->setProgressCallback(context_ptr->getProgressCallback()); + + return counting; + }); + + size_t num_select_threads = pipeline.getNumThreads(); + + pipeline.resize(1); + + if (shouldAddSquashingFroStorage(table)) + { + pipeline.addSimpleTransform([&](const Block & in_header) -> ProcessorPtr + { + return std::make_shared( + in_header, + table->prefersLargeBlocks() ? settings.min_insert_block_size_rows : settings.max_block_size, + table->prefersLargeBlocks() ? settings.min_insert_block_size_bytes : 0ULL); + }); + } + + pipeline.addSimpleTransform([&](const Block &in_header) -> ProcessorPtr + { + return std::make_shared(in_header); + }); + + if (!settings.insert_deduplication_token.value.empty()) + { + pipeline.addSimpleTransform([&](const Block & in_header) -> ProcessorPtr + { + return std::make_shared(settings.insert_deduplication_token.value, in_header); + }); + + pipeline.addSimpleTransform([&](const Block & in_header) -> ProcessorPtr + { + return std::make_shared(in_header); + }); + } + + /// Number of streams works like this: + /// * For the SELECT, use `max_threads`, or `max_insert_threads`, or whatever + /// InterpreterSelectQuery ends up with. + /// * Use `max_insert_threads` streams for various insert-preparation steps, e.g. + /// materializing and squashing (too slow to do in one thread). That's `presink_chains`. + /// * If the table supports parallel inserts, use max_insert_threads for writing to IStorage. + /// Otherwise ResizeProcessor them down to 1 stream. + + size_t presink_streams_size = std::max(settings.max_insert_threads, pipeline.getNumStreams()); + if (settings.max_insert_threads.changed) + presink_streams_size = std::max(1, settings.max_insert_threads); + + size_t sink_streams_size = table->supportsParallelInsert() ? std::max(1, settings.max_insert_threads) : 1; + + size_t views_involved = table->isView() || !DatabaseCatalog::instance().getDependentViews(table->getStorageID()).empty(); + if (!settings.parallel_view_processing && views_involved) + { + sink_streams_size = 1; + } + + auto [presink_chains, sink_chains] = buildPreAndSinkChains( + presink_streams_size, sink_streams_size, + table, metadata_snapshot, query_sample_block); + + pipeline.resize(presink_chains.size()); + + if (shouldAddSquashingFroStorage(table)) + { + pipeline.addSimpleTransform([&](const Block & in_header) -> ProcessorPtr + { + return std::make_shared( + in_header, + table->prefersLargeBlocks() ? settings.min_insert_block_size_rows : settings.max_block_size, + table->prefersLargeBlocks() ? settings.min_insert_block_size_bytes : 0ULL); + }); + } + + for (auto & chain : presink_chains) + pipeline.addResources(chain.detachResources()); + pipeline.addChains(std::move(presink_chains)); + + pipeline.resize(sink_streams_size); + + for (auto & chain : sink_chains) + pipeline.addResources(chain.detachResources()); + pipeline.addChains(std::move(sink_chains)); + + if (!settings.parallel_view_processing && views_involved) + { + /// Don't use more threads for INSERT than for SELECT to reduce memory consumption. + if (pipeline.getNumThreads() > num_select_threads) + pipeline.setMaxThreads(num_select_threads); + } + + pipeline.setSinks([&](const Block & cur_header, QueryPipelineBuilder::StreamType) -> ProcessorPtr + { + return std::make_shared(cur_header); + }); + + return QueryPipelineBuilder::getPipeline(std::move(pipeline)); +} + + +QueryPipeline InterpreterInsertQuery::buildInsertPipeline(ASTInsertQuery & query, StoragePtr table) +{ + const Settings & settings = getContext()->getSettingsRef(); + + auto metadata_snapshot = table->getInMemoryMetadataPtr(); + auto query_sample_block = getSampleBlock(query, table, metadata_snapshot, getContext(), no_destination, allow_materialized); + + Chain chain; + + { + auto [presink_chains, sink_chains] = buildPreAndSinkChains( + /* presink_streams */1, /* sink_streams */1, + table, metadata_snapshot, query_sample_block); + + chain = std::move(presink_chains.front()); + chain.appendChain(std::move(sink_chains.front())); + } + + if (!settings.insert_deduplication_token.value.empty()) + { + chain.addSource(std::make_shared(chain.getInputHeader())); + chain.addSource(std::make_shared(settings.insert_deduplication_token.value, chain.getInputHeader())); + } + + chain.addSource(std::make_shared(chain.getInputHeader())); + + if (shouldAddSquashingFroStorage(table)) + { + bool table_prefers_large_blocks = table->prefersLargeBlocks(); + + auto squashing = std::make_shared( + chain.getInputHeader(), + table_prefers_large_blocks ? settings.min_insert_block_size_rows : settings.max_block_size, + table_prefers_large_blocks ? settings.min_insert_block_size_bytes : 0ULL); + + chain.addSource(std::move(squashing)); + + auto balancing = std::make_shared( + chain.getInputHeader(), + table_prefers_large_blocks ? settings.min_insert_block_size_rows : settings.max_block_size, + table_prefers_large_blocks ? settings.min_insert_block_size_bytes : 0ULL); + + chain.addSource(std::move(balancing)); + } + + auto context_ptr = getContext(); + auto counting = std::make_shared(chain.getInputHeader(), nullptr, context_ptr->getQuota()); + counting->setProcessListElement(context_ptr->getProcessListElement()); + counting->setProgressCallback(context_ptr->getProgressCallback()); + chain.addSource(std::move(counting)); + + QueryPipeline pipeline = QueryPipeline(std::move(chain)); + + pipeline.setNumThreads(std::min(pipeline.getNumThreads(), settings.max_threads)); + pipeline.setConcurrencyControl(settings.use_concurrency_control); + + if (query.hasInlinedData() && !async_insert) + { + /// can execute without additional data + auto format = getInputFormatFromASTInsertQuery(query_ptr, true, query_sample_block, getContext(), nullptr); + for (auto && buffer : owned_buffers) + format->addBuffer(std::move(buffer)); + + auto pipe = getSourceFromInputFormat(query_ptr, std::move(format), getContext(), nullptr); + pipeline.complete(std::move(pipe)); + } + + return pipeline; +} + + BlockIO InterpreterInsertQuery::execute() { const Settings & settings = getContext()->getSettingsRef(); auto & query = query_ptr->as(); - QueryPipelineBuilder pipeline; - std::optional distributed_pipeline; - QueryPlanResourceHolder resources; StoragePtr table = getTable(query); checkStorageSupportsTransactionsIfNeeded(table, getContext()); - StoragePtr inner_table; - if (const auto * mv = dynamic_cast(table.get())) - inner_table = mv->getTargetTable(); - if (query.partition_by && !table->supportsPartitionBy()) throw Exception(ErrorCodes::NOT_IMPLEMENTED, "PARTITION BY clause is not supported by storage"); auto table_lock = table->lockForShare(getContext()->getInitialQueryId(), settings.lock_acquire_timeout); - auto metadata_snapshot = table->getInMemoryMetadataPtr(); + auto metadata_snapshot = table->getInMemoryMetadataPtr(); auto query_sample_block = getSampleBlock(query, table, metadata_snapshot, getContext(), no_destination, allow_materialized); /// For table functions we check access while executing @@ -424,320 +749,45 @@ BlockIO InterpreterInsertQuery::execute() if (!query.table_function) getContext()->checkAccess(AccessType::INSERT, query.table_id, query_sample_block.getNames()); - if (query.select && settings.parallel_distributed_insert_select) - // Distributed INSERT SELECT - distributed_pipeline = table->distributedWrite(query, getContext()); - - std::vector presink_chains; - std::vector sink_chains; - if (!distributed_pipeline) + if (!allow_materialized) { - /// Number of streams works like this: - /// * For the SELECT, use `max_threads`, or `max_insert_threads`, or whatever - /// InterpreterSelectQuery ends up with. - /// * Use `max_insert_threads` streams for various insert-preparation steps, e.g. - /// materializing and squashing (too slow to do in one thread). That's `presink_chains`. - /// * If the table supports parallel inserts, use the same streams for writing to IStorage. - /// Otherwise ResizeProcessor them down to 1 stream. - /// * If it's not an INSERT SELECT, forget all that and use one stream. - size_t pre_streams_size = 1; - size_t sink_streams_size = 1; - - if (query.select) - { - bool is_trivial_insert_select = false; - - if (settings.optimize_trivial_insert_select) - { - const auto & select_query = query.select->as(); - const auto & selects = select_query.list_of_selects->children; - const auto & union_modes = select_query.list_of_modes; - - /// ASTSelectWithUnionQuery is not normalized now, so it may pass some queries which can be Trivial select queries - const auto mode_is_all = [](const auto & mode) { return mode == SelectUnionMode::UNION_ALL; }; - - is_trivial_insert_select = - std::all_of(union_modes.begin(), union_modes.end(), std::move(mode_is_all)) - && std::all_of(selects.begin(), selects.end(), isTrivialSelect); - } - - if (is_trivial_insert_select) - { - /** When doing trivial INSERT INTO ... SELECT ... FROM table, - * don't need to process SELECT with more than max_insert_threads - * and it's reasonable to set block size for SELECT to the desired block size for INSERT - * to avoid unnecessary squashing. - */ - - Settings new_settings = getContext()->getSettings(); - - new_settings.max_threads = std::max(1, settings.max_insert_threads); - - if (table->prefersLargeBlocks()) - { - if (settings.min_insert_block_size_rows) - new_settings.max_block_size = settings.min_insert_block_size_rows; - if (settings.min_insert_block_size_bytes) - new_settings.preferred_block_size_bytes = settings.min_insert_block_size_bytes; - } - - auto new_context = Context::createCopy(context); - new_context->setSettings(new_settings); - new_context->setInsertionTable(getContext()->getInsertionTable(), getContext()->getInsertionTableColumnNames()); - - auto select_query_options = SelectQueryOptions(QueryProcessingStage::Complete, 1); - - if (settings.allow_experimental_analyzer) - { - InterpreterSelectQueryAnalyzer interpreter_select_analyzer(query.select, new_context, select_query_options); - pipeline = interpreter_select_analyzer.buildQueryPipeline(); - } - else - { - InterpreterSelectWithUnionQuery interpreter_select(query.select, new_context, select_query_options); - pipeline = interpreter_select.buildQueryPipeline(); - } - } - else - { - /// Passing 1 as subquery_depth will disable limiting size of intermediate result. - auto select_query_options = SelectQueryOptions(QueryProcessingStage::Complete, 1); - - if (settings.allow_experimental_analyzer) - { - InterpreterSelectQueryAnalyzer interpreter_select_analyzer(query.select, getContext(), select_query_options); - pipeline = interpreter_select_analyzer.buildQueryPipeline(); - } - else - { - InterpreterSelectWithUnionQuery interpreter_select(query.select, getContext(), select_query_options); - pipeline = interpreter_select.buildQueryPipeline(); - } - } - - pipeline.dropTotalsAndExtremes(); - - if (settings.max_insert_threads > 1) - { - auto table_id = table->getStorageID(); - auto views = DatabaseCatalog::instance().getDependentViews(table_id); - - /// It breaks some views-related tests and we have dedicated `parallel_view_processing` for views, so let's just skip them. - /// Also it doesn't make sense to reshuffle data if storage doesn't support parallel inserts. - const bool resize_to_max_insert_threads = !table->isView() && views.empty() && table->supportsParallelInsert(); - pre_streams_size = resize_to_max_insert_threads ? settings.max_insert_threads - : std::min(settings.max_insert_threads, pipeline.getNumStreams()); - - /// Deduplication when passing insert_deduplication_token breaks if using more than one thread - if (!settings.insert_deduplication_token.toString().empty()) - { - LOG_DEBUG( - getLogger("InsertQuery"), - "Insert-select query using insert_deduplication_token, setting streams to 1 to avoid deduplication issues"); - pre_streams_size = 1; - } - - if (table->supportsParallelInsert()) - sink_streams_size = pre_streams_size; - } - - pipeline.resize(pre_streams_size); - - /// Allow to insert Nullable into non-Nullable columns, NULL values will be added as defaults values. - if (getContext()->getSettingsRef().insert_null_as_default) - { - const auto & input_columns = pipeline.getHeader().getColumnsWithTypeAndName(); - const auto & query_columns = query_sample_block.getColumnsWithTypeAndName(); - const auto & output_columns = metadata_snapshot->getColumns(); - - if (input_columns.size() == query_columns.size()) - { - for (size_t col_idx = 0; col_idx < query_columns.size(); ++col_idx) - { - /// Change query sample block columns to Nullable to allow inserting nullable columns, where NULL values will be substituted with - /// default column values (in AddingDefaultsTransform), so all values will be cast correctly. - if (isNullableOrLowCardinalityNullable(input_columns[col_idx].type) - && !isNullableOrLowCardinalityNullable(query_columns[col_idx].type) - && !isVariant(query_columns[col_idx].type) - && !isDynamic(query_columns[col_idx].type) - && output_columns.has(query_columns[col_idx].name)) - query_sample_block.setColumn(col_idx, ColumnWithTypeAndName(makeNullableOrLowCardinalityNullable(query_columns[col_idx].column), makeNullableOrLowCardinalityNullable(query_columns[col_idx].type), query_columns[col_idx].name)); - } - } - } - } - - ThreadGroupPtr running_group; - if (current_thread) - running_group = current_thread->getThreadGroup(); - if (!running_group) - running_group = std::make_shared(getContext()); - for (size_t i = 0; i < sink_streams_size; ++i) - { - auto out = buildSink(table, metadata_snapshot, /* thread_status_holder= */ nullptr, - running_group, /* elapsed_counter_ms= */ nullptr); - sink_chains.emplace_back(std::move(out)); - } - for (size_t i = 0; i < pre_streams_size; ++i) - { - auto out = buildPreSinkChain(sink_chains[0].getInputHeader(), table, metadata_snapshot, query_sample_block); - presink_chains.emplace_back(std::move(out)); - } + for (const auto & column : metadata_snapshot->getColumns()) + if (column.default_desc.kind == ColumnDefaultKind::Materialized && query_sample_block.has(column.name)) + throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Cannot insert column {}, because it is MATERIALIZED column.", column.name); } BlockIO res; - /// What type of query: INSERT or INSERT SELECT or INSERT WATCH? - if (distributed_pipeline) + if (query.select) { - res.pipeline = std::move(*distributed_pipeline); - } - else if (query.select) - { - const auto & header = presink_chains.at(0).getInputHeader(); - auto actions_dag = ActionsDAG::makeConvertingActions( - pipeline.getHeader().getColumnsWithTypeAndName(), - header.getColumnsWithTypeAndName(), - ActionsDAG::MatchColumnsMode::Position); - auto actions = std::make_shared(actions_dag, ExpressionActionsSettings::fromContext(getContext(), CompileExpressions::yes)); - - pipeline.addSimpleTransform([&](const Block & in_header) -> ProcessorPtr + if (settings.parallel_distributed_insert_select) { - return std::make_shared(in_header, actions); - }); - - /// We need to convert Sparse columns to full, because it's destination storage - /// may not support it or may have different settings for applying Sparse serialization. - pipeline.addSimpleTransform([&](const Block & in_header) -> ProcessorPtr - { - return std::make_shared(in_header); - }); - - pipeline.addSimpleTransform([&](const Block & in_header) -> ProcessorPtr - { - auto context_ptr = getContext(); - auto counting = std::make_shared(in_header, nullptr, context_ptr->getQuota()); - counting->setProcessListElement(context_ptr->getProcessListElement()); - counting->setProgressCallback(context_ptr->getProgressCallback()); - - return counting; - }); - - if (shouldAddSquashingFroStorage(table)) - { - bool table_prefers_large_blocks = table->prefersLargeBlocks(); - - size_t threads = presink_chains.size(); - - pipeline.resize(1); - - pipeline.addTransform(std::make_shared( - header, - table_prefers_large_blocks ? settings.min_insert_block_size_rows : settings.max_block_size, - table_prefers_large_blocks ? settings.min_insert_block_size_bytes : 0ULL)); - - pipeline.resize(threads); - - pipeline.addSimpleTransform([&](const Block & in_header) -> ProcessorPtr + auto distributed = table->distributedWrite(query, getContext()); + if (distributed) { - return std::make_shared( - in_header, - table_prefers_large_blocks ? settings.min_insert_block_size_rows : settings.max_block_size, - table_prefers_large_blocks ? settings.min_insert_block_size_bytes : 0ULL); - }); + res.pipeline = std::move(*distributed); + } + else + { + res.pipeline = buildInsertSelectPipeline(query, table); + } } - - size_t num_select_threads = pipeline.getNumThreads(); - - for (auto & chain : presink_chains) - resources = chain.detachResources(); - for (auto & chain : sink_chains) - resources = chain.detachResources(); - - pipeline.addChains(std::move(presink_chains)); - pipeline.resize(sink_chains.size()); - pipeline.addChains(std::move(sink_chains)); - - if (!settings.parallel_view_processing) + else { - /// Don't use more threads for INSERT than for SELECT to reduce memory consumption. - if (pipeline.getNumThreads() > num_select_threads) - pipeline.setMaxThreads(num_select_threads); + res.pipeline = buildInsertSelectPipeline(query, table); } - else if (pipeline.getNumThreads() < settings.max_threads) - { - /// It is possible for query to have max_threads=1, due to optimize_trivial_insert_select, - /// however in case of parallel_view_processing and multiple views, views can still be processed in parallel. - /// - /// Note, number of threads will be limited by buildPushingToViewsChain() to max_threads. - pipeline.setMaxThreads(settings.max_threads); - } - - pipeline.setSinks([&](const Block & cur_header, QueryPipelineBuilder::StreamType) -> ProcessorPtr - { - return std::make_shared(cur_header); - }); - - if (!allow_materialized) - { - for (const auto & column : metadata_snapshot->getColumns()) - if (column.default_desc.kind == ColumnDefaultKind::Materialized && header.has(column.name)) - throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Cannot insert column {}, because it is MATERIALIZED column.", column.name); - } - - res.pipeline = QueryPipelineBuilder::getPipeline(std::move(pipeline)); } else { - auto & chain = presink_chains.at(0); - chain.appendChain(std::move(sink_chains.at(0))); - - if (shouldAddSquashingFroStorage(table)) - { - bool table_prefers_large_blocks = table->prefersLargeBlocks(); - - auto squashing = std::make_shared( - chain.getInputHeader(), - table_prefers_large_blocks ? settings.min_insert_block_size_rows : settings.max_block_size, - table_prefers_large_blocks ? settings.min_insert_block_size_bytes : 0ULL); - - chain.addSource(std::move(squashing)); - - auto balancing = std::make_shared( - chain.getInputHeader(), - table_prefers_large_blocks ? settings.min_insert_block_size_rows : settings.max_block_size, - table_prefers_large_blocks ? settings.min_insert_block_size_bytes : 0ULL); - - chain.addSource(std::move(balancing)); - } - - auto context_ptr = getContext(); - auto counting = std::make_shared(chain.getInputHeader(), nullptr, context_ptr->getQuota()); - counting->setProcessListElement(context_ptr->getProcessListElement()); - counting->setProgressCallback(context_ptr->getProgressCallback()); - chain.addSource(std::move(counting)); - - res.pipeline = QueryPipeline(std::move(presink_chains[0])); - res.pipeline.setNumThreads(std::min(res.pipeline.getNumThreads(), settings.max_threads)); - res.pipeline.setConcurrencyControl(settings.use_concurrency_control); - - if (query.hasInlinedData() && !async_insert) - { - /// can execute without additional data - auto format = getInputFormatFromASTInsertQuery(query_ptr, true, query_sample_block, getContext(), nullptr); - for (auto && buffer : owned_buffers) - format->addBuffer(std::move(buffer)); - - auto pipe = getSourceFromInputFormat(query_ptr, std::move(format), getContext(), nullptr); - res.pipeline.complete(std::move(pipe)); - } + res.pipeline = buildInsertPipeline(query, table); } - res.pipeline.addResources(std::move(resources)); - res.pipeline.addStorageHolder(table); - if (inner_table) - res.pipeline.addStorageHolder(inner_table); + + if (const auto * mv = dynamic_cast(table.get())) + res.pipeline.addStorageHolder(mv->getTargetTable()); + + LOG_TEST(getLogger("InterpreterInsertQuery"), "Pipeline could use up to {} thread", res.pipeline.getNumThreads()); return res; } @@ -758,17 +808,27 @@ void InterpreterInsertQuery::extendQueryLogElemImpl(QueryLogElement & elem, Cont } } + void InterpreterInsertQuery::extendQueryLogElemImpl(QueryLogElement & elem, const ASTPtr &, ContextPtr context_) const { extendQueryLogElemImpl(elem, context_); } + void registerInterpreterInsertQuery(InterpreterFactory & factory) { auto create_fn = [] (const InterpreterFactory::Arguments & args) { - return std::make_unique(args.query, args.context, args.allow_materialized); + return std::make_unique( + args.query, + args.context, + args.allow_materialized, + /* no_squash */false, + /* no_destination */false, + /* async_insert */false); }; factory.registerInterpreter("InterpreterInsertQuery", create_fn); } + + } diff --git a/src/Interpreters/InterpreterInsertQuery.h b/src/Interpreters/InterpreterInsertQuery.h index bf73fb2a319..894c7c42144 100644 --- a/src/Interpreters/InterpreterInsertQuery.h +++ b/src/Interpreters/InterpreterInsertQuery.h @@ -23,10 +23,10 @@ public: InterpreterInsertQuery( const ASTPtr & query_ptr_, ContextPtr context_, - bool allow_materialized_ = false, - bool no_squash_ = false, - bool no_destination_ = false, - bool async_insert_ = false); + bool allow_materialized_, + bool no_squash_, + bool no_destination, + bool async_insert_); /** Prepare a request for execution. Return block streams * - the stream into which you can write data to execute the query, if INSERT; @@ -73,12 +73,17 @@ private: ASTPtr query_ptr; const bool allow_materialized; - const bool no_squash; - const bool no_destination; + bool no_squash = false; + bool no_destination = false; const bool async_insert; std::vector> owned_buffers; + std::pair, std::vector> buildPreAndSinkChains(size_t presink_streams, size_t sink_streams, StoragePtr table, const StorageMetadataPtr & metadata_snapshot, const Block & query_sample_block); + + QueryPipeline buildInsertSelectPipeline(ASTInsertQuery & query, StoragePtr table); + QueryPipeline buildInsertPipeline(ASTInsertQuery & query, StoragePtr table); + Chain buildSink( const StoragePtr & table, const StorageMetadataPtr & metadata_snapshot, diff --git a/src/Interpreters/Squashing.cpp b/src/Interpreters/Squashing.cpp index f8b6a6542cc..5fe0ba9a737 100644 --- a/src/Interpreters/Squashing.cpp +++ b/src/Interpreters/Squashing.cpp @@ -1,6 +1,9 @@ #include #include +#include "Common/Logger.h" +#include "Common/logger_useful.h" #include +#include namespace DB @@ -11,24 +14,33 @@ namespace ErrorCodes } Squashing::Squashing(Block header_, size_t min_block_size_rows_, size_t min_block_size_bytes_) - : header(header_) - , min_block_size_rows(min_block_size_rows_) + : min_block_size_rows(min_block_size_rows_) , min_block_size_bytes(min_block_size_bytes_) + , header(header_) { } Chunk Squashing::flush() { - return convertToChunk(std::move(chunks_to_merge_vec)); + if (!accumulated) + return {}; + + auto result = convertToChunk(extract()); + chassert(result); + return result; } Chunk Squashing::squash(Chunk && input_chunk) { - if (!input_chunk.hasChunkInfo()) + if (!input_chunk) return Chunk(); - const auto *info = getInfoFromChunk(input_chunk); - return squash(info->chunks); + auto squash_info = input_chunk.getChunkInfos().extract(); + + if (!squash_info) + throw Exception(ErrorCodes::LOGICAL_ERROR, "There is no ChunksToSquash in ChunkInfoPtr"); + + return squash(std::move(squash_info->chunks), std::move(input_chunk.getChunkInfos())); } Chunk Squashing::add(Chunk && input_chunk) @@ -37,67 +49,62 @@ Chunk Squashing::add(Chunk && input_chunk) return {}; /// Just read block is already enough. - if (isEnoughSize(input_chunk.getNumRows(), input_chunk.bytes())) + if (isEnoughSize(input_chunk)) { /// If no accumulated data, return just read block. - if (chunks_to_merge_vec.empty()) + if (!accumulated) { - chunks_to_merge_vec.push_back(std::move(input_chunk)); - Chunk res_chunk = convertToChunk(std::move(chunks_to_merge_vec)); - chunks_to_merge_vec.clear(); - return res_chunk; + accumulated.add(std::move(input_chunk)); + return convertToChunk(extract()); } /// Return accumulated data (maybe it has small size) and place new block to accumulated data. - Chunk res_chunk = convertToChunk(std::move(chunks_to_merge_vec)); - chunks_to_merge_vec.clear(); - changeCurrentSize(input_chunk.getNumRows(), input_chunk.bytes()); - chunks_to_merge_vec.push_back(std::move(input_chunk)); + Chunk res_chunk = convertToChunk(extract()); + accumulated.add(std::move(input_chunk)); return res_chunk; } /// Accumulated block is already enough. - if (isEnoughSize(accumulated_size.rows, accumulated_size.bytes)) + if (isEnoughSize()) { /// Return accumulated data and place new block to accumulated data. - Chunk res_chunk = convertToChunk(std::move(chunks_to_merge_vec)); - chunks_to_merge_vec.clear(); - changeCurrentSize(input_chunk.getNumRows(), input_chunk.bytes()); - chunks_to_merge_vec.push_back(std::move(input_chunk)); + Chunk res_chunk = convertToChunk(extract()); + accumulated.add(std::move(input_chunk)); return res_chunk; } /// Pushing data into accumulating vector - expandCurrentSize(input_chunk.getNumRows(), input_chunk.bytes()); - chunks_to_merge_vec.push_back(std::move(input_chunk)); + accumulated.add(std::move(input_chunk)); /// If accumulated data is big enough, we send it - if (isEnoughSize(accumulated_size.rows, accumulated_size.bytes)) - { - Chunk res_chunk = convertToChunk(std::move(chunks_to_merge_vec)); - changeCurrentSize(0, 0); - chunks_to_merge_vec.clear(); - return res_chunk; - } + if (isEnoughSize()) + return convertToChunk(extract()); + return {}; } -Chunk Squashing::convertToChunk(std::vector && chunks) const +Chunk Squashing::convertToChunk(CurrentData && data) const { - if (chunks.empty()) + if (data.chunks.empty()) return {}; auto info = std::make_shared(); - info->chunks = std::move(chunks); + info->chunks = std::move(data.chunks); - chunks.clear(); + // It is imortant that chunk is not empty, it has to have columns even if they are empty + // Sometimes there are could be no columns in header but not empty rows in chunks + // That happens when we intend to add defaults for the missing columns after + auto aggr_chunk = Chunk(header.getColumns(), 0); + if (header.columns() == 0) + aggr_chunk = Chunk(header.getColumns(), data.getRows()); - return Chunk(header.cloneEmptyColumns(), 0, info); + aggr_chunk.getChunkInfos().add(std::move(info)); + chassert(aggr_chunk); + return aggr_chunk; } -Chunk Squashing::squash(std::vector & input_chunks) +Chunk Squashing::squash(std::vector && input_chunks, Chunk::ChunkInfoCollection && infos) { - Chunk accumulated_chunk; std::vector mutable_columns = {}; size_t rows = 0; for (const Chunk & chunk : input_chunks) @@ -119,35 +126,17 @@ Chunk Squashing::squash(std::vector & input_chunks) for (size_t j = 0, size = mutable_columns.size(); j < size; ++j) { const auto source_column = columns[j]; - mutable_columns[j]->insertRangeFrom(*source_column, 0, source_column->size()); } } - accumulated_chunk.setColumns(std::move(mutable_columns), rows); - return accumulated_chunk; -} -const ChunksToSquash* Squashing::getInfoFromChunk(const Chunk & chunk) -{ - const auto& info = chunk.getChunkInfo(); - const auto * agg_info = typeid_cast(info.get()); + Chunk result; + result.setColumns(std::move(mutable_columns), rows); + result.setChunkInfos(infos); + result.getChunkInfos().append(std::move(input_chunks.back().getChunkInfos())); - if (!agg_info) - throw Exception(ErrorCodes::LOGICAL_ERROR, "There is no ChunksToSquash in ChunkInfoPtr"); - - return agg_info; -} - -void Squashing::expandCurrentSize(size_t rows, size_t bytes) -{ - accumulated_size.rows += rows; - accumulated_size.bytes += bytes; -} - -void Squashing::changeCurrentSize(size_t rows, size_t bytes) -{ - accumulated_size.rows = rows; - accumulated_size.bytes = bytes; + chassert(result); + return result; } bool Squashing::isEnoughSize(size_t rows, size_t bytes) const @@ -156,4 +145,29 @@ bool Squashing::isEnoughSize(size_t rows, size_t bytes) const || (min_block_size_rows && rows >= min_block_size_rows) || (min_block_size_bytes && bytes >= min_block_size_bytes); } + +bool Squashing::isEnoughSize() const +{ + return isEnoughSize(accumulated.getRows(), accumulated.getBytes()); +}; + +bool Squashing::isEnoughSize(const Chunk & chunk) const +{ + return isEnoughSize(chunk.getNumRows(), chunk.bytes()); +} + +void Squashing::CurrentData::add(Chunk && chunk) +{ + rows += chunk.getNumRows(); + bytes += chunk.bytes(); + chunks.push_back(std::move(chunk)); +} + +Squashing::CurrentData Squashing::extract() +{ + auto result = std::move(accumulated); + accumulated = {}; + return result; +} + } diff --git a/src/Interpreters/Squashing.h b/src/Interpreters/Squashing.h index d76cca60e41..71ed4c4185a 100644 --- a/src/Interpreters/Squashing.h +++ b/src/Interpreters/Squashing.h @@ -8,9 +8,18 @@ namespace DB { -struct ChunksToSquash : public ChunkInfo +class ChunksToSquash : public ChunkInfoCloneable { - mutable std::vector chunks = {}; +public: + ChunksToSquash() = default; + ChunksToSquash(const ChunksToSquash & other) + { + chunks.reserve(other.chunks.size()); + for (const auto & chunk: other.chunks) + chunks.push_back(chunk.clone()); + } + + std::vector chunks = {}; }; /** Merging consecutive passed blocks to specified minimum size. @@ -36,34 +45,37 @@ public: static Chunk squash(Chunk && input_chunk); Chunk flush(); - bool isDataLeft() - { - return !chunks_to_merge_vec.empty(); - } + void setHeader(Block header_) { header = std::move(header_); } + const Block & getHeader() const { return header; } - Block header; private: - struct CurrentSize + struct CurrentData { + std::vector chunks = {}; size_t rows = 0; size_t bytes = 0; + + explicit operator bool () const { return !chunks.empty(); } + size_t getRows() const { return rows; } + size_t getBytes() const { return bytes; } + void add(Chunk && chunk); }; - std::vector chunks_to_merge_vec = {}; - size_t min_block_size_rows; - size_t min_block_size_bytes; + const size_t min_block_size_rows; + const size_t min_block_size_bytes; + Block header; - CurrentSize accumulated_size; + CurrentData accumulated; - static const ChunksToSquash * getInfoFromChunk(const Chunk & chunk); + static Chunk squash(std::vector && input_chunks, Chunk::ChunkInfoCollection && infos); - static Chunk squash(std::vector & input_chunks); - - void expandCurrentSize(size_t rows, size_t bytes); - void changeCurrentSize(size_t rows, size_t bytes); + bool isEnoughSize() const; bool isEnoughSize(size_t rows, size_t bytes) const; + bool isEnoughSize(const Chunk & chunk) const; - Chunk convertToChunk(std::vector && chunks) const; + CurrentData extract(); + + Chunk convertToChunk(CurrentData && data) const; }; } diff --git a/src/Interpreters/SystemLog.cpp b/src/Interpreters/SystemLog.cpp index 7d84efba1b5..572481e6b12 100644 --- a/src/Interpreters/SystemLog.cpp +++ b/src/Interpreters/SystemLog.cpp @@ -544,7 +544,13 @@ void SystemLog::flushImpl(const std::vector & to_flush, insert_context->makeQueryContext(); addSettingsForQuery(insert_context, IAST::QueryKind::Insert); - InterpreterInsertQuery interpreter(query_ptr, insert_context); + InterpreterInsertQuery interpreter( + query_ptr, + insert_context, + /* allow_materialized */ false, + /* no_squash */ false, + /* no_destination */ false, + /* async_isnert */ false); BlockIO io = interpreter.execute(); PushingPipelineExecutor executor(io.pipeline); diff --git a/src/Interpreters/TreeRewriter.cpp b/src/Interpreters/TreeRewriter.cpp index a3c5a7ed3ed..6ce6f5e454e 100644 --- a/src/Interpreters/TreeRewriter.cpp +++ b/src/Interpreters/TreeRewriter.cpp @@ -1188,7 +1188,7 @@ bool TreeRewriterResult::collectUsedColumns(const ASTPtr & query, bool is_select } } - /// Check for dynamic subcolums in unknown required columns. + /// Check for dynamic subcolumns in unknown required columns. if (!unknown_required_source_columns.empty()) { for (const NameAndTypePair & pair : source_columns_ordinary) diff --git a/src/Processors/Chunk.cpp b/src/Processors/Chunk.cpp index 5f6cf2f7230..4466be5b3a7 100644 --- a/src/Processors/Chunk.cpp +++ b/src/Processors/Chunk.cpp @@ -19,14 +19,6 @@ Chunk::Chunk(DB::Columns columns_, UInt64 num_rows_) : columns(std::move(columns checkNumRowsIsConsistent(); } -Chunk::Chunk(Columns columns_, UInt64 num_rows_, ChunkInfoPtr chunk_info_) - : columns(std::move(columns_)) - , num_rows(num_rows_) - , chunk_info(std::move(chunk_info_)) -{ - checkNumRowsIsConsistent(); -} - static Columns unmuteColumns(MutableColumns && mutable_columns) { Columns columns; @@ -43,17 +35,11 @@ Chunk::Chunk(MutableColumns columns_, UInt64 num_rows_) checkNumRowsIsConsistent(); } -Chunk::Chunk(MutableColumns columns_, UInt64 num_rows_, ChunkInfoPtr chunk_info_) - : columns(unmuteColumns(std::move(columns_))) - , num_rows(num_rows_) - , chunk_info(std::move(chunk_info_)) -{ - checkNumRowsIsConsistent(); -} - Chunk Chunk::clone() const { - return Chunk(getColumns(), getNumRows(), chunk_info); + auto tmp = Chunk(getColumns(), getNumRows()); + tmp.setChunkInfos(chunk_infos.clone()); + return tmp; } void Chunk::setColumns(Columns columns_, UInt64 num_rows_) diff --git a/src/Processors/Chunk.h b/src/Processors/Chunk.h index 4f753798eaa..1348966c0d3 100644 --- a/src/Processors/Chunk.h +++ b/src/Processors/Chunk.h @@ -1,7 +1,9 @@ #pragma once +#include #include -#include + +#include namespace DB { @@ -9,11 +11,29 @@ namespace DB class ChunkInfo { public: - virtual ~ChunkInfo() = default; + using Ptr = std::shared_ptr; + ChunkInfo() = default; + ChunkInfo(const ChunkInfo&) = default; + ChunkInfo(ChunkInfo&&) = default; + + virtual Ptr clone() const = 0; + virtual ~ChunkInfo() = default; }; -using ChunkInfoPtr = std::shared_ptr; + +template +class ChunkInfoCloneable : public ChunkInfo +{ +public: + ChunkInfoCloneable() = default; + ChunkInfoCloneable(const ChunkInfoCloneable & other) = default; + + Ptr clone() const override + { + return std::static_pointer_cast(std::make_shared(*static_cast(this))); + } +}; /** * Chunk is a list of columns with the same length. @@ -32,26 +52,26 @@ using ChunkInfoPtr = std::shared_ptr; class Chunk { public: + using ChunkInfoCollection = CollectionOfDerivedItems; + Chunk() = default; Chunk(const Chunk & other) = delete; Chunk(Chunk && other) noexcept : columns(std::move(other.columns)) , num_rows(other.num_rows) - , chunk_info(std::move(other.chunk_info)) + , chunk_infos(std::move(other.chunk_infos)) { other.num_rows = 0; } Chunk(Columns columns_, UInt64 num_rows_); - Chunk(Columns columns_, UInt64 num_rows_, ChunkInfoPtr chunk_info_); Chunk(MutableColumns columns_, UInt64 num_rows_); - Chunk(MutableColumns columns_, UInt64 num_rows_, ChunkInfoPtr chunk_info_); Chunk & operator=(const Chunk & other) = delete; Chunk & operator=(Chunk && other) noexcept { columns = std::move(other.columns); - chunk_info = std::move(other.chunk_info); + chunk_infos = std::move(other.chunk_infos); num_rows = other.num_rows; other.num_rows = 0; return *this; @@ -62,15 +82,15 @@ public: void swap(Chunk & other) noexcept { columns.swap(other.columns); - chunk_info.swap(other.chunk_info); std::swap(num_rows, other.num_rows); + chunk_infos.swap(other.chunk_infos); } void clear() { num_rows = 0; columns.clear(); - chunk_info.reset(); + chunk_infos.clear(); } const Columns & getColumns() const { return columns; } @@ -81,9 +101,9 @@ public: /** Get empty columns with the same types as in block. */ MutableColumns cloneEmptyColumns() const; - const ChunkInfoPtr & getChunkInfo() const { return chunk_info; } - bool hasChunkInfo() const { return chunk_info != nullptr; } - void setChunkInfo(ChunkInfoPtr chunk_info_) { chunk_info = std::move(chunk_info_); } + ChunkInfoCollection & getChunkInfos() { return chunk_infos; } + const ChunkInfoCollection & getChunkInfos() const { return chunk_infos; } + void setChunkInfos(ChunkInfoCollection chunk_infos_) { chunk_infos = std::move(chunk_infos_); } UInt64 getNumRows() const { return num_rows; } UInt64 getNumColumns() const { return columns.size(); } @@ -107,7 +127,7 @@ public: private: Columns columns; UInt64 num_rows = 0; - ChunkInfoPtr chunk_info; + ChunkInfoCollection chunk_infos; void checkNumRowsIsConsistent(); }; @@ -117,11 +137,15 @@ using Chunks = std::vector; /// AsyncInsert needs two kinds of information: /// - offsets of different sub-chunks /// - tokens of different sub-chunks, which are assigned by setting `insert_deduplication_token`. -class AsyncInsertInfo : public ChunkInfo +class AsyncInsertInfo : public ChunkInfoCloneable { public: AsyncInsertInfo() = default; - explicit AsyncInsertInfo(const std::vector & offsets_, const std::vector & tokens_) : offsets(offsets_), tokens(tokens_) {} + AsyncInsertInfo(const AsyncInsertInfo & other) = default; + AsyncInsertInfo(const std::vector & offsets_, const std::vector & tokens_) + : offsets(offsets_) + , tokens(tokens_) + {} std::vector offsets; std::vector tokens; @@ -130,9 +154,11 @@ public: using AsyncInsertInfoPtr = std::shared_ptr; /// Extension to support delayed defaults. AddingDefaultsProcessor uses it to replace missing values with column defaults. -class ChunkMissingValues : public ChunkInfo +class ChunkMissingValues : public ChunkInfoCloneable { public: + ChunkMissingValues(const ChunkMissingValues & other) = default; + using RowsBitMask = std::vector; /// a bit per row for a column const RowsBitMask & getDefaultsBitmask(size_t column_idx) const; diff --git a/src/Processors/Executors/PullingAsyncPipelineExecutor.cpp b/src/Processors/Executors/PullingAsyncPipelineExecutor.cpp index d27002197d2..d9fab88fe1f 100644 --- a/src/Processors/Executors/PullingAsyncPipelineExecutor.cpp +++ b/src/Processors/Executors/PullingAsyncPipelineExecutor.cpp @@ -147,13 +147,10 @@ bool PullingAsyncPipelineExecutor::pull(Block & block, uint64_t milliseconds) block = lazy_format->getPort(IOutputFormat::PortKind::Main).getHeader().cloneWithColumns(chunk.detachColumns()); - if (auto chunk_info = chunk.getChunkInfo()) + if (auto agg_info = chunk.getChunkInfos().get()) { - if (const auto * agg_info = typeid_cast(chunk_info.get())) - { - block.info.bucket_num = agg_info->bucket_num; - block.info.is_overflows = agg_info->is_overflows; - } + block.info.bucket_num = agg_info->bucket_num; + block.info.is_overflows = agg_info->is_overflows; } return true; diff --git a/src/Processors/Executors/PullingPipelineExecutor.cpp b/src/Processors/Executors/PullingPipelineExecutor.cpp index cbf73c5cb07..25c15d40c9a 100644 --- a/src/Processors/Executors/PullingPipelineExecutor.cpp +++ b/src/Processors/Executors/PullingPipelineExecutor.cpp @@ -73,13 +73,10 @@ bool PullingPipelineExecutor::pull(Block & block) } block = pulling_format->getPort(IOutputFormat::PortKind::Main).getHeader().cloneWithColumns(chunk.detachColumns()); - if (auto chunk_info = chunk.getChunkInfo()) + if (auto agg_info = chunk.getChunkInfos().get()) { - if (const auto * agg_info = typeid_cast(chunk_info.get())) - { - block.info.bucket_num = agg_info->bucket_num; - block.info.is_overflows = agg_info->is_overflows; - } + block.info.bucket_num = agg_info->bucket_num; + block.info.is_overflows = agg_info->is_overflows; } return true; diff --git a/src/Processors/Formats/Impl/ParquetBlockOutputFormat.cpp b/src/Processors/Formats/Impl/ParquetBlockOutputFormat.cpp index a5d334f4f1d..9e499e2c400 100644 --- a/src/Processors/Formats/Impl/ParquetBlockOutputFormat.cpp +++ b/src/Processors/Formats/Impl/ParquetBlockOutputFormat.cpp @@ -179,7 +179,9 @@ void ParquetBlockOutputFormat::consume(Chunk chunk) columns[i]->insertRangeFrom(*concatenated.getColumns()[i], offset, count); Chunks piece; - piece.emplace_back(std::move(columns), count, concatenated.getChunkInfo()); + piece.emplace_back(std::move(columns), count); + piece.back().setChunkInfos(concatenated.getChunkInfos()); + writeRowGroup(std::move(piece)); } } diff --git a/src/Processors/IAccumulatingTransform.cpp b/src/Processors/IAccumulatingTransform.cpp index 4136fc5a5f2..46be6e74693 100644 --- a/src/Processors/IAccumulatingTransform.cpp +++ b/src/Processors/IAccumulatingTransform.cpp @@ -8,8 +8,9 @@ namespace ErrorCodes } IAccumulatingTransform::IAccumulatingTransform(Block input_header, Block output_header) - : IProcessor({std::move(input_header)}, {std::move(output_header)}), - input(inputs.front()), output(outputs.front()) + : IProcessor({std::move(input_header)}, {std::move(output_header)}) + , input(inputs.front()) + , output(outputs.front()) { } diff --git a/src/Processors/Merges/Algorithms/FinishAggregatingInOrderAlgorithm.cpp b/src/Processors/Merges/Algorithms/FinishAggregatingInOrderAlgorithm.cpp index 466adf93538..86675bcb237 100644 --- a/src/Processors/Merges/Algorithms/FinishAggregatingInOrderAlgorithm.cpp +++ b/src/Processors/Merges/Algorithms/FinishAggregatingInOrderAlgorithm.cpp @@ -53,13 +53,11 @@ void FinishAggregatingInOrderAlgorithm::consume(Input & input, size_t source_num if (!input.chunk.hasRows()) return; - const auto & info = input.chunk.getChunkInfo(); - if (!info) + if (input.chunk.getChunkInfos().empty()) throw Exception(ErrorCodes::LOGICAL_ERROR, "Chunk info was not set for chunk in FinishAggregatingInOrderAlgorithm"); Int64 allocated_bytes = 0; - /// Will be set by AggregatingInOrderTransform during local aggregation; will be nullptr during merging on initiator. - if (const auto * arenas_info = typeid_cast(info.get())) + if (auto arenas_info = input.chunk.getChunkInfos().get()) allocated_bytes = arenas_info->allocated_bytes; states[source_num] = State{input.chunk, description, allocated_bytes}; @@ -136,7 +134,7 @@ Chunk FinishAggregatingInOrderAlgorithm::prepareToMerge() info->chunk_num = chunk_num++; Chunk chunk; - chunk.setChunkInfo(std::move(info)); + chunk.getChunkInfos().add(std::move(info)); return chunk; } @@ -163,7 +161,7 @@ void FinishAggregatingInOrderAlgorithm::addToAggregation() chunks.emplace_back(std::move(new_columns), current_rows); } - chunks.back().setChunkInfo(std::make_shared()); + chunks.back().getChunkInfos().add(std::make_shared()); states[i].current_row = states[i].to_row; /// We assume that sizes in bytes of rows are almost the same. diff --git a/src/Processors/Merges/Algorithms/MergeTreePartLevelInfo.h b/src/Processors/Merges/Algorithms/MergeTreePartLevelInfo.h index bcf4e759024..e4f22deec8d 100644 --- a/src/Processors/Merges/Algorithms/MergeTreePartLevelInfo.h +++ b/src/Processors/Merges/Algorithms/MergeTreePartLevelInfo.h @@ -6,18 +6,22 @@ namespace DB { /// To carry part level if chunk is produced by a merge tree source -class MergeTreePartLevelInfo : public ChunkInfo +class MergeTreePartLevelInfo : public ChunkInfoCloneable { public: MergeTreePartLevelInfo() = delete; - explicit MergeTreePartLevelInfo(ssize_t part_level) : origin_merge_tree_part_level(part_level) { } + explicit MergeTreePartLevelInfo(ssize_t part_level) + : origin_merge_tree_part_level(part_level) + { } + MergeTreePartLevelInfo(const MergeTreePartLevelInfo & other) = default; + size_t origin_merge_tree_part_level = 0; }; inline size_t getPartLevelFromChunk(const Chunk & chunk) { - const auto & info = chunk.getChunkInfo(); - if (const auto * part_level_info = typeid_cast(info.get())) + const auto part_level_info = chunk.getChunkInfos().get(); + if (part_level_info) return part_level_info->origin_merge_tree_part_level; return 0; } diff --git a/src/Processors/Merges/Algorithms/ReplacingSortedAlgorithm.cpp b/src/Processors/Merges/Algorithms/ReplacingSortedAlgorithm.cpp index 7b2c7d82a01..cd347d371d9 100644 --- a/src/Processors/Merges/Algorithms/ReplacingSortedAlgorithm.cpp +++ b/src/Processors/Merges/Algorithms/ReplacingSortedAlgorithm.cpp @@ -17,7 +17,7 @@ namespace ErrorCodes static IMergingAlgorithm::Status emitChunk(detail::SharedChunkPtr & chunk, bool finished = false) { - chunk->setChunkInfo(std::make_shared(std::move(chunk->replace_final_selection))); + chunk->getChunkInfos().add(std::make_shared(std::move(chunk->replace_final_selection))); return IMergingAlgorithm::Status(std::move(*chunk), finished); } diff --git a/src/Processors/Merges/Algorithms/ReplacingSortedAlgorithm.h b/src/Processors/Merges/Algorithms/ReplacingSortedAlgorithm.h index a3ccccf0845..2f23f2a5c4d 100644 --- a/src/Processors/Merges/Algorithms/ReplacingSortedAlgorithm.h +++ b/src/Processors/Merges/Algorithms/ReplacingSortedAlgorithm.h @@ -3,6 +3,7 @@ #include #include #include +#include namespace Poco { @@ -14,11 +15,13 @@ namespace DB /** Use in skipping final to keep list of indices of selected row after merging final */ -struct ChunkSelectFinalIndices : public ChunkInfo +struct ChunkSelectFinalIndices : public ChunkInfoCloneable { + explicit ChunkSelectFinalIndices(MutableColumnPtr select_final_indices_); + ChunkSelectFinalIndices(const ChunkSelectFinalIndices & other) = default; + const ColumnPtr column_holder; const ColumnUInt64 * select_final_indices = nullptr; - explicit ChunkSelectFinalIndices(MutableColumnPtr select_final_indices_); }; /** Merges several sorted inputs into one. diff --git a/src/Processors/Merges/IMergingTransform.cpp b/src/Processors/Merges/IMergingTransform.cpp index fbb47969b2f..b1b0182a113 100644 --- a/src/Processors/Merges/IMergingTransform.cpp +++ b/src/Processors/Merges/IMergingTransform.cpp @@ -157,7 +157,7 @@ IProcessor::Status IMergingTransformBase::prepare() bool is_port_full = !output.canPush(); /// Push if has data. - if ((state.output_chunk || state.output_chunk.hasChunkInfo()) && !is_port_full) + if ((state.output_chunk || !state.output_chunk.getChunkInfos().empty()) && !is_port_full) output.push(std::move(state.output_chunk)); if (!is_initialized) diff --git a/src/Processors/Merges/IMergingTransform.h b/src/Processors/Merges/IMergingTransform.h index c218f622870..be629271736 100644 --- a/src/Processors/Merges/IMergingTransform.h +++ b/src/Processors/Merges/IMergingTransform.h @@ -129,7 +129,7 @@ public: IMergingAlgorithm::Status status = algorithm.merge(); - if ((status.chunk && status.chunk.hasRows()) || status.chunk.hasChunkInfo()) + if ((status.chunk && status.chunk.hasRows()) || !status.chunk.getChunkInfos().empty()) { // std::cerr << "Got chunk with " << status.chunk.getNumRows() << " rows" << std::endl; state.output_chunk = std::move(status.chunk); diff --git a/src/Processors/Sinks/RemoteSink.h b/src/Processors/Sinks/RemoteSink.h index 30cf958c072..c05cc1defcb 100644 --- a/src/Processors/Sinks/RemoteSink.h +++ b/src/Processors/Sinks/RemoteSink.h @@ -20,7 +20,7 @@ public: } String getName() const override { return "RemoteSink"; } - void consume (Chunk chunk) override { write(RemoteInserter::getHeader().cloneWithColumns(chunk.detachColumns())); } + void consume (Chunk & chunk) override { write(RemoteInserter::getHeader().cloneWithColumns(chunk.getColumns())); } void onFinish() override { RemoteInserter::onFinish(); } }; diff --git a/src/Processors/Sinks/SinkToStorage.cpp b/src/Processors/Sinks/SinkToStorage.cpp index 5f9f9f9b1a1..36bb70f493f 100644 --- a/src/Processors/Sinks/SinkToStorage.cpp +++ b/src/Processors/Sinks/SinkToStorage.cpp @@ -15,9 +15,8 @@ void SinkToStorage::onConsume(Chunk chunk) */ Nested::validateArraySizes(getHeader().cloneWithColumns(chunk.getColumns())); - consume(chunk.clone()); - if (!lastBlockIsDuplicate()) - cur_chunk = std::move(chunk); + consume(chunk); + cur_chunk = std::move(chunk); } SinkToStorage::GenerateResult SinkToStorage::onGenerate() diff --git a/src/Processors/Sinks/SinkToStorage.h b/src/Processors/Sinks/SinkToStorage.h index 023bbd8b094..c728fa87b1e 100644 --- a/src/Processors/Sinks/SinkToStorage.h +++ b/src/Processors/Sinks/SinkToStorage.h @@ -18,8 +18,7 @@ public: void addTableLock(const TableLockHolder & lock) { table_locks.push_back(lock); } protected: - virtual void consume(Chunk chunk) = 0; - virtual bool lastBlockIsDuplicate() const { return false; } + virtual void consume(Chunk & chunk) = 0; private: std::vector table_locks; @@ -38,7 +37,7 @@ class NullSinkToStorage : public SinkToStorage public: using SinkToStorage::SinkToStorage; std::string getName() const override { return "NullSinkToStorage"; } - void consume(Chunk) override {} + void consume(Chunk &) override {} }; using SinkPtr = std::shared_ptr; diff --git a/src/Processors/Sources/BlocksSource.h b/src/Processors/Sources/BlocksSource.h index ec0dc9609f1..7ac460c14e2 100644 --- a/src/Processors/Sources/BlocksSource.h +++ b/src/Processors/Sources/BlocksSource.h @@ -43,7 +43,10 @@ protected: info->bucket_num = res.info.bucket_num; info->is_overflows = res.info.is_overflows; - return Chunk(res.getColumns(), res.rows(), std::move(info)); + auto chunk = Chunk(res.getColumns(), res.rows()); + chunk.getChunkInfos().add(std::move(info)); + + return chunk; } private: diff --git a/src/Processors/Sources/RemoteSource.cpp b/src/Processors/Sources/RemoteSource.cpp index 3d7dd3f76b8..1578bd389c9 100644 --- a/src/Processors/Sources/RemoteSource.cpp +++ b/src/Processors/Sources/RemoteSource.cpp @@ -176,7 +176,7 @@ std::optional RemoteSource::tryGenerate() auto info = std::make_shared(); info->bucket_num = block.info.bucket_num; info->is_overflows = block.info.is_overflows; - chunk.setChunkInfo(std::move(info)); + chunk.getChunkInfos().add(std::move(info)); } return chunk; diff --git a/src/Processors/Sources/SourceFromSingleChunk.cpp b/src/Processors/Sources/SourceFromSingleChunk.cpp index 00f40a34361..9abe0504d10 100644 --- a/src/Processors/Sources/SourceFromSingleChunk.cpp +++ b/src/Processors/Sources/SourceFromSingleChunk.cpp @@ -5,7 +5,9 @@ namespace DB { -SourceFromSingleChunk::SourceFromSingleChunk(Block header, Chunk chunk_) : ISource(std::move(header)), chunk(std::move(chunk_)) {} +SourceFromSingleChunk::SourceFromSingleChunk(Block header, Chunk chunk_) : ISource(std::move(header)), chunk(std::move(chunk_)) +{ +} SourceFromSingleChunk::SourceFromSingleChunk(Block data) : ISource(data.cloneEmpty()), chunk(data.getColumns(), data.rows()) { @@ -20,7 +22,7 @@ SourceFromSingleChunk::SourceFromSingleChunk(Block data) : ISource(data.cloneEmp auto info = std::make_shared(); info->bucket_num = data.info.bucket_num; info->is_overflows = data.info.is_overflows; - chunk.setChunkInfo(std::move(info)); + chunk.getChunkInfos().add(std::move(info)); } } diff --git a/src/Processors/Transforms/AggregatingInOrderTransform.cpp b/src/Processors/Transforms/AggregatingInOrderTransform.cpp index 9ffe15d0f85..45b0960ec8f 100644 --- a/src/Processors/Transforms/AggregatingInOrderTransform.cpp +++ b/src/Processors/Transforms/AggregatingInOrderTransform.cpp @@ -332,7 +332,7 @@ void AggregatingInOrderTransform::generate() variants.aggregates_pool = variants.aggregates_pools.at(0).get(); /// Pass info about used memory by aggregate functions further. - to_push_chunk.setChunkInfo(std::make_shared(cur_block_bytes)); + to_push_chunk.getChunkInfos().add(std::make_shared(cur_block_bytes)); cur_block_bytes = 0; cur_block_size = 0; @@ -351,11 +351,12 @@ FinalizeAggregatedTransform::FinalizeAggregatedTransform(Block header, Aggregati void FinalizeAggregatedTransform::transform(Chunk & chunk) { if (params->final) - finalizeChunk(chunk, aggregates_mask); - else if (!chunk.getChunkInfo()) { - auto info = std::make_shared(); - chunk.setChunkInfo(std::move(info)); + finalizeChunk(chunk, aggregates_mask); + } + else if (!chunk.getChunkInfos().get()) + { + chunk.getChunkInfos().add(std::make_shared()); } } diff --git a/src/Processors/Transforms/AggregatingInOrderTransform.h b/src/Processors/Transforms/AggregatingInOrderTransform.h index 5d50e97f552..41a0d7fc7f1 100644 --- a/src/Processors/Transforms/AggregatingInOrderTransform.h +++ b/src/Processors/Transforms/AggregatingInOrderTransform.h @@ -5,6 +5,7 @@ #include #include #include +#include namespace DB { @@ -12,10 +13,12 @@ namespace DB struct InputOrderInfo; using InputOrderInfoPtr = std::shared_ptr; -struct ChunkInfoWithAllocatedBytes : public ChunkInfo +struct ChunkInfoWithAllocatedBytes : public ChunkInfoCloneable { + ChunkInfoWithAllocatedBytes(const ChunkInfoWithAllocatedBytes & other) = default; explicit ChunkInfoWithAllocatedBytes(Int64 allocated_bytes_) : allocated_bytes(allocated_bytes_) {} + Int64 allocated_bytes; }; diff --git a/src/Processors/Transforms/AggregatingTransform.cpp b/src/Processors/Transforms/AggregatingTransform.cpp index cdbe194cfac..fda03174c53 100644 --- a/src/Processors/Transforms/AggregatingTransform.cpp +++ b/src/Processors/Transforms/AggregatingTransform.cpp @@ -35,7 +35,7 @@ Chunk convertToChunk(const Block & block) UInt64 num_rows = block.rows(); Chunk chunk(block.getColumns(), num_rows); - chunk.setChunkInfo(std::move(info)); + chunk.getChunkInfos().add(std::move(info)); return chunk; } @@ -44,15 +44,11 @@ namespace { const AggregatedChunkInfo * getInfoFromChunk(const Chunk & chunk) { - const auto & info = chunk.getChunkInfo(); - if (!info) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Chunk info was not set for chunk."); - - const auto * agg_info = typeid_cast(info.get()); + auto agg_info = chunk.getChunkInfos().get(); if (!agg_info) throw Exception(ErrorCodes::LOGICAL_ERROR, "Chunk should have AggregatedChunkInfo."); - return agg_info; + return agg_info.get(); } /// Reads chunks from file in native format. Provide chunks with aggregation info. @@ -210,11 +206,7 @@ private: void process(Chunk && chunk) { - if (!chunk.hasChunkInfo()) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Expected chunk with chunk info in {}", getName()); - - const auto & info = chunk.getChunkInfo(); - const auto * chunks_to_merge = typeid_cast(info.get()); + auto chunks_to_merge = chunk.getChunkInfos().get(); if (!chunks_to_merge) throw Exception(ErrorCodes::LOGICAL_ERROR, "Expected chunk with ChunksToMerge info in {}", getName()); diff --git a/src/Processors/Transforms/AggregatingTransform.h b/src/Processors/Transforms/AggregatingTransform.h index e167acde067..95983c39d1e 100644 --- a/src/Processors/Transforms/AggregatingTransform.h +++ b/src/Processors/Transforms/AggregatingTransform.h @@ -2,6 +2,7 @@ #include #include #include +#include #include #include #include @@ -19,7 +20,7 @@ namespace CurrentMetrics namespace DB { -class AggregatedChunkInfo : public ChunkInfo +class AggregatedChunkInfo : public ChunkInfoCloneable { public: bool is_overflows = false; diff --git a/src/Processors/Transforms/ApplySquashingTransform.h b/src/Processors/Transforms/ApplySquashingTransform.h index 965a084bb13..49a6581e685 100644 --- a/src/Processors/Transforms/ApplySquashingTransform.h +++ b/src/Processors/Transforms/ApplySquashingTransform.h @@ -27,18 +27,12 @@ public: } ExceptionKeepingTransform::work(); - if (finish_chunk) - { - data.chunk = std::move(finish_chunk); - ready_output = true; - } } protected: void onConsume(Chunk chunk) override { - if (auto res_chunk = DB::Squashing::squash(std::move(chunk))) - cur_chunk.setColumns(res_chunk.getColumns(), res_chunk.getNumRows()); + cur_chunk = Squashing::squash(std::move(chunk)); } GenerateResult onGenerate() override @@ -48,16 +42,10 @@ protected: res.is_done = true; return res; } - void onFinish() override - { - auto chunk = DB::Squashing::squash({}); - finish_chunk.setColumns(chunk.getColumns(), chunk.getNumRows()); - } private: Squashing squashing; Chunk cur_chunk; - Chunk finish_chunk; }; } diff --git a/src/Processors/Transforms/CountingTransform.cpp b/src/Processors/Transforms/CountingTransform.cpp index 3dfb9fe178f..2c6b3bd8638 100644 --- a/src/Processors/Transforms/CountingTransform.cpp +++ b/src/Processors/Transforms/CountingTransform.cpp @@ -1,6 +1,7 @@ - -#include #include + +#include +#include #include #include diff --git a/src/Processors/Transforms/DeduplicationTokenTransforms.cpp b/src/Processors/Transforms/DeduplicationTokenTransforms.cpp new file mode 100644 index 00000000000..e6f7e44e026 --- /dev/null +++ b/src/Processors/Transforms/DeduplicationTokenTransforms.cpp @@ -0,0 +1,238 @@ +#include + +#include + +#include +#include +#include + + +#include + + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int LOGICAL_ERROR; +} + +void RestoreChunkInfosTransform::transform(Chunk & chunk) +{ + chunk.getChunkInfos().append(chunk_infos.clone()); +} + +namespace DeduplicationToken +{ + +String TokenInfo::getToken() const +{ + if (!isDefined()) + throw Exception(ErrorCodes::LOGICAL_ERROR, "token is not defined, stage {}, token {}", stage, debugToken()); + + return getTokenImpl(); +} + +String TokenInfo::getTokenImpl() const +{ + String result; + result.reserve(getTotalSize()); + + for (const auto & part : parts) + { + if (!result.empty()) + result.append(":"); + result.append(part); + } + + return result; +} + +String TokenInfo::debugToken() const +{ + return getTokenImpl(); +} + +void TokenInfo::addChunkHash(String part) +{ + if (stage == UNDEFINED && empty()) + stage = DEFINE_SOURCE_WITH_HASHES; + + if (stage != DEFINE_SOURCE_WITH_HASHES) + throw Exception(ErrorCodes::LOGICAL_ERROR, "token is in wrong stage {}, token {}", stage, debugToken()); + + addTokenPart(std::move(part)); +} + +void TokenInfo::finishChunkHashes() +{ + if (stage == UNDEFINED && empty()) + stage = DEFINE_SOURCE_WITH_HASHES; + + if (stage != DEFINE_SOURCE_WITH_HASHES) + throw Exception(ErrorCodes::LOGICAL_ERROR, "token is in wrong stage {}, token {}", stage, debugToken()); + + stage = DEFINED; +} + +void TokenInfo::setUserToken(const String & token) +{ + if (stage == UNDEFINED && empty()) + stage = DEFINE_SOURCE_USER_TOKEN; + + if (stage != DEFINE_SOURCE_USER_TOKEN) + throw Exception(ErrorCodes::LOGICAL_ERROR, "token is in wrong stage {}, token {}", stage, debugToken()); + + addTokenPart(fmt::format("user-token-{}", token)); +} + +void TokenInfo::setSourceWithUserToken(size_t block_number) +{ + if (stage != DEFINE_SOURCE_USER_TOKEN) + throw Exception(ErrorCodes::LOGICAL_ERROR, "token is in wrong stage {}, token {}", stage, debugToken()); + + addTokenPart(fmt::format("source-number-{}", block_number)); + + stage = DEFINED; +} + +void TokenInfo::setViewID(const String & id) +{ + if (stage == DEFINED) + stage = DEFINE_VIEW; + + if (stage != DEFINE_VIEW) + throw Exception(ErrorCodes::LOGICAL_ERROR, "token is in wrong stage {}, token {}", stage, debugToken()); + + addTokenPart(fmt::format("view-id-{}", id)); +} + +void TokenInfo::setViewBlockNumber(size_t block_number) +{ + if (stage != DEFINE_VIEW) + throw Exception(ErrorCodes::LOGICAL_ERROR, "token is in wrong stage {}, token {}", stage, debugToken()); + + addTokenPart(fmt::format("view-block-{}", block_number)); + + stage = DEFINED; +} + +void TokenInfo::reset() +{ + stage = UNDEFINED; + parts.clear(); +} + +void TokenInfo::addTokenPart(String part) +{ + parts.push_back(std::move(part)); +} + +size_t TokenInfo::getTotalSize() const +{ + if (parts.empty()) + return 0; + + size_t size = 0; + for (const auto & part : parts) + size += part.size(); + + // we reserve more size here to be able to add delimenter between parts. + return size + parts.size() - 1; +} + +#ifdef ABORT_ON_LOGICAL_ERROR +void CheckTokenTransform::transform(Chunk & chunk) +{ + auto token_info = chunk.getChunkInfos().get(); + + if (!token_info) + { + throw Exception(ErrorCodes::LOGICAL_ERROR, "Chunk has to have DedupTokenInfo as ChunkInfo, {}", debug); + } + + LOG_TEST(log, "debug: {}, token: {}, columns {} rows {}", debug, token_info->debugToken(), chunk.getNumColumns(), chunk.getNumRows()); +} +#endif + +String DefineSourceWithChunkHashTransform::getChunkHash(const Chunk & chunk) +{ + SipHash hash; + for (const auto & colunm : chunk.getColumns()) + colunm->updateHashFast(hash); + + const auto hash_value = hash.get128(); + return toString(hash_value.items[0]) + "_" + toString(hash_value.items[1]); +} + + +void DefineSourceWithChunkHashTransform::transform(Chunk & chunk) +{ + auto token_info = chunk.getChunkInfos().get(); + + if (!token_info) + throw Exception( + ErrorCodes::LOGICAL_ERROR, + "TokenInfo is expected for consumed chunk in DefineSourceWithChunkHashesTransform"); + + if (token_info->isDefined()) + return; + + token_info->addChunkHash(getChunkHash(chunk)); + token_info->finishChunkHashes(); +} + +void SetUserTokenTransform::transform(Chunk & chunk) +{ + auto token_info = chunk.getChunkInfos().get(); + if (!token_info) + throw Exception( + ErrorCodes::LOGICAL_ERROR, + "TokenInfo is expected for consumed chunk in SetUserTokenTransform"); + token_info->setUserToken(user_token); +} + +void SetSourceBlockNumberTransform::transform(Chunk & chunk) +{ + auto token_info = chunk.getChunkInfos().get(); + if (!token_info) + throw Exception( + ErrorCodes::LOGICAL_ERROR, + "TokenInfo is expected for consumed chunk in SetSourceBlockNumberTransform"); + token_info->setSourceWithUserToken(block_number++); +} + +void SetViewIDTransform::transform(Chunk & chunk) +{ + auto token_info = chunk.getChunkInfos().get(); + if (!token_info) + throw Exception( + ErrorCodes::LOGICAL_ERROR, + "TokenInfo is expected for consumed chunk in SetViewIDTransform"); + token_info->setViewID(view_id); +} + +void SetViewBlockNumberTransform::transform(Chunk & chunk) +{ + auto token_info = chunk.getChunkInfos().get(); + if (!token_info) + throw Exception( + ErrorCodes::LOGICAL_ERROR, + "TokenInfo is expected for consumed chunk in SetViewBlockNumberTransform"); + token_info->setViewBlockNumber(block_number++); +} + +void ResetTokenTransform::transform(Chunk & chunk) +{ + auto token_info = chunk.getChunkInfos().get(); + if (!token_info) + throw Exception( + ErrorCodes::LOGICAL_ERROR, + "TokenInfo is expected for consumed chunk in ResetTokenTransform"); + + token_info->reset(); +} + +} +} diff --git a/src/Processors/Transforms/DeduplicationTokenTransforms.h b/src/Processors/Transforms/DeduplicationTokenTransforms.h new file mode 100644 index 00000000000..d6aff9e1370 --- /dev/null +++ b/src/Processors/Transforms/DeduplicationTokenTransforms.h @@ -0,0 +1,237 @@ +#pragma once + +#include +#include + +#include +#include "Common/Logger.h" + + +namespace DB +{ + class RestoreChunkInfosTransform : public ISimpleTransform + { + public: + RestoreChunkInfosTransform(Chunk::ChunkInfoCollection chunk_infos_, const Block & header_) + : ISimpleTransform(header_, header_, true) + , chunk_infos(std::move(chunk_infos_)) + {} + + String getName() const override { return "RestoreChunkInfosTransform"; } + + void transform(Chunk & chunk) override; + + private: + Chunk::ChunkInfoCollection chunk_infos; + }; + + +namespace DeduplicationToken +{ + class TokenInfo : public ChunkInfoCloneable + { + public: + TokenInfo() = default; + TokenInfo(const TokenInfo & other) = default; + + String getToken() const; + String debugToken() const; + + bool empty() const { return parts.empty(); } + + bool isDefined() const { return stage == DEFINED; } + + void addChunkHash(String part); + void finishChunkHashes(); + + void setUserToken(const String & token); + void setSourceWithUserToken(size_t block_number); + + void setViewID(const String & id); + void setViewBlockNumber(size_t block_number); + + void reset(); + + private: + String getTokenImpl() const; + + void addTokenPart(String part); + size_t getTotalSize() const; + + /* Token has to be prepared in a particular order. + * BuildingStage ensures that token is expanded according the following order. + * Firstly token is expanded with information about the source. + * It could be done with two ways: add several hash sums from the source chunks or provide user defined deduplication token and its sequentional block number. + * + * transition // method + * UNDEFINED -> DEFINE_SOURCE_WITH_HASHES // addChunkHash + * DEFINE_SOURCE_WITH_HASHES -> DEFINE_SOURCE_WITH_HASHES // addChunkHash + * DEFINE_SOURCE_WITH_HASHES -> DEFINED // defineSourceWithChankHashes + * + * transition // method + * UNDEFINED -> DEFINE_SOURCE_USER_TOKEN // setUserToken + * DEFINE_SOURCE_USER_TOKEN -> DEFINED // defineSourceWithUserToken + * + * After token is defined, it could be extended with view id and view block number. Actually it has to be expanded with view details if there is one or several views. + * + * transition // method + * DEFINED -> DEFINE_VIEW // setViewID + * DEFINE_VIEW -> DEFINED // defineViewID + */ + + enum BuildingStage + { + UNDEFINED, + DEFINE_SOURCE_WITH_HASHES, + DEFINE_SOURCE_USER_TOKEN, + DEFINE_VIEW, + DEFINED, + }; + + BuildingStage stage = UNDEFINED; + std::vector parts; + }; + + +#ifdef ABORT_ON_LOGICAL_ERROR + /// use that class only with debug builds in CI for introspection + class CheckTokenTransform : public ISimpleTransform + { + public: + CheckTokenTransform(String debug_, const Block & header_) + : ISimpleTransform(header_, header_, true) + , debug(std::move(debug_)) + { + } + + String getName() const override { return "DeduplicationToken::CheckTokenTransform"; } + + void transform(Chunk & chunk) override; + + private: + String debug; + LoggerPtr log = getLogger("CheckInsertDeduplicationTokenTransform"); + }; +#endif + + + class AddTokenInfoTransform : public ISimpleTransform + { + public: + explicit AddTokenInfoTransform(const Block & header_) + : ISimpleTransform(header_, header_, true) + { + } + + String getName() const override { return "DeduplicationToken::AddTokenInfoTransform"; } + + void transform(Chunk & chunk) override + { + chunk.getChunkInfos().add(std::make_shared()); + } + }; + + + class DefineSourceWithChunkHashTransform : public ISimpleTransform + { + public: + explicit DefineSourceWithChunkHashTransform(const Block & header_) + : ISimpleTransform(header_, header_, true) + { + } + + String getName() const override { return "DeduplicationToken::DefineSourceWithChunkHashesTransform"; } + + // Usually MergeTreeSink/ReplicatedMergeTreeSink calls addChunkHash for the deduplication token with hashes from the parts. + // But if there is some table with different engine, we still need to define the source of the data in deduplication token + // We use that transform to define the source as a hash of entire block in deduplication token + void transform(Chunk & chunk) override; + + static String getChunkHash(const Chunk & chunk); + }; + + class ResetTokenTransform : public ISimpleTransform + { + public: + explicit ResetTokenTransform(const Block & header_) + : ISimpleTransform(header_, header_, true) + { + } + + String getName() const override { return "DeduplicationToken::ResetTokenTransform"; } + + void transform(Chunk & chunk) override; + }; + + + class SetUserTokenTransform : public ISimpleTransform + { + public: + SetUserTokenTransform(String user_token_, const Block & header_) + : ISimpleTransform(header_, header_, true) + , user_token(std::move(user_token_)) + { + } + + String getName() const override { return "DeduplicationToken::SetUserTokenTransform"; } + + void transform(Chunk & chunk) override; + + private: + String user_token; + }; + + + class SetSourceBlockNumberTransform : public ISimpleTransform + { + public: + explicit SetSourceBlockNumberTransform(const Block & header_) + : ISimpleTransform(header_, header_, true) + { + } + + String getName() const override { return "DeduplicationToken::SetSourceBlockNumberTransform"; } + + void transform(Chunk & chunk) override; + + private: + size_t block_number = 0; + }; + + + class SetViewIDTransform : public ISimpleTransform + { + public: + SetViewIDTransform(String view_id_, const Block & header_) + : ISimpleTransform(header_, header_, true) + , view_id(std::move(view_id_)) + { + } + + String getName() const override { return "DeduplicationToken::SetViewIDTransform"; } + + void transform(Chunk & chunk) override; + + private: + String view_id; + }; + + + class SetViewBlockNumberTransform : public ISimpleTransform + { + public: + explicit SetViewBlockNumberTransform(const Block & header_) + : ISimpleTransform(header_, header_, true) + { + } + + String getName() const override { return "DeduplicationToken::SetViewBlockNumberTransform"; } + + void transform(Chunk & chunk) override; + + private: + size_t block_number = 0; + }; + +} +} diff --git a/src/Processors/Transforms/ExpressionTransform.cpp b/src/Processors/Transforms/ExpressionTransform.cpp index 2fbd2c21b8d..04fabc9a3c6 100644 --- a/src/Processors/Transforms/ExpressionTransform.cpp +++ b/src/Processors/Transforms/ExpressionTransform.cpp @@ -1,5 +1,7 @@ #include #include + + namespace DB { diff --git a/src/Processors/Transforms/JoiningTransform.cpp b/src/Processors/Transforms/JoiningTransform.cpp index 3e2a9462e54..ca204bcb482 100644 --- a/src/Processors/Transforms/JoiningTransform.cpp +++ b/src/Processors/Transforms/JoiningTransform.cpp @@ -365,10 +365,9 @@ IProcessor::Status DelayedJoinedBlocksWorkerTransform::prepare() return Status::Finished; } - if (!data.chunk.hasChunkInfo()) + task = data.chunk.getChunkInfos().get(); + if (!task) throw Exception(ErrorCodes::LOGICAL_ERROR, "DelayedJoinedBlocksWorkerTransform must have chunk info"); - - task = std::dynamic_pointer_cast(data.chunk.getChunkInfo()); } else { @@ -479,7 +478,7 @@ IProcessor::Status DelayedJoinedBlocksTransform::prepare() if (output.isFinished()) continue; Chunk chunk; - chunk.setChunkInfo(std::make_shared()); + chunk.getChunkInfos().add(std::make_shared()); output.push(std::move(chunk)); output.finish(); } @@ -496,7 +495,7 @@ IProcessor::Status DelayedJoinedBlocksTransform::prepare() { Chunk chunk; auto task = std::make_shared(delayed_blocks, left_delayed_stream_finished_counter); - chunk.setChunkInfo(task); + chunk.getChunkInfos().add(std::move(task)); output.push(std::move(chunk)); } delayed_blocks = nullptr; diff --git a/src/Processors/Transforms/JoiningTransform.h b/src/Processors/Transforms/JoiningTransform.h index a308af03662..5f6d9d6fff2 100644 --- a/src/Processors/Transforms/JoiningTransform.h +++ b/src/Processors/Transforms/JoiningTransform.h @@ -1,6 +1,7 @@ #pragma once #include - +#include +#include namespace DB { @@ -111,11 +112,12 @@ private: }; -class DelayedBlocksTask : public ChunkInfo +class DelayedBlocksTask : public ChunkInfoCloneable { public: DelayedBlocksTask() = default; + DelayedBlocksTask(const DelayedBlocksTask & other) = default; explicit DelayedBlocksTask(IBlocksStreamPtr delayed_blocks_, JoiningTransform::FinishCounterPtr left_delayed_stream_finish_counter_) : delayed_blocks(std::move(delayed_blocks_)) , left_delayed_stream_finish_counter(left_delayed_stream_finish_counter_) diff --git a/src/Processors/Transforms/MaterializingTransform.cpp b/src/Processors/Transforms/MaterializingTransform.cpp index 1eaa5458d37..9ae80e21a68 100644 --- a/src/Processors/Transforms/MaterializingTransform.cpp +++ b/src/Processors/Transforms/MaterializingTransform.cpp @@ -1,6 +1,7 @@ #include #include + namespace DB { diff --git a/src/Processors/Transforms/MemoryBoundMerging.h b/src/Processors/Transforms/MemoryBoundMerging.h index 607087fb39c..d7bc320173b 100644 --- a/src/Processors/Transforms/MemoryBoundMerging.h +++ b/src/Processors/Transforms/MemoryBoundMerging.h @@ -150,11 +150,7 @@ private: if (!chunk.hasRows()) return; - const auto & info = chunk.getChunkInfo(); - if (!info) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Chunk info was not set for chunk in SortingAggregatedForMemoryBoundMergingTransform."); - - const auto * agg_info = typeid_cast(info.get()); + const auto & agg_info = chunk.getChunkInfos().get(); if (!agg_info) throw Exception( ErrorCodes::LOGICAL_ERROR, "Chunk should have AggregatedChunkInfo in SortingAggregatedForMemoryBoundMergingTransform."); diff --git a/src/Processors/Transforms/MergingAggregatedMemoryEfficientTransform.cpp b/src/Processors/Transforms/MergingAggregatedMemoryEfficientTransform.cpp index fc40c6894bb..ea9ebb0f96e 100644 --- a/src/Processors/Transforms/MergingAggregatedMemoryEfficientTransform.cpp +++ b/src/Processors/Transforms/MergingAggregatedMemoryEfficientTransform.cpp @@ -30,10 +30,10 @@ void GroupingAggregatedTransform::pushData(Chunks chunks, Int32 bucket, bool is_ auto info = std::make_shared(); info->bucket_num = bucket; info->is_overflows = is_overflows; - info->chunks = std::make_unique(std::move(chunks)); + info->chunks = std::make_shared(std::move(chunks)); Chunk chunk; - chunk.setChunkInfo(std::move(info)); + chunk.getChunkInfos().add(std::move(info)); output.push(std::move(chunk)); } @@ -255,11 +255,10 @@ void GroupingAggregatedTransform::addChunk(Chunk chunk, size_t input) if (!chunk.hasRows()) return; - const auto & info = chunk.getChunkInfo(); - if (!info) + if (chunk.getChunkInfos().empty()) throw Exception(ErrorCodes::LOGICAL_ERROR, "Chunk info was not set for chunk in GroupingAggregatedTransform."); - if (const auto * agg_info = typeid_cast(info.get())) + if (auto agg_info = chunk.getChunkInfos().get()) { Int32 bucket = agg_info->bucket_num; bool is_overflows = agg_info->is_overflows; @@ -275,7 +274,7 @@ void GroupingAggregatedTransform::addChunk(Chunk chunk, size_t input) last_bucket_number[input] = bucket; } } - else if (typeid_cast(info.get())) + else if (chunk.getChunkInfos().get()) { single_level_chunks.emplace_back(std::move(chunk)); } @@ -304,7 +303,11 @@ void GroupingAggregatedTransform::work() Int32 bucket = cur_block.info.bucket_num; auto chunk_info = std::make_shared(); chunk_info->bucket_num = bucket; - chunks_map[bucket].emplace_back(Chunk(cur_block.getColumns(), cur_block.rows(), std::move(chunk_info))); + + auto chunk = Chunk(cur_block.getColumns(), cur_block.rows()); + chunk.getChunkInfos().add(std::move(chunk_info)); + + chunks_map[bucket].emplace_back(std::move(chunk)); } } } @@ -319,9 +322,7 @@ MergingAggregatedBucketTransform::MergingAggregatedBucketTransform( void MergingAggregatedBucketTransform::transform(Chunk & chunk) { - const auto & info = chunk.getChunkInfo(); - const auto * chunks_to_merge = typeid_cast(info.get()); - + auto chunks_to_merge = chunk.getChunkInfos().get(); if (!chunks_to_merge) throw Exception(ErrorCodes::LOGICAL_ERROR, "MergingAggregatedSimpleTransform chunk must have ChunkInfo with type ChunksToMerge."); @@ -330,11 +331,10 @@ void MergingAggregatedBucketTransform::transform(Chunk & chunk) BlocksList blocks_list; for (auto & cur_chunk : *chunks_to_merge->chunks) { - const auto & cur_info = cur_chunk.getChunkInfo(); - if (!cur_info) + if (cur_chunk.getChunkInfos().empty()) throw Exception(ErrorCodes::LOGICAL_ERROR, "Chunk info was not set for chunk in MergingAggregatedBucketTransform."); - if (const auto * agg_info = typeid_cast(cur_info.get())) + if (auto agg_info = cur_chunk.getChunkInfos().get()) { Block block = header.cloneWithColumns(cur_chunk.detachColumns()); block.info.is_overflows = agg_info->is_overflows; @@ -342,7 +342,7 @@ void MergingAggregatedBucketTransform::transform(Chunk & chunk) blocks_list.emplace_back(std::move(block)); } - else if (typeid_cast(cur_info.get())) + else if (cur_chunk.getChunkInfos().get()) { Block block = header.cloneWithColumns(cur_chunk.detachColumns()); block.info.is_overflows = false; @@ -361,7 +361,7 @@ void MergingAggregatedBucketTransform::transform(Chunk & chunk) res_info->is_overflows = chunks_to_merge->is_overflows; res_info->bucket_num = chunks_to_merge->bucket_num; res_info->chunk_num = chunks_to_merge->chunk_num; - chunk.setChunkInfo(std::move(res_info)); + chunk.getChunkInfos().add(std::move(res_info)); auto block = params->aggregator.mergeBlocks(blocks_list, params->final, is_cancelled); @@ -405,11 +405,7 @@ bool SortingAggregatedTransform::tryPushChunk() void SortingAggregatedTransform::addChunk(Chunk chunk, size_t from_input) { - const auto & info = chunk.getChunkInfo(); - if (!info) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Chunk info was not set for chunk in SortingAggregatedTransform."); - - const auto * agg_info = typeid_cast(info.get()); + auto agg_info = chunk.getChunkInfos().get(); if (!agg_info) throw Exception(ErrorCodes::LOGICAL_ERROR, "Chunk should have AggregatedChunkInfo in SortingAggregatedTransform."); diff --git a/src/Processors/Transforms/MergingAggregatedMemoryEfficientTransform.h b/src/Processors/Transforms/MergingAggregatedMemoryEfficientTransform.h index 77ee3034ffc..3a3c1bd9c1e 100644 --- a/src/Processors/Transforms/MergingAggregatedMemoryEfficientTransform.h +++ b/src/Processors/Transforms/MergingAggregatedMemoryEfficientTransform.h @@ -3,6 +3,7 @@ #include #include #include +#include #include #include #include @@ -142,9 +143,9 @@ private: void addChunk(Chunk chunk, size_t from_input); }; -struct ChunksToMerge : public ChunkInfo +struct ChunksToMerge : public ChunkInfoCloneable { - std::unique_ptr chunks; + std::shared_ptr chunks; Int32 bucket_num = -1; bool is_overflows = false; UInt64 chunk_num = 0; // chunk number in order of generation, used during memory bound merging to restore chunks order diff --git a/src/Processors/Transforms/MergingAggregatedTransform.cpp b/src/Processors/Transforms/MergingAggregatedTransform.cpp index ad723da7527..446e60a0b81 100644 --- a/src/Processors/Transforms/MergingAggregatedTransform.cpp +++ b/src/Processors/Transforms/MergingAggregatedTransform.cpp @@ -32,11 +32,10 @@ void MergingAggregatedTransform::consume(Chunk chunk) total_input_rows += input_rows; ++total_input_blocks; - const auto & info = chunk.getChunkInfo(); - if (!info) + if (chunk.getChunkInfos().empty()) throw Exception(ErrorCodes::LOGICAL_ERROR, "Chunk info was not set for chunk in MergingAggregatedTransform."); - if (const auto * agg_info = typeid_cast(info.get())) + if (auto agg_info = chunk.getChunkInfos().get()) { /** If the remote servers used a two-level aggregation method, * then blocks will contain information about the number of the bucket. @@ -49,7 +48,7 @@ void MergingAggregatedTransform::consume(Chunk chunk) bucket_to_blocks[agg_info->bucket_num].emplace_back(std::move(block)); } - else if (typeid_cast(info.get())) + else if (chunk.getChunkInfos().get()) { auto block = getInputPort().getHeader().cloneWithColumns(chunk.getColumns()); block.info.is_overflows = false; @@ -89,7 +88,8 @@ Chunk MergingAggregatedTransform::generate() UInt64 num_rows = block.rows(); Chunk chunk(block.getColumns(), num_rows); - chunk.setChunkInfo(std::move(info)); + + chunk.getChunkInfos().add(std::move(info)); return chunk; } diff --git a/src/Processors/Transforms/PlanSquashingTransform.cpp b/src/Processors/Transforms/PlanSquashingTransform.cpp index 0f433165f14..ee4dfa6a64e 100644 --- a/src/Processors/Transforms/PlanSquashingTransform.cpp +++ b/src/Processors/Transforms/PlanSquashingTransform.cpp @@ -10,20 +10,20 @@ namespace ErrorCodes } PlanSquashingTransform::PlanSquashingTransform( - const Block & header, size_t min_block_size_rows, size_t min_block_size_bytes) - : IInflatingTransform(header, header), squashing(header, min_block_size_rows, min_block_size_bytes) + Block header_, size_t min_block_size_rows, size_t min_block_size_bytes) + : IInflatingTransform(header_, header_) + , squashing(header_, min_block_size_rows, min_block_size_bytes) { } void PlanSquashingTransform::consume(Chunk chunk) { - if (Chunk current_chunk = squashing.add(std::move(chunk)); current_chunk.hasChunkInfo()) - squashed_chunk.swap(current_chunk); + squashed_chunk = squashing.add(std::move(chunk)); } Chunk PlanSquashingTransform::generate() { - if (!squashed_chunk.hasChunkInfo()) + if (!squashed_chunk) throw Exception(ErrorCodes::LOGICAL_ERROR, "Can't generate chunk in SimpleSquashingChunksTransform"); Chunk result_chunk; @@ -33,12 +33,11 @@ Chunk PlanSquashingTransform::generate() bool PlanSquashingTransform::canGenerate() { - return squashed_chunk.hasChunkInfo(); + return bool(squashed_chunk); } Chunk PlanSquashingTransform::getRemaining() { - Chunk current_chunk = squashing.flush(); - return current_chunk; + return squashing.flush(); } } diff --git a/src/Processors/Transforms/PlanSquashingTransform.h b/src/Processors/Transforms/PlanSquashingTransform.h index 4ad2ec2d089..e6db245499e 100644 --- a/src/Processors/Transforms/PlanSquashingTransform.h +++ b/src/Processors/Transforms/PlanSquashingTransform.h @@ -10,7 +10,7 @@ class PlanSquashingTransform : public IInflatingTransform { public: PlanSquashingTransform( - const Block & header, size_t min_block_size_rows, size_t min_block_size_bytes); + Block header_, size_t min_block_size_rows, size_t min_block_size_bytes); String getName() const override { return "PlanSquashingTransform"; } @@ -23,7 +23,6 @@ protected: private: Squashing squashing; Chunk squashed_chunk; - Chunk finish_chunk; }; } diff --git a/src/Processors/Transforms/SelectByIndicesTransform.h b/src/Processors/Transforms/SelectByIndicesTransform.h index 480ab1a0f61..b44f5a3203e 100644 --- a/src/Processors/Transforms/SelectByIndicesTransform.h +++ b/src/Processors/Transforms/SelectByIndicesTransform.h @@ -26,7 +26,7 @@ public: void transform(Chunk & chunk) override { size_t num_rows = chunk.getNumRows(); - const auto * select_final_indices_info = typeid_cast(chunk.getChunkInfo().get()); + auto select_final_indices_info = chunk.getChunkInfos().extract(); if (!select_final_indices_info || !select_final_indices_info->select_final_indices) throw Exception(ErrorCodes::LOGICAL_ERROR, "Chunk passed to SelectByIndicesTransform without indices column"); @@ -41,7 +41,6 @@ public: chunk.setColumns(std::move(columns), index_column->size()); } - chunk.setChunkInfo(nullptr); } }; diff --git a/src/Processors/Transforms/SquashingTransform.cpp b/src/Processors/Transforms/SquashingTransform.cpp index b5a40c75c5b..490a57d4e23 100644 --- a/src/Processors/Transforms/SquashingTransform.cpp +++ b/src/Processors/Transforms/SquashingTransform.cpp @@ -1,13 +1,14 @@ +#include #include #include +#include "Processors/Chunk.h" namespace DB { namespace ErrorCodes { -extern const int LOGICAL_ERROR; -extern const int SIZES_OF_COLUMNS_DOESNT_MATCH; + extern const int LOGICAL_ERROR; } SquashingTransform::SquashingTransform( @@ -19,9 +20,7 @@ SquashingTransform::SquashingTransform( void SquashingTransform::onConsume(Chunk chunk) { - Chunk planned_chunk = squashing.add(std::move(chunk)); - if (planned_chunk.hasChunkInfo()) - cur_chunk = DB::Squashing::squash(std::move(planned_chunk)); + cur_chunk = Squashing::squash(squashing.add(std::move(chunk))); } SquashingTransform::GenerateResult SquashingTransform::onGenerate() @@ -34,10 +33,7 @@ SquashingTransform::GenerateResult SquashingTransform::onGenerate() void SquashingTransform::onFinish() { - Chunk chunk = squashing.flush(); - if (chunk.hasChunkInfo()) - chunk = DB::Squashing::squash(std::move(chunk)); - finish_chunk.setColumns(chunk.getColumns(), chunk.getNumRows()); + finish_chunk = Squashing::squash(squashing.flush()); } void SquashingTransform::work() @@ -50,6 +46,7 @@ void SquashingTransform::work() } ExceptionKeepingTransform::work(); + if (finish_chunk) { data.chunk = std::move(finish_chunk); @@ -59,14 +56,14 @@ void SquashingTransform::work() SimpleSquashingChunksTransform::SimpleSquashingChunksTransform( const Block & header, size_t min_block_size_rows, size_t min_block_size_bytes) - : IInflatingTransform(header, header), squashing(min_block_size_rows, min_block_size_bytes) + : IInflatingTransform(header, header) + , squashing(header, min_block_size_rows, min_block_size_bytes) { } void SimpleSquashingChunksTransform::consume(Chunk chunk) { - Block current_block = squashing.add(getInputPort().getHeader().cloneWithColumns(chunk.detachColumns())); - squashed_chunk.setColumns(current_block.getColumns(), current_block.rows()); + squashed_chunk = Squashing::squash(squashing.add(std::move(chunk))); } Chunk SimpleSquashingChunksTransform::generate() @@ -74,7 +71,9 @@ Chunk SimpleSquashingChunksTransform::generate() if (squashed_chunk.empty()) throw Exception(ErrorCodes::LOGICAL_ERROR, "Can't generate chunk in SimpleSquashingChunksTransform"); - return std::move(squashed_chunk); + Chunk result; + result.swap(squashed_chunk); + return result; } bool SimpleSquashingChunksTransform::canGenerate() @@ -84,143 +83,7 @@ bool SimpleSquashingChunksTransform::canGenerate() Chunk SimpleSquashingChunksTransform::getRemaining() { - Block current_block = squashing.add({}); - squashed_chunk.setColumns(current_block.getColumns(), current_block.rows()); - return std::move(squashed_chunk); + return Squashing::squash(squashing.flush()); } -SquashingLegacy::SquashingLegacy(size_t min_block_size_rows_, size_t min_block_size_bytes_) - : min_block_size_rows(min_block_size_rows_) - , min_block_size_bytes(min_block_size_bytes_) -{ -} - -Block SquashingLegacy::add(Block && input_block) -{ - return addImpl(std::move(input_block)); -} - -Block SquashingLegacy::add(const Block & input_block) -{ - return addImpl(input_block); -} - -/* - * To minimize copying, accept two types of argument: const reference for output - * stream, and rvalue reference for input stream, and decide whether to copy - * inside this function. This allows us not to copy Block unless we absolutely - * have to. - */ -template -Block SquashingLegacy::addImpl(ReferenceType input_block) -{ - /// End of input stream. - if (!input_block) - { - Block to_return; - std::swap(to_return, accumulated_block); - return to_return; - } - - /// Just read block is already enough. - if (isEnoughSize(input_block)) - { - /// If no accumulated data, return just read block. - if (!accumulated_block) - { - return std::move(input_block); - } - - /// Return accumulated data (maybe it has small size) and place new block to accumulated data. - Block to_return = std::move(input_block); - std::swap(to_return, accumulated_block); - return to_return; - } - - /// Accumulated block is already enough. - if (isEnoughSize(accumulated_block)) - { - /// Return accumulated data and place new block to accumulated data. - Block to_return = std::move(input_block); - std::swap(to_return, accumulated_block); - return to_return; - } - - append(std::move(input_block)); - if (isEnoughSize(accumulated_block)) - { - Block to_return; - std::swap(to_return, accumulated_block); - return to_return; - } - - /// Squashed block is not ready. - return {}; -} - - -template -void SquashingLegacy::append(ReferenceType input_block) -{ - if (!accumulated_block) - { - accumulated_block = std::move(input_block); - return; - } - - assert(blocksHaveEqualStructure(input_block, accumulated_block)); - - try - { - for (size_t i = 0, size = accumulated_block.columns(); i < size; ++i) - { - const auto source_column = input_block.getByPosition(i).column; - - auto mutable_column = IColumn::mutate(std::move(accumulated_block.getByPosition(i).column)); - mutable_column->insertRangeFrom(*source_column, 0, source_column->size()); - accumulated_block.getByPosition(i).column = std::move(mutable_column); - } - } - catch (...) - { - /// add() may be called again even after a previous add() threw an exception. - /// Keep accumulated_block in a valid state. - /// Seems ok to discard accumulated data because we're throwing an exception, which the caller will - /// hopefully interpret to mean "this block and all *previous* blocks are potentially lost". - accumulated_block.clear(); - throw; - } -} - - -bool SquashingLegacy::isEnoughSize(const Block & block) -{ - size_t rows = 0; - size_t bytes = 0; - - for (const auto & [column, type, name] : block) - { - if (!column) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Invalid column in block."); - - if (!rows) - rows = column->size(); - else if (rows != column->size()) - throw Exception(ErrorCodes::SIZES_OF_COLUMNS_DOESNT_MATCH, "Sizes of columns doesn't match"); - - bytes += column->byteSize(); - } - - return isEnoughSize(rows, bytes); -} - - -bool SquashingLegacy::isEnoughSize(size_t rows, size_t bytes) const -{ - return (!min_block_size_rows && !min_block_size_bytes) - || (min_block_size_rows && rows >= min_block_size_rows) - || (min_block_size_bytes && bytes >= min_block_size_bytes); -} - - } diff --git a/src/Processors/Transforms/SquashingTransform.h b/src/Processors/Transforms/SquashingTransform.h index 452317e7d5e..092f58f2fe0 100644 --- a/src/Processors/Transforms/SquashingTransform.h +++ b/src/Processors/Transforms/SquashingTransform.h @@ -30,35 +30,6 @@ private: Chunk finish_chunk; }; - -class SquashingLegacy -{ -public: - /// Conditions on rows and bytes are OR-ed. If one of them is zero, then corresponding condition is ignored. - SquashingLegacy(size_t min_block_size_rows_, size_t min_block_size_bytes_); - - /** Add next block and possibly returns squashed block. - * At end, you need to pass empty block. As the result for last (empty) block, you will get last Result with ready = true. - */ - Block add(Block && block); - Block add(const Block & block); - -private: - size_t min_block_size_rows; - size_t min_block_size_bytes; - - Block accumulated_block; - - template - Block addImpl(ReferenceType block); - - template - void append(ReferenceType block); - - bool isEnoughSize(const Block & block); - bool isEnoughSize(size_t rows, size_t bytes) const; -}; - class SimpleSquashingChunksTransform : public IInflatingTransform { public: @@ -73,7 +44,7 @@ protected: Chunk getRemaining() override; private: - SquashingLegacy squashing; + Squashing squashing; Chunk squashed_chunk; }; diff --git a/src/Processors/Transforms/TotalsHavingTransform.cpp b/src/Processors/Transforms/TotalsHavingTransform.cpp index 3f278ca884f..567997c24ab 100644 --- a/src/Processors/Transforms/TotalsHavingTransform.cpp +++ b/src/Processors/Transforms/TotalsHavingTransform.cpp @@ -151,11 +151,7 @@ void TotalsHavingTransform::transform(Chunk & chunk) /// Block with values not included in `max_rows_to_group_by`. We'll postpone it. if (overflow_row) { - const auto & info = chunk.getChunkInfo(); - if (!info) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Chunk info was not set for chunk in TotalsHavingTransform."); - - const auto * agg_info = typeid_cast(info.get()); + const auto & agg_info = chunk.getChunkInfos().get(); if (!agg_info) throw Exception(ErrorCodes::LOGICAL_ERROR, "Chunk should have AggregatedChunkInfo in TotalsHavingTransform."); diff --git a/src/Processors/Transforms/buildPushingToViewsChain.cpp b/src/Processors/Transforms/buildPushingToViewsChain.cpp index c616c882ed9..98d66ed77c3 100644 --- a/src/Processors/Transforms/buildPushingToViewsChain.cpp +++ b/src/Processors/Transforms/buildPushingToViewsChain.cpp @@ -5,7 +5,9 @@ #include #include #include +#include #include +#include #include #include #include @@ -16,6 +18,7 @@ #include #include #include +#include #include #include #include @@ -24,10 +27,13 @@ #include #include #include +#include #include +#include #include #include +#include namespace ProfileEvents @@ -106,7 +112,7 @@ private: class ExecutingInnerQueryFromViewTransform final : public ExceptionKeepingTransform { public: - ExecutingInnerQueryFromViewTransform(const Block & header, ViewRuntimeData & view_, ViewsDataPtr views_data_); + ExecutingInnerQueryFromViewTransform(const Block & header, ViewRuntimeData & view_, ViewsDataPtr views_data_, bool disable_deduplication_for_children_); String getName() const override { return "ExecutingInnerQueryFromView"; } @@ -117,6 +123,7 @@ protected: private: ViewsDataPtr views_data; ViewRuntimeData & view; + bool disable_deduplication_for_children; struct State { @@ -139,7 +146,7 @@ class PushingToLiveViewSink final : public SinkToStorage public: PushingToLiveViewSink(const Block & header, StorageLiveView & live_view_, StoragePtr storage_holder_, ContextPtr context_); String getName() const override { return "PushingToLiveViewSink"; } - void consume(Chunk chunk) override; + void consume(Chunk & chunk) override; private: StorageLiveView & live_view; @@ -153,7 +160,7 @@ class PushingToWindowViewSink final : public SinkToStorage public: PushingToWindowViewSink(const Block & header, StorageWindowView & window_view_, StoragePtr storage_holder_, ContextPtr context_); String getName() const override { return "PushingToWindowViewSink"; } - void consume(Chunk chunk) override; + void consume(Chunk & chunk) override; private: StorageWindowView & window_view; @@ -217,45 +224,10 @@ std::optional generateViewChain( const auto & insert_settings = insert_context->getSettingsRef(); - // Do not deduplicate insertions into MV if the main insertion is Ok if (disable_deduplication_for_children) { insert_context->setSetting("insert_deduplicate", Field{false}); } - else if (insert_settings.update_insert_deduplication_token_in_dependent_materialized_views && - !insert_settings.insert_deduplication_token.value.empty()) - { - /** Update deduplication token passed to dependent MV with current view id. So it is possible to properly handle - * deduplication in complex INSERT flows. - * - * Example: - * - * landing -┬--> mv_1_1 ---> ds_1_1 ---> mv_2_1 --┬-> ds_2_1 ---> mv_3_1 ---> ds_3_1 - * | | - * └--> mv_1_2 ---> ds_1_2 ---> mv_2_2 --┘ - * - * Here we want to avoid deduplication for two different blocks generated from `mv_2_1` and `mv_2_2` that will - * be inserted into `ds_2_1`. - * - * We are forced to use view id instead of table id because there are some possible INSERT flows where no tables - * are involved. - * - * Example: - * - * landing -┬--> mv_1_1 --┬-> ds_1_1 - * | | - * └--> mv_1_2 --┘ - * - */ - auto insert_deduplication_token = insert_settings.insert_deduplication_token.value; - - if (view_id.hasUUID()) - insert_deduplication_token += "_" + toString(view_id.uuid); - else - insert_deduplication_token += "_" + view_id.getFullNameNotQuoted(); - - insert_context->setSetting("insert_deduplication_token", insert_deduplication_token); - } // Processing of blocks for MVs is done block by block, and there will // be no parallel reading after (plus it is not a costless operation) @@ -362,7 +334,13 @@ std::optional generateViewChain( insert_columns.emplace_back(column.name); } - InterpreterInsertQuery interpreter(nullptr, insert_context, false, false, false); + InterpreterInsertQuery interpreter( + nullptr, + insert_context, + /* allow_materialized */ false, + /* no_squash */ false, + /* no_destination */ false, + /* async_isnert */ false); /// TODO: remove sql_security_type check after we turn `ignore_empty_sql_security_in_create_view_query=false` bool check_access = !materialized_view->hasInnerTable() && materialized_view->getInMemoryMetadataPtr()->sql_security_type; @@ -379,6 +357,10 @@ std::optional generateViewChain( table_prefers_large_blocks ? settings.min_insert_block_size_bytes : 0ULL)); } +#ifdef ABORT_ON_LOGICAL_ERROR + out.addSource(std::make_shared("Before squashing", out.getInputHeader())); +#endif + auto counting = std::make_shared(out.getInputHeader(), current_thread, insert_context->getQuota()); counting->setProcessListElement(insert_context->getProcessListElement()); counting->setProgressCallback(insert_context->getProgressCallback()); @@ -421,11 +403,19 @@ std::optional generateViewChain( if (type == QueryViewsLogElement::ViewType::MATERIALIZED) { +#ifdef ABORT_ON_LOGICAL_ERROR + out.addSource(std::make_shared("Right after Inner query", out.getInputHeader())); +#endif + auto executing_inner_query = std::make_shared( - storage_header, views_data->views.back(), views_data); + storage_header, views_data->views.back(), views_data, disable_deduplication_for_children); executing_inner_query->setRuntimeData(view_thread_status, view_counter_ms); out.addSource(std::move(executing_inner_query)); + +#ifdef ABORT_ON_LOGICAL_ERROR + out.addSource(std::make_shared("Right before Inner query", out.getInputHeader())); +#endif } return out; @@ -466,11 +456,7 @@ Chain buildPushingToViewsChain( */ result_chain.addTableLock(storage->lockForShare(context->getInitialQueryId(), context->getSettingsRef().lock_acquire_timeout)); - /// If the "root" table deduplicates blocks, there are no need to make deduplication for children - /// Moreover, deduplication for AggregatingMergeTree children could produce false positives due to low size of inserting blocks - bool disable_deduplication_for_children = false; - if (!context->getSettingsRef().deduplicate_blocks_in_dependent_materialized_views) - disable_deduplication_for_children = !no_destination && storage->supportsDeduplication(); + bool disable_deduplication_for_children = !context->getSettingsRef().deduplicate_blocks_in_dependent_materialized_views; auto table_id = storage->getStorageID(); auto views = DatabaseCatalog::instance().getDependentViews(table_id); @@ -561,12 +547,25 @@ Chain buildPushingToViewsChain( auto sink = std::make_shared(live_view_header, *live_view, storage, context); sink->setRuntimeData(thread_status, elapsed_counter_ms); result_chain.addSource(std::move(sink)); + + result_chain.addSource(std::make_shared(result_chain.getInputHeader())); } else if (auto * window_view = dynamic_cast(storage.get())) { auto sink = std::make_shared(window_view->getInputHeader(), *window_view, storage, context); sink->setRuntimeData(thread_status, elapsed_counter_ms); result_chain.addSource(std::move(sink)); + + result_chain.addSource(std::make_shared(result_chain.getInputHeader())); + } + else if (dynamic_cast(storage.get())) + { + auto sink = storage->write(query_ptr, metadata_snapshot, context, async_insert); + metadata_snapshot->check(sink->getHeader().getColumnsWithTypeAndName()); + sink->setRuntimeData(thread_status, elapsed_counter_ms); + result_chain.addSource(std::move(sink)); + + result_chain.addSource(std::make_shared(result_chain.getInputHeader())); } /// Do not push to destination table if the flag is set else if (!no_destination) @@ -574,8 +573,15 @@ Chain buildPushingToViewsChain( auto sink = storage->write(query_ptr, metadata_snapshot, context, async_insert); metadata_snapshot->check(sink->getHeader().getColumnsWithTypeAndName()); sink->setRuntimeData(thread_status, elapsed_counter_ms); + + result_chain.addSource(std::make_shared(sink->getHeader())); + result_chain.addSource(std::move(sink)); } + else + { + result_chain.addSource(std::make_shared(storage_header)); + } if (result_chain.empty()) result_chain.addSink(std::make_shared(storage_header)); @@ -591,7 +597,7 @@ Chain buildPushingToViewsChain( return result_chain; } -static QueryPipeline process(Block block, ViewRuntimeData & view, const ViewsData & views_data) +static QueryPipeline process(Block block, ViewRuntimeData & view, const ViewsData & views_data, Chunk::ChunkInfoCollection && chunk_infos, bool disable_deduplication_for_children) { const auto & context = view.context; @@ -638,6 +644,19 @@ static QueryPipeline process(Block block, ViewRuntimeData & view, const ViewsDat pipeline.getHeader(), std::make_shared(std::move(converting)))); + pipeline.addTransform(std::make_shared(std::move(chunk_infos), pipeline.getHeader())); + + if (!disable_deduplication_for_children) + { + String materialize_view_id = view.table_id.hasUUID() ? toString(view.table_id.uuid) : view.table_id.getFullNameNotQuoted(); + pipeline.addTransform(std::make_shared(std::move(materialize_view_id), pipeline.getHeader())); + pipeline.addTransform(std::make_shared(pipeline.getHeader())); + } + else + { + pipeline.addTransform(std::make_shared(pipeline.getHeader())); + } + return QueryPipelineBuilder::getPipeline(std::move(pipeline)); } @@ -729,17 +748,19 @@ IProcessor::Status CopyingDataToViewsTransform::prepare() ExecutingInnerQueryFromViewTransform::ExecutingInnerQueryFromViewTransform( const Block & header, ViewRuntimeData & view_, - std::shared_ptr views_data_) + std::shared_ptr views_data_, + bool disable_deduplication_for_children_) : ExceptionKeepingTransform(header, view_.sample_block) , views_data(std::move(views_data_)) , view(view_) + , disable_deduplication_for_children(disable_deduplication_for_children_) { } void ExecutingInnerQueryFromViewTransform::onConsume(Chunk chunk) { - auto block = getInputPort().getHeader().cloneWithColumns(chunk.getColumns()); - state.emplace(process(block, view, *views_data)); + auto block = getInputPort().getHeader().cloneWithColumns(chunk.detachColumns()); + state.emplace(process(std::move(block), view, *views_data, std::move(chunk.getChunkInfos()), disable_deduplication_for_children)); } @@ -771,10 +792,10 @@ PushingToLiveViewSink::PushingToLiveViewSink(const Block & header, StorageLiveVi { } -void PushingToLiveViewSink::consume(Chunk chunk) +void PushingToLiveViewSink::consume(Chunk & chunk) { Progress local_progress(chunk.getNumRows(), chunk.bytes(), 0); - live_view.writeBlock(getHeader().cloneWithColumns(chunk.detachColumns()), context); + live_view.writeBlock(live_view, getHeader().cloneWithColumns(chunk.getColumns()), std::move(chunk.getChunkInfos()), context); if (auto process = context->getProcessListElement()) process->updateProgressIn(local_progress); @@ -794,11 +815,11 @@ PushingToWindowViewSink::PushingToWindowViewSink( { } -void PushingToWindowViewSink::consume(Chunk chunk) +void PushingToWindowViewSink::consume(Chunk & chunk) { Progress local_progress(chunk.getNumRows(), chunk.bytes(), 0); StorageWindowView::writeIntoWindowView( - window_view, getHeader().cloneWithColumns(chunk.detachColumns()), context); + window_view, getHeader().cloneWithColumns(chunk.getColumns()), std::move(chunk.getChunkInfos()), context); if (auto process = context->getProcessListElement()) process->updateProgressIn(local_progress); diff --git a/src/QueryPipeline/QueryPipelineBuilder.h b/src/QueryPipeline/QueryPipelineBuilder.h index f0b2ead687e..a9e5b1535c0 100644 --- a/src/QueryPipeline/QueryPipelineBuilder.h +++ b/src/QueryPipeline/QueryPipelineBuilder.h @@ -193,7 +193,7 @@ public: return concurrency_control; } - void addResources(QueryPlanResourceHolder resources_) { resources = std::move(resources_); } + void addResources(QueryPlanResourceHolder resources_) { resources.append(std::move(resources_)); } void setQueryIdHolder(std::shared_ptr query_id_holder) { resources.query_id_holders.emplace_back(std::move(query_id_holder)); } void addContext(ContextPtr context) { resources.interpreter_context.emplace_back(std::move(context)); } diff --git a/src/QueryPipeline/QueryPlanResourceHolder.cpp b/src/QueryPipeline/QueryPlanResourceHolder.cpp index 2cd4dc42a83..bb2be2c8ffb 100644 --- a/src/QueryPipeline/QueryPlanResourceHolder.cpp +++ b/src/QueryPipeline/QueryPlanResourceHolder.cpp @@ -5,7 +5,7 @@ namespace DB { -QueryPlanResourceHolder & QueryPlanResourceHolder::operator=(QueryPlanResourceHolder && rhs) noexcept +QueryPlanResourceHolder & QueryPlanResourceHolder::append(QueryPlanResourceHolder && rhs) noexcept { table_locks.insert(table_locks.end(), rhs.table_locks.begin(), rhs.table_locks.end()); storage_holders.insert(storage_holders.end(), rhs.storage_holders.begin(), rhs.storage_holders.end()); @@ -16,6 +16,12 @@ QueryPlanResourceHolder & QueryPlanResourceHolder::operator=(QueryPlanResourceHo return *this; } +QueryPlanResourceHolder & QueryPlanResourceHolder::operator=(QueryPlanResourceHolder && rhs) noexcept +{ + append(std::move(rhs)); + return *this; +} + QueryPlanResourceHolder::QueryPlanResourceHolder() = default; QueryPlanResourceHolder::QueryPlanResourceHolder(QueryPlanResourceHolder &&) noexcept = default; QueryPlanResourceHolder::~QueryPlanResourceHolder() = default; diff --git a/src/QueryPipeline/QueryPlanResourceHolder.h b/src/QueryPipeline/QueryPlanResourceHolder.h index ed9eb68b7ba..10f7f39ab09 100644 --- a/src/QueryPipeline/QueryPlanResourceHolder.h +++ b/src/QueryPipeline/QueryPlanResourceHolder.h @@ -20,8 +20,11 @@ struct QueryPlanResourceHolder QueryPlanResourceHolder(QueryPlanResourceHolder &&) noexcept; ~QueryPlanResourceHolder(); + QueryPlanResourceHolder & operator=(QueryPlanResourceHolder &) = delete; + /// Custom move assignment does not destroy data from lhs. It appends data from rhs to lhs. QueryPlanResourceHolder & operator=(QueryPlanResourceHolder &&) noexcept; + QueryPlanResourceHolder & append(QueryPlanResourceHolder &&) noexcept; /// Some processors may implicitly use Context or temporary Storage created by Interpreter. /// But lifetime of Streams is not nested in lifetime of Interpreters, so we have to store it here, diff --git a/src/Server/TCPHandler.cpp b/src/Server/TCPHandler.cpp index 8d69df8de76..07366d7cc07 100644 --- a/src/Server/TCPHandler.cpp +++ b/src/Server/TCPHandler.cpp @@ -888,12 +888,11 @@ AsynchronousInsertQueue::PushResult TCPHandler::processAsyncInsertQuery(Asynchro while (readDataNext()) { - squashing.header = state.block_for_insert; - auto planned_chunk = squashing.add({state.block_for_insert.getColumns(), state.block_for_insert.rows()}); - if (planned_chunk.hasChunkInfo()) + squashing.setHeader(state.block_for_insert.cloneEmpty()); + auto result_chunk = Squashing::squash(squashing.add({state.block_for_insert.getColumns(), state.block_for_insert.rows()})); + if (result_chunk) { - Chunk result_chunk = DB::Squashing::squash(std::move(planned_chunk)); - auto result = state.block_for_insert.cloneWithColumns(result_chunk.getColumns()); + auto result = squashing.getHeader().cloneWithColumns(result_chunk.detachColumns()); return PushResult { .status = PushResult::TOO_MUCH_DATA, @@ -902,12 +901,13 @@ AsynchronousInsertQueue::PushResult TCPHandler::processAsyncInsertQuery(Asynchro } } - auto planned_chunk = squashing.flush(); - Chunk result_chunk; - if (planned_chunk.hasChunkInfo()) - result_chunk = DB::Squashing::squash(std::move(planned_chunk)); + Chunk result_chunk = Squashing::squash(squashing.flush()); + if (!result_chunk) + { + return insert_queue.pushQueryWithBlock(state.parsed_query, squashing.getHeader(), query_context); + } - auto result = squashing.header.cloneWithColumns(result_chunk.getColumns()); + auto result = squashing.getHeader().cloneWithColumns(result_chunk.detachColumns()); return insert_queue.pushQueryWithBlock(state.parsed_query, std::move(result), query_context); } diff --git a/src/Storages/Distributed/DistributedSink.cpp b/src/Storages/Distributed/DistributedSink.cpp index a2bd7237854..b2ce62caf0a 100644 --- a/src/Storages/Distributed/DistributedSink.cpp +++ b/src/Storages/Distributed/DistributedSink.cpp @@ -135,7 +135,7 @@ DistributedSink::DistributedSink( } -void DistributedSink::consume(Chunk chunk) +void DistributedSink::consume(Chunk & chunk) { if (is_first_chunk) { @@ -143,7 +143,7 @@ void DistributedSink::consume(Chunk chunk) is_first_chunk = false; } - auto ordinary_block = getHeader().cloneWithColumns(chunk.detachColumns()); + auto ordinary_block = getHeader().cloneWithColumns(chunk.getColumns()); if (insert_sync) writeSync(ordinary_block); @@ -421,7 +421,13 @@ DistributedSink::runWritingJob(JobReplica & job, const Block & current_block, si /// to resolve tables (in InterpreterInsertQuery::getTable()) auto copy_query_ast = query_ast->clone(); - InterpreterInsertQuery interp(copy_query_ast, job.local_context, allow_materialized); + InterpreterInsertQuery interp( + copy_query_ast, + job.local_context, + allow_materialized, + /* no_squash */ false, + /* no_destination */ false, + /* async_isnert */ false); auto block_io = interp.execute(); job.pipeline = std::move(block_io.pipeline); @@ -716,7 +722,13 @@ void DistributedSink::writeToLocal(const Cluster::ShardInfo & shard_info, const try { - InterpreterInsertQuery interp(query_ast, context, allow_materialized); + InterpreterInsertQuery interp( + query_ast, + context, + allow_materialized, + /* no_squash */ false, + /* no_destination */ false, + /* async_isnert */ false); auto block_io = interp.execute(); PushingPipelineExecutor executor(block_io.pipeline); diff --git a/src/Storages/Distributed/DistributedSink.h b/src/Storages/Distributed/DistributedSink.h index a4c95633595..5b7396f2c6f 100644 --- a/src/Storages/Distributed/DistributedSink.h +++ b/src/Storages/Distributed/DistributedSink.h @@ -49,7 +49,7 @@ public: const Names & columns_to_send_); String getName() const override { return "DistributedSink"; } - void consume(Chunk chunk) override; + void consume(Chunk & chunk) override; void onFinish() override; private: diff --git a/src/Storages/FileLog/StorageFileLog.cpp b/src/Storages/FileLog/StorageFileLog.cpp index abd4b4ce23b..0f9bd8b6ff9 100644 --- a/src/Storages/FileLog/StorageFileLog.cpp +++ b/src/Storages/FileLog/StorageFileLog.cpp @@ -740,7 +740,14 @@ bool StorageFileLog::streamToViews() auto new_context = Context::createCopy(getContext()); - InterpreterInsertQuery interpreter(insert, new_context, false, true, true); + InterpreterInsertQuery interpreter( + insert, + new_context, + /* allow_materialized */ false, + /* no_squash */ true, + /* no_destination */ true, + /* async_isnert */ false); + auto block_io = interpreter.execute(); /// Each stream responsible for closing it's files and store meta diff --git a/src/Storages/Kafka/StorageKafka.cpp b/src/Storages/Kafka/StorageKafka.cpp index f5c5d093ce1..809401bb279 100644 --- a/src/Storages/Kafka/StorageKafka.cpp +++ b/src/Storages/Kafka/StorageKafka.cpp @@ -1099,7 +1099,13 @@ bool StorageKafka::streamToViews() // Create a stream for each consumer and join them in a union stream // Only insert into dependent views and expect that input blocks contain virtual columns - InterpreterInsertQuery interpreter(insert, kafka_context, false, true, true); + InterpreterInsertQuery interpreter( + insert, + kafka_context, + /* allow_materialized */ false, + /* no_squash */ true, + /* no_destination */ true, + /* async_isnert */ false); auto block_io = interpreter.execute(); // Create a stream for each consumer and join them in a union stream diff --git a/src/Storages/LiveView/LiveViewSink.h b/src/Storages/LiveView/LiveViewSink.h index 792133ced64..9803fa0a160 100644 --- a/src/Storages/LiveView/LiveViewSink.h +++ b/src/Storages/LiveView/LiveViewSink.h @@ -71,9 +71,9 @@ public: new_hash.reset(); } - void consume(Chunk chunk) override + void consume(Chunk & chunk) override { - auto block = getHeader().cloneWithColumns(chunk.detachColumns()); + auto block = getHeader().cloneWithColumns(chunk.getColumns()); block.updateHash(*new_hash); new_blocks->push_back(std::move(block)); } diff --git a/src/Storages/LiveView/StorageLiveView.cpp b/src/Storages/LiveView/StorageLiveView.cpp index f1323d45c26..71b1a0a73c9 100644 --- a/src/Storages/LiveView/StorageLiveView.cpp +++ b/src/Storages/LiveView/StorageLiveView.cpp @@ -21,6 +21,7 @@ limitations under the License. */ #include #include #include +#include #include #include #include @@ -331,7 +332,7 @@ Pipe StorageLiveView::watch( return reader; } -void StorageLiveView::writeBlock(const Block & block, ContextPtr local_context) +void StorageLiveView::writeBlock(StorageLiveView & live_view, Block && block, Chunk::ChunkInfoCollection && chunk_infos, ContextPtr local_context) { auto output = std::make_shared(*this); @@ -408,6 +409,21 @@ void StorageLiveView::writeBlock(const Block & block, ContextPtr local_context) builder = interpreter.buildQueryPipeline(); } + builder.addSimpleTransform([&](const Block & cur_header) + { + return std::make_shared(chunk_infos.clone(), cur_header); + }); + + String live_view_id = live_view.getStorageID().hasUUID() ? toString(live_view.getStorageID().uuid) : live_view.getStorageID().getFullNameNotQuoted(); + builder.addSimpleTransform([&](const Block & stream_header) + { + return std::make_shared(live_view_id, stream_header); + }); + builder.addSimpleTransform([&](const Block & stream_header) + { + return std::make_shared(stream_header); + }); + builder.addSimpleTransform([&](const Block & cur_header) { return std::make_shared(cur_header); diff --git a/src/Storages/LiveView/StorageLiveView.h b/src/Storages/LiveView/StorageLiveView.h index 91daac32c7b..12d8e898347 100644 --- a/src/Storages/LiveView/StorageLiveView.h +++ b/src/Storages/LiveView/StorageLiveView.h @@ -118,7 +118,7 @@ public: return 0; } - void writeBlock(const Block & block, ContextPtr context); + void writeBlock(StorageLiveView & live_view, Block && block, Chunk::ChunkInfoCollection && chunk_infos, ContextPtr context); void refresh(); diff --git a/src/Storages/MaterializedView/RefreshTask.cpp b/src/Storages/MaterializedView/RefreshTask.cpp index 29631b95542..857cfd78910 100644 --- a/src/Storages/MaterializedView/RefreshTask.cpp +++ b/src/Storages/MaterializedView/RefreshTask.cpp @@ -378,7 +378,13 @@ void RefreshTask::executeRefreshUnlocked(std::shared_ptr(task->getInfo().data_part->info.level)); + return ChunkAndProgress{ - .chunk = Chunk(ordered_columns, res.row_count, add_part_level ? std::make_shared(task->getInfo().data_part->info.level) : nullptr), + .chunk = std::move(chunk), .num_read_rows = res.num_read_rows, .num_read_bytes = res.num_read_bytes, .is_finished = false}; diff --git a/src/Storages/MergeTree/MergeTreeSequentialSource.cpp b/src/Storages/MergeTree/MergeTreeSequentialSource.cpp index ecf7ac08294..07476e8b2e9 100644 --- a/src/Storages/MergeTree/MergeTreeSequentialSource.cpp +++ b/src/Storages/MergeTree/MergeTreeSequentialSource.cpp @@ -265,7 +265,10 @@ try ++it; } - return Chunk(std::move(res_columns), rows_read, add_part_level ? std::make_shared(data_part->info.level) : nullptr); + auto result = Chunk(std::move(res_columns), rows_read); + if (add_part_level) + result.getChunkInfos().add(std::make_shared(data_part->info.level)); + return result; } } else diff --git a/src/Storages/MergeTree/MergeTreeSink.cpp b/src/Storages/MergeTree/MergeTreeSink.cpp index 5c71994d68d..0ca4276b858 100644 --- a/src/Storages/MergeTree/MergeTreeSink.cpp +++ b/src/Storages/MergeTree/MergeTreeSink.cpp @@ -1,15 +1,22 @@ #include #include #include +#include #include #include #include + namespace ProfileEvents { extern const Event DuplicatedInsertedBlocks; } +namespace ErrorCodes +{ + extern const int LOGICAL_ERROR; +} + namespace DB { @@ -59,12 +66,12 @@ void MergeTreeSink::onCancel() { } -void MergeTreeSink::consume(Chunk chunk) +void MergeTreeSink::consume(Chunk & chunk) { if (num_blocks_processed > 0) storage.delayInsertOrThrowIfNeeded(nullptr, context, false); - auto block = getHeader().cloneWithColumns(chunk.detachColumns()); + auto block = getHeader().cloneWithColumns(chunk.getColumns()); if (!storage_snapshot->object_columns.empty()) convertDynamicColumnsToTuples(block, storage_snapshot); @@ -77,6 +84,18 @@ void MergeTreeSink::consume(Chunk chunk) size_t streams = 0; bool support_parallel_write = false; + auto token_info = chunk.getChunkInfos().get(); + if (!token_info) + throw Exception(ErrorCodes::LOGICAL_ERROR, + "TokenInfo is expected for consumed chunk in MergeTreeSink for table: {}", + storage.getStorageID().getNameForLogs()); + + const bool need_to_define_dedup_token = !token_info->isDefined(); + + String block_dedup_token; + if (token_info->isDefined()) + block_dedup_token = token_info->getToken(); + for (auto & current_block : part_blocks) { ProfileEvents::Counters part_counters; @@ -101,22 +120,16 @@ void MergeTreeSink::consume(Chunk chunk) if (!temp_part.part) continue; + if (need_to_define_dedup_token) + { + chassert(temp_part.part); + const auto hash_value = temp_part.part->getPartBlockIDHash(); + token_info->addChunkHash(toString(hash_value.items[0]) + "_" + toString(hash_value.items[1])); + } + if (!support_parallel_write && temp_part.part->getDataPartStorage().supportParallelWrite()) support_parallel_write = true; - String block_dedup_token; - if (storage.getDeduplicationLog()) - { - const String & dedup_token = settings.insert_deduplication_token; - if (!dedup_token.empty()) - { - /// multiple blocks can be inserted within the same insert query - /// an ordinal number is added to dedup token to generate a distinctive block id for each block - block_dedup_token = fmt::format("{}_{}", dedup_token, chunk_dedup_seqnum); - ++chunk_dedup_seqnum; - } - } - size_t max_insert_delayed_streams_for_parallel_write; if (settings.max_insert_delayed_streams_for_parallel_write.changed) @@ -128,6 +141,7 @@ void MergeTreeSink::consume(Chunk chunk) /// In case of too much columns/parts in block, flush explicitly. streams += temp_part.streams.size(); + if (streams > max_insert_delayed_streams_for_parallel_write) { finishDelayedChunk(); @@ -144,11 +158,16 @@ void MergeTreeSink::consume(Chunk chunk) { .temp_part = std::move(temp_part), .elapsed_ns = elapsed_ns, - .block_dedup_token = std::move(block_dedup_token), + .block_dedup_token = block_dedup_token, .part_counters = std::move(part_counters), }); } + if (need_to_define_dedup_token) + { + token_info->finishChunkHashes(); + } + finishDelayedChunk(); delayed_chunk = std::make_unique(); delayed_chunk->partitions = std::move(partitions); @@ -161,6 +180,8 @@ void MergeTreeSink::finishDelayedChunk() if (!delayed_chunk) return; + const Settings & settings = context->getSettingsRef(); + for (auto & partition : delayed_chunk->partitions) { ProfileEventsScope scoped_attach(&partition.part_counters); @@ -179,7 +200,8 @@ void MergeTreeSink::finishDelayedChunk() storage.fillNewPartName(part, lock); auto * deduplication_log = storage.getDeduplicationLog(); - if (deduplication_log) + + if (settings.insert_deduplicate && deduplication_log) { const String block_id = part->getZeroLevelPartBlockID(partition.block_dedup_token); auto res = deduplication_log->addPart(block_id, part->info); diff --git a/src/Storages/MergeTree/MergeTreeSink.h b/src/Storages/MergeTree/MergeTreeSink.h index cf6715a3415..90976020d52 100644 --- a/src/Storages/MergeTree/MergeTreeSink.h +++ b/src/Storages/MergeTree/MergeTreeSink.h @@ -25,7 +25,7 @@ public: ~MergeTreeSink() override; String getName() const override { return "MergeTreeSink"; } - void consume(Chunk chunk) override; + void consume(Chunk & chunk) override; void onStart() override; void onFinish() override; void onCancel() override; @@ -36,7 +36,6 @@ private: size_t max_parts_per_block; ContextPtr context; StorageSnapshotPtr storage_snapshot; - UInt64 chunk_dedup_seqnum = 0; /// input chunk ordinal number in case of dedup token UInt64 num_blocks_processed = 0; /// We can delay processing for previous chunk and start writing a new one. diff --git a/src/Storages/MergeTree/MutateTask.cpp b/src/Storages/MergeTree/MutateTask.cpp index f74cccd518b..9a775db73e2 100644 --- a/src/Storages/MergeTree/MutateTask.cpp +++ b/src/Storages/MergeTree/MutateTask.cpp @@ -1301,6 +1301,7 @@ void PartMergerWriter::prepare() bool PartMergerWriter::mutateOriginalPartAndPrepareProjections() { Block cur_block; + Block projection_header; if (MutationHelpers::checkOperationIsNotCanceled(*ctx->merges_blocker, ctx->mutate_entry) && ctx->mutating_executor->pull(cur_block)) { if (ctx->minmax_idx) @@ -1318,14 +1319,12 @@ bool PartMergerWriter::mutateOriginalPartAndPrepareProjections() ProfileEventTimeIncrement watch(ProfileEvents::MutateTaskProjectionsCalculationMicroseconds); Block block_to_squash = projection.calculate(cur_block, ctx->context); - projection_squashes[i].header = block_to_squash; - Chunk planned_chunk = projection_squashes[i].add({block_to_squash.getColumns(), block_to_squash.rows()}); + projection_squashes[i].setHeader(block_to_squash.cloneEmpty()); - if (planned_chunk.hasChunkInfo()) + Chunk squashed_chunk = Squashing::squash(projection_squashes[i].add({block_to_squash.getColumns(), block_to_squash.rows()})); + if (squashed_chunk) { - Chunk projection_chunk = DB::Squashing::squash(std::move(planned_chunk)); - - auto result = block_to_squash.cloneWithColumns(projection_chunk.getColumns()); + auto result = projection_squashes[i].getHeader().cloneWithColumns(squashed_chunk.detachColumns()); auto tmp_part = MergeTreeDataWriter::writeTempProjectionPart( *ctx->data, ctx->log, result, projection, ctx->new_data_part.get(), ++block_num); tmp_part.finalize(); @@ -1346,12 +1345,10 @@ bool PartMergerWriter::mutateOriginalPartAndPrepareProjections() { const auto & projection = *ctx->projections_to_build[i]; auto & projection_squash_plan = projection_squashes[i]; - auto planned_chunk = projection_squash_plan.flush(); - if (planned_chunk.hasChunkInfo()) + auto squashed_chunk = Squashing::squash(projection_squash_plan.flush()); + if (squashed_chunk) { - Chunk projection_chunk = DB::Squashing::squash(std::move(planned_chunk)); - - auto result = projection_squash_plan.header.cloneWithColumns(projection_chunk.getColumns()); + auto result = projection_squash_plan.getHeader().cloneWithColumns(squashed_chunk.detachColumns()); auto temp_part = MergeTreeDataWriter::writeTempProjectionPart( *ctx->data, ctx->log, result, projection, ctx->new_data_part.get(), ++block_num); temp_part.finalize(); diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp index 83e8a7a3bc5..63e1c12a15f 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp @@ -3,6 +3,7 @@ #include #include #include +#include #include "Common/Exception.h" #include #include @@ -19,6 +20,7 @@ #include #include + namespace ProfileEvents { extern const Event DuplicatedInsertedBlocks; @@ -255,12 +257,12 @@ size_t ReplicatedMergeTreeSinkImpl::checkQuorumPrecondition(const } template -void ReplicatedMergeTreeSinkImpl::consume(Chunk chunk) +void ReplicatedMergeTreeSinkImpl::consume(Chunk & chunk) { if (num_blocks_processed > 0) storage.delayInsertOrThrowIfNeeded(&storage.partial_shutdown_event, context, false); - auto block = getHeader().cloneWithColumns(chunk.detachColumns()); + auto block = getHeader().cloneWithColumns(chunk.getColumns()); const auto & settings = context->getSettingsRef(); @@ -286,13 +288,25 @@ void ReplicatedMergeTreeSinkImpl::consume(Chunk chunk) if constexpr (async_insert) { - const auto & chunk_info = chunk.getChunkInfo(); - if (const auto * async_insert_info_ptr = typeid_cast(chunk_info.get())) + const auto async_insert_info_ptr = chunk.getChunkInfos().get(); + if (async_insert_info_ptr) async_insert_info = std::make_shared(async_insert_info_ptr->offsets, async_insert_info_ptr->tokens); else throw Exception(ErrorCodes::LOGICAL_ERROR, "No chunk info for async inserts"); } + String block_dedup_token; + auto token_info = chunk.getChunkInfos().get(); + if (!token_info) + throw Exception(ErrorCodes::LOGICAL_ERROR, + "TokenInfo is expected for consumed chunk in ReplicatedMergeTreeSink for table: {}", + storage.getStorageID().getNameForLogs()); + + const bool need_to_define_dedup_token = !token_info->isDefined(); + + if (token_info->isDefined()) + block_dedup_token = token_info->getToken(); + auto part_blocks = MergeTreeDataWriter::splitBlockIntoParts(std::move(block), max_parts_per_block, metadata_snapshot, context, async_insert_info); using DelayedPartition = typename ReplicatedMergeTreeSinkImpl::DelayedChunk::Partition; @@ -344,23 +358,10 @@ void ReplicatedMergeTreeSinkImpl::consume(Chunk chunk) } else { - if (deduplicate) { - String block_dedup_token; - /// We add the hash from the data and partition identifier to deduplication ID. /// That is, do not insert the same data to the same partition twice. - - const String & dedup_token = settings.insert_deduplication_token; - if (!dedup_token.empty()) - { - /// multiple blocks can be inserted within the same insert query - /// an ordinal number is added to dedup token to generate a distinctive block id for each block - block_dedup_token = fmt::format("{}_{}", dedup_token, chunk_dedup_seqnum); - ++chunk_dedup_seqnum; - } - block_id = temp_part.part->getZeroLevelPartBlockID(block_dedup_token); LOG_DEBUG(log, "Wrote block with ID '{}', {} rows{}", block_id, current_block.block.rows(), quorumLogMessage(replicas_num)); } @@ -368,6 +369,13 @@ void ReplicatedMergeTreeSinkImpl::consume(Chunk chunk) { LOG_DEBUG(log, "Wrote block with {} rows{}", current_block.block.rows(), quorumLogMessage(replicas_num)); } + + if (need_to_define_dedup_token) + { + chassert(temp_part.part); + const auto hash_value = temp_part.part->getPartBlockIDHash(); + token_info->addChunkHash(toString(hash_value.items[0]) + "_" + toString(hash_value.items[1])); + } } profile_events_scope.reset(); @@ -413,17 +421,15 @@ void ReplicatedMergeTreeSinkImpl::consume(Chunk chunk) )); } + if (need_to_define_dedup_token) + { + token_info->finishChunkHashes(); + } + finishDelayedChunk(zookeeper); delayed_chunk = std::make_unique(); delayed_chunk->partitions = std::move(partitions); - /// If deduplicated data should not be inserted into MV, we need to set proper - /// value for `last_block_is_duplicate`, which is possible only after the part is committed. - /// Othervide we can delay commit. - /// TODO: we can also delay commit if there is no MVs. - if (!settings.deduplicate_blocks_in_dependent_materialized_views) - finishDelayedChunk(zookeeper); - ++num_blocks_processed; } @@ -433,8 +439,6 @@ void ReplicatedMergeTreeSinkImpl::finishDelayedChunk(const ZooKeeperWithF if (!delayed_chunk) return; - last_block_is_duplicate = false; - for (auto & partition : delayed_chunk->partitions) { ProfileEventsScope scoped_attach(&partition.part_counters); @@ -447,8 +451,6 @@ void ReplicatedMergeTreeSinkImpl::finishDelayedChunk(const ZooKeeperWithF { bool deduplicated = commitPart(zookeeper, part, partition.block_id, delayed_chunk->replicas_num).second; - last_block_is_duplicate = last_block_is_duplicate || deduplicated; - /// Set a special error code if the block is duplicate int error = (deduplicate && deduplicated) ? ErrorCodes::INSERT_WAS_DEDUPLICATED : 0; auto counters_snapshot = std::make_shared(partition.part_counters.getPartiallyAtomicSnapshot()); @@ -537,7 +539,7 @@ bool ReplicatedMergeTreeSinkImpl::writeExistingPart(MergeTreeData::Mutabl ProfileEventsScope profile_events_scope; String original_part_dir = part->getDataPartStorage().getPartDirectory(); - auto try_rollback_part_rename = [this, &part, &original_part_dir]() + auto try_rollback_part_rename = [this, &part, &original_part_dir] () { if (original_part_dir == part->getDataPartStorage().getPartDirectory()) return; @@ -576,16 +578,6 @@ bool ReplicatedMergeTreeSinkImpl::writeExistingPart(MergeTreeData::Mutabl } } -template -bool ReplicatedMergeTreeSinkImpl::lastBlockIsDuplicate() const -{ - /// If MV is responsible for deduplication, block is not considered duplicating. - if (context->getSettingsRef().deduplicate_blocks_in_dependent_materialized_views) - return false; - - return last_block_is_duplicate; -} - template std::vector ReplicatedMergeTreeSinkImpl::detectConflictsInAsyncBlockIDs(const std::vector & ids) { @@ -1163,8 +1155,16 @@ void ReplicatedMergeTreeSinkImpl::onStart() template void ReplicatedMergeTreeSinkImpl::onFinish() { - auto zookeeper = storage.getZooKeeper(); - finishDelayedChunk(std::make_shared(zookeeper)); + const auto & settings = context->getSettingsRef(); + + ZooKeeperWithFaultInjectionPtr zookeeper = ZooKeeperWithFaultInjection::createInstance( + settings.insert_keeper_fault_injection_probability, + settings.insert_keeper_fault_injection_seed, + storage.getZooKeeper(), + "ReplicatedMergeTreeSink::onFinish", + log); + + finishDelayedChunk(zookeeper); } template diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeSink.h b/src/Storages/MergeTree/ReplicatedMergeTreeSink.h index 8a37d85a750..7d025361717 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeSink.h +++ b/src/Storages/MergeTree/ReplicatedMergeTreeSink.h @@ -51,7 +51,7 @@ public: ~ReplicatedMergeTreeSinkImpl() override; void onStart() override; - void consume(Chunk chunk) override; + void consume(Chunk & chunk) override; void onFinish() override; String getName() const override { return "ReplicatedMergeTreeSink"; } @@ -59,9 +59,6 @@ public: /// For ATTACHing existing data on filesystem. bool writeExistingPart(MergeTreeData::MutableDataPartPtr & part); - /// For proper deduplication in MaterializedViews - bool lastBlockIsDuplicate() const override; - struct DelayedChunk; private: std::vector detectConflictsInAsyncBlockIDs(const std::vector & ids); @@ -119,7 +116,6 @@ private: bool allow_attach_while_readonly = false; bool quorum_parallel = false; const bool deduplicate = true; - bool last_block_is_duplicate = false; UInt64 num_blocks_processed = 0; LoggerPtr log; diff --git a/src/Storages/MessageQueueSink.cpp b/src/Storages/MessageQueueSink.cpp index 4fb81d69070..36899011e33 100644 --- a/src/Storages/MessageQueueSink.cpp +++ b/src/Storages/MessageQueueSink.cpp @@ -40,7 +40,7 @@ void MessageQueueSink::onFinish() producer->finish(); } -void MessageQueueSink::consume(Chunk chunk) +void MessageQueueSink::consume(Chunk & chunk) { const auto & columns = chunk.getColumns(); if (columns.empty()) diff --git a/src/Storages/MessageQueueSink.h b/src/Storages/MessageQueueSink.h index b3c1e61734f..4a9248c6c4d 100644 --- a/src/Storages/MessageQueueSink.h +++ b/src/Storages/MessageQueueSink.h @@ -35,7 +35,7 @@ public: String getName() const override { return storage_name + "Sink"; } - void consume(Chunk chunk) override; + void consume(Chunk & chunk) override; void onStart() override; void onFinish() override; diff --git a/src/Storages/NATS/StorageNATS.cpp b/src/Storages/NATS/StorageNATS.cpp index 0b88a9e8929..8f0e2d76473 100644 --- a/src/Storages/NATS/StorageNATS.cpp +++ b/src/Storages/NATS/StorageNATS.cpp @@ -644,7 +644,13 @@ bool StorageNATS::streamToViews() insert->table_id = table_id; // Only insert into dependent views and expect that input blocks contain virtual columns - InterpreterInsertQuery interpreter(insert, nats_context, false, true, true); + InterpreterInsertQuery interpreter( + insert, + nats_context, + /* allow_materialized */ false, + /* no_squash */ true, + /* no_destination */ true, + /* async_isnert */ false); auto block_io = interpreter.execute(); auto storage_snapshot = getStorageSnapshot(getInMemoryMetadataPtr(), getContext()); diff --git a/src/Storages/ObjectStorage/StorageObjectStorageSink.cpp b/src/Storages/ObjectStorage/StorageObjectStorageSink.cpp index 000e73c70ce..c864fb9a7e2 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorageSink.cpp +++ b/src/Storages/ObjectStorage/StorageObjectStorageSink.cpp @@ -40,12 +40,12 @@ StorageObjectStorageSink::StorageObjectStorageSink( configuration->format, *write_buf, sample_block, context, format_settings_); } -void StorageObjectStorageSink::consume(Chunk chunk) +void StorageObjectStorageSink::consume(Chunk & chunk) { std::lock_guard lock(cancel_mutex); if (cancelled) return; - writer->write(getHeader().cloneWithColumns(chunk.detachColumns())); + writer->write(getHeader().cloneWithColumns(chunk.getColumns())); } void StorageObjectStorageSink::onCancel() diff --git a/src/Storages/ObjectStorage/StorageObjectStorageSink.h b/src/Storages/ObjectStorage/StorageObjectStorageSink.h index e0081193686..6ab531bb21a 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorageSink.h +++ b/src/Storages/ObjectStorage/StorageObjectStorageSink.h @@ -20,7 +20,7 @@ public: String getName() const override { return "StorageObjectStorageSink"; } - void consume(Chunk chunk) override; + void consume(Chunk & chunk) override; void onCancel() override; diff --git a/src/Storages/ObjectStorageQueue/StorageObjectStorageQueue.cpp b/src/Storages/ObjectStorageQueue/StorageObjectStorageQueue.cpp index 4388864434e..14b828e7268 100644 --- a/src/Storages/ObjectStorageQueue/StorageObjectStorageQueue.cpp +++ b/src/Storages/ObjectStorageQueue/StorageObjectStorageQueue.cpp @@ -454,7 +454,13 @@ bool StorageObjectStorageQueue::streamToViews() while (!shutdown_called && !file_iterator->isFinished()) { - InterpreterInsertQuery interpreter(insert, queue_context, false, true, true); + InterpreterInsertQuery interpreter( + insert, + queue_context, + /* allow_materialized */ false, + /* no_squash */ true, + /* no_destination */ true, + /* async_isnert */ false); auto block_io = interpreter.execute(); auto read_from_format_info = prepareReadingFromFormat( block_io.pipeline.getHeader().getNames(), diff --git a/src/Storages/PartitionedSink.cpp b/src/Storages/PartitionedSink.cpp index 09b009b26d8..ee2570756ed 100644 --- a/src/Storages/PartitionedSink.cpp +++ b/src/Storages/PartitionedSink.cpp @@ -51,7 +51,7 @@ SinkPtr PartitionedSink::getSinkForPartitionKey(StringRef partition_key) return it->second; } -void PartitionedSink::consume(Chunk chunk) +void PartitionedSink::consume(Chunk & chunk) { const auto & columns = chunk.getColumns(); @@ -104,7 +104,7 @@ void PartitionedSink::consume(Chunk chunk) for (const auto & [partition_key, partition_index] : partition_id_to_chunk_index) { auto sink = getSinkForPartitionKey(partition_key); - sink->consume(std::move(partition_index_to_chunk[partition_index])); + sink->consume(partition_index_to_chunk[partition_index]); } } diff --git a/src/Storages/PartitionedSink.h b/src/Storages/PartitionedSink.h index 68edeb6fd73..fcd67556dc9 100644 --- a/src/Storages/PartitionedSink.h +++ b/src/Storages/PartitionedSink.h @@ -20,7 +20,7 @@ public: String getName() const override { return "PartitionedSink"; } - void consume(Chunk chunk) override; + void consume(Chunk & chunk) override; void onException(std::exception_ptr exception) override; diff --git a/src/Storages/PostgreSQL/MaterializedPostgreSQLConsumer.cpp b/src/Storages/PostgreSQL/MaterializedPostgreSQLConsumer.cpp index ba3cc6f58d0..44479bd01e2 100644 --- a/src/Storages/PostgreSQL/MaterializedPostgreSQLConsumer.cpp +++ b/src/Storages/PostgreSQL/MaterializedPostgreSQLConsumer.cpp @@ -697,7 +697,13 @@ void MaterializedPostgreSQLConsumer::syncTables() insert->table_id = storage->getStorageID(); insert->columns = std::make_shared(buffer->columns_ast); - InterpreterInsertQuery interpreter(insert, insert_context, true); + InterpreterInsertQuery interpreter( + insert, + insert_context, + /* allow_materialized */ true, + /* no_squash */ false, + /* no_destination */ false, + /* async_isnert */ false); auto io = interpreter.execute(); auto input = std::make_shared( result_rows.cloneEmpty(), Chunk(result_rows.getColumns(), result_rows.rows())); diff --git a/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp b/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp index 2bb1e2dde0d..f632e553a0d 100644 --- a/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp +++ b/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp @@ -437,7 +437,13 @@ StorageInfo PostgreSQLReplicationHandler::loadFromSnapshot(postgres::Connection auto insert_context = materialized_storage->getNestedTableContext(); - InterpreterInsertQuery interpreter(insert, insert_context); + InterpreterInsertQuery interpreter( + insert, + insert_context, + /* allow_materialized */ false, + /* no_squash */ false, + /* no_destination */ false, + /* async_isnert */ false); auto block_io = interpreter.execute(); const StorageInMemoryMetadata & storage_metadata = nested_storage->getInMemoryMetadata(); diff --git a/src/Storages/RabbitMQ/StorageRabbitMQ.cpp b/src/Storages/RabbitMQ/StorageRabbitMQ.cpp index e4b19992151..f3d2aff68c8 100644 --- a/src/Storages/RabbitMQ/StorageRabbitMQ.cpp +++ b/src/Storages/RabbitMQ/StorageRabbitMQ.cpp @@ -1129,7 +1129,13 @@ bool StorageRabbitMQ::tryStreamToViews() } // Only insert into dependent views and expect that input blocks contain virtual columns - InterpreterInsertQuery interpreter(insert, rabbitmq_context, /* allow_materialized_ */ false, /* no_squash_ */ true, /* no_destination_ */ true); + InterpreterInsertQuery interpreter( + insert, + rabbitmq_context, + /* allow_materialized */ false, + /* no_squash */ true, + /* no_destination */ true, + /* async_isnert */ false); auto block_io = interpreter.execute(); block_io.pipeline.complete(Pipe::unitePipes(std::move(pipes))); diff --git a/src/Storages/RocksDB/EmbeddedRocksDBBulkSink.cpp b/src/Storages/RocksDB/EmbeddedRocksDBBulkSink.cpp index 24411620c21..f6127f3ed8a 100644 --- a/src/Storages/RocksDB/EmbeddedRocksDBBulkSink.cpp +++ b/src/Storages/RocksDB/EmbeddedRocksDBBulkSink.cpp @@ -219,7 +219,7 @@ std::pair EmbeddedRocksDBBulkSink::seriali return {std::move(serialized_key_column), std::move(serialized_value_column)}; } -void EmbeddedRocksDBBulkSink::consume(Chunk chunk_) +void EmbeddedRocksDBBulkSink::consume(Chunk & chunk_) { std::vector chunks_to_write = squash(std::move(chunk_)); @@ -249,7 +249,10 @@ void EmbeddedRocksDBBulkSink::onFinish() { /// If there is any data left, write it. if (!chunks.empty()) - consume({}); + { + Chunk empty; + consume(empty); + } } String EmbeddedRocksDBBulkSink::getTemporarySSTFilePath() diff --git a/src/Storages/RocksDB/EmbeddedRocksDBBulkSink.h b/src/Storages/RocksDB/EmbeddedRocksDBBulkSink.h index 1f548e7813d..64190c8c86f 100644 --- a/src/Storages/RocksDB/EmbeddedRocksDBBulkSink.h +++ b/src/Storages/RocksDB/EmbeddedRocksDBBulkSink.h @@ -32,7 +32,7 @@ public: ~EmbeddedRocksDBBulkSink() override; - void consume(Chunk chunk) override; + void consume(Chunk & chunk) override; void onFinish() override; diff --git a/src/Storages/RocksDB/EmbeddedRocksDBSink.cpp b/src/Storages/RocksDB/EmbeddedRocksDBSink.cpp index c451cfd1bf5..1f7f6939f40 100644 --- a/src/Storages/RocksDB/EmbeddedRocksDBSink.cpp +++ b/src/Storages/RocksDB/EmbeddedRocksDBSink.cpp @@ -29,7 +29,7 @@ EmbeddedRocksDBSink::EmbeddedRocksDBSink( serializations = getHeader().getSerializations(); } -void EmbeddedRocksDBSink::consume(Chunk chunk) +void EmbeddedRocksDBSink::consume(Chunk & chunk) { auto rows = chunk.getNumRows(); const auto & columns = chunk.getColumns(); diff --git a/src/Storages/RocksDB/EmbeddedRocksDBSink.h b/src/Storages/RocksDB/EmbeddedRocksDBSink.h index 011322df829..2e1e0c7b429 100644 --- a/src/Storages/RocksDB/EmbeddedRocksDBSink.h +++ b/src/Storages/RocksDB/EmbeddedRocksDBSink.h @@ -17,7 +17,7 @@ public: StorageEmbeddedRocksDB & storage_, const StorageMetadataPtr & metadata_snapshot_); - void consume(Chunk chunk) override; + void consume(Chunk & chunk) override; String getName() const override { return "EmbeddedRocksDBSink"; } private: diff --git a/src/Storages/RocksDB/StorageEmbeddedRocksDB.cpp b/src/Storages/RocksDB/StorageEmbeddedRocksDB.cpp index 365915cfd68..409703c84c6 100644 --- a/src/Storages/RocksDB/StorageEmbeddedRocksDB.cpp +++ b/src/Storages/RocksDB/StorageEmbeddedRocksDB.cpp @@ -314,7 +314,8 @@ void StorageEmbeddedRocksDB::mutate(const MutationCommands & commands, ContextPt Block block; while (executor.pull(block)) { - sink->consume(Chunk{block.getColumns(), block.rows()}); + auto chunk = Chunk(block.getColumns(), block.rows()); + sink->consume(chunk); } } diff --git a/src/Storages/StorageBuffer.cpp b/src/Storages/StorageBuffer.cpp index c403287c43c..1bcc602f168 100644 --- a/src/Storages/StorageBuffer.cpp +++ b/src/Storages/StorageBuffer.cpp @@ -608,7 +608,7 @@ public: String getName() const override { return "BufferSink"; } - void consume(Chunk chunk) override + void consume(Chunk & chunk) override { size_t rows = chunk.getNumRows(); if (!rows) @@ -1021,7 +1021,13 @@ void StorageBuffer::writeBlockToDestination(const Block & block, StoragePtr tabl auto insert_context = Context::createCopy(getContext()); insert_context->makeQueryContext(); - InterpreterInsertQuery interpreter{insert, insert_context, allow_materialized}; + InterpreterInsertQuery interpreter( + insert, + insert_context, + allow_materialized, + /* no_squash */ false, + /* no_destination */ false, + /* async_isnert */ false); auto block_io = interpreter.execute(); PushingPipelineExecutor executor(block_io.pipeline); diff --git a/src/Storages/StorageDictionary.cpp b/src/Storages/StorageDictionary.cpp index 447fd87cdc9..72f873d1a0e 100644 --- a/src/Storages/StorageDictionary.cpp +++ b/src/Storages/StorageDictionary.cpp @@ -1,3 +1,4 @@ +#include #include #include #include @@ -162,6 +163,7 @@ Pipe StorageDictionary::read( { auto registered_dictionary_name = location == Location::SameDatabaseAndNameAsDictionary ? getStorageID().getInternalDictionaryName() : dictionary_name; auto dictionary = getContext()->getExternalDictionariesLoader().getDictionary(registered_dictionary_name, local_context); + local_context->checkAccess(AccessType::dictGet, dictionary->getDatabaseOrNoDatabaseTag(), dictionary->getDictionaryID().getTableName()); return dictionary->read(column_names, max_block_size, threads); } diff --git a/src/Storages/StorageDistributed.cpp b/src/Storages/StorageDistributed.cpp index 2cf3ced2904..65323b4bb52 100644 --- a/src/Storages/StorageDistributed.cpp +++ b/src/Storages/StorageDistributed.cpp @@ -1033,7 +1033,13 @@ std::optional StorageDistributed::distributedWriteBetweenDistribu const auto & shard_info = shards_info[shard_index]; if (shard_info.isLocal()) { - InterpreterInsertQuery interpreter(new_query, query_context); + InterpreterInsertQuery interpreter( + new_query, + query_context, + /* allow_materialized */ false, + /* no_squash */ false, + /* no_destination */ false, + /* async_isnert */ false); pipeline.addCompletedPipeline(interpreter.execute().pipeline); } else diff --git a/src/Storages/StorageFile.cpp b/src/Storages/StorageFile.cpp index 8301bd6676d..6c31a8a3a71 100644 --- a/src/Storages/StorageFile.cpp +++ b/src/Storages/StorageFile.cpp @@ -1789,12 +1789,12 @@ public: String getName() const override { return "StorageFileSink"; } - void consume(Chunk chunk) override + void consume(Chunk & chunk) override { std::lock_guard cancel_lock(cancel_mutex); if (cancelled) return; - writer->write(getHeader().cloneWithColumns(chunk.detachColumns())); + writer->write(getHeader().cloneWithColumns(chunk.getColumns())); } void onCancel() override diff --git a/src/Storages/StorageKeeperMap.cpp b/src/Storages/StorageKeeperMap.cpp index ce760a8d63f..587cb621362 100644 --- a/src/Storages/StorageKeeperMap.cpp +++ b/src/Storages/StorageKeeperMap.cpp @@ -120,10 +120,10 @@ public: std::string getName() const override { return "StorageKeeperMapSink"; } - void consume(Chunk chunk) override + void consume(Chunk & chunk) override { auto rows = chunk.getNumRows(); - auto block = getHeader().cloneWithColumns(chunk.detachColumns()); + auto block = getHeader().cloneWithColumns(chunk.getColumns()); WriteBufferFromOwnString wb_key; WriteBufferFromOwnString wb_value; @@ -1249,7 +1249,10 @@ void StorageKeeperMap::mutate(const MutationCommands & commands, ContextPtr loca Block block; while (executor.pull(block)) - sink->consume(Chunk{block.getColumns(), block.rows()}); + { + auto chunk = Chunk(block.getColumns(), block.rows()); + sink->consume(chunk); + } sink->finalize(strict); } diff --git a/src/Storages/StorageLog.cpp b/src/Storages/StorageLog.cpp index ee0a37b50e3..303532dfeca 100644 --- a/src/Storages/StorageLog.cpp +++ b/src/Storages/StorageLog.cpp @@ -1,5 +1,6 @@ #include #include +#include #include #include @@ -22,7 +23,6 @@ #include #include -#include "StorageLogSettings.h" #include #include #include @@ -342,7 +342,7 @@ public: } } - void consume(Chunk chunk) override; + void consume(Chunk & chunk) override; void onFinish() override; private: @@ -399,9 +399,9 @@ private: }; -void LogSink::consume(Chunk chunk) +void LogSink::consume(Chunk & chunk) { - auto block = getHeader().cloneWithColumns(chunk.detachColumns()); + auto block = getHeader().cloneWithColumns(chunk.getColumns()); metadata_snapshot->check(block, true); for (auto & stream : streams | boost::adaptors::map_values) diff --git a/src/Storages/StorageMemory.cpp b/src/Storages/StorageMemory.cpp index cd892488b86..42bac783618 100644 --- a/src/Storages/StorageMemory.cpp +++ b/src/Storages/StorageMemory.cpp @@ -62,7 +62,7 @@ public: String getName() const override { return "MemorySink"; } - void consume(Chunk chunk) override + void consume(Chunk & chunk) override { auto block = getHeader().cloneWithColumns(chunk.getColumns()); storage_snapshot->metadata->check(block, true); diff --git a/src/Storages/StorageMongoDB.cpp b/src/Storages/StorageMongoDB.cpp index 62a2a048642..e0818fafae9 100644 --- a/src/Storages/StorageMongoDB.cpp +++ b/src/Storages/StorageMongoDB.cpp @@ -17,7 +17,6 @@ #include #include #include -#include #include @@ -107,12 +106,12 @@ public: String getName() const override { return "StorageMongoDBSink"; } - void consume(Chunk chunk) override + void consume(Chunk & chunk) override { Poco::MongoDB::Database db(db_name); Poco::MongoDB::Document::Vector documents; - auto block = getHeader().cloneWithColumns(chunk.detachColumns()); + auto block = getHeader().cloneWithColumns(chunk.getColumns()); size_t num_rows = block.rows(); size_t num_cols = block.columns(); diff --git a/src/Storages/StorageMySQL.cpp b/src/Storages/StorageMySQL.cpp index f2e19ad4c80..1d1a0ffdeaf 100644 --- a/src/Storages/StorageMySQL.cpp +++ b/src/Storages/StorageMySQL.cpp @@ -152,9 +152,9 @@ public: String getName() const override { return "StorageMySQLSink"; } - void consume(Chunk chunk) override + void consume(Chunk & chunk) override { - auto block = getHeader().cloneWithColumns(chunk.detachColumns()); + auto block = getHeader().cloneWithColumns(chunk.getColumns()); auto blocks = splitBlocks(block, max_batch_rows); mysqlxx::Transaction trans(entry); try diff --git a/src/Storages/StoragePostgreSQL.cpp b/src/Storages/StoragePostgreSQL.cpp index b5a388e8159..d3214e7ed13 100644 --- a/src/Storages/StoragePostgreSQL.cpp +++ b/src/Storages/StoragePostgreSQL.cpp @@ -227,9 +227,9 @@ public: String getName() const override { return "PostgreSQLSink"; } - void consume(Chunk chunk) override + void consume(Chunk & chunk) override { - auto block = getHeader().cloneWithColumns(chunk.detachColumns()); + auto block = getHeader().cloneWithColumns(chunk.getColumns()); if (!inserter) { if (on_conflict.empty()) diff --git a/src/Storages/StorageRedis.cpp b/src/Storages/StorageRedis.cpp index 83bb3c606c9..1a275320f43 100644 --- a/src/Storages/StorageRedis.cpp +++ b/src/Storages/StorageRedis.cpp @@ -147,7 +147,7 @@ class RedisSink : public SinkToStorage public: RedisSink(StorageRedis & storage_, const StorageMetadataPtr & metadata_snapshot_); - void consume(Chunk chunk) override; + void consume(Chunk & chunk) override; String getName() const override { return "RedisSink"; } private: @@ -169,10 +169,10 @@ RedisSink::RedisSink(StorageRedis & storage_, const StorageMetadataPtr & metadat } } -void RedisSink::consume(Chunk chunk) +void RedisSink::consume(Chunk & chunk) { auto rows = chunk.getNumRows(); - auto block = getHeader().cloneWithColumns(chunk.detachColumns()); + auto block = getHeader().cloneWithColumns(chunk.getColumns()); WriteBufferFromOwnString wb_key; WriteBufferFromOwnString wb_value; @@ -567,7 +567,8 @@ void StorageRedis::mutate(const MutationCommands & commands, ContextPtr context_ Block block; while (executor.pull(block)) { - sink->consume(Chunk{block.getColumns(), block.rows()}); + Chunk chunk(block.getColumns(), block.rows()); + sink->consume(chunk); } } diff --git a/src/Storages/StorageSQLite.cpp b/src/Storages/StorageSQLite.cpp index 179e4cee199..85417a2f2a4 100644 --- a/src/Storages/StorageSQLite.cpp +++ b/src/Storages/StorageSQLite.cpp @@ -141,7 +141,7 @@ public: String getName() const override { return "SQLiteSink"; } - void consume(Chunk chunk) override + void consume(Chunk & chunk) override { auto block = getHeader().cloneWithColumns(chunk.getColumns()); WriteBufferFromOwnString sqlbuf; diff --git a/src/Storages/StorageSet.cpp b/src/Storages/StorageSet.cpp index 22bd01e0071..5692da4e454 100644 --- a/src/Storages/StorageSet.cpp +++ b/src/Storages/StorageSet.cpp @@ -45,7 +45,7 @@ public: const String & backup_file_name_, bool persistent_); String getName() const override { return "SetOrJoinSink"; } - void consume(Chunk chunk) override; + void consume(Chunk & chunk) override; void onFinish() override; private: @@ -83,9 +83,9 @@ SetOrJoinSink::SetOrJoinSink( { } -void SetOrJoinSink::consume(Chunk chunk) +void SetOrJoinSink::consume(Chunk & chunk) { - Block block = getHeader().cloneWithColumns(chunk.detachColumns()); + Block block = getHeader().cloneWithColumns(chunk.getColumns()); table.insertBlock(block, getContext()); if (persistent) diff --git a/src/Storages/StorageStripeLog.cpp b/src/Storages/StorageStripeLog.cpp index 68eaa75b416..c892cca4523 100644 --- a/src/Storages/StorageStripeLog.cpp +++ b/src/Storages/StorageStripeLog.cpp @@ -228,9 +228,9 @@ public: } } - void consume(Chunk chunk) override + void consume(Chunk & chunk) override { - block_out->write(getHeader().cloneWithColumns(chunk.detachColumns())); + block_out->write(getHeader().cloneWithColumns(chunk.getColumns())); } void onFinish() override diff --git a/src/Storages/StorageURL.cpp b/src/Storages/StorageURL.cpp index 9cec8c75ebe..fd7c3976803 100644 --- a/src/Storages/StorageURL.cpp +++ b/src/Storages/StorageURL.cpp @@ -567,12 +567,12 @@ StorageURLSink::StorageURLSink( } -void StorageURLSink::consume(Chunk chunk) +void StorageURLSink::consume(Chunk & chunk) { std::lock_guard lock(cancel_mutex); if (cancelled) return; - writer->write(getHeader().cloneWithColumns(chunk.detachColumns())); + writer->write(getHeader().cloneWithColumns(chunk.getColumns())); } void StorageURLSink::onCancel() diff --git a/src/Storages/StorageURL.h b/src/Storages/StorageURL.h index fa7cc6eeeef..1804079e75f 100644 --- a/src/Storages/StorageURL.h +++ b/src/Storages/StorageURL.h @@ -251,7 +251,7 @@ public: const String & method = Poco::Net::HTTPRequest::HTTP_POST); std::string getName() const override { return "StorageURLSink"; } - void consume(Chunk chunk) override; + void consume(Chunk & chunk) override; void onCancel() override; void onException(std::exception_ptr exception) override; void onFinish() override; diff --git a/src/Storages/System/StorageSystemZooKeeper.cpp b/src/Storages/System/StorageSystemZooKeeper.cpp index ba9fac6d289..81d3b0fe659 100644 --- a/src/Storages/System/StorageSystemZooKeeper.cpp +++ b/src/Storages/System/StorageSystemZooKeeper.cpp @@ -120,7 +120,7 @@ public: ZooKeeperSink(const Block & header, ContextPtr context) : SinkToStorage(header), zookeeper(context->getZooKeeper()) { } String getName() const override { return "ZooKeeperSink"; } - void consume(Chunk chunk) override + void consume(Chunk & chunk) override { auto block = getHeader().cloneWithColumns(chunk.getColumns()); size_t rows = block.rows(); diff --git a/src/Storages/WindowView/StorageWindowView.cpp b/src/Storages/WindowView/StorageWindowView.cpp index b842cdda022..e15da0074d5 100644 --- a/src/Storages/WindowView/StorageWindowView.cpp +++ b/src/Storages/WindowView/StorageWindowView.cpp @@ -32,6 +32,7 @@ #include #include #include +#include #include #include #include @@ -305,7 +306,7 @@ namespace public: explicit AddingAggregatedChunkInfoTransform(Block header) : ISimpleTransform(header, header, false) { } - void transform(Chunk & chunk) override { chunk.setChunkInfo(std::make_shared()); } + void transform(Chunk & chunk) override { chunk.getChunkInfos().add(std::make_shared()); } String getName() const override { return "AddingAggregatedChunkInfoTransform"; } }; @@ -690,7 +691,13 @@ inline void StorageWindowView::fire(UInt32 watermark) StoragePtr target_table = getTargetTable(); auto insert = std::make_shared(); insert->table_id = target_table->getStorageID(); - InterpreterInsertQuery interpreter(insert, getContext()); + InterpreterInsertQuery interpreter( + insert, + getContext(), + /* allow_materialized */ false, + /* no_squash */ false, + /* no_destination */ false, + /* async_isnert */ false); auto block_io = interpreter.execute(); auto pipe = Pipe(std::make_shared(blocks, header)); @@ -1414,7 +1421,7 @@ void StorageWindowView::eventTimeParser(const ASTCreateQuery & query) } void StorageWindowView::writeIntoWindowView( - StorageWindowView & window_view, const Block & block, ContextPtr local_context) + StorageWindowView & window_view, Block && block, Chunk::ChunkInfoCollection && chunk_infos, ContextPtr local_context) { window_view.throwIfWindowViewIsDisabled(local_context); while (window_view.modifying_query) @@ -1429,7 +1436,7 @@ void StorageWindowView::writeIntoWindowView( window_view.max_watermark = window_view.getWindowUpperBound(first_record_timestamp); } - Pipe pipe(std::make_shared(block.cloneEmpty(), Chunk(block.getColumns(), block.rows()))); + Pipe pipe(std::make_shared(block)); UInt32 lateness_bound = 0; UInt32 t_max_watermark = 0; @@ -1474,10 +1481,10 @@ void StorageWindowView::writeIntoWindowView( auto syntax_result = TreeRewriter(local_context).analyze(query, columns); auto filter_expression = ExpressionAnalyzer(filter_function, syntax_result, local_context).getActionsDAG(false); - pipe.addSimpleTransform([&](const Block & header) + pipe.addSimpleTransform([&](const Block & header_) { return std::make_shared( - header, std::make_shared(filter_expression), + header_, std::make_shared(filter_expression), filter_function->getColumnName(), true); }); } @@ -1532,6 +1539,30 @@ void StorageWindowView::writeIntoWindowView( QueryProcessingStage::WithMergeableState); builder = select_block.buildQueryPipeline(); + + builder.addSimpleTransform([&](const Block & stream_header) + { + // Can't move chunk_infos here, that function could be called several times + return std::make_shared(chunk_infos.clone(), stream_header); + }); + + String window_view_id = window_view.getStorageID().hasUUID() ? toString(window_view.getStorageID().uuid) : window_view.getStorageID().getFullNameNotQuoted(); + builder.addSimpleTransform([&](const Block & stream_header) + { + return std::make_shared(window_view_id, stream_header); + }); + builder.addSimpleTransform([&](const Block & stream_header) + { + return std::make_shared(stream_header); + }); + +#ifdef ABORT_ON_LOGICAL_ERROR + builder.addSimpleTransform([&](const Block & stream_header) + { + return std::make_shared("StorageWindowView: Afrer tmp table before squashing", stream_header); + }); +#endif + builder.addSimpleTransform([&](const Block & current_header) { return std::make_shared( @@ -1571,6 +1602,13 @@ void StorageWindowView::writeIntoWindowView( lateness_upper_bound); }); +#ifdef ABORT_ON_LOGICAL_ERROR + builder.addSimpleTransform([&](const Block & stream_header) + { + return std::make_shared("StorageWindowView: Afrer WatermarkTransform", stream_header); + }); +#endif + auto inner_table = window_view.getInnerTable(); auto lock = inner_table->lockForShare( local_context->getCurrentQueryId(), local_context->getSettingsRef().lock_acquire_timeout); @@ -1587,9 +1625,16 @@ void StorageWindowView::writeIntoWindowView( auto convert_actions = std::make_shared( convert_actions_dag, ExpressionActionsSettings::fromContext(local_context, CompileExpressions::yes)); - builder.addSimpleTransform([&](const Block & header) { return std::make_shared(header, convert_actions); }); + builder.addSimpleTransform([&](const Block & header_) { return std::make_shared(header_, convert_actions); }); } +#ifdef ABORT_ON_LOGICAL_ERROR + builder.addSimpleTransform([&](const Block & stream_header) + { + return std::make_shared("StorageWindowView: Before out", stream_header); + }); +#endif + builder.addChain(Chain(std::move(output))); builder.setSinks([&](const Block & cur_header, Pipe::StreamType) { diff --git a/src/Storages/WindowView/StorageWindowView.h b/src/Storages/WindowView/StorageWindowView.h index f79867df424..14ac65091d3 100644 --- a/src/Storages/WindowView/StorageWindowView.h +++ b/src/Storages/WindowView/StorageWindowView.h @@ -166,7 +166,7 @@ public: BlockIO populate(); - static void writeIntoWindowView(StorageWindowView & window_view, const Block & block, ContextPtr context); + static void writeIntoWindowView(StorageWindowView & window_view, Block && block, Chunk::ChunkInfoCollection && chunk_infos, ContextPtr context); ASTPtr getMergeableQuery() const { return mergeable_query->clone(); } diff --git a/src/Storages/transformQueryForExternalDatabaseAnalyzer.cpp b/src/Storages/transformQueryForExternalDatabaseAnalyzer.cpp index dc1749b3196..ff40b1fcc2d 100644 --- a/src/Storages/transformQueryForExternalDatabaseAnalyzer.cpp +++ b/src/Storages/transformQueryForExternalDatabaseAnalyzer.cpp @@ -71,8 +71,7 @@ ASTPtr getASTForExternalDatabaseFromQueryTree(const QueryTreeNodePtr & query_tre { if (join_node->getStrictness() != JoinStrictness::All) allow_where = false; - - if (join_node->getKind() == JoinKind::Left) + else if (join_node->getKind() == JoinKind::Left) allow_where = join_node->getLeftTableExpression()->isEqual(*table_expression); else if (join_node->getKind() == JoinKind::Right) allow_where = join_node->getRightTableExpression()->isEqual(*table_expression); diff --git a/src/TableFunctions/TableFunctionDictionary.cpp b/src/TableFunctions/TableFunctionDictionary.cpp index 867fbf5b11e..d58c01385e0 100644 --- a/src/TableFunctions/TableFunctionDictionary.cpp +++ b/src/TableFunctions/TableFunctionDictionary.cpp @@ -80,7 +80,6 @@ ColumnsDescription TableFunctionDictionary::getActualTableStructure(ContextPtr c StoragePtr TableFunctionDictionary::executeImpl( const ASTPtr &, ContextPtr context, const std::string & table_name, ColumnsDescription, bool is_insert_query) const { - context->checkAccess(AccessType::dictGet, getDatabaseName(), table_name); StorageID dict_id(getDatabaseName(), table_name); auto dictionary_table_structure = getActualTableStructure(context, is_insert_query); diff --git a/tests/ci/compress_files.py b/tests/ci/compress_files.py index 70c0aca6a36..f49c872b5a0 100644 --- a/tests/ci/compress_files.py +++ b/tests/ci/compress_files.py @@ -58,14 +58,14 @@ def decompress_fast(archive_path: Path, result_path: Optional[Path] = None) -> N archive_path, result_path, ) - program_part = "--use-compress-program='zstd --threads=0'" + program_part = "--use-compress-program='zstd --threads=0 -d'" elif PIGZ.exists(): logging.info( "pigz found, will compress and decompress faster ('%s' -> '%s')", archive_path, result_path, ) - program_part = "--use-compress-program='pigz'" + program_part = "--use-compress-program='pigz -d'" else: program_part = "-z" logging.info( diff --git a/tests/clickhouse-test b/tests/clickhouse-test index aa703ad814d..7f92ed7989b 100755 --- a/tests/clickhouse-test +++ b/tests/clickhouse-test @@ -3387,13 +3387,13 @@ def parse_args(): parser.add_argument( "--replace-replicated-with-shared", action="store_true", - default=os.environ.get("USE_META_IN_KEEPER_FOR_MERGE_TREE", False), + default=os.environ.get("REPLACE_RMT_WITH_SMT", False), help="Replace ReplicatedMergeTree engine with SharedMergeTree", ) parser.add_argument( "--replace-non-replicated-with-shared", action="store_true", - default=False, + default=os.environ.get("REPLACE_MT_WITH_SMT", False), help="Replace ordinary MergeTree engine with SharedMergeTree", ) diff --git a/tests/queries/0_stateless/02805_distributed_queries_timeouts.reference b/tests/integration/test_accept_invalid_certificate/__init__.py similarity index 100% rename from tests/queries/0_stateless/02805_distributed_queries_timeouts.reference rename to tests/integration/test_accept_invalid_certificate/__init__.py diff --git a/tests/integration/test_accept_invalid_certificate/certs/ca-cert.pem b/tests/integration/test_accept_invalid_certificate/certs/ca-cert.pem new file mode 100644 index 00000000000..c4f88831126 --- /dev/null +++ b/tests/integration/test_accept_invalid_certificate/certs/ca-cert.pem @@ -0,0 +1,32 @@ +-----BEGIN CERTIFICATE----- +MIIFhTCCA22gAwIBAgIUZHeZ4ME7rL1RYGJZGPgOTHl7btMwDQYJKoZIhvcNAQEL +BQAwUjELMAkGA1UEBhMCUlUxEzARBgNVBAgMClNvbWUtU3RhdGUxITAfBgNVBAoM +GEludGVybmV0IFdpZGdpdHMgUHR5IEx0ZDELMAkGA1UEAwwCY2EwHhcNMjQwNzAx +MDUxMTI0WhcNMzQwNjI5MDUxMTI0WjBSMQswCQYDVQQGEwJSVTETMBEGA1UECAwK +U29tZS1TdGF0ZTEhMB8GA1UECgwYSW50ZXJuZXQgV2lkZ2l0cyBQdHkgTHRkMQsw +CQYDVQQDDAJjYTCCAiIwDQYJKoZIhvcNAQEBBQADggIPADCCAgoCggIBALf623IC +1r6Bzwa4dTufRPvFWQinT4uIBv7UvAk8tu8kmLSMOOC2COALNRCSu0s1SFcO6i5G +CgmaWdoTXuw+NYH9n5rU06N991NBfCoLbf0eSoie7bRdTCKkqWWKaw+dCvKL5vWj +LEZRmdh8ZN0lxa05eIsLxErvgWqE8hOOQ7jgRqYLAagp6XkMdB+YRkXFtbVcze83 +ZNxFqNjdgwYYybaqDZKGeOnRDRof7P/cyMkbGNUI7+JYA5eGmDgiOTGbaCZx5AUK +DKmkdmy5IJBWS0rYR8Ol9Sjs6NSYkmYiLFEWFNSvI88K99+Ab3DEi9JMEqMeNTJ5 +1sx48e2OfMG7ywoGj3D3zf7SUxAfwW+7Ls+v7c/kB9hvspA9iVsqjAKcSPAPnJXk +zR5r+zcPwjb/dvGGGB+D7PhvreEpj0n7PRFEB07jviA1E8zkwdk1L9FefYMG2oRl +8J7miWxoyqUOtA6AYhbN36+I0iJkjxIjXLLzB/dhOzerNTzRHcJtmgSyIpcPIV42 +IsJBttLOW0Mpkfw8z88xji+1VWHm+H8sOWCzbfjncScu/z5Xhb92C2T+AG+RpakB +LwyTsovf+xv1gGF/1gmmdKSoV/JZuHSczMmmDLJ9vXRnIpK0AgL16lfs6o0C2Pe2 +OHJv0oViokU5mW+Mge3U9kUrh6BVITk8UjUDAgMBAAGjUzBRMB0GA1UdDgQWBBRA +x0+tgav3Q2gRt34AtKZiGfFH7zAfBgNVHSMEGDAWgBRAx0+tgav3Q2gRt34AtKZi +GfFH7zAPBgNVHRMBAf8EBTADAQH/MA0GCSqGSIb3DQEBCwUAA4ICAQBcV7s7au6r +bEkHaW9RbGe/7iHesOcH2uyiALGeiCF+pfN9QbfkpJJbMRDhKr+sVqJH6lS+Iys5 +MpPaRHOInNs5ajiPQcIxtxAnw2HEsy6LUVvs6RUHrS7Bxlc406YQCY/sPyItAt8t +U/qhx5VNj6Mx1smyKZHFHZIZ0eYieriJkK6NB2E7Ji6EPGe7lkh3kEoVTfaSVCkP +p3d4U603kzCk8PGkYlhjWUCVMhDRwgEy6FXHmU156HivGpYurKcab7cMUfr50MyK +1Mx/NaHOYHhIIQ87t9wF2yQaMskEcyK9XOorp+HFtnTdci+6+J3u2Kmyckq81uy9 +uZreH8bUKgmntOhSqtl4tWGMWxX78c7rX2cGIbSJsZzzjmUdhvnpYeuhCCrUPzY3 +kIt9Q6q7DCKG+f+D4bHQ292emEfKYb9LgftyOX3pcAJ+d3c56Cs7c0RRVFAZyVnX +1O9ZvHrypmii5RARNuyVtttgDpuGBrRCa2kSNlXD9PJJitI2kfoWrCI/88Pw8H/y +bLO9/ZcCOsxBGmfrQDMUSC8rE8JLUSKV80otWwtHcxcnfpsobs64/Rz6aOLYEiS4 +/0M6xyUYdOsEGUh1uyX5qxto8iOWop4S9DLy1QVx+N8/fC6gjhMb4Vg2oTBnHlGr +JcxdYP8jVe6tUgvRKAyNlccPIq20udfGTw== +-----END CERTIFICATE----- diff --git a/tests/integration/test_accept_invalid_certificate/certs/ca-key.pem b/tests/integration/test_accept_invalid_certificate/certs/ca-key.pem new file mode 100644 index 00000000000..66d548d0cf5 --- /dev/null +++ b/tests/integration/test_accept_invalid_certificate/certs/ca-key.pem @@ -0,0 +1,52 @@ +-----BEGIN PRIVATE KEY----- +MIIJQwIBADANBgkqhkiG9w0BAQEFAASCCS0wggkpAgEAAoICAQC3+ttyAta+gc8G +uHU7n0T7xVkIp0+LiAb+1LwJPLbvJJi0jDjgtgjgCzUQkrtLNUhXDuouRgoJmlna +E17sPjWB/Z+a1NOjffdTQXwqC239HkqInu20XUwipKllimsPnQryi+b1oyxGUZnY +fGTdJcWtOXiLC8RK74FqhPITjkO44EamCwGoKel5DHQfmEZFxbW1XM3vN2TcRajY +3YMGGMm2qg2Shnjp0Q0aH+z/3MjJGxjVCO/iWAOXhpg4Ijkxm2gmceQFCgyppHZs +uSCQVktK2EfDpfUo7OjUmJJmIixRFhTUryPPCvffgG9wxIvSTBKjHjUyedbMePHt +jnzBu8sKBo9w983+0lMQH8Fvuy7Pr+3P5AfYb7KQPYlbKowCnEjwD5yV5M0ea/s3 +D8I2/3bxhhgfg+z4b63hKY9J+z0RRAdO474gNRPM5MHZNS/RXn2DBtqEZfCe5ols +aMqlDrQOgGIWzd+viNIiZI8SI1yy8wf3YTs3qzU80R3CbZoEsiKXDyFeNiLCQbbS +zltDKZH8PM/PMY4vtVVh5vh/LDlgs23453EnLv8+V4W/dgtk/gBvkaWpAS8Mk7KL +3/sb9YBhf9YJpnSkqFfyWbh0nMzJpgyyfb10ZyKStAIC9epX7OqNAtj3tjhyb9KF +YqJFOZlvjIHt1PZFK4egVSE5PFI1AwIDAQABAoICACPKkBVqevddznYqvVfHgMaQ +GmWGEwH8iQvbO+d6IcZ9Mp3xJ4GDuE9Jrt2WBgy4BWY2VxWAd6RtnCxCF3XTFlq/ +CeU3oSKNt2dBdpPBXHjhAzP34xtzCQQ2T2QbHP/RF9eeouRHha36YvkMHh0to3qz +EwHN3LaNSHp9+ZAXXZJiQyw7OOvLl4l+hDq/R3B8aYCqWL/AYYU+of3qffpaKSe5 +hmFqeqxWCHv8GQGfpIoHFPogvnYklDJOs188PS8q4gSUpHzceyEnhwhYrX6W4xfg +4nkr2HlJOXReaCCpZkCyEIRdvBQjuJwI4uUnv2G+feOw1T2IGWL25m3GCfwweeuZ +AWqqbj5Xnl1+VgvaNm68aHBVlfZ14rIJ/Hu1yJV/E+5m8+3xewAor0jshWceC3OS +lhrZp+EyrwxsKj17s2IBhk4EIOLjfosoDqdw3E2DrtqKrnI63MLUTaEqHWyXnF82 +Mn880DR/E3Byj2kKtdIVesAdO0ARtIRqedPunpcvhp1eMZ2wrLJXAHMDBJlUZF64 +ZAyO3ll79PhBn/QShGu+KQ682Ns1T3j3J14n2zqBKsU1GzSLBOZO8QXfFELsv6Xj +NGYPg2SbkRwbYHEPr6G9ehsxKaTpMAtPesdz1devKDI4nSpd2qE9ZfREH+CW0MkK +bpiL20d4blIG1DP84qodAoIBAQDEBXbn0IEeyL3j50/AUJY/K4seeDm1ltJmhoNU +HC0dwt2FOHrD1J/Cx6S19zuEVt1qMwJXELxOVzixykKPy73ETW+Qfcpo2J51THrl +fSEUpoheD37Cwjv9vzb6Qb4ZXDr4acLIB06mAUKfo0Pez2x6IybWqS/dZoD1bfh5 +wiSCITXQB4n6mI7chfDDyTqgf3C5U5b2UABLDdIKQZHY5kX55Vfr0Lz+Gt2t6uUy +hUiN4C+Qrx1LoHY4x7g1SG9lKASG5zv5dksc+a6uYsS4N8TM3ZQz7BnZrCQPNiWz +K70p5pMjdVTHmQqThoww9y+rRiHMU3qUSZ0tX4nrY99/MiFNAoIBAQDwRi0GzkeA +ECWfr2R+vh5lnxXBeAvM46v+8CzwJHNxtGAhB1ojod7GxOxHiMnWDXubRiNVlhx0 +OHeCq11hSQh2+a7s8xwxYbC3paYf1ZQq81YmTGd8v+/QX9b21Kq9dpx+G2C3eikZ +rHMuSifrSEwZMxdw/gKvQ+x66PTNeK8fIgAtZsE96VX3u7sCCNhbBElWWg5OwwMf +0RH5Pi/O8UXxkZs2SUKFaXUFMxjYn9r9636QI8LaLsCvcbPjrhshPtHWHqgrZzia +8gD00ty3mcydRU//xNfHruew2iRZUVVB8E0GQ80eQCsHrI0YkP876at55DKbyZLi +pGisswDdyoePAoIBAQCfOFUSDMGksKfVY8Da1FqGKydQc5Vkk4KJdr9b5Zsb7nmi +yXKmeKHgQ+vdRlmYsDofFhnTgKuD8sMBXuncBUE6Hy/8rMBpk3J6ksnMeGwTawWf +aXSsW7sA7H2KnzX7QXhA6R8m/RB8NM/Z/XKWm+Nf/geBG2Hes3ESaeYHeJPRA4rD +VCxWWe7I3x7m2R/+2CVG08fugKMMU8iFPidqh/jU994HlMC+ojHoH8FV3P9HBo98 +NVFVaZw9mLRey1x4TR4IOfD0O4rdmU2zJlTl0Z7QTTchSajhrzqHBJNwyplXXIut +pJNdoUj+9a4P4+hbdZbkcfZCVt0NyumrQq96kMyNAoIBAQC8Jcemd6tHnd1IYPip +wshcvSk1GYLsCzAo7vvRhEmeNSe9VVuBYYToU1XOYk/ivSYYneCXaXeYjosPGI+D +2cO2HqiaxXhDNwNBm4YtkXp385p3XQcAifAJVLMpD7kvqYy7t0aSBiffAcoj6k++ +Dxv/86EcituhUOgCCx9/Mky3bVGEO92AVJJL7oZu/mwlQYMICnb/t/dfV8DiUo7v +VsF1e5Z1+QINJFpNGwkOwIakpV6zEj3Hyip6d+fslu6Jpv570//OFQMfWffjvqa1 +Ysr2MEuHtiVeKRkmxHsIMgc4HirCiB872/WWCclm/kvPJ5huV4JFuZ+6uBF7gqRg +CPMDAoIBAGs0KCL3a/eXIPWhFcOgtpcNTb3o4gK/s38+GLIVbdP7MUNEkMeWq+Pf +PBUca2p1M8CkbpRi0OAvg1gewAR/+bCdyzoipl1+3v3goYC3J5TAs5NT/siBcjrm +QcTKEHmTeVu82Kpi1JVtcfS2TWuSmT6vSB2A2H3R4GEPQQXnXeS/FnRA0I85MFzq +zxMkar5VY7Eg0Z3/bN5dNqAYSzqjgMMhRMPN1W54gjL2ev65Mzt1jYf6xz7gpJOh +6Fw2EOrM23c4OckZx2dZ7Je6ns2U82D81ffhXRwWfQxoKXvMVMccBJmHTbkVEZlE +VAq9BWwNistc+YfRka1OOeq37MgpXq8= +-----END PRIVATE KEY----- diff --git a/tests/integration/test_accept_invalid_certificate/certs/client-cert.pem b/tests/integration/test_accept_invalid_certificate/certs/client-cert.pem new file mode 100644 index 00000000000..2dd497409c3 --- /dev/null +++ b/tests/integration/test_accept_invalid_certificate/certs/client-cert.pem @@ -0,0 +1,30 @@ +-----BEGIN CERTIFICATE----- +MIIFLzCCAxcCFA6o2DM1UGQ1NNCQzp0n4hb3iSd9MA0GCSqGSIb3DQEBCwUAMFIx +CzAJBgNVBAYTAlJVMRMwEQYDVQQIDApTb21lLVN0YXRlMSEwHwYDVQQKDBhJbnRl +cm5ldCBXaWRnaXRzIFB0eSBMdGQxCzAJBgNVBAMMAmNhMB4XDTI0MDcwMTA1MTEy +NloXDTM0MDYyOTA1MTEyNlowVjELMAkGA1UEBhMCUlUxEzARBgNVBAgMClNvbWUt +U3RhdGUxITAfBgNVBAoMGEludGVybmV0IFdpZGdpdHMgUHR5IEx0ZDEPMA0GA1UE +AwwGY2xpZW50MIICIjANBgkqhkiG9w0BAQEFAAOCAg8AMIICCgKCAgEAl9i6heiD +2inNs0lKt7XjK0QyXKyIg63uVO39ZpLY5sj1noZnbFmRo5MYC4MMIrlD3wGnqUo/ +mdev2eg8qYry0CmzhLscVEha7LN+7wmdftYSGkXOl9E08KIv9Uw2bHap4KEIRvJ6 +298bdsuv1l4ljWo3VDkPCczXIaUi94/o1uNu0sbPRluBP8kjFLhwV8Ph/7mxsofu +V5tDYNOZ8VozA7aHAnJqXBCFZuW6igIH3XaROY9LOe2FD+qxtpqJHp7E+ZNsXXH5 +SXMQzE2/X7eKHA7zvdG4z5GLgmz3GEvq3ZBJUGUe6DCnrxqBXUeLTqyummohGE57 +LuqzrH9etFj/91NaZW9uwN42pxGJSkeZaJ5BDnzO3VYxqvvDnFs/2TFbJiZ065vI +NpA0PnuAkGjztaNAm6qXNTXsWo8bdi8TDgIwKxoo5KIlWsxnxKB82tfBovWd+JLa +xLlWNKbKgNb6xc9OmNCyhYkRXhqIsBrpLy6n2yppyYEiCBzPrqwJ2p3Y88yj3dsc +xiiu0ngFSroE2pLwCgi1rQY1x9VO4XQrrq6vBvPhi530+cdXdVTnnyBhaOMX4bWX +OOjI97xJmR/YbOvpgKw9v/na82t2fgKoqIs124iZ21QOiffOIfwwg2AKj8EaWJ9/ +Era8sZMsq/g56PWYwT1J85cCMvbNwGOXjusCAwEAATANBgkqhkiG9w0BAQsFAAOC +AgEAAPSps9k8HEZrfBbzH5SHASNTUc7F81h2EYhaEuQrXoan39u3syCKYMSJDt0P +AVCYbIvXmlHeaBba9YXtZVZJpCM53b+x01297x1bXzRklYxnpuD2IauuQgiaJBXV +VlCjEy7GHBAmRQwV7llkoxx+qHsA0b+oHKvTRc9FJzk/Fx4Eb496gNMvUk47f+15 +WBmWoQolqwbH1Ivhca7BpTvTSeNGkJ14cbmuhwryLstHJCPep+vp2TYS9bHFZIPY +g3APnWv1qShuA7Oj68GLpRVJmImgyPMK0i8LDGcLhrDiG+HYEr89MiRWDvBHun7O +Sp5y8zDnDZSrN+EtRp8DypHeB7R0Ukmn76NP6NsigWCPMlLaHNVsvVAwWwFNae9u +OxsoW07tJo3lZjDx+tFrUb7IRyn/9PLDIwEs4SvDP+O97o0JkCszwfRhkDnjY7Qj +XCgXoCBalIadgnGshbEjFLH0ZSwAoCIC/35X0Zo7cd1+Fz4Xq1tgnV5ec6ci/EHi +qsOmGfTqZz5HJlmfyxUOYicizV6SjNZSFWY07ebYRewx6ymdDFhqrbnJIEUGJYlU +5qDY1nemoqh9QRIhm1+73hAKHtmcaa1u72gj1QBtRD5eG3ToM3GHgdbUUPvFpenx +UPfzw/wbzITSqPoYr1/XIGq9e1muTqHbFNlGVByIW0ejeZQ= +-----END CERTIFICATE----- diff --git a/tests/integration/test_accept_invalid_certificate/certs/client-key.pem b/tests/integration/test_accept_invalid_certificate/certs/client-key.pem new file mode 100644 index 00000000000..09c04c1e815 --- /dev/null +++ b/tests/integration/test_accept_invalid_certificate/certs/client-key.pem @@ -0,0 +1,52 @@ +-----BEGIN PRIVATE KEY----- +MIIJQwIBADANBgkqhkiG9w0BAQEFAASCCS0wggkpAgEAAoICAQCX2LqF6IPaKc2z +SUq3teMrRDJcrIiDre5U7f1mktjmyPWehmdsWZGjkxgLgwwiuUPfAaepSj+Z16/Z +6DypivLQKbOEuxxUSFrss37vCZ1+1hIaRc6X0TTwoi/1TDZsdqngoQhG8nrb3xt2 +y6/WXiWNajdUOQ8JzNchpSL3j+jW427Sxs9GW4E/ySMUuHBXw+H/ubGyh+5Xm0Ng +05nxWjMDtocCcmpcEIVm5bqKAgfddpE5j0s57YUP6rG2mokensT5k2xdcflJcxDM +Tb9ft4ocDvO90bjPkYuCbPcYS+rdkElQZR7oMKevGoFdR4tOrK6aaiEYTnsu6rOs +f160WP/3U1plb27A3janEYlKR5lonkEOfM7dVjGq+8OcWz/ZMVsmJnTrm8g2kDQ+ +e4CQaPO1o0Cbqpc1Nexajxt2LxMOAjArGijkoiVazGfEoHza18Gi9Z34ktrEuVY0 +psqA1vrFz06Y0LKFiRFeGoiwGukvLqfbKmnJgSIIHM+urAnandjzzKPd2xzGKK7S +eAVKugTakvAKCLWtBjXH1U7hdCuurq8G8+GLnfT5x1d1VOefIGFo4xfhtZc46Mj3 +vEmZH9hs6+mArD2/+drza3Z+AqioizXbiJnbVA6J984h/DCDYAqPwRpYn38Stryx +kyyr+Dno9ZjBPUnzlwIy9s3AY5eO6wIDAQABAoICAAl8R6ICPpCcEXrUb5qn2Ujy +IJQdT+D0yJqGTD6QCDvsLDstGcWMcx1him5785IxJ9YpZBbnidO0euXJo8R3Ql8S +aGHK4ZtfFyeKAYNoJ0mSIx03LdiRAEOJ7QDf2LI017yJS9IkeW5ql0fTas0za2Jm +nCBZZrNNzvv7a/BEn8NHkfMQV5Fgy9ESvP3qcHhsNHly/aeD5ZEmbZcmWeb82orS +nxEnR0hUSnI0JpibsycT90DimAOh3LDXfd3E0Iz9VsA8urMpqtpDYzZ1tShed7TP +tyiaxS1+PUNIkEc9xRv2i2FQyhVmpmZdoCXfNi84nUtWhJYE6HkrmgRwnnLQp5rr +4stSQLo9X/5ML/bo2cj1eAiH+P6bAWaZhkMYPGkhBYwwgYqZF0Jn2Ay07BPF5gqq +rDY0mjF+Nm/NOqHfQjWHSj7KYYHm6qU1s5cAnDWLfznXGWZHpoI4UqAZtpAwGWEh +f/nS2LyepFW8RE6mbRDe+PkOM2455QdM8MuQc8drGInJ57A5e7y1LxOvFaUP+QtG +fqXok95XciMfttcR8nfYkUyC5QX+JmDZRf1U/DUxTJtbKXRKVG8WaFagkViiCniw +Q4RbrS4Utu6h+8HXdI9mK2pIHlJPp202b9gPry1j3kya8/jx5cTFWCcPj4IlgY51 +ZC1+vXJUezwq60gWJrRtAoIBAQDLsR9M+ZdBC5sl5fiVGsznAvfEVv/Cf7pmeXqt +q4BZT7kB8zfdfRRMzmZiQIX0zTbBn7Q7kWULcl8HgPbMEGZorc24CFuCq8k0n2ji +0aJsTJlfr7kUfTZ5c3HN1fGhDrnGoCL1eElt/Es4hZTIDnbn3i+RoCqwhCqhC0Lo +L7f6iHrVIcKyoaz+Cye9QKA89PVHMNM0f41zFJTVpSzGfWefG4vNoeU6nxqwkWVJ +Guhkt1EsJUScHueAa7hUYlELDy99tlqOPLdG7/GG6LmE6pM7R1E6DishnL1VgOll +KL6fouoz8nTve59eCDQECNzXN8lxPvx6TdufabGJTSZ3iNjPAoIBAQC+10DFlO3Y +XLHfX9LWK4rdkghx9AoeCFIVZO2G29H4zyDQn7hAMISWeVCqdAKhPQMJmAXyhQcc +PLEUdKVwifbPNO/IxsWL8SOfbI28XoZLYkQLual6JPa10jemFsinVRTPLU7RA1/T +7t/mWvyaAHmODWUPKSs1ltzh5dNwvnSNou6h72TytwlMIk5h8Z8ZtMIY2v5Ca2Mx +I7Wi8/5RebpszBbuLYy0GBqK5oqQov1YH0WK3ae4leaPWlZ49nyNYpx5Bvngm+d8 +4F+HvcxdpckRPfenl9nBW+0UHVXZORepQ+g0CNJxAQTomXy5VeEnrpiylrec4mE7 +YD15GKDxeHclAoIBAQC6efcr4zwDwnpRIc6HwrdfriniR7tx9rlRY1lacS3ZldDL +5jx4uy/AeRzmSifYy56beHfqKNuWtSgmmHxYqZL3tlYlaxFL+bUERgQh0sWd3Cp1 +nUwVY+RhH71KMdk6F18gZi5bB7pBxL2gizqARC27bH+kFDvAyd60/LhOFJGb/SJm +yLeUDctv8Ap7ihSoeaGrgRl6cGOwvWDcjpEFTn10/iTpJpPTLSGkEBfn64hh3Cml +/qBnRSuvN+7S4gdL1qUls7IHkbJAcR5u4dQlyRLtLAdC3QHDjdkP1CrX/4s0XlOH +JyrCrCxkS6JDJ1H2fg4fbQh7d5few0ZTapwSxuelAoIBAQCloou8MTuR6zoShsem +TRp0npen18ikMlKT8QpbXxTJ33Q6vjWbuMCLHXYOyyuxKBQn0oNu71ixgFetSmSA +3a2SP/7SAvtYL43cg8ossNncOciXaKgYtFZQjsjv5Zx351dtTYS30jbs6cn41AWj +4D+gNC3/T5sKIwtJIUE3/IWKnNyDPhUeuBBgvPHM/no5RHKmQgQmP7BS6QxwB1c2 +/fD5FtXcGTqi4LzAy8F+MYizYEjqcuFsSuZWi3C8AisuR2VJge2Eravn7ZHcbbYf +nZF63o7BLgG4WgHV0HQ/WdcYeTDjYLjNjbXl77mZ3ljSKS6s6jUo/C6p6yjPnrmp +ewXlAoIBAHX9aLtoj06zU5dtrMVjhYip6Z5LEtHsXCvOmnAl91uUXKwtMze6Y7xc +4ZggVWBR9lFyq9bAAuIXkRuE1jbvEwl5sLZY5Hf6GrIUR2uD7ZYFbogBTgmWCX9S +vYRpofEwquX0AUt+a4nO09wPBX1BJMViAEZSAWGfCfzVz8rXYIgcmu471wrpQHv4 +/whNA90JX0v6tuOASgR2dGorU//0ejGcm0bcYSuivKPxBfmVNeZNgIJXLM0ZHql1 +qwLnWXS0e0kADWhp/GAWVo0xzefgD3mDmyG2Qbz8MLaDkwPU/L073ZZNT1AHFZZE +xfVijyydtPqa/JDgqcUFBxLtiKGTbbI= +-----END PRIVATE KEY----- diff --git a/tests/integration/test_accept_invalid_certificate/certs/client-req.pem b/tests/integration/test_accept_invalid_certificate/certs/client-req.pem new file mode 100644 index 00000000000..507db7bae6c --- /dev/null +++ b/tests/integration/test_accept_invalid_certificate/certs/client-req.pem @@ -0,0 +1,27 @@ +-----BEGIN CERTIFICATE REQUEST----- +MIIEmzCCAoMCAQAwVjELMAkGA1UEBhMCUlUxEzARBgNVBAgMClNvbWUtU3RhdGUx +ITAfBgNVBAoMGEludGVybmV0IFdpZGdpdHMgUHR5IEx0ZDEPMA0GA1UEAwwGY2xp +ZW50MIICIjANBgkqhkiG9w0BAQEFAAOCAg8AMIICCgKCAgEAl9i6heiD2inNs0lK +t7XjK0QyXKyIg63uVO39ZpLY5sj1noZnbFmRo5MYC4MMIrlD3wGnqUo/mdev2eg8 +qYry0CmzhLscVEha7LN+7wmdftYSGkXOl9E08KIv9Uw2bHap4KEIRvJ6298bdsuv +1l4ljWo3VDkPCczXIaUi94/o1uNu0sbPRluBP8kjFLhwV8Ph/7mxsofuV5tDYNOZ +8VozA7aHAnJqXBCFZuW6igIH3XaROY9LOe2FD+qxtpqJHp7E+ZNsXXH5SXMQzE2/ +X7eKHA7zvdG4z5GLgmz3GEvq3ZBJUGUe6DCnrxqBXUeLTqyummohGE57LuqzrH9e +tFj/91NaZW9uwN42pxGJSkeZaJ5BDnzO3VYxqvvDnFs/2TFbJiZ065vINpA0PnuA +kGjztaNAm6qXNTXsWo8bdi8TDgIwKxoo5KIlWsxnxKB82tfBovWd+JLaxLlWNKbK +gNb6xc9OmNCyhYkRXhqIsBrpLy6n2yppyYEiCBzPrqwJ2p3Y88yj3dscxiiu0ngF +SroE2pLwCgi1rQY1x9VO4XQrrq6vBvPhi530+cdXdVTnnyBhaOMX4bWXOOjI97xJ +mR/YbOvpgKw9v/na82t2fgKoqIs124iZ21QOiffOIfwwg2AKj8EaWJ9/Era8sZMs +q/g56PWYwT1J85cCMvbNwGOXjusCAwEAAaAAMA0GCSqGSIb3DQEBCwUAA4ICAQBr +ZWJGghyeCUESKu9v9dgQYD4Bq5YviYY7BAjdQsn5/Ti+tqkAKHEn/L+264QJMrTU +muTKg09o7n7r+g4sHmZhT2QEtwGFwaEGvz2zRd4po29WlGQAN5a8yzXIb8b9n+C4 +Z5APeAzjBV9ooqbJCURlgeDP247XTcU8yIvvX0YpLmXYRQ5Na8TTfdBdlD3Y23mD +aPe+2YGkK1Go4iNMoyTEoJ0cj+89WMA9o+BiGcptALYMzKBM26f/vz9hb0J49RDV +MkccL8TKNDJQrLFCIcwpLRw08uZZdqJ/cy01hrtkMkp+SSBQO9qSmalDrTLgs4Pp +3ml88OIsQHMkdBUjUAw5FyAJoik4iXab/sLoMmMJ/lqH0in3dT4KeEh8Xe2iAfNU +97/6lAuZOrn2EPvqBJI/gfjwPj9wTgjlvLpmcjhBvfqkuM/dd7zTmwRoM1klRMC2 +07x2YNoG7cGXjAYk1h5HDYVJqdphgQLTS1nRqmiQ8QnPryqlZcI7T/GgpOOulHce +lgMp6FFlGaQMysCbdYJQk4JMQXjwlj5ms/6rZSkgwmrv6WhCY7ImDj2E/7oE5XJd +BqXHFwOQdCSgQOVH+QgmxceYolI73cMNF3Frq8/IztAOtMyBF0bpRpI1U6a3ALf4 +cK2Dkd/TLg8ZwsVBjPaRrtXlntO6tB+tn3suukCKcg== +-----END CERTIFICATE REQUEST----- diff --git a/tests/integration/test_accept_invalid_certificate/certs/generate_certs.sh b/tests/integration/test_accept_invalid_certificate/certs/generate_certs.sh new file mode 100755 index 00000000000..610a1281f8b --- /dev/null +++ b/tests/integration/test_accept_invalid_certificate/certs/generate_certs.sh @@ -0,0 +1,13 @@ +#!/bin/bash + +# 1. Generate CA's private key and self-signed certificate +openssl req -newkey rsa:4096 -x509 -days 3650 -nodes -batch -keyout ca-key.pem -out ca-cert.pem -subj "/C=RU/ST=Some-State/O=Internet Widgits Pty Ltd/CN=ca" + +# 2. Generate self-signed certificate and private key for using as wrong server certificate (because it's not signed by CA) +openssl req -newkey rsa:4096 -x509 -days 3650 -nodes -batch -keyout self-key.pem -out self-cert.pem -subj "/C=RU/ST=Some-State/O=Internet Widgits Pty Ltd/CN=server" + +# 3. Generate client's private key and certificate signing request (CSR) +openssl req -newkey rsa:4096 -nodes -batch -keyout client-key.pem -out client-req.pem -subj "/C=RU/ST=Some-State/O=Internet Widgits Pty Ltd/CN=client" + +# 4. Use CA's private key to sign client's CSR and get back the signed certificate +openssl x509 -req -days 3650 -in client-req.pem -CA ca-cert.pem -CAkey ca-key.pem -CAcreateserial -out client-cert.pem diff --git a/tests/integration/test_accept_invalid_certificate/certs/self-cert.pem b/tests/integration/test_accept_invalid_certificate/certs/self-cert.pem new file mode 100644 index 00000000000..7b3832a3b6b --- /dev/null +++ b/tests/integration/test_accept_invalid_certificate/certs/self-cert.pem @@ -0,0 +1,32 @@ +-----BEGIN CERTIFICATE----- +MIIFjTCCA3WgAwIBAgIUFpPVJytX13EWXwaLqg/rOTdu4UgwDQYJKoZIhvcNAQEL +BQAwVjELMAkGA1UEBhMCUlUxEzARBgNVBAgMClNvbWUtU3RhdGUxITAfBgNVBAoM +GEludGVybmV0IFdpZGdpdHMgUHR5IEx0ZDEPMA0GA1UEAwwGc2VydmVyMB4XDTI0 +MDcwMTA1MTEyNVoXDTM0MDYyOTA1MTEyNVowVjELMAkGA1UEBhMCUlUxEzARBgNV +BAgMClNvbWUtU3RhdGUxITAfBgNVBAoMGEludGVybmV0IFdpZGdpdHMgUHR5IEx0 +ZDEPMA0GA1UEAwwGc2VydmVyMIICIjANBgkqhkiG9w0BAQEFAAOCAg8AMIICCgKC +AgEAsQO7f1Ar0ThZjh00KeuckyJDf/oPQ+67EX2Y82W4/dl8KmHksmIKp2+CJFwg +Y7/c1ti+DjxVXr4tZynzUeDYqujTrAidUMz0zrvjXNeeyoe2hgc2X6YFtVhyEtMQ +6q8VMaqomjuQDIyxUUdt2xHZWyxHPntDrzsmCCDH5Md27CZ4Z5jsSOdgnlrCkjd9 +NtuIzxTlrExe3whIlTL0DURTvAVV/tPoVpVSvHTWCdziaCWm1hG5T6ZCN2W0pBgV +k+UNNtXr1ksr/Ma0Z84ShKjpN+soFOIZ3GDDGY8Xr/74jZo2c4025TpeIPcPBE0Z +b2K24l0UNnQbgm3ryI7NRrZ7psNnhvnWZru4w5Ykr8RKBf3n1AZOt8+dXHNvJQeU +/CUA6pKpc+sIxzAtWtS2ABqdOTJTXj2GyYNsz38dI2uzecUzT7oEtHWq9sr376EO +xMv1wXsUBli8INhPricfojH2/ACnRPysFglAKTx0IpcwqYoV5pGkk0iK1DNkQMv4 +Kf7cD2hLWv9+Fgbrjfu/gNCL0bSJBjaTHngACePsLEQkmGomscDySYXa+PHeB8Ln +QCWsE6MkeDa5g3gUOPV/vC6qv0ZBvQj1p3sOeyIQATDBYX+KheCO+N0ZCCdlT5Sx +thKNDusultu6/lyjGum0c/5oGgoDKTIkibgljekKupBRAykCAwEAAaNTMFEwHQYD +VR0OBBYEFEnPwBjLDPzta3nBrUEH+Gz1nBJRMB8GA1UdIwQYMBaAFEnPwBjLDPzt +a3nBrUEH+Gz1nBJRMA8GA1UdEwEB/wQFMAMBAf8wDQYJKoZIhvcNAQELBQADggIB +AIW829ODEBvBp+vUwt3hbsJdBx9DzwcBBJ+Mew95YVveiosU+3p1HaWCkc4srgOi +hDNbpvfbqBIKVAZzesJbuTEeQ+Aj4M1lcFed9tvidVnmqDgHJApOI61GihT9MDjC +dH6vhcsorE8E5F58FcMJ0QDynVOuCA/gHkS3ucMxBsNKGfYNe0rOGMw23CzHdnz7 +UonOKhrhIQawI3ELLQxo2LWZEmMsluznE4MB+RGdn3k/UzOY9x4bauv4hrHjYvEm +x/zp2iWhZXvvTZ9u9xaHXEzbSU3WFCcXE/l9SDPM7ip9fabUrvaTJkmM190bD9sk +VcEFOAFBVxfx2fvmBIYqvSkVbcWdAT6x8k+MVNfq99/1aIdh+DTB3XUB7FkE/fnU +tXr/1sz0guwfNI/cHvhHisjmbgMhZWeeVV1YZHOJCsK7eq9rGq5Uul8nYrTdYKlN +u0OcrBXYBB6ny7ljCTahyPPmFZh/4Mgr0xROn2HyhTCpDHjnD1M5jT783fvWqu06 +cpOjRIPYNtA/UQAWUZWrTuJTD6Kju9tDI8i1NtMWIXd7w0LDXsXeHpRoDrVvOIPT +P0ivUpt4dFHnwR8Dos4NI/j9mkdGCblUxDsd/7or8gjedOoWjWfGCNlYa72aq0gf +KjKGPISC3BGTNtwRTe+E7k38r+Y6YO/Tnu9RvBdDM3vF +-----END CERTIFICATE----- diff --git a/tests/integration/test_accept_invalid_certificate/certs/self-key.pem b/tests/integration/test_accept_invalid_certificate/certs/self-key.pem new file mode 100644 index 00000000000..10b299a8c0c --- /dev/null +++ b/tests/integration/test_accept_invalid_certificate/certs/self-key.pem @@ -0,0 +1,52 @@ +-----BEGIN PRIVATE KEY----- +MIIJQwIBADANBgkqhkiG9w0BAQEFAASCCS0wggkpAgEAAoICAQCxA7t/UCvROFmO +HTQp65yTIkN/+g9D7rsRfZjzZbj92XwqYeSyYgqnb4IkXCBjv9zW2L4OPFVevi1n +KfNR4Niq6NOsCJ1QzPTOu+Nc157Kh7aGBzZfpgW1WHIS0xDqrxUxqqiaO5AMjLFR +R23bEdlbLEc+e0OvOyYIIMfkx3bsJnhnmOxI52CeWsKSN30224jPFOWsTF7fCEiV +MvQNRFO8BVX+0+hWlVK8dNYJ3OJoJabWEblPpkI3ZbSkGBWT5Q021evWSyv8xrRn +zhKEqOk36ygU4hncYMMZjxev/viNmjZzjTblOl4g9w8ETRlvYrbiXRQ2dBuCbevI +js1Gtnumw2eG+dZmu7jDliSvxEoF/efUBk63z51cc28lB5T8JQDqkqlz6wjHMC1a +1LYAGp05MlNePYbJg2zPfx0ja7N5xTNPugS0dar2yvfvoQ7Ey/XBexQGWLwg2E+u +Jx+iMfb8AKdE/KwWCUApPHQilzCpihXmkaSTSIrUM2RAy/gp/twPaEta/34WBuuN ++7+A0IvRtIkGNpMeeAAJ4+wsRCSYaiaxwPJJhdr48d4HwudAJawToyR4NrmDeBQ4 +9X+8Lqq/RkG9CPWnew57IhABMMFhf4qF4I743RkIJ2VPlLG2Eo0O6y6W27r+XKMa +6bRz/mgaCgMpMiSJuCWN6Qq6kFEDKQIDAQABAoICAA4BaZhifipeTFeo395hPKY3 +k8z7EAd5SrvQ4C/XNefVNAnLwmM1dqUAzEnsMB6JqQN8FVgzAIjagI/PtedzOYbB +nTMuC08VUwP4bGjay+mNC1gCF2nsojmoTH5IzOcvVdH0MBrj6OXVS+z5EIFrdsh2 +nGcegaB1DEagXAXcnQAvmwniA5qmaUfzjmmF58EAuwuyGj9OW5icoyw1jTHE3OCH +hjeqCeav3I1PVpAOi1YKL0UeFIzExCsglPEBfCjoz+mCy4IiYydA4Y1FChoM80bw +CDUpc+UH6D02y9JVA317XMuPuhmGS2RNp+vYiHv+Uide40vbq+w0nkSOvUHXC7lo +UXf26sZWHcQP8TxewjZFcajApk03lCAb/u4skt7OQxbDOcvrcdbCsSpZPS16s3CA +OWA9fS7WJDmFx/dhzm4xNaXpz9z5MMkUfnFqgceG3H1leamL66J+eOzlquQ8LdsJ +/wTvCi1SsR3JmqT+TpqlXTJci7A+I/b0Yt6a0s2HunQ5LA1Xomffhn/00IxUeqyi +4MKh7wnoCaRlwZVORgQfKnosSkbkixDabwr9BMnMryE+NdQI3JKCvchdB0idyk1z +igTgZYpaxZg9hwTlOITGYSsprN7tiwjNQG+Kivatn1NxnxhD8lpDoUqluR7mZs1e +CwOLrB+irCULr7tcRYtxAoIBAQDFV7E1Oz8j8NkvkLztIFJ0ej7zEyNp3HUJb4bd +SRvF2AL6sqSw0xVLK8HbrlfPo1Nex8nZUfY3v2DmPS4+S3mrtk+8XoRkvNn0VdL4 +FqLfygYsURI8xVp4zTsAmR+7grPJbb/u3XG7CBY3T4XGHSbMmYAu7+W3A2t8Q+0U +fptju7Gc9gEACcSfap45CzEnFbabxNq3FJJ9V92N/xVQcOzNLxPxgn6tVhPeQnXV +0WHJQdxB/8DivgOObiTCB7on9tNbSHisTYtGwn6Kz0961ewYU7Xj4epj7A35nWhI +E10FDAf3T7cQa9ubSFZwOiSe0voAbiWbN3/M+zgkZDp2+wlnAoIBAQDloTtzgQaA +//VPNhB2d7iOgAl5cUSWqY0HLJobw1ETsdbkGRTgZrFXe4XlXwHB4S+zVEMmyR3p +HY0sYoAdpmd3Ggo4efJEx9IxkD6UZ71we02szvt4MdV5mwjHOAQfEJffktVkeqEJ +gdekSmaHTeryBf3ESpefq+OH1qaj/IjivySLI4c8fPcxNg0nMmDsdMzRaDOHDDa2 +nUpRBeMFkwiiPJpALPPp/4be0d9mrahPd4zBoBieZwaP6Zhwksi0NaM4WKEfx4Fd +TzH0K7UfLDMJ13wuTsNPzjPlDA+R4E/O3qVZxtHzuqc7fCksE7inYZghIRSjQHmv +mmLPaFdaIWTvAoIBAQC50BQfVyXgpYsaamI5tFFvMxvKrJ6PsvDQbZ7DBW+NHhPQ +Eh1YEajfGthyDPrJm7OIbv30DmSToTOQFtNP/KkXsZXqe+SVJezjT1xSFJ5PsV2W +pM9TQVAqprODoKpsWGkIpLowI3pO7RJeChafsBs0UuSLTX8yj5nJa/vzbDLL+ZMH +g+r1Kj1lDXLlq1AgySUkDnsSH+C+/Ni0NxbOo0WU/Vr98C1u8LnMcS+aoIg+hDDs +Q/BOjzOyDKu3tqblslbYgKh/NJSLj96uvPzNYFfpoAiIBdQ5Oa7+9eOW2gSgF+/4 +QarRX31r/PTGvzh0A5s6hHwMZnmDfJgkS+tw1HbJAoIBAQDfauQBGYROdrsieqPB +3/H9E64kLoRhYk4IsOiwInT8CIfgHlcljkT1rQU1ih6XmJVkkZNUZVFowovcp2mn +b4QzaW6QTHXfXduwxXtdwSGcExwcCra41yOw0ZrU2jeIUgiySpa80GNmcKJzAsCG +3NS+7f1dwoWLoR4byEkSPF6i3v6bMdIelCeeBcz+WhHbfjjZddzMhKGnQ5pPAr9D +aS95SiV6U0JuZWO/38rQiB2+U77rlOzfEbZcxxa+UNJeI+HOrMz3spC3TJp5yxzK +W6K87J+yHogBWcs8C2QKtohv6khO9RbJYM3dHoygif+bO2Vctmbx/4VlyXxh+E8l +LmCZAoIBAFR+dd7cW0pDkOsPsGZbWwDPNRwo2lxmqf2B5w9zoGNFoc4qV5k9tJfu +kPcwol6h2P4s2olYTgYC6hjPmE17U1TyP2DFeOw5IdxzNqy7dZndmaTs4qFLnDv3 +HIbkX42p7v++EU9eKAhYKJzJpuianMZ+v5azV1wNS5TDuN2TFULyIi+1UEw0ONuJ +rA97idb2PrO2sT/eBj02uiOcPhZH6CHSMxDp7QG+S1AVZBgutTQQdeAsb6pE/iW0 +u3IzGNxTYjTmF+u7iNdOPiaTFpdnltDviADESMJGH7PIk7r7PXMI9ID+LbBFGJ7Q +duW6PGuZ9JMBbxbNNvYayVgPeAeUsUI= +-----END PRIVATE KEY----- diff --git a/tests/integration/test_accept_invalid_certificate/configs/ssl_config.xml b/tests/integration/test_accept_invalid_certificate/configs/ssl_config.xml new file mode 100644 index 00000000000..a1dd1a24693 --- /dev/null +++ b/tests/integration/test_accept_invalid_certificate/configs/ssl_config.xml @@ -0,0 +1,17 @@ + + + 9440 + + + + + /etc/clickhouse-server/config.d/self-cert.pem + /etc/clickhouse-server/config.d/self-key.pem + /etc/clickhouse-server/config.d/ca-cert.pem + none + + + diff --git a/tests/integration/test_accept_invalid_certificate/test.py b/tests/integration/test_accept_invalid_certificate/test.py new file mode 100644 index 00000000000..87229d75f90 --- /dev/null +++ b/tests/integration/test_accept_invalid_certificate/test.py @@ -0,0 +1,92 @@ +import pytest +from helpers.client import Client +from helpers.cluster import ClickHouseCluster +import os.path +from os import remove + + +SCRIPT_DIR = os.path.dirname(os.path.realpath(__file__)) +MAX_RETRY = 5 + +cluster = ClickHouseCluster(__file__) +instance = cluster.add_instance( + "node", + main_configs=[ + "configs/ssl_config.xml", + "certs/self-key.pem", + "certs/self-cert.pem", + "certs/ca-cert.pem", + ], +) + + +@pytest.fixture(scope="module", autouse=True) +def started_cluster(): + try: + cluster.start() + yield cluster + + finally: + cluster.shutdown() + + +config_default = """ +""" + +config_accept = """ + 1 +""" + +config_connection_accept = """ + + + {ip_address} + 1 + + +""" + + +def execute_query_native(node, query, config): + config_path = f"{SCRIPT_DIR}/configs/client.xml" + + file = open(config_path, "w") + file.write(config) + file.close() + + client = Client( + node.ip_address, + 9440, + command=cluster.client_bin_path, + secure=True, + config=config_path, + ) + + try: + result = client.query(query) + remove(config_path) + return result + except: + remove(config_path) + raise + + +def test_default(): + with pytest.raises(Exception) as err: + execute_query_native(instance, "SELECT 1", config_default) + assert "certificate verify failed" in str(err.value) + + +def test_accept(): + assert execute_query_native(instance, "SELECT 1", config_accept) == "1\n" + + +def test_connection_accept(): + assert ( + execute_query_native( + instance, + "SELECT 1", + config_connection_accept.format(ip_address=f"{instance.ip_address}"), + ) + == "1\n" + ) diff --git a/tests/integration/test_force_deduplication/test.py b/tests/integration/test_force_deduplication/test.py index 87b2c45bbc5..14c11bc8500 100644 --- a/tests/integration/test_force_deduplication/test.py +++ b/tests/integration/test_force_deduplication/test.py @@ -29,6 +29,8 @@ def get_counts(): def test_basic(start_cluster): + old_src, old_a, old_b, old_c = 0, 0, 0, 0 + node.query( """ CREATE TABLE test (A Int64) ENGINE = ReplicatedMergeTree ('/clickhouse/test/tables/test','1') ORDER BY tuple(); @@ -39,6 +41,15 @@ def test_basic(start_cluster): INSERT INTO test values(999); """ ) + + src, a, b, c = get_counts() + assert src == old_src + 1 + assert a == old_a + 2 + assert b == old_b + 2 + assert c == old_c + 2 + old_src, old_a, old_b, old_c = src, a, b, c + + # that issert fails on test_mv_b due to partitions by A with pytest.raises(QueryRuntimeException): node.query( """ @@ -46,22 +57,23 @@ def test_basic(start_cluster): INSERT INTO test SELECT number FROM numbers(10); """ ) + src, a, b, c = get_counts() + assert src == old_src + 10 + assert a == old_a + 10 + assert b == old_b + assert c == old_c + 10 + old_src, old_a, old_b, old_c = src, a, b, c - old_src, old_a, old_b, old_c = get_counts() - # number of rows in test_mv_a and test_mv_c depends on order of inserts into views - assert old_src == 11 - assert old_a in (1, 11) - assert old_b == 1 - assert old_c in (1, 11) - + # deduplication only for src table node.query("INSERT INTO test SELECT number FROM numbers(10)") src, a, b, c = get_counts() - # no changes because of deduplication in source table assert src == old_src - assert a == old_a - assert b == old_b - assert c == old_c + assert a == old_a + 10 + assert b == old_b + 10 + assert c == old_c + 10 + old_src, old_a, old_b, old_c = src, a, b, c + # deduplication for MV tables does not work, because previous inserts have not written their deduplications tokens to the log due to `deduplicate_blocks_in_dependent_materialized_views = 0`. node.query( """ SET deduplicate_blocks_in_dependent_materialized_views = 1; @@ -69,11 +81,27 @@ def test_basic(start_cluster): """ ) src, a, b, c = get_counts() - assert src == 11 - assert a == old_a + 10 # first insert could be succesfull with disabled dedup - assert b == 11 + assert src == old_src + assert a == old_a + 10 + assert b == old_b + 10 assert c == old_c + 10 + old_src, old_a, old_b, old_c = src, a, b, c + # deduplication for all the tables + node.query( + """ + SET deduplicate_blocks_in_dependent_materialized_views = 1; + INSERT INTO test SELECT number FROM numbers(10); + """ + ) + src, a, b, c = get_counts() + assert src == old_src + assert a == old_a + assert b == old_b + assert c == old_c + old_src, old_a, old_b, old_c = src, a, b, c + + # that issert fails on test_mv_b due to partitions by A, it is an uniq data which is not deduplicated with pytest.raises(QueryRuntimeException): node.query( """ @@ -82,16 +110,23 @@ def test_basic(start_cluster): INSERT INTO test SELECT number FROM numbers(100,10); """ ) + src, a, b, c = get_counts() + assert src == old_src + 10 + assert a == old_a + 10 + assert b == old_b + assert c == old_c + 10 + old_src, old_a, old_b, old_c = src, a, b, c + # deduplication for all tables, except test_mv_b. For test_mv_b it is an uniq data which is not deduplicated due to exception at previous insert node.query( """ SET deduplicate_blocks_in_dependent_materialized_views = 1; INSERT INTO test SELECT number FROM numbers(100,10); """ ) - src, a, b, c = get_counts() - assert src == 21 - assert a == old_a + 20 - assert b == 21 - assert c == old_c + 20 + assert src == old_src + assert a == old_a + assert b == old_b + 10 + assert c == old_c + old_src, old_a, old_b, old_c = src, a, b, c diff --git a/tests/integration/test_storage_postgresql/test.py b/tests/integration/test_storage_postgresql/test.py index ffcff36c47e..12823f1f72d 100644 --- a/tests/integration/test_storage_postgresql/test.py +++ b/tests/integration/test_storage_postgresql/test.py @@ -871,7 +871,7 @@ def test_filter_pushdown(started_cluster): ) assert result1 == result2 - for kind in ["INNER", "LEFT", "RIGHT", "FULL"]: + for kind in ["INNER", "LEFT", "RIGHT", "FULL", "ANY LEFT", "SEMI RIGHT"]: for value in [0, 10]: compare_results( "SELECT * FROM ch_table {kind} JOIN {pg_table} as p ON ch_table.pg_id = p.id WHERE value = {value} ORDER BY ALL", diff --git a/tests/queries/0_stateless/00510_materizlized_view_and_deduplication_zookeeper.reference b/tests/queries/0_stateless/00510_materizlized_view_and_deduplication_zookeeper.reference index adf6abb7298..9c9281dc7e4 100644 --- a/tests/queries/0_stateless/00510_materizlized_view_and_deduplication_zookeeper.reference +++ b/tests/queries/0_stateless/00510_materizlized_view_and_deduplication_zookeeper.reference @@ -1,7 +1,7 @@ 2 3 -2 +3 3 1 diff --git a/tests/queries/0_stateless/00510_materizlized_view_and_deduplication_zookeeper.sql b/tests/queries/0_stateless/00510_materizlized_view_and_deduplication_zookeeper.sql index d3c4da86b41..51e6a513608 100644 --- a/tests/queries/0_stateless/00510_materizlized_view_and_deduplication_zookeeper.sql +++ b/tests/queries/0_stateless/00510_materizlized_view_and_deduplication_zookeeper.sql @@ -29,7 +29,7 @@ INSERT INTO without_deduplication VALUES (43); SELECT count() FROM with_deduplication; SELECT count() FROM without_deduplication; --- Implicit insert isn't deduplicated +-- Implicit insert isn't deduplicated, because deduplicate_blocks_in_dependent_materialized_views = 0 by default SELECT ''; SELECT countMerge(cnt) FROM with_deduplication_mv; SELECT countMerge(cnt) FROM without_deduplication_mv; diff --git a/tests/queries/0_stateless/00633_materialized_view_and_too_many_parts_zookeeper.sh b/tests/queries/0_stateless/00633_materialized_view_and_too_many_parts_zookeeper.sh index 1fb219108da..8f7d19028b0 100755 --- a/tests/queries/0_stateless/00633_materialized_view_and_too_many_parts_zookeeper.sh +++ b/tests/queries/0_stateless/00633_materialized_view_and_too_many_parts_zookeeper.sh @@ -36,8 +36,8 @@ ${CLICKHOUSE_CLIENT} --query "DROP TABLE c" echo ${CLICKHOUSE_CLIENT} --query "CREATE TABLE root (d UInt64) ENGINE = Null" ${CLICKHOUSE_CLIENT} --query "CREATE MATERIALIZED VIEW d (d UInt64) ENGINE = ReplicatedMergeTree('/clickhouse/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/d', '1') ORDER BY d AS SELECT * FROM root" -${CLICKHOUSE_CLIENT} --query "INSERT INTO root VALUES (1)"; -${CLICKHOUSE_CLIENT} --query "INSERT INTO root VALUES (1)"; +${CLICKHOUSE_CLIENT} --query "INSERT INTO root SETTINGS deduplicate_blocks_in_dependent_materialized_views=1 VALUES (1)"; +${CLICKHOUSE_CLIENT} --query "INSERT INTO root SETTINGS deduplicate_blocks_in_dependent_materialized_views=1 VALUES (1)"; ${CLICKHOUSE_CLIENT} --query "SELECT * FROM d"; ${CLICKHOUSE_CLIENT} --query "DROP TABLE root" ${CLICKHOUSE_CLIENT} --query "DROP TABLE d" diff --git a/tests/queries/0_stateless/01275_parallel_mv.reference b/tests/queries/0_stateless/01275_parallel_mv.reference index f5f31c4a563..cf1f7054f6c 100644 --- a/tests/queries/0_stateless/01275_parallel_mv.reference +++ b/tests/queries/0_stateless/01275_parallel_mv.reference @@ -2,13 +2,18 @@ -select 'optimize_trivial_insert_select=0', 'max_insert_threads=0'; -optimize_trivial_insert_select=0 max_insert_threads=0 -insert into testX select number from numbers(10) settings + + + +select 'optimize_trivial_insert_select=0', 'max_insert_threads=0', 'iteration_num='; +optimize_trivial_insert_select=0 max_insert_threads=0 iteration_num= +insert into testX select number from numbers(200) settings + send_logs_level='fatal', + insert_deduplication_token='UT_1', log_queries=1, parallel_view_processing=0, optimize_trivial_insert_select=0, - max_insert_threads=0; -- { serverError FUNCTION_THROW_IF_VALUE_IS_NON_ZERO } + max_insert_threads=0; system flush logs; select peak_threads_usage from system.query_log where current_database = currentDatabase() and @@ -19,20 +24,20 @@ select peak_threads_usage from system.query_log where Settings['max_insert_threads'] = '0'; 2 select count() from testX; -10 +200 select count() from testXA; -10 -select count() from testXB; -0 +200 select count() from testXC; -10 -select 'optimize_trivial_insert_select=0', 'max_insert_threads=16'; -optimize_trivial_insert_select=0 max_insert_threads=16 -insert into testX select number from numbers(10) settings +200 +select 'optimize_trivial_insert_select=0', 'max_insert_threads=5', 'iteration_num='; +optimize_trivial_insert_select=0 max_insert_threads=5 iteration_num= +insert into testX select number from numbers(200) settings + send_logs_level='fatal', + insert_deduplication_token='UT_2', log_queries=1, parallel_view_processing=0, optimize_trivial_insert_select=0, - max_insert_threads=16; -- { serverError FUNCTION_THROW_IF_VALUE_IS_NON_ZERO } + max_insert_threads=5; system flush logs; select peak_threads_usage from system.query_log where current_database = currentDatabase() and @@ -40,23 +45,23 @@ select peak_threads_usage from system.query_log where query like '%insert into testX %' and Settings['parallel_view_processing'] = '0' and Settings['optimize_trivial_insert_select'] = '0' and - Settings['max_insert_threads'] = '16'; + Settings['max_insert_threads'] = '5'; 2 select count() from testX; -20 +400 select count() from testXA; -20 -select count() from testXB; -0 +400 select count() from testXC; -20 -select 'optimize_trivial_insert_select=1', 'max_insert_threads=0'; -optimize_trivial_insert_select=1 max_insert_threads=0 -insert into testX select number from numbers(10) settings +400 +select 'optimize_trivial_insert_select=1', 'max_insert_threads=0', 'iteration_num='; +optimize_trivial_insert_select=1 max_insert_threads=0 iteration_num= +insert into testX select number from numbers(200) settings + send_logs_level='fatal', + insert_deduplication_token='UT_3', log_queries=1, parallel_view_processing=0, optimize_trivial_insert_select=1, - max_insert_threads=0; -- { serverError FUNCTION_THROW_IF_VALUE_IS_NON_ZERO } + max_insert_threads=0; system flush logs; select peak_threads_usage from system.query_log where current_database = currentDatabase() and @@ -67,20 +72,20 @@ select peak_threads_usage from system.query_log where Settings['max_insert_threads'] = '0'; 2 select count() from testX; -30 +600 select count() from testXA; -30 -select count() from testXB; -0 +600 select count() from testXC; -30 -select 'optimize_trivial_insert_select=1', 'max_insert_threads=16'; -optimize_trivial_insert_select=1 max_insert_threads=16 -insert into testX select number from numbers(10) settings +600 +select 'optimize_trivial_insert_select=1', 'max_insert_threads=5', 'iteration_num='; +optimize_trivial_insert_select=1 max_insert_threads=5 iteration_num= +insert into testX select number from numbers(200) settings + send_logs_level='fatal', + insert_deduplication_token='UT_4', log_queries=1, parallel_view_processing=0, optimize_trivial_insert_select=1, - max_insert_threads=16; -- { serverError FUNCTION_THROW_IF_VALUE_IS_NON_ZERO } + max_insert_threads=5; system flush logs; select peak_threads_usage from system.query_log where current_database = currentDatabase() and @@ -88,23 +93,23 @@ select peak_threads_usage from system.query_log where query like '%insert into testX %' and Settings['parallel_view_processing'] = '0' and Settings['optimize_trivial_insert_select'] = '1' and - Settings['max_insert_threads'] = '16'; + Settings['max_insert_threads'] = '5'; 2 select count() from testX; -40 +800 select count() from testXA; -40 -select count() from testXB; -0 +800 select count() from testXC; -40 -select 'optimize_trivial_insert_select=0', 'max_insert_threads=0'; -optimize_trivial_insert_select=0 max_insert_threads=0 -insert into testX select number from numbers(10) settings +800 +select 'optimize_trivial_insert_select=0', 'max_insert_threads=0', 'iteration_num='; +optimize_trivial_insert_select=0 max_insert_threads=0 iteration_num= +insert into testX select number from numbers(200) settings + send_logs_level='fatal', + insert_deduplication_token='UT_5', log_queries=1, parallel_view_processing=1, optimize_trivial_insert_select=0, - max_insert_threads=0; -- { serverError FUNCTION_THROW_IF_VALUE_IS_NON_ZERO } + max_insert_threads=0; system flush logs; select peak_threads_usage from system.query_log where current_database = currentDatabase() and @@ -115,20 +120,20 @@ select peak_threads_usage from system.query_log where Settings['max_insert_threads'] = '0'; 5 select count() from testX; -50 +1000 select count() from testXA; -50 -select count() from testXB; -0 +1000 select count() from testXC; -50 -select 'optimize_trivial_insert_select=0', 'max_insert_threads=16'; -optimize_trivial_insert_select=0 max_insert_threads=16 -insert into testX select number from numbers(10) settings +1000 +select 'optimize_trivial_insert_select=0', 'max_insert_threads=5', 'iteration_num='; +optimize_trivial_insert_select=0 max_insert_threads=5 iteration_num= +insert into testX select number from numbers(200) settings + send_logs_level='fatal', + insert_deduplication_token='UT_6', log_queries=1, parallel_view_processing=1, optimize_trivial_insert_select=0, - max_insert_threads=16; -- { serverError FUNCTION_THROW_IF_VALUE_IS_NON_ZERO } + max_insert_threads=5; system flush logs; select peak_threads_usage from system.query_log where current_database = currentDatabase() and @@ -136,23 +141,23 @@ select peak_threads_usage from system.query_log where query like '%insert into testX %' and Settings['parallel_view_processing'] = '1' and Settings['optimize_trivial_insert_select'] = '0' and - Settings['max_insert_threads'] = '16'; -5 + Settings['max_insert_threads'] = '5'; +12 select count() from testX; -60 +1200 select count() from testXA; -60 -select count() from testXB; -0 +1200 select count() from testXC; -60 -select 'optimize_trivial_insert_select=1', 'max_insert_threads=0'; -optimize_trivial_insert_select=1 max_insert_threads=0 -insert into testX select number from numbers(10) settings +1200 +select 'optimize_trivial_insert_select=1', 'max_insert_threads=0', 'iteration_num='; +optimize_trivial_insert_select=1 max_insert_threads=0 iteration_num= +insert into testX select number from numbers(200) settings + send_logs_level='fatal', + insert_deduplication_token='UT_7', log_queries=1, parallel_view_processing=1, optimize_trivial_insert_select=1, - max_insert_threads=0; -- { serverError FUNCTION_THROW_IF_VALUE_IS_NON_ZERO } + max_insert_threads=0; system flush logs; select peak_threads_usage from system.query_log where current_database = currentDatabase() and @@ -161,22 +166,22 @@ select peak_threads_usage from system.query_log where Settings['parallel_view_processing'] = '1' and Settings['optimize_trivial_insert_select'] = '1' and Settings['max_insert_threads'] = '0'; -5 +2 select count() from testX; -70 +1400 select count() from testXA; -70 -select count() from testXB; -0 +1400 select count() from testXC; -70 -select 'optimize_trivial_insert_select=1', 'max_insert_threads=16'; -optimize_trivial_insert_select=1 max_insert_threads=16 -insert into testX select number from numbers(10) settings +1400 +select 'optimize_trivial_insert_select=1', 'max_insert_threads=5', 'iteration_num='; +optimize_trivial_insert_select=1 max_insert_threads=5 iteration_num= +insert into testX select number from numbers(200) settings + send_logs_level='fatal', + insert_deduplication_token='UT_8', log_queries=1, parallel_view_processing=1, optimize_trivial_insert_select=1, - max_insert_threads=16; -- { serverError FUNCTION_THROW_IF_VALUE_IS_NON_ZERO } + max_insert_threads=5; system flush logs; select peak_threads_usage from system.query_log where current_database = currentDatabase() and @@ -184,13 +189,11 @@ select peak_threads_usage from system.query_log where query like '%insert into testX %' and Settings['parallel_view_processing'] = '1' and Settings['optimize_trivial_insert_select'] = '1' and - Settings['max_insert_threads'] = '16'; -5 + Settings['max_insert_threads'] = '5'; +7 select count() from testX; -80 +1600 select count() from testXA; -80 -select count() from testXB; -0 +1600 select count() from testXC; -80 +1600 diff --git a/tests/queries/0_stateless/01275_parallel_mv.sql.j2 b/tests/queries/0_stateless/01275_parallel_mv.sql.j2 index 5918035e9c3..6805b8902ec 100644 --- a/tests/queries/0_stateless/01275_parallel_mv.sql.j2 +++ b/tests/queries/0_stateless/01275_parallel_mv.sql.j2 @@ -3,7 +3,13 @@ -- no-parallel: it checks the number of threads, which can be lowered in presence of other queries -- avoid settings randomization by clickhouse-test -set max_threads = 0; +set max_threads = 10; + +-- more blocks to process +set max_block_size = 10; +set min_insert_block_size_rows = 10; + +set materialized_views_ignore_errors = 1; drop table if exists testX; drop table if exists testXA; @@ -16,17 +22,25 @@ create materialized view testXA engine=MergeTree order by tuple() as select slee create materialized view testXB engine=MergeTree order by tuple() as select sleep(0.2), throwIf(A=1) from testX; create materialized view testXC engine=MergeTree order by tuple() as select sleep(0.1) from testX; + +{% set count = namespace(value=0) %} + -- { echoOn } {% for parallel_view_processing in [0, 1] %} {% for optimize_trivial_insert_select in [0, 1] %} -{% for max_insert_threads in [0, 16] %} -select 'optimize_trivial_insert_select={{ optimize_trivial_insert_select }}', 'max_insert_threads={{ max_insert_threads }}'; +{% for max_insert_threads in [0, 5] %} -insert into testX select number from numbers(10) settings +{% set count.value = count.value + 1 %} + +select 'optimize_trivial_insert_select={{ optimize_trivial_insert_select }}', 'max_insert_threads={{ max_insert_threads }}', 'iteration_num={{ iteration_num }}'; + +insert into testX select number from numbers(200) settings + send_logs_level='fatal', + insert_deduplication_token='UT_{{ count.value }}', log_queries=1, parallel_view_processing={{ parallel_view_processing }}, optimize_trivial_insert_select={{ optimize_trivial_insert_select }}, - max_insert_threads={{ max_insert_threads }}; -- { serverError FUNCTION_THROW_IF_VALUE_IS_NON_ZERO } + max_insert_threads={{ max_insert_threads }}; system flush logs; select peak_threads_usage from system.query_log where current_database = currentDatabase() and @@ -38,7 +52,6 @@ select peak_threads_usage from system.query_log where select count() from testX; select count() from testXA; -select count() from testXB; select count() from testXC; {% endfor %} {% endfor %} diff --git a/tests/queries/0_stateless/01927_query_views_log_current_database.sql b/tests/queries/0_stateless/01927_query_views_log_current_database.sql index ba42795333c..6287156daaf 100644 --- a/tests/queries/0_stateless/01927_query_views_log_current_database.sql +++ b/tests/queries/0_stateless/01927_query_views_log_current_database.sql @@ -16,6 +16,7 @@ CREATE MATERIALIZED VIEW matview_b_to_c TO table_c AS SELECT SUM(a + sleepEachRo CREATE MATERIALIZED VIEW matview_join_d_e TO table_f AS SELECT table_d.a as a, table_e.count + sleepEachRow(0.000003) as count FROM table_d LEFT JOIN table_e ON table_d.a = table_e.a; -- ENABLE LOGS +SET parallel_view_processing=0; SET log_query_views=1; SET log_queries_min_type='QUERY_FINISH'; SET log_queries=1; diff --git a/tests/queries/0_stateless/02124_insert_deduplication_token_materialized_views.reference b/tests/queries/0_stateless/02124_insert_deduplication_token_materialized_views.reference index e0cc8f0ce63..2d9f236ada9 100644 --- a/tests/queries/0_stateless/02124_insert_deduplication_token_materialized_views.reference +++ b/tests/queries/0_stateless/02124_insert_deduplication_token_materialized_views.reference @@ -1,8 +1,8 @@ -deduplicate_blocks_in_dependent_materialized_views=0, insert_deduplication_token = no, results inconsitent -18 18 9 18 -deduplicate_blocks_in_dependent_materialized_views=1, insert_deduplication_token = no, results inconsitent -18 9 9 9 -deduplicate_blocks_in_dependent_materialized_views=0, insert_deduplication_token = yes, results inconsitent -18 18 9 18 -deduplicate_blocks_in_dependent_materialized_views=1, insert_deduplication_token = yes, results consitent +deduplicate_blocks_in_dependent_materialized_views=0, insert_deduplication_token = no, results: test_mv_a and test_mv_c have all data, test_mv_b has data obly with max_partitions_per_insert_block=0 +18 36 27 36 +deduplicate_blocks_in_dependent_materialized_views=1, insert_deduplication_token = no, results: all tables have deduplicated data +18 18 18 18 +deduplicate_blocks_in_dependent_materialized_views=0, insert_deduplication_token = yes, results: test_mv_a and test_mv_c have all data, test_mv_b has data obly with max_partitions_per_insert_block=0 +18 36 27 36 +deduplicate_blocks_in_dependent_materialized_views=1, insert_deduplication_token = yes, results: all tables have deduplicated data 18 18 18 18 diff --git a/tests/queries/0_stateless/02124_insert_deduplication_token_materialized_views.sql b/tests/queries/0_stateless/02124_insert_deduplication_token_materialized_views.sql index fdd75b91b1f..465c8d6136c 100644 --- a/tests/queries/0_stateless/02124_insert_deduplication_token_materialized_views.sql +++ b/tests/queries/0_stateless/02124_insert_deduplication_token_materialized_views.sql @@ -1,6 +1,6 @@ -- Tags: long -select 'deduplicate_blocks_in_dependent_materialized_views=0, insert_deduplication_token = no, results inconsitent'; +select 'deduplicate_blocks_in_dependent_materialized_views=0, insert_deduplication_token = no, results: test_mv_a and test_mv_c have all data, test_mv_b has data obly with max_partitions_per_insert_block=0'; drop table if exists test sync; drop table if exists test_mv_a sync; @@ -35,7 +35,7 @@ select (select sum(c) from test_mv_c where test='case1'); -select 'deduplicate_blocks_in_dependent_materialized_views=1, insert_deduplication_token = no, results inconsitent'; +select 'deduplicate_blocks_in_dependent_materialized_views=1, insert_deduplication_token = no, results: all tables have deduplicated data'; set deduplicate_blocks_in_dependent_materialized_views=1; @@ -53,7 +53,7 @@ select (select sum(c) from test_mv_c where test='case2'); -select 'deduplicate_blocks_in_dependent_materialized_views=0, insert_deduplication_token = yes, results inconsitent'; +select 'deduplicate_blocks_in_dependent_materialized_views=0, insert_deduplication_token = yes, results: test_mv_a and test_mv_c have all data, test_mv_b has data obly with max_partitions_per_insert_block=0'; set deduplicate_blocks_in_dependent_materialized_views=0; @@ -70,7 +70,7 @@ select (select sum(c) from test_mv_b where test='case3'), (select sum(c) from test_mv_c where test='case3'); -select 'deduplicate_blocks_in_dependent_materialized_views=1, insert_deduplication_token = yes, results consitent'; +select 'deduplicate_blocks_in_dependent_materialized_views=1, insert_deduplication_token = yes, results: all tables have deduplicated data'; set deduplicate_blocks_in_dependent_materialized_views=1; diff --git a/tests/queries/0_stateless/02125_query_views_log.sql b/tests/queries/0_stateless/02125_query_views_log.sql index d2d19b76a1f..ba50902ebea 100644 --- a/tests/queries/0_stateless/02125_query_views_log.sql +++ b/tests/queries/0_stateless/02125_query_views_log.sql @@ -8,7 +8,7 @@ create table dst (key Int) engine=Null(); create materialized view mv1 to dst as select * from src; create materialized view mv2 to dst as select * from src; -insert into src select * from numbers(1e6) settings log_queries=1, max_untracked_memory=0, parallel_view_processing=1; +insert into src select * from numbers(1e6) settings log_queries=1, max_untracked_memory=0, parallel_view_processing=0; system flush logs; -- { echo } diff --git a/tests/queries/0_stateless/02805_distributed_queries_timeouts.sql b/tests/queries/0_stateless/02805_distributed_queries_timeouts.sql deleted file mode 100644 index 0b7337d1255..00000000000 --- a/tests/queries/0_stateless/02805_distributed_queries_timeouts.sql +++ /dev/null @@ -1,3 +0,0 @@ -select * from remote('127.2', view(select sleep(3) from system.one)) settings receive_timeout=1, async_socket_for_remote=0, use_hedged_requests=1 format Null; -select * from remote('127.2', view(select sleep(3) from system.one)) settings receive_timeout=1, async_socket_for_remote=1, use_hedged_requests=0 format Null; -select * from remote('127.2', view(select sleep(3) from system.one)) settings receive_timeout=1, async_socket_for_remote=0, use_hedged_requests=0 format Null; diff --git a/tests/queries/0_stateless/02912_ingestion_mv_deduplication.reference b/tests/queries/0_stateless/02912_ingestion_mv_deduplication.reference index 335b55f05c8..07deb7c2565 100644 --- a/tests/queries/0_stateless/02912_ingestion_mv_deduplication.reference +++ b/tests/queries/0_stateless/02912_ingestion_mv_deduplication.reference @@ -10,13 +10,14 @@ 2022-09-01 12:23:34 42 2023-09-01 12:23:34 42 -- MV -2022-09-01 12:00:00 42 +2022-09-01 12:00:00 84 +2023-09-01 12:00:00 42 -- Original issue with deduplicate_blocks_in_dependent_materialized_views = 1 AND max_insert_delayed_streams_for_parallel_write > 1 -- Landing 2022-09-01 12:23:34 42 2023-09-01 12:23:34 42 -- MV -2022-09-01 12:00:00 42 +2022-09-01 12:00:00 84 2023-09-01 12:00:00 42 -- Regression introduced in https://github.com/ClickHouse/ClickHouse/pull/54184 -- Landing (Agg/Replacing)MergeTree diff --git a/tests/queries/0_stateless/02912_ingestion_mv_deduplication.sql b/tests/queries/0_stateless/02912_ingestion_mv_deduplication.sql index f206f0d7775..a2378fd8f67 100644 --- a/tests/queries/0_stateless/02912_ingestion_mv_deduplication.sql +++ b/tests/queries/0_stateless/02912_ingestion_mv_deduplication.sql @@ -54,8 +54,9 @@ SELECT '-- Original issue with deduplicate_blocks_in_dependent_materialized_view - 1st insert works for landing and mv tables - 2nd insert gets first block 20220901 deduplicated and second one inserted in landing table - - 2nd insert is not inserting anything in mv table due to a bug computing blocks to be discarded + - 2nd insert is not inserting anything in mv table due to a bug computing blocks to be discarded, now that block is inserted because deduplicate_blocks_in_dependent_materialized_views=0 + Now it is fixed. */ SET deduplicate_blocks_in_dependent_materialized_views = 0, max_insert_delayed_streams_for_parallel_write = 1000; @@ -97,7 +98,7 @@ SELECT '-- Original issue with deduplicate_blocks_in_dependent_materialized_view This is what happens now: - 1st insert works for landing and mv tables - - 2nd insert gets first block 20220901 deduplicated and second one inserted for landing and mv tables + - 2nd insert gets first block 20220901 deduplicated for landing and both rows are inserted for mv tables */ SET deduplicate_blocks_in_dependent_materialized_views = 1, max_insert_delayed_streams_for_parallel_write = 1000; diff --git a/tests/queries/0_stateless/03008_deduplication.python b/tests/queries/0_stateless/03008_deduplication.python new file mode 100644 index 00000000000..bfaa496805f --- /dev/null +++ b/tests/queries/0_stateless/03008_deduplication.python @@ -0,0 +1,659 @@ +#!/usr/bin/env python3 + +import os +import sys +import argparse +import string + + +CURDIR = os.path.dirname(os.path.realpath(__file__)) +sys.path.insert(0, os.path.join(CURDIR, "helpers")) + + +def __format(template, **params): + field_names = [v[1] for v in string.Formatter().parse(template) if v[1] is not None] + kv_args = {} + for field in field_names: + if field in params: + kv_args[field] = params[field] + else: + kv_args[field] = "" + + return template.format(**kv_args) + + +def instance_create_statement( + table_name, + table_columns, + table_keys, + table_engine, + with_deduplication, + no_merges=True, +): + template = """ + CREATE TABLE {table_name} + {table_columns} + ENGINE = {table_engine} + ORDER BY {table_keys} + {table_settings}; + {table_no_merges} + """ + + params = dict() + params["table_name"] = table_name + params["table_columns"] = table_columns + params["table_keys"] = table_keys + params["table_no_merges"] = f"SYSTEM STOP MERGES {table_name};" if no_merges else "" + params["table_engine"] = ( + "MergeTree()" + if table_engine == "MergeTree" + else f"ReplicatedMergeTree('/clickhouse/tables/{{database}}/{table_name}', '1')" + ) + + deduplication_window_setting_name = ( + "non_replicated_deduplication_window" + if table_engine == "MergeTree" + else "replicated_deduplication_window" + ) + deduplication_window_setting_value = 1000 if with_deduplication else 0 + + settings = list() + settings += [ + f"{deduplication_window_setting_name}={deduplication_window_setting_value}" + ] + params["table_settings"] = "SETTINGS " + ",".join(settings) + + return __format(template, **params) + + +def instance_insert_statement( + table_name, count, insert_method, insert_unique_blocks, use_insert_token +): + insert_settings = ( + "" if not use_insert_token else "SETTINGS insert_deduplication_token='UDT'" + ) + + if insert_method == "InsertSelect": + template = """ + INSERT INTO {table_name} + SELECT {insert_columns} + FROM numbers({count}) {insert_settings}; + """ + return __format( + template, + table_name=table_name, + count=count, + insert_columns=( + "'src_4', 4" + if not insert_unique_blocks + else "'src_' || toString(number), number" + ), + insert_settings=insert_settings, + ) + + else: + template = """ + INSERT INTO {table_name} + {insert_settings} VALUES {insert_values}; + """ + + values = [] + for i in range(count): + values += ( + [f"('src_{i}', {i})"] if insert_unique_blocks else ["('src_4', 4)"] + ) + insert_values = ", ".join(values) + + return __format( + template, + table_name=table_name, + insert_settings=insert_settings, + insert_values=insert_values, + ) + + +def get_drop_tables_statements(tables): + return "".join( + [f"DROP TABLE IF EXISTS {table_name};\n" for table_name in tables[::-1]] + ) + + +def get_logs_statement(args): + if args.get_logs: + return "SET send_logs_level='test';" + return "" + + +def str2bool(v): + if isinstance(v, bool): + return v + if v.lower() in ("yes", "true", "t", "y", "1"): + return True + elif v.lower() in ("no", "false", "f", "n", "0"): + return False + else: + raise argparse.ArgumentTypeError("Boolean value expected.") + + +class ArgsFactory: + def __init__(self, parser): + self.__parser = parser + + def add_opt_engine(self): + self.__parser.add_argument( + "--table-engine", + choices=["ReplicatedMergeTree", "MergeTree"], + default="MergeTree", + ) + + def add_opt_user_token(self): + self.__parser.add_argument( + "--use-insert-token", type=str2bool, nargs="?", const=True, default=False + ) + + def add_opt_single_thread(self): + self.__parser.add_argument( + "--single-thread", type=str2bool, nargs="?", const=True, default=True + ) + + def add_opt_dedup_src(self): + self.__parser.add_argument( + "--deduplicate-src-table", + type=str2bool, + nargs="?", + const=True, + default=True, + ) + + def add_opt_dedup_dst(self): + self.__parser.add_argument( + "--deduplicate-dst-table", + type=str2bool, + nargs="?", + const=True, + default=True, + ) + + def add_opt_get_logs(self): + self.__parser.add_argument( + "--get-logs", type=str2bool, nargs="?", const=True, default=False + ) + + def add_opt_uniq_blocks(self): + self.__parser.add_argument( + "--insert-unique-blocks", type=str2bool, nargs="?", const=True, default=True + ) + + def add_opt_insert_method(self): + self.__parser.add_argument( + "--insert-method", + choices=["InsertSelect", "InsertValues"], + default="InsertSelect", + ) + + def add_all(self): + self.add_opt_engine() + self.add_opt_user_token() + self.add_opt_single_thread() + self.add_opt_dedup_src() + self.add_opt_dedup_dst() + self.add_opt_get_logs() + self.add_opt_insert_method() + self.add_opt_uniq_blocks() + + +def test_insert_several_blocks(parser): + ArgsFactory(parser).add_all() + + def calle(args): + create_table_a_b_statement = instance_create_statement( + table_name="table_a_b", + table_columns="(a String, b UInt64)", + table_keys="(a, b)", + table_engine=args.table_engine, + with_deduplication=args.deduplicate_src_table, + ) + + create_table_when_b_even_statement = instance_create_statement( + table_name="table_when_b_even", + table_columns="(a String, b UInt64)", + table_keys="(a, b)", + table_engine=args.table_engine, + with_deduplication=args.deduplicate_dst_table, + ) + + create_mv_statement = """ + CREATE MATERIALIZED VIEW mv_b_even + TO table_when_b_even + AS + SELECT a, b + FROM table_a_b + WHERE b % 2 = 0; + """ + + drop_tables_statements = get_drop_tables_statements( + ["table_a_b", "table_when_b_even", "mv_b_even"] + ) + + insert_statement = instance_insert_statement( + "table_a_b", + 10, + args.insert_method, + args.insert_unique_blocks, + args.use_insert_token, + ) + + print_details_statements = f""" + SELECT 'table_a_b'; + SELECT 'count', count() FROM table_a_b; + {"" if not args.get_logs else "SELECT _part, count() FROM table_a_b GROUP BY _part ORDER BY _part;"} + + SELECT 'table_when_b_even'; + SELECT 'count', count() FROM table_when_b_even; + {"" if not args.get_logs else "SELECT _part, count() FROM table_when_b_even GROUP BY _part ORDER BY _part;"} + """ + + if args.insert_unique_blocks: + assert_first_insert_statements = f""" + SELECT throwIf( count() != 10 ) + FROM table_a_b; + SELECT throwIf( count() != 5 ) + FROM table_when_b_even; + """ + assert_second_insert_statements = f""" + SELECT throwIf( count() != {10 if args.deduplicate_src_table else 20} ) + FROM table_a_b; + SELECT throwIf( count() != {5 if args.deduplicate_dst_table else 10} ) + FROM table_when_b_even; + """ + else: + if args.use_insert_token: + assert_first_insert_statements = """ + SELECT throwIf( count() != 10 ) + FROM table_a_b; + SELECT throwIf( count() != 10 ) + FROM table_when_b_even; + """ + assert_second_insert_statements = f""" + SELECT throwIf( count() != {10 if args.deduplicate_src_table else 20} ) + FROM table_a_b; + SELECT throwIf( count() != {10 if args.deduplicate_dst_table else 20} ) + FROM table_when_b_even; + """ + else: + assert_first_insert_statements = f""" + SELECT throwIf( count() != {1 if args.deduplicate_src_table else 10} ) + FROM table_a_b; + SELECT throwIf( count() != {1 if args.deduplicate_dst_table else 10} ) + FROM table_when_b_even; + """ + assert_second_insert_statements = f""" + SELECT throwIf( count() != {1 if args.deduplicate_src_table else 20} ) + FROM table_a_b; + SELECT throwIf( count() != {1 if args.deduplicate_dst_table else 20} ) + FROM table_when_b_even; + """ + + script = f""" + {get_logs_statement(args)} + + SET max_insert_threads={1 if args.single_thread else 10}; + SET update_insert_deduplication_token_in_dependent_materialized_views=1; + SET deduplicate_blocks_in_dependent_materialized_views=1; + + SET max_block_size=1; + SET min_insert_block_size_rows=0; + SET min_insert_block_size_bytes=0; + + {drop_tables_statements} + + {create_table_a_b_statement} + + {create_table_when_b_even_statement} + + {create_mv_statement} + + -- first insert + {insert_statement} + + {print_details_statements} + + {assert_first_insert_statements} + + -- second insert, it is retry + {insert_statement} + + {print_details_statements} + + {assert_second_insert_statements} + + {drop_tables_statements} + """ + + print(script) + + parser.set_defaults(func=calle) + + +def test_mv_generates_several_blocks(parser): + ArgsFactory(parser).add_all() + + def calle(args): + tables = [ + "table_for_join_with", + "table_a_b", + "table_when_b_even_and_joined", + "mv_b_even", + ] + drop_tables_statements = get_drop_tables_statements(tables) + + details_print_for_table_for_join_with = "" + if args.get_logs: + details_print_for_table_for_join_with = """ + SELECT 'table_for_join_with'; + SELECT a_join, b, _part FROM table_for_join_with ORDER BY _part, a_join, b; + """ + + create_table_a_b_statement = instance_create_statement( + table_name="table_a_b", + table_columns="(a_src String, b UInt64)", + table_keys="(a_src, b)", + table_engine=args.table_engine, + with_deduplication=args.deduplicate_src_table, + ) + + create_table_when_b_even_and_joined_statement = instance_create_statement( + table_name="table_when_b_even_and_joined", + table_columns="(a_src String, a_join String, b UInt64)", + table_keys="(a_src, a_join, b)", + table_engine=args.table_engine, + with_deduplication=args.deduplicate_dst_table, + ) + + insert_statement = instance_insert_statement( + "table_a_b", + 5, + args.insert_method, + args.insert_unique_blocks, + args.use_insert_token, + ) + + details_print_statements = f""" + SELECT 'table_a_b'; + SELECT 'count', count() FROM table_a_b; + + SELECT 'table_when_b_even_and_joined'; + SELECT 'count', count() FROM table_when_b_even_and_joined; + {"" if not args.get_logs else "SELECT _part, a_src, a_join, b FROM table_when_b_even_and_joined ORDER BY _part;"} + """ + + if args.insert_unique_blocks: + assert_first_insert_statements = f""" + SELECT throwIf( count() != 5 ) + FROM table_a_b; + + SELECT throwIf( count() != 9 ) + FROM table_when_b_even_and_joined; + """ + assert_second_insert_statements = f""" + SELECT throwIf( count() != {5 if args.deduplicate_src_table else 10} ) + FROM table_a_b; + + SELECT throwIf( count() != {9 if args.deduplicate_dst_table else 18} ) + FROM table_when_b_even_and_joined; + """ + else: + if args.use_insert_token: + assert_first_insert_statements = f""" + SELECT throwIf( count() != {5 if args.deduplicate_src_table else 5} ) + FROM table_a_b; + + SELECT throwIf( count() != {10 if args.deduplicate_dst_table else 10} ) + FROM table_when_b_even_and_joined; + """ + assert_second_insert_statements = f""" + SELECT throwIf( count() != {5 if args.deduplicate_src_table else 10} ) + FROM table_a_b; + + SELECT throwIf( count() != {10 if args.deduplicate_dst_table else 20} ) + FROM table_when_b_even_and_joined; + """ + else: + assert_first_insert_statements = f""" + SELECT throwIf( count() != {1 if args.deduplicate_src_table else 5} ) + FROM table_a_b; + + SELECT throwIf( count() != {2 if args.deduplicate_dst_table else 10} ) + FROM table_when_b_even_and_joined; + """ + assert_second_insert_statements = f""" + SELECT throwIf( count() != {1 if args.deduplicate_src_table else 10} ) + FROM table_a_b; + + SELECT throwIf( count() != {2 if args.deduplicate_dst_table else 20} ) + FROM table_when_b_even_and_joined; + """ + + script = f""" + {get_logs_statement(args)} + + SET max_insert_threads={1 if args.single_thread else 10}; + SET update_insert_deduplication_token_in_dependent_materialized_views=1; + SET deduplicate_blocks_in_dependent_materialized_views=1; + + SET max_block_size=1; + SET min_insert_block_size_rows=0; + SET min_insert_block_size_bytes=0; + + {drop_tables_statements} + + CREATE TABLE table_for_join_with + (a_join String, b UInt64) + ENGINE = MergeTree() + ORDER BY (a_join, b); + INSERT INTO table_for_join_with + SELECT 'joined_' || toString(number), number + FROM numbers(1); + {details_print_for_table_for_join_with} + + {create_table_a_b_statement} + SYSTEM STOP MERGES table_a_b; + + {create_table_when_b_even_and_joined_statement} + SYSTEM STOP MERGES table_when_b_even_and_joined; + + CREATE MATERIALIZED VIEW mv_b_even + TO table_when_b_even_and_joined + AS + SELECT a_src, a_join, table_for_join_with.b as b + FROM table_a_b + FULL OUTER JOIN table_for_join_with + ON table_a_b.b = table_for_join_with.b AND table_a_b.b % 2 = 0 + ORDER BY a_src, a_join, b; + + -- first insert + {insert_statement} + + {details_print_statements} + + -- first assertion + {assert_first_insert_statements} + + -- second insert + {insert_statement} + + {details_print_statements} + + -- second assertion + {assert_second_insert_statements} + + {drop_tables_statements} + """ + + print(script) + + parser.set_defaults(func=calle) + + +def test_several_mv_into_one_table(parser): + ArgsFactory(parser).add_all() + + def calle(args): + tables = ["table_src", "table_dst", "mv_b_even", "mv_b_even_even"] + drop_tables_statements = get_drop_tables_statements(tables) + + create_table_src_statement = instance_create_statement( + table_name="table_src", + table_columns="(a String, b UInt64)", + table_keys="(a, b)", + table_engine=args.table_engine, + with_deduplication=args.deduplicate_src_table, + ) + + create_table_dst_statement = instance_create_statement( + table_name="table_dst", + table_columns="(a String, b UInt64)", + table_keys="(a, b)", + table_engine=args.table_engine, + with_deduplication=args.deduplicate_dst_table, + ) + + insert_statement = instance_insert_statement( + "table_src", + 8, + args.insert_method, + args.insert_unique_blocks, + args.use_insert_token, + ) + + details_print_statements = f""" + SELECT 'table_src count', count() FROM table_src; + + SELECT 'table_dst count', count() FROM table_dst; + {"" if not args.get_logs else "SELECT _part, count() FROM table_dst GROUP BY _part ORDER BY _part;"} + """ + + if args.insert_unique_blocks: + assert_first_insert_statements = f""" + SELECT throwIf( count() != 8 ) + FROM table_src; + + SELECT throwIf( count() != 6 ) + FROM table_dst; + """ + assert_second_insert_statements = f""" + SELECT throwIf( count() != {8 if args.deduplicate_src_table else 16} ) + FROM table_src; + + SELECT throwIf( count() != {6 if args.deduplicate_dst_table else 12} ) + FROM table_dst; + """ + else: + if args.use_insert_token: + assert_first_insert_statements = f""" + SELECT throwIf( count() != {8 if args.deduplicate_src_table else 8} ) + FROM table_src; + + SELECT throwIf( count() != {16 if args.deduplicate_dst_table else 16} ) + FROM table_dst; + """ + assert_second_insert_statements = f""" + SELECT throwIf( count() != {8 if args.deduplicate_src_table else 16} ) + FROM table_src; + + SELECT throwIf( count() != {16 if args.deduplicate_dst_table else 32} ) + FROM table_dst; + """ + else: + assert_first_insert_statements = f""" + SELECT throwIf( count() != {1 if args.deduplicate_src_table else 8} ) + FROM table_src; + + SELECT throwIf( count() != {2 if args.deduplicate_dst_table else 16} ) + FROM table_dst; + """ + assert_second_insert_statements = f""" + SELECT throwIf( count() != {1 if args.deduplicate_src_table else 16} ) + FROM table_src; + + SELECT throwIf( count() != {2 if args.deduplicate_dst_table else 32} ) + FROM table_dst; + """ + + script = f""" + {get_logs_statement(args)} + + SET max_insert_threads={1 if args.single_thread else 10}; + SET update_insert_deduplication_token_in_dependent_materialized_views=1; + SET deduplicate_blocks_in_dependent_materialized_views=1; + + SET max_block_size=1; + SET min_insert_block_size_rows=0; + SET min_insert_block_size_bytes=0; + + {drop_tables_statements} + + {create_table_src_statement} + + {create_table_dst_statement} + + CREATE MATERIALIZED VIEW mv_b_even + TO table_dst + AS + SELECT a, b + FROM table_src + WHERE b % 2 = 0; + + CREATE MATERIALIZED VIEW mv_b_even_even + TO table_dst + AS + SELECT a, b + FROM table_src + WHERE b % 4 = 0; + + -- first insert + {insert_statement} + + {details_print_statements} + + {assert_first_insert_statements} + + -- second insert, retry + {insert_statement} + + {details_print_statements} + + {assert_second_insert_statements} + + {drop_tables_statements} + """ + + print(script) + + parser.set_defaults(func=calle) + + +def parse_args(): + parser = argparse.ArgumentParser() + subparsers = parser.add_subparsers(dest="test") + test_insert_several_blocks( + subparsers.add_parser("insert_several_blocks_into_table") + ) + test_mv_generates_several_blocks( + subparsers.add_parser("mv_generates_several_blocks") + ) + test_several_mv_into_one_table(subparsers.add_parser("several_mv_into_one_table")) + args = parser.parse_args() + if args.test is None: + parser.print_help() + return args + + +def main(): + args = parse_args() + if args.test is not None: + args.func(args) + + +if __name__ == "__main__": + main() diff --git a/tests/queries/0_stateless/03008_deduplication_cases_from_docs.reference b/tests/queries/0_stateless/03008_deduplication_cases_from_docs.reference new file mode 100644 index 00000000000..4893274c1cd --- /dev/null +++ b/tests/queries/0_stateless/03008_deduplication_cases_from_docs.reference @@ -0,0 +1,41 @@ +Different materialized view insert into one underlayed table equal data. +first attempt +from dst 1 A all_1_1_0 +from mv_dst 0 A all_1_1_0 +from mv_dst 0 A all_2_2_0 +second attempt +from dst 1 A all_1_1_0 +from mv_dst 0 A all_1_1_0 +from mv_dst 0 A all_2_2_0 +Different insert operations generate the same data after transformation in underlied table of materialized view. +first attempt +from dst 1 A all_1_1_0 +from mv_dst 0 A all_1_1_0 +second attempt +from dst 1 A all_1_1_0 +from dst 2 A all_2_2_0 +from mv_dst 0 A all_1_1_0 +from mv_dst 0 A all_2_2_0 +Indentical blocks in insertion with `insert_deduplication_token` +first attempt +from dst 0 A all_1_1_0 +from dst 0 A all_2_2_0 +second attempt +from dst 0 A all_1_1_0 +from dst 0 A all_2_2_0 +third attempt +from dst 0 A all_1_1_0 +from dst 0 A all_2_2_0 +Indentical blocks in insertion +from dst 0 A all_1_1_0 +Indentical blocks after materialised view`s transformation +first attempt +from dst 1 B all_1_1_0 +from dst 2 B all_2_2_0 +from mv_dst 0 B all_1_1_0 +from mv_dst 0 B all_2_2_0 +second attempt +from dst 1 B all_1_1_0 +from dst 2 B all_2_2_0 +from mv_dst 0 B all_1_1_0 +from mv_dst 0 B all_2_2_0 diff --git a/tests/queries/0_stateless/03008_deduplication_cases_from_docs.sql b/tests/queries/0_stateless/03008_deduplication_cases_from_docs.sql new file mode 100644 index 00000000000..7927a6b1edf --- /dev/null +++ b/tests/queries/0_stateless/03008_deduplication_cases_from_docs.sql @@ -0,0 +1,331 @@ +-- ######### +select 'Different materialized view insert into one underlayed table equal data.'; + +DROP TABLE IF EXISTS dst; +DROP TABLE IF EXISTS mv_dst; +DROP TABLE IF EXISTS mv_first; +DROP TABLE IF EXISTS mv_second; + +CREATE TABLE dst +( + `key` Int64, + `value` String +) +ENGINE = MergeTree +ORDER BY tuple() +SETTINGS non_replicated_deduplication_window=1000; + +CREATE TABLE mv_dst +( + `key` Int64, + `value` String +) +ENGINE = MergeTree +ORDER BY tuple() +SETTINGS non_replicated_deduplication_window=1000; + +CREATE MATERIALIZED VIEW mv_first +TO mv_dst +AS SELECT + 0 AS key, + value AS value +FROM dst; + +CREATE MATERIALIZED VIEW mv_second +TO mv_dst +AS SELECT + 0 AS key, + value AS value +FROM dst; + +SET deduplicate_blocks_in_dependent_materialized_views=1; + +select 'first attempt'; + +INSERT INTO dst VALUES (1, 'A'); + +SELECT + 'from dst', + *, + _part +FROM dst +ORDER by all; + +SELECT + 'from mv_dst', + *, + _part +FROM mv_dst +ORDER by all; + +select 'second attempt'; + +INSERT INTO dst VALUES (1, 'A'); + +SELECT + 'from dst', + *, + _part +FROM dst +ORDER by all; + +SELECT + 'from mv_dst', + *, + _part +FROM mv_dst +ORDER by all; + +DROP TABLE mv_second; +DROP TABLE mv_first; +DROP TABLE mv_dst; +DROP TABLE dst; + + +-- ######### +select 'Different insert operations generate the same data after transformation in underlied table of materialized view.'; + +DROP TABLE IF EXISTS dst; +DROP TABLE IF EXISTS mv_dst; + +CREATE TABLE dst +( + `key` Int64, + `value` String +) +ENGINE = MergeTree +ORDER BY tuple() +SETTINGS non_replicated_deduplication_window=1000; + +CREATE MATERIALIZED VIEW mv_dst +( + `key` Int64, + `value` String +) +ENGINE = MergeTree +ORDER BY tuple() +SETTINGS non_replicated_deduplication_window=1000 +AS SELECT + 0 AS key, + value AS value +FROM dst; + +SET deduplicate_blocks_in_dependent_materialized_views=1; + +select 'first attempt'; + +INSERT INTO dst VALUES (1, 'A'); + +SELECT + 'from dst', + *, + _part +FROM dst +ORDER by all; + +SELECT + 'from mv_dst', + *, + _part +FROM mv_dst +ORDER by all; + +select 'second attempt'; + +INSERT INTO dst VALUES (2, 'A'); + +SELECT + 'from dst', + *, + _part +FROM dst +ORDER by all; + +SELECT + 'from mv_dst', + *, + _part +FROM mv_dst +ORDER by all; + +DROP TABLE mv_dst; +DROP TABLE dst; + + +-- ######### +select 'Indentical blocks in insertion with `insert_deduplication_token`'; + +DROP TABLE IF EXISTS dst; + +CREATE TABLE dst +( + `key` Int64, + `value` String +) +ENGINE = MergeTree +ORDER BY tuple() +SETTINGS non_replicated_deduplication_window=1000; + +SET max_block_size=1; +SET min_insert_block_size_rows=0; +SET min_insert_block_size_bytes=0; + +select 'first attempt'; + +INSERT INTO dst SELECT + 0 AS key, + 'A' AS value +FROM numbers(2) +SETTINGS insert_deduplication_token='some_user_token'; + +SELECT + 'from dst', + *, + _part +FROM dst +ORDER by all; + +select 'second attempt'; + +INSERT INTO dst SELECT + 0 AS key, + 'A' AS value +FROM numbers(2) +SETTINGS insert_deduplication_token='some_user_token'; + +SELECT + 'from dst', + *, + _part +FROM dst +ORDER by all; + +select 'third attempt'; + +INSERT INTO dst SELECT + 1 AS key, + 'b' AS value +FROM numbers(2) +SETTINGS insert_deduplication_token='some_user_token'; + +SELECT + 'from dst', + *, + _part +FROM dst +ORDER by all; + +DROP TABLE dst; + + +-- ######### +select 'Indentical blocks in insertion'; + +DROP TABLE IF EXISTS dst; + +CREATE TABLE dst +( + `key` Int64, + `value` String +) +ENGINE = MergeTree +ORDER BY tuple() +SETTINGS non_replicated_deduplication_window=1000; + +SET max_block_size=1; +SET min_insert_block_size_rows=0; +SET min_insert_block_size_bytes=0; + +INSERT INTO dst SELECT + 0 AS key, + 'A' AS value +FROM numbers(2); + +SELECT + 'from dst', + *, + _part +FROM dst +ORDER by all; + +DROP TABLE dst; + + +-- ######### +select 'Indentical blocks after materialised view`s transformation'; + +DROP TABLE IF EXISTS dst; +DROP TABLE IF EXISTS mv_dst; + +CREATE TABLE dst +( + `key` Int64, + `value` String +) +ENGINE = MergeTree +ORDER BY tuple() +SETTINGS non_replicated_deduplication_window=1000; + +CREATE MATERIALIZED VIEW mv_dst +( + `key` Int64, + `value` String +) +ENGINE = MergeTree +ORDER BY tuple() +SETTINGS non_replicated_deduplication_window=1000 +AS SELECT + 0 AS key, + value AS value +FROM dst; + +SET max_block_size=1; +SET min_insert_block_size_rows=0; +SET min_insert_block_size_bytes=0; + +SET deduplicate_blocks_in_dependent_materialized_views=1; + +select 'first attempt'; + +INSERT INTO dst SELECT + number + 1 AS key, + IF(key = 0, 'A', 'B') AS value +FROM numbers(2); + +SELECT + 'from dst', + *, + _part +FROM dst +ORDER by all; + +SELECT + 'from mv_dst', + *, + _part +FROM mv_dst +ORDER by all; + +select 'second attempt'; + +INSERT INTO dst SELECT + number + 1 AS key, + IF(key = 0, 'A', 'B') AS value +FROM numbers(2); + +SELECT + 'from dst', + *, + _part +FROM dst +ORDER by all; + +SELECT + 'from mv_dst', + *, + _part +FROM mv_dst +ORDER by all; + +DROP TABLE mv_dst; +DROP TABLE dst; diff --git a/tests/queries/0_stateless/03008_deduplication_insert_into_partitioned_table.reference b/tests/queries/0_stateless/03008_deduplication_insert_into_partitioned_table.reference new file mode 100644 index 00000000000..c82a6eaa213 --- /dev/null +++ b/tests/queries/0_stateless/03008_deduplication_insert_into_partitioned_table.reference @@ -0,0 +1,35 @@ +no user deduplication token +partitioned_table is deduplicated bacause deduplication works in scope of one partiotion: +1 A +1 D +2 B +2 C +mv_table is not deduplicated because the inserted blocks was different: +1 A +1 A +1 D +2 B +2 B +2 C +with user deduplication token +partitioned_table is not deduplicated because different tokens: +1 A +1 A +1 D +2 B +2 B +2 C +mv_table is not deduplicated because different tokens: +1 A +1 A +1 D +2 B +2 B +2 C +with incorrect ussage of user deduplication token +partitioned_table is deduplicated because equal tokens: +1 A +2 B +mv_table is deduplicated because equal tokens: +1 A +2 B diff --git a/tests/queries/0_stateless/03008_deduplication_insert_into_partitioned_table.sql b/tests/queries/0_stateless/03008_deduplication_insert_into_partitioned_table.sql new file mode 100644 index 00000000000..2eb931f7f73 --- /dev/null +++ b/tests/queries/0_stateless/03008_deduplication_insert_into_partitioned_table.sql @@ -0,0 +1,83 @@ +DROP TABLE IF EXISTS partitioned_table; +DROP TABLE IF EXISTS mv_table; + + +SET deduplicate_blocks_in_dependent_materialized_views = 1; + + +SELECT 'no user deduplication token'; + +CREATE TABLE partitioned_table + (key Int64, value String) + ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/03008_deduplication_insert_into_partitioned_table', '{replica}') + partition by key % 10 + order by tuple(); + +CREATE MATERIALIZED VIEW mv_table (key Int64, value String) + ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/03008_deduplication_insert_into_partitioned_table_mv', '{replica}') + ORDER BY tuple() + AS SELECT key, value FROM partitioned_table; + +INSERT INTO partitioned_table VALUES (1, 'A'), (2, 'B'); +INSERT INTO partitioned_table VALUES (1, 'A'), (2, 'C'); +INSERT INTO partitioned_table VALUES (1, 'D'), (2, 'B'); + +SELECT 'partitioned_table is deduplicated bacause deduplication works in scope of one partiotion:'; +SELECT * FROM partitioned_table ORDER BY ALL; +SELECT 'mv_table is not deduplicated because the inserted blocks was different:'; +SELECT * FROM mv_table ORDER BY ALL; + +DROP TABLE partitioned_table; +DROP TABLE mv_table; + + +SELECT 'with user deduplication token'; + +CREATE TABLE partitioned_table + (key Int64, value String) + ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/03008_deduplication_insert_into_partitioned_table', '{replica}') + partition by key % 10 + order by tuple(); + +CREATE MATERIALIZED VIEW mv_table (key Int64, value String) + ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/03008_deduplication_insert_into_partitioned_table_mv', '{replica}') + ORDER BY tuple() + AS SELECT key, value FROM partitioned_table; + +INSERT INTO partitioned_table SETTINGS insert_deduplication_token='token_1' VALUES (1, 'A'), (2, 'B'); +INSERT INTO partitioned_table SETTINGS insert_deduplication_token='token_2' VALUES (1, 'A'), (2, 'C'); +INSERT INTO partitioned_table SETTINGS insert_deduplication_token='token_3' VALUES (1, 'D'), (2, 'B'); + +SELECT 'partitioned_table is not deduplicated because different tokens:'; +SELECT * FROM partitioned_table ORDER BY ALL; +SELECT 'mv_table is not deduplicated because different tokens:'; +SELECT * FROM mv_table ORDER BY ALL; + +DROP TABLE partitioned_table; +DROP TABLE mv_table; + + +SELECT 'with incorrect ussage of user deduplication token'; + +CREATE TABLE partitioned_table + (key Int64, value String) + ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/03008_deduplication_insert_into_partitioned_table', '{replica}') + partition by key % 10 + order by tuple(); + +CREATE MATERIALIZED VIEW mv_table (key Int64, value String) + ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/03008_deduplication_insert_into_partitioned_table_mv', '{replica}') + ORDER BY tuple() + AS SELECT key, value FROM partitioned_table; + +INSERT INTO partitioned_table SETTINGS insert_deduplication_token='token_0' VALUES (1, 'A'), (2, 'B'); +INSERT INTO partitioned_table SETTINGS insert_deduplication_token='token_0' VALUES (1, 'A'), (2, 'C'); +INSERT INTO partitioned_table SETTINGS insert_deduplication_token='token_0' VALUES (1, 'D'), (2, 'B'); + +SELECT 'partitioned_table is deduplicated because equal tokens:'; +SELECT * FROM partitioned_table ORDER BY ALL; +SELECT 'mv_table is deduplicated because equal tokens:'; +SELECT * FROM mv_table ORDER BY ALL; + +DROP TABLE partitioned_table; +DROP TABLE mv_table; diff --git a/tests/queries/0_stateless/03008_deduplication_insert_several_blocks_nonreplicated.reference b/tests/queries/0_stateless/03008_deduplication_insert_several_blocks_nonreplicated.reference new file mode 100644 index 00000000000..bf900aa84d2 --- /dev/null +++ b/tests/queries/0_stateless/03008_deduplication_insert_several_blocks_nonreplicated.reference @@ -0,0 +1,962 @@ + +Test case 0: insert_method=InsertSelect engine=MergeTree use_insert_token=True single_thread=True deduplicate_src_table=True deduplicate_dst_table=True insert_unique_blocks=True +table_a_b +count 10 +table_when_b_even +count 5 +0 +0 +table_a_b +count 10 +table_when_b_even +count 5 +0 +0 +OK + +Test case 1: insert_method=InsertSelect engine=MergeTree use_insert_token=True single_thread=True deduplicate_src_table=True deduplicate_dst_table=True insert_unique_blocks=False +table_a_b +count 10 +table_when_b_even +count 10 +0 +0 +table_a_b +count 10 +table_when_b_even +count 10 +0 +0 +OK + +Test case 2: insert_method=InsertSelect engine=MergeTree use_insert_token=True single_thread=True deduplicate_src_table=True deduplicate_dst_table=False insert_unique_blocks=True +table_a_b +count 10 +table_when_b_even +count 5 +0 +0 +table_a_b +count 10 +table_when_b_even +count 10 +0 +0 +OK + +Test case 3: insert_method=InsertSelect engine=MergeTree use_insert_token=True single_thread=True deduplicate_src_table=True deduplicate_dst_table=False insert_unique_blocks=False +table_a_b +count 10 +table_when_b_even +count 10 +0 +0 +table_a_b +count 10 +table_when_b_even +count 20 +0 +0 +OK + +Test case 4: insert_method=InsertSelect engine=MergeTree use_insert_token=True single_thread=True deduplicate_src_table=False deduplicate_dst_table=True insert_unique_blocks=True +table_a_b +count 10 +table_when_b_even +count 5 +0 +0 +table_a_b +count 20 +table_when_b_even +count 5 +0 +0 +OK + +Test case 5: insert_method=InsertSelect engine=MergeTree use_insert_token=True single_thread=True deduplicate_src_table=False deduplicate_dst_table=True insert_unique_blocks=False +table_a_b +count 10 +table_when_b_even +count 10 +0 +0 +table_a_b +count 20 +table_when_b_even +count 10 +0 +0 +OK + +Test case 6: insert_method=InsertSelect engine=MergeTree use_insert_token=True single_thread=True deduplicate_src_table=False deduplicate_dst_table=False insert_unique_blocks=True +table_a_b +count 10 +table_when_b_even +count 5 +0 +0 +table_a_b +count 20 +table_when_b_even +count 10 +0 +0 +OK + +Test case 7: insert_method=InsertSelect engine=MergeTree use_insert_token=True single_thread=True deduplicate_src_table=False deduplicate_dst_table=False insert_unique_blocks=False +table_a_b +count 10 +table_when_b_even +count 10 +0 +0 +table_a_b +count 20 +table_when_b_even +count 20 +0 +0 +OK + +Test case 8: insert_method=InsertSelect engine=MergeTree use_insert_token=True single_thread=False deduplicate_src_table=True deduplicate_dst_table=True insert_unique_blocks=True +table_a_b +count 10 +table_when_b_even +count 5 +0 +0 +table_a_b +count 10 +table_when_b_even +count 5 +0 +0 +OK + +Test case 9: insert_method=InsertSelect engine=MergeTree use_insert_token=True single_thread=False deduplicate_src_table=True deduplicate_dst_table=True insert_unique_blocks=False +table_a_b +count 10 +table_when_b_even +count 10 +0 +0 +table_a_b +count 10 +table_when_b_even +count 10 +0 +0 +OK + +Test case 10: insert_method=InsertSelect engine=MergeTree use_insert_token=True single_thread=False deduplicate_src_table=True deduplicate_dst_table=False insert_unique_blocks=True +table_a_b +count 10 +table_when_b_even +count 5 +0 +0 +table_a_b +count 10 +table_when_b_even +count 10 +0 +0 +OK + +Test case 11: insert_method=InsertSelect engine=MergeTree use_insert_token=True single_thread=False deduplicate_src_table=True deduplicate_dst_table=False insert_unique_blocks=False +table_a_b +count 10 +table_when_b_even +count 10 +0 +0 +table_a_b +count 10 +table_when_b_even +count 20 +0 +0 +OK + +Test case 12: insert_method=InsertSelect engine=MergeTree use_insert_token=True single_thread=False deduplicate_src_table=False deduplicate_dst_table=True insert_unique_blocks=True +table_a_b +count 10 +table_when_b_even +count 5 +0 +0 +table_a_b +count 20 +table_when_b_even +count 5 +0 +0 +OK + +Test case 13: insert_method=InsertSelect engine=MergeTree use_insert_token=True single_thread=False deduplicate_src_table=False deduplicate_dst_table=True insert_unique_blocks=False +table_a_b +count 10 +table_when_b_even +count 10 +0 +0 +table_a_b +count 20 +table_when_b_even +count 10 +0 +0 +OK + +Test case 14: insert_method=InsertSelect engine=MergeTree use_insert_token=True single_thread=False deduplicate_src_table=False deduplicate_dst_table=False insert_unique_blocks=True +table_a_b +count 10 +table_when_b_even +count 5 +0 +0 +table_a_b +count 20 +table_when_b_even +count 10 +0 +0 +OK + +Test case 15: insert_method=InsertSelect engine=MergeTree use_insert_token=True single_thread=False deduplicate_src_table=False deduplicate_dst_table=False insert_unique_blocks=False +table_a_b +count 10 +table_when_b_even +count 10 +0 +0 +table_a_b +count 20 +table_when_b_even +count 20 +0 +0 +OK + +Test case 16: insert_method=InsertSelect engine=MergeTree use_insert_token=False single_thread=True deduplicate_src_table=True deduplicate_dst_table=True insert_unique_blocks=True +table_a_b +count 10 +table_when_b_even +count 5 +0 +0 +table_a_b +count 10 +table_when_b_even +count 5 +0 +0 +OK + +Test case 17: insert_method=InsertSelect engine=MergeTree use_insert_token=False single_thread=True deduplicate_src_table=True deduplicate_dst_table=True insert_unique_blocks=False +table_a_b +count 1 +table_when_b_even +count 1 +0 +0 +table_a_b +count 1 +table_when_b_even +count 1 +0 +0 +OK + +Test case 18: insert_method=InsertSelect engine=MergeTree use_insert_token=False single_thread=True deduplicate_src_table=True deduplicate_dst_table=False insert_unique_blocks=True +table_a_b +count 10 +table_when_b_even +count 5 +0 +0 +table_a_b +count 10 +table_when_b_even +count 10 +0 +0 +OK + +Test case 19: insert_method=InsertSelect engine=MergeTree use_insert_token=False single_thread=True deduplicate_src_table=True deduplicate_dst_table=False insert_unique_blocks=False +table_a_b +count 1 +table_when_b_even +count 10 +0 +0 +table_a_b +count 1 +table_when_b_even +count 20 +0 +0 +OK + +Test case 20: insert_method=InsertSelect engine=MergeTree use_insert_token=False single_thread=True deduplicate_src_table=False deduplicate_dst_table=True insert_unique_blocks=True +table_a_b +count 10 +table_when_b_even +count 5 +0 +0 +table_a_b +count 20 +table_when_b_even +count 5 +0 +0 +OK + +Test case 21: insert_method=InsertSelect engine=MergeTree use_insert_token=False single_thread=True deduplicate_src_table=False deduplicate_dst_table=True insert_unique_blocks=False +table_a_b +count 10 +table_when_b_even +count 1 +0 +0 +table_a_b +count 20 +table_when_b_even +count 1 +0 +0 +OK + +Test case 22: insert_method=InsertSelect engine=MergeTree use_insert_token=False single_thread=True deduplicate_src_table=False deduplicate_dst_table=False insert_unique_blocks=True +table_a_b +count 10 +table_when_b_even +count 5 +0 +0 +table_a_b +count 20 +table_when_b_even +count 10 +0 +0 +OK + +Test case 23: insert_method=InsertSelect engine=MergeTree use_insert_token=False single_thread=True deduplicate_src_table=False deduplicate_dst_table=False insert_unique_blocks=False +table_a_b +count 10 +table_when_b_even +count 10 +0 +0 +table_a_b +count 20 +table_when_b_even +count 20 +0 +0 +OK + +Test case 24: insert_method=InsertSelect engine=MergeTree use_insert_token=False single_thread=False deduplicate_src_table=True deduplicate_dst_table=True insert_unique_blocks=True +table_a_b +count 10 +table_when_b_even +count 5 +0 +0 +table_a_b +count 10 +table_when_b_even +count 5 +0 +0 +OK + +Test case 25: insert_method=InsertSelect engine=MergeTree use_insert_token=False single_thread=False deduplicate_src_table=True deduplicate_dst_table=True insert_unique_blocks=False +table_a_b +count 1 +table_when_b_even +count 1 +0 +0 +table_a_b +count 1 +table_when_b_even +count 1 +0 +0 +OK + +Test case 26: insert_method=InsertSelect engine=MergeTree use_insert_token=False single_thread=False deduplicate_src_table=True deduplicate_dst_table=False insert_unique_blocks=True +table_a_b +count 10 +table_when_b_even +count 5 +0 +0 +table_a_b +count 10 +table_when_b_even +count 10 +0 +0 +OK + +Test case 27: insert_method=InsertSelect engine=MergeTree use_insert_token=False single_thread=False deduplicate_src_table=True deduplicate_dst_table=False insert_unique_blocks=False +table_a_b +count 1 +table_when_b_even +count 10 +0 +0 +table_a_b +count 1 +table_when_b_even +count 20 +0 +0 +OK + +Test case 28: insert_method=InsertSelect engine=MergeTree use_insert_token=False single_thread=False deduplicate_src_table=False deduplicate_dst_table=True insert_unique_blocks=True +table_a_b +count 10 +table_when_b_even +count 5 +0 +0 +table_a_b +count 20 +table_when_b_even +count 5 +0 +0 +OK + +Test case 29: insert_method=InsertSelect engine=MergeTree use_insert_token=False single_thread=False deduplicate_src_table=False deduplicate_dst_table=True insert_unique_blocks=False +table_a_b +count 10 +table_when_b_even +count 1 +0 +0 +table_a_b +count 20 +table_when_b_even +count 1 +0 +0 +OK + +Test case 30: insert_method=InsertSelect engine=MergeTree use_insert_token=False single_thread=False deduplicate_src_table=False deduplicate_dst_table=False insert_unique_blocks=True +table_a_b +count 10 +table_when_b_even +count 5 +0 +0 +table_a_b +count 20 +table_when_b_even +count 10 +0 +0 +OK + +Test case 31: insert_method=InsertSelect engine=MergeTree use_insert_token=False single_thread=False deduplicate_src_table=False deduplicate_dst_table=False insert_unique_blocks=False +table_a_b +count 10 +table_when_b_even +count 10 +0 +0 +table_a_b +count 20 +table_when_b_even +count 20 +0 +0 +OK + +Test case 32: insert_method=InsertValues engine=MergeTree use_insert_token=True single_thread=True deduplicate_src_table=True deduplicate_dst_table=True insert_unique_blocks=True +table_a_b +count 10 +table_when_b_even +count 5 +0 +0 +table_a_b +count 10 +table_when_b_even +count 5 +0 +0 +OK + +Test case 33: insert_method=InsertValues engine=MergeTree use_insert_token=True single_thread=True deduplicate_src_table=True deduplicate_dst_table=True insert_unique_blocks=False +table_a_b +count 10 +table_when_b_even +count 10 +0 +0 +table_a_b +count 10 +table_when_b_even +count 10 +0 +0 +OK + +Test case 34: insert_method=InsertValues engine=MergeTree use_insert_token=True single_thread=True deduplicate_src_table=True deduplicate_dst_table=False insert_unique_blocks=True +table_a_b +count 10 +table_when_b_even +count 5 +0 +0 +table_a_b +count 10 +table_when_b_even +count 10 +0 +0 +OK + +Test case 35: insert_method=InsertValues engine=MergeTree use_insert_token=True single_thread=True deduplicate_src_table=True deduplicate_dst_table=False insert_unique_blocks=False +table_a_b +count 10 +table_when_b_even +count 10 +0 +0 +table_a_b +count 10 +table_when_b_even +count 20 +0 +0 +OK + +Test case 36: insert_method=InsertValues engine=MergeTree use_insert_token=True single_thread=True deduplicate_src_table=False deduplicate_dst_table=True insert_unique_blocks=True +table_a_b +count 10 +table_when_b_even +count 5 +0 +0 +table_a_b +count 20 +table_when_b_even +count 5 +0 +0 +OK + +Test case 37: insert_method=InsertValues engine=MergeTree use_insert_token=True single_thread=True deduplicate_src_table=False deduplicate_dst_table=True insert_unique_blocks=False +table_a_b +count 10 +table_when_b_even +count 10 +0 +0 +table_a_b +count 20 +table_when_b_even +count 10 +0 +0 +OK + +Test case 38: insert_method=InsertValues engine=MergeTree use_insert_token=True single_thread=True deduplicate_src_table=False deduplicate_dst_table=False insert_unique_blocks=True +table_a_b +count 10 +table_when_b_even +count 5 +0 +0 +table_a_b +count 20 +table_when_b_even +count 10 +0 +0 +OK + +Test case 39: insert_method=InsertValues engine=MergeTree use_insert_token=True single_thread=True deduplicate_src_table=False deduplicate_dst_table=False insert_unique_blocks=False +table_a_b +count 10 +table_when_b_even +count 10 +0 +0 +table_a_b +count 20 +table_when_b_even +count 20 +0 +0 +OK + +Test case 40: insert_method=InsertValues engine=MergeTree use_insert_token=True single_thread=False deduplicate_src_table=True deduplicate_dst_table=True insert_unique_blocks=True +table_a_b +count 10 +table_when_b_even +count 5 +0 +0 +table_a_b +count 10 +table_when_b_even +count 5 +0 +0 +OK + +Test case 41: insert_method=InsertValues engine=MergeTree use_insert_token=True single_thread=False deduplicate_src_table=True deduplicate_dst_table=True insert_unique_blocks=False +table_a_b +count 10 +table_when_b_even +count 10 +0 +0 +table_a_b +count 10 +table_when_b_even +count 10 +0 +0 +OK + +Test case 42: insert_method=InsertValues engine=MergeTree use_insert_token=True single_thread=False deduplicate_src_table=True deduplicate_dst_table=False insert_unique_blocks=True +table_a_b +count 10 +table_when_b_even +count 5 +0 +0 +table_a_b +count 10 +table_when_b_even +count 10 +0 +0 +OK + +Test case 43: insert_method=InsertValues engine=MergeTree use_insert_token=True single_thread=False deduplicate_src_table=True deduplicate_dst_table=False insert_unique_blocks=False +table_a_b +count 10 +table_when_b_even +count 10 +0 +0 +table_a_b +count 10 +table_when_b_even +count 20 +0 +0 +OK + +Test case 44: insert_method=InsertValues engine=MergeTree use_insert_token=True single_thread=False deduplicate_src_table=False deduplicate_dst_table=True insert_unique_blocks=True +table_a_b +count 10 +table_when_b_even +count 5 +0 +0 +table_a_b +count 20 +table_when_b_even +count 5 +0 +0 +OK + +Test case 45: insert_method=InsertValues engine=MergeTree use_insert_token=True single_thread=False deduplicate_src_table=False deduplicate_dst_table=True insert_unique_blocks=False +table_a_b +count 10 +table_when_b_even +count 10 +0 +0 +table_a_b +count 20 +table_when_b_even +count 10 +0 +0 +OK + +Test case 46: insert_method=InsertValues engine=MergeTree use_insert_token=True single_thread=False deduplicate_src_table=False deduplicate_dst_table=False insert_unique_blocks=True +table_a_b +count 10 +table_when_b_even +count 5 +0 +0 +table_a_b +count 20 +table_when_b_even +count 10 +0 +0 +OK + +Test case 47: insert_method=InsertValues engine=MergeTree use_insert_token=True single_thread=False deduplicate_src_table=False deduplicate_dst_table=False insert_unique_blocks=False +table_a_b +count 10 +table_when_b_even +count 10 +0 +0 +table_a_b +count 20 +table_when_b_even +count 20 +0 +0 +OK + +Test case 48: insert_method=InsertValues engine=MergeTree use_insert_token=False single_thread=True deduplicate_src_table=True deduplicate_dst_table=True insert_unique_blocks=True +table_a_b +count 10 +table_when_b_even +count 5 +0 +0 +table_a_b +count 10 +table_when_b_even +count 5 +0 +0 +OK + +Test case 49: insert_method=InsertValues engine=MergeTree use_insert_token=False single_thread=True deduplicate_src_table=True deduplicate_dst_table=True insert_unique_blocks=False +table_a_b +count 1 +table_when_b_even +count 1 +0 +0 +table_a_b +count 1 +table_when_b_even +count 1 +0 +0 +OK + +Test case 50: insert_method=InsertValues engine=MergeTree use_insert_token=False single_thread=True deduplicate_src_table=True deduplicate_dst_table=False insert_unique_blocks=True +table_a_b +count 10 +table_when_b_even +count 5 +0 +0 +table_a_b +count 10 +table_when_b_even +count 10 +0 +0 +OK + +Test case 51: insert_method=InsertValues engine=MergeTree use_insert_token=False single_thread=True deduplicate_src_table=True deduplicate_dst_table=False insert_unique_blocks=False +table_a_b +count 1 +table_when_b_even +count 10 +0 +0 +table_a_b +count 1 +table_when_b_even +count 20 +0 +0 +OK + +Test case 52: insert_method=InsertValues engine=MergeTree use_insert_token=False single_thread=True deduplicate_src_table=False deduplicate_dst_table=True insert_unique_blocks=True +table_a_b +count 10 +table_when_b_even +count 5 +0 +0 +table_a_b +count 20 +table_when_b_even +count 5 +0 +0 +OK + +Test case 53: insert_method=InsertValues engine=MergeTree use_insert_token=False single_thread=True deduplicate_src_table=False deduplicate_dst_table=True insert_unique_blocks=False +table_a_b +count 10 +table_when_b_even +count 1 +0 +0 +table_a_b +count 20 +table_when_b_even +count 1 +0 +0 +OK + +Test case 54: insert_method=InsertValues engine=MergeTree use_insert_token=False single_thread=True deduplicate_src_table=False deduplicate_dst_table=False insert_unique_blocks=True +table_a_b +count 10 +table_when_b_even +count 5 +0 +0 +table_a_b +count 20 +table_when_b_even +count 10 +0 +0 +OK + +Test case 55: insert_method=InsertValues engine=MergeTree use_insert_token=False single_thread=True deduplicate_src_table=False deduplicate_dst_table=False insert_unique_blocks=False +table_a_b +count 10 +table_when_b_even +count 10 +0 +0 +table_a_b +count 20 +table_when_b_even +count 20 +0 +0 +OK + +Test case 56: insert_method=InsertValues engine=MergeTree use_insert_token=False single_thread=False deduplicate_src_table=True deduplicate_dst_table=True insert_unique_blocks=True +table_a_b +count 10 +table_when_b_even +count 5 +0 +0 +table_a_b +count 10 +table_when_b_even +count 5 +0 +0 +OK + +Test case 57: insert_method=InsertValues engine=MergeTree use_insert_token=False single_thread=False deduplicate_src_table=True deduplicate_dst_table=True insert_unique_blocks=False +table_a_b +count 1 +table_when_b_even +count 1 +0 +0 +table_a_b +count 1 +table_when_b_even +count 1 +0 +0 +OK + +Test case 58: insert_method=InsertValues engine=MergeTree use_insert_token=False single_thread=False deduplicate_src_table=True deduplicate_dst_table=False insert_unique_blocks=True +table_a_b +count 10 +table_when_b_even +count 5 +0 +0 +table_a_b +count 10 +table_when_b_even +count 10 +0 +0 +OK + +Test case 59: insert_method=InsertValues engine=MergeTree use_insert_token=False single_thread=False deduplicate_src_table=True deduplicate_dst_table=False insert_unique_blocks=False +table_a_b +count 1 +table_when_b_even +count 10 +0 +0 +table_a_b +count 1 +table_when_b_even +count 20 +0 +0 +OK + +Test case 60: insert_method=InsertValues engine=MergeTree use_insert_token=False single_thread=False deduplicate_src_table=False deduplicate_dst_table=True insert_unique_blocks=True +table_a_b +count 10 +table_when_b_even +count 5 +0 +0 +table_a_b +count 20 +table_when_b_even +count 5 +0 +0 +OK + +Test case 61: insert_method=InsertValues engine=MergeTree use_insert_token=False single_thread=False deduplicate_src_table=False deduplicate_dst_table=True insert_unique_blocks=False +table_a_b +count 10 +table_when_b_even +count 1 +0 +0 +table_a_b +count 20 +table_when_b_even +count 1 +0 +0 +OK + +Test case 62: insert_method=InsertValues engine=MergeTree use_insert_token=False single_thread=False deduplicate_src_table=False deduplicate_dst_table=False insert_unique_blocks=True +table_a_b +count 10 +table_when_b_even +count 5 +0 +0 +table_a_b +count 20 +table_when_b_even +count 10 +0 +0 +OK + +Test case 63: insert_method=InsertValues engine=MergeTree use_insert_token=False single_thread=False deduplicate_src_table=False deduplicate_dst_table=False insert_unique_blocks=False +table_a_b +count 10 +table_when_b_even +count 10 +0 +0 +table_a_b +count 20 +table_when_b_even +count 20 +0 +0 +OK + +All cases executed diff --git a/tests/queries/0_stateless/03008_deduplication_insert_several_blocks_nonreplicated.sh b/tests/queries/0_stateless/03008_deduplication_insert_several_blocks_nonreplicated.sh new file mode 100755 index 00000000000..0791c7566f9 --- /dev/null +++ b/tests/queries/0_stateless/03008_deduplication_insert_several_blocks_nonreplicated.sh @@ -0,0 +1,59 @@ +#!/usr/bin/env bash +# Tags: long, no-fasttest, no-parallel, no-asan + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +ENGINE="MergeTree" + +RUN_ONLY="" +#RUN_ONLY="Test case 52: insert_method=InsertSelect engine=MergeTree use_insert_token=False single_thread=True deduplicate_src_table=False deduplicate_dst_table=True insert_unique_blocks=True" + +i=0 +for insert_method in "InsertSelect" "InsertValues"; do + for use_insert_token in "True" "False"; do + for single_thread in "True" "False"; do + for deduplicate_src_table in "True" "False"; do + for deduplicate_dst_table in "True" "False"; do + for insert_unique_blocks in "True" "False"; do + + THIS_RUN="Test case $i:" + THIS_RUN+=" insert_method=$insert_method" + THIS_RUN+=" engine=$ENGINE" + THIS_RUN+=" use_insert_token=$use_insert_token" + THIS_RUN+=" single_thread=$single_thread" + THIS_RUN+=" deduplicate_src_table=$deduplicate_src_table" + THIS_RUN+=" deduplicate_dst_table=$deduplicate_dst_table" + THIS_RUN+=" insert_unique_blocks=$insert_unique_blocks" + + i=$((i+1)) + + echo + if [ -n "$RUN_ONLY" ] && [ "$RUN_ONLY" != "$THIS_RUN" ]; then + echo "skip $THIS_RUN" + continue + fi + echo "$THIS_RUN" + + $CLICKHOUSE_CLIENT --max_insert_block_size 1 -nmq " + $(python3 $CURDIR/03008_deduplication.python insert_several_blocks_into_table \ + --insert-method $insert_method \ + --table-engine $ENGINE \ + --use-insert-token $use_insert_token \ + --single-thread $single_thread \ + --deduplicate-src-table $deduplicate_src_table \ + --deduplicate-dst-table $deduplicate_dst_table \ + --insert-unique-blocks $insert_unique_blocks \ + --get-logs false \ + ) + " && echo OK || echo FAIL + done + done + done + done + done +done + +echo +echo "All cases executed" diff --git a/tests/queries/0_stateless/03008_deduplication_insert_several_blocks_replicated.reference b/tests/queries/0_stateless/03008_deduplication_insert_several_blocks_replicated.reference new file mode 100644 index 00000000000..c815324b455 --- /dev/null +++ b/tests/queries/0_stateless/03008_deduplication_insert_several_blocks_replicated.reference @@ -0,0 +1,962 @@ + +Test case 0: insert_method=InsertSelect engine=ReplicatedMergeTree use_insert_token=True single_thread=True deduplicate_src_table=True deduplicate_dst_table=True insert_unique_blocks=True +table_a_b +count 10 +table_when_b_even +count 5 +0 +0 +table_a_b +count 10 +table_when_b_even +count 5 +0 +0 +OK + +Test case 1: insert_method=InsertSelect engine=ReplicatedMergeTree use_insert_token=True single_thread=True deduplicate_src_table=True deduplicate_dst_table=True insert_unique_blocks=False +table_a_b +count 10 +table_when_b_even +count 10 +0 +0 +table_a_b +count 10 +table_when_b_even +count 10 +0 +0 +OK + +Test case 2: insert_method=InsertSelect engine=ReplicatedMergeTree use_insert_token=True single_thread=True deduplicate_src_table=True deduplicate_dst_table=False insert_unique_blocks=True +table_a_b +count 10 +table_when_b_even +count 5 +0 +0 +table_a_b +count 10 +table_when_b_even +count 10 +0 +0 +OK + +Test case 3: insert_method=InsertSelect engine=ReplicatedMergeTree use_insert_token=True single_thread=True deduplicate_src_table=True deduplicate_dst_table=False insert_unique_blocks=False +table_a_b +count 10 +table_when_b_even +count 10 +0 +0 +table_a_b +count 10 +table_when_b_even +count 20 +0 +0 +OK + +Test case 4: insert_method=InsertSelect engine=ReplicatedMergeTree use_insert_token=True single_thread=True deduplicate_src_table=False deduplicate_dst_table=True insert_unique_blocks=True +table_a_b +count 10 +table_when_b_even +count 5 +0 +0 +table_a_b +count 20 +table_when_b_even +count 5 +0 +0 +OK + +Test case 5: insert_method=InsertSelect engine=ReplicatedMergeTree use_insert_token=True single_thread=True deduplicate_src_table=False deduplicate_dst_table=True insert_unique_blocks=False +table_a_b +count 10 +table_when_b_even +count 10 +0 +0 +table_a_b +count 20 +table_when_b_even +count 10 +0 +0 +OK + +Test case 6: insert_method=InsertSelect engine=ReplicatedMergeTree use_insert_token=True single_thread=True deduplicate_src_table=False deduplicate_dst_table=False insert_unique_blocks=True +table_a_b +count 10 +table_when_b_even +count 5 +0 +0 +table_a_b +count 20 +table_when_b_even +count 10 +0 +0 +OK + +Test case 7: insert_method=InsertSelect engine=ReplicatedMergeTree use_insert_token=True single_thread=True deduplicate_src_table=False deduplicate_dst_table=False insert_unique_blocks=False +table_a_b +count 10 +table_when_b_even +count 10 +0 +0 +table_a_b +count 20 +table_when_b_even +count 20 +0 +0 +OK + +Test case 8: insert_method=InsertSelect engine=ReplicatedMergeTree use_insert_token=True single_thread=False deduplicate_src_table=True deduplicate_dst_table=True insert_unique_blocks=True +table_a_b +count 10 +table_when_b_even +count 5 +0 +0 +table_a_b +count 10 +table_when_b_even +count 5 +0 +0 +OK + +Test case 9: insert_method=InsertSelect engine=ReplicatedMergeTree use_insert_token=True single_thread=False deduplicate_src_table=True deduplicate_dst_table=True insert_unique_blocks=False +table_a_b +count 10 +table_when_b_even +count 10 +0 +0 +table_a_b +count 10 +table_when_b_even +count 10 +0 +0 +OK + +Test case 10: insert_method=InsertSelect engine=ReplicatedMergeTree use_insert_token=True single_thread=False deduplicate_src_table=True deduplicate_dst_table=False insert_unique_blocks=True +table_a_b +count 10 +table_when_b_even +count 5 +0 +0 +table_a_b +count 10 +table_when_b_even +count 10 +0 +0 +OK + +Test case 11: insert_method=InsertSelect engine=ReplicatedMergeTree use_insert_token=True single_thread=False deduplicate_src_table=True deduplicate_dst_table=False insert_unique_blocks=False +table_a_b +count 10 +table_when_b_even +count 10 +0 +0 +table_a_b +count 10 +table_when_b_even +count 20 +0 +0 +OK + +Test case 12: insert_method=InsertSelect engine=ReplicatedMergeTree use_insert_token=True single_thread=False deduplicate_src_table=False deduplicate_dst_table=True insert_unique_blocks=True +table_a_b +count 10 +table_when_b_even +count 5 +0 +0 +table_a_b +count 20 +table_when_b_even +count 5 +0 +0 +OK + +Test case 13: insert_method=InsertSelect engine=ReplicatedMergeTree use_insert_token=True single_thread=False deduplicate_src_table=False deduplicate_dst_table=True insert_unique_blocks=False +table_a_b +count 10 +table_when_b_even +count 10 +0 +0 +table_a_b +count 20 +table_when_b_even +count 10 +0 +0 +OK + +Test case 14: insert_method=InsertSelect engine=ReplicatedMergeTree use_insert_token=True single_thread=False deduplicate_src_table=False deduplicate_dst_table=False insert_unique_blocks=True +table_a_b +count 10 +table_when_b_even +count 5 +0 +0 +table_a_b +count 20 +table_when_b_even +count 10 +0 +0 +OK + +Test case 15: insert_method=InsertSelect engine=ReplicatedMergeTree use_insert_token=True single_thread=False deduplicate_src_table=False deduplicate_dst_table=False insert_unique_blocks=False +table_a_b +count 10 +table_when_b_even +count 10 +0 +0 +table_a_b +count 20 +table_when_b_even +count 20 +0 +0 +OK + +Test case 16: insert_method=InsertSelect engine=ReplicatedMergeTree use_insert_token=False single_thread=True deduplicate_src_table=True deduplicate_dst_table=True insert_unique_blocks=True +table_a_b +count 10 +table_when_b_even +count 5 +0 +0 +table_a_b +count 10 +table_when_b_even +count 5 +0 +0 +OK + +Test case 17: insert_method=InsertSelect engine=ReplicatedMergeTree use_insert_token=False single_thread=True deduplicate_src_table=True deduplicate_dst_table=True insert_unique_blocks=False +table_a_b +count 1 +table_when_b_even +count 1 +0 +0 +table_a_b +count 1 +table_when_b_even +count 1 +0 +0 +OK + +Test case 18: insert_method=InsertSelect engine=ReplicatedMergeTree use_insert_token=False single_thread=True deduplicate_src_table=True deduplicate_dst_table=False insert_unique_blocks=True +table_a_b +count 10 +table_when_b_even +count 5 +0 +0 +table_a_b +count 10 +table_when_b_even +count 10 +0 +0 +OK + +Test case 19: insert_method=InsertSelect engine=ReplicatedMergeTree use_insert_token=False single_thread=True deduplicate_src_table=True deduplicate_dst_table=False insert_unique_blocks=False +table_a_b +count 1 +table_when_b_even +count 10 +0 +0 +table_a_b +count 1 +table_when_b_even +count 20 +0 +0 +OK + +Test case 20: insert_method=InsertSelect engine=ReplicatedMergeTree use_insert_token=False single_thread=True deduplicate_src_table=False deduplicate_dst_table=True insert_unique_blocks=True +table_a_b +count 10 +table_when_b_even +count 5 +0 +0 +table_a_b +count 20 +table_when_b_even +count 5 +0 +0 +OK + +Test case 21: insert_method=InsertSelect engine=ReplicatedMergeTree use_insert_token=False single_thread=True deduplicate_src_table=False deduplicate_dst_table=True insert_unique_blocks=False +table_a_b +count 10 +table_when_b_even +count 1 +0 +0 +table_a_b +count 20 +table_when_b_even +count 1 +0 +0 +OK + +Test case 22: insert_method=InsertSelect engine=ReplicatedMergeTree use_insert_token=False single_thread=True deduplicate_src_table=False deduplicate_dst_table=False insert_unique_blocks=True +table_a_b +count 10 +table_when_b_even +count 5 +0 +0 +table_a_b +count 20 +table_when_b_even +count 10 +0 +0 +OK + +Test case 23: insert_method=InsertSelect engine=ReplicatedMergeTree use_insert_token=False single_thread=True deduplicate_src_table=False deduplicate_dst_table=False insert_unique_blocks=False +table_a_b +count 10 +table_when_b_even +count 10 +0 +0 +table_a_b +count 20 +table_when_b_even +count 20 +0 +0 +OK + +Test case 24: insert_method=InsertSelect engine=ReplicatedMergeTree use_insert_token=False single_thread=False deduplicate_src_table=True deduplicate_dst_table=True insert_unique_blocks=True +table_a_b +count 10 +table_when_b_even +count 5 +0 +0 +table_a_b +count 10 +table_when_b_even +count 5 +0 +0 +OK + +Test case 25: insert_method=InsertSelect engine=ReplicatedMergeTree use_insert_token=False single_thread=False deduplicate_src_table=True deduplicate_dst_table=True insert_unique_blocks=False +table_a_b +count 1 +table_when_b_even +count 1 +0 +0 +table_a_b +count 1 +table_when_b_even +count 1 +0 +0 +OK + +Test case 26: insert_method=InsertSelect engine=ReplicatedMergeTree use_insert_token=False single_thread=False deduplicate_src_table=True deduplicate_dst_table=False insert_unique_blocks=True +table_a_b +count 10 +table_when_b_even +count 5 +0 +0 +table_a_b +count 10 +table_when_b_even +count 10 +0 +0 +OK + +Test case 27: insert_method=InsertSelect engine=ReplicatedMergeTree use_insert_token=False single_thread=False deduplicate_src_table=True deduplicate_dst_table=False insert_unique_blocks=False +table_a_b +count 1 +table_when_b_even +count 10 +0 +0 +table_a_b +count 1 +table_when_b_even +count 20 +0 +0 +OK + +Test case 28: insert_method=InsertSelect engine=ReplicatedMergeTree use_insert_token=False single_thread=False deduplicate_src_table=False deduplicate_dst_table=True insert_unique_blocks=True +table_a_b +count 10 +table_when_b_even +count 5 +0 +0 +table_a_b +count 20 +table_when_b_even +count 5 +0 +0 +OK + +Test case 29: insert_method=InsertSelect engine=ReplicatedMergeTree use_insert_token=False single_thread=False deduplicate_src_table=False deduplicate_dst_table=True insert_unique_blocks=False +table_a_b +count 10 +table_when_b_even +count 1 +0 +0 +table_a_b +count 20 +table_when_b_even +count 1 +0 +0 +OK + +Test case 30: insert_method=InsertSelect engine=ReplicatedMergeTree use_insert_token=False single_thread=False deduplicate_src_table=False deduplicate_dst_table=False insert_unique_blocks=True +table_a_b +count 10 +table_when_b_even +count 5 +0 +0 +table_a_b +count 20 +table_when_b_even +count 10 +0 +0 +OK + +Test case 31: insert_method=InsertSelect engine=ReplicatedMergeTree use_insert_token=False single_thread=False deduplicate_src_table=False deduplicate_dst_table=False insert_unique_blocks=False +table_a_b +count 10 +table_when_b_even +count 10 +0 +0 +table_a_b +count 20 +table_when_b_even +count 20 +0 +0 +OK + +Test case 32: insert_method=InsertValues engine=ReplicatedMergeTree use_insert_token=True single_thread=True deduplicate_src_table=True deduplicate_dst_table=True insert_unique_blocks=True +table_a_b +count 10 +table_when_b_even +count 5 +0 +0 +table_a_b +count 10 +table_when_b_even +count 5 +0 +0 +OK + +Test case 33: insert_method=InsertValues engine=ReplicatedMergeTree use_insert_token=True single_thread=True deduplicate_src_table=True deduplicate_dst_table=True insert_unique_blocks=False +table_a_b +count 10 +table_when_b_even +count 10 +0 +0 +table_a_b +count 10 +table_when_b_even +count 10 +0 +0 +OK + +Test case 34: insert_method=InsertValues engine=ReplicatedMergeTree use_insert_token=True single_thread=True deduplicate_src_table=True deduplicate_dst_table=False insert_unique_blocks=True +table_a_b +count 10 +table_when_b_even +count 5 +0 +0 +table_a_b +count 10 +table_when_b_even +count 10 +0 +0 +OK + +Test case 35: insert_method=InsertValues engine=ReplicatedMergeTree use_insert_token=True single_thread=True deduplicate_src_table=True deduplicate_dst_table=False insert_unique_blocks=False +table_a_b +count 10 +table_when_b_even +count 10 +0 +0 +table_a_b +count 10 +table_when_b_even +count 20 +0 +0 +OK + +Test case 36: insert_method=InsertValues engine=ReplicatedMergeTree use_insert_token=True single_thread=True deduplicate_src_table=False deduplicate_dst_table=True insert_unique_blocks=True +table_a_b +count 10 +table_when_b_even +count 5 +0 +0 +table_a_b +count 20 +table_when_b_even +count 5 +0 +0 +OK + +Test case 37: insert_method=InsertValues engine=ReplicatedMergeTree use_insert_token=True single_thread=True deduplicate_src_table=False deduplicate_dst_table=True insert_unique_blocks=False +table_a_b +count 10 +table_when_b_even +count 10 +0 +0 +table_a_b +count 20 +table_when_b_even +count 10 +0 +0 +OK + +Test case 38: insert_method=InsertValues engine=ReplicatedMergeTree use_insert_token=True single_thread=True deduplicate_src_table=False deduplicate_dst_table=False insert_unique_blocks=True +table_a_b +count 10 +table_when_b_even +count 5 +0 +0 +table_a_b +count 20 +table_when_b_even +count 10 +0 +0 +OK + +Test case 39: insert_method=InsertValues engine=ReplicatedMergeTree use_insert_token=True single_thread=True deduplicate_src_table=False deduplicate_dst_table=False insert_unique_blocks=False +table_a_b +count 10 +table_when_b_even +count 10 +0 +0 +table_a_b +count 20 +table_when_b_even +count 20 +0 +0 +OK + +Test case 40: insert_method=InsertValues engine=ReplicatedMergeTree use_insert_token=True single_thread=False deduplicate_src_table=True deduplicate_dst_table=True insert_unique_blocks=True +table_a_b +count 10 +table_when_b_even +count 5 +0 +0 +table_a_b +count 10 +table_when_b_even +count 5 +0 +0 +OK + +Test case 41: insert_method=InsertValues engine=ReplicatedMergeTree use_insert_token=True single_thread=False deduplicate_src_table=True deduplicate_dst_table=True insert_unique_blocks=False +table_a_b +count 10 +table_when_b_even +count 10 +0 +0 +table_a_b +count 10 +table_when_b_even +count 10 +0 +0 +OK + +Test case 42: insert_method=InsertValues engine=ReplicatedMergeTree use_insert_token=True single_thread=False deduplicate_src_table=True deduplicate_dst_table=False insert_unique_blocks=True +table_a_b +count 10 +table_when_b_even +count 5 +0 +0 +table_a_b +count 10 +table_when_b_even +count 10 +0 +0 +OK + +Test case 43: insert_method=InsertValues engine=ReplicatedMergeTree use_insert_token=True single_thread=False deduplicate_src_table=True deduplicate_dst_table=False insert_unique_blocks=False +table_a_b +count 10 +table_when_b_even +count 10 +0 +0 +table_a_b +count 10 +table_when_b_even +count 20 +0 +0 +OK + +Test case 44: insert_method=InsertValues engine=ReplicatedMergeTree use_insert_token=True single_thread=False deduplicate_src_table=False deduplicate_dst_table=True insert_unique_blocks=True +table_a_b +count 10 +table_when_b_even +count 5 +0 +0 +table_a_b +count 20 +table_when_b_even +count 5 +0 +0 +OK + +Test case 45: insert_method=InsertValues engine=ReplicatedMergeTree use_insert_token=True single_thread=False deduplicate_src_table=False deduplicate_dst_table=True insert_unique_blocks=False +table_a_b +count 10 +table_when_b_even +count 10 +0 +0 +table_a_b +count 20 +table_when_b_even +count 10 +0 +0 +OK + +Test case 46: insert_method=InsertValues engine=ReplicatedMergeTree use_insert_token=True single_thread=False deduplicate_src_table=False deduplicate_dst_table=False insert_unique_blocks=True +table_a_b +count 10 +table_when_b_even +count 5 +0 +0 +table_a_b +count 20 +table_when_b_even +count 10 +0 +0 +OK + +Test case 47: insert_method=InsertValues engine=ReplicatedMergeTree use_insert_token=True single_thread=False deduplicate_src_table=False deduplicate_dst_table=False insert_unique_blocks=False +table_a_b +count 10 +table_when_b_even +count 10 +0 +0 +table_a_b +count 20 +table_when_b_even +count 20 +0 +0 +OK + +Test case 48: insert_method=InsertValues engine=ReplicatedMergeTree use_insert_token=False single_thread=True deduplicate_src_table=True deduplicate_dst_table=True insert_unique_blocks=True +table_a_b +count 10 +table_when_b_even +count 5 +0 +0 +table_a_b +count 10 +table_when_b_even +count 5 +0 +0 +OK + +Test case 49: insert_method=InsertValues engine=ReplicatedMergeTree use_insert_token=False single_thread=True deduplicate_src_table=True deduplicate_dst_table=True insert_unique_blocks=False +table_a_b +count 1 +table_when_b_even +count 1 +0 +0 +table_a_b +count 1 +table_when_b_even +count 1 +0 +0 +OK + +Test case 50: insert_method=InsertValues engine=ReplicatedMergeTree use_insert_token=False single_thread=True deduplicate_src_table=True deduplicate_dst_table=False insert_unique_blocks=True +table_a_b +count 10 +table_when_b_even +count 5 +0 +0 +table_a_b +count 10 +table_when_b_even +count 10 +0 +0 +OK + +Test case 51: insert_method=InsertValues engine=ReplicatedMergeTree use_insert_token=False single_thread=True deduplicate_src_table=True deduplicate_dst_table=False insert_unique_blocks=False +table_a_b +count 1 +table_when_b_even +count 10 +0 +0 +table_a_b +count 1 +table_when_b_even +count 20 +0 +0 +OK + +Test case 52: insert_method=InsertValues engine=ReplicatedMergeTree use_insert_token=False single_thread=True deduplicate_src_table=False deduplicate_dst_table=True insert_unique_blocks=True +table_a_b +count 10 +table_when_b_even +count 5 +0 +0 +table_a_b +count 20 +table_when_b_even +count 5 +0 +0 +OK + +Test case 53: insert_method=InsertValues engine=ReplicatedMergeTree use_insert_token=False single_thread=True deduplicate_src_table=False deduplicate_dst_table=True insert_unique_blocks=False +table_a_b +count 10 +table_when_b_even +count 1 +0 +0 +table_a_b +count 20 +table_when_b_even +count 1 +0 +0 +OK + +Test case 54: insert_method=InsertValues engine=ReplicatedMergeTree use_insert_token=False single_thread=True deduplicate_src_table=False deduplicate_dst_table=False insert_unique_blocks=True +table_a_b +count 10 +table_when_b_even +count 5 +0 +0 +table_a_b +count 20 +table_when_b_even +count 10 +0 +0 +OK + +Test case 55: insert_method=InsertValues engine=ReplicatedMergeTree use_insert_token=False single_thread=True deduplicate_src_table=False deduplicate_dst_table=False insert_unique_blocks=False +table_a_b +count 10 +table_when_b_even +count 10 +0 +0 +table_a_b +count 20 +table_when_b_even +count 20 +0 +0 +OK + +Test case 56: insert_method=InsertValues engine=ReplicatedMergeTree use_insert_token=False single_thread=False deduplicate_src_table=True deduplicate_dst_table=True insert_unique_blocks=True +table_a_b +count 10 +table_when_b_even +count 5 +0 +0 +table_a_b +count 10 +table_when_b_even +count 5 +0 +0 +OK + +Test case 57: insert_method=InsertValues engine=ReplicatedMergeTree use_insert_token=False single_thread=False deduplicate_src_table=True deduplicate_dst_table=True insert_unique_blocks=False +table_a_b +count 1 +table_when_b_even +count 1 +0 +0 +table_a_b +count 1 +table_when_b_even +count 1 +0 +0 +OK + +Test case 58: insert_method=InsertValues engine=ReplicatedMergeTree use_insert_token=False single_thread=False deduplicate_src_table=True deduplicate_dst_table=False insert_unique_blocks=True +table_a_b +count 10 +table_when_b_even +count 5 +0 +0 +table_a_b +count 10 +table_when_b_even +count 10 +0 +0 +OK + +Test case 59: insert_method=InsertValues engine=ReplicatedMergeTree use_insert_token=False single_thread=False deduplicate_src_table=True deduplicate_dst_table=False insert_unique_blocks=False +table_a_b +count 1 +table_when_b_even +count 10 +0 +0 +table_a_b +count 1 +table_when_b_even +count 20 +0 +0 +OK + +Test case 60: insert_method=InsertValues engine=ReplicatedMergeTree use_insert_token=False single_thread=False deduplicate_src_table=False deduplicate_dst_table=True insert_unique_blocks=True +table_a_b +count 10 +table_when_b_even +count 5 +0 +0 +table_a_b +count 20 +table_when_b_even +count 5 +0 +0 +OK + +Test case 61: insert_method=InsertValues engine=ReplicatedMergeTree use_insert_token=False single_thread=False deduplicate_src_table=False deduplicate_dst_table=True insert_unique_blocks=False +table_a_b +count 10 +table_when_b_even +count 1 +0 +0 +table_a_b +count 20 +table_when_b_even +count 1 +0 +0 +OK + +Test case 62: insert_method=InsertValues engine=ReplicatedMergeTree use_insert_token=False single_thread=False deduplicate_src_table=False deduplicate_dst_table=False insert_unique_blocks=True +table_a_b +count 10 +table_when_b_even +count 5 +0 +0 +table_a_b +count 20 +table_when_b_even +count 10 +0 +0 +OK + +Test case 63: insert_method=InsertValues engine=ReplicatedMergeTree use_insert_token=False single_thread=False deduplicate_src_table=False deduplicate_dst_table=False insert_unique_blocks=False +table_a_b +count 10 +table_when_b_even +count 10 +0 +0 +table_a_b +count 20 +table_when_b_even +count 20 +0 +0 +OK + +All cases executed diff --git a/tests/queries/0_stateless/03008_deduplication_insert_several_blocks_replicated.sh b/tests/queries/0_stateless/03008_deduplication_insert_several_blocks_replicated.sh new file mode 100755 index 00000000000..661dfa5f930 --- /dev/null +++ b/tests/queries/0_stateless/03008_deduplication_insert_several_blocks_replicated.sh @@ -0,0 +1,59 @@ +#!/usr/bin/env bash +# Tags: long, no-fasttest, no-parallel, no-asan + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +ENGINE="ReplicatedMergeTree" + +RUN_ONLY="" +#RUN_ONLY="Test case 52: insert_method=InsertSelect engine=ReplicatedMergeTree use_insert_token=False single_thread=True deduplicate_src_table=False deduplicate_dst_table=True insert_unique_blocks=True" + +i=0 +for insert_method in "InsertSelect" "InsertValues"; do + for use_insert_token in "True" "False"; do + for single_thread in "True" "False"; do + for deduplicate_src_table in "True" "False"; do + for deduplicate_dst_table in "True" "False"; do + for insert_unique_blocks in "True" "False"; do + + THIS_RUN="Test case $i:" + THIS_RUN+=" insert_method=$insert_method" + THIS_RUN+=" engine=$ENGINE" + THIS_RUN+=" use_insert_token=$use_insert_token" + THIS_RUN+=" single_thread=$single_thread" + THIS_RUN+=" deduplicate_src_table=$deduplicate_src_table" + THIS_RUN+=" deduplicate_dst_table=$deduplicate_dst_table" + THIS_RUN+=" insert_unique_blocks=$insert_unique_blocks" + + i=$((i+1)) + + echo + if [ -n "$RUN_ONLY" ] && [ "$RUN_ONLY" != "$THIS_RUN" ]; then + echo "skip $THIS_RUN" + continue + fi + echo "$THIS_RUN" + + $CLICKHOUSE_CLIENT --max_insert_block_size 1 -nmq " + $(python3 $CURDIR/03008_deduplication.python insert_several_blocks_into_table \ + --insert-method $insert_method \ + --table-engine $ENGINE \ + --use-insert-token $use_insert_token \ + --single-thread $single_thread \ + --deduplicate-src-table $deduplicate_src_table \ + --deduplicate-dst-table $deduplicate_dst_table \ + --insert-unique-blocks $insert_unique_blocks \ + --get-logs false \ + ) + " && echo OK || echo FAIL + done + done + done + done + done +done + +echo +echo "All cases executed" diff --git a/tests/queries/0_stateless/03008_deduplication_mv_generates_several_blocks_nonreplicated.reference b/tests/queries/0_stateless/03008_deduplication_mv_generates_several_blocks_nonreplicated.reference new file mode 100644 index 00000000000..6e76ec46aa8 --- /dev/null +++ b/tests/queries/0_stateless/03008_deduplication_mv_generates_several_blocks_nonreplicated.reference @@ -0,0 +1,962 @@ + +Test case 0: insert_method=InsertSelect engine=MergeTree use_insert_token=True single_thread=True deduplicate_src_table=True deduplicate_dst_table=True insert_unique_blocks=True +table_a_b +count 5 +table_when_b_even_and_joined +count 9 +0 +0 +table_a_b +count 5 +table_when_b_even_and_joined +count 9 +0 +0 +OK + +Test case 1: insert_method=InsertSelect engine=MergeTree use_insert_token=True single_thread=True deduplicate_src_table=True deduplicate_dst_table=True insert_unique_blocks=False +table_a_b +count 5 +table_when_b_even_and_joined +count 10 +0 +0 +table_a_b +count 5 +table_when_b_even_and_joined +count 10 +0 +0 +OK + +Test case 2: insert_method=InsertSelect engine=MergeTree use_insert_token=True single_thread=True deduplicate_src_table=True deduplicate_dst_table=False insert_unique_blocks=True +table_a_b +count 5 +table_when_b_even_and_joined +count 9 +0 +0 +table_a_b +count 5 +table_when_b_even_and_joined +count 18 +0 +0 +OK + +Test case 3: insert_method=InsertSelect engine=MergeTree use_insert_token=True single_thread=True deduplicate_src_table=True deduplicate_dst_table=False insert_unique_blocks=False +table_a_b +count 5 +table_when_b_even_and_joined +count 10 +0 +0 +table_a_b +count 5 +table_when_b_even_and_joined +count 20 +0 +0 +OK + +Test case 4: insert_method=InsertSelect engine=MergeTree use_insert_token=True single_thread=True deduplicate_src_table=False deduplicate_dst_table=True insert_unique_blocks=True +table_a_b +count 5 +table_when_b_even_and_joined +count 9 +0 +0 +table_a_b +count 10 +table_when_b_even_and_joined +count 9 +0 +0 +OK + +Test case 5: insert_method=InsertSelect engine=MergeTree use_insert_token=True single_thread=True deduplicate_src_table=False deduplicate_dst_table=True insert_unique_blocks=False +table_a_b +count 5 +table_when_b_even_and_joined +count 10 +0 +0 +table_a_b +count 10 +table_when_b_even_and_joined +count 10 +0 +0 +OK + +Test case 6: insert_method=InsertSelect engine=MergeTree use_insert_token=True single_thread=True deduplicate_src_table=False deduplicate_dst_table=False insert_unique_blocks=True +table_a_b +count 5 +table_when_b_even_and_joined +count 9 +0 +0 +table_a_b +count 10 +table_when_b_even_and_joined +count 18 +0 +0 +OK + +Test case 7: insert_method=InsertSelect engine=MergeTree use_insert_token=True single_thread=True deduplicate_src_table=False deduplicate_dst_table=False insert_unique_blocks=False +table_a_b +count 5 +table_when_b_even_and_joined +count 10 +0 +0 +table_a_b +count 10 +table_when_b_even_and_joined +count 20 +0 +0 +OK + +Test case 8: insert_method=InsertSelect engine=MergeTree use_insert_token=True single_thread=False deduplicate_src_table=True deduplicate_dst_table=True insert_unique_blocks=True +table_a_b +count 5 +table_when_b_even_and_joined +count 9 +0 +0 +table_a_b +count 5 +table_when_b_even_and_joined +count 9 +0 +0 +OK + +Test case 9: insert_method=InsertSelect engine=MergeTree use_insert_token=True single_thread=False deduplicate_src_table=True deduplicate_dst_table=True insert_unique_blocks=False +table_a_b +count 5 +table_when_b_even_and_joined +count 10 +0 +0 +table_a_b +count 5 +table_when_b_even_and_joined +count 10 +0 +0 +OK + +Test case 10: insert_method=InsertSelect engine=MergeTree use_insert_token=True single_thread=False deduplicate_src_table=True deduplicate_dst_table=False insert_unique_blocks=True +table_a_b +count 5 +table_when_b_even_and_joined +count 9 +0 +0 +table_a_b +count 5 +table_when_b_even_and_joined +count 18 +0 +0 +OK + +Test case 11: insert_method=InsertSelect engine=MergeTree use_insert_token=True single_thread=False deduplicate_src_table=True deduplicate_dst_table=False insert_unique_blocks=False +table_a_b +count 5 +table_when_b_even_and_joined +count 10 +0 +0 +table_a_b +count 5 +table_when_b_even_and_joined +count 20 +0 +0 +OK + +Test case 12: insert_method=InsertSelect engine=MergeTree use_insert_token=True single_thread=False deduplicate_src_table=False deduplicate_dst_table=True insert_unique_blocks=True +table_a_b +count 5 +table_when_b_even_and_joined +count 9 +0 +0 +table_a_b +count 10 +table_when_b_even_and_joined +count 9 +0 +0 +OK + +Test case 13: insert_method=InsertSelect engine=MergeTree use_insert_token=True single_thread=False deduplicate_src_table=False deduplicate_dst_table=True insert_unique_blocks=False +table_a_b +count 5 +table_when_b_even_and_joined +count 10 +0 +0 +table_a_b +count 10 +table_when_b_even_and_joined +count 10 +0 +0 +OK + +Test case 14: insert_method=InsertSelect engine=MergeTree use_insert_token=True single_thread=False deduplicate_src_table=False deduplicate_dst_table=False insert_unique_blocks=True +table_a_b +count 5 +table_when_b_even_and_joined +count 9 +0 +0 +table_a_b +count 10 +table_when_b_even_and_joined +count 18 +0 +0 +OK + +Test case 15: insert_method=InsertSelect engine=MergeTree use_insert_token=True single_thread=False deduplicate_src_table=False deduplicate_dst_table=False insert_unique_blocks=False +table_a_b +count 5 +table_when_b_even_and_joined +count 10 +0 +0 +table_a_b +count 10 +table_when_b_even_and_joined +count 20 +0 +0 +OK + +Test case 16: insert_method=InsertSelect engine=MergeTree use_insert_token=False single_thread=True deduplicate_src_table=True deduplicate_dst_table=True insert_unique_blocks=True +table_a_b +count 5 +table_when_b_even_and_joined +count 9 +0 +0 +table_a_b +count 5 +table_when_b_even_and_joined +count 9 +0 +0 +OK + +Test case 17: insert_method=InsertSelect engine=MergeTree use_insert_token=False single_thread=True deduplicate_src_table=True deduplicate_dst_table=True insert_unique_blocks=False +table_a_b +count 1 +table_when_b_even_and_joined +count 2 +0 +0 +table_a_b +count 1 +table_when_b_even_and_joined +count 2 +0 +0 +OK + +Test case 18: insert_method=InsertSelect engine=MergeTree use_insert_token=False single_thread=True deduplicate_src_table=True deduplicate_dst_table=False insert_unique_blocks=True +table_a_b +count 5 +table_when_b_even_and_joined +count 9 +0 +0 +table_a_b +count 5 +table_when_b_even_and_joined +count 18 +0 +0 +OK + +Test case 19: insert_method=InsertSelect engine=MergeTree use_insert_token=False single_thread=True deduplicate_src_table=True deduplicate_dst_table=False insert_unique_blocks=False +table_a_b +count 1 +table_when_b_even_and_joined +count 10 +0 +0 +table_a_b +count 1 +table_when_b_even_and_joined +count 20 +0 +0 +OK + +Test case 20: insert_method=InsertSelect engine=MergeTree use_insert_token=False single_thread=True deduplicate_src_table=False deduplicate_dst_table=True insert_unique_blocks=True +table_a_b +count 5 +table_when_b_even_and_joined +count 9 +0 +0 +table_a_b +count 10 +table_when_b_even_and_joined +count 9 +0 +0 +OK + +Test case 21: insert_method=InsertSelect engine=MergeTree use_insert_token=False single_thread=True deduplicate_src_table=False deduplicate_dst_table=True insert_unique_blocks=False +table_a_b +count 5 +table_when_b_even_and_joined +count 2 +0 +0 +table_a_b +count 10 +table_when_b_even_and_joined +count 2 +0 +0 +OK + +Test case 22: insert_method=InsertSelect engine=MergeTree use_insert_token=False single_thread=True deduplicate_src_table=False deduplicate_dst_table=False insert_unique_blocks=True +table_a_b +count 5 +table_when_b_even_and_joined +count 9 +0 +0 +table_a_b +count 10 +table_when_b_even_and_joined +count 18 +0 +0 +OK + +Test case 23: insert_method=InsertSelect engine=MergeTree use_insert_token=False single_thread=True deduplicate_src_table=False deduplicate_dst_table=False insert_unique_blocks=False +table_a_b +count 5 +table_when_b_even_and_joined +count 10 +0 +0 +table_a_b +count 10 +table_when_b_even_and_joined +count 20 +0 +0 +OK + +Test case 24: insert_method=InsertSelect engine=MergeTree use_insert_token=False single_thread=False deduplicate_src_table=True deduplicate_dst_table=True insert_unique_blocks=True +table_a_b +count 5 +table_when_b_even_and_joined +count 9 +0 +0 +table_a_b +count 5 +table_when_b_even_and_joined +count 9 +0 +0 +OK + +Test case 25: insert_method=InsertSelect engine=MergeTree use_insert_token=False single_thread=False deduplicate_src_table=True deduplicate_dst_table=True insert_unique_blocks=False +table_a_b +count 1 +table_when_b_even_and_joined +count 2 +0 +0 +table_a_b +count 1 +table_when_b_even_and_joined +count 2 +0 +0 +OK + +Test case 26: insert_method=InsertSelect engine=MergeTree use_insert_token=False single_thread=False deduplicate_src_table=True deduplicate_dst_table=False insert_unique_blocks=True +table_a_b +count 5 +table_when_b_even_and_joined +count 9 +0 +0 +table_a_b +count 5 +table_when_b_even_and_joined +count 18 +0 +0 +OK + +Test case 27: insert_method=InsertSelect engine=MergeTree use_insert_token=False single_thread=False deduplicate_src_table=True deduplicate_dst_table=False insert_unique_blocks=False +table_a_b +count 1 +table_when_b_even_and_joined +count 10 +0 +0 +table_a_b +count 1 +table_when_b_even_and_joined +count 20 +0 +0 +OK + +Test case 28: insert_method=InsertSelect engine=MergeTree use_insert_token=False single_thread=False deduplicate_src_table=False deduplicate_dst_table=True insert_unique_blocks=True +table_a_b +count 5 +table_when_b_even_and_joined +count 9 +0 +0 +table_a_b +count 10 +table_when_b_even_and_joined +count 9 +0 +0 +OK + +Test case 29: insert_method=InsertSelect engine=MergeTree use_insert_token=False single_thread=False deduplicate_src_table=False deduplicate_dst_table=True insert_unique_blocks=False +table_a_b +count 5 +table_when_b_even_and_joined +count 2 +0 +0 +table_a_b +count 10 +table_when_b_even_and_joined +count 2 +0 +0 +OK + +Test case 30: insert_method=InsertSelect engine=MergeTree use_insert_token=False single_thread=False deduplicate_src_table=False deduplicate_dst_table=False insert_unique_blocks=True +table_a_b +count 5 +table_when_b_even_and_joined +count 9 +0 +0 +table_a_b +count 10 +table_when_b_even_and_joined +count 18 +0 +0 +OK + +Test case 31: insert_method=InsertSelect engine=MergeTree use_insert_token=False single_thread=False deduplicate_src_table=False deduplicate_dst_table=False insert_unique_blocks=False +table_a_b +count 5 +table_when_b_even_and_joined +count 10 +0 +0 +table_a_b +count 10 +table_when_b_even_and_joined +count 20 +0 +0 +OK + +Test case 32: insert_method=InsertValues engine=MergeTree use_insert_token=True single_thread=True deduplicate_src_table=True deduplicate_dst_table=True insert_unique_blocks=True +table_a_b +count 5 +table_when_b_even_and_joined +count 9 +0 +0 +table_a_b +count 5 +table_when_b_even_and_joined +count 9 +0 +0 +OK + +Test case 33: insert_method=InsertValues engine=MergeTree use_insert_token=True single_thread=True deduplicate_src_table=True deduplicate_dst_table=True insert_unique_blocks=False +table_a_b +count 5 +table_when_b_even_and_joined +count 10 +0 +0 +table_a_b +count 5 +table_when_b_even_and_joined +count 10 +0 +0 +OK + +Test case 34: insert_method=InsertValues engine=MergeTree use_insert_token=True single_thread=True deduplicate_src_table=True deduplicate_dst_table=False insert_unique_blocks=True +table_a_b +count 5 +table_when_b_even_and_joined +count 9 +0 +0 +table_a_b +count 5 +table_when_b_even_and_joined +count 18 +0 +0 +OK + +Test case 35: insert_method=InsertValues engine=MergeTree use_insert_token=True single_thread=True deduplicate_src_table=True deduplicate_dst_table=False insert_unique_blocks=False +table_a_b +count 5 +table_when_b_even_and_joined +count 10 +0 +0 +table_a_b +count 5 +table_when_b_even_and_joined +count 20 +0 +0 +OK + +Test case 36: insert_method=InsertValues engine=MergeTree use_insert_token=True single_thread=True deduplicate_src_table=False deduplicate_dst_table=True insert_unique_blocks=True +table_a_b +count 5 +table_when_b_even_and_joined +count 9 +0 +0 +table_a_b +count 10 +table_when_b_even_and_joined +count 9 +0 +0 +OK + +Test case 37: insert_method=InsertValues engine=MergeTree use_insert_token=True single_thread=True deduplicate_src_table=False deduplicate_dst_table=True insert_unique_blocks=False +table_a_b +count 5 +table_when_b_even_and_joined +count 10 +0 +0 +table_a_b +count 10 +table_when_b_even_and_joined +count 10 +0 +0 +OK + +Test case 38: insert_method=InsertValues engine=MergeTree use_insert_token=True single_thread=True deduplicate_src_table=False deduplicate_dst_table=False insert_unique_blocks=True +table_a_b +count 5 +table_when_b_even_and_joined +count 9 +0 +0 +table_a_b +count 10 +table_when_b_even_and_joined +count 18 +0 +0 +OK + +Test case 39: insert_method=InsertValues engine=MergeTree use_insert_token=True single_thread=True deduplicate_src_table=False deduplicate_dst_table=False insert_unique_blocks=False +table_a_b +count 5 +table_when_b_even_and_joined +count 10 +0 +0 +table_a_b +count 10 +table_when_b_even_and_joined +count 20 +0 +0 +OK + +Test case 40: insert_method=InsertValues engine=MergeTree use_insert_token=True single_thread=False deduplicate_src_table=True deduplicate_dst_table=True insert_unique_blocks=True +table_a_b +count 5 +table_when_b_even_and_joined +count 9 +0 +0 +table_a_b +count 5 +table_when_b_even_and_joined +count 9 +0 +0 +OK + +Test case 41: insert_method=InsertValues engine=MergeTree use_insert_token=True single_thread=False deduplicate_src_table=True deduplicate_dst_table=True insert_unique_blocks=False +table_a_b +count 5 +table_when_b_even_and_joined +count 10 +0 +0 +table_a_b +count 5 +table_when_b_even_and_joined +count 10 +0 +0 +OK + +Test case 42: insert_method=InsertValues engine=MergeTree use_insert_token=True single_thread=False deduplicate_src_table=True deduplicate_dst_table=False insert_unique_blocks=True +table_a_b +count 5 +table_when_b_even_and_joined +count 9 +0 +0 +table_a_b +count 5 +table_when_b_even_and_joined +count 18 +0 +0 +OK + +Test case 43: insert_method=InsertValues engine=MergeTree use_insert_token=True single_thread=False deduplicate_src_table=True deduplicate_dst_table=False insert_unique_blocks=False +table_a_b +count 5 +table_when_b_even_and_joined +count 10 +0 +0 +table_a_b +count 5 +table_when_b_even_and_joined +count 20 +0 +0 +OK + +Test case 44: insert_method=InsertValues engine=MergeTree use_insert_token=True single_thread=False deduplicate_src_table=False deduplicate_dst_table=True insert_unique_blocks=True +table_a_b +count 5 +table_when_b_even_and_joined +count 9 +0 +0 +table_a_b +count 10 +table_when_b_even_and_joined +count 9 +0 +0 +OK + +Test case 45: insert_method=InsertValues engine=MergeTree use_insert_token=True single_thread=False deduplicate_src_table=False deduplicate_dst_table=True insert_unique_blocks=False +table_a_b +count 5 +table_when_b_even_and_joined +count 10 +0 +0 +table_a_b +count 10 +table_when_b_even_and_joined +count 10 +0 +0 +OK + +Test case 46: insert_method=InsertValues engine=MergeTree use_insert_token=True single_thread=False deduplicate_src_table=False deduplicate_dst_table=False insert_unique_blocks=True +table_a_b +count 5 +table_when_b_even_and_joined +count 9 +0 +0 +table_a_b +count 10 +table_when_b_even_and_joined +count 18 +0 +0 +OK + +Test case 47: insert_method=InsertValues engine=MergeTree use_insert_token=True single_thread=False deduplicate_src_table=False deduplicate_dst_table=False insert_unique_blocks=False +table_a_b +count 5 +table_when_b_even_and_joined +count 10 +0 +0 +table_a_b +count 10 +table_when_b_even_and_joined +count 20 +0 +0 +OK + +Test case 48: insert_method=InsertValues engine=MergeTree use_insert_token=False single_thread=True deduplicate_src_table=True deduplicate_dst_table=True insert_unique_blocks=True +table_a_b +count 5 +table_when_b_even_and_joined +count 9 +0 +0 +table_a_b +count 5 +table_when_b_even_and_joined +count 9 +0 +0 +OK + +Test case 49: insert_method=InsertValues engine=MergeTree use_insert_token=False single_thread=True deduplicate_src_table=True deduplicate_dst_table=True insert_unique_blocks=False +table_a_b +count 1 +table_when_b_even_and_joined +count 2 +0 +0 +table_a_b +count 1 +table_when_b_even_and_joined +count 2 +0 +0 +OK + +Test case 50: insert_method=InsertValues engine=MergeTree use_insert_token=False single_thread=True deduplicate_src_table=True deduplicate_dst_table=False insert_unique_blocks=True +table_a_b +count 5 +table_when_b_even_and_joined +count 9 +0 +0 +table_a_b +count 5 +table_when_b_even_and_joined +count 18 +0 +0 +OK + +Test case 51: insert_method=InsertValues engine=MergeTree use_insert_token=False single_thread=True deduplicate_src_table=True deduplicate_dst_table=False insert_unique_blocks=False +table_a_b +count 1 +table_when_b_even_and_joined +count 10 +0 +0 +table_a_b +count 1 +table_when_b_even_and_joined +count 20 +0 +0 +OK + +Test case 52: insert_method=InsertValues engine=MergeTree use_insert_token=False single_thread=True deduplicate_src_table=False deduplicate_dst_table=True insert_unique_blocks=True +table_a_b +count 5 +table_when_b_even_and_joined +count 9 +0 +0 +table_a_b +count 10 +table_when_b_even_and_joined +count 9 +0 +0 +OK + +Test case 53: insert_method=InsertValues engine=MergeTree use_insert_token=False single_thread=True deduplicate_src_table=False deduplicate_dst_table=True insert_unique_blocks=False +table_a_b +count 5 +table_when_b_even_and_joined +count 2 +0 +0 +table_a_b +count 10 +table_when_b_even_and_joined +count 2 +0 +0 +OK + +Test case 54: insert_method=InsertValues engine=MergeTree use_insert_token=False single_thread=True deduplicate_src_table=False deduplicate_dst_table=False insert_unique_blocks=True +table_a_b +count 5 +table_when_b_even_and_joined +count 9 +0 +0 +table_a_b +count 10 +table_when_b_even_and_joined +count 18 +0 +0 +OK + +Test case 55: insert_method=InsertValues engine=MergeTree use_insert_token=False single_thread=True deduplicate_src_table=False deduplicate_dst_table=False insert_unique_blocks=False +table_a_b +count 5 +table_when_b_even_and_joined +count 10 +0 +0 +table_a_b +count 10 +table_when_b_even_and_joined +count 20 +0 +0 +OK + +Test case 56: insert_method=InsertValues engine=MergeTree use_insert_token=False single_thread=False deduplicate_src_table=True deduplicate_dst_table=True insert_unique_blocks=True +table_a_b +count 5 +table_when_b_even_and_joined +count 9 +0 +0 +table_a_b +count 5 +table_when_b_even_and_joined +count 9 +0 +0 +OK + +Test case 57: insert_method=InsertValues engine=MergeTree use_insert_token=False single_thread=False deduplicate_src_table=True deduplicate_dst_table=True insert_unique_blocks=False +table_a_b +count 1 +table_when_b_even_and_joined +count 2 +0 +0 +table_a_b +count 1 +table_when_b_even_and_joined +count 2 +0 +0 +OK + +Test case 58: insert_method=InsertValues engine=MergeTree use_insert_token=False single_thread=False deduplicate_src_table=True deduplicate_dst_table=False insert_unique_blocks=True +table_a_b +count 5 +table_when_b_even_and_joined +count 9 +0 +0 +table_a_b +count 5 +table_when_b_even_and_joined +count 18 +0 +0 +OK + +Test case 59: insert_method=InsertValues engine=MergeTree use_insert_token=False single_thread=False deduplicate_src_table=True deduplicate_dst_table=False insert_unique_blocks=False +table_a_b +count 1 +table_when_b_even_and_joined +count 10 +0 +0 +table_a_b +count 1 +table_when_b_even_and_joined +count 20 +0 +0 +OK + +Test case 60: insert_method=InsertValues engine=MergeTree use_insert_token=False single_thread=False deduplicate_src_table=False deduplicate_dst_table=True insert_unique_blocks=True +table_a_b +count 5 +table_when_b_even_and_joined +count 9 +0 +0 +table_a_b +count 10 +table_when_b_even_and_joined +count 9 +0 +0 +OK + +Test case 61: insert_method=InsertValues engine=MergeTree use_insert_token=False single_thread=False deduplicate_src_table=False deduplicate_dst_table=True insert_unique_blocks=False +table_a_b +count 5 +table_when_b_even_and_joined +count 2 +0 +0 +table_a_b +count 10 +table_when_b_even_and_joined +count 2 +0 +0 +OK + +Test case 62: insert_method=InsertValues engine=MergeTree use_insert_token=False single_thread=False deduplicate_src_table=False deduplicate_dst_table=False insert_unique_blocks=True +table_a_b +count 5 +table_when_b_even_and_joined +count 9 +0 +0 +table_a_b +count 10 +table_when_b_even_and_joined +count 18 +0 +0 +OK + +Test case 63: insert_method=InsertValues engine=MergeTree use_insert_token=False single_thread=False deduplicate_src_table=False deduplicate_dst_table=False insert_unique_blocks=False +table_a_b +count 5 +table_when_b_even_and_joined +count 10 +0 +0 +table_a_b +count 10 +table_when_b_even_and_joined +count 20 +0 +0 +OK + +All cases executed diff --git a/tests/queries/0_stateless/03008_deduplication_mv_generates_several_blocks_nonreplicated.sh b/tests/queries/0_stateless/03008_deduplication_mv_generates_several_blocks_nonreplicated.sh new file mode 100755 index 00000000000..10083506af4 --- /dev/null +++ b/tests/queries/0_stateless/03008_deduplication_mv_generates_several_blocks_nonreplicated.sh @@ -0,0 +1,59 @@ +#!/usr/bin/env bash +# Tags: long, no-fasttest, no-parallel, no-asan + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +ENGINE="MergeTree" + +RUN_ONLY="" +#RUN_ONLY="Test case 20: engine=MergeTree use_insert_token=False single_thread=True deduplicate_src_table=False deduplicate_dst_table=True insert_unique_blocks=True" + +i=0 +for insert_method in "InsertSelect" "InsertValues"; do + for use_insert_token in "True" "False"; do + for single_thread in "True" "False"; do + for deduplicate_src_table in "True" "False"; do + for deduplicate_dst_table in "True" "False"; do + for insert_unique_blocks in "True" "False"; do + + THIS_RUN="Test case $i:" + THIS_RUN+=" insert_method=$insert_method" + THIS_RUN+=" engine=$ENGINE" + THIS_RUN+=" use_insert_token=$use_insert_token" + THIS_RUN+=" single_thread=$single_thread" + THIS_RUN+=" deduplicate_src_table=$deduplicate_src_table" + THIS_RUN+=" deduplicate_dst_table=$deduplicate_dst_table" + THIS_RUN+=" insert_unique_blocks=$insert_unique_blocks" + + i=$((i+1)) + + echo + if [ -n "$RUN_ONLY" ] && [ "$RUN_ONLY" != "$THIS_RUN" ]; then + echo "skip $THIS_RUN" + continue + fi + echo "$THIS_RUN" + + $CLICKHOUSE_CLIENT --max_insert_block_size 1 -nmq " + $(python3 $CURDIR/03008_deduplication.python mv_generates_several_blocks \ + --insert-method $insert_method \ + --table-engine $ENGINE \ + --use-insert-token $use_insert_token \ + --single-thread $single_thread \ + --deduplicate-src-table $deduplicate_src_table \ + --deduplicate-dst-table $deduplicate_dst_table \ + --insert-unique-blocks $insert_unique_blocks \ + --get-logs false \ + ) + " && echo OK || echo FAIL + done + done + done + done + done +done + +echo +echo "All cases executed" diff --git a/tests/queries/0_stateless/03008_deduplication_mv_generates_several_blocks_replicated.reference b/tests/queries/0_stateless/03008_deduplication_mv_generates_several_blocks_replicated.reference new file mode 100644 index 00000000000..a25e8713c61 --- /dev/null +++ b/tests/queries/0_stateless/03008_deduplication_mv_generates_several_blocks_replicated.reference @@ -0,0 +1,962 @@ + +Test case 0: insert_method=InsertSelect engine=ReplicatedMergeTree use_insert_token=True single_thread=True deduplicate_src_table=True deduplicate_dst_table=True insert_unique_blocks=True +table_a_b +count 5 +table_when_b_even_and_joined +count 9 +0 +0 +table_a_b +count 5 +table_when_b_even_and_joined +count 9 +0 +0 +OK + +Test case 1: insert_method=InsertSelect engine=ReplicatedMergeTree use_insert_token=True single_thread=True deduplicate_src_table=True deduplicate_dst_table=True insert_unique_blocks=False +table_a_b +count 5 +table_when_b_even_and_joined +count 10 +0 +0 +table_a_b +count 5 +table_when_b_even_and_joined +count 10 +0 +0 +OK + +Test case 2: insert_method=InsertSelect engine=ReplicatedMergeTree use_insert_token=True single_thread=True deduplicate_src_table=True deduplicate_dst_table=False insert_unique_blocks=True +table_a_b +count 5 +table_when_b_even_and_joined +count 9 +0 +0 +table_a_b +count 5 +table_when_b_even_and_joined +count 18 +0 +0 +OK + +Test case 3: insert_method=InsertSelect engine=ReplicatedMergeTree use_insert_token=True single_thread=True deduplicate_src_table=True deduplicate_dst_table=False insert_unique_blocks=False +table_a_b +count 5 +table_when_b_even_and_joined +count 10 +0 +0 +table_a_b +count 5 +table_when_b_even_and_joined +count 20 +0 +0 +OK + +Test case 4: insert_method=InsertSelect engine=ReplicatedMergeTree use_insert_token=True single_thread=True deduplicate_src_table=False deduplicate_dst_table=True insert_unique_blocks=True +table_a_b +count 5 +table_when_b_even_and_joined +count 9 +0 +0 +table_a_b +count 10 +table_when_b_even_and_joined +count 9 +0 +0 +OK + +Test case 5: insert_method=InsertSelect engine=ReplicatedMergeTree use_insert_token=True single_thread=True deduplicate_src_table=False deduplicate_dst_table=True insert_unique_blocks=False +table_a_b +count 5 +table_when_b_even_and_joined +count 10 +0 +0 +table_a_b +count 10 +table_when_b_even_and_joined +count 10 +0 +0 +OK + +Test case 6: insert_method=InsertSelect engine=ReplicatedMergeTree use_insert_token=True single_thread=True deduplicate_src_table=False deduplicate_dst_table=False insert_unique_blocks=True +table_a_b +count 5 +table_when_b_even_and_joined +count 9 +0 +0 +table_a_b +count 10 +table_when_b_even_and_joined +count 18 +0 +0 +OK + +Test case 7: insert_method=InsertSelect engine=ReplicatedMergeTree use_insert_token=True single_thread=True deduplicate_src_table=False deduplicate_dst_table=False insert_unique_blocks=False +table_a_b +count 5 +table_when_b_even_and_joined +count 10 +0 +0 +table_a_b +count 10 +table_when_b_even_and_joined +count 20 +0 +0 +OK + +Test case 8: insert_method=InsertSelect engine=ReplicatedMergeTree use_insert_token=True single_thread=False deduplicate_src_table=True deduplicate_dst_table=True insert_unique_blocks=True +table_a_b +count 5 +table_when_b_even_and_joined +count 9 +0 +0 +table_a_b +count 5 +table_when_b_even_and_joined +count 9 +0 +0 +OK + +Test case 9: insert_method=InsertSelect engine=ReplicatedMergeTree use_insert_token=True single_thread=False deduplicate_src_table=True deduplicate_dst_table=True insert_unique_blocks=False +table_a_b +count 5 +table_when_b_even_and_joined +count 10 +0 +0 +table_a_b +count 5 +table_when_b_even_and_joined +count 10 +0 +0 +OK + +Test case 10: insert_method=InsertSelect engine=ReplicatedMergeTree use_insert_token=True single_thread=False deduplicate_src_table=True deduplicate_dst_table=False insert_unique_blocks=True +table_a_b +count 5 +table_when_b_even_and_joined +count 9 +0 +0 +table_a_b +count 5 +table_when_b_even_and_joined +count 18 +0 +0 +OK + +Test case 11: insert_method=InsertSelect engine=ReplicatedMergeTree use_insert_token=True single_thread=False deduplicate_src_table=True deduplicate_dst_table=False insert_unique_blocks=False +table_a_b +count 5 +table_when_b_even_and_joined +count 10 +0 +0 +table_a_b +count 5 +table_when_b_even_and_joined +count 20 +0 +0 +OK + +Test case 12: insert_method=InsertSelect engine=ReplicatedMergeTree use_insert_token=True single_thread=False deduplicate_src_table=False deduplicate_dst_table=True insert_unique_blocks=True +table_a_b +count 5 +table_when_b_even_and_joined +count 9 +0 +0 +table_a_b +count 10 +table_when_b_even_and_joined +count 9 +0 +0 +OK + +Test case 13: insert_method=InsertSelect engine=ReplicatedMergeTree use_insert_token=True single_thread=False deduplicate_src_table=False deduplicate_dst_table=True insert_unique_blocks=False +table_a_b +count 5 +table_when_b_even_and_joined +count 10 +0 +0 +table_a_b +count 10 +table_when_b_even_and_joined +count 10 +0 +0 +OK + +Test case 14: insert_method=InsertSelect engine=ReplicatedMergeTree use_insert_token=True single_thread=False deduplicate_src_table=False deduplicate_dst_table=False insert_unique_blocks=True +table_a_b +count 5 +table_when_b_even_and_joined +count 9 +0 +0 +table_a_b +count 10 +table_when_b_even_and_joined +count 18 +0 +0 +OK + +Test case 15: insert_method=InsertSelect engine=ReplicatedMergeTree use_insert_token=True single_thread=False deduplicate_src_table=False deduplicate_dst_table=False insert_unique_blocks=False +table_a_b +count 5 +table_when_b_even_and_joined +count 10 +0 +0 +table_a_b +count 10 +table_when_b_even_and_joined +count 20 +0 +0 +OK + +Test case 16: insert_method=InsertSelect engine=ReplicatedMergeTree use_insert_token=False single_thread=True deduplicate_src_table=True deduplicate_dst_table=True insert_unique_blocks=True +table_a_b +count 5 +table_when_b_even_and_joined +count 9 +0 +0 +table_a_b +count 5 +table_when_b_even_and_joined +count 9 +0 +0 +OK + +Test case 17: insert_method=InsertSelect engine=ReplicatedMergeTree use_insert_token=False single_thread=True deduplicate_src_table=True deduplicate_dst_table=True insert_unique_blocks=False +table_a_b +count 1 +table_when_b_even_and_joined +count 2 +0 +0 +table_a_b +count 1 +table_when_b_even_and_joined +count 2 +0 +0 +OK + +Test case 18: insert_method=InsertSelect engine=ReplicatedMergeTree use_insert_token=False single_thread=True deduplicate_src_table=True deduplicate_dst_table=False insert_unique_blocks=True +table_a_b +count 5 +table_when_b_even_and_joined +count 9 +0 +0 +table_a_b +count 5 +table_when_b_even_and_joined +count 18 +0 +0 +OK + +Test case 19: insert_method=InsertSelect engine=ReplicatedMergeTree use_insert_token=False single_thread=True deduplicate_src_table=True deduplicate_dst_table=False insert_unique_blocks=False +table_a_b +count 1 +table_when_b_even_and_joined +count 10 +0 +0 +table_a_b +count 1 +table_when_b_even_and_joined +count 20 +0 +0 +OK + +Test case 20: insert_method=InsertSelect engine=ReplicatedMergeTree use_insert_token=False single_thread=True deduplicate_src_table=False deduplicate_dst_table=True insert_unique_blocks=True +table_a_b +count 5 +table_when_b_even_and_joined +count 9 +0 +0 +table_a_b +count 10 +table_when_b_even_and_joined +count 9 +0 +0 +OK + +Test case 21: insert_method=InsertSelect engine=ReplicatedMergeTree use_insert_token=False single_thread=True deduplicate_src_table=False deduplicate_dst_table=True insert_unique_blocks=False +table_a_b +count 5 +table_when_b_even_and_joined +count 2 +0 +0 +table_a_b +count 10 +table_when_b_even_and_joined +count 2 +0 +0 +OK + +Test case 22: insert_method=InsertSelect engine=ReplicatedMergeTree use_insert_token=False single_thread=True deduplicate_src_table=False deduplicate_dst_table=False insert_unique_blocks=True +table_a_b +count 5 +table_when_b_even_and_joined +count 9 +0 +0 +table_a_b +count 10 +table_when_b_even_and_joined +count 18 +0 +0 +OK + +Test case 23: insert_method=InsertSelect engine=ReplicatedMergeTree use_insert_token=False single_thread=True deduplicate_src_table=False deduplicate_dst_table=False insert_unique_blocks=False +table_a_b +count 5 +table_when_b_even_and_joined +count 10 +0 +0 +table_a_b +count 10 +table_when_b_even_and_joined +count 20 +0 +0 +OK + +Test case 24: insert_method=InsertSelect engine=ReplicatedMergeTree use_insert_token=False single_thread=False deduplicate_src_table=True deduplicate_dst_table=True insert_unique_blocks=True +table_a_b +count 5 +table_when_b_even_and_joined +count 9 +0 +0 +table_a_b +count 5 +table_when_b_even_and_joined +count 9 +0 +0 +OK + +Test case 25: insert_method=InsertSelect engine=ReplicatedMergeTree use_insert_token=False single_thread=False deduplicate_src_table=True deduplicate_dst_table=True insert_unique_blocks=False +table_a_b +count 1 +table_when_b_even_and_joined +count 2 +0 +0 +table_a_b +count 1 +table_when_b_even_and_joined +count 2 +0 +0 +OK + +Test case 26: insert_method=InsertSelect engine=ReplicatedMergeTree use_insert_token=False single_thread=False deduplicate_src_table=True deduplicate_dst_table=False insert_unique_blocks=True +table_a_b +count 5 +table_when_b_even_and_joined +count 9 +0 +0 +table_a_b +count 5 +table_when_b_even_and_joined +count 18 +0 +0 +OK + +Test case 27: insert_method=InsertSelect engine=ReplicatedMergeTree use_insert_token=False single_thread=False deduplicate_src_table=True deduplicate_dst_table=False insert_unique_blocks=False +table_a_b +count 1 +table_when_b_even_and_joined +count 10 +0 +0 +table_a_b +count 1 +table_when_b_even_and_joined +count 20 +0 +0 +OK + +Test case 28: insert_method=InsertSelect engine=ReplicatedMergeTree use_insert_token=False single_thread=False deduplicate_src_table=False deduplicate_dst_table=True insert_unique_blocks=True +table_a_b +count 5 +table_when_b_even_and_joined +count 9 +0 +0 +table_a_b +count 10 +table_when_b_even_and_joined +count 9 +0 +0 +OK + +Test case 29: insert_method=InsertSelect engine=ReplicatedMergeTree use_insert_token=False single_thread=False deduplicate_src_table=False deduplicate_dst_table=True insert_unique_blocks=False +table_a_b +count 5 +table_when_b_even_and_joined +count 2 +0 +0 +table_a_b +count 10 +table_when_b_even_and_joined +count 2 +0 +0 +OK + +Test case 30: insert_method=InsertSelect engine=ReplicatedMergeTree use_insert_token=False single_thread=False deduplicate_src_table=False deduplicate_dst_table=False insert_unique_blocks=True +table_a_b +count 5 +table_when_b_even_and_joined +count 9 +0 +0 +table_a_b +count 10 +table_when_b_even_and_joined +count 18 +0 +0 +OK + +Test case 31: insert_method=InsertSelect engine=ReplicatedMergeTree use_insert_token=False single_thread=False deduplicate_src_table=False deduplicate_dst_table=False insert_unique_blocks=False +table_a_b +count 5 +table_when_b_even_and_joined +count 10 +0 +0 +table_a_b +count 10 +table_when_b_even_and_joined +count 20 +0 +0 +OK + +Test case 32: insert_method=InsertValues engine=ReplicatedMergeTree use_insert_token=True single_thread=True deduplicate_src_table=True deduplicate_dst_table=True insert_unique_blocks=True +table_a_b +count 5 +table_when_b_even_and_joined +count 9 +0 +0 +table_a_b +count 5 +table_when_b_even_and_joined +count 9 +0 +0 +OK + +Test case 33: insert_method=InsertValues engine=ReplicatedMergeTree use_insert_token=True single_thread=True deduplicate_src_table=True deduplicate_dst_table=True insert_unique_blocks=False +table_a_b +count 5 +table_when_b_even_and_joined +count 10 +0 +0 +table_a_b +count 5 +table_when_b_even_and_joined +count 10 +0 +0 +OK + +Test case 34: insert_method=InsertValues engine=ReplicatedMergeTree use_insert_token=True single_thread=True deduplicate_src_table=True deduplicate_dst_table=False insert_unique_blocks=True +table_a_b +count 5 +table_when_b_even_and_joined +count 9 +0 +0 +table_a_b +count 5 +table_when_b_even_and_joined +count 18 +0 +0 +OK + +Test case 35: insert_method=InsertValues engine=ReplicatedMergeTree use_insert_token=True single_thread=True deduplicate_src_table=True deduplicate_dst_table=False insert_unique_blocks=False +table_a_b +count 5 +table_when_b_even_and_joined +count 10 +0 +0 +table_a_b +count 5 +table_when_b_even_and_joined +count 20 +0 +0 +OK + +Test case 36: insert_method=InsertValues engine=ReplicatedMergeTree use_insert_token=True single_thread=True deduplicate_src_table=False deduplicate_dst_table=True insert_unique_blocks=True +table_a_b +count 5 +table_when_b_even_and_joined +count 9 +0 +0 +table_a_b +count 10 +table_when_b_even_and_joined +count 9 +0 +0 +OK + +Test case 37: insert_method=InsertValues engine=ReplicatedMergeTree use_insert_token=True single_thread=True deduplicate_src_table=False deduplicate_dst_table=True insert_unique_blocks=False +table_a_b +count 5 +table_when_b_even_and_joined +count 10 +0 +0 +table_a_b +count 10 +table_when_b_even_and_joined +count 10 +0 +0 +OK + +Test case 38: insert_method=InsertValues engine=ReplicatedMergeTree use_insert_token=True single_thread=True deduplicate_src_table=False deduplicate_dst_table=False insert_unique_blocks=True +table_a_b +count 5 +table_when_b_even_and_joined +count 9 +0 +0 +table_a_b +count 10 +table_when_b_even_and_joined +count 18 +0 +0 +OK + +Test case 39: insert_method=InsertValues engine=ReplicatedMergeTree use_insert_token=True single_thread=True deduplicate_src_table=False deduplicate_dst_table=False insert_unique_blocks=False +table_a_b +count 5 +table_when_b_even_and_joined +count 10 +0 +0 +table_a_b +count 10 +table_when_b_even_and_joined +count 20 +0 +0 +OK + +Test case 40: insert_method=InsertValues engine=ReplicatedMergeTree use_insert_token=True single_thread=False deduplicate_src_table=True deduplicate_dst_table=True insert_unique_blocks=True +table_a_b +count 5 +table_when_b_even_and_joined +count 9 +0 +0 +table_a_b +count 5 +table_when_b_even_and_joined +count 9 +0 +0 +OK + +Test case 41: insert_method=InsertValues engine=ReplicatedMergeTree use_insert_token=True single_thread=False deduplicate_src_table=True deduplicate_dst_table=True insert_unique_blocks=False +table_a_b +count 5 +table_when_b_even_and_joined +count 10 +0 +0 +table_a_b +count 5 +table_when_b_even_and_joined +count 10 +0 +0 +OK + +Test case 42: insert_method=InsertValues engine=ReplicatedMergeTree use_insert_token=True single_thread=False deduplicate_src_table=True deduplicate_dst_table=False insert_unique_blocks=True +table_a_b +count 5 +table_when_b_even_and_joined +count 9 +0 +0 +table_a_b +count 5 +table_when_b_even_and_joined +count 18 +0 +0 +OK + +Test case 43: insert_method=InsertValues engine=ReplicatedMergeTree use_insert_token=True single_thread=False deduplicate_src_table=True deduplicate_dst_table=False insert_unique_blocks=False +table_a_b +count 5 +table_when_b_even_and_joined +count 10 +0 +0 +table_a_b +count 5 +table_when_b_even_and_joined +count 20 +0 +0 +OK + +Test case 44: insert_method=InsertValues engine=ReplicatedMergeTree use_insert_token=True single_thread=False deduplicate_src_table=False deduplicate_dst_table=True insert_unique_blocks=True +table_a_b +count 5 +table_when_b_even_and_joined +count 9 +0 +0 +table_a_b +count 10 +table_when_b_even_and_joined +count 9 +0 +0 +OK + +Test case 45: insert_method=InsertValues engine=ReplicatedMergeTree use_insert_token=True single_thread=False deduplicate_src_table=False deduplicate_dst_table=True insert_unique_blocks=False +table_a_b +count 5 +table_when_b_even_and_joined +count 10 +0 +0 +table_a_b +count 10 +table_when_b_even_and_joined +count 10 +0 +0 +OK + +Test case 46: insert_method=InsertValues engine=ReplicatedMergeTree use_insert_token=True single_thread=False deduplicate_src_table=False deduplicate_dst_table=False insert_unique_blocks=True +table_a_b +count 5 +table_when_b_even_and_joined +count 9 +0 +0 +table_a_b +count 10 +table_when_b_even_and_joined +count 18 +0 +0 +OK + +Test case 47: insert_method=InsertValues engine=ReplicatedMergeTree use_insert_token=True single_thread=False deduplicate_src_table=False deduplicate_dst_table=False insert_unique_blocks=False +table_a_b +count 5 +table_when_b_even_and_joined +count 10 +0 +0 +table_a_b +count 10 +table_when_b_even_and_joined +count 20 +0 +0 +OK + +Test case 48: insert_method=InsertValues engine=ReplicatedMergeTree use_insert_token=False single_thread=True deduplicate_src_table=True deduplicate_dst_table=True insert_unique_blocks=True +table_a_b +count 5 +table_when_b_even_and_joined +count 9 +0 +0 +table_a_b +count 5 +table_when_b_even_and_joined +count 9 +0 +0 +OK + +Test case 49: insert_method=InsertValues engine=ReplicatedMergeTree use_insert_token=False single_thread=True deduplicate_src_table=True deduplicate_dst_table=True insert_unique_blocks=False +table_a_b +count 1 +table_when_b_even_and_joined +count 2 +0 +0 +table_a_b +count 1 +table_when_b_even_and_joined +count 2 +0 +0 +OK + +Test case 50: insert_method=InsertValues engine=ReplicatedMergeTree use_insert_token=False single_thread=True deduplicate_src_table=True deduplicate_dst_table=False insert_unique_blocks=True +table_a_b +count 5 +table_when_b_even_and_joined +count 9 +0 +0 +table_a_b +count 5 +table_when_b_even_and_joined +count 18 +0 +0 +OK + +Test case 51: insert_method=InsertValues engine=ReplicatedMergeTree use_insert_token=False single_thread=True deduplicate_src_table=True deduplicate_dst_table=False insert_unique_blocks=False +table_a_b +count 1 +table_when_b_even_and_joined +count 10 +0 +0 +table_a_b +count 1 +table_when_b_even_and_joined +count 20 +0 +0 +OK + +Test case 52: insert_method=InsertValues engine=ReplicatedMergeTree use_insert_token=False single_thread=True deduplicate_src_table=False deduplicate_dst_table=True insert_unique_blocks=True +table_a_b +count 5 +table_when_b_even_and_joined +count 9 +0 +0 +table_a_b +count 10 +table_when_b_even_and_joined +count 9 +0 +0 +OK + +Test case 53: insert_method=InsertValues engine=ReplicatedMergeTree use_insert_token=False single_thread=True deduplicate_src_table=False deduplicate_dst_table=True insert_unique_blocks=False +table_a_b +count 5 +table_when_b_even_and_joined +count 2 +0 +0 +table_a_b +count 10 +table_when_b_even_and_joined +count 2 +0 +0 +OK + +Test case 54: insert_method=InsertValues engine=ReplicatedMergeTree use_insert_token=False single_thread=True deduplicate_src_table=False deduplicate_dst_table=False insert_unique_blocks=True +table_a_b +count 5 +table_when_b_even_and_joined +count 9 +0 +0 +table_a_b +count 10 +table_when_b_even_and_joined +count 18 +0 +0 +OK + +Test case 55: insert_method=InsertValues engine=ReplicatedMergeTree use_insert_token=False single_thread=True deduplicate_src_table=False deduplicate_dst_table=False insert_unique_blocks=False +table_a_b +count 5 +table_when_b_even_and_joined +count 10 +0 +0 +table_a_b +count 10 +table_when_b_even_and_joined +count 20 +0 +0 +OK + +Test case 56: insert_method=InsertValues engine=ReplicatedMergeTree use_insert_token=False single_thread=False deduplicate_src_table=True deduplicate_dst_table=True insert_unique_blocks=True +table_a_b +count 5 +table_when_b_even_and_joined +count 9 +0 +0 +table_a_b +count 5 +table_when_b_even_and_joined +count 9 +0 +0 +OK + +Test case 57: insert_method=InsertValues engine=ReplicatedMergeTree use_insert_token=False single_thread=False deduplicate_src_table=True deduplicate_dst_table=True insert_unique_blocks=False +table_a_b +count 1 +table_when_b_even_and_joined +count 2 +0 +0 +table_a_b +count 1 +table_when_b_even_and_joined +count 2 +0 +0 +OK + +Test case 58: insert_method=InsertValues engine=ReplicatedMergeTree use_insert_token=False single_thread=False deduplicate_src_table=True deduplicate_dst_table=False insert_unique_blocks=True +table_a_b +count 5 +table_when_b_even_and_joined +count 9 +0 +0 +table_a_b +count 5 +table_when_b_even_and_joined +count 18 +0 +0 +OK + +Test case 59: insert_method=InsertValues engine=ReplicatedMergeTree use_insert_token=False single_thread=False deduplicate_src_table=True deduplicate_dst_table=False insert_unique_blocks=False +table_a_b +count 1 +table_when_b_even_and_joined +count 10 +0 +0 +table_a_b +count 1 +table_when_b_even_and_joined +count 20 +0 +0 +OK + +Test case 60: insert_method=InsertValues engine=ReplicatedMergeTree use_insert_token=False single_thread=False deduplicate_src_table=False deduplicate_dst_table=True insert_unique_blocks=True +table_a_b +count 5 +table_when_b_even_and_joined +count 9 +0 +0 +table_a_b +count 10 +table_when_b_even_and_joined +count 9 +0 +0 +OK + +Test case 61: insert_method=InsertValues engine=ReplicatedMergeTree use_insert_token=False single_thread=False deduplicate_src_table=False deduplicate_dst_table=True insert_unique_blocks=False +table_a_b +count 5 +table_when_b_even_and_joined +count 2 +0 +0 +table_a_b +count 10 +table_when_b_even_and_joined +count 2 +0 +0 +OK + +Test case 62: insert_method=InsertValues engine=ReplicatedMergeTree use_insert_token=False single_thread=False deduplicate_src_table=False deduplicate_dst_table=False insert_unique_blocks=True +table_a_b +count 5 +table_when_b_even_and_joined +count 9 +0 +0 +table_a_b +count 10 +table_when_b_even_and_joined +count 18 +0 +0 +OK + +Test case 63: insert_method=InsertValues engine=ReplicatedMergeTree use_insert_token=False single_thread=False deduplicate_src_table=False deduplicate_dst_table=False insert_unique_blocks=False +table_a_b +count 5 +table_when_b_even_and_joined +count 10 +0 +0 +table_a_b +count 10 +table_when_b_even_and_joined +count 20 +0 +0 +OK + +All cases executed diff --git a/tests/queries/0_stateless/03008_deduplication_mv_generates_several_blocks_replicated.sh b/tests/queries/0_stateless/03008_deduplication_mv_generates_several_blocks_replicated.sh new file mode 100755 index 00000000000..0d5158d18cd --- /dev/null +++ b/tests/queries/0_stateless/03008_deduplication_mv_generates_several_blocks_replicated.sh @@ -0,0 +1,59 @@ +#!/usr/bin/env bash +# Tags: long, no-fasttest, no-parallel, no-asan + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +ENGINE="ReplicatedMergeTree" + +RUN_ONLY="" +#RUN_ONLY="Test case 20: engine=ReplicatedMergeTree use_insert_token=False single_thread=True deduplicate_src_table=False deduplicate_dst_table=True insert_unique_blocks=True" + +i=0 +for insert_method in "InsertSelect" "InsertValues"; do + for use_insert_token in "True" "False"; do + for single_thread in "True" "False"; do + for deduplicate_src_table in "True" "False"; do + for deduplicate_dst_table in "True" "False"; do + for insert_unique_blocks in "True" "False"; do + + THIS_RUN="Test case $i:" + THIS_RUN+=" insert_method=$insert_method" + THIS_RUN+=" engine=$ENGINE" + THIS_RUN+=" use_insert_token=$use_insert_token" + THIS_RUN+=" single_thread=$single_thread" + THIS_RUN+=" deduplicate_src_table=$deduplicate_src_table" + THIS_RUN+=" deduplicate_dst_table=$deduplicate_dst_table" + THIS_RUN+=" insert_unique_blocks=$insert_unique_blocks" + + i=$((i+1)) + + echo + if [ -n "$RUN_ONLY" ] && [ "$RUN_ONLY" != "$THIS_RUN" ]; then + echo "skip $THIS_RUN" + continue + fi + echo "$THIS_RUN" + + $CLICKHOUSE_CLIENT --max_insert_block_size 1 -nmq " + $(python3 $CURDIR/03008_deduplication.python mv_generates_several_blocks \ + --insert-method $insert_method \ + --table-engine $ENGINE \ + --use-insert-token $use_insert_token \ + --single-thread $single_thread \ + --deduplicate-src-table $deduplicate_src_table \ + --deduplicate-dst-table $deduplicate_dst_table \ + --insert-unique-blocks $insert_unique_blocks \ + --get-logs false \ + ) + " && echo OK || echo FAIL + done + done + done + done + done +done + +echo +echo "All cases executed" diff --git a/tests/queries/0_stateless/03008_deduplication_several_mv_into_one_table_nonreplicated.reference b/tests/queries/0_stateless/03008_deduplication_several_mv_into_one_table_nonreplicated.reference new file mode 100644 index 00000000000..b6a3e0175a7 --- /dev/null +++ b/tests/queries/0_stateless/03008_deduplication_several_mv_into_one_table_nonreplicated.reference @@ -0,0 +1,706 @@ + +Test case 0: insert_method=InsertSelect engine=MergeTree use_insert_token=True single_thread=True deduplicate_src_table=True deduplicate_dst_table=True insert_unique_blocks=True +table_src count 8 +table_dst count 6 +0 +0 +table_src count 8 +table_dst count 6 +0 +0 +OK + +Test case 1: insert_method=InsertSelect engine=MergeTree use_insert_token=True single_thread=True deduplicate_src_table=True deduplicate_dst_table=True insert_unique_blocks=False +table_src count 8 +table_dst count 16 +0 +0 +table_src count 8 +table_dst count 16 +0 +0 +OK + +Test case 2: insert_method=InsertSelect engine=MergeTree use_insert_token=True single_thread=True deduplicate_src_table=True deduplicate_dst_table=False insert_unique_blocks=True +table_src count 8 +table_dst count 6 +0 +0 +table_src count 8 +table_dst count 12 +0 +0 +OK + +Test case 3: insert_method=InsertSelect engine=MergeTree use_insert_token=True single_thread=True deduplicate_src_table=True deduplicate_dst_table=False insert_unique_blocks=False +table_src count 8 +table_dst count 16 +0 +0 +table_src count 8 +table_dst count 32 +0 +0 +OK + +Test case 4: insert_method=InsertSelect engine=MergeTree use_insert_token=True single_thread=True deduplicate_src_table=False deduplicate_dst_table=True insert_unique_blocks=True +table_src count 8 +table_dst count 6 +0 +0 +table_src count 16 +table_dst count 6 +0 +0 +OK + +Test case 5: insert_method=InsertSelect engine=MergeTree use_insert_token=True single_thread=True deduplicate_src_table=False deduplicate_dst_table=True insert_unique_blocks=False +table_src count 8 +table_dst count 16 +0 +0 +table_src count 16 +table_dst count 16 +0 +0 +OK + +Test case 6: insert_method=InsertSelect engine=MergeTree use_insert_token=True single_thread=True deduplicate_src_table=False deduplicate_dst_table=False insert_unique_blocks=True +table_src count 8 +table_dst count 6 +0 +0 +table_src count 16 +table_dst count 12 +0 +0 +OK + +Test case 7: insert_method=InsertSelect engine=MergeTree use_insert_token=True single_thread=True deduplicate_src_table=False deduplicate_dst_table=False insert_unique_blocks=False +table_src count 8 +table_dst count 16 +0 +0 +table_src count 16 +table_dst count 32 +0 +0 +OK + +Test case 8: insert_method=InsertSelect engine=MergeTree use_insert_token=True single_thread=False deduplicate_src_table=True deduplicate_dst_table=True insert_unique_blocks=True +table_src count 8 +table_dst count 6 +0 +0 +table_src count 8 +table_dst count 6 +0 +0 +OK + +Test case 9: insert_method=InsertSelect engine=MergeTree use_insert_token=True single_thread=False deduplicate_src_table=True deduplicate_dst_table=True insert_unique_blocks=False +table_src count 8 +table_dst count 16 +0 +0 +table_src count 8 +table_dst count 16 +0 +0 +OK + +Test case 10: insert_method=InsertSelect engine=MergeTree use_insert_token=True single_thread=False deduplicate_src_table=True deduplicate_dst_table=False insert_unique_blocks=True +table_src count 8 +table_dst count 6 +0 +0 +table_src count 8 +table_dst count 12 +0 +0 +OK + +Test case 11: insert_method=InsertSelect engine=MergeTree use_insert_token=True single_thread=False deduplicate_src_table=True deduplicate_dst_table=False insert_unique_blocks=False +table_src count 8 +table_dst count 16 +0 +0 +table_src count 8 +table_dst count 32 +0 +0 +OK + +Test case 12: insert_method=InsertSelect engine=MergeTree use_insert_token=True single_thread=False deduplicate_src_table=False deduplicate_dst_table=True insert_unique_blocks=True +table_src count 8 +table_dst count 6 +0 +0 +table_src count 16 +table_dst count 6 +0 +0 +OK + +Test case 13: insert_method=InsertSelect engine=MergeTree use_insert_token=True single_thread=False deduplicate_src_table=False deduplicate_dst_table=True insert_unique_blocks=False +table_src count 8 +table_dst count 16 +0 +0 +table_src count 16 +table_dst count 16 +0 +0 +OK + +Test case 14: insert_method=InsertSelect engine=MergeTree use_insert_token=True single_thread=False deduplicate_src_table=False deduplicate_dst_table=False insert_unique_blocks=True +table_src count 8 +table_dst count 6 +0 +0 +table_src count 16 +table_dst count 12 +0 +0 +OK + +Test case 15: insert_method=InsertSelect engine=MergeTree use_insert_token=True single_thread=False deduplicate_src_table=False deduplicate_dst_table=False insert_unique_blocks=False +table_src count 8 +table_dst count 16 +0 +0 +table_src count 16 +table_dst count 32 +0 +0 +OK + +Test case 16: insert_method=InsertSelect engine=MergeTree use_insert_token=False single_thread=True deduplicate_src_table=True deduplicate_dst_table=True insert_unique_blocks=True +table_src count 8 +table_dst count 6 +0 +0 +table_src count 8 +table_dst count 6 +0 +0 +OK + +Test case 17: insert_method=InsertSelect engine=MergeTree use_insert_token=False single_thread=True deduplicate_src_table=True deduplicate_dst_table=True insert_unique_blocks=False +table_src count 1 +table_dst count 2 +0 +0 +table_src count 1 +table_dst count 2 +0 +0 +OK + +Test case 18: insert_method=InsertSelect engine=MergeTree use_insert_token=False single_thread=True deduplicate_src_table=True deduplicate_dst_table=False insert_unique_blocks=True +table_src count 8 +table_dst count 6 +0 +0 +table_src count 8 +table_dst count 12 +0 +0 +OK + +Test case 19: insert_method=InsertSelect engine=MergeTree use_insert_token=False single_thread=True deduplicate_src_table=True deduplicate_dst_table=False insert_unique_blocks=False +table_src count 1 +table_dst count 16 +0 +0 +table_src count 1 +table_dst count 32 +0 +0 +OK + +Test case 20: insert_method=InsertSelect engine=MergeTree use_insert_token=False single_thread=True deduplicate_src_table=False deduplicate_dst_table=True insert_unique_blocks=True +table_src count 8 +table_dst count 6 +0 +0 +table_src count 16 +table_dst count 6 +0 +0 +OK + +Test case 21: insert_method=InsertSelect engine=MergeTree use_insert_token=False single_thread=True deduplicate_src_table=False deduplicate_dst_table=True insert_unique_blocks=False +table_src count 8 +table_dst count 2 +0 +0 +table_src count 16 +table_dst count 2 +0 +0 +OK + +Test case 22: insert_method=InsertSelect engine=MergeTree use_insert_token=False single_thread=True deduplicate_src_table=False deduplicate_dst_table=False insert_unique_blocks=True +table_src count 8 +table_dst count 6 +0 +0 +table_src count 16 +table_dst count 12 +0 +0 +OK + +Test case 23: insert_method=InsertSelect engine=MergeTree use_insert_token=False single_thread=True deduplicate_src_table=False deduplicate_dst_table=False insert_unique_blocks=False +table_src count 8 +table_dst count 16 +0 +0 +table_src count 16 +table_dst count 32 +0 +0 +OK + +Test case 24: insert_method=InsertSelect engine=MergeTree use_insert_token=False single_thread=False deduplicate_src_table=True deduplicate_dst_table=True insert_unique_blocks=True +table_src count 8 +table_dst count 6 +0 +0 +table_src count 8 +table_dst count 6 +0 +0 +OK + +Test case 25: insert_method=InsertSelect engine=MergeTree use_insert_token=False single_thread=False deduplicate_src_table=True deduplicate_dst_table=True insert_unique_blocks=False +table_src count 1 +table_dst count 2 +0 +0 +table_src count 1 +table_dst count 2 +0 +0 +OK + +Test case 26: insert_method=InsertSelect engine=MergeTree use_insert_token=False single_thread=False deduplicate_src_table=True deduplicate_dst_table=False insert_unique_blocks=True +table_src count 8 +table_dst count 6 +0 +0 +table_src count 8 +table_dst count 12 +0 +0 +OK + +Test case 27: insert_method=InsertSelect engine=MergeTree use_insert_token=False single_thread=False deduplicate_src_table=True deduplicate_dst_table=False insert_unique_blocks=False +table_src count 1 +table_dst count 16 +0 +0 +table_src count 1 +table_dst count 32 +0 +0 +OK + +Test case 28: insert_method=InsertSelect engine=MergeTree use_insert_token=False single_thread=False deduplicate_src_table=False deduplicate_dst_table=True insert_unique_blocks=True +table_src count 8 +table_dst count 6 +0 +0 +table_src count 16 +table_dst count 6 +0 +0 +OK + +Test case 29: insert_method=InsertSelect engine=MergeTree use_insert_token=False single_thread=False deduplicate_src_table=False deduplicate_dst_table=True insert_unique_blocks=False +table_src count 8 +table_dst count 2 +0 +0 +table_src count 16 +table_dst count 2 +0 +0 +OK + +Test case 30: insert_method=InsertSelect engine=MergeTree use_insert_token=False single_thread=False deduplicate_src_table=False deduplicate_dst_table=False insert_unique_blocks=True +table_src count 8 +table_dst count 6 +0 +0 +table_src count 16 +table_dst count 12 +0 +0 +OK + +Test case 31: insert_method=InsertSelect engine=MergeTree use_insert_token=False single_thread=False deduplicate_src_table=False deduplicate_dst_table=False insert_unique_blocks=False +table_src count 8 +table_dst count 16 +0 +0 +table_src count 16 +table_dst count 32 +0 +0 +OK + +Test case 32: insert_method=InsertValues engine=MergeTree use_insert_token=True single_thread=True deduplicate_src_table=True deduplicate_dst_table=True insert_unique_blocks=True +table_src count 8 +table_dst count 6 +0 +0 +table_src count 8 +table_dst count 6 +0 +0 +OK + +Test case 33: insert_method=InsertValues engine=MergeTree use_insert_token=True single_thread=True deduplicate_src_table=True deduplicate_dst_table=True insert_unique_blocks=False +table_src count 8 +table_dst count 16 +0 +0 +table_src count 8 +table_dst count 16 +0 +0 +OK + +Test case 34: insert_method=InsertValues engine=MergeTree use_insert_token=True single_thread=True deduplicate_src_table=True deduplicate_dst_table=False insert_unique_blocks=True +table_src count 8 +table_dst count 6 +0 +0 +table_src count 8 +table_dst count 12 +0 +0 +OK + +Test case 35: insert_method=InsertValues engine=MergeTree use_insert_token=True single_thread=True deduplicate_src_table=True deduplicate_dst_table=False insert_unique_blocks=False +table_src count 8 +table_dst count 16 +0 +0 +table_src count 8 +table_dst count 32 +0 +0 +OK + +Test case 36: insert_method=InsertValues engine=MergeTree use_insert_token=True single_thread=True deduplicate_src_table=False deduplicate_dst_table=True insert_unique_blocks=True +table_src count 8 +table_dst count 6 +0 +0 +table_src count 16 +table_dst count 6 +0 +0 +OK + +Test case 37: insert_method=InsertValues engine=MergeTree use_insert_token=True single_thread=True deduplicate_src_table=False deduplicate_dst_table=True insert_unique_blocks=False +table_src count 8 +table_dst count 16 +0 +0 +table_src count 16 +table_dst count 16 +0 +0 +OK + +Test case 38: insert_method=InsertValues engine=MergeTree use_insert_token=True single_thread=True deduplicate_src_table=False deduplicate_dst_table=False insert_unique_blocks=True +table_src count 8 +table_dst count 6 +0 +0 +table_src count 16 +table_dst count 12 +0 +0 +OK + +Test case 39: insert_method=InsertValues engine=MergeTree use_insert_token=True single_thread=True deduplicate_src_table=False deduplicate_dst_table=False insert_unique_blocks=False +table_src count 8 +table_dst count 16 +0 +0 +table_src count 16 +table_dst count 32 +0 +0 +OK + +Test case 40: insert_method=InsertValues engine=MergeTree use_insert_token=True single_thread=False deduplicate_src_table=True deduplicate_dst_table=True insert_unique_blocks=True +table_src count 8 +table_dst count 6 +0 +0 +table_src count 8 +table_dst count 6 +0 +0 +OK + +Test case 41: insert_method=InsertValues engine=MergeTree use_insert_token=True single_thread=False deduplicate_src_table=True deduplicate_dst_table=True insert_unique_blocks=False +table_src count 8 +table_dst count 16 +0 +0 +table_src count 8 +table_dst count 16 +0 +0 +OK + +Test case 42: insert_method=InsertValues engine=MergeTree use_insert_token=True single_thread=False deduplicate_src_table=True deduplicate_dst_table=False insert_unique_blocks=True +table_src count 8 +table_dst count 6 +0 +0 +table_src count 8 +table_dst count 12 +0 +0 +OK + +Test case 43: insert_method=InsertValues engine=MergeTree use_insert_token=True single_thread=False deduplicate_src_table=True deduplicate_dst_table=False insert_unique_blocks=False +table_src count 8 +table_dst count 16 +0 +0 +table_src count 8 +table_dst count 32 +0 +0 +OK + +Test case 44: insert_method=InsertValues engine=MergeTree use_insert_token=True single_thread=False deduplicate_src_table=False deduplicate_dst_table=True insert_unique_blocks=True +table_src count 8 +table_dst count 6 +0 +0 +table_src count 16 +table_dst count 6 +0 +0 +OK + +Test case 45: insert_method=InsertValues engine=MergeTree use_insert_token=True single_thread=False deduplicate_src_table=False deduplicate_dst_table=True insert_unique_blocks=False +table_src count 8 +table_dst count 16 +0 +0 +table_src count 16 +table_dst count 16 +0 +0 +OK + +Test case 46: insert_method=InsertValues engine=MergeTree use_insert_token=True single_thread=False deduplicate_src_table=False deduplicate_dst_table=False insert_unique_blocks=True +table_src count 8 +table_dst count 6 +0 +0 +table_src count 16 +table_dst count 12 +0 +0 +OK + +Test case 47: insert_method=InsertValues engine=MergeTree use_insert_token=True single_thread=False deduplicate_src_table=False deduplicate_dst_table=False insert_unique_blocks=False +table_src count 8 +table_dst count 16 +0 +0 +table_src count 16 +table_dst count 32 +0 +0 +OK + +Test case 48: insert_method=InsertValues engine=MergeTree use_insert_token=False single_thread=True deduplicate_src_table=True deduplicate_dst_table=True insert_unique_blocks=True +table_src count 8 +table_dst count 6 +0 +0 +table_src count 8 +table_dst count 6 +0 +0 +OK + +Test case 49: insert_method=InsertValues engine=MergeTree use_insert_token=False single_thread=True deduplicate_src_table=True deduplicate_dst_table=True insert_unique_blocks=False +table_src count 1 +table_dst count 2 +0 +0 +table_src count 1 +table_dst count 2 +0 +0 +OK + +Test case 50: insert_method=InsertValues engine=MergeTree use_insert_token=False single_thread=True deduplicate_src_table=True deduplicate_dst_table=False insert_unique_blocks=True +table_src count 8 +table_dst count 6 +0 +0 +table_src count 8 +table_dst count 12 +0 +0 +OK + +Test case 51: insert_method=InsertValues engine=MergeTree use_insert_token=False single_thread=True deduplicate_src_table=True deduplicate_dst_table=False insert_unique_blocks=False +table_src count 1 +table_dst count 16 +0 +0 +table_src count 1 +table_dst count 32 +0 +0 +OK + +Test case 52: insert_method=InsertValues engine=MergeTree use_insert_token=False single_thread=True deduplicate_src_table=False deduplicate_dst_table=True insert_unique_blocks=True +table_src count 8 +table_dst count 6 +0 +0 +table_src count 16 +table_dst count 6 +0 +0 +OK + +Test case 53: insert_method=InsertValues engine=MergeTree use_insert_token=False single_thread=True deduplicate_src_table=False deduplicate_dst_table=True insert_unique_blocks=False +table_src count 8 +table_dst count 2 +0 +0 +table_src count 16 +table_dst count 2 +0 +0 +OK + +Test case 54: insert_method=InsertValues engine=MergeTree use_insert_token=False single_thread=True deduplicate_src_table=False deduplicate_dst_table=False insert_unique_blocks=True +table_src count 8 +table_dst count 6 +0 +0 +table_src count 16 +table_dst count 12 +0 +0 +OK + +Test case 55: insert_method=InsertValues engine=MergeTree use_insert_token=False single_thread=True deduplicate_src_table=False deduplicate_dst_table=False insert_unique_blocks=False +table_src count 8 +table_dst count 16 +0 +0 +table_src count 16 +table_dst count 32 +0 +0 +OK + +Test case 56: insert_method=InsertValues engine=MergeTree use_insert_token=False single_thread=False deduplicate_src_table=True deduplicate_dst_table=True insert_unique_blocks=True +table_src count 8 +table_dst count 6 +0 +0 +table_src count 8 +table_dst count 6 +0 +0 +OK + +Test case 57: insert_method=InsertValues engine=MergeTree use_insert_token=False single_thread=False deduplicate_src_table=True deduplicate_dst_table=True insert_unique_blocks=False +table_src count 1 +table_dst count 2 +0 +0 +table_src count 1 +table_dst count 2 +0 +0 +OK + +Test case 58: insert_method=InsertValues engine=MergeTree use_insert_token=False single_thread=False deduplicate_src_table=True deduplicate_dst_table=False insert_unique_blocks=True +table_src count 8 +table_dst count 6 +0 +0 +table_src count 8 +table_dst count 12 +0 +0 +OK + +Test case 59: insert_method=InsertValues engine=MergeTree use_insert_token=False single_thread=False deduplicate_src_table=True deduplicate_dst_table=False insert_unique_blocks=False +table_src count 1 +table_dst count 16 +0 +0 +table_src count 1 +table_dst count 32 +0 +0 +OK + +Test case 60: insert_method=InsertValues engine=MergeTree use_insert_token=False single_thread=False deduplicate_src_table=False deduplicate_dst_table=True insert_unique_blocks=True +table_src count 8 +table_dst count 6 +0 +0 +table_src count 16 +table_dst count 6 +0 +0 +OK + +Test case 61: insert_method=InsertValues engine=MergeTree use_insert_token=False single_thread=False deduplicate_src_table=False deduplicate_dst_table=True insert_unique_blocks=False +table_src count 8 +table_dst count 2 +0 +0 +table_src count 16 +table_dst count 2 +0 +0 +OK + +Test case 62: insert_method=InsertValues engine=MergeTree use_insert_token=False single_thread=False deduplicate_src_table=False deduplicate_dst_table=False insert_unique_blocks=True +table_src count 8 +table_dst count 6 +0 +0 +table_src count 16 +table_dst count 12 +0 +0 +OK + +Test case 63: insert_method=InsertValues engine=MergeTree use_insert_token=False single_thread=False deduplicate_src_table=False deduplicate_dst_table=False insert_unique_blocks=False +table_src count 8 +table_dst count 16 +0 +0 +table_src count 16 +table_dst count 32 +0 +0 +OK + +All cases executed diff --git a/tests/queries/0_stateless/03008_deduplication_several_mv_into_one_table_nonreplicated.sh b/tests/queries/0_stateless/03008_deduplication_several_mv_into_one_table_nonreplicated.sh new file mode 100755 index 00000000000..a9a135d6839 --- /dev/null +++ b/tests/queries/0_stateless/03008_deduplication_several_mv_into_one_table_nonreplicated.sh @@ -0,0 +1,59 @@ +#!/usr/bin/env bash +# Tags: long, no-fasttest, no-parallel, no-asan + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +ENGINE="MergeTree" + +RUN_ONLY="" +#RUN_ONLY="Test case 17: insert_method=InsertSelect engine=MergeTree use_insert_token=False single_thread=True deduplicate_src_table=True deduplicate_dst_table=True insert_unique_blocks=False" + +i=0 +for insert_method in "InsertSelect" "InsertValues"; do + for use_insert_token in "True" "False"; do + for single_thread in "True" "False"; do + for deduplicate_src_table in "True" "False"; do + for deduplicate_dst_table in "True" "False"; do + for insert_unique_blocks in "True" "False"; do + + THIS_RUN="Test case $i:" + THIS_RUN+=" insert_method=$insert_method" + THIS_RUN+=" engine=$ENGINE" + THIS_RUN+=" use_insert_token=$use_insert_token" + THIS_RUN+=" single_thread=$single_thread" + THIS_RUN+=" deduplicate_src_table=$deduplicate_src_table" + THIS_RUN+=" deduplicate_dst_table=$deduplicate_dst_table" + THIS_RUN+=" insert_unique_blocks=$insert_unique_blocks" + + i=$((i+1)) + + echo + if [ -n "$RUN_ONLY" ] && [ "$RUN_ONLY" != "$THIS_RUN" ]; then + echo "skip $THIS_RUN" + continue + fi + echo "$THIS_RUN" + + $CLICKHOUSE_CLIENT --max_insert_block_size 1 -nmq " + $(python3 $CURDIR/03008_deduplication.python several_mv_into_one_table \ + --insert-method $insert_method \ + --table-engine $ENGINE \ + --use-insert-token $use_insert_token \ + --single-thread $single_thread \ + --deduplicate-src-table $deduplicate_src_table \ + --deduplicate-dst-table $deduplicate_dst_table \ + --insert-unique-blocks $insert_unique_blocks \ + --get-logs false \ + ) + " && echo OK || echo FAIL + done + done + done + done + done +done + +echo +echo "All cases executed" diff --git a/tests/queries/0_stateless/03008_deduplication_several_mv_into_one_table_replicated.reference b/tests/queries/0_stateless/03008_deduplication_several_mv_into_one_table_replicated.reference new file mode 100644 index 00000000000..1921103f49e --- /dev/null +++ b/tests/queries/0_stateless/03008_deduplication_several_mv_into_one_table_replicated.reference @@ -0,0 +1,706 @@ + +Test case 0: insert_method=InsertSelect engine=ReplicatedMergeTree use_insert_token=True single_thread=True deduplicate_src_table=True deduplicate_dst_table=True insert_unique_blocks=True +table_src count 8 +table_dst count 6 +0 +0 +table_src count 8 +table_dst count 6 +0 +0 +OK + +Test case 1: insert_method=InsertSelect engine=ReplicatedMergeTree use_insert_token=True single_thread=True deduplicate_src_table=True deduplicate_dst_table=True insert_unique_blocks=False +table_src count 8 +table_dst count 16 +0 +0 +table_src count 8 +table_dst count 16 +0 +0 +OK + +Test case 2: insert_method=InsertSelect engine=ReplicatedMergeTree use_insert_token=True single_thread=True deduplicate_src_table=True deduplicate_dst_table=False insert_unique_blocks=True +table_src count 8 +table_dst count 6 +0 +0 +table_src count 8 +table_dst count 12 +0 +0 +OK + +Test case 3: insert_method=InsertSelect engine=ReplicatedMergeTree use_insert_token=True single_thread=True deduplicate_src_table=True deduplicate_dst_table=False insert_unique_blocks=False +table_src count 8 +table_dst count 16 +0 +0 +table_src count 8 +table_dst count 32 +0 +0 +OK + +Test case 4: insert_method=InsertSelect engine=ReplicatedMergeTree use_insert_token=True single_thread=True deduplicate_src_table=False deduplicate_dst_table=True insert_unique_blocks=True +table_src count 8 +table_dst count 6 +0 +0 +table_src count 16 +table_dst count 6 +0 +0 +OK + +Test case 5: insert_method=InsertSelect engine=ReplicatedMergeTree use_insert_token=True single_thread=True deduplicate_src_table=False deduplicate_dst_table=True insert_unique_blocks=False +table_src count 8 +table_dst count 16 +0 +0 +table_src count 16 +table_dst count 16 +0 +0 +OK + +Test case 6: insert_method=InsertSelect engine=ReplicatedMergeTree use_insert_token=True single_thread=True deduplicate_src_table=False deduplicate_dst_table=False insert_unique_blocks=True +table_src count 8 +table_dst count 6 +0 +0 +table_src count 16 +table_dst count 12 +0 +0 +OK + +Test case 7: insert_method=InsertSelect engine=ReplicatedMergeTree use_insert_token=True single_thread=True deduplicate_src_table=False deduplicate_dst_table=False insert_unique_blocks=False +table_src count 8 +table_dst count 16 +0 +0 +table_src count 16 +table_dst count 32 +0 +0 +OK + +Test case 8: insert_method=InsertSelect engine=ReplicatedMergeTree use_insert_token=True single_thread=False deduplicate_src_table=True deduplicate_dst_table=True insert_unique_blocks=True +table_src count 8 +table_dst count 6 +0 +0 +table_src count 8 +table_dst count 6 +0 +0 +OK + +Test case 9: insert_method=InsertSelect engine=ReplicatedMergeTree use_insert_token=True single_thread=False deduplicate_src_table=True deduplicate_dst_table=True insert_unique_blocks=False +table_src count 8 +table_dst count 16 +0 +0 +table_src count 8 +table_dst count 16 +0 +0 +OK + +Test case 10: insert_method=InsertSelect engine=ReplicatedMergeTree use_insert_token=True single_thread=False deduplicate_src_table=True deduplicate_dst_table=False insert_unique_blocks=True +table_src count 8 +table_dst count 6 +0 +0 +table_src count 8 +table_dst count 12 +0 +0 +OK + +Test case 11: insert_method=InsertSelect engine=ReplicatedMergeTree use_insert_token=True single_thread=False deduplicate_src_table=True deduplicate_dst_table=False insert_unique_blocks=False +table_src count 8 +table_dst count 16 +0 +0 +table_src count 8 +table_dst count 32 +0 +0 +OK + +Test case 12: insert_method=InsertSelect engine=ReplicatedMergeTree use_insert_token=True single_thread=False deduplicate_src_table=False deduplicate_dst_table=True insert_unique_blocks=True +table_src count 8 +table_dst count 6 +0 +0 +table_src count 16 +table_dst count 6 +0 +0 +OK + +Test case 13: insert_method=InsertSelect engine=ReplicatedMergeTree use_insert_token=True single_thread=False deduplicate_src_table=False deduplicate_dst_table=True insert_unique_blocks=False +table_src count 8 +table_dst count 16 +0 +0 +table_src count 16 +table_dst count 16 +0 +0 +OK + +Test case 14: insert_method=InsertSelect engine=ReplicatedMergeTree use_insert_token=True single_thread=False deduplicate_src_table=False deduplicate_dst_table=False insert_unique_blocks=True +table_src count 8 +table_dst count 6 +0 +0 +table_src count 16 +table_dst count 12 +0 +0 +OK + +Test case 15: insert_method=InsertSelect engine=ReplicatedMergeTree use_insert_token=True single_thread=False deduplicate_src_table=False deduplicate_dst_table=False insert_unique_blocks=False +table_src count 8 +table_dst count 16 +0 +0 +table_src count 16 +table_dst count 32 +0 +0 +OK + +Test case 16: insert_method=InsertSelect engine=ReplicatedMergeTree use_insert_token=False single_thread=True deduplicate_src_table=True deduplicate_dst_table=True insert_unique_blocks=True +table_src count 8 +table_dst count 6 +0 +0 +table_src count 8 +table_dst count 6 +0 +0 +OK + +Test case 17: insert_method=InsertSelect engine=ReplicatedMergeTree use_insert_token=False single_thread=True deduplicate_src_table=True deduplicate_dst_table=True insert_unique_blocks=False +table_src count 1 +table_dst count 2 +0 +0 +table_src count 1 +table_dst count 2 +0 +0 +OK + +Test case 18: insert_method=InsertSelect engine=ReplicatedMergeTree use_insert_token=False single_thread=True deduplicate_src_table=True deduplicate_dst_table=False insert_unique_blocks=True +table_src count 8 +table_dst count 6 +0 +0 +table_src count 8 +table_dst count 12 +0 +0 +OK + +Test case 19: insert_method=InsertSelect engine=ReplicatedMergeTree use_insert_token=False single_thread=True deduplicate_src_table=True deduplicate_dst_table=False insert_unique_blocks=False +table_src count 1 +table_dst count 16 +0 +0 +table_src count 1 +table_dst count 32 +0 +0 +OK + +Test case 20: insert_method=InsertSelect engine=ReplicatedMergeTree use_insert_token=False single_thread=True deduplicate_src_table=False deduplicate_dst_table=True insert_unique_blocks=True +table_src count 8 +table_dst count 6 +0 +0 +table_src count 16 +table_dst count 6 +0 +0 +OK + +Test case 21: insert_method=InsertSelect engine=ReplicatedMergeTree use_insert_token=False single_thread=True deduplicate_src_table=False deduplicate_dst_table=True insert_unique_blocks=False +table_src count 8 +table_dst count 2 +0 +0 +table_src count 16 +table_dst count 2 +0 +0 +OK + +Test case 22: insert_method=InsertSelect engine=ReplicatedMergeTree use_insert_token=False single_thread=True deduplicate_src_table=False deduplicate_dst_table=False insert_unique_blocks=True +table_src count 8 +table_dst count 6 +0 +0 +table_src count 16 +table_dst count 12 +0 +0 +OK + +Test case 23: insert_method=InsertSelect engine=ReplicatedMergeTree use_insert_token=False single_thread=True deduplicate_src_table=False deduplicate_dst_table=False insert_unique_blocks=False +table_src count 8 +table_dst count 16 +0 +0 +table_src count 16 +table_dst count 32 +0 +0 +OK + +Test case 24: insert_method=InsertSelect engine=ReplicatedMergeTree use_insert_token=False single_thread=False deduplicate_src_table=True deduplicate_dst_table=True insert_unique_blocks=True +table_src count 8 +table_dst count 6 +0 +0 +table_src count 8 +table_dst count 6 +0 +0 +OK + +Test case 25: insert_method=InsertSelect engine=ReplicatedMergeTree use_insert_token=False single_thread=False deduplicate_src_table=True deduplicate_dst_table=True insert_unique_blocks=False +table_src count 1 +table_dst count 2 +0 +0 +table_src count 1 +table_dst count 2 +0 +0 +OK + +Test case 26: insert_method=InsertSelect engine=ReplicatedMergeTree use_insert_token=False single_thread=False deduplicate_src_table=True deduplicate_dst_table=False insert_unique_blocks=True +table_src count 8 +table_dst count 6 +0 +0 +table_src count 8 +table_dst count 12 +0 +0 +OK + +Test case 27: insert_method=InsertSelect engine=ReplicatedMergeTree use_insert_token=False single_thread=False deduplicate_src_table=True deduplicate_dst_table=False insert_unique_blocks=False +table_src count 1 +table_dst count 16 +0 +0 +table_src count 1 +table_dst count 32 +0 +0 +OK + +Test case 28: insert_method=InsertSelect engine=ReplicatedMergeTree use_insert_token=False single_thread=False deduplicate_src_table=False deduplicate_dst_table=True insert_unique_blocks=True +table_src count 8 +table_dst count 6 +0 +0 +table_src count 16 +table_dst count 6 +0 +0 +OK + +Test case 29: insert_method=InsertSelect engine=ReplicatedMergeTree use_insert_token=False single_thread=False deduplicate_src_table=False deduplicate_dst_table=True insert_unique_blocks=False +table_src count 8 +table_dst count 2 +0 +0 +table_src count 16 +table_dst count 2 +0 +0 +OK + +Test case 30: insert_method=InsertSelect engine=ReplicatedMergeTree use_insert_token=False single_thread=False deduplicate_src_table=False deduplicate_dst_table=False insert_unique_blocks=True +table_src count 8 +table_dst count 6 +0 +0 +table_src count 16 +table_dst count 12 +0 +0 +OK + +Test case 31: insert_method=InsertSelect engine=ReplicatedMergeTree use_insert_token=False single_thread=False deduplicate_src_table=False deduplicate_dst_table=False insert_unique_blocks=False +table_src count 8 +table_dst count 16 +0 +0 +table_src count 16 +table_dst count 32 +0 +0 +OK + +Test case 32: insert_method=InsertValues engine=ReplicatedMergeTree use_insert_token=True single_thread=True deduplicate_src_table=True deduplicate_dst_table=True insert_unique_blocks=True +table_src count 8 +table_dst count 6 +0 +0 +table_src count 8 +table_dst count 6 +0 +0 +OK + +Test case 33: insert_method=InsertValues engine=ReplicatedMergeTree use_insert_token=True single_thread=True deduplicate_src_table=True deduplicate_dst_table=True insert_unique_blocks=False +table_src count 8 +table_dst count 16 +0 +0 +table_src count 8 +table_dst count 16 +0 +0 +OK + +Test case 34: insert_method=InsertValues engine=ReplicatedMergeTree use_insert_token=True single_thread=True deduplicate_src_table=True deduplicate_dst_table=False insert_unique_blocks=True +table_src count 8 +table_dst count 6 +0 +0 +table_src count 8 +table_dst count 12 +0 +0 +OK + +Test case 35: insert_method=InsertValues engine=ReplicatedMergeTree use_insert_token=True single_thread=True deduplicate_src_table=True deduplicate_dst_table=False insert_unique_blocks=False +table_src count 8 +table_dst count 16 +0 +0 +table_src count 8 +table_dst count 32 +0 +0 +OK + +Test case 36: insert_method=InsertValues engine=ReplicatedMergeTree use_insert_token=True single_thread=True deduplicate_src_table=False deduplicate_dst_table=True insert_unique_blocks=True +table_src count 8 +table_dst count 6 +0 +0 +table_src count 16 +table_dst count 6 +0 +0 +OK + +Test case 37: insert_method=InsertValues engine=ReplicatedMergeTree use_insert_token=True single_thread=True deduplicate_src_table=False deduplicate_dst_table=True insert_unique_blocks=False +table_src count 8 +table_dst count 16 +0 +0 +table_src count 16 +table_dst count 16 +0 +0 +OK + +Test case 38: insert_method=InsertValues engine=ReplicatedMergeTree use_insert_token=True single_thread=True deduplicate_src_table=False deduplicate_dst_table=False insert_unique_blocks=True +table_src count 8 +table_dst count 6 +0 +0 +table_src count 16 +table_dst count 12 +0 +0 +OK + +Test case 39: insert_method=InsertValues engine=ReplicatedMergeTree use_insert_token=True single_thread=True deduplicate_src_table=False deduplicate_dst_table=False insert_unique_blocks=False +table_src count 8 +table_dst count 16 +0 +0 +table_src count 16 +table_dst count 32 +0 +0 +OK + +Test case 40: insert_method=InsertValues engine=ReplicatedMergeTree use_insert_token=True single_thread=False deduplicate_src_table=True deduplicate_dst_table=True insert_unique_blocks=True +table_src count 8 +table_dst count 6 +0 +0 +table_src count 8 +table_dst count 6 +0 +0 +OK + +Test case 41: insert_method=InsertValues engine=ReplicatedMergeTree use_insert_token=True single_thread=False deduplicate_src_table=True deduplicate_dst_table=True insert_unique_blocks=False +table_src count 8 +table_dst count 16 +0 +0 +table_src count 8 +table_dst count 16 +0 +0 +OK + +Test case 42: insert_method=InsertValues engine=ReplicatedMergeTree use_insert_token=True single_thread=False deduplicate_src_table=True deduplicate_dst_table=False insert_unique_blocks=True +table_src count 8 +table_dst count 6 +0 +0 +table_src count 8 +table_dst count 12 +0 +0 +OK + +Test case 43: insert_method=InsertValues engine=ReplicatedMergeTree use_insert_token=True single_thread=False deduplicate_src_table=True deduplicate_dst_table=False insert_unique_blocks=False +table_src count 8 +table_dst count 16 +0 +0 +table_src count 8 +table_dst count 32 +0 +0 +OK + +Test case 44: insert_method=InsertValues engine=ReplicatedMergeTree use_insert_token=True single_thread=False deduplicate_src_table=False deduplicate_dst_table=True insert_unique_blocks=True +table_src count 8 +table_dst count 6 +0 +0 +table_src count 16 +table_dst count 6 +0 +0 +OK + +Test case 45: insert_method=InsertValues engine=ReplicatedMergeTree use_insert_token=True single_thread=False deduplicate_src_table=False deduplicate_dst_table=True insert_unique_blocks=False +table_src count 8 +table_dst count 16 +0 +0 +table_src count 16 +table_dst count 16 +0 +0 +OK + +Test case 46: insert_method=InsertValues engine=ReplicatedMergeTree use_insert_token=True single_thread=False deduplicate_src_table=False deduplicate_dst_table=False insert_unique_blocks=True +table_src count 8 +table_dst count 6 +0 +0 +table_src count 16 +table_dst count 12 +0 +0 +OK + +Test case 47: insert_method=InsertValues engine=ReplicatedMergeTree use_insert_token=True single_thread=False deduplicate_src_table=False deduplicate_dst_table=False insert_unique_blocks=False +table_src count 8 +table_dst count 16 +0 +0 +table_src count 16 +table_dst count 32 +0 +0 +OK + +Test case 48: insert_method=InsertValues engine=ReplicatedMergeTree use_insert_token=False single_thread=True deduplicate_src_table=True deduplicate_dst_table=True insert_unique_blocks=True +table_src count 8 +table_dst count 6 +0 +0 +table_src count 8 +table_dst count 6 +0 +0 +OK + +Test case 49: insert_method=InsertValues engine=ReplicatedMergeTree use_insert_token=False single_thread=True deduplicate_src_table=True deduplicate_dst_table=True insert_unique_blocks=False +table_src count 1 +table_dst count 2 +0 +0 +table_src count 1 +table_dst count 2 +0 +0 +OK + +Test case 50: insert_method=InsertValues engine=ReplicatedMergeTree use_insert_token=False single_thread=True deduplicate_src_table=True deduplicate_dst_table=False insert_unique_blocks=True +table_src count 8 +table_dst count 6 +0 +0 +table_src count 8 +table_dst count 12 +0 +0 +OK + +Test case 51: insert_method=InsertValues engine=ReplicatedMergeTree use_insert_token=False single_thread=True deduplicate_src_table=True deduplicate_dst_table=False insert_unique_blocks=False +table_src count 1 +table_dst count 16 +0 +0 +table_src count 1 +table_dst count 32 +0 +0 +OK + +Test case 52: insert_method=InsertValues engine=ReplicatedMergeTree use_insert_token=False single_thread=True deduplicate_src_table=False deduplicate_dst_table=True insert_unique_blocks=True +table_src count 8 +table_dst count 6 +0 +0 +table_src count 16 +table_dst count 6 +0 +0 +OK + +Test case 53: insert_method=InsertValues engine=ReplicatedMergeTree use_insert_token=False single_thread=True deduplicate_src_table=False deduplicate_dst_table=True insert_unique_blocks=False +table_src count 8 +table_dst count 2 +0 +0 +table_src count 16 +table_dst count 2 +0 +0 +OK + +Test case 54: insert_method=InsertValues engine=ReplicatedMergeTree use_insert_token=False single_thread=True deduplicate_src_table=False deduplicate_dst_table=False insert_unique_blocks=True +table_src count 8 +table_dst count 6 +0 +0 +table_src count 16 +table_dst count 12 +0 +0 +OK + +Test case 55: insert_method=InsertValues engine=ReplicatedMergeTree use_insert_token=False single_thread=True deduplicate_src_table=False deduplicate_dst_table=False insert_unique_blocks=False +table_src count 8 +table_dst count 16 +0 +0 +table_src count 16 +table_dst count 32 +0 +0 +OK + +Test case 56: insert_method=InsertValues engine=ReplicatedMergeTree use_insert_token=False single_thread=False deduplicate_src_table=True deduplicate_dst_table=True insert_unique_blocks=True +table_src count 8 +table_dst count 6 +0 +0 +table_src count 8 +table_dst count 6 +0 +0 +OK + +Test case 57: insert_method=InsertValues engine=ReplicatedMergeTree use_insert_token=False single_thread=False deduplicate_src_table=True deduplicate_dst_table=True insert_unique_blocks=False +table_src count 1 +table_dst count 2 +0 +0 +table_src count 1 +table_dst count 2 +0 +0 +OK + +Test case 58: insert_method=InsertValues engine=ReplicatedMergeTree use_insert_token=False single_thread=False deduplicate_src_table=True deduplicate_dst_table=False insert_unique_blocks=True +table_src count 8 +table_dst count 6 +0 +0 +table_src count 8 +table_dst count 12 +0 +0 +OK + +Test case 59: insert_method=InsertValues engine=ReplicatedMergeTree use_insert_token=False single_thread=False deduplicate_src_table=True deduplicate_dst_table=False insert_unique_blocks=False +table_src count 1 +table_dst count 16 +0 +0 +table_src count 1 +table_dst count 32 +0 +0 +OK + +Test case 60: insert_method=InsertValues engine=ReplicatedMergeTree use_insert_token=False single_thread=False deduplicate_src_table=False deduplicate_dst_table=True insert_unique_blocks=True +table_src count 8 +table_dst count 6 +0 +0 +table_src count 16 +table_dst count 6 +0 +0 +OK + +Test case 61: insert_method=InsertValues engine=ReplicatedMergeTree use_insert_token=False single_thread=False deduplicate_src_table=False deduplicate_dst_table=True insert_unique_blocks=False +table_src count 8 +table_dst count 2 +0 +0 +table_src count 16 +table_dst count 2 +0 +0 +OK + +Test case 62: insert_method=InsertValues engine=ReplicatedMergeTree use_insert_token=False single_thread=False deduplicate_src_table=False deduplicate_dst_table=False insert_unique_blocks=True +table_src count 8 +table_dst count 6 +0 +0 +table_src count 16 +table_dst count 12 +0 +0 +OK + +Test case 63: insert_method=InsertValues engine=ReplicatedMergeTree use_insert_token=False single_thread=False deduplicate_src_table=False deduplicate_dst_table=False insert_unique_blocks=False +table_src count 8 +table_dst count 16 +0 +0 +table_src count 16 +table_dst count 32 +0 +0 +OK + +All cases executed diff --git a/tests/queries/0_stateless/03008_deduplication_several_mv_into_one_table_replicated.sh b/tests/queries/0_stateless/03008_deduplication_several_mv_into_one_table_replicated.sh new file mode 100755 index 00000000000..49d556e70f9 --- /dev/null +++ b/tests/queries/0_stateless/03008_deduplication_several_mv_into_one_table_replicated.sh @@ -0,0 +1,59 @@ +#!/usr/bin/env bash +# Tags: long, no-fasttest, no-parallel, no-asan + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +ENGINE="ReplicatedMergeTree" + +RUN_ONLY="" +#RUN_ONLY="Test case 17: insert_method=InsertSelect engine=ReplicatedMergeTree use_insert_token=False single_thread=True deduplicate_src_table=True deduplicate_dst_table=True insert_unique_blocks=False" + +i=0 +for insert_method in "InsertSelect" "InsertValues"; do + for use_insert_token in "True" "False"; do + for single_thread in "True" "False"; do + for deduplicate_src_table in "True" "False"; do + for deduplicate_dst_table in "True" "False"; do + for insert_unique_blocks in "True" "False"; do + + THIS_RUN="Test case $i:" + THIS_RUN+=" insert_method=$insert_method" + THIS_RUN+=" engine=$ENGINE" + THIS_RUN+=" use_insert_token=$use_insert_token" + THIS_RUN+=" single_thread=$single_thread" + THIS_RUN+=" deduplicate_src_table=$deduplicate_src_table" + THIS_RUN+=" deduplicate_dst_table=$deduplicate_dst_table" + THIS_RUN+=" insert_unique_blocks=$insert_unique_blocks" + + i=$((i+1)) + + echo + if [ -n "$RUN_ONLY" ] && [ "$RUN_ONLY" != "$THIS_RUN" ]; then + echo "skip $THIS_RUN" + continue + fi + echo "$THIS_RUN" + + $CLICKHOUSE_CLIENT --max_insert_block_size 1 -nmq " + $(python3 $CURDIR/03008_deduplication.python several_mv_into_one_table \ + --insert-method $insert_method \ + --table-engine $ENGINE \ + --use-insert-token $use_insert_token \ + --single-thread $single_thread \ + --deduplicate-src-table $deduplicate_src_table \ + --deduplicate-dst-table $deduplicate_dst_table \ + --insert-unique-blocks $insert_unique_blocks \ + --get-logs false \ + ) + " && echo OK || echo FAIL + done + done + done + done + done +done + +echo +echo "All cases executed" diff --git a/tests/queries/0_stateless/03008_deduplication_wrong_mv.reference b/tests/queries/0_stateless/03008_deduplication_wrong_mv.reference new file mode 100644 index 00000000000..89b4f324d5e --- /dev/null +++ b/tests/queries/0_stateless/03008_deduplication_wrong_mv.reference @@ -0,0 +1,15 @@ +-- { echo ON } +CREATE TABLE src (x UInt8) ENGINE = Memory; +CREATE TABLE dst (x UInt8) ENGINE = Memory; +CREATE MATERIALIZED VIEW mv1 TO dst AS SELECT * FROM src; +INSERT INTO src VALUES (0); +SELECT * from dst; +0 +TRUNCATE TABLE dst; +--DROP TABLE src SYNC; +--CREATE TABLE src (y String) ENGINE = MergeTree order by tuple(); +ALTER TABLE src ADD COLUMN y UInt8; +ALTER TABLE src DROP COLUMN x; +INSERT INTO src VALUES (0); +SELECT * from dst; +0 diff --git a/tests/queries/0_stateless/03008_deduplication_wrong_mv.sql b/tests/queries/0_stateless/03008_deduplication_wrong_mv.sql new file mode 100644 index 00000000000..dd74c38ab17 --- /dev/null +++ b/tests/queries/0_stateless/03008_deduplication_wrong_mv.sql @@ -0,0 +1,21 @@ +DROP TABLE IF EXISTS mv; +DROP TABLE IF EXISTS src; +DROP TABLE IF EXISTS dst; + +-- { echo ON } +CREATE TABLE src (x UInt8) ENGINE = Memory; +CREATE TABLE dst (x UInt8) ENGINE = Memory; +CREATE MATERIALIZED VIEW mv1 TO dst AS SELECT * FROM src; + +INSERT INTO src VALUES (0); +SELECT * from dst; + +TRUNCATE TABLE dst; + +--DROP TABLE src SYNC; +--CREATE TABLE src (y String) ENGINE = MergeTree order by tuple(); +ALTER TABLE src ADD COLUMN y UInt8; +ALTER TABLE src DROP COLUMN x; + +INSERT INTO src VALUES (0); +SELECT * from dst; diff --git a/tests/queries/0_stateless/03035_max_insert_threads_support.sh b/tests/queries/0_stateless/03035_max_insert_threads_support.sh index 1e6bfb414d8..cedb651a430 100755 --- a/tests/queries/0_stateless/03035_max_insert_threads_support.sh +++ b/tests/queries/0_stateless/03035_max_insert_threads_support.sh @@ -8,7 +8,7 @@ DATA_FILE="data_$CLICKHOUSE_TEST_UNIQUE_NAME.csv" $CLICKHOUSE_CLIENT --max_insert_threads=4 --query=" EXPLAIN PIPELINE INSERT INTO FUNCTION file('$DATA_FILE') SELECT * FROM numbers_mt(1000000) ORDER BY number DESC -" | grep -o MaterializingTransform | wc -l +" | grep -o StorageFileSink | wc -l DATA_FILE_PATH=$($CLICKHOUSE_CLIENT_BINARY --query "select _path from file('$DATA_FILE', 'One')") rm $DATA_FILE_PATH diff --git a/tests/queries/0_stateless/03130_generateSnowflakeId.reference b/tests/queries/0_stateless/03130_generateSnowflakeId.reference index 39669d21bee..fd264f00d36 100644 --- a/tests/queries/0_stateless/03130_generateSnowflakeId.reference +++ b/tests/queries/0_stateless/03130_generateSnowflakeId.reference @@ -1,5 +1,11 @@ +Negative tests +The first bit must be zero 1 +Test disabling of common subexpression elimination via first parameter 0 0 1 +Test user-provided machine ID +1 +Generated Snowflake IDs are unique 100 diff --git a/tests/queries/0_stateless/03130_generateSnowflakeId.sql b/tests/queries/0_stateless/03130_generateSnowflakeId.sql index 0717c81aa0d..6c0a6cc0f2c 100644 --- a/tests/queries/0_stateless/03130_generateSnowflakeId.sql +++ b/tests/queries/0_stateless/03130_generateSnowflakeId.sql @@ -1,13 +1,22 @@ -- Test SQL function 'generateSnowflakeID' -SELECT bitAnd(bitShiftRight(toUInt64(generateSnowflakeID()), 63), 1) = 0; -- check first bit is zero +SELECT 'Negative tests'; +SELECT generateSnowflakeID(1, 2, 3); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } +SELECT generateSnowflakeID(1, 'not_an_int'); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } +SELECT generateSnowflakeID(1, materialize(2)); -- { serverError ILLEGAL_COLUMN } + SELECT 'The first bit must be zero'; +SELECT bitAnd(bitShiftRight(generateSnowflakeID(), 63), 1) = 0; + +SELECT 'Test disabling of common subexpression elimination via first parameter'; SELECT generateSnowflakeID(1) = generateSnowflakeID(2); -- disabled common subexpression elimination --> lhs != rhs SELECT generateSnowflakeID() = generateSnowflakeID(1); -- same as ^^ -SELECT generateSnowflakeID(1) = generateSnowflakeID(1); -- enabled common subexpression elimination +SELECT generateSnowflakeID(1) = generateSnowflakeID(1); -- with common subexpression elimination -SELECT generateSnowflakeID(1, 2); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } +SELECT 'Test user-provided machine ID'; +SELECT bitAnd(bitShiftRight(generateSnowflakeID(1, 123), 12), 1024 - 1) = 123; -- the machine id is actually set in the generated snowflake ID (1024 = 2^10) +SELECT 'Generated Snowflake IDs are unique'; SELECT count(*) FROM ( diff --git a/tests/queries/0_stateless/03199_dictionary_table_access.reference b/tests/queries/0_stateless/03199_dictionary_table_access.reference new file mode 100644 index 00000000000..4a703b3be84 --- /dev/null +++ b/tests/queries/0_stateless/03199_dictionary_table_access.reference @@ -0,0 +1,2 @@ +ACCESS_DENIED +ACCESS_DENIED diff --git a/tests/queries/0_stateless/03199_dictionary_table_access.sh b/tests/queries/0_stateless/03199_dictionary_table_access.sh new file mode 100755 index 00000000000..952b466b5da --- /dev/null +++ b/tests/queries/0_stateless/03199_dictionary_table_access.sh @@ -0,0 +1,41 @@ +#!/usr/bin/env bash + +CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CUR_DIR"/../shell_config.sh + +username="user_${CLICKHOUSE_TEST_UNIQUE_NAME}" +dictname="dict_${CLICKHOUSE_TEST_UNIQUE_NAME}" +dicttablename="dict_table_${CLICKHOUSE_TEST_UNIQUE_NAME}" + +${CLICKHOUSE_CLIENT} -nm --query " + CREATE DICTIONARY IF NOT EXISTS ${dictname} + ( + id UInt64, + value UInt64 + ) + PRIMARY KEY id + SOURCE(NULL()) + LAYOUT(FLAT()) + LIFETIME(MIN 0 MAX 1000); + CREATE USER IF NOT EXISTS ${username} NOT IDENTIFIED; + GRANT SELECT, CREATE TEMPORARY TABLE ON *.* to ${username}; + SELECT * FROM ${dictname}; + CREATE TABLE ${dicttablename} (id UInt64, value UInt64) + ENGINE = Dictionary(${CLICKHOUSE_DATABASE}.${dictname}); + SELECT * FROM ${dicttablename}; +" + +$CLICKHOUSE_CLIENT -nm --user="${username}" --query " + SELECT * FROM ${dictname}; +" 2>&1 | grep -o ACCESS_DENIED | uniq + +$CLICKHOUSE_CLIENT -nm --user="${username}" --query " + SELECT * FROM ${dicttablename}; +" 2>&1 | grep -o ACCESS_DENIED | uniq + +${CLICKHOUSE_CLIENT} -nm --query " + DROP TABLE IF EXISTS ${dicttablename} SYNC; + DROP DICTIONARY IF EXISTS ${dictname}; + DROP USER IF EXISTS ${username}; +" diff --git a/tests/queries/0_stateless/03203_optimize_disjunctions_chain_to_in.reference b/tests/queries/0_stateless/03203_optimize_disjunctions_chain_to_in.reference new file mode 100644 index 00000000000..353c8f98b03 --- /dev/null +++ b/tests/queries/0_stateless/03203_optimize_disjunctions_chain_to_in.reference @@ -0,0 +1,2 @@ +2020-01-01 +2020-01-02 diff --git a/tests/queries/0_stateless/03203_optimize_disjunctions_chain_to_in.sql b/tests/queries/0_stateless/03203_optimize_disjunctions_chain_to_in.sql new file mode 100644 index 00000000000..f9ba28bcd60 --- /dev/null +++ b/tests/queries/0_stateless/03203_optimize_disjunctions_chain_to_in.sql @@ -0,0 +1,12 @@ +SET allow_experimental_analyzer=1; +CREATE TABLE foo (i Date) ENGINE MergeTree ORDER BY i; +INSERT INTO foo VALUES ('2020-01-01'); +INSERT INTO foo VALUES ('2020-01-02'); + +SET optimize_min_equality_disjunction_chain_length = 3; +SELECT * +FROM foo +WHERE (foo.i = parseDateTimeBestEffort('2020-01-01')) + OR (foo.i = parseDateTimeBestEffort('2020-01-02')) + OR (foo.i = parseDateTimeBestEffort('2020-01-03')) +ORDER BY foo.i ASC diff --git a/tests/result b/tests/result new file mode 100644 index 00000000000..b76f44f1e6a --- /dev/null +++ b/tests/result @@ -0,0 +1,12 @@ +Using queries from 'queries' directory +Connecting to ClickHouse server...... OK +Connected to server 24.7.1.1 @ 246f421f2402799fd11b22a608b4d0d497cb8438 chesema-processor-onCancel + +Running 1 stateless tests (MainProcess). + +00993_system_parts_race_condition_drop_zookeeper: [ OK ] + +1 tests passed. 0 tests skipped. 124.59 s elapsed (MainProcess). + +0 tests passed. 0 tests skipped. 0.00 s elapsed (MainProcess). +All tests have finished.