From 3ce4844d4b30f63ed5e7e7c816b1b5fd69f78145 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sat, 2 Apr 2022 18:12:51 +0300 Subject: [PATCH 01/11] Properly cancel the query after client format error In case of format error (i.e. select 2 format Template settings format_template_row='/dev/null') the client will reset the connection, since it will not expect Data blocks. To fix this, catch this client error and properly cancel the query. This should also fix query hang checks (the one that executed after each test). v2: use getCurrentExceptionMessage()/getCurrentExceptionCode() Signed-off-by: Azat Khuzhin --- src/Client/ClientBase.cpp | 31 +++++++++++++++++++++++++++++-- 1 file changed, 29 insertions(+), 2 deletions(-) diff --git a/src/Client/ClientBase.cpp b/src/Client/ClientBase.cpp index 9e61356566c..4faf3a8028e 100644 --- a/src/Client/ClientBase.cpp +++ b/src/Client/ClientBase.cpp @@ -240,6 +240,14 @@ void interruptSignalHandler(int signum) } +/// To cancel the query on local format error. +class LocalFormatError : public DB::Exception +{ +public: + using Exception::Exception; +}; + + ClientBase::~ClientBase() = default; ClientBase::ClientBase() = default; @@ -442,6 +450,7 @@ void ClientBase::onProfileInfo(const ProfileInfo & profile_info) void ClientBase::initBlockOutputStream(const Block & block, ASTPtr parsed_query) +try { if (!output_format) { @@ -530,6 +539,10 @@ void ClientBase::initBlockOutputStream(const Block & block, ASTPtr parsed_query) output_format->setAutoFlush(); } } +catch (...) +{ + throw LocalFormatError(getCurrentExceptionMessage(print_stack_trace), getCurrentExceptionCode()); +} void ClientBase::initLogsOutputStream() @@ -721,6 +734,9 @@ void ClientBase::receiveResult(ASTPtr parsed_query) = std::max(min_poll_interval, std::min(receive_timeout.totalMicroseconds(), default_poll_interval)); bool break_on_timeout = connection->getConnectionType() != IServerConnection::Type::LOCAL; + + std::exception_ptr local_format_error; + while (true) { Stopwatch receive_watch(CLOCK_MONOTONIC_COARSE); @@ -769,10 +785,21 @@ void ClientBase::receiveResult(ASTPtr parsed_query) break; } - if (!receiveAndProcessPacket(parsed_query, cancelled)) - break; + try + { + if (!receiveAndProcessPacket(parsed_query, cancelled)) + break; + } + catch (const LocalFormatError &) + { + local_format_error = std::current_exception(); + connection->sendCancel(); + } } + if (local_format_error) + std::rethrow_exception(local_format_error); + if (cancelled && is_interactive) std::cout << "Query was cancelled." << std::endl; } From df06f9f974102cc6c5b45e622fc3a5a5f4fa1ec5 Mon Sep 17 00:00:00 2001 From: Amos Bird Date: Wed, 6 Apr 2022 14:53:10 +0800 Subject: [PATCH 02/11] Fix performance regression of scalar query --- src/Functions/getScalar.cpp | 2 +- src/Interpreters/Context.cpp | 10 +++++----- src/Interpreters/Context.h | 7 ++++--- .../ExecuteScalarSubqueriesVisitor.cpp | 8 ++++++-- .../IInterpreterUnionOrSelectQuery.h | 4 ++-- tests/performance/scalar2.xml | 17 +++++++++++++++++ 6 files changed, 35 insertions(+), 13 deletions(-) create mode 100644 tests/performance/scalar2.xml diff --git a/src/Functions/getScalar.cpp b/src/Functions/getScalar.cpp index b06fb360366..c165ef26ffa 100644 --- a/src/Functions/getScalar.cpp +++ b/src/Functions/getScalar.cpp @@ -78,7 +78,7 @@ public: static ColumnWithTypeAndName createScalar(ContextPtr context_) { - if (const auto * block = context_->tryGetLocalScalar(Scalar::scalar_name)) + if (const auto * block = context_->tryGetSpecialScalar(Scalar::scalar_name)) return block->getByPosition(0); else if (context_->hasQueryContext()) { diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index e837ce5dae1..493e250ea85 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -930,10 +930,10 @@ const Block & Context::getScalar(const String & name) const return it->second; } -const Block * Context::tryGetLocalScalar(const String & name) const +const Block * Context::tryGetSpecialScalar(const String & name) const { - auto it = local_scalars.find(name); - if (local_scalars.end() == it) + auto it = special_scalars.find(name); + if (special_scalars.end() == it) return nullptr; return &it->second; } @@ -1004,12 +1004,12 @@ void Context::addScalar(const String & name, const Block & block) } -void Context::addLocalScalar(const String & name, const Block & block) +void Context::addSpecialScalar(const String & name, const Block & block) { if (isGlobalContext()) throw Exception(ErrorCodes::LOGICAL_ERROR, "Global context cannot have local scalars"); - local_scalars[name] = block; + special_scalars[name] = block; } diff --git a/src/Interpreters/Context.h b/src/Interpreters/Context.h index ad68f2a2245..31d853318d4 100644 --- a/src/Interpreters/Context.h +++ b/src/Interpreters/Context.h @@ -227,7 +227,8 @@ private: /// Thus, used in HTTP interface. If not specified - then some globally default format is used. TemporaryTablesMapping external_tables_mapping; Scalars scalars; - Scalars local_scalars; + /// Used to store constant values which are different on each instance during distributed plan, such as _shard_num. + Scalars special_scalars; /// Used in s3Cluster table function. With this callback, a worker node could ask an initiator /// about next file to read from s3. @@ -487,8 +488,8 @@ public: void addScalar(const String & name, const Block & block); bool hasScalar(const String & name) const; - const Block * tryGetLocalScalar(const String & name) const; - void addLocalScalar(const String & name, const Block & block); + const Block * tryGetSpecialScalar(const String & name) const; + void addSpecialScalar(const String & name, const Block & block); const QueryAccessInfo & getQueryAccessInfo() const { return query_access_info; } void addQueryAccessInfo( diff --git a/src/Interpreters/ExecuteScalarSubqueriesVisitor.cpp b/src/Interpreters/ExecuteScalarSubqueriesVisitor.cpp index ac8a27484d9..55fdaffdfe9 100644 --- a/src/Interpreters/ExecuteScalarSubqueriesVisitor.cpp +++ b/src/Interpreters/ExecuteScalarSubqueriesVisitor.cpp @@ -113,14 +113,18 @@ void ExecuteScalarSubqueriesMatcher::visit(const ASTSubquery & subquery, ASTPtr bool is_local = false; Block scalar; - if (data.local_scalars.count(scalar_query_hash_str)) + if (data.only_analyze) + { + /// Don't use scalar cache during query analysis + } + else if (data.local_scalars.contains(scalar_query_hash_str)) { hit = true; scalar = data.local_scalars[scalar_query_hash_str]; is_local = true; ProfileEvents::increment(ProfileEvents::ScalarSubqueriesLocalCacheHit); } - else if (data.scalars.count(scalar_query_hash_str)) + else if (data.scalars.contains(scalar_query_hash_str)) { hit = true; scalar = data.scalars[scalar_query_hash_str]; diff --git a/src/Interpreters/IInterpreterUnionOrSelectQuery.h b/src/Interpreters/IInterpreterUnionOrSelectQuery.h index 7906ab189fc..a9262fbfa1e 100644 --- a/src/Interpreters/IInterpreterUnionOrSelectQuery.h +++ b/src/Interpreters/IInterpreterUnionOrSelectQuery.h @@ -18,11 +18,11 @@ public: , max_streams(context->getSettingsRef().max_threads) { if (options.shard_num) - context->addLocalScalar( + context->addSpecialScalar( "_shard_num", Block{{DataTypeUInt32().createColumnConst(1, *options.shard_num), std::make_shared(), "_shard_num"}}); if (options.shard_count) - context->addLocalScalar( + context->addSpecialScalar( "_shard_count", Block{{DataTypeUInt32().createColumnConst(1, *options.shard_count), std::make_shared(), "_shard_count"}}); } diff --git a/tests/performance/scalar2.xml b/tests/performance/scalar2.xml new file mode 100644 index 00000000000..eb427536646 --- /dev/null +++ b/tests/performance/scalar2.xml @@ -0,0 +1,17 @@ + + CREATE TABLE tbl0 (`ds` Date, `x1` String, `x2` UInt32, `x3` UInt32, `x4` UInt32, `bm` AggregateFunction(groupBitmap, UInt32)) ENGINE = MergeTree PARTITION BY (ds, x1) ORDER BY (x2, x3, x4) SETTINGS index_granularity = 1 + + CREATE TABLE tbl (`ds` Date, `y1` UInt32, `x4` UInt32, `y2` UInt32, `y3` UInt32, `bm` AggregateFunction(groupBitmap, UInt32), `y4` UInt32 DEFAULT 0) ENGINE = MergeTree PARTITION BY (ds) ORDER BY (x4, y2, y3) SETTINGS index_granularity = 8192, max_parts_in_total = 10000000 + + insert into tbl0 with murmurHash3_32(toUInt32(rand())) as uid select toDate('2022-03-01')+rand()%7 as ds, concat('xx',toString(rand()%10+1)) as x1, 1 as x2, 2 as x3, bitShiftRight(uid, 22) as x4, groupBitmapState(uid) as bm from numbers(100000000) where x4%40=0 group by ds, x1, x2, x3, x4 + + insert into tbl with murmurHash3_32(toUInt32(rand())) as uid select toDate('2022-03-01')+rand()%7 as ds, rand()%1000+5000 as y1, bitShiftRight(uid, 22) as x4, rand()%100 as y2, rand()%2000 as y3, groupBitmapState(uid) as bm, rand()%1 as y4 from numbers(100000000) where x4%40=0 group by ds, y1, x4, y2, y3, y4 + + CREATE TABLE tmp_acc_hit engine Memory AS SELECT x1, x2, x3, arrayReduceInRanges('groupBitmapMergeState', [(1, 1), (1, 2), (1, 3), (1, 4), (1, 5), (1, 6), (1, 7)], bs) AS bs FROM (SELECT x1, x2, x3, groupArrayInsertAt(b, multiIf(ds = '2022-03-01', 0, ds = '2022-03-02', 1, ds = '2022-03-03', 2, ds = '2022-03-04', 3, ds = '2022-03-05', 4, ds = '2022-03-06', 5, ds = '2022-03-07', 6, 7)) AS bs FROM (SELECT x1, x2, x3, ds, groupBitmapOrState(bm) AS b FROM tbl0 WHERE ((ds >= '2022-03-01') AND (ds <= '2022-03-07')) AND (((x1 = 'xx1') AND (x2 = 1) AND (x3 = 2)) OR ((x1 = 'xx2') AND (x2 = 1) AND (x3 = 2)) OR ((x1 = 'xx3') AND (x2 = 1) AND (x3 = 2)) OR ((x1 = 'xx4') AND (x2 = 1) AND (x3 = 2)) OR ((x1 = 'xx5') AND (x2 = 1) AND (x3 = 2)) OR ((x1 = 'xx6') AND (x2 = 1) AND (x3 = 2)) OR ((x1 = 'xx7') AND (x2 = 1) AND (x3 = 2)) OR ((x1 = 'xx8') AND (x2 = 1) AND (x3 = 2)) OR ((x1 = 'xx9') AND (x2 = 1) AND (x3 = 2)) OR ((x1 = 'xx10') AND (x2 = 1) AND (x3 = 2))) AND (x4 IN (0, 40, 80, 120, 160, 200, 240, 280, 320, 360, 400, 440, 480, 520, 560, 600, 640, 680, 720, 760, 800, 840, 880, 920, 960, 1000)) GROUP BY x1, x2, x3, ds) AS t_hit GROUP BY x1, x2, x3) + + WITH (SELECT groupArrayInsertAt(b, multiIf((x1 = 'xx1') AND (x2 = 1) AND (x3 = 2), 0, (x1 = 'xx2') AND (x2 = 1) AND (x3 = 2), 1, (x1 = 'xx3') AND (x2 = 1) AND (x3 = 2), 2, (x1 = 'xx4') AND (x2 = 1) AND (x3 = 2), 3, (x1 = 'xx5') AND (x2 = 1) AND (x3 = 2), 4, (x1 = 'xx6') AND (x2 = 1) AND (x3 = 2), 5, (x1 = 'xx7') AND (x2 = 1) AND (x3 = 2), 6, (x1 = 'xx8') AND (x2 = 1) AND (x3 = 2), 7, (x1 = 'xx9') AND (x2 = 1) AND (x3 = 2), 8, (x1 = 'xx10') AND (x2 = 1) AND (x3 = 2), 9, 10)) FROM (SELECT x1, x2, x3, bs AS b FROM tmp_acc_hit)) AS bs SELECT y1, x4, toString(flat_arr) AS flat_arr, toString([bitmapAndCardinality(bmor1, (bs[1])[1]), bitmapAndCardinality(bmor2, (bs[1])[1]), bitmapAndCardinality(bmor3, (bs[1])[1]), bitmapAndCardinality(bmor1, (bs[2])[1]), bitmapAndCardinality(bmor2, (bs[2])[1]), bitmapAndCardinality(bmor3, (bs[2])[1]), bitmapAndCardinality(bmor1, (bs[3])[1]), bitmapAndCardinality(bmor2, (bs[3])[1]), bitmapAndCardinality(bmor3, (bs[3])[1]), bitmapAndCardinality(bmor1, (bs[4])[1]), bitmapAndCardinality(bmor2, (bs[4])[1]), bitmapAndCardinality(bmor3, (bs[4])[1]), bitmapAndCardinality(bmor1, (bs[5])[1]), bitmapAndCardinality(bmor2, (bs[5])[1]), bitmapAndCardinality(bmor3, (bs[5])[1]), bitmapAndCardinality(bmor1, (bs[6])[1]), bitmapAndCardinality(bmor2, (bs[6])[1]), bitmapAndCardinality(bmor3, (bs[6])[1]), bitmapAndCardinality(bmor1, (bs[7])[1]), bitmapAndCardinality(bmor2, (bs[7])[1]), bitmapAndCardinality(bmor3, (bs[7])[1]), bitmapAndCardinality(bmor1, (bs[8])[1]), bitmapAndCardinality(bmor2, (bs[8])[1]), bitmapAndCardinality(bmor3, (bs[8])[1]), bitmapAndCardinality(bmor1, (bs[9])[1]), bitmapAndCardinality(bmor2, (bs[9])[1]), bitmapAndCardinality(bmor3, (bs[9])[1]), bitmapAndCardinality(bmor1, (bs[10])[1]), bitmapAndCardinality(bmor2, (bs[10])[1]), bitmapAndCardinality(bmor3, (bs[10])[1])]) AS flat_arr_2 from (SELECT toString(y1) AS y1, toString(x4) AS x4, arrayFlatten(groupArrayInsertAt(flat_arr, multiIf(date_ = '2022-03-01', 0, 1))) AS flat_arr, groupBitmapOrState(bmor1) AS bmor1, groupBitmapOrState(bmor2) AS bmor2, groupBitmapOrState(bmor3) AS bmor3 FROM (WITH '2022-03-01' AS start_ds SELECT y1, x4, groupBitmapOrState(bm) AS bmor1, groupBitmapOrStateIf(bm, y2 > 0) AS bmor2, groupBitmapOrStateIf(bm, y4 = 1) AS bmor3, [sum(y2 * bitmapAndCardinality(bm, (bs[1])[1])), sum(y2 * bitmapAndCardinality(bm, (bs[2])[1])), sum(y2 * bitmapAndCardinality(bm, (bs[3])[1])), sum(y2 * bitmapAndCardinality(bm, (bs[4])[1])), sum(y2 * bitmapAndCardinality(bm, (bs[5])[1])), sum(y2 * bitmapAndCardinality(bm, (bs[6])[1])), sum(y2 * bitmapAndCardinality(bm, (bs[7])[1])), sum(y2 * bitmapAndCardinality(bm, (bs[8])[1])), sum(y2 * bitmapAndCardinality(bm, (bs[9])[1])), sum(y2 * bitmapAndCardinality(bm, (bs[10])[1]))] AS flat_arr, start_ds AS date_ FROM tbl WHERE (ds = start_ds) AND (y1 IN (7063, 5010, 5006, 6788, 6176, 6203, 6769, 6555, 7062, 5119, 5007, 5212, 6814, 6177, 6789, 5095, 4942, 6243, 7061, 6744, 6201, 7196, 6181, 7195, 6178, 5004, 6790, 5008, 6877, 7281, 6791, 6179, 5214, 5005, 7146, 6980, 6322, 5222, 5217, 5137, 6561, 5133, 6937, 5142, 5130, 6885, 7250, 5103, 6867, 7066, 5096, 6868, 6199, 7269, 5131, 6414, 6884, 6560, 5136, 6883, 5158, 6869, 5097, 5132, 5102, 7251, 5219, 4695, 5220, 5202, 4203, 4204, 5098, 6870, 7064, 5101, 5105, 5140, 5135, 5139, 6880, 6194, 5218, 4202, 6655, 5104, 5183, 7245, 5100, 7065, 5099, 6938, 5138, 6881, 5134, 6886, 5141, 5129)) AND (x4 IN (0, 40, 80, 120, 160, 200, 240, 280, 320, 360, 400, 440, 480, 520, 560, 600, 640, 680, 720, 760, 800, 840, 880, 920, 960, 1000)) AND (y4 IN (0, 1)) GROUP BY y1, x4) GROUP BY y1, x4) LIMIT 1 + + DROP TABLE IF EXISTS tbl + DROP TABLE IF EXISTS tbl0 + DROP TABLE IF EXISTS tmp_acc_hit + From 53628092e2ae20d6f4054344438cda041aa1dfde Mon Sep 17 00:00:00 2001 From: Amos Bird Date: Wed, 6 Apr 2022 20:04:13 +0800 Subject: [PATCH 03/11] Fix test --- .../queries/0_stateless/02174_cte_scalar_cache_mv.reference | 2 +- tests/queries/0_stateless/02174_cte_scalar_cache_mv.sql | 6 +++--- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/tests/queries/0_stateless/02174_cte_scalar_cache_mv.reference b/tests/queries/0_stateless/02174_cte_scalar_cache_mv.reference index 246706164df..055c88160ad 100644 --- a/tests/queries/0_stateless/02174_cte_scalar_cache_mv.reference +++ b/tests/queries/0_stateless/02174_cte_scalar_cache_mv.reference @@ -18,7 +18,7 @@ 89 89 89 89 5 94 94 94 94 5 99 99 99 99 5 -02177_MV 7 80 22 +02177_MV 3 80 26 10 40 70 diff --git a/tests/queries/0_stateless/02174_cte_scalar_cache_mv.sql b/tests/queries/0_stateless/02174_cte_scalar_cache_mv.sql index 4d4447c7f31..742d72fe2b2 100644 --- a/tests/queries/0_stateless/02174_cte_scalar_cache_mv.sql +++ b/tests/queries/0_stateless/02174_cte_scalar_cache_mv.sql @@ -39,13 +39,13 @@ SYSTEM FLUSH LOGS; -- The main query should have a cache miss and 3 global hits -- The MV is executed 20 times (100 / 5) and each run does 1 miss and 4 hits to the LOCAL cache -- In addition to this, to prepare the MV, there is an extra preparation to get the list of columns via --- InterpreterSelectQuery, which adds 1 miss and 4 global hits (since it uses the global cache) +-- InterpreterSelectQuery, which adds 5 miss (since we don't use cache for preparation) -- So in total we have: -- Main query: 1 miss, 3 global --- Preparation: 1 miss, 4 global +-- Preparation: 5 miss -- Blocks (20): 20 miss, 0 global, 80 local hits --- TOTAL: 22 miss, 7 global, 80 local +-- TOTAL: 26 miss, 3 global, 80 local SELECT '02177_MV', ProfileEvents['ScalarSubqueriesGlobalCacheHit'] as scalar_cache_global_hit, From 1cbacbc3e61872fce5c14a64f6b80c3404bda2d6 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Wed, 6 Apr 2022 14:28:30 +0000 Subject: [PATCH 04/11] Update version_date.tsv after v22.3.3.44-lts --- 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 e87c4ea2b46..6366aef19ce 100644 --- a/utils/list-versions/version_date.tsv +++ b/utils/list-versions/version_date.tsv @@ -1,3 +1,4 @@ +v22.3.3.44-lts 2022-04-06 v22.3.2.2-lts 2022-03-17 v22.2.3.5-stable 2022-02-25 v22.2.2.1-stable 2022-02-17 From 14b860f9bbf9408e821a70cd37e44d295e08b3d3 Mon Sep 17 00:00:00 2001 From: LAL2211 Date: Wed, 6 Apr 2022 12:39:52 -0400 Subject: [PATCH 05/11] disallow the use of known weak implementations in requirements --- .../requirements/requirements.md | 33 ++-------- .../requirements/requirements.py | 65 +++---------------- 2 files changed, 12 insertions(+), 86 deletions(-) diff --git a/tests/testflows/aes_encryption/requirements/requirements.md b/tests/testflows/aes_encryption/requirements/requirements.md index 80cb614268c..23906f797d0 100644 --- a/tests/testflows/aes_encryption/requirements/requirements.md +++ b/tests/testflows/aes_encryption/requirements/requirements.md @@ -311,7 +311,7 @@ version: 1.0 of the `encrypt` function where the `key_length` SHALL specifies the length of the key and SHALL accept `128`, `192`, or `256` as the values and the `mode` SHALL specify the block encryption -mode and SHALL accept [ECB], [CBC], [CFB128], or [OFB] as well as +mode and SHALL accept [CBC], [CFB128], or [OFB] as well as [CTR] and [GCM] as the values. For example, `aes-256-ofb`. #### RQ.SRS008.AES.Encrypt.Function.Parameters.Mode.Value.Invalid @@ -327,9 +327,6 @@ version: 1.0 [ClickHouse] SHALL support the following [AES] block encryption modes as the value for the `mode` parameter of the `encrypt` function: -* `aes-128-ecb` that SHALL use [ECB] block mode encryption with 128 bit key -* `aes-192-ecb` that SHALL use [ECB] block mode encryption with 192 bit key -* `aes-256-ecb` that SHALL use [ECB] block mode encryption with 256 bit key * `aes-128-cbc` that SHALL use [CBC] block mode encryption with 128 bit key * `aes-192-cbc` that SHALL use [CBC] block mode encryption with 192 bit key * `aes-192-cbc` that SHALL use [CBC] block mode encryption with 256 bit key @@ -403,9 +400,6 @@ version: 1.0 [ClickHouse] SHALL return an error when the `encrypt` function is called with the following parameter values when using non-GCM modes -* `aes-128-ecb` mode and `key` is not 16 bytes or `iv` or `aad` is specified -* `aes-192-ecb` mode and `key` is not 24 bytes or `iv` or `aad` is specified -* `aes-256-ecb` mode and `key` is not 32 bytes or `iv` or `aad` is specified * `aes-128-cbc` mode and `key` is not 16 bytes or if specified `iv` is not 16 bytes or `aad` is specified * `aes-192-cbc` mode and `key` is not 24 bytes or if specified `iv` is not 16 bytes or `aad` is specified * `aes-256-cbc` mode and `key` is not 32 bytes or if specified `iv` is not 16 bytes or `aad` is specified @@ -476,7 +470,7 @@ version: 1.0 of the `decrypt` function where the `key_length` SHALL specifies the length of the key and SHALL accept `128`, `192`, or `256` as the values and the `mode` SHALL specify the block encryption -mode and SHALL accept [ECB], [CBC], [CFB128], or [OFB] as well as +mode and SHALL accept [CBC], [CFB128], or [OFB] as well as [CTR] and [GCM] as the values. For example, `aes-256-ofb`. #### RQ.SRS008.AES.Decrypt.Function.Parameters.Mode.Value.Invalid @@ -492,9 +486,6 @@ version: 1.0 [ClickHouse] SHALL support the following [AES] block encryption modes as the value for the `mode` parameter of the `decrypt` function: -* `aes-128-ecb` that SHALL use [ECB] block mode encryption with 128 bit key -* `aes-192-ecb` that SHALL use [ECB] block mode encryption with 192 bit key -* `aes-256-ecb` that SHALL use [ECB] block mode encryption with 256 bit key * `aes-128-cbc` that SHALL use [CBC] block mode encryption with 128 bit key * `aes-192-cbc` that SHALL use [CBC] block mode encryption with 192 bit key * `aes-192-cbc` that SHALL use [CBC] block mode encryption with 256 bit key @@ -570,9 +561,6 @@ version: 1.0 [ClickHouse] SHALL return an error when the `decrypt` function is called with the following parameter values when using non-GCM modes -* `aes-128-ecb` mode and `key` is not 16 bytes or `iv` or `aad` is specified -* `aes-192-ecb` mode and `key` is not 24 bytes or `iv` or `aad` is specified -* `aes-256-ecb` mode and `key` is not 32 bytes or `iv` or `aad` is specified * `aes-128-cbc` mode and `key` is not 16 bytes or if specified `iv` is not 16 bytes or `aad` is specified * `aes-192-cbc` mode and `key` is not 24 bytes or if specified `iv` is not 16 bytes or `aad` is specified * `aes-256-cbc` mode and `key` is not 32 bytes or if specified `iv` is not 16 bytes or `aad` is specified @@ -644,7 +632,7 @@ version: 1.0 of the `aes_encrypt_mysql` function where the `key_length` SHALL specifies the length of the key and SHALL accept `128`, `192`, or `256` as the values and the `mode` SHALL specify the block encryption -mode and SHALL accept [ECB], [CBC], [CFB128], or [OFB]. For example, `aes-256-ofb`. +mode and SHALL accept [CBC], [CFB128], or [OFB]. For example, `aes-256-ofb`. #### RQ.SRS008.AES.MySQL.Encrypt.Function.Parameters.Mode.Value.Invalid version: 1.0 @@ -659,9 +647,6 @@ version: 1.0 [ClickHouse] SHALL support the following [AES] block encryption modes as the value for the `mode` parameter of the `aes_encrypt_mysql` function: -* `aes-128-ecb` that SHALL use [ECB] block mode encryption with 128 bit key -* `aes-192-ecb` that SHALL use [ECB] block mode encryption with 192 bit key -* `aes-256-ecb` that SHALL use [ECB] block mode encryption with 256 bit key * `aes-128-cbc` that SHALL use [CBC] block mode encryption with 128 bit key * `aes-192-cbc` that SHALL use [CBC] block mode encryption with 192 bit key * `aes-192-cbc` that SHALL use [CBC] block mode encryption with 256 bit key @@ -750,9 +735,6 @@ version: 1.0 [ClickHouse] SHALL return an error when the `aes_encrypt_mysql` function is called with the following parameter values -* `aes-128-ecb` mode and `key` is less than 16 bytes or `iv` is specified -* `aes-192-ecb` mode and `key` is less than 24 bytes or `iv` is specified -* `aes-256-ecb` mode and `key` is less than 32 bytes or `iv` is specified * `aes-128-cbc` mode and `key` is less than 16 bytes or if specified `iv` is less than 16 bytes * `aes-192-cbc` mode and `key` is less than 24 bytes or if specified `iv` is less than 16 bytes * `aes-256-cbc` mode and `key` is less than 32 bytes or if specified `iv` is less than 16 bytes @@ -810,7 +792,7 @@ version: 1.0 of the `aes_decrypt_mysql` function where the `key_length` SHALL specifies the length of the key and SHALL accept `128`, `192`, or `256` as the values and the `mode` SHALL specify the block encryption -mode and SHALL accept [ECB], [CBC], [CFB128], or [OFB]. For example, `aes-256-ofb`. +mode and SHALL accept [CBC], [CFB128], or [OFB]. For example, `aes-256-ofb`. #### RQ.SRS008.AES.MySQL.Decrypt.Function.Parameters.Mode.Value.Invalid version: 1.0 @@ -825,9 +807,6 @@ version: 1.0 [ClickHouse] SHALL support the following [AES] block encryption modes as the value for the `mode` parameter of the `aes_decrypt_mysql` function: -* `aes-128-ecb` that SHALL use [ECB] block mode encryption with 128 bit key -* `aes-192-ecb` that SHALL use [ECB] block mode encryption with 192 bit key -* `aes-256-ecb` that SHALL use [ECB] block mode encryption with 256 bit key * `aes-128-cbc` that SHALL use [CBC] block mode encryption with 128 bit key * `aes-192-cbc` that SHALL use [CBC] block mode encryption with 192 bit key * `aes-192-cbc` that SHALL use [CBC] block mode encryption with 256 bit key @@ -916,9 +895,6 @@ version: 1.0 [ClickHouse] SHALL return an error when the `aes_decrypt_mysql` function is called with the following parameter values -* `aes-128-ecb` mode and `key` is less than 16 bytes or `iv` is specified -* `aes-192-ecb` mode and `key` is less than 24 bytes or `iv` is specified -* `aes-256-ecb` mode and `key` is less than 32 bytes or `iv` is specified * `aes-128-cbc` mode and `key` is less than 16 bytes or if specified `iv` is less than 16 bytes * `aes-192-cbc` mode and `key` is less than 24 bytes or if specified `iv` is less than 16 bytes * `aes-256-cbc` mode and `key` is less than 32 bytes or if specified `iv` is less than 16 bytes @@ -954,7 +930,6 @@ version: 1.0 [GCM]: https://en.wikipedia.org/wiki/Galois/Counter_Mode [CTR]: https://en.wikipedia.org/wiki/Block_cipher_mode_of_operation#Counter_(CTR) [CBC]: https://en.wikipedia.org/wiki/Block_cipher_mode_of_operation#Cipher_block_chaining_(CBC) -[ECB]: https://en.wikipedia.org/wiki/Block_cipher_mode_of_operation#Electronic_codebook_(ECB) [CFB]: https://en.wikipedia.org/wiki/Block_cipher_mode_of_operation#Cipher_feedback_(CFB) [CFB128]: https://en.wikipedia.org/wiki/Block_cipher_mode_of_operation#Cipher_feedback_(CFB) [OFB]: https://en.wikipedia.org/wiki/Block_cipher_mode_of_operation#Output_feedback_(OFB) diff --git a/tests/testflows/aes_encryption/requirements/requirements.py b/tests/testflows/aes_encryption/requirements/requirements.py index 0fbbea7e85a..4523f2d820f 100644 --- a/tests/testflows/aes_encryption/requirements/requirements.py +++ b/tests/testflows/aes_encryption/requirements/requirements.py @@ -429,7 +429,7 @@ RQ_SRS008_AES_Encrypt_Function_Parameters_Mode_ValuesFormat = Requirement( "of the `encrypt` function where\n" "the `key_length` SHALL specifies the length of the key and SHALL accept\n" "`128`, `192`, or `256` as the values and the `mode` SHALL specify the block encryption\n" - "mode and SHALL accept [ECB], [CBC], [CFB128], or [OFB] as well as\n" + "mode and SHALL accept [CBC], [CFB128], or [OFB] as well as\n" "[CTR] and [GCM] as the values. For example, `aes-256-ofb`.\n" "\n" ), @@ -467,9 +467,6 @@ RQ_SRS008_AES_Encrypt_Function_Parameters_Mode_Values = Requirement( "[ClickHouse] SHALL support the following [AES] block encryption modes as the value for the `mode` parameter\n" "of the `encrypt` function:\n" "\n" - "* `aes-128-ecb` that SHALL use [ECB] block mode encryption with 128 bit key\n" - "* `aes-192-ecb` that SHALL use [ECB] block mode encryption with 192 bit key\n" - "* `aes-256-ecb` that SHALL use [ECB] block mode encryption with 256 bit key\n" "* `aes-128-cbc` that SHALL use [CBC] block mode encryption with 128 bit key\n" "* `aes-192-cbc` that SHALL use [CBC] block mode encryption with 192 bit key\n" "* `aes-192-cbc` that SHALL use [CBC] block mode encryption with 256 bit key\n" @@ -642,9 +639,6 @@ RQ_SRS008_AES_Encrypt_Function_NonGCMMode_KeyAndInitializationVector_Length = Re "[ClickHouse] SHALL return an error when the `encrypt` function is called with the following parameter values\n" "when using non-GCM modes\n" "\n" - "* `aes-128-ecb` mode and `key` is not 16 bytes or `iv` or `aad` is specified\n" - "* `aes-192-ecb` mode and `key` is not 24 bytes or `iv` or `aad` is specified\n" - "* `aes-256-ecb` mode and `key` is not 32 bytes or `iv` or `aad` is specified\n" "* `aes-128-cbc` mode and `key` is not 16 bytes or if specified `iv` is not 16 bytes or `aad` is specified\n" "* `aes-192-cbc` mode and `key` is not 24 bytes or if specified `iv` is not 16 bytes or `aad` is specified\n" "* `aes-256-cbc` mode and `key` is not 32 bytes or if specified `iv` is not 16 bytes or `aad` is specified\n" @@ -790,7 +784,7 @@ RQ_SRS008_AES_Decrypt_Function_Parameters_Mode_ValuesFormat = Requirement( "of the `decrypt` function where\n" "the `key_length` SHALL specifies the length of the key and SHALL accept\n" "`128`, `192`, or `256` as the values and the `mode` SHALL specify the block encryption\n" - "mode and SHALL accept [ECB], [CBC], [CFB128], or [OFB] as well as\n" + "mode and SHALL accept [CBC], [CFB128], or [OFB] as well as\n" "[CTR] and [GCM] as the values. For example, `aes-256-ofb`.\n" "\n" ), @@ -828,9 +822,6 @@ RQ_SRS008_AES_Decrypt_Function_Parameters_Mode_Values = Requirement( "[ClickHouse] SHALL support the following [AES] block encryption modes as the value for the `mode` parameter\n" "of the `decrypt` function:\n" "\n" - "* `aes-128-ecb` that SHALL use [ECB] block mode encryption with 128 bit key\n" - "* `aes-192-ecb` that SHALL use [ECB] block mode encryption with 192 bit key\n" - "* `aes-256-ecb` that SHALL use [ECB] block mode encryption with 256 bit key\n" "* `aes-128-cbc` that SHALL use [CBC] block mode encryption with 128 bit key\n" "* `aes-192-cbc` that SHALL use [CBC] block mode encryption with 192 bit key\n" "* `aes-192-cbc` that SHALL use [CBC] block mode encryption with 256 bit key\n" @@ -1005,9 +996,6 @@ RQ_SRS008_AES_Decrypt_Function_NonGCMMode_KeyAndInitializationVector_Length = Re "[ClickHouse] SHALL return an error when the `decrypt` function is called with the following parameter values\n" "when using non-GCM modes\n" "\n" - "* `aes-128-ecb` mode and `key` is not 16 bytes or `iv` or `aad` is specified\n" - "* `aes-192-ecb` mode and `key` is not 24 bytes or `iv` or `aad` is specified\n" - "* `aes-256-ecb` mode and `key` is not 32 bytes or `iv` or `aad` is specified\n" "* `aes-128-cbc` mode and `key` is not 16 bytes or if specified `iv` is not 16 bytes or `aad` is specified\n" "* `aes-192-cbc` mode and `key` is not 24 bytes or if specified `iv` is not 16 bytes or `aad` is specified\n" "* `aes-256-cbc` mode and `key` is not 32 bytes or if specified `iv` is not 16 bytes or `aad` is specified\n" @@ -1154,7 +1142,7 @@ RQ_SRS008_AES_MySQL_Encrypt_Function_Parameters_Mode_ValuesFormat = Requirement( "of the `aes_encrypt_mysql` function where\n" "the `key_length` SHALL specifies the length of the key and SHALL accept\n" "`128`, `192`, or `256` as the values and the `mode` SHALL specify the block encryption\n" - "mode and SHALL accept [ECB], [CBC], [CFB128], or [OFB]. For example, `aes-256-ofb`.\n" + "mode and SHALL accept [CBC], [CFB128], or [OFB]. For example, `aes-256-ofb`.\n" "\n" ), link=None, @@ -1191,9 +1179,6 @@ RQ_SRS008_AES_MySQL_Encrypt_Function_Parameters_Mode_Values = Requirement( "[ClickHouse] SHALL support the following [AES] block encryption modes as the value for the `mode` parameter\n" "of the `aes_encrypt_mysql` function:\n" "\n" - "* `aes-128-ecb` that SHALL use [ECB] block mode encryption with 128 bit key\n" - "* `aes-192-ecb` that SHALL use [ECB] block mode encryption with 192 bit key\n" - "* `aes-256-ecb` that SHALL use [ECB] block mode encryption with 256 bit key\n" "* `aes-128-cbc` that SHALL use [CBC] block mode encryption with 128 bit key\n" "* `aes-192-cbc` that SHALL use [CBC] block mode encryption with 192 bit key\n" "* `aes-192-cbc` that SHALL use [CBC] block mode encryption with 256 bit key\n" @@ -1392,9 +1377,6 @@ RQ_SRS008_AES_MySQL_Encrypt_Function_Mode_KeyAndInitializationVector_Length = Re description=( "[ClickHouse] SHALL return an error when the `aes_encrypt_mysql` function is called with the following parameter values\n" "\n" - "* `aes-128-ecb` mode and `key` is less than 16 bytes or `iv` is specified\n" - "* `aes-192-ecb` mode and `key` is less than 24 bytes or `iv` is specified\n" - "* `aes-256-ecb` mode and `key` is less than 32 bytes or `iv` is specified\n" "* `aes-128-cbc` mode and `key` is less than 16 bytes or if specified `iv` is less than 16 bytes\n" "* `aes-192-cbc` mode and `key` is less than 24 bytes or if specified `iv` is less than 16 bytes\n" "* `aes-256-cbc` mode and `key` is less than 32 bytes or if specified `iv` is less than 16 bytes\n" @@ -1516,7 +1498,7 @@ RQ_SRS008_AES_MySQL_Decrypt_Function_Parameters_Mode_ValuesFormat = Requirement( "of the `aes_decrypt_mysql` function where\n" "the `key_length` SHALL specifies the length of the key and SHALL accept\n" "`128`, `192`, or `256` as the values and the `mode` SHALL specify the block encryption\n" - "mode and SHALL accept [ECB], [CBC], [CFB128], or [OFB]. For example, `aes-256-ofb`.\n" + "mode and SHALL accept [CBC], [CFB128], or [OFB]. For example, `aes-256-ofb`.\n" "\n" ), link=None, @@ -1553,9 +1535,6 @@ RQ_SRS008_AES_MySQL_Decrypt_Function_Parameters_Mode_Values = Requirement( "[ClickHouse] SHALL support the following [AES] block encryption modes as the value for the `mode` parameter\n" "of the `aes_decrypt_mysql` function:\n" "\n" - "* `aes-128-ecb` that SHALL use [ECB] block mode encryption with 128 bit key\n" - "* `aes-192-ecb` that SHALL use [ECB] block mode encryption with 192 bit key\n" - "* `aes-256-ecb` that SHALL use [ECB] block mode encryption with 256 bit key\n" "* `aes-128-cbc` that SHALL use [CBC] block mode encryption with 128 bit key\n" "* `aes-192-cbc` that SHALL use [CBC] block mode encryption with 192 bit key\n" "* `aes-192-cbc` that SHALL use [CBC] block mode encryption with 256 bit key\n" @@ -1754,9 +1733,6 @@ RQ_SRS008_AES_MySQL_Decrypt_Function_Mode_KeyAndInitializationVector_Length = Re description=( "[ClickHouse] SHALL return an error when the `aes_decrypt_mysql` function is called with the following parameter values\n" "\n" - "* `aes-128-ecb` mode and `key` is less than 16 bytes or `iv` is specified\n" - "* `aes-192-ecb` mode and `key` is less than 24 bytes or `iv` is specified\n" - "* `aes-256-ecb` mode and `key` is less than 32 bytes or `iv` is specified\n" "* `aes-128-cbc` mode and `key` is less than 16 bytes or if specified `iv` is less than 16 bytes\n" "* `aes-192-cbc` mode and `key` is less than 24 bytes or if specified `iv` is less than 16 bytes\n" "* `aes-256-cbc` mode and `key` is less than 32 bytes or if specified `iv` is less than 16 bytes\n" @@ -2606,7 +2582,7 @@ version: 1.0 of the `encrypt` function where the `key_length` SHALL specifies the length of the key and SHALL accept `128`, `192`, or `256` as the values and the `mode` SHALL specify the block encryption -mode and SHALL accept [ECB], [CBC], [CFB128], or [OFB] as well as +mode and SHALL accept [CBC], [CFB128], or [OFB] as well as [CTR] and [GCM] as the values. For example, `aes-256-ofb`. #### RQ.SRS008.AES.Encrypt.Function.Parameters.Mode.Value.Invalid @@ -2622,9 +2598,6 @@ version: 1.0 [ClickHouse] SHALL support the following [AES] block encryption modes as the value for the `mode` parameter of the `encrypt` function: -* `aes-128-ecb` that SHALL use [ECB] block mode encryption with 128 bit key -* `aes-192-ecb` that SHALL use [ECB] block mode encryption with 192 bit key -* `aes-256-ecb` that SHALL use [ECB] block mode encryption with 256 bit key * `aes-128-cbc` that SHALL use [CBC] block mode encryption with 128 bit key * `aes-192-cbc` that SHALL use [CBC] block mode encryption with 192 bit key * `aes-192-cbc` that SHALL use [CBC] block mode encryption with 256 bit key @@ -2698,9 +2671,6 @@ version: 1.0 [ClickHouse] SHALL return an error when the `encrypt` function is called with the following parameter values when using non-GCM modes -* `aes-128-ecb` mode and `key` is not 16 bytes or `iv` or `aad` is specified -* `aes-192-ecb` mode and `key` is not 24 bytes or `iv` or `aad` is specified -* `aes-256-ecb` mode and `key` is not 32 bytes or `iv` or `aad` is specified * `aes-128-cbc` mode and `key` is not 16 bytes or if specified `iv` is not 16 bytes or `aad` is specified * `aes-192-cbc` mode and `key` is not 24 bytes or if specified `iv` is not 16 bytes or `aad` is specified * `aes-256-cbc` mode and `key` is not 32 bytes or if specified `iv` is not 16 bytes or `aad` is specified @@ -2771,7 +2741,7 @@ version: 1.0 of the `decrypt` function where the `key_length` SHALL specifies the length of the key and SHALL accept `128`, `192`, or `256` as the values and the `mode` SHALL specify the block encryption -mode and SHALL accept [ECB], [CBC], [CFB128], or [OFB] as well as +mode and SHALL accept [CBC], [CFB128], or [OFB] as well as [CTR] and [GCM] as the values. For example, `aes-256-ofb`. #### RQ.SRS008.AES.Decrypt.Function.Parameters.Mode.Value.Invalid @@ -2787,9 +2757,6 @@ version: 1.0 [ClickHouse] SHALL support the following [AES] block encryption modes as the value for the `mode` parameter of the `decrypt` function: -* `aes-128-ecb` that SHALL use [ECB] block mode encryption with 128 bit key -* `aes-192-ecb` that SHALL use [ECB] block mode encryption with 192 bit key -* `aes-256-ecb` that SHALL use [ECB] block mode encryption with 256 bit key * `aes-128-cbc` that SHALL use [CBC] block mode encryption with 128 bit key * `aes-192-cbc` that SHALL use [CBC] block mode encryption with 192 bit key * `aes-192-cbc` that SHALL use [CBC] block mode encryption with 256 bit key @@ -2865,9 +2832,6 @@ version: 1.0 [ClickHouse] SHALL return an error when the `decrypt` function is called with the following parameter values when using non-GCM modes -* `aes-128-ecb` mode and `key` is not 16 bytes or `iv` or `aad` is specified -* `aes-192-ecb` mode and `key` is not 24 bytes or `iv` or `aad` is specified -* `aes-256-ecb` mode and `key` is not 32 bytes or `iv` or `aad` is specified * `aes-128-cbc` mode and `key` is not 16 bytes or if specified `iv` is not 16 bytes or `aad` is specified * `aes-192-cbc` mode and `key` is not 24 bytes or if specified `iv` is not 16 bytes or `aad` is specified * `aes-256-cbc` mode and `key` is not 32 bytes or if specified `iv` is not 16 bytes or `aad` is specified @@ -2939,7 +2903,7 @@ version: 1.0 of the `aes_encrypt_mysql` function where the `key_length` SHALL specifies the length of the key and SHALL accept `128`, `192`, or `256` as the values and the `mode` SHALL specify the block encryption -mode and SHALL accept [ECB], [CBC], [CFB128], or [OFB]. For example, `aes-256-ofb`. +mode and SHALL accept [CBC], [CFB128], or [OFB]. For example, `aes-256-ofb`. #### RQ.SRS008.AES.MySQL.Encrypt.Function.Parameters.Mode.Value.Invalid version: 1.0 @@ -2954,9 +2918,6 @@ version: 1.0 [ClickHouse] SHALL support the following [AES] block encryption modes as the value for the `mode` parameter of the `aes_encrypt_mysql` function: -* `aes-128-ecb` that SHALL use [ECB] block mode encryption with 128 bit key -* `aes-192-ecb` that SHALL use [ECB] block mode encryption with 192 bit key -* `aes-256-ecb` that SHALL use [ECB] block mode encryption with 256 bit key * `aes-128-cbc` that SHALL use [CBC] block mode encryption with 128 bit key * `aes-192-cbc` that SHALL use [CBC] block mode encryption with 192 bit key * `aes-192-cbc` that SHALL use [CBC] block mode encryption with 256 bit key @@ -3045,9 +3006,6 @@ version: 1.0 [ClickHouse] SHALL return an error when the `aes_encrypt_mysql` function is called with the following parameter values -* `aes-128-ecb` mode and `key` is less than 16 bytes or `iv` is specified -* `aes-192-ecb` mode and `key` is less than 24 bytes or `iv` is specified -* `aes-256-ecb` mode and `key` is less than 32 bytes or `iv` is specified * `aes-128-cbc` mode and `key` is less than 16 bytes or if specified `iv` is less than 16 bytes * `aes-192-cbc` mode and `key` is less than 24 bytes or if specified `iv` is less than 16 bytes * `aes-256-cbc` mode and `key` is less than 32 bytes or if specified `iv` is less than 16 bytes @@ -3105,7 +3063,7 @@ version: 1.0 of the `aes_decrypt_mysql` function where the `key_length` SHALL specifies the length of the key and SHALL accept `128`, `192`, or `256` as the values and the `mode` SHALL specify the block encryption -mode and SHALL accept [ECB], [CBC], [CFB128], or [OFB]. For example, `aes-256-ofb`. +mode and SHALL accept [CBC], [CFB128], or [OFB]. For example, `aes-256-ofb`. #### RQ.SRS008.AES.MySQL.Decrypt.Function.Parameters.Mode.Value.Invalid version: 1.0 @@ -3120,9 +3078,6 @@ version: 1.0 [ClickHouse] SHALL support the following [AES] block encryption modes as the value for the `mode` parameter of the `aes_decrypt_mysql` function: -* `aes-128-ecb` that SHALL use [ECB] block mode encryption with 128 bit key -* `aes-192-ecb` that SHALL use [ECB] block mode encryption with 192 bit key -* `aes-256-ecb` that SHALL use [ECB] block mode encryption with 256 bit key * `aes-128-cbc` that SHALL use [CBC] block mode encryption with 128 bit key * `aes-192-cbc` that SHALL use [CBC] block mode encryption with 192 bit key * `aes-192-cbc` that SHALL use [CBC] block mode encryption with 256 bit key @@ -3211,9 +3166,6 @@ version: 1.0 [ClickHouse] SHALL return an error when the `aes_decrypt_mysql` function is called with the following parameter values -* `aes-128-ecb` mode and `key` is less than 16 bytes or `iv` is specified -* `aes-192-ecb` mode and `key` is less than 24 bytes or `iv` is specified -* `aes-256-ecb` mode and `key` is less than 32 bytes or `iv` is specified * `aes-128-cbc` mode and `key` is less than 16 bytes or if specified `iv` is less than 16 bytes * `aes-192-cbc` mode and `key` is less than 24 bytes or if specified `iv` is less than 16 bytes * `aes-256-cbc` mode and `key` is less than 32 bytes or if specified `iv` is less than 16 bytes @@ -3249,7 +3201,6 @@ version: 1.0 [GCM]: https://en.wikipedia.org/wiki/Galois/Counter_Mode [CTR]: https://en.wikipedia.org/wiki/Block_cipher_mode_of_operation#Counter_(CTR) [CBC]: https://en.wikipedia.org/wiki/Block_cipher_mode_of_operation#Cipher_block_chaining_(CBC) -[ECB]: https://en.wikipedia.org/wiki/Block_cipher_mode_of_operation#Electronic_codebook_(ECB) [CFB]: https://en.wikipedia.org/wiki/Block_cipher_mode_of_operation#Cipher_feedback_(CFB) [CFB128]: https://en.wikipedia.org/wiki/Block_cipher_mode_of_operation#Cipher_feedback_(CFB) [OFB]: https://en.wikipedia.org/wiki/Block_cipher_mode_of_operation#Output_feedback_(OFB) From b3f59537ca3e201b5dc47cc4625f74aff46aa83f Mon Sep 17 00:00:00 2001 From: LAL2211 Date: Wed, 6 Apr 2022 13:34:20 -0400 Subject: [PATCH 06/11] added additional check --- .../external_user_directory/tests/common.py | 25 +++++++++++++------ 1 file changed, 18 insertions(+), 7 deletions(-) diff --git a/tests/testflows/ldap/external_user_directory/tests/common.py b/tests/testflows/ldap/external_user_directory/tests/common.py index 871be815a35..7400459d266 100644 --- a/tests/testflows/ldap/external_user_directory/tests/common.py +++ b/tests/testflows/ldap/external_user_directory/tests/common.py @@ -84,14 +84,25 @@ def rbac_roles(*roles, node=None): node.query(f"DROP ROLE IF EXISTS {role}") -def verify_ldap_user_exists(server, username, password): +# def verify_ldap_user_exists(server, username, password): +# """Check that LDAP user is defined on the LDAP server.""" +# with By("searching LDAP database"): +# ldap_node = current().context.cluster.node(server) +# r = ldap_node.command( +# f"ldapwhoami -H ldap://localhost -D 'cn={user_name},ou=users,dc=company,dc=com' -w {password}" +# ) +# assert r.exitcode == 0, error() + + +def verify_ldap_user_exists(server, username, password, check=False): """Check that LDAP user is defined on the LDAP server.""" - with By("searching LDAP database"): - ldap_node = current().context.cluster.node(server) - r = ldap_node.command( - f"ldapwhoami -H ldap://localhost -D 'cn={user_name},ou=users,dc=company,dc=com' -w {password}" - ) - assert r.exitcode == 0, error() + if check: + with By("searching LDAP database"): + ldap_node = current().context.cluster.node(server) + r = ldap_node.command( + f"ldapwhoami -H ldap://localhost -D 'cn={user_name},ou=users,dc=company,dc=com' -w {password}" + ) + assert r.exitcode == 0, error() def create_ldap_external_user_directory_config_content( From 68bdcdca80cefe8af0479c2e9d6516ac289f6c00 Mon Sep 17 00:00:00 2001 From: LAL2211 Date: Wed, 6 Apr 2022 14:15:30 -0400 Subject: [PATCH 07/11] code fix for Ldap Injection --- .../external_user_directory/tests/common.py | 21 ------------------- 1 file changed, 21 deletions(-) diff --git a/tests/testflows/ldap/external_user_directory/tests/common.py b/tests/testflows/ldap/external_user_directory/tests/common.py index 7400459d266..c0b6e72cd8e 100644 --- a/tests/testflows/ldap/external_user_directory/tests/common.py +++ b/tests/testflows/ldap/external_user_directory/tests/common.py @@ -84,27 +84,6 @@ def rbac_roles(*roles, node=None): node.query(f"DROP ROLE IF EXISTS {role}") -# def verify_ldap_user_exists(server, username, password): -# """Check that LDAP user is defined on the LDAP server.""" -# with By("searching LDAP database"): -# ldap_node = current().context.cluster.node(server) -# r = ldap_node.command( -# f"ldapwhoami -H ldap://localhost -D 'cn={user_name},ou=users,dc=company,dc=com' -w {password}" -# ) -# assert r.exitcode == 0, error() - - -def verify_ldap_user_exists(server, username, password, check=False): - """Check that LDAP user is defined on the LDAP server.""" - if check: - with By("searching LDAP database"): - ldap_node = current().context.cluster.node(server) - r = ldap_node.command( - f"ldapwhoami -H ldap://localhost -D 'cn={user_name},ou=users,dc=company,dc=com' -w {password}" - ) - assert r.exitcode == 0, error() - - def create_ldap_external_user_directory_config_content( server=None, roles=None, **kwargs ): From 77edd41b2e0e2a8e9d89541eb3bf6a8323aaedbb Mon Sep 17 00:00:00 2001 From: jewisliu Date: Wed, 6 Apr 2022 15:23:18 +0800 Subject: [PATCH 08/11] [Improvement] improvement in PARTITION ALL 1. ASTPartition::formatImpl should output ALL while executing ALTER TABLE t DETACH PARTITION ALL 2. prohibit PARTITION ALL excepte DETACH PARTITION ALL --- src/Parsers/ASTPartition.cpp | 13 +++++++---- src/Parsers/ParserPartition.cpp | 4 ---- src/Storages/MergeTree/MergeTreeData.cpp | 20 +++++++++++----- src/Storages/StorageMergeTree.cpp | 5 ++-- .../0_stateless/00753_alter_attach.sql | 23 +++++++++++++++++++ 5 files changed, 49 insertions(+), 16 deletions(-) diff --git a/src/Parsers/ASTPartition.cpp b/src/Parsers/ASTPartition.cpp index 06bfe4f5217..87d159b5817 100644 --- a/src/Parsers/ASTPartition.cpp +++ b/src/Parsers/ASTPartition.cpp @@ -35,10 +35,15 @@ void ASTPartition::formatImpl(const FormatSettings & settings, FormatState & sta } else { - settings.ostr << (settings.hilite ? hilite_keyword : "") << "ID " << (settings.hilite ? hilite_none : ""); - WriteBufferFromOwnString id_buf; - writeQuoted(id, id_buf); - settings.ostr << id_buf.str(); + if (all) + settings.ostr << "ALL"; + else + { + settings.ostr << (settings.hilite ? hilite_keyword : "") << "ID " << (settings.hilite ? hilite_none : ""); + WriteBufferFromOwnString id_buf; + writeQuoted(id, id_buf); + settings.ostr << id_buf.str(); + } } } diff --git a/src/Parsers/ParserPartition.cpp b/src/Parsers/ParserPartition.cpp index 5af442826df..9f1d4d4e889 100644 --- a/src/Parsers/ParserPartition.cpp +++ b/src/Parsers/ParserPartition.cpp @@ -32,10 +32,6 @@ bool ParserPartition::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) } else if (s_all.ignore(pos, expected)) { - ASTPtr value = makeASTFunction("tuple"); - partition->value = value; - partition->children.push_back(value); - partition->fields_count = 0; partition->all = true; } else diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index eacec8f50e5..478c1570a23 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -3385,7 +3385,14 @@ void MergeTreeData::checkAlterPartitionIsPossible( else { /// We are able to parse it - getPartitionIDFromQuery(command.partition, getContext()); + const auto * partition_ast = command.partition->as(); + if (partition_ast && partition_ast->all) + { + if (command.type != PartitionCommand::DROP_PARTITION) + throw DB::Exception("Only support DETACH PARTITION ALL currently", ErrorCodes::SUPPORT_IS_DISABLED); + } + else + getPartitionIDFromQuery(command.partition, getContext()); } } } @@ -3393,14 +3400,15 @@ void MergeTreeData::checkAlterPartitionIsPossible( void MergeTreeData::checkPartitionCanBeDropped(const ASTPtr & partition) { - const String partition_id = getPartitionIDFromQuery(partition, getContext()); DataPartsVector parts_to_remove; const auto * partition_ast = partition->as(); if (partition_ast && partition_ast->all) parts_to_remove = getDataPartsVector(); else + { + const String partition_id = getPartitionIDFromQuery(partition, getContext()); parts_to_remove = getDataPartsVectorInPartition(MergeTreeDataPartState::Active, partition_id); - + } UInt64 partition_size = 0; for (const auto & part : parts_to_remove) @@ -3828,6 +3836,9 @@ String MergeTreeData::getPartitionIDFromQuery(const ASTPtr & ast, ContextPtr loc { const auto & partition_ast = ast->as(); + if (partition_ast.all) + throw Exception("Only Support DETACH PARTITION ALL currently", ErrorCodes::SUPPORT_IS_DISABLED); + if (!partition_ast.value) { MergeTreePartInfo::validatePartitionID(partition_ast.id, format_version); @@ -3847,11 +3858,8 @@ String MergeTreeData::getPartitionIDFromQuery(const ASTPtr & ast, ContextPtr loc } /// Re-parse partition key fields using the information about expected field types. - auto metadata_snapshot = getInMemoryMetadataPtr(); const Block & key_sample_block = metadata_snapshot->getPartitionKey().sample_block; - if (partition_ast.all) - return "ALL"; size_t fields_count = key_sample_block.columns(); if (partition_ast.fields_count != fields_count) throw Exception(ErrorCodes::INVALID_PARTITION_VALUE, diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index b1392f073ea..e7e4528dc83 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -1347,13 +1347,14 @@ void StorageMergeTree::dropPartition(const ASTPtr & partition, bool detach, Cont /// Asks to complete merges and does not allow them to start. /// This protects against "revival" of data for a removed partition after completion of merge. auto merge_blocker = stopMergesAndWait(); - String partition_id = getPartitionIDFromQuery(partition, local_context); const auto * partition_ast = partition->as(); if (partition_ast && partition_ast->all) parts_to_remove = getDataPartsVector(); else + { + String partition_id = getPartitionIDFromQuery(partition, local_context); parts_to_remove = getDataPartsVectorInPartition(MergeTreeDataPartState::Active, partition_id); - + } /// TODO should we throw an exception if parts_to_remove is empty? removePartsFromWorkingSet(parts_to_remove, true); } diff --git a/tests/queries/0_stateless/00753_alter_attach.sql b/tests/queries/0_stateless/00753_alter_attach.sql index 2910bcc222b..9fa4f92c2c1 100644 --- a/tests/queries/0_stateless/00753_alter_attach.sql +++ b/tests/queries/0_stateless/00753_alter_attach.sql @@ -66,6 +66,29 @@ select * from replicated_table_detach_all1 order by id; SYSTEM SYNC REPLICA replicated_table_detach_all2; select * from replicated_table_detach_all2 order by id; +ALTER TABLE replicated_table_detach_all1 FETCH PARTITION ALL FROM '/clickhouse/tables/test_00753_{database}/replicated_table_detach_all1'; -- { serverError 344 } + DROP TABLE replicated_table_detach_all1; DROP TABLE replicated_table_detach_all2; +DROP TABLE IF EXISTS partition_all; +DROP TABLE IF EXISTS partition_all2; + +CREATE TABLE partition_all (x UInt64, p UInt8, q UInt8) ENGINE = MergeTree ORDER BY tuple() PARTITION BY p; +INSERT INTO partition_all VALUES (4, 1, 2), (5, 1, 3), (3, 1, 4); + +CREATE TABLE partition_all2 (x UInt64, p UInt8, q UInt8) ENGINE = MergeTree ORDER BY tuple() PARTITION BY p; +INSERT INTO partition_all2 VALUES (4, 1, 2), (5, 1, 3), (3, 1, 4); + +-- test PARTITION ALL +ALTER TABLE partition_all2 REPLACE PARTITION ALL FROM partition_all; -- { serverError 344 } +ALTER TABLE partition_all MOVE PARTITION ALL TO TABLE partition_all2; -- { serverError 344 } +ALTER TABLE partition_all2 CLEAR INDEX p IN PARTITION ALL; -- { serverError 344 } +ALTER TABLE partition_all2 CLEAR COLUMN q IN PARTITION ALL; -- { serverError 344 } +ALTER TABLE partition_all2 UPDATE q = q + 1 IN PARTITION ALL where p = 1; -- { serverError 344 } +ALTER TABLE partition_all2 FREEZE PARTITION ALL; -- { serverError 344 } +CHECK TABLE partition_all2 PARTITION ALL; -- { serverError 344 } +OPTIMIZE TABLE partition_all2 PARTITION ALL; -- { serverError 344 } + +DROP TABLE partition_all; +DROP TABLE partition_all2; From 44895e5287c27bea160f2371264f9a32ad8b2714 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Thu, 7 Apr 2022 16:28:43 +0200 Subject: [PATCH 09/11] Improve build cache for docker images - Save all layer's cache - Use cache with fallback to a latest tags --- tests/ci/docker_images_check.py | 3 ++- tests/ci/docker_test.py | 15 +++++++++------ 2 files changed, 11 insertions(+), 7 deletions(-) diff --git a/tests/ci/docker_images_check.py b/tests/ci/docker_images_check.py index 43671e3a2f7..57227ef307e 100644 --- a/tests/ci/docker_images_check.py +++ b/tests/ci/docker_images_check.py @@ -237,9 +237,10 @@ def build_and_push_one_image( "docker buildx build --builder default " f"--label build-url={GITHUB_RUN_URL} " f"{from_tag_arg}" - f"--build-arg BUILDKIT_INLINE_CACHE=1 " f"--tag {image.repo}:{version_string} " f"--cache-from type=registry,ref={image.repo}:{version_string} " + f"--cache-from type=registry,ref={image.repo}:latest " + f"--cache-to type=inline,mode=max " f"{push_arg}" f"--progress plain {image.full_path}" ) diff --git a/tests/ci/docker_test.py b/tests/ci/docker_test.py index 0d1c554026f..8a0c4a0dfce 100644 --- a/tests/ci/docker_test.py +++ b/tests/ci/docker_test.py @@ -124,8 +124,9 @@ class TestDockerImageCheck(unittest.TestCase): self.assertIn( f"docker buildx build --builder default --label build-url={GITHUB_RUN_URL} " "--build-arg FROM_TAG=version " - "--build-arg BUILDKIT_INLINE_CACHE=1 --tag name:version --cache-from " - "type=registry,ref=name:version --push --progress plain path", + "--tag name:version --cache-from type=registry,ref=name:version " + "--cache-from type=registry,ref=name:latest " + "--cache-to type=inline,mode=max --push --progress plain path", mock_popen.call_args.args, ) self.assertTrue(result) @@ -141,8 +142,9 @@ class TestDockerImageCheck(unittest.TestCase): self.assertIn( f"docker buildx build --builder default --label build-url={GITHUB_RUN_URL} " "--build-arg FROM_TAG=version2 " - "--build-arg BUILDKIT_INLINE_CACHE=1 --tag name:version2 --cache-from " - "type=registry,ref=name:version2 --progress plain path", + "--tag name:version2 --cache-from type=registry,ref=name:version2 " + "--cache-from type=registry,ref=name:latest " + "--cache-to type=inline,mode=max --progress plain path", mock_popen.call_args.args, ) self.assertTrue(result) @@ -157,8 +159,9 @@ class TestDockerImageCheck(unittest.TestCase): mock_machine.assert_not_called() self.assertIn( f"docker buildx build --builder default --label build-url={GITHUB_RUN_URL} " - "--build-arg BUILDKIT_INLINE_CACHE=1 --tag name:version2 --cache-from " - "type=registry,ref=name:version2 --progress plain path", + "--tag name:version2 --cache-from type=registry,ref=name:version2 " + "--cache-from type=registry,ref=name:latest " + "--cache-to type=inline,mode=max --progress plain path", mock_popen.call_args.args, ) self.assertFalse(result) From 8111010815cb77f177254a79beda2ac433b5487a Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Thu, 7 Apr 2022 17:08:56 +0200 Subject: [PATCH 10/11] Rebuild a builder for test --- docker/docs/builder/Dockerfile | 1 - 1 file changed, 1 deletion(-) diff --git a/docker/docs/builder/Dockerfile b/docker/docs/builder/Dockerfile index 906312a19a2..061251aa7f0 100644 --- a/docker/docs/builder/Dockerfile +++ b/docker/docs/builder/Dockerfile @@ -1,4 +1,3 @@ -# rebuild in #33610 # docker build -t clickhouse/docs-builder . FROM ubuntu:20.04 From 9fa6e6e7dfd3499e147718520b66f49d8fb46d79 Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Thu, 7 Apr 2022 17:42:33 +0200 Subject: [PATCH 11/11] Make backward compatibility check always green --- docker/test/stress/run.sh | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/docker/test/stress/run.sh b/docker/test/stress/run.sh index ba85999caa5..27d5f9c4be4 100755 --- a/docker/test/stress/run.sh +++ b/docker/test/stress/run.sh @@ -330,7 +330,7 @@ then -e "UNFINISHED" \ -e "Renaming unexpected part" \ /var/log/clickhouse-server/clickhouse-server.backward.*.log | zgrep -Fa "" > /test_output/bc_check_error_messages.txt \ - && echo -e 'Backward compatibility check: Error message in clickhouse-server.log (see bc_check_error_messages.txt)\tFAIL' >> /test_output/test_results.tsv \ + && echo -e 'Backward compatibility check: Error message in clickhouse-server.log (see bc_check_error_messages.txt)\tOK' >> /test_output/test_results.tsv \ || echo -e 'Backward compatibility check: No Error messages in clickhouse-server.log\tOK' >> /test_output/test_results.tsv # Remove file bc_check_error_messages.txt if it's empty @@ -346,7 +346,7 @@ then # OOM zgrep -Fa " Application: Child process was terminated by signal 9" /var/log/clickhouse-server/clickhouse-server.backward.*.log > /dev/null \ - && echo -e 'Backward compatibility check: OOM killer (or signal 9) in clickhouse-server.log\tFAIL' >> /test_output/test_results.tsv \ + && echo -e 'Backward compatibility check: OOM killer (or signal 9) in clickhouse-server.log\tOK' >> /test_output/test_results.tsv \ || echo -e 'Backward compatibility check: No OOM messages in clickhouse-server.log\tOK' >> /test_output/test_results.tsv # Logical errors @@ -366,7 +366,7 @@ then # It also checks for crash without stacktrace (printed by watchdog) echo "Check for Fatal message in server log:" zgrep -Fa " " /var/log/clickhouse-server/clickhouse-server.backward.*.log > /test_output/bc_check_fatal_messages.txt \ - && echo -e 'Backward compatibility check: Fatal message in clickhouse-server.log (see bc_check_fatal_messages.txt)\tFAIL' >> /test_output/test_results.tsv \ + && echo -e 'Backward compatibility check: Fatal message in clickhouse-server.log (see bc_check_fatal_messages.txt)\tOK' >> /test_output/test_results.tsv \ || echo -e 'Backward compatibility check: No fatal messages in clickhouse-server.log\tOK' >> /test_output/test_results.tsv # Remove file bc_check_fatal_messages.txt if it's empty