From 6d9b310f61ad4102421cc7a3d7a2bc725f66ee34 Mon Sep 17 00:00:00 2001 From: Pavel Kruglov Date: Tue, 13 Oct 2020 17:51:08 +0300 Subject: [PATCH 01/50] Add optimize_skip_merged_partitions --- .../MergeTree/MergeTreeDataMergerMutator.cpp | 7 +++++++ src/Storages/MergeTree/MergeTreeDataMergerMutator.h | 1 + src/Storages/MergeTree/MergeTreeSettings.h | 1 + src/Storages/StorageMergeTree.cpp | 7 ++++++- src/Storages/StorageReplicatedMergeTree.cpp | 13 ++++++++++--- 5 files changed, 25 insertions(+), 4 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp index fb0a488700c..3c1a40342de 100644 --- a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp +++ b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp @@ -358,6 +358,7 @@ bool MergeTreeDataMergerMutator::selectAllPartsToMergeWithinPartition( const AllowedMergingPredicate & can_merge, const String & partition_id, bool final, + bool * is_single_merged_part, String * out_disable_reason) { MergeTreeData::DataPartsVector parts = selectAllPartsFromPartition(partition_id); @@ -372,6 +373,12 @@ bool MergeTreeDataMergerMutator::selectAllPartsToMergeWithinPartition( return false; } + if (final && data.getSettings()->optimize_skip_merged_partitions && parts.size() == 1 && parts[0]->info.level > 0) + { + *is_single_merged_part = true; + return false; + } + auto it = parts.begin(); auto prev_it = it; diff --git a/src/Storages/MergeTree/MergeTreeDataMergerMutator.h b/src/Storages/MergeTree/MergeTreeDataMergerMutator.h index 0ad525d1901..4118c09fa89 100644 --- a/src/Storages/MergeTree/MergeTreeDataMergerMutator.h +++ b/src/Storages/MergeTree/MergeTreeDataMergerMutator.h @@ -96,6 +96,7 @@ public: const AllowedMergingPredicate & can_merge, const String & partition_id, bool final, + bool * is_single_merged_part, String * out_disable_reason = nullptr); /** Merge the parts. diff --git a/src/Storages/MergeTree/MergeTreeSettings.h b/src/Storages/MergeTree/MergeTreeSettings.h index 97bc73caf5b..c074c6b35b4 100644 --- a/src/Storages/MergeTree/MergeTreeSettings.h +++ b/src/Storages/MergeTree/MergeTreeSettings.h @@ -48,6 +48,7 @@ struct Settings; M(UInt64, write_ahead_log_bytes_to_fsync, 100ULL * 1024 * 1024, "Amount of bytes, accumulated in WAL to do fsync.", 0) \ M(UInt64, write_ahead_log_interval_ms_to_fsync, 100, "Interval in milliseconds after which fsync for WAL is being done.", 0) \ M(Bool, in_memory_parts_insert_sync, false, "If true insert of part with in-memory format will wait for fsync of WAL", 0) \ + M(Bool, optimize_skip_merged_partitions, true, "Skip partitions with one part with level > 0 in optimize final", 0) \ \ /** Inserts settings. */ \ M(UInt64, parts_to_delay_insert, 150, "If table contains at least that many active parts in single partition, artificially slow down insert into table.", 0) \ diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index 0f1afe1bd62..6826ab8f220 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -654,6 +654,7 @@ bool StorageMergeTree::merge( }; bool selected = false; + bool is_single_merged_part = false; if (partition_id.empty()) { @@ -682,7 +683,7 @@ bool StorageMergeTree::merge( { UInt64 disk_space = getStoragePolicy()->getMaxUnreservedFreeSpace(); selected = merger_mutator.selectAllPartsToMergeWithinPartition( - future_part, disk_space, can_merge, partition_id, final, out_disable_reason); + future_part, disk_space, can_merge, partition_id, final, &is_single_merged_part, out_disable_reason); /// If final - we will wait for currently processing merges to finish and continue. /// TODO Respect query settings for timeout @@ -709,6 +710,10 @@ bool StorageMergeTree::merge( if (!selected) { + if (final && is_single_merged_part) + { + return true; + } if (out_disable_reason) { if (!out_disable_reason->empty()) diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 2ac8ddb7846..f8122f59046 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -3761,11 +3761,16 @@ bool StorageReplicatedMergeTree::optimize( ReplicatedMergeTreeMergePredicate can_merge = queue.getMergePredicate(zookeeper); FutureMergedMutatedPart future_merged_part; + bool is_single_merged_part = false; bool selected = merger_mutator.selectAllPartsToMergeWithinPartition( - future_merged_part, disk_space, can_merge, partition_id, true, nullptr); + future_merged_part, disk_space, can_merge, partition_id, true, &is_single_merged_part, nullptr); if (!selected) + { + if (is_single_merged_part) + return true; break; + } ReplicatedMergeTreeLogEntryData merge_entry; CreateMergeEntryResult create_result = createLogEntryToMergeParts( @@ -3798,6 +3803,7 @@ bool StorageReplicatedMergeTree::optimize( FutureMergedMutatedPart future_merged_part; String disable_reason; bool selected = false; + bool is_single_merged_part = false; if (!partition) { selected = merger_mutator.selectPartsToMerge( @@ -3805,15 +3811,16 @@ bool StorageReplicatedMergeTree::optimize( } else { - UInt64 disk_space = getStoragePolicy()->getMaxUnreservedFreeSpace(); String partition_id = getPartitionIDFromQuery(partition, query_context); selected = merger_mutator.selectAllPartsToMergeWithinPartition( - future_merged_part, disk_space, can_merge, partition_id, final, &disable_reason); + future_merged_part, disk_space, can_merge, partition_id, final, &is_single_merged_part, &disable_reason); } if (!selected) { + if (final && is_single_merged_part) + return true; std::stringstream message; message << "Cannot select parts for optimization"; if (!disable_reason.empty()) From 1ccf10505acfc9d564af9e83670156855bab7c4c Mon Sep 17 00:00:00 2001 From: Pavel Kruglov Date: Tue, 13 Oct 2020 21:25:45 +0300 Subject: [PATCH 02/50] add tests and comments --- .../MergeTree/MergeTreeDataMergerMutator.cpp | 6 +++++- .../MergeTree/MergeTreeDataMergerMutator.h | 4 +++- src/Storages/StorageMergeTree.cpp | 4 ++++ src/Storages/StorageReplicatedMergeTree.cpp | 10 +++++++++- ...3_optimize_skip_merged_partitions.reference | 2 ++ .../01533_optimize_skip_merged_partitions.sql | 18 ++++++++++++++++++ 6 files changed, 41 insertions(+), 3 deletions(-) create mode 100644 tests/queries/0_stateless/01533_optimize_skip_merged_partitions.reference create mode 100644 tests/queries/0_stateless/01533_optimize_skip_merged_partitions.sql diff --git a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp index 3c1a40342de..ac08cd31dd2 100644 --- a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp +++ b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp @@ -358,7 +358,7 @@ bool MergeTreeDataMergerMutator::selectAllPartsToMergeWithinPartition( const AllowedMergingPredicate & can_merge, const String & partition_id, bool final, - bool * is_single_merged_part, + bool * is_single_merged_part, String * out_disable_reason) { MergeTreeData::DataPartsVector parts = selectAllPartsFromPartition(partition_id); @@ -373,6 +373,8 @@ bool MergeTreeDataMergerMutator::selectAllPartsToMergeWithinPartition( return false; } + /// If final, optimize_skip_merged_partitions is true and we have only one part in partition with level > 0 + /// than we don't select it to merge if (final && data.getSettings()->optimize_skip_merged_partitions && parts.size() == 1 && parts[0]->info.level > 0) { *is_single_merged_part = true; @@ -637,6 +639,8 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mergePartsToTempor { static const String TMP_PREFIX = "tmp_merge_"; + + if (merges_blocker.isCancelled()) throw Exception("Cancelled merging parts", ErrorCodes::ABORTED); diff --git a/src/Storages/MergeTree/MergeTreeDataMergerMutator.h b/src/Storages/MergeTree/MergeTreeDataMergerMutator.h index 4118c09fa89..481fa24637b 100644 --- a/src/Storages/MergeTree/MergeTreeDataMergerMutator.h +++ b/src/Storages/MergeTree/MergeTreeDataMergerMutator.h @@ -88,7 +88,9 @@ public: String * out_disable_reason = nullptr); /** Select all the parts in the specified partition for merge, if possible. - * final - choose to merge even a single part - that is, allow to merge one part "with itself". + * final - choose to merge even a single part - that is, allow to merge one part "with itself", + * but if setting optimize_skip_merged_partitions is true (it's true as default) than single part with level > 0 + * won't be merged with itself. */ bool selectAllPartsToMergeWithinPartition( FutureMergedMutatedPart & future_part, diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index 6826ab8f220..574c65502f1 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -654,6 +654,9 @@ bool StorageMergeTree::merge( }; bool selected = false; + + /// This flag is true when there is only one part in partition, it's level > 0 + /// and setting optimize_skip_merged_partitions is true bool is_single_merged_part = false; if (partition_id.empty()) @@ -710,6 +713,7 @@ bool StorageMergeTree::merge( if (!selected) { + /// If is_single_merged_part is true we treat this part as already merged if (final && is_single_merged_part) { return true; diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index f8122f59046..19ce88ad0b8 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -3761,12 +3761,16 @@ bool StorageReplicatedMergeTree::optimize( ReplicatedMergeTreeMergePredicate can_merge = queue.getMergePredicate(zookeeper); FutureMergedMutatedPart future_merged_part; + + /// This flag is true when there is only one part in partition, it's level > 0 + /// and setting optimize_skip_merged_partitions is true bool is_single_merged_part = false; bool selected = merger_mutator.selectAllPartsToMergeWithinPartition( future_merged_part, disk_space, can_merge, partition_id, true, &is_single_merged_part, nullptr); if (!selected) { + /// If is_single_merged_part is true we treat this part as already merged if (is_single_merged_part) return true; break; @@ -3789,7 +3793,7 @@ bool StorageReplicatedMergeTree::optimize( } if (try_no == max_retries) return handle_noop("Can't create merge queue node in ZooKeeper, because log was updated in every of " - + toString(max_retries) + " tries"); + + toString(max_retries) + " tries"); } } else @@ -3803,6 +3807,9 @@ bool StorageReplicatedMergeTree::optimize( FutureMergedMutatedPart future_merged_part; String disable_reason; bool selected = false; + + /// This flag is true when there is only one part in partition, it's level > 0 + /// and setting optimize_skip_merged_partitions is true bool is_single_merged_part = false; if (!partition) { @@ -3819,6 +3826,7 @@ bool StorageReplicatedMergeTree::optimize( if (!selected) { + /// If is_single_merged_part is true we treat this part as already merged if (final && is_single_merged_part) return true; std::stringstream message; diff --git a/tests/queries/0_stateless/01533_optimize_skip_merged_partitions.reference b/tests/queries/0_stateless/01533_optimize_skip_merged_partitions.reference new file mode 100644 index 00000000000..300e1103c0f --- /dev/null +++ b/tests/queries/0_stateless/01533_optimize_skip_merged_partitions.reference @@ -0,0 +1,2 @@ +optimize_final 200001 1 1 +optimize_final 202001 1 1 diff --git a/tests/queries/0_stateless/01533_optimize_skip_merged_partitions.sql b/tests/queries/0_stateless/01533_optimize_skip_merged_partitions.sql new file mode 100644 index 00000000000..325aaa2d6bb --- /dev/null +++ b/tests/queries/0_stateless/01533_optimize_skip_merged_partitions.sql @@ -0,0 +1,18 @@ +DROP TABLE IF EXISTS optimize_final; + +CREATE TABLE optimize_final(t DateTime, x Int32) ENGINE = MergeTree() PARTITION BY toYYYYMM(t) ORDER BY x; + +INSERT INTO optimize_final SELECT toDate('2000-01-01'), number FROM numbers(5); +INSERT INTO optimize_final SELECT toDate('2000-01-01'), number + 5 FROM numbers(5); + +OPTIMIZE TABLE optimize_final FINAL; + +INSERT INTO optimize_final SELECT toDate('2020-01-01'), number FROM numbers(5); +INSERT INTO optimize_final SELECT toDate('2020-01-01'), number + 5 FROM numbers(5); + +OPTIMIZE TABLE optimize_final FINAL; + +SELECT table, partition, active, level from system.parts where table = 'optimize_final' and active = 1; + +DROP TABLE optimize_final; + From 51807599e7e438d5c033db87f9ded0921dc7903b Mon Sep 17 00:00:00 2001 From: Pavel Kruglov Date: Tue, 13 Oct 2020 23:10:37 +0300 Subject: [PATCH 03/50] fix style --- src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp | 2 -- 1 file changed, 2 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp index ac08cd31dd2..4b28dbf9949 100644 --- a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp +++ b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp @@ -639,8 +639,6 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mergePartsToTempor { static const String TMP_PREFIX = "tmp_merge_"; - - if (merges_blocker.isCancelled()) throw Exception("Cancelled merging parts", ErrorCodes::ABORTED); From 25c39689c37b3ebc71e2bb95576986c101da38c8 Mon Sep 17 00:00:00 2001 From: Pavel Kruglov Date: Thu, 15 Oct 2020 15:15:02 +0300 Subject: [PATCH 04/50] Check expired ttl --- src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp | 8 ++++++-- src/Storages/MergeTree/MergeTreeDataMergerMutator.h | 1 + src/Storages/StorageMergeTree.cpp | 2 +- src/Storages/StorageReplicatedMergeTree.cpp | 5 +++-- 4 files changed, 11 insertions(+), 5 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp index 4b28dbf9949..a5d0db47929 100644 --- a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp +++ b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp @@ -359,6 +359,7 @@ bool MergeTreeDataMergerMutator::selectAllPartsToMergeWithinPartition( const String & partition_id, bool final, bool * is_single_merged_part, + const StorageMetadataPtr & metadata_snapshot, String * out_disable_reason) { MergeTreeData::DataPartsVector parts = selectAllPartsFromPartition(partition_id); @@ -374,8 +375,9 @@ bool MergeTreeDataMergerMutator::selectAllPartsToMergeWithinPartition( } /// If final, optimize_skip_merged_partitions is true and we have only one part in partition with level > 0 - /// than we don't select it to merge - if (final && data.getSettings()->optimize_skip_merged_partitions && parts.size() == 1 && parts[0]->info.level > 0) + /// than we don't select it to merge. But if there are some expired TTL then merge is needed + if (final && data.getSettings()->optimize_skip_merged_partitions && parts.size() == 1 && parts[0]->info.level > 0 && + (!metadata_snapshot->hasAnyTTL() || parts[0]->checkAllTTLCalculated(metadata_snapshot))) { *is_single_merged_part = true; return false; @@ -639,6 +641,8 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mergePartsToTempor { static const String TMP_PREFIX = "tmp_merge_"; + + if (merges_blocker.isCancelled()) throw Exception("Cancelled merging parts", ErrorCodes::ABORTED); diff --git a/src/Storages/MergeTree/MergeTreeDataMergerMutator.h b/src/Storages/MergeTree/MergeTreeDataMergerMutator.h index 481fa24637b..a3f240f86de 100644 --- a/src/Storages/MergeTree/MergeTreeDataMergerMutator.h +++ b/src/Storages/MergeTree/MergeTreeDataMergerMutator.h @@ -99,6 +99,7 @@ public: const String & partition_id, bool final, bool * is_single_merged_part, + const StorageMetadataPtr & metadata_snapshot, String * out_disable_reason = nullptr); /** Merge the parts. diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index 574c65502f1..69558b13d9b 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -686,7 +686,7 @@ bool StorageMergeTree::merge( { UInt64 disk_space = getStoragePolicy()->getMaxUnreservedFreeSpace(); selected = merger_mutator.selectAllPartsToMergeWithinPartition( - future_part, disk_space, can_merge, partition_id, final, &is_single_merged_part, out_disable_reason); + future_part, disk_space, can_merge, partition_id, final, &is_single_merged_part, metadata_snapshot, out_disable_reason); /// If final - we will wait for currently processing merges to finish and continue. /// TODO Respect query settings for timeout diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 19ce88ad0b8..09d36ddd4e4 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -3739,6 +3739,7 @@ bool StorageReplicatedMergeTree::optimize( }; const auto storage_settings_ptr = getSettings(); + auto metadata_snapshot = getInMemoryMetadataPtr(); if (!partition && final) { @@ -3766,7 +3767,7 @@ bool StorageReplicatedMergeTree::optimize( /// and setting optimize_skip_merged_partitions is true bool is_single_merged_part = false; bool selected = merger_mutator.selectAllPartsToMergeWithinPartition( - future_merged_part, disk_space, can_merge, partition_id, true, &is_single_merged_part, nullptr); + future_merged_part, disk_space, can_merge, partition_id, true, &is_single_merged_part, metadata_snapshot, nullptr); if (!selected) { @@ -3821,7 +3822,7 @@ bool StorageReplicatedMergeTree::optimize( UInt64 disk_space = getStoragePolicy()->getMaxUnreservedFreeSpace(); String partition_id = getPartitionIDFromQuery(partition, query_context); selected = merger_mutator.selectAllPartsToMergeWithinPartition( - future_merged_part, disk_space, can_merge, partition_id, final, &is_single_merged_part, &disable_reason); + future_merged_part, disk_space, can_merge, partition_id, final, &is_single_merged_part, metadata_snapshot, &disable_reason); } if (!selected) From d6fc6d35fae91764da3d3e8d60405faeeaa6765b Mon Sep 17 00:00:00 2001 From: Pavel Kruglov Date: Thu, 15 Oct 2020 19:08:24 +0300 Subject: [PATCH 05/50] Fix style --- src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp | 2 -- 1 file changed, 2 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp index a5d0db47929..624c5f156aa 100644 --- a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp +++ b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp @@ -641,8 +641,6 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mergePartsToTempor { static const String TMP_PREFIX = "tmp_merge_"; - - if (merges_blocker.isCancelled()) throw Exception("Cancelled merging parts", ErrorCodes::ABORTED); From 6c1e0e3df8590784c29f97750ddbea5fe3901037 Mon Sep 17 00:00:00 2001 From: Pavel Kruglov Date: Thu, 15 Oct 2020 19:40:20 +0300 Subject: [PATCH 06/50] Change default to false --- src/Storages/MergeTree/MergeTreeSettings.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/MergeTree/MergeTreeSettings.h b/src/Storages/MergeTree/MergeTreeSettings.h index c074c6b35b4..91c9e7988bd 100644 --- a/src/Storages/MergeTree/MergeTreeSettings.h +++ b/src/Storages/MergeTree/MergeTreeSettings.h @@ -48,7 +48,7 @@ struct Settings; M(UInt64, write_ahead_log_bytes_to_fsync, 100ULL * 1024 * 1024, "Amount of bytes, accumulated in WAL to do fsync.", 0) \ M(UInt64, write_ahead_log_interval_ms_to_fsync, 100, "Interval in milliseconds after which fsync for WAL is being done.", 0) \ M(Bool, in_memory_parts_insert_sync, false, "If true insert of part with in-memory format will wait for fsync of WAL", 0) \ - M(Bool, optimize_skip_merged_partitions, true, "Skip partitions with one part with level > 0 in optimize final", 0) \ + M(Bool, optimize_skip_merged_partitions, false, "Skip partitions with one part with level > 0 in optimize final", 0) \ \ /** Inserts settings. */ \ M(UInt64, parts_to_delay_insert, 150, "If table contains at least that many active parts in single partition, artificially slow down insert into table.", 0) \ From 581bfa425331a0f29b8e7e90ba0a7db5e9b319a6 Mon Sep 17 00:00:00 2001 From: Pavel Kruglov Date: Thu, 15 Oct 2020 22:15:40 +0300 Subject: [PATCH 07/50] Fix test --- .../0_stateless/01533_optimize_skip_merged_partitions.sql | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/01533_optimize_skip_merged_partitions.sql b/tests/queries/0_stateless/01533_optimize_skip_merged_partitions.sql index 325aaa2d6bb..70f73d960fc 100644 --- a/tests/queries/0_stateless/01533_optimize_skip_merged_partitions.sql +++ b/tests/queries/0_stateless/01533_optimize_skip_merged_partitions.sql @@ -1,6 +1,6 @@ DROP TABLE IF EXISTS optimize_final; -CREATE TABLE optimize_final(t DateTime, x Int32) ENGINE = MergeTree() PARTITION BY toYYYYMM(t) ORDER BY x; +CREATE TABLE optimize_final(t DateTime, x Int32) ENGINE = MergeTree() PARTITION BY toYYYYMM(t) ORDER BY x SETTINGS optimize_skip_merged_partitions=1; INSERT INTO optimize_final SELECT toDate('2000-01-01'), number FROM numbers(5); INSERT INTO optimize_final SELECT toDate('2000-01-01'), number + 5 FROM numbers(5); From 6a3ceba9a33818de6fa5cfb4d9aaae8169f4da94 Mon Sep 17 00:00:00 2001 From: filipe Date: Fri, 23 Oct 2020 01:28:25 -0300 Subject: [PATCH 08/50] add countMatches sql function (issue #15413) --- .../functions/string-search-functions.md | 4 + src/Functions/countMatches.cpp | 13 ++ src/Functions/countMatchesImpl.h | 132 ++++++++++++++++++ src/Functions/registerFunctionsString.cpp | 2 + 4 files changed, 151 insertions(+) create mode 100644 src/Functions/countMatches.cpp create mode 100644 src/Functions/countMatchesImpl.h diff --git a/docs/en/sql-reference/functions/string-search-functions.md b/docs/en/sql-reference/functions/string-search-functions.md index a625af14505..5f97ff304b3 100644 --- a/docs/en/sql-reference/functions/string-search-functions.md +++ b/docs/en/sql-reference/functions/string-search-functions.md @@ -394,3 +394,7 @@ For case-insensitive search or/and in UTF-8 format use functions `ngramSearchCas For UTF-8 case we use 3-gram distance. All these are not perfectly fair n-gram distances. We use 2-byte hashes to hash n-grams and then calculate the (non-)symmetric difference between these hash tables – collisions may occur. With UTF-8 case-insensitive format we do not use fair `tolower` function – we zero the 5-th bit (starting from zero) of each codepoint byte and first bit of zeroth byte if bytes more than one – this works for Latin and mostly for all Cyrillic letters. [Original article](https://clickhouse.tech/docs/en/query_language/functions/string_search_functions/) + +## countMatches(haystack, pattern) {#countmatcheshaystack-pattern} + +Returns the number of regular expression matches for a `pattern` in a `haystack`. diff --git a/src/Functions/countMatches.cpp b/src/Functions/countMatches.cpp new file mode 100644 index 00000000000..935b9fb9904 --- /dev/null +++ b/src/Functions/countMatches.cpp @@ -0,0 +1,13 @@ +#include "FunctionFactory.h" +#include "countMatchesImpl.h" + + +namespace DB +{ + +void registerFunctionCountMatches(FunctionFactory & factory) +{ + factory.registerFunction(FunctionFactory::CaseInsensitive); +} + +} diff --git a/src/Functions/countMatchesImpl.h b/src/Functions/countMatchesImpl.h new file mode 100644 index 00000000000..86e25a252ce --- /dev/null +++ b/src/Functions/countMatchesImpl.h @@ -0,0 +1,132 @@ +#pragma once + +#include +#include +#include +#include +#include +#include +#include +#include +#include + + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int ILLEGAL_TYPE_OF_ARGUMENT; + extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; + extern const int BAD_ARGUMENTS; + extern const int ILLEGAL_COLUMN; +} + +using Pos = const char *; + +class FunctionCountMatches : public IFunction +{ +public: + static constexpr auto name = "countMatches"; + static FunctionPtr create(const Context &) { return std::make_shared(); } + + String getName() const override + { + return name; + } + + size_t getNumberOfArguments() const override { return 2; } + + DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override + { + if (!isString(arguments[1])) + throw Exception("Illegal type " + arguments[1]->getName() + " of second argument of function " + getName() + ". Must be String.", + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + + return std::make_shared(); + } + + void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) const override + { + + const ColumnConst * col = checkAndGetColumnConstStringOrFixedString(block.getByPosition(arguments[1]).column.get()); + + if (!col) + throw Exception("Illegal column " + block.getByPosition(arguments[1]).column->getName() + + " of first argument of function " + getName() + ". Must be constant string.", + ErrorCodes::ILLEGAL_COLUMN); + + Regexps::Pool::Pointer re = Regexps::get(col->getValue()); + size_t capture = re->getNumberOfSubpatterns() > 0 ? 1 : 0; + OptimizedRegularExpression::MatchVec matches; + matches.resize(capture + 1); + + size_t array_argument_position = arguments[0]; + + const ColumnString * col_str = checkAndGetColumn(block.getByPosition(array_argument_position).column.get()); + const ColumnConst * col_const_str = + checkAndGetColumnConstStringOrFixedString(block.getByPosition(array_argument_position).column.get()); + + auto col_res = ColumnUInt64::create(); + ColumnUInt64::Container & vec_res = col_res->getData(); + + if (col_str) + { + const ColumnString::Chars & src_chars = col_str->getChars(); + const ColumnString::Offsets & src_offsets = col_str->getOffsets(); + + vec_res.resize(src_offsets.size()); + + size_t size = src_offsets.size(); + ColumnString::Offset current_src_offset = 0; + + for (size_t i = 0; i < size; ++i) + { + Pos pos = reinterpret_cast(&src_chars[current_src_offset]); + current_src_offset = src_offsets[i]; + Pos end = reinterpret_cast(&src_chars[current_src_offset]) - 1; + + uint64_t match_count = 0; + while (true) + { + if (!pos || pos > end) + break; + if (!re->match(pos, end - pos, matches) || !matches[0].length) + break; + pos += matches[0].offset + matches[0].length; + match_count++; + } + vec_res[i] = match_count; + } + + block.getByPosition(result).column = std::move(col_res); + } + else if (col_const_str) + { + String src = col_const_str->getValue(); + + Pos pos = reinterpret_cast(src.data()); + Pos end = reinterpret_cast(src.data() + src.size()); + + uint64_t match_count = 0; + while (true) + { + if (!pos || pos > end) + break; + if (!re->match(pos, end - pos, matches) || !matches[0].length) + break; + pos += matches[0].offset + matches[0].length; + match_count++; + } + + block.getByPosition(result).column = DataTypeUInt64().createColumnConst(col_const_str->size(), match_count); + } + else + throw Exception("Illegal columns " + block.getByPosition(array_argument_position).column->getName() + + ", " + block.getByPosition(array_argument_position).column->getName() + + " of arguments of function " + getName(), + ErrorCodes::ILLEGAL_COLUMN); + } +}; + +} diff --git a/src/Functions/registerFunctionsString.cpp b/src/Functions/registerFunctionsString.cpp index 5d4c165e1e3..647f63fe910 100644 --- a/src/Functions/registerFunctionsString.cpp +++ b/src/Functions/registerFunctionsString.cpp @@ -32,6 +32,7 @@ void registerFunctionTrim(FunctionFactory &); void registerFunctionRegexpQuoteMeta(FunctionFactory &); void registerFunctionNormalizeQuery(FunctionFactory &); void registerFunctionNormalizedQueryHash(FunctionFactory &); +void registerFunctionCountMatches(FunctionFactory &); #if USE_BASE64 void registerFunctionBase64Encode(FunctionFactory &); @@ -66,6 +67,7 @@ void registerFunctionsString(FunctionFactory & factory) registerFunctionRegexpQuoteMeta(factory); registerFunctionNormalizeQuery(factory); registerFunctionNormalizedQueryHash(factory); + registerFunctionCountMatches(factory); #if USE_BASE64 registerFunctionBase64Encode(factory); registerFunctionBase64Decode(factory); From 9120189d8a6457aff678609892b8a114a8f5d29b Mon Sep 17 00:00:00 2001 From: Pavel Kruglov Date: Tue, 10 Nov 2020 17:42:56 +0300 Subject: [PATCH 09/50] Add SelectPartsDecision enum class --- .../MergeTree/MergeTreeDataMergerMutator.cpp | 24 ++++++------- .../MergeTree/MergeTreeDataMergerMutator.h | 14 +++++--- src/Storages/StorageMergeTree.cpp | 25 ++++++------- src/Storages/StorageReplicatedMergeTree.cpp | 36 +++++++------------ 4 files changed, 43 insertions(+), 56 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp index 624c5f156aa..bb999bc71c7 100644 --- a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp +++ b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp @@ -203,7 +203,7 @@ UInt64 MergeTreeDataMergerMutator::getMaxSourcePartSizeForMutation() const return 0; } -bool MergeTreeDataMergerMutator::selectPartsToMerge( +SelectPartsDecision MergeTreeDataMergerMutator::selectPartsToMerge( FutureMergedMutatedPart & future_part, bool aggressive, size_t max_total_size_to_merge, @@ -219,7 +219,7 @@ bool MergeTreeDataMergerMutator::selectPartsToMerge( { if (out_disable_reason) *out_disable_reason = "There are no parts in the table"; - return false; + return SelectPartsDecision::CANNOT_SELECT; } time_t current_time = std::time(nullptr); @@ -335,7 +335,7 @@ bool MergeTreeDataMergerMutator::selectPartsToMerge( { if (out_disable_reason) *out_disable_reason = "There is no need to merge parts according to merge selector algorithm"; - return false; + return SelectPartsDecision::CANNOT_SELECT; } } @@ -349,29 +349,28 @@ bool MergeTreeDataMergerMutator::selectPartsToMerge( LOG_DEBUG(log, "Selected {} parts from {} to {}", parts.size(), parts.front()->name, parts.back()->name); future_part.assign(std::move(parts)); - return true; + return SelectPartsDecision::SELECTED; } -bool MergeTreeDataMergerMutator::selectAllPartsToMergeWithinPartition( +SelectPartsDecision MergeTreeDataMergerMutator::selectAllPartsToMergeWithinPartition( FutureMergedMutatedPart & future_part, UInt64 & available_disk_space, const AllowedMergingPredicate & can_merge, const String & partition_id, bool final, - bool * is_single_merged_part, const StorageMetadataPtr & metadata_snapshot, String * out_disable_reason) { MergeTreeData::DataPartsVector parts = selectAllPartsFromPartition(partition_id); if (parts.empty()) - return false; + return SelectPartsDecision::NOTHING_TO_MERGE; if (!final && parts.size() == 1) { if (out_disable_reason) *out_disable_reason = "There is only one part inside partition"; - return false; + return SelectPartsDecision::CANNOT_SELECT; } /// If final, optimize_skip_merged_partitions is true and we have only one part in partition with level > 0 @@ -379,8 +378,7 @@ bool MergeTreeDataMergerMutator::selectAllPartsToMergeWithinPartition( if (final && data.getSettings()->optimize_skip_merged_partitions && parts.size() == 1 && parts[0]->info.level > 0 && (!metadata_snapshot->hasAnyTTL() || parts[0]->checkAllTTLCalculated(metadata_snapshot))) { - *is_single_merged_part = true; - return false; + return SelectPartsDecision::NOTHING_TO_MERGE; } auto it = parts.begin(); @@ -392,7 +390,7 @@ bool MergeTreeDataMergerMutator::selectAllPartsToMergeWithinPartition( /// For the case of one part, we check that it can be merged "with itself". if ((it != parts.begin() || parts.size() == 1) && !can_merge(*prev_it, *it, out_disable_reason)) { - return false; + return SelectPartsDecision::NOTHING_TO_MERGE; } sum_bytes += (*it)->getBytesOnDisk(); @@ -422,14 +420,14 @@ bool MergeTreeDataMergerMutator::selectAllPartsToMergeWithinPartition( if (out_disable_reason) *out_disable_reason = fmt::format("Insufficient available disk space, required {}", ReadableSize(required_disk_space)); - return false; + return SelectPartsDecision::CANNOT_SELECT; } LOG_DEBUG(log, "Selected {} parts from {} to {}", parts.size(), parts.front()->name, parts.back()->name); future_part.assign(std::move(parts)); available_disk_space -= required_disk_space; - return true; + return SelectPartsDecision::SELECTED; } diff --git a/src/Storages/MergeTree/MergeTreeDataMergerMutator.h b/src/Storages/MergeTree/MergeTreeDataMergerMutator.h index a3f240f86de..4af8ab6bb6b 100644 --- a/src/Storages/MergeTree/MergeTreeDataMergerMutator.h +++ b/src/Storages/MergeTree/MergeTreeDataMergerMutator.h @@ -16,6 +16,13 @@ namespace DB class MergeListEntry; class MergeProgressCallback; +enum class SelectPartsDecision +{ + SELECTED = 0, + CANNOT_SELECT = 1, + NOTHING_TO_MERGE = 2, +}; + /// Auxiliary struct holding metainformation for the future merged or mutated part. struct FutureMergedMutatedPart { @@ -79,7 +86,7 @@ public: * - Parts between which another part can still appear can not be merged. Refer to METR-7001. * - A part that already merges with something in one place, you can not start to merge into something else in another place. */ - bool selectPartsToMerge( + SelectPartsDecision selectPartsToMerge( FutureMergedMutatedPart & future_part, bool aggressive, size_t max_total_size_to_merge, @@ -90,15 +97,14 @@ public: /** Select all the parts in the specified partition for merge, if possible. * final - choose to merge even a single part - that is, allow to merge one part "with itself", * but if setting optimize_skip_merged_partitions is true (it's true as default) than single part with level > 0 - * won't be merged with itself. + * and without expired TTL won't be merged with itself. */ - bool selectAllPartsToMergeWithinPartition( + SelectPartsDecision selectAllPartsToMergeWithinPartition( FutureMergedMutatedPart & future_part, UInt64 & available_disk_space, const AllowedMergingPredicate & can_merge, const String & partition_id, bool final, - bool * is_single_merged_part, const StorageMetadataPtr & metadata_snapshot, String * out_disable_reason = nullptr); diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index 69558b13d9b..44715e51a99 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -653,11 +653,7 @@ bool StorageMergeTree::merge( && getCurrentMutationVersion(left, lock) == getCurrentMutationVersion(right, lock); }; - bool selected = false; - - /// This flag is true when there is only one part in partition, it's level > 0 - /// and setting optimize_skip_merged_partitions is true - bool is_single_merged_part = false; + SelectPartsDecision select_decision = SelectPartsDecision::NOTHING_TO_MERGE; if (partition_id.empty()) { @@ -669,7 +665,7 @@ bool StorageMergeTree::merge( /// possible. if (max_source_parts_size > 0) { - selected = merger_mutator.selectPartsToMerge( + select_decision = merger_mutator.selectPartsToMerge( future_part, aggressive, max_source_parts_size, @@ -685,13 +681,13 @@ bool StorageMergeTree::merge( while (true) { UInt64 disk_space = getStoragePolicy()->getMaxUnreservedFreeSpace(); - selected = merger_mutator.selectAllPartsToMergeWithinPartition( - future_part, disk_space, can_merge, partition_id, final, &is_single_merged_part, metadata_snapshot, out_disable_reason); + select_decision = merger_mutator.selectAllPartsToMergeWithinPartition( + future_part, disk_space, can_merge, partition_id, final, metadata_snapshot, out_disable_reason); /// If final - we will wait for currently processing merges to finish and continue. /// TODO Respect query settings for timeout if (final - && !selected + && select_decision != SelectPartsDecision::SELECTED && !currently_merging_mutating_parts.empty() && out_disable_reason && out_disable_reason->empty()) @@ -711,13 +707,12 @@ bool StorageMergeTree::merge( } } - if (!selected) + /// If final and there is nothing to merge we treat this merge as successful. + if (final && select_decision == SelectPartsDecision::NOTHING_TO_MERGE) + return true; + + if (select_decision != SelectPartsDecision::SELECTED) { - /// If is_single_merged_part is true we treat this part as already merged - if (final && is_single_merged_part) - { - return true; - } if (out_disable_reason) { if (!out_disable_reason->empty()) diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 09d36ddd4e4..c9b8bafdb5a 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -2681,7 +2681,7 @@ void StorageReplicatedMergeTree::mergeSelectingTask() FutureMergedMutatedPart future_merged_part; if (max_source_parts_size_for_merge > 0 && - merger_mutator.selectPartsToMerge(future_merged_part, false, max_source_parts_size_for_merge, merge_pred, merge_with_ttl_allowed, nullptr)) + merger_mutator.selectPartsToMerge(future_merged_part, false, max_source_parts_size_for_merge, merge_pred, merge_with_ttl_allowed, nullptr) == SelectPartsDecision::SELECTED) { create_result = createLogEntryToMergeParts(zookeeper, future_merged_part.parts, future_merged_part.name, future_merged_part.type, deduplicate, nullptr, merge_pred.getVersion(), future_merged_part.merge_type); @@ -3762,20 +3762,11 @@ bool StorageReplicatedMergeTree::optimize( ReplicatedMergeTreeMergePredicate can_merge = queue.getMergePredicate(zookeeper); FutureMergedMutatedPart future_merged_part; + SelectPartsDecision select_decision = merger_mutator.selectAllPartsToMergeWithinPartition( + future_merged_part, disk_space, can_merge, partition_id, true, metadata_snapshot, nullptr); - /// This flag is true when there is only one part in partition, it's level > 0 - /// and setting optimize_skip_merged_partitions is true - bool is_single_merged_part = false; - bool selected = merger_mutator.selectAllPartsToMergeWithinPartition( - future_merged_part, disk_space, can_merge, partition_id, true, &is_single_merged_part, metadata_snapshot, nullptr); - - if (!selected) - { - /// If is_single_merged_part is true we treat this part as already merged - if (is_single_merged_part) - return true; + if (select_decision != SelectPartsDecision::SELECTED) break; - } ReplicatedMergeTreeLogEntryData merge_entry; CreateMergeEntryResult create_result = createLogEntryToMergeParts( @@ -3807,29 +3798,26 @@ bool StorageReplicatedMergeTree::optimize( FutureMergedMutatedPart future_merged_part; String disable_reason; - bool selected = false; + SelectPartsDecision select_decision = SelectPartsDecision::NOTHING_TO_MERGE; - /// This flag is true when there is only one part in partition, it's level > 0 - /// and setting optimize_skip_merged_partitions is true - bool is_single_merged_part = false; if (!partition) { - selected = merger_mutator.selectPartsToMerge( + select_decision = merger_mutator.selectPartsToMerge( future_merged_part, true, storage_settings_ptr->max_bytes_to_merge_at_max_space_in_pool, can_merge, false, &disable_reason); } else { UInt64 disk_space = getStoragePolicy()->getMaxUnreservedFreeSpace(); String partition_id = getPartitionIDFromQuery(partition, query_context); - selected = merger_mutator.selectAllPartsToMergeWithinPartition( - future_merged_part, disk_space, can_merge, partition_id, final, &is_single_merged_part, metadata_snapshot, &disable_reason); + select_decision = merger_mutator.selectAllPartsToMergeWithinPartition( + future_merged_part, disk_space, can_merge, partition_id, final, metadata_snapshot, &disable_reason); } - if (!selected) + if (final && select_decision == SelectPartsDecision::NOTHING_TO_MERGE) + break; + + if (select_decision != SelectPartsDecision::SELECTED) { - /// If is_single_merged_part is true we treat this part as already merged - if (final && is_single_merged_part) - return true; std::stringstream message; message << "Cannot select parts for optimization"; if (!disable_reason.empty()) From 1c35b47073a8dc7999b10721777639cd024463ec Mon Sep 17 00:00:00 2001 From: Pavel Kruglov Date: Tue, 10 Nov 2020 23:01:43 +0300 Subject: [PATCH 10/50] Add checking select decision in merge --- src/Storages/StorageMergeTree.cpp | 11 +++++++++-- src/Storages/StorageMergeTree.h | 10 ++++++++-- 2 files changed, 17 insertions(+), 4 deletions(-) diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index 63cf512fa6e..2abdafd3433 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -635,7 +635,7 @@ void StorageMergeTree::loadMutations() } std::shared_ptr StorageMergeTree::selectPartsToMerge( - const StorageMetadataPtr & metadata_snapshot, bool aggressive, const String & partition_id, bool final, String * out_disable_reason, TableLockHolder & /* table_lock_holder */) + const StorageMetadataPtr & metadata_snapshot, bool aggressive, const String & partition_id, bool final, String * out_disable_reason, TableLockHolder & /* table_lock_holder */, SelectPartsDecision * select_decision_out) { std::unique_lock lock(currently_processing_in_background_mutex); auto data_settings = getSettings(); @@ -743,7 +743,14 @@ bool StorageMergeTree::merge( auto table_lock_holder = lockForShare(RWLockImpl::NO_QUERY, getSettings()->lock_acquire_timeout_for_background_operations); auto metadata_snapshot = getInMemoryMetadataPtr(); - auto merge_mutate_entry = selectPartsToMerge(metadata_snapshot, aggressive, partition_id, final, out_disable_reason, table_lock_holder); + SelectPartsDecision select_decision; + + auto merge_mutate_entry = selectPartsToMerge(metadata_snapshot, aggressive, partition_id, final, out_disable_reason, table_lock_holder, &select_decision); + + /// If final and there is nothing to merge then we treat this merge as successful + if (final && select_decision == SelectPartsDecision::NOTHING_TO_MERGE) + return true; + if (!merge_mutate_entry) return false; diff --git a/src/Storages/StorageMergeTree.h b/src/Storages/StorageMergeTree.h index 6cddfe77fee..b5341f636a4 100644 --- a/src/Storages/StorageMergeTree.h +++ b/src/Storages/StorageMergeTree.h @@ -179,8 +179,14 @@ private: {} }; - std::shared_ptr selectPartsToMerge(const StorageMetadataPtr & metadata_snapshot, bool aggressive, const String & partition_id, bool final, String * disable_reason, TableLockHolder & table_lock_holder); - bool mergeSelectedParts(const StorageMetadataPtr & metadata_snapshot, bool deduplicate, MergeMutateSelectedEntry & entry, TableLockHolder & table_lock_holder); + std::shared_ptr selectPartsToMerge( + const StorageMetadataPtr & metadata_snapshot, + bool aggressive, + const String & partition_id, + bool final, + String * disable_reason, + TableLockHolder & table_lock_holder, + SelectPartsDecision * select_decision_out = nullptr); bool mergeSelectedParts(const StorageMetadataPtr & metadata_snapshot, bool deduplicate, MergeMutateSelectedEntry & entry, TableLockHolder & table_lock_holder); std::shared_ptr selectPartsToMutate(const StorageMetadataPtr & metadata_snapshot, String * disable_reason, TableLockHolder & table_lock_holder); bool mutateSelectedPart(const StorageMetadataPtr & metadata_snapshot, MergeMutateSelectedEntry & entry, TableLockHolder & table_lock_holder); From dd06ff4f942f5432e58276b9e143056d6005a600 Mon Sep 17 00:00:00 2001 From: Pavel Kruglov Date: Wed, 11 Nov 2020 00:02:11 +0300 Subject: [PATCH 11/50] Minor change --- src/Storages/StorageMergeTree.cpp | 6 +++--- src/Storages/StorageReplicatedMergeTree.cpp | 4 ++-- .../0_stateless/01533_optimize_skip_merged_partitions.sql | 8 ++++---- 3 files changed, 9 insertions(+), 9 deletions(-) diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index 2abdafd3433..b4722af62a1 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -653,10 +653,10 @@ std::shared_ptr StorageMergeTree::se if (!left) return !currently_merging_mutating_parts.count(right); return !currently_merging_mutating_parts.count(left) && !currently_merging_mutating_parts.count(right) - && getCurrentMutationVersion(left, lock) == getCurrentMutationVersion(right, lock); + && getCurrentMutationVersion(left, lock) == getCurrentMutationVersion(right, lock); }; - SelectPartsDecision select_decision = SelectPartsDecision::NOTHING_TO_MERGE; + SelectPartsDecision select_decision; if (partition_id.empty()) { @@ -696,7 +696,7 @@ std::shared_ptr StorageMergeTree::se && out_disable_reason->empty()) { LOG_DEBUG(log, "Waiting for currently running merges ({} parts are merging right now) to perform OPTIMIZE FINAL", - currently_merging_mutating_parts.size()); + currently_merging_mutating_parts.size()); if (std::cv_status::timeout == currently_processing_in_background_condition.wait_for( lock, std::chrono::seconds(DBMS_DEFAULT_LOCK_ACQUIRE_TIMEOUT_SEC))) diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index fbe96c200f3..ef30f0987e4 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -3831,7 +3831,7 @@ bool StorageReplicatedMergeTree::optimize( } if (try_no == max_retries) return handle_noop("Can't create merge queue node in ZooKeeper, because log was updated in every of " - + toString(max_retries) + " tries"); + + toString(max_retries) + " tries"); } } else @@ -3844,7 +3844,7 @@ bool StorageReplicatedMergeTree::optimize( FutureMergedMutatedPart future_merged_part; String disable_reason; - SelectPartsDecision select_decision = SelectPartsDecision::NOTHING_TO_MERGE; + SelectPartsDecision select_decision; if (!partition) { diff --git a/tests/queries/0_stateless/01533_optimize_skip_merged_partitions.sql b/tests/queries/0_stateless/01533_optimize_skip_merged_partitions.sql index 70f73d960fc..26d60480114 100644 --- a/tests/queries/0_stateless/01533_optimize_skip_merged_partitions.sql +++ b/tests/queries/0_stateless/01533_optimize_skip_merged_partitions.sql @@ -2,13 +2,13 @@ DROP TABLE IF EXISTS optimize_final; CREATE TABLE optimize_final(t DateTime, x Int32) ENGINE = MergeTree() PARTITION BY toYYYYMM(t) ORDER BY x SETTINGS optimize_skip_merged_partitions=1; -INSERT INTO optimize_final SELECT toDate('2000-01-01'), number FROM numbers(5); -INSERT INTO optimize_final SELECT toDate('2000-01-01'), number + 5 FROM numbers(5); +INSERT INTO optimize_final SELECT toDate('2020-01-01'), number FROM numbers(5); +INSERT INTO optimize_final SELECT toDate('2020-01-01'), number + 5 FROM numbers(5); OPTIMIZE TABLE optimize_final FINAL; -INSERT INTO optimize_final SELECT toDate('2020-01-01'), number FROM numbers(5); -INSERT INTO optimize_final SELECT toDate('2020-01-01'), number + 5 FROM numbers(5); +INSERT INTO optimize_final SELECT toDate('2000-01-01'), number FROM numbers(5); +INSERT INTO optimize_final SELECT toDate('2000-01-01'), number + 5 FROM numbers(5); OPTIMIZE TABLE optimize_final FINAL; From 287d71b3635262a677d3ef0c41ad230edd4fa135 Mon Sep 17 00:00:00 2001 From: Pavel Kruglov Date: Wed, 11 Nov 2020 13:34:32 +0300 Subject: [PATCH 12/50] Fix build --- src/Storages/StorageMergeTree.cpp | 2 +- src/Storages/StorageReplicatedMergeTree.cpp | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index b4722af62a1..17459c7bdb0 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -656,7 +656,7 @@ std::shared_ptr StorageMergeTree::se && getCurrentMutationVersion(left, lock) == getCurrentMutationVersion(right, lock); }; - SelectPartsDecision select_decision; + SelectPartsDecision select_decision = SelectPartsDecision::CANNOT_SELECT; if (partition_id.empty()) { diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index ef30f0987e4..7290088056e 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -3844,7 +3844,7 @@ bool StorageReplicatedMergeTree::optimize( FutureMergedMutatedPart future_merged_part; String disable_reason; - SelectPartsDecision select_decision; + SelectPartsDecision select_decision = SelectPartsDecision::CANNOT_SELECT; if (!partition) { From 148dc71d0eba876a38f54bec069644edfba61282 Mon Sep 17 00:00:00 2001 From: Pavel Kruglov Date: Wed, 11 Nov 2020 13:38:00 +0300 Subject: [PATCH 13/50] Fix StorageMergeTree.h --- src/Storages/StorageMergeTree.h | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/Storages/StorageMergeTree.h b/src/Storages/StorageMergeTree.h index b5341f636a4..a5992614247 100644 --- a/src/Storages/StorageMergeTree.h +++ b/src/Storages/StorageMergeTree.h @@ -186,7 +186,8 @@ private: bool final, String * disable_reason, TableLockHolder & table_lock_holder, - SelectPartsDecision * select_decision_out = nullptr); bool mergeSelectedParts(const StorageMetadataPtr & metadata_snapshot, bool deduplicate, MergeMutateSelectedEntry & entry, TableLockHolder & table_lock_holder); + SelectPartsDecision * select_decision_out = nullptr); + bool mergeSelectedParts(const StorageMetadataPtr & metadata_snapshot, bool deduplicate, MergeMutateSelectedEntry & entry, TableLockHolder & table_lock_holder); std::shared_ptr selectPartsToMutate(const StorageMetadataPtr & metadata_snapshot, String * disable_reason, TableLockHolder & table_lock_holder); bool mutateSelectedPart(const StorageMetadataPtr & metadata_snapshot, MergeMutateSelectedEntry & entry, TableLockHolder & table_lock_holder); From 4c3085775977f3524ea7a8984ebae8bd55127fab Mon Sep 17 00:00:00 2001 From: Pavel Kruglov Date: Fri, 20 Nov 2020 01:22:40 +0300 Subject: [PATCH 14/50] Minor change --- src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp | 4 ++-- src/Storages/StorageMergeTree.cpp | 4 ++-- src/Storages/StorageReplicatedMergeTree.cpp | 3 ++- 3 files changed, 6 insertions(+), 5 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp index 778bc4225a3..271d70ff170 100644 --- a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp +++ b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp @@ -370,7 +370,7 @@ SelectPartsDecision MergeTreeDataMergerMutator::selectAllPartsToMergeWithinParti MergeTreeData::DataPartsVector parts = selectAllPartsFromPartition(partition_id); if (parts.empty()) - return SelectPartsDecision::NOTHING_TO_MERGE; + return SelectPartsDecision::CANNOT_SELECT; if (!final && parts.size() == 1) { @@ -396,7 +396,7 @@ SelectPartsDecision MergeTreeDataMergerMutator::selectAllPartsToMergeWithinParti /// For the case of one part, we check that it can be merged "with itself". if ((it != parts.begin() || parts.size() == 1) && !can_merge(*prev_it, *it, out_disable_reason)) { - return SelectPartsDecision::NOTHING_TO_MERGE; + return SelectPartsDecision::CANNOT_SELECT; } sum_bytes += (*it)->getBytesOnDisk(); diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index 17459c7bdb0..9f358d35cc7 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -747,8 +747,8 @@ bool StorageMergeTree::merge( auto merge_mutate_entry = selectPartsToMerge(metadata_snapshot, aggressive, partition_id, final, out_disable_reason, table_lock_holder, &select_decision); - /// If final and there is nothing to merge then we treat this merge as successful - if (final && select_decision == SelectPartsDecision::NOTHING_TO_MERGE) + /// If there is nothing to merge then we treat this merge as successful (needed for optimize final optimization) + if (select_decision == SelectPartsDecision::NOTHING_TO_MERGE) return true; if (!merge_mutate_entry) diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 7290088056e..61ae1328eb2 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -3859,7 +3859,8 @@ bool StorageReplicatedMergeTree::optimize( future_merged_part, disk_space, can_merge, partition_id, final, metadata_snapshot, &disable_reason); } - if (final && select_decision == SelectPartsDecision::NOTHING_TO_MERGE) + /// If there is nothing to merge then we treat this merge as successful (needed for optimize final optimization) + if (select_decision == SelectPartsDecision::NOTHING_TO_MERGE) break; if (select_decision != SelectPartsDecision::SELECTED) From ca3fe49a2a840ecf1795a9a064500a57f2e2c408 Mon Sep 17 00:00:00 2001 From: Pavel Kruglov Date: Fri, 20 Nov 2020 17:29:13 +0300 Subject: [PATCH 15/50] Make setting global --- src/Core/Settings.h | 1 + .../MergeTree/MergeTreeDataMergerMutator.cpp | 5 +++-- src/Storages/MergeTree/MergeTreeDataMergerMutator.h | 3 ++- src/Storages/MergeTree/MergeTreeSettings.h | 1 - src/Storages/StorageMergeTree.cpp | 13 +++++++------ src/Storages/StorageMergeTree.h | 3 ++- src/Storages/StorageReplicatedMergeTree.cpp | 4 ++-- .../01533_optimize_skip_merged_partitions.sql | 4 +++- 8 files changed, 20 insertions(+), 14 deletions(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 580756361b1..59a36e5452b 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -396,6 +396,7 @@ class IColumn; M(Bool, allow_non_metadata_alters, true, "Allow to execute alters which affects not only tables metadata, but also data on disk", 0) \ M(Bool, enable_global_with_statement, false, "Propagate WITH statements to UNION queries and all subqueries", 0) \ M(Bool, aggregate_functions_null_for_empty, false, "Rewrite all aggregate functions in a query, adding -OrNull suffix to them", 0) \ + M(Bool, optimize_skip_merged_partitions, false, "Skip partitions with one part with level > 0 in optimize final", 0) \ \ /** Obsolete settings that do nothing but left for compatibility reasons. Remove each one after half a year of obsolescence. */ \ \ diff --git a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp index 271d70ff170..9f2d5884121 100644 --- a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp +++ b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp @@ -365,7 +365,8 @@ SelectPartsDecision MergeTreeDataMergerMutator::selectAllPartsToMergeWithinParti const String & partition_id, bool final, const StorageMetadataPtr & metadata_snapshot, - String * out_disable_reason) + String * out_disable_reason, + const Context * context) { MergeTreeData::DataPartsVector parts = selectAllPartsFromPartition(partition_id); @@ -381,7 +382,7 @@ SelectPartsDecision MergeTreeDataMergerMutator::selectAllPartsToMergeWithinParti /// If final, optimize_skip_merged_partitions is true and we have only one part in partition with level > 0 /// than we don't select it to merge. But if there are some expired TTL then merge is needed - if (final && data.getSettings()->optimize_skip_merged_partitions && parts.size() == 1 && parts[0]->info.level > 0 && + if (final && context && context->getSettingsRef().optimize_skip_merged_partitions && parts.size() == 1 && parts[0]->info.level > 0 && (!metadata_snapshot->hasAnyTTL() || parts[0]->checkAllTTLCalculated(metadata_snapshot))) { return SelectPartsDecision::NOTHING_TO_MERGE; diff --git a/src/Storages/MergeTree/MergeTreeDataMergerMutator.h b/src/Storages/MergeTree/MergeTreeDataMergerMutator.h index 30078820734..ae54f44bfcb 100644 --- a/src/Storages/MergeTree/MergeTreeDataMergerMutator.h +++ b/src/Storages/MergeTree/MergeTreeDataMergerMutator.h @@ -105,7 +105,8 @@ public: const String & partition_id, bool final, const StorageMetadataPtr & metadata_snapshot, - String * out_disable_reason = nullptr); + String * out_disable_reason = nullptr, + const Context * context = nullptr); /** Merge the parts. * If `reservation != nullptr`, now and then reduces the size of the reserved space diff --git a/src/Storages/MergeTree/MergeTreeSettings.h b/src/Storages/MergeTree/MergeTreeSettings.h index 9fb7354e182..fe97298fe75 100644 --- a/src/Storages/MergeTree/MergeTreeSettings.h +++ b/src/Storages/MergeTree/MergeTreeSettings.h @@ -48,7 +48,6 @@ struct Settings; M(UInt64, write_ahead_log_bytes_to_fsync, 100ULL * 1024 * 1024, "Amount of bytes, accumulated in WAL to do fsync.", 0) \ M(UInt64, write_ahead_log_interval_ms_to_fsync, 100, "Interval in milliseconds after which fsync for WAL is being done.", 0) \ M(Bool, in_memory_parts_insert_sync, false, "If true insert of part with in-memory format will wait for fsync of WAL", 0) \ - M(Bool, optimize_skip_merged_partitions, false, "Skip partitions with one part with level > 0 in optimize final", 0) \ \ /** Inserts settings. */ \ M(UInt64, parts_to_delay_insert, 150, "If table contains at least that many active parts in single partition, artificially slow down insert into table.", 0) \ diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index 9f358d35cc7..268207b5cfb 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -635,7 +635,7 @@ void StorageMergeTree::loadMutations() } std::shared_ptr StorageMergeTree::selectPartsToMerge( - const StorageMetadataPtr & metadata_snapshot, bool aggressive, const String & partition_id, bool final, String * out_disable_reason, TableLockHolder & /* table_lock_holder */, SelectPartsDecision * select_decision_out) + const StorageMetadataPtr & metadata_snapshot, bool aggressive, const String & partition_id, bool final, String * out_disable_reason, TableLockHolder & /* table_lock_holder */, const Context * context, SelectPartsDecision * select_decision_out) { std::unique_lock lock(currently_processing_in_background_mutex); auto data_settings = getSettings(); @@ -685,7 +685,7 @@ std::shared_ptr StorageMergeTree::se { UInt64 disk_space = getStoragePolicy()->getMaxUnreservedFreeSpace(); select_decision = merger_mutator.selectAllPartsToMergeWithinPartition( - future_part, disk_space, can_merge, partition_id, final, metadata_snapshot, out_disable_reason); + future_part, disk_space, can_merge, partition_id, final, metadata_snapshot, out_disable_reason, context); /// If final - we will wait for currently processing merges to finish and continue. /// TODO Respect query settings for timeout @@ -738,14 +738,15 @@ bool StorageMergeTree::merge( const String & partition_id, bool final, bool deduplicate, - String * out_disable_reason) + String * out_disable_reason, + const Context * context) { auto table_lock_holder = lockForShare(RWLockImpl::NO_QUERY, getSettings()->lock_acquire_timeout_for_background_operations); auto metadata_snapshot = getInMemoryMetadataPtr(); SelectPartsDecision select_decision; - auto merge_mutate_entry = selectPartsToMerge(metadata_snapshot, aggressive, partition_id, final, out_disable_reason, table_lock_holder, &select_decision); + auto merge_mutate_entry = selectPartsToMerge(metadata_snapshot, aggressive, partition_id, final, out_disable_reason, table_lock_holder, context, &select_decision); /// If there is nothing to merge then we treat this merge as successful (needed for optimize final optimization) if (select_decision == SelectPartsDecision::NOTHING_TO_MERGE) @@ -1040,7 +1041,7 @@ bool StorageMergeTree::optimize( for (const String & partition_id : partition_ids) { - if (!merge(true, partition_id, true, deduplicate, &disable_reason)) + if (!merge(true, partition_id, true, deduplicate, &disable_reason, &context)) { std::stringstream message; message.exceptions(std::ios::failbit); @@ -1063,7 +1064,7 @@ bool StorageMergeTree::optimize( if (partition) partition_id = getPartitionIDFromQuery(partition, context); - if (!merge(true, partition_id, final, deduplicate, &disable_reason)) + if (!merge(true, partition_id, final, deduplicate, &disable_reason, &context)) { std::stringstream message; message.exceptions(std::ios::failbit); diff --git a/src/Storages/StorageMergeTree.h b/src/Storages/StorageMergeTree.h index a5992614247..15fa3bb41f4 100644 --- a/src/Storages/StorageMergeTree.h +++ b/src/Storages/StorageMergeTree.h @@ -137,7 +137,7 @@ private: * If aggressive - when selects parts don't takes into account their ratio size and novelty (used for OPTIMIZE query). * Returns true if merge is finished successfully. */ - bool merge(bool aggressive, const String & partition_id, bool final, bool deduplicate, String * out_disable_reason = nullptr); + bool merge(bool aggressive, const String & partition_id, bool final, bool deduplicate, String * out_disable_reason = nullptr, const Context * context = nullptr); ActionLock stopMergesAndWait(); @@ -186,6 +186,7 @@ private: bool final, String * disable_reason, TableLockHolder & table_lock_holder, + const Context * context = nullptr, SelectPartsDecision * select_decision_out = nullptr); bool mergeSelectedParts(const StorageMetadataPtr & metadata_snapshot, bool deduplicate, MergeMutateSelectedEntry & entry, TableLockHolder & table_lock_holder); diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 61ae1328eb2..0b8f6b2c5f5 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -3809,7 +3809,7 @@ bool StorageReplicatedMergeTree::optimize( FutureMergedMutatedPart future_merged_part; SelectPartsDecision select_decision = merger_mutator.selectAllPartsToMergeWithinPartition( - future_merged_part, disk_space, can_merge, partition_id, true, metadata_snapshot, nullptr); + future_merged_part, disk_space, can_merge, partition_id, true, metadata_snapshot, nullptr, &query_context); if (select_decision != SelectPartsDecision::SELECTED) break; @@ -3856,7 +3856,7 @@ bool StorageReplicatedMergeTree::optimize( UInt64 disk_space = getStoragePolicy()->getMaxUnreservedFreeSpace(); String partition_id = getPartitionIDFromQuery(partition, query_context); select_decision = merger_mutator.selectAllPartsToMergeWithinPartition( - future_merged_part, disk_space, can_merge, partition_id, final, metadata_snapshot, &disable_reason); + future_merged_part, disk_space, can_merge, partition_id, final, metadata_snapshot, &disable_reason, &query_context); } /// If there is nothing to merge then we treat this merge as successful (needed for optimize final optimization) diff --git a/tests/queries/0_stateless/01533_optimize_skip_merged_partitions.sql b/tests/queries/0_stateless/01533_optimize_skip_merged_partitions.sql index 26d60480114..63bbbd2d721 100644 --- a/tests/queries/0_stateless/01533_optimize_skip_merged_partitions.sql +++ b/tests/queries/0_stateless/01533_optimize_skip_merged_partitions.sql @@ -1,6 +1,8 @@ DROP TABLE IF EXISTS optimize_final; -CREATE TABLE optimize_final(t DateTime, x Int32) ENGINE = MergeTree() PARTITION BY toYYYYMM(t) ORDER BY x SETTINGS optimize_skip_merged_partitions=1; +SET optimize_skip_merged_partitions=1; + +CREATE TABLE optimize_final(t DateTime, x Int32) ENGINE = MergeTree() PARTITION BY toYYYYMM(t) ORDER BY x; INSERT INTO optimize_final SELECT toDate('2020-01-01'), number FROM numbers(5); INSERT INTO optimize_final SELECT toDate('2020-01-01'), number + 5 FROM numbers(5); From 623dc2df7af1d0fbe1ba1d44cc9fcd1d1a87339b Mon Sep 17 00:00:00 2001 From: Pavel Kruglov Date: Fri, 20 Nov 2020 17:32:39 +0300 Subject: [PATCH 16/50] Update comment --- src/Storages/MergeTree/MergeTreeDataMergerMutator.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/MergeTree/MergeTreeDataMergerMutator.h b/src/Storages/MergeTree/MergeTreeDataMergerMutator.h index ae54f44bfcb..4430e8a48b4 100644 --- a/src/Storages/MergeTree/MergeTreeDataMergerMutator.h +++ b/src/Storages/MergeTree/MergeTreeDataMergerMutator.h @@ -95,7 +95,7 @@ public: /** Select all the parts in the specified partition for merge, if possible. * final - choose to merge even a single part - that is, allow to merge one part "with itself", - * but if setting optimize_skip_merged_partitions is true (it's true as default) than single part with level > 0 + * but if setting optimize_skip_merged_partitions is true than single part with level > 0 * and without expired TTL won't be merged with itself. */ SelectPartsDecision selectAllPartsToMergeWithinPartition( From 5f330a9366e7fc9ba1f5fc7e2d64183dc6f500fe Mon Sep 17 00:00:00 2001 From: vdimir Date: Sat, 28 Nov 2020 23:11:15 +0300 Subject: [PATCH 17/50] IP dictionary works with aliased types `IPv4` and `IPv6` (close #6804) --- src/Dictionaries/IPAddressDictionary.cpp | 9 ++++++--- tests/queries/0_stateless/01018_ip_dictionary.reference | 2 ++ tests/queries/0_stateless/01018_ip_dictionary.sql | 4 ++++ 3 files changed, 12 insertions(+), 3 deletions(-) diff --git a/src/Dictionaries/IPAddressDictionary.cpp b/src/Dictionaries/IPAddressDictionary.cpp index 1e6ac5f7783..82e27361dc9 100644 --- a/src/Dictionaries/IPAddressDictionary.cpp +++ b/src/Dictionaries/IPAddressDictionary.cpp @@ -112,9 +112,12 @@ static void validateKeyTypes(const DataTypes & key_types) if (key_types.empty() || key_types.size() > 2) throw Exception{"Expected a single IP address or IP with mask", ErrorCodes::TYPE_MISMATCH}; - const auto & actual_type = key_types[0]->getName(); - if (actual_type != "UInt32" && actual_type != "FixedString(16)") - throw Exception{"Key does not match, expected either UInt32 or FixedString(16)", ErrorCodes::TYPE_MISMATCH}; + const auto * key_ipv4type = typeid_cast(key_types[0].get()); + const auto * key_ipv6type = typeid_cast(key_types[0].get()); + + if (key_ipv4type == nullptr && (key_ipv6type == nullptr || key_ipv6type->getN() != 16)) + throw Exception{"Key does not match, expected either `IPv4` (`UInt32`) or `IPv6` (`FixedString(16)`)", + ErrorCodes::TYPE_MISMATCH}; if (key_types.size() > 1) { diff --git a/tests/queries/0_stateless/01018_ip_dictionary.reference b/tests/queries/0_stateless/01018_ip_dictionary.reference index 7fcf75f4d6d..d791352e01c 100644 --- a/tests/queries/0_stateless/01018_ip_dictionary.reference +++ b/tests/queries/0_stateless/01018_ip_dictionary.reference @@ -50,6 +50,8 @@ 1 1 1 +1 +1 ***ipv4 trie dict mask*** 1 1 diff --git a/tests/queries/0_stateless/01018_ip_dictionary.sql b/tests/queries/0_stateless/01018_ip_dictionary.sql index 790d38b9306..53ddf1111c7 100644 --- a/tests/queries/0_stateless/01018_ip_dictionary.sql +++ b/tests/queries/0_stateless/01018_ip_dictionary.sql @@ -95,6 +95,10 @@ SELECT 11212 == dictGetUInt32('database_for_dict.dict_ipv4_trie', 'asn', tuple(I SELECT 11211 == dictGetUInt32('database_for_dict.dict_ipv4_trie', 'asn', tuple(IPv4StringToNum('202.79.32.2'))); +-- check that dictionary works with aliased types `IPv4` and `IPv6` +SELECT 11211 == dictGetUInt32('database_for_dict.dict_ipv4_trie', 'asn', tuple(toIPv4('202.79.32.2'))); +SELECT 11212 == dictGetUInt32('database_for_dict.dict_ipv4_trie', 'asn', tuple(toIPv6('::ffff:101.79.55.22'))); + CREATE TABLE database_for_dict.table_from_ipv4_trie_dict ( prefix String, From cb2259cfb3cf2ccc2f36152603a7e8eadb92e3b5 Mon Sep 17 00:00:00 2001 From: vdimir Date: Sun, 29 Nov 2020 19:18:53 +0300 Subject: [PATCH 18/50] Fix mixed statements and typos in 0_stateless/01018_ip_dictionary.sql --- .../0_stateless/01018_ip_dictionary.reference | 15 ++++-- .../0_stateless/01018_ip_dictionary.sql | 50 ++++++++++--------- 2 files changed, 37 insertions(+), 28 deletions(-) diff --git a/tests/queries/0_stateless/01018_ip_dictionary.reference b/tests/queries/0_stateless/01018_ip_dictionary.reference index 7fcf75f4d6d..17c781ebfec 100644 --- a/tests/queries/0_stateless/01018_ip_dictionary.reference +++ b/tests/queries/0_stateless/01018_ip_dictionary.reference @@ -111,6 +111,16 @@ 1 1 1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 ***ipv6 trie dict*** 1 1 @@ -418,8 +428,3 @@ 1 1 1 -1 -1 -1 -1 -1 diff --git a/tests/queries/0_stateless/01018_ip_dictionary.sql b/tests/queries/0_stateless/01018_ip_dictionary.sql index 790d38b9306..0215c4b78e4 100644 --- a/tests/queries/0_stateless/01018_ip_dictionary.sql +++ b/tests/queries/0_stateless/01018_ip_dictionary.sql @@ -217,6 +217,17 @@ SELECT 18 == dictGetUInt32('database_for_dict.dict_ipv4_trie', 'val', tuple(IPv4 SELECT 19 == dictGetUInt32('database_for_dict.dict_ipv4_trie', 'val', tuple(IPv4StringToNum('127.255.128.255'))); SELECT 20 == dictGetUInt32('database_for_dict.dict_ipv4_trie', 'val', tuple(IPv4StringToNum('127.255.255.128'))); +SELECT 3 == dictGetUInt32('database_for_dict.dict_ipv4_trie', 'val', tuple(IPv6StringToNum('::ffff:7f00:0'))); +SELECT 4 == dictGetUInt32('database_for_dict.dict_ipv4_trie', 'val', tuple(IPv6StringToNum('::ffff:7f00:1'))); +SELECT 3 == dictGetUInt32('database_for_dict.dict_ipv4_trie', 'val', tuple(IPv6StringToNum('::ffff:7f00:7f'))); +SELECT 2 == dictGetUInt32('database_for_dict.dict_ipv4_trie', 'val', tuple(IPv6StringToNum('::ffff:7f00:ff7f'))); +SELECT 15 == dictGetUInt32('database_for_dict.dict_ipv4_trie', 'val', tuple(IPv6StringToNum('::ffff:7fff:7f7f'))); +SELECT 16 == dictGetUInt32('database_for_dict.dict_ipv4_trie', 'val', tuple(IPv6StringToNum('::ffff:7fff:8009'))); +SELECT 16 == dictGetUInt32('database_for_dict.dict_ipv4_trie', 'val', tuple(IPv6StringToNum('::ffff:7fff:807f'))); +SELECT 18 == dictGetUInt32('database_for_dict.dict_ipv4_trie', 'val', tuple(IPv6StringToNum('::ffff:7fff:800a'))); +SELECT 19 == dictGetUInt32('database_for_dict.dict_ipv4_trie', 'val', tuple(IPv6StringToNum('::ffff:7fff:80ff'))); +SELECT 20 == dictGetUInt32('database_for_dict.dict_ipv4_trie', 'val', tuple(IPv6StringToNum('::ffff:7fff:ff80'))); + SELECT 1 == dictHas('database_for_dict.dict_ipv4_trie', tuple(IPv4StringToNum('127.0.0.0'))); SELECT 1 == dictHas('database_for_dict.dict_ipv4_trie', tuple(IPv4StringToNum('127.0.0.1'))); SELECT 1 == dictHas('database_for_dict.dict_ipv4_trie', tuple(IPv4StringToNum('127.0.0.127'))); @@ -446,7 +457,7 @@ FROM VALUES ('number UInt32', 5, 13, 24, 48, 49, 99, 127); INSERT INTO database_for_dict.table_ip_trie VALUES ('101.79.55.22', 'JA'); -INSERT INTO database_for_dict.table_ipv4_trie +INSERT INTO database_for_dict.table_ip_trie SELECT '255.255.255.255/' || toString(number) AS prefix, toString(number) AS val @@ -607,27 +618,20 @@ SELECT '99' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6 SELECT '127' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:ffff:ffff:fffe'))); SELECT '127' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv6StringToNum('ffff:ffff:ffff:ffff:ffff:ffff:ffff:ffff'))); - -SELECT '3' == dictGetUInt32('database_for_dict.dict_ipv4_trie', 'val', tuple(IPv4StringToNum('127.0.0.0'))); -SELECT '4' == dictGetUInt32('database_for_dict.dict_ipv4_trie', 'val', tuple(IPv4StringToNum('127.0.0.1'))); -SELECT '3' == dictGetUInt32('database_for_dict.dict_ipv4_trie', 'val', tuple(IPv4StringToNum('127.0.0.127'))); -SELECT '2' == dictGetUInt32('database_for_dict.dict_ipv4_trie', 'val', tuple(IPv4StringToNum('127.0.255.127'))); -SELECT '15' == dictGetUInt32('database_for_dict.dict_ipv4_trie', 'val', tuple(IPv4StringToNum('127.255.127.127'))); -SELECT '16' == dictGetUInt32('database_for_dict.dict_ipv4_trie', 'val', tuple(IPv4StringToNum('127.255.128.9'))); -SELECT '16' == dictGetUInt32('database_for_dict.dict_ipv4_trie', 'val', tuple(IPv4StringToNum('127.255.128.127'))); -SELECT '18' == dictGetUInt32('database_for_dict.dict_ipv4_trie', 'val', tuple(IPv4StringToNum('127.255.128.10'))); -SELECT '19' == dictGetUInt32('database_for_dict.dict_ipv4_trie', 'val', tuple(IPv4StringToNum('127.255.128.255'))); -SELECT '20' == dictGetUInt32('database_for_dict.dict_ipv4_trie', 'val', tuple(IPv4StringToNum('127.255.255.128'))); - -SELECT '3' == dictGetUInt32('database_for_dict.dict_ipv4_trie', 'val', tuple(IPv6StringToNum('::ffff:7f00:0'))); -SELECT '4' == dictGetUInt32('database_for_dict.dict_ipv4_trie', 'val', tuple(IPv6StringToNum('::ffff:7f00:1'))); -SELECT '3' == dictGetUInt32('database_for_dict.dict_ipv4_trie', 'val', tuple(IPv6StringToNum('::ffff:7f00:7f'))); -SELECT '2' == dictGetUInt32('database_for_dict.dict_ipv4_trie', 'val', tuple(IPv6StringToNum('::ffff:7f00:ff7f'))); -SELECT '15' == dictGetUInt32('database_for_dict.dict_ipv4_trie', 'val', tuple(IPv6StringToNum('::ffff:7fff:7f7f'))); -SELECT '16' == dictGetUInt32('database_for_dict.dict_ipv4_trie', 'val', tuple(IPv6StringToNum('::ffff:7fff:8009'))); -SELECT '16' == dictGetUInt32('database_for_dict.dict_ipv4_trie', 'val', tuple(IPv6StringToNum('::ffff:7fff:807f'))); -SELECT '18' == dictGetUInt32('database_for_dict.dict_ipv4_trie', 'val', tuple(IPv6StringToNum('::ffff:7fff:800a'))); -SELECT '19' == dictGetUInt32('database_for_dict.dict_ipv4_trie', 'val', tuple(IPv6StringToNum('::ffff:7fff:80ff'))); -SELECT '20' == dictGetUInt32('database_for_dict.dict_ipv4_trie', 'val', tuple(IPv6StringToNum('::ffff:7fff:ff80'))); +SELECT '' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv4StringToNum('0.0.0.0'))); +SELECT '' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv4StringToNum('128.0.0.0'))); +SELECT '' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv4StringToNum('240.0.0.0'))); +SELECT '5' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv4StringToNum('248.0.0.0'))); +SELECT '5' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv4StringToNum('252.0.0.0'))); +SELECT '5' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv4StringToNum('255.240.0.0'))); +SELECT '13' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv4StringToNum('255.248.0.0'))); +SELECT '13' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv4StringToNum('255.252.0.0'))); +SELECT '13' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv4StringToNum('255.255.254.0'))); +SELECT '24' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv4StringToNum('255.255.255.0'))); +SELECT '24' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv4StringToNum('255.255.255.128'))); +SELECT '24' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv4StringToNum('255.255.255.248'))); +SELECT '30' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv4StringToNum('255.255.255.252'))); +SELECT '30' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv4StringToNum('255.255.255.254'))); +SELECT '30' == dictGetString('database_for_dict.dict_ip_trie', 'val', tuple(IPv4StringToNum('255.255.255.255'))); DROP DATABASE IF EXISTS database_for_dict; From af5e2084ff19eb56e4c63f569aa1f78769b46136 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 15 Oct 2020 22:54:52 +0300 Subject: [PATCH 19/50] Port ClickHouse code to BoringSSL --- src/Common/OpenSSLHelpers.cpp | 16 ++++++++++------ 1 file changed, 10 insertions(+), 6 deletions(-) diff --git a/src/Common/OpenSSLHelpers.cpp b/src/Common/OpenSSLHelpers.cpp index 77abbf99a90..d066e8802ad 100644 --- a/src/Common/OpenSSLHelpers.cpp +++ b/src/Common/OpenSSLHelpers.cpp @@ -37,12 +37,16 @@ void encodeSHA256(const void * text, size_t size, unsigned char * out) String getOpenSSLErrors() { - BIO * mem = BIO_new(BIO_s_mem()); - SCOPE_EXIT(BIO_free(mem)); - ERR_print_errors(mem); - char * buf = nullptr; - size_t size = BIO_get_mem_data(mem, &buf); - return String(buf, size); + String res; + ERR_print_errors_cb([](const char * str, size_t len, void * ctx) + { + String & out = *reinterpret_cast(ctx); + if (!out.empty()) + out += ", "; + out.append(str, len); + return 1; + }, &res); + return res; } } From e42ff1ae0228f24f8bd362fa5d1c14b421146457 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 30 Nov 2020 10:01:55 +0300 Subject: [PATCH 20/50] Port ClickHouse code to BoringSSL --- src/Functions/FunctionsAES.h | 9 ++++----- 1 file changed, 4 insertions(+), 5 deletions(-) diff --git a/src/Functions/FunctionsAES.h b/src/Functions/FunctionsAES.h index 5a5c5dc05b0..fea47074649 100644 --- a/src/Functions/FunctionsAES.h +++ b/src/Functions/FunctionsAES.h @@ -82,10 +82,9 @@ struct KeyHolder return foldEncryptionKeyInMySQLCompatitableMode(cipher_key_size, key, folded_key); } - ~KeyHolder() - { - OPENSSL_cleanse(folded_key.data(), folded_key.size()); - } + /// There is a function to clear key securely. + /// It makes absolutely zero sense to call it here because + /// key comes from column and already copied multiple times through various memory buffers. private: std::array folded_key; @@ -119,7 +118,7 @@ inline void validateCipherMode(const EVP_CIPHER * evp_cipher) } } - throw DB::Exception("Unsupported cipher mode " + std::string(EVP_CIPHER_name(evp_cipher)), DB::ErrorCodes::BAD_ARGUMENTS); + throw DB::Exception("Unsupported cipher mode", DB::ErrorCodes::BAD_ARGUMENTS); } template From aa03162cb8bdce975135a84788c5980f3fbd82cc Mon Sep 17 00:00:00 2001 From: qianmoQ Date: Tue, 1 Dec 2020 22:23:12 +0800 Subject: [PATCH 21/50] translate documentation for playground --- docs/zh/getting-started/playground.md | 74 +++++++++++++++++---------- 1 file changed, 47 insertions(+), 27 deletions(-) diff --git a/docs/zh/getting-started/playground.md b/docs/zh/getting-started/playground.md index 192203c6fe6..670889f303c 100644 --- a/docs/zh/getting-started/playground.md +++ b/docs/zh/getting-started/playground.md @@ -1,12 +1,36 @@ --- toc_priority: 14 -toc_title: "\u266A\u64CD\u573A\u266A" +toc_title: 体验平台 --- # ClickHouse体验平台 {#clickhouse-playground} [ClickHouse体验平台](https://play.clickhouse.tech?file=welcome) 允许人们通过即时运行查询来尝试ClickHouse,而无需设置他们的服务器或集群。 -体验平台中提供了几个示例数据集以及显示ClickHouse特性的示例查询。 + +体验平台中提供几个示例数据集以及显示ClickHouse特性的示例查询。还有一些ClickHouse LTS版本可供尝试。 + +ClickHouse体验平台提供了小型集群[Managed Service for ClickHouse](https://cloud.yandex.com/services/managed-clickhouse)实例配置(4 vCPU, 32 GB RAM)它们托管在[Yandex.Cloud](https://cloud.yandex.com/). 更多信息查询[cloud providers](../commercial/cloud.md). + +您可以使用任何HTTP客户端对ClickHouse体验平台进行查询,例如[curl](https://curl.haxx.se)或者[wget](https://www.gnu.org/software/wget/),或使用[JDBC](../interfaces/jdbc.md)或者[ODBC](../interfaces/odbc.md)驱动连接。关于支持ClickHouse的软件产品的更多信息详见[here](../interfaces/index.md). + +## Credentials {#credentials} + +| 参数 | 值 | +|:--------------------|:----------------------------------------| +| HTTPS端点 | `https://play-api.clickhouse.tech:8443` | +| TCP端点 | `play-api.clickhouse.tech:9440` | +| 用户 | `playground` | +| 密码 | `clickhouse` | + +还有一些带有特定ClickHouse版本的附加信息来试验它们之间的差异(端口和用户/密码与上面相同): + +- 20.3 LTS: `play-api-v20-3.clickhouse.tech` +- 19.14 LTS: `play-api-v19-14.clickhouse.tech` + +!!! note "注意" + 所有这些端点都需要安全的TLS连接。 + +## 查询限制 {#limitations} 查询以只读用户身份执行。 这意味着一些局限性: @@ -14,33 +38,29 @@ toc_title: "\u266A\u64CD\u573A\u266A" - 不允许插入查询 还强制执行以下设置: -- [`max_result_bytes=10485760`](../operations/settings/query_complexity/#max-result-bytes) -- [`max_result_rows=2000`](../operations/settings/query_complexity/#setting-max_result_rows) -- [`result_overflow_mode=break`](../operations/settings/query_complexity/#result-overflow-mode) -- [`max_execution_time=60000`](../operations/settings/query_complexity/#max-execution-time) +- [max_result_bytes=10485760](../operations/settings/query_complexity/#max-result-bytes) +- [max_result_rows=2000](../operations/settings/query_complexity/#setting-max_result_rows) +- [result_overflow_mode=break](../operations/settings/query_complexity/#result-overflow-mode) +- [max_execution_time=60000](../operations/settings/query_complexity/#max-execution-time) -ClickHouse体验还有如下: -[ClickHouse管理服务](https://cloud.yandex.com/services/managed-clickhouse) -实例托管 [Yandex云](https://cloud.yandex.com/). -更多信息 [云提供商](../commercial/cloud.md). +## 示例 {#examples} -ClickHouse体验平台界面实际上是通过ClickHouse [HTTP API](../interfaces/http.md)接口实现的. -体验平台后端只是一个ClickHouse集群,没有任何额外的服务器端应用程序。 -体验平台也同样提供了ClickHouse HTTPS服务端口。 - -您可以使用任何HTTP客户端向体验平台进行查询,例如 [curl](https://curl.haxx.se) 或 [wget](https://www.gnu.org/software/wget/),或使用以下方式建立连接 [JDBC](../interfaces/jdbc.md) 或 [ODBC](../interfaces/odbc.md) 司机 -有关支持ClickHouse的软件产品的更多信息,请访问 [这里](../interfaces/index.md). - -| 参数 | 值 | -|:---------|:--------------------------------------| -| 服务端口 | https://play-api.clickhouse.tech:8443 | -| 用户 | `playground` | -| 密码 | `clickhouse` | - -请注意,此服务端口需要安全连接。 - -示例: +使用`curl`连接Https服务: ``` bash -curl "https://play-api.clickhouse.tech:8443/?query=SELECT+'Play+ClickHouse!';&user=playground&password=clickhouse&database=datasets" +curl "https://play-api.clickhouse.tech:8443/?query=SELECT+'Play+ClickHouse\!';&user=playground&password=clickhouse&database=datasets" ``` + +TCP连接示例[CLI](../interfaces/cli.md): + +``` bash +clickhouse client --secure -h play-api.clickhouse.tech --port 9440 -u playground --password clickhouse -q "SELECT 'Play ClickHouse\!'" +``` + +## Implementation Details {#implementation-details} + +ClickHouse体验平台界面实际上是通过ClickHouse [HTTP API](../interfaces/http.md)接口实现的。 +ClickHouse体验平台是一个ClickHouse集群,没有任何附加的服务器端应用程序。如上所述,ClickHouse的HTTPS和TCP/TLS端点也可以作为体验平台的一部分公开使用, 代理通过[Cloudflare Spectrum](https://www.cloudflare.com/products/cloudflare-spectrum/)增加一层额外的保护和改善连接。 + +!!! warning "注意" + **强烈不推荐**在任何其他情况下将ClickHouse服务器暴露给公共互联网。确保它只在私有网络上侦听,并由正确配置的防火墙监控。 From faf12dc5cfb7e48dba4e0271a170ac0f0563448e Mon Sep 17 00:00:00 2001 From: qianmoQ Date: Tue, 1 Dec 2020 22:29:35 +0800 Subject: [PATCH 22/50] translate documentation for interfaces index --- docs/zh/interfaces/index.md | 34 +++++++++++++++++++++------------- 1 file changed, 21 insertions(+), 13 deletions(-) diff --git a/docs/zh/interfaces/index.md b/docs/zh/interfaces/index.md index 8dd8e38b34c..b678adc765a 100644 --- a/docs/zh/interfaces/index.md +++ b/docs/zh/interfaces/index.md @@ -1,19 +1,27 @@ +--- +toc_folder_title: Interfaces +toc_priority: 14 +toc_title: 客户端 +--- + # 客户端 {#interfaces} -ClickHouse提供了两个网络接口(两者都可以选择包装在TLS中以提高安全性): +ClickHouse提供了两个网络接口(两个都可以选择包装在TLS中以增加安全性): -- [HTTP](http.md),记录在案,易于使用. -- [本地TCP](tcp.md),这有较少的开销. +- [HTTP](http.md), 包含文档,易于使用。 +- [Native TCP](../interfaces/tcp.md),简单,方便使用。 -在大多数情况下,建议使用适当的工具或库,而不是直接与这些工具或库进行交互。 Yandex的官方支持如下: -\* [命令行客户端](cli.md) -\* [JDBC驱动程序](jdbc.md) -\* [ODBC驱动程序](odbc.md) -\* [C++客户端库](cpp.md) +在大多数情况下,建议使用适当的工具或库,而不是直接与它们交互。Yandex官方支持的项目有: -还有许多第三方库可供使用ClickHouse: -\* [客户端库](third-party/client-libraries.md) -\* [集成](third-party/integrations.md) -\* [可视界面](third-party/gui.md) +- [命令行客户端](../interfaces/cli.md) +- [JDBC驱动](../interfaces/jdbc.md) +- [ODBC驱动](../interfaces/odbc.md) +- [C++客户端](../interfaces/cpp.md) -[来源文章](https://clickhouse.tech/docs/zh/interfaces/) +还有一些广泛的第三方库可供ClickHouse使用: + +- [客户端库](../interfaces/third-party/client-libraries.md) +- [第三方集成库](../interfaces/third-party/integrations.md) +- [可视化UI](../interfaces/third-party/gui.md) + +[来源文章](https://clickhouse.tech/docs/en/interfaces/) From 737357418f6fb01d87c3ec229518474ca5e7fa42 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Thu, 26 Nov 2020 23:29:37 +0300 Subject: [PATCH 23/50] Fix and optimize countMatches()/countMatchesCaseInsensitive() - Update after IFunction interfaces changes - move type checks into FunctionCountMatches::getReturnTypeImpl() - Use StringRef over String - Separate out logic for counting sub matches into separate helper - Do not copy other regular expression matches, only the first - Add some comments - Set is_no_capture, to avoid check for number of subpatterns - Add countMatchesCaseInsensitive() - Reguster functions in case-sensitive manner, since this is not SQL standard --- src/Functions/countMatches.cpp | 20 ++++- src/Functions/countMatches.h | 125 +++++++++++++++++++++++++++++ src/Functions/countMatchesImpl.h | 132 ------------------------------- src/Functions/ya.make | 1 + 4 files changed, 144 insertions(+), 134 deletions(-) create mode 100644 src/Functions/countMatches.h delete mode 100644 src/Functions/countMatchesImpl.h diff --git a/src/Functions/countMatches.cpp b/src/Functions/countMatches.cpp index 935b9fb9904..e1ad445befb 100644 --- a/src/Functions/countMatches.cpp +++ b/src/Functions/countMatches.cpp @@ -1,13 +1,29 @@ #include "FunctionFactory.h" -#include "countMatchesImpl.h" +#include "countMatches.h" +namespace +{ + +struct FunctionCountMatchesCaseSensitive +{ + static constexpr auto name = "countMatches"; + static constexpr bool case_insensitive = false; +}; +struct FunctionCountMatchesCaseInsensitive +{ + static constexpr auto name = "countMatchesCaseInsensitive"; + static constexpr bool case_insensitive = true; +}; + +} namespace DB { void registerFunctionCountMatches(FunctionFactory & factory) { - factory.registerFunction(FunctionFactory::CaseInsensitive); + factory.registerFunction>(FunctionFactory::CaseSensitive); + factory.registerFunction>(FunctionFactory::CaseSensitive); } } diff --git a/src/Functions/countMatches.h b/src/Functions/countMatches.h new file mode 100644 index 00000000000..be3d06ffb84 --- /dev/null +++ b/src/Functions/countMatches.h @@ -0,0 +1,125 @@ +#pragma once + +#include +#include +#include +#include +#include +#include +#include +#include + + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int ILLEGAL_TYPE_OF_ARGUMENT; + extern const int ILLEGAL_COLUMN; + extern const int LOGICAL_ERROR; +} + +using Pos = const char *; + +template +class FunctionCountMatches : public IFunction +{ +public: + static constexpr auto name = CountMatchesBase::name; + static FunctionPtr create(const Context &) { return std::make_shared>(); } + + String getName() const override { return name; } + size_t getNumberOfArguments() const override { return 2; } + + DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override + { + if (!isStringOrFixedString(arguments[1].type)) + throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, + "Illegal type {} of second argument (pattern) of function {}. Must be String/FixedString.", + arguments[1].type->getName(), getName()); + if (!isStringOrFixedString(arguments[0].type)) + throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, + "Illegal type {} of first argument (haystack) of function {}. Must be String/FixedString.", + arguments[0].type->getName(), getName()); + const auto * column = arguments[1].column.get(); + if (!column || !checkAndGetColumnConstStringOrFixedString(column)) + throw Exception(ErrorCodes::ILLEGAL_COLUMN, + "The second argument of function {} should be a constant string with the pattern", + getName()); + + return std::make_shared(); + } + + ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t input_rows_count) const override + { + const ColumnConst * column_pattern = checkAndGetColumnConstStringOrFixedString(arguments[1].column.get()); + Regexps::Pool::Pointer re = Regexps::get(column_pattern->getValue()); + OptimizedRegularExpression::MatchVec matches; + + const IColumn * column_haystack = arguments[0].column.get(); + + if (const ColumnString * col_str = checkAndGetColumn(column_haystack)) + { + auto result_column = ColumnUInt64::create(); + + const ColumnString::Chars & src_chars = col_str->getChars(); + const ColumnString::Offsets & src_offsets = col_str->getOffsets(); + + ColumnUInt64::Container & vec_res = result_column->getData(); + vec_res.resize(input_rows_count); + + size_t size = src_offsets.size(); + ColumnString::Offset current_src_offset = 0; + + for (size_t i = 0; i < size; ++i) + { + Pos pos = reinterpret_cast(&src_chars[current_src_offset]); + current_src_offset = src_offsets[i]; + Pos end = reinterpret_cast(&src_chars[current_src_offset]) - 1; + + StringRef str(pos, end - pos); + vec_res[i] = countMatches(str, re, matches); + } + + return result_column; + } + else if (const ColumnConst * col_const_str = checkAndGetColumnConstStringOrFixedString(column_haystack)) + { + StringRef str = col_const_str->getDataColumn().getDataAt(0); + uint64_t matches_count = countMatches(str, re, matches); + return result_type->createColumnConst(input_rows_count, matches_count); + } + else + throw Exception(ErrorCodes::LOGICAL_ERROR, "Error in FunctionCountMatches::getReturnTypeImpl()"); + } + + static uint64_t countMatches(StringRef src, Regexps::Pool::Pointer & re, OptimizedRegularExpression::MatchVec & matches) + { + /// Only one match is required, no need to copy more. + static const unsigned matches_limit = 1; + + Pos pos = reinterpret_cast(src.data); + Pos end = reinterpret_cast(src.data + src.size); + + uint64_t match_count = 0; + while (true) + { + if (pos >= end) + break; + if (!re->match(pos, end - pos, matches, matches_limit)) + break; + /// Progress should be made, but with empty match the progress will not be done. + /// Also note that simply check is pattern empty is not enough, + /// since for example "'[f]{0}'" will match zero bytes: + if (!matches[0].length) + break; + pos += matches[0].offset + matches[0].length; + match_count++; + } + + return match_count; + } +}; + +} diff --git a/src/Functions/countMatchesImpl.h b/src/Functions/countMatchesImpl.h deleted file mode 100644 index 86e25a252ce..00000000000 --- a/src/Functions/countMatchesImpl.h +++ /dev/null @@ -1,132 +0,0 @@ -#pragma once - -#include -#include -#include -#include -#include -#include -#include -#include -#include - - -namespace DB -{ - -namespace ErrorCodes -{ - extern const int ILLEGAL_TYPE_OF_ARGUMENT; - extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; - extern const int BAD_ARGUMENTS; - extern const int ILLEGAL_COLUMN; -} - -using Pos = const char *; - -class FunctionCountMatches : public IFunction -{ -public: - static constexpr auto name = "countMatches"; - static FunctionPtr create(const Context &) { return std::make_shared(); } - - String getName() const override - { - return name; - } - - size_t getNumberOfArguments() const override { return 2; } - - DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override - { - if (!isString(arguments[1])) - throw Exception("Illegal type " + arguments[1]->getName() + " of second argument of function " + getName() + ". Must be String.", - ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); - - return std::make_shared(); - } - - void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) const override - { - - const ColumnConst * col = checkAndGetColumnConstStringOrFixedString(block.getByPosition(arguments[1]).column.get()); - - if (!col) - throw Exception("Illegal column " + block.getByPosition(arguments[1]).column->getName() - + " of first argument of function " + getName() + ". Must be constant string.", - ErrorCodes::ILLEGAL_COLUMN); - - Regexps::Pool::Pointer re = Regexps::get(col->getValue()); - size_t capture = re->getNumberOfSubpatterns() > 0 ? 1 : 0; - OptimizedRegularExpression::MatchVec matches; - matches.resize(capture + 1); - - size_t array_argument_position = arguments[0]; - - const ColumnString * col_str = checkAndGetColumn(block.getByPosition(array_argument_position).column.get()); - const ColumnConst * col_const_str = - checkAndGetColumnConstStringOrFixedString(block.getByPosition(array_argument_position).column.get()); - - auto col_res = ColumnUInt64::create(); - ColumnUInt64::Container & vec_res = col_res->getData(); - - if (col_str) - { - const ColumnString::Chars & src_chars = col_str->getChars(); - const ColumnString::Offsets & src_offsets = col_str->getOffsets(); - - vec_res.resize(src_offsets.size()); - - size_t size = src_offsets.size(); - ColumnString::Offset current_src_offset = 0; - - for (size_t i = 0; i < size; ++i) - { - Pos pos = reinterpret_cast(&src_chars[current_src_offset]); - current_src_offset = src_offsets[i]; - Pos end = reinterpret_cast(&src_chars[current_src_offset]) - 1; - - uint64_t match_count = 0; - while (true) - { - if (!pos || pos > end) - break; - if (!re->match(pos, end - pos, matches) || !matches[0].length) - break; - pos += matches[0].offset + matches[0].length; - match_count++; - } - vec_res[i] = match_count; - } - - block.getByPosition(result).column = std::move(col_res); - } - else if (col_const_str) - { - String src = col_const_str->getValue(); - - Pos pos = reinterpret_cast(src.data()); - Pos end = reinterpret_cast(src.data() + src.size()); - - uint64_t match_count = 0; - while (true) - { - if (!pos || pos > end) - break; - if (!re->match(pos, end - pos, matches) || !matches[0].length) - break; - pos += matches[0].offset + matches[0].length; - match_count++; - } - - block.getByPosition(result).column = DataTypeUInt64().createColumnConst(col_const_str->size(), match_count); - } - else - throw Exception("Illegal columns " + block.getByPosition(array_argument_position).column->getName() - + ", " + block.getByPosition(array_argument_position).column->getName() - + " of arguments of function " + getName(), - ErrorCodes::ILLEGAL_COLUMN); - } -}; - -} diff --git a/src/Functions/ya.make b/src/Functions/ya.make index 181cee53570..436a6a89996 100644 --- a/src/Functions/ya.make +++ b/src/Functions/ya.make @@ -208,6 +208,7 @@ SRCS( cos.cpp cosh.cpp countDigits.cpp + countMatches.cpp countSubstrings.cpp countSubstringsCaseInsensitive.cpp countSubstringsCaseInsensitiveUTF8.cpp From cbd4434a331fc63786ddf5cf04382df77ceb3465 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Thu, 26 Nov 2020 23:58:07 +0300 Subject: [PATCH 24/50] Add a test for countMatches()/countMatchesCaseInsensitive() --- .../0_stateless/01595_countMatches.reference | 24 +++++++++++++++ .../0_stateless/01595_countMatches.sql | 29 +++++++++++++++++++ 2 files changed, 53 insertions(+) create mode 100644 tests/queries/0_stateless/01595_countMatches.reference create mode 100644 tests/queries/0_stateless/01595_countMatches.sql diff --git a/tests/queries/0_stateless/01595_countMatches.reference b/tests/queries/0_stateless/01595_countMatches.reference new file mode 100644 index 00000000000..c65279c0b8e --- /dev/null +++ b/tests/queries/0_stateless/01595_countMatches.reference @@ -0,0 +1,24 @@ +basic +0 +0 +0 +1 +case sensitive +2 +1 +2 +2 +2 +2 +4 +4 +case insensitive +2 +1 +2 +2 +2 +2 +4 +4 +errors diff --git a/tests/queries/0_stateless/01595_countMatches.sql b/tests/queries/0_stateless/01595_countMatches.sql new file mode 100644 index 00000000000..6374fe7bc5b --- /dev/null +++ b/tests/queries/0_stateless/01595_countMatches.sql @@ -0,0 +1,29 @@ +select 'basic'; +select countMatches('', 'foo'); +select countMatches('foo', ''); +-- simply stop if zero bytes was processed +select countMatches('foo', '[f]{0}'); +-- but this is ok +select countMatches('foo', '[f]{0}foo'); + +select 'case sensitive'; +select countMatches('foobarfoo', 'foo'); +select countMatches('foobarfoo', 'foo.*'); +select countMatches('oooo', 'oo'); +select countMatches(concat(toString(number), 'foofoo'), 'foo') from numbers(2); +select countMatches('foobarbazfoobarbaz', 'foo(bar)(?:baz|)'); +select countMatches('foo.com bar.com baz.com bam.com', '([^. ]+)\.([^. ]+)'); +select countMatches('foo.com@foo.com bar.com@foo.com baz.com@foo.com bam.com@foo.com', '([^. ]+)\.([^. ]+)@([^. ]+)\.([^. ]+)'); + +select 'case insensitive'; +select countMatchesCaseInsensitive('foobarfoo', 'FOo'); +select countMatchesCaseInsensitive('foobarfoo', 'FOo.*'); +select countMatchesCaseInsensitive('oooo', 'Oo'); +select countMatchesCaseInsensitive(concat(toString(number), 'Foofoo'), 'foo') from numbers(2); +select countMatchesCaseInsensitive('foOBarBAZfoobarbaz', 'foo(bar)(?:baz|)'); +select countMatchesCaseInsensitive('foo.com BAR.COM baz.com bam.com', '([^. ]+)\.([^. ]+)'); +select countMatchesCaseInsensitive('foo.com@foo.com bar.com@foo.com BAZ.com@foo.com bam.com@foo.com', '([^. ]+)\.([^. ]+)@([^. ]+)\.([^. ]+)'); + +select 'errors'; +select countMatches(1, 'foo') from numbers(1); -- { serverError 43; } +select countMatches('foobarfoo', toString(number)) from numbers(1); -- { serverError 44; } From cb68d5b5e72ba6615484c93eff38d1d0f70aa1c6 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Tue, 1 Dec 2020 22:26:07 +0300 Subject: [PATCH 25/50] Add performance test for countMatches() function --- tests/performance/countMatches.xml | 22 ++++++++++++++++++++++ 1 file changed, 22 insertions(+) create mode 100644 tests/performance/countMatches.xml diff --git a/tests/performance/countMatches.xml b/tests/performance/countMatches.xml new file mode 100644 index 00000000000..be6f76f4066 --- /dev/null +++ b/tests/performance/countMatches.xml @@ -0,0 +1,22 @@ + + + test.hits + + + SELECT count() FROM test.hits WHERE NOT ignore(countMatches(URL, 'yandex')) + SELECT count() FROM test.hits WHERE NOT ignore(countMatches(URL, 'yandex|google')) + SELECT count() FROM test.hits WHERE NOT ignore(countMatches(URL, '(\\w+=\\w+)')) + + SELECT count() FROM test.hits WHERE NOT ignore(countMatches(URL, 'yandex')) SETTINGS max_threads=1 + SELECT count() FROM test.hits WHERE NOT ignore(countMatches(URL, 'yandex|google')) SETTINGS max_threads=1 + SELECT count() FROM test.hits WHERE NOT ignore(countMatches(URL, '(\\w+=\\w+)')) SETTINGS max_threads=1 + + SELECT count() FROM test.hits WHERE NOT ignore(countMatchesCaseInsensitive(URL, 'yandex')) SETTINGS max_threads=1 + SELECT count() FROM test.hits WHERE NOT ignore(countMatchesCaseInsensitive(URL, 'yandex|google')) SETTINGS max_threads=1 + SELECT count() FROM test.hits WHERE NOT ignore(countMatchesCaseInsensitive(URL, '(\\w+=\\w+)')) SETTINGS max_threads=1 + + + SELECT count() FROM test.hits WHERE NOT ignore(countMatches(URL, '(?i)yandex')) SETTINGS max_threads=1 + SELECT count() FROM test.hits WHERE NOT ignore(countMatches(URL, '(?i)yandex|google')) SETTINGS max_threads=1 + SELECT count() FROM test.hits WHERE NOT ignore(countMatches(URL, '(?i)(\\w+=\\w+)')) SETTINGS max_threads=1 + From 88bc73df3a064278f77c222f2e23a2e8e9cf4803 Mon Sep 17 00:00:00 2001 From: tavplubix Date: Tue, 1 Dec 2020 20:01:47 +0300 Subject: [PATCH 26/50] fix seq number in TestKeeper --- src/Common/ZooKeeper/TestKeeper.cpp | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/src/Common/ZooKeeper/TestKeeper.cpp b/src/Common/ZooKeeper/TestKeeper.cpp index 5f34a60c34e..61da1137526 100644 --- a/src/Common/ZooKeeper/TestKeeper.cpp +++ b/src/Common/ZooKeeper/TestKeeper.cpp @@ -216,7 +216,6 @@ std::pair TestKeeperCreateRequest::process(TestKeeper::Contai if (is_sequential) { auto seq_num = it->second.seq_num; - ++it->second.seq_num; std::stringstream seq_num_str; // STYLE_CHECK_ALLOW_STD_STRING_STREAM seq_num_str.exceptions(std::ios::failbit); @@ -225,18 +224,19 @@ std::pair TestKeeperCreateRequest::process(TestKeeper::Contai path_created += seq_num_str.str(); } + /// Increment sequential number even if node is not sequential + ++it->second.seq_num; + response.path_created = path_created; container.emplace(path_created, std::move(created_node)); - undo = [&container, path_created, is_sequential = is_sequential, parent_path = it->first] + undo = [&container, path_created, parent_path = it->first] { container.erase(path_created); auto & undo_parent = container.at(parent_path); --undo_parent.stat.cversion; --undo_parent.stat.numChildren; - - if (is_sequential) - --undo_parent.seq_num; + --undo_parent.seq_num; }; ++it->second.stat.cversion; From bfbf150c67df7ecf967d26a3c8f2c702a25a5cb9 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Wed, 2 Dec 2020 17:49:43 +0300 Subject: [PATCH 27/50] fix segfault when 'not enough space' --- src/Disks/StoragePolicy.cpp | 18 +++++++++++++++++- src/Disks/StoragePolicy.h | 7 +++++-- .../DistributedBlockOutputStream.cpp | 2 +- src/Storages/MergeTree/MergeTreeData.cpp | 3 +-- 4 files changed, 24 insertions(+), 6 deletions(-) diff --git a/src/Disks/StoragePolicy.cpp b/src/Disks/StoragePolicy.cpp index 2215615feda..e3a937cae55 100644 --- a/src/Disks/StoragePolicy.cpp +++ b/src/Disks/StoragePolicy.cpp @@ -30,6 +30,7 @@ namespace ErrorCodes extern const int UNKNOWN_POLICY; extern const int UNKNOWN_VOLUME; extern const int LOGICAL_ERROR; + extern const int NOT_ENOUGH_SPACE; } @@ -210,6 +211,14 @@ ReservationPtr StoragePolicy::reserve(UInt64 bytes) const } +ReservationPtr StoragePolicy::reserveAndCheck(UInt64 bytes) const +{ + if (auto res = reserve(bytes, 0)) + return res; + throw Exception(ErrorCodes::NOT_ENOUGH_SPACE, "Cannot reserve {}, not enough space", ReadableSize(bytes)); +} + + ReservationPtr StoragePolicy::makeEmptyReservationOnLargestDisk() const { UInt64 max_space = 0; @@ -226,7 +235,14 @@ ReservationPtr StoragePolicy::makeEmptyReservationOnLargestDisk() const } } } - return max_disk->reserve(0); + auto reservation = max_disk->reserve(0); + if (!reservation) + { + /// I'm not sure if it's really a logical error, but exception message + /// "Cannot reserve 0 bytes" looks too strange to throw it with another exception code. + throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot reserve 0 bytes"); + } + return reservation; } diff --git a/src/Disks/StoragePolicy.h b/src/Disks/StoragePolicy.h index fc45ed3ed06..9135c27d1c0 100644 --- a/src/Disks/StoragePolicy.h +++ b/src/Disks/StoragePolicy.h @@ -61,10 +61,13 @@ public: const String & getName() const { return name; } - /// Returns valid reservation or null + /// Returns valid reservation or nullptr ReservationPtr reserve(UInt64 bytes) const; - /// Reserve space on any volume with index > min_volume_index + /// Reserves space on any volume or throws + ReservationPtr reserveAndCheck(UInt64 bytes) const; + + /// Reserves space on any volume with index > min_volume_index or returns nullptr ReservationPtr reserve(UInt64 bytes, size_t min_volume_index) const; /// Find volume index, which contains disk diff --git a/src/Storages/Distributed/DistributedBlockOutputStream.cpp b/src/Storages/Distributed/DistributedBlockOutputStream.cpp index 093e5cea8ce..2248c489679 100644 --- a/src/Storages/Distributed/DistributedBlockOutputStream.cpp +++ b/src/Storages/Distributed/DistributedBlockOutputStream.cpp @@ -567,7 +567,7 @@ void DistributedBlockOutputStream::writeToShard(const Block & block, const std:: /// and keep monitor thread out from reading incomplete data std::string first_file_tmp_path{}; - auto reservation = storage.getStoragePolicy()->reserve(block.bytes()); + auto reservation = storage.getStoragePolicy()->reserveAndCheck(block.bytes()); auto disk = reservation->getDisk()->getPath(); auto data_path = storage.getRelativeDataPath(); diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index b213bb7b6f9..1f18c364541 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -3140,8 +3140,7 @@ inline ReservationPtr checkAndReturnReservation(UInt64 expected_size, Reservatio ReservationPtr MergeTreeData::reserveSpace(UInt64 expected_size) const { expected_size = std::max(RESERVATION_MIN_ESTIMATION_SIZE, expected_size); - auto reservation = getStoragePolicy()->reserve(expected_size); - return checkAndReturnReservation(expected_size, std::move(reservation)); + return getStoragePolicy()->reserveAndCheck(expected_size); } ReservationPtr MergeTreeData::reserveSpace(UInt64 expected_size, SpacePtr space) From cab9855dd1d5436ff8f7ee09a802fed90b12187a Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Thu, 3 Dec 2020 16:54:05 +0300 Subject: [PATCH 28/50] Update StorageReplicatedMergeTree.cpp --- src/Storages/StorageReplicatedMergeTree.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index f6d7bbc2b63..01eed30a2c5 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -2645,7 +2645,7 @@ bool StorageReplicatedMergeTree::processQueueEntry(ReplicatedMergeTreeQueue::Sel * See `queue.processEntry` function. */ throw; - } + } catch (...) { tryLogCurrentException(log, __PRETTY_FUNCTION__); From 1e943babcc1a40336e02043677d5e09abc610c47 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 3 Dec 2020 23:20:09 +0300 Subject: [PATCH 29/50] Do not add empty query to browser history --- programs/server/play.html | 21 ++++++++++++--------- 1 file changed, 12 insertions(+), 9 deletions(-) diff --git a/programs/server/play.html b/programs/server/play.html index 37869228c04..12435f55793 100644 --- a/programs/server/play.html +++ b/programs/server/play.html @@ -352,17 +352,20 @@ /// The query is saved in browser history (in state JSON object) /// as well as in URL fragment identifier. if (query != previous_query) { - previous_query = query; + var state = { + query: query, + status: this.status, + response: this.response.length > 100000 ? null : this.response /// Lower than the browser's limit. + }; var title = "ClickHouse Query: " + query; - history.pushState( - { - query: query, - status: this.status, - response: this.response.length > 100000 ? null : this.response /// Lower than the browser's limit. - }, - title, - window.location.pathname + '?user=' + encodeURIComponent(user) + '#' + window.btoa(query)); + var url = window.location.pathname + '?user=' + encodeURIComponent(user) + '#' + window.btoa(query); + if (previous_query == '') { + history.replaceState(state, title, url); + } else { + history.pushState(state, title, url); + } document.title = title; + previous_query = query; } } else { //console.log(this); From 68c4da1203e1e917d1a00897cd960295ab9a3703 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Fri, 4 Dec 2020 07:54:10 +0300 Subject: [PATCH 30/50] Use max_threads=2 for countMatches to keep it under 2 seconds Although I don't like this idea. --- tests/performance/countMatches.xml | 18 +++++++++--------- 1 file changed, 9 insertions(+), 9 deletions(-) diff --git a/tests/performance/countMatches.xml b/tests/performance/countMatches.xml index be6f76f4066..72bd2b132c6 100644 --- a/tests/performance/countMatches.xml +++ b/tests/performance/countMatches.xml @@ -7,16 +7,16 @@ SELECT count() FROM test.hits WHERE NOT ignore(countMatches(URL, 'yandex|google')) SELECT count() FROM test.hits WHERE NOT ignore(countMatches(URL, '(\\w+=\\w+)')) - SELECT count() FROM test.hits WHERE NOT ignore(countMatches(URL, 'yandex')) SETTINGS max_threads=1 - SELECT count() FROM test.hits WHERE NOT ignore(countMatches(URL, 'yandex|google')) SETTINGS max_threads=1 - SELECT count() FROM test.hits WHERE NOT ignore(countMatches(URL, '(\\w+=\\w+)')) SETTINGS max_threads=1 + SELECT count() FROM test.hits WHERE NOT ignore(countMatches(URL, 'yandex')) SETTINGS max_threads=2 + SELECT count() FROM test.hits WHERE NOT ignore(countMatches(URL, 'yandex|google')) SETTINGS max_threads=2 + SELECT count() FROM test.hits WHERE NOT ignore(countMatches(URL, '(\\w+=\\w+)')) SETTINGS max_threads=2 - SELECT count() FROM test.hits WHERE NOT ignore(countMatchesCaseInsensitive(URL, 'yandex')) SETTINGS max_threads=1 - SELECT count() FROM test.hits WHERE NOT ignore(countMatchesCaseInsensitive(URL, 'yandex|google')) SETTINGS max_threads=1 - SELECT count() FROM test.hits WHERE NOT ignore(countMatchesCaseInsensitive(URL, '(\\w+=\\w+)')) SETTINGS max_threads=1 + SELECT count() FROM test.hits WHERE NOT ignore(countMatchesCaseInsensitive(URL, 'yandex')) SETTINGS max_threads=2 + SELECT count() FROM test.hits WHERE NOT ignore(countMatchesCaseInsensitive(URL, 'yandex|google')) SETTINGS max_threads=2 + SELECT count() FROM test.hits WHERE NOT ignore(countMatchesCaseInsensitive(URL, '(\\w+=\\w+)')) SETTINGS max_threads=2 - SELECT count() FROM test.hits WHERE NOT ignore(countMatches(URL, '(?i)yandex')) SETTINGS max_threads=1 - SELECT count() FROM test.hits WHERE NOT ignore(countMatches(URL, '(?i)yandex|google')) SETTINGS max_threads=1 - SELECT count() FROM test.hits WHERE NOT ignore(countMatches(URL, '(?i)(\\w+=\\w+)')) SETTINGS max_threads=1 + SELECT count() FROM test.hits WHERE NOT ignore(countMatches(URL, '(?i)yandex')) SETTINGS max_threads=2 + SELECT count() FROM test.hits WHERE NOT ignore(countMatches(URL, '(?i)yandex|google')) SETTINGS max_threads=2 + SELECT count() FROM test.hits WHERE NOT ignore(countMatches(URL, '(?i)(\\w+=\\w+)')) SETTINGS max_threads=2 From a5e3d89649247384acdd795295febf6a310e402e Mon Sep 17 00:00:00 2001 From: Vladimir Chebotarev Date: Fri, 4 Dec 2020 13:37:08 +0300 Subject: [PATCH 31/50] Attempt to use IOStream in AWS SDK. --- src/IO/S3/PocoHTTPResponseStream.cpp | 2 +- src/IO/S3/PocoHTTPResponseStream.h | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/src/IO/S3/PocoHTTPResponseStream.cpp b/src/IO/S3/PocoHTTPResponseStream.cpp index b35188d9498..93f85d65e30 100644 --- a/src/IO/S3/PocoHTTPResponseStream.cpp +++ b/src/IO/S3/PocoHTTPResponseStream.cpp @@ -10,7 +10,7 @@ namespace DB::S3 { PocoHTTPResponseStream::PocoHTTPResponseStream(std::shared_ptr session_, std::istream & response_stream_) - : Aws::IStream(response_stream_.rdbuf()), session(std::move(session_)) + : Aws::IOStream(response_stream_.rdbuf()), session(std::move(session_)) { } diff --git a/src/IO/S3/PocoHTTPResponseStream.h b/src/IO/S3/PocoHTTPResponseStream.h index 8167ddc4346..fe3df6e52a7 100644 --- a/src/IO/S3/PocoHTTPResponseStream.h +++ b/src/IO/S3/PocoHTTPResponseStream.h @@ -8,7 +8,7 @@ namespace DB::S3 /** * Wrapper of IStream to store response stream and corresponding HTTP session. */ -class PocoHTTPResponseStream : public Aws::IStream +class PocoHTTPResponseStream : public Aws::IOStream { public: PocoHTTPResponseStream(std::shared_ptr session_, std::istream & response_stream_); From 796aee032d9aa36942bfc8bd33271943d25991c2 Mon Sep 17 00:00:00 2001 From: Nicolae Vartolomei Date: Fri, 4 Dec 2020 13:01:12 +0000 Subject: [PATCH 32/50] Update StorageReplicatedMergeTree::waitForAllReplicasToProcessLogEntry to support waiting on foreign shards / tables This is not used anywhere yet but needed for an upcoming PR for part movement between shards. --- src/Storages/StorageReplicatedMergeTree.cpp | 46 ++++++++++++++------- src/Storages/StorageReplicatedMergeTree.h | 4 ++ 2 files changed, 34 insertions(+), 16 deletions(-) diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 492c9b3e957..a9aa65d8f29 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -4566,19 +4566,19 @@ StorageReplicatedMergeTree::allocateBlockNumber( } -Strings StorageReplicatedMergeTree::waitForAllReplicasToProcessLogEntry( - const ReplicatedMergeTreeLogEntryData & entry, bool wait_for_non_active) +Strings StorageReplicatedMergeTree::waitForAllTableReplicasToProcessLogEntry( + const String & table_zookeeper_path, const ReplicatedMergeTreeLogEntryData & entry, bool wait_for_non_active) { LOG_DEBUG(log, "Waiting for all replicas to process {}", entry.znode_name); auto zookeeper = getZooKeeper(); - Strings replicas = zookeeper->getChildren(zookeeper_path + "/replicas"); + Strings replicas = zookeeper->getChildren(table_zookeeper_path + "/replicas"); Strings unwaited; for (const String & replica : replicas) { - if (wait_for_non_active || zookeeper->exists(zookeeper_path + "/replicas/" + replica + "/is_active")) + if (wait_for_non_active || zookeeper->exists(table_zookeeper_path + "/replicas/" + replica + "/is_active")) { - if (!waitForReplicaToProcessLogEntry(replica, entry, wait_for_non_active)) + if (!waitForTableReplicaToProcessLogEntry(table_zookeeper_path, replica, entry, wait_for_non_active)) unwaited.push_back(replica); } else @@ -4592,8 +4592,15 @@ Strings StorageReplicatedMergeTree::waitForAllReplicasToProcessLogEntry( } -bool StorageReplicatedMergeTree::waitForReplicaToProcessLogEntry( - const String & replica, const ReplicatedMergeTreeLogEntryData & entry, bool wait_for_non_active) +Strings StorageReplicatedMergeTree::waitForAllReplicasToProcessLogEntry( + const ReplicatedMergeTreeLogEntryData & entry, bool wait_for_non_active) +{ + return waitForAllTableReplicasToProcessLogEntry(zookeeper_path, entry, wait_for_non_active); +} + + +bool StorageReplicatedMergeTree::waitForTableReplicaToProcessLogEntry( + const String & table_zookeeper_path, const String & replica, const ReplicatedMergeTreeLogEntryData & entry, bool wait_for_non_active) { String entry_str = entry.toString(); String log_node_name; @@ -4619,7 +4626,7 @@ bool StorageReplicatedMergeTree::waitForReplicaToProcessLogEntry( const auto & stop_waiting = [&]() { bool stop_waiting_itself = waiting_itself && is_dropped; - bool stop_waiting_non_active = !wait_for_non_active && !getZooKeeper()->exists(zookeeper_path + "/replicas/" + replica + "/is_active"); + bool stop_waiting_non_active = !wait_for_non_active && !getZooKeeper()->exists(table_zookeeper_path + "/replicas/" + replica + "/is_active"); return stop_waiting_itself || stop_waiting_non_active; }; constexpr auto event_wait_timeout_ms = 1000; @@ -4639,7 +4646,7 @@ bool StorageReplicatedMergeTree::waitForReplicaToProcessLogEntry( { zkutil::EventPtr event = std::make_shared(); - String log_pointer = getZooKeeper()->get(zookeeper_path + "/replicas/" + replica + "/log_pointer", nullptr, event); + String log_pointer = getZooKeeper()->get(table_zookeeper_path + "/replicas/" + replica + "/log_pointer", nullptr, event); if (!log_pointer.empty() && parse(log_pointer) > log_index) break; @@ -4655,9 +4662,9 @@ bool StorageReplicatedMergeTree::waitForReplicaToProcessLogEntry( * looking for a node with the same content. And if we do not find it - then the replica has already taken this entry in its queue. */ - String log_pointer = getZooKeeper()->get(zookeeper_path + "/replicas/" + replica + "/log_pointer"); + String log_pointer = getZooKeeper()->get(table_zookeeper_path + "/replicas/" + replica + "/log_pointer"); - Strings log_entries = getZooKeeper()->getChildren(zookeeper_path + "/log"); + Strings log_entries = getZooKeeper()->getChildren(table_zookeeper_path + "/log"); UInt64 log_index = 0; bool found = false; @@ -4669,7 +4676,7 @@ bool StorageReplicatedMergeTree::waitForReplicaToProcessLogEntry( continue; String log_entry_str; - bool exists = getZooKeeper()->tryGet(zookeeper_path + "/log/" + log_entry_name, log_entry_str); + bool exists = getZooKeeper()->tryGet(table_zookeeper_path + "/log/" + log_entry_name, log_entry_str); if (exists && entry_str == log_entry_str) { found = true; @@ -4687,7 +4694,7 @@ bool StorageReplicatedMergeTree::waitForReplicaToProcessLogEntry( { zkutil::EventPtr event = std::make_shared(); - String log_pointer_new = getZooKeeper()->get(zookeeper_path + "/replicas/" + replica + "/log_pointer", nullptr, event); + String log_pointer_new = getZooKeeper()->get(table_zookeeper_path + "/replicas/" + replica + "/log_pointer", nullptr, event); if (!log_pointer_new.empty() && parse(log_pointer_new) > log_index) break; @@ -4711,13 +4718,13 @@ bool StorageReplicatedMergeTree::waitForReplicaToProcessLogEntry( * Therefore, we search by comparing the content. */ - Strings queue_entries = getZooKeeper()->getChildren(zookeeper_path + "/replicas/" + replica + "/queue"); + Strings queue_entries = getZooKeeper()->getChildren(table_zookeeper_path + "/replicas/" + replica + "/queue"); String queue_entry_to_wait_for; for (const String & entry_name : queue_entries) { String queue_entry_str; - bool exists = getZooKeeper()->tryGet(zookeeper_path + "/replicas/" + replica + "/queue/" + entry_name, queue_entry_str); + bool exists = getZooKeeper()->tryGet(table_zookeeper_path + "/replicas/" + replica + "/queue/" + entry_name, queue_entry_str); if (exists && queue_entry_str == entry_str) { queue_entry_to_wait_for = entry_name; @@ -4735,12 +4742,19 @@ bool StorageReplicatedMergeTree::waitForReplicaToProcessLogEntry( LOG_DEBUG(log, "Waiting for {} to disappear from {} queue", queue_entry_to_wait_for, replica); /// Third - wait until the entry disappears from the replica queue or replica become inactive. - String path_to_wait_on = zookeeper_path + "/replicas/" + replica + "/queue/" + queue_entry_to_wait_for; + String path_to_wait_on = table_zookeeper_path + "/replicas/" + replica + "/queue/" + queue_entry_to_wait_for; return getZooKeeper()->waitForDisappear(path_to_wait_on, stop_waiting); } +bool StorageReplicatedMergeTree::waitForReplicaToProcessLogEntry( + const String & replica, const ReplicatedMergeTreeLogEntryData & entry, bool wait_for_non_active) +{ + return waitForTableReplicaToProcessLogEntry(zookeeper_path, replica, entry, wait_for_non_active); +} + + void StorageReplicatedMergeTree::getStatus(Status & res, bool with_zk_fields) { auto zookeeper = tryGetZooKeeper(); diff --git a/src/Storages/StorageReplicatedMergeTree.h b/src/Storages/StorageReplicatedMergeTree.h index 7176c49dd91..f1094793944 100644 --- a/src/Storages/StorageReplicatedMergeTree.h +++ b/src/Storages/StorageReplicatedMergeTree.h @@ -537,12 +537,16 @@ private: * NOTE: This method must be called without table lock held. * Because it effectively waits for other thread that usually has to also acquire a lock to proceed and this yields deadlock. * TODO: There are wrong usages of this method that are not fixed yet. + * + * One method for convenient use on current table, another for waiting on foregin shards. */ + Strings waitForAllTableReplicasToProcessLogEntry(const String & table_zookeeper_path, const ReplicatedMergeTreeLogEntryData & entry, bool wait_for_non_active = true); Strings waitForAllReplicasToProcessLogEntry(const ReplicatedMergeTreeLogEntryData & entry, bool wait_for_non_active = true); /** Wait until the specified replica executes the specified action from the log. * NOTE: See comment about locks above. */ + bool waitForTableReplicaToProcessLogEntry(const String & table_zookeeper_path, const String & replica_name, const ReplicatedMergeTreeLogEntryData & entry, bool wait_for_non_active = true); bool waitForReplicaToProcessLogEntry(const String & replica_name, const ReplicatedMergeTreeLogEntryData & entry, bool wait_for_non_active = true); /// Throw an exception if the table is readonly. From 9dbced0474e24f00da22091428a920f53fb4b249 Mon Sep 17 00:00:00 2001 From: Pavel Kruglov Date: Fri, 4 Dec 2020 17:01:59 +0300 Subject: [PATCH 33/50] Pass setting instead of context --- .../MergeTree/MergeTreeDataMergerMutator.cpp | 4 ++-- src/Storages/MergeTree/MergeTreeDataMergerMutator.h | 2 +- src/Storages/StorageMergeTree.cpp | 12 ++++++------ src/Storages/StorageMergeTree.h | 4 ++-- src/Storages/StorageReplicatedMergeTree.cpp | 4 ++-- 5 files changed, 13 insertions(+), 13 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp index 5457fdde5a5..1065b992396 100644 --- a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp +++ b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp @@ -366,7 +366,7 @@ SelectPartsDecision MergeTreeDataMergerMutator::selectAllPartsToMergeWithinParti bool final, const StorageMetadataPtr & metadata_snapshot, String * out_disable_reason, - const Context * context) + bool optimize_skip_merged_partitions) { MergeTreeData::DataPartsVector parts = selectAllPartsFromPartition(partition_id); @@ -382,7 +382,7 @@ SelectPartsDecision MergeTreeDataMergerMutator::selectAllPartsToMergeWithinParti /// If final, optimize_skip_merged_partitions is true and we have only one part in partition with level > 0 /// than we don't select it to merge. But if there are some expired TTL then merge is needed - if (final && context && context->getSettingsRef().optimize_skip_merged_partitions && parts.size() == 1 && parts[0]->info.level > 0 && + if (final && optimize_skip_merged_partitions && parts.size() == 1 && parts[0]->info.level > 0 && (!metadata_snapshot->hasAnyTTL() || parts[0]->checkAllTTLCalculated(metadata_snapshot))) { return SelectPartsDecision::NOTHING_TO_MERGE; diff --git a/src/Storages/MergeTree/MergeTreeDataMergerMutator.h b/src/Storages/MergeTree/MergeTreeDataMergerMutator.h index 56fdd474a38..bb74d8b091f 100644 --- a/src/Storages/MergeTree/MergeTreeDataMergerMutator.h +++ b/src/Storages/MergeTree/MergeTreeDataMergerMutator.h @@ -107,7 +107,7 @@ public: bool final, const StorageMetadataPtr & metadata_snapshot, String * out_disable_reason = nullptr, - const Context * context = nullptr); + bool optimize_skip_merged_partitions = false); /** Merge the parts. * If `reservation != nullptr`, now and then reduces the size of the reserved space diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index 72bd0db4606..eb28ccfa3d5 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -635,7 +635,7 @@ void StorageMergeTree::loadMutations() } std::shared_ptr StorageMergeTree::selectPartsToMerge( - const StorageMetadataPtr & metadata_snapshot, bool aggressive, const String & partition_id, bool final, String * out_disable_reason, TableLockHolder & /* table_lock_holder */, const Context * context, SelectPartsDecision * select_decision_out) + const StorageMetadataPtr & metadata_snapshot, bool aggressive, const String & partition_id, bool final, String * out_disable_reason, TableLockHolder & /* table_lock_holder */, bool optimize_skip_merged_partitions, SelectPartsDecision * select_decision_out) { std::unique_lock lock(currently_processing_in_background_mutex); auto data_settings = getSettings(); @@ -688,7 +688,7 @@ std::shared_ptr StorageMergeTree::se { UInt64 disk_space = getStoragePolicy()->getMaxUnreservedFreeSpace(); select_decision = merger_mutator.selectAllPartsToMergeWithinPartition( - future_part, disk_space, can_merge, partition_id, final, metadata_snapshot, out_disable_reason, context); + future_part, disk_space, can_merge, partition_id, final, metadata_snapshot, out_disable_reason, optimize_skip_merged_partitions); /// If final - we will wait for currently processing merges to finish and continue. /// TODO Respect query settings for timeout @@ -742,14 +742,14 @@ bool StorageMergeTree::merge( bool final, bool deduplicate, String * out_disable_reason, - const Context * context) + bool optimize_skip_merged_partitions) { auto table_lock_holder = lockForShare(RWLockImpl::NO_QUERY, getSettings()->lock_acquire_timeout_for_background_operations); auto metadata_snapshot = getInMemoryMetadataPtr(); SelectPartsDecision select_decision; - auto merge_mutate_entry = selectPartsToMerge(metadata_snapshot, aggressive, partition_id, final, out_disable_reason, table_lock_holder, context, &select_decision); + auto merge_mutate_entry = selectPartsToMerge(metadata_snapshot, aggressive, partition_id, final, out_disable_reason, table_lock_holder, optimize_skip_merged_partitions, &select_decision); /// If there is nothing to merge then we treat this merge as successful (needed for optimize final optimization) if (select_decision == SelectPartsDecision::NOTHING_TO_MERGE) @@ -1049,7 +1049,7 @@ bool StorageMergeTree::optimize( for (const String & partition_id : partition_ids) { - if (!merge(true, partition_id, true, deduplicate, &disable_reason, &context)) + if (!merge(true, partition_id, true, deduplicate, &disable_reason, context.getSettingsRef().optimize_skip_merged_partitions)) { constexpr const char * message = "Cannot OPTIMIZE table: {}"; if (disable_reason.empty()) @@ -1068,7 +1068,7 @@ bool StorageMergeTree::optimize( if (partition) partition_id = getPartitionIDFromQuery(partition, context); - if (!merge(true, partition_id, final, deduplicate, &disable_reason, &context)) + if (!merge(true, partition_id, final, deduplicate, &disable_reason, context.getSettingsRef().optimize_skip_merged_partitions)) { constexpr const char * message = "Cannot OPTIMIZE table: {}"; if (disable_reason.empty()) diff --git a/src/Storages/StorageMergeTree.h b/src/Storages/StorageMergeTree.h index bb03dbaceb9..5e0e8478ae6 100644 --- a/src/Storages/StorageMergeTree.h +++ b/src/Storages/StorageMergeTree.h @@ -132,7 +132,7 @@ private: * If aggressive - when selects parts don't takes into account their ratio size and novelty (used for OPTIMIZE query). * Returns true if merge is finished successfully. */ - bool merge(bool aggressive, const String & partition_id, bool final, bool deduplicate, String * out_disable_reason = nullptr, const Context * context = nullptr); + bool merge(bool aggressive, const String & partition_id, bool final, bool deduplicate, String * out_disable_reason = nullptr, bool optimize_skip_merged_partitions = false); ActionLock stopMergesAndWait(); @@ -181,7 +181,7 @@ private: bool final, String * disable_reason, TableLockHolder & table_lock_holder, - const Context * context = nullptr, + bool optimize_skip_merged_partitions = false, SelectPartsDecision * select_decision_out = nullptr); bool mergeSelectedParts(const StorageMetadataPtr & metadata_snapshot, bool deduplicate, MergeMutateSelectedEntry & entry, TableLockHolder & table_lock_holder); diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 01eed30a2c5..6747d992b9a 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -3919,7 +3919,7 @@ bool StorageReplicatedMergeTree::optimize( future_merged_part.uuid = UUIDHelpers::generateV4(); SelectPartsDecision select_decision = merger_mutator.selectAllPartsToMergeWithinPartition( - future_merged_part, disk_space, can_merge, partition_id, true, metadata_snapshot, nullptr, &query_context); + future_merged_part, disk_space, can_merge, partition_id, true, metadata_snapshot, nullptr, query_context.getSettingsRef().optimize_skip_merged_partitions); if (select_decision != SelectPartsDecision::SELECTED) break; @@ -3969,7 +3969,7 @@ bool StorageReplicatedMergeTree::optimize( UInt64 disk_space = getStoragePolicy()->getMaxUnreservedFreeSpace(); String partition_id = getPartitionIDFromQuery(partition, query_context); select_decision = merger_mutator.selectAllPartsToMergeWithinPartition( - future_merged_part, disk_space, can_merge, partition_id, final, metadata_snapshot, &disable_reason, &query_context); + future_merged_part, disk_space, can_merge, partition_id, final, metadata_snapshot, &disable_reason, query_context.getSettingsRef().optimize_skip_merged_partitions); } /// If there is nothing to merge then we treat this merge as successful (needed for optimize final optimization) From e7a151fd1ef68c7df606b6f00dcacba02999d5b6 Mon Sep 17 00:00:00 2001 From: detailyang Date: Fri, 4 Dec 2020 23:14:27 +0800 Subject: [PATCH 34/50] hotfix:check in_memory_parts_enable_wal --- src/Storages/MergeTree/MergeTreeData.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index b213bb7b6f9..459b6b2a91c 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -734,7 +734,7 @@ void MergeTreeData::loadDataParts(bool skip_sanity_checks) for (auto && part : write_ahead_log->restore(metadata_snapshot)) parts_from_wal.push_back(std::move(part)); } - else if (startsWith(it->name(), MergeTreeWriteAheadLog::WAL_FILE_NAME)) + else if (startsWith(it->name(), MergeTreeWriteAheadLog::WAL_FILE_NAME) && settings->in_memory_parts_enable_wal) { MergeTreeWriteAheadLog wal(*this, disk_ptr, it->name()); for (auto && part : wal.restore(metadata_snapshot)) From e2c6430a6ee0002dab427725db0fc368bc4eeec3 Mon Sep 17 00:00:00 2001 From: MyroTk Date: Fri, 4 Dec 2020 18:36:32 +0100 Subject: [PATCH 35/50] Testflows RBAC ACCESS MANAGEMENT tests. --- tests/testflows/rbac/regression.py | 3 + .../rbac/requirements/requirements.md | 255 ++++++- .../rbac/requirements/requirements.py | 717 +++++++++++++++++- .../tests/privileges/alter/alter_quota.py | 137 ++++ .../rbac/tests/privileges/alter/alter_role.py | 118 +++ .../privileges/alter/alter_row_policy.py | 144 ++++ .../alter/alter_settings_profile.py | 138 ++++ .../rbac/tests/privileges/alter/alter_user.py | 118 +++ .../tests/privileges/create/create_quota.py | 128 ++++ .../tests/privileges/create/create_role.py | 128 ++++ .../privileges/create/create_row_policy.py | 133 ++++ .../create/create_settings_profile.py | 129 ++++ .../tests/privileges/create/create_user.py | 276 +++++++ .../rbac/tests/privileges/drop/drop_quota.py | 139 ++++ .../rbac/tests/privileges/drop/drop_role.py | 118 +++ .../tests/privileges/drop/drop_row_policy.py | 144 ++++ .../privileges/drop/drop_settings_profile.py | 140 ++++ .../rbac/tests/privileges/drop/drop_user.py | 118 +++ .../rbac/tests/privileges/feature.py | 21 + .../rbac/tests/privileges/public_tables.py | 22 +- .../rbac/tests/privileges/role_admin.py | 119 +++ .../rbac/tests/privileges/show/show_quotas.py | 171 +++++ .../rbac/tests/privileges/show/show_roles.py | 159 ++++ .../privileges/show/show_row_policies.py | 176 +++++ .../privileges/show/show_settings_profiles.py | 173 +++++ .../rbac/tests/privileges/show/show_users.py | 159 ++++ 26 files changed, 4058 insertions(+), 25 deletions(-) create mode 100644 tests/testflows/rbac/tests/privileges/alter/alter_quota.py create mode 100644 tests/testflows/rbac/tests/privileges/alter/alter_role.py create mode 100644 tests/testflows/rbac/tests/privileges/alter/alter_row_policy.py create mode 100644 tests/testflows/rbac/tests/privileges/alter/alter_settings_profile.py create mode 100644 tests/testflows/rbac/tests/privileges/alter/alter_user.py create mode 100644 tests/testflows/rbac/tests/privileges/create/create_quota.py create mode 100644 tests/testflows/rbac/tests/privileges/create/create_role.py create mode 100644 tests/testflows/rbac/tests/privileges/create/create_row_policy.py create mode 100644 tests/testflows/rbac/tests/privileges/create/create_settings_profile.py create mode 100644 tests/testflows/rbac/tests/privileges/create/create_user.py create mode 100644 tests/testflows/rbac/tests/privileges/drop/drop_quota.py create mode 100644 tests/testflows/rbac/tests/privileges/drop/drop_role.py create mode 100644 tests/testflows/rbac/tests/privileges/drop/drop_row_policy.py create mode 100644 tests/testflows/rbac/tests/privileges/drop/drop_settings_profile.py create mode 100644 tests/testflows/rbac/tests/privileges/drop/drop_user.py create mode 100644 tests/testflows/rbac/tests/privileges/role_admin.py create mode 100644 tests/testflows/rbac/tests/privileges/show/show_quotas.py create mode 100644 tests/testflows/rbac/tests/privileges/show/show_roles.py create mode 100644 tests/testflows/rbac/tests/privileges/show/show_row_policies.py create mode 100644 tests/testflows/rbac/tests/privileges/show/show_settings_profiles.py create mode 100644 tests/testflows/rbac/tests/privileges/show/show_users.py diff --git a/tests/testflows/rbac/regression.py b/tests/testflows/rbac/regression.py index 10907165681..24878332c8a 100755 --- a/tests/testflows/rbac/regression.py +++ b/tests/testflows/rbac/regression.py @@ -24,6 +24,7 @@ issue_17146 = "https://github.com/ClickHouse/ClickHouse/issues/17146" issue_17147 = "https://github.com/ClickHouse/ClickHouse/issues/17147" issue_17653 = "https://github.com/ClickHouse/ClickHouse/issues/17653" issue_17655 = "https://github.com/ClickHouse/ClickHouse/issues/17655" +issue_17766 = "https://github.com/ClickHouse/ClickHouse/issues/17766" xfails = { "syntax/show create quota/I show create quota current": @@ -112,6 +113,8 @@ xfails = { [(Fail, issue_17147)], "privileges/show dictionaries/:/check privilege/:/exists/EXISTS with privilege": [(Fail, issue_17655)], + "privileges/public tables/query log": + [(Fail, issue_17766)] } xflags = { diff --git a/tests/testflows/rbac/requirements/requirements.md b/tests/testflows/rbac/requirements/requirements.md index 45afef6ad74..aea25b04d57 100644 --- a/tests/testflows/rbac/requirements/requirements.md +++ b/tests/testflows/rbac/requirements/requirements.md @@ -354,13 +354,14 @@ * 5.2.8.296 [RQ.SRS-006.RBAC.RowPolicy.ShowRowPolicies.Syntax](#rqsrs-006rbacrowpolicyshowrowpoliciessyntax) * 5.2.9 [Table Privileges](#table-privileges) * 5.2.9.1 [RQ.SRS-006.RBAC.Table.PublicTables](#rqsrs-006rbactablepublictables) - * 5.2.9.2 [Distributed Tables](#distributed-tables) - * 5.2.9.2.1 [RQ.SRS-006.RBAC.Table.DistributedTable.Create](#rqsrs-006rbactabledistributedtablecreate) - * 5.2.9.2.2 [RQ.SRS-006.RBAC.Table.DistributedTable.Select](#rqsrs-006rbactabledistributedtableselect) - * 5.2.9.2.3 [RQ.SRS-006.RBAC.Table.DistributedTable.Insert](#rqsrs-006rbactabledistributedtableinsert) - * 5.2.9.2.4 [RQ.SRS-006.RBAC.Table.DistributedTable.SpecialTables](#rqsrs-006rbactabledistributedtablespecialtables) - * 5.2.9.2.5 [RQ.SRS-006.RBAC.Table.DistributedTable.LocalUser](#rqsrs-006rbactabledistributedtablelocaluser) - * 5.2.9.2.6 [RQ.SRS-006.RBAC.Table.DistributedTable.SameUserDifferentNodesDifferentPrivileges](#rqsrs-006rbactabledistributedtablesameuserdifferentnodesdifferentprivileges) + * 5.2.9.2 [RQ.SRS-006.RBAC.Table.QueryLog](#rqsrs-006rbactablequerylog) + * 5.2.9.3 [Distributed Tables](#distributed-tables) + * 5.2.9.3.1 [RQ.SRS-006.RBAC.Table.DistributedTable.Create](#rqsrs-006rbactabledistributedtablecreate) + * 5.2.9.3.2 [RQ.SRS-006.RBAC.Table.DistributedTable.Select](#rqsrs-006rbactabledistributedtableselect) + * 5.2.9.3.3 [RQ.SRS-006.RBAC.Table.DistributedTable.Insert](#rqsrs-006rbactabledistributedtableinsert) + * 5.2.9.3.4 [RQ.SRS-006.RBAC.Table.DistributedTable.SpecialTables](#rqsrs-006rbactabledistributedtablespecialtables) + * 5.2.9.3.5 [RQ.SRS-006.RBAC.Table.DistributedTable.LocalUser](#rqsrs-006rbactabledistributedtablelocaluser) + * 5.2.9.3.6 [RQ.SRS-006.RBAC.Table.DistributedTable.SameUserDifferentNodesDifferentPrivileges](#rqsrs-006rbactabledistributedtablesameuserdifferentnodesdifferentprivileges) * 5.2.10 [Views](#views) * 5.2.10.1 [View](#view) * 5.2.10.1.1 [RQ.SRS-006.RBAC.View](#rqsrs-006rbacview) @@ -489,11 +490,43 @@ * 5.2.11.48 [RQ.SRS-006.RBAC.Privileges.ShowDictionaries.Query](#rqsrs-006rbacprivilegesshowdictionariesquery) * 5.2.11.49 [RQ.SRS-006.RBAC.Privileges.ShowCreateDictionary](#rqsrs-006rbacprivilegesshowcreatedictionary) * 5.2.11.50 [RQ.SRS-006.RBAC.Privileges.ExistsDictionary](#rqsrs-006rbacprivilegesexistsdictionary) - * 5.2.11.51 [Grant Option](#grant-option) - * 5.2.11.51.1 [RQ.SRS-006.RBAC.Privileges.GrantOption](#rqsrs-006rbacprivilegesgrantoption) - * 5.2.11.52 [RQ.SRS-006.RBAC.Privileges.All](#rqsrs-006rbacprivilegesall) - * 5.2.11.53 [RQ.SRS-006.RBAC.Privileges.All.GrantRevoke](#rqsrs-006rbacprivilegesallgrantrevoke) - * 5.2.11.54 [RQ.SRS-006.RBAC.Privileges.AdminOption](#rqsrs-006rbacprivilegesadminoption) + * 5.2.11.51 [RQ.SRS-006.RBAC.Privileges.CreateUser](#rqsrs-006rbacprivilegescreateuser) + * 5.2.11.52 [RQ.SRS-006.RBAC.Privileges.CreateUser.DefaultRole](#rqsrs-006rbacprivilegescreateuserdefaultrole) + * 5.2.11.53 [RQ.SRS-006.RBAC.Privileges.AlterUser](#rqsrs-006rbacprivilegesalteruser) + * 5.2.11.54 [RQ.SRS-006.RBAC.Privileges.DropUser](#rqsrs-006rbacprivilegesdropuser) + * 5.2.11.55 [RQ.SRS-006.RBAC.Privileges.CreateRole](#rqsrs-006rbacprivilegescreaterole) + * 5.2.11.56 [RQ.SRS-006.RBAC.Privileges.AlterRole](#rqsrs-006rbacprivilegesalterrole) + * 5.2.11.57 [RQ.SRS-006.RBAC.Privileges.DropRole](#rqsrs-006rbacprivilegesdroprole) + * 5.2.11.58 [RQ.SRS-006.RBAC.Privileges.CreateRowPolicy](#rqsrs-006rbacprivilegescreaterowpolicy) + * 5.2.11.59 [RQ.SRS-006.RBAC.Privileges.AlterRowPolicy](#rqsrs-006rbacprivilegesalterrowpolicy) + * 5.2.11.60 [RQ.SRS-006.RBAC.Privileges.DropRowPolicy](#rqsrs-006rbacprivilegesdroprowpolicy) + * 5.2.11.61 [RQ.SRS-006.RBAC.Privileges.CreateQuota](#rqsrs-006rbacprivilegescreatequota) + * 5.2.11.62 [RQ.SRS-006.RBAC.Privileges.AlterQuota](#rqsrs-006rbacprivilegesalterquota) + * 5.2.11.63 [RQ.SRS-006.RBAC.Privileges.DropQuota](#rqsrs-006rbacprivilegesdropquota) + * 5.2.11.64 [RQ.SRS-006.RBAC.Privileges.CreateSettingsProfile](#rqsrs-006rbacprivilegescreatesettingsprofile) + * 5.2.11.65 [RQ.SRS-006.RBAC.Privileges.AlterSettingsProfile](#rqsrs-006rbacprivilegesaltersettingsprofile) + * 5.2.11.66 [RQ.SRS-006.RBAC.Privileges.DropSettingsProfile](#rqsrs-006rbacprivilegesdropsettingsprofile) + * 5.2.11.67 [RQ.SRS-006.RBAC.Privileges.RoleAdmin](#rqsrs-006rbacprivilegesroleadmin) + * 5.2.11.68 [RQ.SRS-006.RBAC.Privileges.ShowUsers](#rqsrs-006rbacprivilegesshowusers) + * 5.2.11.69 [RQ.SRS-006.RBAC.Privileges.ShowUsers.Query](#rqsrs-006rbacprivilegesshowusersquery) + * 5.2.11.70 [RQ.SRS-006.RBAC.Privileges.ShowCreateUser](#rqsrs-006rbacprivilegesshowcreateuser) + * 5.2.11.71 [RQ.SRS-006.RBAC.Privileges.ShowRoles](#rqsrs-006rbacprivilegesshowroles) + * 5.2.11.72 [RQ.SRS-006.RBAC.Privileges.ShowRoles.Query](#rqsrs-006rbacprivilegesshowrolesquery) + * 5.2.11.73 [RQ.SRS-006.RBAC.Privileges.ShowCreateRole](#rqsrs-006rbacprivilegesshowcreaterole) + * 5.2.11.74 [RQ.SRS-006.RBAC.Privileges.ShowRowPolicies](#rqsrs-006rbacprivilegesshowrowpolicies) + * 5.2.11.75 [RQ.SRS-006.RBAC.Privileges.ShowRowPolicies.Query](#rqsrs-006rbacprivilegesshowrowpoliciesquery) + * 5.2.11.76 [RQ.SRS-006.RBAC.Privileges.ShowCreateRowPolicy](#rqsrs-006rbacprivilegesshowcreaterowpolicy) + * 5.2.11.77 [RQ.SRS-006.RBAC.Privileges.ShowQuotas](#rqsrs-006rbacprivilegesshowquotas) + * 5.2.11.78 [RQ.SRS-006.RBAC.Privileges.ShowQuotas.Query](#rqsrs-006rbacprivilegesshowquotasquery) + * 5.2.11.79 [RQ.SRS-006.RBAC.Privileges.ShowCreateQuota](#rqsrs-006rbacprivilegesshowcreatequota) + * 5.2.11.80 [RQ.SRS-006.RBAC.Privileges.ShowSettingsProfiles](#rqsrs-006rbacprivilegesshowsettingsprofiles) + * 5.2.11.81 [RQ.SRS-006.RBAC.Privileges.ShowSettingsProfiles.Query](#rqsrs-006rbacprivilegesshowsettingsprofilesquery) + * 5.2.11.82 [RQ.SRS-006.RBAC.Privileges.ShowCreateSettingsProfile](#rqsrs-006rbacprivilegesshowcreatesettingsprofile) + * 5.2.11.83 [Grant Option](#grant-option) + * 5.2.11.83.1 [RQ.SRS-006.RBAC.Privileges.GrantOption](#rqsrs-006rbacprivilegesgrantoption) + * 5.2.11.84 [RQ.SRS-006.RBAC.Privileges.All](#rqsrs-006rbacprivilegesall) + * 5.2.11.85 [RQ.SRS-006.RBAC.Privileges.All.GrantRevoke](#rqsrs-006rbacprivilegesallgrantrevoke) + * 5.2.11.86 [RQ.SRS-006.RBAC.Privileges.AdminOption](#rqsrs-006rbacprivilegesadminoption) * 5.2.12 [Required Privileges](#required-privileges) * 5.2.12.1 [RQ.SRS-006.RBAC.RequiredPrivileges.Create](#rqsrs-006rbacrequiredprivilegescreate) * 5.2.12.2 [RQ.SRS-006.RBAC.RequiredPrivileges.Alter](#rqsrs-006rbacrequiredprivilegesalter) @@ -2864,6 +2897,11 @@ version: 1.0 * system.contributors * system.functions +##### RQ.SRS-006.RBAC.Table.QueryLog +version: 1.0 + +[ClickHouse] SHALL return only queries executed by the user when the user is selecting from system.query_log. + ##### Distributed Tables ###### RQ.SRS-006.RBAC.Table.DistributedTable.Create @@ -3932,6 +3970,199 @@ version: 1.0 [ClickHouse] SHALL successfully execute `EXISTS dictionary` statement if and only if the user has **show dictionaries** privilege, or any privilege on the dictionary either directly or through a role. +##### RQ.SRS-006.RBAC.Privileges.CreateUser +version: 1.0 + +[ClickHouse] SHALL successfully execute `CREATE USER` statement if and only if the user has **create user** privilege, +or either directly or through a role. + +##### RQ.SRS-006.RBAC.Privileges.CreateUser.DefaultRole +version: 1.0 + +[ClickHouse] SHALL successfully execute `CREATE USER` statement with `DEFAULT ROLE ` clause if and only if +the user has **create user** privilege and the role with **admin option**, or either directly or through a role. + +##### RQ.SRS-006.RBAC.Privileges.AlterUser +version: 1.0 + +[ClickHouse] SHALL successfully execute `ALTER USER` statement if and only if the user has **alter user** privilege, +or either directly or through a role. + +##### RQ.SRS-006.RBAC.Privileges.DropUser +version: 1.0 + +[ClickHouse] SHALL successfully execute `DROP USER` statement if and only if the user has **drop user** privilege, +or either directly or through a role. + +##### RQ.SRS-006.RBAC.Privileges.CreateRole +version: 1.0 + +[ClickHouse] SHALL successfully execute `CREATE ROLE` statement if and only if the user has **create role** privilege, +or either directly or through a role. + +##### RQ.SRS-006.RBAC.Privileges.AlterRole +version: 1.0 + +[ClickHouse] SHALL successfully execute `ALTER ROLE` statement if and only if the user has **alter role** privilege, +or either directly or through a role. + +##### RQ.SRS-006.RBAC.Privileges.DropRole +version: 1.0 + +[ClickHouse] SHALL successfully execute `DROP ROLE` statement if and only if the user has **drop role** privilege, +or either directly or through a role. + +##### RQ.SRS-006.RBAC.Privileges.CreateRowPolicy +version: 1.0 + +[ClickHouse] SHALL successfully execute `CREATE ROW POLICY` statement if and only if the user has **create row policy** privilege, +or either directly or through a role. + +##### RQ.SRS-006.RBAC.Privileges.AlterRowPolicy +version: 1.0 + +[ClickHouse] SHALL successfully execute `ALTER ROW POLICY` statement if and only if the user has **alter row policy** privilege, +or either directly or through a role. + +##### RQ.SRS-006.RBAC.Privileges.DropRowPolicy +version: 1.0 + +[ClickHouse] SHALL successfully execute `DROP ROW POLICY` statement if and only if the user has **drop row policy** privilege, +or either directly or through a role. + +##### RQ.SRS-006.RBAC.Privileges.CreateQuota +version: 1.0 + +[ClickHouse] SHALL successfully execute `CREATE QUOTA` statement if and only if the user has **create quota** privilege, +or either directly or through a role. + +##### RQ.SRS-006.RBAC.Privileges.AlterQuota +version: 1.0 + +[ClickHouse] SHALL successfully execute `ALTER QUOTA` statement if and only if the user has **alter quota** privilege, +or either directly or through a role. + +##### RQ.SRS-006.RBAC.Privileges.DropQuota +version: 1.0 + +[ClickHouse] SHALL successfully execute `DROP QUOTA` statement if and only if the user has **drop quota** privilege, +or either directly or through a role. + +##### RQ.SRS-006.RBAC.Privileges.CreateSettingsProfile +version: 1.0 + +[ClickHouse] SHALL successfully execute `CREATE SETTINGS PROFILE` statement if and only if the user has **create settings profile** privilege, +or either directly or through a role. + +##### RQ.SRS-006.RBAC.Privileges.AlterSettingsProfile +version: 1.0 + +[ClickHouse] SHALL successfully execute `ALTER SETTINGS PROFILE` statement if and only if the user has **alter settings profile** privilege, +or either directly or through a role. + +##### RQ.SRS-006.RBAC.Privileges.DropSettingsProfile +version: 1.0 + +[ClickHouse] SHALL successfully execute `DROP SETTINGS PROFILE` statement if and only if the user has **drop settings profile** privilege, +or either directly or through a role. + +##### RQ.SRS-006.RBAC.Privileges.RoleAdmin +version: 1.0 + +[ClickHouse] SHALL successfully execute any role grant or revoke by a user with `ROLE ADMIN` privilege. + +##### RQ.SRS-006.RBAC.Privileges.ShowUsers +version: 1.0 + +[ClickHouse] SHALL successfully grant `SHOW USERS` privilege when +the user is granted `SHOW USERS`, `SHOW CREATE USER`, `SHOW ACCESS`, or `ACCESS MANAGEMENT`. + +##### RQ.SRS-006.RBAC.Privileges.ShowUsers.Query +version: 1.0 + +[ClickHouse] SHALL successfully execute `SHOW USERS` statement if and only if the user has **show users** privilege, +either directly or through a role. + +##### RQ.SRS-006.RBAC.Privileges.ShowCreateUser +version: 1.0 + +[ClickHouse] SHALL successfully execute `SHOW CREATE USER` statement if and only if the user has **show create user** privilege, +either directly or through a role. + +##### RQ.SRS-006.RBAC.Privileges.ShowRoles +version: 1.0 + +[ClickHouse] SHALL successfully grant `SHOW ROLES` privilege when +the user is granted `SHOW ROLES`, `SHOW CREATE ROLE`, `SHOW ACCESS`, or `ACCESS MANAGEMENT`. + +##### RQ.SRS-006.RBAC.Privileges.ShowRoles.Query +version: 1.0 + +[ClickHouse] SHALL successfully execute `SHOW ROLES` statement if and only if the user has **show roles** privilege, +either directly or through a role. + +##### RQ.SRS-006.RBAC.Privileges.ShowCreateRole +version: 1.0 + +[ClickHouse] SHALL successfully execute `SHOW CREATE ROLE` statement if and only if the user has **show create role** privilege, +either directly or through a role. + +##### RQ.SRS-006.RBAC.Privileges.ShowRowPolicies +version: 1.0 + +[ClickHouse] SHALL successfully grant `SHOW ROW POLICIES` privilege when +the user is granted `SHOW ROW POLICIES`, `SHOW POLICIES`, `SHOW CREATE ROW POLICY`, +`SHOW CREATE POLICY`, `SHOW ACCESS`, or `ACCESS MANAGEMENT`. + +##### RQ.SRS-006.RBAC.Privileges.ShowRowPolicies.Query +version: 1.0 + +[ClickHouse] SHALL successfully execute `SHOW ROW POLICIES` or `SHOW POLICIES` statement if and only if the user has **show row policies** privilege, +either directly or through a role. + +##### RQ.SRS-006.RBAC.Privileges.ShowCreateRowPolicy +version: 1.0 + +[ClickHouse] SHALL successfully execute `SHOW CREATE ROW POLICY` or `SHOW CREATE POLICY` statement if and only if the user has **show create row policy** privilege, +either directly or through a role. + +##### RQ.SRS-006.RBAC.Privileges.ShowQuotas +version: 1.0 + +[ClickHouse] SHALL successfully grant `SHOW QUOTAS` privilege when +the user is granted `SHOW QUOTAS`, `SHOW CREATE QUOTA`, `SHOW ACCESS`, or `ACCESS MANAGEMENT`. + +##### RQ.SRS-006.RBAC.Privileges.ShowQuotas.Query +version: 1.0 + +[ClickHouse] SHALL successfully execute `SHOW QUOTAS` statement if and only if the user has **show quotas** privilege, +either directly or through a role. + +##### RQ.SRS-006.RBAC.Privileges.ShowCreateQuota +version: 1.0 + +[ClickHouse] SHALL successfully execute `SHOW CREATE QUOTA` statement if and only if the user has **show create quota** privilege, +either directly or through a role. + +##### RQ.SRS-006.RBAC.Privileges.ShowSettingsProfiles +version: 1.0 + +[ClickHouse] SHALL successfully grant `SHOW SETTINGS PROFILES` privilege when +the user is granted `SHOW SETTINGS PROFILES`, `SHOW PROFILES`, `SHOW CREATE SETTINGS PROFILE`, +`SHOW SETTINGS PROFILE`, `SHOW ACCESS`, or `ACCESS MANAGEMENT`. + +##### RQ.SRS-006.RBAC.Privileges.ShowSettingsProfiles.Query +version: 1.0 + +[ClickHouse] SHALL successfully execute `SHOW SETTINGS PROFILES` or `SHOW PROFILES` statement if and only if the user has **show settings profiles** privilege, +either directly or through a role. + +##### RQ.SRS-006.RBAC.Privileges.ShowCreateSettingsProfile +version: 1.0 + +[ClickHouse] SHALL successfully execute `SHOW CREATE SETTINGS PROFILE` or `SHOW CREATE PROFILE` statement if and only if the user has **show create settings profile** privilege, +either directly or through a role. + ##### Grant Option ###### RQ.SRS-006.RBAC.Privileges.GrantOption diff --git a/tests/testflows/rbac/requirements/requirements.py b/tests/testflows/rbac/requirements/requirements.py index 7b106f371d1..bcd24853536 100755 --- a/tests/testflows/rbac/requirements/requirements.py +++ b/tests/testflows/rbac/requirements/requirements.py @@ -379,13 +379,14 @@ SRS_006_ClickHouse_Role_Based_Access_Control = Specification( * 5.2.8.296 [RQ.SRS-006.RBAC.RowPolicy.ShowRowPolicies.Syntax](#rqsrs-006rbacrowpolicyshowrowpoliciessyntax) * 5.2.9 [Table Privileges](#table-privileges) * 5.2.9.1 [RQ.SRS-006.RBAC.Table.PublicTables](#rqsrs-006rbactablepublictables) - * 5.2.9.2 [Distributed Tables](#distributed-tables) - * 5.2.9.2.1 [RQ.SRS-006.RBAC.Table.DistributedTable.Create](#rqsrs-006rbactabledistributedtablecreate) - * 5.2.9.2.2 [RQ.SRS-006.RBAC.Table.DistributedTable.Select](#rqsrs-006rbactabledistributedtableselect) - * 5.2.9.2.3 [RQ.SRS-006.RBAC.Table.DistributedTable.Insert](#rqsrs-006rbactabledistributedtableinsert) - * 5.2.9.2.4 [RQ.SRS-006.RBAC.Table.DistributedTable.SpecialTables](#rqsrs-006rbactabledistributedtablespecialtables) - * 5.2.9.2.5 [RQ.SRS-006.RBAC.Table.DistributedTable.LocalUser](#rqsrs-006rbactabledistributedtablelocaluser) - * 5.2.9.2.6 [RQ.SRS-006.RBAC.Table.DistributedTable.SameUserDifferentNodesDifferentPrivileges](#rqsrs-006rbactabledistributedtablesameuserdifferentnodesdifferentprivileges) + * 5.2.9.2 [RQ.SRS-006.RBAC.Table.QueryLog](#rqsrs-006rbactablequerylog) + * 5.2.9.3 [Distributed Tables](#distributed-tables) + * 5.2.9.3.1 [RQ.SRS-006.RBAC.Table.DistributedTable.Create](#rqsrs-006rbactabledistributedtablecreate) + * 5.2.9.3.2 [RQ.SRS-006.RBAC.Table.DistributedTable.Select](#rqsrs-006rbactabledistributedtableselect) + * 5.2.9.3.3 [RQ.SRS-006.RBAC.Table.DistributedTable.Insert](#rqsrs-006rbactabledistributedtableinsert) + * 5.2.9.3.4 [RQ.SRS-006.RBAC.Table.DistributedTable.SpecialTables](#rqsrs-006rbactabledistributedtablespecialtables) + * 5.2.9.3.5 [RQ.SRS-006.RBAC.Table.DistributedTable.LocalUser](#rqsrs-006rbactabledistributedtablelocaluser) + * 5.2.9.3.6 [RQ.SRS-006.RBAC.Table.DistributedTable.SameUserDifferentNodesDifferentPrivileges](#rqsrs-006rbactabledistributedtablesameuserdifferentnodesdifferentprivileges) * 5.2.10 [Views](#views) * 5.2.10.1 [View](#view) * 5.2.10.1.1 [RQ.SRS-006.RBAC.View](#rqsrs-006rbacview) @@ -514,11 +515,43 @@ SRS_006_ClickHouse_Role_Based_Access_Control = Specification( * 5.2.11.48 [RQ.SRS-006.RBAC.Privileges.ShowDictionaries.Query](#rqsrs-006rbacprivilegesshowdictionariesquery) * 5.2.11.49 [RQ.SRS-006.RBAC.Privileges.ShowCreateDictionary](#rqsrs-006rbacprivilegesshowcreatedictionary) * 5.2.11.50 [RQ.SRS-006.RBAC.Privileges.ExistsDictionary](#rqsrs-006rbacprivilegesexistsdictionary) - * 5.2.11.51 [Grant Option](#grant-option) - * 5.2.11.51.1 [RQ.SRS-006.RBAC.Privileges.GrantOption](#rqsrs-006rbacprivilegesgrantoption) - * 5.2.11.52 [RQ.SRS-006.RBAC.Privileges.All](#rqsrs-006rbacprivilegesall) - * 5.2.11.53 [RQ.SRS-006.RBAC.Privileges.All.GrantRevoke](#rqsrs-006rbacprivilegesallgrantrevoke) - * 5.2.11.54 [RQ.SRS-006.RBAC.Privileges.AdminOption](#rqsrs-006rbacprivilegesadminoption) + * 5.2.11.51 [RQ.SRS-006.RBAC.Privileges.CreateUser](#rqsrs-006rbacprivilegescreateuser) + * 5.2.11.52 [RQ.SRS-006.RBAC.Privileges.CreateUser.DefaultRole](#rqsrs-006rbacprivilegescreateuserdefaultrole) + * 5.2.11.53 [RQ.SRS-006.RBAC.Privileges.AlterUser](#rqsrs-006rbacprivilegesalteruser) + * 5.2.11.54 [RQ.SRS-006.RBAC.Privileges.DropUser](#rqsrs-006rbacprivilegesdropuser) + * 5.2.11.55 [RQ.SRS-006.RBAC.Privileges.CreateRole](#rqsrs-006rbacprivilegescreaterole) + * 5.2.11.56 [RQ.SRS-006.RBAC.Privileges.AlterRole](#rqsrs-006rbacprivilegesalterrole) + * 5.2.11.57 [RQ.SRS-006.RBAC.Privileges.DropRole](#rqsrs-006rbacprivilegesdroprole) + * 5.2.11.58 [RQ.SRS-006.RBAC.Privileges.CreateRowPolicy](#rqsrs-006rbacprivilegescreaterowpolicy) + * 5.2.11.59 [RQ.SRS-006.RBAC.Privileges.AlterRowPolicy](#rqsrs-006rbacprivilegesalterrowpolicy) + * 5.2.11.60 [RQ.SRS-006.RBAC.Privileges.DropRowPolicy](#rqsrs-006rbacprivilegesdroprowpolicy) + * 5.2.11.61 [RQ.SRS-006.RBAC.Privileges.CreateQuota](#rqsrs-006rbacprivilegescreatequota) + * 5.2.11.62 [RQ.SRS-006.RBAC.Privileges.AlterQuota](#rqsrs-006rbacprivilegesalterquota) + * 5.2.11.63 [RQ.SRS-006.RBAC.Privileges.DropQuota](#rqsrs-006rbacprivilegesdropquota) + * 5.2.11.64 [RQ.SRS-006.RBAC.Privileges.CreateSettingsProfile](#rqsrs-006rbacprivilegescreatesettingsprofile) + * 5.2.11.65 [RQ.SRS-006.RBAC.Privileges.AlterSettingsProfile](#rqsrs-006rbacprivilegesaltersettingsprofile) + * 5.2.11.66 [RQ.SRS-006.RBAC.Privileges.DropSettingsProfile](#rqsrs-006rbacprivilegesdropsettingsprofile) + * 5.2.11.67 [RQ.SRS-006.RBAC.Privileges.RoleAdmin](#rqsrs-006rbacprivilegesroleadmin) + * 5.2.11.68 [RQ.SRS-006.RBAC.Privileges.ShowUsers](#rqsrs-006rbacprivilegesshowusers) + * 5.2.11.69 [RQ.SRS-006.RBAC.Privileges.ShowUsers.Query](#rqsrs-006rbacprivilegesshowusersquery) + * 5.2.11.70 [RQ.SRS-006.RBAC.Privileges.ShowCreateUser](#rqsrs-006rbacprivilegesshowcreateuser) + * 5.2.11.71 [RQ.SRS-006.RBAC.Privileges.ShowRoles](#rqsrs-006rbacprivilegesshowroles) + * 5.2.11.72 [RQ.SRS-006.RBAC.Privileges.ShowRoles.Query](#rqsrs-006rbacprivilegesshowrolesquery) + * 5.2.11.73 [RQ.SRS-006.RBAC.Privileges.ShowCreateRole](#rqsrs-006rbacprivilegesshowcreaterole) + * 5.2.11.74 [RQ.SRS-006.RBAC.Privileges.ShowRowPolicies](#rqsrs-006rbacprivilegesshowrowpolicies) + * 5.2.11.75 [RQ.SRS-006.RBAC.Privileges.ShowRowPolicies.Query](#rqsrs-006rbacprivilegesshowrowpoliciesquery) + * 5.2.11.76 [RQ.SRS-006.RBAC.Privileges.ShowCreateRowPolicy](#rqsrs-006rbacprivilegesshowcreaterowpolicy) + * 5.2.11.77 [RQ.SRS-006.RBAC.Privileges.ShowQuotas](#rqsrs-006rbacprivilegesshowquotas) + * 5.2.11.78 [RQ.SRS-006.RBAC.Privileges.ShowQuotas.Query](#rqsrs-006rbacprivilegesshowquotasquery) + * 5.2.11.79 [RQ.SRS-006.RBAC.Privileges.ShowCreateQuota](#rqsrs-006rbacprivilegesshowcreatequota) + * 5.2.11.80 [RQ.SRS-006.RBAC.Privileges.ShowSettingsProfiles](#rqsrs-006rbacprivilegesshowsettingsprofiles) + * 5.2.11.81 [RQ.SRS-006.RBAC.Privileges.ShowSettingsProfiles.Query](#rqsrs-006rbacprivilegesshowsettingsprofilesquery) + * 5.2.11.82 [RQ.SRS-006.RBAC.Privileges.ShowCreateSettingsProfile](#rqsrs-006rbacprivilegesshowcreatesettingsprofile) + * 5.2.11.83 [Grant Option](#grant-option) + * 5.2.11.83.1 [RQ.SRS-006.RBAC.Privileges.GrantOption](#rqsrs-006rbacprivilegesgrantoption) + * 5.2.11.84 [RQ.SRS-006.RBAC.Privileges.All](#rqsrs-006rbacprivilegesall) + * 5.2.11.85 [RQ.SRS-006.RBAC.Privileges.All.GrantRevoke](#rqsrs-006rbacprivilegesallgrantrevoke) + * 5.2.11.86 [RQ.SRS-006.RBAC.Privileges.AdminOption](#rqsrs-006rbacprivilegesadminoption) * 5.2.12 [Required Privileges](#required-privileges) * 5.2.12.1 [RQ.SRS-006.RBAC.RequiredPrivileges.Create](#rqsrs-006rbacrequiredprivilegescreate) * 5.2.12.2 [RQ.SRS-006.RBAC.RequiredPrivileges.Alter](#rqsrs-006rbacrequiredprivilegesalter) @@ -2889,6 +2922,11 @@ version: 1.0 * system.contributors * system.functions +##### RQ.SRS-006.RBAC.Table.QueryLog +version: 1.0 + +[ClickHouse] SHALL return only queries executed by the user when the user is selecting from system.query_log. + ##### Distributed Tables ###### RQ.SRS-006.RBAC.Table.DistributedTable.Create @@ -3957,6 +3995,199 @@ version: 1.0 [ClickHouse] SHALL successfully execute `EXISTS dictionary` statement if and only if the user has **show dictionaries** privilege, or any privilege on the dictionary either directly or through a role. +##### RQ.SRS-006.RBAC.Privileges.CreateUser +version: 1.0 + +[ClickHouse] SHALL successfully execute `CREATE USER` statement if and only if the user has **create user** privilege, +or either directly or through a role. + +##### RQ.SRS-006.RBAC.Privileges.CreateUser.DefaultRole +version: 1.0 + +[ClickHouse] SHALL successfully execute `CREATE USER` statement with `DEFAULT ROLE ` clause if and only if +the user has **create user** privilege and the role with **admin option**, or either directly or through a role. + +##### RQ.SRS-006.RBAC.Privileges.AlterUser +version: 1.0 + +[ClickHouse] SHALL successfully execute `ALTER USER` statement if and only if the user has **alter user** privilege, +or either directly or through a role. + +##### RQ.SRS-006.RBAC.Privileges.DropUser +version: 1.0 + +[ClickHouse] SHALL successfully execute `DROP USER` statement if and only if the user has **drop user** privilege, +or either directly or through a role. + +##### RQ.SRS-006.RBAC.Privileges.CreateRole +version: 1.0 + +[ClickHouse] SHALL successfully execute `CREATE ROLE` statement if and only if the user has **create role** privilege, +or either directly or through a role. + +##### RQ.SRS-006.RBAC.Privileges.AlterRole +version: 1.0 + +[ClickHouse] SHALL successfully execute `ALTER ROLE` statement if and only if the user has **alter role** privilege, +or either directly or through a role. + +##### RQ.SRS-006.RBAC.Privileges.DropRole +version: 1.0 + +[ClickHouse] SHALL successfully execute `DROP ROLE` statement if and only if the user has **drop role** privilege, +or either directly or through a role. + +##### RQ.SRS-006.RBAC.Privileges.CreateRowPolicy +version: 1.0 + +[ClickHouse] SHALL successfully execute `CREATE ROW POLICY` statement if and only if the user has **create row policy** privilege, +or either directly or through a role. + +##### RQ.SRS-006.RBAC.Privileges.AlterRowPolicy +version: 1.0 + +[ClickHouse] SHALL successfully execute `ALTER ROW POLICY` statement if and only if the user has **alter row policy** privilege, +or either directly or through a role. + +##### RQ.SRS-006.RBAC.Privileges.DropRowPolicy +version: 1.0 + +[ClickHouse] SHALL successfully execute `DROP ROW POLICY` statement if and only if the user has **drop row policy** privilege, +or either directly or through a role. + +##### RQ.SRS-006.RBAC.Privileges.CreateQuota +version: 1.0 + +[ClickHouse] SHALL successfully execute `CREATE QUOTA` statement if and only if the user has **create quota** privilege, +or either directly or through a role. + +##### RQ.SRS-006.RBAC.Privileges.AlterQuota +version: 1.0 + +[ClickHouse] SHALL successfully execute `ALTER QUOTA` statement if and only if the user has **alter quota** privilege, +or either directly or through a role. + +##### RQ.SRS-006.RBAC.Privileges.DropQuota +version: 1.0 + +[ClickHouse] SHALL successfully execute `DROP QUOTA` statement if and only if the user has **drop quota** privilege, +or either directly or through a role. + +##### RQ.SRS-006.RBAC.Privileges.CreateSettingsProfile +version: 1.0 + +[ClickHouse] SHALL successfully execute `CREATE SETTINGS PROFILE` statement if and only if the user has **create settings profile** privilege, +or either directly or through a role. + +##### RQ.SRS-006.RBAC.Privileges.AlterSettingsProfile +version: 1.0 + +[ClickHouse] SHALL successfully execute `ALTER SETTINGS PROFILE` statement if and only if the user has **alter settings profile** privilege, +or either directly or through a role. + +##### RQ.SRS-006.RBAC.Privileges.DropSettingsProfile +version: 1.0 + +[ClickHouse] SHALL successfully execute `DROP SETTINGS PROFILE` statement if and only if the user has **drop settings profile** privilege, +or either directly or through a role. + +##### RQ.SRS-006.RBAC.Privileges.RoleAdmin +version: 1.0 + +[ClickHouse] SHALL successfully execute any role grant or revoke by a user with `ROLE ADMIN` privilege. + +##### RQ.SRS-006.RBAC.Privileges.ShowUsers +version: 1.0 + +[ClickHouse] SHALL successfully grant `SHOW USERS` privilege when +the user is granted `SHOW USERS`, `SHOW CREATE USER`, `SHOW ACCESS`, or `ACCESS MANAGEMENT`. + +##### RQ.SRS-006.RBAC.Privileges.ShowUsers.Query +version: 1.0 + +[ClickHouse] SHALL successfully execute `SHOW USERS` statement if and only if the user has **show users** privilege, +either directly or through a role. + +##### RQ.SRS-006.RBAC.Privileges.ShowCreateUser +version: 1.0 + +[ClickHouse] SHALL successfully execute `SHOW CREATE USER` statement if and only if the user has **show create user** privilege, +either directly or through a role. + +##### RQ.SRS-006.RBAC.Privileges.ShowRoles +version: 1.0 + +[ClickHouse] SHALL successfully grant `SHOW ROLES` privilege when +the user is granted `SHOW ROLES`, `SHOW CREATE ROLE`, `SHOW ACCESS`, or `ACCESS MANAGEMENT`. + +##### RQ.SRS-006.RBAC.Privileges.ShowRoles.Query +version: 1.0 + +[ClickHouse] SHALL successfully execute `SHOW ROLES` statement if and only if the user has **show roles** privilege, +either directly or through a role. + +##### RQ.SRS-006.RBAC.Privileges.ShowCreateRole +version: 1.0 + +[ClickHouse] SHALL successfully execute `SHOW CREATE ROLE` statement if and only if the user has **show create role** privilege, +either directly or through a role. + +##### RQ.SRS-006.RBAC.Privileges.ShowRowPolicies +version: 1.0 + +[ClickHouse] SHALL successfully grant `SHOW ROW POLICIES` privilege when +the user is granted `SHOW ROW POLICIES`, `SHOW POLICIES`, `SHOW CREATE ROW POLICY`, +`SHOW CREATE POLICY`, `SHOW ACCESS`, or `ACCESS MANAGEMENT`. + +##### RQ.SRS-006.RBAC.Privileges.ShowRowPolicies.Query +version: 1.0 + +[ClickHouse] SHALL successfully execute `SHOW ROW POLICIES` or `SHOW POLICIES` statement if and only if the user has **show row policies** privilege, +either directly or through a role. + +##### RQ.SRS-006.RBAC.Privileges.ShowCreateRowPolicy +version: 1.0 + +[ClickHouse] SHALL successfully execute `SHOW CREATE ROW POLICY` or `SHOW CREATE POLICY` statement if and only if the user has **show create row policy** privilege, +either directly or through a role. + +##### RQ.SRS-006.RBAC.Privileges.ShowQuotas +version: 1.0 + +[ClickHouse] SHALL successfully grant `SHOW QUOTAS` privilege when +the user is granted `SHOW QUOTAS`, `SHOW CREATE QUOTA`, `SHOW ACCESS`, or `ACCESS MANAGEMENT`. + +##### RQ.SRS-006.RBAC.Privileges.ShowQuotas.Query +version: 1.0 + +[ClickHouse] SHALL successfully execute `SHOW QUOTAS` statement if and only if the user has **show quotas** privilege, +either directly or through a role. + +##### RQ.SRS-006.RBAC.Privileges.ShowCreateQuota +version: 1.0 + +[ClickHouse] SHALL successfully execute `SHOW CREATE QUOTA` statement if and only if the user has **show create quota** privilege, +either directly or through a role. + +##### RQ.SRS-006.RBAC.Privileges.ShowSettingsProfiles +version: 1.0 + +[ClickHouse] SHALL successfully grant `SHOW SETTINGS PROFILES` privilege when +the user is granted `SHOW SETTINGS PROFILES`, `SHOW PROFILES`, `SHOW CREATE SETTINGS PROFILE`, +`SHOW SETTINGS PROFILE`, `SHOW ACCESS`, or `ACCESS MANAGEMENT`. + +##### RQ.SRS-006.RBAC.Privileges.ShowSettingsProfiles.Query +version: 1.0 + +[ClickHouse] SHALL successfully execute `SHOW SETTINGS PROFILES` or `SHOW PROFILES` statement if and only if the user has **show settings profiles** privilege, +either directly or through a role. + +##### RQ.SRS-006.RBAC.Privileges.ShowCreateSettingsProfile +version: 1.0 + +[ClickHouse] SHALL successfully execute `SHOW CREATE SETTINGS PROFILE` or `SHOW CREATE PROFILE` statement if and only if the user has **show create settings profile** privilege, +either directly or through a role. + ##### Grant Option ###### RQ.SRS-006.RBAC.Privileges.GrantOption @@ -9065,6 +9296,19 @@ RQ_SRS_006_RBAC_Table_PublicTables = Requirement( ), link=None) +RQ_SRS_006_RBAC_Table_QueryLog = Requirement( + name='RQ.SRS-006.RBAC.Table.QueryLog', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL return only queries executed by the user when the user is selecting from system.query_log.\n' + '\n' + ), + link=None) + RQ_SRS_006_RBAC_Table_DistributedTable_Create = Requirement( name='RQ.SRS-006.RBAC.Table.DistributedTable.Create', version='1.0', @@ -11033,6 +11277,455 @@ RQ_SRS_006_RBAC_Privileges_ExistsDictionary = Requirement( ), link=None) +RQ_SRS_006_RBAC_Privileges_CreateUser = Requirement( + name='RQ.SRS-006.RBAC.Privileges.CreateUser', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL successfully execute `CREATE USER` statement if and only if the user has **create user** privilege,\n' + 'or either directly or through a role.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_Privileges_CreateUser_DefaultRole = Requirement( + name='RQ.SRS-006.RBAC.Privileges.CreateUser.DefaultRole', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL successfully execute `CREATE USER` statement with `DEFAULT ROLE ` clause if and only if\n' + 'the user has **create user** privilege and the role with **admin option**, or either directly or through a role.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_Privileges_AlterUser = Requirement( + name='RQ.SRS-006.RBAC.Privileges.AlterUser', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL successfully execute `ALTER USER` statement if and only if the user has **alter user** privilege,\n' + 'or either directly or through a role.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_Privileges_DropUser = Requirement( + name='RQ.SRS-006.RBAC.Privileges.DropUser', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL successfully execute `DROP USER` statement if and only if the user has **drop user** privilege,\n' + 'or either directly or through a role.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_Privileges_CreateRole = Requirement( + name='RQ.SRS-006.RBAC.Privileges.CreateRole', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL successfully execute `CREATE ROLE` statement if and only if the user has **create role** privilege,\n' + 'or either directly or through a role.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_Privileges_AlterRole = Requirement( + name='RQ.SRS-006.RBAC.Privileges.AlterRole', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL successfully execute `ALTER ROLE` statement if and only if the user has **alter role** privilege,\n' + 'or either directly or through a role.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_Privileges_DropRole = Requirement( + name='RQ.SRS-006.RBAC.Privileges.DropRole', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL successfully execute `DROP ROLE` statement if and only if the user has **drop role** privilege,\n' + 'or either directly or through a role.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_Privileges_CreateRowPolicy = Requirement( + name='RQ.SRS-006.RBAC.Privileges.CreateRowPolicy', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL successfully execute `CREATE ROW POLICY` statement if and only if the user has **create row policy** privilege,\n' + 'or either directly or through a role.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_Privileges_AlterRowPolicy = Requirement( + name='RQ.SRS-006.RBAC.Privileges.AlterRowPolicy', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL successfully execute `ALTER ROW POLICY` statement if and only if the user has **alter row policy** privilege,\n' + 'or either directly or through a role.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_Privileges_DropRowPolicy = Requirement( + name='RQ.SRS-006.RBAC.Privileges.DropRowPolicy', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL successfully execute `DROP ROW POLICY` statement if and only if the user has **drop row policy** privilege,\n' + 'or either directly or through a role.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_Privileges_CreateQuota = Requirement( + name='RQ.SRS-006.RBAC.Privileges.CreateQuota', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL successfully execute `CREATE QUOTA` statement if and only if the user has **create quota** privilege,\n' + 'or either directly or through a role.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_Privileges_AlterQuota = Requirement( + name='RQ.SRS-006.RBAC.Privileges.AlterQuota', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL successfully execute `ALTER QUOTA` statement if and only if the user has **alter quota** privilege,\n' + 'or either directly or through a role.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_Privileges_DropQuota = Requirement( + name='RQ.SRS-006.RBAC.Privileges.DropQuota', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL successfully execute `DROP QUOTA` statement if and only if the user has **drop quota** privilege,\n' + 'or either directly or through a role.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_Privileges_CreateSettingsProfile = Requirement( + name='RQ.SRS-006.RBAC.Privileges.CreateSettingsProfile', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL successfully execute `CREATE SETTINGS PROFILE` statement if and only if the user has **create settings profile** privilege,\n' + 'or either directly or through a role.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_Privileges_AlterSettingsProfile = Requirement( + name='RQ.SRS-006.RBAC.Privileges.AlterSettingsProfile', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL successfully execute `ALTER SETTINGS PROFILE` statement if and only if the user has **alter settings profile** privilege,\n' + 'or either directly or through a role.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_Privileges_DropSettingsProfile = Requirement( + name='RQ.SRS-006.RBAC.Privileges.DropSettingsProfile', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL successfully execute `DROP SETTINGS PROFILE` statement if and only if the user has **drop settings profile** privilege,\n' + 'or either directly or through a role.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_Privileges_RoleAdmin = Requirement( + name='RQ.SRS-006.RBAC.Privileges.RoleAdmin', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL successfully execute any role grant or revoke by a user with `ROLE ADMIN` privilege.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_Privileges_ShowUsers = Requirement( + name='RQ.SRS-006.RBAC.Privileges.ShowUsers', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL successfully grant `SHOW USERS` privilege when\n' + 'the user is granted `SHOW USERS`, `SHOW CREATE USER`, `SHOW ACCESS`, or `ACCESS MANAGEMENT`.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_Privileges_ShowUsers_Query = Requirement( + name='RQ.SRS-006.RBAC.Privileges.ShowUsers.Query', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL successfully execute `SHOW USERS` statement if and only if the user has **show users** privilege,\n' + 'either directly or through a role.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_Privileges_ShowCreateUser = Requirement( + name='RQ.SRS-006.RBAC.Privileges.ShowCreateUser', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL successfully execute `SHOW CREATE USER` statement if and only if the user has **show create user** privilege,\n' + 'either directly or through a role.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_Privileges_ShowRoles = Requirement( + name='RQ.SRS-006.RBAC.Privileges.ShowRoles', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL successfully grant `SHOW ROLES` privilege when\n' + 'the user is granted `SHOW ROLES`, `SHOW CREATE ROLE`, `SHOW ACCESS`, or `ACCESS MANAGEMENT`.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_Privileges_ShowRoles_Query = Requirement( + name='RQ.SRS-006.RBAC.Privileges.ShowRoles.Query', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL successfully execute `SHOW ROLES` statement if and only if the user has **show roles** privilege,\n' + 'either directly or through a role.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_Privileges_ShowCreateRole = Requirement( + name='RQ.SRS-006.RBAC.Privileges.ShowCreateRole', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL successfully execute `SHOW CREATE ROLE` statement if and only if the user has **show create role** privilege,\n' + 'either directly or through a role.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_Privileges_ShowRowPolicies = Requirement( + name='RQ.SRS-006.RBAC.Privileges.ShowRowPolicies', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL successfully grant `SHOW ROW POLICIES` privilege when\n' + 'the user is granted `SHOW ROW POLICIES`, `SHOW POLICIES`, `SHOW CREATE ROW POLICY`,\n' + '`SHOW CREATE POLICY`, `SHOW ACCESS`, or `ACCESS MANAGEMENT`.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_Privileges_ShowRowPolicies_Query = Requirement( + name='RQ.SRS-006.RBAC.Privileges.ShowRowPolicies.Query', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL successfully execute `SHOW ROW POLICIES` or `SHOW POLICIES` statement if and only if the user has **show row policies** privilege,\n' + 'either directly or through a role.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_Privileges_ShowCreateRowPolicy = Requirement( + name='RQ.SRS-006.RBAC.Privileges.ShowCreateRowPolicy', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL successfully execute `SHOW CREATE ROW POLICY` or `SHOW CREATE POLICY` statement if and only if the user has **show create row policy** privilege,\n' + 'either directly or through a role.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_Privileges_ShowQuotas = Requirement( + name='RQ.SRS-006.RBAC.Privileges.ShowQuotas', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL successfully grant `SHOW QUOTAS` privilege when\n' + 'the user is granted `SHOW QUOTAS`, `SHOW CREATE QUOTA`, `SHOW ACCESS`, or `ACCESS MANAGEMENT`.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_Privileges_ShowQuotas_Query = Requirement( + name='RQ.SRS-006.RBAC.Privileges.ShowQuotas.Query', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL successfully execute `SHOW QUOTAS` statement if and only if the user has **show quotas** privilege,\n' + 'either directly or through a role.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_Privileges_ShowCreateQuota = Requirement( + name='RQ.SRS-006.RBAC.Privileges.ShowCreateQuota', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL successfully execute `SHOW CREATE QUOTA` statement if and only if the user has **show create quota** privilege,\n' + 'either directly or through a role.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_Privileges_ShowSettingsProfiles = Requirement( + name='RQ.SRS-006.RBAC.Privileges.ShowSettingsProfiles', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL successfully grant `SHOW SETTINGS PROFILES` privilege when\n' + 'the user is granted `SHOW SETTINGS PROFILES`, `SHOW PROFILES`, `SHOW CREATE SETTINGS PROFILE`,\n' + '`SHOW SETTINGS PROFILE`, `SHOW ACCESS`, or `ACCESS MANAGEMENT`.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_Privileges_ShowSettingsProfiles_Query = Requirement( + name='RQ.SRS-006.RBAC.Privileges.ShowSettingsProfiles.Query', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL successfully execute `SHOW SETTINGS PROFILES` or `SHOW PROFILES` statement if and only if the user has **show settings profiles** privilege,\n' + 'either directly or through a role.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_Privileges_ShowCreateSettingsProfile = Requirement( + name='RQ.SRS-006.RBAC.Privileges.ShowCreateSettingsProfile', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL successfully execute `SHOW CREATE SETTINGS PROFILE` or `SHOW CREATE PROFILE` statement if and only if the user has **show create settings profile** privilege,\n' + 'either directly or through a role.\n' + '\n' + ), + link=None) + RQ_SRS_006_RBAC_Privileges_GrantOption = Requirement( name='RQ.SRS-006.RBAC.Privileges.GrantOption', version='1.0', diff --git a/tests/testflows/rbac/tests/privileges/alter/alter_quota.py b/tests/testflows/rbac/tests/privileges/alter/alter_quota.py new file mode 100644 index 00000000000..4d0d55c86de --- /dev/null +++ b/tests/testflows/rbac/tests/privileges/alter/alter_quota.py @@ -0,0 +1,137 @@ +from contextlib import contextmanager + +from testflows.core import * +from testflows.asserts import error + +from rbac.requirements import * +from rbac.helper.common import * +import rbac.helper.errors as errors + +@contextmanager +def quota(node, name): + try: + with Given("I have a quota"): + node.query(f"CREATE QUOTA {name}") + + yield + + finally: + with Finally("I drop the quota"): + node.query(f"DROP QUOTA IF EXISTS {name}") + +@TestSuite +def alter_quota_granted_directly(self, node=None): + """Check that a user is able to execute `ALTER QUOTA` with privileges are granted directly. + """ + + user_name = f"user_{getuid()}" + + if node is None: + node = self.context.node + + with user(node, f"{user_name}"): + + Suite(run=alter_quota, flags=TE, + examples=Examples("privilege grant_target_name user_name", [ + tuple(list(row)+[user_name,user_name]) for row in alter_quota.examples + ], args=Args(name="check privilege={privilege}", format_name=True))) + +@TestSuite +def alter_quota_granted_via_role(self, node=None): + """Check that a user is able to execute `ALTER QUOTA` with privileges are granted through a role. + """ + + user_name = f"user_{getuid()}" + role_name = f"role_{getuid()}" + + if node is None: + node = self.context.node + + with user(node, f"{user_name}"), role(node, f"{role_name}"): + + with When("I grant the role to the user"): + node.query(f"GRANT {role_name} TO {user_name}") + + Suite(run=alter_quota, flags=TE, + examples=Examples("privilege grant_target_name user_name", [ + tuple(list(row)+[role_name,user_name]) for row in alter_quota.examples + ], args=Args(name="check privilege={privilege}", format_name=True))) + +@TestOutline(Suite) +@Examples("privilege",[ + ("ACCESS MANAGEMENT",), + ("ALTER QUOTA",), +]) +def alter_quota(self, privilege, grant_target_name, user_name, node=None): + """Check that user is only able to execute `ALTER QUOTA` when they have the necessary privilege. + """ + exitcode, message = errors.not_enough_privileges(name=user_name) + + if node is None: + node = self.context.node + + with Scenario("ALTER QUOTA without privilege"): + alter_quota_name = f"alter_quota_{getuid()}" + + with quota(node, alter_quota_name): + + with When("I check the user can't alter a quota"): + node.query(f"ALTER QUOTA {alter_quota_name}", settings=[("user",user_name)], + exitcode=exitcode, message=message) + + with Scenario("ALTER QUOTA with privilege"): + alter_quota_name = f"alter_quota_{getuid()}" + + with quota(node, alter_quota_name): + + with When(f"I grant {privilege}"): + node.query(f"GRANT {privilege} ON *.* TO {grant_target_name}") + + with Then("I check the user can alter a user"): + node.query(f"ALTER QUOTA {alter_quota_name}", settings = [("user", f"{user_name}")]) + + with Scenario("ALTER QUOTA on cluster"): + alter_quota_name = f"alter_quota_{getuid()}" + + try: + with Given("I have a quota on a cluster"): + node.query(f"CREATE QUOTA {alter_quota_name} ON CLUSTER sharded_cluster") + + with When(f"I grant {privilege}"): + node.query(f"GRANT {privilege} ON *.* TO {grant_target_name}") + + with Then("I check the user can alter a quota"): + node.query(f"ALTER QUOTA {alter_quota_name} ON CLUSTER sharded_cluster", + settings = [("user", f"{user_name}")]) + + finally: + with Finally("I drop the quota"): + node.query(f"DROP QUOTA IF EXISTS {alter_quota_name} ON CLUSTER sharded_cluster") + + with Scenario("ALTER QUOTA with revoked privilege"): + alter_quota_name = f"alter_quota_{getuid()}" + + with quota(node, alter_quota_name): + + with When(f"I grant {privilege} on the database"): + node.query(f"GRANT {privilege} ON *.* TO {grant_target_name}") + + with And(f"I revoke {privilege} on the database"): + node.query(f"REVOKE {privilege} ON *.* FROM {grant_target_name}") + + with Then("I check the user can't alter a quota"): + node.query(f"ALTER QUOTA {alter_quota_name}", settings=[("user",user_name)], + exitcode=exitcode, message=message) + +@TestFeature +@Name("alter quota") +@Requirements( + RQ_SRS_006_RBAC_Privileges_AlterQuota("1.0"), +) +def feature(self, node="clickhouse1"): + """Check the RBAC functionality of ALTER QUOTA. + """ + self.context.node = self.context.cluster.node(node) + + Suite(run=alter_quota_granted_directly, setup=instrument_clickhouse_server_log) + Suite(run=alter_quota_granted_via_role, setup=instrument_clickhouse_server_log) diff --git a/tests/testflows/rbac/tests/privileges/alter/alter_role.py b/tests/testflows/rbac/tests/privileges/alter/alter_role.py new file mode 100644 index 00000000000..cf98d66a689 --- /dev/null +++ b/tests/testflows/rbac/tests/privileges/alter/alter_role.py @@ -0,0 +1,118 @@ +from testflows.core import * +from testflows.asserts import error + +from rbac.requirements import * +from rbac.helper.common import * +import rbac.helper.errors as errors + +@TestSuite +def privileges_granted_directly(self, node=None): + """Check that a user is able to execute `ALTER ROLE` with privileges are granted directly. + """ + + user_name = f"user_{getuid()}" + + if node is None: + node = self.context.node + + with user(node, f"{user_name}"): + + Suite(run=alter_role, flags=TE, + examples=Examples("privilege grant_target_name user_name", [ + tuple(list(row)+[user_name,user_name]) for row in alter_role.examples + ], args=Args(name="privilege={privilege}", format_name=True))) + +@TestSuite +def privileges_granted_via_role(self, node=None): + """Check that a user is able to execute `ALTER ROLE` with privileges are granted through a role. + """ + + user_name = f"user_{getuid()}" + role_name = f"role_{getuid()}" + + if node is None: + node = self.context.node + + with user(node, f"{user_name}"), role(node, f"{role_name}"): + + with When("I grant the role to the user"): + node.query(f"GRANT {role_name} TO {user_name}") + + Suite(run=alter_role, flags=TE, + examples=Examples("privilege grant_target_name user_name", [ + tuple(list(row)+[role_name,user_name]) for row in alter_role.examples + ], args=Args(name="privilege={privilege}", format_name=True))) + +@TestOutline(Suite) +@Examples("privilege",[ + ("ACCESS MANAGEMENT",), + ("ALTER ROLE",), +]) +def alter_role(self, privilege, grant_target_name, user_name, node=None): + """Check that user is only able to execute `ALTER ROLE` when they have the necessary privilege. + """ + exitcode, message = errors.not_enough_privileges(name=user_name) + + if node is None: + node = self.context.node + + with Scenario("ALTER ROLE without privilege"): + alter_role_name = f"alter_role_{getuid()}" + with role(node, alter_role_name): + + with When("I check the user can't alter a role"): + node.query(f"ALTER ROLE {alter_role_name}", settings=[("user",user_name)], + exitcode=exitcode, message=message) + + with Scenario("ALTER ROLE with privilege"): + alter_role_name = f"alter_role_{getuid()}" + with role(node, alter_role_name): + + with When(f"I grant {privilege}"): + node.query(f"GRANT {privilege} ON *.* TO {grant_target_name}") + + with Then("I check the user can alter a role"): + node.query(f"ALTER ROLE {alter_role_name}", settings = [("user", f"{user_name}")]) + + with Scenario("ALTER ROLE on cluster"): + alter_role_name = f"alter_role_{getuid()}" + + try: + with Given("I have a role on a cluster"): + node.query(f"CREATE ROLE {alter_role_name} ON CLUSTER sharded_cluster") + + with When(f"I grant {privilege}"): + node.query(f"GRANT {privilege} ON *.* TO {grant_target_name}") + + with Then("I check the user can alter a role"): + node.query(f"ALTER ROLE {alter_role_name} ON CLUSTER sharded_cluster", settings = [("user", f"{user_name}")]) + + finally: + with Finally("I drop the user"): + node.query(f"DROP ROLE IF EXISTS {alter_role_name} ON CLUSTER sharded_cluster") + + with Scenario("ALTER ROLE with revoked privilege"): + alter_role_name = f"alter_role_{getuid()}" + with role(node, alter_role_name): + with When(f"I grant {privilege} on the database"): + node.query(f"GRANT {privilege} ON *.* TO {grant_target_name}") + + with And(f"I revoke {privilege} on the database"): + node.query(f"REVOKE {privilege} ON *.* FROM {grant_target_name}") + + with Then("I check the user cannot alter a role"): + node.query(f"ALTER ROLE {alter_role_name}", settings=[("user",user_name)], + exitcode=exitcode, message=message) + +@TestFeature +@Name("alter role") +@Requirements( + RQ_SRS_006_RBAC_Privileges_AlterRole("1.0"), +) +def feature(self, node="clickhouse1"): + """Check the RBAC functionality of ALTER ROLE. + """ + self.context.node = self.context.cluster.node(node) + + Suite(run=privileges_granted_directly, setup=instrument_clickhouse_server_log) + Suite(run=privileges_granted_via_role, setup=instrument_clickhouse_server_log) diff --git a/tests/testflows/rbac/tests/privileges/alter/alter_row_policy.py b/tests/testflows/rbac/tests/privileges/alter/alter_row_policy.py new file mode 100644 index 00000000000..6d13d30b823 --- /dev/null +++ b/tests/testflows/rbac/tests/privileges/alter/alter_row_policy.py @@ -0,0 +1,144 @@ +from testflows.core import * +from testflows.asserts import error + +from rbac.requirements import * +from rbac.helper.common import * +import rbac.helper.errors as errors + +@TestSuite +def privileges_granted_directly(self, node=None): + """Check that a user is able to execute `ALTER ROW POLICY` with privileges are granted directly. + """ + + user_name = f"user_{getuid()}" + + if node is None: + node = self.context.node + + with user(node, f"{user_name}"): + + Suite(run=alter_row_policy, flags=TE, + examples=Examples("privilege grant_target_name user_name", [ + tuple(list(row)+[user_name,user_name]) for row in alter_row_policy.examples + ], args=Args(name="privilege={privilege}", format_name=True))) + +@TestSuite +def privileges_granted_via_role(self, node=None): + """Check that a user is able to execute `ALTER ROW POLICY` with privileges are granted through a role. + """ + + user_name = f"user_{getuid()}" + role_name = f"role_{getuid()}" + + if node is None: + node = self.context.node + + with user(node, f"{user_name}"), role(node, f"{role_name}"): + + with When("I grant the role to the user"): + node.query(f"GRANT {role_name} TO {user_name}") + + Suite(run=alter_row_policy, flags=TE, + examples=Examples("privilege grant_target_name user_name", [ + tuple(list(row)+[role_name,user_name]) for row in alter_row_policy.examples + ], args=Args(name="privilege={privilege}", format_name=True))) + +@TestOutline(Suite) +@Examples("privilege",[ + ("ACCESS MANAGEMENT",), + ("ALTER ROW POLICY",), + ("ALTER POLICY",), +]) +def alter_row_policy(self, privilege, grant_target_name, user_name, node=None): + """Check that user is only able to execute `ALTER ROW POLICY` when they have the necessary privilege. + """ + exitcode, message = errors.not_enough_privileges(name=user_name) + + if node is None: + node = self.context.node + + with Scenario("ALTER ROW POLICY without privilege"): + alter_row_policy_name = f"alter_row_policy_{getuid()}" + table_name = f"table_name_{getuid()}" + + try: + with Given("I have a row policy"): + node.query(f"CREATE ROW POLICY {alter_row_policy_name} ON {table_name}") + + with When("I check the user can't alter a row policy"): + node.query(f"ALTER ROW POLICY {alter_row_policy_name} ON {table_name}", settings=[("user",user_name)], + exitcode=exitcode, message=message) + + finally: + with Finally("I drop the row policy"): + node.query(f"DROP ROW POLICY IF EXISTS {alter_row_policy_name} ON {table_name}") + + with Scenario("ALTER ROW POLICY with privilege"): + alter_row_policy_name = f"alter_row_policy_{getuid()}" + table_name = f"table_name_{getuid()}" + + try: + with Given("I have a row policy"): + node.query(f"CREATE ROW POLICY {alter_row_policy_name} ON {table_name}") + + with When(f"I grant {privilege}"): + node.query(f"GRANT {privilege} ON *.* TO {grant_target_name}") + + with Then("I check the user can alter a row policy"): + node.query(f"ALTER ROW POLICY {alter_row_policy_name} ON {table_name}", settings = [("user", f"{user_name}")]) + + finally: + with Finally("I drop the row policy"): + node.query(f"DROP ROW POLICY IF EXISTS {alter_row_policy_name} ON {table_name}") + + with Scenario("ALTER ROW POLICY on cluster"): + alter_row_policy_name = f"alter_row_policy_{getuid()}" + table_name = f"table_name_{getuid()}" + + try: + with Given("I have a row policy on a cluster"): + node.query(f"CREATE ROW POLICY {alter_row_policy_name} ON CLUSTER sharded_cluster ON {table_name}") + + with When(f"I grant {privilege}"): + node.query(f"GRANT {privilege} ON *.* TO {grant_target_name}") + + with Then("I check the user can alter a row policy"): + node.query(f"ALTER ROW POLICY {alter_row_policy_name} ON CLUSTER sharded_cluster ON {table_name}", settings = [("user", f"{user_name}")]) + + finally: + with Finally("I drop the user"): + node.query(f"DROP ROW POLICY IF EXISTS {alter_row_policy_name} ON CLUSTER sharded_cluster ON {table_name}") + + with Scenario("ALTER ROW POLICY with revoked privilege"): + alter_row_policy_name = f"alter_row_policy_{getuid()}" + table_name = f"table_name_{getuid()}" + + try: + with Given("I have a row policy"): + node.query(f"CREATE ROW POLICY {alter_row_policy_name} ON {table_name}") + + with When(f"I grant {privilege} on the database"): + node.query(f"GRANT {privilege} ON *.* TO {grant_target_name}") + + with And(f"I revoke {privilege} on the database"): + node.query(f"REVOKE {privilege} ON *.* FROM {grant_target_name}") + + with Then("I check the user cannot alter row policy"): + node.query(f"ALTER ROW POLICY {alter_row_policy_name} ON {table_name}", settings=[("user",user_name)], + exitcode=exitcode, message=message) + finally: + with Finally("I drop the row policy"): + node.query(f"DROP ROW POLICY IF EXISTS {alter_row_policy_name} ON {table_name}") + +@TestFeature +@Name("alter row policy") +@Requirements( + RQ_SRS_006_RBAC_Privileges_AlterRowPolicy("1.0"), +) +def feature(self, node="clickhouse1"): + """Check the RBAC functionality of ALTER ROW POLICY. + """ + self.context.node = self.context.cluster.node(node) + + Suite(run=privileges_granted_directly, setup=instrument_clickhouse_server_log) + Suite(run=privileges_granted_via_role, setup=instrument_clickhouse_server_log) diff --git a/tests/testflows/rbac/tests/privileges/alter/alter_settings_profile.py b/tests/testflows/rbac/tests/privileges/alter/alter_settings_profile.py new file mode 100644 index 00000000000..9212b745544 --- /dev/null +++ b/tests/testflows/rbac/tests/privileges/alter/alter_settings_profile.py @@ -0,0 +1,138 @@ +from contextlib import contextmanager + +from testflows.core import * +from testflows.asserts import error + +from rbac.requirements import * +from rbac.helper.common import * +import rbac.helper.errors as errors + +@contextmanager +def settings_profile(node, name): + try: + with Given("I have a settings_profile"): + node.query(f"CREATE SETTINGS PROFILE {name}") + + yield + + finally: + with Finally("I drop the settings_profile"): + node.query(f"DROP SETTINGS PROFILE IF EXISTS {name}") + +@TestSuite +def alter_settings_profile_granted_directly(self, node=None): + """Check that a user is able to execute `ALTER SETTINGS PROFILE` with privileges are granted directly. + """ + + user_name = f"user_{getuid()}" + + if node is None: + node = self.context.node + + with user(node, f"{user_name}"): + + Suite(run=alter_settings_profile, flags=TE, + examples=Examples("privilege grant_target_name user_name", [ + tuple(list(row)+[user_name,user_name]) for row in alter_settings_profile.examples + ], args=Args(name="check privilege={privilege}", format_name=True))) + +@TestSuite +def alter_settings_profile_granted_via_role(self, node=None): + """Check that a user is able to execute `ALTER SETTINGS PROFILE` with privileges are granted through a role. + """ + + user_name = f"user_{getuid()}" + role_name = f"role_{getuid()}" + + if node is None: + node = self.context.node + + with user(node, f"{user_name}"), role(node, f"{role_name}"): + + with When("I grant the role to the user"): + node.query(f"GRANT {role_name} TO {user_name}") + + Suite(run=alter_settings_profile, flags=TE, + examples=Examples("privilege grant_target_name user_name", [ + tuple(list(row)+[role_name,user_name]) for row in alter_settings_profile.examples + ], args=Args(name="check privilege={privilege}", format_name=True))) + +@TestOutline(Suite) +@Examples("privilege",[ + ("ACCESS MANAGEMENT",), + ("ALTER SETTINGS PROFILE",), + ("ALTER PROFILE",), +]) +def alter_settings_profile(self, privilege, grant_target_name, user_name, node=None): + """Check that user is only able to execute `ALTER SETTINGS PROFILE` when they have the necessary privilege. + """ + exitcode, message = errors.not_enough_privileges(name=user_name) + + if node is None: + node = self.context.node + + with Scenario("ALTER SETTINGS PROFILE without privilege"): + alter_settings_profile_name = f"alter_settings_profile_{getuid()}" + + with settings_profile(node, alter_settings_profile_name): + + with When("I check the user can't alter a settings_profile"): + node.query(f"ALTER SETTINGS PROFILE {alter_settings_profile_name}", settings=[("user",user_name)], + exitcode=exitcode, message=message) + + with Scenario("ALTER SETTINGS PROFILE with privilege"): + alter_settings_profile_name = f"alter_settings_profile_{getuid()}" + + with settings_profile(node, alter_settings_profile_name): + + with When(f"I grant {privilege}"): + node.query(f"GRANT {privilege} ON *.* TO {grant_target_name}") + + with Then("I check the user can alter a user"): + node.query(f"ALTER SETTINGS PROFILE {alter_settings_profile_name}", settings = [("user", f"{user_name}")]) + + with Scenario("ALTER SETTINGS PROFILE on cluster"): + alter_settings_profile_name = f"alter_settings_profile_{getuid()}" + + try: + with Given("I have a settings_profile on a cluster"): + node.query(f"CREATE SETTINGS PROFILE {alter_settings_profile_name} ON CLUSTER sharded_cluster") + + with When(f"I grant {privilege}"): + node.query(f"GRANT {privilege} ON *.* TO {grant_target_name}") + + with Then("I check the user can alter a settings_profile"): + node.query(f"ALTER SETTINGS PROFILE {alter_settings_profile_name} ON CLUSTER sharded_cluster", + settings = [("user", f"{user_name}")]) + + finally: + with Finally("I drop the settings_profile"): + node.query(f"DROP SETTINGS PROFILE IF EXISTS {alter_settings_profile_name} ON CLUSTER sharded_cluster") + + with Scenario("ALTER SETTINGS PROFILE with revoked privilege"): + alter_settings_profile_name = f"alter_settings_profile_{getuid()}" + + with settings_profile(node, alter_settings_profile_name): + + with When(f"I grant {privilege} on the database"): + node.query(f"GRANT {privilege} ON *.* TO {grant_target_name}") + + with And(f"I revoke {privilege} on the database"): + node.query(f"REVOKE {privilege} ON *.* FROM {grant_target_name}") + + with Then("I check the user can't alter a settings_profile"): + node.query(f"ALTER SETTINGS PROFILE {alter_settings_profile_name}", settings=[("user",user_name)], + exitcode=exitcode, message=message) + +@TestFeature +@Name("alter settings profile") +@Requirements( + RQ_SRS_006_RBAC_Privileges_AlterSettingsProfile("1.0"), +) +def feature(self, node="clickhouse1"): + """Check the RBAC functionality of ALTER SETTINGS PROFILE. + """ + self.context.node = self.context.cluster.node(node) + + Suite(run=alter_settings_profile_granted_directly, setup=instrument_clickhouse_server_log) + Suite(run=alter_settings_profile_granted_via_role, setup=instrument_clickhouse_server_log) diff --git a/tests/testflows/rbac/tests/privileges/alter/alter_user.py b/tests/testflows/rbac/tests/privileges/alter/alter_user.py new file mode 100644 index 00000000000..2531f4a3451 --- /dev/null +++ b/tests/testflows/rbac/tests/privileges/alter/alter_user.py @@ -0,0 +1,118 @@ +from testflows.core import * +from testflows.asserts import error + +from rbac.requirements import * +from rbac.helper.common import * +import rbac.helper.errors as errors + +@TestSuite +def alter_user_granted_directly(self, node=None): + """Check that a user is able to execute `ALTER USER` with privileges are granted directly. + """ + + user_name = f"user_{getuid()}" + + if node is None: + node = self.context.node + + with user(node, f"{user_name}"): + + Suite(run=alter_user, flags=TE, + examples=Examples("privilege grant_target_name user_name", [ + tuple(list(row)+[user_name,user_name]) for row in alter_user.examples + ], args=Args(name="check privilege={privilege}", format_name=True))) + +@TestSuite +def alter_user_granted_via_role(self, node=None): + """Check that a user is able to execute `ALTER USER` with privileges are granted through a role. + """ + + user_name = f"user_{getuid()}" + role_name = f"role_{getuid()}" + + if node is None: + node = self.context.node + + with user(node, f"{user_name}"), role(node, f"{role_name}"): + + with When("I grant the role to the user"): + node.query(f"GRANT {role_name} TO {user_name}") + + Suite(run=alter_user, flags=TE, + examples=Examples("privilege grant_target_name user_name", [ + tuple(list(row)+[role_name,user_name]) for row in alter_user.examples + ], args=Args(name="check privilege={privilege}", format_name=True))) + +@TestOutline(Suite) +@Examples("privilege",[ + ("ACCESS MANAGEMENT",), + ("ALTER USER",), +]) +def alter_user(self, privilege, grant_target_name, user_name, node=None): + """Check that user is only able to execute `ALTER USER` when they have the necessary privilege. + """ + exitcode, message = errors.not_enough_privileges(name=user_name) + + if node is None: + node = self.context.node + + with Scenario("ALTER USER without privilege"): + + alter_user_name = f"alter_user_{getuid()}" + with user(node, alter_user_name): + + with When("I check the user can't alter a user"): + node.query(f"ALTER USER {alter_user_name}", settings=[("user",user_name)], + exitcode=exitcode, message=message) + + with Scenario("ALTER USER with privilege"): + alter_user_name = f"alter_user_{getuid()}" + with user(node, alter_user_name): + with When(f"I grant {privilege}"): + node.query(f"GRANT {privilege} ON *.* TO {grant_target_name}") + + with Then("I check the user can alter a user"): + node.query(f"ALTER USER {alter_user_name}", settings = [("user", f"{user_name}")]) + + with Scenario("ALTER USER on cluster"): + alter_user_name = f"alter_user_{getuid()}" + try: + with Given("I have a user on a cluster"): + node.query(f"CREATE USER {alter_user_name} ON CLUSTER sharded_cluster") + + with When(f"I grant {privilege}"): + node.query(f"GRANT {privilege} ON *.* TO {grant_target_name}") + + with Then("I check the user can alter a user"): + node.query(f"ALTER USER {alter_user_name} ON CLUSTER sharded_cluster", + settings = [("user", f"{user_name}")]) + + finally: + with Finally("I drop the user"): + node.query(f"DROP USER IF EXISTS {alter_user_name} ON CLUSTER sharded_cluster") + + with Scenario("ALTER USER with revoked privilege"): + alter_user_name = f"alter_user_{getuid()}" + with user(node, alter_user_name): + with When(f"I grant {privilege} on the database"): + node.query(f"GRANT {privilege} ON *.* TO {grant_target_name}") + + with And(f"I revoke {privilege} on the database"): + node.query(f"REVOKE {privilege} ON *.* FROM {grant_target_name}") + + with Then("I check the user can't alter a user"): + node.query(f"ALTER USER {alter_user_name}", settings=[("user",user_name)], + exitcode=exitcode, message=message) + +@TestFeature +@Name("alter user") +@Requirements( + RQ_SRS_006_RBAC_Privileges_AlterUser("1.0"), +) +def feature(self, node="clickhouse1"): + """Check the RBAC functionality of ALTER USER. + """ + self.context.node = self.context.cluster.node(node) + + Suite(run=alter_user_granted_directly, setup=instrument_clickhouse_server_log) + Suite(run=alter_user_granted_via_role, setup=instrument_clickhouse_server_log) diff --git a/tests/testflows/rbac/tests/privileges/create/create_quota.py b/tests/testflows/rbac/tests/privileges/create/create_quota.py new file mode 100644 index 00000000000..e7f4f4d5f7c --- /dev/null +++ b/tests/testflows/rbac/tests/privileges/create/create_quota.py @@ -0,0 +1,128 @@ +from testflows.core import * +from testflows.asserts import error + +from rbac.requirements import * +from rbac.helper.common import * +import rbac.helper.errors as errors + +@TestSuite +def privileges_granted_directly(self, node=None): + """Check that a user is able to execute `CREATE QUOTA` with privileges are granted directly. + """ + + user_name = f"user_{getuid()}" + + if node is None: + node = self.context.node + + with user(node, f"{user_name}"): + + Suite(run=create_quota, flags=TE, + examples=Examples("privilege grant_target_name user_name", [ + tuple(list(row)+[user_name,user_name]) for row in create_quota.examples + ], args=Args(name="privilege={privilege}", format_name=True))) + +@TestSuite +def privileges_granted_via_role(self, node=None): + """Check that a user is able to execute `CREATE QUOTA` with privileges are granted through a role. + """ + + user_name = f"user_{getuid()}" + role_name = f"role_{getuid()}" + + if node is None: + node = self.context.node + + with user(node, f"{user_name}"), role(node, f"{role_name}"): + + with When("I grant the role to the user"): + node.query(f"GRANT {role_name} TO {user_name}") + + Suite(run=create_quota, flags=TE, + examples=Examples("privilege grant_target_name user_name", [ + tuple(list(row)+[role_name,user_name]) for row in create_quota.examples + ], args=Args(name="privilege={privilege}", format_name=True))) + +@TestOutline(Suite) +@Examples("privilege",[ + ("ACCESS MANAGEMENT",), + ("CREATE QUOTA",), +]) +def create_quota(self, privilege, grant_target_name, user_name, node=None): + """Check that user is only able to execute `CREATE QUOTA` when they have the necessary privilege. + """ + exitcode, message = errors.not_enough_privileges(name=user_name) + + if node is None: + node = self.context.node + + with Scenario("CREATE QUOTA without privilege"): + create_quota_name = f"create_quota_{getuid()}" + + try: + with When("I check the user can't create a quota"): + node.query(f"CREATE QUOTA {create_quota_name}", settings=[("user",user_name)], + exitcode=exitcode, message=message) + + finally: + with Finally("I drop the quota"): + node.query(f"DROP QUOTA IF EXISTS {create_quota_name}") + + with Scenario("CREATE QUOTA with privilege"): + create_quota_name = f"create_quota_{getuid()}" + + try: + with When(f"I grant {privilege}"): + node.query(f"GRANT {privilege} ON *.* TO {grant_target_name}") + + with Then("I check the user can create a quota"): + node.query(f"CREATE QUOTA {create_quota_name}", settings = [("user", f"{user_name}")]) + + finally: + with Finally("I drop the quota"): + node.query(f"DROP QUOTA IF EXISTS {create_quota_name}") + + with Scenario("CREATE QUOTA on cluster"): + create_quota_name = f"create_quota_{getuid()}" + + try: + with When(f"I grant {privilege}"): + node.query(f"GRANT {privilege} ON *.* TO {grant_target_name}") + + with Then("I check the user can create a quota"): + node.query(f"CREATE QUOTA {create_quota_name} ON CLUSTER sharded_cluster", settings = [("user", f"{user_name}")]) + + finally: + with Finally("I drop the quota"): + node.query(f"DROP QUOTA IF EXISTS {create_quota_name} ON CLUSTER sharded_cluster") + + with Scenario("CREATE QUOTA with revoked privilege"): + create_quota_name = f"create_quota_{getuid()}" + + try: + with When(f"I grant {privilege}"): + node.query(f"GRANT {privilege} ON *.* TO {grant_target_name}") + + with And(f"I revoke {privilege}"): + node.query(f"REVOKE {privilege} ON *.* FROM {grant_target_name}") + + with Then("I check the user cannot create a quota"): + node.query(f"CREATE QUOTA {create_quota_name}", settings=[("user",user_name)], + exitcode=exitcode, message=message) + + finally: + with Finally("I drop the quota"): + node.query(f"DROP QUOTA IF EXISTS {create_quota_name}") + +@TestFeature +@Name("create quota") +@Requirements( + RQ_SRS_006_RBAC_Privileges_CreateQuota("1.0"), +) +def feature(self, node="clickhouse1"): + """Check the RBAC functionality of CREATE QUOTA. + """ + self.context.node = self.context.cluster.node(node) + + Suite(run=privileges_granted_directly, setup=instrument_clickhouse_server_log) + Suite(run=privileges_granted_via_role, setup=instrument_clickhouse_server_log) diff --git a/tests/testflows/rbac/tests/privileges/create/create_role.py b/tests/testflows/rbac/tests/privileges/create/create_role.py new file mode 100644 index 00000000000..9d8af913893 --- /dev/null +++ b/tests/testflows/rbac/tests/privileges/create/create_role.py @@ -0,0 +1,128 @@ +from testflows.core import * +from testflows.asserts import error + +from rbac.requirements import * +from rbac.helper.common import * +import rbac.helper.errors as errors + +@TestSuite +def privileges_granted_directly(self, node=None): + """Check that a user is able to execute `CREATE ROLE` with privileges are granted directly. + """ + + user_name = f"user_{getuid()}" + + if node is None: + node = self.context.node + + with user(node, f"{user_name}"): + + Suite(run=create_role, flags=TE, + examples=Examples("privilege grant_target_name user_name", [ + tuple(list(row)+[user_name,user_name]) for row in create_role.examples + ], args=Args(name="privilege={privilege}", format_name=True))) + +@TestSuite +def privileges_granted_via_role(self, node=None): + """Check that a user is able to execute `CREATE ROLE` with privileges are granted through a role. + """ + + user_name = f"user_{getuid()}" + role_name = f"role_{getuid()}" + + if node is None: + node = self.context.node + + with user(node, f"{user_name}"), role(node, f"{role_name}"): + + with When("I grant the role to the user"): + node.query(f"GRANT {role_name} TO {user_name}") + + Suite(run=create_role, flags=TE, + examples=Examples("privilege grant_target_name user_name", [ + tuple(list(row)+[role_name,user_name]) for row in create_role.examples + ], args=Args(name="privilege={privilege}", format_name=True))) + +@TestOutline(Suite) +@Examples("privilege",[ + ("ACCESS MANAGEMENT",), + ("CREATE ROLE",), +]) +def create_role(self, privilege, grant_target_name, user_name, node=None): + """Check that user is only able to execute `CREATE ROLE` when they have the necessary privilege. + """ + exitcode, message = errors.not_enough_privileges(name=user_name) + + if node is None: + node = self.context.node + + with Scenario("CREATE ROLE without privilege"): + create_role_name = f"create_role_{getuid()}" + + try: + with When("I check the user can't create a role"): + node.query(f"CREATE ROLE {create_role_name}", settings=[("user",user_name)], + exitcode=exitcode, message=message) + + finally: + with Finally("I drop the role"): + node.query(f"DROP ROLE IF EXISTS {create_role_name}") + + with Scenario("CREATE ROLE with privilege"): + create_role_name = f"create_role_{getuid()}" + + try: + with When(f"I grant {privilege}"): + node.query(f"GRANT {privilege} ON *.* TO {grant_target_name}") + + with Then("I check the user can create a role"): + node.query(f"CREATE ROLE {create_role_name}", settings = [("user", f"{user_name}")]) + + finally: + with Finally("I drop the role"): + node.query(f"DROP ROLE IF EXISTS {create_role_name}") + + with Scenario("CREATE ROLE on cluster"): + create_role_name = f"create_role_{getuid()}" + + try: + with When(f"I grant {privilege}"): + node.query(f"GRANT {privilege} ON *.* TO {grant_target_name}") + + with Then("I check the user can create a role"): + node.query(f"CREATE ROLE {create_role_name} ON CLUSTER sharded_cluster", settings = [("user", f"{user_name}")]) + + finally: + with Finally("I drop the role"): + node.query(f"DROP ROLE IF EXISTS {create_role_name} ON CLUSTER sharded_cluster") + + with Scenario("CREATE ROLE with revoked privilege"): + create_role_name = f"create_role_{getuid()}" + + try: + with When(f"I grant {privilege}"): + node.query(f"GRANT {privilege} ON *.* TO {grant_target_name}") + + with And(f"I revoke {privilege}"): + node.query(f"REVOKE {privilege} ON *.* FROM {grant_target_name}") + + with Then("I check the user cannot create a role"): + node.query(f"CREATE ROLE {create_role_name}", settings=[("user",user_name)], + exitcode=exitcode, message=message) + + finally: + with Finally("I drop the role"): + node.query(f"DROP ROLE IF EXISTS {create_role_name}") + +@TestFeature +@Name("create role") +@Requirements( + RQ_SRS_006_RBAC_Privileges_CreateRole("1.0"), +) +def feature(self, node="clickhouse1"): + """Check the RBAC functionality of CREATE ROLE. + """ + self.context.node = self.context.cluster.node(node) + + Suite(run=privileges_granted_directly, setup=instrument_clickhouse_server_log) + Suite(run=privileges_granted_via_role, setup=instrument_clickhouse_server_log) diff --git a/tests/testflows/rbac/tests/privileges/create/create_row_policy.py b/tests/testflows/rbac/tests/privileges/create/create_row_policy.py new file mode 100644 index 00000000000..040cc631cc3 --- /dev/null +++ b/tests/testflows/rbac/tests/privileges/create/create_row_policy.py @@ -0,0 +1,133 @@ +from testflows.core import * +from testflows.asserts import error + +from rbac.requirements import * +from rbac.helper.common import * +import rbac.helper.errors as errors + +@TestSuite +def privileges_granted_directly(self, node=None): + """Check that a user is able to execute `CREATE ROW POLICY` with privileges are granted directly. + """ + + user_name = f"user_{getuid()}" + + if node is None: + node = self.context.node + + with user(node, f"{user_name}"): + + Suite(run=create_row_policy, flags=TE, + examples=Examples("privilege grant_target_name user_name", [ + tuple(list(row)+[user_name,user_name]) for row in create_row_policy.examples + ], args=Args(name="privilege={privilege}", format_name=True))) + +@TestSuite +def privileges_granted_via_role(self, node=None): + """Check that a user is able to execute `CREATE ROW POLICY` with privileges are granted through a role. + """ + + user_name = f"user_{getuid()}" + role_name = f"role_{getuid()}" + + if node is None: + node = self.context.node + + with user(node, f"{user_name}"), role(node, f"{role_name}"): + + with When("I grant the role to the user"): + node.query(f"GRANT {role_name} TO {user_name}") + + Suite(run=create_row_policy, flags=TE, + examples=Examples("privilege grant_target_name user_name", [ + tuple(list(row)+[role_name,user_name]) for row in create_row_policy.examples + ], args=Args(name="privilege={privilege}", format_name=True))) + +@TestOutline(Suite) +@Examples("privilege",[ + ("ACCESS MANAGEMENT",), + ("CREATE ROW POLICY",), + ("CREATE POLICY",), +]) +def create_row_policy(self, privilege, grant_target_name, user_name, node=None): + """Check that user is only able to execute `CREATE ROW POLICY` when they have the necessary privilege. + """ + exitcode, message = errors.not_enough_privileges(name=user_name) + + if node is None: + node = self.context.node + + with Scenario("CREATE ROW POLICY without privilege"): + create_row_policy_name = f"create_row_policy_{getuid()}" + table_name = f"table_name_{getuid()}" + + try: + with When("I check the user can't create a row policy"): + node.query(f"CREATE ROW POLICY {create_row_policy_name} ON {table_name}", settings=[("user",user_name)], + exitcode=exitcode, message=message) + + finally: + with Finally("I drop the row policy"): + node.query(f"DROP ROW POLICY IF EXISTS {create_row_policy_name} ON {table_name}") + + with Scenario("CREATE ROW POLICY with privilege"): + create_row_policy_name = f"create_row_policy_{getuid()}" + table_name = f"table_name_{getuid()}" + + try: + with When(f"I grant {privilege}"): + node.query(f"GRANT {privilege} ON *.* TO {grant_target_name}") + + with Then("I check the user can create a row policy"): + node.query(f"CREATE ROW POLICY {create_row_policy_name} ON {table_name}", settings = [("user", f"{user_name}")]) + + finally: + with Finally("I drop the row policy"): + node.query(f"DROP ROW POLICY IF EXISTS {create_row_policy_name} ON {table_name}") + + with Scenario("CREATE ROW POLICY on cluster"): + create_row_policy_name = f"create_row_policy_{getuid()}" + table_name = f"table_name_{getuid()}" + + try: + with When(f"I grant {privilege}"): + node.query(f"GRANT {privilege} ON *.* TO {grant_target_name}") + + with Then("I check the user can create a row policy"): + node.query(f"CREATE ROW POLICY {create_row_policy_name} ON CLUSTER sharded_cluster ON {table_name}", settings = [("user", f"{user_name}")]) + + finally: + with Finally("I drop the row policy"): + node.query(f"DROP ROW POLICY IF EXISTS {create_row_policy_name} ON CLUSTER sharded_cluster ON {table_name}") + + with Scenario("CREATE ROW POLICY with revoked privilege"): + create_row_policy_name = f"create_row_policy_{getuid()}" + table_name = f"table_name_{getuid()}" + + try: + with When(f"I grant {privilege}"): + node.query(f"GRANT {privilege} ON *.* TO {grant_target_name}") + + with And(f"I revoke {privilege}"): + node.query(f"REVOKE {privilege} ON *.* FROM {grant_target_name}") + + with Then("I check the user cannot create a row policy"): + node.query(f"CREATE ROW POLICY {create_row_policy_name} ON {table_name}", settings=[("user",user_name)], + exitcode=exitcode, message=message) + + finally: + with Finally("I drop the row policy"): + node.query(f"DROP ROW POLICY IF EXISTS {create_row_policy_name} ON {table_name}") + +@TestFeature +@Name("create row policy") +@Requirements( + RQ_SRS_006_RBAC_Privileges_CreateRowPolicy("1.0"), +) +def feature(self, node="clickhouse1"): + """Check the RBAC functionality of CREATE ROW POLICY. + """ + self.context.node = self.context.cluster.node(node) + + Suite(run=privileges_granted_directly, setup=instrument_clickhouse_server_log) + Suite(run=privileges_granted_via_role, setup=instrument_clickhouse_server_log) diff --git a/tests/testflows/rbac/tests/privileges/create/create_settings_profile.py b/tests/testflows/rbac/tests/privileges/create/create_settings_profile.py new file mode 100644 index 00000000000..8b206564647 --- /dev/null +++ b/tests/testflows/rbac/tests/privileges/create/create_settings_profile.py @@ -0,0 +1,129 @@ +from testflows.core import * +from testflows.asserts import error + +from rbac.requirements import * +from rbac.helper.common import * +import rbac.helper.errors as errors + +@TestSuite +def privileges_granted_directly(self, node=None): + """Check that a user is able to execute `CREATE SETTINGS PROFILE` with privileges are granted directly. + """ + + user_name = f"user_{getuid()}" + + if node is None: + node = self.context.node + + with user(node, f"{user_name}"): + + Suite(run=create_settings_profile, flags=TE, + examples=Examples("privilege grant_target_name user_name", [ + tuple(list(row)+[user_name,user_name]) for row in create_settings_profile.examples + ], args=Args(name="privilege={privilege}", format_name=True))) + +@TestSuite +def privileges_granted_via_role(self, node=None): + """Check that a user is able to execute `CREATE SETTINGS PROFILE` with privileges are granted through a role. + """ + + user_name = f"user_{getuid()}" + role_name = f"role_{getuid()}" + + if node is None: + node = self.context.node + + with user(node, f"{user_name}"), role(node, f"{role_name}"): + + with When("I grant the role to the user"): + node.query(f"GRANT {role_name} TO {user_name}") + + Suite(run=create_settings_profile, flags=TE, + examples=Examples("privilege grant_target_name user_name", [ + tuple(list(row)+[role_name,user_name]) for row in create_settings_profile.examples + ], args=Args(name="privilege={privilege}", format_name=True))) + +@TestOutline(Suite) +@Examples("privilege",[ + ("ACCESS MANAGEMENT",), + ("CREATE SETTINGS PROFILE",), + ("CREATE PROFILE",), +]) +def create_settings_profile(self, privilege, grant_target_name, user_name, node=None): + """Check that user is only able to execute `CREATE SETTINGS PROFILE` when they have the necessary privilege. + """ + exitcode, message = errors.not_enough_privileges(name=user_name) + + if node is None: + node = self.context.node + + with Scenario("CREATE SETTINGS PROFILE without privilege"): + create_settings_profile_name = f"create_settings_profile_{getuid()}" + + try: + with When("I check the user can't create a settings_profile"): + node.query(f"CREATE SETTINGS PROFILE {create_settings_profile_name}", settings=[("user",user_name)], + exitcode=exitcode, message=message) + + finally: + with Finally("I drop the settings_profile"): + node.query(f"DROP SETTINGS PROFILE IF EXISTS {create_settings_profile_name}") + + with Scenario("CREATE SETTINGS PROFILE with privilege"): + create_settings_profile_name = f"create_settings_profile_{getuid()}" + + try: + with When(f"I grant {privilege}"): + node.query(f"GRANT {privilege} ON *.* TO {grant_target_name}") + + with Then("I check the user can create a settings_profile"): + node.query(f"CREATE SETTINGS PROFILE {create_settings_profile_name}", settings = [("user", f"{user_name}")]) + + finally: + with Finally("I drop the settings_profile"): + node.query(f"DROP SETTINGS PROFILE IF EXISTS {create_settings_profile_name}") + + with Scenario("CREATE SETTINGS PROFILE on cluster"): + create_settings_profile_name = f"create_settings_profile_{getuid()}" + + try: + with When(f"I grant {privilege}"): + node.query(f"GRANT {privilege} ON *.* TO {grant_target_name}") + + with Then("I check the user can create a settings_profile"): + node.query(f"CREATE SETTINGS PROFILE {create_settings_profile_name} ON CLUSTER sharded_cluster", settings = [("user", f"{user_name}")]) + + finally: + with Finally("I drop the settings_profile"): + node.query(f"DROP SETTINGS PROFILE IF EXISTS {create_settings_profile_name} ON CLUSTER sharded_cluster") + + with Scenario("CREATE SETTINGS PROFILE with revoked privilege"): + create_settings_profile_name = f"create_settings_profile_{getuid()}" + + try: + with When(f"I grant {privilege}"): + node.query(f"GRANT {privilege} ON *.* TO {grant_target_name}") + + with And(f"I revoke {privilege}"): + node.query(f"REVOKE {privilege} ON *.* FROM {grant_target_name}") + + with Then("I check the user cannot create a settings_profile"): + node.query(f"CREATE SETTINGS PROFILE {create_settings_profile_name}", settings=[("user",user_name)], + exitcode=exitcode, message=message) + + finally: + with Finally("I drop the settings_profile"): + node.query(f"DROP SETTINGS PROFILE IF EXISTS {create_settings_profile_name}") + +@TestFeature +@Name("create settings profile") +@Requirements( + RQ_SRS_006_RBAC_Privileges_CreateSettingsProfile("1.0"), +) +def feature(self, node="clickhouse1"): + """Check the RBAC functionality of CREATE SETTINGS PROFILE. + """ + self.context.node = self.context.cluster.node(node) + + Suite(run=privileges_granted_directly, setup=instrument_clickhouse_server_log) + Suite(run=privileges_granted_via_role, setup=instrument_clickhouse_server_log) diff --git a/tests/testflows/rbac/tests/privileges/create/create_user.py b/tests/testflows/rbac/tests/privileges/create/create_user.py new file mode 100644 index 00000000000..02fe238b618 --- /dev/null +++ b/tests/testflows/rbac/tests/privileges/create/create_user.py @@ -0,0 +1,276 @@ +from testflows.core import * +from testflows.asserts import error + +from rbac.requirements import * +from rbac.helper.common import * +import rbac.helper.errors as errors + +@TestSuite +def create_user_granted_directly(self, node=None): + """Check that a user is able to execute `CREATE USER` with privileges are granted directly. + """ + + user_name = f"user_{getuid()}" + + if node is None: + node = self.context.node + + with user(node, f"{user_name}"): + + Suite(run=create_user, flags=TE, + examples=Examples("privilege grant_target_name user_name", [ + tuple(list(row)+[user_name,user_name]) for row in create_user.examples + ], args=Args(name="check privilege={privilege}", format_name=True))) + +@TestSuite +def create_user_granted_via_role(self, node=None): + """Check that a user is able to execute `CREATE USER` with privileges are granted through a role. + """ + + user_name = f"user_{getuid()}" + role_name = f"role_{getuid()}" + + if node is None: + node = self.context.node + + with user(node, f"{user_name}"), role(node, f"{role_name}"): + + with When("I grant the role to the user"): + node.query(f"GRANT {role_name} TO {user_name}") + + Suite(run=create_user, flags=TE, + examples=Examples("privilege grant_target_name user_name", [ + tuple(list(row)+[role_name,user_name]) for row in create_user.examples + ], args=Args(name="check privilege={privilege}", format_name=True))) + +@TestOutline(Suite) +@Examples("privilege",[ + ("ACCESS MANAGEMENT",), + ("CREATE USER",), +]) +def create_user(self, privilege, grant_target_name, user_name, node=None): + """Check that user is only able to execute `CREATE USER` when they have the necessary privilege. + """ + exitcode, message = errors.not_enough_privileges(name=user_name) + + if node is None: + node = self.context.node + + with Scenario("CREATE USER without privilege"): + create_user_name = f"create_user_{getuid()}" + + try: + with When("I check the user can't create a user"): + node.query(f"CREATE USER {create_user_name}", settings=[("user",user_name)], + exitcode=exitcode, message=message) + + finally: + with Finally("I drop the user"): + node.query(f"DROP USER IF EXISTS {create_user_name}") + + with Scenario("CREATE USER with privilege"): + create_user_name = f"create_user_{getuid()}" + + try: + with When(f"I grant {privilege}"): + node.query(f"GRANT {privilege} ON *.* TO {grant_target_name}") + + with Then("I check the user can create a user"): + node.query(f"CREATE USER {create_user_name}", settings = [("user", f"{user_name}")]) + + finally: + with Finally("I drop the user"): + node.query(f"DROP USER IF EXISTS {create_user_name}") + + with Scenario("CREATE USER on cluster"): + create_user_name = f"create_user_{getuid()}" + + try: + with When(f"I grant {privilege}"): + node.query(f"GRANT {privilege} ON *.* TO {grant_target_name}") + + with Then("I check the user can create a user"): + node.query(f"CREATE USER {create_user_name} ON CLUSTER sharded_cluster", + settings = [("user", f"{user_name}")]) + + finally: + with Finally("I drop the user"): + node.query(f"DROP USER IF EXISTS {create_user_name} ON CLUSTER sharded_cluster") + + with Scenario("CREATE USER with revoked privilege"): + create_user_name = f"create_user_{getuid()}" + + try: + with When(f"I grant {privilege}"): + node.query(f"GRANT {privilege} ON *.* TO {grant_target_name}") + + with And(f"I revoke {privilege}"): + node.query(f"REVOKE {privilege} ON *.* FROM {grant_target_name}") + + with Then("I check the user can't create a user"): + node.query(f"CREATE USER {create_user_name}", settings=[("user",user_name)], + exitcode=exitcode, message=message) + + finally: + with Finally("I drop the user"): + node.query(f"DROP USER IF EXISTS {create_user_name}") + +@TestSuite +def default_role_granted_directly(self, node=None): + """Check that a user is able to execute `CREATE USER` with `DEFAULT ROLE` with privileges are granted directly. + """ + + user_name = f"user_{getuid()}" + + if node is None: + node = self.context.node + + with user(node, f"{user_name}"): + + Suite(test=default_role, flags=TE)(grant_target_name=user_name, user_name=user_name) + +@TestSuite +def default_role_granted_via_role(self, node=None): + """Check that a user is able to execute `CREATE USER` with `DEFAULT ROLE` with privileges are granted through a role. + """ + + user_name = f"user_{getuid()}" + role_name = f"role_{getuid()}" + + if node is None: + node = self.context.node + + with user(node, f"{user_name}"), role(node, f"{role_name}"): + + with When("I grant the role to the user"): + node.query(f"GRANT {role_name} TO {user_name}") + + Suite(test=default_role, flags=TE)(grant_target_name=role_name, user_name=user_name) + +@TestSuite +@Requirements( + RQ_SRS_006_RBAC_Privileges_CreateUser_DefaultRole("1.0"), +) +def default_role(self, grant_target_name, user_name, node=None): + """Check that user is only able to execute `CREATE USER` with `DEFAULT ROLE` if and only if the user has + `CREATE USER` privilege and the role with `ADMIN OPTION`. + """ + exitcode, message = errors.not_enough_privileges(name=user_name) + + if node is None: + node = self.context.node + + with Scenario("CREATE USER with DEFAULT ROLE without privilege"): + create_user_name = f"create_user_{getuid()}" + default_role_name = f"default_role_{getuid()}" + + with role(node, default_role_name): + try: + with When(f"I grant CREATE USER"): + node.query(f"GRANT CREATE USER ON *.* TO {grant_target_name}") + + with Then("I check the user can't create a user"): + node.query(f"CREATE USER {create_user_name} DEFAULT ROLE {default_role_name}", settings=[("user",user_name)], + exitcode=exitcode, message=message) + + finally: + with Finally("I drop the user"): + node.query(f"DROP USER IF EXISTS {create_user_name}") + + with Scenario("CREATE USER with DEFAULT ROLE with role privilege"): + create_user_name = f"create_user_{getuid()}" + default_role_name = f"default_role_{getuid()}" + + with role(node, default_role_name): + try: + with When(f"I grant CREATE USER"): + node.query(f"GRANT CREATE USER ON *.* TO {grant_target_name}") + + with And(f"I grant the role with ADMIN OPTION"): + node.query(f"GRANT {default_role_name} TO {grant_target_name} WITH ADMIN OPTION") + + with Then("I check the user can create a user"): + node.query(f"CREATE USER {create_user_name} DEFAULT ROLE {default_role_name}", settings=[("user",user_name)]) + + finally: + with Finally("I drop the user"): + node.query(f"DROP USER IF EXISTS {create_user_name}") + + with Scenario("CREATE USER with DEFAULT ROLE on cluster"): + create_user_name = f"create_user_{getuid()}" + default_role_name = f"default_role_{getuid()}" + + try: + with Given("I have role on a cluster"): + node.query(f"CREATE ROLE {default_role_name} ON CLUSTER sharded_cluster") + + with When(f"I grant CREATE USER"): + node.query(f"GRANT CREATE USER ON *.* TO {grant_target_name}") + + with And(f"I grant the role with ADMIN OPTION"): + node.query(f"GRANT {default_role_name} TO {grant_target_name} WITH ADMIN OPTION") + + with Then("I check the user can create a user"): + node.query(f"CREATE USER {create_user_name} ON CLUSTER sharded_cluster DEFAULT ROLE {default_role_name}", + settings = [("user", f"{user_name}")]) + + finally: + with Finally("I drop the user", flags=TE): + node.query(f"DROP USER IF EXISTS {create_user_name} ON CLUSTER sharded_cluster") + + with And("I drop the role from the cluster"): + node.query(f"DROP ROLE {default_role_name} ON CLUSTER sharded_cluster") + + with Scenario("CREATE USER with DEFAULT ROLE with revoked role privilege"): + create_user_name = f"create_user_{getuid()}" + default_role_name = f"default_role_{getuid()}" + + with role(node, default_role_name): + try: + with When(f"I grant CREATE USER"): + node.query(f"GRANT CREATE USER ON *.* TO {grant_target_name}") + + with And(f"I grant the role with ADMIN OPTION"): + node.query(f"GRANT {default_role_name} TO {grant_target_name} WITH ADMIN OPTION") + + with And(f"I revoke the role"): + node.query(f"REVOKE {default_role_name} FROM {grant_target_name}") + + with Then("I check the user can't create a user"): + node.query(f"CREATE USER {create_user_name} DEFAULT ROLE {default_role_name}", settings=[("user",user_name)], + exitcode=exitcode, message=message) + + finally: + with Finally("I drop the user"): + node.query(f"DROP USER IF EXISTS {create_user_name}") + + with Scenario("CREATE USER with DEFAULT ROLE with ACCESS MANAGEMENT privilege"): + create_user_name = f"create_user_{getuid()}" + default_role_name = f"default_role_{getuid()}" + + with role(node, default_role_name): + try: + with When(f"I grant ACCESS MANAGEMENT "): + node.query(f"GRANT ACCESS MANAGEMENT ON *.* TO {grant_target_name}") + + with Then("I check the user can create a user"): + node.query(f"CREATE USER {create_user_name} DEFAULT ROLE {default_role_name}", settings=[("user",user_name)]) + + finally: + with Finally("I drop the user"): + node.query(f"DROP USER IF EXISTS {create_user_name}") + +@TestFeature +@Name("create user") +@Requirements( + RQ_SRS_006_RBAC_Privileges_CreateUser("1.0"), +) +def feature(self, node="clickhouse1"): + """Check the RBAC functionality of CREATE USER. + """ + self.context.node = self.context.cluster.node(node) + + Suite(run=create_user_granted_directly, setup=instrument_clickhouse_server_log) + Suite(run=create_user_granted_via_role, setup=instrument_clickhouse_server_log) + Suite(run=default_role_granted_directly, setup=instrument_clickhouse_server_log) + Suite(run=default_role_granted_via_role, setup=instrument_clickhouse_server_log) diff --git a/tests/testflows/rbac/tests/privileges/drop/drop_quota.py b/tests/testflows/rbac/tests/privileges/drop/drop_quota.py new file mode 100644 index 00000000000..de25f0e15b1 --- /dev/null +++ b/tests/testflows/rbac/tests/privileges/drop/drop_quota.py @@ -0,0 +1,139 @@ +from testflows.core import * +from testflows.asserts import error + +from rbac.requirements import * +from rbac.helper.common import * +import rbac.helper.errors as errors + +@TestSuite +def privileges_granted_directly(self, node=None): + """Check that a user is able to execute `DROP QUOTA` with privileges are granted directly. + """ + + user_name = f"user_{getuid()}" + + if node is None: + node = self.context.node + + with user(node, f"{user_name}"): + + Suite(run=drop_quota, flags=TE, + examples=Examples("privilege grant_target_name user_name", [ + tuple(list(row)+[user_name,user_name]) for row in drop_quota.examples + ], args=Args(name="privilege={privilege}", format_name=True))) + +@TestSuite +def privileges_granted_via_role(self, node=None): + """Check that a user is able to execute `DROP QUOTA` with privileges are granted through a role. + """ + + user_name = f"user_{getuid()}" + role_name = f"role_{getuid()}" + + if node is None: + node = self.context.node + + with user(node, f"{user_name}"), role(node, f"{role_name}"): + + with When("I grant the role to the user"): + node.query(f"GRANT {role_name} TO {user_name}") + + Suite(run=drop_quota, flags=TE, + examples=Examples("privilege grant_target_name user_name", [ + tuple(list(row)+[role_name,user_name]) for row in drop_quota.examples + ], args=Args(name="privilege={privilege}", format_name=True))) + +@TestOutline(Suite) +@Examples("privilege",[ + ("ACCESS MANAGEMENT",), + ("DROP QUOTA",), +]) +def drop_quota(self, privilege, grant_target_name, user_name, node=None): + """Check that user is only able to execute `DROP QUOTA` when they have the necessary privilege. + """ + exitcode, message = errors.not_enough_privileges(name=user_name) + + if node is None: + node = self.context.node + + with Scenario("DROP QUOTA without privilege"): + drop_row_policy_name = f"drop_row_policy_{getuid()}" + + try: + with Given("I have a quota"): + node.query(f"CREATE QUOTA {drop_row_policy_name}") + + with When("I check the user can't drop a quota"): + node.query(f"DROP QUOTA {drop_row_policy_name}", settings=[("user",user_name)], + exitcode=exitcode, message=message) + + finally: + with Finally("I drop the quota"): + node.query(f"DROP QUOTA IF EXISTS {drop_row_policy_name}") + + with Scenario("DROP QUOTA with privilege"): + drop_row_policy_name = f"drop_row_policy_{getuid()}" + + try: + with Given("I have a quota"): + node.query(f"CREATE QUOTA {drop_row_policy_name}") + + with When(f"I grant {privilege}"): + node.query(f"GRANT {privilege} ON *.* TO {grant_target_name}") + + with Then("I check the user can drop a quota"): + node.query(f"DROP QUOTA {drop_row_policy_name}", settings = [("user", f"{user_name}")]) + + finally: + with Finally("I drop the quota"): + node.query(f"DROP QUOTA IF EXISTS {drop_row_policy_name}") + + with Scenario("DROP QUOTA on cluster"): + drop_row_policy_name = f"drop_row_policy_{getuid()}" + + try: + with Given("I have a quota on a cluster"): + node.query(f"CREATE QUOTA {drop_row_policy_name} ON CLUSTER sharded_cluster") + + with When(f"I grant {privilege}"): + node.query(f"GRANT {privilege} ON *.* TO {grant_target_name}") + + with Then("I check the user can drop a quota"): + node.query(f"DROP QUOTA {drop_row_policy_name} ON CLUSTER sharded_cluster", settings = [("user", f"{user_name}")]) + + finally: + with Finally("I drop the user"): + node.query(f"DROP QUOTA IF EXISTS {drop_row_policy_name} ON CLUSTER sharded_cluster") + + with Scenario("DROP QUOTA with revoked privilege"): + drop_row_policy_name = f"drop_row_policy_{getuid()}" + + try: + with Given("I have a quota"): + node.query(f"CREATE QUOTA {drop_row_policy_name}") + + with When(f"I grant {privilege} on the database"): + node.query(f"GRANT {privilege} ON *.* TO {grant_target_name}") + + with And(f"I revoke {privilege} on the database"): + node.query(f"REVOKE {privilege} ON *.* FROM {grant_target_name}") + + with Then("I check the user cannot drop quota"): + node.query(f"DROP QUOTA {drop_row_policy_name}", settings=[("user",user_name)], + exitcode=exitcode, message=message) + finally: + with Finally("I drop the quota"): + node.query(f"DROP QUOTA IF EXISTS {drop_row_policy_name}") + +@TestFeature +@Name("drop quota") +@Requirements( + RQ_SRS_006_RBAC_Privileges_DropQuota("1.0"), +) +def feature(self, node="clickhouse1"): + """Check the RBAC functionality of DROP QUOTA. + """ + self.context.node = self.context.cluster.node(node) + + Suite(run=privileges_granted_directly, setup=instrument_clickhouse_server_log) + Suite(run=privileges_granted_via_role, setup=instrument_clickhouse_server_log) diff --git a/tests/testflows/rbac/tests/privileges/drop/drop_role.py b/tests/testflows/rbac/tests/privileges/drop/drop_role.py new file mode 100644 index 00000000000..4b6b7c04741 --- /dev/null +++ b/tests/testflows/rbac/tests/privileges/drop/drop_role.py @@ -0,0 +1,118 @@ +from testflows.core import * +from testflows.asserts import error + +from rbac.requirements import * +from rbac.helper.common import * +import rbac.helper.errors as errors + +@TestSuite +def privileges_granted_directly(self, node=None): + """Check that a user is able to execute `DROP ROLE` with privileges are granted directly. + """ + + user_name = f"user_{getuid()}" + + if node is None: + node = self.context.node + + with user(node, f"{user_name}"): + + Suite(run=drop_role, flags=TE, + examples=Examples("privilege grant_target_name user_name", [ + tuple(list(row)+[user_name,user_name]) for row in drop_role.examples + ], args=Args(name="privilege={privilege}", format_name=True))) + +@TestSuite +def privileges_granted_via_role(self, node=None): + """Check that a user is able to execute `DROP ROLE` with privileges are granted through a role. + """ + + user_name = f"user_{getuid()}" + role_name = f"role_{getuid()}" + + if node is None: + node = self.context.node + + with user(node, f"{user_name}"), role(node, f"{role_name}"): + + with When("I grant the role to the user"): + node.query(f"GRANT {role_name} TO {user_name}") + + Suite(run=drop_role, flags=TE, + examples=Examples("privilege grant_target_name user_name", [ + tuple(list(row)+[role_name,user_name]) for row in drop_role.examples + ], args=Args(name="privilege={privilege}", format_name=True))) + +@TestOutline(Suite) +@Examples("privilege",[ + ("ACCESS MANAGEMENT",), + ("DROP ROLE",), +]) +def drop_role(self, privilege, grant_target_name, user_name, node=None): + """Check that user is only able to execute `DROP ROLE` when they have the necessary privilege. + """ + exitcode, message = errors.not_enough_privileges(name=user_name) + + if node is None: + node = self.context.node + + with Scenario("DROP ROLE without privilege"): + drop_role_name = f"drop_role_{getuid()}" + with role(node, drop_role_name): + + with When("I check the user can't drop a role"): + node.query(f"DROP ROLE {drop_role_name}", settings=[("user",user_name)], + exitcode=exitcode, message=message) + + with Scenario("DROP ROLE with privilege"): + drop_role_name = f"drop_role_{getuid()}" + with role(node, drop_role_name): + + with When(f"I grant {privilege}"): + node.query(f"GRANT {privilege} ON *.* TO {grant_target_name}") + + with Then("I check the user can drop a role"): + node.query(f"DROP ROLE {drop_role_name}", settings = [("user", f"{user_name}")]) + + with Scenario("DROP ROLE on cluster"): + drop_role_name = f"drop_role_{getuid()}" + + try: + with Given("I have a role on a cluster"): + node.query(f"CREATE ROLE {drop_role_name} ON CLUSTER sharded_cluster") + + with When(f"I grant {privilege}"): + node.query(f"GRANT {privilege} ON *.* TO {grant_target_name}") + + with Then("I check the user can drop a role"): + node.query(f"DROP ROLE {drop_role_name} ON CLUSTER sharded_cluster", settings = [("user", f"{user_name}")]) + + finally: + with Finally("I drop the user"): + node.query(f"DROP ROLE IF EXISTS {drop_role_name} ON CLUSTER sharded_cluster") + + with Scenario("DROP ROLE with revoked privilege"): + drop_role_name = f"drop_role_{getuid()}" + with role(node, drop_role_name): + with When(f"I grant {privilege}"): + node.query(f"GRANT {privilege} ON *.* TO {grant_target_name}") + + with And(f"I revoke {privilege}"): + node.query(f"REVOKE {privilege} ON *.* FROM {grant_target_name}") + + with Then("I check the user can't drop a role"): + node.query(f"DROP ROLE {drop_role_name}", settings=[("user",user_name)], + exitcode=exitcode, message=message) + +@TestFeature +@Name("drop role") +@Requirements( + RQ_SRS_006_RBAC_Privileges_DropRole("1.0"), +) +def feature(self, node="clickhouse1"): + """Check the RBAC functionality of DROP ROLE. + """ + self.context.node = self.context.cluster.node(node) + + Suite(run=privileges_granted_directly, setup=instrument_clickhouse_server_log) + Suite(run=privileges_granted_via_role, setup=instrument_clickhouse_server_log) diff --git a/tests/testflows/rbac/tests/privileges/drop/drop_row_policy.py b/tests/testflows/rbac/tests/privileges/drop/drop_row_policy.py new file mode 100644 index 00000000000..6e8a2aaa3d5 --- /dev/null +++ b/tests/testflows/rbac/tests/privileges/drop/drop_row_policy.py @@ -0,0 +1,144 @@ +from testflows.core import * +from testflows.asserts import error + +from rbac.requirements import * +from rbac.helper.common import * +import rbac.helper.errors as errors + +@TestSuite +def privileges_granted_directly(self, node=None): + """Check that a user is able to execute `DROP ROW POLICY` with privileges are granted directly. + """ + + user_name = f"user_{getuid()}" + + if node is None: + node = self.context.node + + with user(node, f"{user_name}"): + + Suite(run=drop_row_policy, flags=TE, + examples=Examples("privilege grant_target_name user_name", [ + tuple(list(row)+[user_name,user_name]) for row in drop_row_policy.examples + ], args=Args(name="privilege={privilege}", format_name=True))) + +@TestSuite +def privileges_granted_via_role(self, node=None): + """Check that a user is able to execute `DROP ROW POLICY` with privileges are granted through a role. + """ + + user_name = f"user_{getuid()}" + role_name = f"role_{getuid()}" + + if node is None: + node = self.context.node + + with user(node, f"{user_name}"), role(node, f"{role_name}"): + + with When("I grant the role to the user"): + node.query(f"GRANT {role_name} TO {user_name}") + + Suite(run=drop_row_policy, flags=TE, + examples=Examples("privilege grant_target_name user_name", [ + tuple(list(row)+[role_name,user_name]) for row in drop_row_policy.examples + ], args=Args(name="privilege={privilege}", format_name=True))) + +@TestOutline(Suite) +@Examples("privilege",[ + ("ACCESS MANAGEMENT",), + ("DROP ROW POLICY",), + ("DROP POLICY",), +]) +def drop_row_policy(self, privilege, grant_target_name, user_name, node=None): + """Check that user is only able to execute `DROP ROW POLICY` when they have the necessary privilege. + """ + exitcode, message = errors.not_enough_privileges(name=user_name) + + if node is None: + node = self.context.node + + with Scenario("DROP ROW POLICY without privilege"): + drop_row_policy_name = f"drop_row_policy_{getuid()}" + table_name = f"table_name_{getuid()}" + + try: + with Given("I have a row policy"): + node.query(f"CREATE ROW POLICY {drop_row_policy_name} ON {table_name}") + + with When("I check the user can't drop a row policy"): + node.query(f"DROP ROW POLICY {drop_row_policy_name} ON {table_name}", settings=[("user",user_name)], + exitcode=exitcode, message=message) + + finally: + with Finally("I drop the row policy"): + node.query(f"DROP ROW POLICY IF EXISTS {drop_row_policy_name} ON {table_name}") + + with Scenario("DROP ROW POLICY with privilege"): + drop_row_policy_name = f"drop_row_policy_{getuid()}" + table_name = f"table_name_{getuid()}" + + try: + with Given("I have a row policy"): + node.query(f"CREATE ROW POLICY {drop_row_policy_name} ON {table_name}") + + with When(f"I grant {privilege}"): + node.query(f"GRANT {privilege} ON *.* TO {grant_target_name}") + + with Then("I check the user can drop a row policy"): + node.query(f"DROP ROW POLICY {drop_row_policy_name} ON {table_name}", settings = [("user", f"{user_name}")]) + + finally: + with Finally("I drop the row policy"): + node.query(f"DROP ROW POLICY IF EXISTS {drop_row_policy_name} ON {table_name}") + + with Scenario("DROP ROW POLICY on cluster"): + drop_row_policy_name = f"drop_row_policy_{getuid()}" + table_name = f"table_name_{getuid()}" + + try: + with Given("I have a row policy on a cluster"): + node.query(f"CREATE ROW POLICY {drop_row_policy_name} ON CLUSTER sharded_cluster ON {table_name}") + + with When(f"I grant {privilege}"): + node.query(f"GRANT {privilege} ON *.* TO {grant_target_name}") + + with Then("I check the user can drop a row policy"): + node.query(f"DROP ROW POLICY {drop_row_policy_name} ON CLUSTER sharded_cluster ON {table_name}", settings = [("user", f"{user_name}")]) + + finally: + with Finally("I drop the user"): + node.query(f"DROP ROW POLICY IF EXISTS {drop_row_policy_name} ON CLUSTER sharded_cluster ON {table_name}") + + with Scenario("DROP ROW POLICY with revoked privilege"): + drop_row_policy_name = f"drop_row_policy_{getuid()}" + table_name = f"table_name_{getuid()}" + + try: + with Given("I have a row policy"): + node.query(f"CREATE ROW POLICY {drop_row_policy_name} ON {table_name}") + + with When(f"I grant {privilege} on the database"): + node.query(f"GRANT {privilege} ON *.* TO {grant_target_name}") + + with And(f"I revoke {privilege} on the database"): + node.query(f"REVOKE {privilege} ON *.* FROM {grant_target_name}") + + with Then("I check the user cannot drop row policy"): + node.query(f"DROP ROW POLICY {drop_row_policy_name} ON {table_name}", settings=[("user",user_name)], + exitcode=exitcode, message=message) + finally: + with Finally("I drop the row policy"): + node.query(f"DROP ROW POLICY IF EXISTS {drop_row_policy_name} ON {table_name}") + +@TestFeature +@Name("drop row policy") +@Requirements( + RQ_SRS_006_RBAC_Privileges_DropRowPolicy("1.0"), +) +def feature(self, node="clickhouse1"): + """Check the RBAC functionality of DROP ROW POLICY. + """ + self.context.node = self.context.cluster.node(node) + + Suite(run=privileges_granted_directly, setup=instrument_clickhouse_server_log) + Suite(run=privileges_granted_via_role, setup=instrument_clickhouse_server_log) diff --git a/tests/testflows/rbac/tests/privileges/drop/drop_settings_profile.py b/tests/testflows/rbac/tests/privileges/drop/drop_settings_profile.py new file mode 100644 index 00000000000..51139653283 --- /dev/null +++ b/tests/testflows/rbac/tests/privileges/drop/drop_settings_profile.py @@ -0,0 +1,140 @@ +from testflows.core import * +from testflows.asserts import error + +from rbac.requirements import * +from rbac.helper.common import * +import rbac.helper.errors as errors + +@TestSuite +def privileges_granted_directly(self, node=None): + """Check that a user is able to execute `DROP SETTINGS PROFILE` with privileges are granted directly. + """ + + user_name = f"user_{getuid()}" + + if node is None: + node = self.context.node + + with user(node, f"{user_name}"): + + Suite(run=drop_settings_profile, flags=TE, + examples=Examples("privilege grant_target_name user_name", [ + tuple(list(row)+[user_name,user_name]) for row in drop_settings_profile.examples + ], args=Args(name="privilege={privilege}", format_name=True))) + +@TestSuite +def privileges_granted_via_role(self, node=None): + """Check that a user is able to execute `DROP SETTINGS PROFILE` with privileges are granted through a role. + """ + + user_name = f"user_{getuid()}" + role_name = f"role_{getuid()}" + + if node is None: + node = self.context.node + + with user(node, f"{user_name}"), role(node, f"{role_name}"): + + with When("I grant the role to the user"): + node.query(f"GRANT {role_name} TO {user_name}") + + Suite(run=drop_settings_profile, flags=TE, + examples=Examples("privilege grant_target_name user_name", [ + tuple(list(row)+[role_name,user_name]) for row in drop_settings_profile.examples + ], args=Args(name="privilege={privilege}", format_name=True))) + +@TestOutline(Suite) +@Examples("privilege",[ + ("ACCESS MANAGEMENT",), + ("DROP SETTINGS PROFILE",), + ("DROP PROFILE",), +]) +def drop_settings_profile(self, privilege, grant_target_name, user_name, node=None): + """Check that user is only able to execute `DROP SETTINGS PROFILE` when they have the necessary privilege. + """ + exitcode, message = errors.not_enough_privileges(name=user_name) + + if node is None: + node = self.context.node + + with Scenario("DROP SETTINGS PROFILE without privilege"): + drop_row_policy_name = f"drop_row_policy_{getuid()}" + + try: + with Given("I have a settings_profile"): + node.query(f"CREATE SETTINGS PROFILE {drop_row_policy_name}") + + with When("I check the user can't drop a settings_profile"): + node.query(f"DROP SETTINGS PROFILE {drop_row_policy_name}", settings=[("user",user_name)], + exitcode=exitcode, message=message) + + finally: + with Finally("I drop the settings_profile"): + node.query(f"DROP SETTINGS PROFILE IF EXISTS {drop_row_policy_name}") + + with Scenario("DROP SETTINGS PROFILE with privilege"): + drop_row_policy_name = f"drop_row_policy_{getuid()}" + + try: + with Given("I have a settings_profile"): + node.query(f"CREATE SETTINGS PROFILE {drop_row_policy_name}") + + with When(f"I grant {privilege}"): + node.query(f"GRANT {privilege} ON *.* TO {grant_target_name}") + + with Then("I check the user can drop a settings_profile"): + node.query(f"DROP SETTINGS PROFILE {drop_row_policy_name}", settings = [("user", f"{user_name}")]) + + finally: + with Finally("I drop the settings_profile"): + node.query(f"DROP SETTINGS PROFILE IF EXISTS {drop_row_policy_name}") + + with Scenario("DROP SETTINGS PROFILE on cluster"): + drop_row_policy_name = f"drop_row_policy_{getuid()}" + + try: + with Given("I have a settings_profile on a cluster"): + node.query(f"CREATE SETTINGS PROFILE {drop_row_policy_name} ON CLUSTER sharded_cluster") + + with When(f"I grant {privilege}"): + node.query(f"GRANT {privilege} ON *.* TO {grant_target_name}") + + with Then("I check the user can drop a settings_profile"): + node.query(f"DROP SETTINGS PROFILE {drop_row_policy_name} ON CLUSTER sharded_cluster", settings = [("user", f"{user_name}")]) + + finally: + with Finally("I drop the user"): + node.query(f"DROP SETTINGS PROFILE IF EXISTS {drop_row_policy_name} ON CLUSTER sharded_cluster") + + with Scenario("DROP SETTINGS PROFILE with revoked privilege"): + drop_row_policy_name = f"drop_row_policy_{getuid()}" + + try: + with Given("I have a settings_profile"): + node.query(f"CREATE SETTINGS PROFILE {drop_row_policy_name}") + + with When(f"I grant {privilege} on the database"): + node.query(f"GRANT {privilege} ON *.* TO {grant_target_name}") + + with And(f"I revoke {privilege} on the database"): + node.query(f"REVOKE {privilege} ON *.* FROM {grant_target_name}") + + with Then("I check the user cannot drop settings_profile"): + node.query(f"DROP SETTINGS PROFILE {drop_row_policy_name}", settings=[("user",user_name)], + exitcode=exitcode, message=message) + finally: + with Finally("I drop the settings_profile"): + node.query(f"DROP SETTINGS PROFILE IF EXISTS {drop_row_policy_name}") + +@TestFeature +@Name("drop settings profile") +@Requirements( + RQ_SRS_006_RBAC_Privileges_DropSettingsProfile("1.0"), +) +def feature(self, node="clickhouse1"): + """Check the RBAC functionality of DROP SETTINGS PROFILE. + """ + self.context.node = self.context.cluster.node(node) + + Suite(run=privileges_granted_directly, setup=instrument_clickhouse_server_log) + Suite(run=privileges_granted_via_role, setup=instrument_clickhouse_server_log) diff --git a/tests/testflows/rbac/tests/privileges/drop/drop_user.py b/tests/testflows/rbac/tests/privileges/drop/drop_user.py new file mode 100644 index 00000000000..9aec34cca81 --- /dev/null +++ b/tests/testflows/rbac/tests/privileges/drop/drop_user.py @@ -0,0 +1,118 @@ +from testflows.core import * +from testflows.asserts import error + +from rbac.requirements import * +from rbac.helper.common import * +import rbac.helper.errors as errors + +@TestSuite +def drop_user_granted_directly(self, node=None): + """Check that a user is able to execute `DROP USER` with privileges are granted directly. + """ + + user_name = f"user_{getuid()}" + + if node is None: + node = self.context.node + + with user(node, f"{user_name}"): + + Suite(run=drop_user, flags=TE, + examples=Examples("privilege grant_target_name user_name", [ + tuple(list(row)+[user_name,user_name]) for row in drop_user.examples + ], args=Args(name="check privilege={privilege}", format_name=True))) + +@TestSuite +def drop_user_granted_via_role(self, node=None): + """Check that a user is able to execute `DROP USER` with privileges are granted through a role. + """ + + user_name = f"user_{getuid()}" + role_name = f"role_{getuid()}" + + if node is None: + node = self.context.node + + with user(node, f"{user_name}"), role(node, f"{role_name}"): + + with When("I grant the role to the user"): + node.query(f"GRANT {role_name} TO {user_name}") + + Suite(run=drop_user, flags=TE, + examples=Examples("privilege grant_target_name user_name", [ + tuple(list(row)+[role_name,user_name]) for row in drop_user.examples + ], args=Args(name="check privilege={privilege}", format_name=True))) + +@TestOutline(Suite) +@Examples("privilege",[ + ("ACCESS MANAGEMENT",), + ("DROP USER",), +]) +def drop_user(self, privilege, grant_target_name, user_name, node=None): + """Check that user is only able to execute `DROP USER` when they have the necessary privilege. + """ + exitcode, message = errors.not_enough_privileges(name=user_name) + + if node is None: + node = self.context.node + + with Scenario("DROP USER without privilege"): + + drop_user_name = f"drop_user_{getuid()}" + with user(node, drop_user_name): + + with When("I check the user can't drop a user"): + node.query(f"DROP USER {drop_user_name}", settings=[("user",user_name)], + exitcode=exitcode, message=message) + + with Scenario("DROP USER with privilege"): + drop_user_name = f"drop_user_{getuid()}" + with user(node, drop_user_name): + with When(f"I grant {privilege}"): + node.query(f"GRANT {privilege} ON *.* TO {grant_target_name}") + + with Then("I check the user can drop a user"): + node.query(f"DROP USER {drop_user_name}", settings = [("user", f"{user_name}")]) + + with Scenario("DROP USER on cluster"): + drop_user_name = f"drop_user_{getuid()}" + try: + with Given("I have a user on a cluster"): + node.query(f"CREATE USER {drop_user_name} ON CLUSTER sharded_cluster") + + with When(f"I grant {privilege}"): + node.query(f"GRANT {privilege} ON *.* TO {grant_target_name}") + + with Then("I check the user can drop a user"): + node.query(f"DROP USER {drop_user_name} ON CLUSTER sharded_cluster", + settings = [("user", f"{user_name}")]) + + finally: + with Finally("I drop the user"): + node.query(f"DROP USER IF EXISTS {drop_user_name} ON CLUSTER sharded_cluster") + + with Scenario("DROP USER with revoked privilege"): + drop_user_name = f"drop_user_{getuid()}" + with user(node, drop_user_name): + with When(f"I grant {privilege}"): + node.query(f"GRANT {privilege} ON *.* TO {grant_target_name}") + + with And(f"I revoke {privilege}"): + node.query(f"REVOKE {privilege} ON *.* FROM {grant_target_name}") + + with Then("I check the user can't drop a user"): + node.query(f"DROP USER {drop_user_name}", settings=[("user",user_name)], + exitcode=exitcode, message=message) + +@TestFeature +@Name("drop user") +@Requirements( + RQ_SRS_006_RBAC_Privileges_DropUser("1.0"), +) +def feature(self, node="clickhouse1"): + """Check the RBAC functionality of DROP USER. + """ + self.context.node = self.context.cluster.node(node) + + Suite(run=drop_user_granted_directly, setup=instrument_clickhouse_server_log) + Suite(run=drop_user_granted_via_role, setup=instrument_clickhouse_server_log) diff --git a/tests/testflows/rbac/tests/privileges/feature.py b/tests/testflows/rbac/tests/privileges/feature.py index 6773c5b1875..ed9c0ae32e8 100755 --- a/tests/testflows/rbac/tests/privileges/feature.py +++ b/tests/testflows/rbac/tests/privileges/feature.py @@ -20,11 +20,17 @@ def feature(self): run_scenario(pool, tasks, Feature(test=load("rbac.tests.privileges.optimize", "feature"), flags=TE), {}) run_scenario(pool, tasks, Feature(test=load("rbac.tests.privileges.kill_query", "feature"), flags=TE), {}) run_scenario(pool, tasks, Feature(test=load("rbac.tests.privileges.kill_mutation", "feature"), flags=TE), {}) + run_scenario(pool, tasks, Feature(test=load("rbac.tests.privileges.role_admin", "feature"), flags=TE), {}) run_scenario(pool, tasks, Feature(test=load("rbac.tests.privileges.show.show_tables", "feature"), flags=TE), {}) run_scenario(pool, tasks, Feature(test=load("rbac.tests.privileges.show.show_dictionaries", "feature"), flags=TE), {}) run_scenario(pool, tasks, Feature(test=load("rbac.tests.privileges.show.show_databases", "feature"), flags=TE), {}) run_scenario(pool, tasks, Feature(test=load("rbac.tests.privileges.show.show_columns", "feature"), flags=TE), {}) + run_scenario(pool, tasks, Feature(test=load("rbac.tests.privileges.show.show_users", "feature"), flags=TE), {}) + run_scenario(pool, tasks, Feature(test=load("rbac.tests.privileges.show.show_roles", "feature"), flags=TE), {}) + run_scenario(pool, tasks, Feature(test=load("rbac.tests.privileges.show.show_quotas", "feature"), flags=TE), {}) + run_scenario(pool, tasks, Feature(test=load("rbac.tests.privileges.show.show_settings_profiles", "feature"), flags=TE), {}) + run_scenario(pool, tasks, Feature(test=load("rbac.tests.privileges.show.show_row_policies", "feature"), flags=TE), {}) run_scenario(pool, tasks, Feature(test=load("rbac.tests.privileges.alter.alter_column", "feature"), flags=TE), {}) run_scenario(pool, tasks, Feature(test=load("rbac.tests.privileges.alter.alter_index", "feature"), flags=TE), {}) @@ -36,11 +42,21 @@ def feature(self): run_scenario(pool, tasks, Feature(test=load("rbac.tests.privileges.alter.alter_freeze", "feature"), flags=TE), {}) run_scenario(pool, tasks, Feature(test=load("rbac.tests.privileges.alter.alter_fetch", "feature"), flags=TE), {}) run_scenario(pool, tasks, Feature(test=load("rbac.tests.privileges.alter.alter_move", "feature"), flags=TE), {}) + run_scenario(pool, tasks, Feature(test=load("rbac.tests.privileges.alter.alter_user", "feature"), flags=TE), {}) + run_scenario(pool, tasks, Feature(test=load("rbac.tests.privileges.alter.alter_role", "feature"), flags=TE), {}) + run_scenario(pool, tasks, Feature(test=load("rbac.tests.privileges.alter.alter_row_policy", "feature"), flags=TE), {}) + run_scenario(pool, tasks, Feature(test=load("rbac.tests.privileges.alter.alter_quota", "feature"), flags=TE), {}) + run_scenario(pool, tasks, Feature(test=load("rbac.tests.privileges.alter.alter_settings_profile", "feature"), flags=TE), {}) run_scenario(pool, tasks, Feature(test=load("rbac.tests.privileges.create.create_database", "feature"), flags=TE), {}) run_scenario(pool, tasks, Feature(test=load("rbac.tests.privileges.create.create_dictionary", "feature"), flags=TE), {}) run_scenario(pool, tasks, Feature(test=load("rbac.tests.privileges.create.create_temp_table", "feature"), flags=TE), {}) run_scenario(pool, tasks, Feature(test=load("rbac.tests.privileges.create.create_table", "feature"), flags=TE), {}) + run_scenario(pool, tasks, Feature(test=load("rbac.tests.privileges.create.create_user", "feature"), flags=TE), {}) + run_scenario(pool, tasks, Feature(test=load("rbac.tests.privileges.create.create_role", "feature"), flags=TE), {}) + run_scenario(pool, tasks, Feature(test=load("rbac.tests.privileges.create.create_row_policy", "feature"), flags=TE), {}) + run_scenario(pool, tasks, Feature(test=load("rbac.tests.privileges.create.create_quota", "feature"), flags=TE), {}) + run_scenario(pool, tasks, Feature(test=load("rbac.tests.privileges.create.create_settings_profile", "feature"), flags=TE), {}) run_scenario(pool, tasks, Feature(test=load("rbac.tests.privileges.attach.attach_database", "feature"), flags=TE), {}) run_scenario(pool, tasks, Feature(test=load("rbac.tests.privileges.attach.attach_dictionary", "feature"), flags=TE), {}) @@ -50,6 +66,11 @@ def feature(self): run_scenario(pool, tasks, Feature(test=load("rbac.tests.privileges.drop.drop_database", "feature"), flags=TE), {}) run_scenario(pool, tasks, Feature(test=load("rbac.tests.privileges.drop.drop_dictionary", "feature"), flags=TE), {}) run_scenario(pool, tasks, Feature(test=load("rbac.tests.privileges.drop.drop_table", "feature"), flags=TE), {}) + run_scenario(pool, tasks, Feature(test=load("rbac.tests.privileges.drop.drop_user", "feature"), flags=TE), {}) + run_scenario(pool, tasks, Feature(test=load("rbac.tests.privileges.drop.drop_role", "feature"), flags=TE), {}) + run_scenario(pool, tasks, Feature(test=load("rbac.tests.privileges.drop.drop_row_policy", "feature"), flags=TE), {}) + run_scenario(pool, tasks, Feature(test=load("rbac.tests.privileges.drop.drop_quota", "feature"), flags=TE), {}) + run_scenario(pool, tasks, Feature(test=load("rbac.tests.privileges.drop.drop_settings_profile", "feature"), flags=TE), {}) run_scenario(pool, tasks, Feature(test=load("rbac.tests.privileges.detach.detach_database", "feature"), flags=TE), {}) run_scenario(pool, tasks, Feature(test=load("rbac.tests.privileges.detach.detach_dictionary", "feature"), flags=TE), {}) diff --git a/tests/testflows/rbac/tests/privileges/public_tables.py b/tests/testflows/rbac/tests/privileges/public_tables.py index bd3cc55f812..f4e457d71c4 100755 --- a/tests/testflows/rbac/tests/privileges/public_tables.py +++ b/tests/testflows/rbac/tests/privileges/public_tables.py @@ -30,9 +30,29 @@ def public_tables(self, node=None): with And("I check the user is able to select on system.functions"): node.query("SELECT count(*) FROM system.functions", settings = [("user",user_name)]) +@TestScenario +@Requirements( + RQ_SRS_006_RBAC_Table_QueryLog("1.0"), +) +def query_log(self, node=None): + """Check that a user with no privilege is only able to see their own queries. + """ + user_name = f"user_{getuid()}" + if node is None: + node = self.context.node + + with user(node, f"{user_name}"): + with Given("I create a query"): + node.query("SELECT 1") + + with Then("The user reads system.query_log"): + output = node.query("SELECT count() FROM system.query_log", settings = [("user",user_name)]).output + assert output == 0, error() + @TestFeature @Name("public tables") def feature(self, node="clickhouse1"): self.context.node = self.context.cluster.node(node) - Scenario(run=public_tables, setup=instrument_clickhouse_server_log, flags=TE) \ No newline at end of file + Scenario(run=public_tables, setup=instrument_clickhouse_server_log, flags=TE) + Scenario(run=query_log, setup=instrument_clickhouse_server_log, flags=TE) \ No newline at end of file diff --git a/tests/testflows/rbac/tests/privileges/role_admin.py b/tests/testflows/rbac/tests/privileges/role_admin.py new file mode 100644 index 00000000000..91d663d0de8 --- /dev/null +++ b/tests/testflows/rbac/tests/privileges/role_admin.py @@ -0,0 +1,119 @@ +from testflows.core import * +from testflows.asserts import error + +from rbac.requirements import * +from rbac.helper.common import * +import rbac.helper.errors as errors + +@TestSuite +def privileges_granted_directly(self, node=None): + """Check that a user is able to grant role with `ROLE ADMIN` privilege granted directly. + """ + + user_name = f"user_{getuid()}" + + if node is None: + node = self.context.node + + with user(node, f"{user_name}"): + + Suite(test=role_admin, flags=TE)(grant_target_name=user_name, user_name=user_name) + +@TestSuite +def privileges_granted_via_role(self, node=None): + """Check that a user is able to grant role with `ROLE ADMIN` privilege granted through a role. + """ + + user_name = f"user_{getuid()}" + role_name = f"role_{getuid()}" + + if node is None: + node = self.context.node + + with user(node, f"{user_name}"), role(node, f"{role_name}"): + + with When("I grant the role to the user"): + node.query(f"GRANT {role_name} TO {user_name}") + + Suite(test=role_admin, flags=TE)(grant_target_name=role_name, user_name=user_name) + +@TestSuite +def role_admin(self, grant_target_name, user_name, node=None): + """Check that user is able to execute to grant roles if and only if they have `ROLE ADMIN`. + """ + exitcode, message = errors.not_enough_privileges(name=user_name) + + if node is None: + node = self.context.node + + with Scenario("Grant role without privilege"): + role_admin_name = f"role_admin_{getuid()}" + target_user_name = f"target_user_{getuid()}" + + with user(node, target_user_name), role(node, role_admin_name): + + with When("I check the user can't grant a role"): + node.query(f"GRANT {role_admin_name} TO {target_user_name}", settings=[("user",user_name)], + exitcode=exitcode, message=message) + + with Scenario("Grant role with privilege"): + role_admin_name = f"role_admin_{getuid()}" + target_user_name = f"target_user_{getuid()}" + + with user(node, target_user_name), role(node, role_admin_name): + + with When(f"I grant ROLE ADMIN"): + node.query(f"GRANT ROLE ADMIN ON *.* TO {grant_target_name}") + + with Then("I check the user can grant a role"): + node.query(f"GRANT {role_admin_name} TO {target_user_name}", settings = [("user", f"{user_name}")]) + + with Scenario("Grant role on cluster"): + role_admin_name = f"role_admin_{getuid()}" + target_user_name = f"target_user_{getuid()}" + + try: + with Given("I have a role on a cluster"): + node.query(f"CREATE ROLE {role_admin_name} ON CLUSTER sharded_cluster") + + with And("I have a user on a cluster"): + node.query(f"CREATE USER {target_user_name} ON CLUSTER sharded_cluster") + + with When("I grant ROLE ADMIN privilege"): + node.query(f"GRANT ROLE ADMIN ON *.* TO {grant_target_name}") + + with Then("I check the user can grant a role"): + node.query(f"GRANT {role_admin_name} TO {target_user_name} ON CLUSTER sharded_cluster", settings = [("user", f"{user_name}")]) + + finally: + with Finally("I drop the user"): + node.query(f"DROP ROLE IF EXISTS {role_admin_name} ON CLUSTER sharded_cluster") + + with Scenario("Grant role with revoked privilege"): + role_admin_name = f"role_admin_{getuid()}" + target_user_name = f"target_user_{getuid()}" + + with user(node, target_user_name), role(node, role_admin_name): + + with When(f"I grant ROLE ADMIN on the database"): + node.query(f"GRANT ROLE ADMIN ON *.* TO {grant_target_name}") + + with And(f"I revoke ROLE ADMIN on the database"): + node.query(f"REVOKE ROLE ADMIN ON *.* FROM {grant_target_name}") + + with Then("I check the user cannot grant a role"): + node.query(f"GRANT {role_admin_name} TO {target_user_name}", settings=[("user",user_name)], + exitcode=exitcode, message=message) + +@TestFeature +@Name("role admin") +@Requirements( + RQ_SRS_006_RBAC_Privileges_RoleAdmin("1.0"), +) +def feature(self, node="clickhouse1"): + """Check the RBAC functionality of ROLE ADMIN. + """ + self.context.node = self.context.cluster.node(node) + + Suite(run=privileges_granted_directly, setup=instrument_clickhouse_server_log) + Suite(run=privileges_granted_via_role, setup=instrument_clickhouse_server_log) diff --git a/tests/testflows/rbac/tests/privileges/show/show_quotas.py b/tests/testflows/rbac/tests/privileges/show/show_quotas.py new file mode 100644 index 00000000000..74c77db9f98 --- /dev/null +++ b/tests/testflows/rbac/tests/privileges/show/show_quotas.py @@ -0,0 +1,171 @@ +from testflows.core import * +from testflows.asserts import error + +from rbac.requirements import * +from rbac.helper.common import * +import rbac.helper.errors as errors + +@contextmanager +def quota(node, name): + try: + with Given("I have a quota"): + node.query(f"CREATE QUOTA {name}") + + yield + + finally: + with Finally("I drop the quota"): + node.query(f"DROP QUOTA IF EXISTS {name}") + +@TestSuite +def privileges_granted_directly(self, node=None): + """Check that a user is able to execute `SHOW QUOTAS` with privileges are granted directly. + """ + + user_name = f"user_{getuid()}" + + if node is None: + node = self.context.node + + with user(node, f"{user_name}"): + + Suite(run=check_privilege, flags=TE, + examples=Examples("privilege grant_target_name user_name", [ + tuple(list(row)+[user_name,user_name]) for row in check_privilege.examples + ], args=Args(name="privilege={privilege}", format_name=True))) + +@TestSuite +def privileges_granted_via_role(self, node=None): + """Check that a user is able to execute `SHOW QUOTAS` with privileges are granted through a role. + """ + + user_name = f"user_{getuid()}" + role_name = f"role_{getuid()}" + + if node is None: + node = self.context.node + + with user(node, f"{user_name}"), role(node, f"{role_name}"): + + with When("I grant the role to the user"): + node.query(f"GRANT {role_name} TO {user_name}") + + Suite(run=check_privilege, flags=TE, + examples=Examples("privilege grant_target_name user_name", [ + tuple(list(row)+[role_name,user_name]) for row in check_privilege.examples + ], args=Args(name="privilege={privilege}", format_name=True))) + +@TestOutline(Suite) +@Examples("privilege",[ + ("ACCESS MANAGEMENT",), + ("SHOW ACCESS",), + ("SHOW QUOTAS",), + ("SHOW CREATE QUOTA",), +]) +def check_privilege(self, privilege, grant_target_name, user_name, node=None): + """Run checks for commands that require SHOW QUOTAS privilege. + """ + + if node is None: + node = self.context.node + + Suite(test=show_quotas, setup=instrument_clickhouse_server_log)(privilege=privilege, grant_target_name=grant_target_name, user_name=user_name) + Suite(test=show_create, setup=instrument_clickhouse_server_log)(privilege=privilege, grant_target_name=grant_target_name, user_name=user_name) + +@TestSuite +@Requirements( + RQ_SRS_006_RBAC_Privileges_ShowQuotas_Query("1.0"), +) +def show_quotas(self, privilege, grant_target_name, user_name, node=None): + """Check that user is only able to execute `SHOW QUOTAS` when they have the necessary privilege. + """ + exitcode, message = errors.not_enough_privileges(name=user_name) + + if node is None: + node = self.context.node + + with Scenario("SHOW QUOTAS without privilege"): + + with When("I check the user can't use SHOW QUOTAS"): + node.query(f"SHOW QUOTAS", settings=[("user",user_name)], + exitcode=exitcode, message=message) + + with Scenario("SHOW QUOTAS with privilege"): + + with When(f"I grant {privilege}"): + node.query(f"GRANT {privilege} ON *.* TO {grant_target_name}") + + with Then("I check the user can use SHOW QUOTAS"): + node.query(f"SHOW QUOTAS", settings = [("user", f"{user_name}")]) + + with Scenario("SHOW QUOTAS with revoked privilege"): + + with When(f"I grant {privilege}"): + node.query(f"GRANT {privilege} ON *.* TO {grant_target_name}") + + with And(f"I revoke {privilege}"): + node.query(f"REVOKE {privilege} ON *.* FROM {grant_target_name}") + + with Then("I check the user cannot use SHOW QUOTAS"): + node.query(f"SHOW QUOTAS", settings=[("user",user_name)], + exitcode=exitcode, message=message) + +@TestSuite +@Requirements( + RQ_SRS_006_RBAC_Privileges_ShowCreateQuota("1.0"), +) +def show_create(self, privilege, grant_target_name, user_name, node=None): + """Check that user is only able to execute `SHOW CREATE QUOTA` when they have the necessary privilege. + """ + exitcode, message = errors.not_enough_privileges(name=user_name) + + if node is None: + node = self.context.node + + with Scenario("SHOW CREATE QUOTA without privilege"): + target_quota_name = f"target_quota_{getuid()}" + + with quota(node, target_quota_name): + + with When("I check the user can't use SHOW CREATE QUOTA"): + node.query(f"SHOW CREATE QUOTA {target_quota_name}", settings=[("user",user_name)], + exitcode=exitcode, message=message) + + with Scenario("SHOW CREATE QUOTA with privilege"): + target_quota_name = f"target_quota_{getuid()}" + + with quota(node, target_quota_name): + + with When(f"I grant {privilege}"): + node.query(f"GRANT {privilege} ON *.* TO {grant_target_name}") + + with Then("I check the user can use SHOW CREATE QUOTA"): + node.query(f"SHOW CREATE QUOTA {target_quota_name}", settings = [("user", f"{user_name}")]) + + with Scenario("SHOW CREATE QUOTA with revoked privilege"): + target_quota_name = f"target_quota_{getuid()}" + + with quota(node, target_quota_name): + + with When(f"I grant {privilege}"): + node.query(f"GRANT {privilege} ON *.* TO {grant_target_name}") + + with And(f"I revoke {privilege}"): + node.query(f"REVOKE {privilege} ON *.* FROM {grant_target_name}") + + with Then("I check the user cannot use SHOW CREATE QUOTA"): + node.query(f"SHOW CREATE QUOTA {target_quota_name}", settings=[("user",user_name)], + exitcode=exitcode, message=message) + +@TestFeature +@Name("show quotas") +@Requirements( + RQ_SRS_006_RBAC_Privileges_ShowQuotas("1.0"), +) +def feature(self, node="clickhouse1"): + """Check the RBAC functionality of SHOW QUOTAS. + """ + self.context.node = self.context.cluster.node(node) + + Suite(run=privileges_granted_directly, setup=instrument_clickhouse_server_log) + Suite(run=privileges_granted_via_role, setup=instrument_clickhouse_server_log) diff --git a/tests/testflows/rbac/tests/privileges/show/show_roles.py b/tests/testflows/rbac/tests/privileges/show/show_roles.py new file mode 100644 index 00000000000..64eeb5c599d --- /dev/null +++ b/tests/testflows/rbac/tests/privileges/show/show_roles.py @@ -0,0 +1,159 @@ +from testflows.core import * +from testflows.asserts import error + +from rbac.requirements import * +from rbac.helper.common import * +import rbac.helper.errors as errors + +@TestSuite +def privileges_granted_directly(self, node=None): + """Check that a user is able to execute `SHOW ROLES` with privileges are granted directly. + """ + + user_name = f"user_{getuid()}" + + if node is None: + node = self.context.node + + with user(node, f"{user_name}"): + + Suite(run=check_privilege, flags=TE, + examples=Examples("privilege grant_target_name user_name", [ + tuple(list(row)+[user_name,user_name]) for row in check_privilege.examples + ], args=Args(name="privilege={privilege}", format_name=True))) + +@TestSuite +def privileges_granted_via_role(self, node=None): + """Check that a user is able to execute `SHOW ROLES` with privileges are granted through a role. + """ + + user_name = f"user_{getuid()}" + role_name = f"role_{getuid()}" + + if node is None: + node = self.context.node + + with user(node, f"{user_name}"), role(node, f"{role_name}"): + + with When("I grant the role to the user"): + node.query(f"GRANT {role_name} TO {user_name}") + + Suite(run=check_privilege, flags=TE, + examples=Examples("privilege grant_target_name user_name", [ + tuple(list(row)+[role_name,user_name]) for row in check_privilege.examples + ], args=Args(name="privilege={privilege}", format_name=True))) + +@TestOutline(Suite) +@Examples("privilege",[ + ("ACCESS MANAGEMENT",), + ("SHOW ACCESS",), + ("SHOW ROLES",), + ("SHOW CREATE ROLE",), +]) +def check_privilege(self, privilege, grant_target_name, user_name, node=None): + """Run checks for commands that require SHOW ROLES privilege. + """ + + if node is None: + node = self.context.node + + Suite(test=show_roles, setup=instrument_clickhouse_server_log)(privilege=privilege, grant_target_name=grant_target_name, user_name=user_name) + Suite(test=show_create, setup=instrument_clickhouse_server_log)(privilege=privilege, grant_target_name=grant_target_name, user_name=user_name) + +@TestSuite +@Requirements( + RQ_SRS_006_RBAC_Privileges_ShowRoles_Query("1.0"), +) +def show_roles(self, privilege, grant_target_name, user_name, node=None): + """Check that user is only able to execute `SHOW ROLES` when they have the necessary privilege. + """ + exitcode, message = errors.not_enough_privileges(name=user_name) + + if node is None: + node = self.context.node + + with Scenario("SHOW ROLES without privilege"): + + with When("I check the user can't use SHOW ROLES"): + node.query(f"SHOW ROLES", settings=[("user",user_name)], + exitcode=exitcode, message=message) + + with Scenario("SHOW ROLES with privilege"): + + with When(f"I grant {privilege}"): + node.query(f"GRANT {privilege} ON *.* TO {grant_target_name}") + + with Then("I check the user can use SHOW ROLES"): + node.query(f"SHOW ROLES", settings = [("user", f"{user_name}")]) + + with Scenario("SHOW ROLES with revoked privilege"): + + with When(f"I grant {privilege}"): + node.query(f"GRANT {privilege} ON *.* TO {grant_target_name}") + + with And(f"I revoke {privilege}"): + node.query(f"REVOKE {privilege} ON *.* FROM {grant_target_name}") + + with Then("I check the user cannot use SHOW ROLES"): + node.query(f"SHOW ROLES", settings=[("user",user_name)], + exitcode=exitcode, message=message) + +@TestSuite +@Requirements( + RQ_SRS_006_RBAC_Privileges_ShowCreateRole("1.0"), +) +def show_create(self, privilege, grant_target_name, user_name, node=None): + """Check that user is only able to execute `SHOW CREATE ROLE` when they have the necessary privilege. + """ + exitcode, message = errors.not_enough_privileges(name=user_name) + + if node is None: + node = self.context.node + + with Scenario("SHOW CREATE ROLE without privilege"): + target_role_name = f"target_role_{getuid()}" + + with role(node, target_role_name): + + with When("I check the user can't use SHOW CREATE ROLE"): + node.query(f"SHOW CREATE ROLE {target_role_name}", settings=[("user",user_name)], + exitcode=exitcode, message=message) + + with Scenario("SHOW CREATE ROLE with privilege"): + target_role_name = f"target_role_{getuid()}" + + with role(node, target_role_name): + + with When(f"I grant {privilege}"): + node.query(f"GRANT {privilege} ON *.* TO {grant_target_name}") + + with Then("I check the user can use SHOW CREATE ROLE"): + node.query(f"SHOW CREATE ROLE {target_role_name}", settings = [("user", f"{user_name}")]) + + with Scenario("SHOW CREATE ROLE with revoked privilege"): + target_role_name = f"target_role_{getuid()}" + + with role(node, target_role_name): + + with When(f"I grant {privilege}"): + node.query(f"GRANT {privilege} ON *.* TO {grant_target_name}") + + with And(f"I revoke {privilege}"): + node.query(f"REVOKE {privilege} ON *.* FROM {grant_target_name}") + + with Then("I check the user cannot use SHOW CREATE ROLE"): + node.query(f"SHOW CREATE ROLE {target_role_name}", settings=[("user",user_name)], + exitcode=exitcode, message=message) + +@TestFeature +@Name("show roles") +@Requirements( + RQ_SRS_006_RBAC_Privileges_ShowRoles("1.0"), +) +def feature(self, node="clickhouse1"): + """Check the RBAC functionality of SHOW ROLES. + """ + self.context.node = self.context.cluster.node(node) + + Suite(run=privileges_granted_directly, setup=instrument_clickhouse_server_log) + Suite(run=privileges_granted_via_role, setup=instrument_clickhouse_server_log) diff --git a/tests/testflows/rbac/tests/privileges/show/show_row_policies.py b/tests/testflows/rbac/tests/privileges/show/show_row_policies.py new file mode 100644 index 00000000000..a375545e6b8 --- /dev/null +++ b/tests/testflows/rbac/tests/privileges/show/show_row_policies.py @@ -0,0 +1,176 @@ +from testflows.core import * +from testflows.asserts import error + +from rbac.requirements import * +from rbac.helper.common import * +import rbac.helper.errors as errors + +@contextmanager +def row_policy(node, name, table): + try: + with Given("I have a row policy"): + node.query(f"CREATE ROW POLICY {name} ON {table}") + + yield + + finally: + with Finally("I drop the row policy"): + node.query(f"DROP ROW POLICY IF EXISTS {name} ON {table}") + +@TestSuite +def privileges_granted_directly(self, node=None): + """Check that a user is able to execute `SHOW ROW POLICIES` with privileges are granted directly. + """ + + user_name = f"user_{getuid()}" + + if node is None: + node = self.context.node + + with user(node, f"{user_name}"): + + Suite(run=check_privilege, flags=TE, + examples=Examples("privilege grant_target_name user_name", [ + tuple(list(row)+[user_name,user_name]) for row in check_privilege.examples + ], args=Args(name="privilege={privilege}", format_name=True))) + +@TestSuite +def privileges_granted_via_role(self, node=None): + """Check that a user is able to execute `SHOW ROW POLICIES` with privileges are granted through a role. + """ + + user_name = f"user_{getuid()}" + role_name = f"role_{getuid()}" + + if node is None: + node = self.context.node + + with user(node, f"{user_name}"), role(node, f"{role_name}"): + + with When("I grant the role to the user"): + node.query(f"GRANT {role_name} TO {user_name}") + + Suite(run=check_privilege, flags=TE, + examples=Examples("privilege grant_target_name user_name", [ + tuple(list(row)+[role_name,user_name]) for row in check_privilege.examples + ], args=Args(name="privilege={privilege}", format_name=True))) + +@TestOutline(Suite) +@Examples("privilege",[ + ("ACCESS MANAGEMENT",), + ("SHOW ACCESS",), + ("SHOW ROW POLICIES",), + ("SHOW POLICIES",), + ("SHOW CREATE ROW POLICY",), + ("SHOW CREATE POLICY",), +]) +def check_privilege(self, privilege, grant_target_name, user_name, node=None): + """Run checks for commands that require SHOW ROW POLICIES privilege. + """ + + if node is None: + node = self.context.node + + Suite(test=show_row_policies, setup=instrument_clickhouse_server_log)(privilege=privilege, grant_target_name=grant_target_name, user_name=user_name) + Suite(test=show_create, setup=instrument_clickhouse_server_log)(privilege=privilege, grant_target_name=grant_target_name, user_name=user_name) + +@TestSuite +@Requirements( + RQ_SRS_006_RBAC_Privileges_ShowRowPolicies_Query("1.0"), +) +def show_row_policies(self, privilege, grant_target_name, user_name, node=None): + """Check that user is only able to execute `SHOW ROW POLICIES` when they have the necessary privilege. + """ + exitcode, message = errors.not_enough_privileges(name=user_name) + + if node is None: + node = self.context.node + + with Scenario("SHOW ROW POLICIES without privilege"): + + with When("I check the user can't use SHOW ROW POLICIES"): + node.query(f"SHOW ROW POLICIES", settings=[("user",user_name)], + exitcode=exitcode, message=message) + + with Scenario("SHOW ROW POLICIES with privilege"): + + with When(f"I grant {privilege}"): + node.query(f"GRANT {privilege} ON *.* TO {grant_target_name}") + + with Then("I check the user can use SHOW ROW POLICIES"): + node.query(f"SHOW ROW POLICIES", settings = [("user", f"{user_name}")]) + + with Scenario("SHOW ROW POLICIES with revoked privilege"): + + with When(f"I grant {privilege}"): + node.query(f"GRANT {privilege} ON *.* TO {grant_target_name}") + + with And(f"I revoke {privilege}"): + node.query(f"REVOKE {privilege} ON *.* FROM {grant_target_name}") + + with Then("I check the user cannot use SHOW ROW POLICIES"): + node.query(f"SHOW ROW POLICIES", settings=[("user",user_name)], + exitcode=exitcode, message=message) + +@TestSuite +@Requirements( + RQ_SRS_006_RBAC_Privileges_ShowCreateRowPolicy("1.0"), +) +def show_create(self, privilege, grant_target_name, user_name, node=None): + """Check that user is only able to execute `SHOW CREATE ROW POLICY` when they have the necessary privilege. + """ + exitcode, message = errors.not_enough_privileges(name=user_name) + + if node is None: + node = self.context.node + + with Scenario("SHOW CREATE ROW POLICY without privilege"): + target_row_policy_name = f"target_row_policy_{getuid()}" + table_name = f"table_{getuid()}" + + with row_policy(node, target_row_policy_name, table_name): + + with When("I check the user can't use SHOW CREATE ROW POLICY"): + node.query(f"SHOW CREATE ROW POLICY {target_row_policy_name}", settings=[("user",user_name)], + exitcode=exitcode, message=message) + + with Scenario("SHOW CREATE ROW POLICY with privilege"): + target_row_policy_name = f"target_row_policy_{getuid()}" + table_name = f"table_{getuid()}" + + with row_policy(node, target_row_policy_name, table_name): + + with When(f"I grant {privilege}"): + node.query(f"GRANT {privilege} ON *.* TO {grant_target_name}") + + with Then("I check the user can use SHOW CREATE ROW POLICY"): + node.query(f"SHOW CREATE ROW POLICY {target_row_policy_name}", settings = [("user", f"{user_name}")]) + + with Scenario("SHOW CREATE ROW POLICY with revoked privilege"): + target_row_policy_name = f"target_row_policy_{getuid()}" + table_name = f"table_{getuid()}" + + with row_policy(node, target_row_policy_name, table_name): + + with When(f"I grant {privilege}"): + node.query(f"GRANT {privilege} ON *.* TO {grant_target_name}") + + with And(f"I revoke {privilege}"): + node.query(f"REVOKE {privilege} ON *.* FROM {grant_target_name}") + + with Then("I check the user cannot use SHOW CREATE ROW POLICY"): + node.query(f"SHOW CREATE ROW POLICY {target_row_policy_name}", settings=[("user",user_name)], + exitcode=exitcode, message=message) + +@TestFeature +@Name("show row policies") +@Requirements( + RQ_SRS_006_RBAC_Privileges_ShowRowPolicies("1.0"), +) +def feature(self, node="clickhouse1"): + """Check the RBAC functionality of SHOW ROW POLICYS. + """ + self.context.node = self.context.cluster.node(node) + + Suite(run=privileges_granted_directly, setup=instrument_clickhouse_server_log) + Suite(run=privileges_granted_via_role, setup=instrument_clickhouse_server_log) diff --git a/tests/testflows/rbac/tests/privileges/show/show_settings_profiles.py b/tests/testflows/rbac/tests/privileges/show/show_settings_profiles.py new file mode 100644 index 00000000000..1b1e660fffd --- /dev/null +++ b/tests/testflows/rbac/tests/privileges/show/show_settings_profiles.py @@ -0,0 +1,173 @@ +from testflows.core import * +from testflows.asserts import error + +from rbac.requirements import * +from rbac.helper.common import * +import rbac.helper.errors as errors + +@contextmanager +def settings_profile(node, name): + try: + with Given("I have a settings_profile"): + node.query(f"CREATE SETTINGS PROFILE {name}") + + yield + + finally: + with Finally("I drop the settings_profile"): + node.query(f"DROP SETTINGS PROFILE IF EXISTS {name}") + +@TestSuite +def privileges_granted_directly(self, node=None): + """Check that a user is able to execute `SHOW SETTINGS PROFILES` with privileges are granted directly. + """ + + user_name = f"user_{getuid()}" + + if node is None: + node = self.context.node + + with user(node, f"{user_name}"): + + Suite(run=check_privilege, flags=TE, + examples=Examples("privilege grant_target_name user_name", [ + tuple(list(row)+[user_name,user_name]) for row in check_privilege.examples + ], args=Args(name="privilege={privilege}", format_name=True))) + +@TestSuite +def privileges_granted_via_role(self, node=None): + """Check that a user is able to execute `SHOW SETTINGS PROFILES` with privileges are granted through a role. + """ + + user_name = f"user_{getuid()}" + role_name = f"role_{getuid()}" + + if node is None: + node = self.context.node + + with user(node, f"{user_name}"), role(node, f"{role_name}"): + + with When("I grant the role to the user"): + node.query(f"GRANT {role_name} TO {user_name}") + + Suite(run=check_privilege, flags=TE, + examples=Examples("privilege grant_target_name user_name", [ + tuple(list(row)+[role_name,user_name]) for row in check_privilege.examples + ], args=Args(name="privilege={privilege}", format_name=True))) + +@TestOutline(Suite) +@Examples("privilege",[ + ("ACCESS MANAGEMENT",), + ("SHOW ACCESS",), + ("SHOW SETTINGS PROFILES",), + ("SHOW PROFILES",), + ("SHOW CREATE SETTINGS PROFILE",), + ("SHOW CREATE PROFILE",), +]) +def check_privilege(self, privilege, grant_target_name, user_name, node=None): + """Run checks for commands that require SHOW SETTINGS PROFILES privilege. + """ + + if node is None: + node = self.context.node + + Suite(test=show_settings_profiles, setup=instrument_clickhouse_server_log)(privilege=privilege, grant_target_name=grant_target_name, user_name=user_name) + Suite(test=show_create, setup=instrument_clickhouse_server_log)(privilege=privilege, grant_target_name=grant_target_name, user_name=user_name) + +@TestSuite +@Requirements( + RQ_SRS_006_RBAC_Privileges_ShowSettingsProfiles_Query("1.0"), +) +def show_settings_profiles(self, privilege, grant_target_name, user_name, node=None): + """Check that user is only able to execute `SHOW SETTINGS PROFILES` when they have the necessary privilege. + """ + exitcode, message = errors.not_enough_privileges(name=user_name) + + if node is None: + node = self.context.node + + with Scenario("SHOW SETTINGS PROFILES without privilege"): + + with When("I check the user can't use SHOW SETTINGS PROFILES"): + node.query(f"SHOW SETTINGS PROFILES", settings=[("user",user_name)], + exitcode=exitcode, message=message) + + with Scenario("SHOW SETTINGS PROFILES with privilege"): + + with When(f"I grant {privilege}"): + node.query(f"GRANT {privilege} ON *.* TO {grant_target_name}") + + with Then("I check the user can use SHOW SETTINGS PROFILES"): + node.query(f"SHOW SETTINGS PROFILES", settings = [("user", f"{user_name}")]) + + with Scenario("SHOW SETTINGS PROFILES with revoked privilege"): + + with When(f"I grant {privilege}"): + node.query(f"GRANT {privilege} ON *.* TO {grant_target_name}") + + with And(f"I revoke {privilege}"): + node.query(f"REVOKE {privilege} ON *.* FROM {grant_target_name}") + + with Then("I check the user cannot use SHOW SETTINGS PROFILES"): + node.query(f"SHOW SETTINGS PROFILES", settings=[("user",user_name)], + exitcode=exitcode, message=message) + +@TestSuite +@Requirements( + RQ_SRS_006_RBAC_Privileges_ShowCreateSettingsProfile("1.0"), +) +def show_create(self, privilege, grant_target_name, user_name, node=None): + """Check that user is only able to execute `SHOW CREATE SETTINGS PROFILE` when they have the necessary privilege. + """ + exitcode, message = errors.not_enough_privileges(name=user_name) + + if node is None: + node = self.context.node + + with Scenario("SHOW CREATE SETTINGS PROFILE without privilege"): + target_settings_profile_name = f"target_settings_profile_{getuid()}" + + with settings_profile(node, target_settings_profile_name): + + with When("I check the user can't use SHOW CREATE SETTINGS PROFILE"): + node.query(f"SHOW CREATE SETTINGS PROFILE {target_settings_profile_name}", settings=[("user",user_name)], + exitcode=exitcode, message=message) + + with Scenario("SHOW CREATE SETTINGS PROFILE with privilege"): + target_settings_profile_name = f"target_settings_profile_{getuid()}" + + with settings_profile(node, target_settings_profile_name): + + with When(f"I grant {privilege}"): + node.query(f"GRANT {privilege} ON *.* TO {grant_target_name}") + + with Then("I check the user can use SHOW CREATE SETTINGS PROFILE"): + node.query(f"SHOW CREATE SETTINGS PROFILE {target_settings_profile_name}", settings = [("user", f"{user_name}")]) + + with Scenario("SHOW CREATE SETTINGS PROFILE with revoked privilege"): + target_settings_profile_name = f"target_settings_profile_{getuid()}" + + with settings_profile(node, target_settings_profile_name): + + with When(f"I grant {privilege}"): + node.query(f"GRANT {privilege} ON *.* TO {grant_target_name}") + + with And(f"I revoke {privilege}"): + node.query(f"REVOKE {privilege} ON *.* FROM {grant_target_name}") + + with Then("I check the user cannot use SHOW CREATE SETTINGS PROFILE"): + node.query(f"SHOW CREATE SETTINGS PROFILE {target_settings_profile_name}", settings=[("user",user_name)], + exitcode=exitcode, message=message) + +@TestFeature +@Name("show settings profiles") +@Requirements( + RQ_SRS_006_RBAC_Privileges_ShowSettingsProfiles("1.0"), +) +def feature(self, node="clickhouse1"): + """Check the RBAC functionality of SHOW SETTINGS PROFILES. + """ + self.context.node = self.context.cluster.node(node) + + Suite(run=privileges_granted_directly, setup=instrument_clickhouse_server_log) + Suite(run=privileges_granted_via_role, setup=instrument_clickhouse_server_log) diff --git a/tests/testflows/rbac/tests/privileges/show/show_users.py b/tests/testflows/rbac/tests/privileges/show/show_users.py new file mode 100644 index 00000000000..08c5ca65051 --- /dev/null +++ b/tests/testflows/rbac/tests/privileges/show/show_users.py @@ -0,0 +1,159 @@ +from testflows.core import * +from testflows.asserts import error + +from rbac.requirements import * +from rbac.helper.common import * +import rbac.helper.errors as errors + +@TestSuite +def privileges_granted_directly(self, node=None): + """Check that a user is able to execute `SHOW USERS` with privileges are granted directly. + """ + + user_name = f"user_{getuid()}" + + if node is None: + node = self.context.node + + with user(node, f"{user_name}"): + + Suite(run=check_privilege, flags=TE, + examples=Examples("privilege grant_target_name user_name", [ + tuple(list(row)+[user_name,user_name]) for row in check_privilege.examples + ], args=Args(name="privilege={privilege}", format_name=True))) + +@TestSuite +def privileges_granted_via_role(self, node=None): + """Check that a user is able to execute `SHOW USERS` with privileges are granted through a role. + """ + + user_name = f"user_{getuid()}" + role_name = f"role_{getuid()}" + + if node is None: + node = self.context.node + + with user(node, f"{user_name}"), role(node, f"{role_name}"): + + with When("I grant the role to the user"): + node.query(f"GRANT {role_name} TO {user_name}") + + Suite(run=check_privilege, flags=TE, + examples=Examples("privilege grant_target_name user_name", [ + tuple(list(row)+[role_name,user_name]) for row in check_privilege.examples + ], args=Args(name="privilege={privilege}", format_name=True))) + +@TestOutline(Suite) +@Examples("privilege",[ + ("ACCESS MANAGEMENT",), + ("SHOW ACCESS",), + ("SHOW USERS",), + ("SHOW CREATE USER",), +]) +def check_privilege(self, privilege, grant_target_name, user_name, node=None): + """Run checks for commands that require SHOW USERS privilege. + """ + + if node is None: + node = self.context.node + + Suite(test=show_users, setup=instrument_clickhouse_server_log)(privilege=privilege, grant_target_name=grant_target_name, user_name=user_name) + Suite(test=show_create, setup=instrument_clickhouse_server_log)(privilege=privilege, grant_target_name=grant_target_name, user_name=user_name) + +@TestSuite +@Requirements( + RQ_SRS_006_RBAC_Privileges_ShowUsers_Query("1.0"), +) +def show_users(self, privilege, grant_target_name, user_name, node=None): + """Check that user is only able to execute `SHOW USERS` when they have the necessary privilege. + """ + exitcode, message = errors.not_enough_privileges(name=user_name) + + if node is None: + node = self.context.node + + with Scenario("SHOW USERS without privilege"): + + with When("I check the user can't use SHOW USERS"): + node.query(f"SHOW USERS", settings=[("user",user_name)], + exitcode=exitcode, message=message) + + with Scenario("SHOW USERS with privilege"): + + with When(f"I grant {privilege}"): + node.query(f"GRANT {privilege} ON *.* TO {grant_target_name}") + + with Then("I check the user can use SHOW USERS"): + node.query(f"SHOW USERS", settings = [("user", f"{user_name}")]) + + with Scenario("SHOW USERS with revoked privilege"): + + with When(f"I grant {privilege}"): + node.query(f"GRANT {privilege} ON *.* TO {grant_target_name}") + + with And(f"I revoke {privilege}"): + node.query(f"REVOKE {privilege} ON *.* FROM {grant_target_name}") + + with Then("I check the user cannot use SHOW USERS"): + node.query(f"SHOW USERS", settings=[("user",user_name)], + exitcode=exitcode, message=message) + +@TestSuite +@Requirements( + RQ_SRS_006_RBAC_Privileges_ShowCreateUser("1.0"), +) +def show_create(self, privilege, grant_target_name, user_name, node=None): + """Check that user is only able to execute `SHOW CREATE USER` when they have the necessary privilege. + """ + exitcode, message = errors.not_enough_privileges(name=user_name) + + if node is None: + node = self.context.node + + with Scenario("SHOW CREATE USER without privilege"): + target_user_name = f"target_user_{getuid()}" + + with user(node, target_user_name): + + with When("I check the user can't use SHOW CREATE USER"): + node.query(f"SHOW CREATE USER {target_user_name}", settings=[("user",user_name)], + exitcode=exitcode, message=message) + + with Scenario("SHOW CREATE USER with privilege"): + target_user_name = f"target_user_{getuid()}" + + with user(node, target_user_name): + + with When(f"I grant {privilege}"): + node.query(f"GRANT {privilege} ON *.* TO {grant_target_name}") + + with Then("I check the user can use SHOW CREATE USER"): + node.query(f"SHOW CREATE USER {target_user_name}", settings = [("user", f"{user_name}")]) + + with Scenario("SHOW CREATE USER with revoked privilege"): + target_user_name = f"target_user_{getuid()}" + + with user(node, target_user_name): + + with When(f"I grant {privilege}"): + node.query(f"GRANT {privilege} ON *.* TO {grant_target_name}") + + with And(f"I revoke {privilege}"): + node.query(f"REVOKE {privilege} ON *.* FROM {grant_target_name}") + + with Then("I check the user cannot use SHOW CREATE USER"): + node.query(f"SHOW CREATE USER {target_user_name}", settings=[("user",user_name)], + exitcode=exitcode, message=message) + +@TestFeature +@Name("show users") +@Requirements( + RQ_SRS_006_RBAC_Privileges_ShowUsers("1.0"), +) +def feature(self, node="clickhouse1"): + """Check the RBAC functionality of SHOW USERS. + """ + self.context.node = self.context.cluster.node(node) + + Suite(run=privileges_granted_directly, setup=instrument_clickhouse_server_log) + Suite(run=privileges_granted_via_role, setup=instrument_clickhouse_server_log) From c23e3764748f5e7dc1643ed971e3a52976c74954 Mon Sep 17 00:00:00 2001 From: sevirov <72220289+sevirov@users.noreply.github.com> Date: Fri, 4 Dec 2020 21:17:58 +0300 Subject: [PATCH 36/50] DOCSUP-4435: Allow nullable key in MergeTree (#17345) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit * Update settings.md and mergetree.md Задокументировал настройку allow_nullable_key. Добавил пару предложений в mergetree.md. * Update mergetree.md Исправляю битую ссылку. * Update mergetree.md Вставил ссылку на allow_nullable_key. * Update mergetree.md Правлю битую ссылку. * Update mergetree.md Исправляю битую ссылку. * Update settings.md Правлю битую ссылку. * Update mergetree.md and settings.md Выполнил перевод на русский язык и поправил немного английскую версию. * Update mergetree.md Убрал выражение 'для семейства mergetree'. * Update settings.md Внес поправки в русскую версию. * Update mergetree.md and settings.md Сделал поправки в английскую версию на основе комментарий в PR. Co-authored-by: Dmitriy --- .../table-engines/mergetree-family/mergetree.md | 4 ++++ docs/en/operations/settings/settings.md | 11 +++++++++++ .../table-engines/mergetree-family/mergetree.md | 4 ++++ docs/ru/operations/settings/settings.md | 11 +++++++++++ 4 files changed, 30 insertions(+) diff --git a/docs/en/engines/table-engines/mergetree-family/mergetree.md b/docs/en/engines/table-engines/mergetree-family/mergetree.md index 5f99ff99dab..8ca29c3a9b5 100644 --- a/docs/en/engines/table-engines/mergetree-family/mergetree.md +++ b/docs/en/engines/table-engines/mergetree-family/mergetree.md @@ -184,6 +184,10 @@ Sparse indexes allow you to work with a very large number of table rows, because ClickHouse does not require a unique primary key. You can insert multiple rows with the same primary key. +You can use `Nullable`-typed expressions in the `PRIMARY KEY` and `ORDER BY` clauses. To allow this feature, turn on the [allow_nullable_key](../../../operations/settings/settings.md#allow-nullable-key) setting. + +The [NULLS_LAST](../../../sql-reference/statements/select/order-by.md#sorting-of-special-values) principle applies for `NULL` values in the `ORDER BY` clause. + ### Selecting the Primary Key {#selecting-the-primary-key} The number of columns in the primary key is not explicitly limited. Depending on the data structure, you can include more or fewer columns in the primary key. This may: diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index 8346d5ceac9..29c74b76c29 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -2364,4 +2364,15 @@ Allows configurable `NULL` representation for [TSV](../../interfaces/formats.md# Default value: `\N`. +## allow_nullable_key {#allow-nullable-key} + +Allows using of the [Nullable](../../sql-reference/data-types/nullable.md#data_type-nullable)-typed values in a sorting and a primary key for [MergeTree](../../engines/table-engines/mergetree-family/mergetree.md#table_engines-mergetree) tables. + +Possible values: + +- 1 — `Nullable`-type expressions are allowed in keys. +- 0 — `Nullable`-type expressions are not allowed in keys. + +Default value: `0`. + [Original article](https://clickhouse.tech/docs/en/operations/settings/settings/) diff --git a/docs/ru/engines/table-engines/mergetree-family/mergetree.md b/docs/ru/engines/table-engines/mergetree-family/mergetree.md index f738ce13d7c..501f7732745 100644 --- a/docs/ru/engines/table-engines/mergetree-family/mergetree.md +++ b/docs/ru/engines/table-engines/mergetree-family/mergetree.md @@ -177,6 +177,10 @@ Marks numbers: 0 1 2 3 4 5 6 7 8 ClickHouse не требует уникального первичного ключа. Можно вставить много строк с одинаковым первичным ключом. +Ключ в `PRIMARY KEY` и `ORDER BY` может иметь тип `Nullable`. За поддержку этой возможности отвечает настройка [allow_nullable_key](../../../operations/settings/settings.md#allow-nullable-key). + +При сортировке с использованием выражения `ORDER BY` для значений `NULL` всегда работает принцип [NULLS_LAST](../../../sql-reference/statements/select/order-by.md#sorting-of-special-values). + ### Выбор первичного ключа {#vybor-pervichnogo-kliucha} Количество столбцов в первичном ключе не ограничено явным образом. В зависимости от структуры данных в первичный ключ можно включать больше или меньше столбцов. Это может: diff --git a/docs/ru/operations/settings/settings.md b/docs/ru/operations/settings/settings.md index b04a927f944..9750f777990 100644 --- a/docs/ru/operations/settings/settings.md +++ b/docs/ru/operations/settings/settings.md @@ -2235,4 +2235,15 @@ SELECT CAST(toNullable(toInt32(0)) AS Int32) as x, toTypeName(x); Значение по умолчанию: `\N`. +## allow_nullable_key {#allow-nullable-key} + +Включает или отключает поддержку типа [Nullable](../../sql-reference/data-types/nullable.md#data_type-nullable) для ключей таблиц [MergeTree](../../engines/table-engines/mergetree-family/mergetree.md#table_engines-mergetree). + +Возможные значения: + +- 1 — включает поддержку типа `Nullable` для ключей таблиц. +- 0 — отключает поддержку типа `Nullable` для ключей таблиц. + +Значение по умолчанию: `0`. + [Оригинальная статья](https://clickhouse.tech/docs/ru/operations/settings/settings/) From ac0ebd515eaaf70111a11c00562edf6466dbfb85 Mon Sep 17 00:00:00 2001 From: Anna <42538400+adevyatova@users.noreply.github.com> Date: Fri, 4 Dec 2020 21:24:05 +0300 Subject: [PATCH 37/50] DOCSUP-4193: Document the EXPLAIN statement (#17734) * Add EXPLAIN query description * Check examples * update * Update docs/en/sql-reference/statements/explain.md Co-authored-by: BayoNet * Update docs/en/sql-reference/statements/explain.md Co-authored-by: Nikolai Kochetov Co-authored-by: BayoNet Co-authored-by: Nikolai Kochetov --- docs/en/sql-reference/statements/explain.md | 150 ++++++++++++++++++++ docs/en/sql-reference/statements/index.md | 1 + 2 files changed, 151 insertions(+) create mode 100644 docs/en/sql-reference/statements/explain.md diff --git a/docs/en/sql-reference/statements/explain.md b/docs/en/sql-reference/statements/explain.md new file mode 100644 index 00000000000..3cca29801dd --- /dev/null +++ b/docs/en/sql-reference/statements/explain.md @@ -0,0 +1,150 @@ +--- +toc_priority: 39 +toc_title: EXPLAIN +--- + +# EXPLAIN Statement {#explain} + +Show the execution plan of a statement. + +Syntax: + +```sql +EXPLAIN [AST | SYNTAX | PLAN | PIPELINE] [setting = value, ...] SELECT ... [FORMAT ...] +``` + +Example: + +```sql +EXPLAIN SELECT sum(number) FROM numbers(10) UNION ALL SELECT sum(number) FROM numbers(10) ORDER BY sum(number) ASC FORMAT TSV; +``` + +```sql +Union + Expression (Projection) + Expression (Before ORDER BY and SELECT) + Aggregating + Expression (Before GROUP BY) + SettingQuotaAndLimits (Set limits and quota after reading from storage) + ReadFromStorage (SystemNumbers) + Expression (Projection) + MergingSorted (Merge sorted streams for ORDER BY) + MergeSorting (Merge sorted blocks for ORDER BY) + PartialSorting (Sort each block for ORDER BY) + Expression (Before ORDER BY and SELECT) + Aggregating + Expression (Before GROUP BY) + SettingQuotaAndLimits (Set limits and quota after reading from storage) + ReadFromStorage (SystemNumbers) +``` + +## EXPLAIN Types {#explain-types} + +- `AST` — Abstract syntax tree. +- `SYNTAX` — Query text after AST-level optimizations. +- `PLAN` — Query execution plan. +- `PIPELINE` — Query execution pipeline. + +### EXPLAIN AST {#explain-ast} + +Dump query AST. + +Example: + +```sql +EXPLAIN AST SELECT 1; +``` + +```sql +SelectWithUnionQuery (children 1) + ExpressionList (children 1) + SelectQuery (children 1) + ExpressionList (children 1) + Literal UInt64_1 +``` + +### EXPLAIN SYNTAX {#explain-syntax} + +Return query after syntax optimizations. + +Example: + +```sql +EXPLAIN SYNTAX SELECT * FROM system.numbers AS a, system.numbers AS b, system.numbers AS c; +``` + +```sql +SELECT + `--a.number` AS `a.number`, + `--b.number` AS `b.number`, + number AS `c.number` +FROM +( + SELECT + number AS `--a.number`, + b.number AS `--b.number` + FROM system.numbers AS a + CROSS JOIN system.numbers AS b +) AS `--.s` +CROSS JOIN system.numbers AS c +``` +### EXPLAIN PLAN {#explain-plan} + +Dump query plan steps. + +Settings: + +- `header` — Print output header for step. Default: 0. +- `description` — Print step description. Default: 1. +- `actions` — Print detailed information about step actions. Default: 0. + +Example: + +```sql +EXPLAIN SELECT sum(number) FROM numbers(10) GROUP BY number % 4; +``` + +```sql +Union + Expression (Projection) + Expression (Before ORDER BY and SELECT) + Aggregating + Expression (Before GROUP BY) + SettingQuotaAndLimits (Set limits and quota after reading from storage) + ReadFromStorage (SystemNumbers) +``` + +!!! note "Note" + Step and query cost estimation is not supported. + +### EXPLAIN PIPELINE {#explain-pipeline} + +Settings: + +- `header` — Print header for each output port. Default: 0. +- `graph` — Use DOT graph description language. Default: 0. +- `compact` — Print graph in compact mode if graph is enabled. Default: 1. + +Example: + +```sql +EXPLAIN PIPELINE SELECT sum(number) FROM numbers_mt(100000) GROUP BY number % 4; +``` + +```sql +(Union) +(Expression) +ExpressionTransform + (Expression) + ExpressionTransform + (Aggregating) + Resize 2 → 1 + AggregatingTransform × 2 + (Expression) + ExpressionTransform × 2 + (SettingQuotaAndLimits) + (ReadFromStorage) + NumbersMt × 2 0 → 1 +``` + +[Оriginal article](https://clickhouse.tech/docs/en/sql-reference/statements/explain/) diff --git a/docs/en/sql-reference/statements/index.md b/docs/en/sql-reference/statements/index.md index 837f929ff6d..a317e4a47de 100644 --- a/docs/en/sql-reference/statements/index.md +++ b/docs/en/sql-reference/statements/index.md @@ -29,3 +29,4 @@ Statements represent various kinds of action you can perform using SQL queries. - [SET ROLE](../../sql-reference/statements/set-role.md) - [TRUNCATE](../../sql-reference/statements/truncate.md) - [USE](../../sql-reference/statements/use.md) +- [EXPLAIN](../../sql-reference/statements/explain.md) From ea64ec390f8acee2d9302742f2567a63d0edc849 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sun, 29 Nov 2020 19:44:02 +0300 Subject: [PATCH 38/50] Fix flaky test_ttl_move - replace sleep() with waiting for the message in log - due to previous paragaph, use unique table names - increase sleep timeout in some places --- tests/integration/test_ttl_move/test.py | 82 ++++++++++++++++++------- 1 file changed, 59 insertions(+), 23 deletions(-) diff --git a/tests/integration/test_ttl_move/test.py b/tests/integration/test_ttl_move/test.py index cd906cb04e1..7e01b919589 100644 --- a/tests/integration/test_ttl_move/test.py +++ b/tests/integration/test_ttl_move/test.py @@ -3,11 +3,16 @@ import string import threading import time from multiprocessing.dummy import Pool +from helpers.test_tools import assert_logs_contain_with_retry import pytest from helpers.client import QueryRuntimeException from helpers.cluster import ClickHouseCluster +# FIXME: each sleep(1) is a time bomb, and not only this cause false positive +# it also makes the test not reliable (i.e. assertions may be wrong, due timing issues) +# Seems that some SYSTEM query should be added to wait those things insteadof sleep. + cluster = ClickHouseCluster(__file__) node1 = cluster.add_instance('node1', @@ -58,6 +63,14 @@ def check_used_disks_with_retry(node, table_name, expected_disks, retries): time.sleep(0.5) return False +# Use unique table name for flaky checker, that run tests multiple times +def unique_table_name(base_name): + return f'{base_name}_{int(time.time())}' + +def wait_parts_mover(node, table, *args, **kwargs): + # wait for MergeTreePartsMover + assert_logs_contain_with_retry(node, f'default.{table}.*Removed part from old location', *args, **kwargs) + @pytest.mark.parametrize("name,engine,alter", [ ("mt_test_rule_with_invalid_destination", "MergeTree()", 0), @@ -68,6 +81,8 @@ def check_used_disks_with_retry(node, table_name, expected_disks, retries): "ReplicatedMergeTree('/clickhouse/replicated_test_rule_with_invalid_destination', '1')", 1), ]) def test_rule_with_invalid_destination(started_cluster, name, engine, alter): + name = unique_table_name(name) + try: def get_command(x, policy): x = x or "" @@ -129,6 +144,8 @@ def test_rule_with_invalid_destination(started_cluster, name, engine, alter): "ReplicatedMergeTree('/clickhouse/replicated_test_inserts_to_disk_work', '1')", 1), ]) def test_inserts_to_disk_work(started_cluster, name, engine, positive): + name = unique_table_name(name) + try: node1.query(""" CREATE TABLE {name} ( @@ -164,6 +181,8 @@ def test_inserts_to_disk_work(started_cluster, name, engine, positive): "ReplicatedMergeTree('/clickhouse/test_moves_work_after_storage_policy_change', '1')"), ]) def test_moves_work_after_storage_policy_change(started_cluster, name, engine): + name = unique_table_name(name) + try: node1.query(""" CREATE TABLE {name} ( @@ -184,10 +203,6 @@ def test_moves_work_after_storage_policy_change(started_cluster, name, engine): wait_expire_1 = 12 wait_expire_2 = 4 time_1 = time.time() + wait_expire_1 - time_2 = time.time() + wait_expire_1 + wait_expire_2 - - wait_expire_1_thread = threading.Thread(target=time.sleep, args=(wait_expire_1,)) - wait_expire_1_thread.start() data = [] # 10MB in total for i in range(10): @@ -197,8 +212,7 @@ def test_moves_work_after_storage_policy_change(started_cluster, name, engine): used_disks = get_used_disks_for_table(node1, name) assert set(used_disks) == {"jbod1"} - wait_expire_1_thread.join() - time.sleep(wait_expire_2 / 2) + wait_parts_mover(node1, name, retry_count=40) used_disks = get_used_disks_for_table(node1, name) assert set(used_disks) == {"external"} @@ -218,6 +232,8 @@ def test_moves_work_after_storage_policy_change(started_cluster, name, engine): "ReplicatedMergeTree('/clickhouse/replicated_test_moves_to_disk_work', '1')", 1), ]) def test_moves_to_disk_work(started_cluster, name, engine, positive): + name = unique_table_name(name) + try: node1.query(""" CREATE TABLE {name} ( @@ -230,7 +246,7 @@ def test_moves_to_disk_work(started_cluster, name, engine, positive): """.format(name=name, engine=engine)) wait_expire_1 = 12 - wait_expire_2 = 4 + wait_expire_2 = 20 time_1 = time.time() + wait_expire_1 time_2 = time.time() + wait_expire_1 + wait_expire_2 @@ -264,6 +280,8 @@ def test_moves_to_disk_work(started_cluster, name, engine, positive): "ReplicatedMergeTree('/clickhouse/replicated_test_moves_to_volume_work', '1')"), ]) def test_moves_to_volume_work(started_cluster, name, engine): + name = unique_table_name(name) + try: node1.query(""" CREATE TABLE {name} ( @@ -280,9 +298,6 @@ def test_moves_to_volume_work(started_cluster, name, engine): wait_expire_1 = 10 time_1 = time.time() + wait_expire_1 - wait_expire_1_thread = threading.Thread(target=time.sleep, args=(wait_expire_1,)) - wait_expire_1_thread.start() - for p in range(2): data = [] # 10MB in total for i in range(5): @@ -295,8 +310,7 @@ def test_moves_to_volume_work(started_cluster, name, engine): used_disks = get_used_disks_for_table(node1, name) assert set(used_disks) == {'jbod1', 'jbod2'} - wait_expire_1_thread.join() - time.sleep(1) + wait_parts_mover(node1, name, retry_count=40) used_disks = get_used_disks_for_table(node1, name) assert set(used_disks) == {"external"} @@ -316,6 +330,8 @@ def test_moves_to_volume_work(started_cluster, name, engine): "ReplicatedMergeTree('/clickhouse/replicated_test_inserts_to_volume_work', '1')", 1), ]) def test_inserts_to_volume_work(started_cluster, name, engine, positive): + name = unique_table_name(name) + try: node1.query(""" CREATE TABLE {name} ( @@ -355,6 +371,8 @@ def test_inserts_to_volume_work(started_cluster, name, engine, positive): "ReplicatedMergeTree('/clickhouse/replicated_test_moves_to_disk_eventually_work', '1')"), ]) def test_moves_to_disk_eventually_work(started_cluster, name, engine): + name = unique_table_name(name) + try: name_temp = name + "_temp" @@ -395,7 +413,8 @@ def test_moves_to_disk_eventually_work(started_cluster, name, engine): node1.query("DROP TABLE {} NO DELAY".format(name_temp)) - time.sleep(2) + wait_parts_mover(node1, name) + used_disks = get_used_disks_for_table(node1, name) assert set(used_disks) == {"jbod2"} @@ -407,7 +426,7 @@ def test_moves_to_disk_eventually_work(started_cluster, name, engine): def test_replicated_download_ttl_info(started_cluster): - name = "test_replicated_ttl_info" + name = unique_table_name("test_replicated_ttl_info") engine = "ReplicatedMergeTree('/clickhouse/test_replicated_download_ttl_info', '{replica}')" try: for i, node in enumerate((node1, node2), start=1): @@ -426,6 +445,7 @@ def test_replicated_download_ttl_info(started_cluster): node2.query("INSERT INTO {} (s1, d1) VALUES (randomPrintableASCII(1024*1024), toDateTime({}))".format(name, time.time() - 100)) assert set(get_used_disks_for_table(node2, name)) == {"external"} + time.sleep(1) assert node1.query("SELECT count() FROM {}".format(name)).splitlines() == ["1"] @@ -448,6 +468,8 @@ def test_replicated_download_ttl_info(started_cluster): "ReplicatedMergeTree('/clickhouse/replicated_test_merges_to_disk_work', '1')", 1), ]) def test_merges_to_disk_work(started_cluster, name, engine, positive): + name = unique_table_name(name) + try: node1.query(""" CREATE TABLE {name} ( @@ -463,7 +485,7 @@ def test_merges_to_disk_work(started_cluster, name, engine, positive): node1.query("SYSTEM STOP MOVES {}".format(name)) wait_expire_1 = 16 - wait_expire_2 = 4 + wait_expire_2 = 20 time_1 = time.time() + wait_expire_1 time_2 = time.time() + wait_expire_1 + wait_expire_2 @@ -490,7 +512,6 @@ def test_merges_to_disk_work(started_cluster, name, engine, positive): node1.query("SYSTEM START MERGES {}".format(name)) node1.query("OPTIMIZE TABLE {}".format(name)) - time.sleep(1) used_disks = get_used_disks_for_table(node1, name) assert set(used_disks) == {"external" if positive else "jbod1"} assert "1" == node1.query( @@ -508,6 +529,8 @@ def test_merges_to_disk_work(started_cluster, name, engine, positive): "ReplicatedMergeTree('/clickhouse/replicated_test_merges_with_full_disk_work', '1')"), ]) def test_merges_with_full_disk_work(started_cluster, name, engine): + name = unique_table_name(name) + try: name_temp = name + "_temp" @@ -581,6 +604,8 @@ def test_merges_with_full_disk_work(started_cluster, name, engine): "ReplicatedMergeTree('/clickhouse/replicated_test_moves_after_merges_work', '1')", 1), ]) def test_moves_after_merges_work(started_cluster, name, engine, positive): + name = unique_table_name(name) + try: node1.query(""" CREATE TABLE {name} ( @@ -593,7 +618,7 @@ def test_moves_after_merges_work(started_cluster, name, engine, positive): """.format(name=name, engine=engine)) wait_expire_1 = 16 - wait_expire_2 = 4 + wait_expire_2 = 20 time_1 = time.time() + wait_expire_1 time_2 = time.time() + wait_expire_1 + wait_expire_2 @@ -610,7 +635,6 @@ def test_moves_after_merges_work(started_cluster, name, engine, positive): "INSERT INTO {} (s1, d1) VALUES {}".format(name, ",".join(["(" + ",".join(x) + ")" for x in data]))) node1.query("OPTIMIZE TABLE {}".format(name)) - time.sleep(1) used_disks = get_used_disks_for_table(node1, name) assert set(used_disks) == {"jbod1"} @@ -644,6 +668,8 @@ def test_moves_after_merges_work(started_cluster, name, engine, positive): "ReplicatedMergeTree('/clickhouse/replicated_test_moves_after_alter_work', '1')", 1, "TO DISK 'external'"), ]) def test_ttls_do_not_work_after_alter(started_cluster, name, engine, positive, bar): + name = unique_table_name(name) + try: node1.query(""" CREATE TABLE {name} ( @@ -683,6 +709,8 @@ def test_ttls_do_not_work_after_alter(started_cluster, name, engine, positive, b "ReplicatedMergeTree('/clickhouse/test_materialize_ttl_in_partition', '1')"), ]) def test_materialize_ttl_in_partition(started_cluster, name, engine): + name = unique_table_name(name) + try: node1.query(""" CREATE TABLE {name} ( @@ -702,8 +730,6 @@ def test_materialize_ttl_in_partition(started_cluster, name, engine): node1.query( "INSERT INTO {} (p1, s1, d1) VALUES {}".format(name, ",".join(["(" + ",".join(x) + ")" for x in data]))) - time.sleep(0.5) - used_disks = get_used_disks_for_table(node1, name) assert set(used_disks) == {"jbod1"} @@ -713,7 +739,7 @@ def test_materialize_ttl_in_partition(started_cluster, name, engine): d1 TO DISK 'external' SETTINGS materialize_ttl_after_modify = 0 """.format(name=name)) - time.sleep(0.5) + time.sleep(3) used_disks = get_used_disks_for_table(node1, name) assert set(used_disks) == {"jbod1"} @@ -728,7 +754,7 @@ def test_materialize_ttl_in_partition(started_cluster, name, engine): MATERIALIZE TTL IN PARTITION 4 """.format(name=name)) - time.sleep(0.5) + time.sleep(3) used_disks_sets = [] for i in range(len(data)): @@ -751,6 +777,8 @@ def test_materialize_ttl_in_partition(started_cluster, name, engine): "ReplicatedMergeTree('/clickhouse/replicated_test_alter_multiple_ttls_negative', '1')", False), ]) def test_alter_multiple_ttls(started_cluster, name, engine, positive): + name = unique_table_name(name) + """Copyright 2019, Altinity LTD Licensed under the Apache License, Version 2.0 (the "License"); @@ -845,6 +873,8 @@ limitations under the License.""" "ReplicatedMergeTree('/clickhouse/concurrently_altering_ttl_replicated_mt', '1')",), ]) def test_concurrent_alter_with_ttl_move(started_cluster, name, engine): + name = unique_table_name(name) + try: node1.query(""" CREATE TABLE {name} ( @@ -951,6 +981,8 @@ def test_concurrent_alter_with_ttl_move(started_cluster, name, engine): ("test_double_move_while_select_positive", 1), ]) def test_double_move_while_select(started_cluster, name, positive): + name = unique_table_name(name) + try: node1.query(""" CREATE TABLE {name} ( @@ -990,7 +1022,7 @@ def test_double_move_while_select(started_cluster, name, positive): node1.query( "INSERT INTO {name} VALUES (4, randomPrintableASCII(9*1024*1024))".format(name=name)) - time.sleep(1) + wait_parts_mover(node1, name, retry_count=40) # If SELECT locked old part on external, move shall fail. assert node1.query( @@ -1014,6 +1046,8 @@ def test_double_move_while_select(started_cluster, name, positive): "ReplicatedMergeTree('/clickhouse/replicated_test_alter_with_merge_work', '1')", 1), ]) def test_alter_with_merge_work(started_cluster, name, engine, positive): + name = unique_table_name(name) + """Copyright 2019, Altinity LTD Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the License. @@ -1103,6 +1137,8 @@ limitations under the License.""" ("replicated_mt_test_disabled_ttl_move_on_insert_work", "VOLUME", "ReplicatedMergeTree('/clickhouse/replicated_test_disabled_ttl_move_on_insert_work', '1')"), ]) def test_disabled_ttl_move_on_insert(started_cluster, name, dest_type, engine): + name = unique_table_name(name) + try: node1.query(""" CREATE TABLE {name} ( From 4cd6ee04d8dd9cfce79d27b25f56b060b96c153b Mon Sep 17 00:00:00 2001 From: Vitaliy Zakaznikov Date: Fri, 4 Dec 2020 19:11:11 -0500 Subject: [PATCH 39/50] Updating TestFlows README.md to include "How To Debug Why Test Failed" section. --- tests/testflows/README.md | 88 +++++++++++++++++++++++++++++++++++++++ 1 file changed, 88 insertions(+) diff --git a/tests/testflows/README.md b/tests/testflows/README.md index 41b8b38d142..8a8e223191d 100644 --- a/tests/testflows/README.md +++ b/tests/testflows/README.md @@ -72,6 +72,94 @@ If you want to run only a single test such as the `/clickhouse/rbac/syntax/grant For more information, please see [Filtering](https://testflows.com/handbook/#Filtering) section in the [TestFlows Handbook]. +## How To Debug Why Test Failed + +### Step 1: find which tests failed + +If [TestFlows] check does not pass you should look at the end of the `test_run.txt.out.log` to find the list +of failing tests. For example, + +```bash +clickhouse_testflows_tests_volume +Start tests +➤ Dec 02,2020 22:22:24 /clickhouse +... +Failing + +✘ [ Fail ] /clickhouse/rbac/syntax/grant privilege/grant privileges/privilege='SELECT', on=('db0.table0', 'db0.*', '*.*', 'tb0', '*'), allow_column=True, allow_introspection=False +✘ [ Fail ] /clickhouse/rbac/syntax/grant privilege/grant privileges +✘ [ Fail ] /clickhouse/rbac/syntax/grant privilege +✘ [ Fail ] /clickhouse/rbac/syntax +✘ [ Fail ] /clickhouse/rbac +✘ [ Fail ] /clickhouse +``` + +In this case the failing test is + +``` +/clickhouse/rbac/syntax/grant privilege/grant privileges/privilege='SELECT', on=('db0.table0', 'db0.*', '*.*', 'tb0', '*'), allow_column=True, allow_introspection=False +``` + +while the others + +``` +✘ [ Fail ] /clickhouse/rbac/syntax/grant privilege/grant privileges +✘ [ Fail ] /clickhouse/rbac/syntax/grant privilege +✘ [ Fail ] /clickhouse/rbac/syntax +✘ [ Fail ] /clickhouse/rbac +✘ [ Fail ] /clickhouse +``` + +failed because the first fail gets "bubble-up" the test execution tree all the way to the top level test which is the +`/clickhouse`. + +### Step 2: download `test.log` that contains all raw messages + +You need to download the `test.log` that contains all raw messages. + +### Step 3: get messages for the failing test + +Once you know the name of the failing test and you have the `test.log` that contains all the raw messages +for all the tests, you can use `tfs show test messages` command. + +> You get the `tfs` command by installing [TestFlows]. + +For example, + +```bash +cat test.log | tfs show test messages "/clickhouse/rbac/syntax/grant privilege/grant privileges/privilege='SELECT', on=\('db0.table0', 'db0.\*', '\*.\*', 'tb0', '\*'\), allow_column=True, allow_introspection=False" +``` + +> Note: that characters that are treated as special by the shell need to be escaped. In this case +> we have to escape the `*`, `(`, and the `)` characters in the test name. + +### Step 4: working with the `test.log` + +You can use the `test.log` with many of the commands provided by the +`tfs` utility. + +> See `tfs --help` for more information. + +For example, you can get a list of failing tests from the `test.log` using the +`tfs show fails` command as follows + +```bash +$ cat test.log | tfs show fails +``` + +or get the results using the `tfs show results` command as follows + +```bash +$ cat test.log | tfs show results +``` + +or you can transform the log to see only the new fails using the +`tfs transform fail --new` command as follows + +```bash +$ cat test.log | tfs transform fails --new +``` + [Python 3]: https://www.python.org/ [Ubuntu]: https://ubuntu.com/ [TestFlows]: https://testflows.com From c0d529c49d043700cac9dc610437e801fa94a088 Mon Sep 17 00:00:00 2001 From: Vitaliy Zakaznikov Date: Fri, 4 Dec 2020 19:27:41 -0500 Subject: [PATCH 40/50] Small fix to the test/testflows/README.md --- tests/testflows/README.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/testflows/README.md b/tests/testflows/README.md index 8a8e223191d..51ccfd3debd 100644 --- a/tests/testflows/README.md +++ b/tests/testflows/README.md @@ -130,7 +130,7 @@ For example, cat test.log | tfs show test messages "/clickhouse/rbac/syntax/grant privilege/grant privileges/privilege='SELECT', on=\('db0.table0', 'db0.\*', '\*.\*', 'tb0', '\*'\), allow_column=True, allow_introspection=False" ``` -> Note: that characters that are treated as special by the shell need to be escaped. In this case +> Note: that characters that are treated as special in extended regular expressions need to be escaped. In this case > we have to escape the `*`, `(`, and the `)` characters in the test name. ### Step 4: working with the `test.log` From 4920bba7de8f04a59f5d8c5a3d2f4395ed192fcf Mon Sep 17 00:00:00 2001 From: Guillaume Tassery Date: Sat, 5 Dec 2020 09:53:15 +0700 Subject: [PATCH 41/50] Override value for running scripts on docker-entrypoint-initdb.d with CLICKHOUSE_INIT_TIMEOUT --- docker/server/entrypoint.sh | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docker/server/entrypoint.sh b/docker/server/entrypoint.sh index ba352c2bbc2..6048fdffe38 100644 --- a/docker/server/entrypoint.sh +++ b/docker/server/entrypoint.sh @@ -94,8 +94,8 @@ if [ -n "$(ls /docker-entrypoint-initdb.d/)" ] || [ -n "$CLICKHOUSE_DB" ]; then pid="$!" # check if clickhouse is ready to accept connections - # will try to send ping clickhouse via http_port (max 12 retries, with 1 sec delay) - if ! wget --spider --quiet --prefer-family=IPv6 --tries=12 --waitretry=1 --retry-connrefused "http://localhost:$HTTP_PORT/ping" ; then + # will try to send ping clickhouse via http_port (max 12 retries by default, with 1 sec delay) + if ! wget --spider --quiet --prefer-family=IPv6 --tries="${CLICKHOUSE_INIT_TIMEOUT:-12}" --waitretry=1 --retry-connrefused "http://localhost:$HTTP_PORT/ping" ; then echo >&2 'ClickHouse init process failed.' exit 1 fi From 64549702a2c693356800571da8ceb8ec5bc6f27b Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Sun, 6 Dec 2020 00:32:03 +0300 Subject: [PATCH 42/50] DragonBox integration --- .gitmodules | 3 +++ contrib/CMakeLists.txt | 2 ++ contrib/dragonbox | 1 + docker/test/fasttest/run.sh | 1 + src/CMakeLists.txt | 1 + src/IO/WriteHelpers.h | 6 +++--- src/IO/tests/CMakeLists.txt | 1 + src/IO/tests/ryu_test.cpp | 7 ++++++- 8 files changed, 18 insertions(+), 4 deletions(-) create mode 160000 contrib/dragonbox diff --git a/.gitmodules b/.gitmodules index b9a22d13c79..f9c8c191b00 100644 --- a/.gitmodules +++ b/.gitmodules @@ -209,3 +209,6 @@ path = contrib/abseil-cpp url = https://github.com/ClickHouse-Extras/abseil-cpp.git branch = lts_2020_02_25 +[submodule "contrib/dragonbox"] + path = contrib/dragonbox + url = https://github.com/kitaisreal/dragonbox.git diff --git a/contrib/CMakeLists.txt b/contrib/CMakeLists.txt index 57f08cc399c..2b799974b73 100644 --- a/contrib/CMakeLists.txt +++ b/contrib/CMakeLists.txt @@ -322,3 +322,5 @@ endif() if (USE_INTERNAL_ROCKSDB_LIBRARY) add_subdirectory(rocksdb-cmake) endif() + +add_subdirectory(dragonbox) \ No newline at end of file diff --git a/contrib/dragonbox b/contrib/dragonbox new file mode 160000 index 00000000000..17d07b00f9c --- /dev/null +++ b/contrib/dragonbox @@ -0,0 +1 @@ +Subproject commit 17d07b00f9c70d61c60c3bd1046c2ef6814d14c6 diff --git a/docker/test/fasttest/run.sh b/docker/test/fasttest/run.sh index 63c3c679668..7cef484e9f2 100755 --- a/docker/test/fasttest/run.sh +++ b/docker/test/fasttest/run.sh @@ -155,6 +155,7 @@ function clone_submodules contrib/croaring contrib/miniselect contrib/xz + contrib/dragonbox ) git submodule sync diff --git a/src/CMakeLists.txt b/src/CMakeLists.txt index e1ca1729081..bf693a02a3a 100644 --- a/src/CMakeLists.txt +++ b/src/CMakeLists.txt @@ -243,6 +243,7 @@ target_link_libraries (clickhouse_common_io common ${DOUBLE_CONVERSION_LIBRARIES} ryu + dragonbox_to_chars ) if(RE2_LIBRARY) diff --git a/src/IO/WriteHelpers.h b/src/IO/WriteHelpers.h index 201dbb30a86..1997aede564 100644 --- a/src/IO/WriteHelpers.h +++ b/src/IO/WriteHelpers.h @@ -29,7 +29,7 @@ #include #include -#include +#include #include @@ -228,14 +228,14 @@ inline size_t writeFloatTextFastPath(T x, char * buffer) if (DecomposedFloat64(x).is_inside_int64()) result = itoa(Int64(x), buffer) - buffer; else - result = d2s_buffered_n(x, buffer); + result = jkj::dragonbox::to_chars_n(x, buffer) - buffer; } else { if (DecomposedFloat32(x).is_inside_int32()) result = itoa(Int32(x), buffer) - buffer; else - result = f2s_buffered_n(x, buffer); + result = jkj::dragonbox::to_chars_n(x, buffer) - buffer; } if (result <= 0) diff --git a/src/IO/tests/CMakeLists.txt b/src/IO/tests/CMakeLists.txt index 77dfd5404c7..994eed0f737 100644 --- a/src/IO/tests/CMakeLists.txt +++ b/src/IO/tests/CMakeLists.txt @@ -82,6 +82,7 @@ target_link_libraries (zlib_ng_bug PRIVATE ${ZLIB_LIBRARIES}) add_executable (ryu_test ryu_test.cpp) target_link_libraries (ryu_test PRIVATE ryu) +target_link_libraries (ryu_test PRIVATE dragonbox_to_chars) add_executable (zstd_buffers zstd_buffers.cpp) target_link_libraries (zstd_buffers PRIVATE clickhouse_common_io) diff --git a/src/IO/tests/ryu_test.cpp b/src/IO/tests/ryu_test.cpp index 503eec82fd6..e99be60e7b5 100644 --- a/src/IO/tests/ryu_test.cpp +++ b/src/IO/tests/ryu_test.cpp @@ -2,7 +2,7 @@ #include #include #include - +#include struct DecomposedFloat64 { @@ -84,9 +84,14 @@ int main(int argc, char ** argv) double x = argc > 1 ? std::stod(argv[1]) : 0; char buf[32]; + std::cout << "ryu output" << std::endl; d2s_buffered(x, buf); std::cout << buf << "\n"; + std::cout << "dragonbox output" << std::endl; + jkj::dragonbox::to_chars(x, buf); + std::cout << buf << "\n"; + std::cout << DecomposedFloat64(x).isInsideInt64() << "\n"; return 0; From 074d0ee6aeaa5c272d83613d9f526af4e8674166 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Sun, 6 Dec 2020 00:49:32 +0300 Subject: [PATCH 43/50] Updated dragonbox commit --- contrib/dragonbox | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/dragonbox b/contrib/dragonbox index 17d07b00f9c..e8e3cb71425 160000 --- a/contrib/dragonbox +++ b/contrib/dragonbox @@ -1 +1 @@ -Subproject commit 17d07b00f9c70d61c60c3bd1046c2ef6814d14c6 +Subproject commit e8e3cb714256dd99600021005c8ff2b78aadfa90 From a634e4ed800b7a997cf38e4f28ea700295be5f09 Mon Sep 17 00:00:00 2001 From: feng lv Date: Sun, 6 Dec 2020 08:57:02 +0000 Subject: [PATCH 44/50] fix incorrect initialize MergeTreeWriterSettings --- src/Storages/MergeTree/MergeTreeIOSettings.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/MergeTree/MergeTreeIOSettings.h b/src/Storages/MergeTree/MergeTreeIOSettings.h index 42554e51d83..db01f1ae9b7 100644 --- a/src/Storages/MergeTree/MergeTreeIOSettings.h +++ b/src/Storages/MergeTree/MergeTreeIOSettings.h @@ -22,7 +22,7 @@ struct MergeTreeWriterSettings MergeTreeWriterSettings(const Settings & global_settings, bool can_use_adaptive_granularity_, size_t aio_threshold_, bool blocks_are_granules_size_ = false) : min_compress_block_size(global_settings.min_compress_block_size) - , max_compress_block_size(global_settings.min_compress_block_size) + , max_compress_block_size(global_settings.max_compress_block_size) , aio_threshold(aio_threshold_) , can_use_adaptive_granularity(can_use_adaptive_granularity_) , blocks_are_granules_size(blocks_are_granules_size_) {} From 977da3d0529bdd515f823e786abf359be2953d31 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Sun, 6 Dec 2020 15:39:39 +0300 Subject: [PATCH 45/50] Removed ryu --- .gitmodules | 5 +---- contrib/CMakeLists.txt | 1 - contrib/dragonbox | 2 +- contrib/ryu | 1 - contrib/ryu-cmake/CMakeLists.txt | 10 ---------- docker/test/fasttest/run.sh | 1 - src/CMakeLists.txt | 1 - src/IO/tests/CMakeLists.txt | 5 ++--- src/IO/tests/{ryu_test.cpp => dragonbox_test.cpp} | 5 ----- 9 files changed, 4 insertions(+), 27 deletions(-) delete mode 160000 contrib/ryu delete mode 100644 contrib/ryu-cmake/CMakeLists.txt rename src/IO/tests/{ryu_test.cpp => dragonbox_test.cpp} (93%) diff --git a/.gitmodules b/.gitmodules index f9c8c191b00..7bb4764193d 100644 --- a/.gitmodules +++ b/.gitmodules @@ -142,9 +142,6 @@ [submodule "contrib/replxx"] path = contrib/replxx url = https://github.com/ClickHouse-Extras/replxx.git -[submodule "contrib/ryu"] - path = contrib/ryu - url = https://github.com/ClickHouse-Extras/ryu.git [submodule "contrib/avro"] path = contrib/avro url = https://github.com/ClickHouse-Extras/avro.git @@ -211,4 +208,4 @@ branch = lts_2020_02_25 [submodule "contrib/dragonbox"] path = contrib/dragonbox - url = https://github.com/kitaisreal/dragonbox.git + url = https://github.com/ClickHouse-Extras/dragonbox.git diff --git a/contrib/CMakeLists.txt b/contrib/CMakeLists.txt index 2b799974b73..df05046d596 100644 --- a/contrib/CMakeLists.txt +++ b/contrib/CMakeLists.txt @@ -35,7 +35,6 @@ add_subdirectory (libmetrohash) add_subdirectory (lz4-cmake) add_subdirectory (murmurhash) add_subdirectory (replxx-cmake) -add_subdirectory (ryu-cmake) add_subdirectory (unixodbc-cmake) add_subdirectory (xz) diff --git a/contrib/dragonbox b/contrib/dragonbox index e8e3cb71425..b2751c65c05 160000 --- a/contrib/dragonbox +++ b/contrib/dragonbox @@ -1 +1 @@ -Subproject commit e8e3cb714256dd99600021005c8ff2b78aadfa90 +Subproject commit b2751c65c0592c0239aec3becd53d0ea2fde9329 diff --git a/contrib/ryu b/contrib/ryu deleted file mode 160000 index 5b4a853534b..00000000000 --- a/contrib/ryu +++ /dev/null @@ -1 +0,0 @@ -Subproject commit 5b4a853534b47438b4d97935370f6b2397137c2b diff --git a/contrib/ryu-cmake/CMakeLists.txt b/contrib/ryu-cmake/CMakeLists.txt deleted file mode 100644 index bf46fdc61a7..00000000000 --- a/contrib/ryu-cmake/CMakeLists.txt +++ /dev/null @@ -1,10 +0,0 @@ -SET(LIBRARY_DIR ${ClickHouse_SOURCE_DIR}/contrib/ryu) - -add_library(ryu -${LIBRARY_DIR}/ryu/d2fixed.c -${LIBRARY_DIR}/ryu/d2s.c -${LIBRARY_DIR}/ryu/f2s.c -${LIBRARY_DIR}/ryu/generic_128.c -) - -target_include_directories(ryu SYSTEM BEFORE PUBLIC "${LIBRARY_DIR}") diff --git a/docker/test/fasttest/run.sh b/docker/test/fasttest/run.sh index 7cef484e9f2..fb905bdaa5d 100755 --- a/docker/test/fasttest/run.sh +++ b/docker/test/fasttest/run.sh @@ -137,7 +137,6 @@ function clone_submodules contrib/libxml2 contrib/poco contrib/libunwind - contrib/ryu contrib/fmtlib contrib/base64 contrib/cctz diff --git a/src/CMakeLists.txt b/src/CMakeLists.txt index bf693a02a3a..3f5801e310a 100644 --- a/src/CMakeLists.txt +++ b/src/CMakeLists.txt @@ -242,7 +242,6 @@ target_link_libraries (clickhouse_common_io PUBLIC common ${DOUBLE_CONVERSION_LIBRARIES} - ryu dragonbox_to_chars ) diff --git a/src/IO/tests/CMakeLists.txt b/src/IO/tests/CMakeLists.txt index 994eed0f737..da4d330f0a9 100644 --- a/src/IO/tests/CMakeLists.txt +++ b/src/IO/tests/CMakeLists.txt @@ -80,9 +80,8 @@ target_link_libraries (parse_date_time_best_effort PRIVATE clickhouse_common_io) add_executable (zlib_ng_bug zlib_ng_bug.cpp) target_link_libraries (zlib_ng_bug PRIVATE ${ZLIB_LIBRARIES}) -add_executable (ryu_test ryu_test.cpp) -target_link_libraries (ryu_test PRIVATE ryu) -target_link_libraries (ryu_test PRIVATE dragonbox_to_chars) +add_executable (dragonbox_test dragonbox_test.cpp) +target_link_libraries (dragonbox_test PRIVATE dragonbox_to_chars) add_executable (zstd_buffers zstd_buffers.cpp) target_link_libraries (zstd_buffers PRIVATE clickhouse_common_io) diff --git a/src/IO/tests/ryu_test.cpp b/src/IO/tests/dragonbox_test.cpp similarity index 93% rename from src/IO/tests/ryu_test.cpp rename to src/IO/tests/dragonbox_test.cpp index e99be60e7b5..98d41b04d6c 100644 --- a/src/IO/tests/ryu_test.cpp +++ b/src/IO/tests/dragonbox_test.cpp @@ -1,7 +1,6 @@ #include #include #include -#include #include struct DecomposedFloat64 @@ -84,10 +83,6 @@ int main(int argc, char ** argv) double x = argc > 1 ? std::stod(argv[1]) : 0; char buf[32]; - std::cout << "ryu output" << std::endl; - d2s_buffered(x, buf); - std::cout << buf << "\n"; - std::cout << "dragonbox output" << std::endl; jkj::dragonbox::to_chars(x, buf); std::cout << buf << "\n"; From 6a7ffbec1dc3d0aa31c64fc01fb21069f81b66e1 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Sun, 6 Dec 2020 15:41:18 +0300 Subject: [PATCH 46/50] Fixed contrib CMakeLists.txt --- contrib/CMakeLists.txt | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/CMakeLists.txt b/contrib/CMakeLists.txt index df05046d596..a674d542c78 100644 --- a/contrib/CMakeLists.txt +++ b/contrib/CMakeLists.txt @@ -322,4 +322,4 @@ if (USE_INTERNAL_ROCKSDB_LIBRARY) add_subdirectory(rocksdb-cmake) endif() -add_subdirectory(dragonbox) \ No newline at end of file +add_subdirectory(dragonbox) From 459570470e38067049ff03eb635a6aaa8317a6a7 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Sun, 6 Dec 2020 23:31:17 +0300 Subject: [PATCH 47/50] Update version_date.tsv after release 20.11.5.18 --- utils/list-versions/version_date.tsv | 1 + 1 file changed, 1 insertion(+) diff --git a/utils/list-versions/version_date.tsv b/utils/list-versions/version_date.tsv index 95ed2565a61..b1263806a84 100644 --- a/utils/list-versions/version_date.tsv +++ b/utils/list-versions/version_date.tsv @@ -1,3 +1,4 @@ +v20.11.5.18-stable 2020-12-06 v20.11.4.13-stable 2020-11-20 v20.11.3.3-stable 2020-11-13 v20.11.2.1-stable 2020-11-11 From dc5ca434efffdbb9631ddffc78b3fe9b55eb650e Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Sun, 6 Dec 2020 23:46:51 +0300 Subject: [PATCH 48/50] Update version_date.tsv after release 20.10.6.27 --- utils/list-versions/version_date.tsv | 1 + 1 file changed, 1 insertion(+) diff --git a/utils/list-versions/version_date.tsv b/utils/list-versions/version_date.tsv index b1263806a84..78b0739961e 100644 --- a/utils/list-versions/version_date.tsv +++ b/utils/list-versions/version_date.tsv @@ -2,6 +2,7 @@ v20.11.5.18-stable 2020-12-06 v20.11.4.13-stable 2020-11-20 v20.11.3.3-stable 2020-11-13 v20.11.2.1-stable 2020-11-11 +v20.10.6.27-stable 2020-12-06 v20.10.5.10-stable 2020-11-20 v20.10.4.1-stable 2020-11-13 v20.10.3.30-stable 2020-10-29 From f2b63f0702690834151c1cb740fe04fdd39cf4c2 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Mon, 7 Dec 2020 00:11:19 +0300 Subject: [PATCH 49/50] Update version_date.tsv after release 20.9.7.11 --- utils/list-versions/version_date.tsv | 1 + 1 file changed, 1 insertion(+) diff --git a/utils/list-versions/version_date.tsv b/utils/list-versions/version_date.tsv index 78b0739961e..8dce346b561 100644 --- a/utils/list-versions/version_date.tsv +++ b/utils/list-versions/version_date.tsv @@ -7,6 +7,7 @@ v20.10.5.10-stable 2020-11-20 v20.10.4.1-stable 2020-11-13 v20.10.3.30-stable 2020-10-29 v20.10.2.20-stable 2020-10-23 +v20.9.7.11-stable 2020-12-07 v20.9.6.14-stable 2020-11-20 v20.9.5.5-stable 2020-11-13 v20.9.4.76-stable 2020-10-29 From e62ef0ab56ae66c1ebc03ba2db7fa1b287a260d0 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Mon, 7 Dec 2020 00:41:25 +0300 Subject: [PATCH 50/50] Update version_date.tsv after release 20.8.8.2 --- utils/list-versions/version_date.tsv | 1 + 1 file changed, 1 insertion(+) diff --git a/utils/list-versions/version_date.tsv b/utils/list-versions/version_date.tsv index 8dce346b561..eb28f4b5369 100644 --- a/utils/list-versions/version_date.tsv +++ b/utils/list-versions/version_date.tsv @@ -13,6 +13,7 @@ v20.9.5.5-stable 2020-11-13 v20.9.4.76-stable 2020-10-29 v20.9.3.45-stable 2020-10-09 v20.9.2.20-stable 2020-09-22 +v20.8.8.2-lts 2020-12-07 v20.8.7.15-lts 2020-11-20 v20.8.6.6-lts 2020-11-13 v20.8.5.45-lts 2020-10-29