mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-24 08:32:02 +00:00
Merge branch 'master' into optimize_trivial_hive_query
This commit is contained in:
commit
dc4c638af8
@ -1,4 +1,3 @@
|
||||
# rebuild in #33610
|
||||
# docker build -t clickhouse/docs-builder .
|
||||
FROM ubuntu:20.04
|
||||
|
||||
|
@ -330,7 +330,7 @@ then
|
||||
-e "UNFINISHED" \
|
||||
-e "Renaming unexpected part" \
|
||||
/var/log/clickhouse-server/clickhouse-server.backward.*.log | zgrep -Fa "<Error>" > /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 " <Fatal> 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 " <Fatal> " /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
|
||||
|
@ -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<size_t>(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;
|
||||
}
|
||||
|
@ -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();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -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
|
||||
|
@ -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<ASTPartition>();
|
||||
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<ASTPartition>();
|
||||
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<ASTPartition &>();
|
||||
|
||||
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,
|
||||
|
@ -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<ASTPartition>();
|
||||
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);
|
||||
}
|
||||
|
@ -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}"
|
||||
)
|
||||
|
@ -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)
|
||||
|
@ -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;
|
||||
|
@ -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)
|
||||
|
@ -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)
|
||||
|
@ -84,16 +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 create_ldap_external_user_directory_config_content(
|
||||
server=None, roles=None, **kwargs
|
||||
):
|
||||
|
@ -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
|
||||
|
|
Loading…
Reference in New Issue
Block a user