From 90666a43a893aa552240bf64a3a260117d2cc407 Mon Sep 17 00:00:00 2001 From: Nicolae Vartolomei Date: Tue, 3 May 2022 14:16:03 +0000 Subject: [PATCH 001/127] CLICKHOUSE-2153: Move new_data_part shared pointer to allow Outdated part to be deleted immediately on mutation failure --- src/Storages/MergeTree/MergeTreeSettings.h | 3 ++ .../MergeTree/MutateFromLogEntryTask.cpp | 8 ++++ src/Storages/MergeTree/MutateTask.cpp | 2 +- .../test_mutation_fetch_fallback/test.py | 40 +++++++++++++++++++ 4 files changed, 52 insertions(+), 1 deletion(-) create mode 100644 tests/integration/test_mutation_fetch_fallback/test.py diff --git a/src/Storages/MergeTree/MergeTreeSettings.h b/src/Storages/MergeTree/MergeTreeSettings.h index 07659b1c9dc..ba9ccdccb00 100644 --- a/src/Storages/MergeTree/MergeTreeSettings.h +++ b/src/Storages/MergeTree/MergeTreeSettings.h @@ -137,6 +137,9 @@ struct Settings; M(UInt64, min_bytes_to_rebalance_partition_over_jbod, 0, "Minimal amount of bytes to enable part rebalance over JBOD array (0 - disabled).", 0) \ M(Bool, check_sample_column_is_correct, true, "Check columns or columns by hash for sampling are unsigned integer.", 0) \ \ + /** Flags that are relevant only in testing. */ \ + M(Bool, testing_mutate_corrupt_checksums, false, "Tetsing only! Corrupt part checksums during mutation to test fetch fallback.", 0) \ + \ /** Experimental/work in progress feature. Unsafe for production. */ \ M(UInt64, part_moves_between_shards_enable, 0, "Experimental/Incomplete feature to move parts between shards. Does not take into account sharding expressions.", 0) \ M(UInt64, part_moves_between_shards_delay_seconds, 30, "Time to wait before/after moving parts between shards.", 0) \ diff --git a/src/Storages/MergeTree/MutateFromLogEntryTask.cpp b/src/Storages/MergeTree/MutateFromLogEntryTask.cpp index fc8b22865c4..207749c05c8 100644 --- a/src/Storages/MergeTree/MutateFromLogEntryTask.cpp +++ b/src/Storages/MergeTree/MutateFromLogEntryTask.cpp @@ -13,6 +13,11 @@ namespace ProfileEvents namespace DB { +namespace ErrorCodes +{ + extern const int CHECKSUM_DOESNT_MATCH; +} + ReplicatedMergeMutateTaskBase::PrepareResult MutateFromLogEntryTask::prepare() { const String & source_part_name = entry.source_parts.at(0); @@ -202,6 +207,9 @@ bool MutateFromLogEntryTask::finalize(ReplicatedMergeMutateTaskBase::PartLogWrit try { + if (unlikely(storage.getSettings()->testing_mutate_corrupt_checksums)) + throw Exception(ErrorCodes::CHECKSUM_DOESNT_MATCH, "Throwing exception because `testing_mutate_corrupt_checksum` is enabled."); + storage.checkPartChecksumsAndCommit(*transaction_ptr, new_part, mutate_task->getHardlinkedFiles()); } catch (const Exception & e) diff --git a/src/Storages/MergeTree/MutateTask.cpp b/src/Storages/MergeTree/MutateTask.cpp index 254bcd9f7f9..38aa040a60c 100644 --- a/src/Storages/MergeTree/MutateTask.cpp +++ b/src/Storages/MergeTree/MutateTask.cpp @@ -1470,7 +1470,7 @@ bool MutateTask::execute() if (task->executeStep()) return true; - promise.set_value(ctx->new_data_part); + promise.set_value(std::move(ctx->new_data_part)); return false; } } diff --git a/tests/integration/test_mutation_fetch_fallback/test.py b/tests/integration/test_mutation_fetch_fallback/test.py new file mode 100644 index 00000000000..f82013eff33 --- /dev/null +++ b/tests/integration/test_mutation_fetch_fallback/test.py @@ -0,0 +1,40 @@ +import pytest + +from helpers.cluster import ClickHouseCluster + +cluster = ClickHouseCluster(__file__) + +node1 = cluster.add_instance("node1", with_zookeeper=True) +node2 = cluster.add_instance("node2", with_zookeeper=True, stay_alive=True) + + +@pytest.fixture(scope="module") +def start_cluster(): + try: + cluster.start() + for i, node in enumerate([node1, node2]): + # node 2 will corrupt checksums, it must be down while node 1 commits mutation + # with correct checksums to zookeeper. + node.query_with_retry( + """CREATE TABLE fetch_fallback (k int, v int, z int) + ENGINE = ReplicatedMergeTree('/clickhouse/tables/test/t0', '{}') + ORDER BY tuple() + SETTINGS testing_mutate_corrupt_checksums = {}""".format(i, i == 1) + ) + yield cluster + + finally: + cluster.shutdown() + + +def test_mutation_fetch_fallback(start_cluster): + node1.query("INSERT INTO fetch_fallback(k, v) VALUES (1, 3), (2, 7), (3, 4)") + + node2.stop_clickhouse() + node1.query("ALTER TABLE fetch_fallback DROP COLUMN z", settings={"mutations_sync": 1}) + + node2.start_clickhouse() + node1.query("SYSTEM SYNC REPLICA fetch_fallback", timeout=10) + node2.query("SYSTEM SYNC REPLICA fetch_fallback", timeout=10) + + node2.contains_in_log("We will download merged part from replica to force byte-identical result.") From 45f1f572f0f5cd94d395d01c1e6925786902d735 Mon Sep 17 00:00:00 2001 From: Nicolae Vartolomei Date: Tue, 20 Sep 2022 19:03:20 +0000 Subject: [PATCH 002/127] Get rid of scary setting for testing fetching byte identical parts --- src/Storages/MergeTree/MergeTreeSettings.h | 3 --- .../MergeTree/MutateFromLogEntryTask.cpp | 3 --- .../test_mutation_fetch_fallback/test.py | 18 ++++++++++-------- 3 files changed, 10 insertions(+), 14 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeSettings.h b/src/Storages/MergeTree/MergeTreeSettings.h index ba9ccdccb00..07659b1c9dc 100644 --- a/src/Storages/MergeTree/MergeTreeSettings.h +++ b/src/Storages/MergeTree/MergeTreeSettings.h @@ -137,9 +137,6 @@ struct Settings; M(UInt64, min_bytes_to_rebalance_partition_over_jbod, 0, "Minimal amount of bytes to enable part rebalance over JBOD array (0 - disabled).", 0) \ M(Bool, check_sample_column_is_correct, true, "Check columns or columns by hash for sampling are unsigned integer.", 0) \ \ - /** Flags that are relevant only in testing. */ \ - M(Bool, testing_mutate_corrupt_checksums, false, "Tetsing only! Corrupt part checksums during mutation to test fetch fallback.", 0) \ - \ /** Experimental/work in progress feature. Unsafe for production. */ \ M(UInt64, part_moves_between_shards_enable, 0, "Experimental/Incomplete feature to move parts between shards. Does not take into account sharding expressions.", 0) \ M(UInt64, part_moves_between_shards_delay_seconds, 30, "Time to wait before/after moving parts between shards.", 0) \ diff --git a/src/Storages/MergeTree/MutateFromLogEntryTask.cpp b/src/Storages/MergeTree/MutateFromLogEntryTask.cpp index 207749c05c8..ce8368552e6 100644 --- a/src/Storages/MergeTree/MutateFromLogEntryTask.cpp +++ b/src/Storages/MergeTree/MutateFromLogEntryTask.cpp @@ -207,9 +207,6 @@ bool MutateFromLogEntryTask::finalize(ReplicatedMergeMutateTaskBase::PartLogWrit try { - if (unlikely(storage.getSettings()->testing_mutate_corrupt_checksums)) - throw Exception(ErrorCodes::CHECKSUM_DOESNT_MATCH, "Throwing exception because `testing_mutate_corrupt_checksum` is enabled."); - storage.checkPartChecksumsAndCommit(*transaction_ptr, new_part, mutate_task->getHardlinkedFiles()); } catch (const Exception & e) diff --git a/tests/integration/test_mutation_fetch_fallback/test.py b/tests/integration/test_mutation_fetch_fallback/test.py index f82013eff33..e2519e9c058 100644 --- a/tests/integration/test_mutation_fetch_fallback/test.py +++ b/tests/integration/test_mutation_fetch_fallback/test.py @@ -12,14 +12,13 @@ node2 = cluster.add_instance("node2", with_zookeeper=True, stay_alive=True) def start_cluster(): try: cluster.start() - for i, node in enumerate([node1, node2]): - # node 2 will corrupt checksums, it must be down while node 1 commits mutation - # with correct checksums to zookeeper. + for ix, node in enumerate([node1, node2]): node.query_with_retry( - """CREATE TABLE fetch_fallback (k int, v int, z int) + """CREATE TABLE fetch_fallback (k int, v int, z String) ENGINE = ReplicatedMergeTree('/clickhouse/tables/test/t0', '{}') - ORDER BY tuple() - SETTINGS testing_mutate_corrupt_checksums = {}""".format(i, i == 1) + ORDER BY tuple()""".format( + ix + ) ) yield cluster @@ -31,10 +30,13 @@ def test_mutation_fetch_fallback(start_cluster): node1.query("INSERT INTO fetch_fallback(k, v) VALUES (1, 3), (2, 7), (3, 4)") node2.stop_clickhouse() - node1.query("ALTER TABLE fetch_fallback DROP COLUMN z", settings={"mutations_sync": 1}) + node1.query( + "ALTER TABLE fetch_fallback UPDATE z = hostName() WHERE 1 = 1", + settings={"mutations_sync": 1, "allow_nondeterministic_mutations": 1}, + ) node2.start_clickhouse() node1.query("SYSTEM SYNC REPLICA fetch_fallback", timeout=10) node2.query("SYSTEM SYNC REPLICA fetch_fallback", timeout=10) - node2.contains_in_log("We will download merged part from replica to force byte-identical result.") + assert node2.contains_in_log("We will download merged part from replica to force byte-identical result.") From 372545b30697ba7d02147411b77a13f44910d3d1 Mon Sep 17 00:00:00 2001 From: Nicolae Vartolomei Date: Tue, 20 Sep 2022 19:28:40 +0000 Subject: [PATCH 003/127] Remove leftover --- src/Storages/MergeTree/MutateFromLogEntryTask.cpp | 5 ----- 1 file changed, 5 deletions(-) diff --git a/src/Storages/MergeTree/MutateFromLogEntryTask.cpp b/src/Storages/MergeTree/MutateFromLogEntryTask.cpp index ce8368552e6..fc8b22865c4 100644 --- a/src/Storages/MergeTree/MutateFromLogEntryTask.cpp +++ b/src/Storages/MergeTree/MutateFromLogEntryTask.cpp @@ -13,11 +13,6 @@ namespace ProfileEvents namespace DB { -namespace ErrorCodes -{ - extern const int CHECKSUM_DOESNT_MATCH; -} - ReplicatedMergeMutateTaskBase::PrepareResult MutateFromLogEntryTask::prepare() { const String & source_part_name = entry.source_parts.at(0); From 3d6fcf4ae5ae32371270c20ba03c0aeba7959786 Mon Sep 17 00:00:00 2001 From: Nicolae Vartolomei Date: Tue, 20 Sep 2022 19:46:07 +0000 Subject: [PATCH 004/127] Explain bad code --- src/Storages/MergeTree/MutateTask.cpp | 3 +++ 1 file changed, 3 insertions(+) diff --git a/src/Storages/MergeTree/MutateTask.cpp b/src/Storages/MergeTree/MutateTask.cpp index 38aa040a60c..cddf95048f8 100644 --- a/src/Storages/MergeTree/MutateTask.cpp +++ b/src/Storages/MergeTree/MutateTask.cpp @@ -1470,6 +1470,9 @@ bool MutateTask::execute() if (task->executeStep()) return true; + // The `new_data_part` is a shared pointer and must be moved to allow + // part deletion (happening in the destructor) in case it is needed + // in `MutateFromLogEntryTask::finalize`. promise.set_value(std::move(ctx->new_data_part)); return false; } From e93a78faee09bd2325395332632c81a4c7164101 Mon Sep 17 00:00:00 2001 From: Nicolae Vartolomei Date: Tue, 20 Sep 2022 19:56:14 +0000 Subject: [PATCH 005/127] Proper comment for bad code --- src/Storages/MergeTree/MutateTask.cpp | 10 ++++++++-- 1 file changed, 8 insertions(+), 2 deletions(-) diff --git a/src/Storages/MergeTree/MutateTask.cpp b/src/Storages/MergeTree/MutateTask.cpp index cddf95048f8..058d593d22d 100644 --- a/src/Storages/MergeTree/MutateTask.cpp +++ b/src/Storages/MergeTree/MutateTask.cpp @@ -1471,8 +1471,14 @@ bool MutateTask::execute() return true; // The `new_data_part` is a shared pointer and must be moved to allow - // part deletion (happening in the destructor) in case it is needed - // in `MutateFromLogEntryTask::finalize`. + // part deletion in case it is needed in `MutateFromLogEntryTask::finalize`. + // + // `tryRemovePartImmediately` requires `std::shared_ptr::unique() == true` + // to delete the part timely. When there are multiple shared pointers, + // only the part state is changed to `Deleting`. + // + // Fetching a byte-identical part (in case of checksum mismatches) will fail with + // `Part ... should be deleted after previous attempt before fetch`. promise.set_value(std::move(ctx->new_data_part)); return false; } From 3d6545cea1ef2e7e459519c1126985c66c722fbf Mon Sep 17 00:00:00 2001 From: nvartolomei Date: Tue, 20 Sep 2022 22:04:20 +0100 Subject: [PATCH 006/127] Reformat test.py to match style --- tests/integration/test_mutation_fetch_fallback/test.py | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/tests/integration/test_mutation_fetch_fallback/test.py b/tests/integration/test_mutation_fetch_fallback/test.py index e2519e9c058..9249ee06e8f 100644 --- a/tests/integration/test_mutation_fetch_fallback/test.py +++ b/tests/integration/test_mutation_fetch_fallback/test.py @@ -39,4 +39,6 @@ def test_mutation_fetch_fallback(start_cluster): node1.query("SYSTEM SYNC REPLICA fetch_fallback", timeout=10) node2.query("SYSTEM SYNC REPLICA fetch_fallback", timeout=10) - assert node2.contains_in_log("We will download merged part from replica to force byte-identical result.") + assert node2.contains_in_log( + "We will download merged part from replica to force byte-identical result." + ) From 842b0bb44b937768dcfa263060a0f50df667895a Mon Sep 17 00:00:00 2001 From: nvartolomei Date: Tue, 20 Sep 2022 22:41:14 +0100 Subject: [PATCH 007/127] Explain why hostName function is used in test mutation --- tests/integration/test_mutation_fetch_fallback/test.py | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/tests/integration/test_mutation_fetch_fallback/test.py b/tests/integration/test_mutation_fetch_fallback/test.py index 9249ee06e8f..91f5d5d380b 100644 --- a/tests/integration/test_mutation_fetch_fallback/test.py +++ b/tests/integration/test_mutation_fetch_fallback/test.py @@ -30,6 +30,11 @@ def test_mutation_fetch_fallback(start_cluster): node1.query("INSERT INTO fetch_fallback(k, v) VALUES (1, 3), (2, 7), (3, 4)") node2.stop_clickhouse() + + # Run a mutation using non-deterministic `hostName` function to produce + # different results on replicas and exercise the code responsible for + # discarding local mutation results and fetching "byte-identical" parts + # instead from the replica which first committed the mutation. node1.query( "ALTER TABLE fetch_fallback UPDATE z = hostName() WHERE 1 = 1", settings={"mutations_sync": 1, "allow_nondeterministic_mutations": 1}, From 8d55637af837d96c2ebf52ab38d2ffd0cb8db3d3 Mon Sep 17 00:00:00 2001 From: nvartolomei Date: Wed, 21 Sep 2022 10:10:21 +0100 Subject: [PATCH 008/127] Update test.py --- tests/integration/test_mutation_fetch_fallback/test.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/integration/test_mutation_fetch_fallback/test.py b/tests/integration/test_mutation_fetch_fallback/test.py index 91f5d5d380b..8790c9d44b6 100644 --- a/tests/integration/test_mutation_fetch_fallback/test.py +++ b/tests/integration/test_mutation_fetch_fallback/test.py @@ -30,7 +30,7 @@ def test_mutation_fetch_fallback(start_cluster): node1.query("INSERT INTO fetch_fallback(k, v) VALUES (1, 3), (2, 7), (3, 4)") node2.stop_clickhouse() - + # Run a mutation using non-deterministic `hostName` function to produce # different results on replicas and exercise the code responsible for # discarding local mutation results and fetching "byte-identical" parts From 11f93dec3f01e1340650534a72f201ccbf587a52 Mon Sep 17 00:00:00 2001 From: Han Fei Date: Sun, 6 Nov 2022 23:56:26 +0100 Subject: [PATCH 009/127] tmp commit --- src/Interpreters/AsynchronousInsertQueue.cpp | 4 + src/Interpreters/InterpreterInsertQuery.cpp | 2 +- src/Processors/Chunk.h | 10 + .../Transforms/ExpressionTransform.cpp | 1 + .../MergeTree/EphemeralLockInZooKeeper.cpp | 37 +++ .../MergeTree/EphemeralLockInZooKeeper.h | 4 + .../MergeTree/MergeTreeDataWriter.cpp | 46 ++- src/Storages/MergeTree/MergeTreeDataWriter.h | 10 +- .../MergeTree/ReplicatedMergeTreeSink.cpp | 307 +++++++++++++++--- .../MergeTree/ReplicatedMergeTreeSink.h | 14 +- src/Storages/StorageReplicatedMergeTree.cpp | 61 +++- src/Storages/StorageReplicatedMergeTree.h | 7 + 12 files changed, 454 insertions(+), 49 deletions(-) diff --git a/src/Interpreters/AsynchronousInsertQueue.cpp b/src/Interpreters/AsynchronousInsertQueue.cpp index 3aadea918fb..dcaa607a65e 100644 --- a/src/Interpreters/AsynchronousInsertQueue.cpp +++ b/src/Interpreters/AsynchronousInsertQueue.cpp @@ -24,6 +24,7 @@ #include #include #include +#include "Processors/Chunk.h" #include #include @@ -489,11 +490,13 @@ try StreamingFormatExecutor executor(header, format, std::move(on_error), std::move(adding_defaults_transform)); std::unique_ptr last_buffer; + auto chunk_info = std::make_shared(); for (const auto & entry : data->entries) { auto buffer = std::make_unique(entry->bytes); current_entry = entry; total_rows += executor.execute(*buffer); + chunk_info->offsets.push_back(total_rows); /// Keep buffer, because it still can be used /// in destructor, while resetting buffer at next iteration. @@ -534,6 +537,7 @@ try try { auto chunk = Chunk(executor.getResultColumns(), total_rows); + chunk.setChunkInfo(std::move(chunk_info)); size_t total_bytes = chunk.bytes(); auto source = std::make_shared(header, std::move(chunk)); diff --git a/src/Interpreters/InterpreterInsertQuery.cpp b/src/Interpreters/InterpreterInsertQuery.cpp index 107740c3b96..4dc53b53eb4 100644 --- a/src/Interpreters/InterpreterInsertQuery.cpp +++ b/src/Interpreters/InterpreterInsertQuery.cpp @@ -285,7 +285,7 @@ Chain InterpreterInsertQuery::buildChainImpl( /// Do not squash blocks if it is a sync INSERT into Distributed, since it lead to double bufferization on client and server side. /// Client-side bufferization might cause excessive timeouts (especially in case of big blocks). - if (!(settings.insert_distributed_sync && table->isRemote()) && !no_squash && !(query && query->watch)) + if (!(settings.insert_distributed_sync && table->isRemote()) && !async_insert && !no_squash && !(query && query->watch)) { bool table_prefers_large_blocks = table->prefersLargeBlocks(); diff --git a/src/Processors/Chunk.h b/src/Processors/Chunk.h index ec514846f24..398b401c0ff 100644 --- a/src/Processors/Chunk.h +++ b/src/Processors/Chunk.h @@ -113,6 +113,16 @@ private: using Chunks = std::vector; +class ChunkOffsets : public ChunkInfo +{ +public: + ChunkOffsets() = default; + explicit ChunkOffsets(const std::vector & offsets_) : offsets(offsets_) {} + std::vector offsets; +}; + +using ChunkOffsetsPtr = std::shared_ptr; + /// Extension to support delayed defaults. AddingDefaultsProcessor uses it to replace missing values with column defaults. class ChunkMissingValues : public ChunkInfo { diff --git a/src/Processors/Transforms/ExpressionTransform.cpp b/src/Processors/Transforms/ExpressionTransform.cpp index 0d3341b000c..49988932947 100644 --- a/src/Processors/Transforms/ExpressionTransform.cpp +++ b/src/Processors/Transforms/ExpressionTransform.cpp @@ -39,6 +39,7 @@ void ConvertingTransform::onConsume(Chunk chunk) expression->execute(block, num_rows); chunk.setColumns(block.getColumns(), num_rows); + chunk.setChunkInfo(chunk.getChunkInfo()); cur_chunk = std::move(chunk); } diff --git a/src/Storages/MergeTree/EphemeralLockInZooKeeper.cpp b/src/Storages/MergeTree/EphemeralLockInZooKeeper.cpp index 7abed125b7a..9d73bace37f 100644 --- a/src/Storages/MergeTree/EphemeralLockInZooKeeper.cpp +++ b/src/Storages/MergeTree/EphemeralLockInZooKeeper.cpp @@ -61,6 +61,43 @@ std::optional createEphemeralLockInZooKeeper( return EphemeralLockInZooKeeper{path_prefix_, zookeeper_, holder_path}; } +std::tuple, std::vector> createEphemeralLockInZooKeeper( + const String & path_prefix_, const String & temp_path, zkutil::ZooKeeper & zookeeper_, const std::vector & deduplication_paths) +{ + /// The /abandonable_lock- name is for backward compatibility. + String holder_path_prefix = temp_path + "/abandonable_lock-"; + String holder_path; + + /// Check for duplicates in advance, to avoid superfluous block numbers allocation + Coordination::Requests ops; + for (const auto & deduplication_path : deduplication_paths) + { + ops.emplace_back(zkutil::makeCreateRequest(deduplication_path, "", zkutil::CreateMode::Persistent)); + ops.emplace_back(zkutil::makeRemoveRequest(deduplication_path, -1)); + } + ops.emplace_back(zkutil::makeCreateRequest(holder_path_prefix, "", zkutil::CreateMode::EphemeralSequential)); + Coordination::Responses responses; + Coordination::Error e = zookeeper_.tryMulti(ops, responses); + if (e != Coordination::Error::ZOK) + { + /// TODO we should use some cache to check the conflict in advance. + for (const auto & response: responses) + { + if (response->error == Coordination::Error::ZNODEEXISTS) + { + String failed_op_path = zkutil::KeeperMultiException(e, ops, responses).getPathForFirstFailedOp(); + return std::make_pair(std::nullopt, std::vector({failed_op_path})); + } + } + zkutil::KeeperMultiException::check(e, ops, responses); // This should always throw the proper exception + throw Exception("Unable to handle error {} when acquiring ephemeral lock in ZK", ErrorCodes::LOGICAL_ERROR); + } + + holder_path = dynamic_cast(responses.back().get())->path_created; + + return std::make_pair(EphemeralLockInZooKeeper{path_prefix_, zookeeper_, holder_path}, std::vector()); +} + void EphemeralLockInZooKeeper::unlock() { Coordination::Requests ops; diff --git a/src/Storages/MergeTree/EphemeralLockInZooKeeper.h b/src/Storages/MergeTree/EphemeralLockInZooKeeper.h index 276ffab5254..3262620cb3a 100644 --- a/src/Storages/MergeTree/EphemeralLockInZooKeeper.h +++ b/src/Storages/MergeTree/EphemeralLockInZooKeeper.h @@ -26,6 +26,8 @@ class EphemeralLockInZooKeeper : public boost::noncopyable { friend std::optional createEphemeralLockInZooKeeper( const String & path_prefix_, const String & temp_path, zkutil::ZooKeeper & zookeeper_, const String & deduplication_path); + friend std::tuple, std::vector> createEphemeralLockInZooKeeper( + const String & path_prefix_, const String & temp_path, zkutil::ZooKeeper & zookeeper_, const std::vector & deduplication_path); protected: EphemeralLockInZooKeeper() = delete; @@ -95,6 +97,8 @@ private: std::optional createEphemeralLockInZooKeeper( const String & path_prefix_, const String & temp_path, zkutil::ZooKeeper & zookeeper_, const String & deduplication_path); +std::tuple, std::vector> createEphemeralLockInZooKeeper( + const String & path_prefix_, const String & temp_path, zkutil::ZooKeeper & zookeeper_, const std::vector & deduplication_path); /// Acquires block number locks in all partitions. class EphemeralLocksInAllPartitions : public boost::noncopyable diff --git a/src/Storages/MergeTree/MergeTreeDataWriter.cpp b/src/Storages/MergeTree/MergeTreeDataWriter.cpp index 55404324b75..adf101c601f 100644 --- a/src/Storages/MergeTree/MergeTreeDataWriter.cpp +++ b/src/Storages/MergeTree/MergeTreeDataWriter.cpp @@ -1,3 +1,4 @@ +#include #include #include #include @@ -15,6 +16,7 @@ #include #include #include +#include "Processors/Chunk.h" #include #include @@ -146,8 +148,43 @@ void MergeTreeDataWriter::TemporaryPart::finalize() stream.finalizer.finish(); } +std::vector scatterOffsetsBySelector(ChunkOffsetsPtr chunk_offsets, const IColumn::Selector & selector, size_t partition_num) +{ + if (nullptr == chunk_offsets) + { + return {}; + } + if (selector.empty()) + { + return {chunk_offsets}; + } + std::vector result(partition_num); + std::vector last_row_for_partition(partition_num, -1); + size_t offset_idx = 0; + for (size_t i = 0; i < selector.size(); ++i) + { + ++last_row_for_partition[selector[i]]; + if (i + 1 == chunk_offsets->offsets[offset_idx]) + { + for (size_t part_id = 0; part_id < last_row_for_partition.size(); ++part_id) + { + Int64 last_row = last_row_for_partition[part_id]; + if (-1 == last_row) + continue; + size_t offset = static_cast(last_row + 1); + if (result[part_id] == nullptr) + result[part_id] = std::make_shared(); + if (result[part_id]->offsets.empty() || offset > *result[part_id]->offsets.rbegin()) + result[part_id]->offsets.push_back(offset); + } + ++offset_idx; + } + } + return result; +} + BlocksWithPartition MergeTreeDataWriter::splitBlockIntoParts( - const Block & block, size_t max_parts, const StorageMetadataPtr & metadata_snapshot, ContextPtr context) + const Block & block, size_t max_parts, const StorageMetadataPtr & metadata_snapshot, ContextPtr context, ChunkOffsetsPtr chunk_offsets) { BlocksWithPartition result; if (!block || !block.rows()) @@ -174,6 +211,8 @@ BlocksWithPartition MergeTreeDataWriter::splitBlockIntoParts( IColumn::Selector selector; buildScatterSelector(partition_columns, partition_num_to_first_row, selector, max_parts); + auto chunk_offsets_with_partition = scatterOffsetsBySelector(chunk_offsets, selector, partition_num_to_first_row.size()); + size_t partitions_count = partition_num_to_first_row.size(); result.reserve(partitions_count); @@ -191,6 +230,8 @@ BlocksWithPartition MergeTreeDataWriter::splitBlockIntoParts( /// NOTE: returning a copy of the original block so that calculated partition key columns /// do not interfere with possible calculated primary key columns of the same name. result.emplace_back(Block(block), get_partition(0)); + if (!chunk_offsets_with_partition.empty()) + result[0].offsets = chunk_offsets_with_partition[0]; return result; } @@ -204,6 +245,9 @@ BlocksWithPartition MergeTreeDataWriter::splitBlockIntoParts( result[i].block.getByPosition(col).column = std::move(scattered[i]); } + for (size_t i = 0; i < chunk_offsets_with_partition.size(); ++i) + result[i].offsets = chunk_offsets_with_partition[i]; + return result; } diff --git a/src/Storages/MergeTree/MergeTreeDataWriter.h b/src/Storages/MergeTree/MergeTreeDataWriter.h index 8e405016cde..ea1cd8edc06 100644 --- a/src/Storages/MergeTree/MergeTreeDataWriter.h +++ b/src/Storages/MergeTree/MergeTreeDataWriter.h @@ -9,6 +9,8 @@ #include +#include + #include #include @@ -20,11 +22,17 @@ struct BlockWithPartition { Block block; Row partition; + ChunkOffsetsPtr offsets; BlockWithPartition(Block && block_, Row && partition_) : block(block_), partition(std::move(partition_)) { } + + BlockWithPartition(Block && block_, Row && partition_, ChunkOffsetsPtr chunk_offsets_) + : block(block_), partition(std::move(partition_)), offsets(chunk_offsets_) + { + } }; using BlocksWithPartition = std::vector; @@ -43,7 +51,7 @@ public: * (split rows by partition) * Works deterministically: if same block was passed, function will return same result in same order. */ - static BlocksWithPartition splitBlockIntoParts(const Block & block, size_t max_parts, const StorageMetadataPtr & metadata_snapshot, ContextPtr context); + static BlocksWithPartition splitBlockIntoParts(const Block & block, size_t max_parts, const StorageMetadataPtr & metadata_snapshot, ContextPtr context, ChunkOffsetsPtr chunk_offsets = nullptr); /// This structure contains not completely written temporary part. /// Some writes may happen asynchronously, e.g. for blob storages. diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp index 082228d7ebf..22b00a1ae5d 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp @@ -1,3 +1,6 @@ +#include +#include +#include #include #include #include @@ -5,6 +8,11 @@ #include #include #include +#include "Columns/ColumnsNumber.h" +#include "Processors/Chunk.h" +#include "Storages/MergeTree/EphemeralLockInZooKeeper.h" +#include "Storages/MergeTree/IMergeTreeDataPart.h" +#include "Storages/MergeTree/MergeTreeDataWriter.h" #include #include @@ -33,13 +41,39 @@ namespace ErrorCodes extern const int LOGICAL_ERROR; } -struct ReplicatedMergeTreeSink::DelayedChunk +template +struct ReplicatedMergeTreeSink::DelayedChunk { struct Partition { MergeTreeDataWriter::TemporaryPart temp_part; UInt64 elapsed_ns; - String block_id; + BlockIDsType block_id; + BlockWithPartition block; + std::unordered_map block_id_to_offset_idx; + + Partition() = default; + Partition(MergeTreeDataWriter::TemporaryPart && temp_part_, UInt64 elapsed_ns_, BlockIDsType && block_id_, BlockWithPartition && block_) + : temp_part(std::move(temp_part_)), + elapsed_ns(elapsed_ns_), + block_id(std::move(block_id_)), + block(std::move(block_)) + { + initBlockIDMap(); + } + + void initBlockIDMap() + { + if constexpr (async_insert) + { + block_id_to_offset_idx.clear(); + for (size_t i = 0; i < block_id.size(); ++i) + { + block_id_to_offset_idx[block_id[i]] = i; + } + } + } + }; DelayedChunk() = default; @@ -50,7 +84,81 @@ struct ReplicatedMergeTreeSink::DelayedChunk std::vector partitions; }; -ReplicatedMergeTreeSink::ReplicatedMergeTreeSink( +template +inline String toString(const std::vector & vec) +{ + String res = "{"; + for (const auto & item : vec) + res += toString(item) + ","; + return res + "}"; +} + +/// remove the conflict parts of block for rewriting again. +void rewriteBlock(Poco::Logger * log, typename ReplicatedMergeTreeSink::DelayedChunk::Partition & partition, const std::vector & block_paths) +{ + std::vector offset_idx; + for (const auto & raw_path : block_paths) + { + std::filesystem::path p(raw_path); + String conflict_block_id = p.filename(); + auto it = partition.block_id_to_offset_idx.find(conflict_block_id); + if (it == partition.block_id_to_offset_idx.end()) + throw Exception("unknown conflict path " + conflict_block_id, ErrorCodes::LOGICAL_ERROR); + offset_idx.push_back(it->second); + } + std::sort(offset_idx.begin(), offset_idx.end()); + + auto & offsets = partition.block.offsets->offsets; + size_t idx = 0, remove_count = 0; + auto it = offset_idx.begin(); + std::vector new_offsets; + std::vector new_block_ids; + + /// construct filter + size_t rows = partition.block.block.rows(); + auto filter_col = ColumnUInt8::create(rows, 1u); + ColumnUInt8::Container & vec = filter_col->getData(); + UInt8 * pos = vec.data(); + for (auto & offset : offsets) + { + if (it != offset_idx.end() && *it == idx) + { + size_t start_pos = idx > 0 ? offsets[idx - 1] : 0; + size_t end_pos = offset; + remove_count += end_pos - start_pos; + while(start_pos < end_pos) + { + *(pos + start_pos) = 0; + start_pos ++; + } + it++; + } + else + { + new_offsets.push_back(offset - remove_count); + new_block_ids.push_back(partition.block_id[idx]); + } + idx++; + } + + LOG_TRACE(log, "new block ids {}, new offsets{}", toString(new_block_ids), toString(new_offsets)); + + offsets = std::move(new_offsets); + partition.block_id = std::move(new_block_ids); + auto cols = partition.block.block.getColumns(); + for (auto & col : cols) + { + col = col -> filter(vec, rows - remove_count); + } + partition.block.block.setColumns(cols); + + LOG_TRACE(log, "new block rows {}", partition.block.block.rows()); + + partition.initBlockIDMap(); +} + +template +ReplicatedMergeTreeSink::ReplicatedMergeTreeSink( StorageReplicatedMergeTree & storage_, const StorageMetadataPtr & metadata_snapshot_, size_t quorum_size, @@ -79,7 +187,8 @@ ReplicatedMergeTreeSink::ReplicatedMergeTreeSink( required_quorum_size = 0; } -ReplicatedMergeTreeSink::~ReplicatedMergeTreeSink() = default; +template +ReplicatedMergeTreeSink::~ReplicatedMergeTreeSink() = default; /// Allow to verify that the session in ZooKeeper is still alive. static void assertSessionIsNotExpired(zkutil::ZooKeeperPtr & zookeeper) @@ -91,7 +200,8 @@ static void assertSessionIsNotExpired(zkutil::ZooKeeperPtr & zookeeper) throw Exception("ZooKeeper session has been expired.", ErrorCodes::NO_ZOOKEEPER); } -size_t ReplicatedMergeTreeSink::checkQuorumPrecondition(zkutil::ZooKeeperPtr & zookeeper) +template +size_t ReplicatedMergeTreeSink::checkQuorumPrecondition(zkutil::ZooKeeperPtr & zookeeper) { if (!isQuorumEnabled()) return 0; @@ -151,7 +261,33 @@ size_t ReplicatedMergeTreeSink::checkQuorumPrecondition(zkutil::ZooKeeperPtr & z return replicas_number; } -void ReplicatedMergeTreeSink::consume(Chunk chunk) +std::vector getBlockIDVec(BlockWithPartition & block, String partition_id) +{ + size_t start = 0; + auto cols = block.block.getColumns(); + std::vector block_id_vec; + for (auto offset : block.offsets->offsets) + { + SipHash hash; + for (size_t i = start; i < offset; ++i) + for (const auto & col : cols) + col->updateHashWithValue(i, hash); + union + { + char bytes[16]; + UInt64 words[2]; + } hash_value; + hash.get128(hash_value.bytes); + + block_id_vec.push_back(partition_id + "_" + toString(hash_value.words[0]) + "_" + toString(hash_value.words[1])); + + start = offset; + } + return block_id_vec; +} + +template +void ReplicatedMergeTreeSink::consume(Chunk chunk) { auto block = getHeader().cloneWithColumns(chunk.detachColumns()); @@ -166,9 +302,21 @@ void ReplicatedMergeTreeSink::consume(Chunk chunk) size_t replicas_num = checkQuorumPrecondition(zookeeper); deduceTypesOfObjectColumns(storage_snapshot, block); - auto part_blocks = storage.writer.splitBlockIntoParts(block, max_parts_per_block, metadata_snapshot, context); - using DelayedPartitions = std::vector; + ChunkOffsetsPtr chunk_offsets; + + if constexpr (async_insert) + { + const auto & chunk_info = chunk.getChunkInfo(); + if (const auto * chunk_offsets_ptr = typeid_cast(chunk_info.get())) + chunk_offsets = std::make_shared(chunk_offsets_ptr->offsets); + else + throw Exception("miss chunk info for async inserts", ErrorCodes::LOGICAL_ERROR); + } + + auto part_blocks = storage.writer.splitBlockIntoParts(block, max_parts_per_block, metadata_snapshot, context, chunk_offsets); + + using DelayedPartitions = std::vector::DelayedChunk::Partition>; DelayedPartitions partitions; size_t streams = 0; @@ -188,9 +336,15 @@ void ReplicatedMergeTreeSink::consume(Chunk chunk) if (!temp_part.part) continue; - String block_id; + BlockIDsType block_id; - if (deduplicate) + if constexpr (async_insert) + { + /// TODO consider insert_deduplication_token + block_id = getBlockIDVec(current_block, temp_part.part->info.partition_id); + LOG_TRACE(log, "async insert part, part id {}, block id {}, offsets {}", temp_part.part->info.partition_id, toString(block_id), toString(current_block.offsets->offsets)); + } + else if (deduplicate) { String block_dedup_token; @@ -225,7 +379,7 @@ void ReplicatedMergeTreeSink::consume(Chunk chunk) if (streams > max_insert_delayed_streams_for_parallel_write) { finishDelayedChunk(zookeeper); - delayed_chunk = std::make_unique(replicas_num); + delayed_chunk = std::make_unique::DelayedChunk>(replicas_num); delayed_chunk->partitions = std::move(partitions); finishDelayedChunk(zookeeper); @@ -234,11 +388,12 @@ void ReplicatedMergeTreeSink::consume(Chunk chunk) partitions = DelayedPartitions{}; } - partitions.emplace_back(ReplicatedMergeTreeSink::DelayedChunk::Partition{ - .temp_part = std::move(temp_part), - .elapsed_ns = elapsed_ns, - .block_id = std::move(block_id) - }); + partitions.emplace_back(typename ReplicatedMergeTreeSink::DelayedChunk::Partition( + std::move(temp_part), + elapsed_ns, + std::move(block_id), + std::move(current_block) + )); } finishDelayedChunk(zookeeper); @@ -253,7 +408,8 @@ void ReplicatedMergeTreeSink::consume(Chunk chunk) finishDelayedChunk(zookeeper); } -void ReplicatedMergeTreeSink::finishDelayedChunk(zkutil::ZooKeeperPtr & zookeeper) +template<> +void ReplicatedMergeTreeSink::finishDelayedChunk(zkutil::ZooKeeperPtr & zookeeper) { if (!delayed_chunk) return; @@ -287,7 +443,35 @@ void ReplicatedMergeTreeSink::finishDelayedChunk(zkutil::ZooKeeperPtr & zookeepe delayed_chunk.reset(); } -void ReplicatedMergeTreeSink::writeExistingPart(MergeTreeData::MutableDataPartPtr & part) +template<> +void ReplicatedMergeTreeSink::finishDelayedChunk(zkutil::ZooKeeperPtr & zookeeper) +{ + if (!delayed_chunk) + return; + + for (auto & partition: delayed_chunk->partitions) + { + while (true) + { + partition.temp_part.finalize(); + auto conflict_block_ids = commitPart(zookeeper, partition.temp_part.part, partition.block_id, delayed_chunk->replicas_num); + if (conflict_block_ids.empty()) + break; + LOG_DEBUG(log, "depulicate block ids {}", toString(conflict_block_ids)); + /// partition clean conflict + rewriteBlock(log, partition, conflict_block_ids); + if (partition.block_id.empty()) + break; + partition.block.partition = std::move(partition.temp_part.part->partition.value); + partition.temp_part = storage.writer.writeTempPart(partition.block, metadata_snapshot, context); + } + } + + delayed_chunk.reset(); +} + +template +void ReplicatedMergeTreeSink::writeExistingPart(MergeTreeData::MutableDataPartPtr & part) { /// NOTE: No delay in this case. That's Ok. @@ -301,7 +485,7 @@ void ReplicatedMergeTreeSink::writeExistingPart(MergeTreeData::MutableDataPartPt try { part->version.setCreationTID(Tx::PrehistoricTID, nullptr); - commitPart(zookeeper, part, "", replicas_num); + commitPart(zookeeper, part, BlockIDsType(), replicas_num); PartLog::addNewPart(storage.getContext(), part, watch.elapsed()); } catch (...) @@ -311,10 +495,11 @@ void ReplicatedMergeTreeSink::writeExistingPart(MergeTreeData::MutableDataPartPt } } -void ReplicatedMergeTreeSink::commitPart( +template +std::vector ReplicatedMergeTreeSink::commitPart( zkutil::ZooKeeperPtr & zookeeper, MergeTreeData::MutableDataPartPtr & part, - const String & block_id, + const BlockIDsType & block_id, size_t replicas_num) { /// It is possible that we alter a part with different types of source columns. @@ -342,8 +527,22 @@ void ReplicatedMergeTreeSink::commitPart( /// Allocate new block number and check for duplicates bool deduplicate_block = !block_id.empty(); - String block_id_path = deduplicate_block ? storage.zookeeper_path + "/blocks/" + block_id : ""; - auto block_number_lock = storage.allocateBlockNumber(part->info.partition_id, zookeeper, block_id_path); + BlockIDsType block_id_path ; + if constexpr (async_insert) + { + for (const auto & single_block_id : block_id) + { + block_id_path.push_back(storage.zookeeper_path + "/blocks/" + single_block_id); + } + } + else if(deduplicate_block) + block_id_path = storage.zookeeper_path + "/blocks/" + block_id; + std::optional block_number_lock; + std::vector conflict_block_ids; + if constexpr (async_insert) + std::tie(block_number_lock, conflict_block_ids) = storage.allocateBlockNumber(part->info.partition_id, zookeeper, block_id_path); + else + block_number_lock = storage.allocateBlockNumber(part->info.partition_id, zookeeper, block_id_path); /// Prepare transaction to ZooKeeper /// It will simultaneously add information about the part to all the necessary places in ZooKeeper and remove block_number_lock. @@ -384,7 +583,8 @@ void ReplicatedMergeTreeSink::commitPart( log_entry.new_part_name = part->name; /// TODO maybe add UUID here as well? log_entry.quorum = getQuorumSize(replicas_num); - log_entry.block_id = block_id; + if constexpr (!async_insert) + log_entry.block_id = block_id; log_entry.new_part_type = part->getType(); ops.emplace_back(zkutil::makeCreateRequest( @@ -439,6 +639,12 @@ void ReplicatedMergeTreeSink::commitPart( quorum_info.host_node_version)); } } + else if constexpr (async_insert) + { + if (conflict_block_ids.empty()) + throw Exception("conflict block ids and block number lock should not be empty at the same time", ErrorCodes::LOGICAL_ERROR); + return conflict_block_ids; + } else { is_already_existing_part = true; @@ -469,7 +675,7 @@ void ReplicatedMergeTreeSink::commitPart( LOG_INFO(log, "Block with ID {} already exists locally as part {}; ignoring it.", block_id, existing_part_name); } - return; + return {}; } LOG_INFO(log, "Block with ID {} already exists on other replicas as part {}; will write it locally with that name.", block_id, existing_part_name); @@ -512,7 +718,7 @@ void ReplicatedMergeTreeSink::commitPart( if (is_already_existing_part) { LOG_INFO(log, "Part {} is duplicate and it is already written by concurrent request or fetched; ignoring it.", part->name); - return; + return {}; } else throw Exception(ErrorCodes::LOGICAL_ERROR, "Part with name {} is already written by concurrent request." @@ -551,11 +757,24 @@ void ReplicatedMergeTreeSink::commitPart( { String failed_op_path = zkutil::KeeperMultiException(multi_code, ops, responses).getPathForFirstFailedOp(); - if (multi_code == Coordination::Error::ZNODEEXISTS && deduplicate_block && failed_op_path == block_id_path) + auto contains = [](const auto & block_ids, const String & path) + { + if constexpr (async_insert) + { + for (const auto & local_block_id : block_ids) + if (local_block_id == path) + return true; + return false; + } + else + return block_ids == path; + }; + + if (multi_code == Coordination::Error::ZNODEEXISTS && deduplicate_block && contains(block_id_path, failed_op_path)) { /// Block with the same id have just appeared in table (or other replica), rollback thee insertion. LOG_INFO(log, "Block with ID {} already exists (it was just appeared). Renaming part {} back to {}. Will retry write.", - block_id, part->name, temporary_part_relative_path); + toString(block_id), part->name, temporary_part_relative_path); /// We will try to add this part again on the new iteration as it's just a new part. /// So remove it from storage parts set immediately and transfer state to temporary. @@ -564,6 +783,9 @@ void ReplicatedMergeTreeSink::commitPart( part->is_temp = true; part->renameTo(temporary_part_relative_path, false); + if constexpr (async_insert) + return {failed_op_path}; + /// If this part appeared on other replica than it's better to try to write it locally one more time. If it's our part /// than it will be ignored on the next itration. ++loop_counter; @@ -585,7 +807,7 @@ void ReplicatedMergeTreeSink::commitPart( storage.unlockSharedData(*part); /// NOTE: We could be here if the node with the quorum existed, but was quickly removed. transaction.rollback(); - throw Exception("Unexpected logical error while adding block " + toString(block_number) + " with ID '" + block_id + "': " + throw Exception("Unexpected logical error while adding block " + toString(block_number) + " with ID '" + toString(block_id) + "': " + Coordination::errorMessage(multi_code) + ", path " + failed_op_path, ErrorCodes::UNEXPECTED_ZOOKEEPER_ERROR); } @@ -594,14 +816,14 @@ void ReplicatedMergeTreeSink::commitPart( { storage.unlockSharedData(*part); transaction.rollback(); - throw Exception("Unrecoverable network error while adding block " + toString(block_number) + " with ID '" + block_id + "': " + throw Exception("Unrecoverable network error while adding block " + toString(block_number) + " with ID '" + toString(block_id) + "': " + Coordination::errorMessage(multi_code), ErrorCodes::UNEXPECTED_ZOOKEEPER_ERROR); } else { storage.unlockSharedData(*part); transaction.rollback(); - throw Exception("Unexpected ZooKeeper error while adding block " + toString(block_number) + " with ID '" + block_id + "': " + throw Exception("Unexpected ZooKeeper error while adding block " + toString(block_number) + " with ID '" + toString(block_id) + "': " + Coordination::errorMessage(multi_code), ErrorCodes::UNEXPECTED_ZOOKEEPER_ERROR); } @@ -622,23 +844,28 @@ void ReplicatedMergeTreeSink::commitPart( waitForQuorum(zookeeper, part->name, quorum_info.status_path, quorum_info.is_active_node_value, replicas_num); } + + return {}; } -void ReplicatedMergeTreeSink::onStart() +template +void ReplicatedMergeTreeSink::onStart() { /// Only check "too many parts" before write, /// because interrupting long-running INSERT query in the middle is not convenient for users. storage.delayInsertOrThrowIfNeeded(&storage.partial_shutdown_event, context); } -void ReplicatedMergeTreeSink::onFinish() +template +void ReplicatedMergeTreeSink::onFinish() { auto zookeeper = storage.getZooKeeper(); assertSessionIsNotExpired(zookeeper); finishDelayedChunk(zookeeper); } -void ReplicatedMergeTreeSink::waitForQuorum( +template +void ReplicatedMergeTreeSink::waitForQuorum( zkutil::ZooKeeperPtr & zookeeper, const std::string & part_name, const std::string & quorum_path, @@ -691,14 +918,16 @@ void ReplicatedMergeTreeSink::waitForQuorum( LOG_TRACE(log, "Quorum '{}' for part {} satisfied", quorum_path, part_name); } -String ReplicatedMergeTreeSink::quorumLogMessage(size_t replicas_num) const +template +String ReplicatedMergeTreeSink::quorumLogMessage(size_t replicas_num) const { if (!isQuorumEnabled()) return ""; return fmt::format(" (quorum {} of {} replicas)", getQuorumSize(replicas_num), replicas_num); } -size_t ReplicatedMergeTreeSink::getQuorumSize(size_t replicas_num) const +template +size_t ReplicatedMergeTreeSink::getQuorumSize(size_t replicas_num) const { if (!isQuorumEnabled()) return 0; @@ -709,9 +938,13 @@ size_t ReplicatedMergeTreeSink::getQuorumSize(size_t replicas_num) const return replicas_num / 2 + 1; } -bool ReplicatedMergeTreeSink::isQuorumEnabled() const +template +bool ReplicatedMergeTreeSink::isQuorumEnabled() const { return !required_quorum_size.has_value() || required_quorum_size.value() > 1; } +template class ReplicatedMergeTreeSink; +template class ReplicatedMergeTreeSink; + } diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeSink.h b/src/Storages/MergeTree/ReplicatedMergeTreeSink.h index da87ddc0d63..842316c702e 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeSink.h +++ b/src/Storages/MergeTree/ReplicatedMergeTreeSink.h @@ -21,6 +21,12 @@ struct StorageSnapshot; using StorageSnapshotPtr = std::shared_ptr; +/// ReplicatedMergeTreeSink will sink data to replicated merge tree with deduplication. +/// The template argument "async_insert" indicates whether this sink serves for async inserts. +/// Async inserts will have different deduplication policy. We use a vector of "block ids" to +/// identify different async inserts inside the same part. It will remove the duplicate inserts +/// when it encounters lock and retries. +template class ReplicatedMergeTreeSink : public SinkToStorage { public: @@ -59,7 +65,10 @@ public: return last_block_is_duplicate; } + struct DelayedChunk; private: + using BlockIDsType = std::conditional_t, String>; + struct QuorumInfo { String status_path; @@ -75,10 +84,10 @@ private: size_t checkQuorumPrecondition(zkutil::ZooKeeperPtr & zookeeper); /// Rename temporary part and commit to ZooKeeper. - void commitPart( + std::vector commitPart( zkutil::ZooKeeperPtr & zookeeper, MergeTreeData::MutableDataPartPtr & part, - const String & block_id, + const BlockIDsType & block_id, size_t replicas_num); /// Wait for quorum to be satisfied on path (quorum_path) form part (part_name) @@ -114,7 +123,6 @@ private: UInt64 chunk_dedup_seqnum = 0; /// input chunk ordinal number in case of dedup token /// We can delay processing for previous chunk and start writing a new one. - struct DelayedChunk; std::unique_ptr delayed_chunk; void finishDelayedChunk(zkutil::ZooKeeperPtr & zookeeper); diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 3c0fbb162bc..8933ff652c9 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -4467,9 +4467,19 @@ SinkToStoragePtr StorageReplicatedMergeTree::write(const ASTPtr & /*query*/, con const Settings & query_settings = local_context->getSettingsRef(); bool deduplicate = storage_settings_ptr->replicated_deduplication_window != 0 && query_settings.insert_deduplicate; + bool async_insert = query_settings.async_insert; + if (deduplicate && async_insert) + return std::make_shared>( + *this, metadata_snapshot, query_settings.insert_quorum.valueOr(0), + query_settings.insert_quorum_timeout.totalMilliseconds(), + query_settings.max_partitions_per_insert_block, + query_settings.insert_quorum_parallel, + deduplicate, + query_settings.insert_quorum.is_auto, + local_context); // TODO: should we also somehow pass list of columns to deduplicate on to the ReplicatedMergeTreeSink? - return std::make_shared( + return std::make_shared>( *this, metadata_snapshot, query_settings.insert_quorum.valueOr(0), query_settings.insert_quorum_timeout.totalMilliseconds(), query_settings.max_partitions_per_insert_block, @@ -5171,7 +5181,7 @@ PartitionCommandsResultInfo StorageReplicatedMergeTree::attachPartition( MutableDataPartsVector loaded_parts = tryLoadPartsToAttach(partition, attach_part, query_context, renamed_parts); /// TODO Allow to use quorum here. - ReplicatedMergeTreeSink output(*this, metadata_snapshot, 0, 0, 0, false, false, false, query_context, + ReplicatedMergeTreeSink output(*this, metadata_snapshot, 0, 0, 0, false, false, false, query_context, /*is_attach*/true); for (size_t i = 0; i < loaded_parts.size(); ++i) @@ -5280,6 +5290,35 @@ bool StorageReplicatedMergeTree::existsNodeCached(const std::string & path) cons return res; } +std::tuple, std::vector> +StorageReplicatedMergeTree::allocateBlockNumber( + const String & partition_id, const zkutil::ZooKeeperPtr & zookeeper, + const std::vector & zookeeper_block_id_paths) const +{ + String zookeeper_table_path = zookeeper_path; + + String block_numbers_path = fs::path(zookeeper_table_path) / "block_numbers"; + String partition_path = fs::path(block_numbers_path) / partition_id; + + if (!existsNodeCached(partition_path)) + { + Coordination::Requests ops; + ops.push_back(zkutil::makeCreateRequest(partition_path, "", zkutil::CreateMode::Persistent)); + /// We increment data version of the block_numbers node so that it becomes possible + /// to check in a ZK transaction that the set of partitions didn't change + /// (unfortunately there is no CheckChildren op). + ops.push_back(zkutil::makeSetRequest(block_numbers_path, "", -1)); + + Coordination::Responses responses; + Coordination::Error code = zookeeper->tryMulti(ops, responses); + if (code != Coordination::Error::ZOK && code != Coordination::Error::ZNODEEXISTS) + zkutil::KeeperMultiException::check(code, ops, responses); + } + + return createEphemeralLockInZooKeeper( + fs::path(partition_path) / "block-", fs::path(zookeeper_table_path) / "temp", *zookeeper, zookeeper_block_id_paths); +} + std::optional StorageReplicatedMergeTree::allocateBlockNumber( @@ -6994,11 +7033,21 @@ void StorageReplicatedMergeTree::getCommitPartOps( Coordination::Requests & ops, MutableDataPartPtr & part, const String & block_id_path) const +{ + if (block_id_path.empty()) + return getCommitPartOps(ops, part, std::vector()); + else + return getCommitPartOps(ops, part, std::vector({block_id_path})); +} + +void StorageReplicatedMergeTree::getCommitPartOps( + Coordination::Requests & ops, + MutableDataPartPtr & part, + const std::vector & block_id_paths) const { const String & part_name = part->name; const auto storage_settings_ptr = getSettings(); - - if (!block_id_path.empty()) + for (const String & block_id_path : block_id_paths) { /// Make final duplicate check and commit block_id ops.emplace_back( @@ -8596,7 +8645,7 @@ void StorageReplicatedMergeTree::restoreDataFromBackup(RestorerFromBackup & rest void StorageReplicatedMergeTree::attachRestoredParts(MutableDataPartsVector && parts) { auto metadata_snapshot = getInMemoryMetadataPtr(); - auto sink = std::make_shared(*this, metadata_snapshot, 0, 0, 0, false, false, false, getContext(), /*is_attach*/true); + auto sink = std::make_shared>(*this, metadata_snapshot, 0, 0, 0, false, false, false, getContext(), /*is_attach*/true); for (auto part : parts) sink->writeExistingPart(part); } @@ -8606,7 +8655,7 @@ PartsTemporaryRename renamed_parts(*this, "detached/"); MutableDataPartsVector loaded_parts = tryLoadPartsToAttach(partition, attach_part, query_context, renamed_parts); /// TODO Allow to use quorum here. -ReplicatedMergeTreeSink output(*this, metadata_snapshot, 0, 0, 0, false, false, query_context, +ReplicatedMergeTreeSink output(*this, metadata_snapshot, 0, 0, 0, false, false, query_context, /*is_attach*/true); for (size_t i = 0; i < loaded_parts.size(); ++i) diff --git a/src/Storages/StorageReplicatedMergeTree.h b/src/Storages/StorageReplicatedMergeTree.h index 323b1ce02bf..32e56bfc675 100644 --- a/src/Storages/StorageReplicatedMergeTree.h +++ b/src/Storages/StorageReplicatedMergeTree.h @@ -338,6 +338,7 @@ private: /// Delete old parts from disk and from ZooKeeper. void clearOldPartsAndRemoveFromZK(); + template friend class ReplicatedMergeTreeSink; friend class ReplicatedMergeTreePartCheckThread; friend class ReplicatedMergeTreeCleanupThread; @@ -536,6 +537,8 @@ private: void getCommitPartOps(Coordination::Requests & ops, MutableDataPartPtr & part, const String & block_id_path = "") const; + void getCommitPartOps(Coordination::Requests & ops, MutableDataPartPtr & part, const std::vector & block_id_paths) const; + /// Adds actions to `ops` that remove a part from ZooKeeper. /// Set has_children to true for "old-style" parts (those with /columns and /checksums child znodes). void removePartFromZooKeeper(const String & part_name, Coordination::Requests & ops, bool has_children); @@ -712,6 +715,10 @@ private: const String & partition_id, const zkutil::ZooKeeperPtr & zookeeper, const String & zookeeper_block_id_path = "", const String & zookeeper_path_prefix = "") const; + std::tuple, std::vector> allocateBlockNumber( + const String & partition_id, const zkutil::ZooKeeperPtr & zookeeper, + const std::vector & zookeeper_block_id_paths) const; + /** Wait until all replicas, including this, execute the specified action from the log. * If replicas are added at the same time, it can not wait the added replica. * From 0133444433ac071793ebc282dad084e4dd99851f Mon Sep 17 00:00:00 2001 From: HarryLeeIBM Date: Mon, 14 Nov 2022 10:47:32 -0800 Subject: [PATCH 010/127] Fix byte order issue of wide integer for s390x --- base/base/wide_integer_impl.h | 72 +++++++++++-------- src/Common/SipHash.h | 5 ++ src/Common/hex.h | 8 ++- src/Common/tests/gtest_wide_integer.cpp | 17 +++-- .../FunctionsBinaryRepresentation.cpp | 55 ++++++++++---- src/IO/ReadHelpers.h | 2 +- src/IO/WriteHelpers.h | 2 +- 7 files changed, 108 insertions(+), 53 deletions(-) diff --git a/base/base/wide_integer_impl.h b/base/base/wide_integer_impl.h index 1b5f502722c..f5b30cbab55 100644 --- a/base/base/wide_integer_impl.h +++ b/base/base/wide_integer_impl.h @@ -187,8 +187,20 @@ struct integer::_impl static_assert(Bits % base_bits == 0); /// Simple iteration in both directions - static constexpr unsigned little(unsigned idx) { return idx; } - static constexpr unsigned big(unsigned idx) { return item_count - 1 - idx; } + static constexpr unsigned little(unsigned idx) + { + if constexpr (std::endian::native == std::endian::little) + return idx; + else + return item_count - 1 - idx; + } + static constexpr unsigned big(unsigned idx) + { + if constexpr (std::endian::native == std::endian::little) + return item_count - 1 - idx; + else + return idx; + } static constexpr unsigned any(unsigned idx) { return idx; } template @@ -240,20 +252,20 @@ struct integer::_impl { static_assert(sizeof(Integral) <= sizeof(base_type)); - self.items[0] = _impl::to_Integral(rhs); + self.items[little(0)] = _impl::to_Integral(rhs); if constexpr (std::is_signed_v) { if (rhs < 0) { - for (size_t i = 1; i < item_count; ++i) - self.items[i] = -1; + for (unsigned i = 1; i < item_count; ++i) + self.items[little(i)] = -1; return; } } - for (size_t i = 1; i < item_count; ++i) - self.items[i] = 0; + for (unsigned i = 1; i < item_count; ++i) + self.items[little(i)] = 0; } template @@ -348,7 +360,7 @@ struct integer::_impl constexpr const unsigned to_copy = min_bits / base_bits; for (unsigned i = 0; i < to_copy; ++i) - self.items[i] = rhs.items[i]; + self.items[little(i)] = rhs.items[little(i)]; if constexpr (Bits > Bits2) { @@ -357,13 +369,13 @@ struct integer::_impl if (rhs < 0) { for (unsigned i = to_copy; i < item_count; ++i) - self.items[i] = -1; + self.items[little(i)] = -1; return; } } for (unsigned i = to_copy; i < item_count; ++i) - self.items[i] = 0; + self.items[little(i)] = 0; } } @@ -454,7 +466,7 @@ private: { if constexpr (sizeof(T) <= sizeof(base_type)) { - if (0 == idx) + if (little(0) == idx) return static_cast(x); } else if (idx * sizeof(base_type) < sizeof(T)) @@ -475,7 +487,7 @@ private: for (unsigned i = 0; i < op_items; ++i) { - base_type rhs_item = get_item(rhs, i); + base_type rhs_item = get_item(rhs, little(i)); base_type & res_item = res.items[little(i)]; underflows[i] = res_item < rhs_item; @@ -508,7 +520,7 @@ private: for (unsigned i = 0; i < op_items; ++i) { - base_type rhs_item = get_item(rhs, i); + base_type rhs_item = get_item(rhs, little(i)); base_type & res_item = res.items[little(i)]; res_item += rhs_item; @@ -580,12 +592,12 @@ private: else if constexpr (Bits == 128 && sizeof(base_type) == 8) { using CompilerUInt128 = unsigned __int128; - CompilerUInt128 a = (CompilerUInt128(lhs.items[1]) << 64) + lhs.items[0]; // NOLINT(clang-analyzer-core.UndefinedBinaryOperatorResult) - CompilerUInt128 b = (CompilerUInt128(rhs.items[1]) << 64) + rhs.items[0]; // NOLINT(clang-analyzer-core.UndefinedBinaryOperatorResult) + CompilerUInt128 a = (CompilerUInt128(lhs.items[little(1)]) << 64) + lhs.items[little(0)]; // NOLINT(clang-analyzer-core.UndefinedBinaryOperatorResult) + CompilerUInt128 b = (CompilerUInt128(rhs.items[little(1)]) << 64) + rhs.items[little(0)]; // NOLINT(clang-analyzer-core.UndefinedBinaryOperatorResult) CompilerUInt128 c = a * b; integer res; - res.items[0] = c; - res.items[1] = c >> 64; + res.items[little(0)] = c; + res.items[little(1)] = c >> 64; return res; } else @@ -597,7 +609,7 @@ private: #endif for (unsigned i = 0; i < item_count; ++i) { - base_type rhs_item = get_item(rhs, i); + base_type rhs_item = get_item(rhs, little(i)); unsigned pos = i * base_bits; while (rhs_item) @@ -792,7 +804,7 @@ public: integer res; for (unsigned i = 0; i < item_count; ++i) - res.items[little(i)] = lhs.items[little(i)] | get_item(rhs, i); + res.items[little(i)] = lhs.items[little(i)] | get_item(rhs, little(i)); return res; } else @@ -810,7 +822,7 @@ public: integer res; for (unsigned i = 0; i < item_count; ++i) - res.items[little(i)] = lhs.items[little(i)] & get_item(rhs, i); + res.items[little(i)] = lhs.items[little(i)] & get_item(rhs, little(i)); return res; } else @@ -845,17 +857,17 @@ public: { using CompilerUInt128 = unsigned __int128; - CompilerUInt128 a = (CompilerUInt128(numerator.items[1]) << 64) + numerator.items[0]; // NOLINT(clang-analyzer-core.UndefinedBinaryOperatorResult) - CompilerUInt128 b = (CompilerUInt128(denominator.items[1]) << 64) + denominator.items[0]; // NOLINT(clang-analyzer-core.UndefinedBinaryOperatorResult) + CompilerUInt128 a = (CompilerUInt128(numerator.items[little(1)]) << 64) + numerator.items[little(0)]; // NOLINT(clang-analyzer-core.UndefinedBinaryOperatorResult) + CompilerUInt128 b = (CompilerUInt128(denominator.items[little(1)]) << 64) + denominator.items[little(0)]; // NOLINT(clang-analyzer-core.UndefinedBinaryOperatorResult) CompilerUInt128 c = a / b; // NOLINT integer res; - res.items[0] = c; - res.items[1] = c >> 64; + res.items[little(0)] = c; + res.items[little(1)] = c >> 64; CompilerUInt128 remainder = a - b * c; - numerator.items[0] = remainder; - numerator.items[1] = remainder >> 64; + numerator.items[little(0)] = remainder; + numerator.items[little(1)] = remainder >> 64; return res; } @@ -1039,15 +1051,15 @@ constexpr integer::integer(std::initializer_list il) noexcept else { auto it = il.begin(); - for (size_t i = 0; i < _impl::item_count; ++i) + for (unsigned i = 0; i < _impl::item_count; ++i) { if (it < il.end()) { - items[i] = *it; + items[_impl::little(i)] = *it; ++it; } else - items[i] = 0; + items[_impl::little(i)] = 0; } } } @@ -1208,7 +1220,7 @@ constexpr integer::operator T() const noexcept UnsignedT res{}; for (unsigned i = 0; i < _impl::item_count && i < (sizeof(T) + sizeof(base_type) - 1) / sizeof(base_type); ++i) - res += UnsignedT(items[i]) << (sizeof(base_type) * 8 * i); // NOLINT(clang-analyzer-core.UndefinedBinaryOperatorResult) + res += UnsignedT(items[_impl::little(i)]) << (sizeof(base_type) * 8 * i); // NOLINT(clang-analyzer-core.UndefinedBinaryOperatorResult) return res; } diff --git a/src/Common/SipHash.h b/src/Common/SipHash.h index 281a65ca36a..46680fc28df 100644 --- a/src/Common/SipHash.h +++ b/src/Common/SipHash.h @@ -164,8 +164,13 @@ public: void get128(char * out) { finalize(); +#if __BYTE_ORDER__ == __ORDER_BIG_ENDIAN__ + unalignedStore(out + 8, v0 ^ v1); + unalignedStore(out, v2 ^ v3); +#else unalignedStore(out, v0 ^ v1); unalignedStore(out + 8, v2 ^ v3); +#endif } template diff --git a/src/Common/hex.h b/src/Common/hex.h index 424ed1f6c3e..062a6c27f76 100644 --- a/src/Common/hex.h +++ b/src/Common/hex.h @@ -58,9 +58,13 @@ inline void writeHexUIntImpl(TUInt uint_, char * out, const char * const table) value = uint_; - /// Use little endian for (size_t i = 0; i < sizeof(TUInt); ++i) - memcpy(out + i * 2, &table[static_cast(uint8[sizeof(TUInt) - 1 - i]) * 2], 2); + { + if constexpr (std::endian::native == std::endian::little) + memcpy(out + i * 2, &table[static_cast(uint8[sizeof(TUInt) - 1 - i]) * 2], 2); + else + memcpy(out + i * 2, &table[static_cast(uint8[i]) * 2], 2); + } } template diff --git a/src/Common/tests/gtest_wide_integer.cpp b/src/Common/tests/gtest_wide_integer.cpp index fa614e9390a..93c962b23cc 100644 --- a/src/Common/tests/gtest_wide_integer.cpp +++ b/src/Common/tests/gtest_wide_integer.cpp @@ -62,7 +62,7 @@ GTEST_TEST(WideInteger, Conversions) zero += minus_one; #if __BYTE_ORDER__ == __ORDER_BIG_ENDIAN__ - ASSERT_EQ(0, memcmp(&zero, "\xFF\xFF\xFF\xFF\xFF\xFF\xFF\xFE\xFF\xFF\xFF\xFF\xFF\xFF\xFF\xFF", sizeof(zero))); + ASSERT_EQ(0, memcmp(&zero, "\xFF\xFF\xFF\xFF\xFF\xFF\xFF\xFF\xFF\xFF\xFF\xFF\xFF\xFF\xFF\xFE", sizeof(zero))); #else ASSERT_EQ(0, memcmp(&zero, "\xFE\xFF\xFF\xFF\xFF\xFF\xFF\xFF\xFF\xFF\xFF\xFF\xFF\xFF\xFF\xFF", sizeof(zero))); #endif @@ -160,7 +160,7 @@ GTEST_TEST(WideInteger, Arithmetic) zero += minus_one; #if __BYTE_ORDER__ == __ORDER_BIG_ENDIAN__ - ASSERT_EQ(0, memcmp(&zero, "\xFF\xFF\xFF\xFF\xFF\xFF\xFF\xFE\xFF\xFF\xFF\xFF\xFF\xFF\xFF\xFF", sizeof(zero))); + ASSERT_EQ(0, memcmp(&zero, "\xFF\xFF\xFF\xFF\xFF\xFF\xFF\xFF\xFF\xFF\xFF\xFF\xFF\xFF\xFF\xFE", sizeof(zero))); #else ASSERT_EQ(0, memcmp(&zero, "\xFE\xFF\xFF\xFF\xFF\xFF\xFF\xFF\xFF\xFF\xFF\xFF\xFF\xFF\xFF\xFF", sizeof(zero))); #endif @@ -244,7 +244,7 @@ GTEST_TEST(WideInteger, Shift) auto y = x << 64; #if __BYTE_ORDER__ == __ORDER_BIG_ENDIAN__ - ASSERT_EQ(0, memcmp(&y, "\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x01", sizeof(Int128))); + ASSERT_EQ(0, memcmp(&y, "\x00\x00\x00\x00\x00\x00\x00\x01\x00\x00\x00\x00\x00\x00\x00\x00", sizeof(Int128))); #else ASSERT_EQ(0, memcmp(&y, "\x00\x00\x00\x00\x00\x00\x00\x00\x01\x00\x00\x00\x00\x00\x00\x00", sizeof(Int128))); #endif @@ -261,7 +261,7 @@ GTEST_TEST(WideInteger, Shift) y = x << 16; #if __BYTE_ORDER__ == __ORDER_BIG_ENDIAN__ - ASSERT_EQ(0, memcmp(&y, "\xFF\xFF\xFF\xFF\xFF\xFF\x00\x00\xFF\xFF\xFF\xFF\xFF\xFF\xFF\xFF", sizeof(Int128))); + ASSERT_EQ(0, memcmp(&y, "\xFF\xFF\xFF\xFF\xFF\xFF\xFF\xFF\xFF\xFF\xFF\xFF\xFF\xFF\x00\x00", sizeof(Int128))); #else ASSERT_EQ(0, memcmp(&y, "\x00\x00\xFF\xFF\xFF\xFF\xFF\xFF\xFF\xFF\xFF\xFF\xFF\xFF\xFF\xFF", sizeof(Int128))); #endif @@ -269,18 +269,21 @@ GTEST_TEST(WideInteger, Shift) ASSERT_EQ(0, memcmp(&y, "\xFF\xFF\xFF\xFF\xFF\xFF\xFF\xFF\xFF\xFF\xFF\xFF\xFF\xFF\xFF\xFF", sizeof(Int128))); y <<= 64; +#if __BYTE_ORDER__ == __ORDER_BIG_ENDIAN__ + ASSERT_EQ(0, memcmp(&y, "\xFF\xFF\xFF\xFF\xFF\xFF\xFF\xFF\x00\x00\x00\x00\x00\x00\x00\x00", sizeof(Int128))); +#else ASSERT_EQ(0, memcmp(&y, "\x00\x00\x00\x00\x00\x00\x00\x00\xFF\xFF\xFF\xFF\xFF\xFF\xFF\xFF", sizeof(Int128))); - +#endif y >>= 32; #if __BYTE_ORDER__ == __ORDER_BIG_ENDIAN__ - ASSERT_EQ(0, memcmp(&y, "\xFF\xFF\xFF\xFF\x00\x00\x00\x00\xFF\xFF\xFF\xFF\xFF\xFF\xFF\xFF", sizeof(Int128))); + ASSERT_EQ(0, memcmp(&y, "\xFF\xFF\xFF\xFF\xFF\xFF\xFF\xFF\xFF\xFF\xFF\xFF\x00\x00\x00\x00", sizeof(Int128))); #else ASSERT_EQ(0, memcmp(&y, "\x00\x00\x00\x00\xFF\xFF\xFF\xFF\xFF\xFF\xFF\xFF\xFF\xFF\xFF\xFF", sizeof(Int128))); #endif y <<= 64; #if __BYTE_ORDER__ == __ORDER_BIG_ENDIAN__ - ASSERT_EQ(0, memcmp(&y, "\x00\x00\x00\x00\x00\x00\x00\x00\xFF\xFF\xFF\xFF\x00\x00\x00\x00", sizeof(Int128))); + ASSERT_EQ(0, memcmp(&y, "\xFF\xFF\xFF\xFF\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00", sizeof(Int128))); #else ASSERT_EQ(0, memcmp(&y, "\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\xFF\xFF\xFF\xFF", sizeof(Int128))); #endif diff --git a/src/Functions/FunctionsBinaryRepresentation.cpp b/src/Functions/FunctionsBinaryRepresentation.cpp index 775696ded8a..5fac454502b 100644 --- a/src/Functions/FunctionsBinaryRepresentation.cpp +++ b/src/Functions/FunctionsBinaryRepresentation.cpp @@ -65,13 +65,27 @@ struct HexImpl } } - static void executeOneString(const UInt8 * pos, const UInt8 * end, char *& out) + static void executeOneString(const UInt8 * pos, const UInt8 * end, char *& out, bool reverse_order = false) { - while (pos < end) + if (!reverse_order) { - writeHexByteUppercase(*pos, out); - ++pos; - out += word_size; + while (pos < end) + { + writeHexByteUppercase(*pos, out); + ++pos; + out += word_size; + } + } + else + { + auto start_pos = pos; + pos = end - 1; + while (pos >= start_pos) + { + writeHexByteUppercase(*pos, out); + --pos; + out += word_size; + } } *out = '\0'; ++out; @@ -95,7 +109,8 @@ struct HexImpl for (size_t i = 0; i < size; ++i) { const UInt8 * in_pos = reinterpret_cast(&in_vec[i]); - executeOneString(in_pos, in_pos + type_size_in_bytes, out); + bool reverse_order = (std::endian::native == std::endian::big); + executeOneString(in_pos, in_pos + type_size_in_bytes, out, reverse_order); pos += hex_length; out_offsets[i] = pos; @@ -174,7 +189,9 @@ struct BinImpl for (size_t i = 0; i < size; ++i) { const UInt8 * in_pos = reinterpret_cast(&in_vec[i]); - executeOneString(in_pos, in_pos + type_size_in_bytes, out); + + bool reverse_order = (std::endian::native == std::endian::big); + executeOneString(in_pos, in_pos + type_size_in_bytes, out, reverse_order); pos += hex_length; out_offsets[i] = pos; @@ -182,13 +199,27 @@ struct BinImpl col_res = std::move(col_str); } - static void executeOneString(const UInt8 * pos, const UInt8 * end, char *& out) + static void executeOneString(const UInt8 * pos, const UInt8 * end, char *& out, bool reverse_order = false) { - while (pos < end) + if (!reverse_order) { - writeBinByte(*pos, out); - ++pos; - out += word_size; + while (pos < end) + { + writeBinByte(*pos, out); + ++pos; + out += word_size; + } + } + else + { + auto start_pos = pos; + pos = end - 1; + while (pos >= start_pos) + { + writeBinByte(*pos, out); + --pos; + out += word_size; + } } *out = '\0'; ++out; diff --git a/src/IO/ReadHelpers.h b/src/IO/ReadHelpers.h index 27a24eef804..8d7b46e511e 100644 --- a/src/IO/ReadHelpers.h +++ b/src/IO/ReadHelpers.h @@ -1068,7 +1068,7 @@ inline void readBinaryBigEndian(T & x, ReadBuffer & buf) /// Assuming little { for (size_t i = 0; i != std::size(x.items); ++i) { - auto & item = x.items[std::size(x.items) - i - 1]; + auto & item = x.items[(std::endian::native == std::endian::little) ? std::size(x.items) - i - 1 : i]; readBinaryBigEndian(item, buf); } } diff --git a/src/IO/WriteHelpers.h b/src/IO/WriteHelpers.h index 39024b33eb1..003e5a56958 100644 --- a/src/IO/WriteHelpers.h +++ b/src/IO/WriteHelpers.h @@ -1126,7 +1126,7 @@ inline void writeBinaryBigEndian(const T & x, WriteBuffer & buf) /// Assuming { for (size_t i = 0; i != std::size(x.items); ++i) { - const auto & item = x.items[std::size(x.items) - i - 1]; + const auto & item = x.items[(std::endian::native == std::endian::little) ? std::size(x.items) - i - 1 : i]; writeBinaryBigEndian(item, buf); } } From 221c4e5912f358af60b5774e63ed65e918638f24 Mon Sep 17 00:00:00 2001 From: Han Fei Date: Wed, 16 Nov 2022 17:59:08 +0100 Subject: [PATCH 011/127] tiny changes --- src/Interpreters/AsynchronousInsertQueue.cpp | 1 - src/Storages/MergeTree/MergeTreeDataWriter.cpp | 2 -- .../MergeTree/ReplicatedMergeTreeSink.cpp | 17 ++++------------- 3 files changed, 4 insertions(+), 16 deletions(-) diff --git a/src/Interpreters/AsynchronousInsertQueue.cpp b/src/Interpreters/AsynchronousInsertQueue.cpp index 839f49e8a48..a268cb07b0e 100644 --- a/src/Interpreters/AsynchronousInsertQueue.cpp +++ b/src/Interpreters/AsynchronousInsertQueue.cpp @@ -24,7 +24,6 @@ #include #include #include -#include "Processors/Chunk.h" #include #include diff --git a/src/Storages/MergeTree/MergeTreeDataWriter.cpp b/src/Storages/MergeTree/MergeTreeDataWriter.cpp index eaf8c80ae84..69128f23c85 100644 --- a/src/Storages/MergeTree/MergeTreeDataWriter.cpp +++ b/src/Storages/MergeTree/MergeTreeDataWriter.cpp @@ -1,4 +1,3 @@ -#include #include #include #include @@ -16,7 +15,6 @@ #include #include #include -#include "Processors/Chunk.h" #include #include diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp index 163a6b55a03..c3a0d9cdd31 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp @@ -1,19 +1,11 @@ -#include -#include -#include #include #include #include #include #include -#include #include -#include "Columns/ColumnsNumber.h" +#include #include -#include "Processors/Chunk.h" -#include "Storages/MergeTree/EphemeralLockInZooKeeper.h" -#include "Storages/MergeTree/IMergeTreeDataPart.h" -#include "Storages/MergeTree/MergeTreeDataWriter.h" #include #include @@ -128,7 +120,7 @@ void rewriteBlock(Poco::Logger * log, typename ReplicatedMergeTreeSink::De size_t start_pos = idx > 0 ? offsets[idx - 1] : 0; size_t end_pos = offset; remove_count += end_pos - start_pos; - while(start_pos < end_pos) + while (start_pos < end_pos) { *(pos + start_pos) = 0; start_pos ++; @@ -333,7 +325,6 @@ void ReplicatedMergeTreeSink::consume(Chunk chunk) if (!storage_snapshot->object_columns.empty()) convertDynamicColumnsToTuples(block, storage_snapshot); - // deduceTypesOfObjectColumns(storage_snapshot, block); ChunkOffsetsPtr chunk_offsets; @@ -343,7 +334,7 @@ void ReplicatedMergeTreeSink::consume(Chunk chunk) if (const auto * chunk_offsets_ptr = typeid_cast(chunk_info.get())) chunk_offsets = std::make_shared(chunk_offsets_ptr->offsets); else - throw Exception("miss chunk info for async inserts", ErrorCodes::LOGICAL_ERROR); + throw Exception("Miss chunk info for async inserts", ErrorCodes::LOGICAL_ERROR); } auto part_blocks = storage.writer.splitBlockIntoParts(block, max_parts_per_block, metadata_snapshot, context, chunk_offsets); @@ -618,7 +609,7 @@ std::vector ReplicatedMergeTreeSink::commitPart( block_id_path.push_back(storage.zookeeper_path + "/blocks/" + single_block_id); } } - else if(deduplicate_block) + else if (deduplicate_block) block_id_path = storage.zookeeper_path + "/blocks/" + block_id; std::optional block_number_lock; if constexpr (async_insert) From c47f90d197e53fcf57c32d38a556cceec6eb60b7 Mon Sep 17 00:00:00 2001 From: Guillaume Tassery Date: Wed, 16 Nov 2022 20:50:48 +0100 Subject: [PATCH 012/127] add expression index in system.data_skipping_indices --- .../System/StorageSystemDataSkippingIndices.cpp | 11 +++++++++++ .../01917_system_data_skipping_indices.reference | 10 +++++----- 2 files changed, 16 insertions(+), 5 deletions(-) diff --git a/src/Storages/System/StorageSystemDataSkippingIndices.cpp b/src/Storages/System/StorageSystemDataSkippingIndices.cpp index be04261cc4e..15561a87394 100644 --- a/src/Storages/System/StorageSystemDataSkippingIndices.cpp +++ b/src/Storages/System/StorageSystemDataSkippingIndices.cpp @@ -7,6 +7,8 @@ #include #include #include +#include +#include #include #include #include @@ -24,6 +26,7 @@ StorageSystemDataSkippingIndices::StorageSystemDataSkippingIndices(const Storage { "table", std::make_shared() }, { "name", std::make_shared() }, { "type", std::make_shared() }, + { "index_expr", std::make_shared() }, { "expr", std::make_shared() }, { "granularity", std::make_shared() }, { "data_compressed_bytes", std::make_shared() }, @@ -121,6 +124,14 @@ protected: // 'type' column if (column_mask[src_index++]) res_columns[res_index++]->insert(index.type); + // 'index_expr' column + if (column_mask[src_index++]) + { + if (auto expression = index.definition_ast->as()) + res_columns[res_index++]->insert(queryToString(expression->type->clone())); + else + res_columns[res_index++]->insertDefault(); + } // 'expr' column if (column_mask[src_index++]) { diff --git a/tests/queries/0_stateless/01917_system_data_skipping_indices.reference b/tests/queries/0_stateless/01917_system_data_skipping_indices.reference index 115d60f60cc..52bce4efebb 100644 --- a/tests/queries/0_stateless/01917_system_data_skipping_indices.reference +++ b/tests/queries/0_stateless/01917_system_data_skipping_indices.reference @@ -1,8 +1,8 @@ -default data_01917 d1_idx minmax d1 1 0 0 0 -default data_01917 d1_null_idx minmax assumeNotNull(d1_null) 1 0 0 0 -default data_01917_2 memory set frequency * length(name) 5 0 0 0 -default data_01917_2 sample_index1 minmax length(name), name 4 0 0 0 -default data_01917_2 sample_index2 ngrambf_v1 lower(name), name 4 0 0 0 +default data_01917 d1_idx minmax minmax d1 1 0 0 0 +default data_01917 d1_null_idx minmax minmax assumeNotNull(d1_null) 1 0 0 0 +default data_01917_2 memory set set(1000) frequency * length(name) 5 0 0 0 +default data_01917_2 sample_index1 minmax minmax length(name), name 4 0 0 0 +default data_01917_2 sample_index2 ngrambf_v1 ngrambf_v1(3, 256, 2, 0) lower(name), name 4 0 0 0 2 3 d1_idx From 42fb9619b0f2f74bf467a69693859b79361a6169 Mon Sep 17 00:00:00 2001 From: Guillaume Tassery Date: Thu, 17 Nov 2022 09:01:41 +0100 Subject: [PATCH 013/127] fix test --- .../01932_alter_index_with_order.reference | 18 +++++++++--------- ...system_data_skipping_indices_size.reference | 2 +- .../02117_show_create_table_system.reference | 1 + 3 files changed, 11 insertions(+), 10 deletions(-) diff --git a/tests/queries/0_stateless/01932_alter_index_with_order.reference b/tests/queries/0_stateless/01932_alter_index_with_order.reference index eff9ea7da0e..3df860b8ae6 100644 --- a/tests/queries/0_stateless/01932_alter_index_with_order.reference +++ b/tests/queries/0_stateless/01932_alter_index_with_order.reference @@ -1,9 +1,9 @@ -default alter_index_test index_a set a 1 0 0 0 -default alter_index_test index_b minmax b 1 0 0 0 -default alter_index_test index_c set c 2 0 0 0 -default alter_index_test index_a set a 1 0 0 0 -default alter_index_test index_d set d 1 0 0 0 -default alter_index_test index_b minmax b 1 0 0 0 -default alter_index_test index_c set c 2 0 0 0 -default alter_index_test index_a set a 1 0 0 0 -default alter_index_test index_d set d 1 0 0 0 +default alter_index_test index_a set set(0) a 1 0 0 0 +default alter_index_test index_b minmax minmax b 1 0 0 0 +default alter_index_test index_c set set(0) c 2 0 0 0 +default alter_index_test index_a set set(0) a 1 0 0 0 +default alter_index_test index_d set set(0) d 1 0 0 0 +default alter_index_test index_b minmax minmax b 1 0 0 0 +default alter_index_test index_c set set(0) c 2 0 0 0 +default alter_index_test index_a set set(0) a 1 0 0 0 +default alter_index_test index_d set set(0) d 1 0 0 0 diff --git a/tests/queries/0_stateless/02028_system_data_skipping_indices_size.reference b/tests/queries/0_stateless/02028_system_data_skipping_indices_size.reference index d0378511850..e455643c01e 100644 --- a/tests/queries/0_stateless/02028_system_data_skipping_indices_size.reference +++ b/tests/queries/0_stateless/02028_system_data_skipping_indices_size.reference @@ -1 +1 @@ -default test_table value_index minmax value 1 38 12 24 +default test_table value_index minmax minmax value 1 38 12 24 diff --git a/tests/queries/0_stateless/02117_show_create_table_system.reference b/tests/queries/0_stateless/02117_show_create_table_system.reference index 7a0c383b3fb..cc1cd214340 100644 --- a/tests/queries/0_stateless/02117_show_create_table_system.reference +++ b/tests/queries/0_stateless/02117_show_create_table_system.reference @@ -106,6 +106,7 @@ CREATE TABLE system.data_skipping_indices `table` String, `name` String, `type` String, + `index_expr` String, `expr` String, `granularity` UInt64, `data_compressed_bytes` UInt64, From d41094207dc5847f07fce62ef6c3a27b0c919082 Mon Sep 17 00:00:00 2001 From: Han Fei Date: Fri, 18 Nov 2022 11:14:56 +0100 Subject: [PATCH 014/127] Update src/Storages/MergeTree/EphemeralLockInZooKeeper.cpp Co-authored-by: alesapin --- src/Storages/MergeTree/EphemeralLockInZooKeeper.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/MergeTree/EphemeralLockInZooKeeper.cpp b/src/Storages/MergeTree/EphemeralLockInZooKeeper.cpp index 835ae46aa22..9f3927c0fa9 100644 --- a/src/Storages/MergeTree/EphemeralLockInZooKeeper.cpp +++ b/src/Storages/MergeTree/EphemeralLockInZooKeeper.cpp @@ -93,7 +93,7 @@ std::tuple, std::vector> createE } } zkutil::KeeperMultiException::check(e, ops, responses); // This should always throw the proper exception - throw Exception("Unable to handle error {} when acquiring ephemeral lock in ZK", ErrorCodes::LOGICAL_ERROR); + throw Exception(ErrorCodes::LOGICAL_ERROR, "Unable to handle error {} when acquiring ephemeral lock in ZK", toString(e)); } holder_path = dynamic_cast(responses.back().get())->path_created; From f8ac75a054108513232bf3ad9b6d1905b1c862e5 Mon Sep 17 00:00:00 2001 From: Han Fei Date: Fri, 18 Nov 2022 11:15:14 +0100 Subject: [PATCH 015/127] Update src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp Co-authored-by: alesapin --- src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp index c3a0d9cdd31..b8e387c1fdb 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp @@ -479,7 +479,7 @@ void ReplicatedMergeTreeSink::finishDelayedChunk(const ZooKeeperWithFaultI auto conflict_block_ids = commitPart(zookeeper, partition.temp_part.part, partition.block_id, delayed_chunk->replicas_num, false); if (conflict_block_ids.empty()) break; - LOG_DEBUG(log, "depulicate block ids {}", toString(conflict_block_ids)); + LOG_DEBUG(log, "Found depulicate block IDs: {}", toString(conflict_block_ids)); /// partition clean conflict rewriteBlock(log, partition, conflict_block_ids); if (partition.block_id.empty()) From 035e9331963e30246830e3fbc477a91741d21c79 Mon Sep 17 00:00:00 2001 From: Han Fei Date: Fri, 18 Nov 2022 11:15:35 +0100 Subject: [PATCH 016/127] Update src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp Co-authored-by: alesapin --- src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp index b8e387c1fdb..378d4f86d96 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp @@ -263,7 +263,7 @@ size_t ReplicatedMergeTreeSink::checkQuorumPrecondition(const ZooK return replicas_number; } -std::vector getBlockIDVec(BlockWithPartition & block, String partition_id) +std::vector getHashesForBlocks(BlockWithPartition & block, String partition_id) { size_t start = 0; auto cols = block.block.getColumns(); From 8c823bcaabaf1e5d33216c373b8dfb814aff8ebb Mon Sep 17 00:00:00 2001 From: Han Fei Date: Fri, 18 Nov 2022 11:15:45 +0100 Subject: [PATCH 017/127] Update src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp Co-authored-by: alesapin --- src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp index 378d4f86d96..550eb0f3044 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp @@ -146,7 +146,7 @@ void rewriteBlock(Poco::Logger * log, typename ReplicatedMergeTreeSink::De } partition.block.block.setColumns(cols); - LOG_TRACE(log, "new block rows {}", partition.block.block.rows()); + LOG_TRACE(log, "New block rows {}", partition.block.block.rows()); partition.initBlockIDMap(); } From 3dc72b6c56f149aeedadd3351f1fd512aa6e40eb Mon Sep 17 00:00:00 2001 From: Han Fei Date: Fri, 18 Nov 2022 11:15:51 +0100 Subject: [PATCH 018/127] Update src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp Co-authored-by: alesapin --- src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp index 550eb0f3044..05f30b04fbc 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp @@ -334,7 +334,7 @@ void ReplicatedMergeTreeSink::consume(Chunk chunk) if (const auto * chunk_offsets_ptr = typeid_cast(chunk_info.get())) chunk_offsets = std::make_shared(chunk_offsets_ptr->offsets); else - throw Exception("Miss chunk info for async inserts", ErrorCodes::LOGICAL_ERROR); + throw Exception("No chunk info for async inserts", ErrorCodes::LOGICAL_ERROR); } auto part_blocks = storage.writer.splitBlockIntoParts(block, max_parts_per_block, metadata_snapshot, context, chunk_offsets); From 74c9ba8df05eccb1c44ca8a58101e700badff85a Mon Sep 17 00:00:00 2001 From: Han Fei Date: Fri, 18 Nov 2022 11:16:38 +0100 Subject: [PATCH 019/127] Update src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp Co-authored-by: alesapin --- src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp index 05f30b04fbc..2f1630950f6 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp @@ -135,7 +135,7 @@ void rewriteBlock(Poco::Logger * log, typename ReplicatedMergeTreeSink::De idx++; } - LOG_TRACE(log, "new block ids {}, new offsets{}", toString(new_block_ids), toString(new_offsets)); + LOG_TRACE(log, "New block IDs: {}, new offsets: {}", toString(new_block_ids), toString(new_offsets)); offsets = std::move(new_offsets); partition.block_id = std::move(new_block_ids); From b80a2d6c898a4e427fee91519388dca0fe9fa2c2 Mon Sep 17 00:00:00 2001 From: Han Fei Date: Fri, 18 Nov 2022 17:22:05 +0100 Subject: [PATCH 020/127] address comments --- src/Processors/Chunk.h | 1 + .../MergeTree/EphemeralLockInZooKeeper.cpp | 93 ++++---- .../MergeTree/EphemeralLockInZooKeeper.h | 20 +- .../MergeTree/ReplicatedMergeTreeSink.cpp | 209 +++++++++--------- src/Storages/StorageReplicatedMergeTree.cpp | 47 ++-- src/Storages/StorageReplicatedMergeTree.h | 9 +- .../02481_async_insert_dedup.python | 2 +- 7 files changed, 183 insertions(+), 198 deletions(-) diff --git a/src/Processors/Chunk.h b/src/Processors/Chunk.h index 398b401c0ff..15d91431b68 100644 --- a/src/Processors/Chunk.h +++ b/src/Processors/Chunk.h @@ -113,6 +113,7 @@ private: using Chunks = std::vector; +/// ChunkOffsets marks offsets of different sub-chunks, which will be used by async inserts. class ChunkOffsets : public ChunkInfo { public: diff --git a/src/Storages/MergeTree/EphemeralLockInZooKeeper.cpp b/src/Storages/MergeTree/EphemeralLockInZooKeeper.cpp index 9f3927c0fa9..f0474e48a75 100644 --- a/src/Storages/MergeTree/EphemeralLockInZooKeeper.cpp +++ b/src/Storages/MergeTree/EphemeralLockInZooKeeper.cpp @@ -13,16 +13,19 @@ namespace ErrorCodes extern const int LOGICAL_ERROR; } -EphemeralLockInZooKeeper::EphemeralLockInZooKeeper(const String & path_prefix_, const ZooKeeperWithFaultInjectionPtr & zookeeper_, const String & path_) - : zookeeper(zookeeper_), path_prefix(path_prefix_), path(path_) +EphemeralLockInZooKeeper::EphemeralLockInZooKeeper(const String & path_prefix_, const ZooKeeperWithFaultInjectionPtr & zookeeper_, const String & path_, const String & conflict_path_) + : zookeeper(zookeeper_), path_prefix(path_prefix_), path(path_), conflict_path(conflict_path_) { - if (path.size() <= path_prefix.size()) + if (conflict_path.empty() && path.size() <= path_prefix.size()) throw Exception("Logical error: name of the main node is shorter than prefix.", ErrorCodes::LOGICAL_ERROR); } +template std::optional createEphemeralLockInZooKeeper( - const String & path_prefix_, const String & temp_path, const ZooKeeperWithFaultInjectionPtr & zookeeper_, const String & deduplication_path) + const String & path_prefix_, const String & temp_path, const ZooKeeperWithFaultInjectionPtr & zookeeper_, const T & deduplication_path) { + constexpr bool async_insert = std::is_same_v>; + String path; if (deduplication_path.empty()) @@ -36,14 +39,40 @@ std::optional createEphemeralLockInZooKeeper( /// Check for duplicates in advance, to avoid superfluous block numbers allocation Coordination::Requests ops; - ops.emplace_back(zkutil::makeCreateRequest(deduplication_path, "", zkutil::CreateMode::Persistent)); - ops.emplace_back(zkutil::makeRemoveRequest(deduplication_path, -1)); + if constexpr (async_insert) + { + for (const auto & single_dedup_path : deduplication_path) + { + ops.emplace_back(zkutil::makeCreateRequest(single_dedup_path, "", zkutil::CreateMode::Persistent)); + ops.emplace_back(zkutil::makeRemoveRequest(single_dedup_path, -1)); + } + } + else + { + ops.emplace_back(zkutil::makeCreateRequest(deduplication_path, "", zkutil::CreateMode::Persistent)); + ops.emplace_back(zkutil::makeRemoveRequest(deduplication_path, -1)); + } ops.emplace_back(zkutil::makeCreateRequest(path_prefix_, holder_path, zkutil::CreateMode::EphemeralSequential)); Coordination::Responses responses; Coordination::Error e = zookeeper_->tryMulti(ops, responses); if (e != Coordination::Error::ZOK) { - if (responses[0]->error == Coordination::Error::ZNODEEXISTS) + if constexpr (async_insert) + { + for (size_t i = 0; i < deduplication_path.size(); i++) + { + if (responses[i*2]->error == Coordination::Error::ZNODEEXISTS) + { + const String & failed_op_path = deduplication_path[i]; + LOG_DEBUG( + &Poco::Logger::get("createEphemeralLockInZooKeeper"), + "Deduplication path already exists: deduplication_path={}", + failed_op_path); + return EphemeralLockInZooKeeper{"", nullptr, "", failed_op_path}; + } + } + } + else if (responses[0]->error == Coordination::Error::ZNODEEXISTS) { LOG_DEBUG( &Poco::Logger::get("createEphemeralLockInZooKeeper"), @@ -51,11 +80,8 @@ std::optional createEphemeralLockInZooKeeper( deduplication_path); return {}; } - else - { - zkutil::KeeperMultiException::check(e, ops, responses); // This should always throw the proper exception - throw Exception("Unable to handle error {} when acquiring ephemeral lock in ZK", ErrorCodes::LOGICAL_ERROR); - } + zkutil::KeeperMultiException::check(e, ops, responses); // This should always throw the proper exception + throw Exception("Unable to handle error {} when acquiring ephemeral lock in ZK", ErrorCodes::LOGICAL_ERROR); } path = dynamic_cast(responses.back().get())->path_created; @@ -64,43 +90,6 @@ std::optional createEphemeralLockInZooKeeper( return EphemeralLockInZooKeeper{path_prefix_, zookeeper_, path}; } -std::tuple, std::vector> createEphemeralLockInZooKeeper( - const String & path_prefix_, const String & temp_path, const ZooKeeperWithFaultInjectionPtr & zookeeper_, const std::vector & deduplication_paths) -{ - /// The /abandonable_lock- name is for backward compatibility. - String holder_path_prefix = temp_path + "/abandonable_lock-"; - String holder_path; - - /// Check for duplicates in advance, to avoid superfluous block numbers allocation - Coordination::Requests ops; - for (const auto & deduplication_path : deduplication_paths) - { - ops.emplace_back(zkutil::makeCreateRequest(deduplication_path, "", zkutil::CreateMode::Persistent)); - ops.emplace_back(zkutil::makeRemoveRequest(deduplication_path, -1)); - } - ops.emplace_back(zkutil::makeCreateRequest(holder_path_prefix, "", zkutil::CreateMode::EphemeralSequential)); - Coordination::Responses responses; - Coordination::Error e = zookeeper_->tryMulti(ops, responses); - if (e != Coordination::Error::ZOK) - { - /// TODO we should use some cache to check the conflict in advance. - for (const auto & response: responses) - { - if (response->error == Coordination::Error::ZNODEEXISTS) - { - String failed_op_path = zkutil::KeeperMultiException(e, ops, responses).getPathForFirstFailedOp(); - return std::make_pair(std::nullopt, std::vector({failed_op_path})); - } - } - zkutil::KeeperMultiException::check(e, ops, responses); // This should always throw the proper exception - throw Exception(ErrorCodes::LOGICAL_ERROR, "Unable to handle error {} when acquiring ephemeral lock in ZK", toString(e)); - } - - holder_path = dynamic_cast(responses.back().get())->path_created; - - return std::make_pair(EphemeralLockInZooKeeper{path_prefix_, zookeeper_, holder_path}, std::vector()); -} - void EphemeralLockInZooKeeper::unlock() { Coordination::Requests ops; @@ -230,4 +219,10 @@ EphemeralLocksInAllPartitions::~EphemeralLocksInAllPartitions() } } +template std::optional createEphemeralLockInZooKeeper( + const String & path_prefix_, const String & temp_path, const ZooKeeperWithFaultInjectionPtr & zookeeper_, const String & deduplication_path); + +template std::optional createEphemeralLockInZooKeeper>( + const String & path_prefix_, const String & temp_path, const ZooKeeperWithFaultInjectionPtr & zookeeper_, const std::vector & deduplication_path); + } diff --git a/src/Storages/MergeTree/EphemeralLockInZooKeeper.h b/src/Storages/MergeTree/EphemeralLockInZooKeeper.h index 7e972e387e4..eef0366dc8b 100644 --- a/src/Storages/MergeTree/EphemeralLockInZooKeeper.h +++ b/src/Storages/MergeTree/EphemeralLockInZooKeeper.h @@ -26,13 +26,12 @@ namespace ErrorCodes /// Since 22.11 it creates single ephemeral node with `path_prefix` that references persistent fake "secondary node". class EphemeralLockInZooKeeper : public boost::noncopyable { + template friend std::optional createEphemeralLockInZooKeeper( - const String & path_prefix_, const String & temp_path, const ZooKeeperWithFaultInjectionPtr & zookeeper_, const String & deduplication_path); - friend std::tuple, std::vector> createEphemeralLockInZooKeeper( - const String & path_prefix_, const String & temp_path, const ZooKeeperWithFaultInjectionPtr & zookeeper_, const std::vector & deduplication_path); + const String & path_prefix_, const String & temp_path, const ZooKeeperWithFaultInjectionPtr & zookeeper_, const T & deduplication_path); protected: - EphemeralLockInZooKeeper(const String & path_prefix_, const ZooKeeperWithFaultInjectionPtr & zookeeper_, const String & path_); + EphemeralLockInZooKeeper(const String & path_prefix_, const ZooKeeperWithFaultInjectionPtr & zookeeper_, const String & path_, const String & conflict_path_ = ""); public: EphemeralLockInZooKeeper() = delete; @@ -53,6 +52,7 @@ public: rhs.zookeeper = nullptr; path_prefix = std::move(rhs.path_prefix); path = std::move(rhs.path); + conflict_path = std::move(rhs.conflict_path); return *this; } @@ -67,6 +67,11 @@ public: return path; } + String getConflictPath() const + { + return conflict_path; + } + /// Parse the number at the end of the path. UInt64 getNumber() const { @@ -99,13 +104,12 @@ private: ZooKeeperWithFaultInjectionPtr zookeeper; String path_prefix; String path; + String conflict_path; }; +template std::optional createEphemeralLockInZooKeeper( - const String & path_prefix_, const String & temp_path, const ZooKeeperWithFaultInjectionPtr & zookeeper_, const String & deduplication_path); - -std::tuple, std::vector> createEphemeralLockInZooKeeper( - const String & path_prefix_, const String & temp_path, const ZooKeeperWithFaultInjectionPtr & zookeeper_, const std::vector & deduplication_path); + const String & path_prefix_, const String & temp_path, const ZooKeeperWithFaultInjectionPtr & zookeeper_, const T & deduplication_path); /// Acquires block number locks in all partitions. class EphemeralLocksInAllPartitions : public boost::noncopyable diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp index 2f1630950f6..99017392f8f 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp @@ -43,7 +43,7 @@ struct ReplicatedMergeTreeSink::DelayedChunk MergeTreeDataWriter::TemporaryPart temp_part; UInt64 elapsed_ns; BlockIDsType block_id; - BlockWithPartition block; + BlockWithPartition block_with_partition; std::unordered_map block_id_to_offset_idx; Partition() = default; @@ -51,7 +51,7 @@ struct ReplicatedMergeTreeSink::DelayedChunk : temp_part(std::move(temp_part_)), elapsed_ns(elapsed_ns_), block_id(std::move(block_id_)), - block(std::move(block_)) + block_with_partition(std::move(block_)) { initBlockIDMap(); } @@ -78,77 +78,105 @@ struct ReplicatedMergeTreeSink::DelayedChunk std::vector partitions; }; -template -inline String toString(const std::vector & vec) +namespace { - String res = "{"; - for (const auto & item : vec) - res += toString(item) + ","; - return res + "}"; -} - -/// remove the conflict parts of block for rewriting again. -void rewriteBlock(Poco::Logger * log, typename ReplicatedMergeTreeSink::DelayedChunk::Partition & partition, const std::vector & block_paths) -{ - std::vector offset_idx; - for (const auto & raw_path : block_paths) + template + inline String toString(const std::vector & vec) { - std::filesystem::path p(raw_path); - String conflict_block_id = p.filename(); - auto it = partition.block_id_to_offset_idx.find(conflict_block_id); - if (it == partition.block_id_to_offset_idx.end()) - throw Exception("unknown conflict path " + conflict_block_id, ErrorCodes::LOGICAL_ERROR); - offset_idx.push_back(it->second); + String res = "{"; + for (const auto & item : vec) + res += DB::toString(item) + ","; + return res + "}"; } - std::sort(offset_idx.begin(), offset_idx.end()); - auto & offsets = partition.block.offsets->offsets; - size_t idx = 0, remove_count = 0; - auto it = offset_idx.begin(); - std::vector new_offsets; - std::vector new_block_ids; - - /// construct filter - size_t rows = partition.block.block.rows(); - auto filter_col = ColumnUInt8::create(rows, 1u); - ColumnUInt8::Container & vec = filter_col->getData(); - UInt8 * pos = vec.data(); - for (auto & offset : offsets) + /// remove the conflict parts of block for rewriting again. + void rewriteBlock(Poco::Logger * log, typename ReplicatedMergeTreeSink::DelayedChunk::Partition & partition, const std::vector & block_paths) { - if (it != offset_idx.end() && *it == idx) + std::vector offset_idx; + for (const auto & raw_path : block_paths) { - size_t start_pos = idx > 0 ? offsets[idx - 1] : 0; - size_t end_pos = offset; - remove_count += end_pos - start_pos; - while (start_pos < end_pos) + std::filesystem::path p(raw_path); + String conflict_block_id = p.filename(); + auto it = partition.block_id_to_offset_idx.find(conflict_block_id); + if (it == partition.block_id_to_offset_idx.end()) + throw Exception("Unknown conflict path " + conflict_block_id, ErrorCodes::LOGICAL_ERROR); + offset_idx.push_back(it->second); + } + std::sort(offset_idx.begin(), offset_idx.end()); + + auto & offsets = partition.block_with_partition.offsets->offsets; + size_t idx = 0, remove_count = 0; + auto it = offset_idx.begin(); + std::vector new_offsets; + std::vector new_block_ids; + + /// construct filter + size_t rows = partition.block_with_partition.block.rows(); + auto filter_col = ColumnUInt8::create(rows, 1u); + ColumnUInt8::Container & vec = filter_col->getData(); + UInt8 * pos = vec.data(); + for (auto & offset : offsets) + { + if (it != offset_idx.end() && *it == idx) { - *(pos + start_pos) = 0; - start_pos ++; + size_t start_pos = idx > 0 ? offsets[idx - 1] : 0; + size_t end_pos = offset; + remove_count += end_pos - start_pos; + while (start_pos < end_pos) + { + *(pos + start_pos) = 0; + start_pos ++; + } + it++; } - it++; + else + { + new_offsets.push_back(offset - remove_count); + new_block_ids.push_back(partition.block_id[idx]); + } + idx++; } - else + + LOG_TRACE(log, "New block IDs: {}, new offsets: {}", toString(new_block_ids), toString(new_offsets)); + + offsets = std::move(new_offsets); + partition.block_id = std::move(new_block_ids); + auto cols = partition.block_with_partition.block.getColumns(); + for (auto & col : cols) { - new_offsets.push_back(offset - remove_count); - new_block_ids.push_back(partition.block_id[idx]); + col = col -> filter(vec, rows - remove_count); } - idx++; + partition.block_with_partition.block.setColumns(cols); + + LOG_TRACE(log, "New block rows {}", partition.block_with_partition.block.rows()); + + partition.initBlockIDMap(); } - LOG_TRACE(log, "New block IDs: {}, new offsets: {}", toString(new_block_ids), toString(new_offsets)); - - offsets = std::move(new_offsets); - partition.block_id = std::move(new_block_ids); - auto cols = partition.block.block.getColumns(); - for (auto & col : cols) + std::vector getHashesForBlocks(BlockWithPartition & block, String partition_id) { - col = col -> filter(vec, rows - remove_count); + size_t start = 0; + auto cols = block.block.getColumns(); + std::vector block_id_vec; + for (auto offset : block.offsets->offsets) + { + SipHash hash; + for (size_t i = start; i < offset; ++i) + for (const auto & col : cols) + col->updateHashWithValue(i, hash); + union + { + char bytes[16]; + UInt64 words[2]; + } hash_value; + hash.get128(hash_value.bytes); + + block_id_vec.push_back(partition_id + "_" + DB::toString(hash_value.words[0]) + "_" + DB::toString(hash_value.words[1])); + + start = offset; + } + return block_id_vec; } - partition.block.block.setColumns(cols); - - LOG_TRACE(log, "New block rows {}", partition.block.block.rows()); - - partition.initBlockIDMap(); } template @@ -263,31 +291,6 @@ size_t ReplicatedMergeTreeSink::checkQuorumPrecondition(const ZooK return replicas_number; } -std::vector getHashesForBlocks(BlockWithPartition & block, String partition_id) -{ - size_t start = 0; - auto cols = block.block.getColumns(); - std::vector block_id_vec; - for (auto offset : block.offsets->offsets) - { - SipHash hash; - for (size_t i = start; i < offset; ++i) - for (const auto & col : cols) - col->updateHashWithValue(i, hash); - union - { - char bytes[16]; - UInt64 words[2]; - } hash_value; - hash.get128(hash_value.bytes); - - block_id_vec.push_back(partition_id + "_" + toString(hash_value.words[0]) + "_" + toString(hash_value.words[1])); - - start = offset; - } - return block_id_vec; -} - template void ReplicatedMergeTreeSink::consume(Chunk chunk) { @@ -363,7 +366,7 @@ void ReplicatedMergeTreeSink::consume(Chunk chunk) if constexpr (async_insert) { /// TODO consider insert_deduplication_token - block_id = getBlockIDVec(current_block, temp_part.part->info.partition_id); + block_id = getHashesForBlocks(current_block, temp_part.part->info.partition_id); LOG_TRACE(log, "async insert part, part id {}, block id {}, offsets {}, size {}", temp_part.part->info.partition_id, toString(block_id), toString(current_block.offsets->offsets), current_block.offsets->offsets.size()); } else if (deduplicate) @@ -484,8 +487,8 @@ void ReplicatedMergeTreeSink::finishDelayedChunk(const ZooKeeperWithFaultI rewriteBlock(log, partition, conflict_block_ids); if (partition.block_id.empty()) break; - partition.block.partition = std::move(partition.temp_part.part->partition.value); - partition.temp_part = storage.writer.writeTempPart(partition.block, metadata_snapshot, context); + partition.block_with_partition.partition = std::move(partition.temp_part.part->partition.value); + partition.temp_part = storage.writer.writeTempPart(partition.block_with_partition, metadata_snapshot, context); } } @@ -605,17 +608,11 @@ std::vector ReplicatedMergeTreeSink::commitPart( if constexpr (async_insert) { for (const auto & single_block_id : block_id) - { block_id_path.push_back(storage.zookeeper_path + "/blocks/" + single_block_id); - } } else if (deduplicate_block) block_id_path = storage.zookeeper_path + "/blocks/" + block_id; - std::optional block_number_lock; - if constexpr (async_insert) - std::tie(block_number_lock, conflict_block_ids) = storage.allocateBlockNumber(part->info.partition_id, zookeeper, block_id_path); - else - block_number_lock = storage.allocateBlockNumber(part->info.partition_id, zookeeper, block_id_path); + auto block_number_lock = storage.allocateBlockNumber(part->info.partition_id, zookeeper, block_id_path); ThreadFuzzer::maybeInjectSleep(); /// Prepare transaction to ZooKeeper @@ -627,6 +624,20 @@ std::vector ReplicatedMergeTreeSink::commitPart( String existing_part_name; if (block_number_lock) { + if constexpr (async_insert) + { + /// The truth is that we always get only one path from block_number_lock. + /// This is a restriction of Keeper. Here I would like to use vector because + /// I wanna keep extensibility for future optimization, for instance, using + /// cache to resolve conflicts in advance. + String conflict_path = block_number_lock->getConflictPath(); + if (!conflict_path.empty()) + { + LOG_TRACE(log, "Cannot get lock, the conflict path is {}", conflict_path); + conflict_block_ids.push_back(conflict_path); + return; + } + } is_already_existing_part = false; block_number = block_number_lock->getNumber(); @@ -715,14 +726,8 @@ std::vector ReplicatedMergeTreeSink::commitPart( quorum_info.host_node_version)); } } - else if constexpr (async_insert) - { - LOG_TRACE(log, "cannot get lock, the conflict block ids are {}", toString(conflict_block_ids)); - if (conflict_block_ids.empty()) - throw Exception("conflict block ids and block number lock should not be empty at the same time", ErrorCodes::LOGICAL_ERROR); - return; - } - else + /// async_insert will never return null lock, because they need the conflict path. + else if constexpr (!async_insert) { is_already_existing_part = true; @@ -776,6 +781,8 @@ std::vector ReplicatedMergeTreeSink::commitPart( /// Do not check for duplicate on commit to ZK. block_id_path.clear(); } + else + throw Exception("Conflict block ids and block number lock should not be empty at the same time for async inserts", ErrorCodes::LOGICAL_ERROR); /// Information about the part. storage.getCommitPartOps(ops, part, block_id_path); diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 143634ba7ad..505e0406a5d 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -5313,37 +5313,6 @@ bool StorageReplicatedMergeTree::existsNodeCached(const ZooKeeperWithFaultInject return res; } -std::tuple, std::vector> -StorageReplicatedMergeTree::allocateBlockNumber( - const String & partition_id, - const ZooKeeperWithFaultInjectionPtr & zookeeper, - const std::vector & zookeeper_block_id_paths) const -{ - String zookeeper_table_path = zookeeper_path; - - String block_numbers_path = fs::path(zookeeper_table_path) / "block_numbers"; - String partition_path = fs::path(block_numbers_path) / partition_id; - - if (!existsNodeCached(zookeeper, partition_path)) - { - Coordination::Requests ops; - ops.push_back(zkutil::makeCreateRequest(partition_path, "", zkutil::CreateMode::Persistent)); - /// We increment data version of the block_numbers node so that it becomes possible - /// to check in a ZK transaction that the set of partitions didn't change - /// (unfortunately there is no CheckChildren op). - ops.push_back(zkutil::makeSetRequest(block_numbers_path, "", -1)); - - Coordination::Responses responses; - Coordination::Error code = zookeeper->tryMulti(ops, responses); - if (code != Coordination::Error::ZOK && code != Coordination::Error::ZNODEEXISTS) - zkutil::KeeperMultiException::check(code, ops, responses); - } - - return createEphemeralLockInZooKeeper( - fs::path(partition_path) / "block-", fs::path(zookeeper_table_path) / "temp", zookeeper, zookeeper_block_id_paths); -} - - std::optional StorageReplicatedMergeTree::allocateBlockNumber( const String & partition_id, const zkutil::ZooKeeperPtr & zookeeper, @@ -5354,11 +5323,11 @@ std::optional StorageReplicatedMergeTree::allocateBloc partition_id, std::make_shared(zookeeper), zookeeper_block_id_path, zookeeper_path_prefix); } - +template std::optional StorageReplicatedMergeTree::allocateBlockNumber( const String & partition_id, const ZooKeeperWithFaultInjectionPtr & zookeeper, - const String & zookeeper_block_id_path, + const T & zookeeper_block_id_path, const String & zookeeper_path_prefix) const { String zookeeper_table_path; @@ -8789,6 +8758,18 @@ void StorageReplicatedMergeTree::attachRestoredParts(MutableDataPartsVector && p sink->writeExistingPart(part); } +template std::optional StorageReplicatedMergeTree::allocateBlockNumber( + const String & partition_id, + const ZooKeeperWithFaultInjectionPtr & zookeeper, + const String & zookeeper_block_id_path, + const String & zookeeper_path_prefix) const; + +template std::optional StorageReplicatedMergeTree::allocateBlockNumber>( + const String & partition_id, + const ZooKeeperWithFaultInjectionPtr & zookeeper, + const std::vector & zookeeper_block_id_path, + const String & zookeeper_path_prefix) const; + #if 0 PartsTemporaryRename renamed_parts(*this, "detached/"); MutableDataPartsVector loaded_parts = tryLoadPartsToAttach(partition, attach_part, query_context, renamed_parts); diff --git a/src/Storages/StorageReplicatedMergeTree.h b/src/Storages/StorageReplicatedMergeTree.h index 9dc2f29ea9b..e564a31901e 100644 --- a/src/Storages/StorageReplicatedMergeTree.h +++ b/src/Storages/StorageReplicatedMergeTree.h @@ -732,17 +732,14 @@ private: std::optional allocateBlockNumber( const String & partition_id, const zkutil::ZooKeeperPtr & zookeeper, const String & zookeeper_block_id_path = "", const String & zookeeper_path_prefix = "") const; + + template std::optional allocateBlockNumber( const String & partition_id, const ZooKeeperWithFaultInjectionPtr & zookeeper, - const String & zookeeper_block_id_path = "", + const T & zookeeper_block_id_path, const String & zookeeper_path_prefix = "") const; - std::tuple, std::vector> allocateBlockNumber( - const String & partition_id, - const ZooKeeperWithFaultInjectionPtr & zookeeper, - const std::vector & zookeeper_block_id_paths) const; - /** Wait until all replicas, including this, execute the specified action from the log. * If replicas are added at the same time, it can not wait the added replica. * diff --git a/tests/queries/0_stateless/02481_async_insert_dedup.python b/tests/queries/0_stateless/02481_async_insert_dedup.python index 5465e7b988c..d2fcc0fabe2 100644 --- a/tests/queries/0_stateless/02481_async_insert_dedup.python +++ b/tests/queries/0_stateless/02481_async_insert_dedup.python @@ -65,7 +65,7 @@ client.query(''' CREATE TABLE t_async_insert_dedup ( EventDate DateTime, KeyID UInt32 -) Engine = ReplicatedMergeTree('/clickhouse/tables/{shard}/t_async_insert_dedup', '{replica}') +) Engine = ReplicatedMergeTree('/clickhouse/tables/{shard}/{database}/t_async_insert_dedup', '{replica}') PARTITION BY toYYYYMM(EventDate) ORDER BY (KeyID, EventDate) ''') From 81ca8e91e600b6f56255b9f32ce17f65ec244aac Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 20 Nov 2022 14:27:52 +0300 Subject: [PATCH 021/127] Update StorageSystemDataSkippingIndices.cpp --- src/Storages/System/StorageSystemDataSkippingIndices.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/System/StorageSystemDataSkippingIndices.cpp b/src/Storages/System/StorageSystemDataSkippingIndices.cpp index 15561a87394..7040eefd397 100644 --- a/src/Storages/System/StorageSystemDataSkippingIndices.cpp +++ b/src/Storages/System/StorageSystemDataSkippingIndices.cpp @@ -127,7 +127,7 @@ protected: // 'index_expr' column if (column_mask[src_index++]) { - if (auto expression = index.definition_ast->as()) + if (auto * expression = index.definition_ast->as()) res_columns[res_index++]->insert(queryToString(expression->type->clone())); else res_columns[res_index++]->insertDefault(); From 702768469b23dff890b8970a09b4013cc1c9ab66 Mon Sep 17 00:00:00 2001 From: Guillaume Tassery Date: Mon, 21 Nov 2022 09:04:44 +0100 Subject: [PATCH 022/127] cosmetic --- src/Storages/System/StorageSystemDataSkippingIndices.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Storages/System/StorageSystemDataSkippingIndices.cpp b/src/Storages/System/StorageSystemDataSkippingIndices.cpp index 15561a87394..ccb52e9a23a 100644 --- a/src/Storages/System/StorageSystemDataSkippingIndices.cpp +++ b/src/Storages/System/StorageSystemDataSkippingIndices.cpp @@ -26,7 +26,7 @@ StorageSystemDataSkippingIndices::StorageSystemDataSkippingIndices(const Storage { "table", std::make_shared() }, { "name", std::make_shared() }, { "type", std::make_shared() }, - { "index_expr", std::make_shared() }, + { "type_full", std::make_shared() }, { "expr", std::make_shared() }, { "granularity", std::make_shared() }, { "data_compressed_bytes", std::make_shared() }, @@ -124,7 +124,7 @@ protected: // 'type' column if (column_mask[src_index++]) res_columns[res_index++]->insert(index.type); - // 'index_expr' column + // 'type_full' column if (column_mask[src_index++]) { if (auto expression = index.definition_ast->as()) From 9ecda9f45c911047e04e8a089741a19ee49fe16a Mon Sep 17 00:00:00 2001 From: Guillaume Tassery Date: Mon, 21 Nov 2022 11:58:48 +0100 Subject: [PATCH 023/127] fix test --- .../0_stateless/02117_show_create_table_system.reference | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02117_show_create_table_system.reference b/tests/queries/0_stateless/02117_show_create_table_system.reference index a0b67c92017..2561155b71e 100644 --- a/tests/queries/0_stateless/02117_show_create_table_system.reference +++ b/tests/queries/0_stateless/02117_show_create_table_system.reference @@ -106,7 +106,7 @@ CREATE TABLE system.data_skipping_indices `table` String, `name` String, `type` String, - `index_expr` String, + `type_full` String, `expr` String, `granularity` UInt64, `data_compressed_bytes` UInt64, From 32c7fbec556ca3f71da1f90cd85b8cbf7cbf32ee Mon Sep 17 00:00:00 2001 From: Han Fei Date: Tue, 22 Nov 2022 15:12:00 +0100 Subject: [PATCH 024/127] use another dedup path for async inserts --- src/Storages/MergeTree/MergeTreeSettings.h | 2 ++ .../ReplicatedMergeTreeCleanupThread.cpp | 13 ++++++------ .../ReplicatedMergeTreeCleanupThread.h | 2 +- .../MergeTree/ReplicatedMergeTreeSink.cpp | 2 +- src/Storages/StorageReplicatedMergeTree.cpp | 21 +++++++++++++------ src/Storages/StorageReplicatedMergeTree.h | 2 ++ 6 files changed, 28 insertions(+), 14 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeSettings.h b/src/Storages/MergeTree/MergeTreeSettings.h index 844c1ddbfe5..82a0a04257b 100644 --- a/src/Storages/MergeTree/MergeTreeSettings.h +++ b/src/Storages/MergeTree/MergeTreeSettings.h @@ -79,6 +79,8 @@ struct Settings; /** Replication settings. */ \ M(UInt64, replicated_deduplication_window, 100, "How many last blocks of hashes should be kept in ZooKeeper (old blocks will be deleted).", 0) \ M(UInt64, replicated_deduplication_window_seconds, 7 * 24 * 60 * 60 /* one week */, "Similar to \"replicated_deduplication_window\", but determines old blocks by their lifetime. Hash of an inserted block will be deleted (and the block will not be deduplicated after) if it outside of one \"window\". You can set very big replicated_deduplication_window to avoid duplicating INSERTs during that period of time.", 0) \ + M(UInt64, replicated_deduplication_window_for_async_inserts, 10000, "How many last hash values of async_insert blocks should be kept in ZooKeeper (old blocks will be deleted).", 0) \ + M(UInt64, replicated_deduplication_window_seconds_for_async_inserts, 7 * 24 * 60 * 60 /* one week */, "Similar to \"replicated_deduplication_window_for_async_inserts\", but determines old blocks by their lifetime. Hash of an inserted block will be deleted (and the block will not be deduplicated after) if it outside of one \"window\". You can set very big replicated_deduplication_window to avoid duplicating INSERTs during that period of time.", 0) \ M(UInt64, max_replicated_logs_to_keep, 1000, "How many records may be in log, if there is inactive replica. Inactive replica becomes lost when when this number exceed.", 0) \ M(UInt64, min_replicated_logs_to_keep, 10, "Keep about this number of last records in ZooKeeper log, even if they are obsolete. It doesn't affect work of tables: used only to diagnose ZooKeeper log before cleaning.", 0) \ M(Seconds, prefer_fetch_merged_part_time_threshold, 3600, "If time passed after replication log entry creation exceeds this threshold and sum size of parts is greater than \"prefer_fetch_merged_part_size_threshold\", prefer fetching merged part from replica instead of doing merge locally. To speed up very long merges.", 0) \ diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeCleanupThread.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeCleanupThread.cpp index 7993840f1d9..315f471fd5c 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeCleanupThread.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeCleanupThread.cpp @@ -74,7 +74,9 @@ void ReplicatedMergeTreeCleanupThread::iterate() if (storage.is_leader) { clearOldLogs(); - clearOldBlocks(); + auto storage_settings = storage.getSettings(); + clearOldBlocks("blocks", storage_settings->replicated_deduplication_window_seconds, storage_settings->replicated_deduplication_window); + clearOldBlocks("async_blocks", storage_settings->replicated_deduplication_window_seconds_for_async_inserts, storage_settings->replicated_deduplication_window_for_async_inserts); clearOldMutations(); storage.clearEmptyParts(); } @@ -321,10 +323,9 @@ struct ReplicatedMergeTreeCleanupThread::NodeWithStat } }; -void ReplicatedMergeTreeCleanupThread::clearOldBlocks() +void ReplicatedMergeTreeCleanupThread::clearOldBlocks(const String & blocks_dir_name, UInt64 window_seconds, UInt64 window_size) { auto zookeeper = storage.getZooKeeper(); - auto storage_settings = storage.getSettings(); std::vector timed_blocks; getBlocksSortedByTime(*zookeeper, timed_blocks); @@ -336,12 +337,12 @@ void ReplicatedMergeTreeCleanupThread::clearOldBlocks() Int64 current_time = timed_blocks.front().ctime; Int64 time_threshold = std::max( static_cast(0), - current_time - static_cast(1000 * storage_settings->replicated_deduplication_window_seconds)); + current_time - static_cast(1000 * window_seconds)); /// Virtual node, all nodes that are "greater" than this one will be deleted NodeWithStat block_threshold{{}, time_threshold, 0}; - size_t current_deduplication_window = std::min(timed_blocks.size(), storage_settings->replicated_deduplication_window); + size_t current_deduplication_window = std::min(timed_blocks.size(), window_size); auto first_outdated_block_fixed_threshold = timed_blocks.begin() + current_deduplication_window; auto first_outdated_block_time_threshold = std::upper_bound( timed_blocks.begin(), timed_blocks.end(), block_threshold, NodeWithStat::greaterByTime); @@ -359,7 +360,7 @@ void ReplicatedMergeTreeCleanupThread::clearOldBlocks() zkutil::AsyncResponses try_remove_futures; for (auto it = first_outdated_block; it != timed_blocks.end(); ++it) { - String path = storage.zookeeper_path + "/blocks/" + it->node; + String path = storage.zookeeper_path + "/" + blocks_dir_name + "/" + it->node; try_remove_futures.emplace_back(path, zookeeper->asyncTryRemove(path, it->version)); } diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeCleanupThread.h b/src/Storages/MergeTree/ReplicatedMergeTreeCleanupThread.h index 6b4ce78cd35..f8731ca0f43 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeCleanupThread.h +++ b/src/Storages/MergeTree/ReplicatedMergeTreeCleanupThread.h @@ -53,7 +53,7 @@ private: size_t replicas_count, const zkutil::ZooKeeperPtr & zookeeper); /// Remove old block hashes from ZooKeeper. This is done by the leader replica. - void clearOldBlocks(); + void clearOldBlocks(const String & blocks_dir_name, UInt64 window_seconds, UInt64 window_size); /// Remove old mutations that are done from ZooKeeper. This is done by the leader replica. void clearOldMutations(); diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp index 99017392f8f..6c028dc08b1 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp @@ -608,7 +608,7 @@ std::vector ReplicatedMergeTreeSink::commitPart( if constexpr (async_insert) { for (const auto & single_block_id : block_id) - block_id_path.push_back(storage.zookeeper_path + "/blocks/" + single_block_id); + block_id_path.push_back(storage.zookeeper_path + "/async_blocks/" + single_block_id); } else if (deduplicate_block) block_id_path = storage.zookeeper_path + "/blocks/" + block_id; diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 505e0406a5d..b3f75eb1524 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -702,6 +702,8 @@ bool StorageReplicatedMergeTree::createTableIfNotExists(const StorageMetadataPtr zkutil::CreateMode::Persistent)); ops.emplace_back(zkutil::makeCreateRequest(zookeeper_path + "/blocks", "", zkutil::CreateMode::Persistent)); + ops.emplace_back(zkutil::makeCreateRequest(zookeeper_path + "/async_blocks", "", + zkutil::CreateMode::Persistent)); ops.emplace_back(zkutil::makeCreateRequest(zookeeper_path + "/block_numbers", "", zkutil::CreateMode::Persistent)); ops.emplace_back(zkutil::makeCreateRequest(zookeeper_path + "/nonincrement_block_numbers", "", @@ -1012,7 +1014,7 @@ bool StorageReplicatedMergeTree::removeTableNodesFromZooKeeper(zkutil::ZooKeeper /// NOTE /block_numbers/ actually is not flat, because /block_numbers// may have ephemeral children, /// but we assume that all ephemeral block locks are already removed when table is being dropped. - static constexpr std::array flat_nodes = {"block_numbers", "blocks", "leader_election", "log", "mutations", "pinned_part_uuids"}; + static constexpr std::array flat_nodes = {"block_numbers", "blocks", "async_blocks", "leader_election", "log", "mutations", "pinned_part_uuids"}; /// First try to remove paths that are known to be flat for (const auto * node : flat_nodes) @@ -4491,8 +4493,8 @@ SinkToStoragePtr StorageReplicatedMergeTree::write(const ASTPtr & /*query*/, con const auto storage_settings_ptr = getSettings(); const Settings & query_settings = local_context->getSettingsRef(); bool deduplicate = storage_settings_ptr->replicated_deduplication_window != 0 && query_settings.insert_deduplicate; - bool async_insert = query_settings.async_insert; - if (deduplicate && async_insert) + bool async_deduplicate = query_settings.async_insert && storage_settings_ptr->replicated_deduplication_window_for_async_inserts != 0 && query_settings.insert_deduplicate; + if (async_deduplicate) return std::make_shared>( *this, metadata_snapshot, query_settings.insert_quorum.valueOr(0), query_settings.insert_quorum_timeout.totalMilliseconds(), @@ -6499,17 +6501,24 @@ void StorageReplicatedMergeTree::clearLockedBlockNumbersInPartition( void StorageReplicatedMergeTree::getClearBlocksInPartitionOps( Coordination::Requests & ops, zkutil::ZooKeeper & zookeeper, const String & partition_id, Int64 min_block_num, Int64 max_block_num) +{ + getClearBlocksInPartitionOpsImpl(ops, zookeeper, partition_id, min_block_num, max_block_num, "blocks"); + getClearBlocksInPartitionOpsImpl(ops, zookeeper, partition_id, min_block_num, max_block_num, "async_blocks"); +} + +void StorageReplicatedMergeTree::getClearBlocksInPartitionOpsImpl( + Coordination::Requests & ops, zkutil::ZooKeeper & zookeeper, const String & partition_id, Int64 min_block_num, Int64 max_block_num, const String & blocks_dir_name) { Strings blocks; - if (Coordination::Error::ZOK != zookeeper.tryGetChildren(fs::path(zookeeper_path) / "blocks", blocks)) - throw Exception(zookeeper_path + "/blocks doesn't exist", ErrorCodes::NOT_FOUND_NODE); + if (Coordination::Error::ZOK != zookeeper.tryGetChildren(fs::path(zookeeper_path) / blocks_dir_name, blocks)) + throw Exception(zookeeper_path + "/" + blocks_dir_name + "blocks doesn't exist", ErrorCodes::NOT_FOUND_NODE); String partition_prefix = partition_id + "_"; Strings paths_to_get; for (const String & block_id : blocks) if (startsWith(block_id, partition_prefix)) - paths_to_get.push_back(fs::path(zookeeper_path) / "blocks" / block_id); + paths_to_get.push_back(fs::path(zookeeper_path) / blocks_dir_name / block_id); auto results = zookeeper.tryGet(paths_to_get); diff --git a/src/Storages/StorageReplicatedMergeTree.h b/src/Storages/StorageReplicatedMergeTree.h index e564a31901e..bf9e80dee78 100644 --- a/src/Storages/StorageReplicatedMergeTree.h +++ b/src/Storages/StorageReplicatedMergeTree.h @@ -782,6 +782,8 @@ private: void clearLockedBlockNumbersInPartition(zkutil::ZooKeeper & zookeeper, const String & partition_id, Int64 min_block_num, Int64 max_block_num); void getClearBlocksInPartitionOps(Coordination::Requests & ops, zkutil::ZooKeeper & zookeeper, const String & partition_id, Int64 min_block_num, Int64 max_block_num); + + void getClearBlocksInPartitionOpsImpl(Coordination::Requests & ops, zkutil::ZooKeeper & zookeeper, const String & partition_id, Int64 min_block_num, Int64 max_block_num, const String & blocks_dir_name); /// Remove block IDs from `blocks/` in ZooKeeper for the given partition ID in the given block number range. void clearBlocksInPartition( zkutil::ZooKeeper & zookeeper, const String & partition_id, Int64 min_block_num, Int64 max_block_num); From 4bf20d500c96e64a29d09aaa7c3f6e32741b64ff Mon Sep 17 00:00:00 2001 From: Han Fei Date: Tue, 22 Nov 2022 17:27:01 +0100 Subject: [PATCH 025/127] fix tests --- .../0_stateless/02221_system_zookeeper_unrestricted.reference | 2 ++ .../02221_system_zookeeper_unrestricted_like.reference | 2 ++ 2 files changed, 4 insertions(+) diff --git a/tests/queries/0_stateless/02221_system_zookeeper_unrestricted.reference b/tests/queries/0_stateless/02221_system_zookeeper_unrestricted.reference index d250d1c9140..60d3c78d740 100644 --- a/tests/queries/0_stateless/02221_system_zookeeper_unrestricted.reference +++ b/tests/queries/0_stateless/02221_system_zookeeper_unrestricted.reference @@ -6,6 +6,8 @@ abandonable_lock-other abandonable_lock-other alter_partition_version alter_partition_version +async_blocks +async_blocks block_numbers block_numbers blocks diff --git a/tests/queries/0_stateless/02221_system_zookeeper_unrestricted_like.reference b/tests/queries/0_stateless/02221_system_zookeeper_unrestricted_like.reference index 67920b13d71..c59be6a3af5 100644 --- a/tests/queries/0_stateless/02221_system_zookeeper_unrestricted_like.reference +++ b/tests/queries/0_stateless/02221_system_zookeeper_unrestricted_like.reference @@ -2,6 +2,7 @@ abandonable_lock-insert abandonable_lock-other alter_partition_version +async_blocks block_numbers blocks columns @@ -42,6 +43,7 @@ zero_copy_s3 abandonable_lock-insert abandonable_lock-other alter_partition_version +async_blocks block_numbers blocks columns From c20dbb777047c6636f8a90f048f0b859c0e3e46d Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 22 Nov 2022 19:34:38 +0100 Subject: [PATCH 026/127] Move password reset message from client to server --- programs/client/Client.cpp | 12 ++---------- src/Access/AccessControl.cpp | 16 ++++++++++++++-- 2 files changed, 16 insertions(+), 12 deletions(-) diff --git a/programs/client/Client.cpp b/programs/client/Client.cpp index 6e289b57845..115f76174bd 100644 --- a/programs/client/Client.cpp +++ b/programs/client/Client.cpp @@ -348,17 +348,9 @@ void Client::connect() } catch (const Exception & e) { - /// It is typical when users install ClickHouse, type some password and instantly forget it. - /// This problem can't be fixed with reconnection so it is not attempted - if ((connection_parameters.user.empty() || connection_parameters.user == "default") - && e.code() == DB::ErrorCodes::AUTHENTICATION_FAILED) + if (e.code() == DB::ErrorCodes::AUTHENTICATION_FAILED) { - std::cerr << std::endl - << "If you have installed ClickHouse and forgot password you can reset it in the configuration file." << std::endl - << "The password for default user is typically located at /etc/clickhouse-server/users.d/default-password.xml" << std::endl - << "and deleting this file will reset the password." << std::endl - << "See also /etc/clickhouse-server/users.xml on the server where ClickHouse is installed." << std::endl - << std::endl; + /// This problem can't be fixed with reconnection so it is not attempted throw; } else diff --git a/src/Access/AccessControl.cpp b/src/Access/AccessControl.cpp index 04642df6f40..6373df9c703 100644 --- a/src/Access/AccessControl.cpp +++ b/src/Access/AccessControl.cpp @@ -20,7 +20,7 @@ #include #include #include -#include +#include #include #include #include @@ -454,9 +454,21 @@ UUID AccessControl::authenticate(const Credentials & credentials, const Poco::Ne { tryLogCurrentException(getLogger(), "from: " + address.toString() + ", user: " + credentials.getUserName() + ": Authentication failed"); + WriteBufferFromOwnString message; + message << credentials.getUserName() << ": Authentication failed: password is incorrect or there is no user with such name."; + + /// Better exception message for usability. + /// It is typical when users install ClickHouse, type some password and instantly forget it. + if (credentials.getUserName().empty() || credentials.getUserName() == "default") + message << "\n\n" + << "If you have installed ClickHouse and forgot password you can reset it in the configuration file.\n" + << "The password for default user is typically located at /etc/clickhouse-server/users.d/default-password.xml\n" + << "and deleting this file will reset the password.\n" + << "See also /etc/clickhouse-server/users.xml on the server where ClickHouse is installed.\n\n"; + /// We use the same message for all authentication failures because we don't want to give away any unnecessary information for security reasons, /// only the log will show the exact reason. - throw Exception(credentials.getUserName() + ": Authentication failed: password is incorrect or there is no user with such name", ErrorCodes::AUTHENTICATION_FAILED); + throw Exception(message.str(), ErrorCodes::AUTHENTICATION_FAILED); } } From c816ed4f0e533035e2c1f76c85840278bb13c883 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 22 Nov 2022 19:36:52 +0100 Subject: [PATCH 027/127] Move password reset message from client to server --- src/Access/AccessControl.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Access/AccessControl.cpp b/src/Access/AccessControl.cpp index 6373df9c703..c72e66f12cf 100644 --- a/src/Access/AccessControl.cpp +++ b/src/Access/AccessControl.cpp @@ -455,7 +455,7 @@ UUID AccessControl::authenticate(const Credentials & credentials, const Poco::Ne tryLogCurrentException(getLogger(), "from: " + address.toString() + ", user: " + credentials.getUserName() + ": Authentication failed"); WriteBufferFromOwnString message; - message << credentials.getUserName() << ": Authentication failed: password is incorrect or there is no user with such name."; + message << credentials.getUserName() << ": Authentication failed: password is incorrect, or there is no user with such name."; /// Better exception message for usability. /// It is typical when users install ClickHouse, type some password and instantly forget it. From 14d4980e26f434ee6093b2979420fff74ec11019 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Tue, 22 Nov 2022 15:59:03 +0100 Subject: [PATCH 028/127] Fix repainting of the client prompt after fuzzy search Signed-off-by: Azat Khuzhin --- base/base/ReplxxLineReader.cpp | 1 + 1 file changed, 1 insertion(+) diff --git a/base/base/ReplxxLineReader.cpp b/base/base/ReplxxLineReader.cpp index e0dc81af5b0..569fcf3af1f 100644 --- a/base/base/ReplxxLineReader.cpp +++ b/base/base/ReplxxLineReader.cpp @@ -420,6 +420,7 @@ ReplxxLineReader::ReplxxLineReader( auto interactive_history_search = [this](char32_t code) { openInteractiveHistorySearch(); + rx.invoke(Replxx::ACTION::CLEAR_SELF, code); return rx.invoke(Replxx::ACTION::REPAINT, code); }; rx.bind_key(Replxx::KEY::control('R'), interactive_history_search); From 59e647077d7f3537272e5c2bb361d0eff38397ef Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Tue, 22 Nov 2022 15:53:56 +0100 Subject: [PATCH 029/127] Improve fuzzy search in clickhouse-client/clickhouse-local I found few difference in sk/fzf and after this patch I think that it should behave better. So the following commands will be used by default: - fzf --read0 --tac --tiebreak=index --height=30% - sk --read0 --tac --tiebreak=-score --height=30% Those two more or less allows true fuzzy reserve search. Signed-off-by: Azat Khuzhin --- base/base/ReplxxLineReader.cpp | 30 +++++++++++++++++++++--------- base/base/ReplxxLineReader.h | 9 +++++++++ 2 files changed, 30 insertions(+), 9 deletions(-) diff --git a/base/base/ReplxxLineReader.cpp b/base/base/ReplxxLineReader.cpp index 569fcf3af1f..03fffaf3feb 100644 --- a/base/base/ReplxxLineReader.cpp +++ b/base/base/ReplxxLineReader.cpp @@ -38,7 +38,7 @@ std::string getEditor() return editor; } -std::string getFuzzyFinder() +std::pair getFuzzyFinder() { const char * env_path = std::getenv("PATH"); // NOLINT(concurrency-mt-unsafe) @@ -52,14 +52,14 @@ std::string getFuzzyFinder() std::filesystem::path path(path_str); std::filesystem::path sk_bin_path = path / "sk"; if (!access(sk_bin_path.c_str(), X_OK)) - return sk_bin_path; + return {sk_bin_path, FUZZY_FINDER_SKIM}; std::filesystem::path fzf_bin_path = path / "fzf"; if (!access(fzf_bin_path.c_str(), X_OK)) - return fzf_bin_path; + return {fzf_bin_path, FUZZY_FINDER_FZF}; } - return {}; + return {"", FUZZY_FINDER_NONE}; } /// See comments in ShellCommand::executeImpl() @@ -305,11 +305,12 @@ ReplxxLineReader::ReplxxLineReader( replxx::Replxx::highlighter_callback_t highlighter_) : LineReader(history_file_path_, multiline_, std::move(extenders_), std::move(delimiters_)), highlighter(std::move(highlighter_)) , editor(getEditor()) - , fuzzy_finder(getFuzzyFinder()) { using namespace std::placeholders; using Replxx = replxx::Replxx; + std::tie(fuzzy_finder, fuzzy_finder_type) = getFuzzyFinder(); + if (!history_file_path.empty()) { history_file_fd = open(history_file_path.c_str(), O_RDWR); @@ -415,7 +416,7 @@ ReplxxLineReader::ReplxxLineReader( rx.bind_key(Replxx::KEY::meta('#'), insert_comment_action); /// interactive search in history (requires fzf/sk) - if (!fuzzy_finder.empty()) + if (fuzzy_finder_type != FUZZY_FINDER_NONE) { auto interactive_history_search = [this](char32_t code) { @@ -516,9 +517,20 @@ void ReplxxLineReader::openInteractiveHistorySearch() /// /// And also note, that fzf and skim is 95% compatible (at least option /// that is used here) - std::string fuzzy_finder_command = fmt::format( - "{} --read0 --tac --no-sort --tiebreak=index --bind=ctrl-r:toggle-sort --height=30% < {} > {}", - fuzzy_finder, history_file.getPath(), output_file.getPath()); + std::string fuzzy_finder_command = fmt::format("{} --read0 --height=30%", fuzzy_finder); + switch (fuzzy_finder_type) + { + case FUZZY_FINDER_SKIM: + fuzzy_finder_command += " --tac --tiebreak=-score"; + break; + case FUZZY_FINDER_FZF: + fuzzy_finder_command += " --tac --tiebreak=index"; + break; + case FUZZY_FINDER_NONE: + /// assertion for !fuzzy_finder.empty() is enough + break; + } + fuzzy_finder_command += fmt::format(" < {} > {}", history_file.getPath(), output_file.getPath()); char * const argv[] = {sh, sh_c, fuzzy_finder_command.data(), nullptr}; try diff --git a/base/base/ReplxxLineReader.h b/base/base/ReplxxLineReader.h index fea1405a208..9be3b3aa993 100644 --- a/base/base/ReplxxLineReader.h +++ b/base/base/ReplxxLineReader.h @@ -4,6 +4,14 @@ #include +enum FuzzyFinderType +{ + FUZZY_FINDER_NONE, + /// Use https://github.com/junegunn/fzf + FUZZY_FINDER_FZF, + /// Use https://github.com/lotabout/skim + FUZZY_FINDER_SKIM, +}; class ReplxxLineReader : public LineReader { @@ -38,4 +46,5 @@ private: std::string editor; std::string fuzzy_finder; + FuzzyFinderType fuzzy_finder_type = FUZZY_FINDER_NONE; }; From 3103de8664ef76773bb56a69a36aad86df1e5fd3 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 22 Nov 2022 19:44:59 +0100 Subject: [PATCH 030/127] Add a test --- .../0_stateless/02483_password_reset.reference | 4 ++++ tests/queries/0_stateless/02483_password_reset.sh | 13 +++++++++++++ 2 files changed, 17 insertions(+) create mode 100644 tests/queries/0_stateless/02483_password_reset.reference create mode 100755 tests/queries/0_stateless/02483_password_reset.sh diff --git a/tests/queries/0_stateless/02483_password_reset.reference b/tests/queries/0_stateless/02483_password_reset.reference new file mode 100644 index 00000000000..703fe3d06e3 --- /dev/null +++ b/tests/queries/0_stateless/02483_password_reset.reference @@ -0,0 +1,4 @@ +password is incorrect +reset the password +password is incorrect +reset the password diff --git a/tests/queries/0_stateless/02483_password_reset.sh b/tests/queries/0_stateless/02483_password_reset.sh new file mode 100755 index 00000000000..7e2e4eedc85 --- /dev/null +++ b/tests/queries/0_stateless/02483_password_reset.sh @@ -0,0 +1,13 @@ +#!/usr/bin/env bash + +set -e + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +$CLICKHOUSE_CLIENT --password incorrect-password --query "SELECT 1" 2>&1 | grep -o 'password is incorrect' +$CLICKHOUSE_CLIENT --password incorrect-password --query "SELECT 1" 2>&1 | grep -o -P 'reset.+password' + +$CLICKHOUSE_CURL -sS "$CLICKHOUSE_URL" -u default:incorrect-password -d "SELECT 1" | grep -o 'password is incorrect' +$CLICKHOUSE_CURL -sS "$CLICKHOUSE_URL" -u default:incorrect-password -d "SELECT 1" | grep -o -P 'reset.+password' From 6d0e9001bc7a15d38e27f0acd53d362924805bb3 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 22 Nov 2022 19:46:03 +0100 Subject: [PATCH 031/127] Add a test --- tests/queries/0_stateless/02483_password_reset.sh | 2 ++ 1 file changed, 2 insertions(+) diff --git a/tests/queries/0_stateless/02483_password_reset.sh b/tests/queries/0_stateless/02483_password_reset.sh index 7e2e4eedc85..4361954ff2b 100755 --- a/tests/queries/0_stateless/02483_password_reset.sh +++ b/tests/queries/0_stateless/02483_password_reset.sh @@ -6,6 +6,8 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh +# The exception message contains instruction on how to reset the password: + $CLICKHOUSE_CLIENT --password incorrect-password --query "SELECT 1" 2>&1 | grep -o 'password is incorrect' $CLICKHOUSE_CLIENT --password incorrect-password --query "SELECT 1" 2>&1 | grep -o -P 'reset.+password' From ae335e9c220f0dc6f1b8532e4ec41c9841df4162 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 22 Nov 2022 21:58:30 +0300 Subject: [PATCH 032/127] Update AccessControl.cpp --- src/Access/AccessControl.cpp | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/src/Access/AccessControl.cpp b/src/Access/AccessControl.cpp index c72e66f12cf..8c8922e2b7e 100644 --- a/src/Access/AccessControl.cpp +++ b/src/Access/AccessControl.cpp @@ -445,6 +445,8 @@ AccessChangesNotifier & AccessControl::getChangesNotifier() UUID AccessControl::authenticate(const Credentials & credentials, const Poco::Net::IPAddress & address) const { + chassert(!credentials.getUserName().empty()); + try { return MultipleAccessStorage::authenticate(credentials, address, *external_authenticators, allow_no_password, @@ -459,7 +461,7 @@ UUID AccessControl::authenticate(const Credentials & credentials, const Poco::Ne /// Better exception message for usability. /// It is typical when users install ClickHouse, type some password and instantly forget it. - if (credentials.getUserName().empty() || credentials.getUserName() == "default") + if (credentials.getUserName() == "default") message << "\n\n" << "If you have installed ClickHouse and forgot password you can reset it in the configuration file.\n" << "The password for default user is typically located at /etc/clickhouse-server/users.d/default-password.xml\n" From 7a9adbc2859c2469a575fff7817ae432289c68c8 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Tue, 22 Nov 2022 20:06:14 +0100 Subject: [PATCH 033/127] Escape shell arguments for fzf/skim Signed-off-by: Azat Khuzhin --- base/base/ReplxxLineReader.cpp | 11 ++++++++++- 1 file changed, 10 insertions(+), 1 deletion(-) diff --git a/base/base/ReplxxLineReader.cpp b/base/base/ReplxxLineReader.cpp index 03fffaf3feb..b86746365b7 100644 --- a/base/base/ReplxxLineReader.cpp +++ b/base/base/ReplxxLineReader.cpp @@ -17,6 +17,7 @@ #include #include #include +#include #include /// is_any_of namespace @@ -62,6 +63,12 @@ std::pair getFuzzyFinder() return {"", FUZZY_FINDER_NONE}; } +String escapeShellArgument(std::string arg) +{ + boost::replace_all(arg, "'", "'\\''"); + return fmt::format("'{}'", arg); +} + /// See comments in ShellCommand::executeImpl() /// (for the vfork via dlsym()) int executeCommand(char * const argv[]) @@ -530,7 +537,9 @@ void ReplxxLineReader::openInteractiveHistorySearch() /// assertion for !fuzzy_finder.empty() is enough break; } - fuzzy_finder_command += fmt::format(" < {} > {}", history_file.getPath(), output_file.getPath()); + fuzzy_finder_command += fmt::format(" < {} > {}", + escapeShellArgument(history_file.getPath()), + escapeShellArgument(output_file.getPath())); char * const argv[] = {sh, sh_c, fuzzy_finder_command.data(), nullptr}; try From c1283a6bb516dac29df166276c48ed17c3787b42 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 22 Nov 2022 22:17:20 +0300 Subject: [PATCH 034/127] Update StorageSystemDataSkippingIndices.cpp --- src/Storages/System/StorageSystemDataSkippingIndices.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/System/StorageSystemDataSkippingIndices.cpp b/src/Storages/System/StorageSystemDataSkippingIndices.cpp index aa0f186b56a..d2d0e69505d 100644 --- a/src/Storages/System/StorageSystemDataSkippingIndices.cpp +++ b/src/Storages/System/StorageSystemDataSkippingIndices.cpp @@ -128,7 +128,7 @@ protected: if (column_mask[src_index++]) { if (auto * expression = index.definition_ast->as()) - res_columns[res_index++]->insert(queryToString(expression->type->clone())); + res_columns[res_index++]->insert(queryToString(expression->type)); else res_columns[res_index++]->insertDefault(); } From 3125aca59debdc7200371444f772d17ccc9bc30f Mon Sep 17 00:00:00 2001 From: Han Fei Date: Wed, 23 Nov 2022 12:37:00 +0100 Subject: [PATCH 035/127] disable keeper fault injection --- tests/queries/0_stateless/02481_async_insert_dedup.python | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/tests/queries/0_stateless/02481_async_insert_dedup.python b/tests/queries/0_stateless/02481_async_insert_dedup.python index d2fcc0fabe2..1eb9e4c5faf 100644 --- a/tests/queries/0_stateless/02481_async_insert_dedup.python +++ b/tests/queries/0_stateless/02481_async_insert_dedup.python @@ -54,9 +54,9 @@ def generate_data(q, total_number): def fetch_and_insert_data(q, client): while True: insert = q.get() - client.query(insert, settings = {"async_insert": 1, "wait_for_async_insert": 0, "async_insert_busy_timeout_ms": 1500}) + client.query(insert, settings = {"async_insert": 1, "wait_for_async_insert": 0, "async_insert_busy_timeout_ms": 1500, "insert_keeper_fault_injection_probability": 0}) q.task_done() - sleep_time = random.randint(300, 1000) + sleep_time = random.randint(50, 500) time.sleep(sleep_time/1000.0) # main process @@ -82,7 +82,7 @@ for i in range(3): gen.join() -time.sleep(3) +time.sleep(5) result = client.query("select count(*) from t_async_insert_dedup") print(result, flush=True) From a7a3bc4803a1b47e532690ef34747efdae703da0 Mon Sep 17 00:00:00 2001 From: Guillaume Tassery Date: Wed, 23 Nov 2022 12:57:09 +0100 Subject: [PATCH 036/127] queryToString who can accept raw pointer as well --- src/Parsers/queryToString.cpp | 5 +++++ src/Parsers/queryToString.h | 1 + 2 files changed, 6 insertions(+) diff --git a/src/Parsers/queryToString.cpp b/src/Parsers/queryToString.cpp index 9721aa1f128..30a38294d43 100644 --- a/src/Parsers/queryToString.cpp +++ b/src/Parsers/queryToString.cpp @@ -8,6 +8,11 @@ namespace DB return queryToString(*query); } + String queryToString(const IAST * query) + { + return queryToString(*query); + } + String queryToString(const IAST & query) { return serializeAST(query); diff --git a/src/Parsers/queryToString.h b/src/Parsers/queryToString.h index 873de218293..7e4dd5da707 100644 --- a/src/Parsers/queryToString.h +++ b/src/Parsers/queryToString.h @@ -5,5 +5,6 @@ namespace DB { String queryToString(const ASTPtr & query); + String queryToString(const IAST * query); String queryToString(const IAST & query); } From d0f5fcdf0908ce8bf2ff39692bfea64876103275 Mon Sep 17 00:00:00 2001 From: Han Fei Date: Wed, 23 Nov 2022 21:07:59 +0100 Subject: [PATCH 037/127] update ut and refine logs --- .../MergeTree/ReplicatedMergeTreeSink.cpp | 16 +++++-- .../MergeTree/tests/gtest_async_inserts.cpp | 45 +++++++++++++++++++ 2 files changed, 57 insertions(+), 4 deletions(-) create mode 100644 src/Storages/MergeTree/tests/gtest_async_inserts.cpp diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp index 6c028dc08b1..38801b818dc 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp @@ -80,12 +80,19 @@ struct ReplicatedMergeTreeSink::DelayedChunk namespace { + /// Convert block id vector to string. Output at most 50 rows. template inline String toString(const std::vector & vec) { String res = "{"; - for (const auto & item : vec) - res += DB::toString(item) + ","; + size_t size = vec.size(); + if (size > 50) size = 50; + for (size_t i = 0; i < size; ++i) + { + res += DB::toString(vec[i]); + if (i + 1 < size) + res += ","; + } return res + "}"; } @@ -137,7 +144,7 @@ namespace idx++; } - LOG_TRACE(log, "New block IDs: {}, new offsets: {}", toString(new_block_ids), toString(new_offsets)); + LOG_TRACE(log, "New block IDs: {}, new offsets: {}, size: {}", toString(new_block_ids), toString(new_offsets), new_offsets.size()); offsets = std::move(new_offsets); partition.block_id = std::move(new_block_ids); @@ -476,13 +483,14 @@ void ReplicatedMergeTreeSink::finishDelayedChunk(const ZooKeeperWithFaultI for (auto & partition: delayed_chunk->partitions) { + int retry_times = 0; while (true) { partition.temp_part.finalize(); auto conflict_block_ids = commitPart(zookeeper, partition.temp_part.part, partition.block_id, delayed_chunk->replicas_num, false); if (conflict_block_ids.empty()) break; - LOG_DEBUG(log, "Found depulicate block IDs: {}", toString(conflict_block_ids)); + LOG_DEBUG(log, "Found depulicate block IDs: {}, retry times {}", toString(conflict_block_ids), ++retry_times); /// partition clean conflict rewriteBlock(log, partition, conflict_block_ids); if (partition.block_id.empty()) diff --git a/src/Storages/MergeTree/tests/gtest_async_inserts.cpp b/src/Storages/MergeTree/tests/gtest_async_inserts.cpp new file mode 100644 index 00000000000..55d388c1165 --- /dev/null +++ b/src/Storages/MergeTree/tests/gtest_async_inserts.cpp @@ -0,0 +1,45 @@ +#include "config.h" + +#include +#include +#include +#include + + namespace DB { + +std::vector scatterOffsetsBySelector(ChunkOffsetsPtr chunk_offsets, const IColumn::Selector & selector, size_t partition_num); + +class AsyncInsertsTest : public ::testing::TestPartResult +{}; + + +TEST(AsyncInsertsTest, testScatterOffsetsBySelector) +{ + auto testImpl = [](std::vector offsets, std::vector selector_data, size_t part_num, std::vector> expected) + { + auto offset_ptr = std::make_shared(offsets); + IColumn::Selector selector(selector_data.size()); + size_t num_rows = selector_data.size(); + for (size_t i = 0; i < num_rows; i++) + selector[i] = selector_data[i]; + + auto results = scatterOffsetsBySelector(offset_ptr, selector, part_num); + ASSERT_EQ(results.size(), expected.size()); + for (size_t i = 0; i < results.size(); i++) + { + auto result = results[i]->offsets; + auto expect = expected[i]; + ASSERT_EQ(result.size(), expect.size()); + for (size_t j = 0; j < result.size(); j++) + ASSERT_EQ(result[j], expect[j]); + } + }; + + testImpl({5}, {0,1,0,1,0}, 2, {{3},{2}}); + testImpl({5,10}, {0,1,0,1,0,1,0,1,0,1}, 2, {{3,5},{2,5}}); + testImpl({4,8,12}, {0,1,0,1,0,2,0,2,1,2,1,2}, 3, {{2,4},{2,4},{2,4}}); + testImpl({1,2,3,4,5}, {0,1,2,3,4}, 5, {{1},{1},{1},{1},{1}}); + testImpl({3,6,10}, {1,1,1,2,2,2,0,0,0,0}, 3, {{4},{3},{3}}); +} + +} From 58c54226925299e54e67322e72a49e2e9720f304 Mon Sep 17 00:00:00 2001 From: Guillaume Tassery Date: Thu, 24 Nov 2022 10:19:04 +0100 Subject: [PATCH 038/127] cosmetic --- src/Parsers/queryToString.cpp | 5 ----- src/Parsers/queryToString.h | 1 - src/Storages/System/StorageSystemDataSkippingIndices.cpp | 2 +- 3 files changed, 1 insertion(+), 7 deletions(-) diff --git a/src/Parsers/queryToString.cpp b/src/Parsers/queryToString.cpp index 30a38294d43..9721aa1f128 100644 --- a/src/Parsers/queryToString.cpp +++ b/src/Parsers/queryToString.cpp @@ -8,11 +8,6 @@ namespace DB return queryToString(*query); } - String queryToString(const IAST * query) - { - return queryToString(*query); - } - String queryToString(const IAST & query) { return serializeAST(query); diff --git a/src/Parsers/queryToString.h b/src/Parsers/queryToString.h index 7e4dd5da707..873de218293 100644 --- a/src/Parsers/queryToString.h +++ b/src/Parsers/queryToString.h @@ -5,6 +5,5 @@ namespace DB { String queryToString(const ASTPtr & query); - String queryToString(const IAST * query); String queryToString(const IAST & query); } diff --git a/src/Storages/System/StorageSystemDataSkippingIndices.cpp b/src/Storages/System/StorageSystemDataSkippingIndices.cpp index d2d0e69505d..96cd9ab0b32 100644 --- a/src/Storages/System/StorageSystemDataSkippingIndices.cpp +++ b/src/Storages/System/StorageSystemDataSkippingIndices.cpp @@ -128,7 +128,7 @@ protected: if (column_mask[src_index++]) { if (auto * expression = index.definition_ast->as()) - res_columns[res_index++]->insert(queryToString(expression->type)); + res_columns[res_index++]->insert(queryToString(*expression->type)); else res_columns[res_index++]->insertDefault(); } From 4eb97c77a08e29ab4a6ce60019160d8af2420052 Mon Sep 17 00:00:00 2001 From: Han Fei Date: Thu, 24 Nov 2022 16:31:16 +0100 Subject: [PATCH 039/127] introduce new settings "async_insert_max_query_number" --- src/Core/Settings.h | 1 + src/Interpreters/AsynchronousInsertQueue.cpp | 3 ++- src/Interpreters/AsynchronousInsertQueue.h | 1 + 3 files changed, 4 insertions(+), 1 deletion(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index cfc7df6c853..f1eba23abec 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -603,6 +603,7 @@ static constexpr UInt64 operator""_GiB(unsigned long long value) M(Bool, wait_for_async_insert, true, "If true wait for processing of asynchronous insertion", 0) \ M(Seconds, wait_for_async_insert_timeout, DBMS_DEFAULT_LOCK_ACQUIRE_TIMEOUT_SEC, "Timeout for waiting for processing asynchronous insertion", 0) \ M(UInt64, async_insert_max_data_size, 1000000, "Maximum size in bytes of unparsed data collected per query before being inserted", 0) \ + M(UInt64, async_insert_max_query_number, 450, "Maximum number of insert queries before being inserted", 0) \ M(Milliseconds, async_insert_busy_timeout_ms, 200, "Maximum time to wait before dumping collected data per query since the first data appeared", 0) \ M(Milliseconds, async_insert_cleanup_timeout_ms, 1000, "Time to wait before each iteration of cleaning up buffers for INSERT queries which don't appear anymore. Only has meaning at server startup.", 0) \ \ diff --git a/src/Interpreters/AsynchronousInsertQueue.cpp b/src/Interpreters/AsynchronousInsertQueue.cpp index a268cb07b0e..3ea6b2335e4 100644 --- a/src/Interpreters/AsynchronousInsertQueue.cpp +++ b/src/Interpreters/AsynchronousInsertQueue.cpp @@ -248,6 +248,7 @@ void AsynchronousInsertQueue::pushImpl(InsertData::EntryPtr entry, QueueIterator size_t entry_data_size = entry->bytes.size(); + ++data->query_number; data->size += entry_data_size; data->entries.emplace_back(entry); @@ -262,7 +263,7 @@ void AsynchronousInsertQueue::pushImpl(InsertData::EntryPtr entry, QueueIterator /// Here we check whether we hit the limit on maximum data size in the buffer. /// And use setting from query context! /// It works, because queries with the same set of settings are already grouped together. - if (data->size > it->first.settings.async_insert_max_data_size) + if (data->size > it->first.settings.async_insert_max_data_size || data->query_number > it->first.settings.async_insert_max_query_number) scheduleDataProcessingJob(it->first, std::move(data), getContext()); CurrentMetrics::add(CurrentMetrics::PendingAsyncInsert); diff --git a/src/Interpreters/AsynchronousInsertQueue.h b/src/Interpreters/AsynchronousInsertQueue.h index fcf4e3d98d2..b3010d14319 100644 --- a/src/Interpreters/AsynchronousInsertQueue.h +++ b/src/Interpreters/AsynchronousInsertQueue.h @@ -72,6 +72,7 @@ private: std::list entries; size_t size = 0; + size_t query_number = 0; /// Timestamp of the first insert into queue, or after the last queue dump. /// Used to detect for how long the queue is active, so we can dump it by timer. From a2c894bcfdfda83d3f6883a8b0bfaffe1dcba616 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 26 Nov 2022 01:02:29 +0300 Subject: [PATCH 040/127] Just in case --- src/Storages/System/StorageSystemDataSkippingIndices.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/System/StorageSystemDataSkippingIndices.cpp b/src/Storages/System/StorageSystemDataSkippingIndices.cpp index 96cd9ab0b32..2649cf71182 100644 --- a/src/Storages/System/StorageSystemDataSkippingIndices.cpp +++ b/src/Storages/System/StorageSystemDataSkippingIndices.cpp @@ -127,7 +127,7 @@ protected: // 'type_full' column if (column_mask[src_index++]) { - if (auto * expression = index.definition_ast->as()) + if (auto * expression = index.definition_ast->as(); expression && expression->type) res_columns[res_index++]->insert(queryToString(*expression->type)); else res_columns[res_index++]->insertDefault(); From 9aadcefa15685cae14f13d27f1ed4370f94d56eb Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 25 Nov 2022 23:04:42 +0100 Subject: [PATCH 041/127] Update test --- tests/queries/0_stateless/01812_basic_auth_http_server.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/01812_basic_auth_http_server.sh b/tests/queries/0_stateless/01812_basic_auth_http_server.sh index 5f93062ad63..336bb9a0c89 100755 --- a/tests/queries/0_stateless/01812_basic_auth_http_server.sh +++ b/tests/queries/0_stateless/01812_basic_auth_http_server.sh @@ -13,7 +13,7 @@ CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # In this test we do the opposite: passing the invalid credentials while server is accepting default user without a password. # And if the bug exists, they will be ignored (treat as empty credentials) and query succeed. -for i in {3950..4100}; do ${CLICKHOUSE_CURL} --user default:12345 "${CLICKHOUSE_URL}&query=SELECT+1"$(perl -e "print '+'x$i") | grep -v -F 'password' ||:; done +for i in {3950..4100}; do ${CLICKHOUSE_CURL} --user default:12345 "${CLICKHOUSE_URL}&query=SELECT+1"$(perl -e "print '+'x$i") | tr -d '\n' | grep -v -F 'password' ||:; done # You can check that the bug exists in old version by running the old server in Docker: # docker run --network host -it --rm clickhouse/clickhouse-server:1.1.54385 From 114d77d8e8a2bcbb5bd74c346fadf3c35bb66bf8 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 26 Nov 2022 02:42:14 +0100 Subject: [PATCH 042/127] Revert "Update AccessControl.cpp" This reverts commit ae335e9c220f0dc6f1b8532e4ec41c9841df4162. --- src/Access/AccessControl.cpp | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/src/Access/AccessControl.cpp b/src/Access/AccessControl.cpp index 8c8922e2b7e..c72e66f12cf 100644 --- a/src/Access/AccessControl.cpp +++ b/src/Access/AccessControl.cpp @@ -445,8 +445,6 @@ AccessChangesNotifier & AccessControl::getChangesNotifier() UUID AccessControl::authenticate(const Credentials & credentials, const Poco::Net::IPAddress & address) const { - chassert(!credentials.getUserName().empty()); - try { return MultipleAccessStorage::authenticate(credentials, address, *external_authenticators, allow_no_password, @@ -461,7 +459,7 @@ UUID AccessControl::authenticate(const Credentials & credentials, const Poco::Ne /// Better exception message for usability. /// It is typical when users install ClickHouse, type some password and instantly forget it. - if (credentials.getUserName() == "default") + if (credentials.getUserName().empty() || credentials.getUserName() == "default") message << "\n\n" << "If you have installed ClickHouse and forgot password you can reset it in the configuration file.\n" << "The password for default user is typically located at /etc/clickhouse-server/users.d/default-password.xml\n" From 06e6844ec0f82b98e7451ca3c5aed4b5389213a0 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Mon, 28 Nov 2022 18:22:03 +0300 Subject: [PATCH 043/127] Update test.py --- tests/integration/test_mysql_protocol/test.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/integration/test_mysql_protocol/test.py b/tests/integration/test_mysql_protocol/test.py index 02928a24fbd..1bdea0b158c 100644 --- a/tests/integration/test_mysql_protocol/test.py +++ b/tests/integration/test_mysql_protocol/test.py @@ -187,7 +187,7 @@ def test_mysql_client(started_cluster): assert ( stderr.decode() == "mysql: [Warning] Using a password on the command line interface can be insecure.\n" - "ERROR 516 (00000): default: Authentication failed: password is incorrect or there is no user with such name\n" + "ERROR 516 (00000): default: Authentication failed: password is incorrect, or there is no user with such name\n" ) code, (stdout, stderr) = started_cluster.mysql_client_container.exec_run( @@ -587,7 +587,7 @@ def test_python_client(started_cluster): assert exc_info.value.args == ( 516, - "default: Authentication failed: password is incorrect or there is no user with such name", + "default: Authentication failed: password is incorrect, or there is no user with such name", ) client = pymysql.connections.Connection( From cb0727dc87dbea6b7ca444ae3a1a465c9ba97a16 Mon Sep 17 00:00:00 2001 From: Han Fei Date: Mon, 28 Nov 2022 16:47:44 +0100 Subject: [PATCH 044/127] refine test scripts to avoid flaky tests --- .../02481_async_insert_dedup.python | 38 +++++++++++++------ .../02481_async_insert_dedup.reference | 1 - 2 files changed, 27 insertions(+), 12 deletions(-) diff --git a/tests/queries/0_stateless/02481_async_insert_dedup.python b/tests/queries/0_stateless/02481_async_insert_dedup.python index 1eb9e4c5faf..6369fac3747 100644 --- a/tests/queries/0_stateless/02481_async_insert_dedup.python +++ b/tests/queries/0_stateless/02481_async_insert_dedup.python @@ -82,18 +82,34 @@ for i in range(3): gen.join() -time.sleep(5) -result = client.query("select count(*) from t_async_insert_dedup") -print(result, flush=True) -result = client.query("select KeyID from t_async_insert_dedup order by KeyID") -result = result.split() -for i in range(total_number): - expect = str(i+1) - real = result[i] - if expect != real: - print("error, {} is not equal to {} for {}-th elements".format(real, expect, i), flush=True) - break +#result = client.query("select count(*) from t_async_insert_dedup") +#print(result, flush=True) + +retry = 0 + +while (True): + time.sleep(5) + result = client.query("select KeyID from t_async_insert_dedup order by KeyID") + result = result.split() + err = False + errMsg = "" + for i in range(total_number): + expect = str(i+1) + real = result[i] + if expect != real: + err = True + errMsg = "error, {} is not equal to {} for {}-th elements, total rows is {}".format(real, expect, i, len(result)) + break + # retry several times to get stable results. + if err and retry >= 5: + print (errMsg, flush=True) + elif err: + retry += 1 + continue + else: + print(len(result), flush=True) + break client.query("DROP TABLE IF EXISTS t_async_insert_dedup NO DELAY") os._exit(os.EX_OK) diff --git a/tests/queries/0_stateless/02481_async_insert_dedup.reference b/tests/queries/0_stateless/02481_async_insert_dedup.reference index 789c0de7a81..5caff40c4a0 100644 --- a/tests/queries/0_stateless/02481_async_insert_dedup.reference +++ b/tests/queries/0_stateless/02481_async_insert_dedup.reference @@ -1,2 +1 @@ 10000 - From ec3e46b471f18d1542a85e16d935517e0b856c11 Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Mon, 28 Nov 2022 16:07:48 +0000 Subject: [PATCH 045/127] Fix included_elements calculation in AggregateFunctionNullVariadic --- src/AggregateFunctions/AggregateFunctionNull.h | 2 +- .../0_stateless/02417_null_variadic_behaviour.reference | 6 ++++++ tests/queries/0_stateless/02417_null_variadic_behaviour.sql | 6 ++++++ 3 files changed, 13 insertions(+), 1 deletion(-) diff --git a/src/AggregateFunctions/AggregateFunctionNull.h b/src/AggregateFunctions/AggregateFunctionNull.h index deed06b8bf2..b190ff78bc2 100644 --- a/src/AggregateFunctions/AggregateFunctionNull.h +++ b/src/AggregateFunctions/AggregateFunctionNull.h @@ -472,7 +472,7 @@ public: final_flags = std::make_unique(row_end); final_flags_ptr = final_flags.get(); - bool included_elements = 0; + size_t included_elements = 0; const auto & flags = assert_cast(*columns[if_argument_pos]).getData(); for (size_t i = row_begin; i < row_end; i++) { diff --git a/tests/queries/0_stateless/02417_null_variadic_behaviour.reference b/tests/queries/0_stateless/02417_null_variadic_behaviour.reference index bedb69f99b0..e37fa546491 100644 --- a/tests/queries/0_stateless/02417_null_variadic_behaviour.reference +++ b/tests/queries/0_stateless/02417_null_variadic_behaviour.reference @@ -21,6 +21,8 @@ SELECT avgWeighted(if(number < 10000, NULL, number), toNullable(number)) t, toTy \N Nullable(Float64) SELECT avgWeighted(if(number < 50, NULL, number), toNullable(number)) t, toTypeName(t) FROM numbers(100); 77.29530201342281 Nullable(Float64) +SELECT avgWeighted(toNullable(number), if(number < 500, NULL, number)) t, toTypeName(t) FROM numbers(1000); +777.2961974649767 Nullable(Float64) SELECT avgWeighted(if(number < 10000, NULL, number), if(number < 10000, NULL, number)) t, toTypeName(t) FROM numbers(100); \N Nullable(Float64) SELECT avgWeighted(if(number < 50, NULL, number), if(number < 10000, NULL, number)) t, toTypeName(t) FROM numbers(100); @@ -29,6 +31,8 @@ SELECT avgWeighted(if(number < 10000, NULL, number), if(number < 50, NULL, numbe \N Nullable(Float64) SELECT avgWeighted(if(number < 50, NULL, number), if(number < 50, NULL, number)) t, toTypeName(t) FROM numbers(100); 77.29530201342281 Nullable(Float64) +SELECT avgWeighted(if(number < 10000, NULL, number), if(number < 500, NULL, number)) t, toTypeName(t) FROM numbers(1000); +\N Nullable(Float64) SELECT avgWeightedIf(number, number, number % 10) t, toTypeName(t) FROM numbers(100); 66.63333333333334 Float64 SELECT avgWeightedIf(number, number, toNullable(number % 10)) t, toTypeName(t) FROM numbers(100); @@ -39,6 +43,8 @@ SELECT avgWeightedIf(number, number, if(number < 50, NULL, number % 10)) t, toTy 77.75555555555556 Float64 SELECT avgWeightedIf(number, number, if(number < 0, NULL, number % 10)) t, toTypeName(t) FROM numbers(100); 66.63333333333334 Float64 +SELECT avgWeightedIf(number, number, toNullable(number % 10)) t, toTypeName(t) FROM numbers(1000); +666.6633333333333 Float64 SELECT avgWeightedIf(if(number < 10000, NULL, number), if(number < 10000, NULL, number), if(number < 10000, NULL, number % 10)) t, toTypeName(t) FROM numbers(100); \N Nullable(Float64) SELECT avgWeightedIf(if(number < 50, NULL, number), if(number < 10000, NULL, number), if(number < 10000, NULL, number % 10)) t, toTypeName(t) FROM numbers(100); diff --git a/tests/queries/0_stateless/02417_null_variadic_behaviour.sql b/tests/queries/0_stateless/02417_null_variadic_behaviour.sql index 566cf27bb90..00c07ede0b5 100644 --- a/tests/queries/0_stateless/02417_null_variadic_behaviour.sql +++ b/tests/queries/0_stateless/02417_null_variadic_behaviour.sql @@ -14,17 +14,23 @@ SELECT avgWeighted(toNullable(number), if(number < 50, NULL, number)) t, toTypeN SELECT avgWeighted(if(number < 10000, NULL, number), toNullable(number)) t, toTypeName(t) FROM numbers(100); SELECT avgWeighted(if(number < 50, NULL, number), toNullable(number)) t, toTypeName(t) FROM numbers(100); +SELECT avgWeighted(toNullable(number), if(number < 500, NULL, number)) t, toTypeName(t) FROM numbers(1000); + SELECT avgWeighted(if(number < 10000, NULL, number), if(number < 10000, NULL, number)) t, toTypeName(t) FROM numbers(100); SELECT avgWeighted(if(number < 50, NULL, number), if(number < 10000, NULL, number)) t, toTypeName(t) FROM numbers(100); SELECT avgWeighted(if(number < 10000, NULL, number), if(number < 50, NULL, number)) t, toTypeName(t) FROM numbers(100); SELECT avgWeighted(if(number < 50, NULL, number), if(number < 50, NULL, number)) t, toTypeName(t) FROM numbers(100); +SELECT avgWeighted(if(number < 10000, NULL, number), if(number < 500, NULL, number)) t, toTypeName(t) FROM numbers(1000); + SELECT avgWeightedIf(number, number, number % 10) t, toTypeName(t) FROM numbers(100); SELECT avgWeightedIf(number, number, toNullable(number % 10)) t, toTypeName(t) FROM numbers(100); SELECT avgWeightedIf(number, number, if(number < 10000, NULL, number % 10)) t, toTypeName(t) FROM numbers(100); SELECT avgWeightedIf(number, number, if(number < 50, NULL, number % 10)) t, toTypeName(t) FROM numbers(100); SELECT avgWeightedIf(number, number, if(number < 0, NULL, number % 10)) t, toTypeName(t) FROM numbers(100); +SELECT avgWeightedIf(number, number, toNullable(number % 10)) t, toTypeName(t) FROM numbers(1000); + SELECT avgWeightedIf(if(number < 10000, NULL, number), if(number < 10000, NULL, number), if(number < 10000, NULL, number % 10)) t, toTypeName(t) FROM numbers(100); SELECT avgWeightedIf(if(number < 50, NULL, number), if(number < 10000, NULL, number), if(number < 10000, NULL, number % 10)) t, toTypeName(t) FROM numbers(100); SELECT avgWeightedIf(if(number < 10000, NULL, number), if(number < 50, NULL, number), if(number < 10000, NULL, number % 10)) t, toTypeName(t) FROM numbers(100); From 12a7d64ef5932d10408e6a90d3f91b8e600776b6 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Tue, 29 Nov 2022 17:33:02 +0300 Subject: [PATCH 046/127] Update test.py --- tests/integration/test_mysql_protocol/test.py | 11 ++++------- 1 file changed, 4 insertions(+), 7 deletions(-) diff --git a/tests/integration/test_mysql_protocol/test.py b/tests/integration/test_mysql_protocol/test.py index 1bdea0b158c..bce81170c35 100644 --- a/tests/integration/test_mysql_protocol/test.py +++ b/tests/integration/test_mysql_protocol/test.py @@ -185,9 +185,8 @@ def test_mysql_client(started_cluster): ) assert ( - stderr.decode() - == "mysql: [Warning] Using a password on the command line interface can be insecure.\n" - "ERROR 516 (00000): default: Authentication failed: password is incorrect, or there is no user with such name\n" + "mysql: [Warning] Using a password on the command line interface can be insecure.\n" + "ERROR 516 (00000): default: Authentication failed: password is incorrect, or there is no user with such name" in stderr.decode() ) code, (stdout, stderr) = started_cluster.mysql_client_container.exec_run( @@ -585,10 +584,8 @@ def test_python_client(started_cluster): port=server_port, ) - assert exc_info.value.args == ( - 516, - "default: Authentication failed: password is incorrect, or there is no user with such name", - ) + assert exc_info.value.args[0] == 516 + assert "default: Authentication failed: password is incorrect, or there is no user with such name" in exc_info.value.args[1] client = pymysql.connections.Connection( host=started_cluster.get_instance_ip("node"), From ad66e01ace23d166ae197343954200db8ac79b75 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Tue, 29 Nov 2022 14:48:05 +0000 Subject: [PATCH 047/127] Automatic style fix --- tests/integration/test_mysql_protocol/test.py | 8 ++++++-- 1 file changed, 6 insertions(+), 2 deletions(-) diff --git a/tests/integration/test_mysql_protocol/test.py b/tests/integration/test_mysql_protocol/test.py index bce81170c35..2cfb1d41ce0 100644 --- a/tests/integration/test_mysql_protocol/test.py +++ b/tests/integration/test_mysql_protocol/test.py @@ -186,7 +186,8 @@ def test_mysql_client(started_cluster): assert ( "mysql: [Warning] Using a password on the command line interface can be insecure.\n" - "ERROR 516 (00000): default: Authentication failed: password is incorrect, or there is no user with such name" in stderr.decode() + "ERROR 516 (00000): default: Authentication failed: password is incorrect, or there is no user with such name" + in stderr.decode() ) code, (stdout, stderr) = started_cluster.mysql_client_container.exec_run( @@ -585,7 +586,10 @@ def test_python_client(started_cluster): ) assert exc_info.value.args[0] == 516 - assert "default: Authentication failed: password is incorrect, or there is no user with such name" in exc_info.value.args[1] + assert ( + "default: Authentication failed: password is incorrect, or there is no user with such name" + in exc_info.value.args[1] + ) client = pymysql.connections.Connection( host=started_cluster.get_instance_ip("node"), From 564279be83e84f9debbfa217512202b9df3df18b Mon Sep 17 00:00:00 2001 From: Han Fei Date: Wed, 30 Nov 2022 12:00:09 +0100 Subject: [PATCH 048/127] fix integration tests --- src/Storages/MergeTree/MergeTreeDataWriter.cpp | 1 + .../02481_async_insert_dedup.python | 18 ++++++++++++++---- .../02481_async_insert_dedup.reference | 2 ++ 3 files changed, 17 insertions(+), 4 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeDataWriter.cpp b/src/Storages/MergeTree/MergeTreeDataWriter.cpp index 69128f23c85..c50c01ea356 100644 --- a/src/Storages/MergeTree/MergeTreeDataWriter.cpp +++ b/src/Storages/MergeTree/MergeTreeDataWriter.cpp @@ -193,6 +193,7 @@ BlocksWithPartition MergeTreeDataWriter::splitBlockIntoParts( if (!metadata_snapshot->hasPartitionKey()) /// Table is not partitioned. { result.emplace_back(Block(block), Row{}); + result[0].offsets = chunk_offsets; return result; } diff --git a/tests/queries/0_stateless/02481_async_insert_dedup.python b/tests/queries/0_stateless/02481_async_insert_dedup.python index 6369fac3747..fac031434b4 100644 --- a/tests/queries/0_stateless/02481_async_insert_dedup.python +++ b/tests/queries/0_stateless/02481_async_insert_dedup.python @@ -14,6 +14,20 @@ from pure_http_client import ClickHouseClient client = ClickHouseClient() +# test table without partition +client.query("DROP TABLE IF EXISTS t_async_insert_dedup_no_part NO DELAY") +client.query(''' +CREATE TABLE t_async_insert_dedup_no_part ( +KeyID UInt32 +) Engine = ReplicatedMergeTree('/clickhouse/tables/{shard}/{database}/t_async_insert_dedup', '{replica}') +ORDER BY (KeyID) +''') + +client.query("insert into t_async_insert_dedup_no_part values (1), (2), (3), (4), (5)", settings = {"async_insert": 1, "wait_for_async_insert": 1, "insert_keeper_fault_injection_probability": 0}) +result = client.query("select count(*) from t_async_insert_dedup_no_part") +print(result, flush=True) +client.query("DROP TABLE IF EXISTS t_async_insert_dedup_no_part NO DELAY") + # generate data and push to queue def generate_data(q, total_number): old_data = [] @@ -82,10 +96,6 @@ for i in range(3): gen.join() - -#result = client.query("select count(*) from t_async_insert_dedup") -#print(result, flush=True) - retry = 0 while (True): diff --git a/tests/queries/0_stateless/02481_async_insert_dedup.reference b/tests/queries/0_stateless/02481_async_insert_dedup.reference index 5caff40c4a0..a91c59a7fc5 100644 --- a/tests/queries/0_stateless/02481_async_insert_dedup.reference +++ b/tests/queries/0_stateless/02481_async_insert_dedup.reference @@ -1 +1,3 @@ +5 + 10000 From 53e961e7c188a0093a4362fc96e15fc3f4a6491a Mon Sep 17 00:00:00 2001 From: Han Fei Date: Wed, 30 Nov 2022 19:09:57 +0100 Subject: [PATCH 049/127] fix build check --- src/Storages/MergeTree/tests/gtest_async_inserts.cpp | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/src/Storages/MergeTree/tests/gtest_async_inserts.cpp b/src/Storages/MergeTree/tests/gtest_async_inserts.cpp index 55d388c1165..348da5f2ccd 100644 --- a/src/Storages/MergeTree/tests/gtest_async_inserts.cpp +++ b/src/Storages/MergeTree/tests/gtest_async_inserts.cpp @@ -15,7 +15,7 @@ class AsyncInsertsTest : public ::testing::TestPartResult TEST(AsyncInsertsTest, testScatterOffsetsBySelector) { - auto testImpl = [](std::vector offsets, std::vector selector_data, size_t part_num, std::vector> expected) + auto test_impl = [](std::vector offsets, std::vector selector_data, size_t part_num, std::vector> expected) { auto offset_ptr = std::make_shared(offsets); IColumn::Selector selector(selector_data.size()); @@ -35,11 +35,11 @@ TEST(AsyncInsertsTest, testScatterOffsetsBySelector) } }; - testImpl({5}, {0,1,0,1,0}, 2, {{3},{2}}); - testImpl({5,10}, {0,1,0,1,0,1,0,1,0,1}, 2, {{3,5},{2,5}}); - testImpl({4,8,12}, {0,1,0,1,0,2,0,2,1,2,1,2}, 3, {{2,4},{2,4},{2,4}}); - testImpl({1,2,3,4,5}, {0,1,2,3,4}, 5, {{1},{1},{1},{1},{1}}); - testImpl({3,6,10}, {1,1,1,2,2,2,0,0,0,0}, 3, {{4},{3},{3}}); + test_impl({5}, {0,1,0,1,0}, 2, {{3},{2}}); + test_impl({5,10}, {0,1,0,1,0,1,0,1,0,1}, 2, {{3,5},{2,5}}); + test_impl({4,8,12}, {0,1,0,1,0,2,0,2,1,2,1,2}, 3, {{2,4},{2,4},{2,4}}); + test_impl({1,2,3,4,5}, {0,1,2,3,4}, 5, {{1},{1},{1},{1},{1}}); + test_impl({3,6,10}, {1,1,1,2,2,2,0,0,0,0}, 3, {{4},{3},{3}}); } } From 518058b92e33d25c5f415a31d48375e993188343 Mon Sep 17 00:00:00 2001 From: Duc Canh Le Date: Fri, 2 Dec 2022 17:56:23 +0800 Subject: [PATCH 050/127] fix some bad logics in ast level optimization --- src/Analyzer/Passes/NormalizeCountVariantsPass.cpp | 13 +++++++++---- src/Analyzer/Passes/SumIfToCountIfPass.cpp | 2 +- src/Interpreters/RewriteCountVariantsVisitor.cpp | 3 ++- src/Interpreters/RewriteCountVariantsVisitor.h | 8 ++++++-- src/Interpreters/TreeOptimizer.cpp | 6 +++--- src/Interpreters/TreeOptimizer.h | 2 +- src/Interpreters/TreeRewriter.cpp | 4 ++-- ...ing_aggregate_functions_null_for_empty.reference | 3 +++ ...8_setting_aggregate_functions_null_for_empty.sql | 3 +++ ...1706_optimize_normalize_count_variants.reference | 3 +++ .../01706_optimize_normalize_count_variants.sql | 4 ++++ 11 files changed, 37 insertions(+), 14 deletions(-) diff --git a/src/Analyzer/Passes/NormalizeCountVariantsPass.cpp b/src/Analyzer/Passes/NormalizeCountVariantsPass.cpp index b91fc94e6cd..1fb31ba1dd0 100644 --- a/src/Analyzer/Passes/NormalizeCountVariantsPass.cpp +++ b/src/Analyzer/Passes/NormalizeCountVariantsPass.cpp @@ -5,6 +5,7 @@ #include #include +#include namespace DB { @@ -15,7 +16,8 @@ namespace class NormalizeCountVariantsVisitor : public InDepthQueryTreeVisitor { public: - static void visitImpl(QueryTreeNodePtr & node) + explicit NormalizeCountVariantsVisitor(ContextPtr context_) : context(std::move(context_)) {} + void visitImpl(QueryTreeNodePtr & node) { auto * function_node = node->as(); if (!function_node || !function_node->isAggregateFunction() || (function_node->getFunctionName() != "count" && function_node->getFunctionName() != "sum")) @@ -38,13 +40,16 @@ public: } else if (function_node->getFunctionName() == "sum" && first_argument_constant_literal.getType() == Field::Types::UInt64 && - first_argument_constant_literal.get() == 1) + first_argument_constant_literal.get() == 1 && + !context->getSettingsRef().aggregate_functions_null_for_empty) { resolveAsCountAggregateFunction(*function_node); function_node->getArguments().getNodes().clear(); } } private: + ContextPtr context; + static inline void resolveAsCountAggregateFunction(FunctionNode & function_node) { auto function_result_type = function_node.getResultType(); @@ -58,9 +63,9 @@ private: } -void NormalizeCountVariantsPass::run(QueryTreeNodePtr query_tree_node, ContextPtr) +void NormalizeCountVariantsPass::run(QueryTreeNodePtr query_tree_node, ContextPtr context) { - NormalizeCountVariantsVisitor visitor; + NormalizeCountVariantsVisitor visitor(context); visitor.visit(query_tree_node); } diff --git a/src/Analyzer/Passes/SumIfToCountIfPass.cpp b/src/Analyzer/Passes/SumIfToCountIfPass.cpp index f43c90e10eb..56667e6cbfe 100644 --- a/src/Analyzer/Passes/SumIfToCountIfPass.cpp +++ b/src/Analyzer/Passes/SumIfToCountIfPass.cpp @@ -55,7 +55,7 @@ public: if (!isInt64OrUInt64FieldType(constant_value_literal.getType())) return; - if (constant_value_literal.get() != 1) + if (constant_value_literal.get() != 1 || context->getSettingsRef().aggregate_functions_null_for_empty) return; function_node_arguments_nodes[0] = std::move(function_node_arguments_nodes[1]); diff --git a/src/Interpreters/RewriteCountVariantsVisitor.cpp b/src/Interpreters/RewriteCountVariantsVisitor.cpp index 741dc3e8cb7..f207bc51527 100644 --- a/src/Interpreters/RewriteCountVariantsVisitor.cpp +++ b/src/Interpreters/RewriteCountVariantsVisitor.cpp @@ -6,6 +6,7 @@ #include #include #include +#include namespace DB @@ -52,7 +53,7 @@ void RewriteCountVariantsVisitor::visit(ASTFunction & func) if (first_arg_literal->value.getType() == Field::Types::UInt64) { auto constant = first_arg_literal->value.get(); - if (constant == 1) + if (constant == 1 && !context->getSettingsRef().aggregate_functions_null_for_empty) transform = true; } } diff --git a/src/Interpreters/RewriteCountVariantsVisitor.h b/src/Interpreters/RewriteCountVariantsVisitor.h index 6f731c8c463..36c026bdfd7 100644 --- a/src/Interpreters/RewriteCountVariantsVisitor.h +++ b/src/Interpreters/RewriteCountVariantsVisitor.h @@ -1,6 +1,7 @@ #pragma once #include +#include namespace DB { @@ -10,8 +11,11 @@ class ASTFunction; class RewriteCountVariantsVisitor { public: - static void visit(ASTPtr &); - static void visit(ASTFunction &); + explicit RewriteCountVariantsVisitor(ContextPtr context_) : context(context_) {} + void visit(ASTPtr &); + void visit(ASTFunction &); +private: + ContextPtr context; }; } diff --git a/src/Interpreters/TreeOptimizer.cpp b/src/Interpreters/TreeOptimizer.cpp index e4301bad1e8..bac70106f5f 100644 --- a/src/Interpreters/TreeOptimizer.cpp +++ b/src/Interpreters/TreeOptimizer.cpp @@ -788,9 +788,9 @@ void TreeOptimizer::optimizeIf(ASTPtr & query, Aliases & aliases, bool if_chain_ OptimizeIfChainsVisitor().visit(query); } -void TreeOptimizer::optimizeCountConstantAndSumOne(ASTPtr & query) +void TreeOptimizer::optimizeCountConstantAndSumOne(ASTPtr & query, ContextPtr context) { - RewriteCountVariantsVisitor::visit(query); + RewriteCountVariantsVisitor(context).visit(query); } void TreeOptimizer::apply(ASTPtr & query, TreeRewriterResult & result, @@ -835,7 +835,7 @@ void TreeOptimizer::apply(ASTPtr & query, TreeRewriterResult & result, optimizeAnyFunctions(query); if (settings.optimize_normalize_count_variants) - optimizeCountConstantAndSumOne(query); + optimizeCountConstantAndSumOne(query, context); if (settings.optimize_multiif_to_if) optimizeMultiIfToIf(query); diff --git a/src/Interpreters/TreeOptimizer.h b/src/Interpreters/TreeOptimizer.h index ced185373cc..c6130fdc5f3 100644 --- a/src/Interpreters/TreeOptimizer.h +++ b/src/Interpreters/TreeOptimizer.h @@ -24,7 +24,7 @@ public: ContextPtr context); static void optimizeIf(ASTPtr & query, Aliases & aliases, bool if_chain_to_multiif); - static void optimizeCountConstantAndSumOne(ASTPtr & query); + static void optimizeCountConstantAndSumOne(ASTPtr & query, ContextPtr context); }; } diff --git a/src/Interpreters/TreeRewriter.cpp b/src/Interpreters/TreeRewriter.cpp index 30fab527ac5..eb5d2ba7645 100644 --- a/src/Interpreters/TreeRewriter.cpp +++ b/src/Interpreters/TreeRewriter.cpp @@ -147,7 +147,7 @@ struct CustomizeAggregateFunctionsSuffixData void visit(ASTFunction & func, ASTPtr &) const { const auto & instance = AggregateFunctionFactory::instance(); - if (instance.isAggregateFunctionName(func.name) && !endsWith(func.name, customized_func_suffix)) + if (instance.isAggregateFunctionName(func.name) && !endsWith(func.name, customized_func_suffix) && !endsWith(func.name, customized_func_suffix + "If")) { auto properties = instance.tryGetProperties(func.name); if (properties && !properties->returns_default_when_only_null) @@ -1292,7 +1292,7 @@ TreeRewriterResultPtr TreeRewriter::analyzeSelect( /// Perform it before analyzing JOINs, because it may change number of columns with names unique and break some logic inside JOINs if (settings.optimize_normalize_count_variants) - TreeOptimizer::optimizeCountConstantAndSumOne(query); + TreeOptimizer::optimizeCountConstantAndSumOne(query, getContext()); if (tables_with_columns.size() > 1) { diff --git a/tests/queries/0_stateless/01528_setting_aggregate_functions_null_for_empty.reference b/tests/queries/0_stateless/01528_setting_aggregate_functions_null_for_empty.reference index 9c6ae9c65ab..8b1aa83d73c 100644 --- a/tests/queries/0_stateless/01528_setting_aggregate_functions_null_for_empty.reference +++ b/tests/queries/0_stateless/01528_setting_aggregate_functions_null_for_empty.reference @@ -14,3 +14,6 @@ 45 10 10 +SELECT sumOrNullIf(1, number > 0) +FROM numbers(10) +WHERE 0 diff --git a/tests/queries/0_stateless/01528_setting_aggregate_functions_null_for_empty.sql b/tests/queries/0_stateless/01528_setting_aggregate_functions_null_for_empty.sql index e76ce667bbc..b57a492e375 100644 --- a/tests/queries/0_stateless/01528_setting_aggregate_functions_null_for_empty.sql +++ b/tests/queries/0_stateless/01528_setting_aggregate_functions_null_for_empty.sql @@ -33,4 +33,7 @@ SELECT sumOrNull(n) FROM defaults; SELECT count(n) FROM defaults; SELECT countOrNull(n) FROM defaults; + +EXPLAIN SYNTAX SELECT sumIf(1, number > 0) FROM numbers(10) WHERE 0; + DROP TABLE defaults; diff --git a/tests/queries/0_stateless/01706_optimize_normalize_count_variants.reference b/tests/queries/0_stateless/01706_optimize_normalize_count_variants.reference index 0343ad84abb..3080226da32 100644 --- a/tests/queries/0_stateless/01706_optimize_normalize_count_variants.reference +++ b/tests/queries/0_stateless/01706_optimize_normalize_count_variants.reference @@ -4,3 +4,6 @@ SELECT count(), count(), count(NULL) +SELECT sumOrNull(1) +FROM numbers(10) +WHERE 0 diff --git a/tests/queries/0_stateless/01706_optimize_normalize_count_variants.sql b/tests/queries/0_stateless/01706_optimize_normalize_count_variants.sql index d20f23feef8..9c85d6bc2fd 100644 --- a/tests/queries/0_stateless/01706_optimize_normalize_count_variants.sql +++ b/tests/queries/0_stateless/01706_optimize_normalize_count_variants.sql @@ -2,3 +2,7 @@ set optimize_normalize_count_variants = 1; explain syntax select count(), count(1), count(-1), sum(1), count(null); + +set aggregate_functions_null_for_empty = 1; + +explain syntax select sum(1) from numbers(10) where 0; From 006af1b6a43d90c7dbb0f32fce0704a42bb27d7e Mon Sep 17 00:00:00 2001 From: serxa Date: Fri, 2 Dec 2022 19:30:14 +0000 Subject: [PATCH 051/127] constraints on merge tree settings --- .../settings/constraints-on-settings.md | 17 ++++ src/Access/AccessControl.cpp | 7 ++ src/Access/ContextAccess.h | 1 - src/Access/SettingsConstraints.cpp | 77 ++++++++++++++++++- src/Access/SettingsConstraints.h | 7 ++ src/Interpreters/Context.cpp | 5 ++ src/Interpreters/Context.h | 1 + src/Storages/MergeTree/MergeTreeData.cpp | 7 ++ 8 files changed, 120 insertions(+), 2 deletions(-) diff --git a/docs/en/operations/settings/constraints-on-settings.md b/docs/en/operations/settings/constraints-on-settings.md index 651b6465f7e..bb015f80834 100644 --- a/docs/en/operations/settings/constraints-on-settings.md +++ b/docs/en/operations/settings/constraints-on-settings.md @@ -91,4 +91,21 @@ Code: 452, e.displayText() = DB::Exception: Setting force_index_by_date should n **Note:** the `default` profile has special handling: all the constraints defined for the `default` profile become the default constraints, so they restrict all the users until they’re overridden explicitly for these users. +## Constraints on Merge Tree Settings +It is possible to set constraints for [merge tree settings](merge-tree-settings.md). There constraints are applied when table with merge tree engine is created or its storage settings are altered. Name of merge tree setting must be prepended by `merge_tree_` prefix when referenced in `` section. + +**Example:** Forbid to create new tables with explicitly specified `storage_policy` + +``` xml + + + + + + + + + +``` + [Original article](https://clickhouse.com/docs/en/operations/settings/constraints_on_settings/) diff --git a/src/Access/AccessControl.cpp b/src/Access/AccessControl.cpp index 04642df6f40..87333f99423 100644 --- a/src/Access/AccessControl.cpp +++ b/src/Access/AccessControl.cpp @@ -19,6 +19,7 @@ #include #include #include +#include #include #include #include @@ -38,6 +39,10 @@ namespace ErrorCodes extern const int AUTHENTICATION_FAILED; } +namespace +{ + constexpr const char MERGE_TREE_SETTINGS_PREFIX[] = "merge_tree_"; +} namespace { @@ -105,6 +110,8 @@ public: { if (Settings::hasBuiltin(setting_name)) return true; + if (setting_name.starts_with(MERGE_TREE_SETTINGS_PREFIX) && MergeTreeSettings::hasBuiltin(setting_name.substr(strlen(MERGE_TREE_SETTINGS_PREFIX)))) + return true; std::lock_guard lock{mutex}; for (const auto & prefix : registered_prefixes) diff --git a/src/Access/ContextAccess.h b/src/Access/ContextAccess.h index 84ef0ab722d..63604a03b4e 100644 --- a/src/Access/ContextAccess.h +++ b/src/Access/ContextAccess.h @@ -174,7 +174,6 @@ private: void initialize(); void setUser(const UserPtr & user_) const; void setRolesInfo(const std::shared_ptr & roles_info_) const; - void setSettingsAndConstraints() const; void calculateAccessRights() const; template diff --git a/src/Access/SettingsConstraints.cpp b/src/Access/SettingsConstraints.cpp index 0317e43f8d1..e0efb85b707 100644 --- a/src/Access/SettingsConstraints.cpp +++ b/src/Access/SettingsConstraints.cpp @@ -1,6 +1,7 @@ #include #include #include +#include #include #include #include @@ -18,6 +19,11 @@ namespace ErrorCodes extern const int UNKNOWN_SETTING; } +namespace +{ + constexpr const char MERGE_TREE_SETTINGS_PREFIX[] = "merge_tree_"; +} + SettingsConstraints::SettingsConstraints(const AccessControl & access_control_) : access_control(&access_control_) { @@ -97,6 +103,17 @@ void SettingsConstraints::check(const Settings & current_settings, SettingsChang }); } +void SettingsConstraints::check(const MergeTreeSettings & current_settings, const SettingChange & change) const +{ + checkImpl(current_settings, const_cast(change), THROW_ON_VIOLATION); +} + +void SettingsConstraints::check(const MergeTreeSettings & current_settings, const SettingsChanges & changes) const +{ + for (const auto & change : changes) + check(current_settings, change); +} + void SettingsConstraints::clamp(const Settings & current_settings, SettingsChanges & changes) const { boost::range::remove_erase_if( @@ -107,7 +124,6 @@ void SettingsConstraints::clamp(const Settings & current_settings, SettingsChang }); } - bool SettingsConstraints::checkImpl(const Settings & current_settings, SettingChange & change, ReactionOnViolation reaction) const { const String & setting_name = change.name; @@ -177,6 +193,57 @@ bool SettingsConstraints::checkImpl(const Settings & current_settings, SettingCh return getChecker(current_settings, setting_name).check(change, new_value, reaction); } +bool SettingsConstraints::checkImpl(const MergeTreeSettings & current_settings, SettingChange & change, ReactionOnViolation reaction) const +{ + const String & setting_name = change.name; + String prefixed_name = MERGE_TREE_SETTINGS_PREFIX + setting_name; + + bool cannot_cast; + auto cast_value = [&](const Field & x) -> Field + { + cannot_cast = false; + if (reaction == THROW_ON_VIOLATION) + return MergeTreeSettings::castValueUtil(setting_name, x); + else + { + try + { + return MergeTreeSettings::castValueUtil(setting_name, x); + } + catch (...) + { + cannot_cast = true; + return {}; + } + } + }; + + if (reaction == THROW_ON_VIOLATION) + access_control->checkSettingNameIsAllowed(prefixed_name); + else if (!access_control->isSettingNameAllowed(prefixed_name)) + return false; + + Field current_value, new_value; + if (current_settings.tryGet(setting_name, current_value)) + { + /// Setting isn't checked if value has not changed. + if (change.value == current_value) + return false; + + new_value = cast_value(change.value); + if ((new_value == current_value) || cannot_cast) + return false; + } + else + { + new_value = cast_value(change.value); + if (cannot_cast) + return false; + } + + return getMergeTreeChecker(prefixed_name).check(change, new_value, reaction); +} + bool SettingsConstraints::Checker::check(SettingChange & change, const Field & new_value, ReactionOnViolation reaction) const { const String & setting_name = change.name; @@ -280,6 +347,14 @@ SettingsConstraints::Checker SettingsConstraints::getChecker(const Settings & cu return Checker(it->second); } +SettingsConstraints::Checker SettingsConstraints::getMergeTreeChecker(std::string_view setting_name) const +{ + auto it = constraints.find(setting_name); + if (it == constraints.end()) + return Checker(); // Allowed + return Checker(it->second); +} + bool SettingsConstraints::Constraint::operator==(const Constraint & other) const { return writability == other.writability && min_value == other.min_value && max_value == other.max_value; diff --git a/src/Access/SettingsConstraints.h b/src/Access/SettingsConstraints.h index 822bf42861b..4234662055f 100644 --- a/src/Access/SettingsConstraints.h +++ b/src/Access/SettingsConstraints.h @@ -12,6 +12,7 @@ namespace Poco::Util namespace DB { struct Settings; +struct MergeTreeSettings; struct SettingChange; class SettingsChanges; class AccessControl; @@ -75,6 +76,10 @@ public: void check(const Settings & current_settings, const SettingsChanges & changes) const; void check(const Settings & current_settings, SettingsChanges & changes) const; + /// Checks whether `change` with added "merge_tree_" name prefix violates these constraints and throws an exception if so. + void check(const MergeTreeSettings & current_settings, const SettingChange & change) const; + void check(const MergeTreeSettings & current_settings, const SettingsChanges & changes) const; + /// Checks whether `change` violates these and clamps the `change` if so. void clamp(const Settings & current_settings, SettingsChanges & changes) const; @@ -137,8 +142,10 @@ private: }; bool checkImpl(const Settings & current_settings, SettingChange & change, ReactionOnViolation reaction) const; + bool checkImpl(const MergeTreeSettings & current_settings, SettingChange & change, ReactionOnViolation reaction) const; Checker getChecker(const Settings & current_settings, std::string_view setting_name) const; + Checker getMergeTreeChecker(std::string_view setting_name) const; // Special container for heterogeneous lookups: to avoid `String` construction during `find(std::string_view)` using Constraints = std::unordered_map>; diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index 913b0535358..25543165e66 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -1431,6 +1431,11 @@ void Context::clampToSettingsConstraints(SettingsChanges & changes) const getSettingsConstraintsAndCurrentProfiles()->constraints.clamp(settings, changes); } +void Context::checkMergeTreeSettingsConstraints(const MergeTreeSettings & merge_tree_settings, const SettingsChanges & changes) const +{ + getSettingsConstraintsAndCurrentProfiles()->constraints.check(merge_tree_settings, changes); +} + void Context::resetSettingsToDefaultValue(const std::vector & names) { auto lock = getLock(); diff --git a/src/Interpreters/Context.h b/src/Interpreters/Context.h index 63f321db993..ba972baef80 100644 --- a/src/Interpreters/Context.h +++ b/src/Interpreters/Context.h @@ -660,6 +660,7 @@ public: void checkSettingsConstraints(const SettingsChanges & changes) const; void checkSettingsConstraints(SettingsChanges & changes) const; void clampToSettingsConstraints(SettingsChanges & changes) const; + void checkMergeTreeSettingsConstraints(const MergeTreeSettings & merge_tree_settings, const SettingsChanges & changes) const; /// Reset settings to default value void resetSettingsToDefaultValue(const std::vector & names); diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 6e14fec22af..002243786db 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -246,7 +246,11 @@ MergeTreeData::MergeTreeData( /// Check sanity of MergeTreeSettings. Only when table is created. if (!attach) + { + const auto & changes = metadata_.settings_changes->as().changes; + getContext()->checkMergeTreeSettingsConstraints(getContext()->getMergeTreeSettings(), changes); settings->sanityCheck(getContext()->getMergeMutateExecutor()->getMaxTasksCount()); + } MergeTreeDataFormatVersion min_format_version(0); if (!date_column_name.empty()) @@ -2665,6 +2669,8 @@ void MergeTreeData::checkAlterIsPossible(const AlterCommands & commands, Context { const auto current_changes = old_metadata.getSettingsChanges()->as().changes; const auto & new_changes = new_metadata.settings_changes->as().changes; + getContext()->checkMergeTreeSettingsConstraints(*settings_from_storage, new_changes); + for (const auto & changed_setting : new_changes) { const auto & setting_name = changed_setting.name; @@ -2822,6 +2828,7 @@ void MergeTreeData::changeSettings( bool has_storage_policy_changed = false; const auto & new_changes = new_settings->as().changes; + getContext()->checkMergeTreeSettingsConstraints(*getSettings(), new_changes); for (const auto & change : new_changes) { From e908e49dd9113b17b75ded08a6b0097565ac0d81 Mon Sep 17 00:00:00 2001 From: serxa Date: Mon, 5 Dec 2022 11:02:37 +0000 Subject: [PATCH 052/127] review fixes --- src/Access/AccessControl.cpp | 4 +- src/Access/SettingsConstraints.cpp | 136 ++++++++++++----------------- 2 files changed, 56 insertions(+), 84 deletions(-) diff --git a/src/Access/AccessControl.cpp b/src/Access/AccessControl.cpp index 87333f99423..ad1b7b2ca63 100644 --- a/src/Access/AccessControl.cpp +++ b/src/Access/AccessControl.cpp @@ -41,7 +41,7 @@ namespace ErrorCodes namespace { - constexpr const char MERGE_TREE_SETTINGS_PREFIX[] = "merge_tree_"; + constexpr std::string_view MERGE_TREE_SETTINGS_PREFIX = "merge_tree_"; } namespace @@ -110,7 +110,7 @@ public: { if (Settings::hasBuiltin(setting_name)) return true; - if (setting_name.starts_with(MERGE_TREE_SETTINGS_PREFIX) && MergeTreeSettings::hasBuiltin(setting_name.substr(strlen(MERGE_TREE_SETTINGS_PREFIX)))) + if (setting_name.starts_with(MERGE_TREE_SETTINGS_PREFIX) && MergeTreeSettings::hasBuiltin(setting_name.substr(MERGE_TREE_SETTINGS_PREFIX.size()))) return true; std::lock_guard lock{mutex}; diff --git a/src/Access/SettingsConstraints.cpp b/src/Access/SettingsConstraints.cpp index e0efb85b707..217f48d0fa8 100644 --- a/src/Access/SettingsConstraints.cpp +++ b/src/Access/SettingsConstraints.cpp @@ -1,3 +1,4 @@ +#include #include #include #include @@ -8,7 +9,6 @@ #include #include - namespace DB { namespace ErrorCodes @@ -21,7 +21,7 @@ namespace ErrorCodes namespace { - constexpr const char MERGE_TREE_SETTINGS_PREFIX[] = "merge_tree_"; + constexpr std::string_view MERGE_TREE_SETTINGS_PREFIX = "merge_tree_"; } @@ -44,10 +44,21 @@ void SettingsConstraints::clear() void SettingsConstraints::set(const String & setting_name, const Field & min_value, const Field & max_value, SettingConstraintWritability writability) { auto & constraint = constraints[setting_name]; - if (!min_value.isNull()) - constraint.min_value = Settings::castValueUtil(setting_name, min_value); - if (!max_value.isNull()) - constraint.max_value = Settings::castValueUtil(setting_name, max_value); + if (setting_name.starts_with(MERGE_TREE_SETTINGS_PREFIX)) + { + std::string_view name = static_cast(setting_name).substr(MERGE_TREE_SETTINGS_PREFIX.size()); + if (!min_value.isNull()) + constraint.min_value = MergeTreeSettings::castValueUtil(name, min_value); + if (!max_value.isNull()) + constraint.max_value = MergeTreeSettings::castValueUtil(name, max_value); + } + else + { + if (!min_value.isNull()) + constraint.min_value = Settings::castValueUtil(setting_name, min_value); + if (!max_value.isNull()) + constraint.max_value = Settings::castValueUtil(setting_name, max_value); + } constraint.writability = writability; } @@ -124,6 +135,35 @@ void SettingsConstraints::clamp(const Settings & current_settings, SettingsChang }); } +template +bool getNewValueToCheck(const T & current_settings, SettingChange & change, Field & new_value, bool throw_on_failure) +{ + Field current_value; + bool has_current_value = current_settings.tryGet(change.name, current_value); + + /// Setting isn't checked if value has not changed. + if (has_current_value && change.value == current_value) + return false; + + if (throw_on_failure) + new_value = T::castValueUtil(change.name, change.value); + + try + { + new_value = T::castValueUtil(change.name, change.value); + } + catch (...) + { + return false; + } + + /// Setting isn't checked if value has not changed. + if (has_current_value && new_value == current_value) + return false; + + return true; +} + bool SettingsConstraints::checkImpl(const Settings & current_settings, SettingChange & change, ReactionOnViolation reaction) const { const String & setting_name = change.name; @@ -131,26 +171,6 @@ bool SettingsConstraints::checkImpl(const Settings & current_settings, SettingCh if (setting_name == "profile") return true; - bool cannot_cast; - auto cast_value = [&](const Field & x) -> Field - { - cannot_cast = false; - if (reaction == THROW_ON_VIOLATION) - return Settings::castValueUtil(setting_name, x); - else - { - try - { - return Settings::castValueUtil(setting_name, x); - } - catch (...) - { - cannot_cast = true; - return {}; - } - } - }; - if (reaction == THROW_ON_VIOLATION) { try @@ -172,74 +192,26 @@ bool SettingsConstraints::checkImpl(const Settings & current_settings, SettingCh else if (!access_control->isSettingNameAllowed(setting_name)) return false; - Field current_value, new_value; - if (current_settings.tryGet(setting_name, current_value)) - { - /// Setting isn't checked if value has not changed. - if (change.value == current_value) - return false; - - new_value = cast_value(change.value); - if ((new_value == current_value) || cannot_cast) - return false; - } - else - { - new_value = cast_value(change.value); - if (cannot_cast) - return false; - } + Field new_value; + if (!getNewValueToCheck(current_settings, change, new_value, reaction == THROW_ON_VIOLATION)) + return false; return getChecker(current_settings, setting_name).check(change, new_value, reaction); } bool SettingsConstraints::checkImpl(const MergeTreeSettings & current_settings, SettingChange & change, ReactionOnViolation reaction) const { - const String & setting_name = change.name; - String prefixed_name = MERGE_TREE_SETTINGS_PREFIX + setting_name; - - bool cannot_cast; - auto cast_value = [&](const Field & x) -> Field - { - cannot_cast = false; - if (reaction == THROW_ON_VIOLATION) - return MergeTreeSettings::castValueUtil(setting_name, x); - else - { - try - { - return MergeTreeSettings::castValueUtil(setting_name, x); - } - catch (...) - { - cannot_cast = true; - return {}; - } - } - }; + String prefixed_name(MERGE_TREE_SETTINGS_PREFIX); + prefixed_name += change.name; // Just because you cannot concatenate `std::string_view` and `std::string` using operator+ in C++20 yet if (reaction == THROW_ON_VIOLATION) access_control->checkSettingNameIsAllowed(prefixed_name); else if (!access_control->isSettingNameAllowed(prefixed_name)) return false; - Field current_value, new_value; - if (current_settings.tryGet(setting_name, current_value)) - { - /// Setting isn't checked if value has not changed. - if (change.value == current_value) - return false; - - new_value = cast_value(change.value); - if ((new_value == current_value) || cannot_cast) - return false; - } - else - { - new_value = cast_value(change.value); - if (cannot_cast) - return false; - } + Field new_value; + if (!getNewValueToCheck(current_settings, change, new_value, reaction == THROW_ON_VIOLATION)) + return false; return getMergeTreeChecker(prefixed_name).check(change, new_value, reaction); } From dffc1edce8b25444e7af58210c51836102d6b79d Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Mon, 5 Dec 2022 20:09:19 +0000 Subject: [PATCH 053/127] Add test. --- .../0_stateless/02497_remote_disk_fat_column.reference | 1 + .../queries/0_stateless/02497_remote_disk_fat_column.sql | 8 ++++++++ 2 files changed, 9 insertions(+) create mode 100644 tests/queries/0_stateless/02497_remote_disk_fat_column.reference create mode 100644 tests/queries/0_stateless/02497_remote_disk_fat_column.sql diff --git a/tests/queries/0_stateless/02497_remote_disk_fat_column.reference b/tests/queries/0_stateless/02497_remote_disk_fat_column.reference new file mode 100644 index 00000000000..9183bf03fcc --- /dev/null +++ b/tests/queries/0_stateless/02497_remote_disk_fat_column.reference @@ -0,0 +1 @@ +256 diff --git a/tests/queries/0_stateless/02497_remote_disk_fat_column.sql b/tests/queries/0_stateless/02497_remote_disk_fat_column.sql new file mode 100644 index 00000000000..c7672f0ec95 --- /dev/null +++ b/tests/queries/0_stateless/02497_remote_disk_fat_column.sql @@ -0,0 +1,8 @@ +-- Tags: no-parallel, no-random-settings, no-fasttest, no-tsan, no-asan, no-msan +set allow_suspicious_fixed_string_types=1; +create table fat_granularity (x UInt32, fat FixedString(160000)) engine = MergeTree order by x settings storage_policy = 's3_cache'; + +insert into fat_granularity select number, toString(number) || '_' from numbers(100000) settings max_block_size = 8192, max_insert_threads=8; + +-- Too large sizes of FixedString to deserialize +select x from fat_granularity prewhere fat like '256\_%' settings max_threads=2; From b797157437a217442bce84bc6d733d8c843bab75 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Mon, 5 Dec 2022 20:10:31 +0000 Subject: [PATCH 054/127] Respect adaptive index granularity settings while reading from remote disk. --- .../MergeTree/MergeTreeBaseSelectProcessor.cpp | 6 +++--- .../MergeTree/MergeTreeBaseSelectProcessor.h | 14 ++++++++++++++ .../MergeTree/MergeTreeIndexGranularity.cpp | 7 ++++++- src/Storages/MergeTree/MergeTreeIndexGranularity.h | 2 +- .../MergeTree/MergeTreeThreadSelectProcessor.h | 1 - 5 files changed, 24 insertions(+), 6 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeBaseSelectProcessor.cpp b/src/Storages/MergeTree/MergeTreeBaseSelectProcessor.cpp index b3185b1a6af..bd7e3a64749 100644 --- a/src/Storages/MergeTree/MergeTreeBaseSelectProcessor.cpp +++ b/src/Storages/MergeTree/MergeTreeBaseSelectProcessor.cpp @@ -345,7 +345,7 @@ void IMergeTreeSelectAlgorithm::initializeRangeReadersImpl( } static UInt64 estimateNumRows(const MergeTreeReadTask & current_task, UInt64 current_preferred_block_size_bytes, - UInt64 current_max_block_size_rows, UInt64 current_preferred_max_column_in_block_size_bytes, double min_filtration_ratio) + UInt64 current_max_block_size_rows, UInt64 current_preferred_max_column_in_block_size_bytes, double min_filtration_ratio, size_t min_marks_to_read) { const MergeTreeRangeReader & current_reader = current_task.range_reader; @@ -379,7 +379,7 @@ static UInt64 estimateNumRows(const MergeTreeReadTask & current_task, UInt64 cur const MergeTreeIndexGranularity & index_granularity = current_task.data_part->index_granularity; - return index_granularity.countMarksForRows(current_reader.currentMark(), rows_to_read, current_reader.numReadRowsInCurrentGranule()); + return index_granularity.countMarksForRows(current_reader.currentMark(), rows_to_read, current_reader.numReadRowsInCurrentGranule(), min_marks_to_read); } @@ -394,7 +394,7 @@ IMergeTreeSelectAlgorithm::BlockAndProgress IMergeTreeSelectAlgorithm::readFromP const double min_filtration_ratio = 0.00001; UInt64 recommended_rows = estimateNumRows(*task, current_preferred_block_size_bytes, - current_max_block_size_rows, current_preferred_max_column_in_block_size_bytes, min_filtration_ratio); + current_max_block_size_rows, current_preferred_max_column_in_block_size_bytes, min_filtration_ratio, min_marks_to_read); UInt64 rows_to_read = std::max(static_cast(1), std::min(current_max_block_size_rows, recommended_rows)); auto read_result = task->range_reader.read(rows_to_read, task->mark_ranges); diff --git a/src/Storages/MergeTree/MergeTreeBaseSelectProcessor.h b/src/Storages/MergeTree/MergeTreeBaseSelectProcessor.h index 2a5f6871422..b598ad87aab 100644 --- a/src/Storages/MergeTree/MergeTreeBaseSelectProcessor.h +++ b/src/Storages/MergeTree/MergeTreeBaseSelectProcessor.h @@ -169,6 +169,20 @@ protected: std::deque delayed_tasks; std::deque buffered_ranges; + /// This setting is used in base algorithm only an an additionally limit the number of granules to read. + /// It is changed in ctor of MergeTreeThreadSelectAlgorithm. + /// + /// The reason why we have it here is because MergeTreeReadPool takes the full task + /// ignoring min_marks_to_read setting in case of remote disk (see MergeTreeReadPool::getTask). + /// Which makes a logic of adaptive granulatity + /// (merge_tree_min_rows_for_concurrent_read_for_remote_filesystem and + /// merge_tree_min_bytes_for_concurrent_read_for_remote_filesystem settings) a bit useless. + /// + /// On the other hand, big tasks are better for remote disk and prefetches. + /// So, for now it's easier to limit max_rows_to_read. + /// Somebody need to refactor this later. + size_t min_marks_to_read = 0; + private: Poco::Logger * log = &Poco::Logger::get("MergeTreeBaseSelectProcessor"); diff --git a/src/Storages/MergeTree/MergeTreeIndexGranularity.cpp b/src/Storages/MergeTree/MergeTreeIndexGranularity.cpp index ac0146edc3a..2028ce74c23 100644 --- a/src/Storages/MergeTree/MergeTreeIndexGranularity.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexGranularity.cpp @@ -90,7 +90,7 @@ size_t MergeTreeIndexGranularity::getRowsCountInRanges(const MarkRanges & ranges } -size_t MergeTreeIndexGranularity::countMarksForRows(size_t from_mark, size_t number_of_rows, size_t offset_in_rows) const +size_t MergeTreeIndexGranularity::countMarksForRows(size_t from_mark, size_t number_of_rows, size_t offset_in_rows, size_t min_marks_to_read) const { size_t rows_before_mark = getMarkStartingRow(from_mark); size_t last_row_pos = rows_before_mark + offset_in_rows + number_of_rows; @@ -101,6 +101,11 @@ size_t MergeTreeIndexGranularity::countMarksForRows(size_t from_mark, size_t num else to_mark = position - marks_rows_partial_sums.begin(); + /// This is a heuristic to respect min_marks_to_read which is igored by MergeTreeReadPool in case of remote disk. + /// See comment in IMergeTreeSelectAlgorithm. + if (min_marks_to_read && from_mark + 2 * min_marks_to_read <= to_mark) + to_mark = from_mark + min_marks_to_read; + return getRowsCountInRange(from_mark, std::max(1UL, to_mark)) - offset_in_rows; } diff --git a/src/Storages/MergeTree/MergeTreeIndexGranularity.h b/src/Storages/MergeTree/MergeTreeIndexGranularity.h index 0d1480d68cb..f5677995ae0 100644 --- a/src/Storages/MergeTree/MergeTreeIndexGranularity.h +++ b/src/Storages/MergeTree/MergeTreeIndexGranularity.h @@ -34,7 +34,7 @@ public: /// |-----|---------------------------|----|----| /// ^------------------------^-----------^ //// from_mark offset_in_rows number_of_rows - size_t countMarksForRows(size_t from_mark, size_t number_of_rows, size_t offset_in_rows=0) const; + size_t countMarksForRows(size_t from_mark, size_t number_of_rows, size_t offset_in_rows, size_t min_marks_to_read) const; /// Total marks size_t getMarksCount() const; diff --git a/src/Storages/MergeTree/MergeTreeThreadSelectProcessor.h b/src/Storages/MergeTree/MergeTreeThreadSelectProcessor.h index a9104f25d75..ac3dcf0cc41 100644 --- a/src/Storages/MergeTree/MergeTreeThreadSelectProcessor.h +++ b/src/Storages/MergeTree/MergeTreeThreadSelectProcessor.h @@ -49,7 +49,6 @@ private: size_t thread; std::shared_ptr pool; - size_t min_marks_to_read; /// Last part read in this thread std::string last_readed_part_name; From f75250eaf4a2db14aa60a7998f54182afeb742d6 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Mon, 5 Dec 2022 21:27:54 +0000 Subject: [PATCH 055/127] Fix style. --- src/Storages/MergeTree/MergeTreeIndexGranularity.cpp | 3 +-- tests/queries/0_stateless/02497_remote_disk_fat_column.sql | 2 +- 2 files changed, 2 insertions(+), 3 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeIndexGranularity.cpp b/src/Storages/MergeTree/MergeTreeIndexGranularity.cpp index 2028ce74c23..5fdd0555777 100644 --- a/src/Storages/MergeTree/MergeTreeIndexGranularity.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexGranularity.cpp @@ -101,13 +101,12 @@ size_t MergeTreeIndexGranularity::countMarksForRows(size_t from_mark, size_t num else to_mark = position - marks_rows_partial_sums.begin(); - /// This is a heuristic to respect min_marks_to_read which is igored by MergeTreeReadPool in case of remote disk. + /// This is a heuristic to respect min_marks_to_read which is ignored by MergeTreeReadPool in case of remote disk. /// See comment in IMergeTreeSelectAlgorithm. if (min_marks_to_read && from_mark + 2 * min_marks_to_read <= to_mark) to_mark = from_mark + min_marks_to_read; return getRowsCountInRange(from_mark, std::max(1UL, to_mark)) - offset_in_rows; - } void MergeTreeIndexGranularity::resizeWithFixedGranularity(size_t size, size_t fixed_granularity) diff --git a/tests/queries/0_stateless/02497_remote_disk_fat_column.sql b/tests/queries/0_stateless/02497_remote_disk_fat_column.sql index c7672f0ec95..d97109b66f3 100644 --- a/tests/queries/0_stateless/02497_remote_disk_fat_column.sql +++ b/tests/queries/0_stateless/02497_remote_disk_fat_column.sql @@ -1,4 +1,4 @@ --- Tags: no-parallel, no-random-settings, no-fasttest, no-tsan, no-asan, no-msan +-- Tags: no-random-settings, no-fasttest, no-tsan, no-asan, no-msan set allow_suspicious_fixed_string_types=1; create table fat_granularity (x UInt32, fat FixedString(160000)) engine = MergeTree order by x settings storage_policy = 's3_cache'; From 13d74b4f2f52171f30809f167d4eb582f98a9dfd Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Tue, 6 Dec 2022 10:44:38 +0100 Subject: [PATCH 056/127] Analyzer Context refactoring --- src/Analyzer/Passes/CountDistinctPass.cpp | 4 +- src/Analyzer/Passes/QueryAnalysisPass.cpp | 21 +- src/Analyzer/QueryNode.cpp | 10 +- src/Analyzer/QueryNode.h | 57 +++-- src/Analyzer/QueryTreeBuilder.cpp | 216 ++++++++++-------- src/Analyzer/QueryTreeBuilder.h | 2 + src/Analyzer/UnionNode.cpp | 10 +- src/Analyzer/UnionNode.h | 27 ++- .../InterpreterSelectQueryAnalyzer.cpp | 14 +- .../InterpreterSelectQueryAnalyzer.h | 13 +- src/Planner/Planner.cpp | 88 +++---- src/Planner/Planner.h | 6 +- src/Planner/PlannerJoinTree.cpp | 3 +- src/Planner/Utils.cpp | 9 +- src/Planner/Utils.h | 3 + 15 files changed, 276 insertions(+), 207 deletions(-) diff --git a/src/Analyzer/Passes/CountDistinctPass.cpp b/src/Analyzer/Passes/CountDistinctPass.cpp index 2b55efa3552..05c31ec28ba 100644 --- a/src/Analyzer/Passes/CountDistinctPass.cpp +++ b/src/Analyzer/Passes/CountDistinctPass.cpp @@ -3,6 +3,8 @@ #include #include +#include + #include #include #include @@ -56,7 +58,7 @@ public: auto & count_distinct_argument_column_typed = count_distinct_argument_column->as(); /// Build subquery SELECT count_distinct_argument_column FROM table_expression GROUP BY count_distinct_argument_column - auto subquery = std::make_shared(); + auto subquery = std::make_shared(Context::createCopy(query_node->getContext())); subquery->getJoinTree() = query_node->getJoinTree(); subquery->getProjection().getNodes().push_back(count_distinct_argument_column); subquery->getGroupBy().getNodes().push_back(count_distinct_argument_column); diff --git a/src/Analyzer/Passes/QueryAnalysisPass.cpp b/src/Analyzer/Passes/QueryAnalysisPass.cpp index 4c2074fcfeb..8a66e37ac71 100644 --- a/src/Analyzer/Passes/QueryAnalysisPass.cpp +++ b/src/Analyzer/Passes/QueryAnalysisPass.cpp @@ -647,6 +647,11 @@ struct IdentifierResolveScope subquery_depth = parent_scope->subquery_depth; context = parent_scope->context; } + + if (auto * union_node = scope_node->as()) + context = union_node->getContext(); + else if (auto * query_node = scope_node->as()) + context = query_node->getContext(); } QueryTreeNodePtr scope_node; @@ -974,7 +979,9 @@ public: void resolve(QueryTreeNodePtr node, const QueryTreeNodePtr & table_expression, ContextPtr context) { IdentifierResolveScope scope(node, nullptr /*parent_scope*/); - scope.context = context; + + if (!scope.context) + scope.context = context; auto node_type = node->getNodeType(); @@ -4042,7 +4049,7 @@ ProjectionNames QueryAnalyzer::resolveFunction(QueryTreeNodePtr & node, Identifi auto constant_data_type = std::make_shared(); - auto in_subquery = std::make_shared(); + auto in_subquery = std::make_shared(Context::createCopy(scope.context)); in_subquery->getProjection().getNodes().push_back(std::make_shared(1UL, constant_data_type)); in_subquery->getJoinTree() = exists_subquery_argument; in_subquery->getLimit() = std::make_shared(1UL, constant_data_type); @@ -4095,7 +4102,7 @@ ProjectionNames QueryAnalyzer::resolveFunction(QueryTreeNodePtr & node, Identifi projection_columns.emplace_back(column.name, column.type); } - auto in_second_argument_query_node = std::make_shared(); + auto in_second_argument_query_node = std::make_shared(Context::createCopy(scope.context)); in_second_argument_query_node->setIsSubquery(true); in_second_argument_query_node->getProjectionNode() = std::move(column_nodes_to_select); in_second_argument_query_node->getJoinTree() = std::move(in_second_argument); @@ -5756,14 +5763,6 @@ void QueryAnalyzer::resolveQuery(const QueryTreeNodePtr & query_node, Identifier max_subquery_depth); auto & query_node_typed = query_node->as(); - - if (query_node_typed.hasSettingsChanges()) - { - auto updated_scope_context = Context::createCopy(scope.context); - updated_scope_context->applySettingsChanges(query_node_typed.getSettingsChanges()); - scope.context = std::move(updated_scope_context); - } - const auto & settings = scope.context->getSettingsRef(); if (settings.group_by_use_nulls) diff --git a/src/Analyzer/QueryNode.cpp b/src/Analyzer/QueryNode.cpp index 0fc2edc3cce..618cbd6d9e8 100644 --- a/src/Analyzer/QueryNode.cpp +++ b/src/Analyzer/QueryNode.cpp @@ -21,8 +21,10 @@ namespace DB { -QueryNode::QueryNode() +QueryNode::QueryNode(ContextMutablePtr context_, SettingsChanges settings_changes_) : IQueryTreeNode(children_size) + , context(std::move(context_)) + , settings_changes(std::move(settings_changes_)) { children[with_child_index] = std::make_shared(); children[projection_child_index] = std::make_shared(); @@ -32,6 +34,10 @@ QueryNode::QueryNode() children[limit_by_child_index] = std::make_shared(); } +QueryNode::QueryNode(ContextMutablePtr context_) + : QueryNode(context_, {} /*settings_changes*/) +{} + void QueryNode::dumpTreeImpl(WriteBuffer & buffer, FormatState & format_state, size_t indent) const { buffer << std::string(indent, ' ') << "QUERY id: " << format_state.getNodeId(this); @@ -222,7 +228,7 @@ void QueryNode::updateTreeHashImpl(HashState & state) const QueryTreeNodePtr QueryNode::cloneImpl() const { - auto result_query_node = std::make_shared(); + auto result_query_node = std::make_shared(context); result_query_node->is_subquery = is_subquery; result_query_node->is_cte = is_cte; diff --git a/src/Analyzer/QueryNode.h b/src/Analyzer/QueryNode.h index 37c4f60fa9e..54154e1e353 100644 --- a/src/Analyzer/QueryNode.h +++ b/src/Analyzer/QueryNode.h @@ -10,6 +10,8 @@ #include #include +#include + namespace DB { @@ -61,7 +63,41 @@ using QueryNodePtr = std::shared_ptr; class QueryNode final : public IQueryTreeNode { public: - explicit QueryNode(); + /// Construct query node with context and changed settings + explicit QueryNode(ContextMutablePtr context_, SettingsChanges settings_changes_); + + /// Construct query node with context + explicit QueryNode(ContextMutablePtr context_); + + /// Get context + ContextPtr getContext() const + { + return context; + } + + /// Get mutable context + const ContextMutablePtr & getMutableContext() const + { + return context; + } + + /// Get mutable context + ContextMutablePtr & getMutableContext() + { + return context; + } + + /// Returns true if query node has settings changes, false otherwise + bool hasSettingsChanges() const + { + return !settings_changes.empty(); + } + + /// Get query node settings changes + const SettingsChanges & getSettingsChanges() const + { + return settings_changes; + } /// Returns true if query node is subquery, false otherwise bool isSubquery() const @@ -513,24 +549,6 @@ public: return children[offset_child_index]; } - /// Returns true if query node has settings changes specified, false otherwise - bool hasSettingsChanges() const - { - return !settings_changes.empty(); - } - - /// Get query node settings changes - const SettingsChanges & getSettingsChanges() const - { - return settings_changes; - } - - /// Set query node settings changes value - void setSettingsChanges(SettingsChanges settings_changes_value) - { - settings_changes = std::move(settings_changes_value); - } - /// Get query node projection columns const NamesAndTypes & getProjectionColumns() const { @@ -572,6 +590,7 @@ private: std::string cte_name; NamesAndTypes projection_columns; + ContextMutablePtr context; SettingsChanges settings_changes; static constexpr size_t with_child_index = 0; diff --git a/src/Analyzer/QueryTreeBuilder.cpp b/src/Analyzer/QueryTreeBuilder.cpp index 01ecd4ece30..2b2326badfa 100644 --- a/src/Analyzer/QueryTreeBuilder.cpp +++ b/src/Analyzer/QueryTreeBuilder.cpp @@ -77,75 +77,90 @@ public: } private: - QueryTreeNodePtr buildSelectOrUnionExpression(const ASTPtr & select_or_union_query, bool is_subquery, const std::string & cte_name) const; + QueryTreeNodePtr buildSelectOrUnionExpression(const ASTPtr & select_or_union_query, + bool is_subquery, + const std::string & cte_name, + const ContextPtr & context) const; - QueryTreeNodePtr buildSelectWithUnionExpression(const ASTPtr & select_with_union_query, bool is_subquery, const std::string & cte_name) const; + QueryTreeNodePtr buildSelectWithUnionExpression(const ASTPtr & select_with_union_query, + bool is_subquery, + const std::string & cte_name, + const ContextPtr & context) const; - QueryTreeNodePtr buildSelectIntersectExceptQuery(const ASTPtr & select_intersect_except_query, bool is_subquery, const std::string & cte_name) const; + QueryTreeNodePtr buildSelectIntersectExceptQuery(const ASTPtr & select_intersect_except_query, + bool is_subquery, + const std::string & cte_name, + const ContextPtr & context) const; - QueryTreeNodePtr buildSelectExpression(const ASTPtr & select_query, bool is_subquery, const std::string & cte_name) const; + QueryTreeNodePtr buildSelectExpression(const ASTPtr & select_query, + bool is_subquery, + const std::string & cte_name, + const ContextPtr & context) const; - QueryTreeNodePtr buildSortList(const ASTPtr & order_by_expression_list) const; + QueryTreeNodePtr buildSortList(const ASTPtr & order_by_expression_list, const ContextPtr & context) const; - QueryTreeNodePtr buildInterpolateList(const ASTPtr & interpolate_expression_list) const; + QueryTreeNodePtr buildInterpolateList(const ASTPtr & interpolate_expression_list, const ContextPtr & context) const; - QueryTreeNodePtr buildWindowList(const ASTPtr & window_definition_list) const; + QueryTreeNodePtr buildWindowList(const ASTPtr & window_definition_list, const ContextPtr & context) const; - QueryTreeNodePtr buildExpressionList(const ASTPtr & expression_list) const; + QueryTreeNodePtr buildExpressionList(const ASTPtr & expression_list, const ContextPtr & context) const; - QueryTreeNodePtr buildExpression(const ASTPtr & expression) const; + QueryTreeNodePtr buildExpression(const ASTPtr & expression, const ContextPtr & context) const; - QueryTreeNodePtr buildWindow(const ASTPtr & window_definition) const; + QueryTreeNodePtr buildWindow(const ASTPtr & window_definition, const ContextPtr & context) const; - QueryTreeNodePtr buildJoinTree(const ASTPtr & tables_in_select_query) const; + QueryTreeNodePtr buildJoinTree(const ASTPtr & tables_in_select_query, const ContextPtr & context) const; - ColumnTransformersNodes buildColumnTransformers(const ASTPtr & matcher_expression, size_t start_child_index) const; + ColumnTransformersNodes buildColumnTransformers(const ASTPtr & matcher_expression, size_t start_child_index, const ContextPtr & context) const; ASTPtr query; - ContextPtr context; QueryTreeNodePtr query_tree_node; - }; QueryTreeBuilder::QueryTreeBuilder(ASTPtr query_, ContextPtr context_) : query(query_->clone()) - , context(std::move(context_)) { if (query->as() || query->as() || query->as()) - query_tree_node = buildSelectOrUnionExpression(query, false /*is_subquery*/, {} /*cte_name*/); + query_tree_node = buildSelectOrUnionExpression(query, false /*is_subquery*/, {} /*cte_name*/, context_); else if (query->as()) - query_tree_node = buildExpressionList(query); + query_tree_node = buildExpressionList(query, context_); else - query_tree_node = buildExpression(query); + query_tree_node = buildExpression(query, context_); } -QueryTreeNodePtr QueryTreeBuilder::buildSelectOrUnionExpression(const ASTPtr & select_or_union_query, bool is_subquery, const std::string & cte_name) const +QueryTreeNodePtr QueryTreeBuilder::buildSelectOrUnionExpression(const ASTPtr & select_or_union_query, + bool is_subquery, + const std::string & cte_name, + const ContextPtr & context) const { QueryTreeNodePtr query_node; if (select_or_union_query->as()) - query_node = buildSelectWithUnionExpression(select_or_union_query, is_subquery /*is_subquery*/, cte_name /*cte_name*/); + query_node = buildSelectWithUnionExpression(select_or_union_query, is_subquery /*is_subquery*/, cte_name /*cte_name*/, context); else if (select_or_union_query->as()) - query_node = buildSelectIntersectExceptQuery(select_or_union_query, is_subquery /*is_subquery*/, cte_name /*cte_name*/); + query_node = buildSelectIntersectExceptQuery(select_or_union_query, is_subquery /*is_subquery*/, cte_name /*cte_name*/, context); else if (select_or_union_query->as()) - query_node = buildSelectExpression(select_or_union_query, is_subquery /*is_subquery*/, cte_name /*cte_name*/); + query_node = buildSelectExpression(select_or_union_query, is_subquery /*is_subquery*/, cte_name /*cte_name*/, context); else throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "SELECT or UNION query {} is not supported", select_or_union_query->formatForErrorMessage()); return query_node; } -QueryTreeNodePtr QueryTreeBuilder::buildSelectWithUnionExpression(const ASTPtr & select_with_union_query, bool is_subquery, const std::string & cte_name) const +QueryTreeNodePtr QueryTreeBuilder::buildSelectWithUnionExpression(const ASTPtr & select_with_union_query, + bool is_subquery, + const std::string & cte_name, + const ContextPtr & context) const { auto & select_with_union_query_typed = select_with_union_query->as(); auto & select_lists = select_with_union_query_typed.list_of_selects->as(); if (select_lists.children.size() == 1) - return buildSelectOrUnionExpression(select_lists.children[0], is_subquery, cte_name); + return buildSelectOrUnionExpression(select_lists.children[0], is_subquery, cte_name, context); - auto union_node = std::make_shared(select_with_union_query_typed.union_mode); + auto union_node = std::make_shared(Context::createCopy(context), select_with_union_query_typed.union_mode); union_node->setIsSubquery(is_subquery); union_node->setIsCTE(!cte_name.empty()); union_node->setCTEName(cte_name); @@ -156,20 +171,23 @@ QueryTreeNodePtr QueryTreeBuilder::buildSelectWithUnionExpression(const ASTPtr & for (size_t i = 0; i < select_lists_children_size; ++i) { auto & select_list_node = select_lists.children[i]; - QueryTreeNodePtr query_node = buildSelectOrUnionExpression(select_list_node, false /*is_subquery*/, {} /*cte_name*/); + QueryTreeNodePtr query_node = buildSelectOrUnionExpression(select_list_node, false /*is_subquery*/, {} /*cte_name*/, context); union_node->getQueries().getNodes().push_back(std::move(query_node)); } return union_node; } -QueryTreeNodePtr QueryTreeBuilder::buildSelectIntersectExceptQuery(const ASTPtr & select_intersect_except_query, bool is_subquery, const std::string & cte_name) const +QueryTreeNodePtr QueryTreeBuilder::buildSelectIntersectExceptQuery(const ASTPtr & select_intersect_except_query, + bool is_subquery, + const std::string & cte_name, + const ContextPtr & context) const { auto & select_intersect_except_query_typed = select_intersect_except_query->as(); auto select_lists = select_intersect_except_query_typed.getListOfSelects(); if (select_lists.size() == 1) - return buildSelectExpression(select_lists[0], is_subquery, cte_name); + return buildSelectExpression(select_lists[0], is_subquery, cte_name, context); SelectUnionMode union_mode; if (select_intersect_except_query_typed.final_operator == ASTSelectIntersectExceptQuery::Operator::INTERSECT_ALL) @@ -183,7 +201,7 @@ QueryTreeNodePtr QueryTreeBuilder::buildSelectIntersectExceptQuery(const ASTPtr else throw Exception(ErrorCodes::LOGICAL_ERROR, "UNION type is not initialized"); - auto union_node = std::make_shared(union_mode); + auto union_node = std::make_shared(Context::createCopy(context), union_mode); union_node->setIsSubquery(is_subquery); union_node->setIsCTE(!cte_name.empty()); union_node->setCTEName(cte_name); @@ -194,17 +212,32 @@ QueryTreeNodePtr QueryTreeBuilder::buildSelectIntersectExceptQuery(const ASTPtr for (size_t i = 0; i < select_lists_size; ++i) { auto & select_list_node = select_lists[i]; - QueryTreeNodePtr query_node = buildSelectOrUnionExpression(select_list_node, false /*is_subquery*/, {} /*cte_name*/); + QueryTreeNodePtr query_node = buildSelectOrUnionExpression(select_list_node, false /*is_subquery*/, {} /*cte_name*/, context); union_node->getQueries().getNodes().push_back(std::move(query_node)); } return union_node; } -QueryTreeNodePtr QueryTreeBuilder::buildSelectExpression(const ASTPtr & select_query, bool is_subquery, const std::string & cte_name) const +QueryTreeNodePtr QueryTreeBuilder::buildSelectExpression(const ASTPtr & select_query, + bool is_subquery, + const std::string & cte_name, + const ContextPtr & context) const { const auto & select_query_typed = select_query->as(); - auto current_query_tree = std::make_shared(); + + auto updated_context = Context::createCopy(context); + auto select_settings = select_query_typed.settings(); + SettingsChanges settings_changes; + + if (select_settings) + { + auto & set_query = select_settings->as(); + updated_context->applySettingsChanges(set_query.changes); + settings_changes = set_query.changes; + } + + auto current_query_tree = std::make_shared(std::move(updated_context), std::move(settings_changes)); current_query_tree->setIsSubquery(is_subquery); current_query_tree->setIsCTE(!cte_name.empty()); @@ -218,30 +251,25 @@ QueryTreeNodePtr QueryTreeBuilder::buildSelectExpression(const ASTPtr & select_q current_query_tree->setIsGroupByAll(select_query_typed.group_by_all); current_query_tree->setOriginalAST(select_query); - auto select_settings = select_query_typed.settings(); - if (select_settings) - { - auto & set_query = select_settings->as(); - current_query_tree->setSettingsChanges(set_query.changes); - } + auto current_context = current_query_tree->getContext(); - current_query_tree->getJoinTree() = buildJoinTree(select_query_typed.tables()); + current_query_tree->getJoinTree() = buildJoinTree(select_query_typed.tables(), current_context); auto select_with_list = select_query_typed.with(); if (select_with_list) - current_query_tree->getWithNode() = buildExpressionList(select_with_list); + current_query_tree->getWithNode() = buildExpressionList(select_with_list, current_context); auto select_expression_list = select_query_typed.select(); if (select_expression_list) - current_query_tree->getProjectionNode() = buildExpressionList(select_expression_list); + current_query_tree->getProjectionNode() = buildExpressionList(select_expression_list, current_context); auto prewhere_expression = select_query_typed.prewhere(); if (prewhere_expression) - current_query_tree->getPrewhere() = buildExpression(prewhere_expression); + current_query_tree->getPrewhere() = buildExpression(prewhere_expression, current_context); auto where_expression = select_query_typed.where(); if (where_expression) - current_query_tree->getWhere() = buildExpression(where_expression); + current_query_tree->getWhere() = buildExpression(where_expression, current_context); auto group_by_list = select_query_typed.groupBy(); if (group_by_list) @@ -254,56 +282,56 @@ QueryTreeNodePtr QueryTreeBuilder::buildSelectExpression(const ASTPtr & select_q for (auto & grouping_sets_keys : group_by_children) { - auto grouping_sets_keys_list_node = buildExpressionList(grouping_sets_keys); + auto grouping_sets_keys_list_node = buildExpressionList(grouping_sets_keys, current_context); current_query_tree->getGroupBy().getNodes().emplace_back(std::move(grouping_sets_keys_list_node)); } } else { - current_query_tree->getGroupByNode() = buildExpressionList(group_by_list); + current_query_tree->getGroupByNode() = buildExpressionList(group_by_list, current_context); } } auto having_expression = select_query_typed.having(); if (having_expression) - current_query_tree->getHaving() = buildExpression(having_expression); + current_query_tree->getHaving() = buildExpression(having_expression, current_context); auto window_list = select_query_typed.window(); if (window_list) - current_query_tree->getWindowNode() = buildWindowList(window_list); + current_query_tree->getWindowNode() = buildWindowList(window_list, current_context); auto select_order_by_list = select_query_typed.orderBy(); if (select_order_by_list) - current_query_tree->getOrderByNode() = buildSortList(select_order_by_list); + current_query_tree->getOrderByNode() = buildSortList(select_order_by_list, current_context); auto interpolate_list = select_query_typed.interpolate(); if (interpolate_list) - current_query_tree->getInterpolate() = buildInterpolateList(interpolate_list); + current_query_tree->getInterpolate() = buildInterpolateList(interpolate_list, current_context); auto select_limit_by_limit = select_query_typed.limitByLength(); if (select_limit_by_limit) - current_query_tree->getLimitByLimit() = buildExpression(select_limit_by_limit); + current_query_tree->getLimitByLimit() = buildExpression(select_limit_by_limit, current_context); auto select_limit_by_offset = select_query_typed.limitOffset(); if (select_limit_by_offset) - current_query_tree->getLimitByOffset() = buildExpression(select_limit_by_offset); + current_query_tree->getLimitByOffset() = buildExpression(select_limit_by_offset, current_context); auto select_limit_by = select_query_typed.limitBy(); if (select_limit_by) - current_query_tree->getLimitByNode() = buildExpressionList(select_limit_by); + current_query_tree->getLimitByNode() = buildExpressionList(select_limit_by, current_context); auto select_limit = select_query_typed.limitLength(); if (select_limit) - current_query_tree->getLimit() = buildExpression(select_limit); + current_query_tree->getLimit() = buildExpression(select_limit, current_context); auto select_offset = select_query_typed.limitOffset(); if (select_offset) - current_query_tree->getOffset() = buildExpression(select_offset); + current_query_tree->getOffset() = buildExpression(select_offset, current_context); return current_query_tree; } -QueryTreeNodePtr QueryTreeBuilder::buildSortList(const ASTPtr & order_by_expression_list) const +QueryTreeNodePtr QueryTreeBuilder::buildSortList(const ASTPtr & order_by_expression_list, const ContextPtr & context) const { auto list_node = std::make_shared(); @@ -324,7 +352,7 @@ QueryTreeNodePtr QueryTreeBuilder::buildSortList(const ASTPtr & order_by_express collator = std::make_shared(order_by_element.collation->as().value.get()); const auto & sort_expression_ast = order_by_element.children.at(0); - auto sort_expression = buildExpression(sort_expression_ast); + auto sort_expression = buildExpression(sort_expression_ast, context); auto sort_node = std::make_shared(std::move(sort_expression), sort_direction, nulls_sort_direction, @@ -332,11 +360,11 @@ QueryTreeNodePtr QueryTreeBuilder::buildSortList(const ASTPtr & order_by_express order_by_element.with_fill); if (order_by_element.fill_from) - sort_node->getFillFrom() = buildExpression(order_by_element.fill_from); + sort_node->getFillFrom() = buildExpression(order_by_element.fill_from, context); if (order_by_element.fill_to) - sort_node->getFillTo() = buildExpression(order_by_element.fill_to); + sort_node->getFillTo() = buildExpression(order_by_element.fill_to, context); if (order_by_element.fill_step) - sort_node->getFillStep() = buildExpression(order_by_element.fill_step); + sort_node->getFillStep() = buildExpression(order_by_element.fill_step, context); list_node->getNodes().push_back(std::move(sort_node)); } @@ -344,7 +372,7 @@ QueryTreeNodePtr QueryTreeBuilder::buildSortList(const ASTPtr & order_by_express return list_node; } -QueryTreeNodePtr QueryTreeBuilder::buildInterpolateList(const ASTPtr & interpolate_expression_list) const +QueryTreeNodePtr QueryTreeBuilder::buildInterpolateList(const ASTPtr & interpolate_expression_list, const ContextPtr & context) const { auto list_node = std::make_shared(); @@ -355,7 +383,7 @@ QueryTreeNodePtr QueryTreeBuilder::buildInterpolateList(const ASTPtr & interpola { const auto & interpolate_element = expression->as(); auto expression_to_interpolate = std::make_shared(Identifier(interpolate_element.column)); - auto interpolate_expression = buildExpression(interpolate_element.expr); + auto interpolate_expression = buildExpression(interpolate_element.expr, context); auto interpolate_node = std::make_shared(std::move(expression_to_interpolate), std::move(interpolate_expression)); list_node->getNodes().push_back(std::move(interpolate_node)); @@ -364,7 +392,7 @@ QueryTreeNodePtr QueryTreeBuilder::buildInterpolateList(const ASTPtr & interpola return list_node; } -QueryTreeNodePtr QueryTreeBuilder::buildWindowList(const ASTPtr & window_definition_list) const +QueryTreeNodePtr QueryTreeBuilder::buildWindowList(const ASTPtr & window_definition_list, const ContextPtr & context) const { auto list_node = std::make_shared(); @@ -375,7 +403,7 @@ QueryTreeNodePtr QueryTreeBuilder::buildWindowList(const ASTPtr & window_definit { const auto & window_list_element_typed = window_list_element->as(); - auto window_node = buildWindow(window_list_element_typed.definition); + auto window_node = buildWindow(window_list_element_typed.definition, context); window_node->setAlias(window_list_element_typed.name); list_node->getNodes().push_back(std::move(window_node)); @@ -384,7 +412,7 @@ QueryTreeNodePtr QueryTreeBuilder::buildWindowList(const ASTPtr & window_definit return list_node; } -QueryTreeNodePtr QueryTreeBuilder::buildExpressionList(const ASTPtr & expression_list) const +QueryTreeNodePtr QueryTreeBuilder::buildExpressionList(const ASTPtr & expression_list, const ContextPtr & context) const { auto list_node = std::make_shared(); @@ -393,14 +421,14 @@ QueryTreeNodePtr QueryTreeBuilder::buildExpressionList(const ASTPtr & expression for (auto & expression : expression_list_typed.children) { - auto expression_node = buildExpression(expression); + auto expression_node = buildExpression(expression, context); list_node->getNodes().push_back(std::move(expression_node)); } return list_node; } -QueryTreeNodePtr QueryTreeBuilder::buildExpression(const ASTPtr & expression) const +QueryTreeNodePtr QueryTreeBuilder::buildExpression(const ASTPtr & expression, const ContextPtr & context) const { QueryTreeNodePtr result; @@ -411,13 +439,13 @@ QueryTreeNodePtr QueryTreeBuilder::buildExpression(const ASTPtr & expression) co } else if (const auto * asterisk = expression->as()) { - auto column_transformers = buildColumnTransformers(expression, 0 /*start_child_index*/); + auto column_transformers = buildColumnTransformers(expression, 0 /*start_child_index*/, context); result = std::make_shared(std::move(column_transformers)); } else if (const auto * qualified_asterisk = expression->as()) { auto & qualified_identifier = qualified_asterisk->children.at(0)->as(); - auto column_transformers = buildColumnTransformers(expression, 1 /*start_child_index*/); + auto column_transformers = buildColumnTransformers(expression, 1 /*start_child_index*/, context); result = std::make_shared(Identifier(qualified_identifier.name_parts), std::move(column_transformers)); } else if (const auto * ast_literal = expression->as()) @@ -466,7 +494,7 @@ QueryTreeNodePtr QueryTreeBuilder::buildExpression(const ASTPtr & expression) co } const auto & lambda_expression = lambda_arguments_and_expression.at(1); - auto lambda_expression_node = buildExpression(lambda_expression); + auto lambda_expression_node = buildExpression(lambda_expression, context); result = std::make_shared(std::move(lambda_arguments), std::move(lambda_expression_node)); } @@ -478,20 +506,20 @@ QueryTreeNodePtr QueryTreeBuilder::buildExpression(const ASTPtr & expression) co { const auto & function_parameters_list = function->parameters->as()->children; for (const auto & argument : function_parameters_list) - function_node->getParameters().getNodes().push_back(buildExpression(argument)); + function_node->getParameters().getNodes().push_back(buildExpression(argument, context)); } if (function->arguments) { const auto & function_arguments_list = function->arguments->as()->children; for (const auto & argument : function_arguments_list) - function_node->getArguments().getNodes().push_back(buildExpression(argument)); + function_node->getArguments().getNodes().push_back(buildExpression(argument, context)); } if (function->is_window_function) { if (function->window_definition) - function_node->getWindowNode() = buildWindow(function->window_definition); + function_node->getWindowNode() = buildWindow(function->window_definition, context); else function_node->getWindowNode() = std::make_shared(Identifier(function->window_name)); } @@ -502,20 +530,20 @@ QueryTreeNodePtr QueryTreeBuilder::buildExpression(const ASTPtr & expression) co else if (const auto * subquery = expression->as()) { auto subquery_query = subquery->children[0]; - auto query_node = buildSelectWithUnionExpression(subquery_query, true /*is_subquery*/, {} /*cte_name*/); + auto query_node = buildSelectWithUnionExpression(subquery_query, true /*is_subquery*/, {} /*cte_name*/, context); result = std::move(query_node); } else if (const auto * with_element = expression->as()) { auto with_element_subquery = with_element->subquery->as().children.at(0); - auto query_node = buildSelectWithUnionExpression(with_element_subquery, true /*is_subquery*/, with_element->name /*cte_name*/); + auto query_node = buildSelectWithUnionExpression(with_element_subquery, true /*is_subquery*/, with_element->name /*cte_name*/, context); result = std::move(query_node); } else if (const auto * columns_regexp_matcher = expression->as()) { - auto column_transformers = buildColumnTransformers(expression, 0 /*start_child_index*/); + auto column_transformers = buildColumnTransformers(expression, 0 /*start_child_index*/, context); result = std::make_shared(columns_regexp_matcher->getMatcher(), std::move(column_transformers)); } else if (const auto * columns_list_matcher = expression->as()) @@ -529,13 +557,13 @@ QueryTreeNodePtr QueryTreeBuilder::buildExpression(const ASTPtr & expression) co column_list_identifiers.emplace_back(Identifier{column_list_identifier.name_parts}); } - auto column_transformers = buildColumnTransformers(expression, 0 /*start_child_index*/); + auto column_transformers = buildColumnTransformers(expression, 0 /*start_child_index*/, context); result = std::make_shared(std::move(column_list_identifiers), std::move(column_transformers)); } else if (const auto * qualified_columns_regexp_matcher = expression->as()) { auto & qualified_identifier = qualified_columns_regexp_matcher->children.at(0)->as(); - auto column_transformers = buildColumnTransformers(expression, 1 /*start_child_index*/); + auto column_transformers = buildColumnTransformers(expression, 1 /*start_child_index*/, context); result = std::make_shared(Identifier(qualified_identifier.name_parts), qualified_columns_regexp_matcher->getMatcher(), std::move(column_transformers)); } else if (const auto * qualified_columns_list_matcher = expression->as()) @@ -551,7 +579,7 @@ QueryTreeNodePtr QueryTreeBuilder::buildExpression(const ASTPtr & expression) co column_list_identifiers.emplace_back(Identifier{column_list_identifier.name_parts}); } - auto column_transformers = buildColumnTransformers(expression, 1 /*start_child_index*/); + auto column_transformers = buildColumnTransformers(expression, 1 /*start_child_index*/, context); result = std::make_shared(Identifier(qualified_identifier.name_parts), std::move(column_list_identifiers), std::move(column_transformers)); } else @@ -567,7 +595,7 @@ QueryTreeNodePtr QueryTreeBuilder::buildExpression(const ASTPtr & expression) co return result; } -QueryTreeNodePtr QueryTreeBuilder::buildWindow(const ASTPtr & window_definition) const +QueryTreeNodePtr QueryTreeBuilder::buildWindow(const ASTPtr & window_definition, const ContextPtr & context) const { const auto & window_definition_typed = window_definition->as(); WindowFrame window_frame; @@ -586,23 +614,23 @@ QueryTreeNodePtr QueryTreeBuilder::buildWindow(const ASTPtr & window_definition) window_node->setParentWindowName(window_definition_typed.parent_window_name); if (window_definition_typed.partition_by) - window_node->getPartitionByNode() = buildExpressionList(window_definition_typed.partition_by); + window_node->getPartitionByNode() = buildExpressionList(window_definition_typed.partition_by, context); if (window_definition_typed.order_by) - window_node->getOrderByNode() = buildSortList(window_definition_typed.order_by); + window_node->getOrderByNode() = buildSortList(window_definition_typed.order_by, context); if (window_definition_typed.frame_begin_offset) - window_node->getFrameBeginOffsetNode() = buildExpression(window_definition_typed.frame_begin_offset); + window_node->getFrameBeginOffsetNode() = buildExpression(window_definition_typed.frame_begin_offset, context); if (window_definition_typed.frame_end_offset) - window_node->getFrameEndOffsetNode() = buildExpression(window_definition_typed.frame_end_offset); + window_node->getFrameEndOffsetNode() = buildExpression(window_definition_typed.frame_end_offset, context); window_node->setOriginalAST(window_definition); return window_node; } -QueryTreeNodePtr QueryTreeBuilder::buildJoinTree(const ASTPtr & tables_in_select_query) const +QueryTreeNodePtr QueryTreeBuilder::buildJoinTree(const ASTPtr & tables_in_select_query, const ContextPtr & context) const { if (!tables_in_select_query) { @@ -668,7 +696,7 @@ QueryTreeNodePtr QueryTreeBuilder::buildJoinTree(const ASTPtr & tables_in_select auto & subquery_expression = table_expression.subquery->as(); const auto & select_with_union_query = subquery_expression.children[0]; - auto node = buildSelectWithUnionExpression(select_with_union_query, true /*is_subquery*/, {} /*cte_name*/); + auto node = buildSelectWithUnionExpression(select_with_union_query, true /*is_subquery*/, {} /*cte_name*/, context); node->setAlias(subquery_expression.tryGetAlias()); node->setOriginalAST(select_with_union_query); @@ -694,9 +722,9 @@ QueryTreeNodePtr QueryTreeBuilder::buildJoinTree(const ASTPtr & tables_in_select for (const auto & argument : function_arguments_list) { if (argument->as() || argument->as() || argument->as()) - node->getArguments().getNodes().push_back(buildSelectOrUnionExpression(argument, false /*is_subquery*/, {} /*cte_name*/)); + node->getArguments().getNodes().push_back(buildSelectOrUnionExpression(argument, false /*is_subquery*/, {} /*cte_name*/, context)); else - node->getArguments().getNodes().push_back(buildExpression(argument)); + node->getArguments().getNodes().push_back(buildExpression(argument, context)); } } @@ -726,9 +754,9 @@ QueryTreeNodePtr QueryTreeBuilder::buildJoinTree(const ASTPtr & tables_in_select QueryTreeNodePtr join_expression; if (table_join.using_expression_list) - join_expression = buildExpressionList(table_join.using_expression_list); + join_expression = buildExpressionList(table_join.using_expression_list, context); else if (table_join.on_expression) - join_expression = buildExpression(table_join.on_expression); + join_expression = buildExpression(table_join.on_expression, context); const auto & settings = context->getSettingsRef(); auto join_default_strictness = settings.join_default_strictness; @@ -785,7 +813,7 @@ QueryTreeNodePtr QueryTreeBuilder::buildJoinTree(const ASTPtr & tables_in_select auto last_table_expression = std::move(table_expressions.back()); table_expressions.pop_back(); - auto array_join_expressions_list = buildExpressionList(array_join_expression.expression_list); + auto array_join_expressions_list = buildExpressionList(array_join_expression.expression_list, context); auto array_join_node = std::make_shared(std::move(last_table_expression), std::move(array_join_expressions_list), is_left_array_join); /** Original AST is not set because it will contain only array join part and does @@ -805,7 +833,7 @@ QueryTreeNodePtr QueryTreeBuilder::buildJoinTree(const ASTPtr & tables_in_select } -ColumnTransformersNodes QueryTreeBuilder::buildColumnTransformers(const ASTPtr & matcher_expression, size_t start_child_index) const +ColumnTransformersNodes QueryTreeBuilder::buildColumnTransformers(const ASTPtr & matcher_expression, size_t start_child_index, const ContextPtr & context) const { ColumnTransformersNodes column_transformers; size_t children_size = matcher_expression->children.size(); @@ -818,14 +846,14 @@ ColumnTransformersNodes QueryTreeBuilder::buildColumnTransformers(const ASTPtr & { if (apply_transformer->lambda) { - auto lambda_query_tree_node = buildExpression(apply_transformer->lambda); + auto lambda_query_tree_node = buildExpression(apply_transformer->lambda, context); column_transformers.emplace_back(std::make_shared(std::move(lambda_query_tree_node))); } else { auto function_node = std::make_shared(apply_transformer->func_name); if (apply_transformer->parameters) - function_node->getParametersNode() = buildExpressionList(apply_transformer->parameters); + function_node->getParametersNode() = buildExpressionList(apply_transformer->parameters, context); column_transformers.emplace_back(std::make_shared(std::move(function_node))); } @@ -856,7 +884,7 @@ ColumnTransformersNodes QueryTreeBuilder::buildColumnTransformers(const ASTPtr & for (const auto & replace_transformer_child : replace_transformer->children) { auto & replacement = replace_transformer_child->as(); - replacements.emplace_back(ReplaceColumnTransformerNode::Replacement{replacement.name, buildExpression(replacement.expr)}); + replacements.emplace_back(ReplaceColumnTransformerNode::Replacement{replacement.name, buildExpression(replacement.expr, context)}); } column_transformers.emplace_back(std::make_shared(replacements, replace_transformer->is_strict)); diff --git a/src/Analyzer/QueryTreeBuilder.h b/src/Analyzer/QueryTreeBuilder.h index de0f6270230..acff62e07c9 100644 --- a/src/Analyzer/QueryTreeBuilder.h +++ b/src/Analyzer/QueryTreeBuilder.h @@ -13,6 +13,8 @@ namespace DB * AST that represent query ASTSelectWithUnionQuery, ASTSelectIntersectExceptQuery, ASTSelectQuery. * AST that represent a list of expressions ASTExpressionList. * AST that represent expression ASTIdentifier, ASTAsterisk, ASTLiteral, ASTFunction. + * + * For QUERY and UNION nodes contexts are created with respect to specified SETTINGS. */ QueryTreeNodePtr buildQueryTree(ASTPtr query, ContextPtr context); diff --git a/src/Analyzer/UnionNode.cpp b/src/Analyzer/UnionNode.cpp index 19f94e68a33..18733b32437 100644 --- a/src/Analyzer/UnionNode.cpp +++ b/src/Analyzer/UnionNode.cpp @@ -3,8 +3,6 @@ #include #include -#include - #include #include #include @@ -18,9 +16,12 @@ #include #include +#include #include +#include + #include #include @@ -33,8 +34,9 @@ namespace ErrorCodes extern const int BAD_ARGUMENTS; } -UnionNode::UnionNode(SelectUnionMode union_mode_) +UnionNode::UnionNode(ContextMutablePtr context_, SelectUnionMode union_mode_) : IQueryTreeNode(children_size) + , context(std::move(context_)) , union_mode(union_mode_) { if (union_mode == SelectUnionMode::UNION_DEFAULT || @@ -129,7 +131,7 @@ void UnionNode::updateTreeHashImpl(HashState & state) const QueryTreeNodePtr UnionNode::cloneImpl() const { - auto result_union_node = std::make_shared(union_mode); + auto result_union_node = std::make_shared(context, union_mode); result_union_node->is_subquery = is_subquery; result_union_node->is_cte = is_cte; diff --git a/src/Analyzer/UnionNode.h b/src/Analyzer/UnionNode.h index c58d1bc0e12..5e3861da814 100644 --- a/src/Analyzer/UnionNode.h +++ b/src/Analyzer/UnionNode.h @@ -3,12 +3,14 @@ #include #include +#include + #include #include #include #include -#include +#include namespace DB { @@ -37,8 +39,26 @@ using UnionNodePtr = std::shared_ptr; class UnionNode final : public IQueryTreeNode { public: - /// Construct union node with normalized union mode - explicit UnionNode(SelectUnionMode union_mode_); + /// Construct union node with context and normalized union mode + explicit UnionNode(ContextMutablePtr context_, SelectUnionMode union_mode_); + + /// Get context + ContextPtr getContext() const + { + return context; + } + + /// Get mutable context + const ContextMutablePtr & getMutableContext() const + { + return context; + } + + /// Get mutable context + ContextMutablePtr & getMutableContext() + { + return context; + } /// Returns true if union node is subquery, false otherwise bool isSubquery() const @@ -129,6 +149,7 @@ private: bool is_subquery = false; bool is_cte = false; std::string cte_name; + ContextMutablePtr context; SelectUnionMode union_mode; static constexpr size_t queries_child_index = 0; diff --git a/src/Interpreters/InterpreterSelectQueryAnalyzer.cpp b/src/Interpreters/InterpreterSelectQueryAnalyzer.cpp index 61ec5932b7d..ad3ec9b3189 100644 --- a/src/Interpreters/InterpreterSelectQueryAnalyzer.cpp +++ b/src/Interpreters/InterpreterSelectQueryAnalyzer.cpp @@ -62,11 +62,11 @@ InterpreterSelectQueryAnalyzer::InterpreterSelectQueryAnalyzer( const ASTPtr & query_, const SelectQueryOptions & select_query_options_, ContextPtr context_) - : WithContext(context_) - , query(normalizeAndValidateQuery(query_)) + : query(normalizeAndValidateQuery(query_)) , query_tree(buildQueryTreeAndRunPasses(query, context_)) , select_query_options(select_query_options_) - , planner(query_tree, select_query_options, context_) + , context(std::move(context_)) + , planner(query_tree, select_query_options) { } @@ -74,11 +74,11 @@ InterpreterSelectQueryAnalyzer::InterpreterSelectQueryAnalyzer( const QueryTreeNodePtr & query_tree_, const SelectQueryOptions & select_query_options_, ContextPtr context_) - : WithContext(context_) - , query(query_tree_->toAST()) + : query(query_tree_->toAST()) , query_tree(query_tree_) , select_query_options(select_query_options_) - , planner(query_tree, select_query_options, context_) + , context(std::move(context_)) + , planner(query_tree, select_query_options) { } @@ -101,7 +101,7 @@ BlockIO InterpreterSelectQueryAnalyzer::execute() result.pipeline = QueryPipelineBuilder::getPipeline(std::move(*pipeline_builder)); if (!select_query_options.ignore_quota && (select_query_options.to_stage == QueryProcessingStage::Complete)) - result.pipeline.setQuota(getContext()->getQuota()); + result.pipeline.setQuota(context->getQuota()); return result; } diff --git a/src/Interpreters/InterpreterSelectQueryAnalyzer.h b/src/Interpreters/InterpreterSelectQueryAnalyzer.h index e9884567ab0..1dbe5ffccc6 100644 --- a/src/Interpreters/InterpreterSelectQueryAnalyzer.h +++ b/src/Interpreters/InterpreterSelectQueryAnalyzer.h @@ -12,7 +12,7 @@ namespace DB { -class InterpreterSelectQueryAnalyzer : public IInterpreter, public WithContext +class InterpreterSelectQueryAnalyzer : public IInterpreter { public: /// Initialize interpreter with query AST @@ -25,6 +25,16 @@ public: const SelectQueryOptions & select_query_options_, ContextPtr context_); + const ContextPtr & getContext() const + { + return context; + } + + ContextPtr & getContext() + { + return context; + } + Block getSampleBlock(); BlockIO execute() override; @@ -43,6 +53,7 @@ private: ASTPtr query; QueryTreeNodePtr query_tree; SelectQueryOptions select_query_options; + ContextPtr context; Planner planner; }; diff --git a/src/Planner/Planner.cpp b/src/Planner/Planner.cpp index aec5a578774..617f34e8fa9 100644 --- a/src/Planner/Planner.cpp +++ b/src/Planner/Planner.cpp @@ -142,13 +142,11 @@ void addBuildSubqueriesForSetsStepIfNeeded(QueryPlan & query_plan, const SelectQ if (!subquery_node) continue; - auto subquery_context = buildSubqueryContext(planner_context->getQueryContext()); auto subquery_options = select_query_options.subquery(); Planner subquery_planner( subquery_node, subquery_options, - std::move(subquery_context), planner_context->getGlobalPlannerContext()); subquery_planner.buildQueryPlanIfNeeded(); @@ -183,52 +181,26 @@ void extendQueryContextAndStoragesLifetime(QueryPlan & query_plan, const Planner } -Planner::Planner(const QueryTreeNodePtr & query_tree_, - const SelectQueryOptions & select_query_options_, - ContextPtr context_) - : query_tree(query_tree_) - , select_query_options(select_query_options_) - , planner_context(std::make_shared(std::move(context_), std::make_shared())) +PlannerContextPtr buildPlannerContext(const QueryTreeNodePtr & query_tree_node, + const SelectQueryOptions & select_query_options, + GlobalPlannerContextPtr global_planner_context) { - initialize(); -} + auto * query_node = query_tree_node->as(); + auto * union_node = query_tree_node->as(); -Planner::Planner(const QueryTreeNodePtr & query_tree_, - const SelectQueryOptions & select_query_options_, - ContextPtr context_, - GlobalPlannerContextPtr global_planner_context_) - : query_tree(query_tree_) - , select_query_options(select_query_options_) - , planner_context(std::make_shared(std::move(context_), std::move(global_planner_context_))) -{ - initialize(); -} - -void Planner::initialize() -{ - checkStackSize(); - - if (query_tree->getNodeType() != QueryTreeNodeType::QUERY && - query_tree->getNodeType() != QueryTreeNodeType::UNION) + if (!query_node && !union_node) throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "Expected QUERY or UNION node. Actual {}", - query_tree->formatASTForErrorMessage()); + query_tree_node->formatASTForErrorMessage()); - auto & query_context = planner_context->getQueryContext(); - - size_t max_subquery_depth = query_context->getSettingsRef().max_subquery_depth; + auto & mutable_context = query_node ? query_node->getMutableContext() : union_node->getMutableContext(); + size_t max_subquery_depth = mutable_context->getSettingsRef().max_subquery_depth; if (max_subquery_depth && select_query_options.subquery_depth > max_subquery_depth) throw Exception(ErrorCodes::TOO_DEEP_SUBQUERIES, "Too deep subqueries. Maximum: {}", max_subquery_depth); - auto * query_node = query_tree->as(); - if (!query_node) - return; - - bool need_apply_query_settings = query_node->hasSettingsChanges(); - - const auto & client_info = query_context->getClientInfo(); + const auto & client_info = mutable_context->getClientInfo(); auto min_major = static_cast(DBMS_MIN_MAJOR_VERSION_WITH_CURRENT_AGGREGATION_VARIANT_SELECTION_METHOD); auto min_minor = static_cast(DBMS_MIN_MINOR_VERSION_WITH_CURRENT_AGGREGATION_VARIANT_SELECTION_METHOD); @@ -236,22 +208,34 @@ void Planner::initialize() client_info.connection_client_version_major < min_major && client_info.connection_client_version_minor < min_minor; - if (need_apply_query_settings || need_to_disable_two_level_aggregation) + if (need_to_disable_two_level_aggregation) { - auto updated_context = Context::createCopy(query_context); - - if (need_apply_query_settings) - updated_context->applySettingsChanges(query_node->getSettingsChanges()); - /// Disable two-level aggregation due to version incompatibility - if (need_to_disable_two_level_aggregation) - { - updated_context->setSetting("group_by_two_level_threshold", Field(0)); - updated_context->setSetting("group_by_two_level_threshold_bytes", Field(0)); - } - - query_context = std::move(updated_context); + mutable_context->setSetting("group_by_two_level_threshold", Field(0)); + mutable_context->setSetting("group_by_two_level_threshold_bytes", Field(0)); } + + if (select_query_options.is_subquery) + updateContextForSubqueryExecution(mutable_context); + + return std::make_shared(mutable_context, std::move(global_planner_context)); +} + +Planner::Planner(const QueryTreeNodePtr & query_tree_, + const SelectQueryOptions & select_query_options_) + : query_tree(query_tree_) + , select_query_options(select_query_options_) + , planner_context(buildPlannerContext(query_tree, select_query_options, std::make_shared())) +{ +} + +Planner::Planner(const QueryTreeNodePtr & query_tree_, + const SelectQueryOptions & select_query_options_, + GlobalPlannerContextPtr global_planner_context_) + : query_tree(query_tree_) + , select_query_options(select_query_options_) + , planner_context(buildPlannerContext(query_tree_, select_query_options, std::move(global_planner_context_))) +{ } void Planner::buildQueryPlanIfNeeded() @@ -279,7 +263,7 @@ void Planner::buildQueryPlanIfNeeded() for (auto & query_node : union_query_tree->getQueries().getNodes()) { - Planner query_planner(query_node, select_query_options, query_context); + Planner query_planner(query_node, select_query_options); query_planner.buildQueryPlanIfNeeded(); auto query_node_plan = std::make_unique(std::move(query_planner).extractQueryPlan()); query_plans_headers.push_back(query_node_plan->getCurrentDataStream().header); diff --git a/src/Planner/Planner.h b/src/Planner/Planner.h index 03f8e19df56..de4ed5b92e9 100644 --- a/src/Planner/Planner.h +++ b/src/Planner/Planner.h @@ -21,13 +21,11 @@ class Planner public: /// Initialize planner with query tree after analysis phase Planner(const QueryTreeNodePtr & query_tree_, - const SelectQueryOptions & select_query_options_, - ContextPtr context_); + const SelectQueryOptions & select_query_options_); /// Initialize planner with query tree after query analysis phase and global planner context Planner(const QueryTreeNodePtr & query_tree_, const SelectQueryOptions & select_query_options_, - ContextPtr context_, GlobalPlannerContextPtr global_planner_context_); const QueryPlan & getQueryPlan() const @@ -48,8 +46,6 @@ public: } private: - void initialize(); - QueryTreeNodePtr query_tree; QueryPlan query_plan; SelectQueryOptions select_query_options; diff --git a/src/Planner/PlannerJoinTree.cpp b/src/Planner/PlannerJoinTree.cpp index 2fd469986ec..534556e80f9 100644 --- a/src/Planner/PlannerJoinTree.cpp +++ b/src/Planner/PlannerJoinTree.cpp @@ -174,8 +174,7 @@ QueryPlan buildQueryPlanForTableExpression(QueryTreeNodePtr table_expression, else if (query_node || union_node) { auto subquery_options = select_query_options.subquery(); - auto subquery_context = buildSubqueryContext(planner_context->getQueryContext()); - Planner subquery_planner(table_expression, subquery_options, std::move(subquery_context), planner_context->getGlobalPlannerContext()); + Planner subquery_planner(table_expression, subquery_options, planner_context->getGlobalPlannerContext()); subquery_planner.buildQueryPlanIfNeeded(); query_plan = std::move(subquery_planner).extractQueryPlan(); } diff --git a/src/Planner/Utils.cpp b/src/Planner/Utils.cpp index 74918285453..59d174c2877 100644 --- a/src/Planner/Utils.cpp +++ b/src/Planner/Utils.cpp @@ -100,7 +100,7 @@ ASTPtr queryNodeToSelectQuery(const QueryTreeNodePtr & query_node) /** There are no limits on the maximum size of the result for the subquery. * Since the result of the query is not the result of the entire query. */ -ContextPtr buildSubqueryContext(const ContextPtr & context) +void updateContextForSubqueryExecution(ContextMutablePtr & mutable_context) { /** The subquery in the IN / JOIN section does not have any restrictions on the maximum size of the result. * Because the result of this query is not the result of the entire query. @@ -109,15 +109,12 @@ ContextPtr buildSubqueryContext(const ContextPtr & context) * max_rows_in_join, max_bytes_in_join, join_overflow_mode, * which are checked separately (in the Set, Join objects). */ - auto subquery_context = Context::createCopy(context); - Settings subquery_settings = context->getSettings(); + Settings subquery_settings = mutable_context->getSettings(); subquery_settings.max_result_rows = 0; subquery_settings.max_result_bytes = 0; /// The calculation of extremes does not make sense and is not necessary (if you do it, then the extremes of the subquery can be taken for whole query). subquery_settings.extremes = false; - subquery_context->setSettings(subquery_settings); - - return subquery_context; + mutable_context->setSettings(subquery_settings); } namespace diff --git a/src/Planner/Utils.h b/src/Planner/Utils.h index 909cea3bf8f..da99a7e62df 100644 --- a/src/Planner/Utils.h +++ b/src/Planner/Utils.h @@ -34,6 +34,9 @@ ASTPtr queryNodeToSelectQuery(const QueryTreeNodePtr & query_node); /// Build context for subquery execution ContextPtr buildSubqueryContext(const ContextPtr & context); +/// Update mutable context for subquery execution +void updateContextForSubqueryExecution(ContextMutablePtr & mutable_context); + /// Build limits for storage StorageLimits buildStorageLimits(const Context & context, const SelectQueryOptions & options); From 3bdce39121a7cb54994282a92b0c5cd551e656e2 Mon Sep 17 00:00:00 2001 From: vdimir Date: Mon, 5 Dec 2022 17:54:12 +0000 Subject: [PATCH 057/127] Fix logical error in right storage join witn using --- src/Interpreters/HashJoin.cpp | 4 ++-- src/Interpreters/JoinUtils.cpp | 14 +++++++++++--- src/Interpreters/JoinUtils.h | 2 +- src/Interpreters/MergeJoin.cpp | 2 +- src/Interpreters/TableJoin.h | 2 +- .../02497_storage_join_right_assert.reference | 2 ++ .../02497_storage_join_right_assert.sql | 10 ++++++++++ 7 files changed, 28 insertions(+), 8 deletions(-) create mode 100644 tests/queries/0_stateless/02497_storage_join_right_assert.reference create mode 100644 tests/queries/0_stateless/02497_storage_join_right_assert.sql diff --git a/src/Interpreters/HashJoin.cpp b/src/Interpreters/HashJoin.cpp index 6f1634b4e39..8c3cd806ff1 100644 --- a/src/Interpreters/HashJoin.cpp +++ b/src/Interpreters/HashJoin.cpp @@ -1963,7 +1963,7 @@ IBlocksStreamPtr HashJoin::getNonJoinedBlocks(const Block & left_sample_block, /// ... calculate `left_columns_count` ... size_t left_columns_count = left_sample_block.columns(); auto non_joined = std::make_unique>(*this, max_block_size); - return std::make_unique(std::move(non_joined), result_sample_block, left_columns_count, table_join->leftToRightKeyRemap()); + return std::make_unique(std::move(non_joined), result_sample_block, left_columns_count, *table_join); } else @@ -1971,7 +1971,7 @@ IBlocksStreamPtr HashJoin::getNonJoinedBlocks(const Block & left_sample_block, size_t left_columns_count = left_sample_block.columns(); assert(left_columns_count == result_sample_block.columns() - required_right_keys.columns() - sample_block_with_columns_to_add.columns()); auto non_joined = std::make_unique>(*this, max_block_size); - return std::make_unique(std::move(non_joined), result_sample_block, left_columns_count, table_join->leftToRightKeyRemap()); + return std::make_unique(std::move(non_joined), result_sample_block, left_columns_count, *table_join); } } diff --git a/src/Interpreters/JoinUtils.cpp b/src/Interpreters/JoinUtils.cpp index d17d3c0d44e..a4ec64ab70e 100644 --- a/src/Interpreters/JoinUtils.cpp +++ b/src/Interpreters/JoinUtils.cpp @@ -718,11 +718,12 @@ ColumnPtr filterWithBlanks(ColumnPtr src_column, const IColumn::Filter & filter, NotJoinedBlocks::NotJoinedBlocks(std::unique_ptr filler_, const Block & result_sample_block_, size_t left_columns_count, - const LeftToRightKeyRemap & left_to_right_key_remap) + const TableJoin & table_join) : filler(std::move(filler_)) , saved_block_sample(filler->getEmptyBlock()) , result_sample_block(materializeBlock(result_sample_block_)) { + const auto & left_to_right_key_remap = table_join.leftToRightKeyRemap(); for (size_t left_pos = 0; left_pos < left_columns_count; ++left_pos) { /// We need right 'x' for 'RIGHT JOIN ... USING(x)' @@ -739,14 +740,21 @@ NotJoinedBlocks::NotJoinedBlocks(std::unique_ptr filler_, /// `saved_block_sample` may contains non unique column names, get any of them /// (e.g. in case of `... JOIN (SELECT a, a, b FROM table) as t2`) - for (const auto & [name, right_pos] : saved_block_sample.getNamesToIndexesMap()) + for (const auto & [right_name, right_pos] : saved_block_sample.getNamesToIndexesMap()) { + String column_name(right_name); + if (table_join.getStorageJoin()) + { + /// StorageJoin operates with original non qualified column names, so apply renaming here + column_name = table_join.renamedRightColumnName(column_name); + } + /// Start from left_columns_count to don't remap left keys twice. We need only qualified right keys here /// `result_sample_block` may contains non unique column names, need to set index for all of them for (size_t result_pos = left_columns_count; result_pos < result_sample_block.columns(); ++result_pos) { const auto & result_name = result_sample_block.getByPosition(result_pos).name; - if (result_name == name) + if (result_name == column_name) setRightIndex(right_pos, result_pos); } } diff --git a/src/Interpreters/JoinUtils.h b/src/Interpreters/JoinUtils.h index bcff6e60a9a..36be71f2a91 100644 --- a/src/Interpreters/JoinUtils.h +++ b/src/Interpreters/JoinUtils.h @@ -138,7 +138,7 @@ public: NotJoinedBlocks(std::unique_ptr filler_, const Block & result_sample_block_, size_t left_columns_count, - const LeftToRightKeyRemap & left_to_right_key_remap); + const TableJoin & table_join); Block nextImpl() override; diff --git a/src/Interpreters/MergeJoin.cpp b/src/Interpreters/MergeJoin.cpp index 191372cd545..2d54accc76a 100644 --- a/src/Interpreters/MergeJoin.cpp +++ b/src/Interpreters/MergeJoin.cpp @@ -1122,7 +1122,7 @@ IBlocksStreamPtr MergeJoin::getNonJoinedBlocks( size_t left_columns_count = left_sample_block.columns(); assert(left_columns_count == result_sample_block.columns() - right_columns_to_add.columns()); auto non_joined = std::make_unique(*this, max_block_size); - return std::make_unique(std::move(non_joined), result_sample_block, left_columns_count, table_join->leftToRightKeyRemap()); + return std::make_unique(std::move(non_joined), result_sample_block, left_columns_count, *table_join); } return nullptr; } diff --git a/src/Interpreters/TableJoin.h b/src/Interpreters/TableJoin.h index 316beccae80..356687f080c 100644 --- a/src/Interpreters/TableJoin.h +++ b/src/Interpreters/TableJoin.h @@ -346,7 +346,7 @@ public: void setStorageJoin(std::shared_ptr storage); void setStorageJoin(std::shared_ptr storage); - std::shared_ptr getStorageJoin() { return right_storage_join; } + std::shared_ptr getStorageJoin() const { return right_storage_join; } bool isSpecialStorage() const { return !right_storage_name.empty() || right_storage_join || right_kv_storage; } diff --git a/tests/queries/0_stateless/02497_storage_join_right_assert.reference b/tests/queries/0_stateless/02497_storage_join_right_assert.reference new file mode 100644 index 00000000000..b254a03f4ce --- /dev/null +++ b/tests/queries/0_stateless/02497_storage_join_right_assert.reference @@ -0,0 +1,2 @@ +2 2 2 +3 0 3 diff --git a/tests/queries/0_stateless/02497_storage_join_right_assert.sql b/tests/queries/0_stateless/02497_storage_join_right_assert.sql new file mode 100644 index 00000000000..ee9e8713d32 --- /dev/null +++ b/tests/queries/0_stateless/02497_storage_join_right_assert.sql @@ -0,0 +1,10 @@ +DROP TABLE IF EXISTS t1; +DROP TABLE IF EXISTS t2; + +CREATE TABLE t1 (key UInt64, a UInt64) ENGINE = Memory; +CREATE TABLE t2 (key UInt64, a UInt64) ENGINE = Join(ALL, RIGHT, key); + +INSERT INTO t1 VALUES (1, 1), (2, 2); +INSERT INTO t2 VALUES (2, 2), (3, 3); + +SELECT * FROM t1 ALL RIGHT JOIN t2 USING (key) ORDER BY key; From 862da8f459b1a1cafb50748567480ef4098844b8 Mon Sep 17 00:00:00 2001 From: kssenii Date: Tue, 6 Dec 2022 16:04:26 +0100 Subject: [PATCH 058/127] Fix --- src/Interpreters/TreeRewriter.cpp | 56 ++++++++++++------- .../02006_test_positional_arguments.reference | 18 +++++- .../02006_test_positional_arguments.sql | 7 +++ 3 files changed, 61 insertions(+), 20 deletions(-) diff --git a/src/Interpreters/TreeRewriter.cpp b/src/Interpreters/TreeRewriter.cpp index 828f332af1d..6b019609dd9 100644 --- a/src/Interpreters/TreeRewriter.cpp +++ b/src/Interpreters/TreeRewriter.cpp @@ -40,6 +40,7 @@ #include #include #include +#include #include #include @@ -325,6 +326,39 @@ struct ExistsExpressionData using ExistsExpressionVisitor = InDepthNodeVisitor, false>; +struct ReplacePositionalArgumentsData +{ + using TypeToVisit = ASTSelectQuery; + ContextPtr context; + + void visit(ASTSelectQuery & select_query, ASTPtr &) const + { + if (context->getSettingsRef().enable_positional_arguments) + { + if (select_query.groupBy()) + { + for (auto & expr : select_query.groupBy()->children) + replaceForPositionalArguments(expr, &select_query, ASTSelectQuery::Expression::GROUP_BY); + } + if (select_query.orderBy()) + { + for (auto & expr : select_query.orderBy()->children) + { + auto & elem = assert_cast(*expr).children.at(0); + replaceForPositionalArguments(elem, &select_query, ASTSelectQuery::Expression::ORDER_BY); + } + } + if (select_query.limitBy()) + { + for (auto & expr : select_query.limitBy()->children) + replaceForPositionalArguments(expr, &select_query, ASTSelectQuery::Expression::LIMIT_BY); + } + } + } +}; + +using ReplacePositionalArgumentsVisitor = InDepthNodeVisitor, false>; + /// Translate qualified names such as db.table.column, table.column, table_alias.column to names' normal form. /// Expand asterisks and qualified asterisks with column names. /// There would be columns in normal form & column aliases after translation. Column & column alias would be normalized in QueryNormalizer. @@ -1316,25 +1350,6 @@ TreeRewriterResultPtr TreeRewriter::analyzeSelect( all_source_columns_set.insert(name); } - if (getContext()->getSettingsRef().enable_positional_arguments) - { - if (select_query->groupBy()) - { - for (auto & expr : select_query->groupBy()->children) - replaceForPositionalArguments(expr, select_query, ASTSelectQuery::Expression::GROUP_BY); - } - if (select_query->orderBy()) - { - for (auto & expr : select_query->orderBy()->children) - replaceForPositionalArguments(expr, select_query, ASTSelectQuery::Expression::ORDER_BY); - } - if (select_query->limitBy()) - { - for (auto & expr : select_query->limitBy()->children) - replaceForPositionalArguments(expr, select_query, ASTSelectQuery::Expression::LIMIT_BY); - } - } - normalize(query, result.aliases, all_source_columns_set, select_options.ignore_alias, settings, /* allow_self_aliases = */ true, getContext()); // expand GROUP BY ALL @@ -1493,6 +1508,9 @@ void TreeRewriter::normalize( ExistsExpressionVisitor::Data exists; ExistsExpressionVisitor(exists).visit(query); + ReplacePositionalArgumentsVisitor::Data data_replace_positional_arguments{context_}; + ReplacePositionalArgumentsVisitor(data_replace_positional_arguments).visit(query); + if (settings.transform_null_in) { CustomizeInVisitor::Data data_null_in{"nullIn"}; diff --git a/tests/queries/0_stateless/02006_test_positional_arguments.reference b/tests/queries/0_stateless/02006_test_positional_arguments.reference index 56817961b30..e2bbea2149d 100644 --- a/tests/queries/0_stateless/02006_test_positional_arguments.reference +++ b/tests/queries/0_stateless/02006_test_positional_arguments.reference @@ -119,9 +119,25 @@ select b from (select 5 as a, 'Hello' as b order by 1); Hello drop table if exists tp2; create table tp2(first_col String, second_col Int32) engine = MergeTree() order by tuple(); +insert into tp2 select 'bbb', 1; +insert into tp2 select 'aaa', 2; select count(*) from (select first_col, count(second_col) from tp2 group by 1); -0 +2 select total from (select first_col, count(second_col) as total from tp2 group by 1); +1 +1 +select first_col from (select first_col, second_col as total from tp2 order by 1 desc); +bbb +aaa +select first_col from (select first_col, second_col as total from tp2 order by 2 desc); +aaa +bbb +select max from (select max(first_col) as max, second_col as total from tp2 group by 2) order by 1; +aaa +bbb +with res as (select first_col from (select first_col, second_col as total from tp2 order by 2 desc) limit 1) +select * from res; +aaa drop table if exists test; create table test ( diff --git a/tests/queries/0_stateless/02006_test_positional_arguments.sql b/tests/queries/0_stateless/02006_test_positional_arguments.sql index 8829a204ab6..67f4fe24c55 100644 --- a/tests/queries/0_stateless/02006_test_positional_arguments.sql +++ b/tests/queries/0_stateless/02006_test_positional_arguments.sql @@ -51,8 +51,15 @@ select b from (select 5 as a, 'Hello' as b order by 1); drop table if exists tp2; create table tp2(first_col String, second_col Int32) engine = MergeTree() order by tuple(); +insert into tp2 select 'bbb', 1; +insert into tp2 select 'aaa', 2; select count(*) from (select first_col, count(second_col) from tp2 group by 1); select total from (select first_col, count(second_col) as total from tp2 group by 1); +select first_col from (select first_col, second_col as total from tp2 order by 1 desc); +select first_col from (select first_col, second_col as total from tp2 order by 2 desc); +select max from (select max(first_col) as max, second_col as total from tp2 group by 2) order by 1; +with res as (select first_col from (select first_col, second_col as total from tp2 order by 2 desc) limit 1) +select * from res; drop table if exists test; create table test From 1b5b43ec87f86d16656b6e3e33fbe8d3424b2483 Mon Sep 17 00:00:00 2001 From: serxa Date: Tue, 6 Dec 2022 19:09:18 +0000 Subject: [PATCH 059/127] fix checks on CREATE + added resolveSetting.h + improved system table --- src/Access/AccessControl.cpp | 10 +-- src/Access/SettingsConstraints.cpp | 76 ++++++++----------- src/Access/SettingsConstraints.h | 7 +- src/Access/UsersConfigAccessStorage.cpp | 7 +- src/Access/resolveSetting.h | 73 ++++++++++++++++++ src/Storages/MergeTree/MergeTreeData.cpp | 8 +- .../MergeTree/registerStorageMergeTree.cpp | 19 +++-- .../System/StorageSystemMergeTreeSettings.cpp | 30 +++++++- 8 files changed, 155 insertions(+), 75 deletions(-) create mode 100644 src/Access/resolveSetting.h diff --git a/src/Access/AccessControl.cpp b/src/Access/AccessControl.cpp index ad1b7b2ca63..f40eb5250e4 100644 --- a/src/Access/AccessControl.cpp +++ b/src/Access/AccessControl.cpp @@ -16,6 +16,7 @@ #include #include #include +#include #include #include #include @@ -39,11 +40,6 @@ namespace ErrorCodes extern const int AUTHENTICATION_FAILED; } -namespace -{ - constexpr std::string_view MERGE_TREE_SETTINGS_PREFIX = "merge_tree_"; -} - namespace { void checkForUsersNotInMainConfig( @@ -108,9 +104,7 @@ public: bool isSettingNameAllowed(std::string_view setting_name) const { - if (Settings::hasBuiltin(setting_name)) - return true; - if (setting_name.starts_with(MERGE_TREE_SETTINGS_PREFIX) && MergeTreeSettings::hasBuiltin(setting_name.substr(MERGE_TREE_SETTINGS_PREFIX.size()))) + if (settingIsBuiltin(setting_name)) return true; std::lock_guard lock{mutex}; diff --git a/src/Access/SettingsConstraints.cpp b/src/Access/SettingsConstraints.cpp index 217f48d0fa8..92687595371 100644 --- a/src/Access/SettingsConstraints.cpp +++ b/src/Access/SettingsConstraints.cpp @@ -1,5 +1,6 @@ #include #include +#include #include #include #include @@ -19,12 +20,6 @@ namespace ErrorCodes extern const int UNKNOWN_SETTING; } -namespace -{ - constexpr std::string_view MERGE_TREE_SETTINGS_PREFIX = "merge_tree_"; -} - - SettingsConstraints::SettingsConstraints(const AccessControl & access_control_) : access_control(&access_control_) { } @@ -41,30 +36,28 @@ void SettingsConstraints::clear() constraints.clear(); } -void SettingsConstraints::set(const String & setting_name, const Field & min_value, const Field & max_value, SettingConstraintWritability writability) +void SettingsConstraints::set(const String & full_name, const Field & min_value, const Field & max_value, SettingConstraintWritability writability) { - auto & constraint = constraints[setting_name]; - if (setting_name.starts_with(MERGE_TREE_SETTINGS_PREFIX)) - { - std::string_view name = static_cast(setting_name).substr(MERGE_TREE_SETTINGS_PREFIX.size()); - if (!min_value.isNull()) - constraint.min_value = MergeTreeSettings::castValueUtil(name, min_value); - if (!max_value.isNull()) - constraint.max_value = MergeTreeSettings::castValueUtil(name, max_value); - } - else - { - if (!min_value.isNull()) - constraint.min_value = Settings::castValueUtil(setting_name, min_value); - if (!max_value.isNull()) - constraint.max_value = Settings::castValueUtil(setting_name, max_value); - } + auto & constraint = constraints[full_name]; + if (!min_value.isNull()) + constraint.min_value = settingCastValueUtil(full_name, min_value); + if (!max_value.isNull()) + constraint.max_value = settingCastValueUtil(full_name, max_value); constraint.writability = writability; } -void SettingsConstraints::get(const Settings & current_settings, std::string_view setting_name, Field & min_value, Field & max_value, SettingConstraintWritability & writability) const +void SettingsConstraints::get(const Settings & current_settings, std::string_view short_name, Field & min_value, Field & max_value, SettingConstraintWritability & writability) const { - auto checker = getChecker(current_settings, setting_name); + // NOTE: for `Settings` short name is equal to full name + auto checker = getChecker(current_settings, short_name); + min_value = checker.constraint.min_value; + max_value = checker.constraint.max_value; + writability = checker.constraint.writability; +} + +void SettingsConstraints::get(const MergeTreeSettings &, std::string_view short_name, Field & min_value, Field & max_value, SettingConstraintWritability & writability) const +{ + auto checker = getMergeTreeChecker(short_name); min_value = checker.constraint.min_value; max_value = checker.constraint.max_value; writability = checker.constraint.writability; @@ -201,19 +194,10 @@ bool SettingsConstraints::checkImpl(const Settings & current_settings, SettingCh bool SettingsConstraints::checkImpl(const MergeTreeSettings & current_settings, SettingChange & change, ReactionOnViolation reaction) const { - String prefixed_name(MERGE_TREE_SETTINGS_PREFIX); - prefixed_name += change.name; // Just because you cannot concatenate `std::string_view` and `std::string` using operator+ in C++20 yet - - if (reaction == THROW_ON_VIOLATION) - access_control->checkSettingNameIsAllowed(prefixed_name); - else if (!access_control->isSettingNameAllowed(prefixed_name)) - return false; - Field new_value; if (!getNewValueToCheck(current_settings, change, new_value, reaction == THROW_ON_VIOLATION)) return false; - - return getMergeTreeChecker(prefixed_name).check(change, new_value, reaction); + return getMergeTreeChecker(change.name).check(change, new_value, reaction); } bool SettingsConstraints::Checker::check(SettingChange & change, const Field & new_value, ReactionOnViolation reaction) const @@ -224,16 +208,13 @@ bool SettingsConstraints::Checker::check(SettingChange & change, const Field & n { if (reaction == THROW_ON_VIOLATION) return applyVisitor(FieldVisitorAccurateLess{}, left, right); - else + try { - try - { - return applyVisitor(FieldVisitorAccurateLess{}, left, right); - } - catch (...) - { - return true; - } + return applyVisitor(FieldVisitorAccurateLess{}, left, right); + } + catch (...) + { + return true; } }; @@ -319,9 +300,12 @@ SettingsConstraints::Checker SettingsConstraints::getChecker(const Settings & cu return Checker(it->second); } -SettingsConstraints::Checker SettingsConstraints::getMergeTreeChecker(std::string_view setting_name) const +SettingsConstraints::Checker SettingsConstraints::getMergeTreeChecker(std::string_view short_name) const { - auto it = constraints.find(setting_name); + String full_name(MERGE_TREE_SETTINGS_PREFIX); + full_name += short_name; // Just because you cannot concatenate `std::string_view` and `std::string` using operator+ in C++20 yet + + auto it = constraints.find(full_name); if (it == constraints.end()) return Checker(); // Allowed return Checker(it->second); diff --git a/src/Access/SettingsConstraints.h b/src/Access/SettingsConstraints.h index 4234662055f..6cba248f633 100644 --- a/src/Access/SettingsConstraints.h +++ b/src/Access/SettingsConstraints.h @@ -66,8 +66,9 @@ public: void clear(); bool empty() const { return constraints.empty(); } - void set(const String & setting_name, const Field & min_value, const Field & max_value, SettingConstraintWritability writability); - void get(const Settings & current_settings, std::string_view setting_name, Field & min_value, Field & max_value, SettingConstraintWritability & writability) const; + void set(const String & full_name, const Field & min_value, const Field & max_value, SettingConstraintWritability writability); + void get(const Settings & current_settings, std::string_view short_name, Field & min_value, Field & max_value, SettingConstraintWritability & writability) const; + void get(const MergeTreeSettings & current_settings, std::string_view short_name, Field & min_value, Field & max_value, SettingConstraintWritability & writability) const; void merge(const SettingsConstraints & other); @@ -76,7 +77,7 @@ public: void check(const Settings & current_settings, const SettingsChanges & changes) const; void check(const Settings & current_settings, SettingsChanges & changes) const; - /// Checks whether `change` with added "merge_tree_" name prefix violates these constraints and throws an exception if so. + /// Checks whether `change` violates these constraints and throws an exception if so. (setting short name is expected inside `changes`) void check(const MergeTreeSettings & current_settings, const SettingChange & change) const; void check(const MergeTreeSettings & current_settings, const SettingsChanges & changes) const; diff --git a/src/Access/UsersConfigAccessStorage.cpp b/src/Access/UsersConfigAccessStorage.cpp index e3690f36cc3..1feb687d1d3 100644 --- a/src/Access/UsersConfigAccessStorage.cpp +++ b/src/Access/UsersConfigAccessStorage.cpp @@ -4,6 +4,7 @@ #include #include #include +#include #include #include #include @@ -451,9 +452,9 @@ namespace for (const String & constraint_type : constraint_types) { if (constraint_type == "min") - profile_element.min_value = Settings::stringToValueUtil(setting_name, config.getString(path_to_name + "." + constraint_type)); + profile_element.min_value = settingStringToValueUtil(setting_name, config.getString(path_to_name + "." + constraint_type)); else if (constraint_type == "max") - profile_element.max_value = Settings::stringToValueUtil(setting_name, config.getString(path_to_name + "." + constraint_type)); + profile_element.max_value = settingStringToValueUtil(setting_name, config.getString(path_to_name + "." + constraint_type)); else if (constraint_type == "readonly" || constraint_type == "const") { writability_count++; @@ -517,7 +518,7 @@ namespace SettingsProfileElement profile_element; profile_element.setting_name = setting_name; - profile_element.value = Settings::stringToValueUtil(setting_name, config.getString(profile_config + "." + key)); + profile_element.value = settingStringToValueUtil(setting_name, config.getString(profile_config + "." + key)); profile->elements.emplace_back(std::move(profile_element)); } diff --git a/src/Access/resolveSetting.h b/src/Access/resolveSetting.h new file mode 100644 index 00000000000..234715dde9b --- /dev/null +++ b/src/Access/resolveSetting.h @@ -0,0 +1,73 @@ +#pragma once + +#include +#include + +// +// Settings from different classes (Settings, MergeTreeSettings) can coexist in the same "namespace". +// This is, for example, required to define settings constraints inside user profiles. +// `resolveSetting(full_name)` is used to resolve setting name and choose which class is to be used. +// Templated lambda syntax should be used: +// +// return resolveSetting(name, [] (std::string_view name, Overload) -> auto +// { +// return T::castValueUtil(name, value); // T will be deduced into `Settings`, `MergeTreeSettings`, ... +// }); +// + +namespace DB +{ + +constexpr std::string_view MERGE_TREE_SETTINGS_PREFIX = "merge_tree_"; + +template struct Overload {}; + +// Resolve setting name and call function `f` back with short name and class +template +auto resolveSetting(std::string_view full_name, F && f) +{ + if (full_name.starts_with(MERGE_TREE_SETTINGS_PREFIX)) + { + std::string_view short_name = static_cast(full_name).substr(MERGE_TREE_SETTINGS_PREFIX.size()); + if (MergeTreeSettings::hasBuiltin(short_name)) // Check is required because `Settings` also contain names starting with 'merge_tree_' prefix + return f(short_name, Overload()); + } + // NOTE: other setting name resolution rules are to be added here + + // If no rule works - use global namespace + return f(full_name, Overload()); +} + +inline Field settingCastValueUtil(std::string_view full_name, const Field & value) +{ + return resolveSetting(full_name, [&] (std::string_view short_name, Overload) + { + return T::castValueUtil(short_name, value); + }); +} + +inline String settingValueToStringUtil(std::string_view full_name, const Field & value) +{ + return resolveSetting(full_name, [&] (std::string_view short_name, Overload) + { + return T::valueToStringUtil(short_name, value); + }); +} + +inline Field settingStringToValueUtil(std::string_view full_name, const String & str) +{ + return resolveSetting(full_name, [&] (std::string_view short_name, Overload) + { + return T::stringToValueUtil(short_name, str); + }); +} + +inline bool settingIsBuiltin(std::string_view full_name) +{ + return resolveSetting(full_name, [&] (std::string_view short_name, Overload) + { + return T::hasBuiltin(short_name); + }); +} + +} diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index ee5ff85a4b1..e1239fed5c5 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -246,11 +246,7 @@ MergeTreeData::MergeTreeData( /// Check sanity of MergeTreeSettings. Only when table is created. if (!attach) - { - const auto & changes = metadata_.settings_changes->as().changes; - getContext()->checkMergeTreeSettingsConstraints(getContext()->getMergeTreeSettings(), changes); settings->sanityCheck(getContext()->getMergeMutateExecutor()->getMaxTasksCount()); - } MergeTreeDataFormatVersion min_format_version(0); if (!date_column_name.empty()) @@ -2669,7 +2665,7 @@ void MergeTreeData::checkAlterIsPossible(const AlterCommands & commands, Context { const auto current_changes = old_metadata.getSettingsChanges()->as().changes; const auto & new_changes = new_metadata.settings_changes->as().changes; - getContext()->checkMergeTreeSettingsConstraints(*settings_from_storage, new_changes); + getContext()->checkMergeTreeSettingsConstraints(*settings_from_storage, new_changes); // TODO(serxa): use LocalContext() instead for (const auto & changed_setting : new_changes) { @@ -2828,7 +2824,7 @@ void MergeTreeData::changeSettings( bool has_storage_policy_changed = false; const auto & new_changes = new_settings->as().changes; - getContext()->checkMergeTreeSettingsConstraints(*getSettings(), new_changes); + getContext()->checkMergeTreeSettingsConstraints(*getSettings(), new_changes); // TODO(serxa): use LocalContext() instead for (const auto & change : new_changes) { diff --git a/src/Storages/MergeTree/registerStorageMergeTree.cpp b/src/Storages/MergeTree/registerStorageMergeTree.cpp index 05aa8f469b0..ae2abaf8ea5 100644 --- a/src/Storages/MergeTree/registerStorageMergeTree.cpp +++ b/src/Storages/MergeTree/registerStorageMergeTree.cpp @@ -517,11 +517,8 @@ static StoragePtr create(const StorageFactory::Arguments & args) metadata.setColumns(columns); metadata.setComment(args.comment); - std::unique_ptr storage_settings; - if (replicated) - storage_settings = std::make_unique(args.getContext()->getReplicatedMergeTreeSettings()); - else - storage_settings = std::make_unique(args.getContext()->getMergeTreeSettings()); + const auto & initial_storage_settings = replicated ? args.getContext()->getReplicatedMergeTreeSettings() : args.getContext()->getMergeTreeSettings(); + std::unique_ptr storage_settings = std::make_unique(initial_storage_settings); if (is_extended_storage_def) { @@ -609,7 +606,11 @@ static StoragePtr create(const StorageFactory::Arguments & args) // updates the default storage_settings with settings specified via SETTINGS arg in a query if (args.storage_def->settings) + { + if (!args.attach) + args.getLocalContext()->checkMergeTreeSettingsConstraints(initial_storage_settings, args.storage_def->settings->changes); metadata.settings_changes = args.storage_def->settings->ptr(); + } } else { @@ -658,7 +659,15 @@ static StoragePtr create(const StorageFactory::Arguments & args) const auto * ast = engine_args[arg_num]->as(); if (ast && ast->value.getType() == Field::Types::UInt64) + { storage_settings->index_granularity = ast->value.safeGet(); + if (!args.attach) + { + SettingsChanges changes; + changes.emplace_back("index_granularity", Field(storage_settings->index_granularity)); + args.getLocalContext()->checkMergeTreeSettingsConstraints(initial_storage_settings, changes); + } + } else throw Exception( "Index granularity must be a positive integer" + getMergeTreeVerboseHelp(is_extended_storage_def), diff --git a/src/Storages/System/StorageSystemMergeTreeSettings.cpp b/src/Storages/System/StorageSystemMergeTreeSettings.cpp index 626319af63f..6de3fb800f4 100644 --- a/src/Storages/System/StorageSystemMergeTreeSettings.cpp +++ b/src/Storages/System/StorageSystemMergeTreeSettings.cpp @@ -1,7 +1,9 @@ #include #include +#include #include #include +#include namespace DB @@ -15,6 +17,9 @@ NamesAndTypesList SystemMergeTreeSettings::getNamesAndTypes() {"value", std::make_shared()}, {"changed", std::make_shared()}, {"description", std::make_shared()}, + {"min", std::make_shared(std::make_shared())}, + {"max", std::make_shared(std::make_shared())}, + {"readonly", std::make_shared()}, {"type", std::make_shared()}, }; } @@ -22,14 +27,31 @@ NamesAndTypesList SystemMergeTreeSettings::getNamesAndTypes() template void SystemMergeTreeSettings::fillData(MutableColumns & res_columns, ContextPtr context, const SelectQueryInfo &) const { - const auto & settings = replicated ? context->getReplicatedMergeTreeSettings().all() : context->getMergeTreeSettings().all(); - for (const auto & setting : settings) + const auto & settings = replicated ? context->getReplicatedMergeTreeSettings() : context->getMergeTreeSettings(); + auto constraints_and_current_profiles = context->getSettingsConstraintsAndCurrentProfiles(); + const auto & constraints = constraints_and_current_profiles->constraints; + for (const auto & setting : settings.all()) { - res_columns[0]->insert(setting.getName()); + const auto & setting_name = setting.getName(); + res_columns[0]->insert(setting_name); res_columns[1]->insert(setting.getValueString()); res_columns[2]->insert(setting.isValueChanged()); res_columns[3]->insert(setting.getDescription()); - res_columns[4]->insert(setting.getTypeName()); + + Field min, max; + SettingConstraintWritability writability = SettingConstraintWritability::WRITABLE; + constraints.get(settings, setting_name, min, max, writability); + + /// These two columns can accept strings only. + if (!min.isNull()) + min = Settings::valueToStringUtil(setting_name, min); + if (!max.isNull()) + max = Settings::valueToStringUtil(setting_name, max); + + res_columns[4]->insert(min); + res_columns[5]->insert(max); + res_columns[6]->insert(writability == SettingConstraintWritability::CONST); + res_columns[7]->insert(setting.getTypeName()); } } From 2306bd74d7eda92388d267eda5e49bf0b3b4a4e3 Mon Sep 17 00:00:00 2001 From: serxa Date: Tue, 6 Dec 2022 20:46:10 +0000 Subject: [PATCH 060/127] fix for ALTER TABLE ... MODIFY SETTING name=value --- src/Storages/MergeTree/MergeTreeData.cpp | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index e1239fed5c5..7737f487799 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -2665,7 +2665,7 @@ void MergeTreeData::checkAlterIsPossible(const AlterCommands & commands, Context { const auto current_changes = old_metadata.getSettingsChanges()->as().changes; const auto & new_changes = new_metadata.settings_changes->as().changes; - getContext()->checkMergeTreeSettingsConstraints(*settings_from_storage, new_changes); // TODO(serxa): use LocalContext() instead + local_context->checkMergeTreeSettingsConstraints(*settings_from_storage, new_changes); for (const auto & changed_setting : new_changes) { @@ -2824,7 +2824,6 @@ void MergeTreeData::changeSettings( bool has_storage_policy_changed = false; const auto & new_changes = new_settings->as().changes; - getContext()->checkMergeTreeSettingsConstraints(*getSettings(), new_changes); // TODO(serxa): use LocalContext() instead for (const auto & change : new_changes) { From 5206816b6cd01d9f53324cc14c56fea668c1cf7c Mon Sep 17 00:00:00 2001 From: jinjunzh Date: Tue, 6 Dec 2022 16:10:13 -0500 Subject: [PATCH 061/127] Update contrib/qpl to latest v0.3.0 --- contrib/qpl | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/qpl b/contrib/qpl index cdc8442f7a5..becb7a1b15b 160000 --- a/contrib/qpl +++ b/contrib/qpl @@ -1 +1 @@ -Subproject commit cdc8442f7a5e7a6ff6eea39c69665e0c5034d85d +Subproject commit becb7a1b15bdb4845ec3721a550707ffa51d029d From 57ca6ad60b386a57f60d792e5bbfd9d965d54809 Mon Sep 17 00:00:00 2001 From: Nikolay Degterinsky <43110995+evillique@users.noreply.github.com> Date: Wed, 7 Dec 2022 00:40:41 +0100 Subject: [PATCH 062/127] Make massages about GCC fatal --- cmake/tools.cmake | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/cmake/tools.cmake b/cmake/tools.cmake index e8fecd9f30b..6616cbec71d 100644 --- a/cmake/tools.cmake +++ b/cmake/tools.cmake @@ -21,12 +21,13 @@ set (APPLE_CLANG_MINIMUM_VERSION 12.0.0) set (GCC_MINIMUM_VERSION 11) if (COMPILER_GCC) + // ClickHouse really does not support GCC. + message (FATAL_ERROR "Compilation with GCC is unsupported. Please use Clang instead.") + if (CMAKE_CXX_COMPILER_VERSION VERSION_LESS ${GCC_MINIMUM_VERSION}) message (FATAL_ERROR "Compilation with GCC version ${CMAKE_CXX_COMPILER_VERSION} is unsupported, the minimum required version is ${GCC_MINIMUM_VERSION}.") endif () - message (WARNING "Compilation with GCC is unsupported. Please use Clang instead.") - elseif (COMPILER_CLANG) if (CMAKE_CXX_COMPILER_ID MATCHES "AppleClang") # (Experimental!) Specify "-DALLOW_APPLECLANG=ON" when running CMake configuration step, if you want to experiment with using it. @@ -83,7 +84,7 @@ if ((OS_LINUX OR OS_DARWIN) AND NOT LINKER_NAME) if (NOT LINKER_NAME) if (GOLD_PATH) - message (WARNING "Linking with gold is not recommended. Please use lld.") + message (FATAL_ERROR "Linking with gold is not recommended. Please use lld.") if (COMPILER_GCC) set (LINKER_NAME "gold") else () From bfccdb130e516ff2238e9471bc4627ce308f4e48 Mon Sep 17 00:00:00 2001 From: Nikolay Degterinsky <43110995+evillique@users.noreply.github.com> Date: Wed, 7 Dec 2022 00:43:37 +0100 Subject: [PATCH 063/127] Small typo --- cmake/tools.cmake | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/cmake/tools.cmake b/cmake/tools.cmake index 6616cbec71d..3ddf8a869be 100644 --- a/cmake/tools.cmake +++ b/cmake/tools.cmake @@ -21,7 +21,6 @@ set (APPLE_CLANG_MINIMUM_VERSION 12.0.0) set (GCC_MINIMUM_VERSION 11) if (COMPILER_GCC) - // ClickHouse really does not support GCC. message (FATAL_ERROR "Compilation with GCC is unsupported. Please use Clang instead.") if (CMAKE_CXX_COMPILER_VERSION VERSION_LESS ${GCC_MINIMUM_VERSION}) @@ -84,7 +83,7 @@ if ((OS_LINUX OR OS_DARWIN) AND NOT LINKER_NAME) if (NOT LINKER_NAME) if (GOLD_PATH) - message (FATAL_ERROR "Linking with gold is not recommended. Please use lld.") + message (FATAL_ERROR "Linking with gold is unsupported. Please use lld.") if (COMPILER_GCC) set (LINKER_NAME "gold") else () From 8e07e37fe78b83a07fc5b8320b3dd9b2837a476b Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Wed, 7 Dec 2022 09:38:03 +0100 Subject: [PATCH 064/127] Update FunctionsBinaryRepresentation.cpp --- src/Functions/FunctionsBinaryRepresentation.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Functions/FunctionsBinaryRepresentation.cpp b/src/Functions/FunctionsBinaryRepresentation.cpp index 7542ce4bb03..f71f05bbf34 100644 --- a/src/Functions/FunctionsBinaryRepresentation.cpp +++ b/src/Functions/FunctionsBinaryRepresentation.cpp @@ -78,7 +78,7 @@ struct HexImpl } else { - auto start_pos = pos; + const auto * start_pos = pos; pos = end - 1; while (pos >= start_pos) { @@ -212,7 +212,7 @@ struct BinImpl } else { - auto start_pos = pos; + const auto * start_pos = pos; pos = end - 1; while (pos >= start_pos) { From 3725cf4aa7a9aa022b11d08b2f8e4f7f0dd70283 Mon Sep 17 00:00:00 2001 From: kssenii Date: Wed, 7 Dec 2022 11:26:32 +0100 Subject: [PATCH 065/127] Better --- src/Interpreters/TreeRewriter.cpp | 41 +++++++++++++++---------------- 1 file changed, 20 insertions(+), 21 deletions(-) diff --git a/src/Interpreters/TreeRewriter.cpp b/src/Interpreters/TreeRewriter.cpp index 6b019609dd9..805392a9430 100644 --- a/src/Interpreters/TreeRewriter.cpp +++ b/src/Interpreters/TreeRewriter.cpp @@ -329,31 +329,27 @@ using ExistsExpressionVisitor = InDepthNodeVisitorgetSettingsRef().enable_positional_arguments) + if (select_query.groupBy()) { - if (select_query.groupBy()) + for (auto & expr : select_query.groupBy()->children) + replaceForPositionalArguments(expr, &select_query, ASTSelectQuery::Expression::GROUP_BY); + } + if (select_query.orderBy()) + { + for (auto & expr : select_query.orderBy()->children) { - for (auto & expr : select_query.groupBy()->children) - replaceForPositionalArguments(expr, &select_query, ASTSelectQuery::Expression::GROUP_BY); - } - if (select_query.orderBy()) - { - for (auto & expr : select_query.orderBy()->children) - { - auto & elem = assert_cast(*expr).children.at(0); - replaceForPositionalArguments(elem, &select_query, ASTSelectQuery::Expression::ORDER_BY); - } - } - if (select_query.limitBy()) - { - for (auto & expr : select_query.limitBy()->children) - replaceForPositionalArguments(expr, &select_query, ASTSelectQuery::Expression::LIMIT_BY); + auto & elem = assert_cast(*expr).children.at(0); + replaceForPositionalArguments(elem, &select_query, ASTSelectQuery::Expression::ORDER_BY); } } + if (select_query.limitBy()) + { + for (auto & expr : select_query.limitBy()->children) + replaceForPositionalArguments(expr, &select_query, ASTSelectQuery::Expression::LIMIT_BY); + } } }; @@ -1508,8 +1504,11 @@ void TreeRewriter::normalize( ExistsExpressionVisitor::Data exists; ExistsExpressionVisitor(exists).visit(query); - ReplacePositionalArgumentsVisitor::Data data_replace_positional_arguments{context_}; - ReplacePositionalArgumentsVisitor(data_replace_positional_arguments).visit(query); + if (context_->getSettingsRef().enable_positional_arguments) + { + ReplacePositionalArgumentsVisitor::Data data_replace_positional_arguments; + ReplacePositionalArgumentsVisitor(data_replace_positional_arguments).visit(query); + } if (settings.transform_null_in) { From 8eb4e22d80cc3e5a40f15296398f20ec55cc9ad1 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Fri, 2 Dec 2022 09:02:25 +0000 Subject: [PATCH 066/127] Prefer recent clang-tidy over old-ish system clang-tidy --- cmake/clang_tidy.cmake | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/cmake/clang_tidy.cmake b/cmake/clang_tidy.cmake index 57295682487..ceaafdaa9aa 100644 --- a/cmake/clang_tidy.cmake +++ b/cmake/clang_tidy.cmake @@ -5,21 +5,21 @@ if (ENABLE_CLANG_TIDY) find_program (CLANG_TIDY_CACHE_PATH NAMES "clang-tidy-cache") if (CLANG_TIDY_CACHE_PATH) - find_program (_CLANG_TIDY_PATH NAMES "clang-tidy" "clang-tidy-15" "clang-tidy-14" "clang-tidy-13" "clang-tidy-12") + find_program (_CLANG_TIDY_PATH NAMES "clang-tidy-15" "clang-tidy-14" "clang-tidy-13" "clang-tidy-12" "clang-tidy") # Why do we use ';' here? # It's a cmake black magic: https://cmake.org/cmake/help/latest/prop_tgt/LANG_CLANG_TIDY.html#prop_tgt:%3CLANG%3E_CLANG_TIDY # The CLANG_TIDY_PATH is passed to CMAKE_CXX_CLANG_TIDY, which follows CXX_CLANG_TIDY syntax. set (CLANG_TIDY_PATH "${CLANG_TIDY_CACHE_PATH};${_CLANG_TIDY_PATH}" CACHE STRING "A combined command to run clang-tidy with caching wrapper") else () - find_program (CLANG_TIDY_PATH NAMES "clang-tidy" "clang-tidy-15" "clang-tidy-14" "clang-tidy-13" "clang-tidy-12") + find_program (CLANG_TIDY_PATH NAMES "clang-tidy-15" "clang-tidy-14" "clang-tidy-13" "clang-tidy-12" "clang-tidy") endif () if (CLANG_TIDY_PATH) message (STATUS "Using clang-tidy: ${CLANG_TIDY_PATH}. - The checks will be run during build process. - See the .clang-tidy file at the root directory to configure the checks.") + The checks will be run during the build process. + See the .clang-tidy file in the root directory to configure the checks.") set (USE_CLANG_TIDY ON) From 39b34b09c12a2531ac9f3e7cd0815e96256e2aa7 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Fri, 2 Dec 2022 09:02:40 +0000 Subject: [PATCH 067/127] Add comment about dictionary syntax for configuration --- .clang-tidy | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/.clang-tidy b/.clang-tidy index 5da1d309f62..93a732436fc 100644 --- a/.clang-tidy +++ b/.clang-tidy @@ -146,6 +146,11 @@ Checks: '*, WarningsAsErrors: '*' +# TODO: use dictionary syntax for CheckOptions when minimum clang-tidy level rose to 15 +# some-check.SomeOption: 'some value' +# instead of +# - key: some-check.SomeOption +# value: 'some value' CheckOptions: - key: readability-identifier-naming.ClassCase value: CamelCase From 4122bd853dc3c9ebdaeb115dadca1a71defda709 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Fri, 2 Dec 2022 09:17:05 +0000 Subject: [PATCH 068/127] Sort checks --- .clang-tidy | 9 ++++----- 1 file changed, 4 insertions(+), 5 deletions(-) diff --git a/.clang-tidy b/.clang-tidy index 93a732436fc..0d2519e9fea 100644 --- a/.clang-tidy +++ b/.clang-tidy @@ -16,6 +16,7 @@ Checks: '*, -android-*, + -bugprone-assignment-in-if-condition, -bugprone-branch-clone, -bugprone-easily-swappable-parameters, -bugprone-exception-escape, @@ -23,7 +24,6 @@ Checks: '*, -bugprone-narrowing-conversions, -bugprone-not-null-terminated-result, -bugprone-unchecked-optional-access, - -bugprone-assignment-in-if-condition, -cert-dcl16-c, -cert-err58-cpp, @@ -34,7 +34,6 @@ Checks: '*, -clang-analyzer-optin.performance.Padding, -clang-analyzer-optin.portability.UnixAPI, - -clang-analyzer-security.insecureAPI.bzero, -clang-analyzer-security.insecureAPI.strcpy, @@ -103,12 +102,13 @@ Checks: '*, -openmp-*, + -misc-const-correctness, -misc-no-recursion, -misc-non-private-member-variables-in-classes, - -misc-const-correctness, -modernize-avoid-c-arrays, -modernize-concat-nested-namespaces, + -modernize-macro-to-enum, -modernize-pass-by-value, -modernize-return-braced-init-list, -modernize-use-auto, @@ -117,7 +117,6 @@ Checks: '*, -modernize-use-nodiscard, -modernize-use-override, -modernize-use-trailing-return-type, - -modernize-macro-to-enum, -performance-inefficient-string-concatenation, -performance-no-int-to-ptr, @@ -135,11 +134,11 @@ Checks: '*, -readability-magic-numbers, -readability-named-parameter, -readability-redundant-declaration, + -readability-simplify-boolean-expr, -readability-static-accessed-through-instance, -readability-suspicious-call-argument, -readability-uppercase-literal-suffix, -readability-use-anyofallof, - -readability-simplify-boolean-expr, -zirkon-*, ' From 98a11e2ebd8a96bd05becc9be3da787869947c40 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Fri, 2 Dec 2022 09:18:05 +0000 Subject: [PATCH 069/127] Temporarily disable misc-* due to being too slow --- .clang-tidy | 13 +++++++++++++ 1 file changed, 13 insertions(+) diff --git a/.clang-tidy b/.clang-tidy index 0d2519e9fea..e9451272681 100644 --- a/.clang-tidy +++ b/.clang-tidy @@ -141,6 +141,19 @@ Checks: '*, -readability-use-anyofallof, -zirkon-*, + + -misc-*, # temporarily disabled due to being too slow + # also disable checks in other categories which are aliases of checks in misc-*: + # https://releases.llvm.org/15.0.0/tools/clang/tools/extra/docs/clang-tidy/checks/list.html + -cert-dcl54-cpp, # alias of misc-new-delete-overloads + -hicpp-new-delete-operators, # alias of misc-new-delete-overloads + -cert-fio38-c, # alias of misc-non-copyable-objects + -cert-dcl03-c, # alias of misc-static-assert + -hicpp-static-assert, # alias of misc-static-assert + -cert-err09-cpp, # alias of misc-throw-by-value-catch-by-reference + -cert-err61-cpp, # alias of misc-throw-by-value-catch-by-reference + -cppcoreguidelines-c-copy-assignment-signature, # alias of misc-unconventional-assign-operator + -cppcoreguidelines-non-private-member-variables-in-classes, # alias of misc-non-private-member-variables-in-classes ' WarningsAsErrors: '*' From 1197d3fbe37f6bc6e474a8a6c33267cf5da60a88 Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Wed, 7 Dec 2022 13:32:43 +0100 Subject: [PATCH 070/127] Remove test logging of signal "EINTR" --- src/Common/Epoll.cpp | 4 ---- 1 file changed, 4 deletions(-) diff --git a/src/Common/Epoll.cpp b/src/Common/Epoll.cpp index 9b2589f0589..fa31734d432 100644 --- a/src/Common/Epoll.cpp +++ b/src/Common/Epoll.cpp @@ -3,7 +3,6 @@ #include "Epoll.h" #include #include -#include namespace DB { @@ -70,9 +69,6 @@ size_t Epoll::getManyReady(int max_events, epoll_event * events_out, bool blocki if (ready_size == -1 && errno != EINTR) throwFromErrno("Error in epoll_wait", DB::ErrorCodes::EPOLL_ERROR); - - if (errno == EINTR) - LOG_TEST(&Poco::Logger::get("Epoll"), "EINTR"); } while (ready_size <= 0 && (ready_size != 0 || blocking)); From d042664f540dee21515c76eb209b3194de13636c Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Wed, 7 Dec 2022 13:33:29 +0100 Subject: [PATCH 071/127] Remove logging from TimerDescriptor --- src/Common/TimerDescriptor.cpp | 4 ---- 1 file changed, 4 deletions(-) diff --git a/src/Common/TimerDescriptor.cpp b/src/Common/TimerDescriptor.cpp index 1301ebce0ba..a7c74dab8be 100644 --- a/src/Common/TimerDescriptor.cpp +++ b/src/Common/TimerDescriptor.cpp @@ -6,8 +6,6 @@ #include #include -#include - namespace DB { @@ -72,8 +70,6 @@ void TimerDescriptor::drain() const if (errno != EINTR) throwFromErrno("Cannot drain timer_fd", ErrorCodes::CANNOT_READ_FROM_SOCKET); - else - LOG_TEST(&Poco::Logger::get("TimerDescriptor"), "EINTR"); } } } From 9ccfda73ae100e75db4e6209744cede2e3b70ca1 Mon Sep 17 00:00:00 2001 From: serxa Date: Wed, 7 Dec 2022 13:07:28 +0000 Subject: [PATCH 072/127] fixes after review --- src/Access/SettingsConstraints.cpp | 16 +++++++++------- src/Access/resolveSetting.h | 18 +++++++++--------- 2 files changed, 18 insertions(+), 16 deletions(-) diff --git a/src/Access/SettingsConstraints.cpp b/src/Access/SettingsConstraints.cpp index 92687595371..6cd44506304 100644 --- a/src/Access/SettingsConstraints.cpp +++ b/src/Access/SettingsConstraints.cpp @@ -140,14 +140,16 @@ bool getNewValueToCheck(const T & current_settings, SettingChange & change, Fiel if (throw_on_failure) new_value = T::castValueUtil(change.name, change.value); - - try + else { - new_value = T::castValueUtil(change.name, change.value); - } - catch (...) - { - return false; + try + { + new_value = T::castValueUtil(change.name, change.value); + } + catch (...) + { + return false; + } } /// Setting isn't checked if value has not changed. diff --git a/src/Access/resolveSetting.h b/src/Access/resolveSetting.h index 234715dde9b..af9e4a08d61 100644 --- a/src/Access/resolveSetting.h +++ b/src/Access/resolveSetting.h @@ -9,7 +9,7 @@ // `resolveSetting(full_name)` is used to resolve setting name and choose which class is to be used. // Templated lambda syntax should be used: // -// return resolveSetting(name, [] (std::string_view name, Overload) -> auto +// return resolveSetting(name, [] (std::string_view name, SettingsType) // { // return T::castValueUtil(name, value); // T will be deduced into `Settings`, `MergeTreeSettings`, ... // }); @@ -20,27 +20,27 @@ namespace DB constexpr std::string_view MERGE_TREE_SETTINGS_PREFIX = "merge_tree_"; -template struct Overload {}; +template struct SettingsType {}; // Resolve setting name and call function `f` back with short name and class -template +template auto resolveSetting(std::string_view full_name, F && f) { if (full_name.starts_with(MERGE_TREE_SETTINGS_PREFIX)) { std::string_view short_name = static_cast(full_name).substr(MERGE_TREE_SETTINGS_PREFIX.size()); if (MergeTreeSettings::hasBuiltin(short_name)) // Check is required because `Settings` also contain names starting with 'merge_tree_' prefix - return f(short_name, Overload()); + return f(short_name, SettingsType()); } // NOTE: other setting name resolution rules are to be added here // If no rule works - use global namespace - return f(full_name, Overload()); + return f(full_name, SettingsType()); } inline Field settingCastValueUtil(std::string_view full_name, const Field & value) { - return resolveSetting(full_name, [&] (std::string_view short_name, Overload) + return resolveSetting(full_name, [&] (std::string_view short_name, SettingsType) { return T::castValueUtil(short_name, value); }); @@ -48,7 +48,7 @@ inline Field settingCastValueUtil(std::string_view full_name, const Field & valu inline String settingValueToStringUtil(std::string_view full_name, const Field & value) { - return resolveSetting(full_name, [&] (std::string_view short_name, Overload) + return resolveSetting(full_name, [&] (std::string_view short_name, SettingsType) { return T::valueToStringUtil(short_name, value); }); @@ -56,7 +56,7 @@ inline String settingValueToStringUtil(std::string_view full_name, const Field & inline Field settingStringToValueUtil(std::string_view full_name, const String & str) { - return resolveSetting(full_name, [&] (std::string_view short_name, Overload) + return resolveSetting(full_name, [&] (std::string_view short_name, SettingsType) { return T::stringToValueUtil(short_name, str); }); @@ -64,7 +64,7 @@ inline Field settingStringToValueUtil(std::string_view full_name, const String & inline bool settingIsBuiltin(std::string_view full_name) { - return resolveSetting(full_name, [&] (std::string_view short_name, Overload) + return resolveSetting(full_name, [&] (std::string_view short_name, SettingsType) { return T::hasBuiltin(short_name); }); From a8faf196c46fa6651e12da42883462f4bc3b43d0 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Wed, 7 Dec 2022 14:51:16 +0100 Subject: [PATCH 073/127] Add --max-consecutive-errors for clickhouse-benchmark (#43344) * Align Benchmark::Benchmark() Signed-off-by: Azat Khuzhin * Add --max-consecutive-errors for clickhouse-benchmark Unlike --continue_on_errors, it will not leave the benchmark forever if server is unavailable. Signed-off-by: Azat Khuzhin Signed-off-by: Azat Khuzhin Co-authored-by: Nikita Taranov --- programs/benchmark/Benchmark.cpp | 66 ++++++++++++++----- ...benchmark_max_consecutive_errors.reference | 2 + .../02490_benchmark_max_consecutive_errors.sh | 8 +++ 3 files changed, 61 insertions(+), 15 deletions(-) create mode 100644 tests/queries/0_stateless/02490_benchmark_max_consecutive_errors.reference create mode 100755 tests/queries/0_stateless/02490_benchmark_max_consecutive_errors.sh diff --git a/programs/benchmark/Benchmark.cpp b/programs/benchmark/Benchmark.cpp index 4d55a67f0f8..017b28fe082 100644 --- a/programs/benchmark/Benchmark.cpp +++ b/programs/benchmark/Benchmark.cpp @@ -58,22 +58,52 @@ namespace ErrorCodes class Benchmark : public Poco::Util::Application { public: - Benchmark(unsigned concurrency_, double delay_, - Strings && hosts_, Ports && ports_, bool round_robin_, - bool cumulative_, bool secure_, const String & default_database_, - const String & user_, const String & password_, const String & quota_key_, const String & stage, - bool randomize_, size_t max_iterations_, double max_time_, - const String & json_path_, size_t confidence_, - const String & query_id_, const String & query_to_execute_, bool continue_on_errors_, - bool reconnect_, bool display_client_side_time_, bool print_stacktrace_, const Settings & settings_) + Benchmark(unsigned concurrency_, + double delay_, + Strings && hosts_, + Ports && ports_, + bool round_robin_, + bool cumulative_, + bool secure_, + const String & default_database_, + const String & user_, + const String & password_, + const String & quota_key_, + const String & stage, + bool randomize_, + size_t max_iterations_, + double max_time_, + const String & json_path_, + size_t confidence_, + const String & query_id_, + const String & query_to_execute_, + size_t max_consecutive_errors_, + bool continue_on_errors_, + bool reconnect_, + bool display_client_side_time_, + bool print_stacktrace_, + const Settings & settings_) : - round_robin(round_robin_), concurrency(concurrency_), delay(delay_), queue(concurrency), randomize(randomize_), - cumulative(cumulative_), max_iterations(max_iterations_), max_time(max_time_), - json_path(json_path_), confidence(confidence_), query_id(query_id_), - query_to_execute(query_to_execute_), continue_on_errors(continue_on_errors_), reconnect(reconnect_), + round_robin(round_robin_), + concurrency(concurrency_), + delay(delay_), + queue(concurrency), + randomize(randomize_), + cumulative(cumulative_), + max_iterations(max_iterations_), + max_time(max_time_), + json_path(json_path_), + confidence(confidence_), + query_id(query_id_), + query_to_execute(query_to_execute_), + continue_on_errors(continue_on_errors_), + max_consecutive_errors(max_consecutive_errors_), + reconnect(reconnect_), display_client_side_time(display_client_side_time_), - print_stacktrace(print_stacktrace_), settings(settings_), - shared_context(Context::createShared()), global_context(Context::createGlobal(shared_context.get())), + print_stacktrace(print_stacktrace_), + settings(settings_), + shared_context(Context::createShared()), + global_context(Context::createGlobal(shared_context.get())), pool(concurrency) { const auto secure = secure_ ? Protocol::Secure::Enable : Protocol::Secure::Disable; @@ -166,6 +196,7 @@ private: String query_id; String query_to_execute; bool continue_on_errors; + size_t max_consecutive_errors; bool reconnect; bool display_client_side_time; bool print_stacktrace; @@ -174,6 +205,8 @@ private: ContextMutablePtr global_context; QueryProcessingStage::Enum query_processing_stage; + std::atomic consecutive_errors{0}; + /// Don't execute new queries after timelimit or SIGINT or exception std::atomic shutdown{false}; @@ -393,13 +426,14 @@ private: try { execute(connection_entries, query, connection_index); + consecutive_errors = 0; } catch (...) { std::lock_guard lock(mutex); std::cerr << "An error occurred while processing the query " << "'" << query << "'" << ": " << getCurrentExceptionMessage(false) << std::endl; - if (!continue_on_errors) + if (!(continue_on_errors || max_consecutive_errors > ++consecutive_errors)) { shutdown = true; throw; @@ -648,6 +682,7 @@ int mainEntryClickHouseBenchmark(int argc, char ** argv) ("stacktrace", "print stack traces of exceptions") ("confidence", value()->default_value(5), "set the level of confidence for T-test [0=80%, 1=90%, 2=95%, 3=98%, 4=99%, 5=99.5%(default)") ("query_id", value()->default_value(""), "") + ("max-consecutive-errors", value()->default_value(0), "set number of allowed consecutive errors") ("continue_on_errors", "continue testing even if a query fails") ("reconnect", "establish new connection for every query") ("client-side-time", "display the time including network communication instead of server-side time; note that for server versions before 22.8 we always display client-side time") @@ -702,6 +737,7 @@ int mainEntryClickHouseBenchmark(int argc, char ** argv) options["confidence"].as(), options["query_id"].as(), options["query"].as(), + options["max-consecutive-errors"].as(), options.count("continue_on_errors"), options.count("reconnect"), options.count("client-side-time"), diff --git a/tests/queries/0_stateless/02490_benchmark_max_consecutive_errors.reference b/tests/queries/0_stateless/02490_benchmark_max_consecutive_errors.reference new file mode 100644 index 00000000000..b258dc3d8a7 --- /dev/null +++ b/tests/queries/0_stateless/02490_benchmark_max_consecutive_errors.reference @@ -0,0 +1,2 @@ +1 +10 diff --git a/tests/queries/0_stateless/02490_benchmark_max_consecutive_errors.sh b/tests/queries/0_stateless/02490_benchmark_max_consecutive_errors.sh new file mode 100755 index 00000000000..3c046020773 --- /dev/null +++ b/tests/queries/0_stateless/02490_benchmark_max_consecutive_errors.sh @@ -0,0 +1,8 @@ +#!/usr/bin/env bash + +CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CUR_DIR"/../shell_config.sh + +$CLICKHOUSE_BENCHMARK -q 'select throwIf(1)' |& grep '^An error occurred while processing the query.*Exception:' -c +$CLICKHOUSE_BENCHMARK --max-consecutive-errors 10 -q 'select throwIf(1)' |& grep '^An error occurred while processing the query.*Exception:' -c From 4abc36774ed7bea02945f579e97c4a809e98176d Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 1 Dec 2022 19:00:11 +0000 Subject: [PATCH 074/127] Some cleanup of isDeterministic(InScopeOfQuery)() In cases where the implementation simply repeates the implementation inherited from the base class, it can be omitted. --- src/Functions/FunctionConstantBase.h | 1 - src/Functions/FunctionsConversion.h | 2 -- src/Functions/FunctionsMiscellaneous.h | 4 ---- src/Functions/UTCTimestamp.cpp | 1 - src/Functions/array/arrayJoin.cpp | 5 ++++- src/Functions/blockSize.cpp | 5 ++++- src/Functions/getMacro.cpp | 5 ----- src/Functions/getScalar.cpp | 5 ----- src/Functions/getServerPort.cpp | 2 -- src/Functions/now.cpp | 12 +++++++++--- src/Functions/now64.cpp | 12 +++++++++--- src/Functions/randConstant.cpp | 6 ++++-- src/Functions/rowNumberInAllBlocks.cpp | 5 ++++- src/Functions/rowNumberInBlock.cpp | 5 ++++- src/Functions/runningAccumulate.cpp | 5 ++++- src/Functions/runningDifference.h | 6 +++++- src/Functions/today.cpp | 1 - src/Functions/yesterday.cpp | 1 - 18 files changed, 47 insertions(+), 36 deletions(-) diff --git a/src/Functions/FunctionConstantBase.h b/src/Functions/FunctionConstantBase.h index 3a9c31cd201..ad969268713 100644 --- a/src/Functions/FunctionConstantBase.h +++ b/src/Functions/FunctionConstantBase.h @@ -34,7 +34,6 @@ public: } bool isDeterministic() const override { return false; } - bool isDeterministicInScopeOfQuery() const override { return true; } /// Some functions may return different values on different shards/replicas, so it's not constant for distributed query bool isSuitableForConstantFolding() const override { return !is_distributed; } diff --git a/src/Functions/FunctionsConversion.h b/src/Functions/FunctionsConversion.h index c6bb45ddbde..c9638ab95af 100644 --- a/src/Functions/FunctionsConversion.h +++ b/src/Functions/FunctionsConversion.h @@ -2670,8 +2670,6 @@ public: String getName() const override { return cast_name; } - bool isDeterministic() const override { return true; } - bool isDeterministicInScopeOfQuery() const override { return true; } bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return true; } bool hasInformationAboutMonotonicity() const override diff --git a/src/Functions/FunctionsMiscellaneous.h b/src/Functions/FunctionsMiscellaneous.h index 49da8f49c82..da90889fd4d 100644 --- a/src/Functions/FunctionsMiscellaneous.h +++ b/src/Functions/FunctionsMiscellaneous.h @@ -79,8 +79,6 @@ public: String getName() const override { return "FunctionExpression"; } - bool isDeterministic() const override { return true; } - bool isDeterministicInScopeOfQuery() const override { return true; } bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return false; } const DataTypes & getArgumentTypes() const override { return argument_types; } @@ -176,8 +174,6 @@ public: String getName() const override { return name; } - bool isDeterministic() const override { return true; } - bool isDeterministicInScopeOfQuery() const override { return true; } bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return false; } const DataTypes & getArgumentTypes() const override { return capture->captured_types; } diff --git a/src/Functions/UTCTimestamp.cpp b/src/Functions/UTCTimestamp.cpp index 27b5b33bc6e..be137449fd5 100644 --- a/src/Functions/UTCTimestamp.cpp +++ b/src/Functions/UTCTimestamp.cpp @@ -59,7 +59,6 @@ public: } bool isDeterministic() const override { return false; } - bool isDeterministicInScopeOfQuery() const override { return true; } bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return false; } private: diff --git a/src/Functions/array/arrayJoin.cpp b/src/Functions/array/arrayJoin.cpp index 3230886c731..1dbe4cebb14 100644 --- a/src/Functions/array/arrayJoin.cpp +++ b/src/Functions/array/arrayJoin.cpp @@ -38,7 +38,10 @@ public: } /** It could return many different values for single argument. */ - bool isDeterministic() const override { return false; } + bool isDeterministic() const override + { + return false; + } bool isDeterministicInScopeOfQuery() const override { diff --git a/src/Functions/blockSize.cpp b/src/Functions/blockSize.cpp index af3c4ed27b4..deff4b19fc4 100644 --- a/src/Functions/blockSize.cpp +++ b/src/Functions/blockSize.cpp @@ -26,7 +26,10 @@ public: return name; } - bool isDeterministic() const override { return false; } + bool isDeterministic() const override + { + return false; + } bool isDeterministicInScopeOfQuery() const override { diff --git a/src/Functions/getMacro.cpp b/src/Functions/getMacro.cpp index d5c9f8439dd..96c3acc7088 100644 --- a/src/Functions/getMacro.cpp +++ b/src/Functions/getMacro.cpp @@ -50,11 +50,6 @@ public: bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return false; } - bool isDeterministicInScopeOfQuery() const override - { - return true; - } - /// getMacro may return different values on different shards/replicas, so it's not constant for distributed query bool isSuitableForConstantFolding() const override { return !is_distributed; } diff --git a/src/Functions/getScalar.cpp b/src/Functions/getScalar.cpp index a3440dd89f2..d7036255e10 100644 --- a/src/Functions/getScalar.cpp +++ b/src/Functions/getScalar.cpp @@ -105,11 +105,6 @@ public: bool isDeterministic() const override { return false; } - bool isDeterministicInScopeOfQuery() const override - { - return true; - } - bool isSuitableForConstantFolding() const override { return !is_distributed; } bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return false; } diff --git a/src/Functions/getServerPort.cpp b/src/Functions/getServerPort.cpp index 054f7b538f7..faa31e5cac5 100644 --- a/src/Functions/getServerPort.cpp +++ b/src/Functions/getServerPort.cpp @@ -56,7 +56,6 @@ public: } bool isDeterministic() const override { return false; } - bool isDeterministicInScopeOfQuery() const override { return true; } bool isSuitableForConstantFolding() const override { return !is_distributed; } bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return false; } @@ -89,7 +88,6 @@ public: size_t getNumberOfArguments() const override { return 1; } ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {0}; } bool isDeterministic() const override { return false; } - bool isDeterministicInScopeOfQuery() const override { return true; } DataTypePtr getReturnTypeImpl(const DataTypes & data_types) const override { diff --git a/src/Functions/now.cpp b/src/Functions/now.cpp index d257bf4314e..694e6bc97cb 100644 --- a/src/Functions/now.cpp +++ b/src/Functions/now.cpp @@ -62,9 +62,15 @@ public: return std::make_unique(time_value); } - bool isDeterministic() const override { return false; } - bool isDeterministicInScopeOfQuery() const override { return true; } - bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return false; } + bool isDeterministic() const override + { + return false; + } + + bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo &) const override + { + return false; + } private: time_t time_value; diff --git a/src/Functions/now64.cpp b/src/Functions/now64.cpp index a6df4235d60..e075dab6b0f 100644 --- a/src/Functions/now64.cpp +++ b/src/Functions/now64.cpp @@ -87,9 +87,15 @@ public: return std::make_unique(time_value); } - bool isDeterministic() const override { return false; } - bool isDeterministicInScopeOfQuery() const override { return true; } - bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return false; } + bool isDeterministic() const override + { + return false; + } + + bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo &) const override + { + return false; + } private: Field time_value; diff --git a/src/Functions/randConstant.cpp b/src/Functions/randConstant.cpp index 7bf8630f92a..d19518ce765 100644 --- a/src/Functions/randConstant.cpp +++ b/src/Functions/randConstant.cpp @@ -59,8 +59,10 @@ public: return std::make_unique>(value); } - bool isDeterministic() const override { return false; } - bool isDeterministicInScopeOfQuery() const override { return true; } + bool isDeterministic() const override + { + return false; + } private: ToType value; diff --git a/src/Functions/rowNumberInAllBlocks.cpp b/src/Functions/rowNumberInAllBlocks.cpp index 91da512d221..28d61afcdc1 100644 --- a/src/Functions/rowNumberInAllBlocks.cpp +++ b/src/Functions/rowNumberInAllBlocks.cpp @@ -39,7 +39,10 @@ public: return 0; } - bool isDeterministic() const override { return false; } + bool isDeterministic() const override + { + return false; + } bool isDeterministicInScopeOfQuery() const override { diff --git a/src/Functions/rowNumberInBlock.cpp b/src/Functions/rowNumberInBlock.cpp index b3f95d27a93..e5fe2aeb178 100644 --- a/src/Functions/rowNumberInBlock.cpp +++ b/src/Functions/rowNumberInBlock.cpp @@ -34,7 +34,10 @@ public: return 0; } - bool isDeterministic() const override { return false; } + bool isDeterministic() const override + { + return false; + } bool isDeterministicInScopeOfQuery() const override { diff --git a/src/Functions/runningAccumulate.cpp b/src/Functions/runningAccumulate.cpp index 667f722ee92..336c45e49cb 100644 --- a/src/Functions/runningAccumulate.cpp +++ b/src/Functions/runningAccumulate.cpp @@ -52,7 +52,10 @@ public: size_t getNumberOfArguments() const override { return 0; } - bool isDeterministic() const override { return false; } + bool isDeterministic() const override + { + return false; + } bool isDeterministicInScopeOfQuery() const override { diff --git a/src/Functions/runningDifference.h b/src/Functions/runningDifference.h index 053d7cb9736..cf534b30c90 100644 --- a/src/Functions/runningDifference.h +++ b/src/Functions/runningDifference.h @@ -147,7 +147,11 @@ public: return 1; } - bool isDeterministic() const override { return false; } + bool isDeterministic() const override + { + return false; + } + bool isDeterministicInScopeOfQuery() const override { return false; diff --git a/src/Functions/today.cpp b/src/Functions/today.cpp index 504c840efe5..f106e3992a8 100644 --- a/src/Functions/today.cpp +++ b/src/Functions/today.cpp @@ -53,7 +53,6 @@ public: } bool isDeterministic() const override { return false; } - bool isDeterministicInScopeOfQuery() const override { return true; } bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return false; } private: diff --git a/src/Functions/yesterday.cpp b/src/Functions/yesterday.cpp index 53b73a7a71e..43832c1faaa 100644 --- a/src/Functions/yesterday.cpp +++ b/src/Functions/yesterday.cpp @@ -51,7 +51,6 @@ public: } bool isDeterministic() const override { return false; } - bool isDeterministicInScopeOfQuery() const override { return true; } bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return false; } private: From 127373ab063fbfd3831174cd6ac54ce504a88d04 Mon Sep 17 00:00:00 2001 From: vdimir Date: Wed, 7 Dec 2022 15:02:37 +0000 Subject: [PATCH 075/127] Set proper keys order for StorageJoin --- src/Interpreters/TableJoin.cpp | 6 ++- src/Interpreters/TableJoin.h | 1 + src/Storages/StorageJoin.cpp | 33 +++++++++++- src/Storages/StorageJoin.h | 2 + ...02498_storage_join_key_positions.reference | 54 +++++++++++++++++++ .../02498_storage_join_key_positions.sql | 46 ++++++++++++++++ 6 files changed, 139 insertions(+), 3 deletions(-) create mode 100644 tests/queries/0_stateless/02498_storage_join_key_positions.reference create mode 100644 tests/queries/0_stateless/02498_storage_join_key_positions.sql diff --git a/src/Interpreters/TableJoin.cpp b/src/Interpreters/TableJoin.cpp index 5d065e564b2..766378f0fbd 100644 --- a/src/Interpreters/TableJoin.cpp +++ b/src/Interpreters/TableJoin.cpp @@ -35,9 +35,10 @@ namespace DB namespace ErrorCodes { - extern const int TYPE_MISMATCH; + extern const int INCOMPATIBLE_TYPE_OF_JOIN; extern const int LOGICAL_ERROR; extern const int NOT_IMPLEMENTED; + extern const int TYPE_MISMATCH; } namespace @@ -652,6 +653,9 @@ void TableJoin::setStorageJoin(std::shared_ptr storage) void TableJoin::setStorageJoin(std::shared_ptr storage) { + if (clauses.empty()) + throw Exception(ErrorCodes::INCOMPATIBLE_TYPE_OF_JOIN, + "StorageJoin keys should match JOIN keys, expected JOIN ON [{}]", fmt::join(storage->getKeyNames(), ", ")); right_storage_join = storage; } diff --git a/src/Interpreters/TableJoin.h b/src/Interpreters/TableJoin.h index 356687f080c..874e68b0b97 100644 --- a/src/Interpreters/TableJoin.h +++ b/src/Interpreters/TableJoin.h @@ -329,6 +329,7 @@ public: /// StorageJoin overrides key names (cause of different names qualification) void setRightKeys(const Names & keys) { getOnlyClause().key_names_right = keys; } + void setLeftKeys(const Names & keys) { getOnlyClause().key_names_left = keys; } Block getRequiredRightKeys(const Block & right_table_keys, std::vector & keys_sources) const; diff --git a/src/Storages/StorageJoin.cpp b/src/Storages/StorageJoin.cpp index e4f786cd23b..50bdb11952d 100644 --- a/src/Storages/StorageJoin.cpp +++ b/src/Storages/StorageJoin.cpp @@ -165,7 +165,7 @@ HashJoinPtr StorageJoin::getJoinLocked(std::shared_ptr analyzed_join, { auto metadata_snapshot = getInMemoryMetadataPtr(); if (!analyzed_join->sameStrictnessAndKind(strictness, kind)) - throw Exception("Table " + getStorageID().getNameForLogs() + " has incompatible type of JOIN.", ErrorCodes::INCOMPATIBLE_TYPE_OF_JOIN); + throw Exception(ErrorCodes::INCOMPATIBLE_TYPE_OF_JOIN, "Table '{} has incompatible type of JOIN", getStorageID().getNameForLogs()); if ((analyzed_join->forceNullableRight() && !use_nulls) || (!analyzed_join->forceNullableRight() && isLeftOrFull(analyzed_join->kind()) && use_nulls)) @@ -174,12 +174,41 @@ HashJoinPtr StorageJoin::getJoinLocked(std::shared_ptr analyzed_join, "Table {} needs the same join_use_nulls setting as present in LEFT or FULL JOIN", getStorageID().getNameForLogs()); - /// TODO: check key columns + const auto & key_names_right = analyzed_join->getOnlyClause().key_names_right; + const auto & key_names_left = analyzed_join->getOnlyClause().key_names_left; + if (key_names.size() != key_names_right.size() || key_names.size() != key_names_left.size()) + throw Exception(ErrorCodes::INCOMPATIBLE_TYPE_OF_JOIN, + "Number of keys in JOIN ON section ({}) doesn't match number of keys in Join engine ({})", + key_names_right.size(), key_names.size()); + + /* Resort left keys according to right keys order in StorageJoin + * We can't change the order of keys in StorageJoin because the hash table already build. + * + * Example: + * ``` + * CREATE TABLE t_right (a UInt32, b UInt32) ENGINE = Join(ALL, INNER, a, b); + * SELECT * FROM t_left JOIN t_right ON t_left.y = t_right.b AND t_left.x = t_right.a; + * ``` + * In that case right keys should still be (a, b), need to change the order of the left keys to (x, y). + */ + Names left_key_names_resorted; + for (size_t i = 0; i < key_names.size(); ++i) + { + const auto & renamed_key = analyzed_join->renamedRightColumnName(key_names[i]); + /// find position of renamed_key in key_names_right + auto it = std::find(key_names_right.begin(), key_names_right.end(), renamed_key); + if (it == key_names_right.end()) + throw Exception(ErrorCodes::INCOMPATIBLE_TYPE_OF_JOIN, + "Key '{}' not found in JOIN ON section. All Join engine keys '{}' have to be used", key_names[i], fmt::join(key_names, ", ")); + size_t key_position = it - key_names_right.begin(); + left_key_names_resorted.push_back(key_names_left[key_position]); + } /// Set names qualifiers: table.column -> column /// It's required because storage join stores non-qualified names /// Qualifies will be added by join implementation (HashJoin) analyzed_join->setRightKeys(key_names); + analyzed_join->setLeftKeys(left_key_names_resorted); HashJoinPtr join_clone = std::make_shared(analyzed_join, getRightSampleBlock()); diff --git a/src/Storages/StorageJoin.h b/src/Storages/StorageJoin.h index 43515f800d9..c03be74478d 100644 --- a/src/Storages/StorageJoin.h +++ b/src/Storages/StorageJoin.h @@ -85,6 +85,8 @@ public: bool useNulls() const { return use_nulls; } + const Names & getKeyNames() const { return key_names; } + private: Block sample_block; const Names key_names; diff --git a/tests/queries/0_stateless/02498_storage_join_key_positions.reference b/tests/queries/0_stateless/02498_storage_join_key_positions.reference new file mode 100644 index 00000000000..a11b547ca38 --- /dev/null +++ b/tests/queries/0_stateless/02498_storage_join_key_positions.reference @@ -0,0 +1,54 @@ +21 22 23 2000 +31 32 33 3000 +41 42 43 4000 +51 52 53 5000 +21 22 23 2000 +31 32 33 3000 +41 42 43 4000 +51 52 53 5000 +21 22 23 2000 +31 32 33 3000 +41 42 43 4000 +51 52 53 5000 +21 22 23 2000 +31 32 33 3000 +41 42 43 4000 +51 52 53 5000 +21 22 23 22 21 23 2000 +31 32 33 32 31 33 3000 +41 42 43 42 41 43 4000 +51 52 53 52 51 53 5000 +21 22 23 22 21 23 2000 +31 32 33 32 31 33 3000 +41 42 43 42 41 43 4000 +51 52 53 52 51 53 5000 +21 22 23 22 21 23 2000 +31 32 33 32 31 33 3000 +41 42 43 42 41 43 4000 +51 52 53 52 51 53 5000 +21 22 23 22 21 23 2000 +31 32 33 32 31 33 3000 +41 42 43 42 41 43 4000 +51 52 53 52 51 53 5000 +23 21 22 22 21 23 2000 +33 31 32 32 31 33 3000 +43 41 42 42 41 43 4000 +53 51 52 52 51 53 5000 +23 21 22 22 21 23 2000 +33 31 32 32 31 33 3000 +43 41 42 42 41 43 4000 +53 51 52 52 51 53 5000 +23 21 22 22 21 23 2000 +33 31 32 32 31 33 3000 +43 41 42 42 41 43 4000 +53 51 52 52 51 53 5000 +11 12 13 11 11 11 1000 +21 22 23 21 21 21 2000 +31 32 33 31 31 31 3000 +41 42 43 41 41 41 4000 +51 52 53 51 51 51 5000 +11 12 13 11 11 11 1000 +21 22 23 21 21 21 2000 +31 32 33 31 31 31 3000 +41 42 43 41 41 41 4000 +51 52 53 51 51 51 5000 diff --git a/tests/queries/0_stateless/02498_storage_join_key_positions.sql b/tests/queries/0_stateless/02498_storage_join_key_positions.sql new file mode 100644 index 00000000000..34f7e3543aa --- /dev/null +++ b/tests/queries/0_stateless/02498_storage_join_key_positions.sql @@ -0,0 +1,46 @@ +DROP TABLE IF EXISTS t1; +DROP TABLE IF EXISTS tj; +DROP TABLE IF EXISTS tjj; + +CREATE TABLE t1 (key1 UInt64, key2 UInt64, key3 UInt64) ENGINE = Memory; +INSERT INTO t1 VALUES (11, 12, 13), (21, 22, 23), (31, 32, 33), (41, 42, 43), (51, 52, 53); + +CREATE TABLE tj (key2 UInt64, key1 UInt64, key3 UInt64, attr UInt64) ENGINE = Join(ALL, INNER, key3, key2, key1); +INSERT INTO tj VALUES (22, 21, 23, 2000), (32, 31, 33, 3000), (42, 41, 43, 4000), (52, 51, 53, 5000), (62, 61, 63, 6000); + +SELECT * FROM t1 ALL INNER JOIN tj USING (key1, key2, key3) ORDER BY key1; +SELECT * FROM t1 ALL INNER JOIN tj USING (key2, key3, key1) ORDER BY key1; +SELECT * FROM t1 ALL INNER JOIN tj USING (key3, key2, key1) ORDER BY key1; +SELECT * FROM t1 ALL INNER JOIN tj USING (key1, key3, key2) ORDER BY key1; + +SELECT * FROM t1 ALL INNER JOIN tj ON t1.key3 = tj.key3 AND t1.key2 = tj.key2 AND t1.key1 = tj.key1 ORDER BY key1; +SELECT * FROM t1 ALL INNER JOIN tj ON t1.key2 = tj.key2 AND t1.key3 = tj.key3 AND t1.key1 = tj.key1 ORDER BY key1; +SELECT * FROM t1 ALL INNER JOIN tj ON t1.key3 = tj.key3 AND t1.key1 = tj.key1 AND t1.key2 = tj.key2 ORDER BY key1; +SELECT * FROM t1 ALL INNER JOIN tj ON t1.key1 = tj.key1 AND t1.key3 = tj.key3 AND t1.key2 = tj.key2 ORDER BY key1; + +SELECT * FROM (SELECT key3 AS c, key1 AS a, key2 AS b FROM t1) AS t1 ALL INNER JOIN tj ON t1.a = tj.key1 AND t1.c = tj.key3 AND t1.b = tj.key2 ORDER BY t1.a; +SELECT * FROM (SELECT key3 AS c, key1 AS a, key2 AS b FROM t1) AS t1 ALL INNER JOIN tj ON t1.a = tj.key1 AND t1.b = tj.key2 AND t1.c = tj.key3 ORDER BY t1.a; +SELECT * FROM (SELECT key3 AS c, key1 AS a, key2 AS b FROM t1) AS t1 ALL INNER JOIN tj ON t1.c = tj.key3 AND t1.a = tj.key1 AND t1.b = tj.key2 ORDER BY t1.a; + +SELECT * FROM t1 ALL INNER JOIN tj ON t1.key1 = tj.key1 AND t1.key2 = tj.key2 AND t1.key3 = tj.key3 AND 1; -- { serverError INCOMPATIBLE_TYPE_OF_JOIN } +SELECT * FROM t1 ALL INNER JOIN tj ON t1.key1 = tj.key1 AND t1.key2 = tj.key2 AND t1.key3 = tj.key3 AND 0; -- { serverError INCOMPATIBLE_TYPE_OF_JOIN } +SELECT * FROM t1 ALL INNER JOIN tj ON t1.key1 = tj.key1 AND t1.key2 = tj.key2 AND t1.key3 = tj.key3 AND NULL; -- { serverError INCOMPATIBLE_TYPE_OF_JOIN } + +SELECT * FROM t1 ALL INNER JOIN tj USING (key2, key3); -- { serverError INCOMPATIBLE_TYPE_OF_JOIN } +SELECT * FROM t1 ALL INNER JOIN tj USING (key1, key2, attr); -- { serverError INCOMPATIBLE_TYPE_OF_JOIN } +SELECT * FROM t1 ALL INNER JOIN tj USING (key1, key2, key3, attr); -- { serverError INCOMPATIBLE_TYPE_OF_JOIN } +SELECT * FROM t1 ALL INNER JOIN tj ON t1.key1 = tj.attr; -- { serverError INCOMPATIBLE_TYPE_OF_JOIN } +SELECT * FROM t1 ALL INNER JOIN tj ON t1.key1 = tj.key1; -- { serverError INCOMPATIBLE_TYPE_OF_JOIN } +SELECT * FROM t1 ALL INNER JOIN tj ON t1.key1 = tj.key1 AND t1.key2 = tj.key2 AND t1.key3 = tj.attr; -- { serverError INCOMPATIBLE_TYPE_OF_JOIN } +SELECT * FROM t1 ALL INNER JOIN tj ON t1.key1 = tj.key1 AND t1.key2 = tj.key2 AND t1.key3 = tj.key3 AND t1.key1 = tj.key1; -- { serverError INCOMPATIBLE_TYPE_OF_JOIN } + + +CREATE TABLE tjj (key2 UInt64, key1 UInt64, key3 UInt64, attr UInt64) ENGINE = Join(ALL, INNER, key3, key2, key1); +INSERT INTO tjj VALUES (11, 11, 11, 1000), (21, 21, 21, 2000), (31, 31, 31, 3000), (41, 41, 41, 4000), (51, 51, 51, 5000), (61, 61, 61, 6000); + +SELECT * FROM t1 ALL INNER JOIN tjj ON t1.key1 = tjj.key1 AND t1.key1 = tjj.key2 AND t1.key1 = tjj.key3 ORDER BY key1; +SELECT * FROM t1 ALL INNER JOIN tjj ON t1.key1 = tjj.key1 AND t1.key1 = tjj.key3 AND t1.key1 = tjj.key2 ORDER BY key1; + +DROP TABLE IF EXISTS t1; +DROP TABLE IF EXISTS tj; +DROP TABLE IF EXISTS tjj; From 2c098cbcf732f898c38039de66da847984e3653f Mon Sep 17 00:00:00 2001 From: Ivan Blinkov Date: Wed, 7 Dec 2022 18:15:59 +0300 Subject: [PATCH 076/127] [docs] link to GitHub from the install page --- docs/en/getting-started/install.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/getting-started/install.md b/docs/en/getting-started/install.md index 1ed93f7a1cb..391d3a3f59a 100644 --- a/docs/en/getting-started/install.md +++ b/docs/en/getting-started/install.md @@ -9,7 +9,7 @@ slug: /en/install You have two options for getting up and running with ClickHouse: - **[ClickHouse Cloud](https://clickhouse.com/cloud/):** the official ClickHouse as a service, - built by, maintained, and supported by the creators of ClickHouse -- **Self-managed ClickHouse:** ClickHouse can run on any Linux, FreeBSD, or Mac OS X with x86_64, AArch64, or PowerPC64LE CPU architecture +- **[Self-managed ClickHouse](https://github.com/ClickHouse/ClickHouse):** ClickHouse can run on any Linux, FreeBSD, or Mac OS X with x86_64, AArch64, or PowerPC64LE CPU architecture ## ClickHouse Cloud From db50ce063465ef012d3604e91f2f74c79fb5e727 Mon Sep 17 00:00:00 2001 From: serxa Date: Wed, 7 Dec 2022 16:08:33 +0000 Subject: [PATCH 077/127] refactoring --- src/Access/SettingsConstraints.cpp | 5 +---- src/Access/resolveSetting.h | 17 +++++++++++++++++ 2 files changed, 18 insertions(+), 4 deletions(-) diff --git a/src/Access/SettingsConstraints.cpp b/src/Access/SettingsConstraints.cpp index 6cd44506304..9983adcb417 100644 --- a/src/Access/SettingsConstraints.cpp +++ b/src/Access/SettingsConstraints.cpp @@ -304,10 +304,7 @@ SettingsConstraints::Checker SettingsConstraints::getChecker(const Settings & cu SettingsConstraints::Checker SettingsConstraints::getMergeTreeChecker(std::string_view short_name) const { - String full_name(MERGE_TREE_SETTINGS_PREFIX); - full_name += short_name; // Just because you cannot concatenate `std::string_view` and `std::string` using operator+ in C++20 yet - - auto it = constraints.find(full_name); + auto it = constraints.find(settingFullName(short_name)); if (it == constraints.end()) return Checker(); // Allowed return Checker(it->second); diff --git a/src/Access/resolveSetting.h b/src/Access/resolveSetting.h index af9e4a08d61..8469c60bbe3 100644 --- a/src/Access/resolveSetting.h +++ b/src/Access/resolveSetting.h @@ -70,4 +70,21 @@ inline bool settingIsBuiltin(std::string_view full_name) }); } +template +inline String settingFullName(std::string_view short_name); + +template <> +inline String settingFullName(std::string_view short_name) +{ + return String(short_name); +} + +template <> +inline String settingFullName(std::string_view short_name) +{ + String full_name(MERGE_TREE_SETTINGS_PREFIX); + full_name += short_name; // Just because you cannot concatenate `std::string_view` and `std::string` using operator+ in C++20 yet + return full_name; +} + } From 4b50e5c15eec25e2578f0c04f492bf3bd2730f81 Mon Sep 17 00:00:00 2001 From: vdimir Date: Wed, 7 Dec 2022 16:27:01 +0000 Subject: [PATCH 078/127] fix --- src/Interpreters/HashJoin.cpp | 7 ++++--- src/Interpreters/TableJoin.cpp | 3 --- src/Interpreters/TreeRewriter.cpp | 6 ++++++ src/Storages/StorageJoin.cpp | 8 ++++++-- .../0_stateless/02498_storage_join_key_positions.sql | 8 +++++--- 5 files changed, 21 insertions(+), 11 deletions(-) diff --git a/src/Interpreters/HashJoin.cpp b/src/Interpreters/HashJoin.cpp index 8c3cd806ff1..f79ea950436 100644 --- a/src/Interpreters/HashJoin.cpp +++ b/src/Interpreters/HashJoin.cpp @@ -225,7 +225,8 @@ HashJoin::HashJoin(std::shared_ptr table_join_, const Block & right_s , right_sample_block(right_sample_block_) , log(&Poco::Logger::get("HashJoin")) { - LOG_DEBUG(log, "HashJoin. Datatype: {}, kind: {}, strictness: {}", data->type, kind, strictness); + LOG_DEBUG(log, "Datatype: {}, kind: {}, strictness: {}", data->type, kind, strictness); + LOG_DEBUG(log, "Keys: {}", TableJoin::formatClauses(table_join->getClauses(), true)); if (isCrossOrComma(kind)) { @@ -1492,7 +1493,7 @@ void HashJoin::joinBlockImpl( { const auto & right_key = required_right_keys.getByPosition(i); auto right_col_name = getTableJoin().renamedRightColumnName(right_key.name); - if (!block.findByName(right_col_name /*right_key.name*/)) + if (!block.findByName(right_col_name)) { const auto & left_name = required_right_keys_sources[i]; @@ -1512,7 +1513,7 @@ void HashJoin::joinBlockImpl( block.insert(std::move(right_col)); if constexpr (jf.need_replication) - right_keys_to_replicate.push_back(block.getPositionByName(right_key.name)); + right_keys_to_replicate.push_back(block.getPositionByName(right_col_name)); } } } diff --git a/src/Interpreters/TableJoin.cpp b/src/Interpreters/TableJoin.cpp index 766378f0fbd..878455c9fd1 100644 --- a/src/Interpreters/TableJoin.cpp +++ b/src/Interpreters/TableJoin.cpp @@ -653,9 +653,6 @@ void TableJoin::setStorageJoin(std::shared_ptr storage) void TableJoin::setStorageJoin(std::shared_ptr storage) { - if (clauses.empty()) - throw Exception(ErrorCodes::INCOMPATIBLE_TYPE_OF_JOIN, - "StorageJoin keys should match JOIN keys, expected JOIN ON [{}]", fmt::join(storage->getKeyNames(), ", ")); right_storage_join = storage; } diff --git a/src/Interpreters/TreeRewriter.cpp b/src/Interpreters/TreeRewriter.cpp index 828f332af1d..e49ed73fc9a 100644 --- a/src/Interpreters/TreeRewriter.cpp +++ b/src/Interpreters/TreeRewriter.cpp @@ -48,6 +48,7 @@ #include #include +#include #include #include @@ -60,6 +61,7 @@ namespace ErrorCodes extern const int EMPTY_LIST_OF_COLUMNS_QUERIED; extern const int EMPTY_NESTED_TABLE; extern const int EXPECTED_ALL_OR_ANY; + extern const int INCOMPATIBLE_TYPE_OF_JOIN; extern const int INVALID_JOIN_ON_EXPRESSION; extern const int LOGICAL_ERROR; extern const int NOT_IMPLEMENTED; @@ -757,6 +759,10 @@ void collectJoinedColumns(TableJoin & analyzed_join, ASTTableJoin & table_join, throw Exception(ErrorCodes::INVALID_JOIN_ON_EXPRESSION, "Cannot get JOIN keys from JOIN ON section: {}", queryToString(table_join.on_expression)); + if (const auto storage_join = analyzed_join.getStorageJoin()) + throw Exception(ErrorCodes::INCOMPATIBLE_TYPE_OF_JOIN, + "StorageJoin keys should match JOIN keys, expected JOIN ON [{}]", fmt::join(storage_join->getKeyNames(), ", ")); + bool join_on_const_ok = tryJoinOnConst(analyzed_join, table_join.on_expression, context); if (!join_on_const_ok) throw Exception(ErrorCodes::INVALID_JOIN_ON_EXPRESSION, diff --git a/src/Storages/StorageJoin.cpp b/src/Storages/StorageJoin.cpp index 50bdb11952d..2e799b0b0db 100644 --- a/src/Storages/StorageJoin.cpp +++ b/src/Storages/StorageJoin.cpp @@ -174,8 +174,12 @@ HashJoinPtr StorageJoin::getJoinLocked(std::shared_ptr analyzed_join, "Table {} needs the same join_use_nulls setting as present in LEFT or FULL JOIN", getStorageID().getNameForLogs()); - const auto & key_names_right = analyzed_join->getOnlyClause().key_names_right; - const auto & key_names_left = analyzed_join->getOnlyClause().key_names_left; + const auto & join_on = analyzed_join->getOnlyClause(); + if (join_on.on_filter_condition_left || join_on.on_filter_condition_right) + throw Exception(ErrorCodes::INCOMPATIBLE_TYPE_OF_JOIN, "ON section of JOIN with filter conditions is not implemented"); + + const auto & key_names_right = join_on.key_names_right; + const auto & key_names_left = join_on.key_names_left; if (key_names.size() != key_names_right.size() || key_names.size() != key_names_left.size()) throw Exception(ErrorCodes::INCOMPATIBLE_TYPE_OF_JOIN, "Number of keys in JOIN ON section ({}) doesn't match number of keys in Join engine ({})", diff --git a/tests/queries/0_stateless/02498_storage_join_key_positions.sql b/tests/queries/0_stateless/02498_storage_join_key_positions.sql index 34f7e3543aa..96687dab577 100644 --- a/tests/queries/0_stateless/02498_storage_join_key_positions.sql +++ b/tests/queries/0_stateless/02498_storage_join_key_positions.sql @@ -22,9 +22,11 @@ SELECT * FROM (SELECT key3 AS c, key1 AS a, key2 AS b FROM t1) AS t1 ALL INNER J SELECT * FROM (SELECT key3 AS c, key1 AS a, key2 AS b FROM t1) AS t1 ALL INNER JOIN tj ON t1.a = tj.key1 AND t1.b = tj.key2 AND t1.c = tj.key3 ORDER BY t1.a; SELECT * FROM (SELECT key3 AS c, key1 AS a, key2 AS b FROM t1) AS t1 ALL INNER JOIN tj ON t1.c = tj.key3 AND t1.a = tj.key1 AND t1.b = tj.key2 ORDER BY t1.a; -SELECT * FROM t1 ALL INNER JOIN tj ON t1.key1 = tj.key1 AND t1.key2 = tj.key2 AND t1.key3 = tj.key3 AND 1; -- { serverError INCOMPATIBLE_TYPE_OF_JOIN } -SELECT * FROM t1 ALL INNER JOIN tj ON t1.key1 = tj.key1 AND t1.key2 = tj.key2 AND t1.key3 = tj.key3 AND 0; -- { serverError INCOMPATIBLE_TYPE_OF_JOIN } -SELECT * FROM t1 ALL INNER JOIN tj ON t1.key1 = tj.key1 AND t1.key2 = tj.key2 AND t1.key3 = tj.key3 AND NULL; -- { serverError INCOMPATIBLE_TYPE_OF_JOIN } +SELECT * FROM t1 ALL INNER JOIN tj ON 1; -- { serverError INCOMPATIBLE_TYPE_OF_JOIN } +SELECT * FROM t1 ALL INNER JOIN tj ON 0; -- { serverError INCOMPATIBLE_TYPE_OF_JOIN } +SELECT * FROM t1 ALL INNER JOIN tj ON NULL; -- { serverError INCOMPATIBLE_TYPE_OF_JOIN } +SELECT * FROM t1 ALL INNER JOIN tj ON 1 == 1; -- { serverError INCOMPATIBLE_TYPE_OF_JOIN } +SELECT * FROM t1 ALL INNER JOIN tj ON 1 != 1; -- { serverError INCOMPATIBLE_TYPE_OF_JOIN } SELECT * FROM t1 ALL INNER JOIN tj USING (key2, key3); -- { serverError INCOMPATIBLE_TYPE_OF_JOIN } SELECT * FROM t1 ALL INNER JOIN tj USING (key1, key2, attr); -- { serverError INCOMPATIBLE_TYPE_OF_JOIN } From e7a5fd923667b7dedd6612405fb704f31dc91987 Mon Sep 17 00:00:00 2001 From: Vladimir C Date: Wed, 7 Dec 2022 17:38:31 +0100 Subject: [PATCH 079/127] fix --- src/Interpreters/TableJoin.cpp | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/src/Interpreters/TableJoin.cpp b/src/Interpreters/TableJoin.cpp index 878455c9fd1..5d065e564b2 100644 --- a/src/Interpreters/TableJoin.cpp +++ b/src/Interpreters/TableJoin.cpp @@ -35,10 +35,9 @@ namespace DB namespace ErrorCodes { - extern const int INCOMPATIBLE_TYPE_OF_JOIN; + extern const int TYPE_MISMATCH; extern const int LOGICAL_ERROR; extern const int NOT_IMPLEMENTED; - extern const int TYPE_MISMATCH; } namespace From 05788245dde02d85ef12fbde8843a70963e881eb Mon Sep 17 00:00:00 2001 From: serxa Date: Wed, 7 Dec 2022 20:16:10 +0000 Subject: [PATCH 080/127] add test --- .../__init__.py | 0 .../test.py | 43 +++++++++++++++++++ .../users.xml | 28 ++++++++++++ 3 files changed, 71 insertions(+) create mode 100644 tests/integration/test_merge_tree_settings_constraints/__init__.py create mode 100644 tests/integration/test_merge_tree_settings_constraints/test.py create mode 100644 tests/integration/test_merge_tree_settings_constraints/users.xml diff --git a/tests/integration/test_merge_tree_settings_constraints/__init__.py b/tests/integration/test_merge_tree_settings_constraints/__init__.py new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/integration/test_merge_tree_settings_constraints/test.py b/tests/integration/test_merge_tree_settings_constraints/test.py new file mode 100644 index 00000000000..6187a099cfe --- /dev/null +++ b/tests/integration/test_merge_tree_settings_constraints/test.py @@ -0,0 +1,43 @@ +import pytest +import asyncio +import re +import random +import os.path +from helpers.cluster import ClickHouseCluster +from helpers.test_tools import assert_eq_with_retry, TSV + +cluster = ClickHouseCluster(__file__) +instance = cluster.add_instance("instance", user_configs=["users.xml"]) + + +@pytest.fixture(scope="module", autouse=True) +def start_cluster(): + try: + cluster.start() + yield cluster + finally: + cluster.shutdown() + + +def test_merge_tree_settings_constraints(): + expected_error = "Setting min_bytes_for_wide_part should" + + assert expected_error in instance.query_and_get_error( + f"CREATE TABLE wrong_table (number Int64) engine = MergeTree() ORDER BY number SETTINGS min_bytes_for_wide_part = 100" + ) + + assert expected_error in instance.query_and_get_error( + f"CREATE TABLE wrong_table (number Int64) engine = MergeTree() ORDER BY number SETTINGS min_bytes_for_wide_part = 1000000000" + ) + + instance.query( + f"CREATE TABLE good_table (number Int64) engine = MergeTree() ORDER BY number SETTINGS min_bytes_for_wide_part = 10000000" + ) + + assert expected_error in instance.query_and_get_error( + f"ALTER TABLE good_table MODIFY SETTING min_bytes_for_wide_part = 100" + ) + + assert expected_error in instance.query_and_get_error( + f"ALTER TABLE good_table MODIFY SETTING min_bytes_for_wide_part = 1000000000" + ) diff --git a/tests/integration/test_merge_tree_settings_constraints/users.xml b/tests/integration/test_merge_tree_settings_constraints/users.xml new file mode 100644 index 00000000000..82f5995aa67 --- /dev/null +++ b/tests/integration/test_merge_tree_settings_constraints/users.xml @@ -0,0 +1,28 @@ + + + + + + 1000 + 100000000 + + + + + + 1 + + + + + + + + + ::/0 + + + default + + + \ No newline at end of file From 6428b8a1bb072322cdd0040a1310bcdd13ef016c Mon Sep 17 00:00:00 2001 From: serxa Date: Wed, 7 Dec 2022 20:36:04 +0000 Subject: [PATCH 081/127] check one more case in test --- .../integration/test_merge_tree_settings_constraints/test.py | 5 +++++ .../test_merge_tree_settings_constraints/users.xml | 3 +++ 2 files changed, 8 insertions(+) diff --git a/tests/integration/test_merge_tree_settings_constraints/test.py b/tests/integration/test_merge_tree_settings_constraints/test.py index 6187a099cfe..0bb0179108d 100644 --- a/tests/integration/test_merge_tree_settings_constraints/test.py +++ b/tests/integration/test_merge_tree_settings_constraints/test.py @@ -20,6 +20,11 @@ def start_cluster(): def test_merge_tree_settings_constraints(): + + assert "Setting storage_policy should not be changed" in instance.query_and_get_error( + f"CREATE TABLE wrong_table (number Int64) engine = MergeTree() ORDER BY number SETTINGS storage_policy = 'secret_policy'" + ) + expected_error = "Setting min_bytes_for_wide_part should" assert expected_error in instance.query_and_get_error( diff --git a/tests/integration/test_merge_tree_settings_constraints/users.xml b/tests/integration/test_merge_tree_settings_constraints/users.xml index 82f5995aa67..ec4489e0434 100644 --- a/tests/integration/test_merge_tree_settings_constraints/users.xml +++ b/tests/integration/test_merge_tree_settings_constraints/users.xml @@ -6,6 +6,9 @@ 1000 100000000 + + + From b357652c4b3335a195028ad973b94e1958ff7aad Mon Sep 17 00:00:00 2001 From: serxa Date: Wed, 7 Dec 2022 20:40:58 +0000 Subject: [PATCH 082/127] fix stateless tests --- tests/queries/0_stateless/01221_system_settings.reference | 2 +- .../0_stateless/02117_show_create_table_system.reference | 6 ++++++ 2 files changed, 7 insertions(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/01221_system_settings.reference b/tests/queries/0_stateless/01221_system_settings.reference index a90b3eac6dc..554b584635f 100644 --- a/tests/queries/0_stateless/01221_system_settings.reference +++ b/tests/queries/0_stateless/01221_system_settings.reference @@ -1,4 +1,4 @@ send_timeout 300 0 Timeout for sending data to network, in seconds. If client needs to sent some data, but it did not able to send any bytes in this interval, exception is thrown. If you set this setting on client, the \'receive_timeout\' for the socket will be also set on the corresponding connection end on the server. \N \N 0 Seconds -storage_policy default 0 Name of storage disk policy String +storage_policy default 0 Name of storage disk policy \N \N 0 String 1 1 diff --git a/tests/queries/0_stateless/02117_show_create_table_system.reference b/tests/queries/0_stateless/02117_show_create_table_system.reference index c206a41a03e..b0212ffee3c 100644 --- a/tests/queries/0_stateless/02117_show_create_table_system.reference +++ b/tests/queries/0_stateless/02117_show_create_table_system.reference @@ -330,6 +330,9 @@ CREATE TABLE system.merge_tree_settings `value` String, `changed` UInt8, `description` String, + `min` Nullable(String), + `max` Nullable(String), + `readonly` UInt8, `type` String ) ENGINE = SystemMergeTreeSettings @@ -889,6 +892,9 @@ CREATE TABLE system.replicated_merge_tree_settings `value` String, `changed` UInt8, `description` String, + `min` Nullable(String), + `max` Nullable(String), + `readonly` UInt8, `type` String ) ENGINE = SystemReplicatedMergeTreeSettings From e1584c5d3a38b0180b966462aea814d1f485b478 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Wed, 7 Dec 2022 18:24:08 +0100 Subject: [PATCH 083/127] Provide sha512 hashsums for tgz artifacts --- packages/build | 2 ++ tests/ci/build_report_check.py | 2 +- 2 files changed, 3 insertions(+), 1 deletion(-) diff --git a/packages/build b/packages/build index 531e068338d..c8fb77e9371 100755 --- a/packages/build +++ b/packages/build @@ -111,6 +111,8 @@ EOF tar -czf "$TARBALL" -C "$OUTPUT_DIR" "$PKG_DIR" fi + sha512sum "$TARBALL" > "$TARBALL".sha512 + rm -r "$PKG_PATH" } diff --git a/tests/ci/build_report_check.py b/tests/ci/build_report_check.py index 03e18d7766e..4ece21d5449 100644 --- a/tests/ci/build_report_check.py +++ b/tests/ci/build_report_check.py @@ -74,7 +74,7 @@ def group_by_artifacts(build_urls: List[str]) -> Dict[str, List[str]]: groups["apk"].append(url) elif url.endswith(".rpm"): groups["rpm"].append(url) - elif url.endswith(".tgz"): + elif url.endswith(".tgz") or url.endswith(".tgz.sha512"): groups["tgz"].append(url) else: groups["binary"].append(url) From 6d6bcd9ba2a44cc606ad596d8005d408562b89da Mon Sep 17 00:00:00 2001 From: Han Fei Date: Wed, 7 Dec 2022 22:28:55 +0100 Subject: [PATCH 084/127] Update src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp Co-authored-by: alesapin --- src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp index 38801b818dc..a8a101e4115 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp @@ -151,7 +151,7 @@ namespace auto cols = partition.block_with_partition.block.getColumns(); for (auto & col : cols) { - col = col -> filter(vec, rows - remove_count); + col = col->filter(vec, rows - remove_count); } partition.block_with_partition.block.setColumns(cols); From 7a3dcd196eb08204ddf6fd20591bfc409d7eacc7 Mon Sep 17 00:00:00 2001 From: Han Fei Date: Wed, 7 Dec 2022 22:30:03 +0100 Subject: [PATCH 085/127] Update src/Storages/MergeTree/EphemeralLockInZooKeeper.cpp Co-authored-by: alesapin --- src/Storages/MergeTree/EphemeralLockInZooKeeper.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/MergeTree/EphemeralLockInZooKeeper.cpp b/src/Storages/MergeTree/EphemeralLockInZooKeeper.cpp index f0474e48a75..1d89fd4ab2a 100644 --- a/src/Storages/MergeTree/EphemeralLockInZooKeeper.cpp +++ b/src/Storages/MergeTree/EphemeralLockInZooKeeper.cpp @@ -81,7 +81,7 @@ std::optional createEphemeralLockInZooKeeper( return {}; } zkutil::KeeperMultiException::check(e, ops, responses); // This should always throw the proper exception - throw Exception("Unable to handle error {} when acquiring ephemeral lock in ZK", ErrorCodes::LOGICAL_ERROR); + throw Exception(ErrorCodes::LOGICAL_ERROR, "Unable to handle error {} when acquiring ephemeral lock in ZK", toString(e)); } path = dynamic_cast(responses.back().get())->path_created; From 47c3508337e0ee3942ff123d90dc6b890e80acc4 Mon Sep 17 00:00:00 2001 From: jinjunzh Date: Wed, 7 Dec 2022 16:41:12 -0500 Subject: [PATCH 086/127] fixed build issues for QPL 0.3.0 --- contrib/qpl-cmake/CMakeLists.txt | 22 ++++++++++++------- .../CompressionCodecDeflateQpl.cpp | 16 +++++++------- 2 files changed, 22 insertions(+), 16 deletions(-) diff --git a/contrib/qpl-cmake/CMakeLists.txt b/contrib/qpl-cmake/CMakeLists.txt index dc90f07a9bc..beef8432e7a 100644 --- a/contrib/qpl-cmake/CMakeLists.txt +++ b/contrib/qpl-cmake/CMakeLists.txt @@ -15,7 +15,7 @@ set (QPL_SRC_DIR "${ClickHouse_SOURCE_DIR}/contrib/qpl/sources") set (QPL_BINARY_DIR "${ClickHouse_BINARY_DIR}/build/contrib/qpl") set (UUID_DIR "${ClickHouse_SOURCE_DIR}/contrib/qpl-cmake") -set (EFFICIENT_WAIT ON) +set (EFFICIENT_WAIT OFF) set (BLOCK_ON_FAULT ON) set (LOG_HW_INIT OFF) set (SANITIZE_MEMORY OFF) @@ -42,7 +42,7 @@ include("${QPL_PROJECT_DIR}/cmake/CompileOptions.cmake") include(CheckLanguage) check_language(ASM_NASM) if(NOT CMAKE_ASM_NASM_COMPILER) - message(FATAL_ERROR "Please install NASM from 'https://www.nasm.us/' because NASM compiler can not be found!") + message(FATAL_ERROR "Please install NASM from 'https://www.nasm.us/' because NASM compiler can not be found!") endif() # [SUBDIR]isal @@ -110,18 +110,18 @@ target_compile_options(isal PRIVATE "$<$:>" "$<$:>") -target_compile_options(isal_asm PUBLIC "-I${QPL_SRC_DIR}/isal/include/" - PUBLIC "-I${QPL_SRC_DIR}/isal/igzip/" - PUBLIC "-I${QPL_SRC_DIR}/isal/crc/" - PUBLIC "-DQPL_LIB") +target_compile_options(isal_asm PRIVATE "-I${QPL_SRC_DIR}/isal/include/" + PRIVATE "-I${QPL_SRC_DIR}/isal/igzip/" + PRIVATE "-I${QPL_SRC_DIR}/isal/crc/" + PRIVATE "-DQPL_LIB") # AS_FEATURE_LEVEL=10 means "Check SIMD capabilities of the target system at runtime and use up to AVX512 if available". # AS_FEATURE_LEVEL=5 means "Check SIMD capabilities of the target system at runtime and use up to AVX2 if available". # HAVE_KNOWS_AVX512 means rely on AVX512 being available on the target system. if (ENABLE_AVX512) - target_compile_options(isal_asm PUBLIC "-DHAVE_AS_KNOWS_AVX512" "-DAS_FEATURE_LEVEL=10") + target_compile_options(isal_asm PRIVATE "-DHAVE_AS_KNOWS_AVX512" "-DAS_FEATURE_LEVEL=10") else() - target_compile_options(isal_asm PUBLIC "-DAS_FEATURE_LEVEL=5") + target_compile_options(isal_asm PRIVATE "-DAS_FEATURE_LEVEL=5") endif() # Here must remove "-fno-sanitize=undefined" from COMPILE_OPTIONS. @@ -315,7 +315,13 @@ target_compile_definitions(_qpl PRIVATE -DQPL_BADARG_CHECK PUBLIC -DENABLE_QPL_COMPRESSION) +find_library(LIBACCEL accel-config) +if(NOT LIBACCEL) + message(FATAL_ERROR "Please install QPL dependency library:libaccel-config from https://github.com/intel/idxd-config") +endif() + target_link_libraries(_qpl + PRIVATE ${LIBACCEL} PRIVATE ${CMAKE_DL_LIBS}) add_library (ch_contrib::qpl ALIAS _qpl) diff --git a/src/Compression/CompressionCodecDeflateQpl.cpp b/src/Compression/CompressionCodecDeflateQpl.cpp index 348496a2753..117dfae4009 100644 --- a/src/Compression/CompressionCodecDeflateQpl.cpp +++ b/src/Compression/CompressionCodecDeflateQpl.cpp @@ -44,10 +44,10 @@ DeflateQplJobHWPool::DeflateQplJobHWPool() for (UInt32 index = 0; index < MAX_HW_JOB_NUMBER; ++index) { qpl_job * qpl_job_ptr = reinterpret_cast(hw_jobs_buffer.get() + index * job_size); - if (qpl_init_job(qpl_path_hardware, qpl_job_ptr) != QPL_STS_OK) + if (auto status = qpl_init_job(qpl_path_hardware, qpl_job_ptr); status != QPL_STS_OK) { job_pool_ready = false; - LOG_WARNING(log, "Initialization of hardware-assisted DeflateQpl codec failed, falling back to software DeflateQpl codec. Please check if Intel In-Memory Analytics Accelerator (IAA) is properly set up. QPL Version: {}.",qpl_version); + LOG_WARNING(log, "Initialization of hardware-assisted DeflateQpl codec failed: {} , falling back to software DeflateQpl codec. Please check if Intel In-Memory Analytics Accelerator (IAA) is properly set up. QPL Version: {}.", static_cast(status), qpl_version); return; } hw_job_ptr_pool[index] = qpl_job_ptr; @@ -165,7 +165,7 @@ Int32 HardwareCodecDeflateQpl::doCompressData(const char * source, UInt32 source } else { - LOG_WARNING(log, "DeflateQpl HW codec failed, falling back to SW codec.(Details: doCompressData->qpl_execute_job with error code: {} - please refer to qpl_status in ./contrib/qpl/include/qpl/c_api/status.h)", status); + LOG_WARNING(log, "DeflateQpl HW codec failed, falling back to SW codec.(Details: doCompressData->qpl_execute_job with error code: {} - please refer to qpl_status in ./contrib/qpl/include/qpl/c_api/status.h)", static_cast(status)); DeflateQplJobHWPool::instance().releaseJob(job_id); return RET_ERROR; } @@ -193,7 +193,7 @@ Int32 HardwareCodecDeflateQpl::doDecompressDataSynchronous(const char * source, if (auto status = qpl_submit_job(job_ptr); status != QPL_STS_OK) { DeflateQplJobHWPool::instance().releaseJob(job_id); - LOG_WARNING(log, "DeflateQpl HW codec failed, falling back to SW codec.(Details: doDecompressDataSynchronous->qpl_execute_job with error code: {} - please refer to qpl_status in ./contrib/qpl/include/qpl/c_api/status.h)", status); + LOG_WARNING(log, "DeflateQpl HW codec failed, falling back to SW codec.(Details: doDecompressDataSynchronous->qpl_execute_job with error code: {} - please refer to qpl_status in ./contrib/qpl/include/qpl/c_api/status.h)", static_cast(status)); return RET_ERROR; } /// Busy waiting till job complete. @@ -233,7 +233,7 @@ Int32 HardwareCodecDeflateQpl::doDecompressDataAsynchronous(const char * source, else { DeflateQplJobHWPool::instance().releaseJob(job_id); - LOG_WARNING(log, "DeflateQpl HW codec failed, falling back to SW codec.(Details: doDecompressDataAsynchronous->qpl_execute_job with error code: {} - please refer to qpl_status in ./contrib/qpl/include/qpl/c_api/status.h)", status); + LOG_WARNING(log, "DeflateQpl HW codec failed, falling back to SW codec.(Details: doDecompressDataAsynchronous->qpl_execute_job with error code: {} - please refer to qpl_status in ./contrib/qpl/include/qpl/c_api/status.h)", static_cast(status)); return RET_ERROR; } } @@ -289,7 +289,7 @@ qpl_job * SoftwareCodecDeflateQpl::getJobCodecPtr() // Job initialization if (auto status = qpl_init_job(qpl_path_software, sw_job); status != QPL_STS_OK) throw Exception(ErrorCodes::CANNOT_COMPRESS, - "Initialization of DeflateQpl software fallback codec failed. (Details: qpl_init_job with error code: {} - please refer to qpl_status in ./contrib/qpl/include/qpl/c_api/status.h)", status); + "Initialization of DeflateQpl software fallback codec failed. (Details: qpl_init_job with error code: {} - please refer to qpl_status in ./contrib/qpl/include/qpl/c_api/status.h)", static_cast(status)); } return sw_job; } @@ -308,7 +308,7 @@ UInt32 SoftwareCodecDeflateQpl::doCompressData(const char * source, UInt32 sourc if (auto status = qpl_execute_job(job_ptr); status != QPL_STS_OK) throw Exception(ErrorCodes::CANNOT_COMPRESS, - "Execution of DeflateQpl software fallback codec failed. (Details: qpl_execute_job with error code: {} - please refer to qpl_status in ./contrib/qpl/include/qpl/c_api/status.h)", status); + "Execution of DeflateQpl software fallback codec failed. (Details: qpl_execute_job with error code: {} - please refer to qpl_status in ./contrib/qpl/include/qpl/c_api/status.h)", static_cast(status)); return job_ptr->total_out; } @@ -327,7 +327,7 @@ void SoftwareCodecDeflateQpl::doDecompressData(const char * source, UInt32 sourc if (auto status = qpl_execute_job(job_ptr); status != QPL_STS_OK) throw Exception(ErrorCodes::CANNOT_DECOMPRESS, - "Execution of DeflateQpl software fallback codec failed. (Details: qpl_execute_job with error code: {} - please refer to qpl_status in ./contrib/qpl/include/qpl/c_api/status.h)", status); + "Execution of DeflateQpl software fallback codec failed. (Details: qpl_execute_job with error code: {} - please refer to qpl_status in ./contrib/qpl/include/qpl/c_api/status.h)", static_cast(status)); } CompressionCodecDeflateQpl::CompressionCodecDeflateQpl() From 8594274e9956347f58fd546e7cb75064ac86f1cd Mon Sep 17 00:00:00 2001 From: vahid-sohrabloo Date: Wed, 7 Dec 2022 22:50:27 +0100 Subject: [PATCH 087/127] fix docs --- docs/en/sql-reference/statements/alter/update.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/statements/alter/update.md b/docs/en/sql-reference/statements/alter/update.md index 5d27c382982..234812f6ed4 100644 --- a/docs/en/sql-reference/statements/alter/update.md +++ b/docs/en/sql-reference/statements/alter/update.md @@ -7,7 +7,7 @@ sidebar_label: UPDATE # ALTER TABLE … UPDATE Statements ``` sql -ALTER TABLE [db.]table [ON CLUSTER cluster] UPDATE column1 = expr1 [, ...] WHERE filter_expr +ALTER TABLE [db.]table [ON CLUSTER cluster] UPDATE column1 = expr1 [, ...] [IN PARTITION partition_id] WHERE filter_expr ``` Manipulates data matching the specified filtering expression. Implemented as a [mutation](/docs/en/sql-reference/statements/alter/index.md#mutations). From 3c11aca040aa89b22a76600147abbccbb6de8f6f Mon Sep 17 00:00:00 2001 From: Han Fei Date: Wed, 7 Dec 2022 23:40:52 +0100 Subject: [PATCH 088/127] address comments --- .../MergeTree/EphemeralLockInZooKeeper.cpp | 18 +++--- .../MergeTree/EphemeralLockInZooKeeper.h | 2 + .../MergeTree/ReplicatedMergeTreeSink.cpp | 59 +++++++++---------- .../MergeTree/ReplicatedMergeTreeSink.h | 9 ++- src/Storages/StorageReplicatedMergeTree.cpp | 10 ++-- src/Storages/StorageReplicatedMergeTree.h | 2 +- 6 files changed, 49 insertions(+), 51 deletions(-) diff --git a/src/Storages/MergeTree/EphemeralLockInZooKeeper.cpp b/src/Storages/MergeTree/EphemeralLockInZooKeeper.cpp index 1d89fd4ab2a..e62194dc5e5 100644 --- a/src/Storages/MergeTree/EphemeralLockInZooKeeper.cpp +++ b/src/Storages/MergeTree/EphemeralLockInZooKeeper.cpp @@ -59,17 +59,15 @@ std::optional createEphemeralLockInZooKeeper( { if constexpr (async_insert) { - for (size_t i = 0; i < deduplication_path.size(); i++) + auto failed_idx = zkutil::getFailedOpIndex(Coordination::Error::ZNODEEXISTS, responses); + if (failed_idx < deduplication_path.size() * 2) { - if (responses[i*2]->error == Coordination::Error::ZNODEEXISTS) - { - const String & failed_op_path = deduplication_path[i]; - LOG_DEBUG( - &Poco::Logger::get("createEphemeralLockInZooKeeper"), - "Deduplication path already exists: deduplication_path={}", - failed_op_path); - return EphemeralLockInZooKeeper{"", nullptr, "", failed_op_path}; - } + const String & failed_op_path = deduplication_path[failed_idx / 2]; + LOG_DEBUG( + &Poco::Logger::get("createEphemeralLockInZooKeeper"), + "Deduplication path already exists: deduplication_path={}", + failed_op_path); + return EphemeralLockInZooKeeper{"", nullptr, "", failed_op_path}; } } else if (responses[0]->error == Coordination::Error::ZNODEEXISTS) diff --git a/src/Storages/MergeTree/EphemeralLockInZooKeeper.h b/src/Storages/MergeTree/EphemeralLockInZooKeeper.h index eef0366dc8b..a8b60c6ef8a 100644 --- a/src/Storages/MergeTree/EphemeralLockInZooKeeper.h +++ b/src/Storages/MergeTree/EphemeralLockInZooKeeper.h @@ -67,6 +67,8 @@ public: return path; } + // In case of async inserts, we try to get locks for multiple inserts and need to know which insert is conflicted. + // That's why we need this function. String getConflictPath() const { return conflict_path; diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp index a8a101e4115..dbb035693ef 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp @@ -8,6 +8,7 @@ #include #include #include +#include namespace ProfileEvents { @@ -36,7 +37,7 @@ namespace ErrorCodes } template -struct ReplicatedMergeTreeSink::DelayedChunk +struct ReplicatedMergeTreeSinkImpl::DelayedChunk { struct Partition { @@ -80,24 +81,17 @@ struct ReplicatedMergeTreeSink::DelayedChunk namespace { - /// Convert block id vector to string. Output at most 50 rows. + /// Convert block id vector to string. Output at most 50 ids. template inline String toString(const std::vector & vec) { - String res = "{"; size_t size = vec.size(); if (size > 50) size = 50; - for (size_t i = 0; i < size; ++i) - { - res += DB::toString(vec[i]); - if (i + 1 < size) - res += ","; - } - return res + "}"; + return fmt::format("({})", fmt::join(vec.begin(), vec.begin() + size, ",")); } /// remove the conflict parts of block for rewriting again. - void rewriteBlock(Poco::Logger * log, typename ReplicatedMergeTreeSink::DelayedChunk::Partition & partition, const std::vector & block_paths) + void rewriteBlock(Poco::Logger * log, typename ReplicatedMergeTreeSinkImpl::DelayedChunk::Partition & partition, const std::vector & block_paths) { std::vector offset_idx; for (const auto & raw_path : block_paths) @@ -164,7 +158,7 @@ namespace { size_t start = 0; auto cols = block.block.getColumns(); - std::vector block_id_vec; + std::vector block_id_vec(block.offsets->offsets.size()); for (auto offset : block.offsets->offsets) { SipHash hash; @@ -187,7 +181,7 @@ namespace } template -ReplicatedMergeTreeSink::ReplicatedMergeTreeSink( +ReplicatedMergeTreeSinkImpl::ReplicatedMergeTreeSinkImpl( StorageReplicatedMergeTree & storage_, const StorageMetadataPtr & metadata_snapshot_, size_t quorum_size, @@ -217,7 +211,7 @@ ReplicatedMergeTreeSink::ReplicatedMergeTreeSink( } template -ReplicatedMergeTreeSink::~ReplicatedMergeTreeSink() = default; +ReplicatedMergeTreeSinkImpl::~ReplicatedMergeTreeSinkImpl() = default; /// Allow to verify that the session in ZooKeeper is still alive. static void assertSessionIsNotExpired(const zkutil::ZooKeeperPtr & zookeeper) @@ -230,7 +224,7 @@ static void assertSessionIsNotExpired(const zkutil::ZooKeeperPtr & zookeeper) } template -size_t ReplicatedMergeTreeSink::checkQuorumPrecondition(const ZooKeeperWithFaultInjectionPtr & zookeeper) +size_t ReplicatedMergeTreeSinkImpl::checkQuorumPrecondition(const ZooKeeperWithFaultInjectionPtr & zookeeper) { if (!isQuorumEnabled()) return 0; @@ -299,7 +293,7 @@ size_t ReplicatedMergeTreeSink::checkQuorumPrecondition(const ZooK } template -void ReplicatedMergeTreeSink::consume(Chunk chunk) +void ReplicatedMergeTreeSinkImpl::consume(Chunk chunk) { auto block = getHeader().cloneWithColumns(chunk.detachColumns()); @@ -349,7 +343,8 @@ void ReplicatedMergeTreeSink::consume(Chunk chunk) auto part_blocks = storage.writer.splitBlockIntoParts(block, max_parts_per_block, metadata_snapshot, context, chunk_offsets); - using DelayedPartitions = std::vector::DelayedChunk::Partition>; + using DelayedPartition = typename ReplicatedMergeTreeSinkImpl::DelayedChunk::Partition; + using DelayedPartitions = std::vector; DelayedPartitions partitions; size_t streams = 0; @@ -411,7 +406,7 @@ void ReplicatedMergeTreeSink::consume(Chunk chunk) if (streams > max_insert_delayed_streams_for_parallel_write) { finishDelayedChunk(zookeeper); - delayed_chunk = std::make_unique::DelayedChunk>(replicas_num); + delayed_chunk = std::make_unique::DelayedChunk>(replicas_num); delayed_chunk->partitions = std::move(partitions); finishDelayedChunk(zookeeper); @@ -420,7 +415,7 @@ void ReplicatedMergeTreeSink::consume(Chunk chunk) partitions = DelayedPartitions{}; } - partitions.emplace_back(typename ReplicatedMergeTreeSink::DelayedChunk::Partition( + partitions.emplace_back(DelayedPartition( std::move(temp_part), elapsed_ns, std::move(block_id), @@ -429,7 +424,7 @@ void ReplicatedMergeTreeSink::consume(Chunk chunk) } finishDelayedChunk(zookeeper); - delayed_chunk = std::make_unique(); + 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 @@ -441,7 +436,7 @@ void ReplicatedMergeTreeSink::consume(Chunk chunk) } template<> -void ReplicatedMergeTreeSink::finishDelayedChunk(const ZooKeeperWithFaultInjectionPtr & zookeeper) +void ReplicatedMergeTreeSinkImpl::finishDelayedChunk(const ZooKeeperWithFaultInjectionPtr & zookeeper) { if (!delayed_chunk) return; @@ -476,7 +471,7 @@ void ReplicatedMergeTreeSink::finishDelayedChunk(const ZooKeeperWithFault } template<> -void ReplicatedMergeTreeSink::finishDelayedChunk(const ZooKeeperWithFaultInjectionPtr & zookeeper) +void ReplicatedMergeTreeSinkImpl::finishDelayedChunk(const ZooKeeperWithFaultInjectionPtr & zookeeper) { if (!delayed_chunk) return; @@ -504,7 +499,7 @@ void ReplicatedMergeTreeSink::finishDelayedChunk(const ZooKeeperWithFaultI } template -void ReplicatedMergeTreeSink::writeExistingPart(MergeTreeData::MutableDataPartPtr & part) +void ReplicatedMergeTreeSinkImpl::writeExistingPart(MergeTreeData::MutableDataPartPtr & part) { /// NOTE: No delay in this case. That's Ok. @@ -530,7 +525,7 @@ void ReplicatedMergeTreeSink::writeExistingPart(MergeTreeData::Mut } template -std::vector ReplicatedMergeTreeSink::commitPart( +std::vector ReplicatedMergeTreeSinkImpl::commitPart( const ZooKeeperWithFaultInjectionPtr & zookeeper, MergeTreeData::MutableDataPartPtr & part, const BlockIDsType & block_id, @@ -995,7 +990,7 @@ std::vector ReplicatedMergeTreeSink::commitPart( } template -void ReplicatedMergeTreeSink::onStart() +void ReplicatedMergeTreeSinkImpl::onStart() { /// Only check "too many parts" before write, /// because interrupting long-running INSERT query in the middle is not convenient for users. @@ -1003,7 +998,7 @@ void ReplicatedMergeTreeSink::onStart() } template -void ReplicatedMergeTreeSink::onFinish() +void ReplicatedMergeTreeSinkImpl::onFinish() { auto zookeeper = storage.getZooKeeper(); assertSessionIsNotExpired(zookeeper); @@ -1011,7 +1006,7 @@ void ReplicatedMergeTreeSink::onFinish() } template -void ReplicatedMergeTreeSink::waitForQuorum( +void ReplicatedMergeTreeSinkImpl::waitForQuorum( const ZooKeeperWithFaultInjectionPtr & zookeeper, const std::string & part_name, const std::string & quorum_path, @@ -1066,7 +1061,7 @@ void ReplicatedMergeTreeSink::waitForQuorum( } template -String ReplicatedMergeTreeSink::quorumLogMessage(size_t replicas_num) const +String ReplicatedMergeTreeSinkImpl::quorumLogMessage(size_t replicas_num) const { if (!isQuorumEnabled()) return ""; @@ -1074,7 +1069,7 @@ String ReplicatedMergeTreeSink::quorumLogMessage(size_t replicas_n } template -size_t ReplicatedMergeTreeSink::getQuorumSize(size_t replicas_num) const +size_t ReplicatedMergeTreeSinkImpl::getQuorumSize(size_t replicas_num) const { if (!isQuorumEnabled()) return 0; @@ -1086,12 +1081,12 @@ size_t ReplicatedMergeTreeSink::getQuorumSize(size_t replicas_num) } template -bool ReplicatedMergeTreeSink::isQuorumEnabled() const +bool ReplicatedMergeTreeSinkImpl::isQuorumEnabled() const { return !required_quorum_size.has_value() || required_quorum_size.value() > 1; } -template class ReplicatedMergeTreeSink; -template class ReplicatedMergeTreeSink; +template class ReplicatedMergeTreeSinkImpl; +template class ReplicatedMergeTreeSinkImpl; } diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeSink.h b/src/Storages/MergeTree/ReplicatedMergeTreeSink.h index 83b9a92f160..57fd6035471 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeSink.h +++ b/src/Storages/MergeTree/ReplicatedMergeTreeSink.h @@ -29,10 +29,10 @@ using StorageSnapshotPtr = std::shared_ptr; /// identify different async inserts inside the same part. It will remove the duplicate inserts /// when it encounters lock and retries. template -class ReplicatedMergeTreeSink : public SinkToStorage +class ReplicatedMergeTreeSinkImpl : public SinkToStorage { public: - ReplicatedMergeTreeSink( + ReplicatedMergeTreeSinkImpl( StorageReplicatedMergeTree & storage_, const StorageMetadataPtr & metadata_snapshot_, size_t quorum_, @@ -46,7 +46,7 @@ public: // needed to set the special LogEntryType::ATTACH_PART bool is_attach_ = false); - ~ReplicatedMergeTreeSink() override; + ~ReplicatedMergeTreeSinkImpl() override; void onStart() override; void consume(Chunk chunk) override; @@ -134,4 +134,7 @@ private: void finishDelayedChunk(const ZooKeeperWithFaultInjectionPtr & zookeeper); }; +using ReplicatedMergeTreeSinkWithAsyncDeduplicate = ReplicatedMergeTreeSinkImpl; +using ReplicatedMergeTreeSink = ReplicatedMergeTreeSinkImpl; + } diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index b09a9920100..fb9437ea032 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -4525,7 +4525,7 @@ SinkToStoragePtr StorageReplicatedMergeTree::write(const ASTPtr & /*query*/, con bool deduplicate = storage_settings_ptr->replicated_deduplication_window != 0 && query_settings.insert_deduplicate; bool async_deduplicate = query_settings.async_insert && storage_settings_ptr->replicated_deduplication_window_for_async_inserts != 0 && query_settings.insert_deduplicate; if (async_deduplicate) - return std::make_shared>( + return std::make_shared( *this, metadata_snapshot, query_settings.insert_quorum.valueOr(0), query_settings.insert_quorum_timeout.totalMilliseconds(), query_settings.max_partitions_per_insert_block, @@ -4535,7 +4535,7 @@ SinkToStoragePtr StorageReplicatedMergeTree::write(const ASTPtr & /*query*/, con local_context); // TODO: should we also somehow pass list of columns to deduplicate on to the ReplicatedMergeTreeSink? - return std::make_shared>( + return std::make_shared( *this, metadata_snapshot, query_settings.insert_quorum.valueOr(0), query_settings.insert_quorum_timeout.totalMilliseconds(), query_settings.max_partitions_per_insert_block, @@ -5236,7 +5236,7 @@ PartitionCommandsResultInfo StorageReplicatedMergeTree::attachPartition( MutableDataPartsVector loaded_parts = tryLoadPartsToAttach(partition, attach_part, query_context, renamed_parts); /// TODO Allow to use quorum here. - ReplicatedMergeTreeSink output(*this, metadata_snapshot, 0, 0, 0, false, false, false, query_context, + ReplicatedMergeTreeSink output(*this, metadata_snapshot, 0, 0, 0, false, false, false, query_context, /*is_attach*/true); for (size_t i = 0; i < loaded_parts.size(); ++i) @@ -8738,7 +8738,7 @@ void StorageReplicatedMergeTree::restoreDataFromBackup(RestorerFromBackup & rest void StorageReplicatedMergeTree::attachRestoredParts(MutableDataPartsVector && parts) { auto metadata_snapshot = getInMemoryMetadataPtr(); - auto sink = std::make_shared>(*this, metadata_snapshot, 0, 0, 0, false, false, false, getContext(), /*is_attach*/true); + auto sink = std::make_shared(*this, metadata_snapshot, 0, 0, 0, false, false, false, getContext(), /*is_attach*/true); for (auto part : parts) sink->writeExistingPart(part); } @@ -8760,7 +8760,7 @@ PartsTemporaryRename renamed_parts(*this, "detached/"); MutableDataPartsVector loaded_parts = tryLoadPartsToAttach(partition, attach_part, query_context, renamed_parts); /// TODO Allow to use quorum here. -ReplicatedMergeTreeSink output(*this, metadata_snapshot, 0, 0, 0, false, false, query_context, +ReplicatedMergeTreeSink output(*this, metadata_snapshot, 0, 0, 0, false, false, query_context, /*is_attach*/true); for (size_t i = 0; i < loaded_parts.size(); ++i) diff --git a/src/Storages/StorageReplicatedMergeTree.h b/src/Storages/StorageReplicatedMergeTree.h index 1655ad2142b..67e79378b93 100644 --- a/src/Storages/StorageReplicatedMergeTree.h +++ b/src/Storages/StorageReplicatedMergeTree.h @@ -357,7 +357,7 @@ private: void clearOldPartsAndRemoveFromZK(); template - friend class ReplicatedMergeTreeSink; + friend class ReplicatedMergeTreeSinkImpl; friend class ReplicatedMergeTreePartCheckThread; friend class ReplicatedMergeTreeCleanupThread; friend class ReplicatedMergeTreeAlterThread; From 9e2265a6ed4e8361ae66244af803174eb780e975 Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Thu, 8 Dec 2022 00:05:15 +0100 Subject: [PATCH 089/127] Improve hash table preallocation optimisation (#43945) * do not preallocate if max_size_to_preallocate_for_aggregation is too small * skip optimisation for aggr without key * increase default for max_size_to_preallocate_for_aggregation --- src/Core/Settings.h | 2 +- src/Interpreters/Aggregator.cpp | 40 ++++++++++++++++++++++++--------- 2 files changed, 30 insertions(+), 12 deletions(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 33af6710999..2a31f655d41 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -546,7 +546,7 @@ static constexpr UInt64 operator""_GiB(unsigned long long value) \ M(Bool, collect_hash_table_stats_during_aggregation, true, "Enable collecting hash table statistics to optimize memory allocation", 0) \ M(UInt64, max_entries_for_hash_table_stats, 10'000, "How many entries hash table statistics collected during aggregation is allowed to have", 0) \ - M(UInt64, max_size_to_preallocate_for_aggregation, 10'000'000, "For how many elements it is allowed to preallocate space in all hash tables in total before aggregation", 0) \ + M(UInt64, max_size_to_preallocate_for_aggregation, 100'000'000, "For how many elements it is allowed to preallocate space in all hash tables in total before aggregation", 0) \ \ M(Bool, kafka_disable_num_consumers_limit, false, "Disable limit on kafka_num_consumers that depends on the number of available CPU cores", 0) \ M(Bool, enable_software_prefetch_in_aggregation, true, "Enable use of software prefetch in aggregation", 0) \ diff --git a/src/Interpreters/Aggregator.cpp b/src/Interpreters/Aggregator.cpp index 905361d5e00..14113514f1e 100644 --- a/src/Interpreters/Aggregator.cpp +++ b/src/Interpreters/Aggregator.cpp @@ -221,16 +221,31 @@ void initDataVariantsWithSizeHint( const auto max_threads = params.group_by_two_level_threshold != 0 ? std::max(params.max_threads, 1ul) : 1; const auto lower_limit = hint->sum_of_sizes / max_threads; const auto upper_limit = stats_collecting_params.max_size_to_preallocate_for_aggregation / max_threads; - const auto adjusted = std::min(std::max(lower_limit, hint->median_size), upper_limit); - if (worthConvertToTwoLevel( - params.group_by_two_level_threshold, - hint->sum_of_sizes, - /*group_by_two_level_threshold_bytes*/ 0, - /*result_size_bytes*/ 0)) - method_chosen = convertToTwoLevelTypeIfPossible(method_chosen); - result.init(method_chosen, adjusted); - ProfileEvents::increment(ProfileEvents::AggregationHashTablesInitializedAsTwoLevel, result.isTwoLevel()); - return; + if (hint->median_size > upper_limit) + { + /// Since we cannot afford to preallocate as much as we want, we will likely need to do resize anyway. + /// But we will also work with the big (i.e. not so cache friendly) HT from the beginning which may result in a slight slowdown. + /// So let's just do nothing. + LOG_TRACE( + &Poco::Logger::get("Aggregator"), + "No space were preallocated in hash tables because 'max_size_to_preallocate_for_aggregation' has too small value: {}, " + "should be at least {}", + stats_collecting_params.max_size_to_preallocate_for_aggregation, + hint->median_size * max_threads); + } + else + { + const auto adjusted = std::max(lower_limit, hint->median_size); + if (worthConvertToTwoLevel( + params.group_by_two_level_threshold, + hint->sum_of_sizes, + /*group_by_two_level_threshold_bytes*/ 0, + /*result_size_bytes*/ 0)) + method_chosen = convertToTwoLevelTypeIfPossible(method_chosen); + result.init(method_chosen, adjusted); + ProfileEvents::increment(ProfileEvents::AggregationHashTablesInitializedAsTwoLevel, result.isTwoLevel()); + return; + } } } result.init(method_chosen); @@ -930,7 +945,10 @@ void Aggregator::executeOnBlockSmall( /// How to perform the aggregation? if (result.empty()) { - initDataVariantsWithSizeHint(result, method_chosen, params); + if (method_chosen != AggregatedDataVariants::Type::without_key) + initDataVariantsWithSizeHint(result, method_chosen, params); + else + result.init(method_chosen); result.keys_size = params.keys_size; result.key_sizes = key_sizes; } From 77d551788ab505c512fe495d054cbcce0582b093 Mon Sep 17 00:00:00 2001 From: serxa Date: Thu, 8 Dec 2022 00:40:43 +0000 Subject: [PATCH 090/127] Fix relaxed "too many parts" threshold --- src/Storages/MergeTree/MergeTreeData.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index d2b1da9abf8..2ae8a4437dc 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -3675,13 +3675,13 @@ void MergeTreeData::delayInsertOrThrowIfNeeded(Poco::Event * until, ContextPtr q parts_count_in_partition, ReadableSize(average_part_size)); } - if (k_inactive < 0 && parts_count_in_partition < parts_to_delay_insert) + if (k_inactive < 0 && (parts_count_in_partition < parts_to_delay_insert || parts_are_large_enough_in_average)) return; const ssize_t k_active = ssize_t(parts_count_in_partition) - ssize_t(parts_to_delay_insert); size_t max_k; size_t k; - if (k_active > k_inactive && !parts_are_large_enough_in_average) + if (k_active > k_inactive) { max_k = parts_to_throw_insert - parts_to_delay_insert; k = k_active + 1; From 88716b1ec30fc1493da4bf3c33f32f99ec25916d Mon Sep 17 00:00:00 2001 From: Duc Canh Le Date: Thu, 8 Dec 2022 09:39:25 +0800 Subject: [PATCH 091/127] Minor fix --- src/Compression/CompressionCodecGorilla.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Compression/CompressionCodecGorilla.cpp b/src/Compression/CompressionCodecGorilla.cpp index 0da6ff46dbc..88b8c2bc3bb 100644 --- a/src/Compression/CompressionCodecGorilla.cpp +++ b/src/Compression/CompressionCodecGorilla.cpp @@ -344,7 +344,7 @@ UInt8 getDataBytesSize(const IDataType * column_type) if (max_size == 1 || max_size == 2 || max_size == 4 || max_size == 8) return static_cast(max_size); else - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Codec Delta is only applicable for data types of size 1, 2, 4, 8 bytes. Given type {}", + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Codec Gorilla is only applicable for data types of size 1, 2, 4, 8 bytes. Given type {}", column_type->getName()); } From 1bde6d012d3fd1f16b2af82df63f6a1d365007b1 Mon Sep 17 00:00:00 2001 From: Han Fei Date: Thu, 8 Dec 2022 09:57:33 +0100 Subject: [PATCH 092/127] fix --- src/Storages/MergeTree/EphemeralLockInZooKeeper.cpp | 6 +++++- src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp | 2 +- 2 files changed, 6 insertions(+), 2 deletions(-) diff --git a/src/Storages/MergeTree/EphemeralLockInZooKeeper.cpp b/src/Storages/MergeTree/EphemeralLockInZooKeeper.cpp index e62194dc5e5..bc28a555f77 100644 --- a/src/Storages/MergeTree/EphemeralLockInZooKeeper.cpp +++ b/src/Storages/MergeTree/EphemeralLockInZooKeeper.cpp @@ -55,7 +55,7 @@ std::optional createEphemeralLockInZooKeeper( ops.emplace_back(zkutil::makeCreateRequest(path_prefix_, holder_path, zkutil::CreateMode::EphemeralSequential)); Coordination::Responses responses; Coordination::Error e = zookeeper_->tryMulti(ops, responses); - if (e != Coordination::Error::ZOK) + if (e == Coordination::Error::ZNODEEXISTS) { if constexpr (async_insert) { @@ -78,6 +78,10 @@ std::optional createEphemeralLockInZooKeeper( deduplication_path); return {}; } + } + + if (e != Coordination::Error::ZOK) + { zkutil::KeeperMultiException::check(e, ops, responses); // This should always throw the proper exception throw Exception(ErrorCodes::LOGICAL_ERROR, "Unable to handle error {} when acquiring ephemeral lock in ZK", toString(e)); } diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp index dbb035693ef..2d8bf28e700 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp @@ -158,7 +158,7 @@ namespace { size_t start = 0; auto cols = block.block.getColumns(); - std::vector block_id_vec(block.offsets->offsets.size()); + std::vector block_id_vec; for (auto offset : block.offsets->offsets) { SipHash hash; From 16b33c204c5b660c70cd675e9c655dff04d1734a Mon Sep 17 00:00:00 2001 From: Nikolay Degterinsky <43110995+evillique@users.noreply.github.com> Date: Thu, 8 Dec 2022 10:49:25 +0100 Subject: [PATCH 093/127] Fix exception message --- src/Compression/CompressionCodecDoubleDelta.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Compression/CompressionCodecDoubleDelta.cpp b/src/Compression/CompressionCodecDoubleDelta.cpp index c1278cb88de..dd2507ab14a 100644 --- a/src/Compression/CompressionCodecDoubleDelta.cpp +++ b/src/Compression/CompressionCodecDoubleDelta.cpp @@ -445,7 +445,7 @@ UInt8 getDataBytesSize(const IDataType * column_type) if (max_size == 1 || max_size == 2 || max_size == 4 || max_size == 8) return static_cast(max_size); else - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Codec Delta is only applicable for data types of size 1, 2, 4, 8 bytes. Given type {}", + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Codec DoubleDelta is only applicable for data types of size 1, 2, 4, 8 bytes. Given type {}", column_type->getName()); } From 31d2fe63df1b8830d7a044e48847036c98655309 Mon Sep 17 00:00:00 2001 From: vdimir Date: Thu, 8 Dec 2022 10:15:31 +0000 Subject: [PATCH 094/127] Resolve code review comments and apply suggestions --- src/Storages/StorageJoin.cpp | 15 +++++++++------ 1 file changed, 9 insertions(+), 6 deletions(-) diff --git a/src/Storages/StorageJoin.cpp b/src/Storages/StorageJoin.cpp index 2e799b0b0db..aec28b37928 100644 --- a/src/Storages/StorageJoin.cpp +++ b/src/Storages/StorageJoin.cpp @@ -165,7 +165,7 @@ HashJoinPtr StorageJoin::getJoinLocked(std::shared_ptr analyzed_join, { auto metadata_snapshot = getInMemoryMetadataPtr(); if (!analyzed_join->sameStrictnessAndKind(strictness, kind)) - throw Exception(ErrorCodes::INCOMPATIBLE_TYPE_OF_JOIN, "Table '{} has incompatible type of JOIN", getStorageID().getNameForLogs()); + throw Exception(ErrorCodes::INCOMPATIBLE_TYPE_OF_JOIN, "Table '{}' has incompatible type of JOIN", getStorageID().getNameForLogs()); if ((analyzed_join->forceNullableRight() && !use_nulls) || (!analyzed_join->forceNullableRight() && isLeftOrFull(analyzed_join->kind()) && use_nulls)) @@ -186,7 +186,10 @@ HashJoinPtr StorageJoin::getJoinLocked(std::shared_ptr analyzed_join, key_names_right.size(), key_names.size()); /* Resort left keys according to right keys order in StorageJoin - * We can't change the order of keys in StorageJoin because the hash table already build. + * We can't change the order of keys in StorageJoin + * because the hash table was already built with tuples serialized in the order of key_names. + * If we try to use the same hash table with different order of keys, + * then calculated hashes and the result of the comparison will be wrong. * * Example: * ``` @@ -196,15 +199,15 @@ HashJoinPtr StorageJoin::getJoinLocked(std::shared_ptr analyzed_join, * In that case right keys should still be (a, b), need to change the order of the left keys to (x, y). */ Names left_key_names_resorted; - for (size_t i = 0; i < key_names.size(); ++i) + for (const auto & key_name : key_names) { - const auto & renamed_key = analyzed_join->renamedRightColumnName(key_names[i]); + const auto & renamed_key = analyzed_join->renamedRightColumnName(key_name); /// find position of renamed_key in key_names_right auto it = std::find(key_names_right.begin(), key_names_right.end(), renamed_key); if (it == key_names_right.end()) throw Exception(ErrorCodes::INCOMPATIBLE_TYPE_OF_JOIN, - "Key '{}' not found in JOIN ON section. All Join engine keys '{}' have to be used", key_names[i], fmt::join(key_names, ", ")); - size_t key_position = it - key_names_right.begin(); + "Key '{}' not found in JOIN ON section. All Join engine keys '{}' have to be used", key_name, fmt::join(key_names, ", ")); + const size_t key_position = std::distance(key_names_right.begin(), it); left_key_names_resorted.push_back(key_names_left[key_position]); } From c60545a94209642bb174c44890638b633a96545f Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Thu, 8 Dec 2022 11:30:31 +0000 Subject: [PATCH 095/127] Update version_date.tsv and changelogs after v22.8.11.15-lts --- docker/server/Dockerfile.alpine | 2 +- docker/server/Dockerfile.ubuntu | 2 +- docs/changelogs/v22.8.11.15-lts.md | 23 +++++++++++++++++++++++ utils/list-versions/version_date.tsv | 1 + 4 files changed, 26 insertions(+), 2 deletions(-) create mode 100644 docs/changelogs/v22.8.11.15-lts.md diff --git a/docker/server/Dockerfile.alpine b/docker/server/Dockerfile.alpine index b717cec2d33..305fc279414 100644 --- a/docker/server/Dockerfile.alpine +++ b/docker/server/Dockerfile.alpine @@ -33,7 +33,7 @@ RUN arch=${TARGETARCH:-amd64} \ # lts / testing / prestable / etc ARG REPO_CHANNEL="stable" ARG REPOSITORY="https://packages.clickhouse.com/tgz/${REPO_CHANNEL}" -ARG VERSION="22.11.1.1360" +ARG VERSION="22.11.2.30" ARG PACKAGES="clickhouse-client clickhouse-server clickhouse-common-static" # user/group precreated explicitly with fixed uid/gid on purpose. diff --git a/docker/server/Dockerfile.ubuntu b/docker/server/Dockerfile.ubuntu index 8a5dc04681e..f1c4dd097aa 100644 --- a/docker/server/Dockerfile.ubuntu +++ b/docker/server/Dockerfile.ubuntu @@ -21,7 +21,7 @@ RUN sed -i "s|http://archive.ubuntu.com|${apt_archive}|g" /etc/apt/sources.list ARG REPO_CHANNEL="stable" ARG REPOSITORY="deb https://packages.clickhouse.com/deb ${REPO_CHANNEL} main" -ARG VERSION="22.11.1.1360" +ARG VERSION="22.11.2.30" ARG PACKAGES="clickhouse-client clickhouse-server clickhouse-common-static" # set non-empty deb_location_url url to create a docker image diff --git a/docs/changelogs/v22.8.11.15-lts.md b/docs/changelogs/v22.8.11.15-lts.md new file mode 100644 index 00000000000..b0c4a7cc168 --- /dev/null +++ b/docs/changelogs/v22.8.11.15-lts.md @@ -0,0 +1,23 @@ +--- +sidebar_position: 1 +sidebar_label: 2022 +--- + +# 2022 Changelog + +### ClickHouse release v22.8.11.15-lts (65c9506d161) FIXME as compared to v22.8.10.29-lts (d568a57f7af) + +#### Bug Fix +* Backported in [#43098](https://github.com/ClickHouse/ClickHouse/issues/43098): Updated normaliser to clone the alias ast. resolves [#42452](https://github.com/ClickHouse/ClickHouse/issues/42452) Implementation: * Updated QueryNormalizer to clone alias ast, when its replaced. Previously just assigning the same leads to exception in LogicalExpressinsOptimizer as it would be the same parent being inserted again. * This bug is not seen with new analyser (allow_experimental_analyzer), so no changes for it. I added a test for the same. [#42827](https://github.com/ClickHouse/ClickHouse/pull/42827) ([SmitaRKulkarni](https://github.com/SmitaRKulkarni)). + +#### Bug Fix (user-visible misbehavior in official stable or prestable release) + +* Backported in [#43751](https://github.com/ClickHouse/ClickHouse/issues/43751): An issue with the following exception has been reported while trying to read a Parquet file from S3 into ClickHouse:. [#43297](https://github.com/ClickHouse/ClickHouse/pull/43297) ([Arthur Passos](https://github.com/arthurpassos)). +* Backported in [#43617](https://github.com/ClickHouse/ClickHouse/issues/43617): Fix sumMap() for Nullable(Decimal()). [#43414](https://github.com/ClickHouse/ClickHouse/pull/43414) ([Azat Khuzhin](https://github.com/azat)). +* Backported in [#43886](https://github.com/ClickHouse/ClickHouse/issues/43886): Fixed `ALTER ... RESET SETTING` with `ON CLUSTER`. It could be applied to one replica only. Fixes [#43843](https://github.com/ClickHouse/ClickHouse/issues/43843). [#43848](https://github.com/ClickHouse/ClickHouse/pull/43848) ([Elena Torró](https://github.com/elenatorro)). + +#### NOT FOR CHANGELOG / INSIGNIFICANT + +* Use only PRs to our repository in pr_info on push [#43895](https://github.com/ClickHouse/ClickHouse/pull/43895) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Fix tags workflow [#43942](https://github.com/ClickHouse/ClickHouse/pull/43942) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). + diff --git a/utils/list-versions/version_date.tsv b/utils/list-versions/version_date.tsv index 0793f6c816f..23aa0d1dbaf 100644 --- a/utils/list-versions/version_date.tsv +++ b/utils/list-versions/version_date.tsv @@ -10,6 +10,7 @@ v22.9.4.32-stable 2022-10-26 v22.9.3.18-stable 2022-09-30 v22.9.2.7-stable 2022-09-23 v22.9.1.2603-stable 2022-09-22 +v22.8.11.15-lts 2022-12-08 v22.8.10.29-lts 2022-12-02 v22.8.9.24-lts 2022-11-19 v22.8.8.3-lts 2022-10-27 From b81ad6aaf711f3c3fc220e75d1500f1581da7a53 Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Thu, 8 Dec 2022 13:38:08 +0100 Subject: [PATCH 096/127] Add google benchmark to contrib (#43779) * add google benchmark to contrib * rework integer_hash_tables_and_hashes * update readme * keep benchmarks near the benchmarked code * fix fasttests build * rm old target * fix --- .gitmodules | 3 + CMakeLists.txt | 2 + contrib/CMakeLists.txt | 2 + contrib/google-benchmark | 1 + contrib/google-benchmark-cmake/CMakeLists.txt | 34 ++++ src/Common/CMakeLists.txt | 4 + src/Common/benchmarks/CMakeLists.txt | 9 + .../integer_hash_tables_and_hashes.cpp | 159 +++++++++--------- src/Common/examples/CMakeLists.txt | 3 - 9 files changed, 138 insertions(+), 79 deletions(-) create mode 160000 contrib/google-benchmark create mode 100644 contrib/google-benchmark-cmake/CMakeLists.txt create mode 100644 src/Common/benchmarks/CMakeLists.txt rename src/Common/{examples => benchmarks}/integer_hash_tables_and_hashes.cpp (94%) diff --git a/.gitmodules b/.gitmodules index ebeef312ae8..a4cfcc91485 100644 --- a/.gitmodules +++ b/.gitmodules @@ -287,3 +287,6 @@ [submodule "contrib/xxHash"] path = contrib/xxHash url = https://github.com/Cyan4973/xxHash.git +[submodule "contrib/google-benchmark"] + path = contrib/google-benchmark + url = https://github.com/google/benchmark.git diff --git a/CMakeLists.txt b/CMakeLists.txt index 06e6f943fd3..e121559d4e0 100644 --- a/CMakeLists.txt +++ b/CMakeLists.txt @@ -111,6 +111,7 @@ if (ENABLE_FUZZING) set (ENABLE_JEMALLOC 0) set (ENABLE_CHECK_HEAVY_BUILDS 1) set (GLIBC_COMPATIBILITY OFF) + set (ENABLE_BENCHMARKS 0) # For codegen_select_fuzzer set (ENABLE_PROTOBUF 1) @@ -168,6 +169,7 @@ endif () option(ENABLE_TESTS "Provide unit_test_dbms target with Google.Test unit tests" ON) option(ENABLE_EXAMPLES "Build all example programs in 'examples' subdirectories" OFF) +option(ENABLE_BENCHMARKS "Build all benchmark programs in 'benchmarks' subdirectories" OFF) if (OS_LINUX AND (ARCH_AMD64 OR ARCH_AARCH64) AND USE_STATIC_LIBRARIES AND NOT SPLIT_SHARED_LIBRARIES AND NOT USE_MUSL) # Only for Linux, x86_64 or aarch64. diff --git a/contrib/CMakeLists.txt b/contrib/CMakeLists.txt index ec7382846c2..c7419d74aac 100644 --- a/contrib/CMakeLists.txt +++ b/contrib/CMakeLists.txt @@ -171,6 +171,8 @@ add_contrib (annoy-cmake annoy) add_contrib (xxHash-cmake xxHash) +add_contrib (google-benchmark-cmake google-benchmark) + # Put all targets defined here and in subdirectories under "contrib/" folders in GUI-based IDEs. # Some of third-party projects may override CMAKE_FOLDER or FOLDER property of their targets, so they would not appear # in "contrib/..." as originally planned, so we workaround this by fixing FOLDER properties of all targets manually, diff --git a/contrib/google-benchmark b/contrib/google-benchmark new file mode 160000 index 00000000000..2257fa4d6af --- /dev/null +++ b/contrib/google-benchmark @@ -0,0 +1 @@ +Subproject commit 2257fa4d6afb8e5a2ccd510a70f38fe7fcdf1edf diff --git a/contrib/google-benchmark-cmake/CMakeLists.txt b/contrib/google-benchmark-cmake/CMakeLists.txt new file mode 100644 index 00000000000..5d8fa7b838b --- /dev/null +++ b/contrib/google-benchmark-cmake/CMakeLists.txt @@ -0,0 +1,34 @@ +set (SRC_DIR "${ClickHouse_SOURCE_DIR}/contrib/google-benchmark/src") + +set (SRCS + "${SRC_DIR}/benchmark.cc" + "${SRC_DIR}/benchmark_api_internal.cc" + "${SRC_DIR}/benchmark_name.cc" + "${SRC_DIR}/benchmark_register.cc" + "${SRC_DIR}/benchmark_runner.cc" + "${SRC_DIR}/check.cc" + "${SRC_DIR}/colorprint.cc" + "${SRC_DIR}/commandlineflags.cc" + "${SRC_DIR}/complexity.cc" + "${SRC_DIR}/console_reporter.cc" + "${SRC_DIR}/counter.cc" + "${SRC_DIR}/csv_reporter.cc" + "${SRC_DIR}/json_reporter.cc" + "${SRC_DIR}/perf_counters.cc" + "${SRC_DIR}/reporter.cc" + "${SRC_DIR}/sleep.cc" + "${SRC_DIR}/statistics.cc" + "${SRC_DIR}/string_util.cc" + "${SRC_DIR}/sysinfo.cc" + "${SRC_DIR}/timers.cc") + +add_library(google_benchmark "${SRCS}") +target_include_directories(google_benchmark SYSTEM PUBLIC "${SRC_DIR}/../include") + +add_library(google_benchmark_main "${SRC_DIR}/benchmark_main.cc") +target_link_libraries(google_benchmark_main PUBLIC google_benchmark) + +add_library(google_benchmark_all INTERFACE) +target_link_libraries(google_benchmark_all INTERFACE google_benchmark google_benchmark_main) + +add_library(ch_contrib::gbenchmark_all ALIAS google_benchmark_all) diff --git a/src/Common/CMakeLists.txt b/src/Common/CMakeLists.txt index 490628a2180..e527b3dec43 100644 --- a/src/Common/CMakeLists.txt +++ b/src/Common/CMakeLists.txt @@ -1,5 +1,9 @@ add_subdirectory(StringUtils) +if (ENABLE_BENCHMARKS) + add_subdirectory(benchmarks) +endif() + if (ENABLE_EXAMPLES) add_subdirectory(examples) endif() diff --git a/src/Common/benchmarks/CMakeLists.txt b/src/Common/benchmarks/CMakeLists.txt new file mode 100644 index 00000000000..57ed837db8b --- /dev/null +++ b/src/Common/benchmarks/CMakeLists.txt @@ -0,0 +1,9 @@ +clickhouse_add_executable(integer_hash_tables_and_hashes integer_hash_tables_and_hashes.cpp) +target_link_libraries (integer_hash_tables_and_hashes PRIVATE + ch_contrib::gbenchmark_all + dbms + ch_contrib::abseil_swiss_tables + ch_contrib::sparsehash + ch_contrib::wyhash + ch_contrib::farmhash + ch_contrib::xxHash) diff --git a/src/Common/examples/integer_hash_tables_and_hashes.cpp b/src/Common/benchmarks/integer_hash_tables_and_hashes.cpp similarity index 94% rename from src/Common/examples/integer_hash_tables_and_hashes.cpp rename to src/Common/benchmarks/integer_hash_tables_and_hashes.cpp index 0e9390ab3ac..c245fc471cc 100644 --- a/src/Common/examples/integer_hash_tables_and_hashes.cpp +++ b/src/Common/benchmarks/integer_hash_tables_and_hashes.cpp @@ -1,5 +1,8 @@ -#include +#include + #include +#include +#include #include #include @@ -13,12 +16,23 @@ //#define DBMS_HASH_MAP_COUNT_COLLISIONS //#define DBMS_HASH_MAP_DEBUG_RESIZES -#include -#include +#include +#include #include +#include +#include #include #include +#include +#include + +#ifdef __clang__ +# pragma clang diagnostic push +# pragma clang diagnostic ignored "-Wused-but-marked-unused" +#endif +#include + using Key = UInt64; using Value = UInt64; @@ -282,98 +296,91 @@ namespace Hashes return res; } }; + + struct FarmHash + { + size_t operator()(Key x) const { return NAMESPACE_FOR_HASH_FUNCTIONS::Hash64(reinterpret_cast(&x), sizeof(x)); } + }; + + struct WyHash + { + size_t operator()(Key x) const { return wyhash(reinterpret_cast(&x), sizeof(x), 0, _wyp); } + }; + + struct XXH3Hash + { + size_t operator()(Key x) const { return XXH_INLINE_XXH3_64bits(reinterpret_cast(&x), sizeof(x)); } + }; } template