mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-21 07:01:59 +00:00
Merge branch 'master' into mvcc_prototype
This commit is contained in:
commit
6a46da93ae
@ -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,9 +785,20 @@ void ClientBase::receiveResult(ASTPtr parsed_query)
|
||||
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;
|
||||
|
@ -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())
|
||||
{
|
||||
|
@ -938,10 +938,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;
|
||||
}
|
||||
@ -1012,12 +1012,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;
|
||||
}
|
||||
|
||||
|
||||
|
@ -232,7 +232,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.
|
||||
@ -498,8 +499,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(
|
||||
|
@ -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];
|
||||
|
@ -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<DataTypeUInt32>(), "_shard_num"}});
|
||||
if (options.shard_count)
|
||||
context->addLocalScalar(
|
||||
context->addSpecialScalar(
|
||||
"_shard_count",
|
||||
Block{{DataTypeUInt32().createColumnConst(1, *options.shard_count), std::make_shared<DataTypeUInt32>(), "_shard_count"}});
|
||||
}
|
||||
|
@ -34,12 +34,17 @@ void ASTPartition::formatImpl(const FormatSettings & settings, FormatState & sta
|
||||
value->formatImpl(settings, state, frame);
|
||||
}
|
||||
else
|
||||
{
|
||||
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
|
||||
|
@ -3567,6 +3567,13 @@ void MergeTreeData::checkAlterPartitionIsPossible(
|
||||
else
|
||||
{
|
||||
/// We are able to parse it
|
||||
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());
|
||||
}
|
||||
}
|
||||
@ -3575,14 +3582,15 @@ void MergeTreeData::checkAlterPartitionIsPossible(
|
||||
|
||||
void MergeTreeData::checkPartitionCanBeDropped(const ASTPtr & partition, ContextPtr local_context)
|
||||
{
|
||||
const String partition_id = getPartitionIDFromQuery(partition, local_context);
|
||||
DataPartsVector parts_to_remove;
|
||||
const auto * partition_ast = partition->as<ASTPartition>();
|
||||
if (partition_ast && partition_ast->all)
|
||||
parts_to_remove = getVisibleDataPartsVector(local_context);
|
||||
else
|
||||
{
|
||||
const String partition_id = getPartitionIDFromQuery(partition, local_context);
|
||||
parts_to_remove = getVisibleDataPartsVectorInPartition(local_context, partition_id);
|
||||
|
||||
}
|
||||
UInt64 partition_size = 0;
|
||||
|
||||
for (const auto & part : parts_to_remove)
|
||||
@ -4013,6 +4021,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);
|
||||
@ -4032,11 +4043,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,
|
||||
|
@ -1473,13 +1473,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 = getVisibleDataPartsVector(local_context);
|
||||
else
|
||||
{
|
||||
String partition_id = getPartitionIDFromQuery(partition, local_context);
|
||||
parts_to_remove = getVisibleDataPartsVectorInPartition(local_context, partition_id);
|
||||
|
||||
}
|
||||
/// TODO should we throw an exception if parts_to_remove is empty?
|
||||
removePartsFromWorkingSet(local_context->getCurrentTransaction().get(), 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)
|
||||
|
17
tests/performance/scalar2.xml
Normal file
17
tests/performance/scalar2.xml
Normal file
@ -0,0 +1,17 @@
|
||||
<test>
|
||||
<create_query>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_query>
|
||||
|
||||
<create_query>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</create_query>
|
||||
|
||||
<fill_query>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</fill_query>
|
||||
|
||||
<fill_query>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</fill_query>
|
||||
|
||||
<create_query>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)</create_query>
|
||||
|
||||
<query>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</query>
|
||||
|
||||
<drop_query>DROP TABLE IF EXISTS tbl</drop_query>
|
||||
<drop_query>DROP TABLE IF EXISTS tbl0</drop_query>
|
||||
<drop_query>DROP TABLE IF EXISTS tmp_acc_hit</drop_query>
|
||||
</test>
|
@ -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;
|
||||
|
@ -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
|
||||
|
@ -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,
|
||||
|
@ -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