Merge branch 'master' of https://github.com/ClickHouse/ClickHouse into master

This commit is contained in:
George 2021-01-28 19:41:32 +03:00
commit 20abc8f29f
104 changed files with 2138 additions and 778 deletions

2
contrib/aws vendored

@ -1 +1 @@
Subproject commit a220591e335923ce1c19bbf9eb925787f7ab6c13
Subproject commit 7d48b2c8193679cc4516e5bd68ae4a64b94dae7d

View File

@ -43,6 +43,7 @@ RUN apt-get update \
clang-tidy-${LLVM_VERSION} \
cmake \
curl \
lsof \
expect \
fakeroot \
git \

View File

@ -21,13 +21,16 @@ function clone
git init
git remote add origin https://github.com/ClickHouse/ClickHouse
git fetch --depth=100 origin "$SHA_TO_TEST"
git fetch --depth=100 origin master # Used to obtain the list of modified or added tests
# Network is unreliable. GitHub neither.
for _ in {1..100}; do git fetch --depth=100 origin "$SHA_TO_TEST" && break; sleep 1; done
# Used to obtain the list of modified or added tests
for _ in {1..100}; do git fetch --depth=100 origin master && break; sleep 1; done
# If not master, try to fetch pull/.../{head,merge}
if [ "$PR_TO_TEST" != "0" ]
then
git fetch --depth=100 origin "refs/pull/$PR_TO_TEST/*:refs/heads/pull/$PR_TO_TEST/*"
for _ in {1..100}; do git fetch --depth=100 origin "refs/pull/$PR_TO_TEST/*:refs/heads/pull/$PR_TO_TEST/*" && break; sleep 1; done
fi
git checkout "$SHA_TO_TEST"
@ -189,14 +192,14 @@ case "$stage" in
echo "failure" > status.txt
if ! grep -ao "Received signal.*\|Logical error.*\|Assertion.*failed\|Failed assertion.*\|.*runtime error: .*\|.*is located.*\|SUMMARY: MemorySanitizer:.*\|SUMMARY: ThreadSanitizer:.*" server.log > description.txt
then
echo "Lost connection to server. See the logs" > description.txt
echo "Lost connection to server. See the logs." > description.txt
fi
else
# Something different -- maybe the fuzzer itself died? Don't grep the
# server log in this case, because we will find a message about normal
# server termination (Received signal 15), which is confusing.
echo "failure" > status.txt
echo "Fuzzer failed ($fuzzer_exit_code). See the logs" > description.txt
echo "Fuzzer failed ($fuzzer_exit_code). See the logs." > description.txt
fi
;&
"report")

View File

@ -55,12 +55,11 @@ function run_tests()
ADDITIONAL_OPTIONS+=('00000_no_tests_to_skip')
fi
for _ in $(seq 1 "$NUM_TRIES"); do
clickhouse-test --testname --shard --zookeeper --hung-check --print-time "$SKIP_LIST_OPT" "${ADDITIONAL_OPTIONS[@]}" 2>&1 | ts '%Y-%m-%d %H:%M:%S' | tee -a test_output/test_result.txt
if [ "${PIPESTATUS[0]}" -ne "0" ]; then
break;
fi
done
clickhouse-test --testname --shard --zookeeper --hung-check --print-time \
--test-runs "$NUM_TRIES" --jobs 4 \
"$SKIP_LIST_OPT" "${ADDITIONAL_OPTIONS[@]}" 2>&1 \
| ts '%Y-%m-%d %H:%M:%S' \
| tee -a test_output/test_result.txt
}
export -f run_tests

View File

@ -7,16 +7,16 @@ Columns:
- `id` ([UUID](../../sql-reference/data-types/uuid.md)) — Quota ID.
- `storage`([String](../../sql-reference/data-types/string.md)) — Storage of quotas. Possible value: “users.xml” if a quota configured in the users.xml file, “disk” if a quota configured by an SQL-query.
- `keys` ([Array](../../sql-reference/data-types/array.md)([Enum8](../../sql-reference/data-types/enum.md))) — Key specifies how the quota should be shared. If two connections use the same quota and key, they share the same amounts of resources. Values:
- `[]` — All users share the same quota.
- `['user_name']` — Connections with the same user name share the same quota.
- `['ip_address']` — Connections from the same IP share the same quota.
- `['client_key']` — Connections with the same key share the same quota. A key must be explicitly provided by a client. When using [clickhouse-client](../../interfaces/cli.md), pass a key value in the `--quota-key` parameter, or use the `quota_key` parameter in the client configuration file. When using HTTP interface, use the `X-ClickHouse-Quota` header.
- `['user_name', 'client_key']` — Connections with the same `client_key` share the same quota. If a key isnt provided by a client, the qouta is tracked for `user_name`.
- `['client_key', 'ip_address']` — Connections with the same `client_key` share the same quota. If a key isnt provided by a client, the qouta is tracked for `ip_address`.
- `[]` — All users share the same quota.
- `['user_name']` — Connections with the same user name share the same quota.
- `['ip_address']` — Connections from the same IP share the same quota.
- `['client_key']` — Connections with the same key share the same quota. A key must be explicitly provided by a client. When using [clickhouse-client](../../interfaces/cli.md), pass a key value in the `--quota-key` parameter, or use the `quota_key` parameter in the client configuration file. When using HTTP interface, use the `X-ClickHouse-Quota` header.
- `['user_name', 'client_key']` — Connections with the same `client_key` share the same quota. If a key isnt provided by a client, the qouta is tracked for `user_name`.
- `['client_key', 'ip_address']` — Connections with the same `client_key` share the same quota. If a key isnt provided by a client, the qouta is tracked for `ip_address`.
- `durations` ([Array](../../sql-reference/data-types/array.md)([UInt64](../../sql-reference/data-types/int-uint.md))) — Time interval lengths in seconds.
- `apply_to_all` ([UInt8](../../sql-reference/data-types/int-uint.md#uint-ranges)) — Logical value. It shows which users the quota is applied to. Values:
- `0` — The quota applies to users specify in the `apply_to_list`.
- `1` — The quota applies to all users except those listed in `apply_to_except`.
- `0` — The quota applies to users specify in the `apply_to_list`.
- `1` — The quota applies to all users except those listed in `apply_to_except`.
- `apply_to_list` ([Array](../../sql-reference/data-types/array.md)([String](../../sql-reference/data-types/string.md))) — List of user names/[roles](../../operations/access-rights.md#role-management) that the quota should be applied to.
- `apply_to_except` ([Array](../../sql-reference/data-types/array.md)([String](../../sql-reference/data-types/string.md))) — List of user names/roles that the quota should not apply to.

View File

@ -5,16 +5,35 @@ toc_title: QUOTA
# ALTER QUOTA {#alter-quota-statement}
Changes quotas.
Changes [quotas](../../../operations/access-rights.md#quotas-management).
Syntax:
``` sql
ALTER QUOTA [IF EXISTS] name [ON CLUSTER cluster_name]
[RENAME TO new_name]
[KEYED BY {'none' | 'user name' | 'ip address' | 'client key' | 'client key or user name' | 'client key or ip address'}]
[FOR [RANDOMIZED] INTERVAL number {SECOND | MINUTE | HOUR | DAY | WEEK | MONTH | QUARTER | YEAR}
{MAX { {QUERIES | ERRORS | RESULT ROWS | RESULT BYTES | READ ROWS | READ BYTES | EXECUTION TIME} = number } [,...] |
[KEYED BY {user_name | ip_address | client_key | client_key,user_name | client_key,ip_address} | NOT KEYED]
[FOR [RANDOMIZED] INTERVAL number {second | minute | hour | day | week | month | quarter | year}
{MAX { {queries | errors | result_rows | result_bytes | read_rows | read_bytes | execution_time} = number } [,...] |
NO LIMITS | TRACKING ONLY} [,...]]
[TO {role [,...] | ALL | ALL EXCEPT role [,...]}]
```
Keys `user_name`, `ip_address`, `client_key`, `client_key, user_name` and `client_key, ip_address` correspond to the fields in the [system.quotas](../../../operations/system-tables/quotas.md) table.
Parameters `queries`, `errors`, `result_rows`, `result_bytes`, `read_rows`, `read_bytes`, `execution_time` correspond to the fields in the [system.quotas_usage](../../../operations/system-tables/quotas_usage.md) table.
`ON CLUSTER` clause allows creating quotas on a cluster, see [Distributed DDL](../../../sql-reference/distributed-ddl.md).
**Examples**
Limit the maximum number of queries for the current user with 123 queries in 15 months constraint:
``` sql
ALTER QUOTA IF EXISTS qA FOR INTERVAL 15 month MAX queries = 123 TO CURRENT_USER;
```
For the default user limit the maximum execution time with half a second in 30 minutes, and limit the maximum number of queries with 321 and the maximum number of errors with 10 in 5 quaters:
``` sql
ALTER QUOTA IF EXISTS qB FOR INTERVAL 30 minute MAX execution_time = 0.5, FOR INTERVAL 5 quarter MAX queries = 321, errors = 10 TO default;
```

View File

@ -11,19 +11,29 @@ Syntax:
``` sql
CREATE QUOTA [IF NOT EXISTS | OR REPLACE] name [ON CLUSTER cluster_name]
[KEYED BY {'none' | 'user name' | 'ip address' | 'forwarded ip address' | 'client key' | 'client key or user name' | 'client key or ip address'}]
[FOR [RANDOMIZED] INTERVAL number {SECOND | MINUTE | HOUR | DAY | WEEK | MONTH | QUARTER | YEAR}
{MAX { {QUERIES | ERRORS | RESULT ROWS | RESULT BYTES | READ ROWS | READ BYTES | EXECUTION TIME} = number } [,...] |
[KEYED BY {user_name | ip_address | client_key | client_key,user_name | client_key,ip_address} | NOT KEYED]
[FOR [RANDOMIZED] INTERVAL number {second | minute | hour | day | week | month | quarter | year}
{MAX { {queries | errors | result_rows | result_bytes | read_rows | read_bytes | execution_time} = number } [,...] |
NO LIMITS | TRACKING ONLY} [,...]]
[TO {role [,...] | ALL | ALL EXCEPT role [,...]}]
```
Keys `user_name`, `ip_address`, `client_key`, `client_key, user_name` and `client_key, ip_address` correspond to the fields in the [system.quotas](../../../operations/system-tables/quotas.md) table.
Parameters `queries`, `errors`, `result_rows`, `result_bytes`, `read_rows`, `read_bytes`, `execution_time` correspond to the fields in the [system.quotas_usage](../../../operations/system-tables/quotas_usage.md) table.
`ON CLUSTER` clause allows creating quotas on a cluster, see [Distributed DDL](../../../sql-reference/distributed-ddl.md).
## Example {#create-quota-example}
**Examples**
Limit the maximum number of queries for the current user with 123 queries in 15 months constraint:
``` sql
CREATE QUOTA qA FOR INTERVAL 15 MONTH MAX QUERIES 123 TO CURRENT_USER
CREATE QUOTA qA FOR INTERVAL 15 month MAX queries = 123 TO CURRENT_USER;
```
For the default user limit the maximum execution time with half a second in 30 minutes, and limit the maximum number of queries with 321 and the maximum number of errors with 10 in 5 quaters:
``` sql
CREATE QUOTA qB FOR INTERVAL 30 minute MAX execution_time = 0.5, FOR INTERVAL 5 quarter MAX queries = 321, errors = 10 TO default;
```

View File

@ -5,18 +5,38 @@ toc_title: QUOTA
# ALTER QUOTA {#alter-quota-statement}
Изменяет квоту.
Изменяет [квоту](../../../operations/access-rights.md#quotas-management).
## Синтаксис {#alter-quota-syntax}
Синтаксис:
``` sql
ALTER QUOTA [IF EXISTS] name [ON CLUSTER cluster_name]
[RENAME TO new_name]
[KEYED BY {'none' | 'user name' | 'ip address' | 'client key' | 'client key or user name' | 'client key or ip address'}]
[FOR [RANDOMIZED] INTERVAL number {SECOND | MINUTE | HOUR | DAY | WEEK | MONTH | QUARTER | YEAR}
{MAX { {QUERIES | ERRORS | RESULT ROWS | RESULT BYTES | READ ROWS | READ BYTES | EXECUTION TIME} = number } [,...] |
[KEYED BY {user_name | ip_address | client_key | client_key,user_name | client_key,ip_address} | NOT KEYED]
[FOR [RANDOMIZED] INTERVAL number {second | minute | hour | day | week | month | quarter | year}
{MAX { {queries | errors | result_rows | result_bytes | read_rows | read_bytes | execution_time} = number } [,...] |
NO LIMITS | TRACKING ONLY} [,...]]
[TO {role [,...] | ALL | ALL EXCEPT role [,...]}]
```
[Оригинальная статья](https://clickhouse.tech/docs/ru/query_language/alter/quota/) <!--hide-->
Ключи `user_name`, `ip_address`, `client_key`, `client_key, user_name` и `client_key, ip_address` соответствуют полям таблицы [system.quotas](../../../operations/system-tables/quotas.md).
Параметры `queries`, `errors`, `result_rows`, `result_bytes`, `read_rows`, `read_bytes`, `execution_time` соответствуют полям таблицы [system.quotas_usage](../../../operations/system-tables/quotas_usage.md).
В секции `ON CLUSTER` можно указать кластеры, на которых создается квота, см. [Распределенные DDL запросы](../../../sql-reference/distributed-ddl.md).
**Примеры**
Ограничить для текущего пользователя максимальное число запросов — не более 123 запросов за каждые 15 месяцев:
``` sql
ALTER QUOTA IF EXISTS qA FOR INTERVAL 15 month MAX queries = 123 TO CURRENT_USER;
```
Ограничить по умолчанию максимальное время выполнения запроса — не более полсекунды за каждые 30 минут, а также максимальное число запросов — не более 321 и максимальное число ошибок — не более 10 за каждые 5 кварталов:
``` sql
ALTER QUOTA IF EXISTS qB FOR INTERVAL 30 minute MAX execution_time = 0.5, FOR INTERVAL 5 quarter MAX queries = 321, errors = 10 TO default;
```
[Оригинальная статья](https://clickhouse.tech/docs/ru/sql-reference/alter/quota/) <!--hide-->

View File

@ -7,23 +7,34 @@ toc_title: "\u041a\u0432\u043e\u0442\u0430"
Создает [квоту](../../../operations/access-rights.md#quotas-management), которая может быть присвоена пользователю или роли.
### Синтаксис {#create-quota-syntax}
Синтаксис:
``` sql
CREATE QUOTA [IF NOT EXISTS | OR REPLACE] name [ON CLUSTER cluster_name]
[KEYED BY {'none' | 'user name' | 'ip address' | 'client key' | 'client key or user name' | 'client key or ip address'}]
[FOR [RANDOMIZED] INTERVAL number {SECOND | MINUTE | HOUR | DAY | WEEK | MONTH | QUARTER | YEAR}
{MAX { {QUERIES | ERRORS | RESULT ROWS | RESULT BYTES | READ ROWS | READ BYTES | EXECUTION TIME} = number } [,...] |
[KEYED BY {user_name | ip_address | client_key | client_key, user_name | client_key, ip_address} | NOT KEYED]
[FOR [RANDOMIZED] INTERVAL number {second | minute | hour | day | week | month | quarter | year}
{MAX { {queries | errors | result_rows | result_bytes | read_rows | read_bytes | execution_time} = number } [,...] |
NO LIMITS | TRACKING ONLY} [,...]]
[TO {role [,...] | ALL | ALL EXCEPT role [,...]}]
```
Ключи `user_name`, `ip_address`, `client_key`, `client_key, user_name` и `client_key, ip_address` соответствуют полям таблицы [system.quotas](../../../operations/system-tables/quotas.md).
### Пример {#create-quota-example}
Параметры `queries`, `errors`, `result_rows`, `result_bytes`, `read_rows`, `read_bytes`, `execution_time` соответствуют полям таблицы [system.quotas_usage](../../../operations/system-tables/quotas_usage.md).
Ограничить максимальное количество запросов для текущего пользователя до 123 запросов каждые 15 месяцев:
В секции `ON CLUSTER` можно указать кластеры, на которых создается квота, см. [Распределенные DDL запросы](../../../sql-reference/distributed-ddl.md).
**Примеры**
Ограничить максимальное количество запросов для текущего пользователя — не более 123 запросов за каждые 15 месяцев:
``` sql
CREATE QUOTA qA FOR INTERVAL 15 MONTH MAX QUERIES 123 TO CURRENT_USER
CREATE QUOTA qA FOR INTERVAL 15 month MAX queries = 123 TO CURRENT_USER;
```
Ограничить по умолчанию максимальное время выполнения запроса — не более полсекунды за каждые 30 минут, а также максимальное число запросов — не более 321 и максимальное число ошибок — не более 10 за каждые 5 кварталов:
``` sql
CREATE QUOTA qB FOR INTERVAL 30 minute MAX execution_time = 0.5, FOR INTERVAL 5 quarter MAX queries = 321, errors = 10 TO default;
```
[Оригинальная статья](https://clickhouse.tech/docs/ru/sql-reference/statements/create/quota)

View File

@ -229,7 +229,7 @@ public:
{
for (const auto & x : small)
{
if (!rb->contains(static_cast<Value>(x.getValue())))
if (!r1.rb->contains(static_cast<Value>(x.getValue())))
buffer.push_back(x.getValue());
}

View File

@ -29,8 +29,14 @@ namespace ErrorCodes
DataTypePtr DataTypeFactory::get(const String & full_name) const
{
/// Data type parser can be invoked from coroutines with small stack.
/// Value 315 is known to cause stack overflow in some test configurations (debug build, sanitizers)
/// let's make the threshold significantly lower.
/// It is impractical for user to have complex data types with this depth.
static constexpr size_t data_type_max_parse_depth = 200;
ParserDataType parser;
ASTPtr ast = parseQuery(parser, full_name.data(), full_name.data() + full_name.size(), "data type", 0, DBMS_DEFAULT_MAX_PARSER_DEPTH);
ASTPtr ast = parseQuery(parser, full_name.data(), full_name.data() + full_name.size(), "data type", 0, data_type_max_parse_depth);
return get(ast);
}

View File

@ -112,32 +112,33 @@ void registerDiskS3(DiskFactory & factory)
Poco::File disk{context.getPath() + "disks/" + name};
disk.createDirectories();
Aws::Client::ClientConfiguration cfg;
S3::PocoHTTPClientConfiguration client_configuration = S3::ClientFactory::instance().createClientConfiguration(
context.getRemoteHostFilter(),
context.getGlobalContext().getSettingsRef().s3_max_redirects);
S3::URI uri(Poco::URI(config.getString(config_prefix + ".endpoint")));
if (uri.key.back() != '/')
throw Exception("S3 path must ends with '/', but '" + uri.key + "' doesn't.", ErrorCodes::BAD_ARGUMENTS);
cfg.connectTimeoutMs = config.getUInt(config_prefix + ".connect_timeout_ms", 10000);
cfg.httpRequestTimeoutMs = config.getUInt(config_prefix + ".request_timeout_ms", 5000);
cfg.maxConnections = config.getUInt(config_prefix + ".max_connections", 100);
cfg.endpointOverride = uri.endpoint;
client_configuration.connectTimeoutMs = config.getUInt(config_prefix + ".connect_timeout_ms", 10000);
client_configuration.httpRequestTimeoutMs = config.getUInt(config_prefix + ".request_timeout_ms", 5000);
client_configuration.maxConnections = config.getUInt(config_prefix + ".max_connections", 100);
client_configuration.endpointOverride = uri.endpoint;
auto proxy_config = getProxyConfiguration(config_prefix, config);
if (proxy_config)
cfg.perRequestConfiguration = [proxy_config](const auto & request) { return proxy_config->getConfiguration(request); };
client_configuration.perRequestConfiguration = [proxy_config](const auto & request) { return proxy_config->getConfiguration(request); };
cfg.retryStrategy = std::make_shared<Aws::Client::DefaultRetryStrategy>(
client_configuration.retryStrategy = std::make_shared<Aws::Client::DefaultRetryStrategy>(
config.getUInt(config_prefix + ".retry_attempts", 10));
auto client = S3::ClientFactory::instance().create(
cfg,
client_configuration,
uri.is_virtual_hosted_style,
config.getString(config_prefix + ".access_key_id", ""),
config.getString(config_prefix + ".secret_access_key", ""),
config.getBool(config_prefix + ".use_environment_credentials", config.getBool("s3.use_environment_credentials", false)),
context.getRemoteHostFilter(),
context.getGlobalContext().getSettingsRef().s3_max_redirects);
config.getBool(config_prefix + ".use_environment_credentials", config.getBool("s3.use_environment_credentials", false))
);
String metadata_path = config.getString(config_prefix + ".metadata_path", context.getPath() + "disks/" + name + "/");

View File

@ -95,9 +95,11 @@ void registerFunctionsConversion(FunctionFactory & factory)
factory.registerFunction<FunctionToUUIDOrNull>();
factory.registerFunction<FunctionParseDateTimeBestEffort>();
factory.registerFunction<FunctionParseDateTimeBestEffortUS>();
factory.registerFunction<FunctionParseDateTimeBestEffortOrZero>();
factory.registerFunction<FunctionParseDateTimeBestEffortOrNull>();
factory.registerFunction<FunctionParseDateTimeBestEffortUS>();
factory.registerFunction<FunctionParseDateTimeBestEffortUSOrZero>();
factory.registerFunction<FunctionParseDateTimeBestEffortUSOrNull>();
factory.registerFunction<FunctionParseDateTime32BestEffort>();
factory.registerFunction<FunctionParseDateTime32BestEffortOrZero>();
factory.registerFunction<FunctionParseDateTime32BestEffortOrNull>();

View File

@ -955,6 +955,12 @@ struct ConvertThroughParsing
vec_to[i] = res;
}
}
else if constexpr (parsing_mode == ConvertFromStringParsingMode::BestEffortUS)
{
time_t res;
parsed = tryParseDateTimeBestEffortUS(res, read_buffer, *local_time_zone, *utc_time_zone);
vec_to[i] = res;
}
else
{
if constexpr (to_datetime64)
@ -2003,9 +2009,11 @@ using FunctionToDecimal256OrNull = FunctionConvertFromString<DataTypeDecimal<Dec
using FunctionToUUIDOrNull = FunctionConvertFromString<DataTypeUUID, NameToUUIDOrNull, ConvertFromStringExceptionMode::Null>;
struct NameParseDateTimeBestEffort { static constexpr auto name = "parseDateTimeBestEffort"; };
struct NameParseDateTimeBestEffortUS { static constexpr auto name = "parseDateTimeBestEffortUS"; };
struct NameParseDateTimeBestEffortOrZero { static constexpr auto name = "parseDateTimeBestEffortOrZero"; };
struct NameParseDateTimeBestEffortOrNull { static constexpr auto name = "parseDateTimeBestEffortOrNull"; };
struct NameParseDateTimeBestEffortUS { static constexpr auto name = "parseDateTimeBestEffortUS"; };
struct NameParseDateTimeBestEffortUSOrZero { static constexpr auto name = "parseDateTimeBestEffortUSOrZero"; };
struct NameParseDateTimeBestEffortUSOrNull { static constexpr auto name = "parseDateTimeBestEffortUSOrNull"; };
struct NameParseDateTime32BestEffort { static constexpr auto name = "parseDateTime32BestEffort"; };
struct NameParseDateTime32BestEffortOrZero { static constexpr auto name = "parseDateTime32BestEffortOrZero"; };
struct NameParseDateTime32BestEffortOrNull { static constexpr auto name = "parseDateTime32BestEffortOrNull"; };
@ -2016,13 +2024,18 @@ struct NameParseDateTime64BestEffortOrNull { static constexpr auto name = "parse
using FunctionParseDateTimeBestEffort = FunctionConvertFromString<
DataTypeDateTime, NameParseDateTimeBestEffort, ConvertFromStringExceptionMode::Throw, ConvertFromStringParsingMode::BestEffort>;
using FunctionParseDateTimeBestEffortUS = FunctionConvertFromString<
DataTypeDateTime, NameParseDateTimeBestEffortUS, ConvertFromStringExceptionMode::Throw, ConvertFromStringParsingMode::BestEffortUS>;
using FunctionParseDateTimeBestEffortOrZero = FunctionConvertFromString<
DataTypeDateTime, NameParseDateTimeBestEffortOrZero, ConvertFromStringExceptionMode::Zero, ConvertFromStringParsingMode::BestEffort>;
using FunctionParseDateTimeBestEffortOrNull = FunctionConvertFromString<
DataTypeDateTime, NameParseDateTimeBestEffortOrNull, ConvertFromStringExceptionMode::Null, ConvertFromStringParsingMode::BestEffort>;
using FunctionParseDateTimeBestEffortUS = FunctionConvertFromString<
DataTypeDateTime, NameParseDateTimeBestEffortUS, ConvertFromStringExceptionMode::Throw, ConvertFromStringParsingMode::BestEffortUS>;
using FunctionParseDateTimeBestEffortUSOrZero = FunctionConvertFromString<
DataTypeDateTime, NameParseDateTimeBestEffortUSOrZero, ConvertFromStringExceptionMode::Zero, ConvertFromStringParsingMode::BestEffortUS>;
using FunctionParseDateTimeBestEffortUSOrNull = FunctionConvertFromString<
DataTypeDateTime, NameParseDateTimeBestEffortUSOrNull, ConvertFromStringExceptionMode::Null, ConvertFromStringParsingMode::BestEffortUS>;
using FunctionParseDateTime32BestEffort = FunctionConvertFromString<
DataTypeDateTime, NameParseDateTime32BestEffort, ConvertFromStringExceptionMode::Throw, ConvertFromStringParsingMode::BestEffort>;
using FunctionParseDateTime32BestEffortOrZero = FunctionConvertFromString<

View File

@ -50,6 +50,10 @@ public:
bool useDefaultImplementationForConstants() const override { return false; }
/// We do not use default implementation for LowCardinality because this is not a pure function.
/// If used, optimization for LC may execute function only for dictionary, which gives wrong result.
bool useDefaultImplementationForLowCardinalityColumns() const override { return false; }
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
{
size_t number_of_arguments = arguments.size();

View File

@ -6,13 +6,11 @@
#include <utility>
#include <IO/HTTPCommon.h>
#include <IO/S3/SessionAwareAwsStream.h>
#include <IO/WriteBufferFromString.h>
#include <IO/Operators.h>
#include <Common/Stopwatch.h>
#include <aws/core/http/HttpRequest.h>
#include <aws/core/http/HttpResponse.h>
#include <aws/core/http/standard/StandardHttpResponse.h>
#include <aws/core/monitoring/HttpClientMetrics.h>
#include <aws/core/utils/ratelimiter/RateLimiterInterface.h>
#include "Poco/StreamCopier.h"
@ -49,11 +47,9 @@ namespace DB::S3
{
PocoHTTPClientConfiguration::PocoHTTPClientConfiguration(
const Aws::Client::ClientConfiguration & cfg,
const RemoteHostFilter & remote_host_filter_,
unsigned int s3_max_redirects_)
: Aws::Client::ClientConfiguration(cfg)
, remote_host_filter(remote_host_filter_)
: remote_host_filter(remote_host_filter_)
, s3_max_redirects(s3_max_redirects_)
{
}
@ -90,29 +86,19 @@ PocoHTTPClient::PocoHTTPClient(const PocoHTTPClientConfiguration & clientConfigu
{
}
std::shared_ptr<Aws::Http::HttpResponse> PocoHTTPClient::MakeRequest(
Aws::Http::HttpRequest & request,
Aws::Utils::RateLimits::RateLimiterInterface * readLimiter,
Aws::Utils::RateLimits::RateLimiterInterface * writeLimiter) const
{
auto response = Aws::MakeShared<Aws::Http::Standard::StandardHttpResponse>("PocoHTTPClient", request);
makeRequestInternal(request, response, readLimiter, writeLimiter);
return response;
}
std::shared_ptr<Aws::Http::HttpResponse> PocoHTTPClient::MakeRequest(
const std::shared_ptr<Aws::Http::HttpRequest> & request,
Aws::Utils::RateLimits::RateLimiterInterface * readLimiter,
Aws::Utils::RateLimits::RateLimiterInterface * writeLimiter) const
{
auto response = Aws::MakeShared<Aws::Http::Standard::StandardHttpResponse>("PocoHTTPClient", request);
auto response = Aws::MakeShared<PocoHTTPResponse>("PocoHTTPClient", request);
makeRequestInternal(*request, response, readLimiter, writeLimiter);
return response;
}
void PocoHTTPClient::makeRequestInternal(
Aws::Http::HttpRequest & request,
std::shared_ptr<Aws::Http::Standard::StandardHttpResponse> & response,
std::shared_ptr<PocoHTTPResponse> & response,
Aws::Utils::RateLimits::RateLimiterInterface *,
Aws::Utils::RateLimits::RateLimiterInterface *) const
{
@ -278,7 +264,7 @@ void PocoHTTPClient::makeRequestInternal(
}
}
else
response->GetResponseStream().SetUnderlyingStream(std::make_shared<SessionAwareAwsStream<decltype(session)>>(session, response_body_stream));
response->SetResponseBody(response_body_stream, session);
return;
}

View File

@ -2,9 +2,12 @@
#include <Common/RemoteHostFilter.h>
#include <IO/ConnectionTimeouts.h>
#include <IO/HTTPCommon.h>
#include <IO/S3/SessionAwareIOStream.h>
#include <aws/core/client/ClientConfiguration.h>
#include <aws/core/http/HttpClient.h>
#include <aws/core/http/HttpRequest.h>
#include <aws/core/http/standard/StandardHttpResponse.h>
namespace Aws::Http::Standard
{
@ -18,16 +21,52 @@ class Context;
namespace DB::S3
{
class ClientFactory;
struct PocoHTTPClientConfiguration : public Aws::Client::ClientConfiguration
{
const RemoteHostFilter & remote_host_filter;
unsigned int s3_max_redirects;
PocoHTTPClientConfiguration(const Aws::Client::ClientConfiguration & cfg, const RemoteHostFilter & remote_host_filter_,
unsigned int s3_max_redirects_);
void updateSchemeAndRegion();
private:
PocoHTTPClientConfiguration(const RemoteHostFilter & remote_host_filter_, unsigned int s3_max_redirects_);
/// Constructor of Aws::Client::ClientConfiguration must be called after AWS SDK initialization.
friend ClientFactory;
};
class PocoHTTPResponse : public Aws::Http::Standard::StandardHttpResponse
{
public:
using SessionPtr = PooledHTTPSessionPtr;
PocoHTTPResponse(const std::shared_ptr<const Aws::Http::HttpRequest> request)
: Aws::Http::Standard::StandardHttpResponse(request)
, body_stream(request->GetResponseStreamFactory())
{
}
void SetResponseBody(Aws::IStream & incoming_stream, SessionPtr & session_)
{
body_stream = Aws::Utils::Stream::ResponseStream(
Aws::New<SessionAwareIOStream<SessionPtr>>("http result streambuf", session_, incoming_stream.rdbuf())
);
}
Aws::IOStream & GetResponseBody() const override
{
return body_stream.GetUnderlyingStream();
}
Aws::Utils::Stream::ResponseStream && SwapResponseStreamOwnership() override
{
return std::move(body_stream);
}
private:
Aws::Utils::Stream::ResponseStream body_stream;
};
class PocoHTTPClient : public Aws::Http::HttpClient
@ -35,10 +74,6 @@ class PocoHTTPClient : public Aws::Http::HttpClient
public:
explicit PocoHTTPClient(const PocoHTTPClientConfiguration & clientConfiguration);
~PocoHTTPClient() override = default;
std::shared_ptr<Aws::Http::HttpResponse> MakeRequest(
Aws::Http::HttpRequest & request,
Aws::Utils::RateLimits::RateLimiterInterface * readLimiter,
Aws::Utils::RateLimits::RateLimiterInterface * writeLimiter) const override;
std::shared_ptr<Aws::Http::HttpResponse> MakeRequest(
const std::shared_ptr<Aws::Http::HttpRequest> & request,
@ -48,7 +83,7 @@ public:
private:
void makeRequestInternal(
Aws::Http::HttpRequest & request,
std::shared_ptr<Aws::Http::Standard::StandardHttpResponse> & response,
std::shared_ptr<PocoHTTPResponse> & response,
Aws::Utils::RateLimits::RateLimiterInterface * readLimiter,
Aws::Utils::RateLimits::RateLimiterInterface * writeLimiter) const;

View File

@ -1,27 +0,0 @@
#pragma once
#include <IO/HTTPCommon.h>
#include <aws/core/utils/stream/ResponseStream.h>
namespace DB::S3
{
/**
* Wrapper of IOStream to store response stream and corresponding HTTP session.
*/
template <typename Session>
class SessionAwareAwsStream : public Aws::IStream
{
public:
SessionAwareAwsStream(Session session_, std::istream & response_stream_)
: Aws::IStream(response_stream_.rdbuf()), session(std::move(session_))
{
}
private:
/// Poco HTTP session is holder of response stream.
Session session;
};
}

View File

@ -0,0 +1,26 @@
#pragma once
#include <iostream>
namespace DB::S3
{
/**
* Wrapper of IOStream to store response stream and corresponding HTTP session.
*/
template <typename Session>
class SessionAwareIOStream : public std::iostream
{
public:
SessionAwareIOStream(Session session_, std::streambuf * sb)
: std::iostream(sb)
, session(std::move(session_))
{
}
private:
/// Poco HTTP session is holder of response stream.
Session session;
};
}

View File

@ -144,7 +144,7 @@ public:
}
else if (Aws::Utils::StringUtils::ToLower(ec2_metadata_disabled.c_str()) != "true")
{
Aws::Client::ClientConfiguration aws_client_configuration;
DB::S3::PocoHTTPClientConfiguration aws_client_configuration = DB::S3::ClientFactory::instance().createClientConfiguration(remote_host_filter, s3_max_redirects);
/// See MakeDefaultHttpResourceClientConfiguration().
/// This is part of EC2 metadata client, but unfortunately it can't be accessed from outside
@ -165,8 +165,7 @@ public:
aws_client_configuration.requestTimeoutMs = 1000;
aws_client_configuration.retryStrategy = std::make_shared<Aws::Client::DefaultRetryStrategy>(1, 1000);
DB::S3::PocoHTTPClientConfiguration client_configuration(aws_client_configuration, remote_host_filter, s3_max_redirects);
auto ec2_metadata_client = std::make_shared<Aws::Internal::EC2MetadataClient>(client_configuration);
auto ec2_metadata_client = std::make_shared<Aws::Internal::EC2MetadataClient>(aws_client_configuration);
auto config_loader = std::make_shared<Aws::Config::EC2InstanceProfileConfigLoader>(ec2_metadata_client);
AddProvider(std::make_shared<Aws::Auth::InstanceProfileCredentialsProvider>(config_loader));
@ -207,13 +206,32 @@ public:
return result;
}
bool SignRequest(Aws::Http::HttpRequest & request, const char * region, const char * service_name, bool sign_body) const override
{
auto result = Aws::Client::AWSAuthV4Signer::SignRequest(request, region, service_name, sign_body);
for (const auto & header : headers)
request.SetHeaderValue(header.name, header.value);
return result;
}
bool PresignRequest(
Aws::Http::HttpRequest & request,
const char * region,
const char * serviceName,
long long expiration_time_sec) const override // NOLINT
{
auto result = Aws::Client::AWSAuthV4Signer::PresignRequest(request, region, serviceName, expiration_time_sec);
auto result = Aws::Client::AWSAuthV4Signer::PresignRequest(request, region, expiration_time_sec);
for (const auto & header : headers)
request.SetHeaderValue(header.name, header.value);
return result;
}
bool PresignRequest(
Aws::Http::HttpRequest & request,
const char * region,
const char * service_name,
long long expiration_time_sec) const override // NOLINT
{
auto result = Aws::Client::AWSAuthV4Signer::PresignRequest(request, region, service_name, expiration_time_sec);
for (const auto & header : headers)
request.SetHeaderValue(header.name, header.value);
return result;
@ -265,33 +283,28 @@ namespace S3
const RemoteHostFilter & remote_host_filter,
unsigned int s3_max_redirects)
{
Aws::Client::ClientConfiguration cfg;
PocoHTTPClientConfiguration client_configuration(remote_host_filter, s3_max_redirects);
if (!endpoint.empty())
cfg.endpointOverride = endpoint;
client_configuration.endpointOverride = endpoint;
return create(cfg,
return create(client_configuration,
is_virtual_hosted_style,
access_key_id,
secret_access_key,
use_environment_credentials,
remote_host_filter,
s3_max_redirects);
use_environment_credentials);
}
std::shared_ptr<Aws::S3::S3Client> ClientFactory::create( // NOLINT
const Aws::Client::ClientConfiguration & cfg,
const PocoHTTPClientConfiguration & cfg_,
bool is_virtual_hosted_style,
const String & access_key_id,
const String & secret_access_key,
bool use_environment_credentials,
const RemoteHostFilter & remote_host_filter,
unsigned int s3_max_redirects)
bool use_environment_credentials)
{
Aws::Auth::AWSCredentials credentials(access_key_id, secret_access_key);
PocoHTTPClientConfiguration client_configuration(cfg, remote_host_filter, s3_max_redirects);
PocoHTTPClientConfiguration client_configuration = cfg_;
client_configuration.updateSchemeAndRegion();
return std::make_shared<Aws::S3::S3Client>(
@ -301,22 +314,19 @@ namespace S3
use_environment_credentials), // AWS credentials provider.
std::move(client_configuration), // Client configuration.
Aws::Client::AWSAuthV4Signer::PayloadSigningPolicy::Never, // Sign policy.
is_virtual_hosted_style || cfg.endpointOverride.empty() // Use virtual addressing if endpoint is not specified.
is_virtual_hosted_style || client_configuration.endpointOverride.empty() // Use virtual addressing if endpoint is not specified.
);
}
std::shared_ptr<Aws::S3::S3Client> ClientFactory::create( // NOLINT
const Aws::Client::ClientConfiguration & cfg,
const PocoHTTPClientConfiguration & cfg_,
bool is_virtual_hosted_style,
const String & access_key_id,
const String & secret_access_key,
HeaderCollection headers,
bool use_environment_credentials,
const RemoteHostFilter & remote_host_filter,
unsigned int s3_max_redirects)
bool use_environment_credentials)
{
PocoHTTPClientConfiguration client_configuration(cfg, remote_host_filter, s3_max_redirects);
PocoHTTPClientConfiguration client_configuration = cfg_;
client_configuration.updateSchemeAndRegion();
Aws::Auth::AWSCredentials credentials(access_key_id, secret_access_key);
@ -329,6 +339,13 @@ namespace S3
);
}
PocoHTTPClientConfiguration ClientFactory::createClientConfiguration( // NOLINT
const RemoteHostFilter & remote_host_filter,
unsigned int s3_max_redirects)
{
return PocoHTTPClientConfiguration(remote_host_filter, s3_max_redirects);
}
URI::URI(const Poco::URI & uri_)
{
/// Case when bucket name represented in domain name of S3 URL.

View File

@ -7,6 +7,7 @@
#include <common/types.h>
#include <aws/core/Aws.h>
#include <aws/core/client/ClientConfiguration.h>
#include <IO/S3/PocoHTTPClient.h>
#include <Poco/URI.h>
namespace Aws::S3
@ -23,7 +24,6 @@ namespace DB
namespace DB::S3
{
class ClientFactory
{
public:
@ -41,21 +41,21 @@ public:
unsigned int s3_max_redirects);
std::shared_ptr<Aws::S3::S3Client> create(
const Aws::Client::ClientConfiguration & cfg,
const PocoHTTPClientConfiguration & cfg,
bool is_virtual_hosted_style,
const String & access_key_id,
const String & secret_access_key,
bool use_environment_credentials,
const RemoteHostFilter & remote_host_filter,
unsigned int s3_max_redirects);
bool use_environment_credentials);
std::shared_ptr<Aws::S3::S3Client> create(
const Aws::Client::ClientConfiguration & cfg,
const PocoHTTPClientConfiguration & cfg,
bool is_virtual_hosted_style,
const String & access_key_id,
const String & secret_access_key,
HeaderCollection headers,
bool use_environment_credentials,
bool use_environment_credentials);
PocoHTTPClientConfiguration createClientConfiguration(
const RemoteHostFilter & remote_host_filter,
unsigned int s3_max_redirects);

View File

@ -654,6 +654,11 @@ bool tryParseDateTimeBestEffort(time_t & res, ReadBuffer & in, const DateLUTImpl
return parseDateTimeBestEffortImpl<bool, false>(res, in, local_time_zone, utc_time_zone, nullptr);
}
bool tryParseDateTimeBestEffortUS(time_t & res, ReadBuffer & in, const DateLUTImpl & local_time_zone, const DateLUTImpl & utc_time_zone)
{
return parseDateTimeBestEffortImpl<bool, true>(res, in, local_time_zone, utc_time_zone, nullptr);
}
void parseDateTime64BestEffort(DateTime64 & res, UInt32 scale, ReadBuffer & in, const DateLUTImpl & local_time_zone, const DateLUTImpl & utc_time_zone)
{
return parseDateTime64BestEffortImpl<void>(res, scale, in, local_time_zone, utc_time_zone);

View File

@ -57,8 +57,9 @@ class ReadBuffer;
*/
void parseDateTimeBestEffort(time_t & res, ReadBuffer & in, const DateLUTImpl & local_time_zone, const DateLUTImpl & utc_time_zone);
void parseDateTimeBestEffortUS(time_t & res, ReadBuffer & in, const DateLUTImpl & local_time_zone, const DateLUTImpl & utc_time_zone);
bool tryParseDateTimeBestEffort(time_t & res, ReadBuffer & in, const DateLUTImpl & local_time_zone, const DateLUTImpl & utc_time_zone);
void parseDateTimeBestEffortUS(time_t & res, ReadBuffer & in, const DateLUTImpl & local_time_zone, const DateLUTImpl & utc_time_zone);
bool tryParseDateTimeBestEffortUS(time_t & res, ReadBuffer & in, const DateLUTImpl & local_time_zone, const DateLUTImpl & utc_time_zone);
void parseDateTime64BestEffort(DateTime64 & res, UInt32 scale, ReadBuffer & in, const DateLUTImpl & local_time_zone, const DateLUTImpl & utc_time_zone);
bool tryParseDateTime64BestEffort(DateTime64 & res, UInt32 scale, ReadBuffer & in, const DateLUTImpl & local_time_zone, const DateLUTImpl & utc_time_zone);

View File

@ -821,6 +821,13 @@ ActionsDAGPtr ActionsDAG::merge(ActionsDAG && first, ActionsDAG && second)
first.nodes.splice(first.nodes.end(), std::move(second.nodes));
/// Here we rebuild index because some string_view from the first map now may point to string from second.
ActionsDAG::Index first_index;
for (auto * node : first.index)
first_index.insert(node);
first.index.swap(first_index);
#if USE_EMBEDDED_COMPILER
if (first.compilation_cache == nullptr)
first.compilation_cache = second.compilation_cache;

View File

@ -688,7 +688,7 @@ public:
if constexpr (has_defaults)
applyLazyDefaults();
for (size_t j = 0; j < right_indexes.size(); ++j)
for (size_t j = 0, size = right_indexes.size(); j < size; ++j)
columns[j]->insertFrom(*block.getByPosition(right_indexes[j]).column, row_num);
}
@ -701,7 +701,7 @@ public:
{
if (lazy_defaults_count)
{
for (size_t j = 0; j < right_indexes.size(); ++j)
for (size_t j = 0, size = right_indexes.size(); j < size; ++j)
JoinCommon::addDefaultValues(*columns[j], type_name[j].first, lazy_defaults_count);
lazy_defaults_count = 0;
}

View File

@ -693,12 +693,18 @@ void TreeRewriterResult::collectUsedColumns(const ASTPtr & query, bool is_select
if (storage)
{
ss << ", maybe you meant: ";
String hint_name{};
for (const auto & name : columns_context.requiredColumns())
{
auto hints = storage->getHints(name);
if (!hints.empty())
ss << " '" << toString(hints) << "'";
hint_name = hint_name + " '" + toString(hints) + "'";
}
if (!hint_name.empty())
{
ss << ", maybe you meant: ";
ss << hint_name;
}
}
else

View File

@ -14,21 +14,29 @@ namespace
{
/// Wrapper to allow mixed lists of nested and normal types.
class ParserNestedTableOrExpression : public IParserBase
/// Parameters are either:
/// - Nested table elements;
/// - Enum element in form of 'a' = 1;
/// - literal;
/// - another data type (or identifier)
class ParserDataTypeArgument : public IParserBase
{
private:
const char * getName() const override { return "data type or expression"; }
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override
{
ParserNestedTable parser1;
private:
const char * getName() const override { return "data type argument"; }
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override
{
ParserNestedTable nested_parser;
ParserDataType data_type_parser;
ParserLiteral literal_parser;
if (parser1.parse(pos, node, expected))
return true;
const char * operators[] = {"=", "equals", nullptr};
ParserLeftAssociativeBinaryOperatorList enum_parser(operators, std::make_unique<ParserLiteral>());
ParserExpression parser2;
return parser2.parse(pos, node, expected);
}
return nested_parser.parse(pos, node, expected)
|| enum_parser.parse(pos, node, expected)
|| literal_parser.parse(pos, node, expected)
|| data_type_parser.parse(pos, node, expected);
}
};
}
@ -104,7 +112,7 @@ bool ParserDataType::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
++pos;
/// Parse optional parameters
ParserList args_parser(std::make_unique<ParserNestedTableOrExpression>(), std::make_unique<ParserToken>(TokenType::Comma));
ParserList args_parser(std::make_unique<ParserDataTypeArgument>(), std::make_unique<ParserToken>(TokenType::Comma));
ASTPtr expr_list_args;
if (!args_parser.parse(pos, expr_list_args, expected))

View File

@ -116,12 +116,6 @@ public:
: Parent(CurrentMetrics::Merge)
{}
void onEntryCreate(const Parent::Entry & entry) override
{
if (isTTLMergeType(entry->merge_type))
++merges_with_ttl_counter;
}
void onEntryDestroy(const Parent::Entry & entry) override
{
if (isTTLMergeType(entry->merge_type))
@ -140,7 +134,20 @@ public:
}
}
size_t getExecutingMergesWithTTLCount() const
/// Merge consists of two parts: assignment and execution. We add merge to
/// merge list on execution, but checking merge list during merge
/// assignment. This lead to the logical race condition (we can assign more
/// merges with TTL than allowed). So we "book" merge with ttl during
/// assignment, and remove from list after merge execution.
///
/// NOTE: Not important for replicated merge tree, we check count of merges twice:
/// in assignment and in queue before execution.
void bookMergeWithTTL()
{
++merges_with_ttl_counter;
}
size_t getMergesWithTTLCount() const
{
return merges_with_ttl_counter;
}

View File

@ -3955,7 +3955,7 @@ NamesAndTypesList MergeTreeData::getVirtuals() const
size_t MergeTreeData::getTotalMergesWithTTLInMergeList() const
{
return global_context.getMergeList().getExecutingMergesWithTTLCount();
return global_context.getMergeList().getMergesWithTTLCount();
}
void MergeTreeData::addPartContributionToDataVolume(const DataPartPtr & part)

View File

@ -732,6 +732,10 @@ std::shared_ptr<StorageMergeTree::MergeMutateSelectedEntry> StorageMergeTree::se
return {};
}
/// Account TTL merge here to avoid exceeding the max_number_of_merges_with_ttl_in_pool limit
if (isTTLMergeType(future_part.merge_type))
global_context.getMergeList().bookMergeWithTTL();
merging_tagger = std::make_unique<CurrentlyMergingPartsTagger>(future_part, MergeTreeDataMergerMutator::estimateNeededDiskSpace(future_part.parts), *this, metadata_snapshot, false);
return std::make_shared<MergeMutateSelectedEntry>(future_part, std::move(merging_tagger), MutationCommands{});
}

View File

@ -1490,7 +1490,12 @@ bool StorageReplicatedMergeTree::tryExecuteMerge(const LogEntry & entry)
future_merged_part.updatePath(*this, reserved_space);
future_merged_part.merge_type = entry.merge_type;
/// Account TTL merge
if (isTTLMergeType(future_merged_part.merge_type))
global_context.getMergeList().bookMergeWithTTL();
auto table_id = getStorageID();
/// Add merge to list
MergeList::EntryPtr merge_entry = global_context.getMergeList().insert(table_id.database_name, table_id.table_name, future_merged_part);
Transaction transaction(*this);

View File

@ -222,7 +222,10 @@ StorageS3::StorageS3(
if (access_key_id_.empty())
credentials = Aws::Auth::AWSCredentials(std::move(settings.access_key_id), std::move(settings.secret_access_key));
Aws::Client::ClientConfiguration client_configuration;
S3::PocoHTTPClientConfiguration client_configuration = S3::ClientFactory::instance().createClientConfiguration(
context_.getRemoteHostFilter(),
context_.getGlobalContext().getSettingsRef().s3_max_redirects);
client_configuration.endpointOverride = uri_.endpoint;
client_configuration.maxConnections = max_connections_;
@ -232,9 +235,8 @@ StorageS3::StorageS3(
credentials.GetAWSAccessKeyId(),
credentials.GetAWSSecretKey(),
std::move(settings.headers),
settings.use_environment_credentials.value_or(global_context.getConfigRef().getBool("s3.use_environment_credentials", false)),
context_.getRemoteHostFilter(),
context_.getGlobalContext().getSettingsRef().s3_max_redirects);
settings.use_environment_credentials.value_or(global_context.getConfigRef().getBool("s3.use_environment_credentials", false))
);
}

View File

@ -345,9 +345,10 @@ def run_tests_array(all_tests_with_params):
SERVER_DIED = True
break
file_suffix = ('.' + str(os.getpid())) if is_concurrent and args.test_runs > 1 else ''
reference_file = os.path.join(suite_dir, name) + '.reference'
stdout_file = os.path.join(suite_tmp_dir, name) + '.stdout'
stderr_file = os.path.join(suite_tmp_dir, name) + '.stderr'
stdout_file = os.path.join(suite_tmp_dir, name) + file_suffix + '.stdout'
stderr_file = os.path.join(suite_tmp_dir, name) + file_suffix + '.stderr'
proc, stdout, stderr, total_time = run_single_test(args, ext, server_logs_level, client_options, case_file, stdout_file, stderr_file)
@ -434,6 +435,9 @@ def run_tests_array(all_tests_with_params):
if os.path.exists(stderr_file):
os.remove(stderr_file)
if status and not status.endswith('\n'):
status += '\n'
sys.stdout.write(status)
sys.stdout.flush()
except KeyboardInterrupt as e:
@ -721,6 +725,7 @@ def main(args):
all_tests = [case for case in all_tests if is_test_from_dir(suite_dir, case)]
if args.test:
all_tests = [t for t in all_tests if any([re.search(r, t) for r in args.test])]
all_tests = all_tests * args.test_runs
all_tests.sort(key=key_func)
jobs = args.jobs
@ -893,6 +898,7 @@ if __name__ == '__main__':
parser.add_argument('--database', help='Database for tests (random name test_XXXXXX by default)')
parser.add_argument('--parallel', default='1/1', help='One parallel test run number/total')
parser.add_argument('-j', '--jobs', default=1, nargs='?', type=int, help='Run all tests in parallel')
parser.add_argument('--test-runs', default=1, nargs='?', type=int, help='Run each test many times (useful for e.g. flaky check)')
parser.add_argument('-U', '--unified', default=3, type=int, help='output NUM lines of unified context')
parser.add_argument('-r', '--server-check-retries', default=30, type=int, help='Num of tries to execute SELECT 1 before tests started')
parser.add_argument('--skip-list-path', help="Path to skip-list file")

View File

@ -38,6 +38,9 @@ class TSV:
def __str__(self):
return '\n'.join(self.lines)
def __len__(self):
return len(self.lines)
@staticmethod
def toMat(contents):
return [line.split("\t") for line in contents.split("\n") if line.strip()]

View File

@ -2,7 +2,7 @@ import time
import pytest
from helpers.cluster import ClickHouseCluster
from helpers.test_tools import assert_eq_with_retry
from helpers.test_tools import assert_eq_with_retry, TSV
cluster = ClickHouseCluster(__file__)
node1 = cluster.add_instance('node1', main_configs=['configs/fast_background_pool.xml', 'configs/log_conf.xml'], with_zookeeper=True)
@ -28,12 +28,13 @@ def count_ttl_merges_in_queue(node, table):
return int(result.strip())
def count_ttl_merges_in_background_pool(node, table):
result = node.query(
"SELECT count() FROM system.merges WHERE merge_type = 'TTL_DELETE' and table = '{}'".format(table))
if not result:
return 0
return int(result.strip())
def count_ttl_merges_in_background_pool(node, table, level):
result = TSV(node.query(
"SELECT * FROM system.merges WHERE merge_type = 'TTL_DELETE' and table = '{}'".format(table)))
count = len(result)
if count >= level:
print("count_ttl_merges_in_background_pool: merges more than warn level:\n{}".format(result))
return count
def count_regular_merges_in_background_pool(node, table):
@ -67,7 +68,7 @@ def test_no_ttl_merges_in_busy_pool(started_cluster):
while count_running_mutations(node1, "test_ttl") < 6:
print("Mutations count", count_running_mutations(node1, "test_ttl"))
assert count_ttl_merges_in_background_pool(node1, "test_ttl") == 0
assert count_ttl_merges_in_background_pool(node1, "test_ttl", 1) == 0
time.sleep(0.5)
node1.query("SYSTEM START TTL MERGES")
@ -100,7 +101,7 @@ def test_limited_ttl_merges_in_empty_pool(started_cluster):
merges_with_ttl_count = set({})
while True:
merges_with_ttl_count.add(count_ttl_merges_in_background_pool(node1, "test_ttl_v2"))
merges_with_ttl_count.add(count_ttl_merges_in_background_pool(node1, "test_ttl_v2", 3))
time.sleep(0.01)
if node1.query("SELECT COUNT() FROM test_ttl_v2") == "0\n":
break
@ -124,7 +125,7 @@ def test_limited_ttl_merges_in_empty_pool_replicated(started_cluster):
merges_with_ttl_count = set({})
entries_with_ttl_count = set({})
while True:
merges_with_ttl_count.add(count_ttl_merges_in_background_pool(node1, "replicated_ttl"))
merges_with_ttl_count.add(count_ttl_merges_in_background_pool(node1, "replicated_ttl", 3))
entries_with_ttl_count.add(count_ttl_merges_in_queue(node1, "replicated_ttl"))
time.sleep(0.01)
if node1.query("SELECT COUNT() FROM replicated_ttl") == "0\n":
@ -159,8 +160,8 @@ def test_limited_ttl_merges_two_replicas(started_cluster):
merges_with_ttl_count_node1 = set({})
merges_with_ttl_count_node2 = set({})
while True:
merges_with_ttl_count_node1.add(count_ttl_merges_in_background_pool(node1, "replicated_ttl_2"))
merges_with_ttl_count_node2.add(count_ttl_merges_in_background_pool(node2, "replicated_ttl_2"))
merges_with_ttl_count_node1.add(count_ttl_merges_in_background_pool(node1, "replicated_ttl_2", 3))
merges_with_ttl_count_node2.add(count_ttl_merges_in_background_pool(node2, "replicated_ttl_2", 3))
if node1.query("SELECT COUNT() FROM replicated_ttl_2") == "0\n" and node2.query(
"SELECT COUNT() FROM replicated_ttl_2") == "0\n":
break

View File

@ -1 +1 @@
SELECT * FROM system.numbers_mt WHERE concat(materialize('1'), '...', toString(number)) LIKE '%10000000%' LIMIT 1
SELECT extract(toString(number), '10000000') FROM system.numbers_mt WHERE concat(materialize('1'), '...', toString(number)) LIKE '%10000000%' LIMIT 1

View File

@ -48,7 +48,7 @@ SELECT
threads_realtime >= threads_time_user_system_io,
any(length(thread_ids)) >= 1
FROM
(SELECT * FROM system.query_log PREWHERE query='$heavy_cpu_query' WHERE event_date >= today()-1 AND type=2 ORDER BY event_time DESC LIMIT 1)
(SELECT * FROM system.query_log PREWHERE query='$heavy_cpu_query' WHERE event_date >= today()-1 AND current_database = currentDatabase() AND type=2 ORDER BY event_time DESC LIMIT 1)
ARRAY JOIN ProfileEvents.Names AS PN, ProfileEvents.Values AS PV"
# Clean

View File

@ -28,6 +28,6 @@ $CLICKHOUSE_CLIENT $settings -q "$touching_many_parts_query" &> /dev/null
$CLICKHOUSE_CLIENT $settings -q "SYSTEM FLUSH LOGS"
$CLICKHOUSE_CLIENT $settings -q "SELECT pi.Values FROM system.query_log ARRAY JOIN ProfileEvents as pi WHERE query='$touching_many_parts_query' and pi.Names = 'FileOpen' ORDER BY event_time DESC LIMIT 1;"
$CLICKHOUSE_CLIENT $settings -q "SELECT pi.Values FROM system.query_log ARRAY JOIN ProfileEvents as pi WHERE query='$touching_many_parts_query' and current_database = currentDatabase() and pi.Names = 'FileOpen' ORDER BY event_time DESC LIMIT 1;"
$CLICKHOUSE_CLIENT $settings -q "DROP TABLE IF EXISTS merge_tree_table;"

View File

@ -20,7 +20,7 @@ $CLICKHOUSE_CLIENT --use_uncompressed_cache=1 --query_id="test-query-uncompresse
$CLICKHOUSE_CLIENT --query="SYSTEM FLUSH LOGS"
$CLICKHOUSE_CLIENT --query="SELECT ProfileEvents.Values[indexOf(ProfileEvents.Names, 'Seek')], ProfileEvents.Values[indexOf(ProfileEvents.Names, 'ReadCompressedBytes')], ProfileEvents.Values[indexOf(ProfileEvents.Names, 'UncompressedCacheHits')] AS hit FROM system.query_log WHERE (query_id = 'test-query-uncompressed-cache') AND (type = 2) AND event_date >= yesterday() ORDER BY event_time DESC LIMIT 1"
$CLICKHOUSE_CLIENT --query="SELECT ProfileEvents.Values[indexOf(ProfileEvents.Names, 'Seek')], ProfileEvents.Values[indexOf(ProfileEvents.Names, 'ReadCompressedBytes')], ProfileEvents.Values[indexOf(ProfileEvents.Names, 'UncompressedCacheHits')] AS hit FROM system.query_log WHERE (query_id = 'test-query-uncompressed-cache') AND current_database = currentDatabase() AND (type = 2) AND event_date >= yesterday() ORDER BY event_time DESC LIMIT 1"
$CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS small_table"

View File

@ -163,23 +163,23 @@ DROP TABLE IF EXISTS bloom_filter_lc_null_types_test;
DROP TABLE IF EXISTS bloom_filter_array_lc_null_types_test;
CREATE TABLE bloom_filter_array_lc_null_types_test (
order_key Array(LowCardinality(Nullable((UInt64)))),
order_key Array(LowCardinality(Nullable(UInt64))),
i8 Array(LowCardinality(Nullable((Int8)))),
i16 Array(LowCardinality(Nullable((Int16)))),
i32 Array(LowCardinality(Nullable((Int32)))),
i64 Array(LowCardinality(Nullable((Int64)))),
u8 Array(LowCardinality(Nullable((UInt8)))),
u16 Array(LowCardinality(Nullable((UInt16)))),
u32 Array(LowCardinality(Nullable((UInt32)))),
u64 Array(LowCardinality(Nullable((UInt64)))),
f32 Array(LowCardinality(Nullable((Float32)))),
f64 Array(LowCardinality(Nullable((Float64)))),
i8 Array(LowCardinality(Nullable(Int8))),
i16 Array(LowCardinality(Nullable(Int16))),
i32 Array(LowCardinality(Nullable(Int32))),
i64 Array(LowCardinality(Nullable(Int64))),
u8 Array(LowCardinality(Nullable(UInt8))),
u16 Array(LowCardinality(Nullable(UInt16))),
u32 Array(LowCardinality(Nullable(UInt32))),
u64 Array(LowCardinality(Nullable(UInt64))),
f32 Array(LowCardinality(Nullable(Float32))),
f64 Array(LowCardinality(Nullable(Float64))),
date Array(LowCardinality(Nullable((Date)))),
date Array(LowCardinality(Nullable(Date))),
date_time Array(LowCardinality(Nullable(DateTime('Europe/Moscow')))),
str Array(LowCardinality(Nullable((String)))),
str Array(LowCardinality(Nullable(String))),
fixed_string Array(LowCardinality(Nullable(FixedString(5)))),
INDEX idx (i8, i16, i32, i64, u8, u16, u32, u64, f32, f64, date, date_time, str, fixed_string)
TYPE bloom_filter GRANULARITY 1)
@ -286,7 +286,7 @@ SELECT COUNT() FROM bloom_filter_array_lc_null_types_test WHERE has(fixed_string
DROP TABLE IF EXISTS bloom_filter_array_lc_null_types_test;
DROP TABLE IF EXISTS bloom_filter_array_offsets_lc_str;
CREATE TABLE bloom_filter_array_offsets_lc_str (order_key int, str Array(LowCardinality((String))), INDEX idx str TYPE bloom_filter(1.) GRANULARITY 1024) ENGINE = MergeTree() ORDER BY order_key SETTINGS index_granularity = 1024;
CREATE TABLE bloom_filter_array_offsets_lc_str (order_key int, str Array(LowCardinality(String)), INDEX idx str TYPE bloom_filter(1.) GRANULARITY 1024) ENGINE = MergeTree() ORDER BY order_key SETTINGS index_granularity = 1024;
INSERT INTO bloom_filter_array_offsets_lc_str SELECT number AS i, if(i%2, ['value'], []) FROM system.numbers LIMIT 10000;
SELECT count() FROM bloom_filter_array_offsets_lc_str WHERE has(str, 'value');
DROP TABLE IF EXISTS bloom_filter_array_offsets_lc_str;

View File

@ -97,7 +97,7 @@ echo 7
# and finally querylog
$CLICKHOUSE_CLIENT \
--server_logs_file=/dev/null \
--query="select * from system.query_log where event_time > now() - 10 and query like '%TOPSECRET%';"
--query="select * from system.query_log where current_database = currentDatabase() AND event_time > now() - 10 and query like '%TOPSECRET%';"
rm -f "$tmp_file" >/dev/null 2>&1

View File

@ -7,7 +7,7 @@ SET log_queries = 0;
SYSTEM FLUSH LOGS;
WITH addressToLine(arrayJoin(trace) AS addr) || '#' || demangle(addressToSymbol(addr)) AS symbol
SELECT count() > 0 FROM system.trace_log t WHERE query_id = (SELECT query_id FROM system.query_log WHERE query LIKE '%test real time query profiler%' AND query NOT LIKE '%system%' ORDER BY event_time DESC LIMIT 1) AND symbol LIKE '%FunctionSleep%';
SELECT count() > 0 FROM system.trace_log t WHERE query_id = (SELECT query_id FROM system.query_log WHERE current_database = currentDatabase() AND query LIKE '%test real time query profiler%' AND query NOT LIKE '%system%' ORDER BY event_time DESC LIMIT 1) AND symbol LIKE '%FunctionSleep%';
SET query_profiler_real_time_period_ns = 0;
SET query_profiler_cpu_time_period_ns = 1000000;
@ -17,4 +17,4 @@ SET log_queries = 0;
SYSTEM FLUSH LOGS;
WITH addressToLine(arrayJoin(trace) AS addr) || '#' || demangle(addressToSymbol(addr)) AS symbol
SELECT count() > 0 FROM system.trace_log t WHERE query_id = (SELECT query_id FROM system.query_log WHERE query LIKE '%test cpu time query profiler%' AND query NOT LIKE '%system%' ORDER BY event_time DESC LIMIT 1) AND symbol LIKE '%Source%';
SELECT count() > 0 FROM system.trace_log t WHERE query_id = (SELECT query_id FROM system.query_log WHERE current_database = currentDatabase() AND query LIKE '%test cpu time query profiler%' AND query NOT LIKE '%system%' ORDER BY event_time DESC LIMIT 1) AND symbol LIKE '%Source%';

View File

@ -3,5 +3,5 @@ SELECT * FROM test_table_for_01070_exception_code_in_query_log_table; -- { serve
CREATE TABLE test_table_for_01070_exception_code_in_query_log_table (value UInt64) ENGINE=Memory();
SELECT * FROM test_table_for_01070_exception_code_in_query_log_table;
SYSTEM FLUSH LOGS;
SELECT exception_code FROM system.query_log WHERE lower(query) LIKE lower('SELECT * FROM test_table_for_01070_exception_code_in_query_log_table%') AND event_date >= yesterday() AND event_time > now() - INTERVAL 5 MINUTE ORDER BY exception_code;
SELECT exception_code FROM system.query_log WHERE current_database = currentDatabase() AND lower(query) LIKE lower('SELECT * FROM test_table_for_01070_exception_code_in_query_log_table%') AND event_date >= yesterday() AND event_time > now() - INTERVAL 5 MINUTE ORDER BY exception_code;
DROP TABLE IF EXISTS test_table_for_01070_exception_code_in_query_log_table;

View File

@ -5,6 +5,6 @@ SYSTEM FLUSH LOGS;
SELECT arrayJoin AS kv_key
FROM system.query_log
ARRAY JOIN ProfileEvents.Names AS arrayJoin
PREWHERE has(arrayMap(key -> key, ProfileEvents.Names), 'Query')
PREWHERE current_database = currentDatabase() AND has(arrayMap(key -> key, ProfileEvents.Names), 'Query')
WHERE arrayJoin = 'Query'
LIMIT 0;

View File

@ -7,4 +7,4 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
QUERY_ID=$RANDOM
$CLICKHOUSE_BENCHMARK <<< "SELECT 1" --query_id $QUERY_ID -i 10 2>/dev/null
$CLICKHOUSE_CLIENT -q "SYSTEM FLUSH LOGS"
$CLICKHOUSE_CLIENT -q "SELECT count() FROM system.query_log WHERE query_id='$QUERY_ID'"
$CLICKHOUSE_CLIENT -q "SELECT count() FROM system.query_log WHERE current_database = currentDatabase() AND query_id='$QUERY_ID'"

View File

@ -8,7 +8,7 @@ WITH
(
SELECT query_id
FROM system.query_log
WHERE (normalizeQuery(query) like normalizeQuery('WITH 01091 AS id SELECT 1;')) AND (event_date >= (today() - 1))
WHERE current_database = currentDatabase() AND (normalizeQuery(query) like normalizeQuery('WITH 01091 AS id SELECT 1;')) AND (event_date >= (today() - 1))
ORDER BY event_time DESC
LIMIT 1
) AS id
@ -23,7 +23,7 @@ WITH
(
SELECT query_id
FROM system.query_log
WHERE (normalizeQuery(query) = normalizeQuery('with 01091 as id select sum(number) from numbers(1000000);')) AND (event_date >= (today() - 1))
WHERE current_database = currentDatabase() AND (normalizeQuery(query) = normalizeQuery('with 01091 as id select sum(number) from numbers(1000000);')) AND (event_date >= (today() - 1))
ORDER BY event_time DESC
LIMIT 1
) AS id
@ -38,7 +38,7 @@ WITH
(
SELECT query_id
FROM system.query_log
WHERE (normalizeQuery(query) = normalizeQuery('with 01091 as id select sum(number) from numbers_mt(1000000);')) AND (event_date >= (today() - 1))
WHERE current_database = currentDatabase() AND (normalizeQuery(query) = normalizeQuery('with 01091 as id select sum(number) from numbers_mt(1000000);')) AND (event_date >= (today() - 1))
ORDER BY event_time DESC
LIMIT 1
) AS id

View File

@ -3,4 +3,4 @@ SET allow_introspection_functions = 1;
SET memory_profiler_step = 1000000;
SELECT ignore(groupArray(number), 'test memory profiler') FROM numbers(10000000);
SYSTEM FLUSH LOGS;
WITH addressToSymbol(arrayJoin(trace)) AS symbol SELECT count() > 0 FROM system.trace_log t WHERE event_date >= yesterday() AND trace_type = 'Memory' AND query_id = (SELECT query_id FROM system.query_log WHERE event_date >= yesterday() AND query LIKE '%test memory profiler%' ORDER BY event_time DESC LIMIT 1);
WITH addressToSymbol(arrayJoin(trace)) AS symbol SELECT count() > 0 FROM system.trace_log t WHERE event_date >= yesterday() AND trace_type = 'Memory' AND query_id = (SELECT query_id FROM system.query_log WHERE current_database = currentDatabase() AND event_date >= yesterday() AND query LIKE '%test memory profiler%' ORDER BY event_time DESC LIMIT 1);

View File

@ -93,11 +93,11 @@ not optimized (smoke)
-
0 0 0 0
1 1 1 1
3 3 3 3
2 2 2 2
3 3 3 3
-
0 0 0 0
1 1 1 1
-
3 3 3 3
2 2 2 2
3 3 3 3

View File

@ -74,13 +74,13 @@ SELECT * FROM (SELECT number AS key FROM numbers(2)) s1 RIGHT JOIN dict_flat d O
SELECT '-';
SELECT * FROM (SELECT number + 2 AS key FROM numbers(4)) s1 FULL JOIN dict_flat d USING(key) ORDER BY s1.key, d.key;
SELECT '-';
SELECT * FROM (SELECT number AS key FROM numbers(2)) s1 ANY INNER JOIN dict_flat d USING(key);
SELECT * FROM (SELECT number AS key FROM numbers(2)) s1 ANY INNER JOIN dict_flat d USING(key) ORDER BY s1.key;
SELECT '-';
SELECT * FROM (SELECT number AS key FROM numbers(2)) s1 ANY RIGHT JOIN dict_flat d USING(key);
SELECT * FROM (SELECT number AS key FROM numbers(2)) s1 ANY RIGHT JOIN dict_flat d USING(key) ORDER BY s1.key;
SELECT '-';
SELECT * FROM (SELECT number AS key FROM numbers(2)) s1 SEMI RIGHT JOIN dict_flat d USING(key);
SELECT * FROM (SELECT number AS key FROM numbers(2)) s1 SEMI RIGHT JOIN dict_flat d USING(key) ORDER BY s1.key;
SELECT '-';
SELECT * FROM (SELECT number AS key FROM numbers(2)) s1 ANTI RIGHT JOIN dict_flat d USING(key);
SELECT * FROM (SELECT number AS key FROM numbers(2)) s1 ANTI RIGHT JOIN dict_flat d USING(key) ORDER BY s1.key;
DROP DICTIONARY dict_flat;
DROP DICTIONARY dict_hashed;

View File

@ -46,7 +46,7 @@ for i in {1..5}; do
done
$CLICKHOUSE_CLIENT -q "SYSTEM FLUSH LOGS"
$CLICKHOUSE_CLIENT -q "SELECT if(quantile(0.5)(query_duration_ms) < $max_time_ms, 'ok', toString(groupArray(query_duration_ms))) FROM system.query_log WHERE query_id LIKE '$db-%' AND type=2"
$CLICKHOUSE_CLIENT -q "SELECT if(quantile(0.5)(query_duration_ms) < $max_time_ms, 'ok', toString(groupArray(query_duration_ms))) FROM system.query_log WHERE current_database = currentDatabase() AND query_id LIKE '$db-%' AND type=2"
$CLICKHOUSE_CLIENT -q "SELECT count() * $count_multiplier, i, d, s, n.i, n.f FROM $db.table_merge GROUP BY i, d, s, n.i, n.f ORDER BY i"

View File

@ -14,4 +14,4 @@ ${CLICKHOUSE_CURL} -sS "$url" --data "SELECT 'test_01194',$rnd,4" > /dev/null
${CLICKHOUSE_CURL} -sS "$url" --data "SYSTEM FLUSH LOGS"
${CLICKHOUSE_CURL} -sS "$url&query=SELECT+count(DISTINCT+query_id)+FROM+system.query_log+WHERE+query+LIKE+'SELECT+''test_01194'',$rnd%25'"
${CLICKHOUSE_CURL} -sS "$url&query=SELECT+count(DISTINCT+query_id)+FROM+system.query_log+WHERE+current_database+LIKE+currentDatabase()+AND+query+LIKE+'SELECT+''test_01194'',$rnd%25'"

View File

@ -4,4 +4,4 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
# shellcheck source=../shell_config.sh
. "$CURDIR"/../shell_config.sh
$CLICKHOUSE_CLIENT --quota_key Hello --query_id test_quota_key --log_queries 1 --multiquery --query "SELECT 1; SYSTEM FLUSH LOGS; SELECT DISTINCT quota_key FROM system.query_log WHERE event_date >= yesterday() AND event_time >= now() - 300 AND query_id = 'test_quota_key'"
$CLICKHOUSE_CLIENT --quota_key Hello --query_id test_quota_key --log_queries 1 --multiquery --query "SELECT 1; SYSTEM FLUSH LOGS; SELECT DISTINCT quota_key FROM system.query_log WHERE current_database = currentDatabase() AND event_date >= yesterday() AND event_time >= now() - 300 AND query_id = 'test_quota_key'"

View File

@ -2,22 +2,23 @@ set log_queries=1;
select '01231_log_queries_min_type/QUERY_START';
system flush logs;
select count() from system.query_log where query like '%01231_log_queries_min_type/QUERY_START%' and query not like '%system.query_log%' and event_date = today() and event_time >= now() - interval 1 minute;
select count() from system.query_log where current_database = currentDatabase() and query like '%01231_log_queries_min_type/QUERY_START%' and query not like '%system.query_log%' and event_date = today() and event_time >= now() - interval 1 minute;
set log_queries_min_type='EXCEPTION_BEFORE_START';
select '01231_log_queries_min_type/EXCEPTION_BEFORE_START';
system flush logs;
select count() from system.query_log where query like '%01231_log_queries_min_type/EXCEPTION_BEFORE_START%' and query not like '%system.query_log%' and event_date = today() and event_time >= now() - interval 1 minute;
select count() from system.query_log where current_database = currentDatabase() and query like '%01231_log_queries_min_type/EXCEPTION_BEFORE_START%' and query not like '%system.query_log%' and event_date = today() and event_time >= now() - interval 1 minute;
set max_rows_to_read='100K';
set log_queries_min_type='EXCEPTION_WHILE_PROCESSING';
select '01231_log_queries_min_type/EXCEPTION_WHILE_PROCESSING', max(number) from system.numbers limit 1e6; -- { serverError 158; }
system flush logs;
select count() from system.query_log where query like '%01231_log_queries_min_type/EXCEPTION_WHILE_PROCESSING%' and query not like '%system.query_log%' and event_date = today() and event_time >= now() - interval 1 minute and type = 'ExceptionWhileProcessing';
select count() from system.query_log where current_database = currentDatabase() and query like '%01231_log_queries_min_type/EXCEPTION_WHILE_PROCESSING%' and query not like '%system.query_log%' and event_date = today() and event_time >= now() - interval 1 minute and type = 'ExceptionWhileProcessing';
select '01231_log_queries_min_type w/ Settings/EXCEPTION_WHILE_PROCESSING', max(number) from system.numbers limit 1e6; -- { serverError 158; }
system flush logs;
select count() from system.query_log where
current_database = currentDatabase() and
query like '%01231_log_queries_min_type w/ Settings/EXCEPTION_WHILE_PROCESSING%' and
query not like '%system.query_log%' and
event_date = today() and

View File

@ -16,7 +16,7 @@ SELECT
throwIf(count() != 1, 'no query was logged'),
throwIf(length(thread_ids) != 2, 'too many threads used')
FROM system.query_log
WHERE type = 'QueryFinish' AND query LIKE '%data_01283 LIMIT 1%'
WHERE current_database = currentDatabase() AND type = 'QueryFinish' AND query LIKE '%data_01283 LIMIT 1%'
GROUP BY thread_ids
FORMAT Null;

View File

@ -17,6 +17,7 @@ SYSTEM FLUSH LOGS;
SELECT DISTINCT query_duration_ms >= 500
FROM system.query_log
WHERE
current_database = currentDatabase() AND
event_date >= yesterday() AND
query LIKE '%special query for 01290_max_execution_speed_distributed%' AND
query NOT LIKE '%system.query_log%' AND

View File

@ -9,11 +9,11 @@ set log_queries = 1;
select x from table_01323_many_parts limit 10 format Null;
system flush logs;
select arrayUniq(thread_ids) <= 4 from system.query_log where event_date >= today() - 1 and query ilike '%select x from table_01323_many_parts%' and query not like '%system.query_log%' and type = 'QueryFinish' order by query_start_time desc limit 1;
select arrayUniq(thread_ids) <= 4 from system.query_log where current_database = currentDatabase() AND event_date >= today() - 1 and query ilike '%select x from table_01323_many_parts%' and query not like '%system.query_log%' and type = 'QueryFinish' order by query_start_time desc limit 1;
select x from table_01323_many_parts order by x limit 10 format Null;
system flush logs;
select arrayUniq(thread_ids) <= 36 from system.query_log where event_date >= today() - 1 and query ilike '%select x from table_01323_many_parts order by x%' and query not like '%system.query_log%' and type = 'QueryFinish' order by query_start_time desc limit 1;
select arrayUniq(thread_ids) <= 36 from system.query_log where current_database = currentDatabase() AND event_date >= today() - 1 and query ilike '%select x from table_01323_many_parts order by x%' and query not like '%system.query_log%' and type = 'QueryFinish' order by query_start_time desc limit 1;
drop table if exists table_01323_many_parts;

View File

@ -6,6 +6,6 @@ SET min_bytes_to_use_mmap_io = 1;
SELECT * FROM test_01343;
SYSTEM FLUSH LOGS;
SELECT PE.Values FROM system.query_log ARRAY JOIN ProfileEvents AS PE WHERE event_date >= yesterday() AND event_time >= now() - 300 AND query LIKE 'SELECT * FROM test_01343%' AND PE.Names = 'CreatedReadBufferMMap' AND type = 2 ORDER BY event_time DESC LIMIT 1;
SELECT PE.Values FROM system.query_log ARRAY JOIN ProfileEvents AS PE WHERE current_database = currentDatabase() AND event_date >= yesterday() AND event_time >= now() - 300 AND query LIKE 'SELECT * FROM test_01343%' AND PE.Names = 'CreatedReadBufferMMap' AND type = 2 ORDER BY event_time DESC LIMIT 1;
DROP TABLE test_01343;

View File

@ -6,6 +6,6 @@ SET min_bytes_to_use_mmap_io = 1;
SELECT * FROM test_01344 WHERE x = 'Hello, world';
SYSTEM FLUSH LOGS;
SELECT PE.Values FROM system.query_log ARRAY JOIN ProfileEvents AS PE WHERE event_date >= yesterday() AND event_time >= now() - 300 AND query LIKE 'SELECT * FROM test_01344 WHERE x = ''Hello, world''%' AND PE.Names = 'CreatedReadBufferMMap' AND type = 2 ORDER BY event_time DESC LIMIT 1;
SELECT PE.Values FROM system.query_log ARRAY JOIN ProfileEvents AS PE WHERE current_database = currentDatabase() AND event_date >= yesterday() AND event_time >= now() - 300 AND query LIKE 'SELECT * FROM test_01344 WHERE x = ''Hello, world''%' AND PE.Names = 'CreatedReadBufferMMap' AND type = 2 ORDER BY event_time DESC LIMIT 1;
DROP TABLE test_01344;

View File

@ -1,3 +1,4 @@
parseDateTimeBestEffortUS
s a
1970/01/02 010203Z 1970-01-02 01:02:03
@ -16,3 +17,24 @@
12/13/2019 2019-12-13 00:00:00
13/12/2019 2019-12-13 00:00:00
03/04/2019 2019-03-04 00:00:00
parseDateTimeBestEffortUSOrZero parseDateTimeBestEffortUSOrNull
s a b
1970/01/02 010203Z 1970-01-02 01:02:03 1970-01-02 01:02:03
01-02-2001 UTC 2001-01-02 00:00:00 2001-01-02 00:00:00
10.23.1990 1990-10-23 00:00:00 1990-10-23 00:00:00
01-02-2017 03:04:05+1 2017-01-02 02:04:05 2017-01-02 02:04:05
01/02/2017 03:04:05+300 2017-01-02 00:04:05 2017-01-02 00:04:05
01.02.2017 03:04:05GMT 2017-01-02 03:04:05 2017-01-02 03:04:05
01-02-2017 03:04:05 MSD 2017-01-01 23:04:05 2017-01-01 23:04:05
01-02-2017 11:04:05 AM 2017-01-02 11:04:05 2017-01-02 11:04:05
01-02-2017 11:04:05 PM 2017-01-02 23:04:05 2017-01-02 23:04:05
01-02-2017 12:04:05 AM 2017-01-02 00:04:05 2017-01-02 00:04:05
01-02-2017 12:04:05 PM 2017-01-02 12:04:05 2017-01-02 12:04:05
01.02.17 03:04:05 MSD Feb 2017-02-01 23:04:05 2017-02-01 23:04:05
01/02/2017 03:04:05 MSK 2017-01-02 00:04:05 2017-01-02 00:04:05
12/13/2019 2019-12-13 00:00:00 2019-12-13 00:00:00
13/12/2019 2019-12-13 00:00:00 2019-12-13 00:00:00
03/04/2019 2019-03-04 00:00:00 2019-03-04 00:00:00
1970-01-01 00:00:00 ᴺᵁᴸᴸ
xyz 1970-01-01 00:00:00 ᴺᵁᴸᴸ

View File

@ -1,3 +1,5 @@
SELECT 'parseDateTimeBestEffortUS';
SELECT
s,
parseDateTimeBestEffortUS(s, 'UTC') AS a
@ -22,3 +24,32 @@ FROM
'03/04/2019'
]) AS s)
FORMAT PrettySpaceNoEscapes;
SELECT 'parseDateTimeBestEffortUSOrZero', 'parseDateTimeBestEffortUSOrNull';
SELECT
s,
parseDateTimeBestEffortUSOrZero(s, 'UTC') AS a,
parseDateTimeBestEffortUSOrNull(s, 'UTC') AS b
FROM
(
SELECT arrayJoin([
'1970/01/02 010203Z',
'01-02-2001 UTC',
'10.23.1990',
'01-02-2017 03:04:05+1',
'01/02/2017 03:04:05+300',
'01.02.2017 03:04:05GMT',
'01-02-2017 03:04:05 MSD',
'01-02-2017 11:04:05 AM',
'01-02-2017 11:04:05 PM',
'01-02-2017 12:04:05 AM',
'01-02-2017 12:04:05 PM',
'01.02.17 03:04:05 MSD Feb',
'01/02/2017 03:04:05 MSK',
'12/13/2019',
'13/12/2019',
'03/04/2019',
'',
'xyz'
]) AS s)
FORMAT PrettySpaceNoEscapes;

View File

@ -7,6 +7,6 @@ set max_threads = 16;
select g % 2 as gg, sum(s) from table_01356_view_threads group by gg order by gg;
system flush logs;
select length(thread_ids) >= 16 from system.query_log where event_date >= today() - 1 and lower(query) like '%select g % 2 as gg, sum(s) from table_01356_view_threads group by gg order by gg%' and type = 'QueryFinish' order by query_start_time desc limit 1;
select length(thread_ids) >= 16 from system.query_log where current_database = currentDatabase() AND event_date >= today() - 1 and lower(query) like '%select g % 2 as gg, sum(s) from table_01356_view_threads group by gg order by gg%' and type = 'QueryFinish' order by query_start_time desc limit 1;
drop table if exists table_01356_view_threads;

View File

@ -2,4 +2,4 @@ set log_queries = 1;
select count() > 0 from system.settings;
system flush logs;
select result_rows, result_bytes >= 8 from system.query_log where event_date >= today() - 1 and lower(query) like '%select count() > 0 from system.settings%' and type = 'QueryFinish' order by query_start_time desc limit 1;
select result_rows, result_bytes >= 8 from system.query_log where current_database = currentDatabase() AND event_date >= today() - 1 and lower(query) like '%select count() > 0 from system.settings%' and type = 'QueryFinish' order by query_start_time desc limit 1;

View File

@ -4,4 +4,4 @@ set max_threads = 16;
SELECT count() FROM (SELECT number FROM numbers_mt(1000000) ORDER BY number DESC LIMIT 100 UNION ALL SELECT number FROM numbers_mt(1000000) ORDER BY number DESC LIMIT 100 UNION ALL SELECT number FROM numbers_mt(1000000) ORDER BY number DESC LIMIT 100);
system flush logs;
select length(thread_ids) >= 16 from system.query_log where event_date >= today() - 1 and query like '%SELECT count() FROM (SELECT number FROM numbers_mt(1000000) ORDER BY number DESC LIMIT 100 UNION ALL SELECT number FROM numbers_mt(1000000) ORDER BY number DESC LIMIT 100 UNION ALL SELECT number FROM numbers_mt(1000000) ORDER BY number DESC LIMIT 100)%' and type = 'QueryFinish' order by query_start_time desc limit 1;
select length(thread_ids) >= 16 from system.query_log where current_database = currentDatabase() AND event_date >= today() - 1 and query like '%SELECT count() FROM (SELECT number FROM numbers_mt(1000000) ORDER BY number DESC LIMIT 100 UNION ALL SELECT number FROM numbers_mt(1000000) ORDER BY number DESC LIMIT 100 UNION ALL SELECT number FROM numbers_mt(1000000) ORDER BY number DESC LIMIT 100)%' and type = 'QueryFinish' order by query_start_time desc limit 1;

View File

@ -8,6 +8,7 @@ SET log_queries=1;
SELECT 1;
SYSTEM FLUSH LOGS;
-- NOTE: can be rewritten using log_queries_min_query_duration_ms
CREATE MATERIALIZED VIEW slow_log Engine=Memory AS
(
SELECT * FROM
@ -37,7 +38,7 @@ SELECT
extract(query,'/\\*\\s*QUERY_GROUP_ID:(.*?)\\s*\\*/') as QUERY_GROUP_ID,
count()
FROM system.query_log
WHERE type<>1 and event_date >= yesterday() and event_time > now() - 20 and QUERY_GROUP_ID<>''
WHERE current_database = currentDatabase() AND type<>1 and event_date >= yesterday() and event_time > now() - 20 and QUERY_GROUP_ID<>''
GROUP BY QUERY_GROUP_ID
ORDER BY QUERY_GROUP_ID;
@ -47,6 +48,7 @@ SELECT
QUERY_GROUP_ID,
count()
FROM slow_log
WHERE current_database = currentDatabase()
GROUP BY QUERY_GROUP_ID
ORDER BY QUERY_GROUP_ID;

View File

@ -4,25 +4,25 @@ CREATE TABLE rows_events_test (k UInt32, v UInt32) ENGINE = MergeTree ORDER BY k
INSERT INTO /* test 01413, query 1 */ rows_events_test VALUES (1,1);
SYSTEM FLUSH LOGS;
SELECT written_rows FROM system.query_log WHERE query LIKE 'INSERT INTO /* test 01413, query 1 */ rows_events_test%' AND type = 2 AND event_date >= yesterday() ORDER BY event_time DESC LIMIT 1;
SELECT written_rows FROM system.query_log WHERE current_database = currentDatabase() AND query LIKE 'INSERT INTO /* test 01413, query 1 */ rows_events_test%' AND type = 2 AND event_date >= yesterday() ORDER BY event_time DESC LIMIT 1;
SELECT ProfileEvents.Values as value FROM system.query_log ARRAY JOIN ProfileEvents
WHERE ProfileEvents.Names = 'InsertedRows' AND query LIKE 'INSERT INTO /* test 01413, query 1 */ rows_events_test%' AND type = 2 AND event_date >= yesterday() ORDER BY event_time DESC LIMIT 1;
WHERE current_database = currentDatabase() AND ProfileEvents.Names = 'InsertedRows' AND query LIKE 'INSERT INTO /* test 01413, query 1 */ rows_events_test%' AND type = 2 AND event_date >= yesterday() ORDER BY event_time DESC LIMIT 1;
INSERT INTO /* test 01413, query 2 */ rows_events_test VALUES (2,2), (3,3);
SYSTEM FLUSH LOGS;
SELECT written_rows FROM system.query_log WHERE query LIKE 'INSERT INTO /* test 01413, query 2 */ rows_events_test%' AND type = 2 AND event_date >= yesterday() ORDER BY event_time DESC LIMIT 1;
SELECT written_rows FROM system.query_log WHERE current_database = currentDatabase() AND query LIKE 'INSERT INTO /* test 01413, query 2 */ rows_events_test%' AND type = 2 AND event_date >= yesterday() ORDER BY event_time DESC LIMIT 1;
SELECT ProfileEvents.Values as value FROM system.query_log ARRAY JOIN ProfileEvents
WHERE ProfileEvents.Names = 'InsertedRows' AND query LIKE 'INSERT INTO /* test 01413, query 2 */ rows_events_test%' AND type = 2 AND event_date >= yesterday() ORDER BY event_time DESC LIMIT 1;
WHERE current_database = currentDatabase() AND ProfileEvents.Names = 'InsertedRows' AND query LIKE 'INSERT INTO /* test 01413, query 2 */ rows_events_test%' AND type = 2 AND event_date >= yesterday() ORDER BY event_time DESC LIMIT 1;
SELECT * FROM /* test 01413, query 3 */ rows_events_test WHERE v = 2;
SYSTEM FLUSH LOGS;
SELECT read_rows FROM system.query_log WHERE query LIKE 'SELECT * FROM /* test 01413, query 3 */ rows_events_test%' AND type = 2 AND event_date >= yesterday() ORDER BY event_time DESC LIMIT 1;
SELECT read_rows FROM system.query_log WHERE current_database = currentDatabase() AND query LIKE 'SELECT * FROM /* test 01413, query 3 */ rows_events_test%' AND type = 2 AND event_date >= yesterday() ORDER BY event_time DESC LIMIT 1;
SELECT ProfileEvents.Values as value FROM system.query_log ARRAY JOIN ProfileEvents
WHERE ProfileEvents.Names = 'SelectedRows' AND query LIKE 'SELECT * FROM /* test 01413, query 3 */ rows_events_test%' AND type = 2 AND event_date >= yesterday() ORDER BY event_time DESC LIMIT 1;
WHERE current_database = currentDatabase() AND ProfileEvents.Names = 'SelectedRows' AND query LIKE 'SELECT * FROM /* test 01413, query 3 */ rows_events_test%' AND type = 2 AND event_date >= yesterday() ORDER BY event_time DESC LIMIT 1;
DROP TABLE rows_events_test;

View File

@ -1,7 +1,7 @@
DROP TABLE IF EXISTS lc_nullable;
CREATE TABLE lc_nullable (
order_key Array(LowCardinality(Nullable((UInt64)))),
order_key Array(LowCardinality(Nullable(UInt64))),
i8 Array(LowCardinality(Nullable(Int8))),
i16 Array(LowCardinality(Nullable(Int16))),
@ -14,10 +14,10 @@ CREATE TABLE lc_nullable (
f32 Array(LowCardinality(Nullable(Float32))),
f64 Array(LowCardinality(Nullable(Float64))),
date Array(LowCardinality(Nullable((Date)))),
date Array(LowCardinality(Nullable(Date))),
date_time Array(LowCardinality(Nullable(DateTime('Europe/Moscow')))),
str Array(LowCardinality(Nullable((String)))),
str Array(LowCardinality(Nullable(String))),
fixed_string Array(LowCardinality(Nullable(FixedString(5))))
) ENGINE = MergeTree() ORDER BY order_key;

View File

@ -4,4 +4,4 @@ set max_threads = 16;
select sum(number) from remote('127.0.0.{1|2}', numbers_mt(1000000)) group by number % 2 order by number % 2;
system flush logs;
select length(thread_ids) >= 16 from system.query_log where event_date >= today() - 1 and lower(query) like '%select sum(number) from remote(_127.0.0.{1|2}_, numbers_mt(1000000)) group by number %' and type = 'QueryFinish' order by query_start_time desc limit 1;
select length(thread_ids) >= 16 from system.query_log where current_database = currentDatabase() and event_date >= today() - 1 and lower(query) like '%select sum(number) from remote(_127.0.0.{1|2}_, numbers_mt(1000000)) group by number %' and type = 'QueryFinish' order by query_start_time desc limit 1;

View File

@ -29,7 +29,9 @@ select count(*) "'"'"initial query spans with proper parent"'"'"
array join attribute.names as attribute_name,
attribute.values as attribute_value) o
join system.query_log on query_id = o.attribute_value
where trace_id = reinterpretAsUUID(reverse(unhex('$trace_id')))
where
trace_id = reinterpretAsUUID(reverse(unhex('$trace_id')))
and current_database = currentDatabase()
and operation_name = 'query'
and parent_span_id = reinterpretAsUInt64(unhex('73'))
and o.attribute_name = 'clickhouse.query_id'
@ -61,7 +63,7 @@ trace_id=$(${CLICKHOUSE_CLIENT} -q "select lower(hex(reverse(reinterpretAsString
# https://github.com/ClickHouse/ClickHouse/issues/14228
${CLICKHOUSE_CURL} \
--header "traceparent: 00-$trace_id-0000000000000073-01" \
--header "tracestate: some custom state" "http://127.0.0.2:8123/" \
--header "tracestate: some custom state" "$CLICKHOUSE_URL" \
--get \
--data-urlencode "query=select 1 from remote('127.0.0.2', system, one) format Null"

View File

@ -6,12 +6,14 @@ WITH (
(
SELECT query_start_time_microseconds
FROM system.query_log
WHERE current_database = currentDatabase()
ORDER BY query_start_time DESC
LIMIT 1
) AS time_with_microseconds,
(
SELECT query_start_time
FROM system.query_log
WHERE current_database = currentDatabase()
ORDER BY query_start_time DESC
LIMIT 1
) AS t)
@ -24,12 +26,14 @@ WITH (
(
SELECT query_start_time_microseconds
FROM system.query_thread_log
WHERE current_database = currentDatabase()
ORDER BY query_start_time DESC
LIMIT 1
) AS time_with_microseconds,
(
SELECT query_start_time
FROM system.query_thread_log
WHERE current_database = currentDatabase()
ORDER BY query_start_time DESC
LIMIT 1
) AS t)

View File

@ -33,6 +33,7 @@ SELECT '01473_query_log_table_event_start_time_microseconds_test';
WITH (
SELECT event_time_microseconds, event_time
FROM system.query_log
WHERE current_database = currentDatabase()
ORDER BY event_time DESC
LIMIT 1
) AS time
@ -42,6 +43,7 @@ SELECT '01473_query_thread_log_table_event_start_time_microseconds_test';
WITH (
SELECT event_time_microseconds, event_time
FROM system.query_thread_log
WHERE current_database = currentDatabase()
ORDER BY event_time DESC
LIMIT 1
) AS time
@ -51,6 +53,7 @@ SELECT '01473_text_log_table_event_start_time_microseconds_test';
WITH (
SELECT event_time_microseconds, event_time
FROM system.query_thread_log
WHERE current_database = currentDatabase()
ORDER BY event_time DESC
LIMIT 1
) AS time

View File

@ -19,7 +19,7 @@ ${CLICKHOUSE_CLIENT} -n -q "
system flush logs;
select interface, initial_query_id = query_id
from system.query_log
where query_id = '$query_id' and type = 'QueryFinish'
where current_database = currentDatabase() AND query_id = '$query_id' and type = 'QueryFinish'
order by interface
;
"

View File

@ -15,7 +15,7 @@ min_trace_entries=2
query_id_tcp_prefix="01526-tcp-memory-tracking-$RANDOM-$$"
${CLICKHOUSE_CLIENT} --log_queries=1 --max_threads=1 --max_untracked_memory=0 --memory_profiler_sample_probability=1 -q "with '$query_id_tcp_prefix' as __id $query FORMAT Null"
${CLICKHOUSE_CLIENT} -q "SYSTEM FLUSH LOGS"
query_id_tcp="$(${CLICKHOUSE_CLIENT} -q "SELECT DISTINCT query_id FROM system.query_log WHERE query LIKE '%$query_id_tcp_prefix%'")"
query_id_tcp="$(${CLICKHOUSE_CLIENT} -q "SELECT DISTINCT query_id FROM system.query_log WHERE current_database = currentDatabase() AND query LIKE '%$query_id_tcp_prefix%'")"
${CLICKHOUSE_CLIENT} -q "SELECT count()>=$min_trace_entries FROM system.trace_log WHERE query_id = '$query_id_tcp' AND abs(size) < 4e6 AND event_time >= now() - interval 1 hour"
# HTTP

View File

@ -2,4 +2,4 @@
SELECT 1;
SYSTEM FLUSH LOGS;
SELECT any() as t, substring(query, 1, 70) AS query, avg(memory_usage) usage, count() count FROM system.query_log WHERE event_date >= toDate(1604295323) AND event_time >= toDateTime(1604295323) AND type in (1,2,3,4) and initial_user in ('') and('all' = 'all' or(positionCaseInsensitive(query, 'all') = 1)) GROUP BY query ORDER BY usage desc LIMIT 5; -- { serverError 42 }
SELECT any() as t, substring(query, 1, 70) AS query, avg(memory_usage) usage, count() count FROM system.query_log WHERE current_database = currentDatabase() AND event_date >= toDate(1604295323) AND event_time >= toDateTime(1604295323) AND type in (1,2,3,4) and initial_user in ('') and('all' = 'all' or(positionCaseInsensitive(query, 'all') = 1)) GROUP BY query ORDER BY usage desc LIMIT 5; -- { serverError 42 }

View File

@ -26,4 +26,4 @@ ${CLICKHOUSE_CLIENT} -q "drop table if exists test_log_queries.logtable3" "--que
${CLICKHOUSE_CLIENT} -q "drop database if exists test_log_queries" "--query_id=01600_log_queries_with_extensive_info_018"
${CLICKHOUSE_CLIENT} -q "system flush logs"
${CLICKHOUSE_CLIENT} -q "select columns(query, normalized_query_hash, query_kind, databases, tables, columns) apply (any) from system.query_log where type = 'QueryFinish' and query_id like '01600_log_queries_with_extensive_info%' group by query_id order by query_id"
${CLICKHOUSE_CLIENT} -q "select columns(query, normalized_query_hash, query_kind, databases, tables, columns) apply (any) from system.query_log where current_database = currentDatabase() AND type = 'QueryFinish' and query_id like '01600_log_queries_with_extensive_info%' group by query_id order by query_id"

View File

@ -51,7 +51,7 @@ WHERE (query_id =
[NULL, NULL, NULL, NULL, 0.00009999999747378752, NULL, NULL, NULL, NULL, NULL],
query_id
FROM system.query_log
WHERE (query LIKE '%test cpu time query profiler%') AND (query NOT LIKE '%system%')
WHERE current_database = currentDatabase() AND (query LIKE '%test cpu time query profiler%') AND (query NOT LIKE '%system%')
ORDER BY event_time DESC
LIMIT 1
)) AND (symbol LIKE '%Source%');
@ -66,7 +66,7 @@ WHERE greaterOrEquals(event_date, ignore(ignore(ignore(NULL, '')), 256), yesterd
ignore(ignore(ignore(ignore(65536)), ignore(65537), ignore(2)), ''),
query_id
FROM system.query_log
WHERE (event_date >= yesterday()) AND (query LIKE '%test memory profiler%')
WHERE current_database = currentDatabase() AND (event_date >= yesterday()) AND (query LIKE '%test memory profiler%')
ORDER BY event_time DESC
LIMIT 1
)); -- { serverError 42 }
@ -79,6 +79,7 @@ WITH (
(
SELECT query_start_time_microseconds
FROM system.query_log
WHERE current_database = currentDatabase()
ORDER BY query_start_time DESC
LIMIT 1
) AS time_with_microseconds,
@ -87,6 +88,7 @@ WITH (
inf,
query_start_time
FROM system.query_log
WHERE current_database = currentDatabase()
ORDER BY query_start_time DESC
LIMIT 1
) AS t)
@ -96,12 +98,14 @@ WITH (
(
SELECT query_start_time_microseconds
FROM system.query_log
WHERE current_database = currentDatabase()
ORDER BY query_start_time DESC
LIMIT 1
) AS time_with_microseconds,
(
SELECT query_start_time
FROM system.query_log
WHERE current_database = currentDatabase()
ORDER BY query_start_time DESC
LIMIT 1
) AS t)

View File

@ -13,27 +13,27 @@ SELECT '';
SYSTEM FLUSH LOGS;
SELECT arraySort(used_aggregate_functions)
FROM system.query_log WHERE type = 'QueryFinish' AND (query LIKE '%toDate(\'2000-12-05\')%')
FROM system.query_log WHERE current_database = currentDatabase() AND type = 'QueryFinish' AND (query LIKE '%toDate(\'2000-12-05\')%')
ORDER BY query_start_time DESC LIMIT 1 FORMAT TabSeparatedWithNames;
SELECT '';
SELECT arraySort(used_aggregate_function_combinators)
FROM system.query_log WHERE type = 'QueryFinish' AND (query LIKE '%toDate(\'2000-12-05\')%')
FROM system.query_log WHERE current_database = currentDatabase() AND type = 'QueryFinish' AND (query LIKE '%toDate(\'2000-12-05\')%')
ORDER BY query_start_time DESC LIMIT 1 FORMAT TabSeparatedWithNames;
SELECT '';
SELECT arraySort(used_table_functions)
FROM system.query_log WHERE type = 'QueryFinish' AND (query LIKE '%toDate(\'2000-12-05\')%')
FROM system.query_log WHERE current_database = currentDatabase() AND type = 'QueryFinish' AND (query LIKE '%toDate(\'2000-12-05\')%')
ORDER BY query_start_time DESC LIMIT 1 FORMAT TabSeparatedWithNames;
SELECT '';
SELECT arraySort(used_functions)
FROM system.query_log WHERE type = 'QueryFinish' AND (query LIKE '%toDate(\'2000-12-05\')%')
FROM system.query_log WHERE current_database = currentDatabase() AND type = 'QueryFinish' AND (query LIKE '%toDate(\'2000-12-05\')%')
ORDER BY query_start_time DESC LIMIT 1 FORMAT TabSeparatedWithNames;
SELECT '';
SELECT arraySort(used_data_type_families)
FROM system.query_log WHERE type = 'QueryFinish' AND (query LIKE '%toDate(\'2000-12-05\')%')
FROM system.query_log WHERE current_database = currentDatabase() AND type = 'QueryFinish' AND (query LIKE '%toDate(\'2000-12-05\')%')
ORDER BY query_start_time DESC LIMIT 1 FORMAT TabSeparatedWithNames;
SELECT '';
@ -43,7 +43,7 @@ CREATE database test_query_log_factories_info1 ENGINE=Atomic;
SYSTEM FLUSH LOGS;
SELECT used_database_engines
FROM system.query_log
WHERE type == 'QueryFinish' AND (query LIKE '%database test_query_log_factories_info%')
WHERE current_database = currentDatabase() AND type == 'QueryFinish' AND (query LIKE '%database test_query_log_factories_info%')
ORDER BY query_start_time DESC LIMIT 1 FORMAT TabSeparatedWithNames;
SELECT '';
@ -52,7 +52,7 @@ CREATE OR REPLACE TABLE test_query_log_factories_info1.memory_table (id BIGINT,
SYSTEM FLUSH LOGS;
SELECT arraySort(used_data_type_families), used_storages
FROM system.query_log
WHERE type == 'QueryFinish' AND (query LIKE '%TABLE test%')
WHERE current_database = currentDatabase() AND type == 'QueryFinish' AND (query LIKE '%TABLE test%')
ORDER BY query_start_time DESC LIMIT 1 FORMAT TabSeparatedWithNames;
SELECT '';

View File

@ -6,4 +6,4 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}" -d 'SELECT 1' --referer 'https://yandex.ru/'
${CLICKHOUSE_CLIENT} --query "SYSTEM FLUSH LOGS"
${CLICKHOUSE_CLIENT} --query "SELECT http_referer FROM system.query_log WHERE http_referer LIKE '%yandex%' LIMIT 1"
${CLICKHOUSE_CLIENT} --query "SELECT http_referer FROM system.query_log WHERE current_database = currentDatabase() AND http_referer LIKE '%yandex%' LIMIT 1"

View File

@ -1,5 +1,5 @@
SET log_comment = 'log_comment test', log_queries = 1;
SELECT 1;
SYSTEM FLUSH LOGS;
SELECT type, query FROM system.query_log WHERE log_comment = 'log_comment test' AND event_date >= yesterday() AND type = 1 ORDER BY event_time DESC LIMIT 1;
SELECT type, query FROM system.query_log WHERE log_comment = 'log_comment test' AND event_date >= yesterday() AND type = 2 ORDER BY event_time DESC LIMIT 1;
SELECT type, query FROM system.query_log WHERE current_database = currentDatabase() AND log_comment = 'log_comment test' AND event_date >= yesterday() AND type = 1 ORDER BY event_time DESC LIMIT 1;
SELECT type, query FROM system.query_log WHERE current_database = currentDatabase() AND log_comment = 'log_comment test' AND event_date >= yesterday() AND type = 2 ORDER BY event_time DESC LIMIT 1;

View File

@ -0,0 +1,15 @@
0
0
0 0 0
1 1 1
2 2 2
3 3 3
4 4 4
0 0 0
1 1 1
2 2 2
┌─rowNr─┬─val_string─┬─str_m1───┬─str_p1───┬─val_low──┬─low_m1───┬─low_p1───┐
│ 1 │ String 1 │ │ String 1 │ String 1 │ │ String 1 │
│ 2 │ String 1 │ String 1 │ String 2 │ String 1 │ String 1 │ String 2 │
│ 3 │ String 2 │ String 1 │ │ String 2 │ String 1 │ │
└───────┴────────────┴──────────┴──────────┴──────────┴──────────┴──────────┘

View File

@ -0,0 +1,43 @@
SELECT
neighbor(n, -2) AS int,
neighbor(s, -2) AS str,
neighbor(lcs, -2) AS lowCstr
FROM
(
SELECT
number % 5 AS n,
toString(n) AS s,
CAST(s, 'LowCardinality(String)') AS lcs
FROM numbers(10)
);
drop table if exists neighbor_test;
CREATE TABLE neighbor_test
(
`rowNr` UInt8,
`val_string` String,
`val_low` LowCardinality(String)
)
ENGINE = MergeTree
PARTITION BY tuple()
ORDER BY rowNr;
INSERT INTO neighbor_test VALUES (1, 'String 1', 'String 1'), (2, 'String 1', 'String 1'), (3, 'String 2', 'String 2');
SELECT
rowNr,
val_string,
neighbor(val_string, -1) AS str_m1,
neighbor(val_string, 1) AS str_p1,
val_low,
neighbor(val_low, -1) AS low_m1,
neighbor(val_low, 1) AS low_p1
FROM
(
SELECT *
FROM neighbor_test
ORDER BY val_string ASC
) format PrettyCompact;
drop table if exists neighbor_test;

View File

@ -0,0 +1,60 @@
1 50 50 1 0 49
1 50 50 1 0 49
1 50 50 1 0 49
1 50 51 0 1 51
1 50 50 1 0 49
1 50 51 0 1 51
1 50 50 1 0 49
1 50 50 1 0 49
1 50 50 1 0 49
1 50 50 1 0 49
1 50 50 1 0 49
1 50 51 0 1 51
1 50 50 1 0 49
1 50 51 0 1 51
1 50 50 1 0 49
1 50 51 0 1 51
1 50 50 1 0 49
1 50 51 0 1 51
1 50 50 1 0 49
1 50 50 1 0 49
1 50 50 1 0 49
1 50 51 0 1 51
1 50 50 1 0 49
1 50 51 0 1 51
1 50 50 1 0 49
1 50 50 1 0 49
1 50 50 1 0 49
1 50 50 1 0 49
1 50 50 1 0 49
1 50 50 1 0 49
1 50 50 1 0 49
1 50 51 0 1 51
1 50 50 1 0 49
1 50 51 0 1 51
1 50 50 1 0 49
1 50 51 0 1 51
1 50 51 0 1 51
1 50 50 1 0 49
1 50 50 1 0 49
1 50 50 1 0 49
1 50 51 0 1 51
1 50 50 1 0 49
1 50 50 1 0 49
1 50 51 0 1 51
1 50 51 0 1 51
1 50 50 1 0 49
1 50 51 0 1 51
1 50 50 1 0 49
1 50 50 1 0 49
1 50 50 1 0 49
1 50 50 1 0 49
1 50 51 0 1 51
1 50 51 0 1 51
1 50 50 1 0 49
1 50 50 1 0 49
1 50 51 0 1 51
1 50 50 1 0 49
1 50 50 1 0 49
1 50 50 1 0 49
1 50 51 0 1 51

View File

@ -0,0 +1,55 @@
CREATE TABLE group_bitmap_data_test
(
`pickup_date` Date,
`city_id` UInt32,
`uid` UInt32
)
ENGINE = Memory;
INSERT INTO group_bitmap_data_test SELECT
'2019-01-01',
1,
number
FROM numbers(1, 50);
INSERT INTO group_bitmap_data_test SELECT
'2019-01-02',
1,
number
FROM numbers(11, 60);
INSERT INTO group_bitmap_data_test SELECT
'2019-01-03',
2,
number
FROM numbers(1, 10);
SELECT
bitmapCardinality(day_today) AS today_users,
bitmapCardinality(day_before) AS before_users,
bitmapCardinality(bitmapOr(day_today, day_before)) AS ll_users,
bitmapCardinality(bitmapAnd(day_today, day_before)) AS old_users,
bitmapCardinality(bitmapAndnot(day_today, day_before)) AS new_users,
bitmapCardinality(bitmapXor(day_today, day_before)) AS diff_users
FROM
(
SELECT
city_id,
groupBitmapState(uid) AS day_today
FROM group_bitmap_data_test
WHERE pickup_date = '2019-01-02'
GROUP BY
uid,
city_id
) AS js1
ALL LEFT JOIN
(
SELECT
city_id,
groupBitmapState(uid) AS day_before
FROM group_bitmap_data_test
WHERE pickup_date = '2019-01-01'
GROUP BY city_id
) AS js2 USING (city_id);
DROP TABLE IF EXISTS group_bitmap_data_test;

View File

@ -0,0 +1 @@
SELECT [NULL, '25.6', '-0.02', NULL], [NULL], 1024, [NULL, '10485.76', NULL, NULL], [NULL, '-922337203.6854775808', toNullable(NULL)], [NULL] FROM (SELECT [multiIf((number % 1023) = -inf, toString(number), NULL)], NULL, '-1', multiIf((number % NULL) = NULL, toString(number), ''), [NULL, NULL], multiIf((number % NULL) = 65536, toString(number), '') AS s FROM system.numbers) LIMIT 1024 format Null

View File

@ -0,0 +1 @@
Ok

View File

@ -0,0 +1,16 @@
#!/usr/bin/env bash
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
# shellcheck source=../shell_config.sh
. "$CURDIR"/../shell_config.sh
I=0
while true
do
I=$((I + 1))
TYPE=$(perl -e "print 'Array(' x $I; print 'UInt8'; print ')' x $I")
${CLICKHOUSE_CLIENT} --max_parser_depth 1000000 --query "SELECT * FROM remote('127.0.0.{1,2}', generateRandom('x $TYPE', 1, 1, 1)) LIMIT 1 FORMAT Null" 2>&1 | grep -q -F 'Maximum parse depth' && break;
done
#echo "I = ${I}"
echo 'Ok'

View File

@ -196,3 +196,5 @@
01181_db_atomic_drop_on_cluster
01658_test_base64Encode_mysql_compatibility
01659_test_base64Decode_mysql_compatibility
01675_data_type_coroutine
01671_aggregate_function_group_bitmap_data

View File

@ -317,11 +317,14 @@
/// Pessimistic list of tests which work badly in parallel.
/// Probably they need better investigation.
"00062_replicated_merge_tree_alter_zookeeper",
"00080_show_tables_and_system_tables",
"00101_materialized_views_and_insert_without_explicit_database",
"00109_shard_totals_after_having",
"00110_external_sort",
"00116_storage_set",
"00121_drop_column_zookeeper",
"00133_long_shard_memory_tracker_and_exception_safety",
"00158_buffer_and_nonexistent_table",
"00180_attach_materialized_view",
"00226_zookeeper_deduplication_and_unexpected_parts",
"00236_replicated_drop_on_non_leader_zookeeper",
@ -342,9 +345,12 @@
"00571_non_exist_database_when_create_materializ_view",
"00575_illegal_column_exception_when_drop_depen_column",
"00599_create_view_with_subquery",
"00604_show_create_database",
"00612_http_max_query_size",
"00619_union_highlite",
"00620_optimize_on_nonleader_replica_zookeeper",
"00623_truncate_table",
"00623_truncate_table_throw_exception",
"00625_arrays_in_nested",
"00626_replace_partition_from_table",
"00626_replace_partition_from_table_zookeeper",
@ -357,20 +363,28 @@
"00699_materialized_view_mutations",
"00701_rollup",
"00715_fetch_merged_or_mutated_part_zookeeper",
"00716_allow_ddl",
"00719_parallel_ddl_db",
"00740_database_in_nested_view",
"00741_client_comment_multiline",
"00751_default_databasename_for_view",
"00753_alter_attach",
"00754_alter_modify_column_partitions",
"00754_alter_modify_order_by_replicated_zookeeper",
"00763_long_lock_buffer_alter_destination_table",
"00800_versatile_storage_join",
"00804_test_alter_compression_codecs",
"00804_test_custom_compression_codecs",
"00804_test_custom_compression_codes_log_storages",
"00804_test_delta_codec_compression",
"00815_left_join_on_stepanel",
"00834_cancel_http_readonly_queries_on_client_close",
"00834_kill_mutation",
"00834_kill_mutation_replicated_zookeeper",
"00840_long_concurrent_select_and_drop_deadlock",
"00857_global_joinsavel_table_alias",
"00899_long_attach_memory_limit",
"00910_buffer_prewhere",
"00910_zookeeper_custom_compression_codecs_replicated",
"00926_adaptive_index_granularity_merge_tree",
"00926_adaptive_index_granularity_pk",
@ -388,49 +402,94 @@
"00988_constraints_replication_zookeeper",
"00989_parallel_parts_loading",
"00993_system_parts_race_condition_drop_zookeeper",
"01012_show_tables_limit",
"01013_sync_replica_timeout_zookeeper",
"01014_lazy_database_basic",
"01014_lazy_database_concurrent_recreate_reattach_and_show_tables",
"01015_attach_part",
"01015_database_bad_tables",
"01018_ddl_dictionaries_concurrent_requrests",
"01018_ddl_dictionaries_create",
"01018_ddl_dictionaries_select",
"01018_ddl_dictionaries_special",
"01018_dictionaries_from_dictionaries",
"01018_ip_dictionary",
"01021_only_tuple_columns",
"01023_materialized_view_query_context",
"01031_mutations_interpreter_and_context",
"01033_dictionaries_lifetime",
"01035_concurrent_move_partition_from_table_zookeeper",
"01036_no_superfluous_dict_reload_on_create_database",
"01036_no_superfluous_dict_reload_on_create_database_2",
"01037_polygon_dicts_correctness_all",
"01037_polygon_dicts_correctness_fast",
"01037_polygon_dicts_simple_functions",
"01038_dictionary_lifetime_min_zero_sec",
"01040_dictionary_invalidate_query_switchover_long",
"01041_create_dictionary_if_not_exists",
"01042_system_reload_dictionary_reloads_completely",
"01043_dictionary_attribute_properties_values",
"01045_dictionaries_restrictions",
"01045_zookeeper_system_mutations_with_parts_names",
"01048_exists_query",
"01053_drop_database_mat_view",
"01053_ssd_dictionary",
"01054_cache_dictionary_bunch_update",
"01054_cache_dictionary_overflow_cell",
"01055_compact_parts_1",
"01056_create_table_as",
"01060_avro",
"01060_shutdown_table_after_detach",
"01069_database_memory",
"01070_materialize_ttl",
"01070_modify_ttl",
"01070_mutations_with_dependencies",
"01071_live_view_detach_dependency",
"01071_prohibition_secondary_index_with_old_format_merge_tree",
"01073_attach_if_not_exists",
"01073_show_tables_not_like",
"01076_cache_dictionary_datarace_exception_ptr",
"01076_parallel_alter_replicated_zookeeper",
"01079_parallel_alter_add_drop_column_zookeeper",
"01079_parallel_alter_detach_table_zookeeper",
"01080_check_for_error_incorrect_size_of_nested_column",
"01083_expressions_in_engine_arguments",
"01084_regexp_empty",
"01085_max_distributed_connections_http",
"01092_memory_profiler",
"01098_temporary_and_external_tables",
"01103_distributed_product_mode_local_column_renames",
"01107_atomic_db_detach_attach",
"01108_restart_replicas_rename_deadlock_zookeeper",
"01109_exchange_tables",
"01110_dictionary_layout_without_arguments",
"01113_local_dictionary_type_conversion",
"01114_database_atomic",
"01114_mysql_database_engine_segfault",
"01115_join_with_dictionary",
"01125_dict_ddl_cannot_add_column",
"01127_month_partitioning_consistency_select",
"01130_in_memory_parts_partitons",
"01135_default_and_alter_zookeeper",
"01148_zookeeper_path_macros_unfolding",
"01150_ddl_guard_rwr",
"01185_create_or_replace_table",
"01190_full_attach_syntax",
"01191_rename_dictionary",
"01192_rename_database_zookeeper",
"01193_metadata_loading",
"01200_mutations_memory_consumption",
"01224_no_superfluous_dict_reload",
"01225_drop_dictionary_as_table",
"01225_show_create_table_from_dictionary",
"01231_distributed_aggregation_memory_efficient_mix_levels",
"01232_extremes",
"01238_http_memory_tracking",
"01249_bad_arguments_for_bloom_filter",
"01251_dict_is_in_infinite_loop",
"01254_dict_create_without_db",
"01254_dict_load_after_detach_attach",
"01257_dictionary_mismatch_types",
"01259_dictionary_custom_settings_ddl",
"01267_alter_default_key_columns_zookeeper",
"01268_dictionary_direct_layout",
@ -444,18 +503,25 @@
"01293_system_distribution_queue",
"01294_lazy_database_concurrent",
"01294_lazy_database_concurrent_recreate_reattach_and_show_tables",
"01294_system_distributed_on_cluster",
"01296_create_row_policy_in_current_database",
"01305_replica_create_drop_zookeeper",
"01307_multiple_leaders_zookeeper",
"01318_long_unsuccessful_mutation_zookeeper",
"01319_manual_write_to_replicas",
"01320_create_sync_race_condition_zookeeper",
"01338_long_select_and_alter",
"01338_long_select_and_alter_zookeeper",
"01355_alter_column_with_order",
"01355_ilike",
"01357_version_collapsing_attach_detach_zookeeper",
"01375_compact_parts_codecs",
"01376_GROUP_BY_injective_elimination_dictGet",
"01378_alter_rename_with_ttl_zookeeper",
"01383_remote_ambiguous_column_shard",
"01388_clear_all_columns",
"01391_join_on_dict_crash",
"01392_column_resolve",
"01396_inactive_replica_cleanup_nodes_zookeeper",
"01412_cache_dictionary_race",
"01414_mutations_and_errors_zookeeper",
@ -464,20 +530,41 @@
"01417_freeze_partition_verbose",
"01417_freeze_partition_verbose_zookeeper",
"01430_modify_sample_by_zookeeper",
"01444_create_table_drop_database_race",
"01454_storagememory_data_race_challenge",
"01455_rank_correlation_spearman",
"01456_modify_column_type_via_add_drop_update",
"01457_create_as_table_function_structure",
"01459_manual_write_to_replicas",
"01460_DistributedFilesToInsert",
"01465_ttl_recompression",
"01470_show_databases_like",
"01471_calculate_ttl_during_merge",
"01487_distributed_in_not_default_db",
"01493_alter_remove_properties_zookeeper",
"01493_storage_set_persistency",
"01494_storage_join_persistency",
"01501_cache_dictionary_all_fields",
"01507_clickhouse_server_start_with_embedded_config",
"01509_dictionary_preallocate",
"01516_create_table_primary_key",
"01516_drop_table_stress",
"01517_drop_mv_with_inner_table",
"01526_complex_key_dict_direct_layout",
"01527_clickhouse_local_optimize",
"01527_dist_sharding_key_dictGet_reload",
"01530_drop_database_atomic_sync",
"01541_max_memory_usage_for_user",
"01646_system_restart_replicas_smoke", // system restart replicas is a global query
"01542_dictionary_load_exception_race",
"01575_disable_detach_table_of_dictionary",
"01600_count_of_parts_metrics", // tests global system metrics
"01600_detach_permanently",
"01600_log_queries_with_extensive_info",
"01600_multiple_left_join_with_aliases",
"01601_detach_permanently",
"01602_show_create_view",
"01603_rename_overwrite_bug",
"01646_system_restart_replicas_smoke", // system restart replicas is a global query
"attach",
"ddl_dictionaries",
"dictionary",

View File

@ -18,11 +18,15 @@ xfails = {
[(Fail, "known issue")],
"encrypt/invalid key or iv length for mode/mode=\"'aes-???-gcm'\", key_len=??, iv_len=12, aad=True/iv is too long":
[(Fail, "known issue")],
"encrypt/invalid plaintext data type/data_type='IPv6', value=\"toIPv6('2001:0db8:0000:85a3:0000:0000:ac1f:8001')\"":
[(Fail, "known issue as IPv6 is implemented as FixedString(16)")],
# encrypt_mysql
"encrypt_mysql/key or iv length for mode/mode=\"'aes-???-ecb'\", key_len=??, iv_len=None":
[(Fail, issue_18251)],
"encrypt_mysql/invalid parameters/iv not valid for mode":
[(Fail, issue_18251)],
"encrypt_mysql/invalid plaintext data type/data_type='IPv6', value=\"toIPv6('2001:0db8:0000:85a3:0000:0000:ac1f:8001')\"":
[(Fail, "known issue as IPv6 is implemented as FixedString(16)")],
# decrypt_mysql
"decrypt_mysql/key or iv length for mode/mode=\"'aes-???-ecb'\", key_len=??, iv_len=None:":
[(Fail, issue_18251)],

View File

@ -6,102 +6,113 @@
* 1 [Revision History](#revision-history)
* 2 [Introduction](#introduction)
* 3 [Terminology](#terminology)
* 3.1 [AES](#aes)
* 3.2 [AEAD](#aead)
* 4 [Requirements](#requirements)
* 4.1 [Generic](#generic)
* 4.1.1 [RQ.SRS008.AES.Functions](#rqsrs008aesfunctions)
* 4.1.2 [RQ.SRS008.AES.Functions.Compatability.MySQL](#rqsrs008aesfunctionscompatabilitymysql)
* 4.1.3 [RQ.SRS008.AES.Functions.Compatability.Dictionaries](#rqsrs008aesfunctionscompatabilitydictionaries)
* 4.1.4 [RQ.SRS008.AES.Functions.Compatability.Engine.Database.MySQL](#rqsrs008aesfunctionscompatabilityenginedatabasemysql)
* 4.1.5 [RQ.SRS008.AES.Functions.Compatability.Engine.Table.MySQL](#rqsrs008aesfunctionscompatabilityenginetablemysql)
* 4.1.6 [RQ.SRS008.AES.Functions.Compatability.TableFunction.MySQL](#rqsrs008aesfunctionscompatabilitytablefunctionmysql)
* 4.1.7 [RQ.SRS008.AES.Functions.DifferentModes](#rqsrs008aesfunctionsdifferentmodes)
* 4.1.8 [RQ.SRS008.AES.Functions.DataFromMultipleSources](#rqsrs008aesfunctionsdatafrommultiplesources)
* 4.1.9 [RQ.SRS008.AES.Functions.SuppressOutputOfSensitiveValues](#rqsrs008aesfunctionssuppressoutputofsensitivevalues)
* 4.1.10 [RQ.SRS008.AES.Functions.InvalidParameters](#rqsrs008aesfunctionsinvalidparameters)
* 4.1.11 [RQ.SRS008.AES.Functions.Mismatched.Key](#rqsrs008aesfunctionsmismatchedkey)
* 4.1.12 [RQ.SRS008.AES.Functions.Mismatched.IV](#rqsrs008aesfunctionsmismatchediv)
* 4.1.13 [RQ.SRS008.AES.Functions.Mismatched.AAD](#rqsrs008aesfunctionsmismatchedaad)
* 4.1.14 [RQ.SRS008.AES.Functions.Mismatched.Mode](#rqsrs008aesfunctionsmismatchedmode)
* 4.1.15 [RQ.SRS008.AES.Functions.Check.Performance](#rqsrs008aesfunctionscheckperformance)
* 4.1.16 [RQ.SRS008.AES.Function.Check.Performance.BestCase](#rqsrs008aesfunctioncheckperformancebestcase)
* 4.1.17 [RQ.SRS008.AES.Function.Check.Performance.WorstCase](#rqsrs008aesfunctioncheckperformanceworstcase)
* 4.1.18 [RQ.SRS008.AES.Functions.Check.Compression](#rqsrs008aesfunctionscheckcompression)
* 4.1.19 [RQ.SRS008.AES.Functions.Check.Compression.LowCardinality](#rqsrs008aesfunctionscheckcompressionlowcardinality)
* 4.2 [Specific](#specific)
* 4.2.1 [RQ.SRS008.AES.Encrypt.Function](#rqsrs008aesencryptfunction)
* 4.2.2 [RQ.SRS008.AES.Encrypt.Function.Syntax](#rqsrs008aesencryptfunctionsyntax)
* 4.2.3 [RQ.SRS008.AES.Encrypt.Function.NIST.TestVectors](#rqsrs008aesencryptfunctionnisttestvectors)
* 4.2.4 [RQ.SRS008.AES.Encrypt.Function.Parameters.PlainText](#rqsrs008aesencryptfunctionparametersplaintext)
* 4.2.5 [RQ.SRS008.AES.Encrypt.Function.Parameters.Key](#rqsrs008aesencryptfunctionparameterskey)
* 4.2.6 [RQ.SRS008.AES.Encrypt.Function.Parameters.Mode](#rqsrs008aesencryptfunctionparametersmode)
* 4.2.7 [RQ.SRS008.AES.Encrypt.Function.Parameters.Mode.ValuesFormat](#rqsrs008aesencryptfunctionparametersmodevaluesformat)
* 4.2.8 [RQ.SRS008.AES.Encrypt.Function.Parameters.Mode.Value.Invalid](#rqsrs008aesencryptfunctionparametersmodevalueinvalid)
* 4.2.9 [RQ.SRS008.AES.Encrypt.Function.Parameters.Mode.Values](#rqsrs008aesencryptfunctionparametersmodevalues)
* 4.2.10 [RQ.SRS008.AES.Encrypt.Function.Parameters.InitializationVector](#rqsrs008aesencryptfunctionparametersinitializationvector)
* 4.2.11 [RQ.SRS008.AES.Encrypt.Function.Parameters.AdditionalAuthenticatedData](#rqsrs008aesencryptfunctionparametersadditionalauthenticateddata)
* 4.2.12 [RQ.SRS008.AES.Encrypt.Function.Parameters.ReturnValue](#rqsrs008aesencryptfunctionparametersreturnvalue)
* 4.2.13 [RQ.SRS008.AES.Encrypt.Function.Key.Length.InvalidLengthError](#rqsrs008aesencryptfunctionkeylengthinvalidlengtherror)
* 4.2.14 [RQ.SRS008.AES.Encrypt.Function.InitializationVector.Length.InvalidLengthError](#rqsrs008aesencryptfunctioninitializationvectorlengthinvalidlengtherror)
* 4.2.15 [RQ.SRS008.AES.Encrypt.Function.InitializationVector.NotValidForMode](#rqsrs008aesencryptfunctioninitializationvectornotvalidformode)
* 4.2.16 [RQ.SRS008.AES.Encrypt.Function.AdditionalAuthenticationData.NotValidForMode](#rqsrs008aesencryptfunctionadditionalauthenticationdatanotvalidformode)
* 4.2.17 [RQ.SRS008.AES.Encrypt.Function.AdditionalAuthenticationData.Length](#rqsrs008aesencryptfunctionadditionalauthenticationdatalength)
* 4.2.18 [RQ.SRS008.AES.Encrypt.Function.NonGCMMode.KeyAndInitializationVector.Length](#rqsrs008aesencryptfunctionnongcmmodekeyandinitializationvectorlength)
* 4.2.19 [RQ.SRS008.AES.Encrypt.Function.GCMMode.KeyAndInitializationVector.Length](#rqsrs008aesencryptfunctiongcmmodekeyandinitializationvectorlength)
* 4.2.20 [RQ.SRS008.AES.Decrypt.Function](#rqsrs008aesdecryptfunction)
* 4.2.21 [RQ.SRS008.AES.Decrypt.Function.Syntax](#rqsrs008aesdecryptfunctionsyntax)
* 4.2.22 [RQ.SRS008.AES.Decrypt.Function.Parameters.CipherText](#rqsrs008aesdecryptfunctionparametersciphertext)
* 4.2.23 [RQ.SRS008.AES.Decrypt.Function.Parameters.Key](#rqsrs008aesdecryptfunctionparameterskey)
* 4.2.24 [RQ.SRS008.AES.Decrypt.Function.Parameters.Mode](#rqsrs008aesdecryptfunctionparametersmode)
* 4.2.25 [RQ.SRS008.AES.Decrypt.Function.Parameters.Mode.ValuesFormat](#rqsrs008aesdecryptfunctionparametersmodevaluesformat)
* 4.2.26 [RQ.SRS008.AES.Decrypt.Function.Parameters.Mode.Value.Invalid](#rqsrs008aesdecryptfunctionparametersmodevalueinvalid)
* 4.2.27 [RQ.SRS008.AES.Decrypt.Function.Parameters.Mode.Values](#rqsrs008aesdecryptfunctionparametersmodevalues)
* 4.2.28 [RQ.SRS008.AES.Decrypt.Function.Parameters.InitializationVector](#rqsrs008aesdecryptfunctionparametersinitializationvector)
* 4.2.29 [RQ.SRS008.AES.Decrypt.Function.Parameters.AdditionalAuthenticatedData](#rqsrs008aesdecryptfunctionparametersadditionalauthenticateddata)
* 4.2.30 [RQ.SRS008.AES.Decrypt.Function.Parameters.ReturnValue](#rqsrs008aesdecryptfunctionparametersreturnvalue)
* 4.2.31 [RQ.SRS008.AES.Decrypt.Function.Key.Length.InvalidLengthError](#rqsrs008aesdecryptfunctionkeylengthinvalidlengtherror)
* 4.2.32 [RQ.SRS008.AES.Decrypt.Function.InitializationVector.Length.InvalidLengthError](#rqsrs008aesdecryptfunctioninitializationvectorlengthinvalidlengtherror)
* 4.2.33 [RQ.SRS008.AES.Decrypt.Function.InitializationVector.NotValidForMode](#rqsrs008aesdecryptfunctioninitializationvectornotvalidformode)
* 4.2.34 [RQ.SRS008.AES.Decrypt.Function.AdditionalAuthenticationData.NotValidForMode](#rqsrs008aesdecryptfunctionadditionalauthenticationdatanotvalidformode)
* 4.2.35 [RQ.SRS008.AES.Decrypt.Function.AdditionalAuthenticationData.Length](#rqsrs008aesdecryptfunctionadditionalauthenticationdatalength)
* 4.2.36 [RQ.SRS008.AES.Decrypt.Function.NonGCMMode.KeyAndInitializationVector.Length](#rqsrs008aesdecryptfunctionnongcmmodekeyandinitializationvectorlength)
* 4.2.37 [RQ.SRS008.AES.Decrypt.Function.GCMMode.KeyAndInitializationVector.Length](#rqsrs008aesdecryptfunctiongcmmodekeyandinitializationvectorlength)
* 4.3 [MySQL Specific Functions](#mysql-specific-functions)
* 4.3.1 [RQ.SRS008.AES.MySQL.Encrypt.Function](#rqsrs008aesmysqlencryptfunction)
* 4.3.2 [RQ.SRS008.AES.MySQL.Encrypt.Function.Syntax](#rqsrs008aesmysqlencryptfunctionsyntax)
* 4.3.3 [RQ.SRS008.AES.MySQL.Encrypt.Function.Parameters.PlainText](#rqsrs008aesmysqlencryptfunctionparametersplaintext)
* 4.3.4 [RQ.SRS008.AES.MySQL.Encrypt.Function.Parameters.Key](#rqsrs008aesmysqlencryptfunctionparameterskey)
* 4.3.5 [RQ.SRS008.AES.MySQL.Encrypt.Function.Parameters.Mode](#rqsrs008aesmysqlencryptfunctionparametersmode)
* 4.3.6 [RQ.SRS008.AES.MySQL.Encrypt.Function.Parameters.Mode.ValuesFormat](#rqsrs008aesmysqlencryptfunctionparametersmodevaluesformat)
* 4.3.7 [RQ.SRS008.AES.MySQL.Encrypt.Function.Parameters.Mode.Value.Invalid](#rqsrs008aesmysqlencryptfunctionparametersmodevalueinvalid)
* 4.3.8 [RQ.SRS008.AES.MySQL.Encrypt.Function.Parameters.Mode.Values](#rqsrs008aesmysqlencryptfunctionparametersmodevalues)
* 4.3.9 [RQ.SRS008.AES.MySQL.Encrypt.Function.Parameters.Mode.Values.GCM.Error](#rqsrs008aesmysqlencryptfunctionparametersmodevaluesgcmerror)
* 4.3.10 [RQ.SRS008.AES.MySQL.Encrypt.Function.Parameters.Mode.Values.CTR.Error](#rqsrs008aesmysqlencryptfunctionparametersmodevaluesctrerror)
* 4.3.11 [RQ.SRS008.AES.MySQL.Encrypt.Function.Parameters.InitializationVector](#rqsrs008aesmysqlencryptfunctionparametersinitializationvector)
* 4.3.12 [RQ.SRS008.AES.MySQL.Encrypt.Function.Parameters.ReturnValue](#rqsrs008aesmysqlencryptfunctionparametersreturnvalue)
* 4.3.13 [RQ.SRS008.AES.MySQL.Encrypt.Function.Key.Length.TooShortError](#rqsrs008aesmysqlencryptfunctionkeylengthtooshorterror)
* 4.3.14 [RQ.SRS008.AES.MySQL.Encrypt.Function.Key.Length.TooLong](#rqsrs008aesmysqlencryptfunctionkeylengthtoolong)
* 4.3.15 [RQ.SRS008.AES.MySQL.Encrypt.Function.InitializationVector.Length.TooShortError](#rqsrs008aesmysqlencryptfunctioninitializationvectorlengthtooshorterror)
* 4.3.16 [RQ.SRS008.AES.MySQL.Encrypt.Function.InitializationVector.Length.TooLong](#rqsrs008aesmysqlencryptfunctioninitializationvectorlengthtoolong)
* 4.3.17 [RQ.SRS008.AES.MySQL.Encrypt.Function.InitializationVector.NotValidForMode](#rqsrs008aesmysqlencryptfunctioninitializationvectornotvalidformode)
* 4.3.18 [RQ.SRS008.AES.MySQL.Encrypt.Function.Mode.KeyAndInitializationVector.Length](#rqsrs008aesmysqlencryptfunctionmodekeyandinitializationvectorlength)
* 4.3.19 [RQ.SRS008.AES.MySQL.Decrypt.Function](#rqsrs008aesmysqldecryptfunction)
* 4.3.20 [RQ.SRS008.AES.MySQL.Decrypt.Function.Syntax](#rqsrs008aesmysqldecryptfunctionsyntax)
* 4.3.21 [RQ.SRS008.AES.MySQL.Decrypt.Function.Parameters.CipherText](#rqsrs008aesmysqldecryptfunctionparametersciphertext)
* 4.3.22 [RQ.SRS008.AES.MySQL.Decrypt.Function.Parameters.Key](#rqsrs008aesmysqldecryptfunctionparameterskey)
* 4.3.23 [RQ.SRS008.AES.MySQL.Decrypt.Function.Parameters.Mode](#rqsrs008aesmysqldecryptfunctionparametersmode)
* 4.3.24 [RQ.SRS008.AES.MySQL.Decrypt.Function.Parameters.Mode.ValuesFormat](#rqsrs008aesmysqldecryptfunctionparametersmodevaluesformat)
* 4.3.25 [RQ.SRS008.AES.MySQL.Decrypt.Function.Parameters.Mode.Value.Invalid](#rqsrs008aesmysqldecryptfunctionparametersmodevalueinvalid)
* 4.3.26 [RQ.SRS008.AES.MySQL.Decrypt.Function.Parameters.Mode.Values](#rqsrs008aesmysqldecryptfunctionparametersmodevalues)
* 4.3.27 [RQ.SRS008.AES.MySQL.Decrypt.Function.Parameters.Mode.Values.GCM.Error](#rqsrs008aesmysqldecryptfunctionparametersmodevaluesgcmerror)
* 4.3.28 [RQ.SRS008.AES.MySQL.Decrypt.Function.Parameters.Mode.Values.CTR.Error](#rqsrs008aesmysqldecryptfunctionparametersmodevaluesctrerror)
* 4.3.29 [RQ.SRS008.AES.MySQL.Decrypt.Function.Parameters.InitializationVector](#rqsrs008aesmysqldecryptfunctionparametersinitializationvector)
* 4.3.30 [RQ.SRS008.AES.MySQL.Decrypt.Function.Parameters.ReturnValue](#rqsrs008aesmysqldecryptfunctionparametersreturnvalue)
* 4.3.31 [RQ.SRS008.AES.MySQL.Decrypt.Function.Key.Length.TooShortError](#rqsrs008aesmysqldecryptfunctionkeylengthtooshorterror)
* 4.3.32 [RQ.SRS008.AES.MySQL.Decrypt.Function.Key.Length.TooLong](#rqsrs008aesmysqldecryptfunctionkeylengthtoolong)
* 4.3.33 [RQ.SRS008.AES.MySQL.Decrypt.Function.InitializationVector.Length.TooShortError](#rqsrs008aesmysqldecryptfunctioninitializationvectorlengthtooshorterror)
* 4.3.34 [RQ.SRS008.AES.MySQL.Decrypt.Function.InitializationVector.Length.TooLong](#rqsrs008aesmysqldecryptfunctioninitializationvectorlengthtoolong)
* 4.3.35 [RQ.SRS008.AES.MySQL.Decrypt.Function.InitializationVector.NotValidForMode](#rqsrs008aesmysqldecryptfunctioninitializationvectornotvalidformode)
* 4.3.36 [RQ.SRS008.AES.MySQL.Decrypt.Function.Mode.KeyAndInitializationVector.Length](#rqsrs008aesmysqldecryptfunctionmodekeyandinitializationvectorlength)
* 4.2 [Compatibility](#compatibility)
* 4.2.1 [RQ.SRS008.AES.Functions.Compatibility.MySQL](#rqsrs008aesfunctionscompatibilitymysql)
* 4.2.2 [RQ.SRS008.AES.Functions.Compatibility.Dictionaries](#rqsrs008aesfunctionscompatibilitydictionaries)
* 4.2.3 [RQ.SRS008.AES.Functions.Compatibility.Engine.Database.MySQL](#rqsrs008aesfunctionscompatibilityenginedatabasemysql)
* 4.2.4 [RQ.SRS008.AES.Functions.Compatibility.Engine.Table.MySQL](#rqsrs008aesfunctionscompatibilityenginetablemysql)
* 4.2.5 [RQ.SRS008.AES.Functions.Compatibility.TableFunction.MySQL](#rqsrs008aesfunctionscompatibilitytablefunctionmysql)
* 4.3 [Different Modes](#different-modes)
* 4.3.1 [RQ.SRS008.AES.Functions.DifferentModes](#rqsrs008aesfunctionsdifferentmodes)
* 4.4 [Multiple Sources](#multiple-sources)
* 4.4.1 [RQ.SRS008.AES.Functions.DataFromMultipleSources](#rqsrs008aesfunctionsdatafrommultiplesources)
* 4.5 [Suppressing Sensitive Values](#suppressing-sensitive-values)
* 4.5.1 [RQ.SRS008.AES.Functions.SuppressOutputOfSensitiveValues](#rqsrs008aesfunctionssuppressoutputofsensitivevalues)
* 4.6 [Invalid Parameters](#invalid-parameters)
* 4.6.1 [RQ.SRS008.AES.Functions.InvalidParameters](#rqsrs008aesfunctionsinvalidparameters)
* 4.7 [Mismatched Values](#mismatched-values)
* 4.7.1 [RQ.SRS008.AES.Functions.Mismatched.Key](#rqsrs008aesfunctionsmismatchedkey)
* 4.7.2 [RQ.SRS008.AES.Functions.Mismatched.IV](#rqsrs008aesfunctionsmismatchediv)
* 4.7.3 [RQ.SRS008.AES.Functions.Mismatched.AAD](#rqsrs008aesfunctionsmismatchedaad)
* 4.7.4 [RQ.SRS008.AES.Functions.Mismatched.Mode](#rqsrs008aesfunctionsmismatchedmode)
* 4.8 [Performance](#performance)
* 4.8.1 [RQ.SRS008.AES.Functions.Check.Performance](#rqsrs008aesfunctionscheckperformance)
* 4.8.2 [RQ.SRS008.AES.Function.Check.Performance.BestCase](#rqsrs008aesfunctioncheckperformancebestcase)
* 4.8.3 [RQ.SRS008.AES.Function.Check.Performance.WorstCase](#rqsrs008aesfunctioncheckperformanceworstcase)
* 4.8.4 [RQ.SRS008.AES.Functions.Check.Compression](#rqsrs008aesfunctionscheckcompression)
* 4.8.5 [RQ.SRS008.AES.Functions.Check.Compression.LowCardinality](#rqsrs008aesfunctionscheckcompressionlowcardinality)
* 4.9 [Encrypt Function](#encrypt-function)
* 4.9.1 [RQ.SRS008.AES.Encrypt.Function](#rqsrs008aesencryptfunction)
* 4.9.2 [RQ.SRS008.AES.Encrypt.Function.Syntax](#rqsrs008aesencryptfunctionsyntax)
* 4.9.3 [RQ.SRS008.AES.Encrypt.Function.NIST.TestVectors](#rqsrs008aesencryptfunctionnisttestvectors)
* 4.9.4 [RQ.SRS008.AES.Encrypt.Function.Parameters.PlainText](#rqsrs008aesencryptfunctionparametersplaintext)
* 4.9.5 [RQ.SRS008.AES.Encrypt.Function.Parameters.Key](#rqsrs008aesencryptfunctionparameterskey)
* 4.9.6 [RQ.SRS008.AES.Encrypt.Function.Parameters.Mode](#rqsrs008aesencryptfunctionparametersmode)
* 4.9.7 [RQ.SRS008.AES.Encrypt.Function.Parameters.Mode.ValuesFormat](#rqsrs008aesencryptfunctionparametersmodevaluesformat)
* 4.9.8 [RQ.SRS008.AES.Encrypt.Function.Parameters.Mode.Value.Invalid](#rqsrs008aesencryptfunctionparametersmodevalueinvalid)
* 4.9.9 [RQ.SRS008.AES.Encrypt.Function.Parameters.Mode.Values](#rqsrs008aesencryptfunctionparametersmodevalues)
* 4.9.10 [RQ.SRS008.AES.Encrypt.Function.Parameters.InitializationVector](#rqsrs008aesencryptfunctionparametersinitializationvector)
* 4.9.11 [RQ.SRS008.AES.Encrypt.Function.Parameters.AdditionalAuthenticatedData](#rqsrs008aesencryptfunctionparametersadditionalauthenticateddata)
* 4.9.12 [RQ.SRS008.AES.Encrypt.Function.Parameters.ReturnValue](#rqsrs008aesencryptfunctionparametersreturnvalue)
* 4.9.13 [RQ.SRS008.AES.Encrypt.Function.Key.Length.InvalidLengthError](#rqsrs008aesencryptfunctionkeylengthinvalidlengtherror)
* 4.9.14 [RQ.SRS008.AES.Encrypt.Function.InitializationVector.Length.InvalidLengthError](#rqsrs008aesencryptfunctioninitializationvectorlengthinvalidlengtherror)
* 4.9.15 [RQ.SRS008.AES.Encrypt.Function.InitializationVector.NotValidForMode](#rqsrs008aesencryptfunctioninitializationvectornotvalidformode)
* 4.9.16 [RQ.SRS008.AES.Encrypt.Function.AdditionalAuthenticationData.NotValidForMode](#rqsrs008aesencryptfunctionadditionalauthenticationdatanotvalidformode)
* 4.9.17 [RQ.SRS008.AES.Encrypt.Function.AdditionalAuthenticationData.Length](#rqsrs008aesencryptfunctionadditionalauthenticationdatalength)
* 4.9.18 [RQ.SRS008.AES.Encrypt.Function.NonGCMMode.KeyAndInitializationVector.Length](#rqsrs008aesencryptfunctionnongcmmodekeyandinitializationvectorlength)
* 4.9.19 [RQ.SRS008.AES.Encrypt.Function.GCMMode.KeyAndInitializationVector.Length](#rqsrs008aesencryptfunctiongcmmodekeyandinitializationvectorlength)
* 4.10 [Decrypt Function](#decrypt-function)
* 4.10.1 [RQ.SRS008.AES.Decrypt.Function](#rqsrs008aesdecryptfunction)
* 4.10.2 [RQ.SRS008.AES.Decrypt.Function.Syntax](#rqsrs008aesdecryptfunctionsyntax)
* 4.10.3 [RQ.SRS008.AES.Decrypt.Function.Parameters.CipherText](#rqsrs008aesdecryptfunctionparametersciphertext)
* 4.10.4 [RQ.SRS008.AES.Decrypt.Function.Parameters.Key](#rqsrs008aesdecryptfunctionparameterskey)
* 4.10.5 [RQ.SRS008.AES.Decrypt.Function.Parameters.Mode](#rqsrs008aesdecryptfunctionparametersmode)
* 4.10.6 [RQ.SRS008.AES.Decrypt.Function.Parameters.Mode.ValuesFormat](#rqsrs008aesdecryptfunctionparametersmodevaluesformat)
* 4.10.7 [RQ.SRS008.AES.Decrypt.Function.Parameters.Mode.Value.Invalid](#rqsrs008aesdecryptfunctionparametersmodevalueinvalid)
* 4.10.8 [RQ.SRS008.AES.Decrypt.Function.Parameters.Mode.Values](#rqsrs008aesdecryptfunctionparametersmodevalues)
* 4.10.9 [RQ.SRS008.AES.Decrypt.Function.Parameters.InitializationVector](#rqsrs008aesdecryptfunctionparametersinitializationvector)
* 4.10.10 [RQ.SRS008.AES.Decrypt.Function.Parameters.AdditionalAuthenticatedData](#rqsrs008aesdecryptfunctionparametersadditionalauthenticateddata)
* 4.10.11 [RQ.SRS008.AES.Decrypt.Function.Parameters.ReturnValue](#rqsrs008aesdecryptfunctionparametersreturnvalue)
* 4.10.12 [RQ.SRS008.AES.Decrypt.Function.Key.Length.InvalidLengthError](#rqsrs008aesdecryptfunctionkeylengthinvalidlengtherror)
* 4.10.13 [RQ.SRS008.AES.Decrypt.Function.InitializationVector.Length.InvalidLengthError](#rqsrs008aesdecryptfunctioninitializationvectorlengthinvalidlengtherror)
* 4.10.14 [RQ.SRS008.AES.Decrypt.Function.InitializationVector.NotValidForMode](#rqsrs008aesdecryptfunctioninitializationvectornotvalidformode)
* 4.10.15 [RQ.SRS008.AES.Decrypt.Function.AdditionalAuthenticationData.NotValidForMode](#rqsrs008aesdecryptfunctionadditionalauthenticationdatanotvalidformode)
* 4.10.16 [RQ.SRS008.AES.Decrypt.Function.AdditionalAuthenticationData.Length](#rqsrs008aesdecryptfunctionadditionalauthenticationdatalength)
* 4.10.17 [RQ.SRS008.AES.Decrypt.Function.NonGCMMode.KeyAndInitializationVector.Length](#rqsrs008aesdecryptfunctionnongcmmodekeyandinitializationvectorlength)
* 4.10.18 [RQ.SRS008.AES.Decrypt.Function.GCMMode.KeyAndInitializationVector.Length](#rqsrs008aesdecryptfunctiongcmmodekeyandinitializationvectorlength)
* 4.11 [MySQL Encrypt Function](#mysql-encrypt-function)
* 4.11.1 [RQ.SRS008.AES.MySQL.Encrypt.Function](#rqsrs008aesmysqlencryptfunction)
* 4.11.2 [RQ.SRS008.AES.MySQL.Encrypt.Function.Syntax](#rqsrs008aesmysqlencryptfunctionsyntax)
* 4.11.3 [RQ.SRS008.AES.MySQL.Encrypt.Function.Parameters.PlainText](#rqsrs008aesmysqlencryptfunctionparametersplaintext)
* 4.11.4 [RQ.SRS008.AES.MySQL.Encrypt.Function.Parameters.Key](#rqsrs008aesmysqlencryptfunctionparameterskey)
* 4.11.5 [RQ.SRS008.AES.MySQL.Encrypt.Function.Parameters.Mode](#rqsrs008aesmysqlencryptfunctionparametersmode)
* 4.11.6 [RQ.SRS008.AES.MySQL.Encrypt.Function.Parameters.Mode.ValuesFormat](#rqsrs008aesmysqlencryptfunctionparametersmodevaluesformat)
* 4.11.7 [RQ.SRS008.AES.MySQL.Encrypt.Function.Parameters.Mode.Value.Invalid](#rqsrs008aesmysqlencryptfunctionparametersmodevalueinvalid)
* 4.11.8 [RQ.SRS008.AES.MySQL.Encrypt.Function.Parameters.Mode.Values](#rqsrs008aesmysqlencryptfunctionparametersmodevalues)
* 4.11.9 [RQ.SRS008.AES.MySQL.Encrypt.Function.Parameters.Mode.Values.GCM.Error](#rqsrs008aesmysqlencryptfunctionparametersmodevaluesgcmerror)
* 4.11.10 [RQ.SRS008.AES.MySQL.Encrypt.Function.Parameters.Mode.Values.CTR.Error](#rqsrs008aesmysqlencryptfunctionparametersmodevaluesctrerror)
* 4.11.11 [RQ.SRS008.AES.MySQL.Encrypt.Function.Parameters.InitializationVector](#rqsrs008aesmysqlencryptfunctionparametersinitializationvector)
* 4.11.12 [RQ.SRS008.AES.MySQL.Encrypt.Function.Parameters.ReturnValue](#rqsrs008aesmysqlencryptfunctionparametersreturnvalue)
* 4.11.13 [RQ.SRS008.AES.MySQL.Encrypt.Function.Key.Length.TooShortError](#rqsrs008aesmysqlencryptfunctionkeylengthtooshorterror)
* 4.11.14 [RQ.SRS008.AES.MySQL.Encrypt.Function.Key.Length.TooLong](#rqsrs008aesmysqlencryptfunctionkeylengthtoolong)
* 4.11.15 [RQ.SRS008.AES.MySQL.Encrypt.Function.InitializationVector.Length.TooShortError](#rqsrs008aesmysqlencryptfunctioninitializationvectorlengthtooshorterror)
* 4.11.16 [RQ.SRS008.AES.MySQL.Encrypt.Function.InitializationVector.Length.TooLong](#rqsrs008aesmysqlencryptfunctioninitializationvectorlengthtoolong)
* 4.11.17 [RQ.SRS008.AES.MySQL.Encrypt.Function.InitializationVector.NotValidForMode](#rqsrs008aesmysqlencryptfunctioninitializationvectornotvalidformode)
* 4.11.18 [RQ.SRS008.AES.MySQL.Encrypt.Function.Mode.KeyAndInitializationVector.Length](#rqsrs008aesmysqlencryptfunctionmodekeyandinitializationvectorlength)
* 4.12 [MySQL Decrypt Function](#mysql-decrypt-function)
* 4.12.1 [RQ.SRS008.AES.MySQL.Decrypt.Function](#rqsrs008aesmysqldecryptfunction)
* 4.12.2 [RQ.SRS008.AES.MySQL.Decrypt.Function.Syntax](#rqsrs008aesmysqldecryptfunctionsyntax)
* 4.12.3 [RQ.SRS008.AES.MySQL.Decrypt.Function.Parameters.CipherText](#rqsrs008aesmysqldecryptfunctionparametersciphertext)
* 4.12.4 [RQ.SRS008.AES.MySQL.Decrypt.Function.Parameters.Key](#rqsrs008aesmysqldecryptfunctionparameterskey)
* 4.12.5 [RQ.SRS008.AES.MySQL.Decrypt.Function.Parameters.Mode](#rqsrs008aesmysqldecryptfunctionparametersmode)
* 4.12.6 [RQ.SRS008.AES.MySQL.Decrypt.Function.Parameters.Mode.ValuesFormat](#rqsrs008aesmysqldecryptfunctionparametersmodevaluesformat)
* 4.12.7 [RQ.SRS008.AES.MySQL.Decrypt.Function.Parameters.Mode.Value.Invalid](#rqsrs008aesmysqldecryptfunctionparametersmodevalueinvalid)
* 4.12.8 [RQ.SRS008.AES.MySQL.Decrypt.Function.Parameters.Mode.Values](#rqsrs008aesmysqldecryptfunctionparametersmodevalues)
* 4.12.9 [RQ.SRS008.AES.MySQL.Decrypt.Function.Parameters.Mode.Values.GCM.Error](#rqsrs008aesmysqldecryptfunctionparametersmodevaluesgcmerror)
* 4.12.10 [RQ.SRS008.AES.MySQL.Decrypt.Function.Parameters.Mode.Values.CTR.Error](#rqsrs008aesmysqldecryptfunctionparametersmodevaluesctrerror)
* 4.12.11 [RQ.SRS008.AES.MySQL.Decrypt.Function.Parameters.InitializationVector](#rqsrs008aesmysqldecryptfunctionparametersinitializationvector)
* 4.12.12 [RQ.SRS008.AES.MySQL.Decrypt.Function.Parameters.ReturnValue](#rqsrs008aesmysqldecryptfunctionparametersreturnvalue)
* 4.12.13 [RQ.SRS008.AES.MySQL.Decrypt.Function.Key.Length.TooShortError](#rqsrs008aesmysqldecryptfunctionkeylengthtooshorterror)
* 4.12.14 [RQ.SRS008.AES.MySQL.Decrypt.Function.Key.Length.TooLong](#rqsrs008aesmysqldecryptfunctionkeylengthtoolong)
* 4.12.15 [RQ.SRS008.AES.MySQL.Decrypt.Function.InitializationVector.Length.TooShortError](#rqsrs008aesmysqldecryptfunctioninitializationvectorlengthtooshorterror)
* 4.12.16 [RQ.SRS008.AES.MySQL.Decrypt.Function.InitializationVector.Length.TooLong](#rqsrs008aesmysqldecryptfunctioninitializationvectorlengthtoolong)
* 4.12.17 [RQ.SRS008.AES.MySQL.Decrypt.Function.InitializationVector.NotValidForMode](#rqsrs008aesmysqldecryptfunctioninitializationvectornotvalidformode)
* 4.12.18 [RQ.SRS008.AES.MySQL.Decrypt.Function.Mode.KeyAndInitializationVector.Length](#rqsrs008aesmysqldecryptfunctionmodekeyandinitializationvectorlength)
* 5 [References](#references)
## Revision History
@ -122,9 +133,14 @@ accessed on remote [MySQL] servers via [MySQL Dictionary] or [MySQL Database Eng
## Terminology
* **AES** -
### AES
Advanced Encryption Standard ([AES])
### AEAD
Authenticated Encryption with Associated Data
## Requirements
### Generic
@ -134,38 +150,44 @@ version: 1.0
[ClickHouse] SHALL support [AES] encryption functions to encrypt and decrypt data.
#### RQ.SRS008.AES.Functions.Compatability.MySQL
### Compatibility
#### RQ.SRS008.AES.Functions.Compatibility.MySQL
version: 1.0
[ClickHouse] SHALL support [AES] encryption functions compatible with [MySQL 5.7].
#### RQ.SRS008.AES.Functions.Compatability.Dictionaries
#### RQ.SRS008.AES.Functions.Compatibility.Dictionaries
version: 1.0
[ClickHouse] SHALL support encryption and decryption of data accessed on remote
[MySQL] servers using [MySQL Dictionary].
#### RQ.SRS008.AES.Functions.Compatability.Engine.Database.MySQL
#### RQ.SRS008.AES.Functions.Compatibility.Engine.Database.MySQL
version: 1.0
[ClickHouse] SHALL support encryption and decryption of data accessed using [MySQL Database Engine],
#### RQ.SRS008.AES.Functions.Compatability.Engine.Table.MySQL
#### RQ.SRS008.AES.Functions.Compatibility.Engine.Table.MySQL
version: 1.0
[ClickHouse] SHALL support encryption and decryption of data accessed using [MySQL Table Engine].
#### RQ.SRS008.AES.Functions.Compatability.TableFunction.MySQL
#### RQ.SRS008.AES.Functions.Compatibility.TableFunction.MySQL
version: 1.0
[ClickHouse] SHALL support encryption and decryption of data accessed using [MySQL Table Function].
### Different Modes
#### RQ.SRS008.AES.Functions.DifferentModes
version: 1.0
[ClickHouse] SHALL allow different modes to be supported in a single SQL statement
using explicit function parameters.
### Multiple Sources
#### RQ.SRS008.AES.Functions.DataFromMultipleSources
version: 1.0
@ -174,17 +196,23 @@ in the `SELECT` statement, including [ClickHouse] [MergeTree] table as well as [
[MySQL Database Engine], [MySQL Table Engine], and [MySQL Table Function]
with possibly different encryption schemes.
### Suppressing Sensitive Values
#### RQ.SRS008.AES.Functions.SuppressOutputOfSensitiveValues
version: 1.0
[ClickHouse] SHALL suppress output of [AES] `string` and `key` parameters to the system log,
error log, and `query_log` table to prevent leakage of sensitive values.
### Invalid Parameters
#### RQ.SRS008.AES.Functions.InvalidParameters
version: 1.0
[ClickHouse] SHALL return an error when parameters are invalid.
### Mismatched Values
#### RQ.SRS008.AES.Functions.Mismatched.Key
version: 1.0
@ -205,6 +233,8 @@ version: 1.0
[ClickHouse] SHALL return an error or garbage for mismatched mode.
### Performance
#### RQ.SRS008.AES.Functions.Check.Performance
version: 1.0
@ -233,7 +263,7 @@ version: 1.0
Effect of [AES] encryption on the compression of a column with [LowCardinality] data type
SHALL be measured.
### Specific
### Encrypt Function
#### RQ.SRS008.AES.Encrypt.Function
version: 1.0
@ -255,21 +285,23 @@ version: 1.0
[ClickHouse] `encrypt` function output SHALL produce output that matches [NIST test vectors].
#### RQ.SRS008.AES.Encrypt.Function.Parameters.PlainText
version: 1.0
version: 2.0
[ClickHouse] SHALL support `plaintext` with `String`, `FixedString`, `Nullable(String)`,
`Nullable(FixedString)`, `LowCardinality(String)`, or `LowCardinality(FixedString(N))` data types as
the second parameter to the `encrypt` function that SHALL specify the data to be encrypted.
[ClickHouse] SHALL support `plaintext` accepting any data type as
the first parameter to the `encrypt` function that SHALL specify the data to be encrypted.
#### RQ.SRS008.AES.Encrypt.Function.Parameters.Key
version: 1.0
[ClickHouse] SHALL support `key` with `String` or `FixedString` data types
as the second parameter to the `encrypt` function that SHALL specify the encryption key.
as the parameter to the `encrypt` function that SHALL specify the encryption key.
#### RQ.SRS008.AES.Encrypt.Function.Parameters.Mode
version: 1.0
[ClickHouse] SHALL support `mode` with `String` or `FixedString` data types as the third parameter
[ClickHouse] SHALL support `mode` with `String` or `FixedString` data types as the first parameter
to the `encrypt` function that SHALL specify encryption key length and block encryption mode.
#### RQ.SRS008.AES.Encrypt.Function.Parameters.Mode.ValuesFormat
@ -308,13 +340,13 @@ of the `encrypt` function:
* `aes-192-ofb` that SHALL use [OFB] block mode encryption with 192 bit key
* `aes-256-ofb` that SHALL use [OFB] block mode encryption with 256 bit key
* `aes-128-gcm` that SHALL use [GCM] block mode encryption with 128 bit key
and `AEAD` 16-byte tag is appended to the resulting ciphertext according to
and [AEAD] 16-byte tag is appended to the resulting ciphertext according to
the [RFC5116]
* `aes-192-gcm` that SHALL use [GCM] block mode encryption with 192 bit key
and `AEAD` 16-byte tag is appended to the resulting ciphertext according to
and [AEAD] 16-byte tag is appended to the resulting ciphertext according to
the [RFC5116]
* `aes-256-gcm` that SHALL use [GCM] block mode encryption with 256 bit key
and `AEAD` 16-byte tag is appended to the resulting ciphertext according to
and [AEAD] 16-byte tag is appended to the resulting ciphertext according to
the [RFC5116]
* `aes-128-ctr` that SHALL use [CTR] block mode encryption with 128 bit key
* `aes-192-ctr` that SHALL use [CTR] block mode encryption with 192 bit key
@ -403,6 +435,8 @@ when using GCM modes
* `aes-192-gcm` mode and `key` is not 24 bytes or `iv` is not specified
* `aes-256-gcm` mode and `key` is not 32 bytes or `iv` is not specified
### Decrypt Function
#### RQ.SRS008.AES.Decrypt.Function
version: 1.0
@ -421,18 +455,18 @@ decrypt(mode, ciphertext, key, [iv, aad])
version: 1.0
[ClickHouse] SHALL support `ciphertext` accepting `FixedString` or `String` data types as
the first parameter to the `decrypt` function that SHALL specify the data to be decrypted.
the second parameter to the `decrypt` function that SHALL specify the data to be decrypted.
#### RQ.SRS008.AES.Decrypt.Function.Parameters.Key
version: 1.0
[ClickHouse] SHALL support `key` with `String` or `FixedString` data types
as the second parameter to the `decrypt` function that SHALL specify the encryption key.
as the third parameter to the `decrypt` function that SHALL specify the encryption key.
#### RQ.SRS008.AES.Decrypt.Function.Parameters.Mode
version: 1.0
[ClickHouse] SHALL support `mode` with `String` or `FixedString` data types as the third parameter
[ClickHouse] SHALL support `mode` with `String` or `FixedString` data types as the first parameter
to the `decrypt` function that SHALL specify encryption key length and block encryption mode.
#### RQ.SRS008.AES.Decrypt.Function.Parameters.Mode.ValuesFormat
@ -511,7 +545,7 @@ version: 1.0
#### RQ.SRS008.AES.Decrypt.Function.InitializationVector.Length.InvalidLengthError
version: 1.0
[ClickHouse] SHALL return an error if the `iv` is speficified and the length is not exact for the `decrypt` function for a given block mode.
[ClickHouse] SHALL return an error if the `iv` is specified and the length is not exact for the `decrypt` function for a given block mode.
#### RQ.SRS008.AES.Decrypt.Function.InitializationVector.NotValidForMode
version: 1.0
@ -568,7 +602,7 @@ when using GCM modes
* `aes-192-gcm` mode and `key` is not 24 bytes or `iv` is not specified
* `aes-256-gcm` mode and `key` is not 32 bytes or `iv` is not specified
### MySQL Specific Functions
### MySQL Encrypt Function
#### RQ.SRS008.AES.MySQL.Encrypt.Function
version: 1.0
@ -585,21 +619,22 @@ aes_encrypt_mysql(mode, plaintext, key, [iv])
```
#### RQ.SRS008.AES.MySQL.Encrypt.Function.Parameters.PlainText
version: 1.0
version: 2.0
[ClickHouse] SHALL support `plaintext` accepting any data type as
the first parameter to the `aes_encrypt_mysql` function that SHALL specify the data to be encrypted.
[ClickHouse] SHALL support `plaintext` with `String`, `FixedString`, `Nullable(String)`,
`Nullable(FixedString)`, `LowCardinality(String)`, or `LowCardinality(FixedString(N))` data types as
the second parameter to the `aes_encrypt_mysql` function that SHALL specify the data to be encrypted.
#### RQ.SRS008.AES.MySQL.Encrypt.Function.Parameters.Key
version: 1.0
[ClickHouse] SHALL support `key` with `String` or `FixedString` data types
as the second parameter to the `aes_encrypt_mysql` function that SHALL specify the encryption key.
as the third parameter to the `aes_encrypt_mysql` function that SHALL specify the encryption key.
#### RQ.SRS008.AES.MySQL.Encrypt.Function.Parameters.Mode
version: 1.0
[ClickHouse] SHALL support `mode` with `String` or `FixedString` data types as the third parameter
[ClickHouse] SHALL support `mode` with `String` or `FixedString` data types as the first parameter
to the `aes_encrypt_mysql` function that SHALL specify encryption key length and block encryption mode.
#### RQ.SRS008.AES.MySQL.Encrypt.Function.Parameters.Mode.ValuesFormat
@ -734,6 +769,8 @@ version: 1.0
* `aes-192-ofb` mode and `key` is less than 24 bytes or if specified `iv` is less than 16 bytes
* `aes-256-ofb` mode and `key` is less than 32 bytes or if specified `iv` is less than 16 bytes
### MySQL Decrypt Function
#### RQ.SRS008.AES.MySQL.Decrypt.Function
version: 1.0
@ -752,18 +789,18 @@ aes_decrypt_mysql(mode, ciphertext, key, [iv])
version: 1.0
[ClickHouse] SHALL support `ciphertext` accepting any data type as
the first parameter to the `aes_decrypt_mysql` function that SHALL specify the data to be decrypted.
the second parameter to the `aes_decrypt_mysql` function that SHALL specify the data to be decrypted.
#### RQ.SRS008.AES.MySQL.Decrypt.Function.Parameters.Key
version: 1.0
[ClickHouse] SHALL support `key` with `String` or `FixedString` data types
as the second parameter to the `aes_decrypt_mysql` function that SHALL specify the encryption key.
as the third parameter to the `aes_decrypt_mysql` function that SHALL specify the encryption key.
#### RQ.SRS008.AES.MySQL.Decrypt.Function.Parameters.Mode
version: 1.0
[ClickHouse] SHALL support `mode` with `String` or `FixedString` data types as the third parameter
[ClickHouse] SHALL support `mode` with `String` or `FixedString` data types as the first parameter
to the `aes_decrypt_mysql` function that SHALL specify encryption key length and block encryption mode.
#### RQ.SRS008.AES.MySQL.Decrypt.Function.Parameters.Mode.ValuesFormat
@ -906,6 +943,7 @@ version: 1.0
* **ClickHouse:** https://clickhouse.tech
* **Git:** https://git-scm.com/
[AEAD]: #aead
[OpenSSL]: https://www.openssl.org/
[LowCardinality]: https://clickhouse.tech/docs/en/sql-reference/data-types/lowcardinality/
[MergeTree]: https://clickhouse.tech/docs/en/engines/table-engines/mergetree-family/mergetree/

File diff suppressed because it is too large Load Diff

View File

@ -88,32 +88,36 @@ plaintexts = [
("utf8fixedstring", "toFixedString('Gãńdåłf_Thê_Gręât', 24)"),
("String", "'1'"),
("FixedString", "toFixedString('1', 1)"),
("UInt8", "toUInt8('1')"),
("UInt16", "toUInt16('1')"),
("UInt32", "toUInt32('1')"),
("UInt64", "toUInt64('1')"),
("Int8", "toInt8('1')"),
("Int16", "toInt16('1')"),
("Int32", "toInt32('1')"),
("Int64", "toInt64('1')"),
("Float32", "toFloat32('1')"),
("Float64", "toFloat64('1')"),
("Decimal32", "toDecimal32(2, 4)"),
("Decimal64", "toDecimal64(2, 4)"),
("Decimal128", "toDecimal128(2, 4)"),
("UUID", "toUUID('61f0c404-5cb3-11e7-907b-a6006ad3dba0')"),
("Date", "toDate('2020-01-01')"),
("DateTime", "toDateTime('2020-01-01 20:01:02')"),
("DateTime64", "toDateTime64('2020-01-01 20:01:02.123', 3)"),
("UInt8", "reinterpretAsFixedString(toUInt8('1'))"),
("UInt16", "reinterpretAsFixedString(toUInt16('1'))"),
("UInt32", "reinterpretAsFixedString(toUInt32('1'))"),
("UInt64", "reinterpretAsFixedString(toUInt64('1'))"),
("Int8", "reinterpretAsFixedString(toInt8('1'))"),
("Int16", "reinterpretAsFixedString(toInt16('1'))"),
("Int32", "reinterpretAsFixedString(toInt32('1'))"),
("Int64", "reinterpretAsFixedString(toInt64('1'))"),
("Float32", "reinterpretAsFixedString(toFloat32('1'))"),
("Float64", "reinterpretAsFixedString(toFloat64('1'))"),
("Decimal32", "reinterpretAsFixedString(toDecimal32(2, 4))"),
("Decimal64", "reinterpretAsFixedString(toDecimal64(2, 4))"),
("Decimal128", "reinterpretAsFixedString(toDecimal128(2, 4))"),
("UUID", "reinterpretAsFixedString(toUUID('61f0c404-5cb3-11e7-907b-a6006ad3dba0'))"),
("Date", "reinterpretAsFixedString(toDate('2020-01-01'))"),
("DateTime", "reinterpretAsFixedString(toDateTime('2020-01-01 20:01:02'))"),
("DateTime64", "reinterpretAsFixedString(toDateTime64('2020-01-01 20:01:02.123', 3))"),
("LowCardinality", "toLowCardinality('1')"),
("Array", "[1,2]"),
("LowCardinalityFixedString", "toLowCardinality(toFixedString('1',2))"),
#("Array", "[1,2]"), - not supported
#("Tuple", "(1,'a')") - not supported
#("Nullable, "Nullable(X)") - not supported
("NULL", "toDateOrNull('foo')"),
("IPv4", "toIPv4('171.225.130.45')"),
("IPv6", "toIPv6('2001:0db8:0000:85a3:0000:0000:ac1f:8001')"),
("Enum8", r"CAST('a', 'Enum8(\'a\' = 1, \'b\' = 2)')"),
("Enum16", r"CAST('a', 'Enum16(\'a\' = 1, \'b\' = 2)')"),
("NULL", "reinterpretAsFixedString(toDateOrNull('foo'))"),
("NullableString", "toNullable('1')"),
("NullableStringNull", "toNullable(NULL)"),
("NullableFixedString", "toNullable(toFixedString('1',2))"),
("NullableFixedStringNull", "toNullable(toFixedString(NULL,2))"),
("IPv4", "reinterpretAsFixedString(toIPv4('171.225.130.45'))"),
("IPv6", "reinterpretAsFixedString(toIPv6('2001:0db8:0000:85a3:0000:0000:ac1f:8001'))"),
("Enum8", r"reinterpretAsFixedString(CAST('a', 'Enum8(\'a\' = 1, \'b\' = 2)'))"),
("Enum16", r"reinterpretAsFixedString(CAST('a', 'Enum16(\'a\' = 1, \'b\' = 2)'))"),
]
_hex = hex

View File

@ -182,7 +182,7 @@ def encrypt(self, mysql_datatype):
@TestFeature
@Name("database engine")
@Requirements(
RQ_SRS008_AES_Functions_Compatability_Engine_Database_MySQL("1.0")
RQ_SRS008_AES_Functions_Compatibility_Engine_Database_MySQL("1.0")
)
def feature(self, node="clickhouse1", mysql_node="mysql1"):
"""Check usage of encryption functions with [MySQL database engine].

View File

@ -237,7 +237,7 @@ def decrypt(self, mysql_datatype):
@TestFeature
@Name("dictionary")
@Requirements(
RQ_SRS008_AES_Functions_Compatability_Dictionaries("1.0")
RQ_SRS008_AES_Functions_Compatibility_Dictionaries("1.0")
)
def feature(self, node="clickhouse1", mysql_node="mysql1"):
"""Check usage of encryption functions with [MySQL dictionary].

View File

@ -5,7 +5,7 @@ from aes_encryption.requirements import *
@TestFeature
@Name("mysql")
@Requirements(
RQ_SRS008_AES_Functions_Compatability_MySQL("1.0")
RQ_SRS008_AES_Functions_Compatibility_MySQL("1.0")
)
def feature(self, node="clickhouse1"):
"""Check encryption functions usage compatibility with MySQL.

View File

@ -188,7 +188,7 @@ def encrypt(self, mysql_datatype):
@TestFeature
@Name("table engine")
@Requirements(
RQ_SRS008_AES_Functions_Compatability_Engine_Table_MySQL("1.0")
RQ_SRS008_AES_Functions_Compatibility_Engine_Table_MySQL("1.0")
)
def feature(self, node="clickhouse1", mysql_node="mysql1"):
"""Check usage of encryption functions with [MySQL table engine].

View File

@ -169,7 +169,7 @@ def encrypt(self, mysql_datatype):
@TestFeature
@Name("table function")
@Requirements(
RQ_SRS008_AES_Functions_Compatability_TableFunction_MySQL("1.0")
RQ_SRS008_AES_Functions_Compatibility_TableFunction_MySQL("1.0")
)
def feature(self, node="clickhouse1", mysql_node="mysql1"):
"""Check usage of encryption functions with [MySQL table function].

View File

@ -54,7 +54,7 @@ def invalid_ciphertext(self):
d_aad = None if not aad_len else f"'{aad}'"
for datatype, ciphertext in invalid_ciphertexts:
if datatype in ["NULL"]:
if datatype == "NULL" or datatype.endswith("Null"):
continue
with When(f"invalid ciphertext={ciphertext}"):
if "cfb" in mode or "ofb" in mode or "ctr" in mode:
@ -431,17 +431,10 @@ def decryption(self):
ciphertext = f"unhex({ciphertext})"
compare = plaintext
if datatype == "IPv4":
cast = "toIPv4(IPv4NumToString(reinterpretAsUInt32"
endcast = "))"
elif datatype in ["DateTime64", "UUID", "IPv6", "LowCardinality", "Enum8", "Enum16", "Decimal32", "Decimal64", "Decimal128", "Array"]:
xfail(reason="no conversion")
elif datatype == "NULL":
if datatype == "NULL" or datatype.endswith("Null"):
ciphertext = "NULL"
cast = "isNull"
compare = None
elif datatype in ["Float32", "Float64", "Date", "DateTime"] or "Int" in datatype:
cast = f"reinterpretAs{datatype}"
decrypt(ciphertext=ciphertext, key=f"'{key[:key_len]}'", mode=mode,
iv=(None if not iv_len else f"'{iv[:iv_len]}'"),

View File

@ -53,7 +53,7 @@ def invalid_ciphertext(self):
d_iv = None if not iv_len else f"'{iv[:iv_len]}'"
for datatype, ciphertext in invalid_ciphertexts:
if datatype in ["NULL"]:
if datatype == "NULL" or datatype.endswith("Null"):
continue
with When(f"invalid ciphertext={ciphertext}"):
if "cfb" in mode or "ofb" in mode or "ctr" in mode:
@ -364,17 +364,10 @@ def decryption(self):
ciphertext = f"unhex({ciphertext})"
compare = plaintext
if datatype == "IPv4":
cast = "toIPv4(IPv4NumToString(reinterpretAsUInt32"
endcast = "))"
elif datatype in ["DateTime64", "UUID", "IPv6", "LowCardinality", "Enum8", "Enum16", "Decimal32", "Decimal64", "Decimal128", "Array"]:
xfail(reason="no conversion")
elif datatype == "NULL":
if datatype == "NULL" or datatype.endswith("Null"):
ciphertext = "NULL"
cast = "isNull"
compare = None
elif datatype in ["Float32", "Float64", "Date", "DateTime"] or "Int" in datatype:
cast = f"reinterpretAs{datatype}"
aes_decrypt_mysql(ciphertext=ciphertext, key=f"'{key[:key_len]}'", mode=mode,
iv=(None if not iv_len else f"'{iv[:iv_len]}'"),

View File

@ -108,6 +108,43 @@ def invalid_parameters(self):
encrypt(plaintext="'hello there'", key="'0123456789123456'", mode="'AES-128-ECB'", exitcode=36,
message="DB::Exception: Invalid mode: AES-128-ECB")
@TestOutline(Scenario)
@Requirements(
RQ_SRS008_AES_Functions_InvalidParameters("1.0")
)
@Examples("data_type, value", [
("UInt8", "toUInt8('1')"),
("UInt16", "toUInt16('1')"),
("UInt32", "toUInt32('1')"),
("UInt64", "toUInt64('1')"),
("Int8", "toInt8('1')"),
("Int16", "toInt16('1')"),
("Int32", "toInt32('1')"),
("Int64", "toInt64('1')"),
("Float32", "toFloat32('1.0')"),
("Float64", "toFloat64('1.0')"),
("Decimal32", "toDecimal32(2, 4)"),
("Decimal64", "toDecimal64(2, 4)"),
("Decimal128", "toDecimal128(2, 4)"),
("UUID", "toUUID('61f0c404-5cb3-11e7-907b-a6006ad3dba0')"),
("Date", "toDate('2020-01-01')"),
("DateTime", "toDateTime('2020-01-01 20:01:02')"),
("DateTime64", "toDateTime64('2020-01-01 20:01:02.123', 3)"),
("Array", "[1,2]"),
("Tuple", "(1,'a')"),
("IPv4", "toIPv4('171.225.130.45')"),
("IPv6", "toIPv6('2001:0db8:0000:85a3:0000:0000:ac1f:8001')"),
("Enum8", r"CAST('a', 'Enum8(\'a\' = 1, \'b\' = 2)')"),
("Enum16", r"CAST('a', 'Enum16(\'a\' = 1, \'b\' = 2)')")
])
def invalid_plaintext_data_type(self, data_type, value):
"""Check that encrypt function returns an error if the
plaintext parameter has invalid data type.
"""
with When("I try to encrypt plaintext with invalid data type", description=f"{data_type} with value {value}"):
encrypt(plaintext=value, key="'0123456789123456'", mode="'aes-128-cbc'", iv="'0123456789123456'",
exitcode=43, message="DB::Exception: Illegal type of argument")
@TestOutline(Scenario)
@Requirements(
RQ_SRS008_AES_Encrypt_Function_Key_Length_InvalidLengthError("1.0"),
@ -309,7 +346,7 @@ def mode_parameter_types(self):
@TestScenario
@Requirements(
RQ_SRS008_AES_Encrypt_Function_Parameters_PlainText("1.0"),
RQ_SRS008_AES_Encrypt_Function_Parameters_PlainText("2.0"),
RQ_SRS008_AES_Encrypt_Function_Parameters_Mode("1.0"),
RQ_SRS008_AES_Encrypt_Function_Parameters_Mode_ValuesFormat("1.0"),
RQ_SRS008_AES_Encrypt_Function_Parameters_Mode_Values("1.0")

Some files were not shown because too many files have changed in this diff Show More